simplify path concat

This commit is contained in:
bharatnc 2020-12-03 09:53:55 -08:00
parent d05d7e8b06
commit a8fea625ae

View File

@ -223,7 +223,7 @@ public:
const std::string & lock_message_ = "")
:
zookeeper(zookeeper_),
lock_path(lock_prefix_ + "/" + lock_name_),
lock_path(fs::path(lock_prefix_) / lock_name_),
lock_message(lock_message_),
log(&Poco::Logger::get("zkutil::Lock"))
{
@ -391,7 +391,7 @@ void DDLWorker::recoverZooKeeper()
DDLTaskPtr DDLWorker::initAndCheckTask(const String & entry_name, String & out_reason, const ZooKeeperPtr & zookeeper)
{
String node_data;
String entry_path = queue_dir + "/" + entry_name;
String entry_path = fs::path(queue_dir) / entry_name;
if (!zookeeper->tryGet(entry_path, node_data))
{
@ -420,7 +420,7 @@ DDLTaskPtr DDLWorker::initAndCheckTask(const String & entry_name, String & out_r
try
{
createStatusDirs(entry_path, zookeeper);
zookeeper->tryCreate(entry_path + "/finished/" + host_fqdn_id, status, zkutil::CreateMode::Persistent);
zookeeper->tryCreate(fs::path(entry_path) / "finished" / host_fqdn_id, status, zkutil::CreateMode::Persistent);
}
catch (...)
{
@ -501,7 +501,7 @@ void DDLWorker::scheduleTasks()
continue;
}
bool already_processed = zookeeper->exists(task->entry_path + "/finished/" + task->host_id_str);
bool already_processed = zookeeper->exists(fs::path(task->entry_path) / "finished" / task->host_id_str);
if (!server_startup && !task->was_executed && already_processed)
{
throw Exception(ErrorCodes::LOGICAL_ERROR,
@ -859,12 +859,10 @@ bool DDLWorker::tryExecuteQueryOnLeaderReplica(
};
String shard_node_name = get_shard_name(task.cluster->getShardsAddresses().at(task.host_shard_num));
String shard_path = (fs::path(node_path) / fs::path("shards") / fs::path(shard_node_name)).string();
String is_executed_path = (fs::path(shard_path) / fs::path("executed"));
String tries_to_execute_path = fs::path(shard_path) / fs::path("tries_to_execute");
zookeeper->createAncestors((fs::path(shard_path) / "").string()); /* appends "/" at the end of shard_path */
String shard_path = fs::path(node_path) / "shards" / shard_node_name;
String is_executed_path = fs::path(shard_path) / "executed";
String tries_to_execute_path = fs::path(shard_path) / "tries_to_execute";
zookeeper->createAncestors(fs::path(shard_path) / ""); /* appends "/" at the end of shard_path */
/// Node exists, or we will create or we will get an exception
zookeeper->tryCreate(tries_to_execute_path, "0", zkutil::CreateMode::Persistent);
@ -993,8 +991,8 @@ void DDLWorker::cleanupQueue(Int64 current_time_seconds, const ZooKeeperPtr & zo
return;
String node_name = *it;
String node_path = (fs::path(queue_dir) / fs::path(node_name)).string();
String lock_path = (fs::path(node_path) / fs::path("lock")).string();
String node_path = fs::path(queue_dir) / node_name;
String lock_path = fs::path(node_path) / "lock";
Coordination::Stat stat;
String dummy;
@ -1017,7 +1015,7 @@ void DDLWorker::cleanupQueue(Int64 current_time_seconds, const ZooKeeperPtr & zo
continue;
/// Skip if there are active nodes (it is weak guard)
if (zookeeper->exists(node_path + "/active", &stat) && stat.numChildren > 0)
if (zookeeper->exists(fs::path(node_path) / "active", &stat) && stat.numChildren > 0)
{
LOG_INFO(log, "Task {} should be deleted, but there are active workers. Skipping it.", node_name);
continue;
@ -1043,7 +1041,7 @@ void DDLWorker::cleanupQueue(Int64 current_time_seconds, const ZooKeeperPtr & zo
for (const String & child : children)
{
if (child != "lock")
zookeeper->tryRemoveRecursive((fs::path(node_path) / fs::path(child)).string());
zookeeper->tryRemoveRecursive(fs::path(node_path) / child);
}
/// Remove the lock node and its parent atomically
@ -1067,12 +1065,12 @@ void DDLWorker::createStatusDirs(const std::string & node_path, const ZooKeeperP
Coordination::Requests ops;
{
Coordination::CreateRequest request;
request.path = (fs::path(node_path) / fs::path("active")).string();
request.path = fs::path(node_path) / "active";
ops.emplace_back(std::make_shared<Coordination::CreateRequest>(std::move(request)));
}
{
Coordination::CreateRequest request;
request.path = (fs::path(node_path) / fs::path("finished")).string();
request.path = fs::path(node_path) / "finished";
ops.emplace_back(std::make_shared<Coordination::CreateRequest>(std::move(request)));
}
Coordination::Responses responses;
@ -1090,7 +1088,7 @@ String DDLWorker::enqueueQuery(DDLLogEntry & entry)
auto zookeeper = getAndSetZooKeeper();
String query_path_prefix = (fs::path(queue_dir) / fs::path("query-")).string();
String query_path_prefix = fs::path(queue_dir) / "query-";
zookeeper->createAncestors(query_path_prefix);
String node_path = zookeeper->create(query_path_prefix, entry.toString(), zkutil::CreateMode::PersistentSequential);
@ -1120,7 +1118,7 @@ void DDLWorker::runMainThread()
try
{
auto zookeeper = getAndSetZooKeeper();
zookeeper->createAncestors((fs::path(queue_dir) / "").string());
zookeeper->createAncestors(fs::path(queue_dir) / "");
initialized = true;
}
catch (const Coordination::Exception & e)
@ -1293,12 +1291,12 @@ public:
node_path);
}
Strings new_hosts = getNewAndUpdate(getChildrenAllowNoNode(zookeeper, (fs::path(node_path) / fs::path( "finished").string())));
Strings new_hosts = getNewAndUpdate(getChildrenAllowNoNode(zookeeper, fs::path(node_path) / "finished"));
++try_number;
if (new_hosts.empty())
continue;
current_active_hosts = getChildrenAllowNoNode(zookeeper, (fs::path(node_path) / fs::path( "active").string()));
current_active_hosts = getChildrenAllowNoNode(zookeeper, fs::path(node_path) / "active");
MutableColumns columns = sample.cloneEmptyColumns();
for (const String & host_id : new_hosts)
@ -1306,7 +1304,7 @@ public:
ExecutionStatus status(-1, "Cannot obtain error message");
{
String status_data;
if (zookeeper->tryGet((fs::path(node_path) / fs::path("finished") / fs::path(host_id)).string(), status_data))
if (zookeeper->tryGet(fs::path(node_path) / "finished" / host_id, status_data))
status.tryDeserializeText(status_data);
}