2016-10-27 17:48:12 +00:00
|
|
|
#pragma once
|
|
|
|
|
|
|
|
#include <thread>
|
2019-03-21 19:22:38 +00:00
|
|
|
#include <atomic>
|
2020-04-13 01:33:05 +00:00
|
|
|
#include <memory>
|
|
|
|
#include <vector>
|
|
|
|
|
2019-06-19 12:28:34 +00:00
|
|
|
#include <condition_variable>
|
2016-10-27 17:48:12 +00:00
|
|
|
#include <boost/noncopyable.hpp>
|
2017-01-21 04:24:28 +00:00
|
|
|
#include <common/logger_useful.h>
|
2021-01-31 10:02:35 +00:00
|
|
|
#include <ext/scope_guard.h>
|
2020-09-15 09:55:57 +00:00
|
|
|
#include <common/types.h>
|
2020-04-15 20:28:05 +00:00
|
|
|
#include <Core/Defines.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Storages/IStorage.h>
|
|
|
|
#include <Common/Stopwatch.h>
|
|
|
|
#include <Parsers/ASTCreateQuery.h>
|
|
|
|
#include <Parsers/parseQuery.h>
|
2017-09-17 18:49:43 +00:00
|
|
|
#include <Parsers/ParserCreateQuery.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Parsers/ASTRenameQuery.h>
|
|
|
|
#include <Parsers/formatAST.h>
|
2019-10-07 14:18:18 +00:00
|
|
|
#include <Parsers/ASTIndexDeclaration.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Parsers/ASTInsertQuery.h>
|
|
|
|
#include <Interpreters/InterpreterCreateQuery.h>
|
|
|
|
#include <Interpreters/InterpreterRenameQuery.h>
|
|
|
|
#include <Interpreters/InterpreterInsertQuery.h>
|
2020-07-03 17:19:32 +00:00
|
|
|
#include <Interpreters/Context.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Common/setThreadName.h>
|
2019-01-14 19:22:09 +00:00
|
|
|
#include <Common/ThreadPool.h>
|
2018-06-05 19:46:49 +00:00
|
|
|
#include <IO/WriteHelpers.h>
|
2018-06-14 13:03:23 +00:00
|
|
|
#include <Poco/Util/AbstractConfiguration.h>
|
2016-10-27 17:48:12 +00:00
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
|
|
|
|
/** Allow to store structured log in system table.
|
|
|
|
*
|
|
|
|
* Logging is asynchronous. Data is put into queue from where it will be read by separate thread.
|
|
|
|
* That thread inserts log into a table with no more than specified periodicity.
|
|
|
|
*/
|
|
|
|
|
|
|
|
/** Structure of log, template parameter.
|
|
|
|
* Structure could change on server version update.
|
|
|
|
* If on first write, existing table has different structure,
|
|
|
|
* then it get renamed (put aside) and new table is created.
|
|
|
|
*/
|
|
|
|
/* Example:
|
2017-04-01 07:20:54 +00:00
|
|
|
struct LogElement
|
|
|
|
{
|
|
|
|
/// default constructor must be available
|
|
|
|
/// fields
|
2016-10-27 17:48:12 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
static std::string name();
|
|
|
|
static Block createBlock();
|
2020-05-21 20:15:18 +00:00
|
|
|
void appendToBlock(MutableColumns & columns) const;
|
2017-04-01 07:20:54 +00:00
|
|
|
};
|
|
|
|
*/
|
2016-10-27 17:48:12 +00:00
|
|
|
|
2020-01-31 15:45:58 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int TIMEOUT_EXCEEDED;
|
|
|
|
}
|
2016-10-27 17:48:12 +00:00
|
|
|
|
2020-05-23 21:26:45 +00:00
|
|
|
#define DBMS_SYSTEM_LOG_QUEUE_SIZE 1048576
|
2020-05-23 22:30:43 +00:00
|
|
|
|
2020-05-20 20:16:32 +00:00
|
|
|
class QueryLog;
|
|
|
|
class QueryThreadLog;
|
|
|
|
class PartLog;
|
|
|
|
class TextLog;
|
|
|
|
class TraceLog;
|
2020-07-09 04:15:45 +00:00
|
|
|
class CrashLog;
|
2020-05-20 20:16:32 +00:00
|
|
|
class MetricLog;
|
2020-06-10 19:17:30 +00:00
|
|
|
class AsynchronousMetricLog;
|
2020-08-20 20:59:40 +00:00
|
|
|
class OpenTelemetrySpanLog;
|
2020-04-13 01:33:05 +00:00
|
|
|
|
|
|
|
|
|
|
|
class ISystemLog
|
|
|
|
{
|
|
|
|
public:
|
|
|
|
virtual String getName() = 0;
|
|
|
|
virtual ASTPtr getCreateTableQuery() = 0;
|
2020-06-09 18:11:08 +00:00
|
|
|
//// force -- force table creation (used for SYSTEM FLUSH LOGS)
|
|
|
|
virtual void flush(bool force = false) = 0;
|
2020-04-19 23:11:41 +00:00
|
|
|
virtual void prepareTable() = 0;
|
|
|
|
virtual void startup() = 0;
|
2020-04-13 01:33:05 +00:00
|
|
|
virtual void shutdown() = 0;
|
|
|
|
virtual ~ISystemLog() = default;
|
|
|
|
};
|
|
|
|
|
2017-06-05 13:59:38 +00:00
|
|
|
|
2018-03-01 16:52:24 +00:00
|
|
|
/// System logs should be destroyed in destructor of the last Context and before tables,
|
2017-06-05 13:59:38 +00:00
|
|
|
/// because SystemLog destruction makes insert query while flushing data into underlying tables
|
|
|
|
struct SystemLogs
|
|
|
|
{
|
2021-04-10 23:33:54 +00:00
|
|
|
SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConfiguration & config);
|
2017-06-05 13:59:38 +00:00
|
|
|
~SystemLogs();
|
|
|
|
|
2019-06-21 17:25:47 +00:00
|
|
|
void shutdown();
|
|
|
|
|
2019-03-21 19:22:38 +00:00
|
|
|
std::shared_ptr<QueryLog> query_log; /// Used to log queries.
|
|
|
|
std::shared_ptr<QueryThreadLog> query_thread_log; /// Used to log query threads.
|
|
|
|
std::shared_ptr<PartLog> part_log; /// Used to log operations with parts
|
2019-03-23 18:45:22 +00:00
|
|
|
std::shared_ptr<TraceLog> trace_log; /// Used to log traces from query profiler
|
2020-07-09 04:15:45 +00:00
|
|
|
std::shared_ptr<CrashLog> crash_log; /// Used to log server crashes.
|
2019-08-04 15:51:04 +00:00
|
|
|
std::shared_ptr<TextLog> text_log; /// Used to log all text messages.
|
2019-08-13 14:31:46 +00:00
|
|
|
std::shared_ptr<MetricLog> metric_log; /// Used to log all metrics.
|
2020-06-10 19:17:30 +00:00
|
|
|
/// Metrics from system.asynchronous_metrics.
|
|
|
|
std::shared_ptr<AsynchronousMetricLog> asynchronous_metric_log;
|
2020-10-22 16:47:20 +00:00
|
|
|
/// OpenTelemetry trace spans.
|
|
|
|
std::shared_ptr<OpenTelemetrySpanLog> opentelemetry_span_log;
|
2020-04-13 01:33:05 +00:00
|
|
|
|
|
|
|
std::vector<ISystemLog *> logs;
|
2017-06-05 13:59:38 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
|
2016-10-27 17:48:12 +00:00
|
|
|
template <typename LogElement>
|
2021-04-10 23:33:54 +00:00
|
|
|
class SystemLog : public ISystemLog, private boost::noncopyable, WithContext
|
2016-10-27 17:48:12 +00:00
|
|
|
{
|
|
|
|
public:
|
2018-09-13 14:59:03 +00:00
|
|
|
using Self = SystemLog;
|
2018-05-31 15:54:08 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/** Parameter: table name where to write log.
|
|
|
|
* If table is not exists, then it get created with specified engine.
|
|
|
|
* If it already exists, then its structure is checked to be compatible with structure of log record.
|
|
|
|
* If it is compatible, then existing table will be used.
|
|
|
|
* If not - then existing table will be renamed to same name but with suffix '_N' at end,
|
|
|
|
* where N - is a minimal number from 1, for that table with corresponding name doesn't exist yet;
|
|
|
|
* and new table get created - as if previous table was not exist.
|
|
|
|
*/
|
|
|
|
SystemLog(
|
2021-04-10 23:33:54 +00:00
|
|
|
ContextPtr context_,
|
2017-04-01 07:20:54 +00:00
|
|
|
const String & database_name_,
|
|
|
|
const String & table_name_,
|
2017-09-17 18:49:43 +00:00
|
|
|
const String & storage_def_,
|
2020-04-19 23:11:41 +00:00
|
|
|
size_t flush_interval_milliseconds_);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
/** Append a record into log.
|
|
|
|
* Writing to table will be done asynchronously and in case of failure, record could be lost.
|
|
|
|
*/
|
2019-06-19 12:28:34 +00:00
|
|
|
void add(const LogElement & element);
|
2016-10-27 17:48:12 +00:00
|
|
|
|
2020-04-13 01:33:05 +00:00
|
|
|
void stopFlushThread();
|
|
|
|
|
2018-06-13 19:01:07 +00:00
|
|
|
/// Flush data in the buffer to disk
|
2020-06-09 18:11:08 +00:00
|
|
|
void flush(bool force = false) override;
|
2019-03-21 19:22:38 +00:00
|
|
|
|
2020-04-19 23:11:41 +00:00
|
|
|
/// Start the background thread.
|
|
|
|
void startup() override;
|
|
|
|
|
2019-03-21 19:22:38 +00:00
|
|
|
/// Stop the background flush thread before destructor. No more data will be written.
|
2020-04-13 01:33:05 +00:00
|
|
|
void shutdown() override
|
|
|
|
{
|
|
|
|
stopFlushThread();
|
Fix lock-order-inversion during system.*_log shutting down
As TSan reports [1]:
WARNING: ThreadSanitizer: lock-order-inversion (potential deadlock) (pid=24429)
Cycle in lock order graph: M152695175523663992 (0x000000000000) => M2505 (0x7b9000002008) => M152695175523663992
Mutex M2505 acquired here while holding mutex M152695175523663992 in thread T7:
0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a301b6)
1 std::__1::__libcpp_recursive_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:385:10 (clickhouse-tsan+0x17cd6e89)
2 std::__1::recursive_mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:71:14 (clickhouse-tsan+0x17cd6e89)
3 std::__1::unique_lock<std::__1::recursive_mutex>::unique_lock(std::__1::recursive_mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:119:61 (clickhouse-tsan+0x11e32a9f)
4 DB::Context::getLock() const obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:517:12 (clickhouse-tsan+0x11e32a9f)
5 DB::Context::getSchedulePool() const obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:1517:17 (clickhouse-tsan+0x11e32a9f)
6 DB::IBackgroundJobExecutor::start() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:158:42 (clickhouse-tsan+0x12bde50a)
7 DB::StorageMergeTree::startup() obj-x86_64-linux-gnu/../src/Storages/StorageMergeTree.cpp:112:29 (clickhouse-tsan+0x129e9e1e)
8 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_2::operator()(std::__1::shared_ptr<DB::IStorage> const&) const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:230:16 (clickhouse-tsan+0x11d6fa2a)
9 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3::operator()() const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:238:56 (clickhouse-tsan+0x11d6fa2a)
10 decltype(std::__1::forward<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(fp)()) std::__1::__invoke<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x11d6fa2a)
11 void std::__1::__invoke_void_return_wrapper<void>::__call<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x11d6fa2a)
12 std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x11d6fa2a)
13 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x11d6fa2a)
14 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b39350)
15 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b39350)
16 ThreadPoolImpl<ThreadFromGlobalPool>::worker(std::__1::__list_iterator<ThreadFromGlobalPool, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b39350)
17 void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3c070)
18 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(fp0)...)) std::__1::__invoke_constexpr<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3682:1 (clickhouse-tsan+0x8b3c070)
19 decltype(auto) std::__1::__apply_tuple_impl<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::__tuple_indices<std::__1::tuple<>&...>) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1415:1 (clickhouse-tsan+0x8b3c070)
20 decltype(auto) std::__1::apply<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1424:1 (clickhouse-tsan+0x8b3c070)
21 ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()::operator()() obj-x86_64-linux-gnu/../src/Common/ThreadPool.h:178:13 (clickhouse-tsan+0x8b3c070)
22 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3bfd1)
23 void std::__1::__invoke_void_return_wrapper<void>::__call<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x8b3bfd1)
24 std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x8b3bfd1)
25 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x8b3bfd1)
26 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b36c75)
27 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b36c75)
28 ThreadPoolImpl<std::__1::thread>::worker(std::__1::__list_iterator<std::__1::thread, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b36c75)
29 void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3a918)
30 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3a918)
31 void std::__1::__thread_execute<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(std::__1::tuple<void, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>&, std::__1::__tuple_indices<>) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:280:5 (clickhouse-tsan+0x8b3a918)
32 void* std::__1::__thread_proxy<std::__1::tuple<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()> >(void*) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:291:5 (clickhouse-tsan+0x8b3a918)
Hint: use TSAN_OPTIONS=second_deadlock_stack=1 to get more informative warning message
Mutex M152695175523663992 acquired here while holding mutex M2505 in main thread:
0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a301b6)
1 std::__1::__libcpp_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:405:10 (clickhouse-tsan+0x17cd6cf9)
2 std::__1::mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:33:14 (clickhouse-tsan+0x17cd6cf9)
3 std::__1::lock_guard<std::__1::mutex>::lock_guard(std::__1::mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:91:27 (clickhouse-tsan+0x12bdee4b)
4 DB::IBackgroundJobExecutor::finish() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:167:21 (clickhouse-tsan+0x12bdee4b)
5 DB::IBackgroundJobExecutor::~IBackgroundJobExecutor() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:185:5 (clickhouse-tsan+0x12bdee4b)
6 DB::StorageMergeTree::~StorageMergeTree() obj-x86_64-linux-gnu/../src/Storages/StorageMergeTree.cpp:174:1 (clickhouse-tsan+0x129ea118)
7 std::__1::default_delete<DB::StorageMergeTree>::operator()(DB::StorageMergeTree*) const obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:1397:5 (clickhouse-tsan+0x12e4433b)
8 std::__1::__shared_ptr_pointer<DB::StorageMergeTree*, std::__1::shared_ptr<DB::StorageMergeTree>::__shared_ptr_default_delete<DB::StorageMergeTree, DB::StorageMergeTree>, std::__1::allocator<DB::StorageMergeTree> >::__on_zero_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2565:5 (clickhouse-tsan+0x12e4433b)
9 std::__1::__shared_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2475:9 (clickhouse-tsan+0x125b355a)
10 std::__1::__shared_weak_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2517:27 (clickhouse-tsan+0x125b355a)
11 std::__1::shared_ptr<DB::IStorage>::~shared_ptr() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:3212:19 (clickhouse-tsan+0x125b355a)
12 DB::SystemLog<DB::AsynchronousMetricLogElement>::~SystemLog() obj-x86_64-linux-gnu/../src/Interpreters/SystemLog.h:118:7 (clickhouse-tsan+0x125b355a)
13 std::__1::allocator<DB::AsynchronousMetricLog>::destroy(DB::AsynchronousMetricLog*) obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:891:15 (clickhouse-tsan+0x125aff68)
14 void std::__1::allocator_traits<std::__1::allocator<DB::AsynchronousMetricLog> >::__destroy<DB::AsynchronousMetricLog>(std::__1::integral_constant<bool, true>, std::__1::allocator<DB::AsynchronousMetricLog>&, DB::AsynchronousMetricLog*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__memory/allocator_traits.h:539:21 (clickhouse-tsan+0x125aff68)
15 void std::__1::allocator_traits<std::__1::allocator<DB::AsynchronousMetricLog> >::destroy<DB::AsynchronousMetricLog>(std::__1::allocator<DB::AsynchronousMetricLog>&, DB::AsynchronousMetricLog*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__memory/allocator_traits.h:487:14 (clickhouse-tsan+0x125aff68)
16 std::__1::__shared_ptr_emplace<DB::AsynchronousMetricLog, std::__1::allocator<DB::AsynchronousMetricLog> >::__on_zero_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2611:9 (clickhouse-tsan+0x125aff68)
17 std::__1::__shared_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2475:9 (clickhouse-tsan+0x1258e74f)
18 std::__1::__shared_weak_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2517:27 (clickhouse-tsan+0x1258e74f)
19 std::__1::shared_ptr<DB::AsynchronousMetricLog>::~shared_ptr() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:3212:19 (clickhouse-tsan+0x1258e74f)
20 DB::SystemLogs::~SystemLogs() obj-x86_64-linux-gnu/../src/Interpreters/SystemLog.cpp:155:1 (clickhouse-tsan+0x1258e74f)
21 std::__1::__optional_destruct_base<DB::SystemLogs, false>::reset() obj-x86_64-linux-gnu/../contrib/libcxx/include/optional:245:21 (clickhouse-tsan+0x11e41085)
22 DB::ContextShared::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:443:21 (clickhouse-tsan+0x11e41085)
23 DB::Context::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:2251:13 (clickhouse-tsan+0x11e39867)
24 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5::operator()() const obj-x86_64-linux-gnu/../programs/server/Server.cpp:892:5 (clickhouse-tsan+0x8ab8732)
25 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::invoke() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:97:9 (clickhouse-tsan+0x8ab8732)
26 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::~basic_scope_guard() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:47:28 (clickhouse-tsan+0x8ab8732)
27 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&) obj-x86_64-linux-gnu/../programs/server/Server.cpp:1395:1 (clickhouse-tsan+0x8ab3839)
28 Poco::Util::Application::run() obj-x86_64-linux-gnu/../contrib/poco/Util/src/Application.cpp:334:8 (clickhouse-tsan+0x15b464ab)
29 DB::Server::run() obj-x86_64-linux-gnu/../programs/server/Server.cpp:342:25 (clickhouse-tsan+0x8aa71be)
30 Poco::Util::ServerApplication::run(int, char**) obj-x86_64-linux-gnu/../contrib/poco/Util/src/ServerApplication.cpp:611:9 (clickhouse-tsan+0x15b628c3)
31 mainEntryClickHouseServer(int, char**) obj-x86_64-linux-gnu/../programs/server/Server.cpp:134:20 (clickhouse-tsan+0x8aa5d8e)
32 main obj-x86_64-linux-gnu/../programs/main.cpp:368:12 (clickhouse-tsan+0x8aa42f9)
[1]: https://clickhouse-test-reports.s3.yandex.net/21318/38be9ff43ac4c46ce6e803fc125d910bde1d4c71/functional_stateful_tests_(thread).html#fail1
2021-03-02 20:41:29 +00:00
|
|
|
if (table)
|
2021-05-13 07:03:00 +00:00
|
|
|
table->flushAndShutdown();
|
2020-04-13 01:33:05 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
String getName() override
|
|
|
|
{
|
|
|
|
return LogElement::name();
|
|
|
|
}
|
|
|
|
|
|
|
|
ASTPtr getCreateTableQuery() override;
|
2018-06-13 19:01:07 +00:00
|
|
|
|
2018-01-23 22:56:46 +00:00
|
|
|
protected:
|
2020-05-30 21:57:37 +00:00
|
|
|
Poco::Logger * log;
|
2020-01-31 15:45:58 +00:00
|
|
|
|
|
|
|
private:
|
|
|
|
/* Saving thread data */
|
2020-02-17 19:28:25 +00:00
|
|
|
const StorageID table_id;
|
2017-09-17 18:49:43 +00:00
|
|
|
const String storage_def;
|
2017-04-01 07:20:54 +00:00
|
|
|
StoragePtr table;
|
2020-01-31 15:45:58 +00:00
|
|
|
bool is_prepared = false;
|
2017-04-01 07:20:54 +00:00
|
|
|
const size_t flush_interval_milliseconds;
|
2019-01-14 19:22:09 +00:00
|
|
|
ThreadFromGlobalPool saving_thread;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-01-31 15:45:58 +00:00
|
|
|
/* Data shared between callers of add()/flush()/shutdown(), and the saving thread */
|
|
|
|
std::mutex mutex;
|
2020-02-09 14:15:29 +00:00
|
|
|
// Queue is bounded. But its size is quite large to not block in all normal cases.
|
2020-01-31 15:45:58 +00:00
|
|
|
std::vector<LogElement> queue;
|
|
|
|
// An always-incrementing index of the first message currently in the queue.
|
2020-05-18 19:33:12 +00:00
|
|
|
// We use it to give a global sequential index to every message, so that we
|
|
|
|
// can wait until a particular message is flushed. This is used to implement
|
|
|
|
// synchronous log flushing for SYSTEM FLUSH LOGS.
|
2020-01-31 15:45:58 +00:00
|
|
|
uint64_t queue_front_index = 0;
|
|
|
|
bool is_shutdown = false;
|
2021-04-07 13:16:19 +00:00
|
|
|
// A flag that says we must create the tables even if the queue is empty.
|
2020-06-09 18:11:08 +00:00
|
|
|
bool is_force_prepare_tables = false;
|
2020-01-31 15:45:58 +00:00
|
|
|
std::condition_variable flush_event;
|
|
|
|
// Requested to flush logs up to this index, exclusive
|
2021-04-07 13:16:19 +00:00
|
|
|
uint64_t requested_flush_up_to = 0;
|
2020-01-31 15:45:58 +00:00
|
|
|
// Flushed log up to this index, exclusive
|
2021-04-07 13:16:19 +00:00
|
|
|
uint64_t flushed_up_to = 0;
|
2020-05-18 19:33:12 +00:00
|
|
|
// Logged overflow message at this queue front index
|
|
|
|
uint64_t logged_queue_full_at_index = -1;
|
2020-01-31 15:45:58 +00:00
|
|
|
|
|
|
|
void savingThreadFunction();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
/** Creates new table if it does not exist.
|
|
|
|
* Renames old table if its structure is not suitable.
|
|
|
|
* This cannot be done in constructor to avoid deadlock while renaming a table under locked Context when SystemLog object is created.
|
|
|
|
*/
|
2020-04-19 23:11:41 +00:00
|
|
|
void prepareTable() override;
|
2019-03-21 19:22:38 +00:00
|
|
|
|
2019-06-19 12:28:34 +00:00
|
|
|
/// flushImpl can be executed only in saving_thread.
|
2020-01-31 15:45:58 +00:00
|
|
|
void flushImpl(const std::vector<LogElement> & to_flush, uint64_t to_flush_end);
|
2016-10-27 17:48:12 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
|
template <typename LogElement>
|
2021-04-10 23:33:54 +00:00
|
|
|
SystemLog<LogElement>::SystemLog(
|
|
|
|
ContextPtr context_,
|
2017-04-01 07:20:54 +00:00
|
|
|
const String & database_name_,
|
|
|
|
const String & table_name_,
|
2017-09-17 18:49:43 +00:00
|
|
|
const String & storage_def_,
|
2020-04-19 23:11:41 +00:00
|
|
|
size_t flush_interval_milliseconds_)
|
2021-04-10 23:33:54 +00:00
|
|
|
: WithContext(context_)
|
2020-02-17 19:28:25 +00:00
|
|
|
, table_id(database_name_, table_name_)
|
2020-05-21 14:11:56 +00:00
|
|
|
, storage_def(storage_def_)
|
|
|
|
, flush_interval_milliseconds(flush_interval_milliseconds_)
|
2016-10-27 17:48:12 +00:00
|
|
|
{
|
2020-03-02 17:25:36 +00:00
|
|
|
assert(database_name_ == DatabaseCatalog::SYSTEM_DATABASE);
|
2020-05-30 21:57:37 +00:00
|
|
|
log = &Poco::Logger::get("SystemLog (" + database_name_ + "." + table_name_ + ")");
|
2020-04-19 23:11:41 +00:00
|
|
|
}
|
2019-08-13 14:32:32 +00:00
|
|
|
|
2020-04-19 19:53:12 +00:00
|
|
|
|
2020-04-19 23:11:41 +00:00
|
|
|
template <typename LogElement>
|
|
|
|
void SystemLog<LogElement>::startup()
|
|
|
|
{
|
2020-06-11 18:55:23 +00:00
|
|
|
std::lock_guard lock(mutex);
|
2020-01-31 15:45:58 +00:00
|
|
|
saving_thread = ThreadFromGlobalPool([this] { savingThreadFunction(); });
|
2016-10-27 17:48:12 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2021-01-31 10:02:35 +00:00
|
|
|
static thread_local bool recursive_add_call = false;
|
|
|
|
|
2019-06-19 12:28:34 +00:00
|
|
|
template <typename LogElement>
|
|
|
|
void SystemLog<LogElement>::add(const LogElement & element)
|
|
|
|
{
|
2021-01-31 10:02:35 +00:00
|
|
|
/// It is possible that the method will be called recursively.
|
|
|
|
/// Better to drop these events to avoid complications.
|
|
|
|
if (recursive_add_call)
|
|
|
|
return;
|
|
|
|
recursive_add_call = true;
|
|
|
|
SCOPE_EXIT({ recursive_add_call = false; });
|
|
|
|
|
2020-03-15 00:22:06 +00:00
|
|
|
/// Memory can be allocated while resizing on queue.push_back.
|
|
|
|
/// The size of allocation can be in order of a few megabytes.
|
|
|
|
/// But this should not be accounted for query memory usage.
|
|
|
|
/// Otherwise the tests like 01017_uniqCombined_memory_usage.sql will be flacky.
|
2021-02-02 03:05:40 +00:00
|
|
|
MemoryTracker::BlockerInThread temporarily_disable_memory_tracker(VariableContext::Global);
|
2020-03-15 00:22:06 +00:00
|
|
|
|
2020-07-13 05:24:55 +00:00
|
|
|
/// Should not log messages under mutex.
|
|
|
|
bool queue_is_half_full = false;
|
2019-06-19 12:28:34 +00:00
|
|
|
|
2020-05-23 22:30:43 +00:00
|
|
|
{
|
2020-07-13 05:24:55 +00:00
|
|
|
std::unique_lock lock(mutex);
|
2020-03-15 00:22:06 +00:00
|
|
|
|
2020-07-13 05:24:55 +00:00
|
|
|
if (is_shutdown)
|
|
|
|
return;
|
2020-01-31 15:45:58 +00:00
|
|
|
|
2020-07-13 05:24:55 +00:00
|
|
|
if (queue.size() == DBMS_SYSTEM_LOG_QUEUE_SIZE / 2)
|
2020-05-18 19:33:12 +00:00
|
|
|
{
|
2020-07-13 05:24:55 +00:00
|
|
|
queue_is_half_full = true;
|
|
|
|
|
|
|
|
// The queue more than half full, time to flush.
|
|
|
|
// We only check for strict equality, because messages are added one
|
|
|
|
// by one, under exclusive lock, so we will see each message count.
|
|
|
|
// It is enough to only wake the flushing thread once, after the message
|
|
|
|
// count increases past half available size.
|
|
|
|
const uint64_t queue_end = queue_front_index + queue.size();
|
2021-04-07 13:16:19 +00:00
|
|
|
if (requested_flush_up_to < queue_end)
|
|
|
|
requested_flush_up_to = queue_end;
|
2020-07-13 05:24:55 +00:00
|
|
|
|
|
|
|
flush_event.notify_all();
|
|
|
|
}
|
2020-05-18 19:33:12 +00:00
|
|
|
|
2020-07-13 05:24:55 +00:00
|
|
|
if (queue.size() >= DBMS_SYSTEM_LOG_QUEUE_SIZE)
|
|
|
|
{
|
|
|
|
// Ignore all further entries until the queue is flushed.
|
|
|
|
// Log a message about that. Don't spam it -- this might be especially
|
|
|
|
// problematic in case of trace log. Remember what the front index of the
|
|
|
|
// queue was when we last logged the message. If it changed, it means the
|
|
|
|
// queue was flushed, and we can log again.
|
|
|
|
if (queue_front_index != logged_queue_full_at_index)
|
|
|
|
{
|
|
|
|
logged_queue_full_at_index = queue_front_index;
|
|
|
|
|
|
|
|
// TextLog sets its logger level to 0, so this log is a noop and
|
|
|
|
// there is no recursive logging.
|
|
|
|
lock.unlock();
|
|
|
|
LOG_ERROR(log, "Queue is full for system log '{}' at {}", demangle(typeid(*this).name()), queue_front_index);
|
|
|
|
}
|
|
|
|
|
|
|
|
return;
|
2020-05-18 19:33:12 +00:00
|
|
|
}
|
|
|
|
|
2020-07-13 05:24:55 +00:00
|
|
|
queue.push_back(element);
|
2020-01-31 15:45:58 +00:00
|
|
|
}
|
|
|
|
|
2020-07-13 05:24:55 +00:00
|
|
|
if (queue_is_half_full)
|
|
|
|
LOG_INFO(log, "Queue is half full for system log '{}'.", demangle(typeid(*this).name()));
|
2019-06-19 12:28:34 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
template <typename LogElement>
|
2020-06-09 18:11:08 +00:00
|
|
|
void SystemLog<LogElement>::flush(bool force)
|
2019-06-19 12:28:34 +00:00
|
|
|
{
|
2021-04-07 13:16:19 +00:00
|
|
|
uint64_t this_thread_requested_offset;
|
2020-01-31 15:45:58 +00:00
|
|
|
|
2021-04-07 13:16:19 +00:00
|
|
|
{
|
|
|
|
std::unique_lock lock(mutex);
|
2019-06-19 12:28:34 +00:00
|
|
|
|
2021-04-07 13:16:19 +00:00
|
|
|
if (is_shutdown)
|
|
|
|
return;
|
|
|
|
|
|
|
|
this_thread_requested_offset = queue_front_index + queue.size();
|
|
|
|
|
|
|
|
// Publish our flush request, taking care not to overwrite the requests
|
|
|
|
// made by other threads.
|
|
|
|
is_force_prepare_tables |= force;
|
|
|
|
requested_flush_up_to = std::max(requested_flush_up_to,
|
|
|
|
this_thread_requested_offset);
|
2020-01-31 15:45:58 +00:00
|
|
|
|
2020-02-14 13:09:51 +00:00
|
|
|
flush_event.notify_all();
|
2020-01-31 15:45:58 +00:00
|
|
|
}
|
2019-06-19 15:38:06 +00:00
|
|
|
|
2021-04-07 13:16:19 +00:00
|
|
|
LOG_DEBUG(log, "Requested flush up to offset {}",
|
|
|
|
this_thread_requested_offset);
|
|
|
|
|
2021-04-09 12:48:09 +00:00
|
|
|
// Use an arbitrary timeout to avoid endless waiting. 60s proved to be
|
|
|
|
// too fast for our parallel functional tests, probably because they
|
|
|
|
// heavily load the disk.
|
|
|
|
const int timeout_seconds = 180;
|
2021-04-07 13:16:19 +00:00
|
|
|
std::unique_lock lock(mutex);
|
2020-01-31 15:45:58 +00:00
|
|
|
bool result = flush_event.wait_for(lock, std::chrono::seconds(timeout_seconds),
|
2021-04-07 13:16:19 +00:00
|
|
|
[&] { return flushed_up_to >= this_thread_requested_offset
|
|
|
|
&& !is_force_prepare_tables; });
|
2019-06-19 12:28:34 +00:00
|
|
|
|
2020-01-31 15:45:58 +00:00
|
|
|
if (!result)
|
|
|
|
{
|
|
|
|
throw Exception("Timeout exceeded (" + toString(timeout_seconds) + " s) while flushing system log '" + demangle(typeid(*this).name()) + "'.",
|
|
|
|
ErrorCodes::TIMEOUT_EXCEEDED);
|
|
|
|
}
|
2019-06-19 12:28:34 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2016-10-27 17:48:12 +00:00
|
|
|
template <typename LogElement>
|
2020-04-13 01:33:05 +00:00
|
|
|
void SystemLog<LogElement>::stopFlushThread()
|
2016-10-27 17:48:12 +00:00
|
|
|
{
|
2020-01-31 15:45:58 +00:00
|
|
|
{
|
2020-06-11 18:55:23 +00:00
|
|
|
std::lock_guard lock(mutex);
|
2020-01-31 15:45:58 +00:00
|
|
|
|
2020-05-09 13:02:37 +00:00
|
|
|
if (!saving_thread.joinable())
|
|
|
|
{
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
|
2020-01-31 15:45:58 +00:00
|
|
|
if (is_shutdown)
|
|
|
|
{
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
|
|
|
|
is_shutdown = true;
|
|
|
|
|
|
|
|
/// Tell thread to shutdown.
|
|
|
|
flush_event.notify_all();
|
|
|
|
}
|
2019-03-21 19:22:38 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
saving_thread.join();
|
2016-10-27 17:48:12 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
template <typename LogElement>
|
2020-01-31 15:45:58 +00:00
|
|
|
void SystemLog<LogElement>::savingThreadFunction()
|
2016-10-27 17:48:12 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
setThreadName("SystemLogFlush");
|
|
|
|
|
2020-02-14 13:09:51 +00:00
|
|
|
std::vector<LogElement> to_flush;
|
2020-01-31 15:45:58 +00:00
|
|
|
bool exit_this_thread = false;
|
|
|
|
while (!exit_this_thread)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
try
|
|
|
|
{
|
2020-01-31 15:45:58 +00:00
|
|
|
// The end index (exclusive, like std end()) of the messages we are
|
|
|
|
// going to flush.
|
|
|
|
uint64_t to_flush_end = 0;
|
2021-04-07 13:16:19 +00:00
|
|
|
// Should we prepare table even if there are no new messages.
|
|
|
|
bool should_prepare_tables_anyway = false;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-01-31 15:45:58 +00:00
|
|
|
{
|
|
|
|
std::unique_lock lock(mutex);
|
2020-05-21 20:15:18 +00:00
|
|
|
flush_event.wait_for(lock,
|
2020-05-18 19:33:12 +00:00
|
|
|
std::chrono::milliseconds(flush_interval_milliseconds),
|
2020-05-19 14:55:27 +00:00
|
|
|
[&] ()
|
|
|
|
{
|
2021-04-07 13:16:19 +00:00
|
|
|
return requested_flush_up_to > flushed_up_to || is_shutdown || is_force_prepare_tables;
|
2020-05-18 19:33:12 +00:00
|
|
|
}
|
|
|
|
);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-01-31 15:45:58 +00:00
|
|
|
queue_front_index += queue.size();
|
|
|
|
to_flush_end = queue_front_index;
|
2020-02-14 13:09:51 +00:00
|
|
|
// Swap with existing array from previous flush, to save memory
|
|
|
|
// allocations.
|
|
|
|
to_flush.resize(0);
|
2020-01-31 15:45:58 +00:00
|
|
|
queue.swap(to_flush);
|
2018-06-19 20:30:35 +00:00
|
|
|
|
2021-04-07 13:16:19 +00:00
|
|
|
should_prepare_tables_anyway = is_force_prepare_tables;
|
|
|
|
|
2020-01-31 15:45:58 +00:00
|
|
|
exit_this_thread = is_shutdown;
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2020-01-31 15:45:58 +00:00
|
|
|
if (to_flush.empty())
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2021-04-07 13:16:19 +00:00
|
|
|
if (should_prepare_tables_anyway)
|
2020-06-09 18:11:08 +00:00
|
|
|
{
|
|
|
|
prepareTable();
|
|
|
|
LOG_TRACE(log, "Table created (force)");
|
|
|
|
|
|
|
|
std::lock_guard lock(mutex);
|
|
|
|
is_force_prepare_tables = false;
|
|
|
|
flush_event.notify_all();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
flushImpl(to_flush, to_flush_end);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
|
|
|
}
|
|
|
|
}
|
2020-05-23 22:24:01 +00:00
|
|
|
LOG_TRACE(log, "Terminating");
|
2016-10-27 17:48:12 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
template <typename LogElement>
|
2020-01-31 15:45:58 +00:00
|
|
|
void SystemLog<LogElement>::flushImpl(const std::vector<LogElement> & to_flush, uint64_t to_flush_end)
|
2016-10-27 17:48:12 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
try
|
|
|
|
{
|
2021-04-07 13:16:19 +00:00
|
|
|
LOG_TRACE(log, "Flushing system log, {} entries to flush up to offset {}",
|
|
|
|
to_flush.size(), to_flush_end);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-05-18 19:33:12 +00:00
|
|
|
/// We check for existence of the table and create it as needed at every
|
|
|
|
/// flush. This is done to allow user to drop the table at any moment
|
|
|
|
/// (new empty table will be created automatically). BTW, flush method
|
|
|
|
/// is called from single thread.
|
2018-03-29 13:57:16 +00:00
|
|
|
prepareTable();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
Block block = LogElement::createBlock();
|
2020-05-21 20:15:18 +00:00
|
|
|
MutableColumns columns = block.mutateColumns();
|
2020-01-31 15:45:58 +00:00
|
|
|
for (const auto & elem : to_flush)
|
2020-05-21 20:15:18 +00:00
|
|
|
elem.appendToBlock(columns);
|
|
|
|
block.setColumns(std::move(columns));
|
2017-06-05 13:59:38 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// We write to table indirectly, using InterpreterInsertQuery.
|
|
|
|
/// This is needed to support DEFAULT-columns in table.
|
|
|
|
|
|
|
|
std::unique_ptr<ASTInsertQuery> insert = std::make_unique<ASTInsertQuery>();
|
2020-03-02 20:23:58 +00:00
|
|
|
insert->table_id = table_id;
|
2017-04-01 07:20:54 +00:00
|
|
|
ASTPtr query_ptr(insert.release());
|
|
|
|
|
2020-07-06 17:24:33 +00:00
|
|
|
// we need query context to do inserts to target table with MV containing subqueries or joins
|
2021-04-10 23:33:54 +00:00
|
|
|
auto insert_context = Context::createCopy(context);
|
|
|
|
insert_context->makeQueryContext();
|
2020-07-06 17:24:33 +00:00
|
|
|
|
2020-07-03 17:19:32 +00:00
|
|
|
InterpreterInsertQuery interpreter(query_ptr, insert_context);
|
2017-04-01 07:20:54 +00:00
|
|
|
BlockIO io = interpreter.execute();
|
|
|
|
|
|
|
|
io.out->writePrefix();
|
|
|
|
io.out->write(block);
|
|
|
|
io.out->writeSuffix();
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
2019-06-19 12:28:34 +00:00
|
|
|
}
|
2020-01-31 15:45:58 +00:00
|
|
|
|
2020-05-18 19:33:12 +00:00
|
|
|
{
|
2020-06-11 18:55:23 +00:00
|
|
|
std::lock_guard lock(mutex);
|
2021-04-07 13:16:19 +00:00
|
|
|
flushed_up_to = to_flush_end;
|
2020-06-09 18:11:08 +00:00
|
|
|
is_force_prepare_tables = false;
|
2020-05-18 19:33:12 +00:00
|
|
|
flush_event.notify_all();
|
|
|
|
}
|
|
|
|
|
2021-04-07 13:16:19 +00:00
|
|
|
LOG_TRACE(log, "Flushed system log up to offset {}", to_flush_end);
|
2016-10-27 17:48:12 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2016-11-23 20:41:39 +00:00
|
|
|
template <typename LogElement>
|
|
|
|
void SystemLog<LogElement>::prepareTable()
|
|
|
|
{
|
2020-02-17 19:28:25 +00:00
|
|
|
String description = table_id.getNameForLogs();
|
2016-11-23 20:41:39 +00:00
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
table = DatabaseCatalog::instance().tryGetTable(table_id, getContext());
|
2016-11-23 20:41:39 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
if (table)
|
|
|
|
{
|
2020-06-16 12:48:10 +00:00
|
|
|
auto metadata_snapshot = table->getInMemoryMetadataPtr();
|
2017-04-01 07:20:54 +00:00
|
|
|
const Block expected = LogElement::createBlock();
|
2020-06-16 12:48:10 +00:00
|
|
|
const Block actual = metadata_snapshot->getSampleBlockNonMaterialized();
|
2016-11-23 20:41:39 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
if (!blocksHaveEqualStructure(actual, expected))
|
|
|
|
{
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Rename the existing table.
|
2017-04-01 07:20:54 +00:00
|
|
|
int suffix = 0;
|
2021-04-10 23:33:54 +00:00
|
|
|
while (DatabaseCatalog::instance().isTableExist(
|
|
|
|
{table_id.database_name, table_id.table_name + "_" + toString(suffix)}, getContext()))
|
2017-04-01 07:20:54 +00:00
|
|
|
++suffix;
|
2016-11-23 20:41:39 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
auto rename = std::make_shared<ASTRenameQuery>();
|
2016-11-23 20:41:39 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
ASTRenameQuery::Table from;
|
2020-02-17 19:28:25 +00:00
|
|
|
from.database = table_id.database_name;
|
|
|
|
from.table = table_id.table_name;
|
2016-11-23 20:41:39 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
ASTRenameQuery::Table to;
|
2020-02-17 19:28:25 +00:00
|
|
|
to.database = table_id.database_name;
|
|
|
|
to.table = table_id.table_name + "_" + toString(suffix);
|
2016-11-23 20:41:39 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
ASTRenameQuery::Element elem;
|
|
|
|
elem.from = from;
|
|
|
|
elem.to = to;
|
2016-11-23 20:41:39 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
rename->elements.emplace_back(elem);
|
2016-11-23 20:41:39 +00:00
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
LOG_DEBUG(
|
|
|
|
log,
|
|
|
|
"Existing table {} for system log has obsolete or different structure. Renaming it to {}",
|
|
|
|
description,
|
|
|
|
backQuoteIfNeed(to.table));
|
2016-11-23 20:41:39 +00:00
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
auto query_context = Context::createCopy(context);
|
|
|
|
query_context->makeQueryContext();
|
2020-11-20 16:06:27 +00:00
|
|
|
InterpreterRenameQuery(rename, query_context).execute();
|
2016-11-23 20:41:39 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// The required table will be created.
|
2017-04-01 07:20:54 +00:00
|
|
|
table = nullptr;
|
|
|
|
}
|
2018-03-29 13:57:16 +00:00
|
|
|
else if (!is_prepared)
|
2020-05-23 22:24:01 +00:00
|
|
|
LOG_DEBUG(log, "Will use existing table {} for {}", description, LogElement::name());
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2016-11-23 20:41:39 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
if (!table)
|
|
|
|
{
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Create the table.
|
2020-05-23 22:24:01 +00:00
|
|
|
LOG_DEBUG(log, "Creating new table {} for {}", description, LogElement::name());
|
2016-11-23 20:41:39 +00:00
|
|
|
|
2020-04-13 01:33:05 +00:00
|
|
|
auto create = getCreateTableQuery();
|
2016-11-23 20:41:39 +00:00
|
|
|
|
2020-11-20 16:06:27 +00:00
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
auto query_context = Context::createCopy(context);
|
|
|
|
query_context->makeQueryContext();
|
|
|
|
|
2020-11-20 16:06:27 +00:00
|
|
|
InterpreterCreateQuery interpreter(create, query_context);
|
2018-01-18 23:40:32 +00:00
|
|
|
interpreter.setInternal(true);
|
|
|
|
interpreter.execute();
|
2016-11-23 20:41:39 +00:00
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
table = DatabaseCatalog::instance().getTable(table_id, getContext());
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2016-11-23 20:41:39 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
is_prepared = true;
|
2016-11-23 20:41:39 +00:00
|
|
|
}
|
|
|
|
|
2020-04-13 01:33:05 +00:00
|
|
|
|
|
|
|
template <typename LogElement>
|
|
|
|
ASTPtr SystemLog<LogElement>::getCreateTableQuery()
|
|
|
|
{
|
|
|
|
auto create = std::make_shared<ASTCreateQuery>();
|
|
|
|
|
|
|
|
create->database = table_id.database_name;
|
|
|
|
create->table = table_id.table_name;
|
|
|
|
|
|
|
|
Block sample = LogElement::createBlock();
|
|
|
|
|
|
|
|
auto new_columns_list = std::make_shared<ASTColumns>();
|
|
|
|
new_columns_list->set(new_columns_list->columns, InterpreterCreateQuery::formatColumns(sample.getNamesAndTypesList()));
|
|
|
|
create->set(create->columns_list, new_columns_list);
|
|
|
|
|
|
|
|
ParserStorage storage_parser;
|
|
|
|
ASTPtr storage_ast = parseQuery(
|
|
|
|
storage_parser, storage_def.data(), storage_def.data() + storage_def.size(),
|
2020-04-15 20:28:05 +00:00
|
|
|
"Storage to create table for " + LogElement::name(), 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
|
2020-04-13 01:33:05 +00:00
|
|
|
create->set(create->storage, storage_ast);
|
|
|
|
|
|
|
|
return create;
|
|
|
|
}
|
|
|
|
|
2016-10-27 17:48:12 +00:00
|
|
|
}
|