2016-10-27 17:48:12 +00:00
|
|
|
#pragma once
|
|
|
|
|
|
|
|
#include <thread>
|
2019-03-21 19:22:38 +00:00
|
|
|
#include <atomic>
|
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>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Core/Types.h>
|
|
|
|
#include <Common/ConcurrentBoundedQueue.h>
|
|
|
|
#include <Storages/IStorage.h>
|
|
|
|
#include <Interpreters/Context.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>
|
|
|
|
#include <Parsers/ASTInsertQuery.h>
|
|
|
|
#include <Interpreters/InterpreterCreateQuery.h>
|
|
|
|
#include <Interpreters/InterpreterRenameQuery.h>
|
|
|
|
#include <Interpreters/InterpreterInsertQuery.h>
|
|
|
|
#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();
|
|
|
|
void appendToBlock(Block & block) const;
|
|
|
|
};
|
|
|
|
*/
|
2016-10-27 17:48:12 +00:00
|
|
|
|
|
|
|
|
2018-08-22 04:36:53 +00:00
|
|
|
#define DBMS_SYSTEM_LOG_QUEUE_SIZE 1048576
|
2016-10-27 17:48:12 +00:00
|
|
|
|
|
|
|
class Context;
|
2017-06-05 13:59:38 +00:00
|
|
|
class QueryLog;
|
2018-05-31 15:54:08 +00:00
|
|
|
class QueryThreadLog;
|
2017-06-05 13:59:38 +00:00
|
|
|
class PartLog;
|
2019-07-22 13:54:08 +00:00
|
|
|
class TextLog;
|
2019-02-03 21:30:45 +00:00
|
|
|
class TraceLog;
|
2019-08-13 14:31:46 +00:00
|
|
|
class MetricLog;
|
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
|
|
|
|
{
|
2018-07-08 04:54:37 +00:00
|
|
|
SystemLogs(Context & 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
|
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.
|
|
|
|
|
2018-07-08 04:54:37 +00:00
|
|
|
String part_log_database;
|
2017-06-05 13:59:38 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
|
2016-10-27 17:48:12 +00:00
|
|
|
template <typename LogElement>
|
|
|
|
class SystemLog : private boost::noncopyable
|
|
|
|
{
|
|
|
|
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(
|
|
|
|
Context & context_,
|
|
|
|
const String & database_name_,
|
|
|
|
const String & table_name_,
|
2017-09-17 18:49:43 +00:00
|
|
|
const String & storage_def_,
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t flush_interval_milliseconds_);
|
|
|
|
|
|
|
|
~SystemLog();
|
|
|
|
|
|
|
|
/** 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
|
|
|
|
2018-06-13 19:01:07 +00:00
|
|
|
/// Flush data in the buffer to disk
|
2019-06-19 12:28:34 +00:00
|
|
|
void flush();
|
2019-03-21 19:22:38 +00:00
|
|
|
|
|
|
|
/// Stop the background flush thread before destructor. No more data will be written.
|
|
|
|
void shutdown();
|
2018-06-13 19:01:07 +00:00
|
|
|
|
2018-01-23 22:56:46 +00:00
|
|
|
protected:
|
2017-04-01 07:20:54 +00:00
|
|
|
Context & context;
|
|
|
|
const String database_name;
|
|
|
|
const String table_name;
|
2017-09-17 18:49:43 +00:00
|
|
|
const String storage_def;
|
2017-04-01 07:20:54 +00:00
|
|
|
StoragePtr table;
|
|
|
|
const size_t flush_interval_milliseconds;
|
2019-03-21 19:22:38 +00:00
|
|
|
std::atomic<bool> is_shutdown{false};
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-06-19 16:50:17 +00:00
|
|
|
enum class EntryType
|
2019-06-19 12:28:34 +00:00
|
|
|
{
|
2019-06-19 16:50:17 +00:00
|
|
|
LOG_ELEMENT = 0,
|
|
|
|
AUTO_FLUSH,
|
|
|
|
FORCE_FLUSH,
|
2019-06-19 12:28:34 +00:00
|
|
|
SHUTDOWN,
|
|
|
|
};
|
|
|
|
|
2019-06-19 16:50:17 +00:00
|
|
|
using QueueItem = std::pair<EntryType, LogElement>;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
/// Queue is bounded. But its size is quite large to not block in all normal cases.
|
|
|
|
ConcurrentBoundedQueue<QueueItem> queue {DBMS_SYSTEM_LOG_QUEUE_SIZE};
|
|
|
|
|
|
|
|
/** Data that was pulled from queue. Data is accumulated here before enough time passed.
|
|
|
|
* It's possible to implement double-buffering, but we assume that insertion into table is faster
|
|
|
|
* than accumulation of large amount of log records (for example, for query log - processing of large amount of queries).
|
|
|
|
*/
|
|
|
|
std::vector<LogElement> data;
|
|
|
|
|
|
|
|
Logger * log;
|
|
|
|
|
|
|
|
/** In this thread, data is pulled from 'queue' and stored in 'data', and then written into table.
|
|
|
|
*/
|
2019-01-14 19:22:09 +00:00
|
|
|
ThreadFromGlobalPool saving_thread;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
void threadFunction();
|
|
|
|
|
|
|
|
/** 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.
|
|
|
|
*/
|
|
|
|
bool is_prepared = false;
|
|
|
|
void prepareTable();
|
2019-03-21 19:22:38 +00:00
|
|
|
|
2019-06-19 12:28:34 +00:00
|
|
|
std::mutex flush_mutex;
|
|
|
|
std::mutex condvar_mutex;
|
|
|
|
std::condition_variable flush_condvar;
|
|
|
|
bool force_flushing = false;
|
|
|
|
|
|
|
|
/// flushImpl can be executed only in saving_thread.
|
2019-06-19 16:50:17 +00:00
|
|
|
void flushImpl(EntryType reason);
|
2016-10-27 17:48:12 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
|
template <typename LogElement>
|
|
|
|
SystemLog<LogElement>::SystemLog(Context & 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_,
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t flush_interval_milliseconds_)
|
|
|
|
: context(context_),
|
2017-09-17 18:49:43 +00:00
|
|
|
database_name(database_name_), table_name(table_name_), storage_def(storage_def_),
|
2017-04-01 07:20:54 +00:00
|
|
|
flush_interval_milliseconds(flush_interval_milliseconds_)
|
2016-10-27 17:48:12 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
log = &Logger::get("SystemLog (" + database_name + "." + table_name + ")");
|
2019-08-13 14:32:32 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
data.reserve(DBMS_SYSTEM_LOG_QUEUE_SIZE);
|
2019-01-14 19:22:09 +00:00
|
|
|
saving_thread = ThreadFromGlobalPool([this] { threadFunction(); });
|
2016-10-27 17:48:12 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-06-19 12:28:34 +00:00
|
|
|
template <typename LogElement>
|
|
|
|
void SystemLog<LogElement>::add(const LogElement & element)
|
|
|
|
{
|
|
|
|
if (is_shutdown)
|
|
|
|
return;
|
|
|
|
|
|
|
|
/// Without try we could block here in case of queue overflow.
|
2019-06-19 16:50:17 +00:00
|
|
|
if (!queue.tryPush({EntryType::LOG_ELEMENT, element}))
|
2019-06-19 12:28:34 +00:00
|
|
|
LOG_ERROR(log, "SystemLog queue is full");
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
template <typename LogElement>
|
|
|
|
void SystemLog<LogElement>::flush()
|
|
|
|
{
|
|
|
|
if (is_shutdown)
|
|
|
|
return;
|
|
|
|
|
|
|
|
std::lock_guard flush_lock(flush_mutex);
|
2019-06-19 15:38:06 +00:00
|
|
|
force_flushing = true;
|
|
|
|
|
2019-06-19 12:28:34 +00:00
|
|
|
/// Tell thread to execute extra flush.
|
2019-06-19 16:50:17 +00:00
|
|
|
queue.push({EntryType::FORCE_FLUSH, {}});
|
2019-06-19 12:28:34 +00:00
|
|
|
|
|
|
|
/// Wait for flush being finished.
|
|
|
|
std::unique_lock lock(condvar_mutex);
|
|
|
|
while (force_flushing)
|
|
|
|
flush_condvar.wait(lock);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2016-10-27 17:48:12 +00:00
|
|
|
template <typename LogElement>
|
2019-03-21 19:22:38 +00:00
|
|
|
void SystemLog<LogElement>::shutdown()
|
2016-10-27 17:48:12 +00:00
|
|
|
{
|
2019-03-21 19:22:38 +00:00
|
|
|
bool old_val = false;
|
|
|
|
if (!is_shutdown.compare_exchange_strong(old_val, true))
|
|
|
|
return;
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Tell thread to shutdown.
|
2019-06-19 16:50:17 +00:00
|
|
|
queue.push({EntryType::SHUTDOWN, {}});
|
2017-04-01 07:20:54 +00:00
|
|
|
saving_thread.join();
|
2016-10-27 17:48:12 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-03-21 19:22:38 +00:00
|
|
|
template <typename LogElement>
|
|
|
|
SystemLog<LogElement>::~SystemLog()
|
|
|
|
{
|
|
|
|
shutdown();
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2016-10-27 17:48:12 +00:00
|
|
|
template <typename LogElement>
|
|
|
|
void SystemLog<LogElement>::threadFunction()
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
setThreadName("SystemLogFlush");
|
|
|
|
|
|
|
|
Stopwatch time_after_last_write;
|
|
|
|
bool first = true;
|
|
|
|
|
|
|
|
while (true)
|
|
|
|
{
|
|
|
|
try
|
|
|
|
{
|
|
|
|
if (first)
|
|
|
|
{
|
|
|
|
time_after_last_write.restart();
|
|
|
|
first = false;
|
|
|
|
}
|
|
|
|
|
|
|
|
QueueItem element;
|
|
|
|
bool has_element = false;
|
|
|
|
|
2018-06-19 20:30:35 +00:00
|
|
|
/// data.size() is increased only in this function
|
|
|
|
/// TODO: get rid of data and queue duality
|
|
|
|
|
2019-06-19 12:28:34 +00:00
|
|
|
if (data.empty())
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
queue.pop(element);
|
|
|
|
has_element = true;
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
size_t milliseconds_elapsed = time_after_last_write.elapsed() / 1000000;
|
|
|
|
if (milliseconds_elapsed < flush_interval_milliseconds)
|
|
|
|
has_element = queue.tryPop(element, flush_interval_milliseconds - milliseconds_elapsed);
|
|
|
|
}
|
|
|
|
|
|
|
|
if (has_element)
|
|
|
|
{
|
2019-06-19 16:50:17 +00:00
|
|
|
if (element.first == EntryType::SHUTDOWN)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-09-04 17:49:39 +00:00
|
|
|
/// NOTE: MergeTree engine can write data even it is already in shutdown state.
|
2019-06-19 16:50:17 +00:00
|
|
|
flushImpl(element.first);
|
2017-04-01 07:20:54 +00:00
|
|
|
break;
|
|
|
|
}
|
2019-06-19 16:50:17 +00:00
|
|
|
else if (element.first == EntryType::FORCE_FLUSH)
|
2018-06-19 20:30:35 +00:00
|
|
|
{
|
2019-06-19 16:50:17 +00:00
|
|
|
flushImpl(element.first);
|
2019-06-19 12:28:34 +00:00
|
|
|
time_after_last_write.restart();
|
|
|
|
continue;
|
2018-06-19 20:30:35 +00:00
|
|
|
}
|
2019-06-19 12:28:34 +00:00
|
|
|
else
|
|
|
|
data.push_back(element.second);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
size_t milliseconds_elapsed = time_after_last_write.elapsed() / 1000000;
|
|
|
|
if (milliseconds_elapsed >= flush_interval_milliseconds)
|
|
|
|
{
|
|
|
|
/// Write data to a table.
|
2019-06-19 16:50:17 +00:00
|
|
|
flushImpl(EntryType::AUTO_FLUSH);
|
2017-04-01 07:20:54 +00:00
|
|
|
time_after_last_write.restart();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
/// In case of exception we lost accumulated data - to avoid locking.
|
|
|
|
data.clear();
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
|
|
|
}
|
|
|
|
}
|
2016-10-27 17:48:12 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
template <typename LogElement>
|
2019-06-19 16:50:17 +00:00
|
|
|
void SystemLog<LogElement>::flushImpl(EntryType reason)
|
2016-10-27 17:48:12 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
try
|
|
|
|
{
|
2019-06-19 16:50:17 +00:00
|
|
|
if ((reason == EntryType::AUTO_FLUSH || reason == EntryType::SHUTDOWN) && data.empty())
|
2018-06-19 20:30:35 +00:00
|
|
|
return;
|
|
|
|
|
2017-04-27 05:58:16 +00:00
|
|
|
LOG_TRACE(log, "Flushing system log");
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-03-29 20:54:57 +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();
|
|
|
|
for (const LogElement & elem : data)
|
|
|
|
elem.appendToBlock(block);
|
2017-06-05 13:59:38 +00:00
|
|
|
|
2017-04-27 05:58:16 +00:00
|
|
|
/// Clear queue early, because insertion to the table could lead to generation of more log entrites
|
|
|
|
/// and pushing them to already full queue will lead to deadlock.
|
|
|
|
data.clear();
|
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>();
|
|
|
|
insert->database = database_name;
|
|
|
|
insert->table = table_name;
|
|
|
|
ASTPtr query_ptr(insert.release());
|
|
|
|
|
|
|
|
InterpreterInsertQuery interpreter(query_ptr, context);
|
|
|
|
BlockIO io = interpreter.execute();
|
|
|
|
|
|
|
|
io.out->writePrefix();
|
|
|
|
io.out->write(block);
|
|
|
|
io.out->writeSuffix();
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
2017-04-25 12:34:03 +00:00
|
|
|
/// In case of exception, also clean accumulated data - to avoid locking.
|
|
|
|
data.clear();
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2019-06-19 16:50:17 +00:00
|
|
|
if (reason == EntryType::FORCE_FLUSH)
|
2019-06-19 12:28:34 +00:00
|
|
|
{
|
|
|
|
std::lock_guard lock(condvar_mutex);
|
|
|
|
force_flushing = false;
|
|
|
|
flush_condvar.notify_one();
|
|
|
|
}
|
2016-10-27 17:48:12 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2016-11-23 20:41:39 +00:00
|
|
|
template <typename LogElement>
|
|
|
|
void SystemLog<LogElement>::prepareTable()
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
String description = backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(table_name);
|
2016-11-23 20:41:39 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
table = context.tryGetTable(database_name, table_name);
|
2016-11-23 20:41:39 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
if (table)
|
|
|
|
{
|
|
|
|
const Block expected = LogElement::createBlock();
|
|
|
|
const Block actual = table->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;
|
|
|
|
while (context.isTableExist(database_name, table_name + "_" + toString(suffix)))
|
|
|
|
++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;
|
|
|
|
from.database = database_name;
|
|
|
|
from.table = table_name;
|
2016-11-23 20:41:39 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
ASTRenameQuery::Table to;
|
|
|
|
to.database = database_name;
|
|
|
|
to.table = 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
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
LOG_DEBUG(log, "Existing table " << description << " for system log has obsolete or different structure."
|
|
|
|
" Renaming it to " << backQuoteIfNeed(to.table));
|
2016-11-23 20:41:39 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
InterpreterRenameQuery(rename, 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)
|
2017-04-01 07:20:54 +00:00
|
|
|
LOG_DEBUG(log, "Will use existing table " << description << " for " + LogElement::name());
|
|
|
|
}
|
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.
|
2017-04-01 07:20:54 +00:00
|
|
|
LOG_DEBUG(log, "Creating new table " << description << " for " + LogElement::name());
|
2016-11-23 20:41:39 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
auto create = std::make_shared<ASTCreateQuery>();
|
2016-11-23 20:41:39 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
create->database = database_name;
|
|
|
|
create->table = table_name;
|
2016-11-23 20:41:39 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
Block sample = LogElement::createBlock();
|
2019-02-05 14:50:25 +00:00
|
|
|
|
|
|
|
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);
|
2016-11-23 20:41:39 +00:00
|
|
|
|
2017-09-17 18:49:43 +00:00
|
|
|
ParserStorage storage_parser;
|
|
|
|
ASTPtr storage_ast = parseQuery(
|
|
|
|
storage_parser, storage_def.data(), storage_def.data() + storage_def.size(),
|
2018-04-16 15:11:13 +00:00
|
|
|
"Storage to create table for " + LogElement::name(), 0);
|
2017-09-17 18:49:43 +00:00
|
|
|
create->set(create->storage, storage_ast);
|
2016-11-23 20:41:39 +00:00
|
|
|
|
2018-01-18 23:40:32 +00:00
|
|
|
InterpreterCreateQuery interpreter(create, context);
|
|
|
|
interpreter.setInternal(true);
|
|
|
|
interpreter.execute();
|
2016-11-23 20:41:39 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
table = context.getTable(database_name, table_name);
|
|
|
|
}
|
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
|
|
|
}
|
|
|
|
|
2016-10-27 17:48:12 +00:00
|
|
|
}
|