make force_drop_table work with materialized view

This commit is contained in:
Alexander Tokmakov 2021-02-17 17:23:07 +03:00
parent dab2356833
commit 13ae988eff
7 changed files with 62 additions and 35 deletions

View File

@ -759,7 +759,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
global_context->updateStorageConfiguration(*config);
},
/* already_loaded = */ true);
/* already_loaded = */ false);
auto & access_control = global_context->getAccessControlManager();
if (config().has("custom_settings_prefixes"))

View File

@ -106,8 +106,15 @@ 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)
{
if (auto * mv = dynamic_cast<StorageMaterializedView *>(tryGetTable(table_name, context).get()))
{
/// Remove the inner table (if any) to avoid deadlock
/// (due to attempt to execute DROP from the worker thread)
mv->dropInnerTable(no_delay);
}
String table_metadata_path = getObjectMetadataPath(table_name);
String table_metadata_path_drop;
StoragePtr table;
@ -121,10 +128,6 @@ void DatabaseAtomic::dropTable(const Context &, const String & table_name, bool
}
if (table->storesDataOnDisk())
tryRemoveSymlink(table_name);
/// 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);
/// 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);

View File

@ -389,32 +389,6 @@ Strings StorageMaterializedView::getDataPaths() const
return {};
}
void StorageMaterializedView::checkTableCanBeDropped() const
{
/// Don't drop the target table if it was created manually via 'TO inner_table' statement
if (!has_inner_table)
return;
auto target_table = tryGetTargetTable();
if (!target_table)
return;
target_table->checkTableCanBeDropped();
}
void StorageMaterializedView::checkPartitionCanBeDropped(const ASTPtr & partition)
{
/// Don't drop the partition in target table if it was created manually via 'TO inner_table' statement
if (!has_inner_table)
return;
auto target_table = tryGetTargetTable();
if (!target_table)
return;
target_table->checkPartitionCanBeDropped(partition);
}
ActionLock StorageMaterializedView::getActionLock(StorageActionBlockType type)
{
return has_inner_table ? getTargetTable()->getActionLock(type) : ActionLock{};

View File

@ -64,9 +64,6 @@ public:
void shutdown() override;
void checkTableCanBeDropped() const override;
void checkPartitionCanBeDropped(const ASTPtr & partition) override;
QueryProcessingStage::Enum getQueryProcessingStage(const Context &, QueryProcessingStage::Enum /*to_stage*/, SelectQueryInfo &) const override;
StoragePtr getTargetTable() const;

View File

@ -0,0 +1,4 @@
<yandex>
<max_table_size_to_drop>1</max_table_size_to_drop>
<max_partition_size_to_drop>1</max_partition_size_to_drop>
</yandex>

View File

@ -0,0 +1,49 @@
import pytest
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance('node', main_configs=["configs/config.xml"], with_zookeeper=True)
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def create_force_drop_flag(node):
force_drop_flag_path = "/var/lib/clickhouse/flags/force_drop_table"
node.exec_in_container(["bash", "-c", "touch {} && chmod a=rw {}".format(force_drop_flag_path, force_drop_flag_path)], user="root")
@pytest.mark.parametrize("engine", ['Ordinary', 'Atomic'])
def test_drop_materialized_view(started_cluster, engine):
node.query("CREATE DATABASE d ENGINE={}".format(engine))
node.query("CREATE TABLE d.rmt (n UInt64) ENGINE=ReplicatedMergeTree('/test/rmt', 'r1') ORDER BY n PARTITION BY n % 2")
node.query("CREATE MATERIALIZED VIEW d.mv (n UInt64, s String) ENGINE=MergeTree ORDER BY n PARTITION BY n % 2 AS SELECT n, toString(n) AS s FROM d.rmt")
node.query("INSERT INTO d.rmt VALUES (1), (2)")
assert "is greater than max" in node.query_and_get_error("DROP TABLE d.rmt")
assert "is greater than max" in node.query_and_get_error("DROP TABLE d.mv")
assert "is greater than max" in node.query_and_get_error("TRUNCATE TABLE d.rmt")
assert "is greater than max" in node.query_and_get_error("TRUNCATE TABLE d.mv")
assert "is greater than max" in node.query_and_get_error("ALTER TABLE d.rmt DROP PARTITION '0'")
assert node.query("SELECT * FROM d.rmt ORDER BY n") == "1\n2\n"
assert node.query("SELECT * FROM d.mv ORDER BY n") == "1\t1\n2\t2\n"
create_force_drop_flag(node)
node.query("ALTER TABLE d.rmt DROP PARTITION '0'")
assert node.query("SELECT * FROM d.rmt ORDER BY n") == "1\n"
assert "is greater than max" in node.query_and_get_error("ALTER TABLE d.mv DROP PARTITION '0'")
create_force_drop_flag(node)
node.query("ALTER TABLE d.mv DROP PARTITION '0'")
assert node.query("SELECT * FROM d.mv ORDER BY n") == "1\t1\n"
assert "is greater than max" in node.query_and_get_error("DROP TABLE d.rmt SYNC")
create_force_drop_flag(node)
node.query("DROP TABLE d.rmt SYNC")
assert "is greater than max" in node.query_and_get_error("DROP TABLE d.mv SYNC")
create_force_drop_flag(node)
node.query("DROP TABLE d.mv SYNC")
node.query("DROP DATABASE d")