mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 16:42:05 +00:00
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:
commit
1c1e12f06b
@ -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();
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
|
@ -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();
|
||||
|
||||
|
@ -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();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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_);
|
||||
};
|
||||
|
@ -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) \
|
||||
\
|
||||
|
@ -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") \
|
||||
|
@ -302,6 +302,7 @@ using SettingDateTimeInputFormat = SettingEnum<FormatSettings::DateTimeInputForm
|
||||
enum class LogsLevel
|
||||
{
|
||||
none = 0, /// Disable
|
||||
fatal,
|
||||
error,
|
||||
warning,
|
||||
information,
|
||||
|
@ -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)
|
||||
|
@ -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
|
||||
|
||||
|
@ -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):
|
||||
|
@ -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):
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
|
@ -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');
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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));
|
||||
|
@ -1,4 +1,4 @@
|
||||
SET send_logs_level = 'none';
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
SELECT if(); -- { serverError 42 }
|
||||
SELECT if(1); -- { serverError 42 }
|
||||
|
@ -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 }
|
||||
|
@ -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;
|
||||
|
@ -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 (
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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 }
|
||||
|
@ -1,4 +1,4 @@
|
||||
SET send_logs_level = 'none';
|
||||
SET send_logs_level = 'fatal';
|
||||
SET allow_ddl = 0;
|
||||
|
||||
CREATE DATABASE some_db; -- { serverError 392 }
|
||||
|
@ -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;
|
||||
|
@ -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 }
|
||||
|
@ -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);
|
||||
|
@ -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';
|
||||
|
@ -1,4 +1,4 @@
|
||||
SET send_logs_level = 'none';
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
DROP TABLE IF EXISTS quorum1;
|
||||
DROP TABLE IF EXISTS quorum2;
|
||||
|
@ -1,4 +1,4 @@
|
||||
SET send_logs_level = 'none';
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
DROP TABLE IF EXISTS quorum1;
|
||||
DROP TABLE IF EXISTS quorum2;
|
||||
|
@ -1,4 +1,4 @@
|
||||
SET send_logs_level = 'none';
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
DROP TABLE IF EXISTS quorum1;
|
||||
DROP TABLE IF EXISTS quorum2;
|
||||
|
@ -1,4 +1,4 @@
|
||||
SET send_logs_level = 'none';
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
DROP TABLE IF EXISTS quorum1;
|
||||
DROP TABLE IF EXISTS quorum2;
|
||||
|
@ -1,4 +1,4 @@
|
||||
SET send_logs_level = 'none';
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
DROP TABLE IF EXISTS quorum1;
|
||||
DROP TABLE IF EXISTS quorum2;
|
||||
|
@ -1,4 +1,4 @@
|
||||
SET send_logs_level = 'none';
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
DROP TABLE IF EXISTS quorum1;
|
||||
DROP TABLE IF EXISTS quorum2;
|
||||
|
@ -1,4 +1,4 @@
|
||||
SET send_logs_level = 'none';
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
SELECT 'value vs value';
|
||||
|
||||
|
@ -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 }
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
|
@ -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 }
|
||||
|
@ -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';
|
||||
|
@ -1,4 +1,4 @@
|
||||
SET send_logs_level = 'none';
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
select lcase('FOO');
|
||||
select ucase('foo');
|
||||
|
@ -1,4 +1,4 @@
|
||||
SET send_logs_level = 'none';
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
DROP TABLE IF EXISTS alter_compression_codec;
|
||||
|
||||
|
@ -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;
|
||||
|
@ -1,4 +1,4 @@
|
||||
SET send_logs_level = 'none';
|
||||
SET send_logs_level = 'fatal';
|
||||
SET allow_suspicious_codecs = 1;
|
||||
|
||||
-- copy-paste for storage log
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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 }
|
||||
|
@ -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]);
|
||||
|
@ -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);
|
||||
|
@ -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));
|
||||
|
@ -1,4 +1,4 @@
|
||||
SET send_logs_level = 'none';
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
SELECT globalNotIn(['"wh'], [NULL]);
|
||||
SELECT globalIn([''], [NULL]);
|
||||
|
@ -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 }
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -1,4 +1,4 @@
|
||||
SET send_logs_level = 'none';
|
||||
SET send_logs_level = 'fatal';
|
||||
SELECT '----00489----';
|
||||
DROP TABLE IF EXISTS pk;
|
||||
|
||||
|
@ -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;
|
||||
|
@ -1,4 +1,4 @@
|
||||
set send_logs_level = 'none';
|
||||
set send_logs_level = 'fatal';
|
||||
|
||||
drop table if exists ttl;
|
||||
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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;
|
||||
|
@ -1,4 +1,4 @@
|
||||
SET send_logs_level = 'none';
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
DROP TABLE IF EXISTS mt_with_pk;
|
||||
|
||||
|
@ -1,4 +1,4 @@
|
||||
SET send_logs_level = 'none';
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
DROP DATABASE IF EXISTS database_for_dict;
|
||||
|
||||
|
@ -1,4 +1,4 @@
|
||||
SET send_logs_level = 'none';
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
DROP DATABASE IF EXISTS database_for_dict;
|
||||
|
||||
|
@ -1,4 +1,4 @@
|
||||
SET send_logs_level = 'none';
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
DROP DATABASE IF EXISTS database_for_dict;
|
||||
|
||||
|
@ -1,4 +1,4 @@
|
||||
SET send_logs_level = 'none';
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
DROP DATABASE IF EXISTS database_for_dict;
|
||||
|
||||
|
@ -1,4 +1,4 @@
|
||||
SET send_logs_level = 'none';
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
DROP DATABASE IF EXISTS database_for_dict;
|
||||
|
||||
|
@ -1,4 +1,4 @@
|
||||
SET send_logs_level = 'none';
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
DROP DATABASE IF EXISTS test_01037;
|
||||
|
||||
|
@ -1,4 +1,4 @@
|
||||
SET send_logs_level = 'none';
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
DROP DATABASE IF EXISTS test_01037;
|
||||
|
||||
|
@ -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;
|
||||
|
||||
|
@ -1,4 +1,4 @@
|
||||
SET send_logs_level = 'none';
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
DROP TABLE IF EXISTS array_of_tuples;
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
Loading…
Reference in New Issue
Block a user