diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 24bab42cad2..e5d2b23ace0 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -460,7 +460,7 @@ void DatabaseOnDisk::renameTable( if (from_atomic_to_ordinary) { - auto & atomic_db = assert_cast(*this); + auto & atomic_db = dynamic_cast(*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()) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index d0a1e4d37bf..766b14dea42 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -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); } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index f6ee28aca22..8b59b225480 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -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 diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index f08f47b1c0e..c342a994395 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -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(); 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); diff --git a/src/Interpreters/DDLWorker.h b/src/Interpreters/DDLWorker.h index 0985884eef7..c39a832c098 100644 --- a/src/Interpreters/DDLWorker.h +++ b/src/Interpreters/DDLWorker.h @@ -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(); diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index fb75a933910..32317968fe5 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -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(); @@ -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(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 diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 5c75fc0300b..52cef210748 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -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",