dbms: initial implementation of QueryLog [#METR-16946].

This commit is contained in:
Alexey Milovidov 2015-06-26 23:48:10 +03:00
parent 04f6d1eb60
commit c36fdf0e56
11 changed files with 107 additions and 27 deletions

View File

@ -78,5 +78,7 @@
/// Граница, на которых должны быть выровнены блоки для асинхронных файловых операций.
#define DEFAULT_AIO_FILE_BLOCK_SIZE 4096
#define DEFAULT_QUERY_LOG_FLUSH_INTERVAL_MILLISECONDS_STR "10000"
#define ALWAYS_INLINE __attribute__((__always_inline__))
#define NO_INLINE __attribute__((__noinline__))

View File

@ -40,6 +40,7 @@ class ProcessListElement;
class Macros;
class Progress;
class Clusters;
class QueryLog;
/// имя таблицы -> таблица
@ -64,12 +65,30 @@ typedef std::vector<DatabaseAndTableName> Dependencies;
*/
class Context
{
public:
enum class Interface
{
TCP = 1,
HTTP = 2,
OLAP_HTTP = 3,
};
enum class HTTPMethod
{
UNKNOWN = 0,
GET = 1,
POST = 2,
};
private:
typedef std::shared_ptr<ContextShared> Shared;
Shared shared;
String user; /// Текущий пользователь.
Poco::Net::IPAddress ip_address; /// IP-адрес, с которого задан запрос.
Interface interface = Interface::TCP;
HTTPMethod http_method = HTTPMethod::UNKNOWN; /// NOTE Возможно, перенести это в отдельный struct ClientInfo.
std::shared_ptr<QuotaForIntervals> quota; /// Текущая квота. По-умолчанию - пустая квота, которая ничего не ограничивает.
String current_database; /// Текущая БД.
String current_query_id; /// Id текущего запроса.
@ -108,6 +127,12 @@ public:
String getUser() const { return user; }
Poco::Net::IPAddress getIPAddress() const { return ip_address; }
Interface getInterface() const { return interface; }
void setInterface(Interface interface_) { interface = interface_; }
HTTPMethod getHTTPMethod() const { return http_method; }
void setHTTPMethod(HTTPMethod http_method_) { http_method = http_method_; }
void setQuota(const String & name, const String & quota_key, const String & user_name, const Poco::Net::IPAddress & address);
QuotaForIntervals & getQuota();
@ -242,6 +267,8 @@ public:
Compiler & getCompiler();
QueryLog & getQueryLog();
/// Позволяет выбрать метод сжатия по условиям, описанным в конфигурационном файле.
CompressionMethod chooseCompressionMethod(size_t part_size, double part_size_ratio) const;

View File

@ -6,6 +6,7 @@
#include <DB/Core/Types.h>
#include <DB/Common/ConcurrentBoundedQueue.h>
#include <DB/Storages/IStorage.h>
#include <DB/Interpreters/Context.h>
namespace DB
@ -35,38 +36,24 @@ struct QueryLogElement
QUERY_FINISH = 2,
};
enum Interface
{
TCP = 1,
HTTP = 2,
OLAP_HTTP = 3,
};
enum HTTPMethod
{
UNKNOWN = 0,
GET = 1,
POST = 2,
};
Type type;
Type type = QUERY_START;
/// В зависимости от типа, не все поля могут быть заполнены.
time_t event_time;
time_t query_start_time;
UInt64 query_duration_ms;
time_t event_time{};
time_t query_start_time{};
UInt64 query_duration_ms{};
UInt64 read_rows;
UInt64 read_bytes;
UInt64 read_rows{};
UInt64 read_bytes{};
UInt64 result_rows;
UInt64 result_bytes;
UInt64 result_rows{};
UInt64 result_bytes{};
String query;
Interface interface;
HTTPMethod http_method;
Context::Interface interface = Context::Interface::TCP;
Context::HTTPMethod http_method = Context::HTTPMethod::UNKNOWN;
Poco::Net::IPAddress ip_address;
String user;
String query_id;

View File

@ -135,6 +135,9 @@ struct Settings
\
/** Приоритет запроса. 1 - самый высокий, больше - ниже; 0 - не использовать приоритеты. */ \
M(SettingUInt64, priority, 0) \
\
/** Логгировать запросы и писать лог в системную таблицу. */ \
M(SettingBool, log_queries, 0) \
/// Всевозможные ограничения на выполнение запроса.
Limits limits;

View File

@ -27,6 +27,7 @@
#include <DB/Interpreters/Cluster.h>
#include <DB/Interpreters/InterserverIOHandler.h>
#include <DB/Interpreters/Compiler.h>
#include <DB/Interpreters/QueryLog.h>
#include <DB/Interpreters/Context.h>
#include <DB/IO/ReadBufferFromFile.h>
#include <DB/IO/WriteBufferFromString.h>
@ -86,6 +87,7 @@ struct ContextShared
BackgroundProcessingPoolPtr background_pool; /// Пул потоков для фоновой работы, выполняемой таблицами.
Macros macros; /// Подстановки из конфига.
std::unique_ptr<Compiler> compiler; /// Для динамической компиляции частей запроса, при необходимости.
std::unique_ptr<QueryLog> query_log; /// Для логгирования запросов.
mutable std::unique_ptr<CompressionMethodSelector> compression_method_selector; /// Правила для выбора метода сжатия в зависимости от размера куска.
/// Кластеры для distributed таблиц
@ -830,6 +832,26 @@ Compiler & Context::getCompiler()
}
QueryLog & Context::getQueryLog()
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
if (!shared->query_log)
{
auto & config = Poco::Util::Application::instance().config();
String database = config.getString("query_log.database", "system");
String table = config.getString("query_log.table", "query_log");
size_t flush_interval_milliseconds = parse<size_t>(
config.getString("query_log.flush_interval_milliseconds", DEFAULT_QUERY_LOG_FLUSH_INTERVAL_MILLISECONDS_STR));
shared->query_log.reset(new QueryLog{ *this, database, table, flush_interval_milliseconds });
}
return *shared->query_log;
}
CompressionMethod Context::chooseCompressionMethod(size_t part_size, double part_size_ratio) const
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);

