diff --git a/CMakeLists.txt b/CMakeLists.txt index ab976612401..8f54b0b2236 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -433,6 +433,11 @@ else() link_libraries(global-group) endif () +option (ENABLE_GWP_ASAN "Enable Gwp-Asan" ON) +if (NOT OS_LINUX AND NOT OS_ANDROID) + set(ENABLE_GWP_ASAN OFF) +endif () + option(WERROR "Enable -Werror compiler option" ON) if (WERROR) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index cb2e62fd027..7d22f54bf89 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -114,6 +114,7 @@ endif() add_contrib (llvm-project-cmake llvm-project) add_contrib (libfuzzer-cmake llvm-project) +add_contrib (gwpasan-cmake llvm-project) add_contrib (libxml2-cmake libxml2) add_contrib (aws-cmake diff --git a/contrib/gwpasan-cmake/CMakeLists.txt b/contrib/gwpasan-cmake/CMakeLists.txt new file mode 100644 index 00000000000..cc88903eecd --- /dev/null +++ b/contrib/gwpasan-cmake/CMakeLists.txt @@ -0,0 +1,24 @@ +if (NOT ENABLE_GWP_ASAN) + message (STATUS "Not using gwp-asan") + return () +endif () + +set(COMPILER_RT_GWP_ASAN_SRC_DIR "${ClickHouse_SOURCE_DIR}/contrib/llvm-project/compiler-rt/lib/gwp_asan") + +set(GWP_ASAN_SOURCES + ${COMPILER_RT_GWP_ASAN_SRC_DIR}/common.cpp + ${COMPILER_RT_GWP_ASAN_SRC_DIR}/crash_handler.cpp + ${COMPILER_RT_GWP_ASAN_SRC_DIR}/platform_specific/common_posix.cpp + ${COMPILER_RT_GWP_ASAN_SRC_DIR}/platform_specific/guarded_pool_allocator_posix.cpp + ${COMPILER_RT_GWP_ASAN_SRC_DIR}/platform_specific/mutex_posix.cpp + ${COMPILER_RT_GWP_ASAN_SRC_DIR}/platform_specific/utilities_posix.cpp + ${COMPILER_RT_GWP_ASAN_SRC_DIR}/guarded_pool_allocator.cpp + ${COMPILER_RT_GWP_ASAN_SRC_DIR}/stack_trace_compressor.cpp + ${COMPILER_RT_GWP_ASAN_SRC_DIR}/optional/options_parser.cpp +) + +set(GWP_ASAN_HEADERS "${ClickHouse_SOURCE_DIR}/contrib/llvm-project/compiler-rt/lib") + +add_library(_gwp_asan ${GWP_ASAN_SOURCES}) +target_include_directories (_gwp_asan SYSTEM PUBLIC ${GWP_ASAN_HEADERS}) +add_library(ch_contrib::gwp_asan ALIAS _gwp_asan) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 32224056114..da90840ab90 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3310,6 +3310,15 @@ SELECT FROM fuse_tbl ``` +## optimize_rewrite_aggregate_function_with_if + +Rewrite aggregate functions with if expression as argument when logically equivalent. +For example, `avg(if(cond, col, null))` can be rewritten to `avgOrNullIf(cond, col)`. It may improve performance. + +:::note +Supported only with experimental analyzer (`allow_experimental_analyzer = 1`). +::: + ## allow_experimental_database_replicated {#allow_experimental_database_replicated} Enables to create databases with [Replicated](../../engines/database-engines/replicated.md) engine. diff --git a/programs/main.cpp b/programs/main.cpp index 389eae92091..83e64b8c932 100644 --- a/programs/main.cpp +++ b/programs/main.cpp @@ -433,7 +433,6 @@ extern "C" } #endif - /// This allows to implement assert to forbid initialization of a class in static constructors. /// Usage: /// diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 82958394224..8c9893c5c93 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -304,6 +304,11 @@ if (TARGET ch_contrib::llvm) dbms_target_link_libraries (PUBLIC ch_contrib::llvm) endif () +if (TARGET ch_contrib::gwp_asan) + target_link_libraries (clickhouse_common_io PRIVATE ch_contrib::gwp_asan) + target_link_libraries (clickhouse_new_delete PRIVATE ch_contrib::gwp_asan) +endif() + # Otherwise it will slow down stack traces printing too much. set_source_files_properties( Common/Elf.cpp diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index ca0423ebc1b..b632ea95928 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -157,6 +157,19 @@ static void deleteAttributesRecursive(Node * root) } } +static void mergeAttributes(Element & config_element, Element & with_element) +{ + auto * with_element_attributes = with_element.attributes(); + + for (size_t i = 0; i < with_element_attributes->length(); ++i) + { + auto * attr = with_element_attributes->item(i); + config_element.setAttribute(attr->nodeName(), attr->getNodeValue()); + } + + with_element_attributes->release(); +} + void ConfigProcessor::mergeRecursive(XMLDocumentPtr config, Node * config_root, const Node * with_root) { const NodeListPtr with_nodes = with_root->childNodes(); @@ -211,6 +224,9 @@ void ConfigProcessor::mergeRecursive(XMLDocumentPtr config, Node * config_root, } else { + Element & config_element = dynamic_cast(*config_node); + + mergeAttributes(config_element, with_element); mergeRecursive(config, config_node, with_node); } merged = true; diff --git a/src/Common/config.h.in b/src/Common/config.h.in index 36dfd13534e..66fa014c5af 100644 --- a/src/Common/config.h.in +++ b/src/Common/config.h.in @@ -24,6 +24,7 @@ #cmakedefine01 USE_ODBC #cmakedefine01 USE_REPLXX #cmakedefine01 USE_JEMALLOC +#cmakedefine01 USE_GWP_ASAN #cmakedefine01 USE_H3 #cmakedefine01 USE_S2_GEOMETRY #cmakedefine01 USE_FASTOPS diff --git a/src/Common/memory.h b/src/Common/memory.h index 4cb1c535e56..f5d665c4dc2 100644 --- a/src/Common/memory.h +++ b/src/Common/memory.h @@ -17,6 +17,12 @@ # include #endif +#if USE_GWP_ASAN +# include + +static gwp_asan::GuardedPoolAllocator GuardedAlloc; +#endif + namespace Memory { @@ -29,6 +35,23 @@ template ... TAlign> requires DB::OptionalArgument inline ALWAYS_INLINE void * newImpl(std::size_t size, TAlign... align) { +#if USE_GWP_ASAN + if (unlikely(GuardedAlloc.shouldSample())) + { + if constexpr (sizeof...(TAlign) == 1) + { + if (void * ptr = GuardedAlloc.allocate(size, alignToSizeT(align...))) + return ptr; + } + else + { + if (void * ptr = GuardedAlloc.allocate(size)) + return ptr; + + } + } +#endif + void * ptr = nullptr; if constexpr (sizeof...(TAlign) == 1) ptr = aligned_alloc(alignToSizeT(align...), size); @@ -44,16 +67,37 @@ inline ALWAYS_INLINE void * newImpl(std::size_t size, TAlign... align) inline ALWAYS_INLINE void * newNoExept(std::size_t size) noexcept { +#if USE_GWP_ASAN + if (unlikely(GuardedAlloc.shouldSample())) + { + if (void * ptr = GuardedAlloc.allocate(size)) + return ptr; + } +#endif return malloc(size); } inline ALWAYS_INLINE void * newNoExept(std::size_t size, std::align_val_t align) noexcept { +#if USE_GWP_ASAN + if (unlikely(GuardedAlloc.shouldSample())) + { + if (void * ptr = GuardedAlloc.allocate(size, alignToSizeT(align))) + return ptr; + } +#endif return aligned_alloc(static_cast(align), size); } inline ALWAYS_INLINE void deleteImpl(void * ptr) noexcept { +#if USE_GWP_ASAN + if (unlikely(GuardedAlloc.pointerIsMine(ptr))) + { + GuardedAlloc.deallocate(ptr); + return; + } +#endif free(ptr); } @@ -66,6 +110,14 @@ inline ALWAYS_INLINE void deleteSized(void * ptr, std::size_t size, TAlign... al if (unlikely(ptr == nullptr)) return; +#if USE_GWP_ASAN + if (unlikely(GuardedAlloc.pointerIsMine(ptr))) + { + GuardedAlloc.deallocate(ptr); + return; + } +#endif + if constexpr (sizeof...(TAlign) == 1) sdallocx(ptr, size, MALLOCX_ALIGN(alignToSizeT(align...))); else @@ -78,6 +130,13 @@ template ... TAlign> requires DB::OptionalArgument inline ALWAYS_INLINE void deleteSized(void * ptr, std::size_t size [[maybe_unused]], TAlign... /* align */) noexcept { +#if USE_GWP_ASAN + if (unlikely(GuardedAlloc.pointerIsMine(ptr))) + { + GuardedAlloc.deallocate(ptr); + return; + } +#endif free(ptr); } @@ -122,6 +181,16 @@ template ... TAlign> requires DB::OptionalArgument inline ALWAYS_INLINE void untrackMemory(void * ptr [[maybe_unused]], std::size_t size [[maybe_unused]] = 0, TAlign... align [[maybe_unused]]) noexcept { +#if USE_GWP_ASAN + if (unlikely(GuardedAlloc.pointerIsMine(ptr))) + { + if (!size) + size = GuardedAlloc.getSize(ptr); + CurrentMemoryTracker::free(size); + return; + } +#endif + try { #if USE_JEMALLOC diff --git a/src/Common/new_delete.cpp b/src/Common/new_delete.cpp index 871ab750907..7abc47ad1e1 100644 --- a/src/Common/new_delete.cpp +++ b/src/Common/new_delete.cpp @@ -1,4 +1,5 @@ #include +#include #include #include "config.h" #include @@ -41,6 +42,26 @@ static struct InitializeJemallocZoneAllocatorForOSX } initializeJemallocZoneAllocatorForOSX; #endif +#if USE_GWP_ASAN + +#include + +/// Both clickhouse_new_delete and clickhouse_common_io links gwp_asan, but It should only init once, otherwise it +/// will cause unexpected deadlock. +static struct InitGwpAsan +{ + InitGwpAsan() + { + gwp_asan::options::initOptions(); + gwp_asan::options::Options &opts = gwp_asan::options::getOptions(); + GuardedAlloc.init(opts); + + ///std::cerr << "GwpAsan is initialized, the options are { Enabled: " << opts.Enabled + /// << ", MaxSimultaneousAllocations: " << opts.MaxSimultaneousAllocations + /// << ", SampleRate: " << opts.SampleRate << " }\n"; + } +} init_gwp_asan; +#endif /// Replace default new/delete with memory tracking versions. /// @sa https://en.cppreference.com/w/cpp/memory/new/operator_new diff --git a/src/Disks/IO/createReadBufferFromFileBase.cpp b/src/Disks/IO/createReadBufferFromFileBase.cpp index d4ee53a4066..04496e36826 100644 --- a/src/Disks/IO/createReadBufferFromFileBase.cpp +++ b/src/Disks/IO/createReadBufferFromFileBase.cpp @@ -149,13 +149,13 @@ std::unique_ptr createReadBufferFromFileBase( if (buffer_size % min_alignment) { - existing_memory = nullptr; /// Cannot reuse existing memory is it has unaligned size. + existing_memory = nullptr; /// Cannot reuse existing memory as it has unaligned size. buffer_size = align_up(buffer_size); } if (reinterpret_cast(existing_memory) % min_alignment) { - existing_memory = nullptr; /// Cannot reuse existing memory is it has unaligned offset. + existing_memory = nullptr; /// Cannot reuse existing memory as it has unaligned offset. } /// Attempt to open a file with O_DIRECT diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 9a2252844d1..66ab8e43f7e 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -886,8 +886,8 @@ void InterpreterSystemQuery::syncReplica() if (auto * storage_replicated = dynamic_cast(table.get())) { - LOG_TRACE(log, "Synchronizing entries in replica's queue with table's log and waiting for it to become empty"); - if (!storage_replicated->waitForShrinkingQueueSize(0, getContext()->getSettingsRef().receive_timeout.totalMilliseconds())) + LOG_TRACE(log, "Synchronizing entries in replica's queue with table's log and waiting for current last entry to be processed"); + if (!storage_replicated->waitForProcessingQueue(getContext()->getSettingsRef().receive_timeout.totalMilliseconds())) { LOG_ERROR(log, "SYNC REPLICA {}: Timed out!", table_id.getNameForLogs()); throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "SYNC REPLICA {}: command timed out. " \ diff --git a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp index fbbf8c119ce..0a7a9025b30 100644 --- a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp @@ -117,7 +117,7 @@ bool PullingAsyncPipelineExecutor::pull(Chunk & chunk, uint64_t milliseconds) data->rethrowExceptionIfHas(); bool is_execution_finished - = !data->executor->checkTimeLimitSoft() || lazy_format ? lazy_format->isFinished() : data->is_finished.load(); + = !data->executor->checkTimeLimitSoft() || (lazy_format ? lazy_format->isFinished() : data->is_finished.load()); if (is_execution_finished) { diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index aa5ab8a5e85..9cfee088eab 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -544,8 +544,7 @@ void ReplicatedMergeTreeQueue::removeProcessedEntry(zkutil::ZooKeeperPtr zookeep if (!found && need_remove_from_zk) throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't find {} in the memory queue. It is a bug. Entry: {}", entry->znode_name, entry->toString()); - - notifySubscribers(queue_size); + notifySubscribers(queue_size, entry->znode_name); if (!need_remove_from_zk) return; @@ -2466,12 +2465,17 @@ ReplicatedMergeTreeQueue::SubscriberHandler ReplicatedMergeTreeQueue::addSubscriber(ReplicatedMergeTreeQueue::SubscriberCallBack && callback) { std::lock_guard lock(state_mutex); + std::unordered_set result; + result.reserve(queue.size()); + for (const auto & entry : queue) + result.insert(entry->znode_name); + std::lock_guard lock_subscribers(subscribers_mutex); auto it = subscribers.emplace(subscribers.end(), std::move(callback)); - /// Atomically notify about current size - (*it)(queue.size()); + /// Notify queue size & log entry ids to avoid waiting for removed entries + (*it)(result.size(), result, std::nullopt); return SubscriberHandler(it, *this); } @@ -2482,16 +2486,16 @@ ReplicatedMergeTreeQueue::SubscriberHandler::~SubscriberHandler() queue.subscribers.erase(it); } -void ReplicatedMergeTreeQueue::notifySubscribers(size_t new_queue_size) +void ReplicatedMergeTreeQueue::notifySubscribers(size_t new_queue_size, std::optional removed_log_entry_id) { std::lock_guard lock_subscribers(subscribers_mutex); for (auto & subscriber_callback : subscribers) - subscriber_callback(new_queue_size); + subscriber_callback(new_queue_size, {}, removed_log_entry_id); } ReplicatedMergeTreeQueue::~ReplicatedMergeTreeQueue() { - notifySubscribers(0); + notifySubscribers(0, std::nullopt); } String padIndex(Int64 index) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 36f1ee07ad4..3c0fe571bfd 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -163,7 +163,7 @@ private: /// A subscriber callback is called when an entry queue is deleted mutable std::mutex subscribers_mutex; - using SubscriberCallBack = std::function; + using SubscriberCallBack = std::function /*wait_for_ids*/, std::optional /* removed_log_entry_id */)>; using Subscribers = std::list; using SubscriberIterator = Subscribers::iterator; @@ -180,8 +180,8 @@ private: Subscribers subscribers; - /// Notify subscribers about queue change - void notifySubscribers(size_t new_queue_size); + /// Notify subscribers about queue change (new queue size and entry that was removed) + void notifySubscribers(size_t new_queue_size, std::optional removed_log_entry_id); /// Check that entry_ptr is REPLACE_RANGE entry and can be removed from queue because current entry covers it bool checkReplaceRangeCanBeRemoved( diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 16a2f49b8df..5037734f21d 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -7549,26 +7549,37 @@ void StorageReplicatedMergeTree::onActionLockRemove(StorageActionBlockType actio background_moves_assignee.trigger(); } -bool StorageReplicatedMergeTree::waitForShrinkingQueueSize(size_t queue_size, UInt64 max_wait_milliseconds) +bool StorageReplicatedMergeTree::waitForProcessingQueue(UInt64 max_wait_milliseconds) { Stopwatch watch; /// Let's fetch new log entries firstly queue.pullLogsToQueue(getZooKeeperAndAssertNotReadonly(), {}, ReplicatedMergeTreeQueue::SYNC); - /// This is significant, because the execution of this task could be delayed at BackgroundPool. /// And we force it to be executed. background_operations_assignee.trigger(); - Poco::Event target_size_event; - auto callback = [&target_size_event, queue_size] (size_t new_queue_size) + std::unordered_set wait_for_ids; + bool set_ids_to_wait = true; + + Poco::Event target_entry_event; + auto callback = [&target_entry_event, &wait_for_ids, &set_ids_to_wait](size_t new_queue_size, std::unordered_set log_entry_ids, std::optional removed_log_entry_id) { - if (new_queue_size <= queue_size) - target_size_event.set(); + if (set_ids_to_wait) + { + wait_for_ids = log_entry_ids; + set_ids_to_wait = false; + } + + if (removed_log_entry_id.has_value()) + wait_for_ids.erase(removed_log_entry_id.value()); + + if (wait_for_ids.empty() || new_queue_size == 0) + target_entry_event.set(); }; const auto handler = queue.addSubscriber(std::move(callback)); - while (!target_size_event.tryWait(50)) + while (!target_entry_event.tryWait(50)) { if (max_wait_milliseconds && watch.elapsedMilliseconds() > max_wait_milliseconds) return false; @@ -7576,7 +7587,6 @@ bool StorageReplicatedMergeTree::waitForShrinkingQueueSize(size_t queue_size, UI if (partial_shutdown_called) throw Exception(ErrorCodes::ABORTED, "Shutdown is called for table"); } - return true; } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 3db54430b34..1e31fad6669 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -178,9 +178,9 @@ public: void onActionLockRemove(StorageActionBlockType action_type) override; - /// Wait when replication queue size becomes less or equal than queue_size + /// Wait till replication queue's current last entry is processed or till size becomes 0 /// If timeout is exceeded returns false - bool waitForShrinkingQueueSize(size_t queue_size = 0, UInt64 max_wait_milliseconds = 0); + bool waitForProcessingQueue(UInt64 max_wait_milliseconds = 0); /// Get the status of the table. If with_zk_fields = false - do not fill in the fields that require queries to ZK. void getStatus(ReplicatedTableStatus & res, bool with_zk_fields = true); diff --git a/src/configure_config.cmake b/src/configure_config.cmake index fe298855bcd..38ea803f5e9 100644 --- a/src/configure_config.cmake +++ b/src/configure_config.cmake @@ -109,6 +109,9 @@ endif() if (TARGET ch_contrib::jemalloc) set(USE_JEMALLOC 1) endif() +if (TARGET ch_contrib::gwp_asan) + set(USE_GWP_ASAN 1) +endif() if (TARGET ch_contrib::h3) set(USE_H3 1) endif() diff --git a/tests/ci/merge_pr.py b/tests/ci/merge_pr.py index 6be02e497d9..4a21bfcdd70 100644 --- a/tests/ci/merge_pr.py +++ b/tests/ci/merge_pr.py @@ -56,6 +56,13 @@ class Reviews: logging.info("There aren't reviews for PR #%s", self.pr.number) return False + logging.info( + "The following users have reviewed the PR:\n %s", + "\n ".join( + f"{user.login}: {review.state}" for user, review in self.reviews.items() + ), + ) + filtered_reviews = { user: review for user, review in self.reviews.items() @@ -125,7 +132,11 @@ class Reviews: return False return True - logging.info("The PR #%s is not approved", self.pr.number) + logging.info( + "The PR #%s is not approved by any of %s team member", + self.pr.number, + TEAM_NAME, + ) return False diff --git a/tests/clickhouse-test b/tests/clickhouse-test index a65662440f6..2215982affe 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -91,7 +91,7 @@ class HTTPError(Exception): # Helpers to execute queries via HTTP interface. def clickhouse_execute_http( - base_args, query, timeout=30, settings=None, default_format=None + base_args, query, timeout=30, settings=None, default_format=None, max_http_retries=5 ): if args.secure: client = http.client.HTTPSConnection( @@ -120,7 +120,7 @@ def clickhouse_execute_http( if default_format is not None: params["default_format"] = default_format - for i in range(MAX_RETRIES): + for i in range(max_http_retries): try: client.request( "POST", @@ -130,7 +130,7 @@ def clickhouse_execute_http( data = res.read() break except Exception as ex: - if i == MAX_RETRIES - 1: + if i == max_http_retries - 1: raise ex sleep(i + 1) @@ -140,13 +140,12 @@ def clickhouse_execute_http( return data - -def clickhouse_execute(base_args, query, timeout=30, settings=None): - return clickhouse_execute_http(base_args, query, timeout, settings).strip() +def clickhouse_execute(base_args, query, timeout=30, settings=None, max_http_retries=5): + return clickhouse_execute_http(base_args, query, timeout, settings, max_http_retries=max_http_retries).strip() -def clickhouse_execute_json(base_args, query, timeout=60, settings=None): - data = clickhouse_execute_http(base_args, query, timeout, settings, "JSONEachRow") +def clickhouse_execute_json(base_args, query, timeout=60, settings=None, max_http_retries=5): + data = clickhouse_execute_http(base_args, query, timeout, settings, "JSONEachRow", max_http_retries=max_http_retries) if not data: return None rows = [] @@ -641,7 +640,7 @@ class TestCase: clickhouse_execute( args, - "CREATE DATABASE " + database + get_db_engine(testcase_args, database), + "CREATE DATABASE IF NOT EXISTS " + database + get_db_engine(testcase_args, database), settings=get_create_database_settings(args, testcase_args), ) @@ -1139,7 +1138,7 @@ class TestCase: seconds_left = max( args.timeout - (datetime.now() - start_time).total_seconds(), 20 ) - drop_database_query = "DROP DATABASE " + database + drop_database_query = "DROP DATABASE IF EXISTS " + database if args.replicated_database: drop_database_query += " ON CLUSTER test_cluster_database_replicated" @@ -1670,7 +1669,7 @@ def check_server_started(args): retry_count = args.server_check_retries while retry_count > 0: try: - clickhouse_execute(args, "SELECT 1") + clickhouse_execute(args, "SELECT 1", max_http_retries=1) print(" OK") sys.stdout.flush() return True diff --git a/tests/integration/test_config_xml_yaml_mix/configs/config.d/mark_cache_size.yaml b/tests/integration/test_config_xml_yaml_mix/configs/config.d/mark_cache_size.yaml new file mode 100644 index 00000000000..b7ed5fcac82 --- /dev/null +++ b/tests/integration/test_config_xml_yaml_mix/configs/config.d/mark_cache_size.yaml @@ -0,0 +1,2 @@ +mark_cache_size: + '@from_env': CONFIG_TEST_ENV diff --git a/tests/integration/test_config_xml_yaml_mix/configs/config.xml b/tests/integration/test_config_xml_yaml_mix/configs/config.xml index dc1cb85e155..52835f014ca 100644 --- a/tests/integration/test_config_xml_yaml_mix/configs/config.xml +++ b/tests/integration/test_config_xml_yaml_mix/configs/config.xml @@ -271,4 +271,6 @@ false https://6f33034cfe684dd7a3ab9875e57b1c8d@o388870.ingest.sentry.io/5226277 + + 123451234 diff --git a/tests/integration/test_config_xml_yaml_mix/test.py b/tests/integration/test_config_xml_yaml_mix/test.py index 4138441b881..373c42b2dea 100644 --- a/tests/integration/test_config_xml_yaml_mix/test.py +++ b/tests/integration/test_config_xml_yaml_mix/test.py @@ -21,6 +21,7 @@ def test_extra_yaml_mix(): "configs/config.d/logging_no_rotate.xml", "configs/config.d/log_to_console.yaml", "configs/config.d/macros.yaml", + "configs/config.d/mark_cache_size.yaml", "configs/config.d/metric_log.xml", "configs/config.d/more_clusters.yaml", "configs/config.d/part_log.xml", @@ -46,6 +47,7 @@ def test_extra_yaml_mix(): users_config_name="users.yaml", copy_common_configs=False, config_root_name="clickhouse", + env_variables={"CONFIG_TEST_ENV": "8956"}, ) try: