mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-17 20:02:05 +00:00
make force_drop_table work with materialized view
This commit is contained in:
parent
dab2356833
commit
13ae988eff
@ -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"))
|
||||
|
@ -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);
|
||||
|
@ -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{};
|
||||
|
@ -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;
|
||||
|
0
tests/integration/test_force_drop_table/__init__.py
Normal file
0
tests/integration/test_force_drop_table/__init__.py
Normal 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>
|
49
tests/integration/test_force_drop_table/test.py
Normal file
49
tests/integration/test_force_drop_table/test.py
Normal 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")
|
||||
|
Loading…
Reference in New Issue
Block a user