mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 17:41:59 +00:00
fix style. And del getMaxBlock...()
This commit is contained in:
parent
8b8697bc5d
commit
33545a41b6
@ -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);
|
||||||
|
@ -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;
|
||||||
|
@ -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);
|
||||||
|
@ -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();
|
||||||
|
|
||||||
|
Loading…
Reference in New Issue
Block a user