mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-29 11:02:08 +00:00
fix
This commit is contained in:
parent
00bf90186a
commit
e40917a9c5
@ -6,6 +6,7 @@
|
|||||||
#include <Access/Common/AccessRightsElement.h>
|
#include <Access/Common/AccessRightsElement.h>
|
||||||
#include <Common/typeid_cast.h>
|
#include <Common/typeid_cast.h>
|
||||||
#include <Parsers/ASTExpressionList.h>
|
#include <Parsers/ASTExpressionList.h>
|
||||||
|
#include <Storages/MergeTree/MergeTreeData.h>
|
||||||
|
|
||||||
#include <Interpreters/processColumnTransformers.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());
|
table->optimize(query_ptr, metadata_snapshot, ast.partition, ast.final, ast.deduplicate, column_names, getContext());
|
||||||
|
|
||||||
return {};
|
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);
|
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);
|
part_to_detach->makeCloneInDetached(prefix, metadata_snapshot);
|
||||||
|
removePartsFromWorkingSet(NO_TRANSACTION_RAW, {part_to_detach}, true);
|
||||||
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);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
void MergeTreeData::forcefullyMovePartToDetachedAndRemoveFromMemory(const MergeTreeData::DataPartPtr & part_to_detach, const String & prefix, bool restore_covered)
|
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(
|
node1.query(
|
||||||
"""
|
"""
|
||||||
CREATE TABLE replicated_mt(date Date, id UInt32, value Int32)
|
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(
|
""".format(
|
||||||
replica=node1.name
|
replica=node1.name
|
||||||
)
|
)
|
||||||
|
@ -40,7 +40,8 @@ def remove_part_from_disk(node, table, part_name):
|
|||||||
def test_lost_part_same_replica(start_cluster):
|
def test_lost_part_same_replica(start_cluster):
|
||||||
for node in [node1, node2]:
|
for node in [node1, node2]:
|
||||||
node.query(
|
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
|
node.name
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
@ -104,7 +105,8 @@ def test_lost_part_same_replica(start_cluster):
|
|||||||
def test_lost_part_other_replica(start_cluster):
|
def test_lost_part_other_replica(start_cluster):
|
||||||
for node in [node1, node2]:
|
for node in [node1, node2]:
|
||||||
node.query(
|
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
|
node.name
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
@ -168,7 +170,8 @@ def test_lost_part_other_replica(start_cluster):
|
|||||||
def test_lost_part_mutation(start_cluster):
|
def test_lost_part_mutation(start_cluster):
|
||||||
for node in [node1, node2]:
|
for node in [node1, node2]:
|
||||||
node.query(
|
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
|
node.name
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
@ -225,7 +228,9 @@ def test_lost_last_part(start_cluster):
|
|||||||
for node in [node1, node2]:
|
for node in [node1, node2]:
|
||||||
node.query(
|
node.query(
|
||||||
"CREATE TABLE mt3 (id UInt64, p String) ENGINE ReplicatedMergeTree('/clickhouse/tables/t3', '{}') "
|
"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")
|
node1.query("SYSTEM STOP MERGES mt3")
|
||||||
|
Loading…
Reference in New Issue
Block a user