#pragma once #include "Aliases.h" #include "Internals.h" #include "TaskCluster.h" #include "TaskTableAndShard.h" #include "ShardPartition.h" #include "ShardPartitionPiece.h" #include "ZooKeeperStaff.h" namespace DB { using ConfigurationPtr = Poco::AutoPtr; ConfigurationPtr getConfigurationFromXMLString(const std::string & xml_data) { std::stringstream ss(xml_data); Poco::XML::InputSource input_source{ss}; return {new Poco::Util::XMLConfiguration{&input_source}}; } class ClusterCopier { public: ClusterCopier(String task_path_, String host_id_, String proxy_database_name_, Context & context_) : task_zookeeper_path(std::move(task_path_)), host_id(std::move(host_id_)), working_database_name(std::move(proxy_database_name_)), context(context_), log(&Poco::Logger::get("ClusterCopier")) { } void init(); template decltype(auto) retry(T && func, UInt64 max_tries = 100); void discoverShardPartitions(const ConnectionTimeouts & timeouts, const TaskShardPtr & task_shard); /// Compute set of partitions, assume set of partitions aren't changed during the processing void discoverTablePartitions(const ConnectionTimeouts & timeouts, TaskTable & task_table, UInt64 num_threads = 0); void uploadTaskDescription(const std::string & task_path, const std::string & task_file, const bool force); void reloadTaskDescription(); void updateConfigIfNeeded(); void process(const ConnectionTimeouts & timeouts); /// Disables DROP PARTITION commands that used to clear data after errors void setSafeMode(bool is_safe_mode_ = true); void setCopyFaultProbability(double copy_fault_probability_); protected: String getWorkersPath() const; String getWorkersPathVersion() const; String getCurrentWorkerNodePath() const; zkutil::EphemeralNodeHolder::Ptr createTaskWorkerNodeAndWaitIfNeed( const zkutil::ZooKeeperPtr & zookeeper, const String & description, bool unprioritized); /** Checks that the whole partition of a table was copied. We should do it carefully due to dirty lock. * State of some task could change during the processing. * We have to ensure that all shards have the finished state and there is no dirty flag. * Moreover, we have to check status twice and check zxid, because state can change during the checking. */ bool checkPartitionIsDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition); bool checkAllPieceInPartitionDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition); /* The same as function above * Assume that we don't know on which shards do we have partition certain piece. * We'll check them all (I mean shards that contain the whole partition) * And shards that don't have certain piece MUST mark that piece is_done true. * */ bool checkPartitionPieceIsDone(const TaskTable & task_table, const String & partition_name, size_t piece_number, const TasksShard & shards_with_partition); /// Removes MATERIALIZED and ALIAS columns from create table query ASTPtr removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast); /// Replaces ENGINE and table name in a create query std::shared_ptr rewriteCreateQueryStorage(const ASTPtr & create_query_ast, const DatabaseAndTableName & new_table, const ASTPtr & new_storage_ast); bool tryDropPartition(ShardPartition & task_partition, const zkutil::ZooKeeperPtr & zookeeper, const CleanStateClock & clean_state_clock); static constexpr UInt64 max_table_tries = 1000; static constexpr UInt64 max_shard_partition_tries = 600; bool tryProcessTable(const ConnectionTimeouts & timeouts, TaskTable & task_table); /// Execution status of a task enum class PartitionTaskStatus { Active, Finished, Error, }; enum class PartititonPieceTaskStatus { Active, Finished, Error, }; /// Job for copying partition from particular shard. PartitionTaskStatus tryProcessPartitionTask(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, bool is_unprioritized_task); PartitionTaskStatus iterateThroughAllPiecesInPartition(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, bool is_unprioritized_task); PartitionTaskStatus processPartitionPieceTaskImpl(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, const size_t current_piece_number, bool is_unprioritized_task); void dropAndCreateLocalTable(const ASTPtr & create_ast); void dropLocalTableIfExists(const DatabaseAndTableName & table_name) const; String getRemoteCreateTable(const DatabaseAndTableName & table, Connection & connection, const Settings * settings = nullptr); ASTPtr getCreateTableForPullShard(const ConnectionTimeouts & timeouts, TaskShard & task_shard); /// If it is implicitly asked to create split Distributed table for certain piece on current shard, we will do it. void createShardInternalTables(const ConnectionTimeouts & timeouts, TaskShard & task_shard, bool create_split = true, const size_t piece_number = 0); std::set getShardPartitions(const ConnectionTimeouts & timeouts, TaskShard & task_shard); bool checkShardHasPartition(const ConnectionTimeouts & timeouts, TaskShard & task_shard, const String & partition_quoted_name); /// TODO: Implement checkPresentPartitionPiecesOnCurrentShard(); /// Just copypaste the function above bool checkPresentPartitionPiecesOnCurrentShard(const ConnectionTimeouts & timeouts, TaskShard & task_shard, const String & partition_quoted_name, size_t current_piece_number); /** Executes simple query (without output streams, for example DDL queries) on each shard of the cluster * Returns number of shards for which at least one replica executed query successfully */ UInt64 executeQueryOnCluster( const ClusterPtr & cluster, const String & query, const ASTPtr & query_ast_ = nullptr, const Settings * settings = nullptr, PoolMode pool_mode = PoolMode::GET_ALL, UInt64 max_successful_executions_per_shard = 0) const; private: String task_zookeeper_path; String task_description_path; String host_id; String working_database_name; /// Auto update config stuff UInt64 task_descprtion_current_version = 1; std::atomic task_descprtion_version{1}; Coordination::WatchCallback task_description_watch_callback; /// ZooKeeper session used to set the callback zkutil::ZooKeeperPtr task_description_watch_zookeeper; ConfigurationPtr task_cluster_initial_config; ConfigurationPtr task_cluster_current_config; Coordination::Stat task_description_current_stat{}; std::unique_ptr task_cluster; bool is_safe_mode = false; double copy_fault_probability = 0.0; Context & context; Poco::Logger * log; std::chrono::milliseconds default_sleep_time{1000}; }; }