2014-04-02 07:59:43 +00:00
|
|
|
#pragma once
|
|
|
|
|
2021-07-23 19:33:59 +00:00
|
|
|
#include <Processors/Sinks/SinkToStorage.h>
|
2017-06-25 00:01:10 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreeData.h>
|
2020-09-15 09:55:57 +00:00
|
|
|
#include <common/types.h>
|
2014-04-02 07:59:43 +00:00
|
|
|
|
2017-06-25 00:01:10 +00:00
|
|
|
|
2017-02-02 22:08:19 +00:00
|
|
|
namespace Poco { class Logger; }
|
|
|
|
|
2017-06-25 00:01:10 +00:00
|
|
|
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;
|
2016-01-11 21:46:36 +00:00
|
|
|
|
|
|
|
|
2021-07-23 19:33:59 +00:00
|
|
|
class ReplicatedMergeTreeSink : public SinkToStorage
|
2014-04-02 07:59:43 +00:00
|
|
|
{
|
|
|
|
public:
|
2021-07-23 19:33:59 +00:00
|
|
|
ReplicatedMergeTreeSink(
|
2020-06-16 15:51:29 +00:00
|
|
|
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_,
|
2021-02-10 14:12:49 +00:00
|
|
|
ContextPtr context_,
|
2021-02-15 15:06:48 +00:00
|
|
|
// 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);
|
2015-09-11 02:13:59 +00:00
|
|
|
|
2021-07-26 14:47:29 +00:00
|
|
|
void onStart() override;
|
2021-07-23 19:33:59 +00:00
|
|
|
void consume(Chunk chunk) override;
|
|
|
|
|
|
|
|
String getName() const override { return "ReplicatedMergeTreeSink"; }
|
2014-04-02 07:59:43 +00:00
|
|
|
|
2017-06-25 00:51:51 +00:00
|
|
|
/// For ATTACHing existing data on filesystem.
|
|
|
|
void writeExistingPart(MergeTreeData::MutableDataPartPtr & part);
|
|
|
|
|
2017-10-24 19:32:23 +00:00
|
|
|
/// For proper deduplication in MaterializedViews
|
2021-08-31 13:50:07 +00:00
|
|
|
bool lastBlockIsDuplicate() const override
|
2017-10-24 19:32:23 +00:00
|
|
|
{
|
2021-08-31 13:50:07 +00:00
|
|
|
/// If MV is responsible for deduplication, block is not considered duplicating.
|
|
|
|
if (context->getSettingsRef().deduplicate_blocks_in_dependent_materialized_views)
|
|
|
|
return false;
|
|
|
|
|
2017-10-24 19:32:23 +00:00
|
|
|
return last_block_is_duplicate;
|
|
|
|
}
|
|
|
|
|
2014-04-02 07:59:43 +00:00
|
|
|
private:
|
2017-06-25 00:01:10 +00:00
|
|
|
struct QuorumInfo
|
|
|
|
{
|
|
|
|
String status_path;
|
|
|
|
String is_active_node_value;
|
|
|
|
int is_active_node_version = -1;
|
|
|
|
int host_node_version = -1;
|
|
|
|
};
|
|
|
|
|
|
|
|
QuorumInfo quorum_info;
|
|
|
|
void checkQuorumPrecondition(zkutil::ZooKeeperPtr & zookeeper);
|
|
|
|
|
|
|
|
/// Rename temporary part and commit to ZooKeeper.
|
2017-06-25 00:51:51 +00:00
|
|
|
void commitPart(zkutil::ZooKeeperPtr & zookeeper, MergeTreeData::MutableDataPartPtr & part, const String & block_id);
|
2017-06-25 00:01:10 +00:00
|
|
|
|
2021-04-08 10:35:38 +00:00
|
|
|
/// Wait for quorum to be satisfied on path (quorum_path) form part (part_name)
|
|
|
|
/// Also checks that replica still alive.
|
|
|
|
void waitForQuorum(
|
|
|
|
zkutil::ZooKeeperPtr & zookeeper, const std::string & part_name,
|
2021-04-08 10:38:40 +00:00
|
|
|
const std::string & quorum_path, const std::string & is_active_node_value) const;
|
2021-04-08 10:35:38 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
StorageReplicatedMergeTree & storage;
|
2020-06-16 15:51:29 +00:00
|
|
|
StorageMetadataPtr metadata_snapshot;
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t quorum;
|
|
|
|
size_t quorum_timeout_ms;
|
2019-03-29 09:33:39 +00:00
|
|
|
size_t max_parts_per_block;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-02-15 15:06:48 +00:00
|
|
|
bool is_attach = false;
|
2020-09-30 23:16:27 +00:00
|
|
|
bool quorum_parallel = false;
|
2017-10-24 19:32:23 +00:00
|
|
|
bool deduplicate = true;
|
|
|
|
bool last_block_is_duplicate = false;
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
using Logger = Poco::Logger;
|
2020-05-30 21:57:37 +00:00
|
|
|
Poco::Logger * log;
|
2020-11-13 07:54:05 +00:00
|
|
|
|
2021-02-10 14:12:49 +00:00
|
|
|
ContextPtr context;
|
2014-04-02 07:59:43 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
}
|