diff --git a/base/daemon/BaseDaemon.cpp b/base/daemon/BaseDaemon.cpp index e40884a44b5..33dee5d4a63 100644 --- a/base/daemon/BaseDaemon.cpp +++ b/base/daemon/BaseDaemon.cpp @@ -85,7 +85,8 @@ static const size_t signal_pipe_buf_size = + sizeof(ucontext_t) + sizeof(StackTrace) + sizeof(UInt32) - + max_query_id_size + 1; /// query_id + varint encoded length + + max_query_id_size + 1 /// query_id + varint encoded length + + sizeof(void*); using signal_function = void(int, siginfo_t*, void*); @@ -135,6 +136,7 @@ static void signalHandler(int sig, siginfo_t * info, void * context) DB::writePODBinary(stack_trace, out); DB::writeBinary(UInt32(getThreadId()), out); DB::writeStringBinary(query_id, out); + DB::writePODBinary(DB::current_thread, out); out.next(); @@ -218,16 +220,18 @@ public: StackTrace stack_trace(NoCapture{}); UInt32 thread_num; std::string query_id; + DB::ThreadStatus * thread_ptr{}; DB::readPODBinary(info, in); DB::readPODBinary(context, in); DB::readPODBinary(stack_trace, in); DB::readBinary(thread_num, in); DB::readBinary(query_id, in); + DB::readPODBinary(thread_ptr, in); /// This allows to receive more signals if failure happens inside onFault function. /// Example: segfault while symbolizing stack trace. - std::thread([=, this] { onFault(sig, info, context, stack_trace, thread_num, query_id); }).detach(); + std::thread([=, this] { onFault(sig, info, context, stack_trace, thread_num, query_id, thread_ptr); }).detach(); } } } @@ -248,8 +252,19 @@ private: const ucontext_t & context, const StackTrace & stack_trace, UInt32 thread_num, - const std::string & query_id) const + const std::string & query_id, + DB::ThreadStatus * thread_ptr) const { + DB::ThreadStatus thread_status; + + /// Send logs from this thread to client if possible. + /// It will allow client to see failure messages directly. + if (thread_ptr) + { + if (auto logs_queue = thread_ptr->getInternalTextLogsQueue()) + DB::CurrentThread::attachInternalTextLogsQueue(logs_queue, DB::LogsLevel::trace); + } + LOG_FATAL(log, "########################################"); if (query_id.empty()) @@ -280,6 +295,10 @@ private: /// Write symbolized stack trace line by line for better grep-ability. stack_trace.toStringEveryLine([&](const std::string & s) { LOG_FATAL(log, s); }); + + /// When everything is done, we will try to send these error messages to client. + if (thread_ptr) + thread_ptr->onFatalError(); } }; diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 5e0bc081526..05fc1ba9141 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -123,7 +123,7 @@ private: }; bool is_interactive = true; /// Use either interactive line editing interface or batch mode. bool need_render_progress = true; /// Render query execution progress. - bool send_logs = false; /// send_logs_level passed, do not use previous cursor position, to avoid overlaps with logs + bool has_received_logs = false; /// We have received some logs, do not use previous cursor position, to avoid overlaps with logs bool echo_queries = false; /// Print queries before execution in batch mode. bool ignore_error = false; /// In case of errors, don't print error message, continue to next query. Only applicable for non-interactive mode. bool print_time_to_stderr = false; /// Output execution time to stderr in batch mode. @@ -908,8 +908,6 @@ private: connection->forceConnected(connection_parameters.timeouts); - send_logs = context.getSettingsRef().send_logs_level != LogsLevel::none; - ASTPtr input_function; if (insert && insert->select) insert->tryFindInputFunction(input_function); @@ -1518,6 +1516,7 @@ private: void onLogData(Block & block) { + has_received_logs = true; initLogsOutputStream(); logs_out_stream->write(block); logs_out_stream->flush(); @@ -1553,7 +1552,7 @@ private: void clearProgress() { written_progress_chars = 0; - if (!send_logs) + if (!has_received_logs) std::cerr << "\r" CLEAR_TO_END_OF_LINE; } @@ -1581,7 +1580,7 @@ private: const char * indicator = indicators[increment % 8]; - if (!send_logs && written_progress_chars) + if (!has_received_logs && written_progress_chars) message << '\r'; size_t prefix_size = message.count(); @@ -1635,7 +1634,7 @@ private: message << CLEAR_TO_END_OF_LINE; - if (send_logs) + if (has_received_logs) message << '\n'; ++increment; diff --git a/src/Common/CurrentThread.cpp b/src/Common/CurrentThread.cpp index 884cf2bfa9c..c93b22c443a 100644 --- a/src/Common/CurrentThread.cpp +++ b/src/Common/CurrentThread.cpp @@ -72,6 +72,13 @@ void CurrentThread::attachInternalTextLogsQueue(const std::shared_ptrattachInternalTextLogsQueue(logs_queue, client_logs_level); } +void CurrentThread::setFatalErrorCallback(std::function callback) +{ + if (unlikely(!current_thread)) + return; + current_thread->setFatalErrorCallback(callback); +} + std::shared_ptr CurrentThread::getInternalTextLogsQueue() { /// NOTE: this method could be called at early server startup stage diff --git a/src/Common/CurrentThread.h b/src/Common/CurrentThread.h index dd9bba2c785..b58429a69d6 100644 --- a/src/Common/CurrentThread.h +++ b/src/Common/CurrentThread.h @@ -46,6 +46,8 @@ public: LogsLevel client_logs_level); static std::shared_ptr getInternalTextLogsQueue(); + static void setFatalErrorCallback(std::function callback); + /// Makes system calls to update ProfileEvents that contain info from rusage and taskstats static void updatePerformanceCounters(); diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index ddb0b96df0e..9e2731cfc22 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -98,4 +98,15 @@ void ThreadStatus::attachInternalTextLogsQueue(const InternalTextLogsQueuePtr & thread_group->client_logs_level = client_logs_level; } +void ThreadStatus::setFatalErrorCallback(std::function callback) +{ + fatal_error_callback = std::move(callback); +} + +void ThreadStatus::onFatalError() +{ + if (fatal_error_callback) + fatal_error_callback(); +} + } diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index d0952c3ab28..a3394dc7458 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -145,6 +145,10 @@ public: void attachInternalTextLogsQueue(const InternalTextLogsQueuePtr & logs_queue, LogsLevel client_logs_level); + /// Callback that is used to trigger sending fatal error messages to client. + void setFatalErrorCallback(std::function callback); + void onFatalError(); + /// Sets query context for current thread and its thread group /// NOTE: query_context have to be alive until detachQuery() is called void attachQueryContext(Context & query_context); @@ -200,6 +204,9 @@ protected: std::unique_ptr last_rusage; std::unique_ptr taskstats; + /// Is used to send logs from logs_queue to client in case of fatal errors. + std::function fatal_error_callback; + private: void setupState(const ThreadGroupStatusPtr & thread_group_); }; diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 6aaebc4e995..d11fd2b0aaa 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -316,7 +316,7 @@ struct Settings : public SettingsCollection M(SettingBool, log_profile_events, true, "Log query performance statistics into the query_log and query_thread_log.", 0) \ M(SettingBool, log_query_settings, true, "Log query settings into the query_log.", 0) \ M(SettingBool, log_query_threads, true, "Log query threads into system.query_thread_log table. This setting have effect only when 'log_queries' is true.", 0) \ - M(SettingLogsLevel, send_logs_level, LogsLevel::none, "Send server text logs with specified minimum level to client. Valid values: 'trace', 'debug', 'information', 'warning', 'error', 'none'", 0) \ + M(SettingLogsLevel, send_logs_level, LogsLevel::fatal, "Send server text logs with specified minimum level to client. Valid values: 'trace', 'debug', 'information', 'warning', 'error', 'fatal', 'none'", 0) \ M(SettingBool, enable_optimize_predicate_expression, 1, "If it is set to true, optimize predicates to subqueries.", 0) \ M(SettingBool, enable_optimize_predicate_expression_to_final_subquery, 1, "Allow push predicate to final subquery.", 0) \ \ diff --git a/src/Core/SettingsCollection.cpp b/src/Core/SettingsCollection.cpp index b36884fad22..32bf1f29c90 100644 --- a/src/Core/SettingsCollection.cpp +++ b/src/Core/SettingsCollection.cpp @@ -542,6 +542,7 @@ IMPLEMENT_SETTING_ENUM(FormatSettings::DateTimeInputFormat, DATE_TIME_INPUT_FORM #define LOGS_LEVEL_LIST_OF_NAMES(M) \ M(none, "none") \ + M(fatal, "fatal") \ M(error, "error") \ M(warning, "warning") \ M(information, "information") \ diff --git a/src/Core/SettingsCollection.h b/src/Core/SettingsCollection.h index e85baf82fb6..67f6dd05614 100644 --- a/src/Core/SettingsCollection.h +++ b/src/Core/SettingsCollection.h @@ -302,6 +302,7 @@ using SettingDateTimeInputFormat = SettingEnum(); state.logs_queue->max_priority = Poco::Logger::parseLevel(client_logs_level.toString()); CurrentThread::attachInternalTextLogsQueue(state.logs_queue, client_logs_level); + CurrentThread::setFatalErrorCallback([this]{ sendLogs(); }); } query_context->setExternalTablesInitializer([&connection_settings, this] (Context & context) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 219546c92f1..7126f204fdf 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -271,6 +271,10 @@ def run_tests_array(all_tests_with_params): if stderr: print(stderr.encode('utf-8')) + # Stop on fatal errors like segmentation fault. They are send to client via logs. + if ' ' in stderr: + SERVER_DIED = True + if args.stop and ('Connection refused' in stderr or 'Attempt to read after eof' in stderr) and not 'Received exception from server' in stderr: SERVER_DIED = True diff --git a/tests/integration/test_distributed_over_live_view/test.py b/tests/integration/test_distributed_over_live_view/test.py index f932379e5c6..46c0bada535 100644 --- a/tests/integration/test_distributed_over_live_view/test.py +++ b/tests/integration/test_distributed_over_live_view/test.py @@ -81,7 +81,10 @@ class TestLiveViewOverDistributedSuite: client1.expect(prompt) client1.send(select_query) - client1.expect('"node1",0,0\r\n.*"node1",1,1\r\n.*"node2",0,10\r\n.*"node2",1,11\r\n') + client1.expect('"node1",0,0') + client1.expect('"node1",1,1') + client1.expect('"node2",0,10') + client1.expect('"node2",1,11') client1.expect(prompt) client1.send("INSERT INTO distributed_table VALUES ('node1', 1, 3), ('node1', 2, 3)") @@ -90,7 +93,13 @@ class TestLiveViewOverDistributedSuite: client2.expect(prompt) time.sleep(2) client1.send(select_query) - client1.expect('"node1",0,0\r\n.*"node1",1,1\r\n.*"node1",1,3\r\n.*"node1",2,3\r\n.*"node1",3,3\r\n.*"node2",0,10\r\n.*"node2",1,11\r\n') + client1.expect('"node1",0,0') + client1.expect('"node1",1,1') + client1.expect('"node1",1,3') + client1.expect('"node1",2,3') + client1.expect('"node1",3,3') + client1.expect('"node2",0,10') + client1.expect('"node2",1,11') client1.expect(prompt) def test_distributed_over_live_view_order_by_key(self, started_cluster, node, source): @@ -110,7 +119,10 @@ class TestLiveViewOverDistributedSuite: client1.expect(prompt) client1.send(select_query) - client1.expect('"node1",0,0\r\n"node2",0,10\r\n"node1",1,1\r\n.*"node2",1,11\r\n') + client1.expect('"node1",0,0') + client1.expect('"node2",0,10') + client1.expect('"node1",1,1') + client1.expect('"node2",1,11') client1.expect(prompt) client1.send("INSERT INTO distributed_table VALUES ('node1', 1, 3), ('node1', 2, 3)") @@ -119,7 +131,13 @@ class TestLiveViewOverDistributedSuite: client2.expect(prompt) time.sleep(2) client1.send(select_query) - client1.expect('"node1",0,0\r\n.*"node2",0,10.*\r\n"node1",1,1\r\n.*"node1",1,3\r\n.*"node2",1,11\r\n.*"node1",2,3\r\n.*"node1",3,3\r\n') + client1.expect('"node1",0,0') + client1.expect('"node2",0,10') + client1.expect('"node1",1,1') + client1.expect('"node1",1,3') + client1.expect('"node2",1,11') + client1.expect('"node1",2,3') + client1.expect('"node1",3,3') client1.expect(prompt) def test_distributed_over_live_view_group_by_node(self, started_cluster, node, source): @@ -139,14 +157,16 @@ class TestLiveViewOverDistributedSuite: client1.expect(prompt) client1.send(select_query) - client1.expect('"node1",1\r\n"node2",21\r\n') + client1.expect('"node1",1') + client1.expect('"node2",21') client1.expect(prompt) client2.send("INSERT INTO distributed_table VALUES ('node1', 2, 2)") client2.expect(prompt) time.sleep(2) client1.send(select_query) - client1.expect('"node1",3\r\n.*"node2",21\r\n') + client1.expect('"node1",3') + client1.expect('"node2",21') client1.expect(prompt) client1.send("INSERT INTO distributed_table VALUES ('node1', 1, 3), ('node1', 3, 3)") @@ -155,7 +175,8 @@ class TestLiveViewOverDistributedSuite: client2.expect(prompt) time.sleep(2) client1.send(select_query) - client1.expect('"node1",12\r\n.*"node2",21\r\n') + client1.expect('"node1",12') + client1.expect('"node2",21') client1.expect(prompt) def test_distributed_over_live_view_group_by_key(self, started_cluster, node, source): @@ -175,21 +196,27 @@ class TestLiveViewOverDistributedSuite: client1.expect(prompt) client1.send(select_query) - client1.expect("0,10\r\n1,12\r\n") + client1.expect('0,10') + client1.expect('1,12') client1.expect(prompt) client2.send("INSERT INTO distributed_table VALUES ('node1', 2, 2)") client2.expect(prompt) time.sleep(2) client1.send(select_query) - client1.expect("0,10\r\n1,12\r\n2,2\r\n") + client1.expect('0,10') + client1.expect('1,12') + client1.expect('2,2') client1.expect(prompt) client2.send("INSERT INTO distributed_table VALUES ('node1', 1, 3), ('node1', 3, 3)") client2.expect(prompt) time.sleep(2) client1.send(select_query) - client1.expect("0,10\r\n.*1,15\r\n.*2,2\r\n.*3,3\r\n") + client1.expect('0,10') + client1.expect('1,15') + client1.expect('2,2') + client1.expect('3,3') client1.expect(prompt) def test_distributed_over_live_view_sum(self, started_cluster, node, source): diff --git a/tests/integration/test_live_view_over_distributed/test.py b/tests/integration/test_live_view_over_distributed/test.py index c7b9c452725..1ae9a4b5199 100644 --- a/tests/integration/test_live_view_over_distributed/test.py +++ b/tests/integration/test_live_view_over_distributed/test.py @@ -111,15 +111,28 @@ node2\t1\t11 client1.expect(prompt) client1.send("WATCH lv FORMAT CSV") - client1.expect('"node1",0,0,1\r\n.*"node1",1,1,1\r\n.*"node2",0,10,1\r\n.*"node2",1,11,1\r\n') + client1.expect('"node1",0,0,1') + client1.expect('"node1",1,1,1') + client1.expect('"node2",0,10,1') + client1.expect('"node2",1,11,1') client2.send("INSERT INTO distributed_table VALUES ('node1', 2, 2)") client2.expect(prompt) - client1.expect('"node1",0,0,2\r\n.*"node1",1,1,2\r\n.*"node1",2,2,2\r\n.*"node2",0,10,2\r\n.*"node2",1,11,2\r\n') + client1.expect('"node1",0,0,2') + client1.expect('"node1",1,1,2') + client1.expect('"node1",2,2,2') + client1.expect('"node2",0,10,2') + client1.expect('"node2",1,11,2') client2.send("INSERT INTO distributed_table VALUES ('node1', 0, 3), ('node3', 3, 3)") client2.expect(prompt) - client1.expect('"node1",0,0,3\r\n.*"node1",0,3,3\r\n.*"node1",1,1,3\r\n.*"node1",2,2,3\r\n.*"node2",0,10,3\r\n.*"node2",1,11,3\r\n.*"node3",3,3,3\r\n') + client1.expect('"node1",0,0,3') + client1.expect('"node1",0,3,3') + client1.expect('"node1",1,1,3') + client1.expect('"node1",2,2,3') + client1.expect('"node2",0,10,3') + client1.expect('"node2",1,11,3') + client1.expect('"node3",3,3,3') def test_watch_live_view_order_by_key(self, started_cluster, node, source): log = sys.stdout @@ -141,15 +154,28 @@ node2\t1\t11 client1.expect(prompt) client1.send("WATCH lv FORMAT CSV") - client1.expect('"node1",0,0,1\r\n.*"node2",0,10,1\r\n.*"node1",1,1,1\r\n.*"node2",1,11,1\r\n') + client1.expect('"node1",0,0,1') + client1.expect('"node2",0,10,1') + client1.expect('"node1",1,1,1') + client1.expect('"node2",1,11,1') client2.send("INSERT INTO distributed_table VALUES ('node1', 2, 2)") client2.expect(prompt) - client1.expect('"node1",0,0,2\r\n.*"node2",0,10,2\r\n.*"node1",1,1,2\r\n.*"node2",1,11,2\r\n.*"node1",2,2,2\r\n') + client1.expect('"node1",0,0,2') + client1.expect('"node2",0,10,2') + client1.expect('"node1",1,1,2') + client1.expect('"node2",1,11,2') + client1.expect('"node1",2,2,2') client2.send("INSERT INTO distributed_table VALUES ('node1', 0, 3), ('node3', 3, 3)") client2.expect(prompt) - client1.expect('"node1",0,0,3\r\n.*"node1",0,3,3\r\n.*"node2",0,10,3\r\n.*"node1",1,1,3\r\n.*"node2",1,11,3\r\n.*"node1",2,2,3\r\n.*"node3",3,3,3\r\n') + client1.expect('"node1",0,0,3') + client1.expect('"node1",0,3,3') + client1.expect('"node2",0,10,3') + client1.expect('"node1",1,1,3') + client1.expect('"node2",1,11,3') + client1.expect('"node1",2,2,3') + client1.expect('"node3",3,3,3') def test_watch_live_view_group_by_node(self, started_cluster, node, source): log = sys.stdout @@ -171,15 +197,19 @@ node2\t1\t11 client1.expect(prompt) client1.send("WATCH lv FORMAT CSV") - client1.expect('"node1",1,1\r\n.*"node2",21,1\r\n') + client1.expect('"node1",1,1') + client1.expect('"node2",21,1') client2.send("INSERT INTO distributed_table VALUES ('node1', 2, 2)") client2.expect(prompt) - client1.expect('"node1",3,2\r\n.*"node2",21,2\r\n') + client1.expect('"node1",3,2') + client1.expect('"node2",21,2') client2.send("INSERT INTO distributed_table VALUES ('node1', 0, 3), ('node3', 3, 3)") client2.expect(prompt) - client1.expect('"node1",6,3\r\n.*"node2",21,3\r\n.*"node3",3,3\r\n') + client1.expect('"node1",6,3') + client1.expect('"node2",21,3') + client1.expect('"node3",3,3') def test_watch_live_view_group_by_key(self, started_cluster, node, source): log = sys.stdout @@ -201,15 +231,21 @@ node2\t1\t11 client1.expect(prompt) client1.send("WATCH lv FORMAT CSV") - client1.expect("0,10,1\r\n.*1,12,1\r\n") + client1.expect('0,10,1') + client1.expect('1,12,1') client2.send("INSERT INTO distributed_table VALUES ('node1', 2, 2)") client2.expect(prompt) - client1.expect("0,10,2\r\n.*1,12,2\r\n.*2,2,2\r\n") + client1.expect('0,10,2') + client1.expect('1,12,2') + client1.expect('2,2,2') client2.send("INSERT INTO distributed_table VALUES ('node1', 0, 3), ('node1', 3, 3)") client2.expect(prompt) - client1.expect("0,13,3\r\n.*1,12,3\r\n.*2,2,3\r\n.*3,3,3\r\n") + client1.expect('0,13,3') + client1.expect('1,12,3') + client1.expect('2,2,3') + client1.expect('3,3,3') def test_watch_live_view_sum(self, started_cluster, node, source): diff --git a/tests/queries/0_stateless/00002_system_numbers.sql b/tests/queries/0_stateless/00002_system_numbers.sql index 8f1580e9127..95f75573201 100644 --- a/tests/queries/0_stateless/00002_system_numbers.sql +++ b/tests/queries/0_stateless/00002_system_numbers.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SELECT * FROM system.numbers LIMIT 3; SELECT sys_num.number FROM system.numbers AS sys_num WHERE number > 2 LIMIT 2; diff --git a/tests/queries/0_stateless/00158_buffer_and_nonexistent_table.sql b/tests/queries/0_stateless/00158_buffer_and_nonexistent_table.sql index 5fc41e30889..8d5d297b629 100644 --- a/tests/queries/0_stateless/00158_buffer_and_nonexistent_table.sql +++ b/tests/queries/0_stateless/00158_buffer_and_nonexistent_table.sql @@ -2,7 +2,7 @@ CREATE DATABASE IF NOT EXISTS test2_00158; DROP TABLE IF EXISTS test2_00158.mt_buffer_00158; DROP TABLE IF EXISTS test2_00158.mt_00158; CREATE TABLE test2_00158.mt_buffer_00158 (d Date DEFAULT today(), x UInt64) ENGINE = Buffer(test2_00158, mt_00158, 16, 100, 100, 1000000, 1000000, 1000000000, 1000000000); -SET send_logs_level = 'none'; -- Supress "Destination table test2.mt doesn't exist. Block of data is discarded." +SET send_logs_level = 'fatal'; -- Supress "Destination table test2.mt doesn't exist. Block of data is discarded." INSERT INTO test2_00158.mt_buffer_00158 (x) SELECT number AS x FROM system.numbers LIMIT 100000; INSERT INTO test2_00158.mt_buffer_00158 (x) SELECT number AS x FROM system.numbers LIMIT 1000000; DROP TABLE IF EXISTS test2_00158.mt_buffer_00158; diff --git a/tests/queries/0_stateless/00183_skip_unavailable_shards.sql b/tests/queries/0_stateless/00183_skip_unavailable_shards.sql index 2f2fe407400..4aa7cc72605 100644 --- a/tests/queries/0_stateless/00183_skip_unavailable_shards.sql +++ b/tests/queries/0_stateless/00183_skip_unavailable_shards.sql @@ -1,3 +1,3 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SELECT count() FROM remote('{127,1}.0.0.{2,3}', system.one) SETTINGS skip_unavailable_shards = 1; SELECT count() FROM remote('{1,127}.0.0.{2,3}', system.one) SETTINGS skip_unavailable_shards = 1; diff --git a/tests/queries/0_stateless/00205_scalar_subqueries.sql b/tests/queries/0_stateless/00205_scalar_subqueries.sql index 35febd5a75b..14244377e5f 100644 --- a/tests/queries/0_stateless/00205_scalar_subqueries.sql +++ b/tests/queries/0_stateless/00205_scalar_subqueries.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SELECT (SELECT (SELECT (SELECT (SELECT (SELECT count() FROM (SELECT * FROM system.numbers LIMIT 10)))))) = (SELECT 10), ((SELECT 1, 'Hello', [1, 2]).3)[1]; SELECT toUInt64((SELECT 9)) IN (SELECT number FROM system.numbers LIMIT 10); diff --git a/tests/queries/0_stateless/00233_position_function_family.sql b/tests/queries/0_stateless/00233_position_function_family.sql index 874e093d3c5..1bfd17310e1 100644 --- a/tests/queries/0_stateless/00233_position_function_family.sql +++ b/tests/queries/0_stateless/00233_position_function_family.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; select 1 = position('', ''); select 1 = position('abc', ''); select 0 = position('', 'abc'); diff --git a/tests/queries/0_stateless/00492_drop_temporary_table.sql b/tests/queries/0_stateless/00492_drop_temporary_table.sql index 5bc3652a411..a065b7f225d 100644 --- a/tests/queries/0_stateless/00492_drop_temporary_table.sql +++ b/tests/queries/0_stateless/00492_drop_temporary_table.sql @@ -2,7 +2,7 @@ DROP TEMPORARY TABLE IF EXISTS temp_tab; CREATE TEMPORARY TABLE temp_tab (number UInt64); INSERT INTO temp_tab SELECT number FROM system.numbers LIMIT 1; SELECT number FROM temp_tab; -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; EXISTS TEMPORARY TABLE temp_tab; DROP TABLE temp_tab; EXISTS TEMPORARY TABLE temp_tab; diff --git a/tests/queries/0_stateless/00502_sum_map.sql b/tests/queries/0_stateless/00502_sum_map.sql index 6a4035a3782..021aaf3cd3b 100644 --- a/tests/queries/0_stateless/00502_sum_map.sql +++ b/tests/queries/0_stateless/00502_sum_map.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS sum_map; CREATE TABLE sum_map(date Date, timeslot DateTime, statusMap Nested(status UInt16, requests UInt64)) ENGINE = Log; diff --git a/tests/queries/0_stateless/00597_push_down_predicate.sql b/tests/queries/0_stateless/00597_push_down_predicate.sql index 774c855c369..7fd30a82d8d 100644 --- a/tests/queries/0_stateless/00597_push_down_predicate.sql +++ b/tests/queries/0_stateless/00597_push_down_predicate.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SET any_join_distinct_right_table_keys = 1; SET joined_subquery_requires_alias = 0; diff --git a/tests/queries/0_stateless/00647_multiply_aggregation_state.sql b/tests/queries/0_stateless/00647_multiply_aggregation_state.sql index ebf0c22d9d0..b0361458221 100644 --- a/tests/queries/0_stateless/00647_multiply_aggregation_state.sql +++ b/tests/queries/0_stateless/00647_multiply_aggregation_state.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SELECT countMerge(x) AS y FROM ( SELECT countState() * 2 AS x FROM ( SELECT 1 )); SELECT countMerge(x) AS y FROM ( SELECT countState() * 0 AS x FROM ( SELECT 1 UNION ALL SELECT 2)); diff --git a/tests/queries/0_stateless/00692_if_exception_code.sql b/tests/queries/0_stateless/00692_if_exception_code.sql index c37b877648d..f9d06f2e3a5 100644 --- a/tests/queries/0_stateless/00692_if_exception_code.sql +++ b/tests/queries/0_stateless/00692_if_exception_code.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SELECT if(); -- { serverError 42 } SELECT if(1); -- { serverError 42 } diff --git a/tests/queries/0_stateless/00694_max_block_size_zero.sql b/tests/queries/0_stateless/00694_max_block_size_zero.sql index d0dc826160f..ba5b513bb5d 100644 --- a/tests/queries/0_stateless/00694_max_block_size_zero.sql +++ b/tests/queries/0_stateless/00694_max_block_size_zero.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SET max_block_size = 0; SELECT number FROM system.numbers; -- { serverError 12 } diff --git a/tests/queries/0_stateless/00698_validate_array_sizes_for_nested.sql b/tests/queries/0_stateless/00698_validate_array_sizes_for_nested.sql index 8bdc7c4c1a1..ec238797dca 100644 --- a/tests/queries/0_stateless/00698_validate_array_sizes_for_nested.sql +++ b/tests/queries/0_stateless/00698_validate_array_sizes_for_nested.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS mergetree_00698; CREATE TABLE mergetree_00698 (k UInt32, `n.x` Array(UInt64), `n.y` Array(UInt64)) ENGINE = MergeTree ORDER BY k; diff --git a/tests/queries/0_stateless/00698_validate_array_sizes_for_nested_kshvakov.sql b/tests/queries/0_stateless/00698_validate_array_sizes_for_nested_kshvakov.sql index 2d8dcc1f4f3..010d53dbcac 100644 --- a/tests/queries/0_stateless/00698_validate_array_sizes_for_nested_kshvakov.sql +++ b/tests/queries/0_stateless/00698_validate_array_sizes_for_nested_kshvakov.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS Issue_2231_Invalid_Nested_Columns_Size; CREATE TABLE Issue_2231_Invalid_Nested_Columns_Size ( diff --git a/tests/queries/0_stateless/00701_join_default_strictness.sql b/tests/queries/0_stateless/00701_join_default_strictness.sql index 873911a944d..6a10b4b88ad 100644 --- a/tests/queries/0_stateless/00701_join_default_strictness.sql +++ b/tests/queries/0_stateless/00701_join_default_strictness.sql @@ -1,7 +1,7 @@ DROP TABLE IF EXISTS a1; DROP TABLE IF EXISTS a2; -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; CREATE TABLE a1(a UInt8, b UInt8) ENGINE=Memory; CREATE TABLE a2(a UInt8, b UInt8) ENGINE=Memory; diff --git a/tests/queries/0_stateless/00705_aggregate_states_addition.sql b/tests/queries/0_stateless/00705_aggregate_states_addition.sql index a2d9660955d..661197d15ef 100644 --- a/tests/queries/0_stateless/00705_aggregate_states_addition.sql +++ b/tests/queries/0_stateless/00705_aggregate_states_addition.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS add_aggregate; CREATE TABLE add_aggregate(a UInt32, b UInt32) ENGINE = Memory; diff --git a/tests/queries/0_stateless/00714_alter_uuid.sql b/tests/queries/0_stateless/00714_alter_uuid.sql index 00a8f793e0e..ab08e943175 100644 --- a/tests/queries/0_stateless/00714_alter_uuid.sql +++ b/tests/queries/0_stateless/00714_alter_uuid.sql @@ -37,7 +37,7 @@ ENGINE = MergeTree PARTITION BY toDate(created_at) ORDER BY (created_at, id0, id1); -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; ALTER TABLE uuid MODIFY COLUMN id0 UUID; -- { serverError 524 } ALTER TABLE uuid MODIFY COLUMN id1 UUID; -- { serverError 524 } diff --git a/tests/queries/0_stateless/00716_allow_ddl.sql b/tests/queries/0_stateless/00716_allow_ddl.sql index c060bf37290..d33d8b7eec5 100755 --- a/tests/queries/0_stateless/00716_allow_ddl.sql +++ b/tests/queries/0_stateless/00716_allow_ddl.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SET allow_ddl = 0; CREATE DATABASE some_db; -- { serverError 392 } diff --git a/tests/queries/0_stateless/00717_merge_and_distributed.sql b/tests/queries/0_stateless/00717_merge_and_distributed.sql index 908df53db44..f0d34b5165f 100644 --- a/tests/queries/0_stateless/00717_merge_and_distributed.sql +++ b/tests/queries/0_stateless/00717_merge_and_distributed.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS test_local_1; DROP TABLE IF EXISTS test_local_2; diff --git a/tests/queries/0_stateless/00718_format_datetime.sql b/tests/queries/0_stateless/00718_format_datetime.sql index 21dc504985d..cd679fe9735 100644 --- a/tests/queries/0_stateless/00718_format_datetime.sql +++ b/tests/queries/0_stateless/00718_format_datetime.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SELECT formatDateTime(); -- { serverError 42 } SELECT formatDateTime('not a datetime', 'IGNORED'); -- { serverError 43 } diff --git a/tests/queries/0_stateless/00729_prewhere_array_join.sql b/tests/queries/0_stateless/00729_prewhere_array_join.sql index 2fdd66c11f7..ba10dd38bd2 100644 --- a/tests/queries/0_stateless/00729_prewhere_array_join.sql +++ b/tests/queries/0_stateless/00729_prewhere_array_join.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; drop table if exists t1_00729; create table t1_00729 (id UInt64, val Array(String),nid UInt64, eDate Date)ENGINE = MergeTree(eDate, (id, eDate), 8192); diff --git a/tests/queries/0_stateless/00732_base64_functions.sql b/tests/queries/0_stateless/00732_base64_functions.sql index a2dd4c4b41b..0d3b7e43645 100644 --- a/tests/queries/0_stateless/00732_base64_functions.sql +++ b/tests/queries/0_stateless/00732_base64_functions.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SELECT base64Encode(val) FROM (select arrayJoin(['', 'f', 'fo', 'foo', 'foob', 'fooba', 'foobar']) val); SELECT base64Decode(val) FROM (select arrayJoin(['', 'Zg==', 'Zm8=', 'Zm9v', 'Zm9vYg==', 'Zm9vYmE=', 'Zm9vYmFy']) val); SELECT base64Decode(base64Encode('foo')) = 'foo', base64Encode(base64Decode('Zm9v')) == 'Zm9v'; diff --git a/tests/queries/0_stateless/00732_quorum_insert_have_data_before_quorum_zookeeper.sql b/tests/queries/0_stateless/00732_quorum_insert_have_data_before_quorum_zookeeper.sql index 9348c94c981..6346dde9b76 100644 --- a/tests/queries/0_stateless/00732_quorum_insert_have_data_before_quorum_zookeeper.sql +++ b/tests/queries/0_stateless/00732_quorum_insert_have_data_before_quorum_zookeeper.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS quorum1; DROP TABLE IF EXISTS quorum2; diff --git a/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.sql b/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.sql index 5bfd54c3768..bf9f63ee3c1 100644 --- a/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.sql +++ b/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS quorum1; DROP TABLE IF EXISTS quorum2; diff --git a/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.sql b/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.sql index 001932c852e..7d2ee796b48 100644 --- a/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.sql +++ b/tests/queries/0_stateless/00732_quorum_insert_lost_part_zookeeper.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS quorum1; DROP TABLE IF EXISTS quorum2; diff --git a/tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper.sql b/tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper.sql index 1644856dc8d..0f66598fc51 100644 --- a/tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper.sql +++ b/tests/queries/0_stateless/00732_quorum_insert_select_with_old_data_and_without_quorum_zookeeper.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS quorum1; DROP TABLE IF EXISTS quorum2; diff --git a/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.sql b/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.sql index 69fd709add8..397aa7853df 100644 --- a/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.sql +++ b/tests/queries/0_stateless/00732_quorum_insert_simple_test_1_parts_zookeeper.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS quorum1; DROP TABLE IF EXISTS quorum2; diff --git a/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.sql b/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.sql index 74e625cafa4..a77f3377202 100644 --- a/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.sql +++ b/tests/queries/0_stateless/00732_quorum_insert_simple_test_2_parts_zookeeper.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS quorum1; DROP TABLE IF EXISTS quorum2; diff --git a/tests/queries/0_stateless/00735_conditional.sql b/tests/queries/0_stateless/00735_conditional.sql index ce49c26ca3d..001a4aa7430 100644 --- a/tests/queries/0_stateless/00735_conditional.sql +++ b/tests/queries/0_stateless/00735_conditional.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SELECT 'value vs value'; diff --git a/tests/queries/0_stateless/00742_require_join_strictness.sql b/tests/queries/0_stateless/00742_require_join_strictness.sql index 958d349aa51..5659a0f6833 100644 --- a/tests/queries/0_stateless/00742_require_join_strictness.sql +++ b/tests/queries/0_stateless/00742_require_join_strictness.sql @@ -1,3 +1,3 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SET join_default_strictness = ''; SELECT * FROM system.one INNER JOIN (SELECT number AS k FROM system.numbers) js2 ON dummy = k; -- { serverError 417 } diff --git a/tests/queries/0_stateless/00754_alter_modify_column_partitions.sql b/tests/queries/0_stateless/00754_alter_modify_column_partitions.sql index d2fffabb8c6..4aa7ab8ad64 100644 --- a/tests/queries/0_stateless/00754_alter_modify_column_partitions.sql +++ b/tests/queries/0_stateless/00754_alter_modify_column_partitions.sql @@ -1,6 +1,6 @@ -- check ALTER MODIFY COLUMN with partitions -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS alter_column; diff --git a/tests/queries/0_stateless/00754_alter_modify_order_by.sql b/tests/queries/0_stateless/00754_alter_modify_order_by.sql index 6cf3da0e29f..f8c584ed052 100644 --- a/tests/queries/0_stateless/00754_alter_modify_order_by.sql +++ b/tests/queries/0_stateless/00754_alter_modify_order_by.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS old_style; CREATE TABLE old_style(d Date, x UInt32) ENGINE MergeTree(d, x, 8192); diff --git a/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql b/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql index c94c0f3c55b..e973259d4b7 100644 --- a/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql +++ b/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS old_style; CREATE TABLE old_style(d Date, x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/old_style', 'r1', d, x, 8192); diff --git a/tests/queries/0_stateless/00758_array_reverse.sql b/tests/queries/0_stateless/00758_array_reverse.sql index 8a2417e68b0..11192535dc1 100644 --- a/tests/queries/0_stateless/00758_array_reverse.sql +++ b/tests/queries/0_stateless/00758_array_reverse.sql @@ -11,5 +11,5 @@ SELECT reverse(NULL); SELECT reverse([]); SELECT reverse([[[[]]]]); -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SELECT '[RE7', ( SELECT '\0' ) AS riwwq, ( SELECT reverse([( SELECT bitTestAll(NULL) ) , ( SELECT '\0' ) AS ddfweeuy]) ) AS xuvv, '', ( SELECT * FROM file() ) AS wqgdswyc, ( SELECT * FROM file() ); -- { serverError 42 } diff --git a/tests/queries/0_stateless/00762_date_comparsion.sql b/tests/queries/0_stateless/00762_date_comparsion.sql index b874cb0b0b3..cc054bc7047 100644 --- a/tests/queries/0_stateless/00762_date_comparsion.sql +++ b/tests/queries/0_stateless/00762_date_comparsion.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; select today() < 2018-11-14; -- { serverError 43 } select toDate('2018-01-01') < '2018-11-14'; diff --git a/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql b/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql index a7f1f3ad98a..4e16768b373 100644 --- a/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql +++ b/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; select lcase('FOO'); select ucase('foo'); diff --git a/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql b/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql index 8a4cb9f81d7..a9e6c12735c 100644 --- a/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql +++ b/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS alter_compression_codec; diff --git a/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql b/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql index 2c7b3b39469..61298de2548 100644 --- a/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql +++ b/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SET allow_suspicious_codecs = 1; DROP TABLE IF EXISTS compression_codec; diff --git a/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.sql b/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.sql index f61adccefd5..fba6a216762 100644 --- a/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.sql +++ b/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SET allow_suspicious_codecs = 1; -- copy-paste for storage log diff --git a/tests/queries/0_stateless/00804_test_delta_codec_compression.sql b/tests/queries/0_stateless/00804_test_delta_codec_compression.sql index 5f4e3d9be79..ad104eff92c 100644 --- a/tests/queries/0_stateless/00804_test_delta_codec_compression.sql +++ b/tests/queries/0_stateless/00804_test_delta_codec_compression.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SET joined_subquery_requires_alias = 0; DROP TABLE IF EXISTS delta_codec_synthetic; diff --git a/tests/queries/0_stateless/00804_test_delta_codec_no_type_alter.sql b/tests/queries/0_stateless/00804_test_delta_codec_no_type_alter.sql index f49b8b2ee32..4cb34b6384e 100644 --- a/tests/queries/0_stateless/00804_test_delta_codec_no_type_alter.sql +++ b/tests/queries/0_stateless/00804_test_delta_codec_no_type_alter.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SET allow_suspicious_codecs = 1; DROP TABLE IF EXISTS delta_codec_for_alter; diff --git a/tests/queries/0_stateless/00805_round_down.sql b/tests/queries/0_stateless/00805_round_down.sql index bf25f7be952..6d59cb0af1a 100644 --- a/tests/queries/0_stateless/00805_round_down.sql +++ b/tests/queries/0_stateless/00805_round_down.sql @@ -4,7 +4,7 @@ SELECT toInt32(number) as x, roundDown(x, [e(), pi(), pi(), e()]) FROM system.nu SELECT number as x, roundDown(x, [6, 5, 4]) FROM system.numbers LIMIT 10; SELECT 1 as x, roundDown(x, [6, 5, 4]); -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SELECT 1 as x, roundDown(x, []); -- { serverError 43 } SELECT 1 as x, roundDown(x, emptyArrayUInt8()); -- { serverError 44 } SELECT roundDown(number, [number]) FROM system.numbers LIMIT 10; -- { serverError 44 } diff --git a/tests/queries/0_stateless/00808_array_enumerate_segfault.sql b/tests/queries/0_stateless/00808_array_enumerate_segfault.sql index b492d3114f8..88f9b821685 100644 --- a/tests/queries/0_stateless/00808_array_enumerate_segfault.sql +++ b/tests/queries/0_stateless/00808_array_enumerate_segfault.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SELECT arrayEnumerateUniq(anyHeavy([]), []); SELECT arrayEnumerateDense([], [sequenceCount(NULL)]); -- { serverError 190 } SELECT arrayEnumerateDense([STDDEV_SAMP(NULL, 910947.571364)], [NULL]); diff --git a/tests/queries/0_stateless/00808_not_optimize_predicate.sql b/tests/queries/0_stateless/00808_not_optimize_predicate.sql index b4204f0c3bb..ad3df16e4bb 100644 --- a/tests/queries/0_stateless/00808_not_optimize_predicate.sql +++ b/tests/queries/0_stateless/00808_not_optimize_predicate.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS test_00808; CREATE TABLE test_00808(date Date, id Int8, name String, value Int64, sign Int8) ENGINE = CollapsingMergeTree(sign) ORDER BY (id, date); diff --git a/tests/queries/0_stateless/00809_add_days_segfault.sql b/tests/queries/0_stateless/00809_add_days_segfault.sql index 3d2e11ece77..6791439708a 100644 --- a/tests/queries/0_stateless/00809_add_days_segfault.sql +++ b/tests/queries/0_stateless/00809_add_days_segfault.sql @@ -4,7 +4,7 @@ SELECT ignore(subtractDays(toDateTime(0), 1)); SELECT ignore(addDays(toDate(0), -1)); SELECT ignore(subtractDays(toDate(0), 1)); -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SELECT ignore(addDays((CAST((96.338) AS DateTime)), -3)); SELECT ignore(subtractDays((CAST((-5263074.47) AS DateTime)), -737895)); diff --git a/tests/queries/0_stateless/00810_in_operators_segfault.sql b/tests/queries/0_stateless/00810_in_operators_segfault.sql index 8e4a4723608..99ce7231780 100644 --- a/tests/queries/0_stateless/00810_in_operators_segfault.sql +++ b/tests/queries/0_stateless/00810_in_operators_segfault.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SELECT globalNotIn(['"wh'], [NULL]); SELECT globalIn([''], [NULL]); diff --git a/tests/queries/0_stateless/00811_garbage.sql b/tests/queries/0_stateless/00811_garbage.sql index 65db1c736fa..93d9ec963c2 100644 --- a/tests/queries/0_stateless/00811_garbage.sql +++ b/tests/queries/0_stateless/00811_garbage.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SELECT truncate(895, -16); SELECT ( SELECT toDecimal128([], rowNumberInBlock()) ) , lcm('', [[(CAST(('>A') AS String))]]); -- { serverError 44 } diff --git a/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated.sql b/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated.sql index be83f040e04..68823386853 100644 --- a/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated.sql +++ b/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SET allow_suspicious_codecs = 1; DROP TABLE IF EXISTS test.compression_codec_replicated1; diff --git a/tests/queries/0_stateless/00910_zookeeper_test_alter_compression_codecs.sql b/tests/queries/0_stateless/00910_zookeeper_test_alter_compression_codecs.sql index cec9ff86cd0..cd0512d6be8 100644 --- a/tests/queries/0_stateless/00910_zookeeper_test_alter_compression_codecs.sql +++ b/tests/queries/0_stateless/00910_zookeeper_test_alter_compression_codecs.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SET replication_alter_partitions_sync = 2; DROP TABLE IF EXISTS alter_compression_codec1; diff --git a/tests/queries/0_stateless/00926_adaptive_index_granularity_pk.sql b/tests/queries/0_stateless/00926_adaptive_index_granularity_pk.sql index 53e640a4032..58e336e03c7 100644 --- a/tests/queries/0_stateless/00926_adaptive_index_granularity_pk.sql +++ b/tests/queries/0_stateless/00926_adaptive_index_granularity_pk.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SELECT '----00489----'; DROP TABLE IF EXISTS pk; diff --git a/tests/queries/0_stateless/00929_multi_match_edit_distance.sql b/tests/queries/0_stateless/00929_multi_match_edit_distance.sql index c0f39a4f201..50f2359a685 100644 --- a/tests/queries/0_stateless/00929_multi_match_edit_distance.sql +++ b/tests/queries/0_stateless/00929_multi_match_edit_distance.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; select 0 = multiFuzzyMatchAny('abc', 0, ['a1c']) from system.numbers limit 5; select 1 = multiFuzzyMatchAny('abc', 1, ['a1c']) from system.numbers limit 5; diff --git a/tests/queries/0_stateless/00933_alter_ttl.sql b/tests/queries/0_stateless/00933_alter_ttl.sql index 4e0fde00952..1aacb9a6694 100644 --- a/tests/queries/0_stateless/00933_alter_ttl.sql +++ b/tests/queries/0_stateless/00933_alter_ttl.sql @@ -1,4 +1,4 @@ -set send_logs_level = 'none'; +set send_logs_level = 'fatal'; drop table if exists ttl; diff --git a/tests/queries/0_stateless/00933_ttl_simple.sql b/tests/queries/0_stateless/00933_ttl_simple.sql index 6c750f8749d..b924faad3f5 100644 --- a/tests/queries/0_stateless/00933_ttl_simple.sql +++ b/tests/queries/0_stateless/00933_ttl_simple.sql @@ -83,7 +83,7 @@ select sleep(0.7) format Null; -- wait if very fast merge happen optimize table ttl_00933_1 final; select * from ttl_00933_1; -set send_logs_level = 'none'; +set send_logs_level = 'fatal'; drop table if exists ttl_00933_1; diff --git a/tests/queries/0_stateless/00949_format.sql b/tests/queries/0_stateless/00949_format.sql index 433ababde9d..0683b2b6952 100644 --- a/tests/queries/0_stateless/00949_format.sql +++ b/tests/queries/0_stateless/00949_format.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; select format('Hello {1} World {0}', materialize('first'), materialize('second')) from system.numbers limit 1; select format('Hello {0} World {1}', materialize('first'), materialize('second')) from system.numbers limit 2; diff --git a/tests/queries/0_stateless/00955_test_final_mark.sql b/tests/queries/0_stateless/00955_test_final_mark.sql index d58bdec7472..f77f1b7b30a 100644 --- a/tests/queries/0_stateless/00955_test_final_mark.sql +++ b/tests/queries/0_stateless/00955_test_final_mark.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS mt_with_pk; diff --git a/tests/queries/0_stateless/01018_ddl_dictionaries_create.sql b/tests/queries/0_stateless/01018_ddl_dictionaries_create.sql index d420ecdd445..d7d7c02baa8 100644 --- a/tests/queries/0_stateless/01018_ddl_dictionaries_create.sql +++ b/tests/queries/0_stateless/01018_ddl_dictionaries_create.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP DATABASE IF EXISTS database_for_dict; diff --git a/tests/queries/0_stateless/01018_ddl_dictionaries_select.sql b/tests/queries/0_stateless/01018_ddl_dictionaries_select.sql index 7f2e1d08671..f4de269e774 100644 --- a/tests/queries/0_stateless/01018_ddl_dictionaries_select.sql +++ b/tests/queries/0_stateless/01018_ddl_dictionaries_select.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP DATABASE IF EXISTS database_for_dict; diff --git a/tests/queries/0_stateless/01018_ddl_dictionaries_special.sql b/tests/queries/0_stateless/01018_ddl_dictionaries_special.sql index 2cea0c8e61d..6d9b499a247 100644 --- a/tests/queries/0_stateless/01018_ddl_dictionaries_special.sql +++ b/tests/queries/0_stateless/01018_ddl_dictionaries_special.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP DATABASE IF EXISTS database_for_dict; diff --git a/tests/queries/0_stateless/01018_dictionaries_from_dictionaries.sql b/tests/queries/0_stateless/01018_dictionaries_from_dictionaries.sql index 029dc9755d8..4d2cd6351b5 100644 --- a/tests/queries/0_stateless/01018_dictionaries_from_dictionaries.sql +++ b/tests/queries/0_stateless/01018_dictionaries_from_dictionaries.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP DATABASE IF EXISTS database_for_dict; diff --git a/tests/queries/0_stateless/01033_dictionaries_lifetime.sql b/tests/queries/0_stateless/01033_dictionaries_lifetime.sql index 8b16c401afe..57776e1fec1 100644 --- a/tests/queries/0_stateless/01033_dictionaries_lifetime.sql +++ b/tests/queries/0_stateless/01033_dictionaries_lifetime.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP DATABASE IF EXISTS database_for_dict; diff --git a/tests/queries/0_stateless/01037_polygon_dict_multi_polygons.sql b/tests/queries/0_stateless/01037_polygon_dict_multi_polygons.sql index 4c446c28b51..9750ec04daf 100644 --- a/tests/queries/0_stateless/01037_polygon_dict_multi_polygons.sql +++ b/tests/queries/0_stateless/01037_polygon_dict_multi_polygons.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP DATABASE IF EXISTS test_01037; diff --git a/tests/queries/0_stateless/01037_polygon_dict_simple_polygons.sql b/tests/queries/0_stateless/01037_polygon_dict_simple_polygons.sql index 56e5865a5a1..9ffbac78bbb 100644 --- a/tests/queries/0_stateless/01037_polygon_dict_simple_polygons.sql +++ b/tests/queries/0_stateless/01037_polygon_dict_simple_polygons.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP DATABASE IF EXISTS test_01037; diff --git a/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.sql b/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.sql index 1d195b0388f..d8f5356d145 100644 --- a/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.sql +++ b/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.sql @@ -1,5 +1,5 @@ SET check_query_single_value_result = 0; -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS mt_without_pk; diff --git a/tests/queries/0_stateless/01038_array_of_unnamed_tuples.sql b/tests/queries/0_stateless/01038_array_of_unnamed_tuples.sql index c9805599549..5da319f1a25 100644 --- a/tests/queries/0_stateless/01038_array_of_unnamed_tuples.sql +++ b/tests/queries/0_stateless/01038_array_of_unnamed_tuples.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS array_of_tuples; diff --git a/tests/queries/0_stateless/01115_join_with_dictionary.sql b/tests/queries/0_stateless/01115_join_with_dictionary.sql index 65704f2b3eb..f1477df7df2 100644 --- a/tests/queries/0_stateless/01115_join_with_dictionary.sql +++ b/tests/queries/0_stateless/01115_join_with_dictionary.sql @@ -1,4 +1,4 @@ -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; DROP DATABASE IF EXISTS db_01115; CREATE DATABASE db_01115 Engine = Ordinary; diff --git a/tests/queries/0_stateless/01247_dist_on_dist_group_by_sharding_key_optimization.sql b/tests/queries/0_stateless/01247_dist_on_dist_group_by_sharding_key_optimization.sql index 29e45ffdb80..b4852793e7c 100644 --- a/tests/queries/0_stateless/01247_dist_on_dist_group_by_sharding_key_optimization.sql +++ b/tests/queries/0_stateless/01247_dist_on_dist_group_by_sharding_key_optimization.sql @@ -1,7 +1,7 @@ -- TODO: correct testing with real unique shards -- Avoid "Connection failed at try №1" messages. -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SET connect_timeout_with_failover_ms = 5000; set optimize_distributed_group_by_sharding_key=1; diff --git a/tests/queries/0_stateless/01247_distributed_group_by_no_merge_GROUP_BY_injective_sharding_key.sql b/tests/queries/0_stateless/01247_distributed_group_by_no_merge_GROUP_BY_injective_sharding_key.sql index d4753e95f79..4a8842ca71f 100644 --- a/tests/queries/0_stateless/01247_distributed_group_by_no_merge_GROUP_BY_injective_sharding_key.sql +++ b/tests/queries/0_stateless/01247_distributed_group_by_no_merge_GROUP_BY_injective_sharding_key.sql @@ -1,7 +1,7 @@ -- TODO: correct testing with real unique shards -- Avoid "Connection failed at try №1" messages. -SET send_logs_level = 'none'; +SET send_logs_level = 'fatal'; SET connect_timeout_with_failover_ms = 5000; set optimize_distributed_group_by_sharding_key=1;