fix style. And del getMaxBlock...()

This commit is contained in:
VadimPE 2018-09-21 18:07:43 +03:00
parent 8b8697bc5d
commit 33545a41b6
4 changed files with 13 additions and 29 deletions

View File

@ -1785,21 +1785,6 @@ size_t MergeTreeData::getMaxPartsCountForPartition() const
} }
using PartitionIdToMaxBlock = std::unordered_map<String, Int64>;
PartitionIdToMaxBlock MergeTreeData::getMaxBlocksForPartition() const
{
std::lock_guard<std::mutex> lock(data_parts_mutex);
PartitionIdToMaxBlock max_blocks;
for (const auto & part : getDataPartsStateRange(DataPartState::Committed))
max_blocks[part->info.partition_id] = part->info.max_block;
return max_blocks;
}
std::optional<Int64> MergeTreeData::getMinPartDataVersion() const std::optional<Int64> MergeTreeData::getMinPartDataVersion() const
{ {
std::lock_guard lock(data_parts_mutex); std::lock_guard lock(data_parts_mutex);

View File

@ -388,10 +388,6 @@ public:
size_t getMaxPartsCountForPartition() const; size_t getMaxPartsCountForPartition() const;
using PartitionIdToMaxBlock = std::unordered_map<String, Int64>;
PartitionIdToMaxBlock getMaxBlocksForPartition() const;
/// Get min value of part->info.getDataVersion() for all active parts. /// Get min value of part->info.getDataVersion() for all active parts.
/// Makes sense only for ordinary MergeTree engines because for them block numbering doesn't depend on partition. /// Makes sense only for ordinary MergeTree engines because for them block numbering doesn't depend on partition.
std::optional<Int64> getMinPartDataVersion() const; std::optional<Int64> getMinPartDataVersion() const;

View File

@ -15,9 +15,9 @@ namespace DB
struct ReplicatedMergeTreeQuorumAddedParts struct ReplicatedMergeTreeQuorumAddedParts
{ {
using PartitionIdToMaxBlock = std::unordered_map<String, Int64>; using PartitionIdToMaxBlock = std::unordered_map<String, Int64>;
using PartitonIdToPartName = std::unordered_map<String, String>; using PartitionIdToPartName = std::unordered_map<String, String>;
PartitonIdToPartName added_parts; PartitionIdToPartName added_parts;
MergeTreeDataFormatVersion format_version; MergeTreeDataFormatVersion format_version;
@ -28,7 +28,7 @@ struct ReplicatedMergeTreeQuorumAddedParts
} }
/// Write new parts in buffer with added parts. /// Write new parts in buffer with added parts.
void write(WriteBufferFromOwnString & out) void write(WriteBuffer & out)
{ {
out << "version: " << 2 << '\n'; out << "version: " << 2 << '\n';
out << "parts count: " << added_parts.size() << '\n'; out << "parts count: " << added_parts.size() << '\n';
@ -50,7 +50,7 @@ struct ReplicatedMergeTreeQuorumAddedParts
return max_added_blocks; return max_added_blocks;
} }
void read(ReadBufferFromString & in) void read(ReadBuffer & in)
{ {
if (checkString("version: ", in)) if (checkString("version: ", in))
{ {
@ -67,9 +67,9 @@ struct ReplicatedMergeTreeQuorumAddedParts
} }
/// Read added bloks when node in ZooKeeper supports only one partition. /// Read added bloks when node in ZooKeeper supports only one partition.
PartitonIdToPartName read_v1(ReadBufferFromString & in) PartitionIdToPartName read_v1(ReadBuffer & in)
{ {
PartitonIdToPartName parts_in_quorum; PartitionIdToPartName parts_in_quorum;
std::string partition_name; std::string partition_name;
@ -82,11 +82,11 @@ struct ReplicatedMergeTreeQuorumAddedParts
} }
/// Read blocks when node in ZooKeeper suppors multiple partitions. /// Read blocks when node in ZooKeeper suppors multiple partitions.
PartitonIdToPartName read_v2(ReadBufferFromString & in) PartitionIdToPartName read_v2(ReadBuffer & in)
{ {
assertString("parts count: ", in); assertString("parts count: ", in);
PartitonIdToPartName parts_in_quorum; PartitionIdToPartName parts_in_quorum;
uint64_t parts_count; uint64_t parts_count;
readText(parts_count, in); readText(parts_count, in);

View File

@ -2954,10 +2954,13 @@ BlockInputStreams StorageReplicatedMergeTree::read(
* 2. Do not read parts that have not yet been written to the quorum of the replicas. * 2. Do not read parts that have not yet been written to the quorum of the replicas.
* For this you have to synchronously go to ZooKeeper. * For this you have to synchronously go to ZooKeeper.
*/ */
DB::ReplicatedMergeTreeQuorumAddedParts::PartitionIdToMaxBlock max_added_blocks; ReplicatedMergeTreeQuorumAddedParts::PartitionIdToMaxBlock max_added_blocks;
if (settings.select_sequential_consistency) if (settings.select_sequential_consistency)
{ {
max_added_blocks = data.getMaxBlocksForPartition(); for (const auto & data_part : data.getDataParts())
{
max_added_blocks[data_part->info.partition_id] = data_part->info.max_block;
}
auto zookeeper = getZooKeeper(); auto zookeeper = getZooKeeper();