View File

@ -12,7 +12,6 @@
#include <DB/DataTypes/DataTypeDate.h>
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataTypes/DataTypeFixedString.h>
#include <DB/Interpreters/Context.h>
#include <DB/Interpreters/InterpreterCreateQuery.h>
#include <DB/Interpreters/InterpreterRenameQuery.h>
#include <DB/Interpreters/QueryLog.h>
@ -263,6 +262,7 @@ void QueryLog::flush()
tryLogCurrentException(__PRETTY_FUNCTION__);
}
/// В случае ошибки тоже очищаем накопленные записи, чтобы не блокироваться.
data.clear();
}

View File

@ -15,6 +15,7 @@
#include <DB/Interpreters/Quota.h>
#include <DB/Interpreters/InterpreterFactory.h>
#include <DB/Interpreters/ProcessList.h>
#include <DB/Interpreters/QueryLog.h>
#include <DB/Interpreters/executeQuery.h>
@ -31,6 +32,7 @@ static void checkLimits(const IAST & ast, const Limits & limits)
}
/// Логгировать запрос в обычный лог (не в таблицу).
static void logQuery(const String & query, const Context & context)
{
String logged_query = query;
@ -85,12 +87,12 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
quota.checkExceeded(current_time);
const Settings & settings = context.getSettingsRef();
/// Положим запрос в список процессов. Но запрос SHOW PROCESSLIST класть не будем.
ProcessList::EntryPtr process_list_entry;
if (!internal && nullptr == typeid_cast<const ASTShowProcesslistQuery *>(&*ast))
{
const Settings & settings = context.getSettingsRef();
process_list_entry = context.getProcessList().insert(
query, context.getUser(), context.getCurrentQueryId(), context.getIPAddress(),
settings.limits.max_memory_usage,
@ -101,6 +103,27 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
context.setProcessListElement(&process_list_entry->get());
}
/// Логгируем в таблицу начало выполнения запроса, если нужно.
if (settings.log_queries)
{
QueryLogElement elem;
elem.type = QueryLogElement::QUERY_START;
elem.event_time = current_time;
elem.query_start_time = current_time;
elem.query = query;
elem.interface = context.getInterface();
elem.http_method = context.getHTTPMethod();
elem.ip_address = context.getIPAddress();
elem.user = context.getUser();
elem.query_id = context.getCurrentQueryId();
context.getQueryLog().add(elem);
}
BlockIO res;
try

View File

@ -136,6 +136,16 @@ void HTTPHandler::processQuery(Poco::Net::HTTPServerRequest & request, Poco::Net
if (readonly)
context.getSettingsRef().limits.readonly = true;
context.setInterface(Context::Interface::HTTP);
Context::HTTPMethod http_method = Context::HTTPMethod::UNKNOWN;
if (request.getMethod() == Poco::Net::HTTPServerRequest::HTTP_GET)
http_method = Context::HTTPMethod::GET;
else if (request.getMethod() == Poco::Net::HTTPServerRequest::HTTP_POST)
http_method = Context::HTTPMethod::POST;
context.setHTTPMethod(http_method);
Stopwatch watch;
executeQuery(*in, *used_output.out_maybe_compressed, context, query_plan);
watch.stop();

View File

@ -52,6 +52,9 @@ void OLAPHTTPHandler::processQuery(Poco::Net::HTTPServerRequest & request, Poco:
context.setUser(user, password, request.clientAddress().host(), quota_key);
context.setInterface(Context::Interface::HTTP);
context.setHTTPMethod(Context::HTTPMethod::POST);
OLAP::QueryParseResult olap_query = server.olap_parser->parse(request_istream);
std::string clickhouse_query;

View File

@ -533,6 +533,8 @@ int Server::main(const std::vector<std::string> & args)
LOG_DEBUG(log, "Loaded metadata.");
/// Создаём системные таблицы.
Poco::File(path + "data/system").createDirectories();
Poco::File(path + "metadata/system").createDirectories();
global_context->addDatabase("system");
global_context->addTable("system", "one", StorageSystemOne::create("one"));

View File

@ -111,6 +111,7 @@ void TCPHandler::runImpl()
{
/// Восстанавливаем контекст запроса.
query_context = connection_context;
query_context.setInterface(Context::Interface::TCP);
/** Если Query - обрабатываем. Если Ping или Cancel - возвращаемся в начало.
* Могут прийти настройки на отдельный запрос, которые модифицируют query_context.