1
0
mirror of https://github.com/mariadb-corporation/mariadb-columnstore-engine.git synced 2025-07-30 19:23:07 +03:00

chore(codestyle): mark virtual methods as override

This commit is contained in:
Aleksei Antipovskii
2024-09-12 12:27:02 +02:00
committed by Leonid Fedorov
parent 6001db44ab
commit 5556d818f8
303 changed files with 4091 additions and 4894 deletions

View File

@ -18,7 +18,7 @@
#include "AppendTask.h"
#include "messageFormat.h"
#include "SMLogging.h"
#include <errno.h>
#include <cerrno>
using namespace std;
@ -50,7 +50,7 @@ bool AppendTask::run()
success = read(cmdbuf, sizeof(append_cmd));
check_error("AppendTask read", false);
append_cmd* cmd = (append_cmd*)cmdbuf;
auto* cmd = (append_cmd*)cmdbuf;
if (cmd->flen > 1023 - sizeof(*cmd))
{

View File

@ -24,13 +24,11 @@ namespace storagemanager
class AppendTask : public PosixTask
{
public:
AppendTask() = delete;
AppendTask(int sock, uint length);
virtual ~AppendTask();
~AppendTask() override;
bool run();
private:
AppendTask();
bool run() override;
};
} // namespace storagemanager

View File

@ -46,7 +46,7 @@ class Cache : public boost::noncopyable, public ConfigListener
{
public:
static Cache* get();
virtual ~Cache();
~Cache() override;
// reading fcns
// read() marks objects to be read s.t. they do not get flushed.
@ -98,7 +98,7 @@ class Cache : public boost::noncopyable, public ConfigListener
void reset();
void validateCacheSize();
virtual void configListener() override;
void configListener() override;
private:
Cache();

View File

@ -26,11 +26,12 @@ namespace storagemanager
class CloudStorage
{
public:
virtual ~CloudStorage(){};
virtual ~CloudStorage() = default;
/* These behave like syscalls. return code -1 means an error, and errno is set */
virtual int getObject(const std::string& sourceKey, const std::string& destFile, size_t* size = NULL) = 0;
virtual int getObject(const std::string& sourceKey, const std::string& destFile,
size_t* size = nullptr) = 0;
virtual int getObject(const std::string& sourceKey, std::shared_ptr<uint8_t[]>* data,
size_t* size = NULL) = 0;
size_t* size = nullptr) = 0;
virtual int putObject(const std::string& sourceFile, const std::string& destKey) = 0;
virtual int putObject(const std::shared_ptr<uint8_t[]> data, size_t len, const std::string& destKey) = 0;
virtual int deleteObject(const std::string& key) = 0;

View File

@ -18,14 +18,14 @@
#pragma once
#ifndef __clang__
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wmaybe-uninitialized"
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wmaybe-uninitialized"
#endif
#include <boost/property_tree/ptree.hpp>
#ifndef __clang__
#pragma GCC diagnostic pop
#pragma GCC diagnostic pop
#endif
#include <boost/thread.hpp>
@ -41,7 +41,7 @@ namespace storagemanager
class ConfigListener
{
public:
virtual ~ConfigListener(){};
virtual ~ConfigListener() = default;
virtual void configListener() = 0;
};
@ -61,7 +61,7 @@ class Config : public boost::noncopyable
private:
Config();
Config(const std::string&);
explicit Config(const std::string&);
bool reload();
void reloadThreadFcn();

View File

@ -54,10 +54,10 @@ bool CopyTask::run()
success = read(buf, getLength());
check_error("CopyTask read", false);
copy_cmd* cmd = (copy_cmd*)buf;
auto* cmd = (copy_cmd*)buf;
string filename1(cmd->file1.filename,
cmd->file1.flen); // need to copy this in case it's not null terminated
f_name* filename2 = (f_name*)&buf[sizeof(copy_cmd) + cmd->file1.flen];
auto* filename2 = (f_name*)&buf[sizeof(copy_cmd) + cmd->file1.flen];
#ifdef SM_TRACE
logger->log(LOG_DEBUG, "copy %s to %s.", filename1.c_str(), filename2->filename);
@ -80,7 +80,7 @@ bool CopyTask::run()
return true;
}
sm_response* resp = (sm_response*)buf;
auto* resp = (sm_response*)buf;
resp->returnCode = 0;
return write(*resp, 0);
}

