#include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include namespace DB { namespace ErrorCodes { extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; extern const int NOT_IMPLEMENTED; } namespace { class StorageWithComment : public IAST { public: ASTPtr storage; ASTPtr comment; String getID(char) const override { return "Storage with comment definition"; } ASTPtr clone() const override { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method clone is not supported"); } void formatImpl(const FormatSettings &, FormatState &, FormatStateStacked) const override { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method formatImpl is not supported"); } }; class ParserStorageWithComment : public IParserBase { protected: const char * getName() const override { return "storage definition with comment"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override { ParserStorage storage_p; ASTPtr storage; if (!storage_p.parse(pos, storage, expected)) return false; ParserKeyword s_comment("COMMENT"); ParserStringLiteral string_literal_parser; ASTPtr comment; if (s_comment.ignore(pos, expected)) string_literal_parser.parse(pos, comment, expected); auto storage_with_comment = std::make_shared(); storage_with_comment->storage = std::move(storage); storage_with_comment->comment = std::move(comment); node = storage_with_comment; return true; } }; } namespace { constexpr size_t DEFAULT_SYSTEM_LOG_FLUSH_INTERVAL_MILLISECONDS = 7500; constexpr size_t DEFAULT_METRIC_LOG_COLLECT_INTERVAL_MILLISECONDS = 1000; /// Creates a system log with MergeTree engine using parameters from config template std::shared_ptr createSystemLog( ContextPtr context, const String & default_database_name, const String & default_table_name, const Poco::Util::AbstractConfiguration & config, const String & config_prefix) { if (!config.has(config_prefix)) { LOG_DEBUG(&Poco::Logger::get("SystemLog"), "Not creating {}.{} since corresponding section '{}' is missing from config", default_database_name, default_table_name, config_prefix); return {}; } String database = config.getString(config_prefix + ".database", default_database_name); String table = config.getString(config_prefix + ".table", default_table_name); if (database != default_database_name) { /// System tables must be loaded before other tables, but loading order is undefined for all databases except `system` LOG_ERROR(&Poco::Logger::get("SystemLog"), "Custom database name for a system table specified in config." " Table `{}` will be created in `system` database instead of `{}`", table, database); database = default_database_name; } String engine; if (config.has(config_prefix + ".engine")) { if (config.has(config_prefix + ".partition_by")) throw Exception("If 'engine' is specified for system table, " "PARTITION BY parameters should be specified directly inside 'engine' and 'partition_by' setting doesn't make sense", ErrorCodes::BAD_ARGUMENTS); if (config.has(config_prefix + ".ttl")) throw Exception("If 'engine' is specified for system table, " "TTL parameters should be specified directly inside 'engine' and 'ttl' setting doesn't make sense", ErrorCodes::BAD_ARGUMENTS); engine = config.getString(config_prefix + ".engine"); } else { String partition_by = config.getString(config_prefix + ".partition_by", "toYYYYMM(event_date)"); engine = "ENGINE = MergeTree"; if (!partition_by.empty()) engine += " PARTITION BY (" + partition_by + ")"; String ttl = config.getString(config_prefix + ".ttl", ""); if (!ttl.empty()) engine += " TTL " + ttl; engine += " ORDER BY "; engine += TSystemLog::getDefaultOrderBy(); } /// Validate engine definition syntax to prevent some configuration errors. ParserStorageWithComment storage_parser; parseQuery(storage_parser, engine.data(), engine.data() + engine.size(), "Storage to create table for " + config_prefix, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); size_t flush_interval_milliseconds = config.getUInt64(config_prefix + ".flush_interval_milliseconds", DEFAULT_SYSTEM_LOG_FLUSH_INTERVAL_MILLISECONDS); return std::make_shared(context, database, table, engine, flush_interval_milliseconds); } /// returns CREATE TABLE query, but with removed UUID /// That way it can be used to compare with the SystemLog::getCreateTableQuery() ASTPtr getCreateTableQueryClean(const StorageID & table_id, ContextPtr context) { DatabasePtr database = DatabaseCatalog::instance().getDatabase(table_id.database_name); ASTPtr old_ast = database->getCreateTableQuery(table_id.table_name, context); auto & old_create_query_ast = old_ast->as(); /// Reset UUID old_create_query_ast.uuid = UUIDHelpers::Nil; return old_ast; } } SystemLogs::SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConfiguration & config) { query_log = createSystemLog(global_context, "system", "query_log", config, "query_log"); query_thread_log = createSystemLog(global_context, "system", "query_thread_log", config, "query_thread_log"); part_log = createSystemLog(global_context, "system", "part_log", config, "part_log"); trace_log = createSystemLog(global_context, "system", "trace_log", config, "trace_log"); crash_log = createSystemLog(global_context, "system", "crash_log", config, "crash_log"); text_log = createSystemLog(global_context, "system", "text_log", config, "text_log"); metric_log = createSystemLog(global_context, "system", "metric_log", config, "metric_log"); cache_log = createSystemLog(global_context, "system", "filesystem_cache_log", config, "filesystem_cache_log"); asynchronous_metric_log = createSystemLog( global_context, "system", "asynchronous_metric_log", config, "asynchronous_metric_log"); opentelemetry_span_log = createSystemLog( global_context, "system", "opentelemetry_span_log", config, "opentelemetry_span_log"); query_views_log = createSystemLog(global_context, "system", "query_views_log", config, "query_views_log"); zookeeper_log = createSystemLog(global_context, "system", "zookeeper_log", config, "zookeeper_log"); session_log = createSystemLog(global_context, "system", "session_log", config, "session_log"); transactions_info_log = createSystemLog( global_context, "system", "transactions_info_log", config, "transactions_info_log"); processors_profile_log = createSystemLog(global_context, "system", "processors_profile_log", config, "processors_profile_log"); if (query_log) logs.emplace_back(query_log.get()); if (query_thread_log) logs.emplace_back(query_thread_log.get()); if (part_log) logs.emplace_back(part_log.get()); if (trace_log) logs.emplace_back(trace_log.get()); if (crash_log) logs.emplace_back(crash_log.get()); if (text_log) logs.emplace_back(text_log.get()); if (metric_log) logs.emplace_back(metric_log.get()); if (asynchronous_metric_log) logs.emplace_back(asynchronous_metric_log.get()); if (opentelemetry_span_log) logs.emplace_back(opentelemetry_span_log.get()); if (query_views_log) logs.emplace_back(query_views_log.get()); if (zookeeper_log) logs.emplace_back(zookeeper_log.get()); if (session_log) { logs.emplace_back(session_log.get()); global_context->addWarningMessage("Table system.session_log is enabled. It's unreliable and may contain garbage. Do not use it for any kind of security monitoring."); } if (transactions_info_log) logs.emplace_back(transactions_info_log.get()); if (processors_profile_log) logs.emplace_back(processors_profile_log.get()); if (cache_log) logs.emplace_back(cache_log.get()); try { for (auto & log : logs) log->startup(); } catch (...) { /// join threads shutdown(); throw; } if (metric_log) { size_t collect_interval_milliseconds = config.getUInt64("metric_log.collect_interval_milliseconds", DEFAULT_METRIC_LOG_COLLECT_INTERVAL_MILLISECONDS); metric_log->startCollectMetric(collect_interval_milliseconds); } if (crash_log) { CrashLog::initialize(crash_log); } } SystemLogs::~SystemLogs() { shutdown(); } void SystemLogs::shutdown() { for (auto & log : logs) log->shutdown(); } template SystemLog::SystemLog( ContextPtr context_, const String & database_name_, const String & table_name_, const String & storage_def_, size_t flush_interval_milliseconds_) : WithContext(context_) , table_id(database_name_, table_name_) , storage_def(storage_def_) , create_query(serializeAST(*getCreateTableQuery())) , flush_interval_milliseconds(flush_interval_milliseconds_) { assert(database_name_ == DatabaseCatalog::SYSTEM_DATABASE); log = &Poco::Logger::get("SystemLog (" + database_name_ + "." + table_name_ + ")"); } template void SystemLog::shutdown() { stopFlushThread(); auto table = DatabaseCatalog::instance().tryGetTable(table_id, getContext()); if (table) table->flushAndShutdown(); } template void SystemLog::savingThreadFunction() { setThreadName("SystemLogFlush"); std::vector to_flush; bool exit_this_thread = false; while (!exit_this_thread) { try { // The end index (exclusive, like std end()) of the messages we are // going to flush. uint64_t to_flush_end = 0; // Should we prepare table even if there are no new messages. bool should_prepare_tables_anyway = false; { std::unique_lock lock(mutex); flush_event.wait_for(lock, std::chrono::milliseconds(flush_interval_milliseconds), [&] () { return requested_flush_up_to > flushed_up_to || is_shutdown || is_force_prepare_tables; } ); queue_front_index += queue.size(); to_flush_end = queue_front_index; // Swap with existing array from previous flush, to save memory // allocations. to_flush.resize(0); queue.swap(to_flush); should_prepare_tables_anyway = is_force_prepare_tables; exit_this_thread = is_shutdown; } if (to_flush.empty()) { if (should_prepare_tables_anyway) { 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); } } catch (...) { tryLogCurrentException(__PRETTY_FUNCTION__); } } LOG_TRACE(log, "Terminating"); } template void SystemLog::flushImpl(const std::vector & to_flush, uint64_t to_flush_end) { try { LOG_TRACE(log, "Flushing system log, {} entries to flush up to offset {}", to_flush.size(), to_flush_end); /// 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. prepareTable(); ColumnsWithTypeAndName log_element_columns; auto log_element_names_and_types = LogElement::getNamesAndTypes(); for (const auto & name_and_type : log_element_names_and_types) log_element_columns.emplace_back(name_and_type.type, name_and_type.name); Block block(std::move(log_element_columns)); MutableColumns columns = block.mutateColumns(); for (const auto & elem : to_flush) elem.appendToBlock(columns); block.setColumns(std::move(columns)); /// We write to table indirectly, using InterpreterInsertQuery. /// This is needed to support DEFAULT-columns in table. std::unique_ptr insert = std::make_unique(); insert->table_id = table_id; ASTPtr query_ptr(insert.release()); // we need query context to do inserts to target table with MV containing subqueries or joins auto insert_context = Context::createCopy(context); insert_context->makeQueryContext(); InterpreterInsertQuery interpreter(query_ptr, insert_context); BlockIO io = interpreter.execute(); PushingPipelineExecutor executor(io.pipeline); executor.start(); executor.push(block); executor.finish(); } catch (...) { tryLogCurrentException(__PRETTY_FUNCTION__); } { std::lock_guard lock(mutex); flushed_up_to = to_flush_end; is_force_prepare_tables = false; flush_event.notify_all(); } LOG_TRACE(log, "Flushed system log up to offset {}", to_flush_end); } template void SystemLog::prepareTable() { String description = table_id.getNameForLogs(); auto table = DatabaseCatalog::instance().tryGetTable(table_id, getContext()); if (table) { if (old_create_query.empty()) { old_create_query = serializeAST(*getCreateTableQueryClean(table_id, getContext())); if (old_create_query.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty CREATE QUERY for {}", backQuoteIfNeed(table_id.table_name)); } if (old_create_query != create_query) { /// Rename the existing table. int suffix = 0; while (DatabaseCatalog::instance().isTableExist( {table_id.database_name, table_id.table_name + "_" + toString(suffix)}, getContext())) ++suffix; auto rename = std::make_shared(); ASTRenameQuery::Table from; from.database = table_id.database_name; from.table = table_id.table_name; ASTRenameQuery::Table to; to.database = table_id.database_name; to.table = table_id.table_name + "_" + toString(suffix); ASTRenameQuery::Element elem; elem.from = from; elem.to = to; rename->elements.emplace_back(elem); LOG_DEBUG( log, "Existing table {} for system log has obsolete or different structure. Renaming it to {}.\nOld: {}\nNew: {}\n.", description, backQuoteIfNeed(to.table), old_create_query, create_query); auto query_context = Context::createCopy(context); query_context->makeQueryContext(); InterpreterRenameQuery(rename, query_context).execute(); /// The required table will be created. table = nullptr; } else if (!is_prepared) LOG_DEBUG(log, "Will use existing table {} for {}", description, LogElement::name()); } if (!table) { /// Create the table. LOG_DEBUG(log, "Creating new table {} for {}", description, LogElement::name()); auto query_context = Context::createCopy(context); query_context->makeQueryContext(); auto create_query_ast = getCreateTableQuery(); InterpreterCreateQuery interpreter(create_query_ast, query_context); interpreter.setInternal(true); interpreter.execute(); table = DatabaseCatalog::instance().getTable(table_id, getContext()); old_create_query.clear(); } is_prepared = true; } template ASTPtr SystemLog::getCreateTableQuery() { auto create = std::make_shared(); create->setDatabase(table_id.database_name); create->setTable(table_id.table_name); auto new_columns_list = std::make_shared(); if (const char * custom_column_list = LogElement::getCustomColumnList()) { ParserColumnDeclarationList parser; const Settings & settings = getContext()->getSettingsRef(); ASTPtr columns_list_raw = parseQuery(parser, custom_column_list, "columns declaration list", settings.max_query_size, settings.max_parser_depth); new_columns_list->set(new_columns_list->columns, columns_list_raw); } else { auto ordinary_columns = LogElement::getNamesAndTypes(); auto alias_columns = LogElement::getNamesAndAliases(); new_columns_list->set(new_columns_list->columns, InterpreterCreateQuery::formatColumns(ordinary_columns, alias_columns)); } create->set(create->columns_list, new_columns_list); ParserStorageWithComment storage_parser; ASTPtr storage_with_comment_ast = parseQuery( storage_parser, storage_def.data(), storage_def.data() + storage_def.size(), "Storage to create table for " + LogElement::name(), 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); StorageWithComment & storage_with_comment = storage_with_comment_ast->as(); create->set(create->storage, storage_with_comment.storage); create->set(create->comment, storage_with_comment.comment); /// Write additional (default) settings for MergeTree engine to make it make it possible to compare ASTs /// and recreate tables on settings changes. const auto & engine = create->storage->engine->as(); if (endsWith(engine.name, "MergeTree")) { auto storage_settings = std::make_unique(getContext()->getMergeTreeSettings()); storage_settings->loadFromQuery(*create->storage); } return create; } #define INSTANTIATE_SYSTEM_LOG(ELEMENT) template class SystemLog; SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG) }