This commit is contained in:
Alexander Tokmakov 2021-02-16 17:05:58 +03:00
parent bf6f64a3fb
commit 7b54b892b5
7 changed files with 34 additions and 11 deletions

View File

@ -460,7 +460,7 @@ void DatabaseOnDisk::renameTable(
if (from_atomic_to_ordinary)
{
auto & atomic_db = assert_cast<DatabaseAtomic &>(*this);
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())

View File

@ -2553,10 +2553,10 @@ StorageID Context::resolveStorageIDImpl(StorageID storage_id, StorageNamespace w
return StorageID::createEmpty();
}
void Context::initMetadataTransaction(MetadataTransactionPtr txn)
void Context::initMetadataTransaction(MetadataTransactionPtr txn, [[maybe_unused]] bool attach_existing)
{
assert(!metadata_transaction);
assert(query_context == this);
assert(attach_existing || query_context == this);
metadata_transaction = std::move(txn);
}

View File

@ -746,7 +746,7 @@ public:
IHostContextPtr & getHostContext();
const IHostContextPtr & getHostContext() const;
void initMetadataTransaction(MetadataTransactionPtr txn);
void initMetadataTransaction(MetadataTransactionPtr txn, bool attach_to_context = false);
MetadataTransactionPtr getMetadataTransaction() const;
struct MySQLWireContext

View File

@ -328,6 +328,8 @@ void DDLWorker::scheduleTasks()
LOG_TRACE(log, "No tasks to schedule");
return;
}
else if (max_tasks_in_queue < queue_nodes.size())
cleanup_event->set();
bool server_startup = current_tasks.empty();
auto begin_node = queue_nodes.begin();
@ -489,9 +491,8 @@ void DDLWorker::processTask(DDLTaskBase & task)
if (create_active_res == Coordination::Error::ZNODEEXISTS)
{
/// Connection has been lost and now we are retrying to write query status,
/// Connection has been lost and now we are retrying,
/// but our previous ephemeral node still exists.
assert(task.was_executed);
zkutil::EventPtr eph_node_disappeared = std::make_shared<Poco::Event>();
String dummy;
if (zookeeper->tryGet(active_node_path, dummy, nullptr, eph_node_disappeared))
@ -826,6 +827,7 @@ void DDLWorker::cleanupQueue(Int64, const ZooKeeperPtr & zookeeper)
ops.emplace_back(zkutil::makeRemoveRequest(fs::path(node_path) / "finished", -1));
ops.emplace_back(zkutil::makeRemoveRequest(node_path, -1));
auto rm_entry_res = zookeeper->tryMulti(ops, res);
if (rm_entry_res == Coordination::Error::ZNONODE)
{
/// Most likely both node_path/finished and node_path were removed concurrently.
@ -888,8 +890,11 @@ void DDLWorker::createStatusDirs(const std::string & node_path, const ZooKeeperP
return;
if (is_currently_deleting)
{
cleanup_event->set();
throw Exception(ErrorCodes::UNFINISHED, "Cannot create status dirs for {}, "
"most likely because someone is deleting it concurrently", node_path);
}
/// Connection lost or entry was removed
assert(Coordination::isHardwareError(code) || code == Coordination::Error::ZNONODE);

View File

@ -102,7 +102,7 @@ protected:
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();

View File

@ -194,9 +194,9 @@ BlockOutputStreamPtr StorageMaterializedView::write(const ASTPtr & query, const
}
static void executeDropQuery(ASTDropQuery::Kind kind, const 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>();
@ -206,7 +206,18 @@ static void executeDropQuery(ASTDropQuery::Kind kind, const Context & global_con
drop_query->no_delay = no_delay;
drop_query->if_exists = true;
ASTPtr ast_drop_query = drop_query;
/// 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.getMetadataTransaction())
{
/// For Replicated database
drop_context.setQueryContext(const_cast<Context &>(current_context));
drop_context.initMetadataTransaction(txn, true);
}
InterpreterDropQuery drop_interpreter(ast_drop_query, drop_context);
drop_interpreter.execute();
}
@ -226,13 +237,13 @@ void StorageMaterializedView::drop()
void StorageMaterializedView::dropInnerTable(bool no_delay, const Context & context)
{
if (has_inner_table && tryGetTargetTable())
executeDropQuery(ASTDropQuery::Kind::Drop, 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 & context, TableExclusiveLockHolder &)
{
if (has_inner_table)
executeDropQuery(ASTDropQuery::Kind::Truncate, context, target_table_id, true);
executeDropQuery(ASTDropQuery::Kind::Truncate, global_context, context, target_table_id, true);
}
void StorageMaterializedView::checkStatementCanBeForwarded() const

View File

@ -103,9 +103,16 @@
"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",
"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",