View File

@ -24,13 +24,11 @@ namespace storagemanager
class CopyTask : public PosixTask
{
public:
CopyTask() = delete;
CopyTask(int sock, uint length);
virtual ~CopyTask();
~CopyTask() override;
bool run();
private:
CopyTask();
bool run() override;
};
} // namespace storagemanager

View File

@ -19,9 +19,10 @@
#include "Config.h"
#include "SMLogging.h"
#include <string>
#include <errno.h>
#include <cerrno>
#include <iostream>
#include <boost/filesystem.hpp>
#include <utility>
using namespace std;
namespace bf = boost::filesystem;
@ -135,14 +136,20 @@ const bf::path& Downloader::getTmpPath() const
return tmpPath;
}
/* The helper fcns */
Downloader::Download::Download(const string& source, const bf::path& _dlPath, boost::mutex* _lock,
Downloader* _dl)
: dlPath(_dlPath), key(source), dl_errno(0), size(0), lock(_lock), finished(false), itRan(false), dl(_dl)
Downloader::Download::Download(string source, bf::path _dlPath, boost::mutex* _lock, Downloader* _dl)
: dlPath(std::move(_dlPath))
, key(std::move(source))
, dl_errno(0)
, size(0)
, lock(_lock)
, finished(false)
, itRan(false)
, dl(_dl)
{
}
Downloader::Download::Download(const string& source)
: key(source), dl_errno(0), size(0), lock(NULL), finished(false), itRan(false), dl(NULL)
Downloader::Download::Download(string source)
: key(std::move(source)), dl_errno(0), size(0), lock(nullptr), finished(false), itRan(false), dl(nullptr)
{
}
@ -154,12 +161,12 @@ Downloader::Download::~Download()
void Downloader::Download::operator()()
{
itRan = true;
CloudStorage* storage = CloudStorage::get();
CloudStorage* stor = CloudStorage::get();
// download to a tmp path
if (!bf::exists(dlPath / dl->getTmpPath()))
bf::create_directories(dlPath / dl->getTmpPath());
bf::path tmpFile = dlPath / dl->getTmpPath() / key;
int err = storage->getObject(key, tmpFile.string(), &size);
int err = stor->getObject(key, tmpFile.string(), &size);
if (err != 0)
{
dl_errno = errno;
@ -179,8 +186,10 @@ void Downloader::Download::operator()()
lock->lock();
finished = true;
for (uint i = 0; i < listeners.size(); i++)
listeners[i]->downloadFinished();
for (auto* listener : listeners)
{
listener->downloadFinished();
}
lock->unlock();
}
@ -217,7 +226,7 @@ void Downloader::configListener()
{
maxDownloads = 20;
workers.setMaxThreads(maxDownloads);
logger->log(LOG_INFO, "max_concurrent_downloads = %u",maxDownloads);
logger->log(LOG_INFO, "max_concurrent_downloads = %u", maxDownloads);
}
if (stmp.empty())
{

View File

@ -38,7 +38,7 @@ class Downloader : public ConfigListener
{
public:
Downloader();
virtual ~Downloader();
~Downloader() override;
// caller owns the memory for the strings.
// errors are reported through errnos
@ -49,7 +49,7 @@ class Downloader : public ConfigListener
void printKPIs() const;
virtual void configListener() override;
void configListener() override;
private:
uint maxDownloads;
@ -73,10 +73,10 @@ class Downloader : public ConfigListener
*/
struct Download : public ThreadPool::Job
{
Download(const std::string& source, const boost::filesystem::path& _dlPath, boost::mutex*, Downloader*);
Download(const std::string& source);
virtual ~Download();
void operator()();
Download(std::string source, boost::filesystem::path _dlPath, boost::mutex*, Downloader*);
explicit Download(std::string source);
~Download() override;
void operator()() override;
boost::filesystem::path dlPath;
const std::string key;
int dl_errno; // to propagate errors from the download job to the caller

View File

@ -18,7 +18,7 @@
#pragma once
#include <sys/types.h>
#include <stdint.h>
#include <cstdint>
#include <sys/stat.h>
#include <vector>
#include <string>
@ -58,7 +58,7 @@ class IOCoordinator : public boost::noncopyable
// The shared logic for merging a journal file with its base file.
// len should be set to the length of the data requested
std::shared_ptr<uint8_t[]> mergeJournal(const char* objectPath, const char* journalPath, off_t offset,
size_t len, size_t* sizeRead) const;
size_t len, size_t* sizeRead) const;
// this version modifies object data in memory, given the journal filename. Processes the whole object
// and whole journal file.

View File

@ -25,14 +25,13 @@ namespace storagemanager
class ListDirectoryTask : public PosixTask
{
public:
ListDirectoryTask() = delete;
ListDirectoryTask(int sock, uint length);
virtual ~ListDirectoryTask();
~ListDirectoryTask() override;
bool run();
bool run() override;
private:
ListDirectoryTask();
bool writeString(uint8_t* buf, int* offset, int size, const std::string& str);
};

View File

@ -29,18 +29,19 @@ class LocalStorage : public CloudStorage
{
public:
LocalStorage();
virtual ~LocalStorage();
~LocalStorage() override;
int getObject(const std::string& sourceKey, const std::string& destFile, size_t* size = NULL);
int getObject(const std::string& sourceKey, std::shared_ptr<uint8_t[]>* data, size_t* size = NULL);
int putObject(const std::string& sourceFile, const std::string& destKey);
int putObject(const std::shared_ptr<uint8_t[]> data, size_t len, const std::string& destKey);
int deleteObject(const std::string& key);
int copyObject(const std::string& sourceKey, const std::string& destKey);
int exists(const std::string& key, bool* out);
int getObject(const std::string& sourceKey, const std::string& destFile, size_t* size = nullptr) override;
int getObject(const std::string& sourceKey, std::shared_ptr<uint8_t[]>* data,
size_t* size = nullptr) override;
int putObject(const std::string& sourceFile, const std::string& destKey) override;
int putObject(const std::shared_ptr<uint8_t[]> data, size_t len, const std::string& destKey) override;
int deleteObject(const std::string& key) override;
int copyObject(const std::string& sourceKey, const std::string& destKey) override;
int exists(const std::string& key, bool* out) override;
const boost::filesystem::path& getPrefix() const;
void printKPIs() const;
void printKPIs() const override;
protected:
size_t bytesRead, bytesWritten;

View File

@ -22,7 +22,7 @@
#include <string>
#include <sys/types.h>
#include <sys/stat.h>
#include <stdint.h>
#include <cstdint>
#include <vector>
#include <iostream>
#include <unordered_map>
@ -33,7 +33,7 @@ namespace storagemanager
struct metadataObject
{
metadataObject();
metadataObject(uint64_t offset); // so we can search mObjects by integer
explicit metadataObject(uint64_t offset); // so we can search mObjects by integer
metadataObject(uint64_t offset, uint64_t length, const std::string& key);
uint64_t offset;
mutable uint64_t length;
@ -51,7 +51,7 @@ class MetadataFile
{
};
MetadataFile();
MetadataFile(const boost::filesystem::path& filename);
explicit MetadataFile(const boost::filesystem::path& filename);
MetadataFile(const boost::filesystem::path& path, no_create_t,
bool appendExt); // this one won't create it if it doesn't exist

View File

@ -19,8 +19,7 @@
#include "messageFormat.h"
#include "SMLogging.h"
#include <sys/stat.h>
#include <errno.h>
#include <string.h>
#include <cerrno>
using namespace std;
@ -30,9 +29,7 @@ OpenTask::OpenTask(int sock, uint len) : PosixTask(sock, len)
{
}
OpenTask::~OpenTask()
{
}
OpenTask::~OpenTask() = default;
bool OpenTask::run()
{
@ -58,12 +55,12 @@ bool OpenTask::run()
return false;
}
open_cmd* cmd = (open_cmd*)buf;
auto* cmd = (open_cmd*)buf;
#ifdef SM_TRACE
logger->log(LOG_DEBUG, "open filename %s mode %o.", cmd->filename, cmd->openmode);
#endif
sm_response* resp = (sm_response*)buf;
auto* resp = (sm_response*)buf;
int err;
try

View File

@ -24,13 +24,11 @@ namespace storagemanager
class OpenTask : public PosixTask
{
public:
OpenTask() = delete;
OpenTask(int sock, uint length);
virtual ~OpenTask();
~OpenTask() override;
bool run();
private:
OpenTask();
bool run() override;
};
} // namespace storagemanager

View File

@ -62,7 +62,7 @@ class Ownership : public boost::noncopyable
struct Monitor
{
Monitor(Ownership*);
explicit Monitor(Ownership*);
~Monitor();
boost::thread thread;
Ownership* owner;

View File

@ -17,7 +17,7 @@
#include "PingTask.h"
#include "messageFormat.h"
#include <errno.h>
#include <cerrno>
namespace storagemanager
{
@ -25,9 +25,7 @@ PingTask::PingTask(int sock, uint len) : PosixTask(sock, len)
{
}
PingTask::~PingTask()
{
}
PingTask::~PingTask() = default;
bool PingTask::run()
{
@ -49,7 +47,7 @@ bool PingTask::run()
}
// send generic success response
sm_response ret;
sm_response ret{};
ret.returnCode = 0;
success = write(ret, 0);
return success;

View File

@ -24,13 +24,11 @@ namespace storagemanager
class PingTask : public PosixTask
{
public:
PingTask() = delete;
PingTask(int sock, uint length);
virtual ~PingTask();
~PingTask() override;
bool run();
private:
PingTask();
bool run() override;
};
} // namespace storagemanager

View File

@ -18,10 +18,9 @@
#include "PosixTask.h"
#include "messageFormat.h"
#include "SMLogging.h"
#include <iostream>
#include <sys/types.h>
#include <sys/socket.h>
#include <string.h>
#include <cstring>
#define min(x, y) (x < y ? x : y)
@ -53,7 +52,7 @@ void PosixTask::handleError(const char* name, int errCode)
char buf[80];
// send an error response if possible
sm_response* resp = (sm_response*)buf;
auto* resp = (sm_response*)buf;
resp->returnCode = -1;
*((int*)resp->payload) = errCode;
bool success = write(*resp, 4);
@ -67,7 +66,7 @@ uint PosixTask::getRemainingLength()
return remainingLengthForCaller;
}
uint PosixTask::getLength()
uint PosixTask::getLength() const
{
return totalLength;
}
@ -152,7 +151,7 @@ void PosixTask::primeBuffer()
}
}
bool PosixTask::write(const uint8_t* buf, uint len)
bool PosixTask::write(const uint8_t* buf, uint len) const
{
int err;
uint count = 0;
@ -167,11 +166,11 @@ bool PosixTask::write(const uint8_t* buf, uint len)
return true;
}
bool PosixTask::write(sm_response& resp, uint payloadLength)
bool PosixTask::write(sm_response& resp, uint payloadLength) const
{
int err;
uint count = 0;
uint8_t* buf = (uint8_t*)&resp;
auto* buf = (uint8_t*)&resp;
resp.header.type = SM_MSG_START;
resp.header.flags = 0;

View File

@ -29,6 +29,7 @@ namespace storagemanager
class PosixTask
{
public:
PosixTask() = delete;
PosixTask(int sock, uint length);
virtual ~PosixTask();
@ -39,18 +40,16 @@ class PosixTask
protected:
int read(uint8_t* buf, uint length);
bool write(const std::vector<uint8_t>& buf);
bool write(sm_response& resp, uint payloadLength);
bool write(const uint8_t* buf, uint length);
bool write(sm_response& resp, uint payloadLength) const;
bool write(const uint8_t* buf, uint length) const;
void consumeMsg(); // drains the remaining portion of the message
uint getLength(); // returns the total length of the msg
uint getLength() const; // returns the total length of the msg
uint getRemainingLength(); // returns the remaining length from the caller's perspective
void handleError(const char* name, int errCode);
IOCoordinator* ioc;
private:
PosixTask();
int sock;
int totalLength;
uint remainingLengthInStream;

View File

@ -39,7 +39,8 @@ namespace storagemanager
class PrefixCache : public boost::noncopyable
{
public:
PrefixCache(const boost::filesystem::path& prefix);
PrefixCache() = delete;
explicit PrefixCache(const boost::filesystem::path& prefix);
virtual ~PrefixCache();
// reading fcns
@ -82,8 +83,6 @@ class PrefixCache : public boost::noncopyable
void validateCacheSize();
private:
PrefixCache();
boost::filesystem::path cachePrefix;
boost::filesystem::path journalPrefix;
boost::filesystem::path firstDir;

View File

@ -24,14 +24,13 @@ namespace storagemanager
class ProcessTask : public ThreadPool::Job
{
public:
ProcessTask() = delete;
ProcessTask(int sock, uint length); // _sock is the socket to read from
virtual ~ProcessTask();
~ProcessTask() override;
void operator()();
void operator()() override;
private:
ProcessTask();
void handleError(int errCode);
int sock;
uint length;

View File

@ -24,13 +24,11 @@ namespace storagemanager
class ReadTask : public PosixTask
{
public:
ReadTask() = delete;
ReadTask(int sock, uint length);
virtual ~ReadTask();
~ReadTask() override;
bool run();
private:
ReadTask();
bool run() override;
};
} // namespace storagemanager

View File

@ -17,11 +17,11 @@
#pragma once
//#include "ThreadPool.h"
// #include "ThreadPool.h"
#include "MetadataFile.h"
#include <boost/filesystem.hpp>
#include <sys/types.h>
#include <stdint.h>
#include <cstdint>
#define JOURNAL_ENTRY_HEADER_SIZE 16

View File

@ -36,8 +36,9 @@ class S3Storage : public CloudStorage
~S3Storage() override;
int getObject(const std::string& sourceKey, const std::string& destFile, size_t* size = NULL) override;
int getObject(const std::string& sourceKey, std::shared_ptr<uint8_t[]>* data, size_t* size = NULL) override;
int getObject(const std::string& sourceKey, const std::string& destFile, size_t* size = nullptr) override;
int getObject(const std::string& sourceKey, std::shared_ptr<uint8_t[]>* data,
size_t* size = nullptr) override;
int putObject(const std::string& sourceFile, const std::string& destKey) override;
int putObject(const std::shared_ptr<uint8_t[]> data, size_t len, const std::string& destKey) override;
int deleteObject(const std::string& key) override;
@ -77,7 +78,9 @@ class S3Storage : public CloudStorage
struct Connection
{
Connection(uint64_t id): id(id) {}
explicit Connection(uint64_t id) : id(id)
{
}
uint64_t id;
ms3_st* conn{nullptr};
timespec touchedAt{};
@ -96,7 +99,8 @@ class S3Storage : public CloudStorage
mutable boost::mutex connMutex;
std::deque<std::shared_ptr<Connection>> freeConns; // using this as a stack to keep lru objects together
std::unordered_map<uint64_t, std::shared_ptr<Connection>> usedConns; // using this for displaying and killing tasks
std::unordered_map<uint64_t, std::shared_ptr<Connection>>
usedConns; // using this for displaying and killing tasks
uint64_t nextConnId = 0;
const time_t maxIdleSecs = 30;
};

View File

@ -18,11 +18,10 @@
#include "StatTask.h"
#include "messageFormat.h"
#include "SMLogging.h"
#include <errno.h>
#include <cerrno>
#include <sys/types.h>
#include <sys/stat.h>
#include <stdint.h>
#include <string.h>
#include <cstdint>
using namespace std;

View File

@ -24,13 +24,11 @@ namespace storagemanager
class StatTask : public PosixTask
{
public:
StatTask() = delete;
StatTask(int sock, uint length);
virtual ~StatTask();
~StatTask() override;
bool run();
private:
StatTask();
bool run() override;
};
} // namespace storagemanager

View File

@ -18,7 +18,7 @@
#include "SyncTask.h"
#include "Synchronizer.h"
#include "messageFormat.h"
#include <errno.h>
#include <cerrno>
namespace storagemanager
{
@ -26,9 +26,7 @@ SyncTask::SyncTask(int sock, uint len) : PosixTask(sock, len)
{
}
SyncTask::~SyncTask()
{
}
SyncTask::~SyncTask() = default;
bool SyncTask::run()
{

View File

@ -24,13 +24,11 @@ namespace storagemanager
class SyncTask : public PosixTask
{
public:
SyncTask() = delete;
SyncTask(int sock, uint length);
virtual ~SyncTask();
~SyncTask() override;
bool run();
private:
SyncTask();
bool run() override;
};
} // namespace storagemanager

View File

@ -40,7 +40,7 @@ class Synchronizer : public boost::noncopyable, public ConfigListener
{
public:
static Synchronizer* get();
virtual ~Synchronizer();
~Synchronizer() override;
// these take keys as parameters, not full path names, ex, pass in '12345' not
// 'cache/12345'.
@ -61,7 +61,7 @@ class Synchronizer : public boost::noncopyable, public ConfigListener
boost::filesystem::path getCachePath();
void printKPIs() const;
virtual void configListener() override;
void configListener() override;
private:
Synchronizer();
@ -77,7 +77,7 @@ class Synchronizer : public boost::noncopyable, public ConfigListener
// this struct kind of got sloppy. Need to clean it at some point.
struct PendingOps
{
PendingOps(int flags);
explicit PendingOps(int flags);
~PendingOps();
int opFlags;
int waiters;
@ -89,9 +89,9 @@ class Synchronizer : public boost::noncopyable, public ConfigListener
struct Job : public ThreadPool::Job
{
virtual ~Job(){};
~Job() override = default;
Job(Synchronizer* s, std::list<std::string>::iterator i);
void operator()();
void operator()() override;
Synchronizer* sync;
std::list<std::string>::iterator it;
};

View File

@ -37,7 +37,7 @@ class ThreadPool : public boost::noncopyable
class Job
{
public:
virtual ~Job(){};
virtual ~Job() = default;
virtual void operator()() = 0;
};

View File

@ -24,13 +24,11 @@ namespace storagemanager
class TruncateTask : public PosixTask
{
public:
TruncateTask() = delete;
TruncateTask(int sock, uint length);
virtual ~TruncateTask();
~TruncateTask() override;
bool run();
private:
TruncateTask();
bool run() override;
};
} // namespace storagemanager

View File

@ -24,13 +24,11 @@ namespace storagemanager
class UnlinkTask : public PosixTask
{
public:
UnlinkTask() = delete;
UnlinkTask(int sock, uint length);
virtual ~UnlinkTask();
~UnlinkTask() override;
bool run();
private:
UnlinkTask();
bool run() override;
};
} // namespace storagemanager

View File

@ -42,23 +42,23 @@ struct ScopedFileLock
struct ScopedReadLock : public ScopedFileLock
{
ScopedReadLock(IOCoordinator* i, const std::string& k);
virtual ~ScopedReadLock();
void lock();
void unlock();
~ScopedReadLock() override;
void lock() override;
void unlock() override;
};
struct ScopedWriteLock : public ScopedFileLock
{
ScopedWriteLock(IOCoordinator* i, const std::string& k);
virtual ~ScopedWriteLock();
void lock();
void unlock();
~ScopedWriteLock() override;
void lock() override;
void unlock() override;
};
struct ScopedCloser
{
ScopedCloser();
ScopedCloser(int f);
explicit ScopedCloser(int f);
~ScopedCloser();
int fd;
};

View File

@ -24,13 +24,11 @@ namespace storagemanager
class WriteTask : public PosixTask
{
public:
WriteTask() = delete;
WriteTask(int sock, uint length);
virtual ~WriteTask();
~WriteTask() override;
bool run();
private:
WriteTask();
bool run() override;
};
} // namespace storagemanager

View File

@ -15,15 +15,9 @@
Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston,
MA 02110-1301, USA. */
#include <unistd.h>
#include <string>
#include <iostream>
#include <stdio.h>
#include <cstdlib>
#include <sys/types.h>
#include <sys/socket.h>
#include <netinet/in.h>
#include <signal.h>
#include <csignal>
using namespace std;
@ -54,7 +48,7 @@ class ServiceStorageManager : public Service, public Opt
void setupChildSignalHandlers();
public:
ServiceStorageManager(const Opt& opt) : Service("StorageManager"), Opt(opt)
explicit ServiceStorageManager(const Opt& opt) : Service("StorageManager"), Opt(opt)
{
}
void LogErrno() override
@ -74,14 +68,14 @@ class ServiceStorageManager : public Service, public Opt
bool signalCaught = false;
void printCacheUsage(int sig)
void printCacheUsage(int /*sig*/)
{
Cache::get()->validateCacheSize();
cout << "Current cache size = " << Cache::get()->getCurrentCacheSize() << endl;
cout << "Cache element count = " << Cache::get()->getCurrentCacheElementCount() << endl;
}
void printKPIs(int sig)
void printKPIs(int /*sig*/)
{
IOCoordinator::get()->printKPIs();
Cache::get()->printKPIs();
@ -120,29 +114,29 @@ void ServiceStorageManager::setupChildSignalHandlers()
sa.sa_handler = shutdownSM;
for (int sig : shutdownSignals)
sigaction(sig, &sa, NULL);
sigaction(sig, &sa, nullptr);
sa.sa_handler = coreSM;
for (int sig : coreSignals)
sigaction(sig, &sa, NULL);
sigaction(sig, &sa, nullptr);
sa.sa_handler = SIG_IGN;
sigaction(SIGPIPE, &sa, NULL);
sigaction(SIGPIPE, &sa, nullptr);
sa.sa_handler = printCacheUsage;
sigaction(SIGUSR1, &sa, NULL);
sigaction(SIGUSR1, &sa, nullptr);
sa.sa_handler = printKPIs;
sigaction(SIGUSR2, &sa, NULL);
sigaction(SIGUSR2, &sa, nullptr);
}
int ServiceStorageManager::Child()
{
SMLogging* logger = SMLogging::get();
IOCoordinator* ioc = NULL;
Cache* cache = NULL;
Synchronizer* sync = NULL;
Replicator* rep = NULL;
IOCoordinator* ioc = nullptr;
Cache* cache = nullptr;
Synchronizer* sync = nullptr;
Replicator* rep = nullptr;
/* Instantiate objects to have them verify config settings before continuing */
try