Merge pull request #11826 from ClickHouse/send-logs-on-fatal-errors

Send logs to client on fatal errors if possible
This commit is contained in:
alexey-milovidov 2020-06-23 09:49:48 +03:00 committed by GitHub
commit 1c1e12f06b
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
79 changed files with 213 additions and 98 deletions

View File

@ -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();
}
};

View File

@ -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;

View File

@ -72,6 +72,13 @@ void CurrentThread::attachInternalTextLogsQueue(const std::shared_ptr<InternalTe
current_thread->attachInternalTextLogsQueue(logs_queue, client_logs_level);
}
void CurrentThread::setFatalErrorCallback(std::function<void()> callback)
{
if (unlikely(!current_thread))
return;
current_thread->setFatalErrorCallback(callback);
}
std::shared_ptr<InternalTextLogsQueue> CurrentThread::getInternalTextLogsQueue()
{
/// NOTE: this method could be called at early server startup stage

View File

@ -46,6 +46,8 @@ public:
LogsLevel client_logs_level);
static std::shared_ptr<InternalTextLogsQueue> getInternalTextLogsQueue();
static void setFatalErrorCallback(std::function<void()> callback);
/// Makes system calls to update ProfileEvents that contain info from rusage and taskstats
static void updatePerformanceCounters();

View File

@ -98,4 +98,15 @@ void ThreadStatus::attachInternalTextLogsQueue(const InternalTextLogsQueuePtr &
thread_group->client_logs_level = client_logs_level;
}
void ThreadStatus::setFatalErrorCallback(std::function<void()> callback)
{
fatal_error_callback = std::move(callback);
}
void ThreadStatus::onFatalError()
{
if (fatal_error_callback)
fatal_error_callback();
}
}

View File

@ -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<void()> 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<RUsageCounters> last_rusage;
std::unique_ptr<TasksStatsCounters> taskstats;
/// Is used to send logs from logs_queue to client in case of fatal errors.
std::function<void()> fatal_error_callback;
private:
void setupState(const ThreadGroupStatusPtr & thread_group_);
};

View File

@ -316,7 +316,7 @@ struct Settings : public SettingsCollection<Settings>
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) \
\

View File

@ -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") \

View File

@ -302,6 +302,7 @@ using SettingDateTimeInputFormat = SettingEnum<FormatSettings::DateTimeInputForm
enum class LogsLevel
{
none = 0, /// Disable
fatal,
error,
warning,
information,

View File

@ -189,6 +189,7 @@ void TCPHandler::runImpl()
state.logs_queue = std::make_shared<InternalTextLogsQueue>();
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)

View File

@ -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 ' <Fatal> ' 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

View File

@ -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):

View File

@ -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):

View File

@ -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;

View File

@ -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;

View File

@ -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;

View File

@ -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);

View File

@ -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');

View File

@ -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;

View File

@ -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;

View File

@ -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;

View File

@ -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));

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
SELECT if(); -- { serverError 42 }
SELECT if(1); -- { serverError 42 }

View File

@ -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 }

View File

@ -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;

View File

@ -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 (

View File

@ -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;

View File

@ -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;

View File

@ -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 }

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
SET allow_ddl = 0;
CREATE DATABASE some_db; -- { serverError 392 }

View File

@ -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;

View File

@ -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 }

View File

@ -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);

View File

@ -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';

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
DROP TABLE IF EXISTS quorum1;
DROP TABLE IF EXISTS quorum2;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
DROP TABLE IF EXISTS quorum1;
DROP TABLE IF EXISTS quorum2;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
DROP TABLE IF EXISTS quorum1;
DROP TABLE IF EXISTS quorum2;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
DROP TABLE IF EXISTS quorum1;
DROP TABLE IF EXISTS quorum2;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
DROP TABLE IF EXISTS quorum1;
DROP TABLE IF EXISTS quorum2;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
DROP TABLE IF EXISTS quorum1;
DROP TABLE IF EXISTS quorum2;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
SELECT 'value vs value';

View File

@ -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 }

View File

@ -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;

View File

@ -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);

View File

@ -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);

View File

@ -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 }

View File

@ -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';

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
select lcase('FOO');
select ucase('foo');

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
DROP TABLE IF EXISTS alter_compression_codec;

View File

@ -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;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
SET allow_suspicious_codecs = 1;
-- copy-paste for storage log

View File

@ -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;

View File

@ -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;

View File

@ -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 }

View File

@ -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]);

View File

@ -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);

View File

@ -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));

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
SELECT globalNotIn(['"wh'], [NULL]);
SELECT globalIn([''], [NULL]);

View File

@ -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 }

View File

@ -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;

View File

@ -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;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
SELECT '----00489----';
DROP TABLE IF EXISTS pk;

View File

@ -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;

View File

@ -1,4 +1,4 @@
set send_logs_level = 'none';
set send_logs_level = 'fatal';
drop table if exists ttl;

View File

@ -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;

View File

@ -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;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
DROP TABLE IF EXISTS mt_with_pk;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
DROP DATABASE IF EXISTS database_for_dict;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
DROP DATABASE IF EXISTS database_for_dict;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
DROP DATABASE IF EXISTS database_for_dict;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
DROP DATABASE IF EXISTS database_for_dict;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
DROP DATABASE IF EXISTS database_for_dict;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
DROP DATABASE IF EXISTS test_01037;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
DROP DATABASE IF EXISTS test_01037;

View File

@ -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;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
DROP TABLE IF EXISTS array_of_tuples;

View File

@ -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;

View File

@ -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;

View File

@ -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;