Removed obolete code. [#CLICKHOUSE-3346]

This commit is contained in:
Vitaliy Lyudvichenko 2017-11-14 20:45:15 +03:00
parent 1517422f4d
commit 30028f14e0

View File

@ -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;
}
}