diff --git a/contrib/grpc b/contrib/grpc index f5b7fdc2dff..1716359d2e2 160000 --- a/contrib/grpc +++ b/contrib/grpc @@ -1 +1 @@ -Subproject commit f5b7fdc2dff09ada06dbf6c75df298fb40f898df +Subproject commit 1716359d2e28d304a250f9df0e6c0ccad03de8db diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile index 2512268be0f..4cac2ee6135 100644 --- a/docker/test/fasttest/Dockerfile +++ b/docker/test/fasttest/Dockerfile @@ -26,7 +26,10 @@ RUN apt-get update \ zstd \ --yes --no-install-recommends \ && apt-get clean \ - && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* + && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* \ + && groupadd --system --gid 1000 clickhouse \ + && useradd --system --gid 1000 --uid 1000 -m clickhouse +# ^ For some reason, groupadd and useradd are needed for tests with 'expect', but I don't know, why. COPY requirements.txt / RUN pip3 install --no-cache-dir -r /requirements.txt diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 0d975d64010..970bf12a81a 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -9,7 +9,7 @@ trap 'kill $(jobs -pr) ||:' EXIT stage=${stage:-} # Compiler version, normally set by Dockerfile -export LLVM_VERSION=${LLVM_VERSION:-17} +export LLVM_VERSION=${LLVM_VERSION:-18} # A variable to pass additional flags to CMake. # Here we explicitly default it to nothing so that bash doesn't complain about diff --git a/docker/test/util/Dockerfile b/docker/test/util/Dockerfile index f13bb576f79..babddbd228c 100644 --- a/docker/test/util/Dockerfile +++ b/docker/test/util/Dockerfile @@ -17,6 +17,7 @@ ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=18 RUN apt-get update \ && apt-get install \ + sudo \ apt-transport-https \ apt-utils \ ca-certificates \ diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md index e18ff6f1a3f..849b3d4b486 100644 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -185,6 +185,7 @@ You can pass parameters to `clickhouse-client` (all parameters have a default va - `--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. +- `--memory-usage` – If specified, print memory usage to ‘stderr’ in non-interactive mode]. Possible values: 'none' - do not print memory usage, 'default' - print number of bytes, 'readable' - print memory usage in human-readable format. - `--stacktrace` – If specified, also print the stack trace if an exception occurs. - `--config-file` – The name of the configuration file. - `--secure` – If specified, will connect to server over secure connection (TLS). You might need to configure your CA certificates in the [configuration file](#configuration_files). The available configuration settings are the same as for [server-side TLS configuration](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-openssl). @@ -339,7 +340,7 @@ 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 +clickhouse-client clickhouse://192.168.1.15,192.168.1.25 ``` ### Configuration Files {#configuration_files} @@ -367,7 +368,7 @@ Example of a config file: ``` Or the same config in a YAML format: - + ```yaml user: username password: 'password' diff --git a/docs/en/operations/system-tables/detached_tables.md b/docs/en/operations/system-tables/detached_tables.md new file mode 100644 index 00000000000..f669c8fd251 --- /dev/null +++ b/docs/en/operations/system-tables/detached_tables.md @@ -0,0 +1,35 @@ +--- +slug: /en/operations/system-tables/detached_tables +--- +# detached_tables + +Contains information of each detached table. + +Columns: + +- `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. + +- `uuid` ([UUID](../../sql-reference/data-types/uuid.md)) — Table uuid (Atomic database). + +- `metadata_path` ([String](../../sql-reference/data-types/string.md)) - Path to the table metadata in the file system. + +- `is_permanently` ([UInt8](../../sql-reference/data-types/int-uint.md)) - Flag indicates that the table was detached PERMANENTLY. + + +**Example** + +```sql +SELECT * FROM system.detached_tables FORMAT Vertical; +``` + +```text +Row 1: +────── +database: base +table: t1 +uuid: 81b1c20a-b7c6-4116-a2ce-7583fb6b6736 +metadata_path: /var/lib/clickhouse/store/461/461cf698-fd0b-406d-8c01-5d8fd5748a91/t1.sql +is_permanently: 1 +``` diff --git a/docs/en/sql-reference/data-types/lowcardinality.md b/docs/en/sql-reference/data-types/lowcardinality.md index 133ac2bd72e..a92f6f4dce6 100644 --- a/docs/en/sql-reference/data-types/lowcardinality.md +++ b/docs/en/sql-reference/data-types/lowcardinality.md @@ -56,7 +56,6 @@ Functions: ## Related content -- [Reducing ClickHouse Storage Cost with the Low Cardinality Type – Lessons from an Instana Engineer](https://altinity.com/blog/2020-5-20-reducing-clickhouse-storage-cost-with-the-low-cardinality-type-lessons-from-an-instana-engineer) -- [String Optimization (video presentation in Russian)](https://youtu.be/rqf-ILRgBdY?list=PL0Z2YDlm0b3iwXCpEFiOOYmwXzVmjJfEt). [Slides in English](https://github.com/ClickHouse/clickhouse-presentations/raw/master/meetup19/string_optimization.pdf) - Blog: [Optimizing ClickHouse with Schemas and Codecs](https://clickhouse.com/blog/optimize-clickhouse-codecs-compression-schema) - Blog: [Working with time series data in ClickHouse](https://clickhouse.com/blog/working-with-time-series-data-and-functions-ClickHouse) +- [String Optimization (video presentation in Russian)](https://youtu.be/rqf-ILRgBdY?list=PL0Z2YDlm0b3iwXCpEFiOOYmwXzVmjJfEt). [Slides in English](https://github.com/ClickHouse/clickhouse-presentations/raw/master/meetup19/string_optimization.pdf) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 260457b3be1..b7e4094f30e 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -76,7 +76,7 @@ WHERE macro = 'test'; └───────┴──────────────┘ ``` -## FQDN +## fqdn Returns the fully qualified domain name of the ClickHouse server. @@ -86,7 +86,7 @@ Returns the fully qualified domain name of the ClickHouse server. fqdn(); ``` -Aliases: `fullHostName`, 'FQDN'. +Aliases: `fullHostName`, `FQDN`. **Returned value** diff --git a/docs/zh/changelog/index.md b/docs/zh/changelog/index.md index c91d8bcf4d1..cd77a8c03cf 100644 --- a/docs/zh/changelog/index.md +++ b/docs/zh/changelog/index.md @@ -356,7 +356,7 @@ sidebar_label: "\u53D8\u66F4\u65E5\u5FD7" #### 新功能 {#new-feature-1} -- 添加 `deduplicate_blocks_in_dependent_materialized_views` 用于控制具有实例化视图的表中幂等插入的行为的选项。 这个新功能是由Altinity的特殊要求添加到错误修正版本中的。 +- 添加 `deduplicate_blocks_in_dependent_materialized_views` 用于控制具有实例化视图的表中幂等插入的行为的选项。 [#9070](https://github.com/ClickHouse/ClickHouse/pull/9070) [(urykhy)](https://github.com/urykhy) ### ClickHouse版本v20.1.2.4,2020-01-22 {#clickhouse-release-v20-1-2-4-2020-01-22} diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 46b543e49e9..48e0cca7b73 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -423,6 +423,7 @@ void LocalServer::connect() { connection_parameters = ConnectionParameters(getClientConfiguration(), "localhost"); + /// This is needed for table function input(...). ReadBuffer * in; auto table_file = getClientConfiguration().getString("table-file", "-"); if (table_file == "-" || table_file == "stdin") diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index c6a070219a3..d2e0a7191b1 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -2070,9 +2070,18 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin progress_indication.writeFinalProgress(); output_stream << std::endl << std::endl; } - else if (getClientConfiguration().getBool("print-time-to-stderr", false)) + else { - error_stream << progress_indication.elapsedSeconds() << "\n"; + const auto & config = getClientConfiguration(); + if (config.getBool("print-time-to-stderr", false)) + error_stream << progress_indication.elapsedSeconds() << "\n"; + + const auto & print_memory_mode = config.getString("print-memory-to-stderr", ""); + auto peak_memeory_usage = std::max(progress_indication.getMemoryUsage().peak, 0); + if (print_memory_mode == "default") + error_stream << peak_memeory_usage << "\n"; + else if (print_memory_mode == "readable") + error_stream << formatReadableSizeWithBinarySuffix(peak_memeory_usage) << "\n"; } if (!is_interactive && getClientConfiguration().getBool("print-num-processed-rows", false)) @@ -3036,6 +3045,7 @@ void ClientBase::init(int argc, char ** argv) ("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.") ("wait_for_suggestions_to_load", "Load suggestion data synchonously.") ("time,t", "print query execution time to stderr in non-interactive mode (for benchmarks)") + ("memory-usage", po::value()->implicit_value("default")->default_value("none"), "print memory usage to stderr in non-interactive mode (for benchmarks). Values: 'none', 'default', 'readable'") ("echo", "in batch mode, print query before execution") @@ -3121,6 +3131,14 @@ void ClientBase::init(int argc, char ** argv) /// Output execution time to stderr in batch mode. if (options.count("time")) getClientConfiguration().setBool("print-time-to-stderr", true); + if (options.count("memory-usage")) + { + const auto & memory_usage_mode = options["memory-usage"].as(); + if (memory_usage_mode != "none" && memory_usage_mode != "default" && memory_usage_mode != "readable") + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown memory-usage mode: {}", memory_usage_mode); + getClientConfiguration().setString("print-memory-to-stderr", memory_usage_mode); + } + if (options.count("query")) queries = options["query"].as>(); if (options.count("query_id")) diff --git a/src/Client/IConnections.h b/src/Client/IConnections.h index ebc71511834..09211de53b0 100644 --- a/src/Client/IConnections.h +++ b/src/Client/IConnections.h @@ -54,8 +54,6 @@ public: struct ReplicaInfo { - bool collaborate_with_initiator{false}; - size_t all_replicas_count{0}; size_t number_of_current_replica{0}; }; diff --git a/src/Client/LocalConnection.h b/src/Client/LocalConnection.h index fb6fa1b55eb..b424c5b5aa3 100644 --- a/src/Client/LocalConnection.h +++ b/src/Client/LocalConnection.h @@ -175,4 +175,5 @@ private: ReadBuffer * in; }; + } diff --git a/src/Client/MultiplexedConnections.cpp b/src/Client/MultiplexedConnections.cpp index 207bf6c9e07..244eccf1ed9 100644 --- a/src/Client/MultiplexedConnections.cpp +++ b/src/Client/MultiplexedConnections.cpp @@ -142,13 +142,12 @@ void MultiplexedConnections::sendQuery( modified_settings.group_by_two_level_threshold = 0; modified_settings.group_by_two_level_threshold_bytes = 0; } + } - if (replica_info) - { - client_info.collaborate_with_initiator = true; - client_info.count_participating_replicas = replica_info->all_replicas_count; - client_info.number_of_current_replica = replica_info->number_of_current_replica; - } + if (replica_info) + { + client_info.collaborate_with_initiator = true; + client_info.number_of_current_replica = replica_info->number_of_current_replica; } /// FIXME: Remove once we will make `allow_experimental_analyzer` obsolete setting. diff --git a/src/Common/Allocator.cpp b/src/Common/Allocator.cpp index bfc85559fe8..bcc5d187047 100644 --- a/src/Common/Allocator.cpp +++ b/src/Common/Allocator.cpp @@ -187,12 +187,9 @@ void * Allocator::realloc(void * buf, size_t old_size, #if USE_GWP_ASAN if (unlikely(GWPAsan::GuardedAlloc.shouldSample())) { + auto trace_alloc = CurrentMemoryTracker::alloc(new_size); if (void * ptr = GWPAsan::GuardedAlloc.allocate(new_size, alignment)) { - auto trace_free = CurrentMemoryTracker::free(old_size); - auto trace_alloc = CurrentMemoryTracker::alloc(new_size); - trace_free.onFree(buf, old_size); - memcpy(ptr, buf, std::min(old_size, new_size)); free(buf, old_size); trace_alloc.onAlloc(buf, new_size); @@ -209,6 +206,7 @@ void * Allocator::realloc(void * buf, size_t old_size, } else { + [[maybe_unused]] auto trace_free = CurrentMemoryTracker::free(old_size); ProfileEvents::increment(ProfileEvents::GWPAsanAllocateFailed); } } @@ -231,13 +229,17 @@ void * Allocator::realloc(void * buf, size_t old_size, if (alignment <= MALLOC_MIN_ALIGNMENT) { /// Resize malloc'd memory region with no special alignment requirement. - auto trace_free = CurrentMemoryTracker::free(old_size); + /// Realloc can do 2 possible things: + /// - expand existing memory region + /// - allocate new memory block and free the old one + /// Because we don't know which option will be picked we need to make sure there is enough + /// memory for all options auto trace_alloc = CurrentMemoryTracker::alloc(new_size); - trace_free.onFree(buf, old_size); void * new_buf = ::realloc(buf, new_size); if (nullptr == new_buf) { + [[maybe_unused]] auto trace_free = CurrentMemoryTracker::free(old_size); throw DB::ErrnoException( DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Allocator: Cannot realloc from {} to {}", @@ -246,6 +248,8 @@ void * Allocator::realloc(void * buf, size_t old_size, } buf = new_buf; + auto trace_free = CurrentMemoryTracker::free(old_size); + trace_free.onFree(buf, old_size); trace_alloc.onAlloc(buf, new_size); if constexpr (clear_memory) diff --git a/src/Common/ProgressIndication.h b/src/Common/ProgressIndication.h index 188ec4ed764..9baa1c7ffc4 100644 --- a/src/Common/ProgressIndication.h +++ b/src/Common/ProgressIndication.h @@ -72,11 +72,6 @@ public: /// How much seconds passed since query execution start. double elapsedSeconds() const { return getElapsedNanoseconds() / 1e9; } - void updateThreadEventData(HostToTimesMap & new_hosts_data); - -private: - double getCPUUsage(); - struct MemoryUsage { UInt64 total = 0; @@ -86,6 +81,11 @@ private: MemoryUsage getMemoryUsage() const; + void updateThreadEventData(HostToTimesMap & new_hosts_data); + +private: + double getCPUUsage(); + UInt64 getElapsedNanoseconds() const; /// This flag controls whether to show the progress bar. We start showing it after diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index 9b132c3e199..4bc8d436df7 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -1306,6 +1306,10 @@ void BaseDaemon::setupWatchdog() int status = 0; do { + // Close log files to prevent keeping descriptors of unlinked rotated files. + // On next log write files will be reopened. + closeLogs(logger()); + if (-1 != waitpid(pid, &status, WUNTRACED | WCONTINUED) || errno == ECHILD) { if (WIFSTOPPED(status)) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 9e7004e72a9..d86e29ca915 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -39,8 +39,10 @@ namespace ErrorCodes class AtomicDatabaseTablesSnapshotIterator final : public DatabaseTablesSnapshotIterator { public: - explicit AtomicDatabaseTablesSnapshotIterator(DatabaseTablesSnapshotIterator && base) - : DatabaseTablesSnapshotIterator(std::move(base)) {} + explicit AtomicDatabaseTablesSnapshotIterator(DatabaseTablesSnapshotIterator && base) noexcept + : DatabaseTablesSnapshotIterator(std::move(base)) + { + } UUID uuid() const override { return table()->getStorageID().uuid; } }; @@ -111,12 +113,12 @@ StoragePtr DatabaseAtomic::detachTable(ContextPtr /* context */, const String & // it is important to call the destructors of not_in_use without // locked mutex to avoid potential deadlock. DetachedTables not_in_use; - StoragePtr table; + StoragePtr detached_table; { std::lock_guard lock(mutex); - table = DatabaseOrdinary::detachTableUnlocked(name); + detached_table = DatabaseOrdinary::detachTableUnlocked(name); table_name_to_path.erase(name); - detached_tables.emplace(table->getStorageID().uuid, table); + detached_tables.emplace(detached_table->getStorageID().uuid, detached_table); not_in_use = cleanupDetachedTables(); } @@ -126,7 +128,7 @@ StoragePtr DatabaseAtomic::detachTable(ContextPtr /* context */, const String & LOG_DEBUG(log, "Finished removing not used detached tables"); } - return table; + return detached_table; } void DatabaseAtomic::dropTable(ContextPtr local_context, const String & table_name, bool sync) diff --git a/src/Databases/DatabaseAtomic.h b/src/Databases/DatabaseAtomic.h index b59edd479ba..4a4ccfa2573 100644 --- a/src/Databases/DatabaseAtomic.h +++ b/src/Databases/DatabaseAtomic.h @@ -1,7 +1,8 @@ #pragma once -#include #include +#include +#include namespace DB diff --git a/src/Databases/DatabaseLazy.cpp b/src/Databases/DatabaseLazy.cpp index da942cebf8f..5017c9b25cb 100644 --- a/src/Databases/DatabaseLazy.cpp +++ b/src/Databases/DatabaseLazy.cpp @@ -188,6 +188,13 @@ void DatabaseLazy::attachTable(ContextPtr /* context_ */, const String & table_n it->second.expiration_iterator = cache_expiration_queue.emplace(cache_expiration_queue.end(), current_time, table_name); + LOG_DEBUG(log, "Add info for detached table {} to snapshot.", backQuote(table_name)); + if (snapshot_detached_tables.contains(table_name)) + { + LOG_DEBUG(log, "Clean info about detached table {} from snapshot.", backQuote(table_name)); + snapshot_detached_tables.erase(table_name); + } + CurrentMetrics::add(CurrentMetrics::AttachedTable, 1); } @@ -204,6 +211,15 @@ StoragePtr DatabaseLazy::detachTable(ContextPtr /* context */, const String & ta if (it->second.expiration_iterator != cache_expiration_queue.end()) cache_expiration_queue.erase(it->second.expiration_iterator); tables_cache.erase(it); + LOG_DEBUG(log, "Add info for detached table {} to snapshot.", backQuote(table_name)); + snapshot_detached_tables.emplace( + table_name, + SnapshotDetachedTable{ + .database = res->getStorageID().database_name, + .table = res->getStorageID().table_name, + .uuid = res->getStorageID().uuid, + .metadata_path = getObjectMetadataPath(table_name), + .is_permanently = false}); CurrentMetrics::sub(CurrentMetrics::AttachedTable, 1); } diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 07a250e72c7..42730cfd53d 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -23,6 +23,7 @@ #include #include #include +#include #include #include #include @@ -308,6 +309,16 @@ void DatabaseOnDisk::detachTablePermanently(ContextPtr query_context, const Stri try { FS::createFile(detached_permanently_flag); + + std::lock_guard lock(mutex); + if (const auto it = snapshot_detached_tables.find(table_name); it == snapshot_detached_tables.end()) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table={}", table_name); + } + else + { + it->second.is_permanently = true; + } } catch (Exception & e) { diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 6555c4444e2..3ab5d3fa697 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -189,7 +189,7 @@ void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTables size_t prev_tables_count = metadata.parsed_tables.size(); size_t prev_total_dictionaries = metadata.total_dictionaries; - auto process_metadata = [&metadata, is_startup, this](const String & file_name) + auto process_metadata = [&metadata, is_startup, local_context, this](const String & file_name) { fs::path path(getMetadataPath()); fs::path file_path(file_name); @@ -197,7 +197,7 @@ void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTables try { - auto ast = parseQueryFromMetadata(log, getContext(), full_path.string(), /*throw_on_error*/ true, /*remove_empty*/ false); + auto ast = parseQueryFromMetadata(log, local_context, full_path.string(), /*throw_on_error*/ true, /*remove_empty*/ false); if (ast) { FunctionNameNormalizer::visit(ast.get()); @@ -226,8 +226,23 @@ void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTables if (fs::exists(full_path.string() + detached_suffix)) { const std::string table_name = unescapeForFileName(file_name.substr(0, file_name.size() - 4)); - permanently_detached_tables.push_back(table_name); LOG_DEBUG(log, "Skipping permanently detached table {}.", backQuote(table_name)); + + std::lock_guard lock(mutex); + permanently_detached_tables.push_back(table_name); + + const auto detached_table_name = create_query->getTable(); + + snapshot_detached_tables.emplace( + detached_table_name, + SnapshotDetachedTable{ + .database = create_query->getDatabase(), + .table = detached_table_name, + .uuid = create_query->uuid, + .metadata_path = getObjectMetadataPath(detached_table_name), + .is_permanently = true}); + + LOG_TRACE(log, "Add permanently detached table {} to system.detached_tables", detached_table_name); return; } @@ -489,6 +504,12 @@ DatabaseTablesIteratorPtr DatabaseOrdinary::getTablesIterator(ContextPtr local_c return DatabaseWithOwnTablesBase::getTablesIterator(local_context, filter_by_table_name, skip_not_loaded); } +DatabaseDetachedTablesSnapshotIteratorPtr DatabaseOrdinary::getDetachedTablesIterator( + ContextPtr local_context, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name, bool skip_not_loaded) const +{ + return DatabaseWithOwnTablesBase::getDetachedTablesIterator(local_context, filter_by_table_name, skip_not_loaded); +} + Strings DatabaseOrdinary::getAllTableNames(ContextPtr) const { std::set unique_names; diff --git a/src/Databases/DatabaseOrdinary.h b/src/Databases/DatabaseOrdinary.h index ef00ac8fdfa..c2c5775e5ab 100644 --- a/src/Databases/DatabaseOrdinary.h +++ b/src/Databases/DatabaseOrdinary.h @@ -57,6 +57,9 @@ public: LoadTaskPtr startupDatabaseAsync(AsyncLoader & async_loader, LoadJobSet startup_after, LoadingStrictnessLevel mode) override; DatabaseTablesIteratorPtr getTablesIterator(ContextPtr local_context, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name, bool skip_not_loaded) const override; + DatabaseDetachedTablesSnapshotIteratorPtr getDetachedTablesIterator( + ContextPtr local_context, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name, bool skip_not_loaded) const override; + Strings getAllTableNames(ContextPtr context) const override; void alterTable( @@ -64,7 +67,11 @@ public: const StorageID & table_id, const StorageInMemoryMetadata & metadata) override; - Strings getNamesOfPermanentlyDetachedTables() const override { return permanently_detached_tables; } + Strings getNamesOfPermanentlyDetachedTables() const override + { + std::lock_guard lock(mutex); + return permanently_detached_tables; + } protected: virtual void commitAlterTable( @@ -74,7 +81,7 @@ protected: const String & statement, ContextPtr query_context); - Strings permanently_detached_tables; + Strings permanently_detached_tables TSA_GUARDED_BY(mutex); std::unordered_map load_table TSA_GUARDED_BY(mutex); std::unordered_map startup_table TSA_GUARDED_BY(mutex); diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index 6426123bb4f..fe0baf30e57 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -2,12 +2,9 @@ #include #include -#include -#include -#include -#include #include #include +#include #include #include #include @@ -16,6 +13,10 @@ #include #include #include +#include +#include +#include +#include namespace DB @@ -237,6 +238,24 @@ DatabaseTablesIteratorPtr DatabaseWithOwnTablesBase::getTablesIterator(ContextPt return std::make_unique(std::move(filtered_tables), database_name); } +DatabaseDetachedTablesSnapshotIteratorPtr DatabaseWithOwnTablesBase::getDetachedTablesIterator( + ContextPtr, const FilterByNameFunction & filter_by_table_name, bool /* skip_not_loaded */) const +{ + std::lock_guard lock(mutex); + if (!filter_by_table_name) + return std::make_unique(snapshot_detached_tables); + + SnapshotDetachedTables filtered_detached_tables; + for (const auto & [detached_table_name, snapshot] : snapshot_detached_tables) + if (filter_by_table_name(detached_table_name)) + { + filtered_detached_tables.emplace(detached_table_name, snapshot); + } + + + return std::make_unique(std::move(filtered_detached_tables)); +} + bool DatabaseWithOwnTablesBase::empty() const { std::lock_guard lock(mutex); @@ -251,27 +270,36 @@ StoragePtr DatabaseWithOwnTablesBase::detachTable(ContextPtr /* context_ */, con StoragePtr DatabaseWithOwnTablesBase::detachTableUnlocked(const String & table_name) { - StoragePtr res; - auto it = tables.find(table_name); if (it == tables.end()) throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist", backQuote(database_name), backQuote(table_name)); - res = it->second; + + auto table_storage = it->second; + + snapshot_detached_tables.emplace( + table_name, + SnapshotDetachedTable{ + .database = it->second->getStorageID().getDatabaseName(), + .table = table_name, + .uuid = it->second->getStorageID().uuid, + .metadata_path = getObjectMetadataPath(table_name), + .is_permanently = false}); + tables.erase(it); - res->is_detached = true; + table_storage->is_detached = true; - if (res->isSystemStorage() == false) - CurrentMetrics::sub(getAttachedCounterForStorage(res), 1); + if (table_storage->isSystemStorage() == false) + CurrentMetrics::sub(getAttachedCounterForStorage(table_storage), 1); - auto table_id = res->getStorageID(); + auto table_id = table_storage->getStorageID(); if (table_id.hasUUID()) { assert(database_name == DatabaseCatalog::TEMPORARY_DATABASE || getUUID() != UUIDHelpers::Nil); DatabaseCatalog::instance().removeUUIDMapping(table_id.uuid); } - return res; + return table_storage; } void DatabaseWithOwnTablesBase::attachTable(ContextPtr /* context_ */, const String & table_name, const StoragePtr & table, const String &) @@ -300,6 +328,8 @@ void DatabaseWithOwnTablesBase::attachTableUnlocked(const String & table_name, c throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {} already exists.", table_id.getFullTableName()); } + snapshot_detached_tables.erase(table_name); + /// It is important to reset is_detached here since in case of RENAME in /// non-Atomic database the is_detached is set to true before RENAME. table->is_detached = false; @@ -337,6 +367,7 @@ void DatabaseWithOwnTablesBase::shutdown() std::lock_guard lock(mutex); tables.clear(); + snapshot_detached_tables.clear(); } DatabaseWithOwnTablesBase::~DatabaseWithOwnTablesBase() diff --git a/src/Databases/DatabasesCommon.h b/src/Databases/DatabasesCommon.h index 2eecf8a564f..1ca49e90c23 100644 --- a/src/Databases/DatabasesCommon.h +++ b/src/Databases/DatabasesCommon.h @@ -37,6 +37,9 @@ public: DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name, bool skip_not_loaded) const override; + DatabaseDetachedTablesSnapshotIteratorPtr + getDetachedTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name, bool skip_not_loaded) const 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; @@ -46,12 +49,13 @@ public: protected: Tables tables TSA_GUARDED_BY(mutex); + SnapshotDetachedTables snapshot_detached_tables TSA_GUARDED_BY(mutex); LoggerPtr log; DatabaseWithOwnTablesBase(const String & name_, const String & logger, ContextPtr context); void attachTableUnlocked(const String & table_name, const StoragePtr & table) TSA_REQUIRES(mutex); - StoragePtr detachTableUnlocked(const String & table_name) TSA_REQUIRES(mutex); + StoragePtr detachTableUnlocked(const String & table_name) TSA_REQUIRES(mutex); StoragePtr getTableUnlocked(const String & table_name) const TSA_REQUIRES(mutex); StoragePtr tryGetTableNoWait(const String & table_name) const; }; diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index b00f2fe4baf..ce5a52b1b0f 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -5,20 +5,22 @@ #include #include #include +#include +#include #include #include -#include #include +#include #include #include -#include #include #include +#include #include #include +#include #include -#include namespace DB @@ -110,6 +112,57 @@ public: using DatabaseTablesIteratorPtr = std::unique_ptr; +struct SnapshotDetachedTable final +{ + String database; + String table; + UUID uuid = UUIDHelpers::Nil; + String metadata_path; + bool is_permanently{}; +}; + +class DatabaseDetachedTablesSnapshotIterator +{ +private: + SnapshotDetachedTables snapshot; + SnapshotDetachedTables::iterator it; + +protected: + DatabaseDetachedTablesSnapshotIterator(DatabaseDetachedTablesSnapshotIterator && other) noexcept + { + size_t idx = std::distance(other.snapshot.begin(), other.it); + std::swap(snapshot, other.snapshot); + other.it = other.snapshot.end(); + it = snapshot.begin(); + std::advance(it, idx); + } + +public: + explicit DatabaseDetachedTablesSnapshotIterator(const SnapshotDetachedTables & tables_) : snapshot(tables_), it(snapshot.begin()) + { + } + + explicit DatabaseDetachedTablesSnapshotIterator(SnapshotDetachedTables && tables_) : snapshot(std::move(tables_)), it(snapshot.begin()) + { + } + + void next() { ++it; } + + bool isValid() const { return it != snapshot.end(); } + + String database() const { return it->second.database; } + + String table() const { return it->second.table; } + + UUID uuid() const { return it->second.uuid; } + + String metadataPath() const { return it->second.metadata_path; } + + bool isPermanently() const { return it->second.is_permanently; } +}; + +using DatabaseDetachedTablesSnapshotIteratorPtr = std::unique_ptr; + /** Database engine. * It is responsible for: @@ -232,6 +285,12 @@ public: /// Wait for all tables to be loaded and started up. If `skip_not_loaded` is true, then not yet loaded or not yet started up (at the moment of iterator creation) tables are excluded. virtual DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name = {}, bool skip_not_loaded = false) const = 0; /// NOLINT + virtual DatabaseDetachedTablesSnapshotIteratorPtr getDetachedTablesIterator( + ContextPtr /*context*/, const FilterByNameFunction & /*filter_by_table_name = {}*/, bool /*skip_not_loaded = false*/) const + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot get detached tables for Database{}", getEngineName()); + } + /// Returns list of table names. virtual Strings getAllTableNames(ContextPtr context) const { diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 27717ea3611..95c54ac9528 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -1184,7 +1184,7 @@ private: if (icolumn->size() != vec_to.size()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Argument column '{}' size {} doesn't match result column size {} of function {}", - icolumn->getName(), icolumn->size(), vec_to.size(), getName()); + icolumn->getName(), icolumn->size(), vec_to.size(), getName()); if constexpr (Keyed) if (key_cols.size() != vec_to.size() && key_cols.size() != 1) @@ -1223,6 +1223,9 @@ private: else executeGeneric(key_cols, icolumn, vec_to); } + /// Return a fixed random-looking magic number when input is empty. + static constexpr auto filler = 0xe28dbde7fe22e41c; + void executeForArgument(const KeyColumnsType & key_cols, const IDataType * type, const IColumn * column, typename ColumnVector::Container & vec_to, bool & is_first) const { /// Flattening of tuples. @@ -1231,6 +1234,11 @@ private: const auto & tuple_columns = tuple->getColumns(); const DataTypes & tuple_types = typeid_cast(*type).getElements(); size_t tuple_size = tuple_columns.size(); + + if (0 == tuple_size && is_first) + for (auto & hash : vec_to) + hash = static_cast(filler); + for (size_t i = 0; i < tuple_size; ++i) executeForArgument(key_cols, tuple_types[i].get(), tuple_columns[i].get(), vec_to, is_first); } @@ -1239,6 +1247,11 @@ private: const auto & tuple_columns = tuple_const->getColumns(); const DataTypes & tuple_types = typeid_cast(*type).getElements(); size_t tuple_size = tuple_columns.size(); + + if (0 == tuple_size && is_first) + for (auto & hash : vec_to) + hash = static_cast(filler); + for (size_t i = 0; i < tuple_size; ++i) { auto tmp = ColumnConst::create(tuple_columns[i], column->size()); @@ -1300,10 +1313,7 @@ public: constexpr size_t first_data_argument = Keyed; if (arguments.size() <= first_data_argument) - { - /// Return a fixed random-looking magic number when input is empty - vec_to.assign(input_rows_count, static_cast(0xe28dbde7fe22e41c)); - } + vec_to.assign(input_rows_count, static_cast(filler)); KeyColumnsType key_cols{}; if constexpr (Keyed) diff --git a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp index 32c51b745c7..855aa36b159 100644 --- a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp @@ -114,6 +114,34 @@ namespace else if (query.grantees) user.grantees = *query.grantees; } + + time_t getValidUntilFromAST(ASTPtr valid_until, ContextPtr context) + { + if (context) + valid_until = evaluateConstantExpressionAsLiteral(valid_until, context); + + const String valid_until_str = checkAndGetLiteralArgument(valid_until, "valid_until"); + + if (valid_until_str == "infinity") + return 0; + + time_t time = 0; + ReadBufferFromString in(valid_until_str); + + if (context) + { + const auto & time_zone = DateLUT::instance(""); + const auto & utc_time_zone = DateLUT::instance("UTC"); + + parseDateTimeBestEffort(time, in, time_zone, utc_time_zone); + } + else + { + readDateTimeText(time, in); + } + + return time; + } } BlockIO InterpreterCreateUserQuery::execute() @@ -134,23 +162,7 @@ BlockIO InterpreterCreateUserQuery::execute() std::optional valid_until; if (query.valid_until) - { - const ASTPtr valid_until_literal = evaluateConstantExpressionAsLiteral(query.valid_until, getContext()); - const String valid_until_str = checkAndGetLiteralArgument(valid_until_literal, "valid_until"); - - time_t time = 0; - - if (valid_until_str != "infinity") - { - const auto & time_zone = DateLUT::instance(""); - const auto & utc_time_zone = DateLUT::instance("UTC"); - - ReadBufferFromString in(valid_until_str); - parseDateTimeBestEffort(time, in, time_zone, utc_time_zone); - } - - valid_until = time; - } + valid_until = getValidUntilFromAST(query.valid_until, getContext()); std::optional default_roles_from_query; if (query.default_roles) @@ -259,7 +271,11 @@ void InterpreterCreateUserQuery::updateUserFromQuery(User & user, const ASTCreat if (query.auth_data) auth_data = AuthenticationData::fromAST(*query.auth_data, {}, !query.attach); - updateUserFromQueryImpl(user, query, auth_data, {}, {}, {}, {}, {}, allow_no_password, allow_plaintext_password, true); + std::optional valid_until; + if (query.valid_until) + valid_until = getValidUntilFromAST(query.valid_until, {}); + + updateUserFromQueryImpl(user, query, auth_data, {}, {}, {}, {}, valid_until, allow_no_password, allow_plaintext_password, true); } void registerInterpreterCreateUserQuery(InterpreterFactory & factory) diff --git a/src/Interpreters/ClientInfo.cpp b/src/Interpreters/ClientInfo.cpp index ce1efb61cc0..39fdef23baa 100644 --- a/src/Interpreters/ClientInfo.cpp +++ b/src/Interpreters/ClientInfo.cpp @@ -95,7 +95,7 @@ void ClientInfo::write(WriteBuffer & out, UInt64 server_protocol_revision) const if (server_protocol_revision >= DBMS_MIN_REVISION_WITH_PARALLEL_REPLICAS) { writeVarUInt(static_cast(collaborate_with_initiator), out); - writeVarUInt(count_participating_replicas, out); + writeVarUInt(obsolete_count_participating_replicas, out); writeVarUInt(number_of_current_replica, out); } } @@ -185,7 +185,7 @@ void ClientInfo::read(ReadBuffer & in, UInt64 client_protocol_revision) UInt64 value; readVarUInt(value, in); collaborate_with_initiator = static_cast(value); - readVarUInt(count_participating_replicas, in); + readVarUInt(obsolete_count_participating_replicas, in); readVarUInt(number_of_current_replica, in); } } diff --git a/src/Interpreters/ClientInfo.h b/src/Interpreters/ClientInfo.h index 3054667e264..ca32b4c5cfa 100644 --- a/src/Interpreters/ClientInfo.h +++ b/src/Interpreters/ClientInfo.h @@ -127,7 +127,7 @@ public: /// For parallel processing on replicas bool collaborate_with_initiator{false}; - UInt64 count_participating_replicas{0}; + UInt64 obsolete_count_participating_replicas{0}; UInt64 number_of_current_replica{0}; enum class BackgroundOperationType : uint8_t diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 771b8e9e558..6a24e049998 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -5004,13 +5004,6 @@ void Context::setConnectionClientVersion(UInt64 client_version_major, UInt64 cli client_info.connection_tcp_protocol_version = client_tcp_protocol_version; } -void Context::setReplicaInfo(bool collaborate_with_initiator, size_t all_replicas_count, size_t number_of_current_replica) -{ - client_info.collaborate_with_initiator = collaborate_with_initiator; - client_info.count_participating_replicas = all_replicas_count; - client_info.number_of_current_replica = number_of_current_replica; -} - void Context::increaseDistributedDepth() { ++client_info.distributed_depth; diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index f183a72e8e2..33b742d20ad 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -700,7 +700,6 @@ public: void setInitialQueryStartTime(std::chrono::time_point initial_query_start_time); void setQuotaClientKey(const String & quota_key); void setConnectionClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version); - void setReplicaInfo(bool collaborate_with_initiator, size_t all_replicas_count, size_t number_of_current_replica); void increaseDistributedDepth(); const OpenTelemetry::TracingContext & getClientTraceContext() const { return client_info.client_trace_context; } OpenTelemetry::TracingContext & getClientTraceContext() { return client_info.client_trace_context; } diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index 5fe0ba9a737..3a7f28ed837 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -8,6 +8,7 @@ namespace DB { + namespace ErrorCodes { extern const int LOGICAL_ERROR; diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index b5148174043..4d17891f9f8 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -385,7 +385,7 @@ static void maybeConvertOrdinaryDatabaseToAtomic(ContextMutablePtr context, cons if (database->getEngineName() != "Ordinary") return; - Strings permanently_detached_tables = database->getNamesOfPermanentlyDetachedTables(); + const Strings permanently_detached_tables = database->getNamesOfPermanentlyDetachedTables(); if (!permanently_detached_tables.empty()) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot automatically convert database {} from Ordinary to Atomic, " diff --git a/src/Parsers/ASTCreateQuery.cpp b/src/Parsers/ASTCreateQuery.cpp index d56a2724914..6dc009da9a8 100644 --- a/src/Parsers/ASTCreateQuery.cpp +++ b/src/Parsers/ASTCreateQuery.cpp @@ -265,6 +265,16 @@ ASTPtr ASTCreateQuery::clone() const return res; } +String ASTCreateQuery::getID(char delim) const +{ + String res = attach ? "AttachQuery" : "CreateQuery"; + String database = getDatabase(); + if (!database.empty()) + res += (delim + getDatabase()); + res += (delim + getTable()); + return res; +} + void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { frame.need_parens = false; diff --git a/src/Parsers/ASTCreateQuery.h b/src/Parsers/ASTCreateQuery.h index 6fbf045915b..9e4364b1f25 100644 --- a/src/Parsers/ASTCreateQuery.h +++ b/src/Parsers/ASTCreateQuery.h @@ -136,7 +136,7 @@ public: bool create_or_replace{false}; /** Get the text that identifies this element. */ - String getID(char delim) const override { return (attach ? "AttachQuery" : "CreateQuery") + (delim + getDatabase()) + delim + getTable(); } + String getID(char delim) const override; ASTPtr clone() const override; diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index fe724a8a198..e27515a62a4 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -412,8 +412,8 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings); const auto & shard = cluster->getShardsInfo().at(0); - size_t all_replicas_count = current_settings.max_parallel_replicas; - if (all_replicas_count > shard.getAllNodeCount()) + size_t max_replicas_to_use = current_settings.max_parallel_replicas; + if (max_replicas_to_use > shard.getAllNodeCount()) { LOG_INFO( getLogger("ReadFromParallelRemoteReplicasStep"), @@ -421,14 +421,14 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder "Will use the latter number to execute the query.", current_settings.max_parallel_replicas, shard.getAllNodeCount()); - all_replicas_count = shard.getAllNodeCount(); + max_replicas_to_use = shard.getAllNodeCount(); } std::vector shuffled_pool; - if (all_replicas_count < shard.getAllNodeCount()) + if (max_replicas_to_use < shard.getAllNodeCount()) { shuffled_pool = shard.pool->getShuffledPools(current_settings); - shuffled_pool.resize(all_replicas_count); + shuffled_pool.resize(max_replicas_to_use); } else { @@ -438,11 +438,10 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder shuffled_pool = shard.pool->getShuffledPools(current_settings, priority_func); } - for (size_t i=0; i < all_replicas_count; ++i) + for (size_t i=0; i < max_replicas_to_use; ++i) { IConnections::ReplicaInfo replica_info { - .all_replicas_count = all_replicas_count, /// we should use this number specifically because efficiency of data distribution by consistent hash depends on it. .number_of_current_replica = i, }; diff --git a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp index a3ae035afdd..b070bbe739b 100644 --- a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp +++ b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp @@ -36,18 +36,33 @@ inline void iotaWithStepOptimized(T * begin, size_t count, T first_value, T step iotaWithStep(begin, count, first_value, step); } +/// The range is defined as [start, end) +UInt64 itemCountInRange(UInt64 start, UInt64 end, UInt64 step) +{ + const auto range_count = end - start; + if (step == 1) + return range_count; + + return (range_count - 1) / step + 1; +} + class NumbersSource : public ISource { public: - NumbersSource(UInt64 block_size_, UInt64 offset_, std::optional limit_, UInt64 chunk_step_, const std::string & column_name, UInt64 step_) + NumbersSource( + UInt64 block_size_, + UInt64 offset_, + std::optional end_, + const std::string & column_name, + UInt64 step_in_chunk_, + UInt64 step_between_chunks_) : ISource(createHeader(column_name)) , block_size(block_size_) , next(offset_) - , chunk_step(chunk_step_) - , step(step_) + , end(end_) + , step_in_chunk(step_in_chunk_) + , step_between_chunks(step_between_chunks_) { - if (limit_.has_value()) - end = limit_.value() + offset_; } String getName() const override { return "Numbers"; } @@ -64,7 +79,10 @@ protected: { if (end.value() <= next) return {}; - real_block_size = std::min(block_size, end.value() - next); + + auto max_items_to_generate = itemCountInRange(next, *end, step_in_chunk); + + real_block_size = std::min(block_size, max_items_to_generate); } auto column = ColumnUInt64::create(real_block_size); ColumnUInt64::Container & vec = column->getData(); @@ -74,21 +92,20 @@ protected: UInt64 * current_end = &vec[real_block_size]; - iotaWithStepOptimized(pos, static_cast(current_end - pos), curr, step); + iotaWithStepOptimized(pos, static_cast(current_end - pos), curr, step_in_chunk); - next += chunk_step; + next += step_between_chunks; progress(column->size(), column->byteSize()); - return {Columns{std::move(column)}, real_block_size}; } private: UInt64 block_size; UInt64 next; - UInt64 chunk_step; std::optional end; /// not included - UInt64 step; + UInt64 step_in_chunk; + UInt64 step_between_chunks; }; struct RangeWithStep @@ -549,20 +566,39 @@ Pipe ReadFromSystemNumbersStep::makePipe() return pipe; } + const auto end = std::invoke( + [&]() -> std::optional + { + if (numbers_storage.limit.has_value()) + return *(numbers_storage.limit) + numbers_storage.offset; + return {}; + }); + /// Fall back to NumbersSource + /// Range in a single block + const auto block_range = max_block_size * numbers_storage.step; + /// Step between chunks in a single source. + /// It is bigger than block_range in case of multiple threads, because we have to account for other sources as well. + const auto step_between_chunks = num_streams * block_range; for (size_t i = 0; i < num_streams; ++i) { + const auto source_offset = i * block_range; + if (numbers_storage.limit.has_value() && *numbers_storage.limit < source_offset) + break; + + const auto source_start = numbers_storage.offset + source_offset; + auto source = std::make_shared( max_block_size, - numbers_storage.offset + i * max_block_size * numbers_storage.step, - numbers_storage.limit, - num_streams * max_block_size * numbers_storage.step, + source_start, + end, numbers_storage.column_name, - numbers_storage.step); + numbers_storage.step, + step_between_chunks); if (numbers_storage.limit && i == 0) { - auto rows_appr = (*numbers_storage.limit - 1) / numbers_storage.step + 1; + auto rows_appr = itemCountInRange(numbers_storage.offset, *numbers_storage.limit, numbers_storage.step); if (limit > 0 && limit < rows_appr) rows_appr = query_info_limit; source->addTotalRowsApprox(rows_appr); @@ -571,19 +607,6 @@ Pipe ReadFromSystemNumbersStep::makePipe() pipe.addSource(std::move(source)); } - if (numbers_storage.limit) - { - size_t i = 0; - auto storage_limit = (*numbers_storage.limit - 1) / numbers_storage.step + 1; - /// This formula is how to split 'limit' elements to 'num_streams' chunks almost uniformly. - pipe.addSimpleTransform( - [&](const Block & header) - { - ++i; - return std::make_shared(header, storage_limit * i / num_streams - storage_limit * (i - 1) / num_streams, 0); - }); - } - return pipe; } diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 4849f5827c1..b61df45133a 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -622,8 +622,9 @@ void KeeperTCPHandler::updateStats(Coordination::ZooKeeperResponsePtr & response { LOG_INFO( log, - "Total time to process a request took too long ({}ms).\nRequest info: {}", - elapsed, + "Total time to process a request in session {} took too long ({}ms).\nRequest info: {}", + session_id, + elapsed_ms, request->toString(/*short_format=*/true)); } diff --git a/src/Server/PrometheusRequestHandler.cpp b/src/Server/PrometheusRequestHandler.cpp index 1f3e038a1f5..87c106c3fc0 100644 --- a/src/Server/PrometheusRequestHandler.cpp +++ b/src/Server/PrometheusRequestHandler.cpp @@ -30,15 +30,8 @@ void PrometheusRequestHandler::handleRequest(HTTPServerRequest & request, HTTPSe response.setContentType("text/plain; version=0.0.4; charset=UTF-8"); WriteBufferFromHTTPServerResponse wb(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD, keep_alive_timeout, write_event); - try - { - metrics_writer->write(wb); - wb.finalize(); - } - catch (...) - { - wb.finalize(); - } + metrics_writer->write(wb); + wb.finalize(); } catch (...) { diff --git a/src/Storages/IStorage_fwd.h b/src/Storages/IStorage_fwd.h index b9243b029b0..4cbc586a745 100644 --- a/src/Storages/IStorage_fwd.h +++ b/src/Storages/IStorage_fwd.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include @@ -9,9 +10,10 @@ namespace DB { class IStorage; +struct SnapshotDetachedTable; using ConstStoragePtr = std::shared_ptr; using StoragePtr = std::shared_ptr; using Tables = std::map; - +using SnapshotDetachedTables = std::map; } diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index b9edff39b82..a904b29e12f 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -21,7 +21,6 @@ #include #include -#include #include #include #include diff --git a/src/Storages/System/StorageSystemDetachedTables.cpp b/src/Storages/System/StorageSystemDetachedTables.cpp new file mode 100644 index 00000000000..56c5e49b467 --- /dev/null +++ b/src/Storages/System/StorageSystemDetachedTables.cpp @@ -0,0 +1,241 @@ +#include "StorageSystemDetachedTables.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + + +namespace DB +{ + +namespace +{ + +class DetachedTablesBlockSource : public ISource +{ +public: + DetachedTablesBlockSource( + std::vector columns_mask_, + Block header_, + UInt64 max_block_size_, + ColumnPtr databases_, + ColumnPtr detached_tables_, + ContextPtr context_) + : ISource(std::move(header_)) + , columns_mask(std::move(columns_mask_)) + , max_block_size(max_block_size_) + , databases(std::move(databases_)) + , context(Context::createCopy(context_)) + { + size_t size = detached_tables_->size(); + detached_tables.reserve(size); + for (size_t idx = 0; idx < size; ++idx) + { + detached_tables.insert(detached_tables_->getDataAt(idx).toString()); + } + } + + String getName() const override { return "DetachedTables"; } + +protected: + Chunk generate() override + { + if (done) + return {}; + + MutableColumns result_columns = getPort().getHeader().cloneEmptyColumns(); + + const auto access = context->getAccess(); + const bool need_to_check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES); + + size_t database_idx = 0; + size_t rows_count = 0; + for (; database_idx < databases->size() && rows_count < max_block_size; ++database_idx) + { + database_name = databases->getDataAt(database_idx).toString(); + database = DatabaseCatalog::instance().tryGetDatabase(database_name); + + if (!database) + continue; + + const bool need_to_check_access_for_tables + = need_to_check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, database_name); + + if (!detached_tables_it || !detached_tables_it->isValid()) + detached_tables_it = database->getDetachedTablesIterator(context, {}, false); + + for (; rows_count < max_block_size && detached_tables_it->isValid(); detached_tables_it->next()) + { + const auto detached_table_name = detached_tables_it->table(); + + if (!detached_tables.contains(detached_table_name)) + continue; + + if (need_to_check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, database_name, detached_table_name)) + continue; + + fillResultColumnsByDetachedTableIterator(result_columns); + ++rows_count; + } + } + + if (databases->size() == database_idx && (!detached_tables_it || !detached_tables_it->isValid())) + { + done = true; + } + const UInt64 num_rows = result_columns.at(0)->size(); + return Chunk(std::move(result_columns), num_rows); + } + +private: + const std::vector columns_mask; + const UInt64 max_block_size; + const ColumnPtr databases; + NameSet detached_tables; + DatabaseDetachedTablesSnapshotIteratorPtr detached_tables_it; + ContextPtr context; + bool done = false; + DatabasePtr database; + std::string database_name; + + void fillResultColumnsByDetachedTableIterator(MutableColumns & result_columns) const + { + size_t src_index = 0; + size_t res_index = 0; + + if (columns_mask[src_index++]) + result_columns[res_index++]->insert(detached_tables_it->database()); + + if (columns_mask[src_index++]) + result_columns[res_index++]->insert(detached_tables_it->table()); + + + if (columns_mask[src_index++]) + result_columns[res_index++]->insert(detached_tables_it->uuid()); + + if (columns_mask[src_index++]) + result_columns[res_index++]->insert(detached_tables_it->metadataPath()); + + if (columns_mask[src_index++]) + result_columns[res_index++]->insert(detached_tables_it->isPermanently()); + } +}; + +} + +class ReadFromSystemDetachedTables : public SourceStepWithFilter +{ +public: + std::string getName() const override { return "ReadFromSystemDetachedTables"; } + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + + ReadFromSystemDetachedTables( + const Names & column_names_, + const SelectQueryInfo & query_info_, + const StorageSnapshotPtr & storage_snapshot_, + const ContextPtr & context_, + Block sample_block, + std::vector columns_mask_, + size_t max_block_size_); + + void applyFilters(ActionDAGNodes added_filter_nodes) override; + +private: + std::vector columns_mask; + size_t max_block_size; + + ColumnPtr filtered_databases_column; + ColumnPtr filtered_tables_column; +}; + +StorageSystemDetachedTables::StorageSystemDetachedTables(const StorageID & table_id_) : IStorage(table_id_) +{ + StorageInMemoryMetadata storage_metadata; + + auto description = ColumnsDescription{ + ColumnDescription{"database", std::make_shared(), "The name of the database the table is in."}, + ColumnDescription{"table", std::make_shared(), "Table name."}, + ColumnDescription{"uuid", std::make_shared(), "Table uuid (Atomic database)."}, + ColumnDescription{"metadata_path", std::make_shared(), "Path to the table metadata in the file system."}, + ColumnDescription{"is_permanently", std::make_shared(), "Table was detached permanently."}, + }; + + storage_metadata.setColumns(std::move(description)); + + setInMemoryMetadata(storage_metadata); +} + +void StorageSystemDetachedTables::read( + QueryPlan & query_plan, + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum /*processed_stage*/, + const size_t max_block_size, + size_t /*num_streams*/) +{ + storage_snapshot->check(column_names); + auto sample_block = storage_snapshot->metadata->getSampleBlock(); + + auto [columns_mask, res_block] = getQueriedColumnsMaskAndHeader(sample_block, column_names); + + auto reading = std::make_unique( + column_names, query_info, storage_snapshot, context, std::move(res_block), std::move(columns_mask), max_block_size); + + query_plan.addStep(std::move(reading)); +} + +ReadFromSystemDetachedTables::ReadFromSystemDetachedTables( + const Names & column_names_, + const SelectQueryInfo & query_info_, + const StorageSnapshotPtr & storage_snapshot_, + const ContextPtr & context_, + Block sample_block, + std::vector columns_mask_, + size_t max_block_size_) + : SourceStepWithFilter(DataStream{.header = std::move(sample_block)}, column_names_, query_info_, storage_snapshot_, context_) + , columns_mask(std::move(columns_mask_)) + , max_block_size(max_block_size_) +{ +} + +void ReadFromSystemDetachedTables::applyFilters(ActionDAGNodes added_filter_nodes) +{ + SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); + + const ActionsDAG::Node * predicate = nullptr; + if (filter_actions_dag) + predicate = filter_actions_dag->getOutputs().at(0); + + filtered_databases_column = detail::getFilteredDatabases(predicate, context); + filtered_tables_column = detail::getFilteredTables(predicate, filtered_databases_column, context, true); +} + +void ReadFromSystemDetachedTables::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +{ + auto pipe = Pipe(std::make_shared( + std::move(columns_mask), + getOutputStream().header, + max_block_size, + std::move(filtered_databases_column), + std::move(filtered_tables_column), + context)); + pipeline.init(std::move(pipe)); +} +} diff --git a/src/Storages/System/StorageSystemDetachedTables.h b/src/Storages/System/StorageSystemDetachedTables.h new file mode 100644 index 00000000000..cd042f51eaa --- /dev/null +++ b/src/Storages/System/StorageSystemDetachedTables.h @@ -0,0 +1,32 @@ +#pragma once + +#include + + +namespace DB +{ + +class Context; + +/** Implements the system table `detached_tables`, which allows you to get information about detached tables. + */ +class StorageSystemDetachedTables final : public IStorage +{ +public: + explicit StorageSystemDetachedTables(const StorageID & table_id_); + + std::string getName() const override { return "SystemDetachedTables"; } + + void read( + QueryPlan & query_plan, + 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; + + bool isSystemStorage() const override { return true; } +}; +} diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index 783b899c978..43b761d84b1 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -1,29 +1,30 @@ +#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 @@ -31,6 +32,105 @@ namespace DB { +namespace +{ + +/// Avoid heavy operation on tables if we only queried columns that we can get without table object. +/// Otherwise it will require table initialization for Lazy database. +bool needTable(const DatabasePtr & database, const Block & header) +{ + if (database->getEngineName() != "Lazy") + return true; + + static const std::set columns_without_table = {"database", "name", "uuid", "metadata_modification_time"}; + for (const auto & column : header.getColumnsWithTypeAndName()) + { + if (columns_without_table.find(column.name) == columns_without_table.end()) + return true; + } + return false; +} +} + +namespace detail +{ +ColumnPtr getFilteredDatabases(const ActionsDAG::Node * predicate, ContextPtr context) +{ + MutableColumnPtr column = ColumnString::create(); + + const auto databases = DatabaseCatalog::instance().getDatabases(); + for (const auto & database_name : databases | boost::adaptors::map_keys) + { + if (database_name == DatabaseCatalog::TEMPORARY_DATABASE) + continue; /// We don't want to show the internal database for temporary tables in system.tables + + column->insert(database_name); + } + + Block block{ColumnWithTypeAndName(std::move(column), std::make_shared(), "database")}; + VirtualColumnUtils::filterBlockWithPredicate(predicate, block, context); + return block.getByPosition(0).column; +} + +ColumnPtr getFilteredTables( + const ActionsDAG::Node * predicate, const ColumnPtr & filtered_databases_column, ContextPtr context, const bool is_detached) +{ + Block sample{ + ColumnWithTypeAndName(nullptr, std::make_shared(), "name"), + ColumnWithTypeAndName(nullptr, std::make_shared(), "engine")}; + + MutableColumnPtr database_column = ColumnString::create(); + MutableColumnPtr engine_column; + + auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(predicate, &sample); + if (dag) + { + bool filter_by_engine = false; + for (const auto * input : dag->getInputs()) + if (input->result_name == "engine") + filter_by_engine = true; + + if (filter_by_engine) + engine_column = ColumnString::create(); + } + + for (size_t database_idx = 0; database_idx < filtered_databases_column->size(); ++database_idx) + { + const auto & database_name = filtered_databases_column->getDataAt(database_idx).toString(); + DatabasePtr database = DatabaseCatalog::instance().tryGetDatabase(database_name); + if (!database) + continue; + + if (is_detached) + { + auto table_it = database->getDetachedTablesIterator(context, {}, false); + for (; table_it->isValid(); table_it->next()) + { + database_column->insert(table_it->table()); + } + } + else + { + for (auto table_it = database->getTablesIterator(context); table_it->isValid(); table_it->next()) + { + database_column->insert(table_it->name()); + if (engine_column) + engine_column->insert(table_it->table()->getName()); + } + } + } + + Block block{ColumnWithTypeAndName(std::move(database_column), std::make_shared(), "name")}; + if (engine_column) + block.insert(ColumnWithTypeAndName(std::move(engine_column), std::make_shared(), "engine")); + + if (dag) + VirtualColumnUtils::filterBlockWithDAG(dag, block, context); + + return block.getByPosition(0).column; +} + +} StorageSystemTables::StorageSystemTables(const StorageID & table_id_) : IStorage(table_id_) @@ -105,92 +205,6 @@ StorageSystemTables::StorageSystemTables(const StorageID & table_id_) setInMemoryMetadata(storage_metadata); } - -namespace -{ - -ColumnPtr getFilteredDatabases(const ActionsDAG::Node * predicate, ContextPtr context) -{ - MutableColumnPtr column = ColumnString::create(); - - const auto databases = DatabaseCatalog::instance().getDatabases(); - for (const auto & database_name : databases | boost::adaptors::map_keys) - { - if (database_name == DatabaseCatalog::TEMPORARY_DATABASE) - continue; /// We don't want to show the internal database for temporary tables in system.tables - - column->insert(database_name); - } - - Block block { ColumnWithTypeAndName(std::move(column), std::make_shared(), "database") }; - VirtualColumnUtils::filterBlockWithPredicate(predicate, block, context); - return block.getByPosition(0).column; -} - -ColumnPtr getFilteredTables(const ActionsDAG::Node * predicate, const ColumnPtr & filtered_databases_column, ContextPtr context) -{ - Block sample { - ColumnWithTypeAndName(nullptr, std::make_shared(), "name"), - ColumnWithTypeAndName(nullptr, std::make_shared(), "engine") - }; - - MutableColumnPtr database_column = ColumnString::create(); - MutableColumnPtr engine_column; - - auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(predicate, &sample); - if (dag) - { - bool filter_by_engine = false; - for (const auto * input : dag->getInputs()) - if (input->result_name == "engine") - filter_by_engine = true; - - if (filter_by_engine) - engine_column = ColumnString::create(); - } - - for (size_t database_idx = 0; database_idx < filtered_databases_column->size(); ++database_idx) - { - const auto & database_name = filtered_databases_column->getDataAt(database_idx).toString(); - DatabasePtr database = DatabaseCatalog::instance().tryGetDatabase(database_name); - if (!database) - continue; - - for (auto table_it = database->getTablesIterator(context); table_it->isValid(); table_it->next()) - { - database_column->insert(table_it->name()); - if (engine_column) - engine_column->insert(table_it->table()->getName()); - } - } - - Block block {ColumnWithTypeAndName(std::move(database_column), std::make_shared(), "name")}; - if (engine_column) - block.insert(ColumnWithTypeAndName(std::move(engine_column), std::make_shared(), "engine")); - - if (dag) - VirtualColumnUtils::filterBlockWithDAG(dag, block, context); - - return block.getByPosition(0).column; -} - -/// Avoid heavy operation on tables if we only queried columns that we can get without table object. -/// Otherwise it will require table initialization for Lazy database. -bool needTable(const DatabasePtr & database, const Block & header) -{ - if (database->getEngineName() != "Lazy") - return true; - - static const std::set columns_without_table = { "database", "name", "uuid", "metadata_modification_time" }; - for (const auto & column : header.getColumnsWithTypeAndName()) - { - if (columns_without_table.find(column.name) == columns_without_table.end()) - return true; - } - return false; -} - - class TablesBlockSource : public ISource { public: @@ -690,8 +704,6 @@ private: std::string database_name; }; -} - class ReadFromSystemTables : public SourceStepWithFilter { public: @@ -756,8 +768,8 @@ void ReadFromSystemTables::applyFilters(ActionDAGNodes added_filter_nodes) if (filter_actions_dag) predicate = filter_actions_dag->getOutputs().at(0); - filtered_databases_column = getFilteredDatabases(predicate, context); - filtered_tables_column = getFilteredTables(predicate, filtered_databases_column, context); + filtered_databases_column = detail::getFilteredDatabases(predicate, context); + filtered_tables_column = detail::getFilteredTables(predicate, filtered_databases_column, context, false); } void ReadFromSystemTables::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) diff --git a/src/Storages/System/StorageSystemTables.h b/src/Storages/System/StorageSystemTables.h index 87cdf1b6a8e..d6e3996b8e3 100644 --- a/src/Storages/System/StorageSystemTables.h +++ b/src/Storages/System/StorageSystemTables.h @@ -8,6 +8,15 @@ namespace DB class Context; +namespace detail +{ + +ColumnPtr getFilteredDatabases(const ActionsDAG::Node * predicate, ContextPtr context); +ColumnPtr +getFilteredTables(const ActionsDAG::Node * predicate, const ColumnPtr & filtered_databases_column, ContextPtr context, bool is_detached); + +} + /** Implements the system table `tables`, which allows you to get information about all tables. */ diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 6ff86b26ca9..97eda1db3fa 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -129,6 +130,7 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b attachNoDescription(context, system_database, "zeros_mt", "Multithreaded version of system.zeros.", true); attach(context, system_database, "databases", "Lists all databases of the current server."); attachNoDescription(context, system_database, "tables", "Lists all tables of the current server."); + attachNoDescription(context, system_database, "detached_tables", "Lists all detached tables of the current server."); attachNoDescription(context, system_database, "columns", "Lists all columns from all tables of the current server."); attach(context, system_database, "functions", "Contains a list of all available ordinary and aggregate functions with their descriptions."); attach(context, system_database, "events", "Contains profiling events and their current value."); diff --git a/tests/ci/autoscale_runners_lambda/app.py b/tests/ci/autoscale_runners_lambda/app.py index 520abcb41bb..c790a5081dd 100644 --- a/tests/ci/autoscale_runners_lambda/app.py +++ b/tests/ci/autoscale_runners_lambda/app.py @@ -99,7 +99,8 @@ def set_capacity( continue raise ValueError("Queue status is not in ['in_progress', 'queued']") - scale_down, scale_up = get_scales(runner_type) + # scale_down, scale_up = get_scales(runner_type) + _, scale_up = get_scales(runner_type) # With lyfecycle hooks some instances are actually free because some of # them are in 'Terminating:Wait' state effective_capacity = max( @@ -110,7 +111,7 @@ def set_capacity( # How much nodes are free (positive) or need to be added (negative) capacity_reserve = effective_capacity - running - queued stop = False - if capacity_reserve < 0: + if capacity_reserve <= 0: # This part is about scaling up capacity_deficit = -capacity_reserve # It looks that we are still OK, since no queued jobs exist @@ -158,41 +159,43 @@ def set_capacity( ) return - # Now we will calculate if we need to scale down - stop = stop or asg["DesiredCapacity"] == asg["MinSize"] - new_capacity = asg["DesiredCapacity"] - (capacity_reserve // scale_down) - new_capacity = max(new_capacity, asg["MinSize"]) - new_capacity = min(new_capacity, asg["MaxSize"]) - stop = stop or asg["DesiredCapacity"] == new_capacity - if stop: - logging.info( - "Do not decrease ASG %s capacity, current capacity=%s, effective " - "capacity=%s, minimum capacity=%s, running jobs=%s, queue size=%s", - asg["AutoScalingGroupName"], - asg["DesiredCapacity"], - effective_capacity, - asg["MinSize"], - running, - queued, - ) - return - - logging.info( - "The ASG %s capacity will be decreased to %s, current capacity=%s, effective " - "capacity=%s, minimum capacity=%s, running jobs=%s, queue size=%s", - asg["AutoScalingGroupName"], - new_capacity, - asg["DesiredCapacity"], - effective_capacity, - asg["MinSize"], - running, - queued, - ) - if not dry_run: - client.set_desired_capacity( - AutoScalingGroupName=asg["AutoScalingGroupName"], - DesiredCapacity=new_capacity, - ) + # FIXME: try decreasing capacity from runners that finished their jobs and have no job assigned + # IMPORTANT: Runner init script must be of version that supports ASG decrease + # # Now we will calculate if we need to scale down + # stop = stop or asg["DesiredCapacity"] == asg["MinSize"] + # new_capacity = asg["DesiredCapacity"] - (capacity_reserve // scale_down) + # new_capacity = max(new_capacity, asg["MinSize"]) + # new_capacity = min(new_capacity, asg["MaxSize"]) + # stop = stop or asg["DesiredCapacity"] == new_capacity + # if stop: + # logging.info( + # "Do not decrease ASG %s capacity, current capacity=%s, effective " + # "capacity=%s, minimum capacity=%s, running jobs=%s, queue size=%s", + # asg["AutoScalingGroupName"], + # asg["DesiredCapacity"], + # effective_capacity, + # asg["MinSize"], + # running, + # queued, + # ) + # return + # + # logging.info( + # "The ASG %s capacity will be decreased to %s, current capacity=%s, effective " + # "capacity=%s, minimum capacity=%s, running jobs=%s, queue size=%s", + # asg["AutoScalingGroupName"], + # new_capacity, + # asg["DesiredCapacity"], + # effective_capacity, + # asg["MinSize"], + # running, + # queued, + # ) + # if not dry_run: + # client.set_desired_capacity( + # AutoScalingGroupName=asg["AutoScalingGroupName"], + # DesiredCapacity=new_capacity, + # ) def main(dry_run: bool = True) -> None: diff --git a/tests/ci/autoscale_runners_lambda/test_autoscale.py b/tests/ci/autoscale_runners_lambda/test_autoscale.py index 3aeab49ffc7..d1a1f9b358f 100644 --- a/tests/ci/autoscale_runners_lambda/test_autoscale.py +++ b/tests/ci/autoscale_runners_lambda/test_autoscale.py @@ -97,19 +97,34 @@ class TestSetCapacity(unittest.TestCase): ), TestCase("lower-min", 10, 5, 20, [Queue("queued", 5, "lower-min")], 10), # Decrease capacity - TestCase("w/reserve", 1, 13, 20, [Queue("queued", 5, "w/reserve")], 5), + # FIXME: Tests changed for lambda that can only scale up + # TestCase("w/reserve", 1, 13, 20, [Queue("queued", 5, "w/reserve")], 5), + TestCase("w/reserve", 1, 13, 20, [Queue("queued", 5, "w/reserve")], -1), + # TestCase( + # "style-checker", 1, 13, 20, [Queue("queued", 5, "style-checker")], 5 + # ), TestCase( - "style-checker", 1, 13, 20, [Queue("queued", 5, "style-checker")], 5 + "style-checker", 1, 13, 20, [Queue("queued", 5, "style-checker")], -1 ), - TestCase("w/reserve", 1, 23, 20, [Queue("queued", 17, "w/reserve")], 17), - TestCase("decrease", 1, 13, 20, [Queue("in_progress", 3, "decrease")], 3), + # TestCase("w/reserve", 1, 23, 20, [Queue("queued", 17, "w/reserve")], 17), + TestCase("w/reserve", 1, 23, 20, [Queue("queued", 17, "w/reserve")], -1), + # TestCase("decrease", 1, 13, 20, [Queue("in_progress", 3, "decrease")], 3), + TestCase("decrease", 1, 13, 20, [Queue("in_progress", 3, "decrease")], -1), + # TestCase( + # "style-checker", + # 1, + # 13, + # 20, + # [Queue("in_progress", 5, "style-checker")], + # 5, + # ), TestCase( "style-checker", 1, 13, 20, [Queue("in_progress", 5, "style-checker")], - 5, + -1, ), ) for t in test_cases: diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 9b9ddee5326..f5e39f343b2 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -415,7 +415,8 @@ class CI: JobNames.INTEGRATION_TEST_FLAKY: CommonJobConfigs.INTEGRATION_TEST.with_properties( required_builds=[BuildNames.PACKAGE_ASAN], pr_only=True, - reference_job_name=JobNames.INTEGRATION_TEST_TSAN, + # TODO: approach with reference job names does not work because digest may not be calculated if job skipped in wf + # reference_job_name=JobNames.INTEGRATION_TEST_TSAN, ), JobNames.COMPATIBILITY_TEST: CommonJobConfigs.COMPATIBILITY_TEST.with_properties( required_builds=[BuildNames.PACKAGE_RELEASE], @@ -460,7 +461,8 @@ class CI: required_builds=[BuildNames.PACKAGE_ASAN], pr_only=True, timeout=3600, - reference_job_name=JobNames.STATELESS_TEST_RELEASE, + # TODO: approach with reference job names does not work because digest may not be calculated if job skipped in wf + # reference_job_name=JobNames.STATELESS_TEST_RELEASE, ), JobNames.JEPSEN_KEEPER: JobConfig( required_builds=[BuildNames.BINARY_RELEASE], diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index 41c7ed963c9..74dd4d8fbd7 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -17,9 +17,19 @@ from download_release_packages import download_last_release from env_helper import REPO_COPY, REPORT_PATH, TEMP_PATH from get_robot_token import get_parameter_from_ssm from pr_info import PRInfo -from report import ERROR, SUCCESS, JobReport, StatusType, TestResults, read_test_results +from report import ( + ERROR, + SUCCESS, + JobReport, + StatusType, + TestResults, + read_test_results, + FAILURE, +) from stopwatch import Stopwatch from tee_popen import TeePopen +from ci_config import CI +from ci_utils import Utils NO_CHANGES_MSG = "Nothing to run" @@ -351,7 +361,23 @@ def main(): additional_files=additional_logs, ).dump(to_file=args.report_to_file if args.report_to_file else None) + should_block_ci = False if state != SUCCESS: + should_block_ci = True + + if state == FAILURE and CI.is_required(check_name): + failed_cnt = Utils.get_failed_tests_number(description) + print( + f"Job status is [{state}] with [{failed_cnt}] failed test cases. status description [{description}]" + ) + if ( + failed_cnt + and failed_cnt <= CI.MAX_TOTAL_FAILURES_PER_JOB_BEFORE_BLOCKING_CI + ): + print(f"Won't block the CI workflow") + should_block_ci = False + + if should_block_ci: sys.exit(1) diff --git a/tests/ci/integration_test_check.py b/tests/ci/integration_test_check.py index 80ac1935d95..6245f0490fc 100644 --- a/tests/ci/integration_test_check.py +++ b/tests/ci/integration_test_check.py @@ -23,10 +23,13 @@ from report import ( TestResult, TestResults, read_test_results, + FAILURE, ) from stopwatch import Stopwatch import integration_tests_runner as runner +from ci_config import CI +from ci_utils import Utils def get_json_params_dict( @@ -233,7 +236,23 @@ def main(): additional_files=additional_logs, ).dump(to_file=args.report_to_file if args.report_to_file else None) + should_block_ci = False if state != SUCCESS: + should_block_ci = True + + if state == FAILURE and CI.is_required(check_name): + failed_cnt = Utils.get_failed_tests_number(description) + print( + f"Job status is [{state}] with [{failed_cnt}] failed test cases. status description [{description}]" + ) + if ( + failed_cnt + and failed_cnt <= CI.MAX_TOTAL_FAILURES_PER_JOB_BEFORE_BLOCKING_CI + ): + print(f"Won't block the CI workflow") + should_block_ci = False + + if should_block_ci: sys.exit(1) diff --git a/tests/ci/lambda_shared_package/lambda_shared/pr.py b/tests/ci/lambda_shared_package/lambda_shared/pr.py index e981e28a454..95130fc2a0f 100644 --- a/tests/ci/lambda_shared_package/lambda_shared/pr.py +++ b/tests/ci/lambda_shared_package/lambda_shared/pr.py @@ -3,48 +3,27 @@ import re from typing import Tuple -# Individual trusted contirbutors who are not in any trusted organization. +# Individual trusted contributors 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", # Kakao corp - "Algunenano", # Raúl Marín, ClickHouse, Inc "amosbird", "azat", # SEMRush "bharatnc", # Many contributions. - "bobrik", # Seasoned contributor, CloudFlare "cwurm", # ClickHouse, Inc "den-crane", # Documentation contributor - "hagen1778", # Roman Khavronenko, seasoned contributor - "hczhcz", - "hexiaoting", # Seasoned contributor "ildus", # adjust, ex-pgpro - "javisantana", # a Spanish ClickHouse enthusiast, ex-Carto - "kreuzerkrieg", - "nikvas0", "nvartolomei", # Seasoned contributor, CloudFlare - "spongedu", # Seasoned contributor "taiyang-li", "ucasFL", # Amos Bird's friend - "vdimir", # ClickHouse, Inc - "YiuRULE", - "zlobober", # Developer of YT - "ilejn", # Arenadata, responsible for Kerberized Kafka "thomoco", # ClickHouse, Inc - "BoloniniD", # Seasoned contributor, HSE "tonickkozlov", # Cloudflare "tylerhannan", # ClickHouse, Inc - "myrrc", # Mike Kot, DoubleCloud - "thevar1able", # ClickHouse, Inc - "aalexfvk", - "MikhailBurdukov", "tsolodov", # ClickHouse, Inc - "kitaisreal", - "k-morozov", # Konstantin Morozov, Yandex Cloud "justindeguzman", # ClickHouse, Inc - "jrdi", # ClickHouse contributor, TinyBird + "XuJia0210", # ClickHouse, Inc ] } diff --git a/tests/ci/merge_pr.py b/tests/ci/merge_pr.py index 94456506879..4d8facafb84 100644 --- a/tests/ci/merge_pr.py +++ b/tests/ci/merge_pr.py @@ -27,7 +27,6 @@ from report import SUCCESS, FAILURE from env_helper import GITHUB_UPSTREAM_REPOSITORY, GITHUB_REPOSITORY from synchronizer_utils import SYNC_BRANCH_PREFIX from ci_config import CI -from ci_utils import Utils # The team name for accepted approvals TEAM_NAME = getenv("GITHUB_TEAM_NAME", "core") @@ -249,74 +248,22 @@ def main(): repo = gh.get_repo(args.repo) if args.set_ci_status: + # set Mergeable check status and exit assert args.wf_status in (FAILURE, SUCCESS) - # set mergeable check status and exit commit = get_commit(gh, args.pr_info.sha) statuses = get_commit_filtered_statuses(commit) - max_failed_tests_per_job = 0 - job_name_with_max_failures = None - total_failed_tests = 0 - failed_to_get_info = False has_failed_statuses = False for status in statuses: - if not CI.is_required(status.context) or status.context in ( - CI.StatusNames.SYNC, - CI.StatusNames.PR_CHECK, - ): - # CI.StatusNames.SYNC or CI.StatusNames.PR_CHECK should not be checked - continue print(f"Check status [{status.context}], [{status.state}]") - if status.state == FAILURE: + if CI.is_required(status.context) and status.state != SUCCESS: + print(f"WARNING: Failed status [{status.context}], [{status.state}]") has_failed_statuses = True - failed_cnt = Utils.get_failed_tests_number(status.description) - if failed_cnt is None: - failed_to_get_info = True - print( - f"WARNING: failed to get number of failed tests from [{status.description}]" - ) - else: - if failed_cnt > max_failed_tests_per_job: - job_name_with_max_failures = status.context - max_failed_tests_per_job = failed_cnt - total_failed_tests += failed_cnt - print( - f"Failed test cases in [{status.context}] is [{failed_cnt}], total failures [{total_failed_tests}]" - ) - elif status.state != SUCCESS and status.context not in ( - CI.StatusNames.SYNC, - CI.StatusNames.PR_CHECK, - ): - # do not block CI on failures in (CI.StatusNames.SYNC, CI.StatusNames.PR_CHECK) - has_failed_statuses = True - print( - f"Unexpected status for [{status.context}]: [{status.state}] - block further testing" - ) - failed_to_get_info = True - - can_continue = True - if total_failed_tests > CI.MAX_TOTAL_FAILURES_BEFORE_BLOCKING_CI: - print( - f"Required check has [{total_failed_tests}] failed - block further testing" - ) - can_continue = False - if max_failed_tests_per_job > CI.MAX_TOTAL_FAILURES_PER_JOB_BEFORE_BLOCKING_CI: - print( - f"Job [{job_name_with_max_failures}] has [{max_failed_tests_per_job}] failures - block further testing" - ) - can_continue = False - if failed_to_get_info: - print("Unexpected commit status state - block further testing") - can_continue = False - if args.wf_status != SUCCESS and not has_failed_statuses: - # workflow failed but reason is unknown as no failed statuses present - can_continue = False - print( - "WARNING: Either the runner is faulty or the operating status is unknown. The first is self-healing, the second requires investigation." - ) if args.wf_status == SUCCESS or has_failed_statuses: - # do not set mergeable check status if args.wf_status == failure, apparently it has died runners and is to be restarted + # set Mergeable check if workflow is successful (green) + # or if we have GH statuses with failures (red) + # to avoid false-green on a died runner state = trigger_mergeable_check( commit, statuses, @@ -333,10 +280,10 @@ def main(): print( "Workflow failed but no failed statuses found (died runner?) - cannot set Mergeable Check status" ) - - if not can_continue: + if args.wf_status == SUCCESS and not has_failed_statuses: + sys.exit(0) + else: sys.exit(1) - sys.exit(0) # An ugly and not nice fix to patch the wrong organization URL, # see https://github.com/PyGithub/PyGithub/issues/2395#issuecomment-1378629710 diff --git a/tests/ci/worker/deploy-runner-init.sh b/tests/ci/worker/deploy-runner-init.sh index 06edede48fa..96fbd82a99c 100755 --- a/tests/ci/worker/deploy-runner-init.sh +++ b/tests/ci/worker/deploy-runner-init.sh @@ -1,5 +1,7 @@ #!/usr/bin/env bash +set -e + usage() { echo "Usage: $0 ENVIRONMENT" >&2 echo "Valid values for ENVIRONMENT: staging, production" >&2 @@ -55,7 +57,7 @@ EOF body() { local first_line - first_line=$(sed -n '/^# THE SCRIPT START$/{=;q}' "$SOURCE_SCRIPT") + first_line=$(sed -n '/^# THE SCRIPT START$/{=;q;}' "$SOURCE_SCRIPT") if [ -z "$first_line" ]; then echo "The pattern '# THE SCRIPT START' is not found in $SOURCE_SCRIPT" >&2 exit 1 diff --git a/tests/ci/worker/init_runner.sh b/tests/ci/worker/init_runner.sh index ec7793db2aa..d6cdb6d9c57 100644 --- a/tests/ci/worker/init_runner.sh +++ b/tests/ci/worker/init_runner.sh @@ -50,7 +50,7 @@ set -uo pipefail # set accordingly to a runner role # #################################### -echo "Running init script" +echo "Running init v1" export DEBIAN_FRONTEND=noninteractive export RUNNER_HOME=/home/ubuntu/actions-runner @@ -90,7 +90,6 @@ terminate_delayed() { # IF `sleep` IS CHANGED, CHANGE ANOTHER VALUE IN `pgrep` sleep=13.14159265358979323846 echo "Going to terminate the runner's instance in $sleep seconds" - INSTANCE_ID=$(ec2metadata --instance-id) # We execute it with `at` to not have it as an orphan process, but launched independently # GH Runners kill all remain processes echo "sleep '$sleep'; aws ec2 terminate-instances --instance-ids $INSTANCE_ID" | at now || \ @@ -111,11 +110,17 @@ declare -f terminate_delayed >> /tmp/actions-hooks/common.sh terminate_and_exit() { # Terminate instance and exit from the script instantly echo "Going to terminate the runner's instance" - INSTANCE_ID=$(ec2metadata --instance-id) aws ec2 terminate-instances --instance-ids "$INSTANCE_ID" exit 0 } +terminate_decrease_and_exit() { + # Terminate instance and exit from the script instantly + echo "Going to terminate the runner's instance and decrease asg capacity" + aws autoscaling terminate-instance-in-auto-scaling-group --instance-id "$INSTANCE_ID" --should-decrement-desired-capacity + exit 0 +} + declare -f terminate_and_exit >> /tmp/actions-hooks/common.sh check_spot_instance_is_old() { @@ -324,7 +329,7 @@ while true; do sudo -u ubuntu ./config.sh remove --token "$(get_runner_token)" \ || continue echo "Runner didn't launch or have assigned jobs after ${RUNNER_AGE} seconds, shutting down" - terminate_and_exit + terminate_decrease_and_exit fi fi else diff --git a/tests/integration/test_memory_limit/__init__.py b/tests/integration/test_memory_limit/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_memory_limit/configs/async_metrics_no.xml b/tests/integration/test_memory_limit/configs/async_metrics_no.xml new file mode 100644 index 00000000000..96cae3bf387 --- /dev/null +++ b/tests/integration/test_memory_limit/configs/async_metrics_no.xml @@ -0,0 +1,21 @@ + + + 86400 + + + + + + + + + + + + + + + + + + diff --git a/tests/integration/test_memory_limit/test.py b/tests/integration/test_memory_limit/test.py new file mode 100644 index 00000000000..6d6745711da --- /dev/null +++ b/tests/integration/test_memory_limit/test.py @@ -0,0 +1,54 @@ +#!/usr/bin/env python3 +import logging +import time +import pytest +from helpers.cluster import ClickHouseCluster +from multiprocessing.dummy import Pool + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance( + "node", + main_configs=[ + "configs/async_metrics_no.xml", + ], + mem_limit="4g", + env_variables={"MALLOC_CONF": "dirty_decay_ms:0"}, +) + + +@pytest.fixture(scope="module", autouse=True) +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_multiple_queries(): + if node.is_built_with_sanitizer(): + return + + p = Pool(15) + + def run_query(node): + try: + node.query("SELECT * FROM system.numbers GROUP BY number") + except Exception as ex: + print("Exception", ex) + raise ex + + tasks = [] + for i in range(30): + tasks.append(p.apply_async(run_query, (node,))) + time.sleep(i * 0.1) + + for task in tasks: + try: + task.get() + except Exception as ex: + print("Exception", ex) + + # test that we didn't kill the server + node.query("SELECT 1") diff --git a/tests/integration/test_system_detached_tables/__init__.py b/tests/integration/test_system_detached_tables/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_system_detached_tables/configs/remote_servers.xml b/tests/integration/test_system_detached_tables/configs/remote_servers.xml new file mode 100644 index 00000000000..8b579724282 --- /dev/null +++ b/tests/integration/test_system_detached_tables/configs/remote_servers.xml @@ -0,0 +1,13 @@ + + + + + true + + replica1 + 9000 + + + + + \ No newline at end of file diff --git a/tests/integration/test_system_detached_tables/test.py b/tests/integration/test_system_detached_tables/test.py new file mode 100644 index 00000000000..2eb870efcbc --- /dev/null +++ b/tests/integration/test_system_detached_tables/test.py @@ -0,0 +1,112 @@ +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance( + "replica1", + with_zookeeper=True, + main_configs=["configs/remote_servers.xml"], + macros={"replica": "replica1"}, + stay_alive=True, +) + + +@pytest.fixture(scope="module", autouse=True) +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +@pytest.mark.parametrize( + "db_name,db_engine,table_engine,table_engine_params", + [ + pytest.param( + "test_db_atomic", + "Atomic", + "MergeTree", + "ORDER BY n", + id="Atomic db with MergeTree table", + ), + pytest.param( + "test_db_lazy", + "Lazy(60)", + "Log", + "", + id="Lazy db with Log table", + ), + pytest.param( + "test_db_repl", + "Replicated('/clickhouse/tables/test_table','shard1', 'replica1')", + "ReplicatedMergeTree", + "ORDER BY n", + id="Replicated db with ReplicatedMergeTree table", + ), + ], +) +def test_system_detached_tables( + start_cluster, db_name, db_engine, table_engine, table_engine_params +): + node.query(f"CREATE DATABASE IF NOT EXISTS {db_name} ENGINE={db_engine};") + + node.query( + f"CREATE TABLE {db_name}.test_table (n Int64) ENGINE={table_engine} {table_engine_params};" + ) + node.query( + f"CREATE TABLE {db_name}.test_table_perm (n Int64) ENGINE={table_engine} {table_engine_params};" + ) + + test_table_uuid = node.query( + "SELECT uuid FROM system.tables WHERE table='test_table'" + ).rstrip("\n") + test_table_metadata_path = node.query( + "SELECT metadata_path FROM system.tables WHERE table='test_table'" + ).rstrip("\n") + + test_table_perm_uuid = node.query( + "SELECT uuid FROM system.tables WHERE table='test_table_perm'" + ).rstrip("\n") + test_table_perm_metadata_path = node.query( + "SELECT metadata_path FROM system.tables WHERE table='test_table_perm'" + ).rstrip("\n") + + assert "" == node.query( + f"SELECT * FROM system.detached_tables WHERE database='{db_name}'" + ) + + node.query( + f"SET database_replicated_always_detach_permanently=1; DETACH TABLE {db_name}.test_table" + ) + node.query(f"DETACH TABLE {db_name}.test_table_perm PERMANENTLY") + + querry = f"SELECT database, table, is_permanently, uuid, metadata_path FROM system.detached_tables WHERE database='{db_name}' FORMAT Values" + result = node.query(querry) + + if db_engine.startswith("Repl"): + expected_before_restart = f"('{db_name}','test_table',1,'{test_table_uuid}','{test_table_metadata_path}'),('{db_name}','test_table_perm',1,'{test_table_perm_uuid}','{test_table_perm_metadata_path}')" + else: + expected_before_restart = f"('{db_name}','test_table',0,'{test_table_uuid}','{test_table_metadata_path}'),('{db_name}','test_table_perm',1,'{test_table_perm_uuid}','{test_table_perm_metadata_path}')" + + assert result == expected_before_restart + + if db_engine.startswith("Lazy"): + return + + node.restart_clickhouse() + + if db_engine.startswith("Repl"): + expected_after_restart = expected_before_restart + else: + expected_after_restart = f"('{db_name}','test_table_perm',1,'{test_table_perm_uuid}','{test_table_perm_metadata_path}')" + + result = node.query(querry) + assert result == expected_after_restart + + node.restart_clickhouse() + + result = node.query(querry) + assert result == expected_after_restart + + node.query(f"DROP DATABASE {db_name}") diff --git a/tests/integration/test_user_valid_until/test.py b/tests/integration/test_user_valid_until/test.py index d6d5bf8b18e..39ca5997067 100644 --- a/tests/integration/test_user_valid_until/test.py +++ b/tests/integration/test_user_valid_until/test.py @@ -5,7 +5,7 @@ from time import sleep from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node = cluster.add_instance("node") +node = cluster.add_instance("node", stay_alive=True) @pytest.fixture(scope="module") @@ -84,3 +84,22 @@ def test_details(started_cluster): node.query("SHOW CREATE USER user_details_time_only") == f"CREATE USER user_details_time_only VALID UNTIL \\'{until_year}-01-01 22:03:40\\'\n" ) + + +def test_restart(started_cluster): + node.query("CREATE USER user_restart VALID UNTIL '06/11/2010 08:03:20 Z+3'") + + assert ( + node.query("SHOW CREATE USER user_restart") + == "CREATE USER user_restart VALID UNTIL \\'2010-11-06 05:03:20\\'\n" + ) + + node.restart_clickhouse() + + assert ( + node.query("SHOW CREATE USER user_restart") + == "CREATE USER user_restart VALID UNTIL \\'2010-11-06 05:03:20\\'\n" + ) + + error = "Authentication failed" + assert error in node.query_and_get_error("SELECT 1", user="user_restart") diff --git a/tests/queries/0_stateless/02044_url_glob_parallel_connection_refused.sh b/tests/queries/0_stateless/02044_url_glob_parallel_connection_refused.sh index 7e8579f7cbe..b4b0ee8a023 100755 --- a/tests/queries/0_stateless/02044_url_glob_parallel_connection_refused.sh +++ b/tests/queries/0_stateless/02044_url_glob_parallel_connection_refused.sh @@ -8,7 +8,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) i=0 retries=5 # Connecting to wrong address and checking for race condition +# http_max_tries is limited to 2 because with the default 10 retries the execution time might go as high as around 3 minutes (because of exponential back-off). +# because of that we might see wrong 'tests hung' reports depending on how close to the end of tests run this particular test was executed. +# proper fix should be implemented in https://github.com/ClickHouse/ClickHouse/issues/66656 while [[ $i -lt $retries ]]; do - timeout 5s ${CLICKHOUSE_CLIENT} --max_threads 10 --query "SELECT * FROM url('http://128.0.0.{1..10}:${CLICKHOUSE_PORT_HTTP}/?query=SELECT+sleep(1)', TSV, 'x UInt8')" --format Null 2>/dev/null + timeout 5s ${CLICKHOUSE_CLIENT} --max_threads 10 --http_max_tries 2 --query "SELECT * FROM url('http://128.0.0.{1..10}:${CLICKHOUSE_PORT_HTTP}/?query=SELECT+sleep(1)', TSV, 'x UInt8')" --format Null 2>/dev/null ((++i)) done diff --git a/tests/queries/0_stateless/02231_buffer_aggregate_states_leak.sql b/tests/queries/0_stateless/02231_buffer_aggregate_states_leak.sql index f1f7c876ba6..3d6b69fe161 100644 --- a/tests/queries/0_stateless/02231_buffer_aggregate_states_leak.sql +++ b/tests/queries/0_stateless/02231_buffer_aggregate_states_leak.sql @@ -29,7 +29,7 @@ from in_02231 group by key; set optimize_trivial_insert_select = 1; -insert into in_02231 select * from numbers(10e6) settings max_memory_usage='310Mi', max_threads=1; +insert into in_02231 select * from numbers(10e6) settings max_memory_usage='400Mi', max_threads=1; drop table buffer_02231; drop table out_02231; diff --git a/tests/queries/0_stateless/02834_apache_arrow_abort.sql b/tests/queries/0_stateless/02834_apache_arrow_abort.sql index 47e1c5d3951..cb0eaebe554 100644 --- a/tests/queries/0_stateless/02834_apache_arrow_abort.sql +++ b/tests/queries/0_stateless/02834_apache_arrow_abort.sql @@ -1,4 +1,4 @@ --- Tags: no-fasttest +-- Tags: no-fasttest, no-tsan, no-asan, no-msan, no-ubsan -- This tests depends on internet access, but it does not matter, because it only has to check that there is no abort due to a bug in Apache Arrow library. SET optimize_trivial_insert_select=1; INSERT INTO TABLE FUNCTION url('https://clickhouse-public-datasets.s3.amazonaws.com/hits_compatible/athena_partitioned/hits_9.parquet') SELECT * FROM url('https://clickhouse-public-datasets.s3.amazonaws.com/hits_compatible/athena_partitioned/hits_9.parquet'); -- { serverError CANNOT_WRITE_TO_OSTREAM, RECEIVED_ERROR_FROM_REMOTE_IO_SERVER, POCO_EXCEPTION } diff --git a/tests/queries/0_stateless/02903_parameterized_view_explain_ast.reference b/tests/queries/0_stateless/02903_parameterized_view_explain_ast.reference index 6ee8d0c3d23..27222968b1a 100644 --- a/tests/queries/0_stateless/02903_parameterized_view_explain_ast.reference +++ b/tests/queries/0_stateless/02903_parameterized_view_explain_ast.reference @@ -1,4 +1,4 @@ -CreateQuery numbers_pv (children 2) +CreateQuery numbers_pv (children 2) Identifier numbers_pv SelectWithUnionQuery (children 1) ExpressionList (children 1) diff --git a/tests/queries/0_stateless/02970_generate_series.reference b/tests/queries/0_stateless/02970_generate_series.reference index 6e6f3c81587..f8a8e1891b7 100644 --- a/tests/queries/0_stateless/02970_generate_series.reference +++ b/tests/queries/0_stateless/02970_generate_series.reference @@ -5,6 +5,7 @@ 501 50 17928 +17928 0 10 13 diff --git a/tests/queries/0_stateless/02970_generate_series.sql b/tests/queries/0_stateless/02970_generate_series.sql index 0844714b3a6..edae884a561 100644 --- a/tests/queries/0_stateless/02970_generate_series.sql +++ b/tests/queries/0_stateless/02970_generate_series.sql @@ -5,6 +5,7 @@ SELECT count() FROM generate_series(7, 77, 10); SELECT count() FROM generate_series(0, 1000, 2); SELECT count() FROM generate_series(0, 999, 20); SELECT sum(generate_series) FROM generate_series(4, 1008, 4) WHERE generate_series % 7 = 1; +SELECT sum(generate_series) FROM generate_series(4, 1008, 4) WHERE generate_series % 7 = 1 SETTINGS max_block_size = 71; SELECT * FROM generate_series(5, 4); SELECT * FROM generate_series(0, 0); diff --git a/tests/queries/0_stateless/03172_system_detached_tables.reference b/tests/queries/0_stateless/03172_system_detached_tables.reference new file mode 100644 index 00000000000..83d1ff13942 --- /dev/null +++ b/tests/queries/0_stateless/03172_system_detached_tables.reference @@ -0,0 +1,11 @@ +database atomic tests +test03172_system_detached_tables test_table 0 +test03172_system_detached_tables test_table_perm 1 +test03172_system_detached_tables test_table 0 +test03172_system_detached_tables test_table_perm 1 +test03172_system_detached_tables test_table 0 +----------------------- +database lazy tests +before attach test03172_system_detached_tables_lazy test_table 0 +before attach test03172_system_detached_tables_lazy test_table_perm 1 +DROP TABLE diff --git a/tests/queries/0_stateless/03172_system_detached_tables.sql b/tests/queries/0_stateless/03172_system_detached_tables.sql new file mode 100644 index 00000000000..1a3c2d7cc0f --- /dev/null +++ b/tests/queries/0_stateless/03172_system_detached_tables.sql @@ -0,0 +1,53 @@ +-- Tags: no-parallel + +SELECT 'database atomic tests'; +DROP DATABASE IF EXISTS test03172_system_detached_tables; +CREATE DATABASE IF NOT EXISTS test03172_system_detached_tables ENGINE=Atomic; + +CREATE TABLE test03172_system_detached_tables.test_table (n Int64) ENGINE=MergeTree ORDER BY n; +SELECT * FROM system.detached_tables WHERE database='test03172_system_detached_tables'; + +DETACH TABLE test03172_system_detached_tables.test_table; +SELECT database, table, is_permanently FROM system.detached_tables WHERE database='test03172_system_detached_tables'; + +ATTACH TABLE test03172_system_detached_tables.test_table; + +CREATE TABLE test03172_system_detached_tables.test_table_perm (n Int64) ENGINE=MergeTree ORDER BY n; +SELECT * FROM system.detached_tables WHERE database='test03172_system_detached_tables'; + +DETACH TABLE test03172_system_detached_tables.test_table_perm PERMANENTLY; +SELECT database, table, is_permanently FROM system.detached_tables WHERE database='test03172_system_detached_tables'; + +DETACH TABLE test03172_system_detached_tables.test_table SYNC; +SELECT database, table, is_permanently FROM system.detached_tables WHERE database='test03172_system_detached_tables'; + +SELECT database, table, is_permanently FROM system.detached_tables WHERE database='test03172_system_detached_tables' AND table='test_table'; + +DROP DATABASE test03172_system_detached_tables SYNC; + +SELECT '-----------------------'; +SELECT 'database lazy tests'; + +DROP DATABASE IF EXISTS test03172_system_detached_tables_lazy; +CREATE DATABASE test03172_system_detached_tables_lazy Engine=Lazy(10); + +CREATE TABLE test03172_system_detached_tables_lazy.test_table (number UInt64) engine=Log; +INSERT INTO test03172_system_detached_tables_lazy.test_table SELECT * FROM numbers(100); +DETACH TABLE test03172_system_detached_tables_lazy.test_table; + +CREATE TABLE test03172_system_detached_tables_lazy.test_table_perm (number UInt64) engine=Log; +INSERT INTO test03172_system_detached_tables_lazy.test_table_perm SELECT * FROM numbers(100); +DETACH table test03172_system_detached_tables_lazy.test_table_perm PERMANENTLY; + +SELECT 'before attach', database, table, is_permanently FROM system.detached_tables WHERE database='test03172_system_detached_tables_lazy'; + +ATTACH TABLE test03172_system_detached_tables_lazy.test_table; +ATTACH TABLE test03172_system_detached_tables_lazy.test_table_perm; + +SELECT 'after attach', database, table, is_permanently FROM system.detached_tables WHERE database='test03172_system_detached_tables_lazy'; + +SELECT 'DROP TABLE'; +DROP TABLE test03172_system_detached_tables_lazy.test_table SYNC; +DROP TABLE test03172_system_detached_tables_lazy.test_table_perm SYNC; + +DROP DATABASE test03172_system_detached_tables_lazy SYNC; diff --git a/tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql b/tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql index 27fca179580..5d473064c68 100644 --- a/tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql +++ b/tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql @@ -1,4 +1,5 @@ --- Tags: no-random-settings, no-s3-storage +-- Tags: no-random-settings, no-object-storage +-- Tag no-object-storage: this test relies on the number of opened files in MergeTree that can differ in object storages SET allow_experimental_dynamic_type = 1; DROP TABLE IF EXISTS test_dynamic; diff --git a/tests/queries/0_stateless/03203_client_benchmark_options.reference b/tests/queries/0_stateless/03203_client_benchmark_options.reference new file mode 100644 index 00000000000..fd2996b1c78 --- /dev/null +++ b/tests/queries/0_stateless/03203_client_benchmark_options.reference @@ -0,0 +1,6 @@ +Ok +Ok +Ok +Ok +Ok +Ok diff --git a/tests/queries/0_stateless/03203_client_benchmark_options.sh b/tests/queries/0_stateless/03203_client_benchmark_options.sh new file mode 100755 index 00000000000..a9b9d69822b --- /dev/null +++ b/tests/queries/0_stateless/03203_client_benchmark_options.sh @@ -0,0 +1,13 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} -t -q "SELECT sleepEachRow(2) FORMAT Null" 2>&1 | grep -q "^2\." && echo "Ok" || echo "Fail" +${CLICKHOUSE_CLIENT} --time -q "SELECT sleepEachRow(2) FORMAT Null" 2>&1 | grep -q "^2\." && echo "Ok" || echo "Fail" +${CLICKHOUSE_CLIENT} --memory-usage -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" 2>&1 | grep -q "^[0-9]\+$" && echo "Ok" || echo "Fail" +${CLICKHOUSE_CLIENT} --memory-usage=none -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" # expected no output +${CLICKHOUSE_CLIENT} --memory-usage=default -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" 2>&1 | grep -q "^[0-9]\+$" && echo "Ok" || echo "Fail" +${CLICKHOUSE_CLIENT} --memory-usage=readable -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" 2>&1 | grep -q "^[0-9].*B$" && echo "Ok" || echo "Fail" +${CLICKHOUSE_CLIENT} --memory-usage=unknown -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" 2>&1 | grep -q "BAD_ARGUMENTS" && echo "Ok" || echo "Fail" diff --git a/tests/queries/0_stateless/03203_system_numbers_limit_and_offset_complex.reference b/tests/queries/0_stateless/03203_system_numbers_limit_and_offset_complex.reference new file mode 100644 index 00000000000..d38b21d2d05 --- /dev/null +++ b/tests/queries/0_stateless/03203_system_numbers_limit_and_offset_complex.reference @@ -0,0 +1,14 @@ +18679 31 +0 +10 +20 +30 +40 +50 +60 +70 +80 +90 +100 +110 +4250 diff --git a/tests/queries/0_stateless/03203_system_numbers_limit_and_offset_complex.sql b/tests/queries/0_stateless/03203_system_numbers_limit_and_offset_complex.sql new file mode 100644 index 00000000000..756e08da27d --- /dev/null +++ b/tests/queries/0_stateless/03203_system_numbers_limit_and_offset_complex.sql @@ -0,0 +1,38 @@ +--- The following query was buggy before, so let's use it as a test case +WITH + (num > 1) AND (arraySum(arrayMap(y -> ((y > 1) AND (y < num) AND ((num % y) = 0)), range(toUInt64(sqrt(num)) + 1))) = 0) AS is_prime_slow +SELECT + num, + ds, +FROM +( + WITH + arraySum(arrayMap(y -> toUInt8(y), splitByString('', toString(num)))) AS digits_sum + SELECT + 1 + (number * 2) AS num, + digits_sum AS ds + FROM numbers_mt(10000) + WHERE ds IN ( + WITH + (number > 1) AND (arraySum(arrayMap(y -> ((y > 1) AND (y < number) AND ((number % y) = 0)), range(toUInt64(sqrt(number)) + 1))) = 0) AS is_prime_slow + SELECT number + FROM numbers(180 + 1) + WHERE is_prime_slow + ) +) +WHERE is_prime_slow +ORDER BY num ASC +LIMIT 998, 1 +SETTINGS max_block_size = 64, max_threads=16; + +SELECT number +FROM numbers_mt(120) +WHERE (number % 10) = 0 +ORDER BY number ASC +SETTINGS max_block_size = 31, max_threads = 11; + +SELECT number +FROM numbers_mt(4242, 9) +WHERE (number % 10) = 0 +ORDER BY number ASC +SETTINGS max_block_size = 31, max_threads = 11; diff --git a/tests/queries/0_stateless/03203_system_numbers_limit_and_offset_simple.reference b/tests/queries/0_stateless/03203_system_numbers_limit_and_offset_simple.reference new file mode 100644 index 00000000000..6aad3ee0b4b --- /dev/null +++ b/tests/queries/0_stateless/03203_system_numbers_limit_and_offset_simple.reference @@ -0,0 +1,6 @@ +case 1 +9900 +9910 +9920 +case 2 +9990 diff --git a/tests/queries/0_stateless/03203_system_numbers_limit_and_offset_simple.sql b/tests/queries/0_stateless/03203_system_numbers_limit_and_offset_simple.sql new file mode 100644 index 00000000000..df01ddf4312 --- /dev/null +++ b/tests/queries/0_stateless/03203_system_numbers_limit_and_offset_simple.sql @@ -0,0 +1,11 @@ +SELECT 'case 1'; +SELECT number FROM numbers_mt(10000) +WHERE (number % 10) = 0 +ORDER BY number ASC +LIMIT 990, 3; + +SELECT 'case 2'; +SELECT number FROM numbers_mt(10000) +WHERE (number % 10) = 0 +ORDER BY number ASC +LIMIT 999, 20 SETTINGS max_block_size = 31; diff --git a/tests/queries/0_stateless/03205_hashing_empty_tuples.reference b/tests/queries/0_stateless/03205_hashing_empty_tuples.reference new file mode 100644 index 00000000000..e24b5809aee --- /dev/null +++ b/tests/queries/0_stateless/03205_hashing_empty_tuples.reference @@ -0,0 +1,22 @@ +16324913028386710556 +16324913028386710556 +5049034479224883533 +7385293435322750976 +12248912094175844631 +5049034479224883533 +5887129541803688833 +5887129541803688833 +13747979201178469747 +5887129541803688833 +15520217392480966957 +16324913028386710556 +16324913028386710556 +5049034479224883533 +7385293435322750976 +12248912094175844631 +5049034479224883533 +5887129541803688833 +5887129541803688833 +13747979201178469747 +5887129541803688833 +15520217392480966957 diff --git a/tests/queries/0_stateless/03205_hashing_empty_tuples.sql b/tests/queries/0_stateless/03205_hashing_empty_tuples.sql new file mode 100644 index 00000000000..4a97f30ced3 --- /dev/null +++ b/tests/queries/0_stateless/03205_hashing_empty_tuples.sql @@ -0,0 +1,23 @@ +SELECT sipHash64(()); +SELECT sipHash64((), ()); +SELECT sipHash64((), 1); +SELECT sipHash64(1, ()); +SELECT sipHash64(1, (), 1); +SELECT sipHash64((), 1, ()); +SELECT sipHash64((), (1, 2)); +SELECT sipHash64((), (1, 2)); +SELECT sipHash64((1, 2), ()); +SELECT sipHash64((), (1, 2), ()); +SELECT sipHash64((1, 2), (), (3, 4)); + +SELECT sipHash64(materialize(())); +SELECT sipHash64(materialize(()), materialize(())); +SELECT sipHash64(materialize(()), 1); +SELECT sipHash64(1, materialize(())); +SELECT sipHash64(1, materialize(()), 1); +SELECT sipHash64((), 1, materialize(())); +SELECT sipHash64(materialize(()), (1, 2)); +SELECT sipHash64(materialize(()), (1, 2)); +SELECT sipHash64((1, 2), materialize(())); +SELECT sipHash64(materialize(()), (1, 2), ()); +SELECT sipHash64((1, 2), materialize(()), (3, 4)); diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 4e7bf7f686e..b4b18f9bbf6 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1653,6 +1653,7 @@ formated formatschema formatter formatters +fqdn frac freezed fromDaysSinceYearZero