mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 01:22:04 +00:00
fix
This commit is contained in:
parent
00bf90186a
commit
e40917a9c5
@ -6,6 +6,7 @@
|
||||
#include <Access/Common/AccessRightsElement.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
|
||||
#include <Interpreters/processColumnTransformers.h>
|
||||
|
||||
@ -75,6 +76,9 @@ BlockIO InterpreterOptimizeQuery::execute()
|
||||
}
|
||||
}
|
||||
|
||||
if (auto * snapshot_data = dynamic_cast<MergeTreeData::SnapshotData *>(storage_snapshot->data.get()))
|
||||
snapshot_data->parts = {};
|
||||
|
||||
table->optimize(query_ptr, metadata_snapshot, ast.partition, ast.final, ast.deduplicate, column_names, getContext());
|
||||
|
||||
return {};
|
||||
|
@ -3220,13 +3220,7 @@ void MergeTreeData::outdateBrokenPartAndCloneToDetached(const DataPartPtr & part
|
||||
LOG_INFO(log, "Cloning part {} to {}_{} and making it obsolete.", part_to_detach->data_part_storage->getPartDirectory(), prefix, part_to_detach->name);
|
||||
|
||||
part_to_detach->makeCloneInDetached(prefix, metadata_snapshot);
|
||||
|
||||
auto lock = lockParts();
|
||||
|
||||
if (auto it_part = data_parts_by_info.find(part_to_detach->info); it_part == data_parts_by_info.end())
|
||||
throw Exception("No such data part " + part_to_detach->getNameWithState(), ErrorCodes::NO_SUCH_DATA_PART);
|
||||
|
||||
removePartsFromWorkingSet(nullptr, {part_to_detach}, true, lock);
|
||||
removePartsFromWorkingSet(NO_TRANSACTION_RAW, {part_to_detach}, true);
|
||||
}
|
||||
|
||||
void MergeTreeData::forcefullyMovePartToDetachedAndRemoveFromMemory(const MergeTreeData::DataPartPtr & part_to_detach, const String & prefix, bool restore_covered)
|
||||
|
@ -24,7 +24,8 @@ def test_merge_and_part_corruption(started_cluster):
|
||||
node1.query(
|
||||
"""
|
||||
CREATE TABLE replicated_mt(date Date, id UInt32, value Int32)
|
||||
ENGINE = ReplicatedMergeTree('/clickhouse/tables/replicated_mt', '{replica}') ORDER BY id;
|
||||
ENGINE = ReplicatedMergeTree('/clickhouse/tables/replicated_mt', '{replica}') ORDER BY id
|
||||
SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1;
|
||||
""".format(
|
||||
replica=node1.name
|
||||
)
|
||||
|
@ -40,7 +40,8 @@ def remove_part_from_disk(node, table, part_name):
|
||||
def test_lost_part_same_replica(start_cluster):
|
||||
for node in [node1, node2]:
|
||||
node.query(
|
||||
"CREATE TABLE mt0 (id UInt64, date Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/t', '{}') ORDER BY tuple() PARTITION BY date".format(
|
||||
"CREATE TABLE mt0 (id UInt64, date Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/t', '{}') ORDER BY tuple() PARTITION BY date "
|
||||
"SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1".format(
|
||||
node.name
|
||||
)
|
||||
)
|
||||
@ -104,7 +105,8 @@ def test_lost_part_same_replica(start_cluster):
|
||||
def test_lost_part_other_replica(start_cluster):
|
||||
for node in [node1, node2]:
|
||||
node.query(
|
||||
"CREATE TABLE mt1 (id UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/t1', '{}') ORDER BY tuple()".format(
|
||||
"CREATE TABLE mt1 (id UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/t1', '{}') ORDER BY tuple() "
|
||||
"SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1".format(
|
||||
node.name
|
||||
)
|
||||
)
|
||||
@ -168,7 +170,8 @@ def test_lost_part_other_replica(start_cluster):
|
||||
def test_lost_part_mutation(start_cluster):
|
||||
for node in [node1, node2]:
|
||||
node.query(
|
||||
"CREATE TABLE mt2 (id UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/t2', '{}') ORDER BY tuple()".format(
|
||||
"CREATE TABLE mt2 (id UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/t2', '{}') ORDER BY tuple() "
|
||||
"SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1".format(
|
||||
node.name
|
||||
)
|
||||
)
|
||||
@ -225,7 +228,9 @@ def test_lost_last_part(start_cluster):
|
||||
for node in [node1, node2]:
|
||||
node.query(
|
||||
"CREATE TABLE mt3 (id UInt64, p String) ENGINE ReplicatedMergeTree('/clickhouse/tables/t3', '{}') "
|
||||
"ORDER BY tuple() PARTITION BY p".format(node.name)
|
||||
"ORDER BY tuple() PARTITION BY p SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1".format(
|
||||
node.name
|
||||
)
|
||||
)
|
||||
|
||||
node1.query("SYSTEM STOP MERGES mt3")
|
||||
|
Loading…
Reference in New Issue
Block a user