ClickHouse/src/Storages/MergeTree/ReplicatedMergeTreeSink.h

146 lines
4.7 KiB
C++
Raw Normal View History

2014-04-02 07:59:43 +00:00
#pragma once
2021-07-23 19:33:59 +00:00
#include <Processors/Sinks/SinkToStorage.h>
#include <Storages/MergeTree/MergeTreeData.h>
2021-10-02 07:13:14 +00:00
#include <base/types.h>
#include <Storages/MergeTree/ZooKeeperRetries.h>
2022-12-22 13:31:42 +00:00
#include <Common/ZooKeeper/ZooKeeperWithFaultInjection.h>
#include <Storages/MergeTree/AsyncBlockIDsCache.h>
2014-04-02 07:59:43 +00:00
namespace Poco { class Logger; }
namespace zkutil
{
class ZooKeeper;
using ZooKeeperPtr = std::shared_ptr<ZooKeeper>;
}
2014-04-02 07:59:43 +00:00
namespace DB
{
2016-01-17 05:22:22 +00:00
class StorageReplicatedMergeTree;
struct StorageSnapshot;
using StorageSnapshotPtr = std::shared_ptr<StorageSnapshot>;
2022-11-06 22:56:26 +00:00
/// ReplicatedMergeTreeSink will sink data to replicated merge tree with deduplication.
/// The template argument "async_insert" indicates whether this sink serves for async inserts.
/// Async inserts will have different deduplication policy. We use a vector of "block ids" to
/// identify different async inserts inside the same part. It will remove the duplicate inserts
/// when it encounters lock and retries.
template<bool async_insert>
2022-12-07 22:40:52 +00:00
class ReplicatedMergeTreeSinkImpl : public SinkToStorage
2014-04-02 07:59:43 +00:00
{
public:
2022-12-07 22:40:52 +00:00
ReplicatedMergeTreeSinkImpl(
StorageReplicatedMergeTree & storage_,
const StorageMetadataPtr & metadata_snapshot_,
size_t quorum_,
size_t quorum_timeout_ms_,
size_t max_parts_per_block_,
2020-09-30 23:16:27 +00:00
bool quorum_parallel_,
2020-11-13 07:54:05 +00:00
bool deduplicate_,
bool majority_quorum_,
ContextPtr context_,
// special flag to determine the ALTER TABLE ATTACH PART without the query context,
// needed to set the special LogEntryType::ATTACH_PART
bool is_attach_ = false);
2022-12-07 22:40:52 +00:00
~ReplicatedMergeTreeSinkImpl() override;
2021-07-26 14:47:29 +00:00
void onStart() override;
2021-07-23 19:33:59 +00:00
void consume(Chunk chunk) override;
void onFinish() override;
2021-07-23 19:33:59 +00:00
String getName() const override { return "ReplicatedMergeTreeSink"; }
2014-04-02 07:59:43 +00:00
/// For ATTACHing existing data on filesystem.
void writeExistingPart(MergeTreeData::MutableDataPartPtr & part);
/// For proper deduplication in MaterializedViews
bool lastBlockIsDuplicate() const override
{
/// If MV is responsible for deduplication, block is not considered duplicating.
if (context->getSettingsRef().deduplicate_blocks_in_dependent_materialized_views)
return false;
return last_block_is_duplicate;
}
2022-11-06 22:56:26 +00:00
struct DelayedChunk;
2014-04-02 07:59:43 +00:00
private:
2022-11-06 22:56:26 +00:00
using BlockIDsType = std::conditional_t<async_insert, std::vector<String>, String>;
ZooKeeperRetriesInfo zookeeper_retries_info;
struct QuorumInfo
{
String status_path;
int is_active_node_version = -1;
int host_node_version = -1;
};
QuorumInfo quorum_info;
/// Checks active replicas.
/// Returns total number of replicas.
size_t checkQuorumPrecondition(const ZooKeeperWithFaultInjectionPtr & zookeeper);
/// Rename temporary part and commit to ZooKeeper.
2023-06-02 15:42:04 +00:00
/// Returns a list of conflicting async blocks and true if the whole parts was deduplicated
std::pair<std::vector<String>, bool> commitPart(
const ZooKeeperWithFaultInjectionPtr & zookeeper,
2022-06-23 16:21:46 +00:00
MergeTreeData::MutableDataPartPtr & part,
2022-11-06 22:56:26 +00:00
const BlockIDsType & block_id,
size_t replicas_num,
bool writing_existing_part);
/// Wait for quorum to be satisfied on path (quorum_path) form part (part_name)
/// Also checks that replica still alive.
void waitForQuorum(
const ZooKeeperWithFaultInjectionPtr & zookeeper,
const std::string & part_name,
const std::string & quorum_path,
int is_active_node_version,
size_t replicas_num) const;
StorageReplicatedMergeTree & storage;
StorageMetadataPtr metadata_snapshot;
/// Empty means use majority quorum.
std::optional<size_t> required_quorum_size;
size_t getQuorumSize(size_t replicas_num) const;
bool isQuorumEnabled() const;
String quorumLogMessage(size_t replicas_num) const; /// Used in logs for debug purposes
size_t quorum_timeout_ms;
size_t max_parts_per_block;
UInt64 cache_version = 0;
bool is_attach = false;
2020-09-30 23:16:27 +00:00
bool quorum_parallel = false;
const bool deduplicate = true;
bool last_block_is_duplicate = false;
2023-06-11 15:51:54 +00:00
UInt64 num_blocks_processed = 0;
using Logger = Poco::Logger;
2020-05-30 21:57:37 +00:00
Poco::Logger * log;
2020-11-13 07:54:05 +00:00
ContextPtr context;
StorageSnapshotPtr storage_snapshot;
UInt64 chunk_dedup_seqnum = 0; /// input chunk ordinal number in case of dedup token
/// We can delay processing for previous chunk and start writing a new one.
std::unique_ptr<DelayedChunk> delayed_chunk;
void finishDelayedChunk(const ZooKeeperWithFaultInjectionPtr & zookeeper);
2014-04-02 07:59:43 +00:00
};
2022-12-07 22:40:52 +00:00
using ReplicatedMergeTreeSinkWithAsyncDeduplicate = ReplicatedMergeTreeSinkImpl<true>;
using ReplicatedMergeTreeSink = ReplicatedMergeTreeSinkImpl<false>;
2014-04-02 07:59:43 +00:00
}