mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Removed obolete code. [#CLICKHOUSE-3346]
This commit is contained in:
parent
1517422f4d
commit
30028f14e0
@ -8,6 +8,7 @@
|
||||
#include <Poco/Util/Application.h>
|
||||
#include <Poco/UUIDGenerator.h>
|
||||
#include <Poco/File.h>
|
||||
#include <Poco/Process.h>
|
||||
#include <chrono>
|
||||
|
||||
#include <Common/Exception.h>
|
||||
@ -17,46 +18,41 @@
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/Settings.h>
|
||||
#include <Interpreters/Cluster.h>
|
||||
#include <Interpreters/InterpreterCheckQuery.h>
|
||||
#include <Interpreters/ClusterProxy/executeQuery.h>
|
||||
#include <Interpreters/ClusterProxy/DescribeStreamFactory.h>
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
#include <Interpreters/InterpreterInsertQuery.h>
|
||||
#include <Interpreters/InterpreterFactory.h>
|
||||
#include <Interpreters/InterpreterDropQuery.h>
|
||||
#include <Interpreters/InterpreterCreateQuery.h>
|
||||
|
||||
#include <common/logger_useful.h>
|
||||
#include <common/ApplicationServerExt.h>
|
||||
#include <Parsers/ASTCheckQuery.h>
|
||||
#include <common/ThreadPool.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/ClickHouseRevision.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Storages/StorageDistributed.h>
|
||||
#include <Parsers/ASTCheckQuery.h>
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Databases/DatabaseMemory.h>
|
||||
#include <DataStreams/RemoteBlockInputStream.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <DataStreams/SquashingBlockInputStream.h>
|
||||
#include <Common/isLocalAddress.h>
|
||||
#include <Parsers/ParserQuery.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Parsers/ASTDropQuery.h>
|
||||
#include <Interpreters/InterpreterDropQuery.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Databases/DatabaseMemory.h>
|
||||
#include <DataStreams/RemoteBlockInputStream.h>
|
||||
#include <DataStreams/SquashingBlockInputStream.h>
|
||||
#include <Common/isLocalAddress.h>
|
||||
#include <DataStreams/copyData.h>
|
||||
#include <Interpreters/InterpreterCreateQuery.h>
|
||||
#include <common/ThreadPool.h>
|
||||
#include <DataStreams/NullBlockOutputStream.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <IO/WriteBufferNull.h>
|
||||
#include <Functions/registerFunctions.h>
|
||||
#include <TableFunctions/registerTableFunctions.h>
|
||||
#include <AggregateFunctions/registerAggregateFunctions.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Interpreters/executeQuery.h>
|
||||
#include <IO/WriteBufferNull.h>
|
||||
#include <Parsers/ParserQuery.h>
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
#include <Interpreters/InterpreterInsertQuery.h>
|
||||
#include <Interpreters/InterpreterFactory.h>
|
||||
#include <Poco/Process.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -100,6 +96,7 @@ enum class TaskState
|
||||
};
|
||||
|
||||
|
||||
/// Used to mark status of shard partition tasks
|
||||
struct TaskStateWithOwner
|
||||
{
|
||||
TaskStateWithOwner() = default;
|
||||
@ -145,11 +142,10 @@ struct TaskCluster;
|
||||
|
||||
using TasksPartition = std::map<String, TaskPartition>;
|
||||
using ShardInfo = Cluster::ShardInfo;
|
||||
using TaskShardPtr = TaskShard *;
|
||||
using TasksShard = std::vector<TaskShard>;
|
||||
using TasksShardPtrs = std::vector<TaskShard *>;
|
||||
using TaskShardPtr = std::shared_ptr<TaskShard>;
|
||||
using TasksShard = std::vector<TaskShardPtr>;
|
||||
using TasksTable = std::list<TaskTable>;
|
||||
using PartitionToShards = std::map<String, TasksShardPtrs>;
|
||||
using PartitionToShards = std::map<String, TasksShard>;
|
||||
|
||||
struct TaskPartition
|
||||
{
|
||||
@ -182,8 +178,6 @@ struct ShardPriority
|
||||
}
|
||||
};
|
||||
|
||||
ShardPriority getReplicasPriority(const Cluster::Addresses & replicas, const std::string & local_hostname, UInt8 random);
|
||||
|
||||
|
||||
struct TaskShard
|
||||
{
|
||||
@ -239,15 +233,9 @@ struct TaskTable
|
||||
ClusterPtr cluster_pull;
|
||||
ClusterPtr cluster_push;
|
||||
|
||||
struct Shards
|
||||
{
|
||||
TasksShard all_shards;
|
||||
TasksShardPtrs all_shards_prioritized;
|
||||
TasksShardPtrs local_shards_prioritized;
|
||||
};
|
||||
|
||||
/// Prioritized list of shards
|
||||
Shards shards;
|
||||
TasksShard all_shards;
|
||||
TasksShard local_shards;
|
||||
|
||||
PartitionToShards partition_to_shards;
|
||||
|
||||
@ -459,7 +447,7 @@ TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfigurati
|
||||
}
|
||||
|
||||
|
||||
ShardPriority getReplicasPriority(const Cluster::Addresses & replicas, const std::string & local_hostname, UInt8 random)
|
||||
static ShardPriority getReplicasPriority(const Cluster::Addresses & replicas, const std::string & local_hostname, UInt8 random)
|
||||
{
|
||||
ShardPriority res;
|
||||
|
||||
@ -496,34 +484,28 @@ void TaskTable::initShards(URNG && urng)
|
||||
// Compute the priority
|
||||
for (auto & shard_info : cluster_pull->getShardsInfo())
|
||||
{
|
||||
shards.all_shards.emplace_back(*this, shard_info);
|
||||
TaskShardPtr task_shard = std::make_shared<TaskShard>(*this, shard_info);
|
||||
const auto & replicas = cluster_pull->getShardsAddresses().at(task_shard->indexInCluster());
|
||||
task_shard->priority = getReplicasPriority(replicas, fqdn_name, get_rand(urng));
|
||||
|
||||
TaskShard & task_shard = shards.all_shards.back();
|
||||
const auto & replicas = cluster_pull->getShardsAddresses().at(task_shard.indexInCluster());
|
||||
task_shard.priority = getReplicasPriority(replicas, fqdn_name, get_rand(urng));
|
||||
all_shards.emplace_back(task_shard);
|
||||
}
|
||||
|
||||
// Sort by priority
|
||||
auto is_more_priority = [] (TaskShardPtr lhs, TaskShardPtr rhs)
|
||||
{
|
||||
return ShardPriority::isMorePriority(lhs->priority, rhs->priority);
|
||||
};
|
||||
std::sort(all_shards.begin(), all_shards.end(),
|
||||
[] (const TaskShardPtr & lhs, const TaskShardPtr & rhs)
|
||||
{
|
||||
return ShardPriority::isMorePriority(lhs->priority, rhs->priority);
|
||||
});
|
||||
|
||||
for (auto & task_shard : shards.all_shards)
|
||||
shards.all_shards_prioritized.push_back(&task_shard);
|
||||
// Cut local shards
|
||||
auto it_first_remote = std::lower_bound(all_shards.begin(), all_shards.end(), 1,
|
||||
[] (const TaskShardPtr & lhs, UInt8 is_remote)
|
||||
{
|
||||
return lhs->priority.is_remote < is_remote;
|
||||
});
|
||||
|
||||
std::sort(shards.all_shards_prioritized.begin(), shards.all_shards_prioritized.end(), is_more_priority);
|
||||
|
||||
// Select local shards
|
||||
auto has_less_is_remote = [] (TaskShardPtr lhs, UInt8 is_remote)
|
||||
{
|
||||
return lhs->priority.is_remote < is_remote;
|
||||
};
|
||||
|
||||
auto it_first_remote = std::lower_bound(shards.all_shards_prioritized.begin(), shards.all_shards_prioritized.end(), 1,
|
||||
has_less_is_remote);
|
||||
|
||||
shards.local_shards_prioritized.assign(shards.all_shards_prioritized.begin(), it_first_remote);
|
||||
local_shards.assign(all_shards.begin(), it_first_remote);
|
||||
}
|
||||
|
||||
TaskCluster::TaskCluster(const String & task_zookeeper_path_, const Poco::Util::AbstractConfiguration & config, const String & base_key,
|
||||
@ -623,9 +605,7 @@ public:
|
||||
/// Compute set of partitions, set of partitions aren't changed
|
||||
for (auto & task_table : task_cluster->table_tasks)
|
||||
{
|
||||
auto & shards = task_table.shards.all_shards_prioritized;
|
||||
|
||||
for (TaskShardPtr task_shard : shards)
|
||||
for (const TaskShardPtr & task_shard : task_table.all_shards)
|
||||
{
|
||||
if (task_shard->info.pool == nullptr)
|
||||
{
|
||||
@ -639,8 +619,8 @@ public:
|
||||
<< ", shard " << task_shard->info.shard_num << ")");
|
||||
|
||||
LOG_DEBUG(log, "There are "
|
||||
<< task_table.shards.all_shards_prioritized.size() << " shards, and "
|
||||
<< task_table.shards.local_shards_prioritized.size() << " remote ones");
|
||||
<< task_table.all_shards.size() << " shards, and "
|
||||
<< task_table.local_shards.size() << " remote ones");
|
||||
|
||||
auto connection_entry = task_shard->info.pool->get(&task_cluster->settings_pull);
|
||||
LOG_DEBUG(log, "Will get meta information for shard " << task_shard->numberInCluster()
|
||||
@ -665,20 +645,15 @@ public:
|
||||
{
|
||||
for (TaskTable & task_table : task_cluster->table_tasks)
|
||||
{
|
||||
auto & shards = task_table.shards.all_shards_prioritized;
|
||||
|
||||
if (shards.empty())
|
||||
if (task_table.all_shards.empty())
|
||||
continue;
|
||||
|
||||
/// Try process all partitions of the first shard
|
||||
for (auto shard_ptr : shards)
|
||||
/// An optimization: first of all, try to process all partitions of the local shards
|
||||
for (const TaskShardPtr & shard : task_table.local_shards)
|
||||
{
|
||||
if (shard_ptr->priority.is_remote)
|
||||
continue;
|
||||
|
||||
for (auto & task_partition : shards[0]->partitions)
|
||||
for (auto & task_partition : shard->partitions)
|
||||
{
|
||||
LOG_DEBUG(log, "Processing partition " << task_partition.first << " for local shard " << shard_ptr->numberInCluster());
|
||||
LOG_DEBUG(log, "Processing partition " << task_partition.first << " for local shard " << shard->numberInCluster());
|
||||
processPartitionTask(task_partition.second);
|
||||
}
|
||||
}
|
||||
@ -687,7 +662,7 @@ public:
|
||||
for (const auto & partition_with_shards : task_table.partition_to_shards)
|
||||
{
|
||||
const String & partition_name = partition_with_shards.first;
|
||||
const TasksShardPtrs & shards_with_partition = partition_with_shards.second;
|
||||
const TasksShard & shards_with_partition = partition_with_shards.second;
|
||||
bool is_done;
|
||||
|
||||
do
|
||||
@ -697,7 +672,7 @@ public:
|
||||
|
||||
size_t num_successful_shards = 0;
|
||||
|
||||
for (auto & shard : shards_with_partition)
|
||||
for (const TaskShardPtr & shard : shards_with_partition)
|
||||
{
|
||||
auto it_shard_partition = shard->partitions.find(partition_name);
|
||||
if (it_shard_partition == shard->partitions.end())
|
||||
@ -731,7 +706,7 @@ public:
|
||||
* We have to ensure that all shards have the finished state and there are no dirty flag.
|
||||
* Moreover, we have to check status twice and check zxid, because state could be changed during the checking.
|
||||
*/
|
||||
bool checkPartitionIsDone(const TaskTable & task_table, const String & partition_name, const TasksShardPtrs & shards_with_partition)
|
||||
bool checkPartitionIsDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition)
|
||||
{
|
||||
LOG_DEBUG(log, "Check that all shards processed partition " << partition_name << " successfully");
|
||||
|
||||
@ -1017,8 +992,8 @@ protected:
|
||||
|
||||
// Task is abandoned, initialize DROP PARTITION
|
||||
LOG_DEBUG(log, "Task " << current_task_status_path << " has not been successfully finished by " << status.owner);
|
||||
create_is_dirty_node();
|
||||
|
||||
create_is_dirty_node();
|
||||
return false;
|
||||
}
|
||||
}
|
||||
@ -1229,7 +1204,7 @@ protected:
|
||||
}
|
||||
}
|
||||
|
||||
LOG_DEBUG(log, "Data partition copied");
|
||||
LOG_INFO(log, "Partition " << task_partition.name << " copied");
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -1389,45 +1364,6 @@ protected:
|
||||
return successful_shards;
|
||||
}
|
||||
|
||||
String getTableStructureAndCheckConsistency(TaskTable & table_task)
|
||||
{
|
||||
InterpreterCheckQuery::RemoteTablesInfo remotes_info;
|
||||
remotes_info.cluster = table_task.cluster_pull;
|
||||
remotes_info.remote_database = table_task.table_pull.first;
|
||||
remotes_info.remote_table = table_task.table_pull.second;
|
||||
|
||||
Context local_context = context;
|
||||
InterpreterCheckQuery check_table(std::move(remotes_info), local_context);
|
||||
|
||||
BlockIO io = check_table.execute();
|
||||
if (io.out != nullptr)
|
||||
throw Exception("Expected empty io.out", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
String columns_structure;
|
||||
size_t rows = 0;
|
||||
|
||||
Block block;
|
||||
while ((block = io.in->read()))
|
||||
{
|
||||
auto & structure_col = typeid_cast<ColumnString &>(*block.getByName("structure").column);
|
||||
auto & structure_class_col = typeid_cast<ColumnUInt32 &>(*block.getByName("structure_class").column);
|
||||
|
||||
for (size_t i = 0; i < block.rows(); ++i)
|
||||
{
|
||||
if (structure_class_col.getElement(i) != 0)
|
||||
throw Exception("Structures of table " + getDatabaseDotTable(table_task.table_pull) + " are different on cluster " +
|
||||
table_task.cluster_pull_name, ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
if (rows == 0)
|
||||
columns_structure = structure_col.getDataAt(i).toString();
|
||||
|
||||
++rows;
|
||||
}
|
||||
}
|
||||
|
||||
return columns_structure;
|
||||
}
|
||||
|
||||
void initZooKeeper()
|
||||
{
|
||||
current_zookeeper = std::make_shared<zkutil::ZooKeeper>(*zookeeper_config, "zookeeper");
|
||||
@ -1514,7 +1450,7 @@ public:
|
||||
std::cerr << DB::getCurrentExceptionMessage(true) << "\n";
|
||||
auto code = getCurrentExceptionCode();
|
||||
|
||||
return (code ? code : -1);
|
||||
return (code) ? code : -1;
|
||||
}
|
||||
|
||||
return 0;
|
||||
@ -1595,6 +1531,6 @@ int mainEntryClickHouseClusterCopier(int argc, char ** argv)
|
||||
std::cerr << DB::getCurrentExceptionMessage(true) << "\n";
|
||||
auto code = DB::getCurrentExceptionCode();
|
||||
|
||||
return (code ? code : -1);
|
||||
return (code) ? code : -1;
|
||||
}
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user