mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
automatically convert system database to Atomic
This commit is contained in:
parent
0d304f7b8c
commit
dbf2763788
@ -284,6 +284,11 @@ then
|
||||
|
||||
rm -rf /var/lib/clickhouse/*
|
||||
|
||||
# Make BC check more funny by forcing Ordinary engine for system database
|
||||
# New version will try to convert it to Atomic on startup
|
||||
mkdir /var/lib/clickhouse/metadata
|
||||
echo "ATTACH DATABASE system ENGINE=Ordinary" > /var/lib/clickhouse/metadata/system.sql
|
||||
|
||||
# Install previous release packages
|
||||
install_packages previous_release_package_folder
|
||||
|
||||
|
@ -1488,6 +1488,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
/// We load temporary database first, because projections need it.
|
||||
database_catalog.initializeAndLoadTemporaryDatabase();
|
||||
loadMetadataSystem(global_context);
|
||||
maybeConvertOrdinaryDatabaseToAtomic(global_context, DatabaseCatalog::instance().getSystemDatabase());
|
||||
/// After attaching system databases we can initialize system log.
|
||||
global_context->initializeSystemLogs();
|
||||
global_context->setSystemZooKeeperLogAfterInitializationIfNeeded();
|
||||
|
@ -61,36 +61,19 @@ namespace ErrorCodes
|
||||
|
||||
DatabasePtr DatabaseFactory::get(const ASTCreateQuery & create, const String & metadata_path, ContextPtr context)
|
||||
{
|
||||
bool created = false;
|
||||
/// Creates store/xxx/ for Atomic
|
||||
fs::create_directories(fs::path(metadata_path).parent_path());
|
||||
|
||||
try
|
||||
{
|
||||
/// Creates store/xxx/ for Atomic
|
||||
fs::create_directories(fs::path(metadata_path).parent_path());
|
||||
DatabasePtr impl = getImpl(create, metadata_path, context);
|
||||
|
||||
/// Before 20.7 it's possible that .sql metadata file does not exist for some old database.
|
||||
/// In this case Ordinary database is created on server startup if the corresponding metadata directory exists.
|
||||
/// So we should remove metadata directory if database creation failed.
|
||||
/// TODO remove this code
|
||||
created = fs::create_directory(metadata_path);
|
||||
if (impl && context->hasQueryContext() && context->getSettingsRef().log_queries)
|
||||
context->getQueryContext()->addQueryFactoriesInfo(Context::QueryLogFactories::Database, impl->getEngineName());
|
||||
|
||||
DatabasePtr impl = getImpl(create, metadata_path, context);
|
||||
/// Attach database metadata
|
||||
if (impl && create.comment)
|
||||
impl->setDatabaseComment(create.comment->as<ASTLiteral>()->value.safeGet<String>());
|
||||
|
||||
if (impl && context->hasQueryContext() && context->getSettingsRef().log_queries)
|
||||
context->getQueryContext()->addQueryFactoriesInfo(Context::QueryLogFactories::Database, impl->getEngineName());
|
||||
|
||||
// Attach database metadata
|
||||
if (impl && create.comment)
|
||||
impl->setDatabaseComment(create.comment->as<ASTLiteral>()->value.safeGet<String>());
|
||||
|
||||
return impl;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
if (created && fs::exists(metadata_path))
|
||||
fs::remove_all(metadata_path);
|
||||
throw;
|
||||
}
|
||||
return impl;
|
||||
}
|
||||
|
||||
template <typename ValueType>
|
||||
|
@ -9,6 +9,7 @@
|
||||
#include <Interpreters/InterpreterCreateQuery.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/loadMetadata.h>
|
||||
#include <Interpreters/executeQuery.h>
|
||||
|
||||
#include <Databases/DatabaseOrdinary.h>
|
||||
#include <Databases/TablesLoader.h>
|
||||
@ -26,6 +27,11 @@ namespace fs = std::filesystem;
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
}
|
||||
|
||||
static void executeCreateQuery(
|
||||
const String & query,
|
||||
ContextMutablePtr context,
|
||||
@ -111,43 +117,33 @@ void loadMetadata(ContextMutablePtr context, const String & default_database_nam
|
||||
if (it->is_symlink())
|
||||
continue;
|
||||
|
||||
const auto current_file = it->path().filename().string();
|
||||
if (!it->is_directory())
|
||||
{
|
||||
/// TODO: DETACH DATABASE PERMANENTLY ?
|
||||
if (fs::path(current_file).extension() == ".sql")
|
||||
{
|
||||
String db_name = fs::path(current_file).stem();
|
||||
if (!isSystemOrInformationSchema(db_name))
|
||||
databases.emplace(unescapeForFileName(db_name), fs::path(path) / db_name);
|
||||
}
|
||||
|
||||
/// Temporary fails may be left from previous server runs.
|
||||
if (fs::path(current_file).extension() == ".tmp")
|
||||
{
|
||||
LOG_WARNING(log, "Removing temporary file {}", it->path().string());
|
||||
try
|
||||
{
|
||||
fs::remove(it->path());
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// It does not prevent server to startup.
|
||||
tryLogCurrentException(log);
|
||||
}
|
||||
}
|
||||
|
||||
if (it->is_directory())
|
||||
continue;
|
||||
|
||||
const auto current_file = it->path().filename().string();
|
||||
|
||||
/// TODO: DETACH DATABASE PERMANENTLY ?
|
||||
if (fs::path(current_file).extension() == ".sql")
|
||||
{
|
||||
String db_name = fs::path(current_file).stem();
|
||||
if (!isSystemOrInformationSchema(db_name))
|
||||
databases.emplace(unescapeForFileName(db_name), fs::path(path) / db_name);
|
||||
}
|
||||
|
||||
/// For '.svn', '.gitignore' directory and similar.
|
||||
if (current_file.at(0) == '.')
|
||||
continue;
|
||||
|
||||
if (isSystemOrInformationSchema(current_file))
|
||||
continue;
|
||||
|
||||
databases.emplace(unescapeForFileName(current_file), it->path().string());
|
||||
/// Temporary fails may be left from previous server runs.
|
||||
if (fs::path(current_file).extension() == ".tmp")
|
||||
{
|
||||
LOG_WARNING(log, "Removing temporary file {}", it->path().string());
|
||||
try
|
||||
{
|
||||
fs::remove(it->path());
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// It does not prevent server to startup.
|
||||
tryLogCurrentException(log);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// clickhouse-local creates DatabaseMemory as default database by itself
|
||||
@ -185,11 +181,18 @@ static void loadSystemDatabaseImpl(ContextMutablePtr context, const String & dat
|
||||
{
|
||||
String path = context->getPath() + "metadata/" + database_name;
|
||||
String metadata_file = path + ".sql";
|
||||
if (fs::exists(fs::path(path)) || fs::exists(fs::path(metadata_file)))
|
||||
if (fs::exists(fs::path(metadata_file)))
|
||||
{
|
||||
/// 'has_force_restore_data_flag' is true, to not fail on loading query_log table, if it is corrupted.
|
||||
loadDatabase(context, database_name, path, true);
|
||||
}
|
||||
else if (fs::exists(fs::path(path)))
|
||||
{
|
||||
chassert(database_name == "system");
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Data directory for {} database exists, but metadata file does not. "
|
||||
"Probably you are trying to upgrade from version older than 20.7. "
|
||||
"If so, you should upgrade through intermediate version.", database_name);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Initialize system database manually
|
||||
@ -201,6 +204,110 @@ static void loadSystemDatabaseImpl(ContextMutablePtr context, const String & dat
|
||||
}
|
||||
}
|
||||
|
||||
static void convertOrdinaryDatabaseToAtomic(ContextMutablePtr context, const DatabasePtr & database)
|
||||
{
|
||||
/// It's kind of C++ script that creates temporary table with Atomic engine,
|
||||
/// moves all tables to it, drops old database and then renames new one to old name.
|
||||
|
||||
Poco::Logger * log = &Poco::Logger::get("loadMetadata");
|
||||
|
||||
String name = database->getDatabaseName();
|
||||
|
||||
String tmp_name = fmt::format(".tmp_convert.{}.{}", name, thread_local_rng());
|
||||
|
||||
String name_quoted = backQuoteIfNeed(name);
|
||||
String tmp_name_quoted = backQuoteIfNeed(tmp_name);
|
||||
|
||||
LOG_INFO(log, "Will convert database {} from Ordinary to Atomic", name_quoted);
|
||||
|
||||
String create_database_query = fmt::format("CREATE DATABASE IF NOT EXISTS {}", tmp_name_quoted);
|
||||
auto res = executeQuery(create_database_query, context, true);
|
||||
executeTrivialBlockIO(res, context);
|
||||
res = {};
|
||||
auto tmp_database = DatabaseCatalog::instance().getDatabase(tmp_name);
|
||||
assert(tmp_database->getEngineName() == "Atomic");
|
||||
|
||||
size_t num_tables = 0;
|
||||
for (auto iterator = database->getTablesIterator(context); iterator->isValid(); iterator->next())
|
||||
{
|
||||
++num_tables;
|
||||
auto id = iterator->table()->getStorageID();
|
||||
id.database_name = tmp_name;
|
||||
iterator->table()->checkTableCanBeRenamed(id);
|
||||
}
|
||||
|
||||
LOG_INFO(log, "Will move {} tables to {}", num_tables, tmp_name_quoted);
|
||||
|
||||
for (auto iterator = database->getTablesIterator(context); iterator->isValid(); iterator->next())
|
||||
{
|
||||
auto id = iterator->table()->getStorageID();
|
||||
String qualified_quoted_name = id.getFullTableName();
|
||||
id.database_name = tmp_name;
|
||||
String tmp_qualified_quoted_name = id.getFullTableName();
|
||||
|
||||
String move_table_query = fmt::format("RENAME TABLE {} TO {}", qualified_quoted_name, tmp_qualified_quoted_name);
|
||||
res = executeQuery(move_table_query, context, true);
|
||||
executeTrivialBlockIO(res, context);
|
||||
res = {};
|
||||
}
|
||||
|
||||
LOG_INFO(log, "Moved all tables from {} to {}", name_quoted, tmp_name_quoted);
|
||||
|
||||
if (!database->empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Database {} is not empty after moving tables", name_quoted);
|
||||
|
||||
String drop_query = fmt::format("DROP DATABASE {}", name_quoted);
|
||||
res = executeQuery(drop_query, context, true);
|
||||
executeTrivialBlockIO(res, context);
|
||||
res = {};
|
||||
|
||||
String rename_query = fmt::format("RENAME DATABASE {} TO {}", tmp_name_quoted, name_quoted);
|
||||
res = executeQuery(rename_query, context, true);
|
||||
executeTrivialBlockIO(res, context);
|
||||
|
||||
LOG_INFO(log, "Finished database engine conversion of {}", name_quoted);
|
||||
}
|
||||
|
||||
void maybeConvertOrdinaryDatabaseToAtomic(ContextMutablePtr context, const DatabasePtr & database)
|
||||
{
|
||||
if (database->getEngineName() != "Ordinary")
|
||||
return;
|
||||
|
||||
if (context->getSettingsRef().allow_deprecated_database_ordinary)
|
||||
return;
|
||||
|
||||
try
|
||||
{
|
||||
/// It's not quite correct to run DDL queries while database is not started up.
|
||||
startupSystemTables();
|
||||
|
||||
auto local_context = Context::createCopy(context);
|
||||
local_context->setSetting("check_table_dependencies", false);
|
||||
convertOrdinaryDatabaseToAtomic(local_context, database);
|
||||
|
||||
/// Reload database just in case (and update logger name)
|
||||
String detach_query = fmt::format("DETACH DATABASE {}", backQuoteIfNeed(DatabaseCatalog::SYSTEM_DATABASE));
|
||||
auto res = executeQuery(detach_query, context, true);
|
||||
executeTrivialBlockIO(res, context);
|
||||
res = {};
|
||||
|
||||
loadSystemDatabaseImpl(context, DatabaseCatalog::SYSTEM_DATABASE, "Atomic");
|
||||
TablesLoader::Databases databases =
|
||||
{
|
||||
{DatabaseCatalog::SYSTEM_DATABASE, DatabaseCatalog::instance().getSystemDatabase()},
|
||||
};
|
||||
TablesLoader loader{context, databases, /* force_restore */ true, /* force_attach */ true};
|
||||
loader.loadTables();
|
||||
|
||||
/// Will startup tables usual way
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
e.addMessage("While trying to convert {} to Atomic", database->getDatabaseName());
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void startupSystemTables()
|
||||
{
|
||||
|
@ -19,4 +19,8 @@ void loadMetadata(ContextMutablePtr context, const String & default_database_nam
|
||||
/// so we startup system tables after all databases are loaded.
|
||||
void startupSystemTables();
|
||||
|
||||
/// Converts database with Ordinary engine to Atomic. Does nothing if database is not Ordinary.
|
||||
/// Can be called only during server startup when there are no queries from users.
|
||||
void maybeConvertOrdinaryDatabaseToAtomic(ContextMutablePtr context, const DatabasePtr & database);
|
||||
|
||||
}
|
||||
|
@ -314,6 +314,13 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
String replica_name;
|
||||
StorageReplicatedMergeTree::RenamingRestrictions renaming_restrictions = StorageReplicatedMergeTree::RenamingRestrictions::ALLOW_ANY;
|
||||
|
||||
bool is_on_cluster = args.getLocalContext()->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY;
|
||||
bool is_replicated_database = args.getLocalContext()->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY &&
|
||||
DatabaseCatalog::instance().getDatabase(args.table_id.database_name)->getEngineName() == "Replicated";
|
||||
|
||||
if (is_replicated_database && !is_extended_storage_def)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Old syntax is not allowed for *MergeTree tables in Replicated databases");
|
||||
|
||||
if (replicated)
|
||||
{
|
||||
bool has_arguments = arg_num + 2 <= arg_cnt;
|
||||
@ -376,17 +383,11 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
throw Exception("Expected two string literal arguments: zookeeper_path and replica_name", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
/// Allow implicit {uuid} macros only for zookeeper_path in ON CLUSTER queries
|
||||
bool is_on_cluster = args.getLocalContext()->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY;
|
||||
bool is_replicated_database = args.getLocalContext()->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY &&
|
||||
DatabaseCatalog::instance().getDatabase(args.table_id.database_name)->getEngineName() == "Replicated";
|
||||
bool allow_uuid_macro = is_on_cluster || is_replicated_database || args.query.attach;
|
||||
|
||||
/// Unfold {database} and {table} macro on table creation, so table can be renamed.
|
||||
if (!args.attach)
|
||||
{
|
||||
if (is_replicated_database && !is_extended_storage_def)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Old syntax is not allowed for ReplicatedMergeTree tables in Replicated databases");
|
||||
|
||||
Macros::MacroExpansionInfo info;
|
||||
/// NOTE: it's not recursive
|
||||
info.expand_special_macros_only = true;
|
||||
|
@ -3390,14 +3390,6 @@ class ClickHouseInstance:
|
||||
],
|
||||
user="root",
|
||||
)
|
||||
self.exec_in_container(
|
||||
[
|
||||
"bash",
|
||||
"-c",
|
||||
"cp /usr/share/clickhouse-odbc-bridge_fresh /usr/bin/clickhouse-odbc-bridge && chmod 777 /usr/bin/clickhouse",
|
||||
],
|
||||
user="root",
|
||||
)
|
||||
self.exec_in_container(
|
||||
["bash", "-c", "{} --daemon".format(self.clickhouse_start_command)],
|
||||
user=str(os.getuid()),
|
||||
@ -3411,7 +3403,7 @@ class ClickHouseInstance:
|
||||
self.wait_start(time_left)
|
||||
|
||||
def restart_with_latest_version(
|
||||
self, stop_start_wait_sec=300, callback_onstop=None, signal=15
|
||||
self, stop_start_wait_sec=300, callback_onstop=None, signal=15, fix_metadata=False
|
||||
):
|
||||
begin_time = time.time()
|
||||
if not self.stay_alive:
|
||||
@ -3458,14 +3450,15 @@ class ClickHouseInstance:
|
||||
"echo 'restart_with_latest_version: From version' && /usr/share/clickhouse_original server --version && echo 'To version' /usr/share/clickhouse_fresh server --version",
|
||||
]
|
||||
)
|
||||
self.exec_in_container(
|
||||
[
|
||||
"bash",
|
||||
"-c",
|
||||
"cp /usr/share/clickhouse-odbc-bridge_fresh /usr/bin/clickhouse-odbc-bridge && chmod 777 /usr/bin/clickhouse",
|
||||
],
|
||||
user="root",
|
||||
)
|
||||
if fix_metadata:
|
||||
# Versions older than 20.7 might not create .sql file for system and default database
|
||||
# Create it manually if upgrading from older version
|
||||
self.exec_in_container(
|
||||
["bash", "-c", "echo 'ATTACH DATABASE system ENGINE=Ordinary' > /var/lib/clickhouse/metadata/system.sql"],
|
||||
)
|
||||
self.exec_in_container(
|
||||
["bash", "-c", "echo 'ATTACH DATABASE system ENGINE=Ordinary' > /var/lib/clickhouse/metadata/default.sql"],
|
||||
)
|
||||
self.exec_in_container(
|
||||
["bash", "-c", "{} --daemon".format(self.clickhouse_start_command)],
|
||||
user=str(os.getuid()),
|
||||
|
@ -54,7 +54,7 @@ def test_backup_from_old_version(started_cluster):
|
||||
|
||||
node1.query("ALTER TABLE source_table FREEZE PARTITION tuple();")
|
||||
|
||||
node1.restart_with_latest_version()
|
||||
node1.restart_with_latest_version(fix_metadata=True)
|
||||
|
||||
node1.query(
|
||||
"CREATE TABLE dest_table (A Int64, B String, Y String) ENGINE = ReplicatedMergeTree('/test/dest_table1', '1') ORDER BY tuple()"
|
||||
@ -107,7 +107,7 @@ def test_backup_from_old_version_setting(started_cluster):
|
||||
|
||||
node2.query("ALTER TABLE source_table FREEZE PARTITION tuple();")
|
||||
|
||||
node2.restart_with_latest_version()
|
||||
node2.restart_with_latest_version(fix_metadata=True)
|
||||
|
||||
node2.query(
|
||||
"CREATE TABLE dest_table (A Int64, B String, Y String) ENGINE = ReplicatedMergeTree('/test/dest_table2', '1') ORDER BY tuple() SETTINGS enable_mixed_granularity_parts = 1"
|
||||
@ -163,7 +163,7 @@ def test_backup_from_old_version_config(started_cluster):
|
||||
"<clickhouse><merge_tree><enable_mixed_granularity_parts>1</enable_mixed_granularity_parts></merge_tree></clickhouse>",
|
||||
)
|
||||
|
||||
node3.restart_with_latest_version(callback_onstop=callback)
|
||||
node3.restart_with_latest_version(callback_onstop=callback, fix_metadata=True)
|
||||
|
||||
node3.query(
|
||||
"CREATE TABLE dest_table (A Int64, B String, Y String) ENGINE = ReplicatedMergeTree('/test/dest_table3', '1') ORDER BY tuple() SETTINGS enable_mixed_granularity_parts = 1"
|
||||
|
@ -71,7 +71,7 @@ def test_backward_compatability(start_cluster):
|
||||
|
||||
assert node1.query("SELECT avgMerge(x) FROM state") == "2.5\n"
|
||||
|
||||
node1.restart_with_latest_version()
|
||||
node1.restart_with_latest_version(fix_metadata=True)
|
||||
|
||||
assert node1.query("SELECT avgMerge(x) FROM state") == "2.5\n"
|
||||
|
||||
|
@ -0,0 +1,59 @@
|
||||
import pytest
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
|
||||
cluster = ClickHouseCluster(__file__, name="convert_ordinary")
|
||||
node = cluster.add_instance(
|
||||
"node",
|
||||
image="yandex/clickhouse-server",
|
||||
tag="19.17.8.54",
|
||||
stay_alive=True,
|
||||
with_installed_binary=True,
|
||||
)
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def start_cluster():
|
||||
try:
|
||||
cluster.start()
|
||||
yield cluster
|
||||
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
def q(query):
|
||||
return node.query(query, settings={"log_queries": 1})
|
||||
|
||||
def test_convert_system_db_to_atomic(start_cluster):
|
||||
q("CREATE TABLE t(date Date, id UInt32) ENGINE = MergeTree PARTITION BY toYYYYMM(date) ORDER BY id")
|
||||
q("INSERT INTO t VALUES (today(), 1)")
|
||||
q("INSERT INTO t SELECT number % 1000, number FROM system.numbers LIMIT 1000000")
|
||||
|
||||
assert "1000001\n" == q("SELECT count() FROM t")
|
||||
assert "499999500001\n" == q("SELECT sum(id) FROM t")
|
||||
assert "1970-01-01\t1000\t499500000\n1970-01-02\t1000\t499501000\n" == q("SELECT date, count(), sum(id) FROM t GROUP BY date ORDER BY date LIMIT 2")
|
||||
q("SYSTEM FLUSH LOGS")
|
||||
|
||||
assert "query_log" in q("SHOW TABLES FROM system")
|
||||
assert "part_log" in q("SHOW TABLES FROM system")
|
||||
q("SYSTEM FLUSH LOGS")
|
||||
assert "1\n" == q("SELECT count() != 0 FROM system.query_log")
|
||||
assert "1\n" == q("SELECT count() != 0 FROM system.part_log")
|
||||
|
||||
node.restart_with_latest_version(fix_metadata=True)
|
||||
|
||||
assert "Ordinary" in node.query("SHOW CREATE DATABASE default")
|
||||
assert "Atomic" in node.query("SHOW CREATE DATABASE system")
|
||||
assert "query_log" in node.query("SHOW TABLES FROM system")
|
||||
assert "part_log" in node.query("SHOW TABLES FROM system")
|
||||
node.query("SYSTEM FLUSH LOGS")
|
||||
|
||||
assert "query_log_0" in node.query("SHOW TABLES FROM system")
|
||||
assert "part_log_0" in node.query("SHOW TABLES FROM system")
|
||||
assert "1\n" == node.query("SELECT count() != 0 FROM system.query_log_0")
|
||||
assert "1\n" == node.query("SELECT count() != 0 FROM system.part_log_0")
|
||||
assert "1970-01-01\t1000\t499500000\n1970-01-02\t1000\t499501000\n" == node.query("SELECT date, count(), sum(id) FROM t GROUP BY date ORDER BY date LIMIT 2")
|
||||
assert "INFORMATION_SCHEMA\ndefault\ninformation_schema\nsystem\n" == node.query("SELECT name FROM system.databases ORDER BY name")
|
||||
|
||||
assert "0\n" == node.count_in_log("<Error>")
|
||||
assert "0\n" == node.count_in_log("<Warning> Database")
|
||||
assert "0\n" == node.count_in_log("always include the lines below")
|
||||
|
@ -421,7 +421,7 @@ def test_default_codec_version_update(start_cluster):
|
||||
)
|
||||
|
||||
old_version = node3.query("SELECT version()")
|
||||
node3.restart_with_latest_version()
|
||||
node3.restart_with_latest_version(fix_metadata=True)
|
||||
new_version = node3.query("SELECT version()")
|
||||
logging.debug(f"Updated from {old_version} to {new_version}")
|
||||
assert (
|
||||
|
@ -495,7 +495,7 @@ def test_polymorphic_parts_diff_versions_2(start_cluster_diff_versions):
|
||||
with pytest.raises(Exception):
|
||||
node_old.query("SYSTEM SYNC REPLICA polymorphic_table_2", timeout=3)
|
||||
|
||||
node_old.restart_with_latest_version()
|
||||
node_old.restart_with_latest_version(fix_metadata=True)
|
||||
|
||||
node_old.query("SYSTEM SYNC REPLICA polymorphic_table_2", timeout=20)
|
||||
|
||||
|
@ -58,8 +58,8 @@ def test_mutate_and_upgrade(start_cluster):
|
||||
|
||||
node2.query("DETACH TABLE mt") # stop being leader
|
||||
node1.query("DETACH TABLE mt") # stop being leader
|
||||
node1.restart_with_latest_version(signal=9)
|
||||
node2.restart_with_latest_version(signal=9)
|
||||
node1.restart_with_latest_version(signal=9, fix_metadata=True)
|
||||
node2.restart_with_latest_version(signal=9, fix_metadata=True)
|
||||
|
||||
# After hard restart table can be in readonly mode
|
||||
exec_query_with_retry(
|
||||
@ -111,7 +111,7 @@ def test_upgrade_while_mutation(start_cluster):
|
||||
node3.query("ALTER TABLE mt1 DELETE WHERE id % 2 == 0")
|
||||
|
||||
node3.query("DETACH TABLE mt1") # stop being leader
|
||||
node3.restart_with_latest_version(signal=9)
|
||||
node3.restart_with_latest_version(signal=9, fix_metadata=True)
|
||||
|
||||
# checks for readonly
|
||||
exec_query_with_retry(node3, "OPTIMIZE TABLE mt1", sleep_time=5, retry_count=60)
|
||||
|
Loading…
Reference in New Issue
Block a user