提交 2340e55d 编写于 作者: A Alexey Zatelepin 提交者: alexey-milovidov

revert to old behaviour of getPartIfExists() [#CLICKHOUSE-3508]

上级 3124ffea
......@@ -141,9 +141,10 @@ void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & /*bo
MergeTreeData::DataPartPtr Service::findPart(const String & name)
{
/// It is important to include PreCommitted parts here
/// Because part could be actually committed into ZooKeeper, but response from ZooKeeper to the server could be delayed
auto part = data.getPartIfExists(name, {MergeTreeDataPart::State::PreCommitted, MergeTreeDataPart::State::Committed});
/// It is important to include PreCommitted and Outdated parts here because remote replicas cannot reliably
/// determine the local state of the part, so queries for the parts in these states are completely normal.
auto part = data.getPartIfExists(
name, {MergeTreeDataPartState::PreCommitted, MergeTreeDataPartState::Committed, MergeTreeDataPartState::Outdated});
if (part)
return part;
......
......@@ -1718,7 +1718,7 @@ MergeTreeData::DataPartPtr MergeTreeData::getPartIfExists(const String & part_na
for (auto state : valid_states)
{
if ((*it)->state == state)
return *it;
return *it;
}
return nullptr;
......
......@@ -352,8 +352,8 @@ public:
/// Returns an comitted part with the given name or a part containing it. If there is no such part, returns nullptr.
DataPartPtr getActiveContainingPart(const String & part_name);
/// Returns the part with the given name (and state) or nullptr if no such part.
DataPartPtr getPartIfExists(const String & part_name, const DataPartStates & valid_states = {DataPartState::Committed});
/// Returns the part with the given name and state or nullptr if no such part.
DataPartPtr getPartIfExists(const String & part_name, const DataPartStates & valid_states);
/// Total size of active parts in bytes.
size_t getTotalActiveSizeInBytes() const;
......
#include <Storages/MergeTree/ReplicatedMergeTreeQueue.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Storages/MergeTree/MergeTreeDataPart.h>
#include <Storages/MergeTree/MergeTreeDataMerger.h>
#include <Common/StringUtils.h>
......@@ -635,7 +636,7 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry(
return false;
}
auto part = data.getPartIfExists(name);
auto part = data.getPartIfExists(name, {MergeTreeDataPartState::Committed, MergeTreeDataPartState::Outdated});
if (part)
sum_parts_size_in_bytes += part->size_in_bytes;
}
......
......@@ -253,7 +253,8 @@ void ReplicatedMergeTreeRestartingThread::removeFailedQuorumParts()
for (auto part_name : failed_parts)
{
auto part = storage.data.getPartIfExists(part_name);
auto part = storage.data.getPartIfExists(
part_name, {MergeTreeDataPartState::PreCommitted, MergeTreeDataPartState::Committed, MergeTreeDataPartState::Outdated});
if (part)
{
LOG_DEBUG(log, "Found part " << part_name << " with failed quorum. Moving to detached. This shouldn't happen often.");
......
......@@ -5,6 +5,7 @@
#include <Storages/ColumnsDescription.h>
#include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/MergeTree/MergeTreeDataPart.h>
#include <Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h>
#include <Storages/MergeTree/ReplicatedMergeTreeQuorumEntry.h>
#include <Storages/MergeTree/MergeList.h>
......@@ -2125,7 +2126,7 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Strin
if (auto part = data.getPartIfExists(part_name, {MergeTreeDataPart::State::Outdated, MergeTreeDataPart::State::Deleting}))
{
LOG_DEBUG(log, "Part " << part->getNameWithState() << " should be deleted after previous attempt before fetch");
/// Force premature parts cleanup
/// Force immediate parts cleanup to delete the part that was left from the previous fetch attempt.
cleanup_thread->schedule();
return false;
}
......@@ -2322,7 +2323,7 @@ BlockInputStreams StorageReplicatedMergeTree::read(
String last_part;
zookeeper->tryGet(zookeeper_path + "/quorum/last_part", last_part);
if (!last_part.empty() && !data.getPartIfExists(last_part)) /// TODO Disable replica for distributed queries.
if (!last_part.empty() && !data.getActiveContainingPart(last_part)) /// TODO Disable replica for distributed queries.
throw Exception("Replica doesn't have part " + last_part + " which was successfully written to quorum of other replicas."
" Send query to another replica or disable 'select_sequential_consistency' setting.", ErrorCodes::REPLICA_IS_NOT_IN_QUORUM);
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册