From 89d2270e1d4afd18267d8556a24e4874d4ad8817 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 25 Nov 2021 21:06:04 +0300 Subject: [PATCH 1/2] Add ability to reset any type of AST --- src/Parsers/IAST.cpp | 18 ------------------ src/Parsers/IAST.h | 18 +++++++++++++++++- 2 files changed, 17 insertions(+), 19 deletions(-) diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index 9a8838956f9..3a21d704eb9 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -14,7 +14,6 @@ namespace ErrorCodes extern const int TOO_DEEP_AST; extern const int BAD_ARGUMENTS; extern const int UNKNOWN_ELEMENT_IN_AST; - extern const int LOGICAL_ERROR; } @@ -48,23 +47,6 @@ size_t IAST::checkSize(size_t max_size) const return res; } -void IAST::reset(IAST *& field) -{ - if (field == nullptr) - return; - - const auto child = std::find_if(children.begin(), children.end(), [field](const auto & p) - { - return p.get() == field; - }); - - if (child == children.end()) - throw Exception("AST subtree not found in children", ErrorCodes::LOGICAL_ERROR); - - children.erase(child); - field = nullptr; -} - IAST::Hash IAST::getTreeHash() const { diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index 1c9c81ad378..2f30a1f7bee 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -157,7 +157,23 @@ public: set(field, child); } - void reset(IAST *& field); + template + void reset(T * & field) + { + if (field == nullptr) + return; + + const auto child = std::find_if(children.begin(), children.end(), [field](const auto & p) + { + return p.get() == field; + }); + + if (child == children.end()) + throw Exception("AST subtree not found in children", ErrorCodes::LOGICAL_ERROR); + + children.erase(child); + field = nullptr; + } /// Convert to a string. From 5fafeea76330db8037f2beddd8a4386cad54505a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 25 Nov 2021 21:06:04 +0300 Subject: [PATCH 2/2] Recreate system.*_log tables in case of different engine/partition_by. - Mark getCreateTableQuery() non virtual to call it from ctor - Make check for CREATE TABLE simpler (by caching result) - Extend error message. v2: at SystemLog tables has not been initialized yet, so SHOW CREATE cannot be obtained for the existing table, move the check to the prepareTable() call. v3: reset UUID and SETTINGS of MergeTree in CREATE TABLE query v4: fix re-creating table with the same UUID --- src/Interpreters/SystemLog.cpp | 16 +++++ src/Interpreters/SystemLog.h | 41 +++++++---- .../test_system_logs_recreate/__init__.py | 0 .../test_system_logs_recreate/test.py | 70 +++++++++++++++++++ 4 files changed, 112 insertions(+), 15 deletions(-) create mode 100644 tests/integration/test_system_logs_recreate/__init__.py create mode 100644 tests/integration/test_system_logs_recreate/test.py diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 71155760aea..fc2a5b620e2 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -96,6 +96,22 @@ std::shared_ptr createSystemLog( } +ASTPtr ISystemLog::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; + /// Existing table has default settings (i.e. `index_granularity = 8192`), reset them. + if (ASTStorage * storage = old_create_query_ast.storage) + { + storage->reset(storage->settings); + } + return old_ast; +} + + SystemLogs::SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConfiguration & config) { query_log = createSystemLog(global_context, "system", "query_log", config, "query_log"); diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index ee3e3062e13..d6342e3973e 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -61,6 +61,7 @@ namespace DB namespace ErrorCodes { extern const int TIMEOUT_EXCEEDED; + extern const int LOGICAL_ERROR; } #define DBMS_SYSTEM_LOG_QUEUE_SIZE 1048576 @@ -83,13 +84,18 @@ class ISystemLog { public: virtual String getName() = 0; - virtual ASTPtr getCreateTableQuery() = 0; //// force -- force table creation (used for SYSTEM FLUSH LOGS) virtual void flush(bool force = false) = 0; virtual void prepareTable() = 0; virtual void startup() = 0; virtual void shutdown() = 0; virtual ~ISystemLog() = default; + + /// returns CREATE TABLE query, but with removed: + /// - UUID + /// - SETTINGS (for MergeTree) + /// That way it can be used to compare with the SystemLog::getCreateTableQuery() + static ASTPtr getCreateTableQueryClean(const StorageID & table_id, ContextPtr context); }; @@ -171,7 +177,7 @@ public: return LogElement::name(); } - ASTPtr getCreateTableQuery() override; + ASTPtr getCreateTableQuery(); protected: Poco::Logger * log; @@ -181,6 +187,8 @@ private: const StorageID table_id; const String storage_def; StoragePtr table; + String create_query; + String old_create_query; bool is_prepared = false; const size_t flush_interval_milliseconds; ThreadFromGlobalPool saving_thread; @@ -228,6 +236,7 @@ SystemLog::SystemLog( : 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); @@ -520,14 +529,14 @@ void SystemLog::prepareTable() if (table) { - auto metadata_columns = table->getInMemoryMetadataPtr()->getColumns(); - auto old_query = InterpreterCreateQuery::formatColumns(metadata_columns); + 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)); + } - auto ordinary_columns = LogElement::getNamesAndTypes(); - auto alias_columns = LogElement::getNamesAndAliases(); - auto current_query = InterpreterCreateQuery::formatColumns(ordinary_columns, alias_columns); - - if (serializeAST(*old_query) != serializeAST(*current_query)) + if (old_create_query != create_query) { /// Rename the existing table. int suffix = 0; @@ -553,9 +562,11 @@ void SystemLog::prepareTable() LOG_DEBUG( log, - "Existing table {} for system log has obsolete or different structure. Renaming it to {}", + "Existing table {} for system log has obsolete or different structure. Renaming it to {}.\nOld: {}\nNew: {}\n.", description, - backQuoteIfNeed(to.table)); + backQuoteIfNeed(to.table), + old_create_query, + create_query); auto query_context = Context::createCopy(context); query_context->makeQueryContext(); @@ -573,17 +584,17 @@ void SystemLog::prepareTable() /// Create the table. LOG_DEBUG(log, "Creating new table {} for {}", description, LogElement::name()); - auto create = getCreateTableQuery(); - - auto query_context = Context::createCopy(context); query_context->makeQueryContext(); - InterpreterCreateQuery interpreter(create, query_context); + 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; diff --git a/tests/integration/test_system_logs_recreate/__init__.py b/tests/integration/test_system_logs_recreate/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_system_logs_recreate/test.py b/tests/integration/test_system_logs_recreate/test.py new file mode 100644 index 00000000000..3ab0269b42e --- /dev/null +++ b/tests/integration/test_system_logs_recreate/test.py @@ -0,0 +1,70 @@ +# pylint: disable=line-too-long +# pylint: disable=unused-argument +# pylint: disable=redefined-outer-name + +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance('node_default', stay_alive=True) + +@pytest.fixture(scope='module', autouse=True) +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_system_logs_recreate(): + system_logs = [ + # enabled by default + 'query_log', + 'query_thread_log', + 'part_log', + 'trace_log', + 'metric_log', + ] + + node.query('SYSTEM FLUSH LOGS') + for table in system_logs: + assert 'ENGINE = MergeTree' in node.query(f'SHOW CREATE TABLE system.{table}') + assert 'ENGINE = Null' not in node.query(f'SHOW CREATE TABLE system.{table}') + assert len(node.query(f"SHOW TABLES FROM system LIKE '{table}%'").strip().split('\n')) == 1 + + # NOTE: we use zzz- prefix to make it the last file, + # so that it will be applied last. + for table in system_logs: + node.exec_in_container(['bash', '-c', f"""echo " + + <{table}> + ENGINE = Null + + + + " > /etc/clickhouse-server/config.d/zzz-override-{table}.xml + """]) + + node.restart_clickhouse() + node.query('SYSTEM FLUSH LOGS') + for table in system_logs: + assert 'ENGINE = MergeTree' not in node.query(f'SHOW CREATE TABLE system.{table}') + assert 'ENGINE = Null' in node.query(f'SHOW CREATE TABLE system.{table}') + assert len(node.query(f"SHOW TABLES FROM system LIKE '{table}%'").strip().split('\n')) == 2 + + for table in system_logs: + node.exec_in_container(['rm', f'/etc/clickhouse-server/config.d/zzz-override-{table}.xml']) + + node.restart_clickhouse() + node.query('SYSTEM FLUSH LOGS') + for table in system_logs: + assert 'ENGINE = MergeTree' in node.query(f'SHOW CREATE TABLE system.{table}') + assert 'ENGINE = Null' not in node.query(f'SHOW CREATE TABLE system.{table}') + assert len(node.query(f"SHOW TABLES FROM system LIKE '{table}%'").strip().split('\n')) == 3 + + node.query('SYSTEM FLUSH LOGS') + # Ensure that there was no superfluous RENAME's + # IOW that the table created only when the structure is indeed different. + for table in system_logs: + assert len(node.query(f"SHOW TABLES FROM system LIKE '{table}%'").strip().split('\n')) == 3