diff --git a/README.md b/README.md index 3b5209dcbe9..96dec2ca607 100644 --- a/README.md +++ b/README.md @@ -42,17 +42,18 @@ Keep an eye out for upcoming meetups and events around the world. Somewhere else Upcoming meetups -* [Jakarta Meetup](https://www.meetup.com/clickhouse-indonesia-user-group/events/303191359/) - October 1 -* [Singapore Meetup](https://www.meetup.com/clickhouse-singapore-meetup-group/events/303212064/) - October 3 -* [Madrid Meetup](https://www.meetup.com/clickhouse-spain-user-group/events/303096564/) - October 22 * [Oslo Meetup](https://www.meetup.com/open-source-real-time-data-warehouse-real-time-analytics/events/302938622) - October 31 * [Barcelona Meetup](https://www.meetup.com/clickhouse-spain-user-group/events/303096876/) - November 12 * [Ghent Meetup](https://www.meetup.com/clickhouse-belgium-user-group/events/303049405/) - November 19 * [Dubai Meetup](https://www.meetup.com/clickhouse-dubai-meetup-group/events/303096989/) - November 21 * [Paris Meetup](https://www.meetup.com/clickhouse-france-user-group/events/303096434) - November 26 +* [New York Meetup](https://www.meetup.com/clickhouse-new-york-user-group/events/304268174) - December 9 Recently completed meetups +* [Madrid Meetup](https://www.meetup.com/clickhouse-spain-user-group/events/303096564/) - October 22 +* [Singapore Meetup](https://www.meetup.com/clickhouse-singapore-meetup-group/events/303212064/) - October 3 +* [Jakarta Meetup](https://www.meetup.com/clickhouse-indonesia-user-group/events/303191359/) - October 1 * [ClickHouse Guangzhou User Group Meetup](https://mp.weixin.qq.com/s/GSvo-7xUoVzCsuUvlLTpCw) - August 25 * [Seattle Meetup (Statsig)](https://www.meetup.com/clickhouse-seattle-user-group/events/302518075/) - August 27 * [Melbourne Meetup](https://www.meetup.com/clickhouse-australia-user-group/events/302732666/) - August 27 diff --git a/docs/en/sql-reference/statements/grant.md b/docs/en/sql-reference/statements/grant.md index c11299baf38..19305675ec8 100644 --- a/docs/en/sql-reference/statements/grant.md +++ b/docs/en/sql-reference/statements/grant.md @@ -78,6 +78,10 @@ Specifying privileges you can use asterisk (`*`) instead of a table or a databas Also, you can omit database name. In this case privileges are granted for current database. For example, `GRANT SELECT ON * TO john` grants the privilege on all the tables in the current database, `GRANT SELECT ON mytable TO john` grants the privilege on the `mytable` table in the current database. +:::note +The feature described below is available starting with the 24.10 ClickHouse version. +::: + You can also put asterisks at the end of a table or a database name. This feature allows you to grant privileges on an abstract prefix of the table's path. Example: `GRANT SELECT ON db.my_tables* TO john`. This query allows `john` to execute the `SELECT` query over all the `db` database tables with the prefix `my_tables*`. diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index f256c4d740c..468b39dbe3b 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -207,7 +207,6 @@ namespace ServerSetting extern const ServerSettingsBool format_alter_operations_with_parentheses; extern const ServerSettingsUInt64 global_profiler_cpu_time_period_ns; extern const ServerSettingsUInt64 global_profiler_real_time_period_ns; - extern const ServerSettingsDouble gwp_asan_force_sample_probability; extern const ServerSettingsUInt64 http_connections_soft_limit; extern const ServerSettingsUInt64 http_connections_store_limit; extern const ServerSettingsUInt64 http_connections_warn_limit; @@ -622,7 +621,7 @@ void sanityChecks(Server & server) #if defined(OS_LINUX) try { - const std::unordered_set fastClockSources = { + const std::unordered_set fast_clock_sources = { // ARM clock "arch_sys_counter", // KVM guest clock @@ -631,7 +630,7 @@ void sanityChecks(Server & server) "tsc", }; const char * filename = "/sys/devices/system/clocksource/clocksource0/current_clocksource"; - if (!fastClockSources.contains(readLine(filename))) + if (!fast_clock_sources.contains(readLine(filename))) server.context()->addWarningMessage("Linux is not using a fast clock source. Performance can be degraded. Check " + String(filename)); } catch (...) // NOLINT(bugprone-empty-catch) @@ -1930,10 +1929,6 @@ try if (global_context->isServerCompletelyStarted()) CannotAllocateThreadFaultInjector::setFaultProbability(new_server_settings[ServerSetting::cannot_allocate_thread_fault_injection_probability]); -#if USE_GWP_ASAN - GWPAsan::setForceSampleProbability(new_server_settings[ServerSetting::gwp_asan_force_sample_probability]); -#endif - ProfileEvents::increment(ProfileEvents::MainConfigLoads); /// Must be the last. @@ -2441,7 +2436,6 @@ try #if USE_GWP_ASAN GWPAsan::initFinished(); - GWPAsan::setForceSampleProbability(server_settings[ServerSetting::gwp_asan_force_sample_probability]); #endif try diff --git a/src/Common/GWPAsan.cpp b/src/Common/GWPAsan.cpp index de6991191ea..a210fb3a73a 100644 --- a/src/Common/GWPAsan.cpp +++ b/src/Common/GWPAsan.cpp @@ -57,7 +57,7 @@ static bool guarded_alloc_initialized = [] opts.MaxSimultaneousAllocations = 1024; if (!env_options_raw || !std::string_view{env_options_raw}.contains("SampleRate")) - opts.SampleRate = 10000; + opts.SampleRate = 0; const char * collect_stacktraces = std::getenv("GWP_ASAN_COLLECT_STACKTRACES"); // NOLINT(concurrency-mt-unsafe) if (collect_stacktraces && std::string_view{collect_stacktraces} == "1") diff --git a/src/Common/GWPAsan.h b/src/Common/GWPAsan.h index 846c3417db4..c01a1130739 100644 --- a/src/Common/GWPAsan.h +++ b/src/Common/GWPAsan.h @@ -8,7 +8,6 @@ #include #include -#include namespace GWPAsan { @@ -39,14 +38,6 @@ inline bool shouldSample() return init_finished.load(std::memory_order_relaxed) && GuardedAlloc.shouldSample(); } -inline bool shouldForceSample() -{ - if (!init_finished.load(std::memory_order_relaxed)) - return false; - std::bernoulli_distribution dist(force_sample_probability.load(std::memory_order_relaxed)); - return dist(thread_local_rng); -} - } #endif diff --git a/src/Common/PODArray.h b/src/Common/PODArray.h index 48f2ffee8ce..2d69b8ac26c 100644 --- a/src/Common/PODArray.h +++ b/src/Common/PODArray.h @@ -115,11 +115,6 @@ protected: template void alloc(size_t bytes, TAllocatorParams &&... allocator_params) { -#if USE_GWP_ASAN - if (unlikely(GWPAsan::shouldForceSample())) - gwp_asan::getThreadLocals()->NextSampleCounter = 1; -#endif - char * allocated = reinterpret_cast(TAllocator::alloc(bytes, std::forward(allocator_params)...)); c_start = allocated + pad_left; @@ -149,11 +144,6 @@ protected: return; } -#if USE_GWP_ASAN - if (unlikely(GWPAsan::shouldForceSample())) - gwp_asan::getThreadLocals()->NextSampleCounter = 1; -#endif - unprotect(); ptrdiff_t end_diff = c_end - c_start; diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp index 34b1f306e86..ead40061493 100644 --- a/src/Core/ServerSettings.cpp +++ b/src/Core/ServerSettings.cpp @@ -184,7 +184,6 @@ namespace DB DECLARE(String, merge_workload, "default", "Name of workload to be used to access resources for all merges (may be overridden by a merge tree setting)", 0) \ DECLARE(String, mutation_workload, "default", "Name of workload to be used to access resources for all mutations (may be overridden by a merge tree setting)", 0) \ DECLARE(Bool, prepare_system_log_tables_on_startup, false, "If true, ClickHouse creates all configured `system.*_log` tables before the startup. It can be helpful if some startup scripts depend on these tables.", 0) \ - DECLARE(Double, gwp_asan_force_sample_probability, 0.0003, "Probability that an allocation from specific places will be sampled by GWP Asan (i.e. PODArray allocations)", 0) \ DECLARE(UInt64, config_reload_interval_ms, 2000, "How often clickhouse will reload config and check for new changes", 0) \ DECLARE(UInt64, memory_worker_period_ms, 0, "Tick period of background memory worker which corrects memory tracker memory usages and cleans up unused pages during higher memory usage. If set to 0, default value will be used depending on the memory usage source", 0) \ DECLARE(Bool, disable_insertion_and_mutation, false, "Disable all insert/alter/delete queries. This setting will be enabled if someone needs read-only nodes to prevent insertion and mutation affect reading performance.", 0) \ diff --git a/src/Functions/IFunction.cpp b/src/Functions/IFunction.cpp index 10a25cfe0d0..68d4f25f08d 100644 --- a/src/Functions/IFunction.cpp +++ b/src/Functions/IFunction.cpp @@ -1,27 +1,28 @@ #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 "config.h" +#include +#include + #if USE_EMBEDDED_COMPILER # include #endif @@ -451,6 +452,7 @@ FunctionBasePtr IFunctionOverloadResolver::build(const ColumnsWithTypeAndName & /// Use FunctionBaseDynamicAdaptor if default implementation for Dynamic is enabled and we have Dynamic type in arguments. if (useDefaultImplementationForDynamic()) { + checkNumberOfArguments(arguments.size()); for (const auto & arg : arguments) { if (isDynamic(arg.type)) diff --git a/src/IO/BufferWithOwnMemory.h b/src/IO/BufferWithOwnMemory.h index da38bccdea1..79b1bb67aaa 100644 --- a/src/IO/BufferWithOwnMemory.h +++ b/src/IO/BufferWithOwnMemory.h @@ -44,16 +44,10 @@ struct Memory : boost::noncopyable, Allocator char * m_data = nullptr; size_t alignment = 0; - [[maybe_unused]] bool allow_gwp_asan_force_sample{false}; - Memory() = default; /// If alignment != 0, then allocate memory aligned to specified value. - explicit Memory(size_t size_, size_t alignment_ = 0, bool allow_gwp_asan_force_sample_ = false) - : alignment(alignment_), allow_gwp_asan_force_sample(allow_gwp_asan_force_sample_) - { - alloc(size_); - } + explicit Memory(size_t size_, size_t alignment_ = 0) : alignment(alignment_) { alloc(size_); } ~Memory() { @@ -133,11 +127,6 @@ private: ProfileEvents::increment(ProfileEvents::IOBufferAllocs); ProfileEvents::increment(ProfileEvents::IOBufferAllocBytes, new_capacity); -#if USE_GWP_ASAN - if (unlikely(allow_gwp_asan_force_sample && GWPAsan::shouldForceSample())) - gwp_asan::getThreadLocals()->NextSampleCounter = 1; -#endif - m_data = static_cast(Allocator::alloc(new_capacity, alignment)); m_capacity = new_capacity; m_size = new_size; @@ -165,7 +154,7 @@ protected: public: /// If non-nullptr 'existing_memory' is passed, then buffer will not create its own memory and will use existing_memory without ownership. explicit BufferWithOwnMemory(size_t size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr, size_t alignment = 0) - : Base(nullptr, 0), memory(existing_memory ? 0 : size, alignment, /*allow_gwp_asan_force_sample_=*/true) + : Base(nullptr, 0), memory(existing_memory ? 0 : size, alignment) { Base::set(existing_memory ? existing_memory : memory.data(), size); Base::padded = !existing_memory; diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 921c53b6bcb..e7e4ae25a68 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -301,6 +301,9 @@ void TCPHandler::runImpl() { receiveHello(); + if (!default_database.empty()) + DatabaseCatalog::instance().assertDatabaseExists(default_database); + /// In interserver mode queries are executed without a session context. if (!is_interserver_mode) session->makeSessionContext(); diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueOrderedFileMetadata.cpp b/src/Storages/ObjectStorageQueue/ObjectStorageQueueOrderedFileMetadata.cpp index 75a96328051..72e9e073f27 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueOrderedFileMetadata.cpp +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueOrderedFileMetadata.cpp @@ -381,10 +381,10 @@ void ObjectStorageQueueOrderedFileMetadata::setProcessedImpl() /// In one zookeeper transaction do the following: enum RequestType { - SET_MAX_PROCESSED_PATH = 0, - CHECK_PROCESSING_ID_PATH = 1, /// Optional. - REMOVE_PROCESSING_ID_PATH = 2, /// Optional. - REMOVE_PROCESSING_PATH = 3, /// Optional. + CHECK_PROCESSING_ID_PATH = 0, + REMOVE_PROCESSING_ID_PATH = 1, + REMOVE_PROCESSING_PATH = 2, + SET_MAX_PROCESSED_PATH = 3, }; const auto zk_client = getZooKeeper(); @@ -409,8 +409,18 @@ void ObjectStorageQueueOrderedFileMetadata::setProcessedImpl() return; } + bool unexpected_error = false; if (Coordination::isHardwareError(code)) failure_reason = "Lost connection to keeper"; + else if (is_request_failed(CHECK_PROCESSING_ID_PATH)) + failure_reason = "Version of processing id node changed"; + else if (is_request_failed(REMOVE_PROCESSING_PATH)) + { + /// Remove processing_id node should not actually fail + /// because we just checked in a previous keeper request that it exists and has a certain version. + unexpected_error = true; + failure_reason = "Failed to remove processing id path"; + } else if (is_request_failed(SET_MAX_PROCESSED_PATH)) { LOG_TRACE(log, "Cannot set file {} as processed. " @@ -418,13 +428,12 @@ void ObjectStorageQueueOrderedFileMetadata::setProcessedImpl() "Will retry.", path, code); continue; } - else if (is_request_failed(CHECK_PROCESSING_ID_PATH)) - failure_reason = "Version of processing id node changed"; - else if (is_request_failed(REMOVE_PROCESSING_PATH)) - failure_reason = "Failed to remove processing path"; else throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected state of zookeeper transaction: {}", code); + if (unexpected_error) + throw Exception(ErrorCodes::LOGICAL_ERROR, "{}", failure_reason); + LOG_WARNING(log, "Cannot set file {} as processed: {}. Reason: {}", path, code, failure_reason); return; } diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueUnorderedFileMetadata.cpp b/src/Storages/ObjectStorageQueue/ObjectStorageQueueUnorderedFileMetadata.cpp index 40751d9c332..2050797a2ea 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueUnorderedFileMetadata.cpp +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueUnorderedFileMetadata.cpp @@ -103,29 +103,46 @@ void ObjectStorageQueueUnorderedFileMetadata::setProcessedImpl() /// In one zookeeper transaction do the following: enum RequestType { - SET_MAX_PROCESSED_PATH = 0, - CHECK_PROCESSING_ID_PATH = 1, /// Optional. - REMOVE_PROCESSING_ID_PATH = 2, /// Optional. - REMOVE_PROCESSING_PATH = 3, /// Optional. + CHECK_PROCESSING_ID_PATH, + REMOVE_PROCESSING_ID_PATH, + REMOVE_PROCESSING_PATH, + SET_PROCESSED_PATH, }; const auto zk_client = getZooKeeper(); - std::string failure_reason; - Coordination::Requests requests; - requests.push_back( - zkutil::makeCreateRequest( - processed_node_path, node_metadata.toString(), zkutil::CreateMode::Persistent)); + std::map request_index; if (processing_id_version.has_value()) { requests.push_back(zkutil::makeCheckRequest(processing_node_id_path, processing_id_version.value())); requests.push_back(zkutil::makeRemoveRequest(processing_node_id_path, processing_id_version.value())); requests.push_back(zkutil::makeRemoveRequest(processing_node_path, -1)); + + /// The order is important: + /// we must first check processing nodes and set processed_path the last. + request_index[CHECK_PROCESSING_ID_PATH] = 0; + request_index[REMOVE_PROCESSING_ID_PATH] = 1; + request_index[REMOVE_PROCESSING_PATH] = 2; + request_index[SET_PROCESSED_PATH] = 3; + } + else + { + request_index[SET_PROCESSED_PATH] = 0; } + requests.push_back( + zkutil::makeCreateRequest( + processed_node_path, node_metadata.toString(), zkutil::CreateMode::Persistent)); + Coordination::Responses responses; - auto is_request_failed = [&](RequestType type) { return responses[type]->error != Coordination::Error::ZOK; }; + auto is_request_failed = [&](RequestType type) + { + if (!request_index.contains(type)) + return false; + chassert(request_index[type] < responses.size()); + return responses[request_index[type]]->error != Coordination::Error::ZOK; + }; const auto code = zk_client->tryMulti(requests, responses); if (code == Coordination::Error::ZOK) @@ -140,18 +157,41 @@ void ObjectStorageQueueUnorderedFileMetadata::setProcessedImpl() return; } + bool unexpected_error = false; + std::string failure_reason; + if (Coordination::isHardwareError(code)) + { failure_reason = "Lost connection to keeper"; - else if (is_request_failed(SET_MAX_PROCESSED_PATH)) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Cannot create a persistent node in /processed since it already exists"); + } else if (is_request_failed(CHECK_PROCESSING_ID_PATH)) + { + /// This is normal in case of expired session with keeper. failure_reason = "Version of processing id node changed"; + } + else if (is_request_failed(REMOVE_PROCESSING_ID_PATH)) + { + /// Remove processing_id node should not actually fail + /// because we just checked in a previous keeper request that it exists and has a certain version. + unexpected_error = true; + failure_reason = "Failed to remove processing id path"; + } else if (is_request_failed(REMOVE_PROCESSING_PATH)) + { + /// This is normal in case of expired session with keeper as this node is ephemeral. failure_reason = "Failed to remove processing path"; + } + else if (is_request_failed(SET_PROCESSED_PATH)) + { + unexpected_error = true; + failure_reason = "Cannot create a persistent node in /processed since it already exists"; + } else throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected state of zookeeper transaction: {}", code); + if (unexpected_error) + throw Exception(ErrorCodes::LOGICAL_ERROR, "{}", failure_reason); + LOG_WARNING(log, "Cannot set file {} as processed: {}. Reason: {}", path, code, failure_reason); } diff --git a/tests/queries/0_stateless/03252_check_number_of_arguments_for_dynamic.reference b/tests/queries/0_stateless/03252_check_number_of_arguments_for_dynamic.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03252_check_number_of_arguments_for_dynamic.sql b/tests/queries/0_stateless/03252_check_number_of_arguments_for_dynamic.sql new file mode 100644 index 00000000000..79a8617930e --- /dev/null +++ b/tests/queries/0_stateless/03252_check_number_of_arguments_for_dynamic.sql @@ -0,0 +1,18 @@ +set enable_analyzer=1; +set allow_experimental_json_type=1; + +CREATE TABLE t +( + `a` JSON +) +ENGINE = MergeTree() +ORDER BY tuple(); + +insert into t values ('{"a":1}'), ('{"a":2.0}'); + +SELECT 1 +FROM +( + SELECT 1 AS c0 +) AS tx +FULL OUTER JOIN t AS t2 ON equals(t2.a.Float32); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } diff --git a/tests/queries/0_stateless/03258_nonexistent_db.reference b/tests/queries/0_stateless/03258_nonexistent_db.reference new file mode 100644 index 00000000000..825bae3beaa --- /dev/null +++ b/tests/queries/0_stateless/03258_nonexistent_db.reference @@ -0,0 +1,2 @@ +UNKNOWN_DATABASE +OK diff --git a/tests/queries/0_stateless/03258_nonexistent_db.sh b/tests/queries/0_stateless/03258_nonexistent_db.sh new file mode 100755 index 00000000000..847d692c440 --- /dev/null +++ b/tests/queries/0_stateless/03258_nonexistent_db.sh @@ -0,0 +1,7 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +timeout 5 ${CLICKHOUSE_CLIENT_BINARY} --database "nonexistent" 2>&1 | grep -o "UNKNOWN_DATABASE" && echo "OK" || echo "FAIL"