New upstream version 8.1.0
This commit is contained in:
530
meta/source/components/buddyresyncer/BuddyResyncJob.cpp
Normal file
530
meta/source/components/buddyresyncer/BuddyResyncJob.cpp
Normal file
@@ -0,0 +1,530 @@
|
||||
#include <program/Program.h>
|
||||
|
||||
#include <common/components/worker/IncSyncedCounterWork.h>
|
||||
#include <common/net/message/nodes/SetTargetConsistencyStatesMsg.h>
|
||||
#include <common/net/message/nodes/SetTargetConsistencyStatesRespMsg.h>
|
||||
#include <common/net/message/storage/mirroring/StorageResyncStartedMsg.h>
|
||||
#include <common/net/message/storage/mirroring/StorageResyncStartedRespMsg.h>
|
||||
#include <common/threading/Barrier.h>
|
||||
#include <common/toolkit/DebugVariable.h>
|
||||
#include <common/toolkit/SynchronizedCounter.h>
|
||||
|
||||
#include <app/App.h>
|
||||
#include <components/buddyresyncer/BuddyResyncerBulkSyncSlave.h>
|
||||
#include <components/buddyresyncer/BuddyResyncerModSyncSlave.h>
|
||||
#include <components/worker/BarrierWork.h>
|
||||
#include <toolkit/BuddyCommTk.h>
|
||||
|
||||
#include "BuddyResyncJob.h"
|
||||
|
||||
BuddyResyncJob::BuddyResyncJob() :
|
||||
PThread("BuddyResyncJob"),
|
||||
state(BuddyResyncJobState_NOTSTARTED),
|
||||
startTime(0), endTime(0),
|
||||
gatherSlave(boost::make_unique<BuddyResyncerGatherSlave>(&syncCandidates))
|
||||
{
|
||||
App* app = Program::getApp();
|
||||
Config* cfg = app->getConfig();
|
||||
buddyNodeID =
|
||||
NumNodeID(app->getMetaBuddyGroupMapper()->getBuddyTargetID(app->getLocalNodeNumID().val()));
|
||||
|
||||
const unsigned numSyncSlaves = std::max<unsigned>(cfg->getTuneNumResyncSlaves(), 1);
|
||||
|
||||
for (size_t i = 0; i < numSyncSlaves; i++)
|
||||
bulkSyncSlaves.emplace_back(
|
||||
boost::make_unique<BuddyResyncerBulkSyncSlave>(*this, &syncCandidates, i, buddyNodeID));
|
||||
|
||||
sessionStoreResyncer = boost::make_unique<SessionStoreResyncer>(buddyNodeID);
|
||||
modSyncSlave = boost::make_unique<BuddyResyncerModSyncSlave>(*this, &syncCandidates, 1, buddyNodeID);
|
||||
}
|
||||
|
||||
BuddyResyncJob::~BuddyResyncJob() = default;
|
||||
|
||||
void BuddyResyncJob::run()
|
||||
{
|
||||
const char* logContext = "Run resync job";
|
||||
|
||||
InternodeSyncer* internodeSyncer = Program::getApp()->getInternodeSyncer();
|
||||
App* app = Program::getApp();
|
||||
WorkerList* workers = app->getWorkers();
|
||||
NodeStore* metaNodes = app->getMetaNodes();
|
||||
const std::string metaPath = app->getMetaPath();
|
||||
const std::string metaBuddyMirPath = app->getMetaPath() + "/" + CONFIG_BUDDYMIRROR_SUBDIR_NAME;
|
||||
Barrier workerBarrier(workers->size() + 1);
|
||||
bool workersStopped = false;
|
||||
|
||||
startTime = time(NULL);
|
||||
|
||||
syncCandidates.clear();
|
||||
|
||||
auto buddyNode = metaNodes->referenceNode(buddyNodeID);
|
||||
|
||||
if (!buddyNode)
|
||||
{
|
||||
LOG(MIRRORING, ERR, "Unable to resolve buddy node. Resync will not start.");
|
||||
setState(BuddyResyncJobState_FAILURE);
|
||||
goto cleanup;
|
||||
}
|
||||
|
||||
DEBUG_ENV_VAR(unsigned, DIE_AT_RESYNC_N, 0, "BEEGFS_RESYNC_DIE_AT_N");
|
||||
if (DIE_AT_RESYNC_N) {
|
||||
static unsigned resyncs = 0;
|
||||
// for #479: terminating a server at this point caused the workers to terminate before the
|
||||
// resyncer had communicated with them, causing a deadlock on shutdown
|
||||
if (++resyncs == DIE_AT_RESYNC_N) {
|
||||
::kill(0, SIGTERM);
|
||||
sleep(4);
|
||||
}
|
||||
}
|
||||
stopAllWorkersOn(workerBarrier);
|
||||
{
|
||||
// Notify buddy that resync started and wait for confirmation
|
||||
StorageResyncStartedMsg msg(buddyNodeID.val());
|
||||
const auto respMsg = MessagingTk::requestResponse(*buddyNode, msg,
|
||||
NETMSGTYPE_StorageResyncStartedResp);
|
||||
|
||||
if (!respMsg)
|
||||
{
|
||||
LogContext(logContext).logErr("Unable to notify buddy about resync attempt. "
|
||||
"Resync will not start.");
|
||||
setState(BuddyResyncJobState_FAILURE);
|
||||
workerBarrier.wait();
|
||||
goto cleanup;
|
||||
}
|
||||
|
||||
// resync could have been aborted before we got here. if so, exit as soon as possible without
|
||||
// setting the resync job state to something else.
|
||||
{
|
||||
std::unique_lock<Mutex> lock(stateMutex);
|
||||
|
||||
if (state == BuddyResyncJobState_INTERRUPTED)
|
||||
{
|
||||
lock.unlock();
|
||||
workerBarrier.wait();
|
||||
goto cleanup;
|
||||
}
|
||||
|
||||
state = BuddyResyncJobState_RUNNING;
|
||||
}
|
||||
internodeSyncer->setResyncInProgress(true);
|
||||
|
||||
const bool startGatherSlaveRes = startGatherSlaves();
|
||||
if (!startGatherSlaveRes)
|
||||
{
|
||||
setState(BuddyResyncJobState_FAILURE);
|
||||
workerBarrier.wait();
|
||||
goto cleanup;
|
||||
}
|
||||
|
||||
const bool startResyncSlaveRes = startSyncSlaves();
|
||||
if (!startResyncSlaveRes)
|
||||
{
|
||||
setState(BuddyResyncJobState_FAILURE);
|
||||
workerBarrier.wait();
|
||||
goto cleanup;
|
||||
}
|
||||
}
|
||||
workerBarrier.wait();
|
||||
|
||||
LOG_DEBUG(__func__, Log_DEBUG, "Going to join gather slaves.");
|
||||
joinGatherSlaves();
|
||||
LOG_DEBUG(__func__, Log_DEBUG, "Joined gather slaves.");
|
||||
|
||||
LOG_DEBUG(__func__, Log_DEBUG, "Going to join sync slaves.");
|
||||
|
||||
// gather slaves have finished. Tell sync slaves to stop when work packages are empty and wait.
|
||||
for (auto it = bulkSyncSlaves.begin(); it != bulkSyncSlaves.end(); ++it)
|
||||
{
|
||||
(*it)->setOnlyTerminateIfIdle(true);
|
||||
(*it)->selfTerminate();
|
||||
}
|
||||
|
||||
for (auto it = bulkSyncSlaves.begin(); it != bulkSyncSlaves.end(); ++it)
|
||||
(*it)->join();
|
||||
|
||||
// here we can be in one of two situations:
|
||||
// 1. bulk resync has succeeded. we then totally stop the workers: the session store must be in
|
||||
// a quiescent state for resync, so for simplicitly, we suspend all client operations here.
|
||||
// we do not want to do this any earlier than this point, because bulk syncers may take a
|
||||
// very long time to complete.
|
||||
// 2. bulk resync has failed. in this case, the bulk syncers have aborted the currently running
|
||||
// job, and the mod syncer is either dead or in the process of dying. here we MUST NOT stop
|
||||
// the workers, because they are very likely blocked on the mod sync queue already and will
|
||||
// not unblock before the queue is cleared.
|
||||
if (getState() == BuddyResyncJobState_RUNNING)
|
||||
{
|
||||
stopAllWorkersOn(workerBarrier);
|
||||
workersStopped = true;
|
||||
}
|
||||
|
||||
modSyncSlave->setOnlyTerminateIfIdle(true);
|
||||
modSyncSlave->selfTerminate();
|
||||
modSyncSlave->join();
|
||||
|
||||
// gatherers are done and the workers have been stopped, we can safely resync the session now.
|
||||
|
||||
LOG_DEBUG(__func__, Log_DEBUG, "Joined sync slaves.");
|
||||
|
||||
// Perform session store resync
|
||||
// the job may have been aborted or terminated by errors. in this case, do not resync the session
|
||||
// store. end the sync as quickly as possible.
|
||||
if (getState() == BuddyResyncJobState_RUNNING)
|
||||
sessionStoreResyncer->doSync();
|
||||
|
||||
// session store is now synced, and future actions can be forwarded safely. we do not restart
|
||||
// the workers here because the resync may still enter FAILED state, and we don't want to forward
|
||||
// to the secondary in this case.
|
||||
|
||||
cleanup:
|
||||
bool syncErrors = false;
|
||||
|
||||
{
|
||||
std::lock_guard<Mutex> lock(gatherSlave->stateMutex);
|
||||
while (gatherSlave->isRunning)
|
||||
gatherSlave->isRunningChangeCond.wait(&gatherSlave->stateMutex);
|
||||
|
||||
syncErrors |= gatherSlave->getStats().errors != 0;
|
||||
}
|
||||
|
||||
for (auto it = bulkSyncSlaves.begin(); it != bulkSyncSlaves.end(); ++it)
|
||||
{
|
||||
BuddyResyncerBulkSyncSlave* slave = it->get();
|
||||
std::lock_guard<Mutex> lock(slave->stateMutex);
|
||||
while (slave->isRunning)
|
||||
slave->isRunningChangeCond.wait(&slave->stateMutex);
|
||||
|
||||
syncErrors |= slave->getStats().dirErrors != 0;
|
||||
syncErrors |= slave->getStats().fileErrors != 0;
|
||||
}
|
||||
|
||||
syncErrors |= sessionStoreResyncer->getStats().errors;
|
||||
|
||||
{
|
||||
while (modSyncSlave->isRunning)
|
||||
modSyncSlave->isRunningChangeCond.wait(&modSyncSlave->stateMutex);
|
||||
|
||||
syncErrors |= modSyncSlave->getStats().errors != 0;
|
||||
}
|
||||
|
||||
|
||||
if (getState() == BuddyResyncJobState_RUNNING || getState() == BuddyResyncJobState_INTERRUPTED)
|
||||
{
|
||||
if (syncErrors)
|
||||
setState(BuddyResyncJobState_ERRORS);
|
||||
else if (getState() == BuddyResyncJobState_RUNNING)
|
||||
setState(BuddyResyncJobState_SUCCESS);
|
||||
|
||||
// delete timestamp override file if it exists.
|
||||
BuddyCommTk::setBuddyNeedsResync(metaPath, false);
|
||||
|
||||
const TargetConsistencyState buddyState = newBuddyState();
|
||||
informBuddy(buddyState);
|
||||
informMgmtd(buddyState);
|
||||
|
||||
const bool interrupted = getState() != BuddyResyncJobState_SUCCESS;
|
||||
LOG(MIRRORING, WARNING, "Resync finished.", interrupted, syncErrors);
|
||||
}
|
||||
|
||||
internodeSyncer->setResyncInProgress(false);
|
||||
endTime = time(NULL);
|
||||
|
||||
// restart all the worker threads
|
||||
if (workersStopped)
|
||||
workerBarrier.wait();
|
||||
|
||||
// if the resync was aborted, the mod sync queue may still contain items. additionally, workers
|
||||
// may be waiting for a changeset slot, or they may have started executing after the resync was
|
||||
// aborted by the sync slaves, but before the resync was officially set to "not running".
|
||||
// we cannot set the resync to "not running" in abort() because we have no upper bound for the
|
||||
// number of worker threads. even if we did set the resync to "not running" in abort() and
|
||||
// cleared the sync queues at the same time, there may still be an arbitrary number of threads
|
||||
// waiting for a changeset slot.
|
||||
// instead, we have to wait for each thread to "see" that the resync is over, and periodically
|
||||
// clear the sync queue to unblock those workers that are still waiting for slots.
|
||||
if (syncErrors)
|
||||
{
|
||||
SynchronizedCounter counter;
|
||||
|
||||
for (auto it = workers->begin(); it != workers->end(); ++it)
|
||||
{
|
||||
auto& worker = **it;
|
||||
|
||||
worker.getWorkQueue()->addPersonalWork(
|
||||
new IncSyncedCounterWork(&counter),
|
||||
worker.getPersonalWorkQueue());
|
||||
}
|
||||
|
||||
while (!counter.timedWaitForCount(workers->size(), 100))
|
||||
{
|
||||
while (!syncCandidates.isFilesEmpty())
|
||||
{
|
||||
MetaSyncCandidateFile candidate;
|
||||
syncCandidates.fetch(candidate, this);
|
||||
candidate.signal();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void BuddyResyncJob::stopAllWorkersOn(Barrier& barrier)
|
||||
{
|
||||
WorkerList* workers = Program::getApp()->getWorkers();
|
||||
|
||||
for (WorkerListIter workerIt = workers->begin(); workerIt != workers->end(); ++workerIt)
|
||||
{
|
||||
Worker* worker = *workerIt;
|
||||
PersonalWorkQueue* personalQ = worker->getPersonalWorkQueue();
|
||||
MultiWorkQueue* workQueue = worker->getWorkQueue();
|
||||
workQueue->addPersonalWork(new BarrierWork(&barrier), personalQ);
|
||||
}
|
||||
|
||||
barrier.wait(); // Wait until all workers are blocked.
|
||||
}
|
||||
|
||||
void BuddyResyncJob::abort(bool wait_for_completion)
|
||||
{
|
||||
setState(BuddyResyncJobState_INTERRUPTED);
|
||||
|
||||
gatherSlave->selfTerminate();
|
||||
|
||||
// set onlyTerminateIfIdle on the slaves to false - they will be stopped by the main loop then.
|
||||
for (auto it = bulkSyncSlaves.begin(); it != bulkSyncSlaves.end(); ++it)
|
||||
{
|
||||
BuddyResyncerBulkSyncSlave* slave = it->get();
|
||||
slave->setOnlyTerminateIfIdle(false);
|
||||
}
|
||||
|
||||
modSyncSlave->selfTerminate();
|
||||
|
||||
int retry = 600;
|
||||
/* Wait till all on-going thread events are fetched or max 30mins.
|
||||
* (fetch waits for 3secs if there are no files to be fetched)
|
||||
*/
|
||||
if (wait_for_completion)
|
||||
{
|
||||
modSyncSlave->join();
|
||||
while (threadCount > 0 && retry)
|
||||
{
|
||||
LOG(MIRRORING, WARNING, "Wait for pending worker threads to finish");
|
||||
if (!syncCandidates.isFilesEmpty())
|
||||
{
|
||||
MetaSyncCandidateFile candidate;
|
||||
syncCandidates.fetch(candidate, this);
|
||||
candidate.signal();
|
||||
}
|
||||
retry--;
|
||||
}
|
||||
if (threadCount)
|
||||
LOG(MIRRORING, ERR, "Cleanup of aborted resync failed: I/O worker threads"
|
||||
" did not finish properly: ",
|
||||
("threadCount", threadCount.load()));
|
||||
}
|
||||
}
|
||||
|
||||
bool BuddyResyncJob::startGatherSlaves()
|
||||
{
|
||||
try
|
||||
{
|
||||
gatherSlave->resetSelfTerminate();
|
||||
gatherSlave->start();
|
||||
gatherSlave->setIsRunning(true);
|
||||
}
|
||||
catch (PThreadCreateException& e)
|
||||
{
|
||||
LogContext(__func__).logErr(std::string("Unable to start thread: ") + e.what());
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool BuddyResyncJob::startSyncSlaves()
|
||||
{
|
||||
App* app = Program::getApp();
|
||||
const NumNodeID localNodeID = app->getLocalNodeNumID();
|
||||
const NumNodeID buddyNodeID(
|
||||
app->getMetaBuddyGroupMapper()->getBuddyTargetID(localNodeID.val(), NULL) );
|
||||
|
||||
for (size_t i = 0; i < bulkSyncSlaves.size(); i++)
|
||||
{
|
||||
try
|
||||
{
|
||||
bulkSyncSlaves[i]->resetSelfTerminate();
|
||||
bulkSyncSlaves[i]->start();
|
||||
bulkSyncSlaves[i]->setIsRunning(true);
|
||||
}
|
||||
catch (PThreadCreateException& e)
|
||||
{
|
||||
LogContext(__func__).logErr(std::string("Unable to start thread: ") + e.what() );
|
||||
|
||||
for (size_t j = 0; j < i; j++)
|
||||
bulkSyncSlaves[j]->selfTerminate();
|
||||
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
try
|
||||
{
|
||||
modSyncSlave->resetSelfTerminate();
|
||||
modSyncSlave->start();
|
||||
modSyncSlave->setIsRunning(true);
|
||||
}
|
||||
catch (PThreadCreateException& e)
|
||||
{
|
||||
LogContext(__func__).logErr(std::string("Unable to start thread: ") + e.what() );
|
||||
|
||||
for (size_t j = 0; j < bulkSyncSlaves.size(); j++)
|
||||
bulkSyncSlaves[j]->selfTerminate();
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
void BuddyResyncJob::joinGatherSlaves()
|
||||
{
|
||||
gatherSlave->join();
|
||||
}
|
||||
|
||||
MetaBuddyResyncJobStatistics BuddyResyncJob::getJobStats()
|
||||
{
|
||||
std::lock_guard<Mutex> lock(stateMutex);
|
||||
|
||||
BuddyResyncerGatherSlave::Stats gatherStats = gatherSlave->getStats();
|
||||
const uint64_t dirsDiscovered = gatherStats.dirsDiscovered;
|
||||
const uint64_t gatherErrors = gatherStats.errors;
|
||||
|
||||
uint64_t dirsSynced = 0;
|
||||
uint64_t filesSynced = 0;
|
||||
uint64_t dirErrors = 0;
|
||||
uint64_t fileErrors = 0;
|
||||
|
||||
for (auto syncerIt = bulkSyncSlaves.begin(); syncerIt != bulkSyncSlaves.end(); ++syncerIt)
|
||||
{
|
||||
BuddyResyncerBulkSyncSlave::Stats bulkSyncStats = (*syncerIt)->getStats();
|
||||
dirsSynced += bulkSyncStats.dirsSynced;
|
||||
filesSynced += bulkSyncStats.filesSynced;
|
||||
dirErrors += bulkSyncStats.dirErrors;
|
||||
fileErrors += bulkSyncStats.fileErrors;
|
||||
}
|
||||
|
||||
SessionStoreResyncer::Stats sessionSyncStats = sessionStoreResyncer->getStats();
|
||||
const uint64_t sessionsToSync = sessionSyncStats.sessionsToSync;
|
||||
const uint64_t sessionsSynced = sessionSyncStats.sessionsSynced;
|
||||
const bool sessionSyncErrors = sessionSyncStats.errors;
|
||||
|
||||
BuddyResyncerModSyncSlave::Stats modSyncStats = modSyncSlave->getStats();
|
||||
uint64_t modObjectsSynced = modSyncStats.objectsSynced;
|
||||
uint64_t modSyncErrors = modSyncStats.errors;
|
||||
|
||||
return MetaBuddyResyncJobStatistics(
|
||||
state, startTime, endTime,
|
||||
dirsDiscovered, gatherErrors,
|
||||
dirsSynced, filesSynced, dirErrors, fileErrors,
|
||||
sessionsToSync, sessionsSynced, sessionSyncErrors,
|
||||
modObjectsSynced, modSyncErrors);
|
||||
}
|
||||
|
||||
/**
|
||||
* Determine the state for the buddy after the end of a resync job.
|
||||
* @returns the new state to be set on the buddy accroding to this job's JobState.
|
||||
*/
|
||||
TargetConsistencyState BuddyResyncJob::newBuddyState()
|
||||
{
|
||||
switch (getState())
|
||||
{
|
||||
case BuddyResyncJobState_ERRORS:
|
||||
case BuddyResyncJobState_INTERRUPTED:
|
||||
case BuddyResyncJobState_FAILURE:
|
||||
return TargetConsistencyState_BAD;
|
||||
|
||||
case BuddyResyncJobState_SUCCESS:
|
||||
return TargetConsistencyState_GOOD;
|
||||
|
||||
default:
|
||||
LOG(MIRRORING, ERR, "Undefined resync state.", state);
|
||||
return TargetConsistencyState_BAD;
|
||||
}
|
||||
}
|
||||
|
||||
void BuddyResyncJob::informBuddy(const TargetConsistencyState newTargetState)
|
||||
{
|
||||
App* app = Program::getApp();
|
||||
NodeStore* metaNodes = app->getMetaNodes();
|
||||
MirrorBuddyGroupMapper* buddyGroups = app->getMetaBuddyGroupMapper();
|
||||
|
||||
NumNodeID buddyNodeID =
|
||||
NumNodeID(buddyGroups->getBuddyTargetID(app->getLocalNodeNumID().val()));
|
||||
auto metaNode = metaNodes->referenceNode(buddyNodeID);
|
||||
|
||||
if (!metaNode)
|
||||
{
|
||||
LOG(MIRRORING, ERR, "Unable to inform buddy about finished resync", buddyNodeID.str());
|
||||
return;
|
||||
}
|
||||
|
||||
UInt16List nodeIDs(1, buddyNodeID.val());
|
||||
UInt8List states(1, newTargetState);
|
||||
SetTargetConsistencyStatesMsg msg(NODETYPE_Meta, &nodeIDs, &states, false);
|
||||
|
||||
const auto respMsg = MessagingTk::requestResponse(*metaNode, msg,
|
||||
NETMSGTYPE_SetTargetConsistencyStatesResp);
|
||||
if (!respMsg)
|
||||
{
|
||||
LogContext(__func__).logErr(
|
||||
"Unable to inform buddy about finished resync. "
|
||||
"BuddyNodeID: " + buddyNodeID.str() + "; "
|
||||
"error: Communication Error");
|
||||
return;
|
||||
}
|
||||
|
||||
{
|
||||
auto* respMsgCast = static_cast<SetTargetConsistencyStatesRespMsg*>(respMsg.get());
|
||||
FhgfsOpsErr result = respMsgCast->getResult();
|
||||
|
||||
if (result != FhgfsOpsErr_SUCCESS)
|
||||
{
|
||||
LogContext(__func__).logErr(
|
||||
"Error while informing buddy about finished resync. "
|
||||
"BuddyNodeID: " + buddyNodeID.str() + "; "
|
||||
"error: " + boost::lexical_cast<std::string>(result) );
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void BuddyResyncJob::informMgmtd(const TargetConsistencyState newTargetState)
|
||||
{
|
||||
App* app = Program::getApp();
|
||||
NodeStore* mgmtNodes = app->getMgmtNodes();
|
||||
|
||||
auto mgmtNode = mgmtNodes->referenceFirstNode();
|
||||
|
||||
if (!mgmtNode)
|
||||
{
|
||||
LOG(MIRRORING, ERR, "Unable to communicate with management node.");
|
||||
return;
|
||||
}
|
||||
|
||||
UInt16List nodeIDs(1, buddyNodeID.val());
|
||||
UInt8List states(1, newTargetState);
|
||||
SetTargetConsistencyStatesMsg msg(NODETYPE_Meta, &nodeIDs, &states, false);
|
||||
|
||||
const auto respMsg = MessagingTk::requestResponse(*mgmtNode, msg,
|
||||
NETMSGTYPE_SetTargetConsistencyStatesResp);
|
||||
if (!respMsg)
|
||||
{
|
||||
LOG(MIRRORING, ERR,
|
||||
"Unable to inform management node about finished resync: Communication error.");
|
||||
return;
|
||||
}
|
||||
|
||||
{
|
||||
auto* respMsgCast = static_cast<SetTargetConsistencyStatesRespMsg*>(respMsg.get());
|
||||
FhgfsOpsErr result = respMsgCast->getResult();
|
||||
|
||||
if (result != FhgfsOpsErr_SUCCESS)
|
||||
LOG(MIRRORING, ERR, "Error informing management node about finished resync.", result);
|
||||
}
|
||||
}
|
||||
92
meta/source/components/buddyresyncer/BuddyResyncJob.h
Normal file
92
meta/source/components/buddyresyncer/BuddyResyncJob.h
Normal file
@@ -0,0 +1,92 @@
|
||||
#pragma once
|
||||
|
||||
#include <components/buddyresyncer/BuddyResyncerGatherSlave.h>
|
||||
#include <components/buddyresyncer/SessionStoreResyncer.h>
|
||||
#include <common/storage/mirroring/BuddyResyncJobStatistics.h>
|
||||
#include <common/storage/mirroring/SyncCandidateStore.h>
|
||||
#include <common/threading/PThread.h>
|
||||
#include <common/nodes/NumNodeID.h>
|
||||
#include <common/nodes/TargetStateInfo.h>
|
||||
|
||||
#include <atomic>
|
||||
#include <mutex>
|
||||
|
||||
class BuddyResyncerBulkSyncSlave;
|
||||
class BuddyResyncerModSyncSlave;
|
||||
|
||||
class BuddyResyncJob : public PThread
|
||||
{
|
||||
public:
|
||||
BuddyResyncJob();
|
||||
~BuddyResyncJob();
|
||||
|
||||
virtual void run();
|
||||
|
||||
void abort(bool wait_for_completion);
|
||||
MetaBuddyResyncJobStatistics getJobStats();
|
||||
std::atomic<unsigned int> threadCount{ 0 };
|
||||
|
||||
private:
|
||||
BuddyResyncJobState state;
|
||||
Mutex stateMutex;
|
||||
|
||||
int64_t startTime;
|
||||
int64_t endTime;
|
||||
|
||||
NumNodeID buddyNodeID;
|
||||
|
||||
MetaSyncCandidateStore syncCandidates;
|
||||
|
||||
std::unique_ptr<BuddyResyncerGatherSlave> gatherSlave;
|
||||
std::vector<std::unique_ptr<BuddyResyncerBulkSyncSlave>> bulkSyncSlaves;
|
||||
std::unique_ptr<BuddyResyncerModSyncSlave> modSyncSlave;
|
||||
std::unique_ptr<SessionStoreResyncer> sessionStoreResyncer;
|
||||
|
||||
bool startGatherSlaves();
|
||||
bool startSyncSlaves();
|
||||
void joinGatherSlaves();
|
||||
|
||||
public:
|
||||
BuddyResyncJobState getState()
|
||||
{
|
||||
std::lock_guard<Mutex> lock(stateMutex);
|
||||
return state;
|
||||
}
|
||||
|
||||
bool isRunning()
|
||||
{
|
||||
std::lock_guard<Mutex> lock(stateMutex);
|
||||
return state == BuddyResyncJobState_RUNNING;
|
||||
}
|
||||
|
||||
void enqueue(MetaSyncCandidateFile syncCandidate, PThread* caller)
|
||||
{
|
||||
syncCandidates.add(std::move(syncCandidate), caller);
|
||||
}
|
||||
|
||||
void registerOps()
|
||||
{
|
||||
this->threadCount += 1;
|
||||
}
|
||||
|
||||
void unregisterOps()
|
||||
{
|
||||
this->threadCount -= 1;
|
||||
}
|
||||
|
||||
private:
|
||||
void setState(const BuddyResyncJobState state)
|
||||
{
|
||||
LOG_DEBUG(__func__, Log_DEBUG, "Setting state: "
|
||||
+ StringTk::uintToStr(static_cast<int>(state) ) );
|
||||
std::lock_guard<Mutex> lock(stateMutex);
|
||||
this->state = state;
|
||||
}
|
||||
|
||||
TargetConsistencyState newBuddyState();
|
||||
void informBuddy(const TargetConsistencyState newTargetState);
|
||||
void informMgmtd(const TargetConsistencyState newTargetState);
|
||||
|
||||
void stopAllWorkersOn(Barrier& barrier);
|
||||
};
|
||||
|
||||
89
meta/source/components/buddyresyncer/BuddyResyncer.cpp
Normal file
89
meta/source/components/buddyresyncer/BuddyResyncer.cpp
Normal file
@@ -0,0 +1,89 @@
|
||||
#include "BuddyResyncer.h"
|
||||
|
||||
#include <program/Program.h>
|
||||
|
||||
__thread MetaSyncCandidateFile* BuddyResyncer::currentThreadChangeSet = 0;
|
||||
|
||||
BuddyResyncer::~BuddyResyncer()
|
||||
{
|
||||
if (job)
|
||||
{
|
||||
job->abort(false);
|
||||
job->join();
|
||||
|
||||
SAFE_DELETE(job);
|
||||
}
|
||||
}
|
||||
|
||||
FhgfsOpsErr BuddyResyncer::startResync()
|
||||
{
|
||||
std::lock_guard<Mutex> lock(jobMutex);
|
||||
|
||||
if (noNewResyncs)
|
||||
return FhgfsOpsErr_INTERRUPTED;
|
||||
|
||||
if (!job)
|
||||
{
|
||||
job = new BuddyResyncJob();
|
||||
job->start();
|
||||
return FhgfsOpsErr_SUCCESS;
|
||||
}
|
||||
|
||||
switch (job->getState())
|
||||
{
|
||||
case BuddyResyncJobState_NOTSTARTED:
|
||||
case BuddyResyncJobState_RUNNING:
|
||||
return FhgfsOpsErr_INUSE;
|
||||
|
||||
default:
|
||||
// a job must never be started more than once. to ensure this, we create a new job for
|
||||
// every resync process, because doing so allows us to use NOTSTARTED and RUNNING as
|
||||
// "job is currently active" values. otherwise, a second resync may see state SUCCESS and
|
||||
// allow duplicate resyncer activity.
|
||||
// if a job is still active, don't wait for very long - it may take a while to finish. the
|
||||
// internode syncer will retry periodically, so this will work fine.
|
||||
if (!job->timedjoin(10))
|
||||
return FhgfsOpsErr_INUSE;
|
||||
|
||||
delete job;
|
||||
job = new BuddyResyncJob();
|
||||
job->start();
|
||||
return FhgfsOpsErr_SUCCESS;
|
||||
}
|
||||
}
|
||||
|
||||
void BuddyResyncer::shutdown()
|
||||
{
|
||||
std::unique_ptr<BuddyResyncJob> job;
|
||||
|
||||
{
|
||||
std::lock_guard<Mutex> lock(jobMutex);
|
||||
|
||||
job.reset(this->job);
|
||||
this->job = nullptr;
|
||||
noNewResyncs = true;
|
||||
}
|
||||
|
||||
if (job)
|
||||
{
|
||||
job->abort(false);
|
||||
job->join();
|
||||
}
|
||||
}
|
||||
|
||||
void BuddyResyncer::commitThreadChangeSet()
|
||||
{
|
||||
BEEGFS_BUG_ON(!currentThreadChangeSet, "no change set active");
|
||||
|
||||
auto* job = Program::getApp()->getBuddyResyncer()->getResyncJob();
|
||||
|
||||
std::unique_ptr<MetaSyncCandidateFile> candidate(currentThreadChangeSet);
|
||||
currentThreadChangeSet = nullptr;
|
||||
|
||||
Barrier syncDone(2);
|
||||
|
||||
candidate->prepareSignal(syncDone);
|
||||
|
||||
job->enqueue(std::move(*candidate), PThread::getCurrentThread());
|
||||
syncDone.wait();
|
||||
}
|
||||
69
meta/source/components/buddyresyncer/BuddyResyncer.h
Normal file
69
meta/source/components/buddyresyncer/BuddyResyncer.h
Normal file
@@ -0,0 +1,69 @@
|
||||
#pragma once
|
||||
|
||||
#include <components/buddyresyncer/BuddyResyncJob.h>
|
||||
#include <common/storage/StorageErrors.h>
|
||||
#include <common/Common.h>
|
||||
|
||||
#include <mutex>
|
||||
|
||||
/**
|
||||
* This component does not represent a thread by itself. Instead, it manages a group of "slave
|
||||
* threads" that are started and stopped when needed.
|
||||
*
|
||||
* Other components should only use this component as an interface and not access the slave threads
|
||||
* directly.
|
||||
*/
|
||||
class BuddyResyncer
|
||||
{
|
||||
public:
|
||||
BuddyResyncer()
|
||||
: job(NULL), noNewResyncs(false)
|
||||
{ }
|
||||
|
||||
~BuddyResyncer();
|
||||
|
||||
FhgfsOpsErr startResync();
|
||||
void shutdown();
|
||||
|
||||
static void commitThreadChangeSet();
|
||||
|
||||
private:
|
||||
BuddyResyncJob* job; // Note: In the Storage Server, this is a Map. Here it's just one pointer
|
||||
// that's set to NULL when no job is present.
|
||||
Mutex jobMutex;
|
||||
|
||||
public:
|
||||
BuddyResyncJob* getResyncJob()
|
||||
{
|
||||
std::lock_guard<Mutex> lock(jobMutex);
|
||||
return job;
|
||||
}
|
||||
|
||||
static void registerSyncChangeset()
|
||||
{
|
||||
BEEGFS_BUG_ON(currentThreadChangeSet, "current changeset not nullptr");
|
||||
|
||||
currentThreadChangeSet = new MetaSyncCandidateFile;
|
||||
}
|
||||
|
||||
static void abandonSyncChangeset()
|
||||
{
|
||||
delete currentThreadChangeSet;
|
||||
currentThreadChangeSet = nullptr;
|
||||
}
|
||||
|
||||
static MetaSyncCandidateFile* getSyncChangeset()
|
||||
{
|
||||
return currentThreadChangeSet;
|
||||
}
|
||||
|
||||
private:
|
||||
static __thread MetaSyncCandidateFile* currentThreadChangeSet;
|
||||
|
||||
bool noNewResyncs;
|
||||
|
||||
// No copy allowed
|
||||
BuddyResyncer(const BuddyResyncer&);
|
||||
BuddyResyncer& operator=(const BuddyResyncer&);
|
||||
};
|
||||
|
||||
@@ -0,0 +1,234 @@
|
||||
#include "BuddyResyncerBulkSyncSlave.h"
|
||||
|
||||
#include <common/net/message/storage/mirroring/ResyncRawInodesRespMsg.h>
|
||||
#include <common/toolkit/StringTk.h>
|
||||
#include <common/toolkit/MessagingTk.h>
|
||||
#include <common/Common.h>
|
||||
|
||||
#include <net/message/storage/mirroring/ResyncRawInodesMsgEx.h>
|
||||
#include <net/msghelpers/MsgHelperXAttr.h>
|
||||
#include <program/Program.h>
|
||||
#include <toolkit/XAttrTk.h>
|
||||
|
||||
#include <dirent.h>
|
||||
|
||||
BuddyResyncerBulkSyncSlave::BuddyResyncerBulkSyncSlave(BuddyResyncJob& parentJob,
|
||||
MetaSyncCandidateStore* syncCandidates, uint8_t slaveID, const NumNodeID& buddyNodeID) :
|
||||
SyncSlaveBase("BuddyResyncerBulkSyncSlave_" + StringTk::uintToStr(slaveID), parentJob,
|
||||
buddyNodeID),
|
||||
syncCandidates(syncCandidates)
|
||||
{
|
||||
}
|
||||
|
||||
void BuddyResyncerBulkSyncSlave::syncLoop()
|
||||
{
|
||||
EntryLockStore* const lockStore = Program::getApp()->getMirroredSessions()->getEntryLockStore();
|
||||
|
||||
while (!getSelfTerminateNotIdle())
|
||||
{
|
||||
MetaSyncCandidateDir candidate;
|
||||
syncCandidates->fetch(candidate, this);
|
||||
|
||||
// the sync candidate we have retrieved may be invalid if this thread was ordered to
|
||||
// terminate and the sync candidate store has no more directories queued for us.
|
||||
// in this case, we may end the sync because we have no more candidates, and the resync job
|
||||
// guarantees that all gather threads have completed before the bulk syncers are ordered to
|
||||
// finish.
|
||||
if (syncCandidates->isDirsEmpty() && candidate.getRelativePath().empty() &&
|
||||
getSelfTerminate())
|
||||
return;
|
||||
|
||||
if (candidate.getType() == MetaSyncDirType::InodesHashDir ||
|
||||
candidate.getType() == MetaSyncDirType::DentriesHashDir)
|
||||
{
|
||||
// lock the hash path in accordance with MkLocalDir, RmLocalDir and RmDir.
|
||||
const auto& hashDir = candidate.getRelativePath();
|
||||
auto slash1 = hashDir.find('/');
|
||||
auto slash2 = hashDir.find('/', slash1 + 1);
|
||||
auto hash1 = StringTk::strHexToUInt(hashDir.substr(slash1 + 1, slash2 - slash1 - 1));
|
||||
auto hash2 = StringTk::strHexToUInt(hashDir.substr(slash2 + 1));
|
||||
HashDirLock hashLock = {lockStore, {hash1, hash2}};
|
||||
|
||||
const FhgfsOpsErr resyncRes = resyncDirectory(candidate, "");
|
||||
if (resyncRes == FhgfsOpsErr_SUCCESS)
|
||||
continue;
|
||||
|
||||
numDirErrors.increase();
|
||||
parentJob->abort(false);
|
||||
return;
|
||||
}
|
||||
|
||||
// not a hash dir, so it must be a content directory. sync the #fSiDs# first, then the actual
|
||||
// content directory. we lock the directory inode the content directory belongs to because we
|
||||
// must not allow a concurrent meta action to delete the content directory while we are
|
||||
// resyncing it. concurrent modification of directory contents could be allowed, though.
|
||||
|
||||
const std::string dirInodeID = Path(candidate.getRelativePath()).back();
|
||||
const std::string fullPath = META_BUDDYMIRROR_SUBDIR_NAME "/" + candidate.getRelativePath();
|
||||
|
||||
FileIDLock dirLock(lockStore, dirInodeID, false);
|
||||
|
||||
// first ensure that the directory still exists - a concurrent modification may have deleted
|
||||
// it. this would not be an error; bulk resync should not touch it, an modification sync
|
||||
// would remove it completely.
|
||||
if (::access(fullPath.c_str(), F_OK) != 0 && errno == ENOENT)
|
||||
{
|
||||
numDirsSynced.increase(); // Count it anyway, so the sums match up.
|
||||
continue;
|
||||
}
|
||||
|
||||
MetaSyncCandidateDir fsIDs(
|
||||
candidate.getRelativePath() + "/" + META_DIRENTRYID_SUB_STR,
|
||||
MetaSyncDirType::InodesHashDir);
|
||||
|
||||
FhgfsOpsErr resyncRes = resyncDirectory(fsIDs, dirInodeID);
|
||||
if (resyncRes == FhgfsOpsErr_SUCCESS)
|
||||
resyncRes = resyncDirectory(candidate, dirInodeID);
|
||||
|
||||
if (resyncRes != FhgfsOpsErr_SUCCESS)
|
||||
{
|
||||
numDirErrors.increase();
|
||||
parentJob->abort(false);
|
||||
return;
|
||||
}
|
||||
else
|
||||
{
|
||||
numDirsSynced.increase();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
FhgfsOpsErr BuddyResyncerBulkSyncSlave::resyncDirectory(const MetaSyncCandidateDir& root,
|
||||
const std::string& inodeID)
|
||||
{
|
||||
StreamCandidateArgs args(*this, root, inodeID);
|
||||
|
||||
return resyncAt(Path(root.getRelativePath()), true, streamCandidateDir, &args);
|
||||
}
|
||||
|
||||
FhgfsOpsErr BuddyResyncerBulkSyncSlave::streamCandidateDir(Socket& socket,
|
||||
const MetaSyncCandidateDir& candidate, const std::string& inodeID)
|
||||
{
|
||||
EntryLockStore* const lockStore = Program::getApp()->getMirroredSessions()->getEntryLockStore();
|
||||
|
||||
Path candidatePath(META_BUDDYMIRROR_SUBDIR_NAME "/" + candidate.getRelativePath());
|
||||
|
||||
std::unique_ptr<DIR, StorageTk::CloseDirDeleter> dir(opendir(candidatePath.str().c_str()));
|
||||
|
||||
if (!dir)
|
||||
{
|
||||
LOG(MIRRORING, ERR, "Could not open candidate directory.", candidatePath, sysErr);
|
||||
return FhgfsOpsErr_INTERNAL;
|
||||
}
|
||||
|
||||
int dirFD = ::dirfd(dir.get());
|
||||
if (dirFD < 0)
|
||||
{
|
||||
LOG(MIRRORING, ERR, "Could not open candidate directory.", candidatePath, sysErr);
|
||||
return FhgfsOpsErr_INTERNAL;
|
||||
}
|
||||
|
||||
while (true)
|
||||
{
|
||||
struct dirent* entry;
|
||||
|
||||
#if USE_READDIR_P
|
||||
struct dirent entryBuf;
|
||||
int err = ::readdir_r(dir.get(), &entryBuf, &entry);
|
||||
#else
|
||||
errno = 0;
|
||||
entry = readdir(dir.get());
|
||||
int err = entry ? 0 : errno;
|
||||
#endif
|
||||
if (err > 0)
|
||||
{
|
||||
LOG(MIRRORING, ERR, "Could not read candidate directory.", candidatePath, sysErr);
|
||||
numDirErrors.increase();
|
||||
break;
|
||||
}
|
||||
|
||||
if (!entry)
|
||||
break;
|
||||
|
||||
if (strcmp(entry->d_name, ".") == 0 || strcmp(entry->d_name, "..") == 0)
|
||||
continue;
|
||||
|
||||
struct stat statData;
|
||||
if (::fstatat(dirFD, entry->d_name, &statData, AT_SYMLINK_NOFOLLOW) < 0)
|
||||
{
|
||||
// the file/directory may have gone away. this is not an error, and the secondary will
|
||||
// delete the file/directory as well.
|
||||
if (errno == ENOENT)
|
||||
continue;
|
||||
|
||||
LOG(MIRRORING, ERR, "Could not stat resync candidate.",
|
||||
candidatePath, entry->d_name, sysErr);
|
||||
numFileErrors.increase();
|
||||
continue;
|
||||
}
|
||||
|
||||
if (!S_ISDIR(statData.st_mode) && !S_ISREG(statData.st_mode))
|
||||
{
|
||||
LOG(MIRRORING, ERR, "Resync candidate is neither file nor directory.",
|
||||
candidatePath, entry->d_name, statData.st_mode);
|
||||
numFileErrors.increase();
|
||||
continue;
|
||||
}
|
||||
|
||||
if (candidate.getType() == MetaSyncDirType::ContentDir)
|
||||
{
|
||||
// if it's in a content directory and a directory, it can really only be the fsids dir.
|
||||
// locking for this case is already sorted, so we only have to transfer the (empty)
|
||||
// inode metadata to tell the secondary that the directory may stay.
|
||||
if (S_ISDIR(statData.st_mode))
|
||||
{
|
||||
const FhgfsOpsErr streamRes = streamInode(socket, Path(entry->d_name), true);
|
||||
if (streamRes != FhgfsOpsErr_SUCCESS)
|
||||
return streamRes;
|
||||
}
|
||||
else
|
||||
{
|
||||
ParentNameLock dentryLock(lockStore, inodeID, entry->d_name);
|
||||
|
||||
const auto streamRes = streamDentry(socket, Path(), entry->d_name);
|
||||
if (streamRes != FhgfsOpsErr_SUCCESS)
|
||||
{
|
||||
numFileErrors.increase();
|
||||
return streamRes;
|
||||
}
|
||||
else
|
||||
{
|
||||
numFilesSynced.increase();
|
||||
}
|
||||
}
|
||||
|
||||
continue;
|
||||
}
|
||||
|
||||
// we are now either in a fsids (file inode) directory or a second-level inode hash-dir,
|
||||
// which may contain either file or directory inodes. taking a lock unnecessarily is stilll
|
||||
// cheaper than reading the inode from disk to determine its type, so just lock the inode id
|
||||
// as file
|
||||
FileIDLock dirLock(lockStore, entry->d_name, true);
|
||||
|
||||
// access the file once more, because it may have been deleted in the meantime. a new entry
|
||||
// with the same name cannot appear in a sane filesystem (that would indicate an ID being
|
||||
// reused).
|
||||
if (faccessat(dirFD, entry->d_name, F_OK, 0) < 0 && errno == ENOENT)
|
||||
continue;
|
||||
|
||||
const FhgfsOpsErr streamRes = streamInode(socket, Path(entry->d_name),
|
||||
S_ISDIR(statData.st_mode));
|
||||
if (streamRes != FhgfsOpsErr_SUCCESS)
|
||||
{
|
||||
numFileErrors.increase();
|
||||
return streamRes;
|
||||
}
|
||||
else
|
||||
{
|
||||
numFilesSynced.increase();
|
||||
}
|
||||
}
|
||||
|
||||
return sendResyncPacket(socket, std::tuple<>());
|
||||
}
|
||||
@@ -0,0 +1,67 @@
|
||||
#pragma once
|
||||
|
||||
#include <common/storage/mirroring/SyncCandidateStore.h>
|
||||
#include <common/threading/PThread.h>
|
||||
#include <common/storage/StorageErrors.h>
|
||||
#include <common/nodes/Node.h>
|
||||
#include <app/App.h>
|
||||
#include <components/buddyresyncer/SyncCandidate.h>
|
||||
|
||||
#include "SyncSlaveBase.h"
|
||||
|
||||
class DirEntry;
|
||||
|
||||
class BuddyResyncerBulkSyncSlave : public SyncSlaveBase
|
||||
{
|
||||
friend class BuddyResyncer;
|
||||
friend class BuddyResyncJob;
|
||||
|
||||
public:
|
||||
BuddyResyncerBulkSyncSlave(BuddyResyncJob& parentJob, MetaSyncCandidateStore* syncCandidates, uint8_t slaveID,
|
||||
const NumNodeID& buddyNodeID);
|
||||
|
||||
struct Stats
|
||||
{
|
||||
uint64_t dirsSynced;
|
||||
uint64_t filesSynced;
|
||||
uint64_t dirErrors;
|
||||
uint64_t fileErrors;
|
||||
};
|
||||
|
||||
Stats getStats()
|
||||
{
|
||||
return Stats{ numDirsSynced.read(), numFilesSynced.read(),
|
||||
numDirErrors.read(), numFileErrors.read() };
|
||||
}
|
||||
|
||||
private:
|
||||
MetaSyncCandidateStore* syncCandidates;
|
||||
|
||||
AtomicUInt64 numDirsSynced;
|
||||
AtomicUInt64 numFilesSynced;
|
||||
AtomicUInt64 numDirErrors;
|
||||
AtomicUInt64 numFileErrors;
|
||||
|
||||
void syncLoop();
|
||||
|
||||
FhgfsOpsErr resyncDirectory(const MetaSyncCandidateDir& root, const std::string& inodeID);
|
||||
|
||||
FhgfsOpsErr streamCandidateDir(Socket& socket, const MetaSyncCandidateDir& candidate,
|
||||
const std::string& inodeID);
|
||||
|
||||
|
||||
private:
|
||||
typedef std::tuple<
|
||||
BuddyResyncerBulkSyncSlave&,
|
||||
const MetaSyncCandidateDir&,
|
||||
const std::string&> StreamCandidateArgs;
|
||||
|
||||
static FhgfsOpsErr streamCandidateDir(Socket* socket, void* context)
|
||||
{
|
||||
using std::get;
|
||||
|
||||
auto& args = *(StreamCandidateArgs*) context;
|
||||
return get<0>(args).streamCandidateDir(*socket, get<1>(args), get<2>(args));
|
||||
}
|
||||
};
|
||||
|
||||
@@ -0,0 +1,134 @@
|
||||
#include <common/storage/Metadata.h>
|
||||
#include <common/app/log/LogContext.h>
|
||||
#include <common/toolkit/StringTk.h>
|
||||
#include <toolkit/BuddyCommTk.h>
|
||||
#include <program/Program.h>
|
||||
|
||||
#include "BuddyResyncerGatherSlave.h"
|
||||
|
||||
BuddyResyncerGatherSlave::BuddyResyncerGatherSlave(MetaSyncCandidateStore* syncCandidates) :
|
||||
PThread("BuddyResyncerGatherSlave"),
|
||||
isRunning(false),
|
||||
syncCandidates(syncCandidates)
|
||||
{
|
||||
metaBuddyPath = Program::getApp()->getMetaPath() + "/" CONFIG_BUDDYMIRROR_SUBDIR_NAME;
|
||||
}
|
||||
|
||||
void BuddyResyncerGatherSlave::run()
|
||||
{
|
||||
setIsRunning(true);
|
||||
|
||||
try
|
||||
{
|
||||
LOG(MIRRORING, DEBUG, "Component started");
|
||||
registerSignalHandler();
|
||||
workLoop();
|
||||
LOG(MIRRORING, DEBUG, "Component stopped");
|
||||
}
|
||||
catch (std::exception& e)
|
||||
{
|
||||
PThread::getCurrentThreadApp()->handleComponentException(e);
|
||||
}
|
||||
|
||||
setIsRunning(false);
|
||||
}
|
||||
|
||||
void BuddyResyncerGatherSlave::workLoop()
|
||||
{
|
||||
crawlDir(metaBuddyPath + "/" META_INODES_SUBDIR_NAME, MetaSyncDirType::InodesHashDir);
|
||||
crawlDir(metaBuddyPath + "/" META_DENTRIES_SUBDIR_NAME, MetaSyncDirType::DentriesHashDir);
|
||||
}
|
||||
|
||||
void BuddyResyncerGatherSlave::crawlDir(const std::string& path, const MetaSyncDirType type,
|
||||
const unsigned level)
|
||||
{
|
||||
LOG_DBG(MIRRORING, DEBUG, "Entering hash dir.", level, path);
|
||||
|
||||
std::unique_ptr<DIR, StorageTk::CloseDirDeleter> dirHandle(::opendir(path.c_str()));
|
||||
|
||||
if (!dirHandle)
|
||||
{
|
||||
LOG(MIRRORING, ERR, "Unable to open path", path, sysErr);
|
||||
numErrors.increase();
|
||||
return;
|
||||
}
|
||||
|
||||
while (!getSelfTerminate())
|
||||
{
|
||||
struct dirent* entry;
|
||||
|
||||
#if USE_READDIR_R
|
||||
struct dirent buffer;
|
||||
const int readRes = ::readdir_r(dirHandle.get(), &buffer, &entry);
|
||||
#else
|
||||
errno = 0;
|
||||
entry = ::readdir(dirHandle.get());
|
||||
const int readRes = entry ? 0 : errno;
|
||||
#endif
|
||||
if (readRes != 0)
|
||||
{
|
||||
LOG(MIRRORING, ERR, "Could not read dir entry.", path, sysErr(readRes));
|
||||
numErrors.increase();
|
||||
return;
|
||||
}
|
||||
|
||||
if (!entry)
|
||||
break;
|
||||
|
||||
if (strcmp(entry->d_name, ".") == 0 || strcmp(entry->d_name, "..") == 0)
|
||||
continue;
|
||||
|
||||
const std::string& candidatePath = path + "/" + entry->d_name;
|
||||
|
||||
struct stat statBuf;
|
||||
const int statRes = ::stat(candidatePath.c_str(), &statBuf);
|
||||
if (statRes)
|
||||
{
|
||||
// in a 2nd level dentry hashdir, content directories may disappear - this is not an error,
|
||||
// it was most likely caused by an rmdir issued by a user.
|
||||
if (!(errno == ENOENT && type == MetaSyncDirType::DentriesHashDir && level == 2))
|
||||
{
|
||||
LOG(MIRRORING, ERR, "Could not stat dir entry.", candidatePath, sysErr);
|
||||
numErrors.increase();
|
||||
}
|
||||
|
||||
continue;
|
||||
}
|
||||
|
||||
if (!S_ISDIR(statBuf.st_mode))
|
||||
{
|
||||
LOG(MIRRORING, ERR, "Found a non-dir where only directories are expected.", candidatePath,
|
||||
oct(statBuf.st_mode));
|
||||
numErrors.increase();
|
||||
continue;
|
||||
}
|
||||
|
||||
// layout is: (dentries|inodes)/l1/l2/...
|
||||
// -> level 0 correlates with type
|
||||
// -> level 1 is not very interesting, except for reporting
|
||||
// -> level 2 must be synced. if it is a dentry hashdir, its contents must also be crawled.
|
||||
if (level == 0)
|
||||
{
|
||||
crawlDir(candidatePath, type, level + 1);
|
||||
continue;
|
||||
}
|
||||
|
||||
if (level == 1)
|
||||
{
|
||||
LOG_DBG(MIRRORING, DEBUG, "Adding hashdir sync candidate.", candidatePath);
|
||||
addCandidate(candidatePath, type);
|
||||
|
||||
if (type == MetaSyncDirType::DentriesHashDir)
|
||||
crawlDir(candidatePath, type, level + 1);
|
||||
|
||||
continue;
|
||||
}
|
||||
|
||||
// so here we read a 2nd level dentry hashdir. crawl that too, add sync candidates for each
|
||||
// entry we find - non-directories have already been reported, and the bulk resyncer will
|
||||
// take care of the fsids directories.
|
||||
numDirsDiscovered.increase();
|
||||
LOG_DBG(MIRRORING, DEBUG, "Adding contdir sync candidate.", candidatePath);
|
||||
addCandidate(candidatePath, MetaSyncDirType::ContentDir);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,73 @@
|
||||
#pragma once
|
||||
|
||||
#include <common/app/log/LogContext.h>
|
||||
#include <common/threading/PThread.h>
|
||||
#include <components/buddyresyncer/SyncCandidate.h>
|
||||
|
||||
#include <mutex>
|
||||
|
||||
class BuddyResyncerGatherSlave : public PThread
|
||||
{
|
||||
// Grant access to internal mutex
|
||||
friend class BuddyResyncer;
|
||||
friend class BuddyResyncJob;
|
||||
|
||||
public:
|
||||
BuddyResyncerGatherSlave(MetaSyncCandidateStore* syncCandidates);
|
||||
|
||||
void workLoop();
|
||||
|
||||
private:
|
||||
Mutex stateMutex;
|
||||
Condition isRunningChangeCond;
|
||||
|
||||
AtomicUInt64 numDirsDiscovered;
|
||||
AtomicUInt64 numErrors;
|
||||
|
||||
std::string metaBuddyPath;
|
||||
|
||||
bool isRunning;
|
||||
|
||||
MetaSyncCandidateStore* syncCandidates;
|
||||
|
||||
virtual void run();
|
||||
|
||||
void crawlDir(const std::string& path, const MetaSyncDirType type, const unsigned level = 0);
|
||||
|
||||
public:
|
||||
bool getIsRunning()
|
||||
{
|
||||
std::lock_guard<Mutex> lock(stateMutex);
|
||||
return this->isRunning;
|
||||
}
|
||||
|
||||
struct Stats
|
||||
{
|
||||
uint64_t dirsDiscovered;
|
||||
uint64_t errors;
|
||||
};
|
||||
|
||||
Stats getStats()
|
||||
{
|
||||
return Stats{ numDirsDiscovered.read(), numErrors.read() };
|
||||
}
|
||||
|
||||
|
||||
private:
|
||||
void setIsRunning(const bool isRunning)
|
||||
{
|
||||
std::lock_guard<Mutex> lock(stateMutex);
|
||||
this->isRunning = isRunning;
|
||||
isRunningChangeCond.broadcast();
|
||||
}
|
||||
|
||||
void addCandidate(const std::string& path, const MetaSyncDirType type)
|
||||
{
|
||||
const std::string& relPath = path.substr(metaBuddyPath.size() + 1);
|
||||
syncCandidates->add(MetaSyncCandidateDir(relPath, type), this);
|
||||
}
|
||||
};
|
||||
|
||||
typedef std::vector<BuddyResyncerGatherSlave*> BuddyResyncerGatherSlaveVec;
|
||||
typedef BuddyResyncerGatherSlaveVec::iterator BuddyResyncerGatherSlaveVecIter;
|
||||
|
||||
@@ -0,0 +1,142 @@
|
||||
#include "BuddyResyncerModSyncSlave.h"
|
||||
|
||||
#include <common/net/message/storage/mirroring/ResyncRawInodesRespMsg.h>
|
||||
#include <common/toolkit/StringTk.h>
|
||||
#include <common/toolkit/MessagingTk.h>
|
||||
#include <common/toolkit/DebugVariable.h>
|
||||
#include <common/Common.h>
|
||||
|
||||
#include <net/message/storage/mirroring/ResyncRawInodesMsgEx.h>
|
||||
#include <net/msghelpers/MsgHelperXAttr.h>
|
||||
#include <program/Program.h>
|
||||
#include <toolkit/XAttrTk.h>
|
||||
|
||||
BuddyResyncerModSyncSlave::BuddyResyncerModSyncSlave(BuddyResyncJob& parentJob,
|
||||
MetaSyncCandidateStore* syncCandidates, uint8_t slaveID, const NumNodeID& buddyNodeID) :
|
||||
SyncSlaveBase("BuddyResyncerModSyncSlave_" + StringTk::uintToStr(slaveID), parentJob,
|
||||
buddyNodeID),
|
||||
syncCandidates(syncCandidates)
|
||||
{
|
||||
}
|
||||
|
||||
void BuddyResyncerModSyncSlave::syncLoop()
|
||||
{
|
||||
while (!getSelfTerminateNotIdle())
|
||||
{
|
||||
if (syncCandidates->waitForFiles(this))
|
||||
resyncAt(Path(), false, streamCandidates, this);
|
||||
else if (getOnlyTerminateIfIdle())
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
namespace {
|
||||
struct CandidateSignaler
|
||||
{
|
||||
void operator()(MetaSyncCandidateFile* candidate) const
|
||||
{
|
||||
candidate->signal();
|
||||
}
|
||||
};
|
||||
|
||||
bool resyncElemCmp(const MetaSyncCandidateFile::Element& a, const MetaSyncCandidateFile::Element& b)
|
||||
{
|
||||
// we must sync deletions before updates and inodes before everything else:
|
||||
//
|
||||
// deletions may fail on the secondary, so they *can* be synced first to begin with.
|
||||
// any item that is deleted and then recreated with an update must be deleted first.
|
||||
// we also guarantee that no item is created and deleted in the same changeset.
|
||||
//
|
||||
// inodes must be synced before dentries because the dentries may link to inodes in the same
|
||||
// changeset - and if the secondary does not have the appropriate inode yet, the changeset
|
||||
// must create it.
|
||||
if (a.isDeletion && !b.isDeletion)
|
||||
return true;
|
||||
|
||||
if (a.type == MetaSyncFileType::Inode && b.type != MetaSyncFileType::Inode)
|
||||
return true;
|
||||
|
||||
return std::make_pair(int(a.type), a.path) < std::make_pair(int(b.type), b.path);
|
||||
}
|
||||
}
|
||||
|
||||
FhgfsOpsErr BuddyResyncerModSyncSlave::streamCandidates(Socket& socket)
|
||||
{
|
||||
DEBUG_ENV_VAR(unsigned, DEBUG_FAIL_MODSYNC, 0, "BEEGFS_DEBUG_FAIL_MODSYNC");
|
||||
|
||||
while (!getSelfTerminateNotIdle())
|
||||
{
|
||||
if (syncCandidates->isFilesEmpty())
|
||||
break;
|
||||
|
||||
MetaSyncCandidateFile candidate;
|
||||
syncCandidates->fetch(candidate, this);
|
||||
|
||||
// signal the candidate at the end of this loop iteration.
|
||||
// do it like this because we have a few exit points and also have exceptions to take into
|
||||
// account.
|
||||
std::unique_ptr<MetaSyncCandidateFile, CandidateSignaler> signaler(&candidate);
|
||||
|
||||
auto resyncElems = candidate.releaseElements();
|
||||
|
||||
std::sort(resyncElems.begin(), resyncElems.end(), resyncElemCmp);
|
||||
|
||||
for (auto it = resyncElems.begin(); it != resyncElems.end(); ++it)
|
||||
{
|
||||
const auto& element = *it;
|
||||
|
||||
// element.path is relative to the meta root, so we have to chop off the buddymir/ prefix
|
||||
const Path itemPath(element.path.substr(strlen(META_BUDDYMIRROR_SUBDIR_NAME) + 1));
|
||||
|
||||
FhgfsOpsErr resyncRes;
|
||||
|
||||
LOG_DBG(MIRRORING, DEBUG, "Syncing one modification.", element.path, element.isDeletion,
|
||||
int(element.type));
|
||||
|
||||
switch (element.type)
|
||||
{
|
||||
case MetaSyncFileType::Dentry:
|
||||
resyncRes = element.isDeletion
|
||||
? deleteDentry(socket, itemPath.dirname(), itemPath.back())
|
||||
: streamDentry(socket, itemPath.dirname(), itemPath.back());
|
||||
break;
|
||||
|
||||
case MetaSyncFileType::Directory:
|
||||
case MetaSyncFileType::Inode:
|
||||
resyncRes = element.isDeletion
|
||||
? deleteInode(socket, itemPath, element.type == MetaSyncFileType::Directory)
|
||||
: streamInode(socket, itemPath, element.type == MetaSyncFileType::Directory);
|
||||
break;
|
||||
|
||||
default:
|
||||
LOG(MIRRORING, ERR, "this should never happen");
|
||||
return FhgfsOpsErr_INTERNAL;
|
||||
}
|
||||
|
||||
if (resyncRes != FhgfsOpsErr_SUCCESS || DEBUG_FAIL_MODSYNC)
|
||||
{
|
||||
LOG(MIRRORING, ERR, "Modification resync failed.", element.path, element.isDeletion,
|
||||
resyncRes);
|
||||
numErrors.increase();
|
||||
|
||||
// Since this error prevents the resync from reaching a GOOD state on the secondary,
|
||||
// we abort here.
|
||||
parentJob->abort(true);
|
||||
|
||||
// terminate the current stream, start a new one if necessary. we could (in theory)
|
||||
// reuse the current stream, but terminating a stream that has seen an error is simpler
|
||||
// to handle than keeping it open. also, bulk resync would like "fail on error"
|
||||
// semantics very much.
|
||||
sendResyncPacket(socket, std::tuple<>());
|
||||
return FhgfsOpsErr_SUCCESS;
|
||||
}
|
||||
else
|
||||
{
|
||||
numObjectsSynced.increase();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
sendResyncPacket(socket, std::tuple<>());
|
||||
return FhgfsOpsErr_SUCCESS;
|
||||
}
|
||||
@@ -0,0 +1,50 @@
|
||||
#pragma once
|
||||
|
||||
#include <common/storage/mirroring/SyncCandidateStore.h>
|
||||
#include <common/threading/PThread.h>
|
||||
#include <common/storage/StorageErrors.h>
|
||||
#include <common/nodes/Node.h>
|
||||
#include <app/App.h>
|
||||
#include <components/buddyresyncer/SyncCandidate.h>
|
||||
|
||||
#include "SyncSlaveBase.h"
|
||||
|
||||
class DirEntry;
|
||||
|
||||
class BuddyResyncerModSyncSlave : public SyncSlaveBase
|
||||
{
|
||||
friend class BuddyResyncer;
|
||||
friend class BuddyResyncJob;
|
||||
|
||||
public:
|
||||
BuddyResyncerModSyncSlave(BuddyResyncJob& parentJob, MetaSyncCandidateStore* syncCandidates,
|
||||
uint8_t slaveID, const NumNodeID& buddyNodeID);
|
||||
|
||||
struct Stats
|
||||
{
|
||||
uint64_t objectsSynced;
|
||||
uint64_t errors;
|
||||
};
|
||||
|
||||
Stats getStats()
|
||||
{
|
||||
return Stats{ numObjectsSynced.read(), numErrors.read() };
|
||||
}
|
||||
|
||||
private:
|
||||
MetaSyncCandidateStore* syncCandidates;
|
||||
|
||||
AtomicUInt64 numObjectsSynced;
|
||||
AtomicUInt64 numErrors;
|
||||
|
||||
void syncLoop();
|
||||
|
||||
FhgfsOpsErr streamCandidates(Socket& socket);
|
||||
|
||||
private:
|
||||
static FhgfsOpsErr streamCandidates(Socket* socket, void* context)
|
||||
{
|
||||
return static_cast<BuddyResyncerModSyncSlave*>(context)->streamCandidates(*socket);
|
||||
}
|
||||
};
|
||||
|
||||
@@ -0,0 +1,59 @@
|
||||
#include "SessionStoreResyncer.h"
|
||||
|
||||
#include <common/toolkit/MessagingTk.h>
|
||||
#include <common/net/message/storage/mirroring/ResyncSessionStoreMsg.h>
|
||||
#include <common/net/message/storage/mirroring/ResyncSessionStoreRespMsg.h>
|
||||
#include <common/toolkit/StringTk.h>
|
||||
#include <program/Program.h>
|
||||
#include <app/App.h>
|
||||
|
||||
#include <boost/scoped_array.hpp>
|
||||
|
||||
SessionStoreResyncer::SessionStoreResyncer(const NumNodeID& buddyNodeID)
|
||||
: buddyNodeID(buddyNodeID) {}
|
||||
|
||||
void SessionStoreResyncer::doSync()
|
||||
{
|
||||
App* app = Program::getApp();
|
||||
SessionStore* sessions = app->getMirroredSessions();
|
||||
NodeStoreServers* metaNodes = app->getMetaNodes();
|
||||
const uint64_t numSessions = sessions->getSize();
|
||||
|
||||
numSessionsToSync.set(numSessions);
|
||||
|
||||
// Serialize sessions store into buffer
|
||||
std::pair<std::unique_ptr<char[]>, size_t> sessionStoreSerBuf = sessions->serializeToBuf();
|
||||
|
||||
if (sessionStoreSerBuf.second == 0)
|
||||
{
|
||||
// Serialization failed.
|
||||
errors.set(1);
|
||||
return;
|
||||
}
|
||||
|
||||
LOG(MIRRORING, DEBUG, "Serialized session store", ("size", sessionStoreSerBuf.second));
|
||||
|
||||
ResyncSessionStoreMsg msg(sessionStoreSerBuf.first.get(), sessionStoreSerBuf.second);
|
||||
RequestResponseArgs rrArgs(NULL, &msg, NETMSGTYPE_ResyncSessionStoreResp);
|
||||
RequestResponseNode rrNode(buddyNodeID, metaNodes);
|
||||
msg.registerStreamoutHook(rrArgs);
|
||||
|
||||
FhgfsOpsErr requestRes = MessagingTk::requestResponseNode(&rrNode, &rrArgs);
|
||||
|
||||
if (requestRes != FhgfsOpsErr_SUCCESS)
|
||||
{
|
||||
errors.set(1);
|
||||
LOG(MIRRORING, ERR, "Request failed.", requestRes);
|
||||
return;
|
||||
}
|
||||
|
||||
ResyncSessionStoreRespMsg* resp = (ResyncSessionStoreRespMsg*)rrArgs.outRespMsg.get();
|
||||
FhgfsOpsErr retVal = resp->getResult();
|
||||
|
||||
LOG(MIRRORING, DEBUG, "ResyncSessionStoreRespMsg", retVal);
|
||||
|
||||
if (retVal != FhgfsOpsErr_SUCCESS)
|
||||
errors.set(1);
|
||||
else
|
||||
numSessionsSynced.set(numSessions);
|
||||
}
|
||||
35
meta/source/components/buddyresyncer/SessionStoreResyncer.h
Normal file
35
meta/source/components/buddyresyncer/SessionStoreResyncer.h
Normal file
@@ -0,0 +1,35 @@
|
||||
#pragma once
|
||||
|
||||
#include <common/nodes/Node.h>
|
||||
#include <common/threading/PThread.h>
|
||||
|
||||
class SessionStoreResyncer
|
||||
{
|
||||
friend class BuddyResyncer;
|
||||
friend class BuddyResyncJob;
|
||||
|
||||
public:
|
||||
SessionStoreResyncer(const NumNodeID& buddyNodeID);
|
||||
|
||||
struct Stats
|
||||
{
|
||||
uint64_t sessionsToSync;
|
||||
uint64_t sessionsSynced;
|
||||
bool errors;
|
||||
};
|
||||
|
||||
Stats getStats()
|
||||
{
|
||||
return Stats{ numSessionsToSync.read(), numSessionsSynced.read(), errors.read() != 0 };
|
||||
}
|
||||
|
||||
private:
|
||||
NumNodeID buddyNodeID;
|
||||
|
||||
AtomicUInt64 numSessionsToSync;
|
||||
AtomicUInt64 numSessionsSynced;
|
||||
AtomicSizeT errors; // 0 / 1
|
||||
|
||||
void doSync();
|
||||
};
|
||||
|
||||
113
meta/source/components/buddyresyncer/SyncCandidate.h
Normal file
113
meta/source/components/buddyresyncer/SyncCandidate.h
Normal file
@@ -0,0 +1,113 @@
|
||||
#pragma once
|
||||
|
||||
#include <common/toolkit/serialization/Serialization.h>
|
||||
#include <common/storage/mirroring/SyncCandidateStore.h>
|
||||
#include <common/threading/Barrier.h>
|
||||
|
||||
#include <string>
|
||||
|
||||
enum class MetaSyncDirType
|
||||
{
|
||||
InodesHashDir,
|
||||
DentriesHashDir,
|
||||
ContentDir,
|
||||
};
|
||||
GCC_COMPAT_ENUM_CLASS_OPEQNEQ(MetaSyncDirType)
|
||||
|
||||
class MetaSyncCandidateDir
|
||||
{
|
||||
public:
|
||||
MetaSyncCandidateDir(const std::string& relativePath, MetaSyncDirType type):
|
||||
relPath(relativePath), type(type)
|
||||
{}
|
||||
|
||||
MetaSyncCandidateDir() = default;
|
||||
|
||||
private:
|
||||
std::string relPath;
|
||||
MetaSyncDirType type;
|
||||
|
||||
public:
|
||||
const std::string& getRelativePath() const { return relPath; }
|
||||
MetaSyncDirType getType() const { return type; }
|
||||
};
|
||||
|
||||
enum class MetaSyncFileType
|
||||
{
|
||||
Inode,
|
||||
Dentry,
|
||||
Directory,
|
||||
};
|
||||
GCC_COMPAT_ENUM_CLASS_OPEQNEQ(MetaSyncFileType)
|
||||
|
||||
template<>
|
||||
struct SerializeAs<MetaSyncFileType> {
|
||||
typedef uint8_t type;
|
||||
};
|
||||
|
||||
class MetaSyncCandidateFile
|
||||
{
|
||||
public:
|
||||
struct Element
|
||||
{
|
||||
std::string path;
|
||||
MetaSyncFileType type;
|
||||
bool isDeletion;
|
||||
};
|
||||
|
||||
MetaSyncCandidateFile(): barrier(nullptr) {}
|
||||
|
||||
MetaSyncCandidateFile(MetaSyncCandidateFile&& src):
|
||||
barrier(nullptr)
|
||||
{
|
||||
swap(src);
|
||||
}
|
||||
|
||||
MetaSyncCandidateFile& operator=(MetaSyncCandidateFile&& other)
|
||||
{
|
||||
MetaSyncCandidateFile(std::move(other)).swap(*this);
|
||||
return *this;
|
||||
}
|
||||
|
||||
void swap(MetaSyncCandidateFile& other)
|
||||
{
|
||||
paths.swap(other.paths);
|
||||
std::swap(barrier, other.barrier);
|
||||
}
|
||||
|
||||
void signal()
|
||||
{
|
||||
barrier->wait();
|
||||
}
|
||||
|
||||
friend void swap(MetaSyncCandidateFile& a, MetaSyncCandidateFile& b)
|
||||
{
|
||||
a.swap(b);
|
||||
}
|
||||
|
||||
private:
|
||||
std::vector<Element> paths;
|
||||
Barrier* barrier;
|
||||
|
||||
public:
|
||||
const std::vector<Element>& getElements() const { return paths; }
|
||||
std::vector<Element> releaseElements() { return std::move(paths); }
|
||||
|
||||
void addModification(std::string path, MetaSyncFileType type)
|
||||
{
|
||||
paths.push_back(Element{std::move(path), type, false});
|
||||
}
|
||||
|
||||
void addDeletion(std::string path, MetaSyncFileType type)
|
||||
{
|
||||
paths.push_back(Element{std::move(path), type, true});
|
||||
}
|
||||
|
||||
void prepareSignal(Barrier& barrier)
|
||||
{
|
||||
this->barrier = &barrier;
|
||||
}
|
||||
};
|
||||
|
||||
typedef SyncCandidateStore<MetaSyncCandidateDir, MetaSyncCandidateFile> MetaSyncCandidateStore;
|
||||
|
||||
249
meta/source/components/buddyresyncer/SyncSlaveBase.cpp
Normal file
249
meta/source/components/buddyresyncer/SyncSlaveBase.cpp
Normal file
@@ -0,0 +1,249 @@
|
||||
#include "SyncSlaveBase.h"
|
||||
|
||||
#include <common/net/message/storage/mirroring/ResyncRawInodesRespMsg.h>
|
||||
#include <net/message/storage/mirroring/ResyncRawInodesMsgEx.h>
|
||||
#include <net/msghelpers/MsgHelperXAttr.h>
|
||||
#include <program/Program.h>
|
||||
#include <toolkit/XAttrTk.h>
|
||||
|
||||
void SyncSlaveBase::run()
|
||||
{
|
||||
setIsRunning(true);
|
||||
|
||||
try
|
||||
{
|
||||
LOG(MIRRORING, DEBUG, "Component started");
|
||||
|
||||
registerSignalHandler();
|
||||
|
||||
syncLoop();
|
||||
|
||||
LOG(MIRRORING, DEBUG, "Component stopped");
|
||||
}
|
||||
catch (std::exception& e)
|
||||
{
|
||||
PThread::getCurrentThreadApp()->handleComponentException(e);
|
||||
}
|
||||
|
||||
setIsRunning(false);
|
||||
}
|
||||
|
||||
FhgfsOpsErr SyncSlaveBase::receiveAck(Socket& socket)
|
||||
{
|
||||
auto resp = MessagingTk::recvMsgBuf(socket);
|
||||
if (resp.empty())
|
||||
return FhgfsOpsErr_INTERNAL;
|
||||
|
||||
const auto respMsg = PThread::getCurrentThreadApp()->getNetMessageFactory()->createFromBuf(
|
||||
std::move(resp));
|
||||
if (respMsg->getMsgType() != NETMSGTYPE_ResyncRawInodesResp)
|
||||
return FhgfsOpsErr_COMMUNICATION;
|
||||
|
||||
return static_cast<ResyncRawInodesRespMsg&>(*respMsg).getResult();
|
||||
}
|
||||
|
||||
FhgfsOpsErr SyncSlaveBase::resyncAt(const Path& basePath, bool wholeDirectory,
|
||||
FhgfsOpsErr (*streamFn)(Socket*, void*), void* context)
|
||||
{
|
||||
const bool sendXAttrs = Program::getApp()->getConfig()->getStoreClientXAttrs();
|
||||
|
||||
this->basePath = META_BUDDYMIRROR_SUBDIR_NAME / basePath;
|
||||
|
||||
ResyncRawInodesMsgEx msg(basePath, sendXAttrs, wholeDirectory);
|
||||
|
||||
RequestResponseNode rrNode(buddyNodeID, Program::getApp()->getMetaNodes());
|
||||
RequestResponseArgs rrArgs(nullptr, &msg, NETMSGTYPE_ResyncRawInodesResp,
|
||||
streamFn, context);
|
||||
|
||||
// resync processing may take a very long time for each step, eg if a very large directory must
|
||||
// be cleaned out on the secondary. do not use timeouts for resync communication right now.
|
||||
|
||||
rrArgs.minTimeoutMS = -1;
|
||||
|
||||
const auto commRes = MessagingTk::requestResponseNode(&rrNode, &rrArgs);
|
||||
|
||||
if (commRes != FhgfsOpsErr_SUCCESS)
|
||||
{
|
||||
LOG(MIRRORING, ERR, "Error during communication with secondary.", commRes);
|
||||
return commRes;
|
||||
}
|
||||
|
||||
const auto resyncRes = static_cast<ResyncRawInodesRespMsg&>(*rrArgs.outRespMsg).getResult();
|
||||
|
||||
if (resyncRes != FhgfsOpsErr_SUCCESS)
|
||||
LOG(MIRRORING, ERR, "Error while resyncing directory.", basePath, resyncRes);
|
||||
|
||||
return resyncRes;
|
||||
}
|
||||
|
||||
FhgfsOpsErr SyncSlaveBase::streamDentry(Socket& socket, const Path& contDirRelPath,
|
||||
const std::string& name)
|
||||
{
|
||||
std::unique_ptr<DirEntry> dentry(
|
||||
DirEntry::createFromFile((basePath / contDirRelPath).str(), name));
|
||||
if (!dentry)
|
||||
{
|
||||
LOG(MIRRORING, ERR, "Could not open dentry.", basePath, contDirRelPath, name);
|
||||
return FhgfsOpsErr_INTERNAL;
|
||||
}
|
||||
|
||||
if (dentry->getIsInodeInlined())
|
||||
{
|
||||
auto err = sendResyncPacket(socket, LinkDentryInfo(
|
||||
MetaSyncFileType::Dentry,
|
||||
(contDirRelPath / name).str(),
|
||||
true,
|
||||
dentry->getID(),
|
||||
false));
|
||||
if (err != FhgfsOpsErr_SUCCESS)
|
||||
return err;
|
||||
|
||||
return receiveAck(socket);
|
||||
}
|
||||
|
||||
std::vector<char> dentryContent;
|
||||
|
||||
{
|
||||
Serializer ser;
|
||||
dentry->serializeDentry(ser);
|
||||
dentryContent.resize(ser.size());
|
||||
|
||||
ser = Serializer(&dentryContent[0], dentryContent.size());
|
||||
dentry->serializeDentry(ser);
|
||||
if (!ser.good())
|
||||
{
|
||||
LOG(MIRRORING, ERR, "Could not serialize dentry for secondary.");
|
||||
return FhgfsOpsErr_INTERNAL;
|
||||
}
|
||||
}
|
||||
|
||||
const FhgfsOpsErr sendRes = sendResyncPacket(socket, FullDentryInfo(
|
||||
MetaSyncFileType::Dentry,
|
||||
(contDirRelPath / name).str(),
|
||||
false,
|
||||
dentryContent,
|
||||
false));
|
||||
if (sendRes != FhgfsOpsErr_SUCCESS)
|
||||
return sendRes;
|
||||
|
||||
return receiveAck(socket);
|
||||
}
|
||||
|
||||
FhgfsOpsErr SyncSlaveBase::streamInode(Socket& socket, const Path& inodeRelPath,
|
||||
const bool isDirectory)
|
||||
{
|
||||
const Path fullPath(basePath / inodeRelPath);
|
||||
|
||||
MetaStore& store = *Program::getApp()->getMetaStore();
|
||||
|
||||
// Map to store attribute name and its data
|
||||
std::map<std::string, std::vector<char>> contents;
|
||||
|
||||
if (!isDirectory)
|
||||
{
|
||||
std::vector<char> attrData;
|
||||
FhgfsOpsErr readRes;
|
||||
|
||||
// Helper function to read and store attribute data in map
|
||||
auto readAndStoreMetaAttribute = [&](const std::string& attrName)
|
||||
{
|
||||
attrData.clear();
|
||||
readRes = store.getRawMetadata(fullPath, attrName.c_str(), attrData);
|
||||
if (readRes != FhgfsOpsErr_SUCCESS)
|
||||
return false;
|
||||
contents.insert(std::make_pair(attrName, std::move(attrData)));
|
||||
return true;
|
||||
};
|
||||
|
||||
// Handle META_XATTR_NAME ("user.fhgfs") separately because it can be stored as either
|
||||
// file contents or an extended attribute, depending on the 'storeUseExtendedAttribs'
|
||||
// configuration setting in the meta config. In contrast, all other metadata-specific
|
||||
// attributes are strictly stored as extended attributes and do not have the option to
|
||||
// be stored as file contents.
|
||||
if (!readAndStoreMetaAttribute(META_XATTR_NAME))
|
||||
return readRes;
|
||||
|
||||
// Now handle all remaining metadata attributes
|
||||
std::pair<FhgfsOpsErr, std::vector<std::string>> listXAttrs = XAttrTk::listXAttrs(fullPath.str());
|
||||
if (listXAttrs.first != FhgfsOpsErr_SUCCESS)
|
||||
return listXAttrs.first;
|
||||
|
||||
for (auto const& attrName : listXAttrs.second)
|
||||
{
|
||||
// Process all metadata-specific attributes except META_XATTR_NAME (already handled above)
|
||||
// This approach ensures we only process attribute(s) that:
|
||||
// 1. Exist on the inode.
|
||||
// 2. Are listed in METADATA_XATTR_NAME_LIST, our collection of known metadata attributes.
|
||||
// 3. Is not META_XATTR_NAME, to prevent duplicate processing.
|
||||
if (std::find(METADATA_XATTR_NAME_LIST.begin(), METADATA_XATTR_NAME_LIST.end(), attrName)
|
||||
!= METADATA_XATTR_NAME_LIST.end() && (attrName != META_XATTR_NAME))
|
||||
{
|
||||
if (!readAndStoreMetaAttribute(attrName))
|
||||
return readRes;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
const FhgfsOpsErr sendRes = sendResyncPacket(socket, InodeInfo(
|
||||
isDirectory
|
||||
? MetaSyncFileType::Directory
|
||||
: MetaSyncFileType::Inode,
|
||||
inodeRelPath.str(),
|
||||
contents,
|
||||
false));
|
||||
if (sendRes != FhgfsOpsErr_SUCCESS)
|
||||
return sendRes;
|
||||
|
||||
if (Program::getApp()->getConfig()->getStoreClientXAttrs())
|
||||
{
|
||||
auto xattrs = XAttrTk::listUserXAttrs(fullPath.str());
|
||||
if (xattrs.first != FhgfsOpsErr_SUCCESS)
|
||||
{
|
||||
LOG(MIRRORING, ERR, "Could not list resync candidate xattrs.", fullPath, ("error", xattrs.first));
|
||||
xattrs.second.clear();
|
||||
return FhgfsOpsErr_INTERNAL;
|
||||
}
|
||||
|
||||
MsgHelperXAttr::StreamXAttrState state(fullPath.str(), std::move(xattrs.second));
|
||||
|
||||
const FhgfsOpsErr xattrRes = MsgHelperXAttr::StreamXAttrState::streamXattrFn(&socket, &state);
|
||||
if (xattrRes != FhgfsOpsErr_SUCCESS)
|
||||
{
|
||||
LOG(MIRRORING, ERR, "Error while sending xattrs to secondary.", fullPath, xattrRes);
|
||||
return FhgfsOpsErr_INTERNAL;
|
||||
}
|
||||
}
|
||||
|
||||
return receiveAck(socket);
|
||||
}
|
||||
|
||||
FhgfsOpsErr SyncSlaveBase::deleteDentry(Socket& socket, const Path& contDirRelPath,
|
||||
const std::string& name)
|
||||
{
|
||||
auto err = sendResyncPacket(socket, LinkDentryInfo(
|
||||
MetaSyncFileType::Dentry,
|
||||
(contDirRelPath / name).str(),
|
||||
true,
|
||||
{},
|
||||
true));
|
||||
if (err != FhgfsOpsErr_SUCCESS)
|
||||
return err;
|
||||
|
||||
return receiveAck(socket);
|
||||
}
|
||||
|
||||
FhgfsOpsErr SyncSlaveBase::deleteInode(Socket& socket, const Path& inodeRelPath,
|
||||
const bool isDirectory)
|
||||
{
|
||||
auto err = sendResyncPacket(socket, InodeInfo(
|
||||
isDirectory
|
||||
? MetaSyncFileType::Directory
|
||||
: MetaSyncFileType::Inode,
|
||||
inodeRelPath.str(),
|
||||
{},
|
||||
true));
|
||||
if (err != FhgfsOpsErr_SUCCESS)
|
||||
return err;
|
||||
|
||||
return receiveAck(socket);
|
||||
}
|
||||
129
meta/source/components/buddyresyncer/SyncSlaveBase.h
Normal file
129
meta/source/components/buddyresyncer/SyncSlaveBase.h
Normal file
@@ -0,0 +1,129 @@
|
||||
#pragma once
|
||||
|
||||
#include <common/net/sock/Socket.h>
|
||||
#include <common/storage/StorageErrors.h>
|
||||
#include <common/threading/PThread.h>
|
||||
#include <app/App.h>
|
||||
|
||||
class DirEntry;
|
||||
|
||||
class SyncSlaveBase : public PThread
|
||||
{
|
||||
public:
|
||||
bool getIsRunning()
|
||||
{
|
||||
std::lock_guard<Mutex> lock(stateMutex);
|
||||
return this->isRunning;
|
||||
}
|
||||
|
||||
void setOnlyTerminateIfIdle(bool value)
|
||||
{
|
||||
onlyTerminateIfIdle.set(value);
|
||||
}
|
||||
|
||||
bool getOnlyTerminateIfIdle()
|
||||
{
|
||||
return onlyTerminateIfIdle.read();
|
||||
}
|
||||
|
||||
protected:
|
||||
BuddyResyncJob* parentJob;
|
||||
|
||||
NumNodeID buddyNodeID;
|
||||
|
||||
Mutex stateMutex;
|
||||
Condition isRunningChangeCond;
|
||||
|
||||
AtomicSizeT onlyTerminateIfIdle;
|
||||
|
||||
bool isRunning;
|
||||
|
||||
Path basePath;
|
||||
|
||||
SyncSlaveBase(const std::string& threadName, BuddyResyncJob& parentJob,
|
||||
const NumNodeID buddyNodeID):
|
||||
PThread(threadName), parentJob(&parentJob), buddyNodeID(buddyNodeID), isRunning(false)
|
||||
{
|
||||
}
|
||||
|
||||
virtual void run() override;
|
||||
virtual void syncLoop() = 0;
|
||||
|
||||
FhgfsOpsErr resyncAt(const Path& basePath, bool wholeDirectory,
|
||||
FhgfsOpsErr (*streamFn)(Socket*, void*), void* context);
|
||||
|
||||
FhgfsOpsErr streamDentry(Socket& socket, const Path& contDirRelPath, const std::string& name);
|
||||
FhgfsOpsErr streamInode(Socket& socket, const Path& inodeRelPath, const bool isDirectory);
|
||||
|
||||
FhgfsOpsErr deleteDentry(Socket& socket, const Path& contDirRelPath, const std::string& name);
|
||||
FhgfsOpsErr deleteInode(Socket& socket, const Path& inodeRelPath, const bool isDirectory);
|
||||
|
||||
void setIsRunning(bool isRunning)
|
||||
{
|
||||
std::lock_guard<Mutex> lock(stateMutex);
|
||||
this->isRunning = isRunning;
|
||||
isRunningChangeCond.broadcast();
|
||||
}
|
||||
|
||||
bool getSelfTerminateNotIdle()
|
||||
{
|
||||
return getSelfTerminate() && !getOnlyTerminateIfIdle();
|
||||
}
|
||||
|
||||
template<typename ValueT>
|
||||
static FhgfsOpsErr sendResyncPacket(Socket& socket, const ValueT& value)
|
||||
{
|
||||
Serializer ser;
|
||||
ser % value;
|
||||
|
||||
const unsigned packetSize = ser.size();
|
||||
const unsigned totalSize = packetSize + sizeof(uint32_t);
|
||||
|
||||
const std::tuple<uint32_t, const ValueT&> packet(packetSize, value);
|
||||
|
||||
std::unique_ptr<char[]> buffer(new (std::nothrow) char[totalSize]);
|
||||
if (!buffer)
|
||||
{
|
||||
LOG(MIRRORING, ERR, "Could not allocate memory for resync packet.");
|
||||
return FhgfsOpsErr_OUTOFMEM;
|
||||
}
|
||||
|
||||
ser = {buffer.get(), totalSize};
|
||||
ser % packet;
|
||||
if (!ser.good())
|
||||
{
|
||||
LOG(MIRRORING, ERR, "Serialization of resync packet failed.");
|
||||
return FhgfsOpsErr_INTERNAL;
|
||||
}
|
||||
|
||||
socket.send(buffer.get(), totalSize, 0);
|
||||
return FhgfsOpsErr_SUCCESS;
|
||||
}
|
||||
|
||||
static FhgfsOpsErr receiveAck(Socket& socket);
|
||||
|
||||
private:
|
||||
typedef std::tuple<
|
||||
MetaSyncFileType,
|
||||
const std::string&, // relative path
|
||||
bool, // is hardlink?
|
||||
const std::string&, // link target entry id
|
||||
bool // is deletion?
|
||||
> LinkDentryInfo;
|
||||
|
||||
typedef std::tuple<
|
||||
MetaSyncFileType,
|
||||
const std::string&, // relative path
|
||||
bool, // is hardlink?
|
||||
const std::vector<char>&, // dentry raw content
|
||||
bool // is deletion?
|
||||
> FullDentryInfo;
|
||||
|
||||
typedef std::tuple<
|
||||
MetaSyncFileType,
|
||||
const std::string&, // relative path
|
||||
std::map<std::string, std::vector<char>>, // metadata specific attribute's raw contents
|
||||
bool // is deletion?
|
||||
> InodeInfo;
|
||||
};
|
||||
|
||||
Reference in New Issue
Block a user