Merge pull request #16193 from ClickHouse/database_replicated

DatabaseReplicated: continuation of #10485
This commit is contained in:
tavplubix 2021-02-20 19:39:34 +03:00 committed by GitHub
commit a697b578bc
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
80 changed files with 3611 additions and 1111 deletions

View File

@ -339,7 +339,7 @@ function run_tests
# Look at DistributedFilesToInsert, so cannot run in parallel.
01460_DistributedFilesToInsert
01541_max_memory_usage_for_user
01541_max_memory_usage_for_user_long
# Require python libraries like scipy, pandas and numpy
01322_ttest_scipy

View File

@ -60,4 +60,8 @@ fi
# more idiologically correct.
read -ra ADDITIONAL_OPTIONS <<< "${ADDITIONAL_OPTIONS:-}"
if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
ADDITIONAL_OPTIONS+=('--replicated-database')
fi
clickhouse-test --testname --shard --zookeeper --no-stateless --hung-check --print-time "$SKIP_LIST_OPT" "${ADDITIONAL_OPTIONS[@]}" "$SKIP_TESTS_OPTION" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt

View File

@ -57,6 +57,10 @@ function run_tests()
ADDITIONAL_OPTIONS+=('4')
fi
if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
ADDITIONAL_OPTIONS+=('--replicated-database')
fi
clickhouse-test --testname --shard --zookeeper --hung-check --print-time \
--test-runs "$NUM_TRIES" \
"$SKIP_LIST_OPT" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \

View File

@ -23,12 +23,15 @@ def get_options(i):
if 0 < i:
options += " --order=random"
if i % 2 == 1:
if i % 3 == 1:
options += " --db-engine=Ordinary"
if i % 3 == 2:
options += ''' --db-engine="Replicated('/test/db/test_{}', 's1', 'r1')"'''.format(i)
# If database name is not specified, new database is created for each functional test.
# Run some threads with one database for all tests.
if i % 3 == 1:
if i % 2 == 1:
options += " --database=test_{}".format(i)
if i == 13:

View File

@ -103,6 +103,7 @@ namespace CurrentMetrics
extern const Metric Revision;
extern const Metric VersionInteger;
extern const Metric MemoryTracking;
extern const Metric MaxDDLEntryID;
}
@ -1012,7 +1013,8 @@ int Server::main(const std::vector<std::string> & /*args*/)
int pool_size = config().getInt("distributed_ddl.pool_size", 1);
if (pool_size < 1)
throw Exception("distributed_ddl.pool_size should be greater then 0", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
global_context->setDDLWorker(std::make_unique<DDLWorker>(pool_size, ddl_zookeeper_path, *global_context, &config(), "distributed_ddl"));
global_context->setDDLWorker(std::make_unique<DDLWorker>(pool_size, ddl_zookeeper_path, *global_context, &config(),
"distributed_ddl", "DDLWorker", &CurrentMetrics::MaxDDLEntryID));
}
std::unique_ptr<DNSCacheUpdater> dns_cache_updater;

View File

@ -537,6 +537,7 @@
M(568, RAFT_ERROR) \
M(569, MULTIPLE_COLUMNS_SERIALIZED_TO_SAME_PROTOBUF_FIELD) \
M(570, DATA_TYPE_INCOMPATIBLE_WITH_PROTOBUF_FIELD) \
M(571, DATABASE_REPLICATION_FAILED) \
\
M(999, KEEPER_EXCEPTION) \
M(1000, POCO_EXCEPTION) \

View File

@ -59,7 +59,7 @@ static void addRootPath(String & path, const String & root_path)
throw Exception("Path cannot be empty", Error::ZBADARGUMENTS);
if (path[0] != '/')
throw Exception("Path must begin with /", Error::ZBADARGUMENTS);
throw Exception("Path must begin with /, got " + path, Error::ZBADARGUMENTS);
if (root_path.empty())
return;

View File

@ -194,6 +194,7 @@ public:
void removeChildren(const std::string & path);
using WaitCondition = std::function<bool()>;
/// Wait for the node to disappear or return immediately if it doesn't exist.
/// If condition is specified, it is used to return early (when condition returns false)
/// The function returns true if waited and false if waiting was interrupted by condition.
@ -314,8 +315,15 @@ public:
return std::make_shared<EphemeralNodeHolder>(path, zookeeper, false, false, "");
}
void setAlreadyRemoved()
{
need_remove = false;
}
~EphemeralNodeHolder()
{
if (!need_remove)
return;
try
{
zookeeper.tryRemove(path);
@ -331,6 +339,7 @@ private:
std::string path;
ZooKeeper & zookeeper;
CurrentMetrics::Increment metric_increment{CurrentMetrics::EphemeralNode};
bool need_remove = true;
};
using EphemeralNodeHolderPtr = EphemeralNodeHolder::Ptr;

View File

@ -421,6 +421,9 @@ class IColumn;
M(Bool, optimize_rewrite_sum_if_to_count_if, true, "Rewrite sumIf() and sum(if()) function countIf() function when logically equivalent", 0) \
M(UInt64, insert_shard_id, 0, "If non zero, when insert into a distributed table, the data will be inserted into the shard `insert_shard_id` synchronously. Possible values range from 1 to `shards_number` of corresponding distributed table", 0) \
M(Bool, allow_experimental_query_deduplication, false, "Allow sending parts' UUIDs for a query in order to deduplicate data parts if any", 0) \
M(Bool, allow_experimental_database_replicated, false, "Allow to create databases with Replicated engine", 0) \
M(UInt64, database_replicated_initial_query_timeout_sec, 300, "How long initial DDL query should wait for Replicated database to precess previous DDL queue entries", 0) \
M(Bool, database_replicated_ddl_output, true, "Return table with query execution status as a result of DDL query", 0) \
\
/** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \
\

View File

@ -4,13 +4,14 @@
#include <Poco/Path.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadBufferFromFile.h>
#include <Parsers/formatAST.h>
#include <Common/renameat2.h>
#include <Storages/StorageMaterializedView.h>
#include <Interpreters/Context.h>
#include <Interpreters/ExternalDictionariesLoader.h>
#include <filesystem>
#include <Interpreters/DDLTask.h>
namespace DB
{
@ -34,7 +35,6 @@ public:
UUID uuid() const override { return table()->getStorageID().uuid; }
};
DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, UUID uuid, const String & logger_name, const Context & context_)
: DatabaseOrdinary(name_, std::move(metadata_path_), "store/", logger_name, context_)
, path_to_table_symlinks(global_context.getPath() + "data/" + escapeForFileName(name_) + "/")
@ -106,7 +106,7 @@ StoragePtr DatabaseAtomic::detachTable(const String & name)
return table;
}
void DatabaseAtomic::dropTable(const Context &, const String & table_name, bool no_delay)
void DatabaseAtomic::dropTable(const Context & context, const String & table_name, bool no_delay)
{
String table_metadata_path = getObjectMetadataPath(table_name);
String table_metadata_path_drop;
@ -115,6 +115,16 @@ void DatabaseAtomic::dropTable(const Context &, const String & table_name, bool
std::unique_lock lock(mutex);
table = getTableUnlocked(table_name, lock);
table_metadata_path_drop = DatabaseCatalog::instance().getPathForDroppedMetadata(table->getStorageID());
auto txn = context.getZooKeeperMetadataTransaction();
if (txn && !context.isInternalSubquery())
txn->commit(); /// Commit point (a sort of) for Replicated database
/// NOTE: replica will be lost if server crashes before the following rename
/// We apply changes in ZooKeeper before applying changes in local metadata file
/// to reduce probability of failures between these operations
/// (it's more likely to lost connection, than to fail before applying local changes).
/// TODO better detection and recovery
Poco::File(table_metadata_path).renameTo(table_metadata_path_drop); /// Mark table as dropped
DatabaseWithDictionaries::detachTableUnlocked(table_name, lock); /// Should never throw
table_name_to_path.erase(table_name);
@ -124,7 +134,7 @@ void DatabaseAtomic::dropTable(const Context &, const String & table_name, bool
/// Remove the inner table (if any) to avoid deadlock
/// (due to attempt to execute DROP from the worker thread)
if (auto * mv = dynamic_cast<StorageMaterializedView *>(table.get()))
mv->dropInnerTable(no_delay);
mv->dropInnerTable(no_delay, context);
/// Notify DatabaseCatalog that table was dropped. It will remove table data in background.
/// Cleanup is performed outside of database to allow easily DROP DATABASE without waiting for cleanup to complete.
DatabaseCatalog::instance().enqueueDroppedTableCleanup(table->getStorageID(), table, table_metadata_path_drop, no_delay);
@ -144,6 +154,8 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n
if (exchange && dictionary)
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot exchange dictionaries");
if (exchange && !supportsRenameat2())
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "RENAME EXCHANGE is not supported");
auto & other_db = dynamic_cast<DatabaseAtomic &>(to_database);
bool inside_database = this == &other_db;
@ -232,6 +244,13 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n
}
/// Table renaming actually begins here
auto txn = context.getZooKeeperMetadataTransaction();
if (txn && !context.isInternalSubquery())
txn->commit(); /// Commit point (a sort of) for Replicated database
/// NOTE: replica will be lost if server crashes before the following rename
/// TODO better detection and recovery
if (exchange)
renameExchange(old_metadata_path, new_metadata_path);
else
@ -267,7 +286,8 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n
}
void DatabaseAtomic::commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table,
const String & table_metadata_tmp_path, const String & table_metadata_path)
const String & table_metadata_tmp_path, const String & table_metadata_path,
const Context & query_context)
{
DetachedTables not_in_use;
auto table_data_path = getTableDataPath(query);
@ -284,6 +304,14 @@ void DatabaseAtomic::commitCreateTable(const ASTCreateQuery & query, const Stora
/// We will get en exception if some table with the same UUID exists (even if it's detached table or table from another database)
DatabaseCatalog::instance().addUUIDMapping(query.uuid);
locked_uuid = true;
auto txn = query_context.getZooKeeperMetadataTransaction();
if (txn && !query_context.isInternalSubquery())
txn->commit(); /// Commit point (a sort of) for Replicated database
/// NOTE: replica will be lost if server crashes before the following renameNoReplace(...)
/// TODO better detection and recovery
/// It throws if `table_metadata_path` already exists (it's possible if table was detached)
renameNoReplace(table_metadata_tmp_path, table_metadata_path); /// Commit point (a sort of)
attachTableUnlocked(query.table, table, lock); /// Should never throw
@ -300,7 +328,8 @@ void DatabaseAtomic::commitCreateTable(const ASTCreateQuery & query, const Stora
tryCreateSymlink(query.table, table_data_path);
}
void DatabaseAtomic::commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path)
void DatabaseAtomic::commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path,
const String & /*statement*/, const Context & query_context)
{
bool check_file_exists = true;
SCOPE_EXIT({ std::error_code code; if (check_file_exists) std::filesystem::remove(table_metadata_tmp_path, code); });
@ -311,6 +340,13 @@ void DatabaseAtomic::commitAlterTable(const StorageID & table_id, const String &
if (table_id.uuid != actual_table_id.uuid)
throw Exception("Cannot alter table because it was renamed", ErrorCodes::CANNOT_ASSIGN_ALTER);
auto txn = query_context.getZooKeeperMetadataTransaction();
if (txn && !query_context.isInternalSubquery())
txn->commit(); /// Commit point (a sort of) for Replicated database
/// NOTE: replica will be lost if server crashes before the following rename
/// TODO better detection and recovery
check_file_exists = renameExchangeIfSupported(table_metadata_tmp_path, table_metadata_path);
if (!check_file_exists)
std::filesystem::rename(table_metadata_tmp_path, table_metadata_path);
@ -329,6 +365,12 @@ void DatabaseAtomic::assertDetachedTableNotInUse(const UUID & uuid)
", because it was detached but still used by some query. Retry later.", ErrorCodes::TABLE_ALREADY_EXISTS);
}
void DatabaseAtomic::setDetachedTableNotInUseForce(const UUID & uuid)
{
std::unique_lock lock{mutex};
detached_tables.erase(uuid);
}
DatabaseAtomic::DetachedTables DatabaseAtomic::cleanupDetachedTables()
{
DetachedTables not_in_use;

View File

@ -58,11 +58,12 @@ public:
void tryRemoveSymlink(const String & table_name);
void waitDetachedTableNotInUse(const UUID & uuid) override;
void setDetachedTableNotInUseForce(const UUID & uuid);
private:
void commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path) override;
protected:
void commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path, const String & statement, const Context & query_context) override;
void commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table,
const String & table_metadata_tmp_path, const String & table_metadata_path) override;
const String & table_metadata_tmp_path, const String & table_metadata_path, const Context & query_context) override;
void assertDetachedTableNotInUse(const UUID & uuid);
typedef std::unordered_map<UUID, StoragePtr> DetachedTables;

View File

@ -1,6 +1,7 @@
#include <Databases/DatabaseFactory.h>
#include <Databases/DatabaseAtomic.h>
#include <Databases/DatabaseReplicated.h>
#include <Databases/DatabaseDictionary.h>
#include <Databases/DatabaseLazy.h>
#include <Databases/DatabaseMemory.h>
@ -13,6 +14,7 @@
#include <Poco/File.h>
#include <Poco/Path.h>
#include <Interpreters/Context.h>
#include <Common/Macros.h>
#if !defined(ARCADIA_BUILD)
# include "config_core.h"
@ -96,11 +98,16 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String
const String & engine_name = engine_define->engine->name;
const UUID & uuid = create.uuid;
if (engine_name != "MySQL" && engine_name != "MaterializeMySQL" && engine_name != "Lazy" && engine_name != "PostgreSQL" && engine_define->engine->arguments)
bool engine_may_have_arguments = engine_name == "MySQL" || engine_name == "MaterializeMySQL" || engine_name == "Lazy" ||
engine_name == "Replicated" || engine_name == "PostgreSQL";
if (engine_define->engine->arguments && !engine_may_have_arguments)
throw Exception("Database engine " + engine_name + " cannot have arguments", ErrorCodes::BAD_ARGUMENTS);
if (engine_define->engine->parameters || engine_define->partition_by || engine_define->primary_key || engine_define->order_by ||
engine_define->sample_by || (!endsWith(engine_name, "MySQL") && engine_define->settings))
bool has_unexpected_element = engine_define->engine->parameters || engine_define->partition_by ||
engine_define->primary_key || engine_define->order_by ||
engine_define->sample_by;
bool may_have_settings = endsWith(engine_name, "MySQL") || engine_name == "Replicated";
if (has_unexpected_element || (!may_have_settings && engine_define->settings))
throw Exception("Database engine " + engine_name + " cannot have parameters, primary_key, order_by, sample_by, settings",
ErrorCodes::UNKNOWN_ELEMENT_IN_AST);
@ -184,6 +191,32 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String
return std::make_shared<DatabaseLazy>(database_name, metadata_path, cache_expiration_time_seconds, context);
}
else if (engine_name == "Replicated")
{
const ASTFunction * engine = engine_define->engine;
if (!engine->arguments || engine->arguments->children.size() != 3)
throw Exception("Replicated database requires 3 arguments: zookeeper path, shard name and replica name", ErrorCodes::BAD_ARGUMENTS);
const auto & arguments = engine->arguments->children;
String zookeeper_path = safeGetLiteralValue<String>(arguments[0], "Replicated");
String shard_name = safeGetLiteralValue<String>(arguments[1], "Replicated");
String replica_name = safeGetLiteralValue<String>(arguments[2], "Replicated");
zookeeper_path = context.getMacros()->expand(zookeeper_path);
shard_name = context.getMacros()->expand(shard_name);
replica_name = context.getMacros()->expand(replica_name);
DatabaseReplicatedSettings database_replicated_settings{};
if (engine_define->settings)
database_replicated_settings.loadFromQuery(*engine_define);
return std::make_shared<DatabaseReplicated>(database_name, metadata_path, uuid,
zookeeper_path, shard_name, replica_name,
std::move(database_replicated_settings), context);
}
#if USE_LIBPQXX
else if (engine_name == "PostgreSQL")

View File

@ -129,6 +129,60 @@ String getObjectDefinitionFromCreateQuery(const ASTPtr & query)
return statement_buf.str();
}
void applyMetadataChangesToCreateQuery(const ASTPtr & query, const StorageInMemoryMetadata & metadata)
{
auto & ast_create_query = query->as<ASTCreateQuery &>();
bool has_structure = ast_create_query.columns_list && ast_create_query.columns_list->columns;
if (ast_create_query.as_table_function && !has_structure)
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot alter table {} because it was created AS table function"
" and doesn't have structure in metadata", backQuote(ast_create_query.table));
assert(has_structure);
ASTPtr new_columns = InterpreterCreateQuery::formatColumns(metadata.columns);
ASTPtr new_indices = InterpreterCreateQuery::formatIndices(metadata.secondary_indices);
ASTPtr new_constraints = InterpreterCreateQuery::formatConstraints(metadata.constraints);
ast_create_query.columns_list->replace(ast_create_query.columns_list->columns, new_columns);
ast_create_query.columns_list->setOrReplace(ast_create_query.columns_list->indices, new_indices);
ast_create_query.columns_list->setOrReplace(ast_create_query.columns_list->constraints, new_constraints);
if (metadata.select.select_query)
{
query->replace(ast_create_query.select, metadata.select.select_query);
}
/// MaterializedView is one type of CREATE query without storage.
if (ast_create_query.storage)
{
ASTStorage & storage_ast = *ast_create_query.storage;
bool is_extended_storage_def
= storage_ast.partition_by || storage_ast.primary_key || storage_ast.order_by || storage_ast.sample_by || storage_ast.settings;
if (is_extended_storage_def)
{
if (metadata.sorting_key.definition_ast)
storage_ast.set(storage_ast.order_by, metadata.sorting_key.definition_ast);
if (metadata.primary_key.definition_ast)
storage_ast.set(storage_ast.primary_key, metadata.primary_key.definition_ast);
if (metadata.sampling_key.definition_ast)
storage_ast.set(storage_ast.sample_by, metadata.sampling_key.definition_ast);
if (metadata.table_ttl.definition_ast)
storage_ast.set(storage_ast.ttl_table, metadata.table_ttl.definition_ast);
else if (storage_ast.ttl_table != nullptr) /// TTL was removed
storage_ast.ttl_table = nullptr;
if (metadata.settings_changes)
storage_ast.set(storage_ast.settings, metadata.settings_changes);
}
}
}
DatabaseOnDisk::DatabaseOnDisk(
const String & name,
const String & metadata_path_,
@ -214,7 +268,7 @@ void DatabaseOnDisk::createTable(
out.close();
}
commitCreateTable(create, table, table_metadata_tmp_path, table_metadata_path);
commitCreateTable(create, table, table_metadata_tmp_path, table_metadata_path, context);
removeDetachedPermanentlyFlag(table_name, table_metadata_path);
}
@ -238,7 +292,8 @@ void DatabaseOnDisk::removeDetachedPermanentlyFlag(const String & table_name, co
}
void DatabaseOnDisk::commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table,
const String & table_metadata_tmp_path, const String & table_metadata_path)
const String & table_metadata_tmp_path, const String & table_metadata_path,
const Context & /*query_context*/)
{
try
{
@ -256,7 +311,7 @@ void DatabaseOnDisk::commitCreateTable(const ASTCreateQuery & query, const Stora
}
}
void DatabaseOnDisk::detachTablePermanently(const String & table_name)
void DatabaseOnDisk::detachTablePermanently(const Context &, const String & table_name)
{
auto table = detachTable(table_name);
@ -352,6 +407,8 @@ void DatabaseOnDisk::renameTable(
from_ordinary_to_atomic = true;
else if (typeid_cast<DatabaseAtomic *>(this) && typeid_cast<DatabaseOrdinary *>(&to_database))
from_atomic_to_ordinary = true;
else if (dynamic_cast<DatabaseAtomic *>(this) && typeid_cast<DatabaseOrdinary *>(&to_database) && getEngineName() == "Replicated")
from_atomic_to_ordinary = true;
else
throw Exception("Moving tables between databases of different engines is not supported", ErrorCodes::NOT_IMPLEMENTED);
}
@ -363,6 +420,7 @@ void DatabaseOnDisk::renameTable(
/// DatabaseLazy::detachTable may return nullptr even if table exists, so we need tryGetTable for this case.
StoragePtr table = tryGetTable(table_name, global_context);
detachTable(table_name);
UUID prev_uuid = UUIDHelpers::Nil;
try
{
table_lock = table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
@ -375,7 +433,7 @@ void DatabaseOnDisk::renameTable(
if (from_ordinary_to_atomic)
create.uuid = UUIDHelpers::generateV4();
if (from_atomic_to_ordinary)
create.uuid = UUIDHelpers::Nil;
std::swap(create.uuid, prev_uuid);
if (auto * target_db = dynamic_cast<DatabaseOnDisk *>(&to_database))
target_db->checkMetadataFilenameAvailability(to_table_name);
@ -400,12 +458,16 @@ void DatabaseOnDisk::renameTable(
Poco::File(table_metadata_path).remove();
/// Special case: usually no actions with symlinks are required when detaching/attaching table,
/// but not when moving from Atomic database to Ordinary
if (from_atomic_to_ordinary && table->storesDataOnDisk())
if (from_atomic_to_ordinary)
{
auto & atomic_db = assert_cast<DatabaseAtomic &>(*this);
atomic_db.tryRemoveSymlink(table_name);
auto & atomic_db = dynamic_cast<DatabaseAtomic &>(*this);
/// Special case: usually no actions with symlinks are required when detaching/attaching table,
/// but not when moving from Atomic database to Ordinary
if (table->storesDataOnDisk())
atomic_db.tryRemoveSymlink(table_name);
/// Forget about UUID, now it's possible to reuse it for new table
DatabaseCatalog::instance().removeUUIDMappingFinally(prev_uuid);
atomic_db.setDetachedTableNotInUseForce(prev_uuid);
}
}

View File

@ -25,6 +25,8 @@ std::pair<String, StoragePtr> createTableFromAST(
*/
String getObjectDefinitionFromCreateQuery(const ASTPtr & query);
void applyMetadataChangesToCreateQuery(const ASTPtr & query, const StorageInMemoryMetadata & metadata);
/* Class to provide basic operations with tables when metadata is stored on disk in .sql files.
*/
@ -39,7 +41,7 @@ public:
const StoragePtr & table,
const ASTPtr & query) override;
void detachTablePermanently(const String & table_name) override;
void detachTablePermanently(const Context & context, const String & table_name) override;
void dropTable(
const Context & context,
@ -90,7 +92,7 @@ protected:
ASTPtr getCreateQueryFromMetadata(const String & metadata_path, bool throw_on_error) const;
virtual void commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table,
const String & table_metadata_tmp_path, const String & table_metadata_path);
const String & table_metadata_tmp_path, const String & table_metadata_path, const Context & query_context);
const String metadata_path;
const String data_path;

View File

@ -33,11 +33,6 @@ static constexpr size_t PRINT_MESSAGE_EACH_N_OBJECTS = 256;
static constexpr size_t PRINT_MESSAGE_EACH_N_SECONDS = 5;
static constexpr size_t METADATA_FILE_BUFFER_SIZE = 32768;
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
namespace
{
void tryAttachTable(
@ -272,55 +267,7 @@ void DatabaseOrdinary::alterTable(const Context & context, const StorageID & tab
0,
context.getSettingsRef().max_parser_depth);
auto & ast_create_query = ast->as<ASTCreateQuery &>();
bool has_structure = ast_create_query.columns_list && ast_create_query.columns_list->columns;
if (ast_create_query.as_table_function && !has_structure)
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot alter table {} because it was created AS table function"
" and doesn't have structure in metadata", backQuote(table_name));
assert(has_structure);
ASTPtr new_columns = InterpreterCreateQuery::formatColumns(metadata.columns);
ASTPtr new_indices = InterpreterCreateQuery::formatIndices(metadata.secondary_indices);
ASTPtr new_constraints = InterpreterCreateQuery::formatConstraints(metadata.constraints);
ast_create_query.columns_list->replace(ast_create_query.columns_list->columns, new_columns);
ast_create_query.columns_list->setOrReplace(ast_create_query.columns_list->indices, new_indices);
ast_create_query.columns_list->setOrReplace(ast_create_query.columns_list->constraints, new_constraints);
if (metadata.select.select_query)
{
ast->replace(ast_create_query.select, metadata.select.select_query);
}
/// MaterializedView is one type of CREATE query without storage.
if (ast_create_query.storage)
{
ASTStorage & storage_ast = *ast_create_query.storage;
bool is_extended_storage_def
= storage_ast.partition_by || storage_ast.primary_key || storage_ast.order_by || storage_ast.sample_by || storage_ast.settings;
if (is_extended_storage_def)
{
if (metadata.sorting_key.definition_ast)
storage_ast.set(storage_ast.order_by, metadata.sorting_key.definition_ast);
if (metadata.primary_key.definition_ast)
storage_ast.set(storage_ast.primary_key, metadata.primary_key.definition_ast);
if (metadata.sampling_key.definition_ast)
storage_ast.set(storage_ast.sample_by, metadata.sampling_key.definition_ast);
if (metadata.table_ttl.definition_ast)
storage_ast.set(storage_ast.ttl_table, metadata.table_ttl.definition_ast);
else if (storage_ast.ttl_table != nullptr) /// TTL was removed
storage_ast.ttl_table = nullptr;
if (metadata.settings_changes)
storage_ast.set(storage_ast.settings, metadata.settings_changes);
}
}
applyMetadataChangesToCreateQuery(ast, metadata);
statement = getObjectDefinitionFromCreateQuery(ast);
{
@ -332,10 +279,10 @@ void DatabaseOrdinary::alterTable(const Context & context, const StorageID & tab
out.close();
}
commitAlterTable(table_id, table_metadata_tmp_path, table_metadata_path);
commitAlterTable(table_id, table_metadata_tmp_path, table_metadata_path, statement, context);
}
void DatabaseOrdinary::commitAlterTable(const StorageID &, const String & table_metadata_tmp_path, const String & table_metadata_path)
void DatabaseOrdinary::commitAlterTable(const StorageID &, const String & table_metadata_tmp_path, const String & table_metadata_path, const String & /*statement*/, const Context & /*query_context*/)
{
try
{

View File

@ -30,7 +30,7 @@ public:
const StorageInMemoryMetadata & metadata) override;
protected:
virtual void commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path);
virtual void commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path, const String & statement, const Context & query_context);
void startupTables(ThreadPool & thread_pool);
};

View File

@ -0,0 +1,719 @@
#include <DataTypes/DataTypeString.h>
#include <Databases/DatabaseReplicated.h>
#include <IO/ReadBufferFromFile.h>
#include <IO/ReadBufferFromString.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Interpreters/Context.h>
#include <Interpreters/executeQuery.h>
#include <Parsers/queryToString.h>
#include <Common/Exception.h>
#include <Common/Stopwatch.h>
#include <Common/ZooKeeper/KeeperException.h>
#include <Common/ZooKeeper/Types.h>
#include <Common/ZooKeeper/ZooKeeper.h>
#include <Databases/DatabaseReplicatedWorker.h>
#include <Interpreters/DDLTask.h>
#include <Interpreters/executeDDLQueryOnCluster.h>
#include <Interpreters/Cluster.h>
#include <common/getFQDNOrHostName.h>
#include <Parsers/ASTAlterQuery.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/parseQuery.h>
#include <Interpreters/InterpreterCreateQuery.h>
#include <Parsers/formatAST.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NO_ZOOKEEPER;
extern const int LOGICAL_ERROR;
extern const int BAD_ARGUMENTS;
extern const int REPLICA_IS_ALREADY_EXIST;
extern const int DATABASE_REPLICATION_FAILED;
extern const int UNKNOWN_DATABASE;
extern const int UNKNOWN_TABLE;
extern const int NOT_IMPLEMENTED;
extern const int INCORRECT_QUERY;
extern const int ALL_CONNECTION_TRIES_FAILED;
}
static constexpr const char * DROPPED_MARK = "DROPPED";
static constexpr const char * BROKEN_TABLES_SUFFIX = "_broken_tables";
zkutil::ZooKeeperPtr DatabaseReplicated::getZooKeeper() const
{
return global_context.getZooKeeper();
}
static inline String getHostID(const Context & global_context, const UUID & db_uuid)
{
return Cluster::Address::toString(getFQDNOrHostName(), global_context.getTCPPort()) + ':' + toString(db_uuid);
}
DatabaseReplicated::~DatabaseReplicated() = default;
DatabaseReplicated::DatabaseReplicated(
const String & name_,
const String & metadata_path_,
UUID uuid,
const String & zookeeper_path_,
const String & shard_name_,
const String & replica_name_,
DatabaseReplicatedSettings db_settings_,
const Context & context_)
: DatabaseAtomic(name_, metadata_path_, uuid, "DatabaseReplicated (" + name_ + ")", context_)
, zookeeper_path(zookeeper_path_)
, shard_name(shard_name_)
, replica_name(replica_name_)
, db_settings(std::move(db_settings_))
{
if (zookeeper_path.empty() || shard_name.empty() || replica_name.empty())
throw Exception("ZooKeeper path, shard and replica names must be non-empty", ErrorCodes::BAD_ARGUMENTS);
if (shard_name.find('/') != std::string::npos || replica_name.find('/') != std::string::npos)
throw Exception("Shard and replica names should not contain '/'", ErrorCodes::BAD_ARGUMENTS);
if (shard_name.find('|') != std::string::npos || replica_name.find('|') != std::string::npos)
throw Exception("Shard and replica names should not contain '|'", ErrorCodes::BAD_ARGUMENTS);
if (zookeeper_path.back() == '/')
zookeeper_path.resize(zookeeper_path.size() - 1);
/// If zookeeper chroot prefix is used, path should start with '/', because chroot concatenates without it.
if (zookeeper_path.front() != '/')
zookeeper_path = "/" + zookeeper_path;
}
String DatabaseReplicated::getFullReplicaName() const
{
return shard_name + '|' + replica_name;
}
std::pair<String, String> DatabaseReplicated::parseFullReplicaName(const String & name)
{
String shard;
String replica;
auto pos = name.find('|');
if (pos == std::string::npos || name.find('|', pos + 1) != std::string::npos)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Incorrect replica identifier: {}", name);
shard = name.substr(0, pos);
replica = name.substr(pos + 1);
return {shard, replica};
}
ClusterPtr DatabaseReplicated::getCluster() const
{
/// TODO Maintain up-to-date Cluster and allow to use it in Distributed tables
Strings hosts;
Strings host_ids;
auto zookeeper = global_context.getZooKeeper();
constexpr int max_retries = 10;
int iteration = 0;
bool success = false;
while (++iteration <= max_retries)
{
host_ids.resize(0);
Coordination::Stat stat;
hosts = zookeeper->getChildren(zookeeper_path + "/replicas", &stat);
if (hosts.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "No hosts found");
Int32 cver = stat.cversion;
std::sort(hosts.begin(), hosts.end());
std::vector<zkutil::ZooKeeper::FutureGet> futures;
futures.reserve(hosts.size());
host_ids.reserve(hosts.size());
for (const auto & host : hosts)
futures.emplace_back(zookeeper->asyncTryGet(zookeeper_path + "/replicas/" + host));
success = true;
for (auto & future : futures)
{
auto res = future.get();
if (res.error != Coordination::Error::ZOK)
success = false;
host_ids.emplace_back(res.data);
}
zookeeper->get(zookeeper_path + "/replicas", &stat);
if (success && cver == stat.version)
break;
}
if (!success)
throw Exception(ErrorCodes::ALL_CONNECTION_TRIES_FAILED, "Cannot get consistent cluster snapshot,"
"because replicas are created or removed concurrently");
assert(!hosts.empty());
assert(hosts.size() == host_ids.size());
String current_shard = parseFullReplicaName(hosts.front()).first;
std::vector<Strings> shards;
shards.emplace_back();
for (size_t i = 0; i < hosts.size(); ++i)
{
const auto & id = host_ids[i];
if (id == DROPPED_MARK)
continue;
auto [shard, replica] = parseFullReplicaName(hosts[i]);
auto pos = id.find(':');
String host = id.substr(0, pos);
if (shard != current_shard)
{
current_shard = shard;
if (!shards.back().empty())
shards.emplace_back();
}
shards.back().emplace_back(unescapeForFileName(host));
}
/// TODO make it configurable
String username = "default";
String password;
return std::make_shared<Cluster>(global_context.getSettingsRef(), shards, username, password, global_context.getTCPPort(), false);
}
void DatabaseReplicated::tryConnectToZooKeeperAndInitDatabase(bool force_attach)
{
try
{
if (!global_context.hasZooKeeper())
{
throw Exception("Can't create replicated database without ZooKeeper", ErrorCodes::NO_ZOOKEEPER);
}
auto current_zookeeper = global_context.getZooKeeper();
if (!current_zookeeper->exists(zookeeper_path))
{
/// Create new database, multiple nodes can execute it concurrently
createDatabaseNodesInZooKeeper(current_zookeeper);
}
replica_path = zookeeper_path + "/replicas/" + getFullReplicaName();
String replica_host_id;
if (current_zookeeper->tryGet(replica_path, replica_host_id))
{
String host_id = getHostID(global_context, db_uuid);
if (replica_host_id != host_id)
throw Exception(ErrorCodes::REPLICA_IS_ALREADY_EXIST,
"Replica {} of shard {} of replicated database at {} already exists. Replica host ID: '{}', current host ID: '{}'",
replica_name, shard_name, zookeeper_path, replica_host_id, host_id);
}
else
{
/// Throws if replica with the same name already exists
createReplicaNodesInZooKeeper(current_zookeeper);
}
is_readonly = false;
}
catch (...)
{
if (!force_attach)
throw;
/// It's server startup, ignore error.
/// Worker thread will try to setup ZooKeeper connection
tryLogCurrentException(log);
}
}
bool DatabaseReplicated::createDatabaseNodesInZooKeeper(const zkutil::ZooKeeperPtr & current_zookeeper)
{
current_zookeeper->createAncestors(zookeeper_path);
Coordination::Requests ops;
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path, "", zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/log", "", zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/replicas", "", zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/counter", "", zkutil::CreateMode::Persistent));
/// We create and remove counter/cnt- node to increment sequential number of counter/ node and make log entry numbers start from 1.
/// New replicas are created with log pointer equal to 0 and log pointer is a number of the last executed entry.
/// It means that we cannot have log entry with number 0.
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/counter/cnt-", "", zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_path + "/counter/cnt-", -1));
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/metadata", "", zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/max_log_ptr", "1", zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/logs_to_keep", "1000", zkutil::CreateMode::Persistent));
Coordination::Responses responses;
auto res = current_zookeeper->tryMulti(ops, responses);
if (res == Coordination::Error::ZOK)
return true; /// Created new database (it's the first replica)
if (res == Coordination::Error::ZNODEEXISTS)
return false; /// Database exists, we will add new replica
/// Other codes are unexpected, will throw
zkutil::KeeperMultiException::check(res, ops, responses);
assert(false);
__builtin_unreachable();
}
void DatabaseReplicated::createReplicaNodesInZooKeeper(const zkutil::ZooKeeperPtr & current_zookeeper)
{
/// Write host name to replica_path, it will protect from multiple replicas with the same name
auto host_id = getHostID(global_context, db_uuid);
/// On replica creation add empty entry to log. Can be used to trigger some actions on other replicas (e.g. update cluster info).
DDLLogEntry entry{};
String query_path_prefix = zookeeper_path + "/log/query-";
String counter_prefix = zookeeper_path + "/counter/cnt-";
String counter_path = current_zookeeper->create(counter_prefix, "", zkutil::CreateMode::EphemeralSequential);
String query_path = query_path_prefix + counter_path.substr(counter_prefix.size());
Coordination::Requests ops;
ops.emplace_back(zkutil::makeCreateRequest(replica_path, host_id, zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/log_ptr", "0", zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(query_path, entry.toString(), zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeRemoveRequest(counter_path, -1));
current_zookeeper->multi(ops);
}
void DatabaseReplicated::loadStoredObjects(Context & context, bool has_force_restore_data_flag, bool force_attach)
{
tryConnectToZooKeeperAndInitDatabase(force_attach);
DatabaseAtomic::loadStoredObjects(context, has_force_restore_data_flag, force_attach);
ddl_worker = std::make_unique<DatabaseReplicatedDDLWorker>(this, global_context);
ddl_worker->startup();
}
BlockIO DatabaseReplicated::tryEnqueueReplicatedDDL(const ASTPtr & query, const Context & query_context)
{
if (is_readonly)
throw Exception(ErrorCodes::NO_ZOOKEEPER, "Database is in readonly mode, because it cannot connect to ZooKeeper");
if (query_context.getClientInfo().query_kind != ClientInfo::QueryKind::INITIAL_QUERY)
throw Exception(ErrorCodes::INCORRECT_QUERY, "It's not initial query. ON CLUSTER is not allowed for Replicated database.");
/// Replicas will set correct name of current database in query context (database name can be different on replicas)
if (auto * ddl_query = query->as<ASTQueryWithTableAndOutput>())
ddl_query->database.clear();
if (const auto * query_alter = query->as<ASTAlterQuery>())
{
for (const auto & command : query_alter->command_list->children)
{
if (!isSupportedAlterType(command->as<ASTAlterCommand&>().type))
throw Exception("Unsupported type of ALTER query", ErrorCodes::NOT_IMPLEMENTED);
}
}
LOG_DEBUG(log, "Proposing query: {}", queryToString(query));
/// TODO maybe write current settings to log entry?
DDLLogEntry entry;
entry.query = queryToString(query);
entry.initiator = ddl_worker->getCommonHostID();
String node_path = ddl_worker->tryEnqueueAndExecuteEntry(entry, query_context);
BlockIO io;
if (query_context.getSettingsRef().distributed_ddl_task_timeout == 0)
return io;
Strings hosts_to_wait = getZooKeeper()->getChildren(zookeeper_path + "/replicas");
auto stream = std::make_shared<DDLQueryStatusInputStream>(node_path, entry, query_context, hosts_to_wait);
if (query_context.getSettingsRef().database_replicated_ddl_output)
io.in = std::move(stream);
return io;
}
static UUID getTableUUIDIfReplicated(const String & metadata, const Context & context)
{
bool looks_like_replicated = metadata.find("ReplicatedMergeTree") != std::string::npos;
if (!looks_like_replicated)
return UUIDHelpers::Nil;
ParserCreateQuery parser;
auto size = context.getSettingsRef().max_query_size;
auto depth = context.getSettingsRef().max_parser_depth;
ASTPtr query = parseQuery(parser, metadata, size, depth);
const ASTCreateQuery & create = query->as<const ASTCreateQuery &>();
if (!create.storage || !create.storage->engine)
return UUIDHelpers::Nil;
if (!startsWith(create.storage->engine->name, "Replicated") || !endsWith(create.storage->engine->name, "MergeTree"))
return UUIDHelpers::Nil;
assert(create.uuid != UUIDHelpers::Nil);
return create.uuid;
}
void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeeper, UInt32 our_log_ptr, UInt32 max_log_ptr)
{
/// Let's compare local (possibly outdated) metadata with (most actual) metadata stored in ZooKeeper
/// and try to update the set of local tables.
/// We could drop all local tables and create the new ones just like it's new replica.
/// But it will cause all ReplicatedMergeTree tables to fetch all data parts again and data in other tables will be lost.
bool new_replica = our_log_ptr == 0;
if (new_replica)
LOG_INFO(log, "Will create new replica from log pointer {}", max_log_ptr);
else
LOG_WARNING(log, "Will recover replica with staled log pointer {} from log pointer {}", our_log_ptr, max_log_ptr);
if (new_replica && !empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "It's new replica, but database is not empty");
auto table_name_to_metadata = tryGetConsistentMetadataSnapshot(current_zookeeper, max_log_ptr);
/// For ReplicatedMergeTree tables we can compare only UUIDs to ensure that it's the same table.
/// Metadata can be different, it's handled on table replication level.
/// We need to handle renamed tables only.
/// TODO maybe we should also update MergeTree SETTINGS if required?
std::unordered_map<UUID, String> zk_replicated_id_to_name;
for (const auto & zk_table : table_name_to_metadata)
{
UUID zk_replicated_id = getTableUUIDIfReplicated(zk_table.second, global_context);
if (zk_replicated_id != UUIDHelpers::Nil)
zk_replicated_id_to_name.emplace(zk_replicated_id, zk_table.first);
}
/// We will drop or move tables which exist only in local metadata
Strings tables_to_detach;
std::vector<std::pair<String, String>> replicated_tables_to_rename;
size_t total_tables = 0;
std::vector<UUID> replicated_ids;
for (auto existing_tables_it = getTablesIterator(global_context, {}); existing_tables_it->isValid(); existing_tables_it->next(), ++total_tables)
{
String name = existing_tables_it->name();
UUID local_replicated_id = UUIDHelpers::Nil;
if (existing_tables_it->table()->supportsReplication())
{
/// Check if replicated tables have the same UUID
local_replicated_id = existing_tables_it->table()->getStorageID().uuid;
auto it = zk_replicated_id_to_name.find(local_replicated_id);
if (it != zk_replicated_id_to_name.end())
{
if (name != it->second)
{
/// Need just update table name
replicated_tables_to_rename.emplace_back(name, it->second);
}
continue;
}
}
auto in_zk = table_name_to_metadata.find(name);
if (in_zk == table_name_to_metadata.end() || in_zk->second != readMetadataFile(name))
{
/// Local table does not exits in ZooKeeper or has different metadata
tables_to_detach.emplace_back(std::move(name));
}
}
String db_name = getDatabaseName();
String to_db_name = getDatabaseName() + BROKEN_TABLES_SUFFIX;
if (total_tables * db_settings.max_broken_tables_ratio < tables_to_detach.size())
throw Exception(ErrorCodes::DATABASE_REPLICATION_FAILED, "Too many tables to recreate: {} of {}", tables_to_detach.size(), total_tables);
else if (!tables_to_detach.empty())
{
LOG_WARNING(log, "Will recreate {} broken tables to recover replica", tables_to_detach.size());
/// It's too dangerous to automatically drop tables, so we will move them to special database.
/// We use Ordinary engine for destination database, because it's the only way to discard table UUID
/// and make possible creation of new table with the same UUID.
String query = fmt::format("CREATE DATABASE IF NOT EXISTS {} ENGINE=Ordinary", backQuoteIfNeed(to_db_name));
Context query_context = global_context;
executeQuery(query, query_context, true);
}
size_t dropped_dicts = 0;
size_t moved_tables = 0;
std::vector<UUID> dropped_tables;
for (const auto & table_name : tables_to_detach)
{
DDLGuardPtr table_guard = DatabaseCatalog::instance().getDDLGuard(db_name, table_name);
if (getDatabaseName() != db_name)
throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database was renamed, will retry");
auto table = tryGetTable(table_name, global_context);
if (isDictionaryExist(table_name))
{
/// We can safely drop any dictionaries because they do not store data
LOG_DEBUG(log, "Will DROP DICTIONARY {}", backQuoteIfNeed(table_name));
DatabaseAtomic::removeDictionary(global_context, table_name);
++dropped_dicts;
}
else if (!table->storesDataOnDisk())
{
LOG_DEBUG(log, "Will DROP TABLE {}, because it does not store data on disk and can be safely dropped", backQuoteIfNeed(table_name));
dropped_tables.push_back(tryGetTableUUID(table_name));
table->shutdown();
DatabaseAtomic::dropTable(global_context, table_name, true);
}
else
{
/// Table probably stores some data. Let's move it to another database.
String to_name = fmt::format("{}_{}_{}", table_name, max_log_ptr, thread_local_rng() % 1000);
LOG_DEBUG(log, "Will RENAME TABLE {} TO {}.{}", backQuoteIfNeed(table_name), backQuoteIfNeed(to_db_name), backQuoteIfNeed(to_name));
assert(db_name < to_db_name);
DDLGuardPtr to_table_guard = DatabaseCatalog::instance().getDDLGuard(to_db_name, to_name);
auto to_db_ptr = DatabaseCatalog::instance().getDatabase(to_db_name);
DatabaseAtomic::renameTable(global_context, table_name, *to_db_ptr, to_name, false, false);
++moved_tables;
}
}
if (!tables_to_detach.empty())
LOG_WARNING(log, "Cleaned {} outdated objects: dropped {} dictionaries and {} tables, moved {} tables",
tables_to_detach.size(), dropped_dicts, dropped_tables.size(), moved_tables);
/// Now database is cleared from outdated tables, let's rename ReplicatedMergeTree tables to actual names
for (const auto & old_to_new : replicated_tables_to_rename)
{
const String & from = old_to_new.first;
const String & to = old_to_new.second;
LOG_DEBUG(log, "Will RENAME TABLE {} TO {}", backQuoteIfNeed(from), backQuoteIfNeed(to));
/// TODO Maybe we should do it in two steps: rename all tables to temporary names and then rename them to actual names?
DDLGuardPtr table_guard = DatabaseCatalog::instance().getDDLGuard(db_name, std::min(from, to));
DDLGuardPtr to_table_guard = DatabaseCatalog::instance().getDDLGuard(db_name, std::max(from, to));
DatabaseAtomic::renameTable(global_context, from, *this, to, false, false);
}
for (const auto & id : dropped_tables)
DatabaseCatalog::instance().waitTableFinallyDropped(id);
for (const auto & name_and_meta : table_name_to_metadata)
{
if (isTableExist(name_and_meta.first, global_context))
{
assert(name_and_meta.second == readMetadataFile(name_and_meta.first));
continue;
}
auto query_ast = parseQueryFromMetadataInZooKeeper(name_and_meta.first, name_and_meta.second);
Context query_context = global_context;
query_context.makeQueryContext();
query_context.getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY;
query_context.setCurrentDatabase(database_name);
query_context.setCurrentQueryId(""); // generate random query_id
LOG_INFO(log, "Executing {}", serializeAST(*query_ast));
InterpreterCreateQuery(query_ast, query_context).execute();
}
current_zookeeper->set(replica_path + "/log_ptr", toString(max_log_ptr));
}
std::map<String, String> DatabaseReplicated::tryGetConsistentMetadataSnapshot(const ZooKeeperPtr & zookeeper, UInt32 & max_log_ptr)
{
std::map<String, String> table_name_to_metadata;
constexpr int max_retries = 10;
int iteration = 0;
while (++iteration <= max_retries)
{
table_name_to_metadata.clear();
LOG_DEBUG(log, "Trying to get consistent metadata snapshot for log pointer {}", max_log_ptr);
Strings table_names = zookeeper->getChildren(zookeeper_path + "/metadata");
std::vector<zkutil::ZooKeeper::FutureGet> futures;
futures.reserve(table_names.size());
for (const auto & table : table_names)
futures.emplace_back(zookeeper->asyncTryGet(zookeeper_path + "/metadata/" + table));
for (size_t i = 0; i < table_names.size(); ++i)
{
auto res = futures[i].get();
if (res.error != Coordination::Error::ZOK)
break;
table_name_to_metadata.emplace(unescapeForFileName(table_names[i]), res.data);
}
UInt32 new_max_log_ptr = parse<UInt32>(zookeeper->get(zookeeper_path + "/max_log_ptr"));
if (new_max_log_ptr == max_log_ptr && table_names.size() == table_name_to_metadata.size())
break;
if (max_log_ptr < new_max_log_ptr)
{
LOG_DEBUG(log, "Log pointer moved from {} to {}, will retry", max_log_ptr, new_max_log_ptr);
max_log_ptr = new_max_log_ptr;
}
else
{
assert(max_log_ptr == new_max_log_ptr);
assert(table_names.size() != table_name_to_metadata.size());
LOG_DEBUG(log, "Cannot get metadata of some tables due to ZooKeeper error, will retry");
}
}
if (max_retries < iteration)
throw Exception(ErrorCodes::DATABASE_REPLICATION_FAILED, "Cannot get consistent metadata snapshot");
LOG_DEBUG(log, "Got consistent metadata snapshot for log pointer {}", max_log_ptr);
return table_name_to_metadata;
}
ASTPtr DatabaseReplicated::parseQueryFromMetadataInZooKeeper(const String & node_name, const String & query)
{
ParserCreateQuery parser;
String description = "in ZooKeeper " + zookeeper_path + "/metadata/" + node_name;
auto ast = parseQuery(parser, query, description, 0, global_context.getSettingsRef().max_parser_depth);
auto & create = ast->as<ASTCreateQuery &>();
if (create.uuid == UUIDHelpers::Nil || create.table != TABLE_WITH_UUID_NAME_PLACEHOLDER || ! create.database.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Got unexpected query from {}: {}", node_name, query);
create.database = getDatabaseName();
create.table = unescapeForFileName(node_name);
create.attach = false;
return ast;
}
void DatabaseReplicated::drop(const Context & context_)
{
auto current_zookeeper = getZooKeeper();
current_zookeeper->set(replica_path, DROPPED_MARK);
DatabaseAtomic::drop(context_);
current_zookeeper->tryRemoveRecursive(replica_path);
/// TODO it may leave garbage in ZooKeeper if the last node lost connection here
if (current_zookeeper->tryRemove(zookeeper_path + "/replicas") == Coordination::Error::ZOK)
{
/// It was the last replica, remove all metadata
current_zookeeper->tryRemoveRecursive(zookeeper_path);
}
}
void DatabaseReplicated::stopReplication()
{
if (ddl_worker)
ddl_worker->shutdown();
}
void DatabaseReplicated::shutdown()
{
stopReplication();
ddl_worker = nullptr;
DatabaseAtomic::shutdown();
}
void DatabaseReplicated::dropTable(const Context & context, const String & table_name, bool no_delay)
{
auto txn = context.getZooKeeperMetadataTransaction();
assert(!ddl_worker->isCurrentlyActive() || txn);
if (txn && txn->isInitialQuery())
{
String metadata_zk_path = zookeeper_path + "/metadata/" + escapeForFileName(table_name);
txn->addOp(zkutil::makeRemoveRequest(metadata_zk_path, -1));
}
DatabaseAtomic::dropTable(context, table_name, no_delay);
}
void DatabaseReplicated::renameTable(const Context & context, const String & table_name, IDatabase & to_database,
const String & to_table_name, bool exchange, bool dictionary)
{
auto txn = context.getZooKeeperMetadataTransaction();
assert(txn);
if (txn->isInitialQuery())
{
if (this != &to_database)
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Moving tables between databases is not supported for Replicated engine");
if (table_name == to_table_name)
throw Exception(ErrorCodes::INCORRECT_QUERY, "Cannot rename table to itself");
if (!isTableExist(table_name, context))
throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} does not exist", table_name);
if (exchange && !to_database.isTableExist(to_table_name, context))
throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} does not exist", to_table_name);
String statement = readMetadataFile(table_name);
String metadata_zk_path = zookeeper_path + "/metadata/" + escapeForFileName(table_name);
String metadata_zk_path_to = zookeeper_path + "/metadata/" + escapeForFileName(to_table_name);
txn->addOp(zkutil::makeRemoveRequest(metadata_zk_path, -1));
if (exchange)
{
String statement_to = readMetadataFile(to_table_name);
txn->addOp(zkutil::makeRemoveRequest(metadata_zk_path_to, -1));
txn->addOp(zkutil::makeCreateRequest(metadata_zk_path, statement_to, zkutil::CreateMode::Persistent));
}
txn->addOp(zkutil::makeCreateRequest(metadata_zk_path_to, statement, zkutil::CreateMode::Persistent));
}
DatabaseAtomic::renameTable(context, table_name, to_database, to_table_name, exchange, dictionary);
}
void DatabaseReplicated::commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table,
const String & table_metadata_tmp_path, const String & table_metadata_path,
const Context & query_context)
{
auto txn = query_context.getZooKeeperMetadataTransaction();
assert(!ddl_worker->isCurrentlyActive() || txn);
if (txn && txn->isInitialQuery())
{
String metadata_zk_path = zookeeper_path + "/metadata/" + escapeForFileName(query.table);
String statement = getObjectDefinitionFromCreateQuery(query.clone());
/// zk::multi(...) will throw if `metadata_zk_path` exists
txn->addOp(zkutil::makeCreateRequest(metadata_zk_path, statement, zkutil::CreateMode::Persistent));
}
DatabaseAtomic::commitCreateTable(query, table, table_metadata_tmp_path, table_metadata_path, query_context);
}
void DatabaseReplicated::commitAlterTable(const StorageID & table_id,
const String & table_metadata_tmp_path, const String & table_metadata_path,
const String & statement, const Context & query_context)
{
auto txn = query_context.getZooKeeperMetadataTransaction();
if (txn && txn->isInitialQuery())
{
String metadata_zk_path = zookeeper_path + "/metadata/" + escapeForFileName(table_id.table_name);
txn->addOp(zkutil::makeSetRequest(metadata_zk_path, statement, -1));
}
DatabaseAtomic::commitAlterTable(table_id, table_metadata_tmp_path, table_metadata_path, statement, query_context);
}
void DatabaseReplicated::createDictionary(const Context & context,
const String & dictionary_name,
const ASTPtr & query)
{
auto txn = context.getZooKeeperMetadataTransaction();
assert(!ddl_worker->isCurrentlyActive() || txn);
if (txn && txn->isInitialQuery())
{
String metadata_zk_path = zookeeper_path + "/metadata/" + escapeForFileName(dictionary_name);
String statement = getObjectDefinitionFromCreateQuery(query->clone());
txn->addOp(zkutil::makeCreateRequest(metadata_zk_path, statement, zkutil::CreateMode::Persistent));
}
DatabaseAtomic::createDictionary(context, dictionary_name, query);
}
void DatabaseReplicated::removeDictionary(const Context & context, const String & dictionary_name)
{
auto txn = context.getZooKeeperMetadataTransaction();
assert(!ddl_worker->isCurrentlyActive() || txn);
if (txn && txn->isInitialQuery())
{
String metadata_zk_path = zookeeper_path + "/metadata/" + escapeForFileName(dictionary_name);
txn->addOp(zkutil::makeRemoveRequest(metadata_zk_path, -1));
}
DatabaseAtomic::removeDictionary(context, dictionary_name);
}
void DatabaseReplicated::detachTablePermanently(const Context & context, const String & table_name)
{
auto txn = context.getZooKeeperMetadataTransaction();
assert(!ddl_worker->isCurrentlyActive() || txn);
if (txn && txn->isInitialQuery())
{
String metadata_zk_path = zookeeper_path + "/metadata/" + escapeForFileName(table_name);
txn->addOp(zkutil::makeRemoveRequest(metadata_zk_path, -1));
}
DatabaseAtomic::detachTablePermanently(context, table_name);
}
String DatabaseReplicated::readMetadataFile(const String & table_name) const
{
String statement;
ReadBufferFromFile in(getObjectMetadataPath(table_name), 4096);
readStringUntilEOF(statement, in);
return statement;
}
}

View File

@ -0,0 +1,91 @@
#pragma once
#include <Databases/DatabaseAtomic.h>
#include <Databases/DatabaseReplicatedSettings.h>
#include <Common/ZooKeeper/ZooKeeper.h>
#include <Core/BackgroundSchedulePool.h>
#include <DataStreams/BlockIO.h>
#include <DataStreams/OneBlockInputStream.h>
#include <Interpreters/Context.h>
namespace DB
{
class DatabaseReplicatedDDLWorker;
using ZooKeeperPtr = std::shared_ptr<zkutil::ZooKeeper>;
class Cluster;
using ClusterPtr = std::shared_ptr<Cluster>;
class DatabaseReplicated : public DatabaseAtomic
{
public:
DatabaseReplicated(const String & name_, const String & metadata_path_, UUID uuid,
const String & zookeeper_path_, const String & shard_name_, const String & replica_name_,
DatabaseReplicatedSettings db_settings_,
const Context & context);
~DatabaseReplicated() override;
String getEngineName() const override { return "Replicated"; }
/// If current query is initial, then the following methods add metadata updating ZooKeeper operations to current ZooKeeperMetadataTransaction.
void dropTable(const Context &, const String & table_name, bool no_delay) override;
void renameTable(const Context & context, const String & table_name, IDatabase & to_database,
const String & to_table_name, bool exchange, bool dictionary) override;
void commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table,
const String & table_metadata_tmp_path, const String & table_metadata_path,
const Context & query_context) override;
void commitAlterTable(const StorageID & table_id,
const String & table_metadata_tmp_path, const String & table_metadata_path,
const String & statement, const Context & query_context) override;
void createDictionary(const Context & context,
const String & dictionary_name,
const ASTPtr & query) override;
void removeDictionary(const Context & context, const String & dictionary_name) override;
void detachTablePermanently(const Context & context, const String & table_name) override;
/// Try to execute DLL query on current host as initial query. If query is succeed,
/// then it will be executed on all replicas.
BlockIO tryEnqueueReplicatedDDL(const ASTPtr & query, const Context & query_context);
void stopReplication();
String getFullReplicaName() const;
static std::pair<String, String> parseFullReplicaName(const String & name);
/// Returns cluster consisting of database replicas
ClusterPtr getCluster() const;
void drop(const Context & /*context*/) override;
void loadStoredObjects(Context & context, bool has_force_restore_data_flag, bool force_attach) override;
void shutdown() override;
friend struct DatabaseReplicatedTask;
friend class DatabaseReplicatedDDLWorker;
private:
void tryConnectToZooKeeperAndInitDatabase(bool force_attach);
bool createDatabaseNodesInZooKeeper(const ZooKeeperPtr & current_zookeeper);
void createReplicaNodesInZooKeeper(const ZooKeeperPtr & current_zookeeper);
void recoverLostReplica(const ZooKeeperPtr & current_zookeeper, UInt32 our_log_ptr, UInt32 max_log_ptr);
std::map<String, String> tryGetConsistentMetadataSnapshot(const ZooKeeperPtr & zookeeper, UInt32 & max_log_ptr);
ASTPtr parseQueryFromMetadataInZooKeeper(const String & node_name, const String & query);
String readMetadataFile(const String & table_name) const;
String zookeeper_path;
String shard_name;
String replica_name;
String replica_path;
DatabaseReplicatedSettings db_settings;
zkutil::ZooKeeperPtr getZooKeeper() const;
std::atomic_bool is_readonly = true;
std::unique_ptr<DatabaseReplicatedDDLWorker> ddl_worker;
};
}

View File

@ -0,0 +1,23 @@
#include <Databases/DatabaseReplicatedSettings.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTCreateQuery.h>
namespace DB
{
IMPLEMENT_SETTINGS_TRAITS(DatabaseReplicatedSettingsTraits, LIST_OF_DATABASE_REPLICATED_SETTINGS)
void DatabaseReplicatedSettings::loadFromQuery(ASTStorage & storage_def)
{
if (storage_def.settings)
{
applyChanges(storage_def.settings->changes);
return;
}
auto settings_ast = std::make_shared<ASTSetQuery>();
settings_ast->is_standalone = false;
storage_def.set(storage_def.settings, settings_ast);
}
}

View File

@ -0,0 +1,26 @@
#pragma once
#include <Core/Defines.h>
#include <Core/BaseSettings.h>
namespace DB
{
class ASTStorage;
#define LIST_OF_DATABASE_REPLICATED_SETTINGS(M) \
M(Float, max_broken_tables_ratio, 0.5, "Do not recover replica automatically if the ratio of staled tables to all tables is greater", 0) \
M(UInt64, max_replication_lag_to_enqueue, 10, "Replica will throw exception on attempt to execute query if its replication lag greater", 0) \
M(UInt64, wait_entry_commited_timeout_sec, 3600, "Replicas will try to cancel query if timeout exceed, but initiator host has not executed it yet", 0) \
DECLARE_SETTINGS_TRAITS(DatabaseReplicatedSettingsTraits, LIST_OF_DATABASE_REPLICATED_SETTINGS)
/** Settings for the MaterializeMySQL database engine.
* Could be loaded from a CREATE DATABASE query (SETTINGS clause).
*/
struct DatabaseReplicatedSettings : public BaseSettings<DatabaseReplicatedSettingsTraits>
{
void loadFromQuery(ASTStorage & storage_def);
};
}

View File

@ -0,0 +1,260 @@
#include <Databases/DatabaseReplicatedWorker.h>
#include <Databases/DatabaseReplicated.h>
#include <Interpreters/DDLTask.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int DATABASE_REPLICATION_FAILED;
extern const int NOT_A_LEADER;
extern const int UNFINISHED;
}
DatabaseReplicatedDDLWorker::DatabaseReplicatedDDLWorker(DatabaseReplicated * db, const Context & context_)
: DDLWorker(/* pool_size */ 1, db->zookeeper_path + "/log", context_, nullptr, {}, fmt::format("DDLWorker({})", db->getDatabaseName()))
, database(db)
{
/// Pool size must be 1 to avoid reordering of log entries.
/// TODO Make a dependency graph of DDL queries. It will allow to execute independent entries in parallel.
/// We also need similar graph to load tables on server startup in order of topsort.
}
void DatabaseReplicatedDDLWorker::initializeMainThread()
{
while (!stop_flag)
{
try
{
auto zookeeper = getAndSetZooKeeper();
if (database->is_readonly)
database->tryConnectToZooKeeperAndInitDatabase(false);
initializeReplication();
initialized = true;
return;
}
catch (...)
{
tryLogCurrentException(log, fmt::format("Error on initialization of {}", database->getDatabaseName()));
sleepForSeconds(5);
}
}
}
void DatabaseReplicatedDDLWorker::shutdown()
{
DDLWorker::shutdown();
wait_current_task_change.notify_all();
}
void DatabaseReplicatedDDLWorker::initializeReplication()
{
/// Check if we need to recover replica.
/// Invariant: replica is lost if it's log_ptr value is less then max_log_ptr - logs_to_keep.
String log_ptr_str = current_zookeeper->get(database->replica_path + "/log_ptr");
UInt32 our_log_ptr = parse<UInt32>(log_ptr_str);
UInt32 max_log_ptr = parse<UInt32>(current_zookeeper->get(database->zookeeper_path + "/max_log_ptr"));
logs_to_keep = parse<UInt32>(current_zookeeper->get(database->zookeeper_path + "/logs_to_keep"));
if (our_log_ptr == 0 || our_log_ptr + logs_to_keep < max_log_ptr)
database->recoverLostReplica(current_zookeeper, our_log_ptr, max_log_ptr);
else
last_skipped_entry_name.emplace(log_ptr_str);
}
String DatabaseReplicatedDDLWorker::enqueueQuery(DDLLogEntry & entry)
{
auto zookeeper = getAndSetZooKeeper();
const String query_path_prefix = queue_dir + "/query-";
/// We cannot create sequential node and it's ephemeral child in a single transaction, so allocate sequential number another way
String counter_prefix = database->zookeeper_path + "/counter/cnt-";
String counter_path = zookeeper->create(counter_prefix, "", zkutil::CreateMode::EphemeralSequential);
String node_path = query_path_prefix + counter_path.substr(counter_prefix.size());
Coordination::Requests ops;
/// Query is not committed yet, but we have to write it into log to avoid reordering
ops.emplace_back(zkutil::makeCreateRequest(node_path, entry.toString(), zkutil::CreateMode::Persistent));
/// '/try' will be replaced with '/committed' or will be removed due to expired session or other error
ops.emplace_back(zkutil::makeCreateRequest(node_path + "/try", database->getFullReplicaName(), zkutil::CreateMode::Ephemeral));
/// We don't need it anymore
ops.emplace_back(zkutil::makeRemoveRequest(counter_path, -1));
/// Create status dirs
ops.emplace_back(zkutil::makeCreateRequest(node_path + "/active", "", zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(node_path + "/finished", "", zkutil::CreateMode::Persistent));
zookeeper->multi(ops);
return node_path;
}
String DatabaseReplicatedDDLWorker::tryEnqueueAndExecuteEntry(DDLLogEntry & entry, const Context & query_context)
{
/// NOTE Possibly it would be better to execute initial query on the most up-to-date node,
/// but it requires more complex logic around /try node.
auto zookeeper = getAndSetZooKeeper();
UInt32 our_log_ptr = parse<UInt32>(zookeeper->get(database->replica_path + "/log_ptr"));
UInt32 max_log_ptr = parse<UInt32>(zookeeper->get(database->zookeeper_path + "/max_log_ptr"));
assert(our_log_ptr <= max_log_ptr);
if (database->db_settings.max_replication_lag_to_enqueue < max_log_ptr - our_log_ptr)
throw Exception(ErrorCodes::NOT_A_LEADER, "Cannot enqueue query on this replica, "
"because it has replication lag of {} queries. Try other replica.", max_log_ptr - our_log_ptr);
String entry_path = enqueueQuery(entry);
auto try_node = zkutil::EphemeralNodeHolder::existing(entry_path + "/try", *zookeeper);
String entry_name = entry_path.substr(entry_path.rfind('/') + 1);
auto task = std::make_unique<DatabaseReplicatedTask>(entry_name, entry_path, database);
task->entry = entry;
task->parseQueryFromEntry(context);
assert(!task->entry.query.empty());
assert(!zookeeper->exists(task->getFinishedNodePath()));
task->is_initial_query = true;
LOG_DEBUG(log, "Waiting for worker thread to process all entries before {}", entry_name);
UInt64 timeout = query_context.getSettingsRef().database_replicated_initial_query_timeout_sec;
{
std::unique_lock lock{mutex};
bool processed = wait_current_task_change.wait_for(lock, std::chrono::seconds(timeout), [&]()
{
assert(zookeeper->expired() || current_task <= entry_name);
return zookeeper->expired() || current_task == entry_name || stop_flag;
});
if (!processed)
throw Exception(ErrorCodes::UNFINISHED, "Timeout: Cannot enqueue query on this replica,"
"most likely because replica is busy with previous queue entries");
}
if (zookeeper->expired() || stop_flag)
throw Exception(ErrorCodes::DATABASE_REPLICATION_FAILED, "ZooKeeper session expired or replication stopped, try again");
processTask(*task, zookeeper);
if (!task->was_executed)
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "Entry {} was executed, but was not committed: code {}: {}",
task->execution_status.code, task->execution_status.message);
}
try_node->setAlreadyRemoved();
return entry_path;
}
DDLTaskPtr DatabaseReplicatedDDLWorker::initAndCheckTask(const String & entry_name, String & out_reason, const ZooKeeperPtr & zookeeper)
{
{
std::lock_guard lock{mutex};
if (current_task < entry_name)
{
current_task = entry_name;
wait_current_task_change.notify_all();
}
}
UInt32 our_log_ptr = parse<UInt32>(current_zookeeper->get(database->replica_path + "/log_ptr"));
UInt32 entry_num = DatabaseReplicatedTask::getLogEntryNumber(entry_name);
if (entry_num <= our_log_ptr)
{
out_reason = fmt::format("Task {} already executed according to log pointer {}", entry_name, our_log_ptr);
return {};
}
String entry_path = queue_dir + "/" + entry_name;
auto task = std::make_unique<DatabaseReplicatedTask>(entry_name, entry_path, database);
String initiator_name;
zkutil::EventPtr wait_committed_or_failed = std::make_shared<Poco::Event>();
String try_node_path = entry_path + "/try";
if (zookeeper->tryGet(try_node_path, initiator_name, nullptr, wait_committed_or_failed))
{
task->is_initial_query = initiator_name == task->host_id_str;
/// Query is not committed yet. We cannot just skip it and execute next one, because reordering may break replication.
LOG_TRACE(log, "Waiting for initiator {} to commit or rollback entry {}", initiator_name, entry_path);
constexpr size_t wait_time_ms = 1000;
size_t max_iterations = database->db_settings.wait_entry_commited_timeout_sec;
size_t iteration = 0;
while (!wait_committed_or_failed->tryWait(wait_time_ms))
{
if (stop_flag)
{
/// We cannot return task to process and we cannot return nullptr too,
/// because nullptr means "task should not be executed".
/// We can only exit by exception.
throw Exception(ErrorCodes::UNFINISHED, "Replication was stopped");
}
if (max_iterations <= ++iteration)
{
/// What can we do if initiator hangs for some reason? Seems like we can remove /try node.
/// Initiator will fail to commit ZooKeeperMetadataTransaction (including ops for replicated table) if /try does not exist.
/// But it's questionable.
/// We use tryRemove(...) because multiple hosts (including initiator) may try to do it concurrently.
auto code = zookeeper->tryRemove(try_node_path);
if (code != Coordination::Error::ZOK && code != Coordination::Error::ZNONODE)
throw Coordination::Exception(code, try_node_path);
if (!zookeeper->exists(entry_path + "/committed"))
{
out_reason = fmt::format("Entry {} was forcefully cancelled due to timeout", entry_name);
return {};
}
}
}
}
if (!zookeeper->exists(entry_path + "/committed"))
{
out_reason = fmt::format("Entry {} hasn't been committed", entry_name);
return {};
}
if (task->is_initial_query)
{
assert(!zookeeper->exists(entry_path + "/try"));
assert(zookeeper->exists(entry_path + "/committed") == (zookeeper->get(task->getFinishedNodePath()) == ExecutionStatus(0).serializeText()));
out_reason = fmt::format("Entry {} has been executed as initial query", entry_name);
return {};
}
String node_data;
if (!zookeeper->tryGet(entry_path, node_data))
{
LOG_ERROR(log, "Cannot get log entry {}", entry_path);
throw Exception(ErrorCodes::LOGICAL_ERROR, "should be unreachable");
}
task->entry.parse(node_data);
if (task->entry.query.empty())
{
out_reason = fmt::format("Entry {} is a dummy task", entry_name);
return {};
}
task->parseQueryFromEntry(context);
if (zookeeper->exists(task->getFinishedNodePath()))
{
out_reason = fmt::format("Task {} has been already processed", entry_name);
return {};
}
return task;
}
bool DatabaseReplicatedDDLWorker::canRemoveQueueEntry(const String & entry_name, const Coordination::Stat &)
{
UInt32 entry_number = DDLTaskBase::getLogEntryNumber(entry_name);
UInt32 max_log_ptr = parse<UInt32>(getAndSetZooKeeper()->get(database->zookeeper_path + "/max_log_ptr"));
return entry_number + logs_to_keep < max_log_ptr;
}
}

View File

@ -0,0 +1,46 @@
#pragma once
#include <Interpreters/DDLWorker.h>
namespace DB
{
class DatabaseReplicated;
/// It's similar to DDLWorker, but has the following differences:
/// 1. DDL queue in ZooKeeper is not shared between multiple clusters and databases,
/// each DatabaseReplicated has its own queue in ZooKeeper and DatabaseReplicatedDDLWorker object.
/// 2. Shards and replicas are identified by shard_name and replica_name arguments of database engine,
/// not by address:port pairs. Cluster (of multiple database replicas) is identified by its zookeeper_path.
/// 3. After creation of an entry in DDL queue initiator tries to execute the entry locally
/// and other hosts wait for query to finish on initiator host.
/// If query succeed on initiator, then all hosts must execute it, so they will retry until query succeed.
/// We assume that cluster is homogeneous, so if replicas are in consistent state and query succeed on one host,
/// then all hosts can execute it (maybe after several retries).
/// 4. Each database replica stores its log pointer in ZooKeeper. Cleanup thread removes old entry
/// if its number < max_log_ptr - logs_to_keep.
class DatabaseReplicatedDDLWorker : public DDLWorker
{
public:
DatabaseReplicatedDDLWorker(DatabaseReplicated * db, const Context & context_);
String enqueueQuery(DDLLogEntry & entry) override;
String tryEnqueueAndExecuteEntry(DDLLogEntry & entry, const Context & query_context);
void shutdown() override;
private:
void initializeMainThread() override;
void initializeReplication();
DDLTaskPtr initAndCheckTask(const String & entry_name, String & out_reason, const ZooKeeperPtr & zookeeper) override;
bool canRemoveQueueEntry(const String & entry_name, const Coordination::Stat & stat) override;
DatabaseReplicated * const database;
mutable std::mutex mutex;
std::condition_variable wait_current_task_change;
String current_task;
UInt32 logs_to_keep = std::numeric_limits<UInt32>::max();
};
}

View File

@ -4,6 +4,7 @@
#include <Interpreters/ExternalDictionariesLoader.h>
#include <Interpreters/ExternalLoaderTempConfigRepository.h>
#include <Interpreters/ExternalLoaderDatabaseConfigRepository.h>
#include <Interpreters/DDLTask.h>
#include <Dictionaries/getDictionaryConfigurationFromAST.h>
#include <Dictionaries/DictionaryStructure.h>
#include <Parsers/ASTCreateQuery.h>
@ -193,6 +194,10 @@ void DatabaseWithDictionaries::createDictionary(const Context & context, const S
detachDictionary(dictionary_name);
});
auto txn = context.getZooKeeperMetadataTransaction();
if (txn && !context.isInternalSubquery())
txn->commit(); /// Commit point (a sort of) for Replicated database
/// If it was ATTACH query and file with dictionary metadata already exist
/// (so, ATTACH is done after DETACH), then rename atomically replaces old file with new one.
Poco::File(dictionary_metadata_tmp_path).renameTo(dictionary_metadata_path);
@ -205,7 +210,7 @@ void DatabaseWithDictionaries::createDictionary(const Context & context, const S
succeeded = true;
}
void DatabaseWithDictionaries::removeDictionary(const Context &, const String & dictionary_name)
void DatabaseWithDictionaries::removeDictionary(const Context & context, const String & dictionary_name)
{
DictionaryAttachInfo attach_info;
detachDictionaryImpl(dictionary_name, attach_info);
@ -213,6 +218,11 @@ void DatabaseWithDictionaries::removeDictionary(const Context &, const String &
try
{
String dictionary_metadata_path = getObjectMetadataPath(dictionary_name);
auto txn = context.getZooKeeperMetadataTransaction();
if (txn && !context.isInternalSubquery())
txn->commit(); /// Commit point (a sort of) for Replicated database
Poco::File(dictionary_metadata_path).remove();
CurrentStatusInfo::unset(CurrentStatusInfo::DictionaryStatus,
StorageID(attach_info.create_query).getInternalDictionaryName());

View File

@ -249,7 +249,7 @@ public:
/// Forget about the table without deleting it's data, but rename metadata file to prevent reloading it
/// with next restart. The database may not support this method.
virtual void detachTablePermanently(const String & /*name*/)
virtual void detachTablePermanently(const Context & /*context*/, const String & /*name*/)
{
throw Exception("There is no DETACH TABLE PERMANENTLY query for Database" + getEngineName(), ErrorCodes::NOT_IMPLEMENTED);
}

View File

@ -395,7 +395,7 @@ void DatabaseConnectionMySQL::loadStoredObjects(Context &, bool, bool /*force_at
}
}
void DatabaseConnectionMySQL::detachTablePermanently(const String & table_name)
void DatabaseConnectionMySQL::detachTablePermanently(const Context &, const String & table_name)
{
std::lock_guard<std::mutex> lock{mutex};
@ -429,9 +429,9 @@ void DatabaseConnectionMySQL::detachTablePermanently(const String & table_name)
table_iter->second.second->is_dropped = true;
}
void DatabaseConnectionMySQL::dropTable(const Context &, const String & table_name, bool /*no_delay*/)
void DatabaseConnectionMySQL::dropTable(const Context & context, const String & table_name, bool /*no_delay*/)
{
detachTablePermanently(table_name);
detachTablePermanently(context, table_name);
}
DatabaseConnectionMySQL::~DatabaseConnectionMySQL()

View File

@ -72,9 +72,9 @@ public:
StoragePtr detachTable(const String & table_name) override;
void detachTablePermanently(const String & table_name) override;
void detachTablePermanently(const Context & context, const String & table_name) override;
void dropTable(const Context &, const String & table_name, bool no_delay) override;
void dropTable(const Context & context, const String & table_name, bool no_delay) override;
void attachTable(const String & table_name, const StoragePtr & storage, const String & relative_table_path) override;

View File

@ -16,6 +16,9 @@ SRCS(
DatabaseMemory.cpp
DatabaseOnDisk.cpp
DatabaseOrdinary.cpp
DatabaseReplicated.cpp
DatabaseReplicatedSettings.cpp
DatabaseReplicatedWorker.cpp
DatabaseWithDictionaries.cpp
DatabasesCommon.cpp
MySQL/ConnectionMySQLSettings.cpp

View File

@ -342,6 +342,7 @@ struct ContextShared
ReplicatedFetchList replicated_fetch_list;
ConfigurationPtr users_config; /// Config with the users, profiles and quotas sections.
InterserverIOHandler interserver_io_handler; /// Handler for interserver communication.
mutable std::optional<BackgroundSchedulePool> buffer_flush_schedule_pool; /// A thread pool that can do background flush for Buffer tables.
mutable std::optional<BackgroundSchedulePool> schedule_pool; /// A thread pool that can run different jobs in background (used in replicated tables)
mutable std::optional<BackgroundSchedulePool> distributed_schedule_pool; /// A thread pool that can run different jobs in background (used for distributed sends)
@ -1552,6 +1553,7 @@ void Context::setDDLWorker(std::unique_ptr<DDLWorker> ddl_worker)
auto lock = getLock();
if (shared->ddl_worker)
throw Exception("DDL background thread has already been initialized", ErrorCodes::LOGICAL_ERROR);
ddl_worker->startup();
shared->ddl_worker = std::move(ddl_worker);
}
@ -2551,6 +2553,19 @@ StorageID Context::resolveStorageIDImpl(StorageID storage_id, StorageNamespace w
return StorageID::createEmpty();
}
void Context::initZooKeeperMetadataTransaction(ZooKeeperMetadataTransactionPtr txn, [[maybe_unused]] bool attach_existing)
{
assert(!metadata_transaction);
assert(attach_existing || query_context == this);
metadata_transaction = std::move(txn);
}
ZooKeeperMetadataTransactionPtr Context::getZooKeeperMetadataTransaction() const
{
assert(!metadata_transaction || hasQueryContext());
return metadata_transaction;
}
PartUUIDsPtr Context::getPartUUIDs()
{
auto lock = getLock();

View File

@ -117,6 +117,8 @@ using VolumePtr = std::shared_ptr<IVolume>;
struct NamedSession;
struct BackgroundTaskSchedulingSettings;
class ZooKeeperMetadataTransaction;
using ZooKeeperMetadataTransactionPtr = std::shared_ptr<ZooKeeperMetadataTransaction>;
#if USE_EMBEDDED_COMPILER
class CompiledExpressionCache;
@ -279,6 +281,12 @@ private:
/// to be customized in HTTP and TCP servers by overloading the customizeContext(DB::Context&)
/// methods.
ZooKeeperMetadataTransactionPtr metadata_transaction; /// Distributed DDL context. I'm not sure if it's a suitable place for this,
/// but it's the easiest way to pass this through the whole stack from executeQuery(...)
/// to DatabaseOnDisk::commitCreateTable(...) or IStorage::alter(...) without changing
/// thousands of signatures.
/// And I hope it will be replaced with more common Transaction sometime.
/// Use copy constructor or createGlobal() instead
Context();
@ -534,6 +542,7 @@ public:
const Context & getQueryContext() const;
Context & getQueryContext();
bool hasQueryContext() const { return query_context != nullptr; }
bool isInternalSubquery() const { return hasQueryContext() && query_context != this; }
const Context & getSessionContext() const;
Context & getSessionContext();
@ -737,6 +746,11 @@ public:
IHostContextPtr & getHostContext();
const IHostContextPtr & getHostContext() const;
/// Initialize context of distributed DDL query with Replicated database.
void initZooKeeperMetadataTransaction(ZooKeeperMetadataTransactionPtr txn, bool attach_existing = false);
/// Returns context of current distributed DDL query or nullptr.
ZooKeeperMetadataTransactionPtr getZooKeeperMetadataTransaction() const;
struct MySQLWireContext
{
uint8_t sequence_id = 0;

View File

@ -0,0 +1,344 @@
#include <Interpreters/DDLTask.h>
#include <Common/DNSResolver.h>
#include <Common/isLocalAddress.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <IO/Operators.h>
#include <IO/ReadBufferFromString.h>
#include <Poco/Net/NetException.h>
#include <common/logger_useful.h>
#include <Parsers/ParserQuery.h>
#include <Parsers/parseQuery.h>
#include <Parsers/ASTQueryWithOnCluster.h>
#include <Parsers/ASTQueryWithTableAndOutput.h>
#include <Databases/DatabaseReplicated.h>
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_FORMAT_VERSION;
extern const int UNKNOWN_TYPE_OF_QUERY;
extern const int INCONSISTENT_CLUSTER_DEFINITION;
}
HostID HostID::fromString(const String & host_port_str)
{
HostID res;
std::tie(res.host_name, res.port) = Cluster::Address::fromString(host_port_str);
return res;
}
bool HostID::isLocalAddress(UInt16 clickhouse_port) const
{
try
{
return DB::isLocalAddress(DNSResolver::instance().resolveAddress(host_name, port), clickhouse_port);
}
catch (const Poco::Net::NetException &)
{
/// Avoid "Host not found" exceptions
return false;
}
}
String DDLLogEntry::toString() const
{
WriteBufferFromOwnString wb;
Strings host_id_strings(hosts.size());
std::transform(hosts.begin(), hosts.end(), host_id_strings.begin(), HostID::applyToString);
auto version = CURRENT_VERSION;
wb << "version: " << version << "\n";
wb << "query: " << escape << query << "\n";
wb << "hosts: " << host_id_strings << "\n";
wb << "initiator: " << initiator << "\n";
return wb.str();
}
void DDLLogEntry::parse(const String & data)
{
ReadBufferFromString rb(data);
int version;
rb >> "version: " >> version >> "\n";
if (version != CURRENT_VERSION)
throw Exception(ErrorCodes::UNKNOWN_FORMAT_VERSION, "Unknown DDLLogEntry format version: {}", version);
Strings host_id_strings;
rb >> "query: " >> escape >> query >> "\n";
rb >> "hosts: " >> host_id_strings >> "\n";
if (!rb.eof())
rb >> "initiator: " >> initiator >> "\n";
else
initiator.clear();
assertEOF(rb);
hosts.resize(host_id_strings.size());
std::transform(host_id_strings.begin(), host_id_strings.end(), hosts.begin(), HostID::fromString);
}
void DDLTaskBase::parseQueryFromEntry(const Context & context)
{
const char * begin = entry.query.data();
const char * end = begin + entry.query.size();
ParserQuery parser_query(end);
String description;
query = parseQuery(parser_query, begin, end, description, 0, context.getSettingsRef().max_parser_depth);
}
std::unique_ptr<Context> DDLTaskBase::makeQueryContext(Context & from_context, const ZooKeeperPtr & /*zookeeper*/)
{
auto query_context = std::make_unique<Context>(from_context);
query_context->makeQueryContext();
query_context->setCurrentQueryId(""); // generate random query_id
query_context->getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY;
return query_context;
}
bool DDLTask::findCurrentHostID(const Context & global_context, Poco::Logger * log)
{
bool host_in_hostlist = false;
for (const HostID & host : entry.hosts)
{
auto maybe_secure_port = global_context.getTCPPortSecure();
/// The port is considered local if it matches TCP or TCP secure port that the server is listening.
bool is_local_port = (maybe_secure_port && host.isLocalAddress(*maybe_secure_port))
|| host.isLocalAddress(global_context.getTCPPort());
if (!is_local_port)
continue;
if (host_in_hostlist)
{
/// This check could be slow a little bit
LOG_WARNING(log, "There are two the same ClickHouse instances in task {}: {} and {}. Will use the first one only.",
entry_name, host_id.readableString(), host.readableString());
}
else
{
host_in_hostlist = true;
host_id = host;
host_id_str = host.toString();
}
}
return host_in_hostlist;
}
void DDLTask::setClusterInfo(const Context & context, Poco::Logger * log)
{
auto * query_on_cluster = dynamic_cast<ASTQueryWithOnCluster *>(query.get());
if (!query_on_cluster)
throw Exception("Received unknown DDL query", ErrorCodes::UNKNOWN_TYPE_OF_QUERY);
cluster_name = query_on_cluster->cluster;
cluster = context.tryGetCluster(cluster_name);
if (!cluster)
throw Exception(ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION,
"DDL task {} contains current host {} in cluster {}, but there are no such cluster here.",
entry_name, host_id.readableString(), cluster_name);
/// Try to find host from task host list in cluster
/// At the first, try find exact match (host name and ports should be literally equal)
/// If the attempt fails, try find it resolving host name of each instance
if (!tryFindHostInCluster())
{
LOG_WARNING(log, "Not found the exact match of host {} from task {} in cluster {} definition. Will try to find it using host name resolving.",
host_id.readableString(), entry_name, cluster_name);
if (!tryFindHostInClusterViaResolving(context))
throw Exception(ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION, "Not found host {} in definition of cluster {}",
host_id.readableString(), cluster_name);
LOG_INFO(log, "Resolved host {} from task {} as host {} in definition of cluster {}",
host_id.readableString(), entry_name, address_in_cluster.readableString(), cluster_name);
}
query = query_on_cluster->getRewrittenASTWithoutOnCluster(address_in_cluster.default_database);
query_on_cluster = nullptr;
}
bool DDLTask::tryFindHostInCluster()
{
const auto & shards = cluster->getShardsAddresses();
bool found_exact_match = false;
String default_database;
for (size_t shard_num = 0; shard_num < shards.size(); ++shard_num)
{
for (size_t replica_num = 0; replica_num < shards[shard_num].size(); ++replica_num)
{
const Cluster::Address & address = shards[shard_num][replica_num];
if (address.host_name == host_id.host_name && address.port == host_id.port)
{
if (found_exact_match)
{
if (default_database == address.default_database)
{
throw Exception(ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION,
"There are two exactly the same ClickHouse instances {} in cluster {}",
address.readableString(), cluster_name);
}
else
{
/* Circular replication is used.
* It is when every physical node contains
* replicas of different shards of the same table.
* To distinguish one replica from another on the same node,
* every shard is placed into separate database.
* */
is_circular_replicated = true;
auto * query_with_table = dynamic_cast<ASTQueryWithTableAndOutput *>(query.get());
if (!query_with_table || query_with_table->database.empty())
{
throw Exception(ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION,
"For a distributed DDL on circular replicated cluster its table name must be qualified by database name.");
}
if (default_database == query_with_table->database)
return true;
}
}
found_exact_match = true;
host_shard_num = shard_num;
host_replica_num = replica_num;
address_in_cluster = address;
default_database = address.default_database;
}
}
}
return found_exact_match;
}
bool DDLTask::tryFindHostInClusterViaResolving(const Context & context)
{
const auto & shards = cluster->getShardsAddresses();
bool found_via_resolving = false;
for (size_t shard_num = 0; shard_num < shards.size(); ++shard_num)
{
for (size_t replica_num = 0; replica_num < shards[shard_num].size(); ++replica_num)
{
const Cluster::Address & address = shards[shard_num][replica_num];
if (auto resolved = address.getResolvedAddress();
resolved && (isLocalAddress(*resolved, context.getTCPPort())
|| (context.getTCPPortSecure() && isLocalAddress(*resolved, *context.getTCPPortSecure()))))
{
if (found_via_resolving)
{
throw Exception(ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION,
"There are two the same ClickHouse instances in cluster {} : {} and {}",
cluster_name, address_in_cluster.readableString(), address.readableString());
}
else
{
found_via_resolving = true;
host_shard_num = shard_num;
host_replica_num = replica_num;
address_in_cluster = address;
}
}
}
}
return found_via_resolving;
}
String DDLTask::getShardID() const
{
/// Generate unique name for shard node, it will be used to execute the query by only single host
/// Shard node name has format 'replica_name1,replica_name2,...,replica_nameN'
/// Where replica_name is 'replica_config_host_name:replica_port'
auto shard_addresses = cluster->getShardsAddresses().at(host_shard_num);
Strings replica_names;
for (const Cluster::Address & address : shard_addresses)
replica_names.emplace_back(address.readableString());
std::sort(replica_names.begin(), replica_names.end());
String res;
for (auto it = replica_names.begin(); it != replica_names.end(); ++it)
res += *it + (std::next(it) != replica_names.end() ? "," : "");
return res;
}
DatabaseReplicatedTask::DatabaseReplicatedTask(const String & name, const String & path, DatabaseReplicated * database_)
: DDLTaskBase(name, path)
, database(database_)
{
host_id_str = database->getFullReplicaName();
}
String DatabaseReplicatedTask::getShardID() const
{
return database->shard_name;
}
std::unique_ptr<Context> DatabaseReplicatedTask::makeQueryContext(Context & from_context, const ZooKeeperPtr & zookeeper)
{
auto query_context = DDLTaskBase::makeQueryContext(from_context, zookeeper);
query_context->getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY;
query_context->setCurrentDatabase(database->getDatabaseName());
auto txn = std::make_shared<ZooKeeperMetadataTransaction>(zookeeper, database->zookeeper_path, is_initial_query);
query_context->initZooKeeperMetadataTransaction(txn);
if (is_initial_query)
{
txn->addOp(zkutil::makeRemoveRequest(entry_path + "/try", -1));
txn->addOp(zkutil::makeCreateRequest(entry_path + "/committed", host_id_str, zkutil::CreateMode::Persistent));
txn->addOp(zkutil::makeSetRequest(database->zookeeper_path + "/max_log_ptr", toString(getLogEntryNumber(entry_name)), -1));
}
txn->addOp(zkutil::makeSetRequest(database->replica_path + "/log_ptr", toString(getLogEntryNumber(entry_name)), -1));
for (auto & op : ops)
txn->addOp(std::move(op));
ops.clear();
return query_context;
}
String DDLTaskBase::getLogEntryName(UInt32 log_entry_number)
{
constexpr size_t seq_node_digits = 10;
String number = toString(log_entry_number);
String name = "query-" + String(seq_node_digits - number.size(), '0') + number;
return name;
}
UInt32 DDLTaskBase::getLogEntryNumber(const String & log_entry_name)
{
constexpr const char * name = "query-";
assert(startsWith(log_entry_name, name));
return parse<UInt32>(log_entry_name.substr(strlen(name)));
}
void ZooKeeperMetadataTransaction::commit()
{
assert(state == CREATED);
state = FAILED;
current_zookeeper->multi(ops);
state = COMMITTED;
}
}

195
src/Interpreters/DDLTask.h Normal file
View File

@ -0,0 +1,195 @@
#pragma once
#include <Core/Types.h>
#include <Interpreters/Cluster.h>
#include <Common/ZooKeeper/Types.h>
namespace Poco
{
class Logger;
}
namespace zkutil
{
class ZooKeeper;
}
namespace DB
{
class ASTQueryWithOnCluster;
using ZooKeeperPtr = std::shared_ptr<zkutil::ZooKeeper>;
class DatabaseReplicated;
class ZooKeeperMetadataTransaction;
using ZooKeeperMetadataTransactionPtr = std::shared_ptr<ZooKeeperMetadataTransaction>;
struct HostID
{
String host_name;
UInt16 port;
HostID() = default;
explicit HostID(const Cluster::Address & address)
: host_name(address.host_name), port(address.port) {}
static HostID fromString(const String & host_port_str);
String toString() const
{
return Cluster::Address::toString(host_name, port);
}
String readableString() const
{
return host_name + ":" + DB::toString(port);
}
bool isLocalAddress(UInt16 clickhouse_port) const;
static String applyToString(const HostID & host_id)
{
return host_id.toString();
}
};
struct DDLLogEntry
{
String query;
std::vector<HostID> hosts;
String initiator; // optional
static constexpr int CURRENT_VERSION = 1;
String toString() const;
void parse(const String & data);
};
struct DDLTaskBase
{
const String entry_name;
const String entry_path;
DDLLogEntry entry;
String host_id_str;
ASTPtr query;
bool is_initial_query = false;
bool is_circular_replicated = false;
bool execute_on_leader = false;
Coordination::Requests ops;
ExecutionStatus execution_status;
bool was_executed = false;
std::atomic_bool completely_processed = false;
DDLTaskBase(const String & name, const String & path) : entry_name(name), entry_path(path) {}
DDLTaskBase(const DDLTaskBase &) = delete;
virtual ~DDLTaskBase() = default;
void parseQueryFromEntry(const Context & context);
virtual String getShardID() const = 0;
virtual std::unique_ptr<Context> makeQueryContext(Context & from_context, const ZooKeeperPtr & zookeeper);
inline String getActiveNodePath() const { return entry_path + "/active/" + host_id_str; }
inline String getFinishedNodePath() const { return entry_path + "/finished/" + host_id_str; }
inline String getShardNodePath() const { return entry_path + "/shards/" + getShardID(); }
static String getLogEntryName(UInt32 log_entry_number);
static UInt32 getLogEntryNumber(const String & log_entry_name);
};
struct DDLTask : public DDLTaskBase
{
DDLTask(const String & name, const String & path) : DDLTaskBase(name, path) {}
bool findCurrentHostID(const Context & global_context, Poco::Logger * log);
void setClusterInfo(const Context & context, Poco::Logger * log);
String getShardID() const override;
private:
bool tryFindHostInCluster();
bool tryFindHostInClusterViaResolving(const Context & context);
HostID host_id;
String cluster_name;
ClusterPtr cluster;
Cluster::Address address_in_cluster;
size_t host_shard_num;
size_t host_replica_num;
};
struct DatabaseReplicatedTask : public DDLTaskBase
{
DatabaseReplicatedTask(const String & name, const String & path, DatabaseReplicated * database_);
String getShardID() const override;
std::unique_ptr<Context> makeQueryContext(Context & from_context, const ZooKeeperPtr & zookeeper) override;
DatabaseReplicated * database;
};
/// The main purpose of ZooKeeperMetadataTransaction is to execute all zookeeper operation related to query
/// in a single transaction when we performed all required checks and ready to "commit" changes.
/// For example, create ALTER_METADATA entry in ReplicatedMergeTree log,
/// create path/to/entry/finished/host_id node in distributed DDL queue to mark query as executed and
/// update metadata in path/to/replicated_database/metadata/table_name
/// It's used for DatabaseReplicated.
/// TODO we can also use it for ordinary ON CLUSTER queries
class ZooKeeperMetadataTransaction
{
enum State
{
CREATED,
COMMITTED,
FAILED
};
State state = CREATED;
ZooKeeperPtr current_zookeeper;
String zookeeper_path;
bool is_initial_query;
Coordination::Requests ops;
public:
ZooKeeperMetadataTransaction(const ZooKeeperPtr & current_zookeeper_, const String & zookeeper_path_, bool is_initial_query_)
: current_zookeeper(current_zookeeper_)
, zookeeper_path(zookeeper_path_)
, is_initial_query(is_initial_query_)
{
}
bool isInitialQuery() const { return is_initial_query; }
bool isExecuted() const { return state != CREATED; }
String getDatabaseZooKeeperPath() const { return zookeeper_path; }
void addOp(Coordination::RequestPtr && op)
{
assert(!isExecuted());
ops.emplace_back(op);
}
void moveOpsTo(Coordination::Requests & other_ops)
{
assert(!isExecuted());
std::move(ops.begin(), ops.end(), std::back_inserter(other_ops));
ops.clear();
state = COMMITTED;
}
void commit();
~ZooKeeperMetadataTransaction() { assert(isExecuted() || std::uncaught_exception()); }
};
}

File diff suppressed because it is too large Load Diff

View File

@ -1,15 +1,11 @@
#pragma once
#include <DataStreams/BlockIO.h>
#include <Interpreters/Cluster.h>
#include <Interpreters/Context.h>
#include <Storages/IStorage_fwd.h>
#include <Poco/Net/NetException.h>
#include <Common/CurrentThread.h>
#include <Common/DNSResolver.h>
#include <Common/ThreadPool.h>
#include <Common/isLocalAddress.h>
#include <common/logger_useful.h>
#include <Storages/IStorage_fwd.h>
#include <Parsers/IAST_fwd.h>
#include <Interpreters/Context.h>
#include <atomic>
#include <chrono>
@ -19,90 +15,39 @@
namespace zkutil
{
class ZooKeeper;
class ZooKeeper;
}
namespace Poco
{
class Logger;
namespace Util { class AbstractConfiguration; }
}
namespace Coordination
{
struct Stat;
}
namespace DB
{
class Context;
class ASTAlterQuery;
struct DDLLogEntry;
struct DDLTaskBase;
using DDLTaskPtr = std::unique_ptr<DDLTaskBase>;
using ZooKeeperPtr = std::shared_ptr<zkutil::ZooKeeper>;
class AccessRightsElements;
struct HostID
{
String host_name;
UInt16 port;
HostID() = default;
explicit HostID(const Cluster::Address & address) : host_name(address.host_name), port(address.port) { }
static HostID fromString(const String & host_port_str)
{
HostID res;
std::tie(res.host_name, res.port) = Cluster::Address::fromString(host_port_str);
return res;
}
String toString() const { return Cluster::Address::toString(host_name, port); }
String readableString() const { return host_name + ":" + DB::toString(port); }
bool isLocalAddress(UInt16 clickhouse_port) const
{
try
{
return DB::isLocalAddress(DNSResolver::instance().resolveAddress(host_name, port), clickhouse_port);
}
catch (const Poco::Net::NetException &)
{
/// Avoid "Host not found" exceptions
return false;
}
}
static String applyToString(const HostID & host_id) { return host_id.toString(); }
};
struct DDLLogEntry
{
String query;
std::vector<HostID> hosts;
String initiator; // optional
static constexpr int CURRENT_VERSION = 1;
public:
String toString();
void parse(const String & data);
};
struct DDLTask;
using DDLTaskPtr = std::unique_ptr<DDLTask>;
/// Pushes distributed DDL query to the queue
BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, const Context & context);
BlockIO executeDDLQueryOnCluster(
const ASTPtr & query_ptr,
const Context & context,
const AccessRightsElements & query_requires_access,
bool query_requires_grant_option = false);
BlockIO executeDDLQueryOnCluster(
const ASTPtr & query_ptr,
const Context & context,
AccessRightsElements && query_requires_access,
bool query_requires_grant_option = false);
class DDLWorker
{
public:
DDLWorker(int pool_size_, const std::string & zk_root_dir, Context & context_, const Poco::Util::AbstractConfiguration * config, const String & prefix);
~DDLWorker();
DDLWorker(int pool_size_, const std::string & zk_root_dir, const Context & context_, const Poco::Util::AbstractConfiguration * config, const String & prefix,
const String & logger_name = "DDLWorker", const CurrentMetrics::Metric * max_entry_metric_ = nullptr);
virtual ~DDLWorker();
/// Pushes query into DDL queue, returns path to created node
String enqueueQuery(DDLLogEntry & entry);
virtual String enqueueQuery(DDLLogEntry & entry);
/// Host ID (name:port) for logging purposes
/// Note that in each task hosts are identified individually by name:port from initiator server cluster config
@ -111,30 +56,32 @@ public:
return host_fqdn_id;
}
private:
using ZooKeeperPtr = std::shared_ptr<zkutil::ZooKeeper>;
void startup();
virtual void shutdown();
bool isCurrentlyActive() const { return initialized && !stop_flag; }
protected:
/// Returns cached ZooKeeper session (possibly expired).
ZooKeeperPtr tryGetZooKeeper() const;
/// If necessary, creates a new session and caches it.
ZooKeeperPtr getAndSetZooKeeper();
/// ZooKeeper recover loop (while not stopped).
void recoverZooKeeper();
void checkCurrentTasks();
/// Iterates through queue tasks in ZooKeeper, runs execution of new tasks
void scheduleTasks();
void saveTask(const String & entry_name);
DDLTaskBase & saveTask(DDLTaskPtr && task);
/// Reads entry and check that the host belongs to host list of the task
/// Returns non-empty DDLTaskPtr if entry parsed and the check is passed
DDLTaskPtr initAndCheckTask(const String & entry_name, String & out_reason, const ZooKeeperPtr & zookeeper);
virtual DDLTaskPtr initAndCheckTask(const String & entry_name, String & out_reason, const ZooKeeperPtr & zookeeper);
void updateMaxDDLEntryID(const DDLTask & task);
void enqueueTask(DDLTaskPtr task);
void processTask(DDLTask & task);
void processTask(DDLTaskBase & task, const ZooKeeperPtr & zookeeper);
void updateMaxDDLEntryID(const String & entry_name);
/// Check that query should be executed on leader replica only
static bool taskShouldBeExecutedOnLeader(const ASTPtr ast_ddl, StoragePtr storage);
static bool taskShouldBeExecutedOnLeader(const ASTPtr & ast_ddl, StoragePtr storage);
/// Executes query only on leader replica in case of replicated table.
/// Queries like TRUNCATE/ALTER .../OPTIMIZE have to be executed only on one node of shard.
@ -142,27 +89,27 @@ private:
/// query via RemoteBlockOutputStream to leader, so to avoid such "2-phase" query execution we
/// execute query directly on leader.
bool tryExecuteQueryOnLeaderReplica(
DDLTask & task,
DDLTaskBase & task,
StoragePtr storage,
const String & rewritten_query,
const String & node_path,
const ZooKeeperPtr & zookeeper);
void parseQueryAndResolveHost(DDLTask & task);
bool tryExecuteQuery(const String & query, const DDLTask & task, ExecutionStatus & status);
bool tryExecuteQuery(const String & query, DDLTaskBase & task, const ZooKeeperPtr & zookeeper);
/// Checks and cleanups queue's nodes
void cleanupQueue(Int64 current_time_seconds, const ZooKeeperPtr & zookeeper);
virtual bool canRemoveQueueEntry(const String & entry_name, const Coordination::Stat & stat);
/// Init task node
static void createStatusDirs(const std::string & node_path, const ZooKeeperPtr & zookeeper);
void createStatusDirs(const std::string & node_path, const ZooKeeperPtr & zookeeper);
virtual void initializeMainThread();
void runMainThread();
void runCleanupThread();
private:
protected:
Context context;
Poco::Logger * log;
@ -174,10 +121,12 @@ private:
ZooKeeperPtr current_zookeeper;
/// Save state of executed task to avoid duplicate execution on ZK error
std::vector<std::string> last_tasks;
std::optional<String> last_skipped_entry_name;
std::list<DDLTaskPtr> current_tasks;
std::shared_ptr<Poco::Event> queue_updated_event = std::make_shared<Poco::Event>();
std::shared_ptr<Poco::Event> cleanup_event = std::make_shared<Poco::Event>();
std::atomic<bool> initialized = false;
std::atomic<bool> stop_flag = false;
ThreadFromGlobalPool main_thread;
@ -195,9 +144,7 @@ private:
size_t max_tasks_in_queue = 1000;
std::atomic<UInt64> max_id = 0;
friend class DDLQueryStatusInputStream;
friend struct DDLTask;
const CurrentMetrics::Metric * max_entry_metric;
};

View File

@ -609,7 +609,7 @@ DatabaseCatalog::updateDependency(const StorageID & old_from, const StorageID &
view_dependencies[{new_from.getDatabaseName(), new_from.getTableName()}].insert(new_where);
}
std::unique_ptr<DDLGuard> DatabaseCatalog::getDDLGuard(const String & database, const String & table)
DDLGuardPtr DatabaseCatalog::getDDLGuard(const String & database, const String & table)
{
std::unique_lock lock(ddl_guards_mutex);
auto db_guard_iter = ddl_guards.try_emplace(database).first;
@ -956,36 +956,38 @@ DDLGuard::DDLGuard(Map & map_, std::shared_mutex & db_mutex_, std::unique_lock<s
++it->second.counter;
guards_lock.unlock();
table_lock = std::unique_lock(*it->second.mutex);
bool is_database = elem.empty();
if (!is_database)
is_database_guard = elem.empty();
if (!is_database_guard)
{
bool locked_database_for_read = db_mutex.try_lock_shared();
if (!locked_database_for_read)
{
removeTableLock();
releaseTableLock();
throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database {} is currently dropped or renamed", database_name);
}
}
}
void DDLGuard::removeTableLock()
void DDLGuard::releaseTableLock() noexcept
{
if (table_lock_removed)
return;
table_lock_removed = true;
guards_lock.lock();
--it->second.counter;
if (!it->second.counter)
{
table_lock.unlock();
UInt32 counter = --it->second.counter;
table_lock.unlock();
if (counter == 0)
map.erase(it);
}
guards_lock.unlock();
}
DDLGuard::~DDLGuard()
{
bool is_database = it->first.empty();
if (!is_database)
if (!is_database_guard)
db_mutex.unlock_shared();
removeTableLock();
releaseTableLock();
}
}

View File

@ -54,16 +54,21 @@ public:
DDLGuard(Map & map_, std::shared_mutex & db_mutex_, std::unique_lock<std::mutex> guards_lock_, const String & elem, const String & database_name);
~DDLGuard();
/// Unlocks table name, keeps holding read lock for database name
void releaseTableLock() noexcept;
private:
Map & map;
std::shared_mutex & db_mutex;
Map::iterator it;
std::unique_lock<std::mutex> guards_lock;
std::unique_lock<std::mutex> table_lock;
void removeTableLock();
bool table_lock_removed = false;
bool is_database_guard = false;
};
using DDLGuardPtr = std::unique_ptr<DDLGuard>;
/// Creates temporary table in `_temporary_and_external_tables` with randomly generated unique StorageID.
/// Such table can be accessed from everywhere by its ID.
@ -117,7 +122,7 @@ public:
void loadDatabases();
/// Get an object that protects the table from concurrently executing multiple DDL operations.
std::unique_ptr<DDLGuard> getDDLGuard(const String & database, const String & table);
DDLGuardPtr getDDLGuard(const String & database, const String & table);
/// Get an object that protects the database from concurrent DDL queries all tables in the database
std::unique_lock<std::shared_mutex> getExclusiveDDLGuardForDatabase(const String & database);

View File

@ -1,5 +1,5 @@
#include <Interpreters/InterpreterAlterQuery.h>
#include <Interpreters/DDLWorker.h>
#include <Interpreters/executeDDLQueryOnCluster.h>
#include <Interpreters/MutationsInterpreter.h>
#include <Interpreters/AddDefaultDatabaseVisitor.h>
#include <Interpreters/Context.h>
@ -16,6 +16,9 @@
#include <Common/typeid_cast.h>
#include <boost/range/algorithm_ext/push_back.hpp>
#include <algorithm>
#include <Databases/IDatabase.h>
#include <Databases/DatabaseReplicated.h>
#include <Databases/DatabaseFactory.h>
namespace DB
@ -25,6 +28,7 @@ namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int INCORRECT_QUERY;
extern const int NOT_IMPLEMENTED;
}
@ -38,11 +42,21 @@ BlockIO InterpreterAlterQuery::execute()
BlockIO res;
const auto & alter = query_ptr->as<ASTAlterQuery &>();
if (!alter.cluster.empty())
return executeDDLQueryOnCluster(query_ptr, context, getRequiredAccess());
context.checkAccess(getRequiredAccess());
auto table_id = context.resolveStorageID(alter, Context::ResolveOrdinary);
DatabasePtr database = DatabaseCatalog::instance().getDatabase(table_id.database_name);
if (typeid_cast<DatabaseReplicated *>(database.get()) && context.getClientInfo().query_kind != ClientInfo::QueryKind::SECONDARY_QUERY)
{
auto guard = DatabaseCatalog::instance().getDDLGuard(table_id.database_name, table_id.table_name);
guard->releaseTableLock();
return typeid_cast<DatabaseReplicated *>(database.get())->tryEnqueueReplicatedDDL(query_ptr, context);
}
StoragePtr table = DatabaseCatalog::instance().getTable(table_id, context);
auto alter_lock = table->lockForAlter(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
auto metadata_snapshot = table->getInMemoryMetadataPtr();
@ -80,6 +94,14 @@ BlockIO InterpreterAlterQuery::execute()
throw Exception("Wrong parameter type in ALTER query", ErrorCodes::LOGICAL_ERROR);
}
if (typeid_cast<DatabaseReplicated *>(database.get()))
{
int command_types_count = !mutation_commands.empty() + !partition_commands.empty() + !live_view_commands.empty() + !alter_commands.empty();
if (1 < command_types_count)
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "For Replicated databases it's not allowed "
"to execute ALTERs of different types in single query");
}
if (!mutation_commands.empty())
{
MutationsInterpreter(table, metadata_snapshot, mutation_commands, context, false).validate();

View File

@ -30,7 +30,8 @@
#include <Storages/StorageInMemoryMetadata.h>
#include <Interpreters/Context.h>
#include <Interpreters/DDLWorker.h>
#include <Interpreters/executeDDLQueryOnCluster.h>
#include <Interpreters/Cluster.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/InterpreterCreateQuery.h>
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
@ -46,6 +47,7 @@
#include <DataTypes/DataTypeNullable.h>
#include <Databases/DatabaseFactory.h>
#include <Databases/DatabaseReplicated.h>
#include <Databases/IDatabase.h>
#include <Databases/DatabaseOnDisk.h>
@ -79,6 +81,7 @@ namespace ErrorCodes
extern const int ILLEGAL_SYNTAX_FOR_DATA_TYPE;
extern const int ILLEGAL_COLUMN;
extern const int LOGICAL_ERROR;
extern const int UNKNOWN_DATABASE;
extern const int PATH_ACCESS_DENIED;
extern const int NOT_IMPLEMENTED;
extern const int UNKNOWN_TABLE;
@ -146,7 +149,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
throw Exception(ErrorCodes::UNKNOWN_DATABASE_ENGINE, "Unknown database engine: {}", serializeAST(*create.storage));
}
if (create.storage->engine->name == "Atomic")
if (create.storage->engine->name == "Atomic" || create.storage->engine->name == "Replicated")
{
if (create.attach && create.uuid == UUIDHelpers::Nil)
throw Exception(ErrorCodes::INCORRECT_QUERY, "UUID must be specified for ATTACH. "
@ -205,6 +208,12 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
"Enable allow_experimental_database_materialize_mysql to use it.", ErrorCodes::UNKNOWN_DATABASE_ENGINE);
}
if (create.storage->engine->name == "Replicated" && !context.getSettingsRef().allow_experimental_database_replicated && !internal)
{
throw Exception("Replicated is an experimental database engine. "
"Enable allow_experimental_database_replicated to use it.", ErrorCodes::UNKNOWN_DATABASE_ENGINE);
}
DatabasePtr database = DatabaseFactory::get(create, metadata_path / "", context);
if (create.uuid != UUIDHelpers::Nil)
@ -556,6 +565,11 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::setProperties(AS
validateTableStructure(create, properties);
/// Set the table engine if it was not specified explicitly.
setEngine(create);
assert(as_database_saved.empty() && as_table_saved.empty());
std::swap(create.as_database, as_database_saved);
std::swap(create.as_table, as_table_saved);
return properties;
}
@ -702,6 +716,12 @@ void InterpreterCreateQuery::assertOrSetUUID(ASTCreateQuery & create, const Data
const auto * kind = create.is_dictionary ? "Dictionary" : "Table";
const auto * kind_upper = create.is_dictionary ? "DICTIONARY" : "TABLE";
if (database->getEngineName() == "Replicated" && context.getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY && !internal)
{
if (create.uuid == UUIDHelpers::Nil)
throw Exception("Table UUID is not specified in DDL log", ErrorCodes::LOGICAL_ERROR);
}
bool from_path = create.attach_from_path.has_value();
if (database->getUUID() != UUIDHelpers::Nil)
@ -776,11 +796,11 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
ErrorCodes::BAD_DATABASE_FOR_TEMPORARY_TABLE);
String current_database = context.getCurrentDatabase();
auto database_name = create.database.empty() ? current_database : create.database;
// If this is a stub ATTACH query, read the query definition from the database
if (create.attach && !create.storage && !create.columns_list)
{
auto database_name = create.database.empty() ? current_database : create.database;
auto database = DatabaseCatalog::instance().getDatabase(database_name);
bool if_not_exists = create.if_not_exists;
@ -800,19 +820,30 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
if (create.attach_from_path)
{
fs::path data_path = fs::path(*create.attach_from_path).lexically_normal();
fs::path user_files = fs::path(context.getUserFilesPath()).lexically_normal();
if (data_path.is_relative())
data_path = (user_files / data_path).lexically_normal();
if (!startsWith(data_path, user_files))
throw Exception(ErrorCodes::PATH_ACCESS_DENIED,
"Data directory {} must be inside {} to attach it", String(data_path), String(user_files));
fs::path root_path = fs::path(context.getPath()).lexically_normal();
/// Data path must be relative to root_path
create.attach_from_path = fs::relative(data_path, root_path) / "";
if (context.getClientInfo().query_kind == ClientInfo::QueryKind::INITIAL_QUERY)
{
fs::path data_path = fs::path(*create.attach_from_path).lexically_normal();
if (data_path.is_relative())
data_path = (user_files / data_path).lexically_normal();
if (!startsWith(data_path, user_files))
throw Exception(ErrorCodes::PATH_ACCESS_DENIED,
"Data directory {} must be inside {} to attach it", String(data_path), String(user_files));
/// Data path must be relative to root_path
create.attach_from_path = fs::relative(data_path, root_path) / "";
}
else
{
fs::path data_path = (root_path / *create.attach_from_path).lexically_normal();
if (!startsWith(data_path, user_files))
throw Exception(ErrorCodes::PATH_ACCESS_DENIED,
"Data directory {} must be inside {} to attach it", String(data_path), String(user_files));
}
}
else if (create.attach && !create.attach_short_syntax)
else if (create.attach && !create.attach_short_syntax && context.getClientInfo().query_kind != ClientInfo::QueryKind::SECONDARY_QUERY)
{
auto * log = &Poco::Logger::get("InterpreterCreateQuery");
LOG_WARNING(log, "ATTACH TABLE query with full table definition is not recommended: "
@ -836,11 +867,29 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
/// Set and retrieve list of columns, indices and constraints. Set table engine if needed. Rewrite query in canonical way.
TableProperties properties = setProperties(create);
DatabasePtr database;
bool need_add_to_database = !create.temporary;
if (need_add_to_database)
database = DatabaseCatalog::instance().getDatabase(database_name);
if (need_add_to_database && database->getEngineName() == "Replicated")
{
auto guard = DatabaseCatalog::instance().getDDLGuard(create.database, create.table);
database = DatabaseCatalog::instance().getDatabase(create.database);
if (typeid_cast<DatabaseReplicated *>(database.get()) && context.getClientInfo().query_kind != ClientInfo::QueryKind::SECONDARY_QUERY)
{
assertOrSetUUID(create, database);
guard->releaseTableLock();
return typeid_cast<DatabaseReplicated *>(database.get())->tryEnqueueReplicatedDDL(query_ptr, context);
}
}
if (create.replace_table)
return doCreateOrReplaceTable(create, properties);
/// Actually creates table
bool created = doCreateTable(create, properties);
if (!created) /// Table already exists
return {};
@ -880,7 +929,8 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create,
drop_ast->table = create.table;
drop_ast->no_ddl_lock = true;
InterpreterDropQuery interpreter(drop_ast, context);
Context drop_context = context;
InterpreterDropQuery interpreter(drop_ast, drop_context);
interpreter.execute();
}
else
@ -1037,6 +1087,14 @@ BlockIO InterpreterCreateQuery::createDictionary(ASTCreateQuery & create)
auto guard = DatabaseCatalog::instance().getDDLGuard(database_name, dictionary_name);
DatabasePtr database = DatabaseCatalog::instance().getDatabase(database_name);
if (typeid_cast<DatabaseReplicated *>(database.get()) && context.getClientInfo().query_kind != ClientInfo::QueryKind::SECONDARY_QUERY)
{
if (!create.attach)
assertOrSetUUID(create, database);
guard->releaseTableLock();
return typeid_cast<DatabaseReplicated *>(database.get())->tryEnqueueReplicatedDDL(query_ptr, context);
}
if (database->isDictionaryExist(dictionary_name))
{
/// TODO Check structure of dictionary
@ -1189,15 +1247,14 @@ AccessRightsElements InterpreterCreateQuery::getRequiredAccess() const
return required_access;
}
void InterpreterCreateQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, const Context &) const
void InterpreterCreateQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr &, const Context &) const
{
const auto & create = ast->as<const ASTCreateQuery &>();
elem.query_kind = "Create";
if (!create.as_table.empty())
if (!as_table_saved.empty())
{
String database = backQuoteIfNeed(create.as_database.empty() ? context.getCurrentDatabase() : create.as_database);
String database = backQuoteIfNeed(as_database_saved.empty() ? context.getCurrentDatabase() : as_database_saved);
elem.query_databases.insert(database);
elem.query_tables.insert(database + "." + backQuoteIfNeed(create.as_table));
elem.query_tables.insert(database + "." + backQuoteIfNeed(as_table_saved));
}
}

View File

@ -95,5 +95,8 @@ private:
/// Is this an internal query - not from the user.
bool internal = false;
bool force_attach = false;
mutable String as_database_saved;
mutable String as_table_saved;
};
}

View File

@ -2,7 +2,7 @@
#include <Parsers/ASTCreateQuotaQuery.h>
#include <Parsers/ASTRolesOrUsersSet.h>
#include <Interpreters/Context.h>
#include <Interpreters/DDLWorker.h>
#include <Interpreters/executeDDLQueryOnCluster.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessFlags.h>
#include <ext/range.h>

View File

@ -1,7 +1,7 @@
#include <Interpreters/InterpreterCreateRoleQuery.h>
#include <Parsers/ASTCreateRoleQuery.h>
#include <Interpreters/Context.h>
#include <Interpreters/DDLWorker.h>
#include <Interpreters/executeDDLQueryOnCluster.h>
#include <Access/AccessControlManager.h>
#include <Access/Role.h>

View File

@ -4,7 +4,7 @@
#include <Parsers/ASTRolesOrUsersSet.h>
#include <Parsers/formatAST.h>
#include <Interpreters/Context.h>
#include <Interpreters/DDLWorker.h>
#include <Interpreters/executeDDLQueryOnCluster.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessFlags.h>
#include <boost/range/algorithm/sort.hpp>

View File

@ -2,7 +2,7 @@
#include <Parsers/ASTCreateSettingsProfileQuery.h>
#include <Parsers/ASTRolesOrUsersSet.h>
#include <Interpreters/Context.h>
#include <Interpreters/DDLWorker.h>
#include <Interpreters/executeDDLQueryOnCluster.h>
#include <Access/AccessControlManager.h>
#include <Access/SettingsProfile.h>
#include <Access/AccessFlags.h>

View File

@ -1,7 +1,7 @@
#include <Interpreters/InterpreterCreateUserQuery.h>
#include <Interpreters/Context.h>
#include <Interpreters/InterpreterSetRoleQuery.h>
#include <Interpreters/DDLWorker.h>
#include <Interpreters/executeDDLQueryOnCluster.h>
#include <Parsers/ASTCreateUserQuery.h>
#include <Parsers/ASTUserNameWithHost.h>
#include <Parsers/ASTRolesOrUsersSet.h>

View File

@ -2,7 +2,7 @@
#include <Parsers/ASTDropAccessEntityQuery.h>
#include <Parsers/ASTRowPolicyName.h>
#include <Interpreters/Context.h>
#include <Interpreters/DDLWorker.h>
#include <Interpreters/executeDDLQueryOnCluster.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessFlags.h>
#include <Access/User.h>

View File

@ -2,7 +2,7 @@
#include <Databases/IDatabase.h>
#include <Interpreters/Context.h>
#include <Interpreters/DDLWorker.h>
#include <Interpreters/executeDDLQueryOnCluster.h>
#include <Interpreters/InterpreterDropQuery.h>
#include <Interpreters/ExternalDictionariesLoader.h>
#include <Interpreters/QueryLog.h>
@ -12,6 +12,7 @@
#include <Common/escapeForFileName.h>
#include <Common/quoteString.h>
#include <Common/typeid_cast.h>
#include <Databases/DatabaseReplicated.h>
#if !defined(ARCADIA_BUILD)
# include "config_core.h"
@ -32,6 +33,7 @@ namespace ErrorCodes
extern const int UNKNOWN_TABLE;
extern const int UNKNOWN_DICTIONARY;
extern const int NOT_IMPLEMENTED;
extern const int INCORRECT_QUERY;
}
@ -118,32 +120,55 @@ BlockIO InterpreterDropQuery::executeToTableImpl(const ASTDropQuery & query, Dat
if (database && table)
{
if (query_ptr->as<ASTDropQuery &>().is_view && !table->isView())
if (query.as<ASTDropQuery &>().is_view && !table->isView())
throw Exception("Table " + table_id.getNameForLogs() + " is not a View", ErrorCodes::LOGICAL_ERROR);
/// Now get UUID, so we can wait for table data to be finally dropped
table_id.uuid = database->tryGetTableUUID(table_id.table_name);
/// Prevents recursive drop from drop database query. The original query must specify a table.
bool is_drop_or_detach_database = query_ptr->as<ASTDropQuery>()->table.empty();
bool is_replicated_ddl_query = typeid_cast<DatabaseReplicated *>(database.get()) &&
context.getClientInfo().query_kind != ClientInfo::QueryKind::SECONDARY_QUERY &&
!is_drop_or_detach_database;
if (is_replicated_ddl_query)
{
if (query.kind == ASTDropQuery::Kind::Detach && !query.permanently)
throw Exception(ErrorCodes::INCORRECT_QUERY, "DETACH TABLE is not allowed for Replicated databases. "
"Use DETACH TABLE PERMANENTLY or SYSTEM RESTART REPLICA");
if (query.kind == ASTDropQuery::Kind::Detach)
context.checkAccess(table->isView() ? AccessType::DROP_VIEW : AccessType::DROP_TABLE, table_id);
else if (query.kind == ASTDropQuery::Kind::Truncate)
context.checkAccess(AccessType::TRUNCATE, table_id);
else if (query.kind == ASTDropQuery::Kind::Drop)
context.checkAccess(table->isView() ? AccessType::DROP_VIEW : AccessType::DROP_TABLE, table_id);
ddl_guard->releaseTableLock();
table.reset();
return typeid_cast<DatabaseReplicated *>(database.get())->tryEnqueueReplicatedDDL(query.clone(), context);
}
if (query.kind == ASTDropQuery::Kind::Detach)
{
context.checkAccess(table->isView() ? AccessType::DROP_VIEW : AccessType::DROP_TABLE, table_id);
table->checkTableCanBeDetached();
table->shutdown();
TableExclusiveLockHolder table_lock;
if (database->getUUID() == UUIDHelpers::Nil)
table_lock = table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
if (query.permanently)
{
/// Drop table from memory, don't touch data, metadata file renamed and will be skipped during server restart
database->detachTablePermanently(table_id.table_name);
database->detachTablePermanently(context, table_id.table_name);
}
else
{
/// Drop table from memory, don't touch data and metadata
database->detachTable(table_id.table_name);
}
}
else if (query.kind == ASTDropQuery::Kind::Truncate)
{
@ -194,6 +219,21 @@ BlockIO InterpreterDropQuery::executeToDictionary(
DatabasePtr database = tryGetDatabase(database_name, if_exists);
bool is_drop_or_detach_database = query_ptr->as<ASTDropQuery>()->table.empty();
bool is_replicated_ddl_query = typeid_cast<DatabaseReplicated *>(database.get()) &&
context.getClientInfo().query_kind != ClientInfo::QueryKind::SECONDARY_QUERY &&
!is_drop_or_detach_database;
if (is_replicated_ddl_query)
{
if (kind == ASTDropQuery::Kind::Detach)
throw Exception(ErrorCodes::INCORRECT_QUERY, "DETACH DICTIONARY is not allowed for Replicated databases.");
context.checkAccess(AccessType::DROP_DICTIONARY, database_name, dictionary_name);
ddl_guard->releaseTableLock();
return typeid_cast<DatabaseReplicated *>(database.get())->tryEnqueueReplicatedDDL(query_ptr, context);
}
if (!database || !database->isDictionaryExist(dictionary_name))
{
if (!if_exists)
@ -307,6 +347,8 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query,
if (database->getEngineName() == "MaterializeMySQL")
stopDatabaseSynchronization(database);
#endif
if (auto * replicated = typeid_cast<DatabaseReplicated *>(database.get()))
replicated->stopReplication();
if (database->shouldBeEmptyOnDetach())
{

View File

@ -2,7 +2,7 @@
#include <Parsers/ASTGrantQuery.h>
#include <Parsers/ASTRolesOrUsersSet.h>
#include <Interpreters/Context.h>
#include <Interpreters/DDLWorker.h>
#include <Interpreters/executeDDLQueryOnCluster.h>
#include <Access/AccessControlManager.h>
#include <Access/ContextAccess.h>
#include <Access/RolesOrUsersSet.h>

View File

@ -2,7 +2,7 @@
#include <Parsers/ASTKillQueryQuery.h>
#include <Parsers/queryToString.h>
#include <Interpreters/Context.h>
#include <Interpreters/DDLWorker.h>
#include <Interpreters/executeDDLQueryOnCluster.h>
#include <Interpreters/ProcessList.h>
#include <Interpreters/executeQuery.h>
#include <Interpreters/CancellationCode.h>

View File

@ -1,7 +1,7 @@
#include <Storages/IStorage.h>
#include <Parsers/ASTOptimizeQuery.h>
#include <Interpreters/Context.h>
#include <Interpreters/DDLWorker.h>
#include <Interpreters/executeDDLQueryOnCluster.h>
#include <Interpreters/InterpreterOptimizeQuery.h>
#include <Access/AccessRightsElement.h>
#include <Common/typeid_cast.h>

View File

@ -3,14 +3,20 @@
#include <Interpreters/Context.h>
#include <Interpreters/InterpreterRenameQuery.h>
#include <Storages/IStorage.h>
#include <Interpreters/DDLWorker.h>
#include <Interpreters/executeDDLQueryOnCluster.h>
#include <Interpreters/QueryLog.h>
#include <Access/AccessRightsElement.h>
#include <Common/typeid_cast.h>
#include <Databases/DatabaseReplicated.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
InterpreterRenameQuery::InterpreterRenameQuery(const ASTPtr & query_ptr_, Context & context_)
: query_ptr(query_ptr_), context(context_)
@ -61,10 +67,10 @@ BlockIO InterpreterRenameQuery::execute()
if (rename.database)
return executeToDatabase(rename, descriptions);
else
return executeToTables(rename, descriptions);
return executeToTables(rename, descriptions, table_guards);
}
BlockIO InterpreterRenameQuery::executeToTables(const ASTRenameQuery & rename, const RenameDescriptions & descriptions)
BlockIO InterpreterRenameQuery::executeToTables(const ASTRenameQuery & rename, const RenameDescriptions & descriptions, TableGuards & ddl_guards)
{
auto & database_catalog = DatabaseCatalog::instance();
@ -73,13 +79,29 @@ BlockIO InterpreterRenameQuery::executeToTables(const ASTRenameQuery & rename, c
if (!rename.exchange)
database_catalog.assertTableDoesntExist(StorageID(elem.to_database_name, elem.to_table_name), context);
database_catalog.getDatabase(elem.from_database_name)->renameTable(
DatabasePtr database = database_catalog.getDatabase(elem.from_database_name);
if (typeid_cast<DatabaseReplicated *>(database.get()) && context.getClientInfo().query_kind != ClientInfo::QueryKind::SECONDARY_QUERY)
{
if (1 < descriptions.size())
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Database {} is Replicated, "
"it does not support renaming of multiple tables in single query.", elem.from_database_name);
UniqueTableName from(elem.from_database_name, elem.from_table_name);
UniqueTableName to(elem.to_database_name, elem.to_table_name);
ddl_guards[from]->releaseTableLock();
ddl_guards[to]->releaseTableLock();
return typeid_cast<DatabaseReplicated *>(database.get())->tryEnqueueReplicatedDDL(query_ptr, context);
}
else
{
database->renameTable(
context,
elem.from_table_name,
*database_catalog.getDatabase(elem.to_database_name),
elem.to_table_name,
rename.exchange,
rename.dictionary);
}
}
return {};

View File

@ -57,7 +57,7 @@ public:
void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, const Context &) const override;
private:
BlockIO executeToTables(const ASTRenameQuery & rename, const RenameDescriptions & descriptions);
BlockIO executeToTables(const ASTRenameQuery & rename, const RenameDescriptions & descriptions, TableGuards & ddl_guards);
static BlockIO executeToDatabase(const ASTRenameQuery & rename, const RenameDescriptions & descriptions);
AccessRightsElements getRequiredAccess() const;

View File

@ -16,7 +16,7 @@
#include <Interpreters/InterpreterCreateQuery.h>
#include <Interpreters/InterpreterRenameQuery.h>
#include <Interpreters/QueryLog.h>
#include <Interpreters/DDLWorker.h>
#include <Interpreters/executeDDLQueryOnCluster.h>
#include <Interpreters/PartLog.h>
#include <Interpreters/QueryThreadLog.h>
#include <Interpreters/TraceLog.h>

View File

@ -515,7 +515,9 @@ void SystemLog<LogElement>::prepareTable()
LOG_DEBUG(log, "Existing table {} for system log has obsolete or different structure. Renaming it to {}", description, backQuoteIfNeed(to.table));
InterpreterRenameQuery(rename, context).execute();
Context query_context = context;
query_context.makeQueryContext();
InterpreterRenameQuery(rename, query_context).execute();
/// The required table will be created.
table = nullptr;
@ -531,7 +533,10 @@ void SystemLog<LogElement>::prepareTable()
auto create = getCreateTableQuery();
InterpreterCreateQuery interpreter(create, context);
Context query_context = context;
query_context.makeQueryContext();
InterpreterCreateQuery interpreter(create, query_context);
interpreter.setInternal(true);
interpreter.execute();

View File

@ -0,0 +1,337 @@
#include <Interpreters/executeDDLQueryOnCluster.h>
#include <Interpreters/DDLWorker.h>
#include <Interpreters/DDLTask.h>
#include <Interpreters/AddDefaultDatabaseVisitor.h>
#include <Interpreters/Context.h>
#include <Parsers/ASTQueryWithOutput.h>
#include <Parsers/ASTQueryWithOnCluster.h>
#include <Parsers/ASTAlterQuery.h>
#include <Parsers/queryToString.h>
#include <Access/AccessRightsElement.h>
#include <Access/ContextAccess.h>
#include <Common/Macros.h>
#include <Common/ZooKeeper/ZooKeeper.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeString.h>
#include <filesystem>
namespace fs = std::filesystem;
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
extern const int TIMEOUT_EXCEEDED;
extern const int UNFINISHED;
extern const int QUERY_IS_PROHIBITED;
extern const int LOGICAL_ERROR;
}
bool isSupportedAlterType(int type)
{
assert(type != ASTAlterCommand::NO_TYPE);
static const std::unordered_set<int> unsupported_alter_types{
/// It's dangerous, because it may duplicate data if executed on multiple replicas. We can allow it after #18978
ASTAlterCommand::ATTACH_PARTITION,
/// Usually followed by ATTACH PARTITION
ASTAlterCommand::FETCH_PARTITION,
/// Logical error
ASTAlterCommand::NO_TYPE,
};
return unsupported_alter_types.count(type) == 0;
}
BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, const Context & context)
{
return executeDDLQueryOnCluster(query_ptr_, context, {});
}
BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, const Context & context, const AccessRightsElements & query_requires_access, bool query_requires_grant_option)
{
return executeDDLQueryOnCluster(query_ptr, context, AccessRightsElements{query_requires_access}, query_requires_grant_option);
}
BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, const Context & context, AccessRightsElements && query_requires_access, bool query_requires_grant_option)
{
/// Remove FORMAT <fmt> and INTO OUTFILE <file> if exists
ASTPtr query_ptr = query_ptr_->clone();
ASTQueryWithOutput::resetOutputASTIfExist(*query_ptr);
// XXX: serious design flaw since `ASTQueryWithOnCluster` is not inherited from `IAST`!
auto * query = dynamic_cast<ASTQueryWithOnCluster *>(query_ptr.get());
if (!query)
{
throw Exception("Distributed execution is not supported for such DDL queries", ErrorCodes::NOT_IMPLEMENTED);
}
if (!context.getSettingsRef().allow_distributed_ddl)
throw Exception("Distributed DDL queries are prohibited for the user", ErrorCodes::QUERY_IS_PROHIBITED);
if (const auto * query_alter = query_ptr->as<ASTAlterQuery>())
{
for (const auto & command : query_alter->command_list->children)
{
if (!isSupportedAlterType(command->as<ASTAlterCommand&>().type))
throw Exception("Unsupported type of ALTER query", ErrorCodes::NOT_IMPLEMENTED);
}
}
query->cluster = context.getMacros()->expand(query->cluster);
ClusterPtr cluster = context.getCluster(query->cluster);
DDLWorker & ddl_worker = context.getDDLWorker();
/// Enumerate hosts which will be used to send query.
Cluster::AddressesWithFailover shards = cluster->getShardsAddresses();
std::vector<HostID> hosts;
for (const auto & shard : shards)
{
for (const auto & addr : shard)
hosts.emplace_back(addr);
}
if (hosts.empty())
throw Exception("No hosts defined to execute distributed DDL query", ErrorCodes::LOGICAL_ERROR);
/// The current database in a distributed query need to be replaced with either
/// the local current database or a shard's default database.
bool need_replace_current_database
= (std::find_if(
query_requires_access.begin(),
query_requires_access.end(),
[](const AccessRightsElement & elem) { return elem.isEmptyDatabase(); })
!= query_requires_access.end());
bool use_local_default_database = false;
const String & current_database = context.getCurrentDatabase();
if (need_replace_current_database)
{
Strings shard_default_databases;
for (const auto & shard : shards)
{
for (const auto & addr : shard)
{
if (!addr.default_database.empty())
shard_default_databases.push_back(addr.default_database);
else
use_local_default_database = true;
}
}
std::sort(shard_default_databases.begin(), shard_default_databases.end());
shard_default_databases.erase(std::unique(shard_default_databases.begin(), shard_default_databases.end()), shard_default_databases.end());
assert(use_local_default_database || !shard_default_databases.empty());
if (use_local_default_database && !shard_default_databases.empty())
throw Exception("Mixed local default DB and shard default DB in DDL query", ErrorCodes::NOT_IMPLEMENTED);
if (use_local_default_database)
{
query_requires_access.replaceEmptyDatabase(current_database);
}
else
{
for (size_t i = 0; i != query_requires_access.size();)
{
auto & element = query_requires_access[i];
if (element.isEmptyDatabase())
{
query_requires_access.insert(query_requires_access.begin() + i + 1, shard_default_databases.size() - 1, element);
for (size_t j = 0; j != shard_default_databases.size(); ++j)
query_requires_access[i + j].replaceEmptyDatabase(shard_default_databases[j]);
i += shard_default_databases.size();
}
else
++i;
}
}
}
AddDefaultDatabaseVisitor visitor(current_database, !use_local_default_database);
visitor.visitDDL(query_ptr);
/// Check access rights, assume that all servers have the same users config
if (query_requires_grant_option)
context.getAccess()->checkGrantOption(query_requires_access);
else
context.checkAccess(query_requires_access);
DDLLogEntry entry;
entry.hosts = std::move(hosts);
entry.query = queryToString(query_ptr);
entry.initiator = ddl_worker.getCommonHostID();
String node_path = ddl_worker.enqueueQuery(entry);
BlockIO io;
if (context.getSettingsRef().distributed_ddl_task_timeout == 0)
return io;
auto stream = std::make_shared<DDLQueryStatusInputStream>(node_path, entry, context);
io.in = std::move(stream);
return io;
}
DDLQueryStatusInputStream::DDLQueryStatusInputStream(const String & zk_node_path, const DDLLogEntry & entry, const Context & context_,
const std::optional<Strings> & hosts_to_wait)
: node_path(zk_node_path)
, context(context_)
, watch(CLOCK_MONOTONIC_COARSE)
, log(&Poco::Logger::get("DDLQueryStatusInputStream"))
{
sample = Block{
{std::make_shared<DataTypeString>(), "host"},
{std::make_shared<DataTypeUInt16>(), "port"},
{std::make_shared<DataTypeInt64>(), "status"},
{std::make_shared<DataTypeString>(), "error"},
{std::make_shared<DataTypeUInt64>(), "num_hosts_remaining"},
{std::make_shared<DataTypeUInt64>(), "num_hosts_active"},
};
if (hosts_to_wait)
{
waiting_hosts = NameSet(hosts_to_wait->begin(), hosts_to_wait->end());
by_hostname = false;
}
else
{
for (const HostID & host : entry.hosts)
waiting_hosts.emplace(host.toString());
}
addTotalRowsApprox(waiting_hosts.size());
timeout_seconds = context.getSettingsRef().distributed_ddl_task_timeout;
}
Block DDLQueryStatusInputStream::readImpl()
{
Block res;
if (num_hosts_finished >= waiting_hosts.size())
{
if (first_exception)
throw Exception(*first_exception);
return res;
}
auto zookeeper = context.getZooKeeper();
size_t try_number = 0;
while (res.rows() == 0)
{
if (isCancelled())
{
if (first_exception)
throw Exception(*first_exception);
return res;
}
if (timeout_seconds >= 0 && watch.elapsedSeconds() > timeout_seconds)
{
size_t num_unfinished_hosts = waiting_hosts.size() - num_hosts_finished;
size_t num_active_hosts = current_active_hosts.size();
throw Exception(ErrorCodes::TIMEOUT_EXCEEDED,
"Watching task {} is executing longer than distributed_ddl_task_timeout (={}) seconds. "
"There are {} unfinished hosts ({} of them are currently active), they are going to execute the query in background",
node_path, timeout_seconds, num_unfinished_hosts, num_active_hosts);
}
if (num_hosts_finished != 0 || try_number != 0)
{
sleepForMilliseconds(std::min<size_t>(1000, 50 * (try_number + 1)));
}
if (!zookeeper->exists(node_path))
{
throw Exception(ErrorCodes::UNFINISHED,
"Cannot provide query execution status. The query's node {} has been deleted by the cleaner since it was finished (or its lifetime is expired)",
node_path);
}
Strings new_hosts = getNewAndUpdate(getChildrenAllowNoNode(zookeeper, fs::path(node_path) / "finished"));
++try_number;
if (new_hosts.empty())
continue;
current_active_hosts = getChildrenAllowNoNode(zookeeper, fs::path(node_path) / "active");
MutableColumns columns = sample.cloneEmptyColumns();
for (const String & host_id : new_hosts)
{
ExecutionStatus status(-1, "Cannot obtain error message");
{
String status_data;
if (zookeeper->tryGet(fs::path(node_path) / "finished" / host_id, status_data))
status.tryDeserializeText(status_data);
}
String host = host_id;
UInt16 port = 0;
if (by_hostname)
{
auto host_and_port = Cluster::Address::fromString(host_id);
host = host_and_port.first;
port = host_and_port.second;
}
if (status.code != 0 && first_exception == nullptr)
first_exception = std::make_unique<Exception>(status.code, "There was an error on [{}:{}]: {}", host, port, status.message);
++num_hosts_finished;
columns[0]->insert(host);
columns[1]->insert(port);
columns[2]->insert(status.code);
columns[3]->insert(status.message);
columns[4]->insert(waiting_hosts.size() - num_hosts_finished);
columns[5]->insert(current_active_hosts.size());
}
res = sample.cloneWithColumns(std::move(columns));
}
return res;
}
Strings DDLQueryStatusInputStream::getChildrenAllowNoNode(const std::shared_ptr<zkutil::ZooKeeper> & zookeeper, const String & node_path)
{
Strings res;
Coordination::Error code = zookeeper->tryGetChildren(node_path, res);
if (code != Coordination::Error::ZOK && code != Coordination::Error::ZNONODE)
throw Coordination::Exception(code, node_path);
return res;
}
Strings DDLQueryStatusInputStream::getNewAndUpdate(const Strings & current_list_of_finished_hosts)
{
Strings diff;
for (const String & host : current_list_of_finished_hosts)
{
if (!waiting_hosts.count(host))
{
if (!ignoring_hosts.count(host))
{
ignoring_hosts.emplace(host);
LOG_INFO(log, "Unexpected host {} appeared in task {}", host, node_path);
}
continue;
}
if (!finished_hosts.count(host))
{
diff.emplace_back(host);
finished_hosts.emplace(host);
}
}
return diff;
}
}

View File

@ -0,0 +1,67 @@
#pragma once
#include <DataStreams/BlockIO.h>
#include <Parsers/IAST_fwd.h>
namespace zkutil
{
class ZooKeeper;
}
namespace DB
{
class Context;
class AccessRightsElements;
struct DDLLogEntry;
/// Returns true if provided ALTER type can be executed ON CLUSTER
bool isSupportedAlterType(int type);
/// Pushes distributed DDL query to the queue.
/// Returns DDLQueryStatusInputStream, which reads results of query execution on each host in the cluster.
BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, const Context & context);
BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, const Context & context, const AccessRightsElements & query_requires_access, bool query_requires_grant_option = false);
BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, const Context & context, AccessRightsElements && query_requires_access, bool query_requires_grant_option = false);
class DDLQueryStatusInputStream final : public IBlockInputStream
{
public:
DDLQueryStatusInputStream(const String & zk_node_path, const DDLLogEntry & entry, const Context & context_, const std::optional<Strings> & hosts_to_wait = {});
String getName() const override { return "DDLQueryStatusInputStream"; }
Block getHeader() const override { return sample; }
Block getSampleBlock() const { return sample.cloneEmpty(); }
Block readImpl() override;
private:
static Strings getChildrenAllowNoNode(const std::shared_ptr<zkutil::ZooKeeper> & zookeeper, const String & node_path);
Strings getNewAndUpdate(const Strings & current_list_of_finished_hosts);
String node_path;
const Context & context;
Stopwatch watch;
Poco::Logger * log;
Block sample;
NameSet waiting_hosts; /// hosts from task host list
NameSet finished_hosts; /// finished hosts from host list
NameSet ignoring_hosts; /// appeared hosts that are not in hosts list
Strings current_active_hosts; /// Hosts that were in active state at the last check
size_t num_hosts_finished = 0;
/// Save the first detected error and throw it at the end of execution
std::unique_ptr<Exception> first_exception;
Int64 timeout_seconds = 120;
bool by_hostname = true;
};
}

View File

@ -41,6 +41,7 @@ SRCS(
Context.cpp
CrashLog.cpp
CrossToInnerJoinVisitor.cpp
DDLTask.cpp
DDLWorker.cpp
DNSCacheUpdater.cpp
DatabaseAndTableWithAlias.cpp
@ -153,6 +154,7 @@ SRCS(
convertFieldToType.cpp
createBlockSelector.cpp
evaluateConstantExpression.cpp
executeDDLQueryOnCluster.cpp
executeQuery.cpp
getClusterName.cpp
getHeaderForProcessingStage.cpp

View File

@ -344,7 +344,7 @@ void ASTAlterCommand::formatImpl(
throw Exception("Unexpected type of ALTER", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
}
bool ASTAlterQuery::isSettingsAlter() const
bool ASTAlterQuery::isOneCommandTypeOnly(const ASTAlterCommand::Type & type) const
{
if (command_list)
{
@ -353,7 +353,7 @@ bool ASTAlterQuery::isSettingsAlter() const
for (const auto & child : command_list->children)
{
const auto & command = child->as<const ASTAlterCommand &>();
if (command.type != ASTAlterCommand::MODIFY_SETTING)
if (command.type != type)
return false;
}
return true;
@ -361,6 +361,16 @@ bool ASTAlterQuery::isSettingsAlter() const
return false;
}
bool ASTAlterQuery::isSettingsAlter() const
{
return isOneCommandTypeOnly(ASTAlterCommand::MODIFY_SETTING);
}
bool ASTAlterQuery::isFreezeAlter() const
{
return isOneCommandTypeOnly(ASTAlterCommand::FREEZE_PARTITION) || isOneCommandTypeOnly(ASTAlterCommand::FREEZE_ALL);
}
/** Get the text that identifies this element. */
String ASTAlterQuery::getID(char delim) const
{

View File

@ -189,6 +189,8 @@ public:
bool isSettingsAlter() const;
bool isFreezeAlter() const;
String getID(char) const override;
ASTPtr clone() const override;
@ -200,6 +202,8 @@ public:
protected:
void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
bool isOneCommandTypeOnly(const ASTAlterCommand::Type & type) const;
};
}

View File

@ -450,16 +450,21 @@ static StoragePtr create(const StorageFactory::Arguments & args)
arg_cnt += 2;
}
else
throw Exception("Expected two string literal arguments: zookeper_path and replica_name", ErrorCodes::BAD_ARGUMENTS);
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.local_context.getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY;
bool allow_uuid_macro = is_on_cluster || args.query.attach;
bool is_replicated_database = args.local_context.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.
/// We also unfold {uuid} macro, so path will not be broken after moving table from Atomic to Ordinary database.
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;

View File

@ -89,6 +89,7 @@ StorageMaterializedView::StorageMaterializedView(
else
{
/// We will create a query to create an internal table.
auto create_context = Context(local_context);
auto manual_create_query = std::make_shared<ASTCreateQuery>();
manual_create_query->database = getStorageID().database_name;
manual_create_query->table = generateInnerTableName(getStorageID());
@ -99,7 +100,7 @@ StorageMaterializedView::StorageMaterializedView(
manual_create_query->set(manual_create_query->columns_list, new_columns_list);
manual_create_query->set(manual_create_query->storage, query.storage->ptr());
InterpreterCreateQuery create_interpreter(manual_create_query, local_context);
InterpreterCreateQuery create_interpreter(manual_create_query, create_context);
create_interpreter.setInternal(true);
create_interpreter.execute();
@ -193,9 +194,9 @@ BlockOutputStreamPtr StorageMaterializedView::write(const ASTPtr & query, const
}
static void executeDropQuery(ASTDropQuery::Kind kind, Context & global_context, const StorageID & target_table_id, bool no_delay)
static void executeDropQuery(ASTDropQuery::Kind kind, const Context & global_context, const Context & current_context, const StorageID & target_table_id, bool no_delay)
{
if (DatabaseCatalog::instance().tryGetTable(target_table_id, global_context))
if (DatabaseCatalog::instance().tryGetTable(target_table_id, current_context))
{
/// We create and execute `drop` query for internal table.
auto drop_query = std::make_shared<ASTDropQuery>();
@ -205,7 +206,19 @@ static void executeDropQuery(ASTDropQuery::Kind kind, Context & global_context,
drop_query->no_delay = no_delay;
drop_query->if_exists = true;
ASTPtr ast_drop_query = drop_query;
InterpreterDropQuery drop_interpreter(ast_drop_query, global_context);
/// FIXME We have to use global context to execute DROP query for inner table
/// to avoid "Not enough privileges" error if current user has only DROP VIEW ON mat_view_name privilege
/// and not allowed to drop inner table explicitly. Allowing to drop inner table without explicit grant
/// looks like expected behaviour and we have tests for it.
auto drop_context = Context(global_context);
drop_context.getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY;
if (auto txn = current_context.getZooKeeperMetadataTransaction())
{
/// For Replicated database
drop_context.setQueryContext(const_cast<Context &>(current_context));
drop_context.initZooKeeperMetadataTransaction(txn, true);
}
InterpreterDropQuery drop_interpreter(ast_drop_query, drop_context);
drop_interpreter.execute();
}
}
@ -218,19 +231,19 @@ void StorageMaterializedView::drop()
if (!select_query.select_table_id.empty())
DatabaseCatalog::instance().removeDependency(select_query.select_table_id, table_id);
dropInnerTable(true);
dropInnerTable(true, global_context);
}
void StorageMaterializedView::dropInnerTable(bool no_delay)
void StorageMaterializedView::dropInnerTable(bool no_delay, const Context & context)
{
if (has_inner_table && tryGetTargetTable())
executeDropQuery(ASTDropQuery::Kind::Drop, global_context, target_table_id, no_delay);
executeDropQuery(ASTDropQuery::Kind::Drop, global_context, context, target_table_id, no_delay);
}
void StorageMaterializedView::truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder &)
void StorageMaterializedView::truncate(const ASTPtr &, const StorageMetadataPtr &, const Context & context, TableExclusiveLockHolder &)
{
if (has_inner_table)
executeDropQuery(ASTDropQuery::Kind::Truncate, global_context, target_table_id, true);
executeDropQuery(ASTDropQuery::Kind::Truncate, global_context, context, target_table_id, true);
}
void StorageMaterializedView::checkStatementCanBeForwarded() const

View File

@ -37,7 +37,7 @@ public:
BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override;
void drop() override;
void dropInnerTable(bool no_delay);
void dropInnerTable(bool no_delay, const Context & context);
void truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder &) override;

View File

@ -29,6 +29,7 @@
#include <Databases/IDatabase.h>
#include <Databases/DatabaseOnDisk.h>
#include <Parsers/formatAST.h>
#include <Parsers/ASTDropQuery.h>
@ -46,6 +47,7 @@
#include <Interpreters/InterpreterAlterQuery.h>
#include <Interpreters/PartLog.h>
#include <Interpreters/Context.h>
#include <Interpreters/DDLTask.h>
#include <DataStreams/RemoteBlockInputStream.h>
#include <DataStreams/copyData.h>
@ -4218,6 +4220,8 @@ void StorageReplicatedMergeTree::alter(
future_metadata_in_zk.constraints = new_constraints_str;
Coordination::Requests ops;
size_t alter_path_idx = std::numeric_limits<size_t>::max();
size_t mutation_path_idx = std::numeric_limits<size_t>::max();
String new_metadata_str = future_metadata_in_zk.toString();
ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/metadata", new_metadata_str, metadata_version));
@ -4249,6 +4253,7 @@ void StorageReplicatedMergeTree::alter(
*current_metadata, query_context.getSettingsRef().materialize_ttl_after_modify, query_context);
alter_entry->have_mutation = !maybe_mutation_commands.empty();
alter_path_idx = ops.size();
ops.emplace_back(zkutil::makeCreateRequest(
zookeeper_path + "/log/log-", alter_entry->toString(), zkutil::CreateMode::PersistentSequential));
@ -4272,10 +4277,22 @@ void StorageReplicatedMergeTree::alter(
mutation_entry.create_time = time(nullptr);
ops.emplace_back(zkutil::makeSetRequest(mutations_path, String(), mutations_stat.version));
mutation_path_idx = ops.size();
ops.emplace_back(
zkutil::makeCreateRequest(mutations_path + "/", mutation_entry.toString(), zkutil::CreateMode::PersistentSequential));
}
if (auto txn = query_context.getZooKeeperMetadataTransaction())
{
txn->moveOpsTo(ops);
/// NOTE: IDatabase::alterTable(...) is called when executing ALTER_METADATA queue entry without query context,
/// so we have to update metadata of DatabaseReplicated here.
String metadata_zk_path = txn->getDatabaseZooKeeperPath() + "/metadata/" + escapeForFileName(table_id.table_name);
auto ast = DatabaseCatalog::instance().getDatabase(table_id.database_name)->getCreateTableQuery(table_id.table_name, query_context);
applyMetadataChangesToCreateQuery(ast, future_metadata);
ops.emplace_back(zkutil::makeSetRequest(metadata_zk_path, getObjectDefinitionFromCreateQuery(ast), -1));
}
Coordination::Responses results;
Coordination::Error rc = zookeeper->tryMulti(ops, results);
@ -4289,17 +4306,17 @@ void StorageReplicatedMergeTree::alter(
if (alter_entry->have_mutation)
{
/// ALTER_METADATA record in replication /log
String alter_path = dynamic_cast<const Coordination::CreateResponse &>(*results[2]).path_created;
String alter_path = dynamic_cast<const Coordination::CreateResponse &>(*results[alter_path_idx]).path_created;
alter_entry->znode_name = alter_path.substr(alter_path.find_last_of('/') + 1);
/// ReplicatedMergeTreeMutationEntry record in /mutations
String mutation_path = dynamic_cast<const Coordination::CreateResponse &>(*results.back()).path_created;
String mutation_path = dynamic_cast<const Coordination::CreateResponse &>(*results[mutation_path_idx]).path_created;
mutation_znode = mutation_path.substr(mutation_path.find_last_of('/') + 1);
}
else
{
/// ALTER_METADATA record in replication /log
String alter_path = dynamic_cast<const Coordination::CreateResponse &>(*results.back()).path_created;
String alter_path = dynamic_cast<const Coordination::CreateResponse &>(*results[alter_path_idx]).path_created;
alter_entry->znode_name = alter_path.substr(alter_path.find_last_of('/') + 1);
}
break;
@ -4421,7 +4438,7 @@ void StorageReplicatedMergeTree::dropPartition(const ASTPtr & partition, bool de
else
{
String partition_id = getPartitionIDFromQuery(partition, query_context);
did_drop = dropAllPartsInPartition(*zookeeper, partition_id, entry, detach);
did_drop = dropAllPartsInPartition(*zookeeper, partition_id, entry, query_context, detach);
}
if (did_drop)
@ -4445,7 +4462,7 @@ void StorageReplicatedMergeTree::dropPartition(const ASTPtr & partition, bool de
void StorageReplicatedMergeTree::truncate(
const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder & table_lock)
const ASTPtr &, const StorageMetadataPtr &, const Context & query_context, TableExclusiveLockHolder & table_lock)
{
table_lock.release(); /// Truncate is done asynchronously.
@ -4461,7 +4478,7 @@ void StorageReplicatedMergeTree::truncate(
{
LogEntry entry;
if (dropAllPartsInPartition(*zookeeper, partition_id, entry, false))
if (dropAllPartsInPartition(*zookeeper, partition_id, entry, query_context, false))
waitForAllReplicasToProcessLogEntry(entry);
}
}
@ -5245,6 +5262,9 @@ void StorageReplicatedMergeTree::mutate(const MutationCommands & commands, const
requests.emplace_back(zkutil::makeCreateRequest(
mutations_path + "/", mutation_entry.toString(), zkutil::CreateMode::PersistentSequential));
if (auto txn = query_context.getZooKeeperMetadataTransaction())
txn->moveOpsTo(requests);
Coordination::Responses responses;
Coordination::Error rc = zookeeper->tryMulti(requests, responses);
@ -5746,6 +5766,9 @@ void StorageReplicatedMergeTree::replacePartitionFrom(
}
}
if (auto txn = context.getZooKeeperMetadataTransaction())
txn->moveOpsTo(ops);
ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/log", "", -1)); /// Just update version
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential));
@ -6214,7 +6237,7 @@ bool StorageReplicatedMergeTree::dropPart(
}
bool StorageReplicatedMergeTree::dropAllPartsInPartition(
zkutil::ZooKeeper & zookeeper, String & partition_id, LogEntry & entry, bool detach)
zkutil::ZooKeeper & zookeeper, String & partition_id, LogEntry & entry, const Context & query_context, bool detach)
{
MergeTreePartInfo drop_range_info;
if (!getFakePartCoveringAllPartsInPartition(partition_id, drop_range_info))
@ -6246,6 +6269,8 @@ bool StorageReplicatedMergeTree::dropAllPartsInPartition(
Coordination::Requests ops;
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential));
ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/log", "", -1)); /// Just update version.
if (auto txn = query_context.getZooKeeperMetadataTransaction())
txn->moveOpsTo(ops);
Coordination::Responses responses = zookeeper.multi(ops);
String log_znode_path = dynamic_cast<const Coordination::CreateResponse &>(*responses.front()).path_created;

View File

@ -134,7 +134,7 @@ public:
*/
void drop() override;
void truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder &) override;
void truncate(const ASTPtr &, const StorageMetadataPtr &, const Context & query_context, TableExclusiveLockHolder &) override;
void checkTableCanBeRenamed() const override;
@ -577,7 +577,7 @@ private:
bool dropPart(zkutil::ZooKeeperPtr & zookeeper, String part_name, LogEntry & entry, bool detach, bool throw_if_noop);
bool dropAllPartsInPartition(
zkutil::ZooKeeper & zookeeper, String & partition_id, LogEntry & entry, bool detach);
zkutil::ZooKeeper & zookeeper, String & partition_id, LogEntry & entry, const Context & query_context, bool detach);
// Partition helpers
void dropPartition(const ASTPtr & partition, bool detach, bool drop_part, const Context & query_context, bool throw_if_noop) override;

View File

@ -3,6 +3,7 @@
#include <Interpreters/Cluster.h>
#include <Interpreters/Context.h>
#include <Storages/System/StorageSystemClusters.h>
#include <Databases/DatabaseReplicated.h>
namespace DB
{
@ -26,40 +27,51 @@ NamesAndTypesList StorageSystemClusters::getNamesAndTypes()
};
}
void StorageSystemClusters::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const
{
for (const auto & name_and_cluster : context.getClusters().getContainer())
writeCluster(res_columns, name_and_cluster);
const auto databases = DatabaseCatalog::instance().getDatabases();
for (const auto & name_and_database : databases)
{
const String & cluster_name = name_and_cluster.first;
const ClusterPtr & cluster = name_and_cluster.second;
const auto & shards_info = cluster->getShardsInfo();
const auto & addresses_with_failover = cluster->getShardsAddresses();
if (const auto * replicated = typeid_cast<const DatabaseReplicated *>(name_and_database.second.get()))
writeCluster(res_columns, {name_and_database.first, replicated->getCluster()});
}
}
for (size_t shard_index = 0; shard_index < shards_info.size(); ++shard_index)
void StorageSystemClusters::writeCluster(MutableColumns & res_columns, const NameAndCluster & name_and_cluster)
{
const String & cluster_name = name_and_cluster.first;
const ClusterPtr & cluster = name_and_cluster.second;
const auto & shards_info = cluster->getShardsInfo();
const auto & addresses_with_failover = cluster->getShardsAddresses();
for (size_t shard_index = 0; shard_index < shards_info.size(); ++shard_index)
{
const auto & shard_info = shards_info[shard_index];
const auto & shard_addresses = addresses_with_failover[shard_index];
const auto pool_status = shard_info.pool->getStatus();
for (size_t replica_index = 0; replica_index < shard_addresses.size(); ++replica_index)
{
const auto & shard_info = shards_info[shard_index];
const auto & shard_addresses = addresses_with_failover[shard_index];
const auto pool_status = shard_info.pool->getStatus();
size_t i = 0;
const auto & address = shard_addresses[replica_index];
for (size_t replica_index = 0; replica_index < shard_addresses.size(); ++replica_index)
{
size_t i = 0;
const auto & address = shard_addresses[replica_index];
res_columns[i++]->insert(cluster_name);
res_columns[i++]->insert(shard_info.shard_num);
res_columns[i++]->insert(shard_info.weight);
res_columns[i++]->insert(replica_index + 1);
res_columns[i++]->insert(address.host_name);
auto resolved = address.getResolvedAddress();
res_columns[i++]->insert(resolved ? resolved->host().toString() : String());
res_columns[i++]->insert(address.port);
res_columns[i++]->insert(address.is_local);
res_columns[i++]->insert(address.user);
res_columns[i++]->insert(address.default_database);
res_columns[i++]->insert(pool_status[replica_index].error_count);
res_columns[i++]->insert(pool_status[replica_index].estimated_recovery_time.count());
}
res_columns[i++]->insert(cluster_name);
res_columns[i++]->insert(shard_info.shard_num);
res_columns[i++]->insert(shard_info.weight);
res_columns[i++]->insert(replica_index + 1);
res_columns[i++]->insert(address.host_name);
auto resolved = address.getResolvedAddress();
res_columns[i++]->insert(resolved ? resolved->host().toString() : String());
res_columns[i++]->insert(address.port);
res_columns[i++]->insert(address.is_local);
res_columns[i++]->insert(address.user);
res_columns[i++]->insert(address.default_database);
res_columns[i++]->insert(pool_status[replica_index].error_count);
res_columns[i++]->insert(pool_status[replica_index].estimated_recovery_time.count());
}
}
}

View File

@ -10,6 +10,7 @@ namespace DB
{
class Context;
class Cluster;
/** Implements system table 'clusters'
* that allows to obtain information about available clusters
@ -25,8 +26,10 @@ public:
protected:
using IStorageSystemOneBlock::IStorageSystemOneBlock;
using NameAndCluster = std::pair<String, std::shared_ptr<Cluster>>;
void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override;
static void writeCluster(MutableColumns & res_columns, const NameAndCluster & name_and_cluster);
};
}

View File

@ -4,7 +4,7 @@
#include "StorageSystemDDLWorkerQueue.h"
#include <Columns/ColumnArray.h>
#include <Interpreters/DDLWorker.h>
#include <Interpreters/DDLTask.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeDateTime.h>

View File

@ -261,6 +261,18 @@
"with_coverage": false
}
},
"Functional stateful tests (release, DatabaseReplicated)": {
"required_build_properties": {
"compiler": "clang-11",
"package_type": "deb",
"build_type": "relwithdebuginfo",
"sanitizer": "none",
"bundled": "bundled",
"splitted": "unsplitted",
"clang-tidy": "disable",
"with_coverage": false
}
},
"Functional stateless tests (address)": {
"required_build_properties": {
"compiler": "clang-11",
@ -381,6 +393,18 @@
"with_coverage": false
}
},
"Functional stateless tests (release, DatabaseReplicated)": {
"required_build_properties": {
"compiler": "clang-11",
"package_type": "deb",
"build_type": "relwithdebuginfo",
"sanitizer": "none",
"bundled": "bundled",
"splitted": "unsplitted",
"clang-tidy": "disable",
"with_coverage": false
}
},
"Stress test (address)": {
"required_build_properties": {
"compiler": "clang-11",

View File

@ -105,7 +105,9 @@ def remove_control_characters(s):
s = re.sub(r"[\x00-\x08\x0b\x0e-\x1f\x7f]", "", s)
return s
def get_db_engine(args):
def get_db_engine(args, database_name):
if args.replicated_database:
return " ENGINE=Replicated('/test/clickhouse/db/{}', 's1', 'r1')".format(database_name)
if args.db_engine:
return " ENGINE=" + args.db_engine
return "" # Will use default engine
@ -128,7 +130,7 @@ def run_single_test(args, ext, server_logs_level, client_options, case_file, std
clickhouse_proc_create = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE, universal_newlines=True)
try:
clickhouse_proc_create.communicate(("CREATE DATABASE " + database + get_db_engine(args)), timeout=args.timeout)
clickhouse_proc_create.communicate(("CREATE DATABASE " + database + get_db_engine(args, database)), timeout=args.timeout)
except TimeoutExpired:
total_time = (datetime.now() - start_time).total_seconds()
return clickhouse_proc_create, "", "Timeout creating database {} before test".format(database), total_time
@ -161,7 +163,12 @@ def run_single_test(args, ext, server_logs_level, client_options, case_file, std
while (datetime.now() - start_time).total_seconds() < args.timeout and proc.poll() is None:
sleep(0.01)
if not args.database:
need_drop_database = not args.database
if need_drop_database and args.no_drop_if_fail:
maybe_passed = (proc.returncode == 0) and (proc.stderr is None) and (proc.stdout is None or 'Exception' not in proc.stdout)
need_drop_database = not maybe_passed
if need_drop_database:
clickhouse_proc_create = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE, universal_newlines=True)
seconds_left = max(args.timeout - (datetime.now() - start_time).total_seconds(), 10)
try:
@ -182,7 +189,8 @@ def run_single_test(args, ext, server_logs_level, client_options, case_file, std
# Normalize randomized database names in stdout, stderr files.
os.system("LC_ALL=C sed -i -e 's/{test_db}/default/g' {file}".format(test_db=database, file=stdout_file))
os.system("LC_ALL=C sed -i -e 's/{test_db}/default/g' {file}".format(test_db=database, file=stderr_file))
if not args.show_db_name:
os.system("LC_ALL=C sed -i -e 's/{test_db}/default/g' {file}".format(test_db=database, file=stderr_file))
stdout = open(stdout_file, 'rb').read() if os.path.exists(stdout_file) else b''
stdout = str(stdout, errors='replace', encoding='utf-8')
@ -526,6 +534,8 @@ class BuildFlags():
RELEASE = 'release-build'
DATABASE_ORDINARY = 'database-ordinary'
POLYMORPHIC_PARTS = 'polymorphic-parts'
ANTLR = 'antlr'
DATABASE_REPLICATED = 'database-replicated'
def collect_build_flags(client):
@ -607,7 +617,9 @@ def main(args):
build_flags = collect_build_flags(args.client)
if args.antlr:
build_flags.append('antlr')
build_flags.append(BuildFlags.ANTLR)
if args.replicated_database:
build_flags.append(BuildFlags.DATABASE_REPLICATED)
if args.use_skip_list:
tests_to_skip_from_list = collect_tests_to_skip(args.skip_list_path, build_flags)
@ -660,10 +672,10 @@ def main(args):
if args.database and args.database != "test":
clickhouse_proc_create = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE, universal_newlines=True)
clickhouse_proc_create.communicate(("CREATE DATABASE IF NOT EXISTS " + args.database + get_db_engine(args)))
clickhouse_proc_create.communicate(("CREATE DATABASE IF NOT EXISTS " + args.database + get_db_engine(args, args.database)))
clickhouse_proc_create = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE, universal_newlines=True)
clickhouse_proc_create.communicate(("CREATE DATABASE IF NOT EXISTS test" + get_db_engine(args)))
clickhouse_proc_create.communicate(("CREATE DATABASE IF NOT EXISTS test" + get_db_engine(args, 'test')))
def is_test_from_dir(suite_dir, case):
case_file = os.path.join(suite_dir, case)
@ -907,6 +919,8 @@ if __name__ == '__main__':
parser.add_argument('--hung-check', action='store_true', default=False)
parser.add_argument('--force-color', action='store_true', default=False)
parser.add_argument('--database', help='Database for tests (random name test_XXXXXX by default)')
parser.add_argument('--no-drop-if-fail', action='store_true', help='Do not drop database for test if test has failed')
parser.add_argument('--show-db-name', action='store_true', help='Do not replace random database name with "default"')
parser.add_argument('--parallel', default='1/1', help='One parallel test run number/total')
parser.add_argument('-j', '--jobs', default=1, nargs='?', type=int, help='Run all tests in parallel')
parser.add_argument('--test-runs', default=1, nargs='?', type=int, help='Run each test many times (useful for e.g. flaky check)')
@ -915,6 +929,7 @@ if __name__ == '__main__':
parser.add_argument('--skip-list-path', help="Path to skip-list file")
parser.add_argument('--use-skip-list', action='store_true', default=False, help="Use skip list to skip tests if found")
parser.add_argument('--db-engine', help='Database engine name')
parser.add_argument('--replicated-database', action='store_true', default=False, help='Run tests with Replicated database engine')
parser.add_argument('--antlr', action='store_true', default=False, dest='antlr', help='Use new ANTLR parser in tests')
parser.add_argument('--no-stateless', action='store_true', help='Disable all stateless tests')

View File

@ -61,5 +61,8 @@ fi
if [[ -n "$USE_DATABASE_ORDINARY" ]] && [[ "$USE_DATABASE_ORDINARY" -eq 1 ]]; then
ln -sf $SRC_PATH/users.d/database_ordinary.xml $DEST_SERVER_PATH/users.d/
fi
if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
ln -sf $SRC_PATH/users.d/database_replicated.xml $DEST_SERVER_PATH/users.d/
fi
ln -sf $SRC_PATH/client_config.xml $DEST_CLIENT_PATH/config.xml

View File

@ -0,0 +1,10 @@
<yandex>
<profiles>
<default>
<allow_experimental_database_replicated>1</allow_experimental_database_replicated>
<database_replicated_ddl_output>0</database_replicated_ddl_output>
<database_replicated_initial_query_timeout_sec>30</database_replicated_initial_query_timeout_sec>
<distributed_ddl_task_timeout>30</distributed_ddl_task_timeout>
</default>
</profiles>
</yandex>

View File

@ -47,20 +47,20 @@ class TSV:
def assert_eq_with_retry(instance, query, expectation, retry_count=20, sleep_time=0.5, stdin=None, timeout=None,
settings=None, user=None, ignore_error=False):
settings=None, user=None, ignore_error=False, get_result=lambda x: x):
expectation_tsv = TSV(expectation)
for i in range(retry_count):
try:
if TSV(instance.query(query, user=user, stdin=stdin, timeout=timeout, settings=settings,
ignore_error=ignore_error)) == expectation_tsv:
if TSV(get_result(instance.query(query, user=user, stdin=stdin, timeout=timeout, settings=settings,
ignore_error=ignore_error))) == expectation_tsv:
break
time.sleep(sleep_time)
except Exception as ex:
print(("assert_eq_with_retry retry {} exception {}".format(i + 1, ex)))
time.sleep(sleep_time)
else:
val = TSV(instance.query(query, user=user, stdin=stdin, timeout=timeout, settings=settings,
ignore_error=ignore_error))
val = TSV(get_result(instance.query(query, user=user, stdin=stdin, timeout=timeout, settings=settings,
ignore_error=ignore_error)))
if expectation_tsv != val:
raise AssertionError("'{}' != '{}'\n{}".format(expectation_tsv, val, '\n'.join(
expectation_tsv.diff(val, n1="expectation", n2="query"))))

View File

@ -0,0 +1,34 @@
<yandex>
<database_atomic_delay_before_drop_table_sec>10</database_atomic_delay_before_drop_table_sec>
<remote_servers>
<cluster>
<shard>
<internal_replication>true</internal_replication>
<replica>
<host>main_node</host>
<port>9000</port>
</replica>
<replica>
<host>dummy_node</host>
<port>9000</port>
</replica>
<replica>
<host>competing_node</host>
<port>9000</port>
</replica>
</shard>
<shard>
<internal_replication>true</internal_replication>
<replica>
<host>snapshotting_node</host>
<port>9000</port>
</replica>
<replica>
<host>snapshot_recovering_node</host>
<port>9000</port>
</replica>
</shard>
</cluster>
</remote_servers>
</yandex>

View File

@ -0,0 +1,12 @@
<yandex>
<profiles>
<default>
<allow_experimental_database_replicated>1</allow_experimental_database_replicated>
</default>
</profiles>
<users>
<default>
<profile>default</profile>
</default>
</users>
</yandex>

View File

@ -0,0 +1,278 @@
import time
import re
import pytest
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import assert_eq_with_retry, assert_logs_contain
from helpers.network import PartitionManager
cluster = ClickHouseCluster(__file__)
main_node = cluster.add_instance('main_node', main_configs=['configs/config.xml'], user_configs=['configs/settings.xml'], with_zookeeper=True, stay_alive=True, macros={"shard": 1, "replica": 1})
dummy_node = cluster.add_instance('dummy_node', main_configs=['configs/config.xml'], user_configs=['configs/settings.xml'], with_zookeeper=True, stay_alive=True, macros={"shard": 1, "replica": 2})
competing_node = cluster.add_instance('competing_node', main_configs=['configs/config.xml'], user_configs=['configs/settings.xml'], with_zookeeper=True, macros={"shard": 1, "replica": 3})
snapshotting_node = cluster.add_instance('snapshotting_node', main_configs=['configs/config.xml'], user_configs=['configs/settings.xml'], with_zookeeper=True, macros={"shard": 2, "replica": 1})
snapshot_recovering_node = cluster.add_instance('snapshot_recovering_node', main_configs=['configs/config.xml'], user_configs=['configs/settings.xml'], with_zookeeper=True, macros={"shard": 2, "replica": 2})
all_nodes = [main_node, dummy_node, competing_node, snapshotting_node, snapshot_recovering_node]
uuid_regex = re.compile("[0-9a-f]{8}\-[0-9a-f]{4}\-[0-9a-f]{4}\-[0-9a-f]{4}\-[0-9a-f]{12}")
def assert_create_query(nodes, table_name, expected):
replace_uuid = lambda x: re.sub(uuid_regex, "uuid", x)
query = "show create table {}".format(table_name)
for node in nodes:
assert_eq_with_retry(node, query, expected, get_result=replace_uuid)
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
main_node.query("CREATE DATABASE testdb ENGINE = Replicated('/clickhouse/databases/test1', 'shard1', 'replica1');")
dummy_node.query("CREATE DATABASE testdb ENGINE = Replicated('/clickhouse/databases/test1', 'shard1', 'replica2');")
yield cluster
finally:
cluster.shutdown()
def test_create_replicated_table(started_cluster):
assert "Old syntax is not allowed" in \
main_node.query_and_get_error("CREATE TABLE testdb.replicated_table (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/test/tmp', 'r', d, k, 8192);")
main_node.query("CREATE TABLE testdb.replicated_table (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree ORDER BY k PARTITION BY toYYYYMM(d);")
expected = "CREATE TABLE testdb.replicated_table\\n(\\n `d` Date,\\n `k` UInt64,\\n `i32` Int32\\n)\\n" \
"ENGINE = ReplicatedMergeTree(\\'/clickhouse/tables/uuid/{shard}\\', \\'{replica}\\')\\n" \
"PARTITION BY toYYYYMM(d)\\nORDER BY k\\nSETTINGS index_granularity = 8192"
assert_create_query([main_node, dummy_node], "testdb.replicated_table", expected)
# assert without replacing uuid
assert main_node.query("show create testdb.replicated_table") == dummy_node.query("show create testdb.replicated_table")
@pytest.mark.parametrize("engine", ['MergeTree', 'ReplicatedMergeTree'])
def test_simple_alter_table(started_cluster, engine):
# test_simple_alter_table
name = "testdb.alter_test_{}".format(engine)
main_node.query("CREATE TABLE {} "
"(CounterID UInt32, StartDate Date, UserID UInt32, VisitID UInt32, NestedColumn Nested(A UInt8, S String), ToDrop UInt32) "
"ENGINE = {} PARTITION BY StartDate ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID);".format(name, engine))
main_node.query("ALTER TABLE {} ADD COLUMN Added0 UInt32;".format(name))
main_node.query("ALTER TABLE {} ADD COLUMN Added2 UInt32;".format(name))
main_node.query("ALTER TABLE {} ADD COLUMN Added1 UInt32 AFTER Added0;".format(name))
main_node.query("ALTER TABLE {} ADD COLUMN AddedNested1 Nested(A UInt32, B UInt64) AFTER Added2;".format(name))
main_node.query("ALTER TABLE {} ADD COLUMN AddedNested1.C Array(String) AFTER AddedNested1.B;".format(name))
main_node.query("ALTER TABLE {} ADD COLUMN AddedNested2 Nested(A UInt32, B UInt64) AFTER AddedNested1;".format(name))
full_engine = engine if not "Replicated" in engine else engine + "(\\'/clickhouse/tables/uuid/{shard}\\', \\'{replica}\\')"
expected = "CREATE TABLE {}\\n(\\n `CounterID` UInt32,\\n `StartDate` Date,\\n `UserID` UInt32,\\n" \
" `VisitID` UInt32,\\n `NestedColumn.A` Array(UInt8),\\n `NestedColumn.S` Array(String),\\n" \
" `ToDrop` UInt32,\\n `Added0` UInt32,\\n `Added1` UInt32,\\n `Added2` UInt32,\\n" \
" `AddedNested1.A` Array(UInt32),\\n `AddedNested1.B` Array(UInt64),\\n `AddedNested1.C` Array(String),\\n" \
" `AddedNested2.A` Array(UInt32),\\n `AddedNested2.B` Array(UInt64)\\n)\\n" \
"ENGINE = {}\\nPARTITION BY StartDate\\nORDER BY (CounterID, StartDate, intHash32(UserID), VisitID)\\n" \
"SETTINGS index_granularity = 8192".format(name, full_engine)
assert_create_query([main_node, dummy_node], name, expected)
# test_create_replica_after_delay
competing_node.query("CREATE DATABASE IF NOT EXISTS testdb ENGINE = Replicated('/clickhouse/databases/test1', 'shard1', 'replica3');")
name = "testdb.alter_test_{}".format(engine)
main_node.query("ALTER TABLE {} ADD COLUMN Added3 UInt32;".format(name))
main_node.query("ALTER TABLE {} DROP COLUMN AddedNested1;".format(name))
main_node.query("ALTER TABLE {} RENAME COLUMN Added1 TO AddedNested1;".format(name))
full_engine = engine if not "Replicated" in engine else engine + "(\\'/clickhouse/tables/uuid/{shard}\\', \\'{replica}\\')"
expected = "CREATE TABLE {}\\n(\\n `CounterID` UInt32,\\n `StartDate` Date,\\n `UserID` UInt32,\\n" \
" `VisitID` UInt32,\\n `NestedColumn.A` Array(UInt8),\\n `NestedColumn.S` Array(String),\\n" \
" `ToDrop` UInt32,\\n `Added0` UInt32,\\n `AddedNested1` UInt32,\\n `Added2` UInt32,\\n" \
" `AddedNested2.A` Array(UInt32),\\n `AddedNested2.B` Array(UInt64),\\n `Added3` UInt32\\n)\\n" \
"ENGINE = {}\\nPARTITION BY StartDate\\nORDER BY (CounterID, StartDate, intHash32(UserID), VisitID)\\n" \
"SETTINGS index_granularity = 8192".format(name, full_engine)
assert_create_query([main_node, dummy_node, competing_node], name, expected)
def test_alters_from_different_replicas(started_cluster):
# test_alters_from_different_replicas
competing_node.query("CREATE DATABASE IF NOT EXISTS testdb ENGINE = Replicated('/clickhouse/databases/test1', 'shard1', 'replica3');")
main_node.query("CREATE TABLE testdb.concurrent_test "
"(CounterID UInt32, StartDate Date, UserID UInt32, VisitID UInt32, NestedColumn Nested(A UInt8, S String), ToDrop UInt32) "
"ENGINE = MergeTree(StartDate, intHash32(UserID), (CounterID, StartDate, intHash32(UserID), VisitID), 8192);")
main_node.query("CREATE TABLE testdb.dist AS testdb.concurrent_test ENGINE = Distributed(cluster, testdb, concurrent_test, CounterID)")
dummy_node.stop_clickhouse(kill=True)
settings = {"distributed_ddl_task_timeout": 10}
assert "There are 1 unfinished hosts (0 of them are currently active)" in \
competing_node.query_and_get_error("ALTER TABLE testdb.concurrent_test ADD COLUMN Added0 UInt32;", settings=settings)
dummy_node.start_clickhouse()
main_node.query("ALTER TABLE testdb.concurrent_test ADD COLUMN Added2 UInt32;")
competing_node.query("ALTER TABLE testdb.concurrent_test ADD COLUMN Added1 UInt32 AFTER Added0;")
main_node.query("ALTER TABLE testdb.concurrent_test ADD COLUMN AddedNested1 Nested(A UInt32, B UInt64) AFTER Added2;")
competing_node.query("ALTER TABLE testdb.concurrent_test ADD COLUMN AddedNested1.C Array(String) AFTER AddedNested1.B;")
main_node.query("ALTER TABLE testdb.concurrent_test ADD COLUMN AddedNested2 Nested(A UInt32, B UInt64) AFTER AddedNested1;")
expected = "CREATE TABLE testdb.concurrent_test\\n(\\n `CounterID` UInt32,\\n `StartDate` Date,\\n `UserID` UInt32,\\n" \
" `VisitID` UInt32,\\n `NestedColumn.A` Array(UInt8),\\n `NestedColumn.S` Array(String),\\n `ToDrop` UInt32,\\n" \
" `Added0` UInt32,\\n `Added1` UInt32,\\n `Added2` UInt32,\\n `AddedNested1.A` Array(UInt32),\\n" \
" `AddedNested1.B` Array(UInt64),\\n `AddedNested1.C` Array(String),\\n `AddedNested2.A` Array(UInt32),\\n" \
" `AddedNested2.B` Array(UInt64)\\n)\\n" \
"ENGINE = MergeTree(StartDate, intHash32(UserID), (CounterID, StartDate, intHash32(UserID), VisitID), 8192)"
assert_create_query([main_node, competing_node], "testdb.concurrent_test", expected)
# test_create_replica_after_delay
main_node.query("DROP TABLE testdb.concurrent_test")
main_node.query("CREATE TABLE testdb.concurrent_test "
"(CounterID UInt32, StartDate Date, UserID UInt32, VisitID UInt32, NestedColumn Nested(A UInt8, S String), ToDrop UInt32) "
"ENGINE = ReplicatedMergeTree ORDER BY CounterID;")
expected = "CREATE TABLE testdb.concurrent_test\\n(\\n `CounterID` UInt32,\\n `StartDate` Date,\\n `UserID` UInt32,\\n" \
" `VisitID` UInt32,\\n `NestedColumn.A` Array(UInt8),\\n `NestedColumn.S` Array(String),\\n `ToDrop` UInt32\\n)\\n" \
"ENGINE = ReplicatedMergeTree(\\'/clickhouse/tables/uuid/{shard}\\', \\'{replica}\\')\\nORDER BY CounterID\\nSETTINGS index_granularity = 8192"
assert_create_query([main_node, competing_node], "testdb.concurrent_test", expected)
main_node.query("INSERT INTO testdb.dist (CounterID, StartDate, UserID) SELECT number, addDays(toDate('2020-02-02'), number), intHash32(number) FROM numbers(10)")
# test_replica_restart
main_node.restart_clickhouse()
expected = "CREATE TABLE testdb.concurrent_test\\n(\\n `CounterID` UInt32,\\n `StartDate` Date,\\n `UserID` UInt32,\\n" \
" `VisitID` UInt32,\\n `NestedColumn.A` Array(UInt8),\\n `NestedColumn.S` Array(String),\\n `ToDrop` UInt32\\n)\\n" \
"ENGINE = ReplicatedMergeTree(\\'/clickhouse/tables/uuid/{shard}\\', \\'{replica}\\')\\nORDER BY CounterID\\nSETTINGS index_granularity = 8192"
# test_snapshot_and_snapshot_recover
snapshotting_node.query("CREATE DATABASE testdb ENGINE = Replicated('/clickhouse/databases/test1', 'shard2', 'replica1');")
snapshot_recovering_node.query("CREATE DATABASE testdb ENGINE = Replicated('/clickhouse/databases/test1', 'shard2', 'replica2');")
assert_create_query(all_nodes, "testdb.concurrent_test", expected)
main_node.query("SYSTEM FLUSH DISTRIBUTED testdb.dist")
main_node.query("ALTER TABLE testdb.concurrent_test UPDATE StartDate = addYears(StartDate, 1) WHERE 1")
res = main_node.query("ALTER TABLE testdb.concurrent_test DELETE WHERE UserID % 2")
assert "shard1|replica1" in res and "shard1|replica2" in res and "shard1|replica3" in res
assert "shard2|replica1" in res and "shard2|replica2" in res
expected = "1\t1\tmain_node\n" \
"1\t2\tdummy_node\n" \
"1\t3\tcompeting_node\n" \
"2\t1\tsnapshotting_node\n" \
"2\t2\tsnapshot_recovering_node\n"
assert main_node.query("SELECT shard_num, replica_num, host_name FROM system.clusters WHERE cluster='testdb'") == expected
# test_drop_and_create_replica
main_node.query("DROP DATABASE testdb SYNC")
main_node.query("CREATE DATABASE testdb ENGINE = Replicated('/clickhouse/databases/test1', 'shard1', 'replica1');")
expected = "CREATE TABLE testdb.concurrent_test\\n(\\n `CounterID` UInt32,\\n `StartDate` Date,\\n `UserID` UInt32,\\n" \
" `VisitID` UInt32,\\n `NestedColumn.A` Array(UInt8),\\n `NestedColumn.S` Array(String),\\n `ToDrop` UInt32\\n)\\n" \
"ENGINE = ReplicatedMergeTree(\\'/clickhouse/tables/uuid/{shard}\\', \\'{replica}\\')\\nORDER BY CounterID\\nSETTINGS index_granularity = 8192"
assert_create_query([main_node, competing_node], "testdb.concurrent_test", expected)
assert_create_query(all_nodes, "testdb.concurrent_test", expected)
for node in all_nodes:
node.query("SYSTEM SYNC REPLICA testdb.concurrent_test")
expected = "0\t2021-02-02\t4249604106\n" \
"1\t2021-02-03\t1343103100\n" \
"4\t2021-02-06\t3902320246\n" \
"7\t2021-02-09\t3844986530\n" \
"9\t2021-02-11\t1241149650\n"
assert_eq_with_retry(dummy_node, "SELECT CounterID, StartDate, UserID FROM testdb.dist ORDER BY CounterID", expected)
def test_recover_staled_replica(started_cluster):
main_node.query("CREATE DATABASE recover ENGINE = Replicated('/clickhouse/databases/recover', 'shard1', 'replica1');")
started_cluster.get_kazoo_client('zoo1').set('/clickhouse/databases/recover/logs_to_keep', b'10')
dummy_node.query("CREATE DATABASE recover ENGINE = Replicated('/clickhouse/databases/recover', 'shard1', 'replica2');")
settings = {"distributed_ddl_task_timeout": 0}
main_node.query("CREATE TABLE recover.t1 (n int) ENGINE=Memory", settings=settings)
dummy_node.query("CREATE TABLE recover.t2 (s String) ENGINE=Memory", settings=settings)
main_node.query("CREATE TABLE recover.mt1 (n int) ENGINE=MergeTree order by n", settings=settings)
dummy_node.query("CREATE TABLE recover.mt2 (n int) ENGINE=MergeTree order by n", settings=settings)
main_node.query("CREATE TABLE recover.rmt1 (n int) ENGINE=ReplicatedMergeTree order by n", settings=settings)
dummy_node.query("CREATE TABLE recover.rmt2 (n int) ENGINE=ReplicatedMergeTree order by n", settings=settings)
main_node.query("CREATE TABLE recover.rmt3 (n int) ENGINE=ReplicatedMergeTree order by n", settings=settings)
dummy_node.query("CREATE TABLE recover.rmt5 (n int) ENGINE=ReplicatedMergeTree order by n", settings=settings)
main_node.query("CREATE DICTIONARY recover.d1 (n int DEFAULT 0, m int DEFAULT 1) PRIMARY KEY n SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'rmt1' PASSWORD '' DB 'recover')) LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT())")
dummy_node.query("CREATE DICTIONARY recover.d2 (n int DEFAULT 0, m int DEFAULT 1) PRIMARY KEY n SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'rmt2' PASSWORD '' DB 'recover')) LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT())")
for table in ['t1', 't2', 'mt1', 'mt2', 'rmt1', 'rmt2', 'rmt3', 'rmt5']:
main_node.query("INSERT INTO recover.{} VALUES (42)".format(table))
for table in ['t1', 't2', 'mt1', 'mt2']:
dummy_node.query("INSERT INTO recover.{} VALUES (42)".format(table))
for table in ['rmt1', 'rmt2', 'rmt3', 'rmt5']:
main_node.query("SYSTEM SYNC REPLICA recover.{}".format(table))
with PartitionManager() as pm:
pm.drop_instance_zk_connections(dummy_node)
dummy_node.query_and_get_error("RENAME TABLE recover.t1 TO recover.m1")
main_node.query("RENAME TABLE recover.t1 TO recover.m1", settings=settings)
main_node.query("ALTER TABLE recover.mt1 ADD COLUMN m int", settings=settings)
main_node.query("ALTER TABLE recover.rmt1 ADD COLUMN m int", settings=settings)
main_node.query("RENAME TABLE recover.rmt3 TO recover.rmt4", settings=settings)
main_node.query("DROP TABLE recover.rmt5", settings=settings)
main_node.query("DROP DICTIONARY recover.d2", settings=settings)
main_node.query("CREATE DICTIONARY recover.d2 (n int DEFAULT 0, m int DEFAULT 1) PRIMARY KEY n SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'rmt1' PASSWORD '' DB 'recover')) LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT());", settings=settings)
main_node.query("CREATE TABLE recover.tmp AS recover.m1", settings=settings)
main_node.query("DROP TABLE recover.tmp", settings=settings)
main_node.query("CREATE TABLE recover.tmp AS recover.m1", settings=settings)
main_node.query("DROP TABLE recover.tmp", settings=settings)
main_node.query("CREATE TABLE recover.tmp AS recover.m1", settings=settings)
main_node.query("DROP TABLE recover.tmp", settings=settings)
main_node.query("CREATE TABLE recover.tmp AS recover.m1", settings=settings)
assert main_node.query("SELECT name FROM system.tables WHERE database='recover' ORDER BY name") == "d1\nd2\nm1\nmt1\nmt2\nrmt1\nrmt2\nrmt4\nt2\ntmp\n"
query = "SELECT name, uuid, create_table_query FROM system.tables WHERE database='recover' ORDER BY name"
expected = main_node.query(query)
assert_eq_with_retry(dummy_node, query, expected)
for table in ['m1', 't2', 'mt1', 'mt2', 'rmt1', 'rmt2', 'rmt4', 'd1', 'd2']:
assert main_node.query("SELECT (*,).1 FROM recover.{}".format(table)) == "42\n"
for table in ['t2', 'rmt1', 'rmt2', 'rmt4', 'd1', 'd2', 'mt2']:
assert dummy_node.query("SELECT (*,).1 FROM recover.{}".format(table)) == "42\n"
for table in ['m1', 'mt1']:
assert dummy_node.query("SELECT count() FROM recover.{}".format(table)) == "0\n"
assert dummy_node.query("SELECT count() FROM system.tables WHERE database='recover_broken_tables'") == "2\n"
table = dummy_node.query("SHOW TABLES FROM recover_broken_tables LIKE 'mt1_26_%'").strip()
assert dummy_node.query("SELECT (*,).1 FROM recover_broken_tables.{}".format(table)) == "42\n"
table = dummy_node.query("SHOW TABLES FROM recover_broken_tables LIKE 'rmt5_26_%'").strip()
assert dummy_node.query("SELECT (*,).1 FROM recover_broken_tables.{}".format(table)) == "42\n"
expected = "Cleaned 4 outdated objects: dropped 1 dictionaries and 1 tables, moved 2 tables"
assert_logs_contain(dummy_node, expected)
dummy_node.query("DROP TABLE recover.tmp")
assert_eq_with_retry(main_node, "SELECT count() FROM system.tables WHERE database='recover' AND name='tmp'", "0\n")
def test_startup_without_zk(started_cluster):
main_node.query("DROP DATABASE IF EXISTS testdb SYNC")
main_node.query("DROP DATABASE IF EXISTS recover SYNC")
with PartitionManager() as pm:
pm.drop_instance_zk_connections(main_node)
err = main_node.query_and_get_error("CREATE DATABASE startup ENGINE = Replicated('/clickhouse/databases/startup', 'shard1', 'replica1');")
assert "ZooKeeper" in err
main_node.query("CREATE DATABASE startup ENGINE = Replicated('/clickhouse/databases/startup', 'shard1', 'replica1');")
#main_node.query("CREATE TABLE startup.rmt (n int) ENGINE=ReplicatedMergeTree order by n")
main_node.query("CREATE TABLE startup.rmt (n int) ENGINE=MergeTree order by n")
main_node.query("INSERT INTO startup.rmt VALUES (42)")
with PartitionManager() as pm:
pm.drop_instance_zk_connections(main_node)
main_node.restart_clickhouse(stop_start_wait_sec=30)
assert main_node.query("SELECT (*,).1 FROM startup.rmt") == "42\n"
for _ in range(10):
try:
main_node.query("CREATE TABLE startup.m (n int) ENGINE=Memory")
break
except:
time.sleep(1)
main_node.query("EXCHANGE TABLES startup.rmt AND startup.m")
assert main_node.query("SELECT (*,).1 FROM startup.m") == "42\n"

View File

@ -113,8 +113,8 @@ timeout $TIMEOUT bash -c thread7 2> /dev/null &
wait
$CLICKHOUSE_CLIENT -q "SELECT 'Still alive'"
$CLICKHOUSE_CLIENT -q "ATTACH DICTIONARY database_for_dict.dict1"
$CLICKHOUSE_CLIENT -q "ATTACH DICTIONARY database_for_dict.dict2"
$CLICKHOUSE_CLIENT -q "ATTACH DICTIONARY IF NOT EXISTS database_for_dict.dict1"
$CLICKHOUSE_CLIENT -q "ATTACH DICTIONARY IF NOT EXISTS database_for_dict.dict2"
$CLICKHOUSE_CLIENT -n -q "
DROP TABLE table_for_dict1;

View File

@ -18,3 +18,6 @@ yes 'SELECT 1' 2>/dev/null | {
} | grep -x -c 1
wait
# Reset max_memory_usage_for_user, so it will not affect other tests
${CLICKHOUSE_CLIENT} --max_memory_usage_for_user=0 -q "SELECT 1 FORMAT Null"

View File

@ -42,3 +42,6 @@ execute_group_by
# if memory accounting will be incorrect, the second query will be failed with MEMORY_LIMIT_EXCEEDED
execute_group_by
wait
# Reset max_memory_usage_for_user, so it will not affect other tests
${CLICKHOUSE_CLIENT} --max_memory_usage_for_user=0 -q "SELECT 1 FORMAT Null"

View File

@ -66,4 +66,7 @@ echo 'OK'
${CLICKHOUSE_CLIENT} --query "DROP USER test_01541";
# Reset max_memory_usage_for_user, so it will not affect other tests
${CLICKHOUSE_CLIENT} --max_memory_usage_for_user=0 -q "SELECT 1 FORMAT Null"
exit 0

View File

@ -105,6 +105,158 @@
"00510_materizlized_view_and_deduplication_zookeeper",
"00738_lock_for_inner_table"
],
"database-replicated": [
/// Tests with DETACH TABLE (it's not allowed)
/// and tests with SET (session and query settings are not supported)
"memory_tracking",
"memory_usage",
"live_view",
"00825_protobuf_format_map",
"00152_insert_different_granularity",
"01715_background_checker_blather_zookeeper",
"01714_alter_drop_version",
"01114_materialize_clear_index_compact_parts",
"00814_replicated_minimalistic_part_header_zookeeper",
"01188_attach_table_from_pat",
"01415_sticking_mutations",
"01130_in_memory_parts",
"01110_dictionary_layout_without_arguments",
"01018_ddl_dictionaries_create",
"01018_ddl_dictionaries_select",
"01414_freeze_does_not_prevent_alters",
"01018_ddl_dictionaries_bad_queries",
"01686_rocksdb",
"01550_mutation_subquery",
"01070_mutations_with_dependencies",
"01070_materialize_ttl",
"01055_compact_parts",
"01017_mutations_with_nondeterministic_functions_zookeeper",
"00926_adaptive_index_granularity_pk",
"00910_zookeeper_test_alter_compression_codecs",
"00908_bloom_filter_index",
"00616_final_single_part",
"00446_clear_column_in_partition_zookeeper",
"01533_multiple_nested",
"01213_alter_rename_column_zookeeper",
"01575_disable_detach_table_of_dictionary",
"01457_create_as_table_function_structure",
"01415_inconsistent_merge_tree_settings",
"01413_allow_non_metadata_alters",
"01378_alter_rename_with_ttl_zookeeper",
"01349_mutation_datetime_key",
"01325_freeze_mutation_stuck",
"01272_suspicious_codecs",
"01181_db_atomic_drop_on_cluster",
"00957_delta_diff_bug",
"00910_zookeeper_custom_compression_codecs_replicated",
"00899_long_attach_memory_limit",
"00804_test_custom_compression_codes_log_storages",
"00804_test_alter_compression_codecs",
"00804_test_delta_codec_no_type_alter",
"00804_test_custom_compression_codecs",
"00753_alter_attach",
"00715_fetch_merged_or_mutated_part_zookeeper",
"00688_low_cardinality_serialization",
"01575_disable_detach_table_of_dictionary",
"00738_lock_for_inner_table",
"01666_blns",
"01652_ignore_and_low_cardinality",
"01651_map_functions",
"01650_fetch_patition_with_macro_in_zk_path",
"01648_mutations_and_escaping",
"01640_marks_corruption_regression",
"01622_byte_size",
"01611_string_to_low_cardinality_key_alter",
"01602_show_create_view",
"01600_log_queries_with_extensive_info",
"01560_ttl_remove_empty_parts",
"01554_bloom_filter_index_big_integer_uuid",
"01550_type_map_formats_input",
"01550_type_map_formats",
"01550_create_map_type",
"01532_primary_key_without_order_by_zookeeper",
"01511_alter_version_versioned_collapsing_merge_tree_zookeeper",
"01509_parallel_quorum_insert_no_replicas",
"01504_compression_multiple_streams",
"01494_storage_join_persistency",
"01493_storage_set_persistency",
"01493_alter_remove_properties_zookeeper",
"01475_read_subcolumns_storages",
"01475_read_subcolumns",
"01451_replicated_detach_drop_part",
"01451_detach_drop_part",
"01440_big_int_exotic_casts",
"01430_modify_sample_by_zookeeper",
"01417_freeze_partition_verbose_zookeeper",
"01417_freeze_partition_verbose",
"01396_inactive_replica_cleanup_nodes_zookeeper",
"01375_compact_parts_codecs",
"01357_version_collapsing_attach_detach_zookeeper",
"01355_alter_column_with_order",
"01291_geo_types",
"01270_optimize_skip_unused_shards_low_cardinality",
"01182_materialized_view_different_structure",
"01150_ddl_guard_rwr",
"01148_zookeeper_path_macros_unfolding",
"01135_default_and_alter_zookeeper",
"01130_in_memory_parts_partitons",
"01127_month_partitioning_consistency_select",
"01114_database_atomic",
"01083_expressions_in_engine_arguments",
"01073_attach_if_not_exists",
"01072_optimize_skip_unused_shards_const_expr_eval",
"01071_prohibition_secondary_index_with_old_format_merge_tree",
"01062_alter_on_mutataion_zookeeper",
"01060_shutdown_table_after_detach",
"01056_create_table_as",
"01035_avg",
"01021_only_tuple_columns",
"01019_alter_materialized_view_query",
"01019_alter_materialized_view_consistent",
"01019_alter_materialized_view_atomic",
"01015_attach_part",
"00989_parallel_parts_loading",
"00980_zookeeper_merge_tree_alter_settings",
"00980_merge_alter_settings",
"00955_test_final_mark",
"00933_reserved_word",
"00926_zookeeper_adaptive_index_granularity_replicated_merge_tree",
"00926_adaptive_index_granularity_replacing_merge_tree",
"00926_adaptive_index_granularity_merge_tree",
"00925_zookeeper_empty_replicated_merge_tree_optimize_final",
"00800_low_cardinality_distinct_numeric",
"00754_alter_modify_order_by_replicated_zookeeper",
"00751_low_cardinality_nullable_group_by",
"00751_default_databasename_for_view",
"00719_parallel_ddl_table",
"00718_low_cardinaliry_alter",
"00717_low_cardinaliry_distributed_group_by",
"00688_low_cardinality_syntax",
"00688_low_cardinality_nullable_cast",
"00688_low_cardinality_in",
"00652_replicated_mutations_zookeeper",
"00634_rename_view",
"00626_replace_partition_from_table",
"00625_arrays_in_nested",
"00623_replicated_truncate_table_zookeeper",
"00619_union_highlite",
"00599_create_view_with_subquery",
"00571_non_exist_database_when_create_materializ_view",
"00553_buff_exists_materlized_column",
"00516_deduplication_after_drop_partition_zookeeper",
"00508_materialized_view_to",
"00446_clear_column_in_partition_concurrent_zookeeper",
"00423_storage_log_single_thread",
"00311_array_primary_key",
"00236_replicated_drop_on_non_leader_zookeeper",
"00226_zookeeper_deduplication_and_unexpected_parts",
"00215_primary_key_order_zookeeper",
"00180_attach_materialized_view",
"00121_drop_column_zookeeper",
"00116_storage_set",
"00083_create_merge_tree_zookeeper",
"00062_replicated_merge_tree_alter_zookeeper"
],
"polymorphic-parts": [
"01508_partition_pruning_long", /// bug, shoud be fixed
"01482_move_to_prewhere_and_cast" /// bug, shoud be fixed
@ -161,6 +313,7 @@
"01015_attach_part",
"01015_database_bad_tables",
"01017_uniqCombined_memory_usage",
"01018_ddl_dictionaries_concurrent_requrests", /// Cannot parse ATTACH DICTIONARY IF NOT EXISTS
"01019_alter_materialized_view_atomic",
"01019_alter_materialized_view_consistent",
"01019_alter_materialized_view_query",
@ -290,7 +443,7 @@
"01530_drop_database_atomic_sync",
"01532_execute_merges_on_single_replica",
"01532_primary_key_without_order_by_zookeeper",
"01541_max_memory_usage_for_user",
"01541_max_memory_usage_for_user_long",
"01551_mergetree_read_in_order_spread",
"01552_dict_fixedstring",
"01554_bloom_filter_index_big_integer_uuid",
@ -567,7 +720,7 @@
"01527_clickhouse_local_optimize",
"01527_dist_sharding_key_dictGet_reload",
"01530_drop_database_atomic_sync",
"01541_max_memory_usage_for_user",
"01541_max_memory_usage_for_user_long",
"01542_dictionary_load_exception_race",
"01575_disable_detach_table_of_dictionary",
"01593_concurrent_alter_mutations_kill",