diff --git a/dist-test/dev_server.sh b/dist-test/dev_server.sh index 0c1d26516..0c87340ea 100755 --- a/dist-test/dev_server.sh +++ b/dist-test/dev_server.sh @@ -5,6 +5,8 @@ THIS_DIR=$(dirname $(readlink -f $0)) PROJ_ROOT=${THIS_DIR}/.. pushd ${PROJ_ROOT} > /dev/null +export OVERRIDE_CPU_COUNT=4 + if [[ -z "$1" ]]; then docker-compose up -d dist-test-server elif [[ "$1" == "restart" ]]; then diff --git a/dist-test/run.sh b/dist-test/run.sh index 9f3f596c3..747ac212c 100755 --- a/dist-test/run.sh +++ b/dist-test/run.sh @@ -6,6 +6,8 @@ pushd ${PROJ_ROOT} >> /dev/null export CONAN_CACHE_MOUNT_SOURCE=$HOME/.conan/ RETURN_VAL=0 +export OVERRIDE_CPU_COUNT=4 + # Run the test server in the background docker-compose \ up \ diff --git a/docker-compose.yml b/docker-compose.yml index c9840afac..deb8452e7 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -20,6 +20,10 @@ services: - LOG_LEVEL=debug - REDIS_STATE_HOST=redis - REDIS_QUEUE_HOST=redis + - OVERRIDE_CPU_COUNT=${OVERRIDE_CPU_COUNT:-0} + - ASAN_OPTIONS=verbosity=1:halt_on_error=1 + - TSAN_OPTIONS=halt_on_error=1:suppressions=/code/faabric/thread-sanitizer-ignorelist.txt:history_size=7:second_deadlock_stack=1 + - UBSAN_OPTIONS="print_stacktrace=1:halt_on_error=1 depends_on: - redis @@ -34,6 +38,7 @@ services: - LOG_LEVEL=debug - REDIS_STATE_HOST=redis - REDIS_QUEUE_HOST=redis + - OVERRIDE_CPU_COUNT=${OVERRIDE_CPU_COUNT:-0} command: ./bin/faabric_dist_test_server depends_on: - redis diff --git a/include/faabric/scheduler/Scheduler.h b/include/faabric/scheduler/Scheduler.h index 6ce6f2a55..e588ccb1d 100644 --- a/include/faabric/scheduler/Scheduler.h +++ b/include/faabric/scheduler/Scheduler.h @@ -32,13 +32,13 @@ class ExecutorTask ExecutorTask(int messageIndexIn, std::shared_ptr reqIn, std::shared_ptr> batchCounterIn, - bool needsSnapshotPushIn, + bool needsSnapshotSyncIn, bool skipResetIn); std::shared_ptr req; std::shared_ptr> batchCounter; int messageIndex = 0; - bool needsSnapshotPush = false; + bool needsSnapshotSync = false; bool skipReset = false; }; @@ -69,7 +69,7 @@ class Executor void releaseClaim(); - virtual faabric::util::SnapshotData snapshot(); + virtual faabric::util::MemoryView getMemoryView(); protected: virtual void restore(faabric::Message& msg); diff --git a/include/faabric/snapshot/SnapshotClient.h b/include/faabric/snapshot/SnapshotClient.h index 2c1aedded..5c88b7cee 100644 --- a/include/faabric/snapshot/SnapshotClient.h +++ b/include/faabric/snapshot/SnapshotClient.h @@ -12,7 +12,8 @@ namespace faabric::snapshot { // Mocking // ----------------------------------- -std::vector> +std::vector< + std::pair>> getSnapshotPushes(); std::vector>> @@ -35,12 +36,12 @@ class SnapshotClient final : public faabric::transport::MessageEndpointClient explicit SnapshotClient(const std::string& hostIn); void pushSnapshot(const std::string& key, - int32_t groupId, - const faabric::util::SnapshotData& data); + std::shared_ptr data); - void pushSnapshotDiffs(std::string snapshotKey, - int32_t groupId, - std::vector diffs); + void pushSnapshotDiffs( + std::string snapshotKey, + bool force, + const std::vector& diffs); void deleteSnapshot(const std::string& key); diff --git a/include/faabric/snapshot/SnapshotRegistry.h b/include/faabric/snapshot/SnapshotRegistry.h index 0c3e97f8c..7d6952a57 100644 --- a/include/faabric/snapshot/SnapshotRegistry.h +++ b/include/faabric/snapshot/SnapshotRegistry.h @@ -22,13 +22,12 @@ class SnapshotRegistry void mapSnapshot(const std::string& key, uint8_t* target); - void takeSnapshot(const std::string& key, - faabric::util::SnapshotData data, - bool locallyRestorable = true); + void registerSnapshot(const std::string& key, + std::shared_ptr data); - void takeSnapshotIfNotExists(const std::string& key, - faabric::util::SnapshotData data, - bool locallyRestorable = true); + void registerSnapshotIfNotExists( + const std::string& key, + std::shared_ptr data); void deleteSnapshot(const std::string& key); @@ -46,10 +45,9 @@ class SnapshotRegistry int writeSnapshotToFd(const std::string& key, faabric::util::SnapshotData& data); - void doTakeSnapshot(const std::string& key, - faabric::util::SnapshotData data, - bool locallyRestorable, - bool overwrite); + void doRegisterSnapshot(const std::string& key, + std::shared_ptr data, + bool overwrite); }; SnapshotRegistry& getSnapshotRegistry(); diff --git a/include/faabric/snapshot/SnapshotServer.h b/include/faabric/snapshot/SnapshotServer.h index 566bef2a6..a01c418c4 100644 --- a/include/faabric/snapshot/SnapshotServer.h +++ b/include/faabric/snapshot/SnapshotServer.h @@ -14,10 +14,6 @@ class SnapshotServer final : public faabric::transport::MessageEndpointServer public: SnapshotServer(); - // Returns how many diffs have been applied since started, useful for - // testing - size_t diffsApplied() const; - protected: void doAsyncRecv(int header, const uint8_t* buffer, @@ -40,6 +36,5 @@ class SnapshotServer final : public faabric::transport::MessageEndpointServer private: faabric::transport::PointToPointBroker& broker; - std::atomic_size_t diffsAppliedCounter = 0; }; } diff --git a/include/faabric/util/bytes.h b/include/faabric/util/bytes.h index a24876289..620fb8f2d 100644 --- a/include/faabric/util/bytes.h +++ b/include/faabric/util/bytes.h @@ -26,18 +26,18 @@ int safeCopyToBuffer(const uint8_t* dataIn, int bufferLen); template -T unalignedRead(const std::byte* bytes) +T unalignedRead(const uint8_t* bytes) { T value; - std::copy_n(bytes, sizeof(T), reinterpret_cast(&value)); + std::copy_n(bytes, sizeof(T), reinterpret_cast(&value)); return value; } template -void unalignedWrite(const T& value, std::byte* destination) +void unalignedWrite(const T& value, uint8_t* destination) { std::copy_n( - reinterpret_cast(&value), sizeof(T), destination); + reinterpret_cast(&value), sizeof(T), destination); } template diff --git a/include/faabric/util/memory.h b/include/faabric/util/memory.h index d7dddf983..510c7acb5 100644 --- a/include/faabric/util/memory.h +++ b/include/faabric/util/memory.h @@ -1,6 +1,10 @@ #pragma once #include +#include +#include +#include +#include #include #include @@ -22,7 +26,7 @@ struct AlignedChunk static const long HOST_PAGE_SIZE = sysconf(_SC_PAGESIZE); -bool isPageAligned(void* ptr); +bool isPageAligned(const void* ptr); size_t getRequiredHostPages(size_t nBytes); @@ -41,4 +45,27 @@ std::vector getDirtyPageNumbers(const uint8_t* ptr, int nPages); std::vector> getDirtyRegions(const uint8_t* ptr, int nPages); + +// ------------------------- +// Allocation +// ------------------------- +typedef std::unique_ptr> MemoryRegion; + +MemoryRegion allocateSharedMemory(size_t size); + +MemoryRegion allocateVirtualMemory(size_t size); + +void claimVirtualMemory(std::span region); + +void mapMemoryPrivate(std::span target, int fd); + +void mapMemoryShared(std::span target, int fd); + +void resizeFd(int fd, size_t size); + +void writeToFd(int fd, off_t offset, std::span data); + +int createFd(size_t size, const std::string& fdLabel); + +void appendDataToFd(int fd, std::span data); } diff --git a/include/faabric/util/snapshot.h b/include/faabric/util/snapshot.h index 0a87e32a4..7cd6e9130 100644 --- a/include/faabric/util/snapshot.h +++ b/include/faabric/util/snapshot.h @@ -3,11 +3,14 @@ #include #include #include +#include +#include #include #include #include #include +#include namespace faabric::util { @@ -30,28 +33,28 @@ enum SnapshotMergeOperation class SnapshotDiff { public: - const uint8_t* data = nullptr; - size_t size = 0; - SnapshotDataType dataType = SnapshotDataType::Raw; - SnapshotMergeOperation operation = SnapshotMergeOperation::Overwrite; - uint32_t offset = 0; - - bool noChange = false; - SnapshotDiff() = default; SnapshotDiff(SnapshotDataType dataTypeIn, SnapshotMergeOperation operationIn, uint32_t offsetIn, - const uint8_t* dataIn, - size_t sizeIn) - { - dataType = dataTypeIn; - operation = operationIn; - offset = offsetIn; - data = dataIn; - size = sizeIn; - } + std::span dataIn); + + SnapshotDataType getDataType() const { return dataType; } + + SnapshotMergeOperation getOperation() const { return operation; } + + uint32_t getOffset() const { return offset; } + + std::span getData() const { return data; } + + std::vector getDataCopy() const; + + private: + SnapshotDataType dataType = SnapshotDataType::Raw; + SnapshotMergeOperation operation = SnapshotMergeOperation::Overwrite; + uint32_t offset = 0; + std::vector data; }; class SnapshotMergeRegion @@ -73,16 +76,31 @@ class SnapshotMergeRegion class SnapshotData { public: - size_t size = 0; - uint8_t* data = nullptr; - int fd = 0; - SnapshotData() = default; - std::vector getDirtyPages(); + explicit SnapshotData(size_t sizeIn); + + SnapshotData(size_t sizeIn, size_t maxSizeIn); + + explicit SnapshotData(std::span dataIn); + + SnapshotData(std::span dataIn, size_t maxSizeIn); + + SnapshotData(const SnapshotData&) = delete; + + SnapshotData& operator=(const SnapshotData&) = delete; + + ~SnapshotData(); + + void copyInData(std::span buffer, uint32_t offset = 0); + + const uint8_t* getDataPtr(uint32_t offset = 0); + + std::vector getDataCopy(); - std::vector getChangeDiffs(const uint8_t* updated, - size_t updatedSize); + std::vector getDataCopy(uint32_t offset, size_t dataSize); + + void mapToMemory(uint8_t* target); void addMergeRegion(uint32_t offset, size_t length, @@ -90,10 +108,61 @@ class SnapshotData SnapshotMergeOperation operation, bool overwrite = false); + void clearMergeRegions(); + + std::map getMergeRegions(); + + size_t getQueuedDiffsCount(); + + void queueDiffs(std::span diffs); + + void writeQueuedDiffs(); + + size_t getSize() const { return size; } + + size_t getMaxSize() const { return maxSize; } + private: + size_t size = 0; + size_t maxSize = 0; + + int fd = -1; + + std::shared_mutex snapMx; + + MemoryRegion data = nullptr; + + std::vector queuedDiffs; + // Note - we care about the order of this map, as we iterate through it // in order of offsets std::map mergeRegions; + + uint8_t* validatedOffsetPtr(uint32_t offset); + + void mapToMemory(uint8_t* target, bool shared); + + void writeData(std::span buffer, uint32_t offset = 0); +}; + +class MemoryView +{ + public: + // Note - this object is just a view of a section of memory, and does not + // own the underlying data + MemoryView() = default; + + explicit MemoryView(std::span dataIn); + + std::vector getDirtyRegions(); + + std::vector diffWithSnapshot( + std::shared_ptr snap); + + std::span getData() { return data; } + + private: + std::span data; }; std::string snapshotDataTypeStr(SnapshotDataType dt); diff --git a/src/flat/faabric.fbs b/src/flat/faabric.fbs index db2e6d3f5..1f7a7ef5e 100644 --- a/src/flat/faabric.fbs +++ b/src/flat/faabric.fbs @@ -1,6 +1,6 @@ table SnapshotPushRequest { key:string; - groupid:int; + maxSize:ulong; contents:[ubyte]; } @@ -17,7 +17,7 @@ table SnapshotDiffChunk { table SnapshotDiffPushRequest { key:string; - groupid:int; + force:bool; chunks:[SnapshotDiffChunk]; } diff --git a/src/scheduler/Executor.cpp b/src/scheduler/Executor.cpp index 11b9b1733..dbe5767d8 100644 --- a/src/scheduler/Executor.cpp +++ b/src/scheduler/Executor.cpp @@ -22,12 +22,12 @@ namespace faabric::scheduler { ExecutorTask::ExecutorTask(int messageIndexIn, std::shared_ptr reqIn, std::shared_ptr> batchCounterIn, - bool needsSnapshotPushIn, + bool needsSnapshotSyncIn, bool skipResetIn) : req(std::move(reqIn)) , batchCounter(std::move(batchCounterIn)) , messageIndex(messageIndexIn) - , needsSnapshotPush(needsSnapshotPushIn) + , needsSnapshotSync(needsSnapshotSyncIn) , skipReset(skipResetIn) {} @@ -115,8 +115,6 @@ void Executor::executeTasks(std::vector msgIdxs, // competing, hence is rare so we can afford to be conservative here. faabric::util::UniqueLock lock(threadsMutex); - // Restore if necessary. If we're executing threads on the master host we - // assume we don't need to restore, but for everything else we do. faabric::Message& firstMsg = req->mutable_messages()->at(0); std::string snapshotKey = firstMsg.snapshotkey(); std::string thisHost = faabric::util::getSystemConfig().endpointHost; @@ -125,26 +123,27 @@ void Executor::executeTasks(std::vector msgIdxs, bool isThreads = req->type() == faabric::BatchExecuteRequest::THREADS; bool isSnapshot = !snapshotKey.empty(); - if (isSnapshot && !isMaster) { + // Restore if we have a snapshot + if (isSnapshot) { SPDLOG_DEBUG("Restoring {} from snapshot {}", funcStr, snapshotKey); restore(firstMsg); } // Reset dirty page tracking if we're executing threads. - // Note this must be done after the restore has happened - bool needsSnapshotPush = false; - if (isThreads && isSnapshot && !isMaster) { + // Note this must be done after the restore has happened. + bool needsSnapshotSync = false; + if (isThreads && isSnapshot) { faabric::util::resetDirtyTracking(); - needsSnapshotPush = true; + needsSnapshotSync = true; } // Set up shared counter for this batch of tasks auto batchCounter = std::make_shared>(msgIdxs.size()); - // Work out if we should skip the reset after this batch. This only needs to - // happen when we're executing threads on the master host, in which case the - // original function call will cause a reset - bool skipReset = isMaster && isThreads; + // Work out if we should skip the reset after this batch. This happens for + // threads, as they will be restored from their respective snapshot on the + // next execution. + bool skipReset = isThreads; // Iterate through and invoke tasks. By default, we allocate tasks // one-to-one with thread pool threads. Only once the pool is exhausted do @@ -192,7 +191,7 @@ void Executor::executeTasks(std::vector msgIdxs, // Enqueue the task threadTaskQueues[threadPoolIdx].enqueue(ExecutorTask( - msgIdx, req, batchCounter, needsSnapshotPush, skipReset)); + msgIdx, req, batchCounter, needsSnapshotSync, skipReset)); // Lazily create the thread if (threadPoolThreads.at(threadPoolIdx) == nullptr) { @@ -243,14 +242,23 @@ void Executor::threadPoolThread(int threadPoolIdx) faabric::Message& msg = task.req->mutable_messages()->at(task.messageIndex); + // Check ptp group + std::shared_ptr group = nullptr; + if (msg.groupid() > 0) { + group = + faabric::transport::PointToPointGroup::getGroup(msg.groupid()); + } + + bool isMaster = msg.masterhost() == conf.endpointHost; bool isThreads = task.req->type() == faabric::BatchExecuteRequest::THREADS; - SPDLOG_TRACE("Thread {}:{} executing task {} ({}, thread={})", + SPDLOG_TRACE("Thread {}:{} executing task {} ({}, thread={}, group={})", id, threadPoolIdx, task.messageIndex, msg.id(), - isThreads); + isThreads, + msg.groupid()); int32_t returnValue; try { @@ -269,6 +277,7 @@ void Executor::threadPoolThread(int threadPoolIdx) msg.set_returnvalue(returnValue); // Decrement the task count + std::atomic_thread_fence(std::memory_order_release); int oldTaskCount = task.batchCounter->fetch_sub(1); assert(oldTaskCount >= 0); bool isLastInBatch = oldTaskCount == 1; @@ -280,25 +289,38 @@ void Executor::threadPoolThread(int threadPoolIdx) oldTaskCount - 1); // Handle snapshot diffs _before_ we reset the executor - if (isLastInBatch && task.needsSnapshotPush) { - // Get diffs between original snapshot and after execution - auto snapshotPostExecution = snapshot(); + faabric::util::MemoryView funcMemory = getMemoryView(); + if (!funcMemory.getData().empty() && isLastInBatch && + task.needsSnapshotSync) { + auto snap = faabric::snapshot::getSnapshotRegistry().getSnapshot( + msg.snapshotkey()); - auto snapshotPreExecution = - faabric::snapshot::getSnapshotRegistry().getSnapshot( - msg.snapshotkey()); - - SPDLOG_TRACE("Diffing pre and post execution snapshots for {}", + SPDLOG_TRACE("Diffing memory with pre-execution snapshot for {}", msg.snapshotkey()); + // If we're on master, we write the diffs straight to the snapshot + // otherwise we push them to the master. std::vector diffs = - snapshotPreExecution->getChangeDiffs(snapshotPostExecution.data, - snapshotPostExecution.size); + funcMemory.diffWithSnapshot(snap); + + if (isMaster) { + SPDLOG_DEBUG("Queueing {} diffs for {} to snapshot {} on " + "master (group {})", + diffs.size(), + faabric::util::funcToString(msg, false), + msg.snapshotkey(), + msg.groupid()); - sch.pushSnapshotDiffs(msg, diffs); + snap->queueDiffs(diffs); + } else { + sch.pushSnapshotDiffs(msg, diffs); + + // Reset dirty page tracking on non-master + faabric::util::resetDirtyTracking(); + } - // Reset dirty page tracking now that we've pushed the diffs - faabric::util::resetDirtyTracking(); + SPDLOG_DEBUG("Clearing merge regions for {}", msg.snapshotkey()); + snap->clearMergeRegions(); } // If this batch is finished, reset the executor and release its claim. @@ -403,11 +425,11 @@ void Executor::postFinish() {} void Executor::reset(faabric::Message& msg) {} -faabric::util::SnapshotData Executor::snapshot() +faabric::util::MemoryView Executor::getMemoryView() { - SPDLOG_WARN("Executor has not implemented snapshot method"); - faabric::util::SnapshotData d; - return d; + SPDLOG_WARN("Executor for {} has not implemented memory view method", + faabric::util::funcToString(boundMessage, false)); + return faabric::util::MemoryView(); } void Executor::restore(faabric::Message& msg) diff --git a/src/scheduler/Scheduler.cpp b/src/scheduler/Scheduler.cpp index e5d259240..403395e95 100644 --- a/src/scheduler/Scheduler.cpp +++ b/src/scheduler/Scheduler.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include #include #include @@ -83,8 +84,7 @@ void Scheduler::addHostToGlobalSet() void Scheduler::resetThreadLocalCache() { - auto tid = (pid_t)syscall(SYS_gettid); - SPDLOG_DEBUG("Resetting scheduler thread-local cache for thread {}", tid); + SPDLOG_DEBUG("Resetting scheduler thread-local cache"); functionCallClients.clear(); snapshotClients.clear(); @@ -497,29 +497,30 @@ faabric::util::SchedulingDecision Scheduler::doCallFunctions( if (!snapshotKey.empty()) { for (const auto& host : getFunctionRegisteredHosts(firstMsg, false)) { SnapshotClient& c = getSnapshotClient(host); - auto snapshotData = + auto snap = faabric::snapshot::getSnapshotRegistry().getSnapshot(snapshotKey); // See if we've already pushed this snapshot to the given host, // if so, just push the diffs if (pushedSnapshotsMap[snapshotKey].contains(host)) { + MemoryView snapMemView({ snap->getDataPtr(), snap->getSize() }); + std::vector snapshotDiffs = - snapshotData->getDirtyPages(); - c.pushSnapshotDiffs( - snapshotKey, firstMsg.groupid(), snapshotDiffs); + snapMemView.getDirtyRegions(); + + c.pushSnapshotDiffs(snapshotKey, true, snapshotDiffs); } else { - c.pushSnapshot(snapshotKey, firstMsg.groupid(), *snapshotData); + c.pushSnapshot(snapshotKey, snap); pushedSnapshotsMap[snapshotKey].insert(host); } } } // Now reset the dirty page tracking just before we start executing - SPDLOG_DEBUG("Resetting dirty tracking after pushing diffs {}", funcStr); faabric::util::resetDirtyTracking(); // ------------------------------------------- - // EXECTUION + // EXECUTION // ------------------------------------------- // Records for tests - copy messages before execution to avoid racing on msg @@ -785,9 +786,11 @@ std::shared_ptr Scheduler::claimExecutor( // We have no warm executors available, so scale up if (claimed == nullptr) { - int nExecutors = thisExecutors.size(); - SPDLOG_DEBUG( - "Scaling {} from {} -> {}", funcStr, nExecutors, nExecutors + 1); + SPDLOG_DEBUG("Scaling {} from {} -> {}", + funcStr, + thisExecutors.size(), + thisExecutors.size() + 1); + // Spinning up a new executor can be lengthy, allow other things // to run in parallel schedulerLock.unlock(); @@ -899,12 +902,21 @@ void Scheduler::pushSnapshotDiffs( const faabric::Message& msg, const std::vector& diffs) { - bool isMaster = msg.masterhost() == conf.endpointHost; + if (diffs.empty()) { + return; + } - if (!isMaster && !diffs.empty()) { - SnapshotClient& c = getSnapshotClient(msg.masterhost()); - c.pushSnapshotDiffs(msg.snapshotkey(), msg.groupid(), diffs); + bool isMaster = msg.masterhost() == conf.endpointHost; + const std::string& snapKey = msg.snapshotkey(); + if (isMaster) { + SPDLOG_ERROR("{} pushing snapshot diffs for {} on master", + faabric::util::funcToString(msg, false), + snapKey); + throw std::runtime_error("Cannot push snapshot diffs on master"); } + + SnapshotClient& c = getSnapshotClient(msg.masterhost()); + c.pushSnapshotDiffs(snapKey, false, diffs); } void Scheduler::setThreadResultLocally(uint32_t msgId, int32_t returnValue) diff --git a/src/snapshot/SnapshotClient.cpp b/src/snapshot/SnapshotClient.cpp index c741d5408..dd8108c4d 100644 --- a/src/snapshot/SnapshotClient.cpp +++ b/src/snapshot/SnapshotClient.cpp @@ -14,7 +14,8 @@ namespace faabric::snapshot { static std::mutex mockMutex; -static std::vector> +static std::vector< + std::pair>> snapshotPushes; static std::vector< @@ -26,7 +27,8 @@ static std::vector> snapshotDeletes; static std::vector>> threadResults; -std::vector> +std::vector< + std::pair>> getSnapshotPushes() { faabric::util::UniqueLock lock(mockMutex); @@ -71,28 +73,31 @@ SnapshotClient::SnapshotClient(const std::string& hostIn) SNAPSHOT_SYNC_PORT) {} -void SnapshotClient::pushSnapshot(const std::string& key, - int groupId, - const faabric::util::SnapshotData& data) +void SnapshotClient::pushSnapshot( + const std::string& key, + std::shared_ptr data) { - if (data.size == 0) { + if (data->getSize() == 0) { SPDLOG_ERROR("Cannot push snapshot {} with size zero to {}", key, host); throw std::runtime_error("Pushing snapshot with zero size"); } - SPDLOG_DEBUG("Pushing snapshot {} to {} ({} bytes)", key, host, data.size); + SPDLOG_DEBUG( + "Pushing snapshot {} to {} ({} bytes)", key, host, data->getSize()); if (faabric::util::isMockMode()) { faabric::util::UniqueLock lock(mockMutex); + snapshotPushes.emplace_back(host, data); } else { // Set up the main request - // TODO - avoid copying data here + // TODO - avoid copying data here? flatbuffers::FlatBufferBuilder mb; auto keyOffset = mb.CreateString(key); - auto dataOffset = mb.CreateVector(data.data, data.size); - auto requestOffset = - CreateSnapshotPushRequest(mb, keyOffset, groupId, dataOffset); + auto dataOffset = + mb.CreateVector(data->getDataPtr(), data->getSize()); + auto requestOffset = CreateSnapshotPushRequest( + mb, keyOffset, data->getMaxSize(), dataOffset); mb.Finish(requestOffset); // Send it @@ -102,8 +107,8 @@ void SnapshotClient::pushSnapshot(const std::string& key, void SnapshotClient::pushSnapshotDiffs( std::string snapshotKey, - int groupId, - std::vector diffs) + bool force, + const std::vector& diffs) { if (faabric::util::isMockMode()) { faabric::util::UniqueLock lock(mockMutex); @@ -119,19 +124,20 @@ void SnapshotClient::pushSnapshotDiffs( // Create objects for all the chunks std::vector> diffsFbVector; for (const auto& d : diffs) { - auto dataOffset = mb.CreateVector(d.data, d.size); + std::span diffData = d.getData(); + auto dataOffset = + mb.CreateVector(diffData.data(), diffData.size()); auto chunk = CreateSnapshotDiffChunk( - mb, d.offset, d.dataType, d.operation, dataOffset); + mb, d.getOffset(), d.getDataType(), d.getOperation(), dataOffset); diffsFbVector.push_back(chunk); } - // Set up the main request - // TODO - avoid copying data here + // Set up the request auto keyOffset = mb.CreateString(snapshotKey); auto diffsOffset = mb.CreateVector(diffsFbVector); auto requestOffset = - CreateSnapshotDiffPushRequest(mb, keyOffset, groupId, diffsOffset); + CreateSnapshotDiffPushRequest(mb, keyOffset, force, diffsOffset); mb.Finish(requestOffset); SEND_FB_MSG(SnapshotCalls::PushSnapshotDiffs, mb); diff --git a/src/snapshot/SnapshotRegistry.cpp b/src/snapshot/SnapshotRegistry.cpp index 1fa343ec8..9175cdcdb 100644 --- a/src/snapshot/SnapshotRegistry.cpp +++ b/src/snapshot/SnapshotRegistry.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include @@ -27,101 +28,50 @@ std::shared_ptr SnapshotRegistry::getSnapshot( bool SnapshotRegistry::snapshotExists(const std::string& key) { + faabric::util::SharedLock lock(snapshotsMx); return snapshotMap.find(key) != snapshotMap.end(); } void SnapshotRegistry::mapSnapshot(const std::string& key, uint8_t* target) { auto d = getSnapshot(key); - - if (!faabric::util::isPageAligned((void*)target)) { - SPDLOG_ERROR( - "Mapping snapshot {} to non page-aligned address {}", key, target); - throw std::runtime_error( - "Mapping snapshot to non page-aligned address"); - } - - if (d->fd == 0) { - SPDLOG_ERROR("Attempting to map non-restorable snapshot"); - throw std::runtime_error("Mapping non-restorable snapshot"); - } - - void* mmapRes = - mmap(target, d->size, PROT_WRITE, MAP_PRIVATE | MAP_FIXED, d->fd, 0); - - if (mmapRes == MAP_FAILED) { - SPDLOG_ERROR( - "mmapping snapshot failed: {} ({})", errno, ::strerror(errno)); - throw std::runtime_error("mmapping snapshot failed"); - } + d->mapToMemory(target); } -void SnapshotRegistry::takeSnapshotIfNotExists(const std::string& key, - faabric::util::SnapshotData data, - bool locallyRestorable) +void SnapshotRegistry::registerSnapshotIfNotExists( + const std::string& key, + std::shared_ptr data) { - doTakeSnapshot(key, data, locallyRestorable, false); + doRegisterSnapshot(key, std::move(data), false); } -void SnapshotRegistry::takeSnapshot(const std::string& key, - faabric::util::SnapshotData data, - bool locallyRestorable) +void SnapshotRegistry::registerSnapshot( + const std::string& key, + std::shared_ptr data) { - doTakeSnapshot(key, data, locallyRestorable, true); + doRegisterSnapshot(key, std::move(data), true); } -void SnapshotRegistry::doTakeSnapshot(const std::string& key, - faabric::util::SnapshotData data, - bool locallyRestorable, - bool overwrite) +void SnapshotRegistry::doRegisterSnapshot( + const std::string& key, + std::shared_ptr data, + bool overwrite) { - if (data.size == 0) { - SPDLOG_ERROR("Cannot take snapshot {} of size zero", key); - throw std::runtime_error("Taking snapshot size zero"); - } - faabric::util::FullLock lock(snapshotsMx); - if (snapshotExists(key) && !overwrite) { + if (!overwrite && (snapshotMap.find(key) != snapshotMap.end())) { SPDLOG_TRACE("Skipping already existing snapshot {}", key); return; } - SPDLOG_TRACE("Registering snapshot {} size {} (restorable={})", - key, - data.size, - locallyRestorable); + SPDLOG_TRACE("Registering snapshot {} size {}", key, data->getSize()); - // Note - we only preserve the snapshot in the in-memory file, and do not - // take ownership for the original data referenced in SnapshotData - auto shared_data = - std::make_shared(std::move(data)); - snapshotMap[key] = shared_data; - - // Write to fd to be locally restorable - if (locallyRestorable) { - writeSnapshotToFd(key, *shared_data); - } + snapshotMap.insert_or_assign(key, std::move(data)); } void SnapshotRegistry::deleteSnapshot(const std::string& key) { faabric::util::FullLock lock(snapshotsMx); - - if (snapshotMap.count(key) == 0) { - return; - } - - auto d = snapshotMap[key]; - - // Note - the data referenced by the SnapshotData object is not owned by the - // snapshot registry so we don't delete it here. We only remove the file - // descriptor used for mapping memory - if (d->fd > 0) { - ::close(d->fd); - d->fd = 0; - } - snapshotMap.erase(key); } @@ -140,38 +90,6 @@ SnapshotRegistry& getSnapshotRegistry() void SnapshotRegistry::clear() { faabric::util::FullLock lock(snapshotsMx); - for (auto p : snapshotMap) { - if (p.second->fd > 0) { - ::close(p.second->fd); - } - } - snapshotMap.clear(); } - -int SnapshotRegistry::writeSnapshotToFd(const std::string& key, - faabric::util::SnapshotData& data) -{ - int fd = ::memfd_create(key.c_str(), 0); - - // Make the fd big enough - int ferror = ::ftruncate(fd, data.size); - if (ferror) { - SPDLOG_ERROR("ferror call failed with error {}", ferror); - throw std::runtime_error("Failed writing memory to fd (ftruncate)"); - } - - // Write the data - ssize_t werror = ::write(fd, data.data, data.size); - if (werror == -1) { - SPDLOG_ERROR("Write call failed with error {}", werror); - throw std::runtime_error("Failed writing memory to fd (write)"); - } - - // Record the fd - data.fd = fd; - - SPDLOG_DEBUG("Wrote snapshot {} to fd {}", key, fd); - return fd; -} } diff --git a/src/snapshot/SnapshotServer.cpp b/src/snapshot/SnapshotServer.cpp index 9e800c2ed..21a7e529c 100644 --- a/src/snapshot/SnapshotServer.cpp +++ b/src/snapshot/SnapshotServer.cpp @@ -6,11 +6,13 @@ #include #include #include +#include #include #include +#include #include -#include +using namespace faabric::util; namespace faabric::snapshot { SnapshotServer::SnapshotServer() @@ -18,15 +20,10 @@ SnapshotServer::SnapshotServer() SNAPSHOT_ASYNC_PORT, SNAPSHOT_SYNC_PORT, SNAPSHOT_INPROC_LABEL, - faabric::util::getSystemConfig().snapshotServerThreads) + getSystemConfig().snapshotServerThreads) , broker(faabric::transport::getPointToPointBroker()) {} -size_t SnapshotServer::diffsApplied() const -{ - return diffsAppliedCounter.load(std::memory_order_acquire); -} - void SnapshotServer::doAsyncRecv(int header, const uint8_t* buffer, size_t bufferSize) @@ -76,26 +73,22 @@ std::unique_ptr SnapshotServer::recvPushSnapshot( throw std::runtime_error("Received snapshot with zero size"); } - SPDLOG_DEBUG("Receiving snapshot {} (size {})", + SPDLOG_DEBUG("Receiving snapshot {} (size {}, max {})", r->key()->c_str(), - r->contents()->size()); + r->contents()->size(), + r->maxSize()); faabric::snapshot::SnapshotRegistry& reg = faabric::snapshot::getSnapshotRegistry(); // Set up the snapshot - faabric::util::SnapshotData data; - data.size = r->contents()->size(); + size_t snapSize = r->contents()->size(); + std::string snapKey = r->key()->str(); + auto d = std::make_shared( + std::span((uint8_t*)r->contents()->Data(), snapSize), r->maxSize()); - // TODO - avoid this copy by changing server superclass to allow subclasses - // to provide a buffer to receive data. - // TODO - work out snapshot ownership here, how do we know when to delete - // this data? - data.data = (uint8_t*)mmap( - nullptr, data.size, PROT_WRITE, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0); - std::memcpy(data.data, r->contents()->Data(), data.size); - - reg.takeSnapshot(r->key()->str(), data, true); + // Register snapshot + reg.registerSnapshot(snapKey, d); // Send response return std::make_unique(); @@ -119,7 +112,6 @@ SnapshotServer::recvPushSnapshotDiffs(const uint8_t* buffer, size_t bufferSize) { const SnapshotDiffPushRequest* r = flatbuffers::GetRoot(buffer); - int groupId = r->groupid(); SPDLOG_DEBUG( "Applying {} diffs to snapshot {}", r->chunks()->size(), r->key()->str()); @@ -129,92 +121,25 @@ SnapshotServer::recvPushSnapshotDiffs(const uint8_t* buffer, size_t bufferSize) faabric::snapshot::getSnapshotRegistry(); auto snap = reg.getSnapshot(r->key()->str()); - // Lock the function group if it exists - if (groupId > 0 && - faabric::transport::PointToPointGroup::groupExists(groupId)) { - faabric::transport::PointToPointGroup::getGroup(r->groupid()) - ->localLock(); - } - - // Iterate through the chunks passed in the request + // Convert chunks to snapshot diff objects + std::vector diffs; + diffs.reserve(r->chunks()->size()); for (const auto* chunk : *r->chunks()) { - uint8_t* dest = snap->data + chunk->offset(); - - SPDLOG_TRACE("Applying snapshot diff to {} at {}-{}", - r->key()->str(), - chunk->offset(), - chunk->offset() + chunk->data()->size()); - - switch (chunk->dataType()) { - case (faabric::util::SnapshotDataType::Raw): { - switch (chunk->mergeOp()) { - case (faabric::util::SnapshotMergeOperation::Overwrite): { - std::memcpy( - dest, chunk->data()->data(), chunk->data()->size()); - break; - } - default: { - SPDLOG_ERROR("Unsupported raw merge operation: {}", - chunk->mergeOp()); - throw std::runtime_error( - "Unsupported raw merge operation"); - } - } - break; - } - case (faabric::util::SnapshotDataType::Int): { - const auto* value = - reinterpret_cast(chunk->data()->data()); - auto* destValue = reinterpret_cast(dest); - switch (chunk->mergeOp()) { - case (faabric::util::SnapshotMergeOperation::Sum): { - *destValue += *value; - break; - } - case (faabric::util::SnapshotMergeOperation::Subtract): { - *destValue -= *value; - break; - } - case (faabric::util::SnapshotMergeOperation::Product): { - *destValue *= *value; - break; - } - case (faabric::util::SnapshotMergeOperation::Min): { - *destValue = std::min(*destValue, *value); - break; - } - case (faabric::util::SnapshotMergeOperation::Max): { - *destValue = std::max(*destValue, *value); - break; - } - default: { - SPDLOG_ERROR("Unsupported int merge operation: {}", - chunk->mergeOp()); - throw std::runtime_error( - "Unsupported int merge operation"); - } - } - break; - } - default: { - SPDLOG_ERROR("Unsupported data type: {}", chunk->dataType()); - throw std::runtime_error("Unsupported merge data type"); - } - } - // make changes visible to other threads - std::atomic_thread_fence(std::memory_order_release); - this->diffsAppliedCounter.fetch_add(1, std::memory_order_acq_rel); + diffs.emplace_back( + static_cast(chunk->dataType()), + static_cast(chunk->mergeOp()), + chunk->offset(), + std::span(chunk->data()->data(), + chunk->data()->size())); } - // Unlock group if exists - if (groupId > 0 && - faabric::transport::PointToPointGroup::groupExists(groupId)) { - faabric::transport::PointToPointGroup::getGroup(r->groupid()) - ->localUnlock(); - } + // Queue on the snapshot + snap->queueDiffs(diffs); - // Reset dirty tracking having applied diffs - SPDLOG_DEBUG("Resetting dirty page tracking having applied diffs"); + // Write if necessary + if (r->force()) { + snap->writeQueuedDiffs(); + } // Send response return std::make_unique(); diff --git a/src/transport/PointToPointBroker.cpp b/src/transport/PointToPointBroker.cpp index c8eac5623..3d70e3f4f 100644 --- a/src/transport/PointToPointBroker.cpp +++ b/src/transport/PointToPointBroker.cpp @@ -68,6 +68,8 @@ std::string getPointToPointKey(int groupId, int recvIdx) std::shared_ptr PointToPointGroup::getGroup(int groupId) { + faabric::util::SharedLock lock(groupsMutex); + if (groups.find(groupId) == groups.end()) { SPDLOG_ERROR("Did not find group ID {} on this host", groupId); throw std::runtime_error("Group ID not found on host"); @@ -241,7 +243,6 @@ void PointToPointGroup::localLock() bool PointToPointGroup::localTryLock() { - SPDLOG_TRACE("Trying local lock on {}", groupId); return localMx.try_lock(); } diff --git a/src/util/memory.cpp b/src/util/memory.cpp index 09150ce2f..35ef062a6 100644 --- a/src/util/memory.cpp +++ b/src/util/memory.cpp @@ -21,9 +21,9 @@ namespace faabric::util { // Alignment // ------------------------- -bool isPageAligned(void* ptr) +bool isPageAligned(const void* ptr) { - return (((uintptr_t)(const void*)(ptr)) % (HOST_PAGE_SIZE) == 0); + return (((uintptr_t)(ptr)) % (HOST_PAGE_SIZE) == 0); } size_t getRequiredHostPages(size_t nBytes) @@ -177,4 +177,157 @@ std::vector> getDirtyRegions(const uint8_t* ptr, return regions; } + +// ------------------------- +// Allocation +// ------------------------- + +MemoryRegion doAlloc(size_t size, int prot, int flags) +{ + auto deleter = [size](uint8_t* u) { munmap(u, size); }; + MemoryRegion mem((uint8_t*)::mmap(nullptr, size, prot, flags, -1, 0), + deleter); + + if (mem.get() == MAP_FAILED) { + SPDLOG_ERROR("Allocating memory with mmap failed: {} ({})", + errno, + ::strerror(errno)); + throw std::runtime_error("Allocating memory failed"); + } + + return mem; +} + +MemoryRegion allocateSharedMemory(size_t size) +{ + return doAlloc(size, PROT_READ | PROT_WRITE, MAP_SHARED | MAP_ANONYMOUS); +} + +MemoryRegion allocateVirtualMemory(size_t size) +{ + return doAlloc(size, PROT_NONE, MAP_PRIVATE | MAP_ANONYMOUS); +} + +void claimVirtualMemory(std::span region) +{ + int protectRes = + ::mprotect(region.data(), region.size(), PROT_READ | PROT_WRITE); + if (protectRes != 0) { + SPDLOG_ERROR("Failed claiming virtual memory: {}", strerror(errno)); + throw std::runtime_error("Failed claiming virtual memory"); + } +} + +void mapMemory(std::span target, int fd, int flags) +{ + if (!faabric::util::isPageAligned((void*)target.data())) { + SPDLOG_ERROR("Mapping memory to non page-aligned address"); + throw std::runtime_error("Mapping memory to non page-aligned address"); + } + + if (fd <= 0) { + SPDLOG_ERROR("Mapping invalid or zero fd ({})", fd); + throw std::runtime_error("Invalid fd for mapping"); + } + + void* mmapRes = ::mmap( + target.data(), target.size(), PROT_READ | PROT_WRITE, flags, fd, 0); + + if (mmapRes == MAP_FAILED) { + SPDLOG_ERROR("mapping memory to fd {} failed: {} ({})", + fd, + errno, + ::strerror(errno)); + throw std::runtime_error("mmapping memory failed"); + } +} + +void mapMemoryPrivate(std::span target, int fd) +{ + mapMemory(target, fd, MAP_PRIVATE | MAP_FIXED); +} + +void mapMemoryShared(std::span target, int fd) +{ + mapMemory(target, fd, MAP_SHARED | MAP_FIXED); +} + +void resizeFd(int fd, size_t size) +{ + int ferror = ::ftruncate(fd, size); + if (ferror != 0) { + SPDLOG_ERROR("ftruncate call failed with error {}", ferror); + throw std::runtime_error("Failed writing memory to fd (ftruncate)"); + } +} + +void writeToFd(int fd, off_t offset, std::span data) +{ + // Seek to the right point + off_t lseekRes = ::lseek(fd, offset, SEEK_SET); + if (lseekRes == -1) { + SPDLOG_ERROR("Failed to set fd {} to offset {}", fd, offset); + throw std::runtime_error("Failed changing fd size"); + } + + // Write the data + ssize_t werror = ::write(fd, data.data(), data.size()); + if (werror == -1) { + SPDLOG_ERROR("Write call failed with error {}", werror); + throw std::runtime_error("Failed writing memory to fd (write)"); + } + + // Set back to end + ::lseek(fd, 0, SEEK_END); +} + +int createFd(size_t size, const std::string& fdLabel) +{ + // Create new fd + int fd = ::memfd_create(fdLabel.c_str(), 0); + if (fd == -1) { + SPDLOG_ERROR("Failed to create file descriptor: {}", strerror(errno)); + throw std::runtime_error("Failed to create file descriptor"); + } + + // Make the fd big enough + resizeFd(fd, size); + + return fd; +} + +void appendDataToFd(int fd, std::span data) +{ + off_t oldSize = ::lseek(fd, 0, SEEK_END); + if (oldSize == -1) { + SPDLOG_ERROR("lseek to get old size failed: {}", strerror(errno)); + throw std::runtime_error("Failed seeking existing size of fd"); + } + + if (data.empty()) { + return; + } + + // Extend the fd + off_t newSize = oldSize + data.size(); + int ferror = ::ftruncate(fd, newSize); + if (ferror != 0) { + SPDLOG_ERROR("Extending with ftruncate failed with error {}", ferror); + throw std::runtime_error("Failed appending data to fd (ftruncate)"); + } + + // Skip to the end of the old data + off_t seekRes = ::lseek(fd, oldSize, SEEK_SET); + if (seekRes == -1) { + SPDLOG_ERROR("lseek call failed with error {}", strerror(errno)); + throw std::runtime_error("Failed appending data to fd"); + } + + // Write the data + ssize_t werror = ::write(fd, data.data(), data.size()); + if (werror == -1) { + SPDLOG_ERROR("Appending with write failed with error {}", werror); + throw std::runtime_error("Failed appending memory to fd (write)"); + } +} } diff --git a/src/util/snapshot.cpp b/src/util/snapshot.cpp index e44300f8e..16e6db1fe 100644 --- a/src/util/snapshot.cpp +++ b/src/util/snapshot.cpp @@ -1,81 +1,146 @@ +#include +#include #include #include #include #include #include +#include + namespace faabric::util { -// TODO - this would be better as an instance variable on the SnapshotData -// class, but it can't be copy-constructed. -static std::mutex snapMx; +SnapshotDiff::SnapshotDiff(SnapshotDataType dataTypeIn, + SnapshotMergeOperation operationIn, + uint32_t offsetIn, + std::span dataIn) + : dataType(dataTypeIn) + , operation(operationIn) + , offset(offsetIn) + , data(dataIn.begin(), dataIn.end()) +{} + +std::vector SnapshotDiff::getDataCopy() const +{ + return std::vector(data.begin(), data.end()); +} -std::vector SnapshotData::getDirtyPages() +SnapshotData::SnapshotData(size_t sizeIn) + : SnapshotData(sizeIn, sizeIn) +{} + +SnapshotData::SnapshotData(size_t sizeIn, size_t maxSizeIn) + : size(sizeIn) + , maxSize(maxSizeIn) { - if (data == nullptr || size == 0) { - std::vector empty; - return empty; + if (maxSize == 0) { + maxSize = size; } - // Get dirty pages - int nPages = getRequiredHostPages(size); - std::vector dirtyPageNumbers = getDirtyPageNumbers(data, nPages); + // Allocate virtual memory big enough for the max size if provided + data = faabric::util::allocateVirtualMemory(maxSize); - // Convert to snapshot diffs - // TODO - reduce number of diffs by merging adjacent dirty pages - std::vector diffs; - for (int i : dirtyPageNumbers) { - uint32_t offset = i * HOST_PAGE_SIZE; - diffs.emplace_back(SnapshotDataType::Raw, - SnapshotMergeOperation::Overwrite, - offset, - data + offset, - HOST_PAGE_SIZE); - } + // Claim just the snapshot region + faabric::util::claimVirtualMemory({ BYTES(data.get()), size }); - SPDLOG_DEBUG("Snapshot has {}/{} dirty pages", diffs.size(), nPages); + // Set up the fd with a two-way mapping to the data + std::string fdLabel = "snap_" + std::to_string(generateGid()); + fd = createFd(size, fdLabel); + mapMemoryShared({ data.get(), size }, fd); +} - return diffs; +SnapshotData::SnapshotData(std::span dataIn) + : SnapshotData(dataIn.size()) +{ + writeData(dataIn); } -std::vector SnapshotData::getChangeDiffs(const uint8_t* updated, - size_t updatedSize) +SnapshotData::SnapshotData(std::span dataIn, size_t maxSizeIn) + : SnapshotData(dataIn.size(), maxSizeIn) { - std::vector diffs; - if (mergeRegions.empty()) { - SPDLOG_DEBUG("No merge regions set, thus no diffs"); - return diffs; + writeData(dataIn); +} + +SnapshotData::~SnapshotData() +{ + if (fd > 0) { + SPDLOG_TRACE("Closing fd {}", fd); + ::close(fd); + fd = -1; } +} - // Work out which regions of memory have changed - size_t nThisPages = getRequiredHostPages(updatedSize); - std::vector> dirtyRegions = - getDirtyRegions(updated, nThisPages); - SPDLOG_TRACE("Found {} dirty regions", dirtyRegions.size()); +void SnapshotData::copyInData(std::span buffer, uint32_t offset) +{ + faabric::util::FullLock lock(snapMx); - // Iterate through merge regions, see which ones overlap with dirty memory - // regions, and add corresponding diffs - for (auto& mrPair : mergeRegions) { - SnapshotMergeRegion& mr = mrPair.second; + writeData(buffer, offset); +} - SPDLOG_TRACE("Merge region {} {} at {}-{}", - snapshotDataTypeStr(mr.dataType), - snapshotMergeOpStr(mr.operation), - mr.offset, - mr.offset + mr.length); +void SnapshotData::writeData(std::span buffer, uint32_t offset) +{ + // Try to allocate more memory on top of existing data if necessary. + // Will throw an exception if not possible + size_t newSize = offset + buffer.size(); + if (newSize > size) { + if (newSize > maxSize) { + SPDLOG_ERROR( + "Copying snapshot data over max: {} > {}", newSize, maxSize); + throw std::runtime_error("Copying snapshot data over max"); + } - for (auto& dirtyRegion : dirtyRegions) { - // Add the diffs - mr.addDiffs(diffs, - data, - size, - updated, - dirtyRegion.first, - dirtyRegion.second); + claimVirtualMemory({ data.get(), newSize }); + size = newSize; + + // Update fd + if (fd > 0) { + resizeFd(fd, newSize); } + + // Remap data + mapMemoryShared({ data.get(), size }, fd); } - return diffs; + // Copy in new data + uint8_t* copyTarget = validatedOffsetPtr(offset); + ::memcpy(copyTarget, buffer.data(), buffer.size()); +} + +const uint8_t* SnapshotData::getDataPtr(uint32_t offset) +{ + faabric::util::SharedLock lock(snapMx); + return validatedOffsetPtr(offset); +} + +uint8_t* SnapshotData::validatedOffsetPtr(uint32_t offset) +{ + if (offset > size) { + SPDLOG_ERROR("Out of bounds snapshot access: {} > {}", offset, size); + throw std::runtime_error("Out of bounds snapshot access"); + } + + return data.get() + offset; +} + +std::vector SnapshotData::getDataCopy() +{ + return getDataCopy(0, size); +} + +std::vector SnapshotData::getDataCopy(uint32_t offset, size_t dataSize) +{ + faabric::util::SharedLock lock(snapMx); + + if ((offset + dataSize) > size) { + SPDLOG_ERROR("Out of bounds snapshot access: {} + {} > {}", + offset, + dataSize, + size); + throw std::runtime_error("Out of bounds snapshot access"); + } + + uint8_t* ptr = validatedOffsetPtr(offset); + return std::vector(ptr, ptr + dataSize); } void SnapshotData::addMergeRegion(uint32_t offset, @@ -84,14 +149,13 @@ void SnapshotData::addMergeRegion(uint32_t offset, SnapshotMergeOperation operation, bool overwrite) { + faabric::util::FullLock lock(snapMx); + SnapshotMergeRegion region{ .offset = offset, .length = length, .dataType = dataType, .operation = operation }; - // Locking as this may be called in bursts by multiple threads - faabric::util::UniqueLock lock(snapMx); - if (mergeRegions.find(region.offset) != mergeRegions.end()) { if (!overwrite) { SPDLOG_ERROR("Attempting to overwrite existing merge region at {} " @@ -123,6 +187,228 @@ void SnapshotData::addMergeRegion(uint32_t offset, mergeRegions[region.offset] = region; } +void SnapshotData::mapToMemory(uint8_t* target) +{ + faabric::util::FullLock lock(snapMx); + + if (fd <= 0) { + std::string msg = "Attempting to map memory of non-restorable snapshot"; + SPDLOG_ERROR(msg); + throw std::runtime_error(msg); + } + + mapMemoryPrivate({ target, size }, fd); +} + +std::map SnapshotData::getMergeRegions() +{ + faabric::util::SharedLock lock(snapMx); + return mergeRegions; +} + +void SnapshotData::clearMergeRegions() +{ + faabric::util::FullLock lock(snapMx); + mergeRegions.clear(); +} + +size_t SnapshotData::getQueuedDiffsCount() +{ + faabric::util::SharedLock lock(snapMx); + return queuedDiffs.size(); +} + +void SnapshotData::queueDiffs(const std::span diffs) +{ + faabric::util::FullLock lock(snapMx); + for (const auto& diff : diffs) { + queuedDiffs.emplace_back(std::move(diff)); + } +} + +void SnapshotData::writeQueuedDiffs() +{ + faabric::util::FullLock lock(snapMx); + + // Iterate through diffs + for (auto& diff : queuedDiffs) { + switch (diff.getDataType()) { + case (faabric::util::SnapshotDataType::Raw): { + switch (diff.getOperation()) { + case (faabric::util::SnapshotMergeOperation::Overwrite): { + SPDLOG_TRACE("Copying raw snapshot diff into {}-{}", + diff.getOffset(), + diff.getOffset() + diff.getData().size()); + + writeData(diff.getData(), diff.getOffset()); + break; + } + default: { + SPDLOG_ERROR("Unsupported raw merge operation: {}", + diff.getOperation()); + throw std::runtime_error( + "Unsupported raw merge operation"); + } + } + break; + } + case (faabric::util::SnapshotDataType::Int): { + auto diffValue = unalignedRead(diff.getData().data()); + + auto original = faabric::util::unalignedRead( + validatedOffsetPtr(diff.getOffset())); + + int32_t finalValue = 0; + switch (diff.getOperation()) { + case (faabric::util::SnapshotMergeOperation::Sum): { + finalValue = original + diffValue; + + SPDLOG_TRACE("Applying int sum diff {}: {} = {} + {}", + diff.getOffset(), + finalValue, + original, + diffValue); + break; + } + case (faabric::util::SnapshotMergeOperation::Subtract): { + finalValue = original - diffValue; + + SPDLOG_TRACE("Applying int sub diff {}: {} = {} - {}", + diff.getOffset(), + finalValue, + original, + diffValue); + break; + } + case (faabric::util::SnapshotMergeOperation::Product): { + finalValue = original * diffValue; + + SPDLOG_TRACE("Applying int mult diff {}: {} = {} * {}", + diff.getOffset(), + finalValue, + original, + diffValue); + break; + } + case (faabric::util::SnapshotMergeOperation::Min): { + finalValue = std::min(original, diffValue); + + SPDLOG_TRACE("Applying int min diff {}: min({}, {})", + diff.getOffset(), + original, + diffValue); + break; + } + case (faabric::util::SnapshotMergeOperation::Max): { + finalValue = std::max(original, diffValue); + + SPDLOG_TRACE("Applying int max diff {}: max({}, {})", + diff.getOffset(), + original, + diffValue); + break; + } + default: { + SPDLOG_ERROR("Unsupported int merge operation: {}", + diff.getOperation()); + throw std::runtime_error( + "Unsupported int merge operation"); + } + } + + writeData({ BYTES(&finalValue), sizeof(int32_t) }, + diff.getOffset()); + break; + } + default: { + SPDLOG_ERROR("Unsupported data type: {}", diff.getDataType()); + throw std::runtime_error("Unsupported merge data type"); + } + } + } + + // Clear queue + queuedDiffs.clear(); +} + +MemoryView::MemoryView(std::span dataIn) + : data(dataIn) +{} + +std::vector MemoryView::getDirtyRegions() +{ + if (data.empty()) { + return {}; + } + + // Get dirty regions + int nPages = getRequiredHostPages(data.size()); + std::vector dirtyPageNumbers = + getDirtyPageNumbers(data.data(), nPages); + + std::vector> regions = + faabric::util::getDirtyRegions(data.data(), nPages); + + // Convert to snapshot diffs + std::vector diffs; + diffs.reserve(regions.size()); + for (auto [regionBegin, regionEnd] : regions) { + diffs.emplace_back(SnapshotDataType::Raw, + SnapshotMergeOperation::Overwrite, + regionBegin, + data.subspan(regionBegin, regionEnd - regionBegin)); + } + + SPDLOG_DEBUG("Memory view has {}/{} dirty pages", diffs.size(), nPages); + + return diffs; +} + +std::vector MemoryView::diffWithSnapshot( + std::shared_ptr snap) +{ + std::vector diffs; + std::map mergeRegions = + snap->getMergeRegions(); + if (mergeRegions.empty()) { + SPDLOG_DEBUG("No merge regions set, thus no diffs"); + return diffs; + } + + // Work out which regions of memory have changed + size_t nThisPages = getRequiredHostPages(data.size()); + std::vector> dirtyRegions = + faabric::util::getDirtyRegions(data.data(), nThisPages); + SPDLOG_TRACE("Found {} dirty regions at {} over {} pages", + dirtyRegions.size(), + (void*)data.data(), + nThisPages); + + // Iterate through merge regions, see which ones overlap with dirty memory + // regions, and add corresponding diffs + for (auto& mrPair : mergeRegions) { + SnapshotMergeRegion& mr = mrPair.second; + + SPDLOG_TRACE("Merge region {} {} at {}-{}", + snapshotDataTypeStr(mr.dataType), + snapshotMergeOpStr(mr.operation), + mr.offset, + mr.offset + mr.length); + + for (auto& dirtyRegion : dirtyRegions) { + // Add the diffs + mr.addDiffs(diffs, + snap->getDataPtr(), + snap->getSize(), + data.data(), + dirtyRegion.first, + dirtyRegion.second); + } + } + + return diffs; +} + std::string snapshotDataTypeStr(SnapshotDataType dt) { switch (dt) { @@ -185,9 +471,11 @@ void SnapshotMergeRegion::addDiffs(std::vector& diffs, return; } - SPDLOG_TRACE("Checking for {} {} merge region in dirty region {}-{}", + SPDLOG_TRACE("{} {} merge region {}-{} aligns with dirty region {}-{}", snapshotDataTypeStr(dataType), snapshotMergeOpStr(operation), + offset, + offset + length, dirtyRegionStart, dirtyRegionEnd); @@ -249,10 +537,12 @@ void SnapshotMergeRegion::addDiffs(std::vector& diffs, (uint8_t*)updatedValue, BYTES(&updatedInt), sizeof(int32_t)); // Add the diff - diffs.emplace_back( - dataType, operation, offset, updatedValue, length); + diffs.emplace_back(dataType, + operation, + offset, + std::span(updatedValue, length)); - SPDLOG_TRACE("Adding {} {} diff at {}-{} ({})", + SPDLOG_TRACE("Found {} {} diff at {}-{} ({})", snapshotDataTypeStr(dataType), snapshotMergeOpStr(operation), offset, @@ -293,18 +583,19 @@ void SnapshotMergeRegion::addDiffs(std::vector& diffs, // Diff ends if it's not different and diff is // in progress int diffLength = b - diffStart; - SPDLOG_TRACE("Adding {} {} diff at {}-{}", + SPDLOG_TRACE("Found {} {} diff at {}-{}", snapshotDataTypeStr(dataType), snapshotMergeOpStr(operation), diffStart, diffStart + diffLength); diffInProgress = false; - diffs.emplace_back(dataType, - operation, - diffStart, - updated + diffStart, - diffLength); + diffs.emplace_back( + dataType, + operation, + diffStart, + std::span(updated + diffStart, + diffLength)); } } @@ -319,11 +610,12 @@ void SnapshotMergeRegion::addDiffs(std::vector& diffs, diffStart, diffStart + finalDiffLength); - diffs.emplace_back(dataType, - operation, - diffStart, - updated + diffStart, - finalDiffLength); + diffs.emplace_back( + dataType, + operation, + diffStart, + std::span(updated + diffStart, + finalDiffLength)); } break; } diff --git a/tasks/dev.py b/tasks/dev.py index 65b833a7d..c0921e624 100644 --- a/tasks/dev.py +++ b/tasks/dev.py @@ -82,7 +82,7 @@ def install(ctx, target, shared=False): @task -def sanitise(ctx, mode, noclean=False, shared=False): +def sanitise(ctx, mode, target="faabric_tests", noclean=False, shared=False): """ Build the tests with different sanitisers """ @@ -95,4 +95,4 @@ def sanitise(ctx, mode, noclean=False, shared=False): sanitise_mode=mode, ) - cc(ctx, "faabric_tests", shared=shared) + cc(ctx, target, shared=shared) diff --git a/tests/dist/DistTestExecutor.cpp b/tests/dist/DistTestExecutor.cpp index f1130c21b..12622f49c 100644 --- a/tests/dist/DistTestExecutor.cpp +++ b/tests/dist/DistTestExecutor.cpp @@ -3,7 +3,9 @@ #include #include +#include #include +#include using namespace faabric::scheduler; @@ -52,28 +54,44 @@ int32_t DistTestExecutor::executeTask( return callback(this, threadPoolIdx, msgIdx, req); } -faabric::util::SnapshotData DistTestExecutor::snapshot() +void DistTestExecutor::reset(faabric::Message& msg) { - faabric::util::SnapshotData snap; - snap.data = snapshotMemory; - snap.size = snapshotSize; - - return snap; + SPDLOG_DEBUG("Dist test executor resetting for {}", + faabric::util::funcToString(msg, false)); } void DistTestExecutor::restore(faabric::Message& msg) { - // Initialise the dummy memory and map to snapshot + SPDLOG_DEBUG("Dist test executor restoring for {}", + faabric::util::funcToString(msg, false)); + faabric::snapshot::SnapshotRegistry& reg = faabric::snapshot::getSnapshotRegistry(); + auto snap = reg.getSnapshot(msg.snapshotkey()); - // Note this has to be mmapped to be page-aligned - snapshotMemory = (uint8_t*)mmap( - nullptr, snap->size, PROT_WRITE, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0); - snapshotSize = snap->size; + setUpDummyMemory(snap->getSize()); - reg.mapSnapshot(msg.snapshotkey(), snapshotMemory); + reg.mapSnapshot(msg.snapshotkey(), dummyMemory.get()); +} + +faabric::util::MemoryView DistTestExecutor::getMemoryView() +{ + return faabric::util::MemoryView({ dummyMemory.get(), dummyMemorySize }); +} + +std::span DistTestExecutor::getDummyMemory() +{ + return { dummyMemory.get(), dummyMemorySize }; +} + +void DistTestExecutor::setUpDummyMemory(size_t memSize) +{ + if (dummyMemory.get() == nullptr) { + SPDLOG_DEBUG("Dist test executor initialising memory size {}", memSize); + dummyMemory = faabric::util::allocateSharedMemory(memSize); + dummyMemorySize = memSize; + } } std::shared_ptr DistTestExecutorFactory::createExecutor( diff --git a/tests/dist/DistTestExecutor.h b/tests/dist/DistTestExecutor.h index 7eecd73fc..45a1fc749 100644 --- a/tests/dist/DistTestExecutor.h +++ b/tests/dist/DistTestExecutor.h @@ -7,16 +7,6 @@ namespace tests { -typedef int (*ExecutorFunction)( - faabric::scheduler::Executor* exec, - int threadPoolIdx, - int msgIdx, - std::shared_ptr req); - -void registerDistTestExecutorCallback(const char* user, - const char* funcName, - ExecutorFunction func); - class DistTestExecutor final : public faabric::scheduler::Executor { public: @@ -29,13 +19,20 @@ class DistTestExecutor final : public faabric::scheduler::Executor int msgIdx, std::shared_ptr req) override; - faabric::util::SnapshotData snapshot() override; - - uint8_t* snapshotMemory = nullptr; - size_t snapshotSize = 0; + void reset(faabric::Message& msg) override; - protected: void restore(faabric::Message& msg) override; + + faabric::util::MemoryView getMemoryView() override; + + std::span getDummyMemory(); + + void setUpDummyMemory(size_t memSize); + + private: + faabric::util::MemoryRegion dummyMemory = nullptr; + + size_t dummyMemorySize = 0; }; class DistTestExecutorFactory : public faabric::scheduler::ExecutorFactory @@ -44,4 +41,15 @@ class DistTestExecutorFactory : public faabric::scheduler::ExecutorFactory std::shared_ptr createExecutor( faabric::Message& msg) override; }; + +typedef int (*ExecutorFunction)( + tests::DistTestExecutor* exec, + int threadPoolIdx, + int msgIdx, + std::shared_ptr req); + +void registerDistTestExecutorCallback(const char* user, + const char* funcName, + ExecutorFunction func); + } diff --git a/tests/dist/scheduler/functions.cpp b/tests/dist/scheduler/functions.cpp index 8680964ee..f0b755767 100644 --- a/tests/dist/scheduler/functions.cpp +++ b/tests/dist/scheduler/functions.cpp @@ -12,13 +12,16 @@ #include #include #include +#include #include #include #include +using namespace faabric::util; + namespace tests { -int handleSimpleThread(faabric::scheduler::Executor* exec, +int handleSimpleThread(tests::DistTestExecutor* exec, int threadPoolIdx, int msgIdx, std::shared_ptr req) @@ -37,7 +40,7 @@ int handleSimpleThread(faabric::scheduler::Executor* exec, return returnValue; } -int handleSimpleFunction(faabric::scheduler::Executor* exec, +int handleSimpleFunction(tests::DistTestExecutor* exec, int threadPoolIdx, int msgIdx, std::shared_ptr req) @@ -54,7 +57,7 @@ int handleSimpleFunction(faabric::scheduler::Executor* exec, return 0; } -int handleFakeDiffsFunction(faabric::scheduler::Executor* exec, +int handleFakeDiffsFunction(tests::DistTestExecutor* exec, int threadPoolIdx, int msgIdx, std::shared_ptr req) @@ -68,7 +71,6 @@ int handleFakeDiffsFunction(faabric::scheduler::Executor* exec, faabric::snapshot::getSnapshotRegistry(); auto originalSnap = reg.getSnapshot(snapshotKey); - faabric::util::SnapshotData updatedSnap = exec->snapshot(); // Add a single merge region to catch both diffs int offsetA = 10; @@ -83,15 +85,19 @@ int handleFakeDiffsFunction(faabric::scheduler::Executor* exec, // Modify the executor's memory std::vector keyBytes = faabric::util::stringToBytes(snapshotKey); - std::memcpy(updatedSnap.data + offsetA, keyBytes.data(), keyBytes.size()); - std::memcpy( - updatedSnap.data + offsetB, inputBytes.data(), inputBytes.size()); + + std::memcpy(exec->getDummyMemory().data() + offsetA, + keyBytes.data(), + keyBytes.size()); + std::memcpy(exec->getDummyMemory().data() + offsetB, + inputBytes.data(), + inputBytes.size()); return 123; } int handleFakeDiffsThreadedFunction( - faabric::scheduler::Executor* exec, + tests::DistTestExecutor* exec, int threadPoolIdx, int msgIdx, std::shared_ptr req) @@ -112,14 +118,8 @@ int handleFakeDiffsThreadedFunction( // Set up the snapshot size_t snapSize = (nThreads * 4) * faabric::util::HOST_PAGE_SIZE; - uint8_t* snapMemory = (uint8_t*)mmap( - nullptr, snapSize, PROT_WRITE, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0); - - faabric::util::SnapshotData snap; - snap.data = snapMemory; - snap.size = snapSize; - - reg.takeSnapshot(snapshotKey, snap); + auto snap = std::make_shared(snapSize); + reg.registerSnapshot(snapshotKey, snap); auto req = faabric::util::batchExecFactory(msg.user(), msg.function(), nThreads); @@ -135,8 +135,7 @@ int handleFakeDiffsThreadedFunction( // the child thread to make sure it's not overwritten std::vector localChange(3, i); int offset = 2 * i * faabric::util::HOST_PAGE_SIZE; - std::memcpy( - snapMemory + offset, localChange.data(), localChange.size()); + snap->copyInData(localChange, offset); } // Dispatch the message, expecting them all to execute on other hosts @@ -165,15 +164,17 @@ int handleFakeDiffsThreadedFunction( sch.awaitThreadResult(m.id()); } - // Check that the changes have been made to the snapshot memory + // Write queued diffs + snap->writeQueuedDiffs(); + + // Check changes have been applied bool success = true; for (int i = 0; i < nThreads; i++) { // Check local modifications std::vector expectedLocal(3, i); int localOffset = 2 * i * faabric::util::HOST_PAGE_SIZE; - std::vector actualLocal(snapMemory + localOffset, - snapMemory + localOffset + - expectedLocal.size()); + std::vector actualLocal = + snap->getDataCopy(localOffset, expectedLocal.size()); if (actualLocal != expectedLocal) { SPDLOG_ERROR("Local modifications not present for {}", i); @@ -183,7 +184,8 @@ int handleFakeDiffsThreadedFunction( // Check remote modifications int offset = 2 * i * faabric::util::HOST_PAGE_SIZE + 10; std::string expectedData("thread_" + std::to_string(i)); - auto* charPtr = reinterpret_cast(snapMemory + offset); + const auto* charPtr = + reinterpret_cast(snap->getDataPtr(offset)); std::string actual(charPtr); if (actual != expectedData) { @@ -210,7 +212,6 @@ int handleFakeDiffsThreadedFunction( faabric::util::stringToBytes(msgInput); auto originalSnap = reg.getSnapshot(snapshotKey); - faabric::util::SnapshotData updatedSnap = exec->snapshot(); // Make sure it's captured by the region int regionLength = 20 + inputBytes.size(); @@ -221,7 +222,7 @@ int handleFakeDiffsThreadedFunction( faabric::util::SnapshotMergeOperation::Overwrite); // Now modify the memory - std::memcpy(updatedSnap.data + changeOffset, + std::memcpy(exec->getDummyMemory().data() + changeOffset, inputBytes.data(), inputBytes.size()); @@ -231,6 +232,208 @@ int handleFakeDiffsThreadedFunction( return 333; } +/* + * This function performs two reductions and non-conflicting updates to a shared + * array in a loop to check distributed snapshot synchronisation and merge + * strategies. + */ +int handleReductionFunction(tests::DistTestExecutor* exec, + int threadPoolIdx, + int msgIdx, + std::shared_ptr req) +{ + faabric::snapshot::SnapshotRegistry& reg = + faabric::snapshot::getSnapshotRegistry(); + + faabric::Message& msg = req->mutable_messages()->at(msgIdx); + + int nThreads = 4; + int nRepeats = 20; + size_t snapSize = 4 * HOST_PAGE_SIZE; + int groupId = 1234; + + // Perform two reductions and one array modification. One reduction on same + // page as array change + uint32_t reductionAOffset = HOST_PAGE_SIZE; + uint32_t reductionBOffset = 2 * HOST_PAGE_SIZE; + uint32_t arrayOffset = HOST_PAGE_SIZE + 10 * sizeof(int32_t); + + // Initialise message + bool isThread = req->type() == faabric::BatchExecuteRequest::THREADS; + + // Set up snapshot + std::string snapKey = "dist-reduction-" + std::to_string(generateGid()); + std::shared_ptr snap = + std::make_shared(snapSize); + reg.registerSnapshot(snapKey, snap); + + // Main function will set up the snapshot and merge regions, while the child + // threads will modify an array and perform a reduction operation + if (!isThread) { + for (int r = 0; r < nRepeats; r++) { + // Set up thread request + auto req = faabric::util::batchExecFactory( + msg.user(), msg.function(), nThreads); + req->set_type(faabric::BatchExecuteRequest::THREADS); + for (int i = 0; i < nThreads; i++) { + auto& m = req->mutable_messages()->at(i); + m.set_snapshotkey(snapKey); + + // Set app/ group info + m.set_groupid(groupId); + m.set_groupidx(i); + m.set_appidx(i); + } + + // Make the request + faabric::scheduler::Scheduler& sch = + faabric::scheduler::getScheduler(); + std::vector actualHosts = sch.callFunctions(req).hosts; + + // Check hosts + std::string thisHost = getSystemConfig().endpointHost; + int nThisHost = 0; + int nOtherHost = 0; + for (const auto& h : actualHosts) { + if (h == thisHost) { + nThisHost++; + } else { + nOtherHost++; + } + } + + if (nThisHost != 2 || nOtherHost != 2) { + SPDLOG_ERROR("Threads not scheduled as expected: {} {}", + nThisHost, + nOtherHost); + return 1; + } + + // Wait for the threads + for (const auto& m : req->messages()) { + int32_t thisRes = sch.awaitThreadResult(m.id()); + if (thisRes != 0) { + SPDLOG_ERROR( + "Distributed reduction test thread {} failed: {}", + m.id(), + thisRes); + + return 1; + } + } + + SPDLOG_DEBUG("Reduce test threads finished"); + + // Write queued snapshot diffs + snap->writeQueuedDiffs(); + + // Remap memory to snapshot + snap->mapToMemory(exec->getDummyMemory().data()); + + uint8_t* reductionAPtr = + exec->getDummyMemory().data() + reductionAOffset; + uint8_t* reductionBPtr = + exec->getDummyMemory().data() + reductionBOffset; + uint8_t* arrayPtr = exec->getDummyMemory().data() + arrayOffset; + + // Check everything as expected + int expectedReductionA = (r + 1) * nThreads * 10; + int expectedReductionB = (r + 1) * nThreads * 20; + auto actualReductionA = unalignedRead(reductionAPtr); + auto actualReductionB = unalignedRead(reductionBPtr); + + bool success = true; + + for (int i = 0; i < nThreads; i++) { + uint8_t* thisPtr = arrayPtr + (i * sizeof(int32_t)); + int expectedValue = i * 30; + auto actualValue = unalignedRead(thisPtr); + + if (expectedValue != actualValue) { + success = false; + SPDLOG_ERROR("Dist array merge at {} failed: {} != {}", + i, + expectedValue, + actualValue); + } + } + + if (expectedReductionA != actualReductionA) { + success = false; + SPDLOG_ERROR("Dist reduction A failed: {} != {}", + expectedReductionA, + actualReductionA); + } + + if (expectedReductionB != actualReductionB) { + success = false; + SPDLOG_ERROR("Dist reduction B failed: {} != {}", + expectedReductionB, + actualReductionB); + } + + if (!success) { + return 1; + } + } + } else { + uint8_t* reductionAPtr = + exec->getDummyMemory().data() + reductionAOffset; + uint8_t* reductionBPtr = + exec->getDummyMemory().data() + reductionBOffset; + + uint8_t* arrayPtr = exec->getDummyMemory().data() + arrayOffset; + uint32_t thisIdx = msg.appidx(); + uint8_t* thisArrayPtr = arrayPtr + (sizeof(int32_t) * thisIdx); + + // Set merge regions + std::shared_ptr snap = reg.getSnapshot(msg.snapshotkey()); + snap->addMergeRegion(reductionAOffset, + sizeof(int32_t), + SnapshotDataType::Int, + SnapshotMergeOperation::Sum, + true); + + snap->addMergeRegion(reductionBOffset, + sizeof(int32_t), + SnapshotDataType::Int, + SnapshotMergeOperation::Sum, + true); + + snap->addMergeRegion(arrayOffset, + sizeof(int32_t) * nThreads, + SnapshotDataType::Raw, + SnapshotMergeOperation::Overwrite, + true); + + // Lock group locally while doing reduction + std::shared_ptr group = + faabric::transport::PointToPointGroup::getGroup(groupId); + group->localLock(); + + // Make modifications + int32_t initialA = unalignedRead(reductionAPtr); + int32_t initialB = unalignedRead(reductionBPtr); + + unalignedWrite(initialA + 10, reductionAPtr); + unalignedWrite(initialB + 20, reductionBPtr); + + int arrayValue = thisIdx * 30; + unalignedWrite(arrayValue, thisArrayPtr); + + SPDLOG_DEBUG("Reduce test thread {}: {} {} {}", + thisIdx, + arrayValue, + initialA, + initialB); + + // Unlock group + group->localUnlock(); + } + + return 0; +} + void registerSchedulerTestFunctions() { registerDistTestExecutorCallback("threads", "simple", handleSimpleThread); @@ -242,5 +445,8 @@ void registerSchedulerTestFunctions() registerDistTestExecutorCallback( "snapshots", "fake-diffs-threaded", handleFakeDiffsThreadedFunction); + + registerDistTestExecutorCallback( + "snapshots", "reduction", handleReductionFunction); } } diff --git a/tests/dist/scheduler/test_snapshots.cpp b/tests/dist/scheduler/test_snapshots.cpp index a2e0aa084..1d983deda 100644 --- a/tests/dist/scheduler/test_snapshots.cpp +++ b/tests/dist/scheduler/test_snapshots.cpp @@ -20,7 +20,7 @@ namespace tests { TEST_CASE_METHOD(DistTestsFixture, - "Check snapshots sent back from worker are applied", + "Check snapshots sent back from worker are queued", "[snapshots]") { std::string user = "snapshots"; @@ -28,14 +28,10 @@ TEST_CASE_METHOD(DistTestsFixture, std::string snapshotKey = "dist-snap-check"; size_t snapSize = 2 * faabric::util::HOST_PAGE_SIZE; - uint8_t* snapMemory = (uint8_t*)mmap( - nullptr, snapSize, PROT_WRITE, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0); + auto snap = std::make_shared(snapSize); // Set up snapshot - faabric::util::SnapshotData snap; - snap.data = snapMemory; - snap.size = snapSize; - reg.takeSnapshot(snapshotKey, snap); + reg.registerSnapshot(snapshotKey, snap); // Set up the message std::shared_ptr req = @@ -61,11 +57,15 @@ TEST_CASE_METHOD(DistTestsFixture, int actualResult = sch.awaitThreadResult(m.id()); REQUIRE(actualResult == 123); - // Check the diffs have been applied + // Write the diffs and check they've been applied + REQUIRE(snap->getQueuedDiffsCount() == 2); + snap->writeQueuedDiffs(); + size_t sizeA = snapshotKey.size(); size_t sizeB = inputData.size(); - uint8_t* startA = snapMemory + 10; - uint8_t* startB = snapMemory + 100; + + const uint8_t* startA = snap->getDataPtr() + 10; + const uint8_t* startB = snap->getDataPtr() + 100; std::vector actualA(startA, startA + sizeA); std::vector actualB(startB, startB + sizeB); @@ -103,4 +103,27 @@ TEST_CASE_METHOD(DistTestsFixture, faabric::Message actualResult = sch.getFunctionResult(m.id(), 10000); REQUIRE(actualResult.returnvalue() == 333); } + +TEST_CASE_METHOD(DistTestsFixture, "Check repeated reduction", "[snapshots]") +{ + std::string user = "snapshots"; + std::string function = "reduction"; + + std::shared_ptr req = + faabric::util::batchExecFactory(user, function, 1); + faabric::Message& m = req->mutable_messages()->at(0); + + // Main function and one thread execute on this host, others on another + faabric::HostResources res; + res.set_slots(3); + sch.setThisHostResources(res); + + std::vector expectedHosts = { getMasterIP() }; + faabric::util::SchedulingDecision decision = sch.callFunctions(req); + std::vector executedHosts = decision.hosts; + REQUIRE(expectedHosts == executedHosts); + + faabric::Message actualResult = sch.getFunctionResult(m.id(), 10000); + REQUIRE(actualResult.returnvalue() == 0); +} } diff --git a/tests/dist/scheduler/test_threads.cpp b/tests/dist/scheduler/test_threads.cpp index 86f0d1b3f..258be0a70 100644 --- a/tests/dist/scheduler/test_threads.cpp +++ b/tests/dist/scheduler/test_threads.cpp @@ -1,6 +1,6 @@ -#include "faabric_utils.h" #include +#include "faabric_utils.h" #include "fixtures.h" #include "init.h" @@ -13,6 +13,7 @@ #include #include #include +#include namespace tests { @@ -27,25 +28,23 @@ TEST_CASE_METHOD(DistTestsFixture, res.set_slots(nLocalSlots); sch.setThisHostResources(res); - // Set up the messages - std::shared_ptr req = - faabric::util::batchExecFactory("threads", "simple", 4); - req->set_type(faabric::BatchExecuteRequest::THREADS); - // Set up a snapshot size_t snapshotSize = 5 * faabric::util::HOST_PAGE_SIZE; - auto* snapshotData = (uint8_t*)mmap( - nullptr, snapshotSize, PROT_WRITE, MAP_SHARED | MAP_ANONYMOUS, -1, 0); - - faabric::util::SnapshotData snap; - snap.data = snapshotData; - snap.size = snapshotSize; + auto snap = std::make_shared(snapshotSize); std::string snapKey = std::to_string(faabric::util::generateGid()); - reg.takeSnapshot(snapKey, snap); + reg.registerSnapshot(snapKey, snap); - faabric::Message& firstMsg = req->mutable_messages()->at(0); - firstMsg.set_snapshotkey(snapKey); + // Set up the message + std::shared_ptr req = + faabric::util::batchExecFactory("threads", "simple", nThreads); + req->set_type(faabric::BatchExecuteRequest::THREADS); + + for (int i = 0; i < nThreads; i++) { + faabric::Message& m = req->mutable_messages()->at(i); + m.set_appidx(i); + m.set_snapshotkey(snapKey); + } // Call the functions sch.callFunctions(req); @@ -63,7 +62,5 @@ TEST_CASE_METHOD(DistTestsFixture, int res = sch.awaitThreadResult(m.id()); REQUIRE(res == m.id() / 2); } - - munmap(snapshotData, snapshotSize); } } diff --git a/tests/dist/transport/functions.cpp b/tests/dist/transport/functions.cpp index f145e7d14..09c329963 100644 --- a/tests/dist/transport/functions.cpp +++ b/tests/dist/transport/functions.cpp @@ -19,7 +19,7 @@ using namespace faabric::util; namespace tests { int handlePointToPointFunction( - faabric::scheduler::Executor* exec, + tests::DistTestExecutor* exec, int threadPoolIdx, int msgIdx, std::shared_ptr req) @@ -62,7 +62,7 @@ int handlePointToPointFunction( return 0; } -int handleDistributedLock(faabric::scheduler::Executor* exec, +int handleDistributedLock(tests::DistTestExecutor* exec, int threadPoolIdx, int msgIdx, std::shared_ptr req) @@ -280,7 +280,7 @@ class DistributedCoordinationTestRunner int groupId = 123; }; -int handleDistributedBarrier(faabric::scheduler::Executor* exec, +int handleDistributedBarrier(tests::DistTestExecutor* exec, int threadPoolIdx, int msgIdx, std::shared_ptr req) @@ -298,7 +298,7 @@ int handleDistributedBarrier(faabric::scheduler::Executor* exec, } int handleDistributedBarrierWorker( - faabric::scheduler::Executor* exec, + tests::DistTestExecutor* exec, int threadPoolIdx, int msgIdx, std::shared_ptr req) @@ -334,7 +334,7 @@ int handleDistributedBarrierWorker( return runner.checkResults(expectedResults); } -int handleDistributedNotify(faabric::scheduler::Executor* exec, +int handleDistributedNotify(tests::DistTestExecutor* exec, int threadPoolIdx, int msgIdx, std::shared_ptr req) @@ -352,7 +352,7 @@ int handleDistributedNotify(faabric::scheduler::Executor* exec, } int handleDistributedNotifyWorker( - faabric::scheduler::Executor* exec, + tests::DistTestExecutor* exec, int threadPoolIdx, int msgIdx, std::shared_ptr req) diff --git a/tests/test/scheduler/test_executor.cpp b/tests/test/scheduler/test_executor.cpp index ad8f67481..ae92cbf9a 100644 --- a/tests/test/scheduler/test_executor.cpp +++ b/tests/test/scheduler/test_executor.cpp @@ -33,15 +33,6 @@ TestExecutor::TestExecutor(faabric::Message& msg) : Executor(msg) {} -TestExecutor::~TestExecutor() = default; - -void TestExecutor::postFinish() -{ - if (dummyMemory != nullptr) { - munmap(dummyMemory, dummyMemorySize); - } -} - void TestExecutor::reset(faabric::Message& msg) { SPDLOG_DEBUG("Resetting TestExecutor"); @@ -53,25 +44,18 @@ void TestExecutor::restore(faabric::Message& msg) SPDLOG_DEBUG("Restoring TestExecutor"); restoreCount += 1; - // Initialise the dummy memory and map to snapshot faabric::snapshot::SnapshotRegistry& reg = faabric::snapshot::getSnapshotRegistry(); auto snap = reg.getSnapshot(msg.snapshotkey()); - // Note this has to be mmapped to be page-aligned - dummyMemorySize = snap->size; - dummyMemory = (uint8_t*)mmap( - nullptr, snap->size, PROT_WRITE, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0); - - reg.mapSnapshot(msg.snapshotkey(), dummyMemory); + dummyMemorySize = snap->getSize(); + dummyMemory = faabric::util::allocateSharedMemory(snap->getSize()); + reg.mapSnapshot(msg.snapshotkey(), dummyMemory.get()); } -faabric::util::SnapshotData TestExecutor::snapshot() +faabric::util::MemoryView TestExecutor::getMemoryView() { - faabric::util::SnapshotData snap; - snap.data = dummyMemory; - snap.size = dummyMemorySize; - return snap; + return faabric::util::MemoryView({ dummyMemory.get(), dummyMemorySize }); } int32_t TestExecutor::executeTask( @@ -79,9 +63,7 @@ int32_t TestExecutor::executeTask( int msgIdx, std::shared_ptr reqOrig) { - faabric::Message& msg = reqOrig->mutable_messages()->at(msgIdx); - std::string funcStr = faabric::util::funcToString(msg, true); bool isThread = reqOrig->type() == faabric::BatchExecuteRequest::THREADS; @@ -111,12 +93,9 @@ int32_t TestExecutor::executeTask( std::string snapKey = funcStr + "-snap"; faabric::snapshot::SnapshotRegistry& reg = faabric::snapshot::getSnapshotRegistry(); - faabric::util::SnapshotData snap; - snap.size = 10; - auto snapDataAllocation = std::make_unique(snap.size); - snap.data = snapDataAllocation.get(); - - reg.takeSnapshot(snapKey, snap); + size_t snapSize = 10; + auto snap = std::make_shared(snapSize); + reg.registerSnapshot(snapKey, snap); for (int i = 0; i < chainedReq->messages_size(); i++) { faabric::Message& m = chainedReq->mutable_messages()->at(i); @@ -217,8 +196,7 @@ int32_t TestExecutor::executeTask( offset, offset + data.size()); - uint8_t* offsetPtr = dummyMemory + offset; - std::memcpy(offsetPtr, data.data(), data.size()); + ::memcpy(dummyMemory.get() + offset, data.data(), data.size()); } if (msg.function() == "echo") { @@ -260,20 +238,22 @@ class TestExecutorFixture std::make_shared(); setExecutorFactory(fac); - setUpDummySnapshot(); + dummySnap = setUpSnapshot(snapshotKey, snapshotNPages); + faabric::util::resetDirtyTracking(); restoreCount = 0; resetCount = 0; } - ~TestExecutorFixture() { munmap(snapshotData, snapshotSize); } + ~TestExecutorFixture() = default; protected: std::string snapshotKey = "foobar"; - uint8_t* snapshotData = nullptr; int snapshotNPages = 10; size_t snapshotSize = snapshotNPages * faabric::util::HOST_PAGE_SIZE; + std::shared_ptr dummySnap; + std::vector executeWithTestExecutor( std::shared_ptr req, bool forceLocal) @@ -289,13 +269,6 @@ class TestExecutorFixture return sch.callFunctions(req, topologyHint).hosts; } - - private: - void setUpDummySnapshot() - { - takeSnapshot(snapshotKey, snapshotNPages, true); - faabric::util::resetDirtyTracking(); - } }; TEST_CASE_METHOD(TestExecutorFixture, @@ -403,8 +376,8 @@ TEST_CASE_METHOD(TestExecutorFixture, REQUIRE(result == msgId / 100); } - // Check that restore has not been called as we're on the master - REQUIRE(restoreCount == 0); + // Check that restore has been called + REQUIRE(restoreCount == 1); } TEST_CASE_METHOD(TestExecutorFixture, @@ -429,8 +402,8 @@ TEST_CASE_METHOD(TestExecutorFixture, faabric::Message res = sch.getFunctionResult(msg.id(), 5000); REQUIRE(res.returnvalue() == 0); - // Check that restore has not been called as we're on the master - REQUIRE(restoreCount == 0); + // Check that restore has been called + REQUIRE(restoreCount == 1); } TEST_CASE_METHOD(TestExecutorFixture, @@ -700,17 +673,20 @@ TEST_CASE_METHOD(TestExecutorFixture, for (int i = 0; i < diffList.size(); i++) { // Check offset and data (according to logic defined in the dummy // executor) - REQUIRE(diffList.at(i).offset == i * faabric::util::HOST_PAGE_SIZE); + REQUIRE(diffList.at(i).getOffset() == + i * faabric::util::HOST_PAGE_SIZE); std::vector expected = { (uint8_t)(i + 1), (uint8_t)(i + 2), (uint8_t)(i + 3) }; - std::vector actual(diffList.at(i).data, - diffList.at(i).data + 3); + std::vector actual = diffList.at(i).getDataCopy(); REQUIRE(actual == expected); } + + // Check no merge regions left on the snapshot + REQUIRE(reg.getSnapshot(snapshotKey)->getMergeRegions().empty()); } TEST_CASE_METHOD(TestExecutorFixture, @@ -735,7 +711,7 @@ TEST_CASE_METHOD(TestExecutorFixture, resOther.set_slots(10); faabric::scheduler::queueResourceResponse(otherHost, resOther); - // Execute a batch of threads + // Set up message for a batch of threads std::shared_ptr req = faabric::util::batchExecFactory("dummy", "blah", nThreads); req->set_type(faabric::BatchExecuteRequest::THREADS); @@ -770,23 +746,26 @@ TEST_CASE_METHOD(TestExecutorFixture, // Check snapshot has been pushed auto pushes = faabric::snapshot::getSnapshotPushes(); + REQUIRE(pushes.size() == 1); REQUIRE(pushes.at(0).first == otherHost); - REQUIRE(pushes.at(0).second.size == snapshotSize); + REQUIRE(pushes.at(0).second->getSize() == snapshotSize); REQUIRE(faabric::snapshot::getSnapshotDiffPushes().empty()); - // Check that we're not registering any dirty pages on the snapshot - auto snap = reg.getSnapshot(snapshotKey); - REQUIRE(snap->getDirtyPages().empty()); - // Now reset snapshot pushes of all kinds faabric::snapshot::clearMockSnapshotRequests(); - // Make an edit to the snapshot memory and get the expected diffs - snap->data[0] = 9; - snap->data[(2 * faabric::util::HOST_PAGE_SIZE) + 1] = 9; + // Update the snapshot and check we get expected diffs + auto snap = reg.getSnapshot(snapshotKey); + int newValue = 8; + snap->copyInData({ BYTES(&newValue), sizeof(int) }); + snap->copyInData({ BYTES(&newValue), sizeof(int) }, + 2 * faabric::util::HOST_PAGE_SIZE + 1); + std::vector expectedDiffs = - snap->getDirtyPages(); + faabric::util::MemoryView({ snap->getDataPtr(), snap->getSize() }) + .getDirtyRegions(); + REQUIRE(expectedDiffs.size() == 2); // Set up another function @@ -815,13 +794,15 @@ TEST_CASE_METHOD(TestExecutorFixture, diffPushes.at(0).second; for (int i = 0; i < actualDiffs.size(); i++) { - REQUIRE(actualDiffs.at(i).offset == expectedDiffs.at(i).offset); - REQUIRE(actualDiffs.at(i).size == expectedDiffs.at(i).size); + REQUIRE(actualDiffs.at(i).getOffset() == + expectedDiffs.at(i).getOffset()); + REQUIRE(actualDiffs.at(i).getData().size() == + expectedDiffs.at(i).getData().size()); } } TEST_CASE_METHOD(TestExecutorFixture, - "Test reset not called on master threads", + "Test reset not called on threads", "[executor]") { faabric::util::setMockMode(true); @@ -840,6 +821,7 @@ TEST_CASE_METHOD(TestExecutorFixture, hostOverride = "foobar"; nMessages = 3; requestType = faabric::BatchExecuteRequest::THREADS; + expectedResets = 0; } SECTION("Master threads") diff --git a/tests/test/scheduler/test_scheduler.cpp b/tests/test/scheduler/test_scheduler.cpp index 565965138..98820cfca 100644 --- a/tests/test/scheduler/test_scheduler.cpp +++ b/tests/test/scheduler/test_scheduler.cpp @@ -222,17 +222,13 @@ TEST_CASE_METHOD(SlowExecutorFixture, "Test batch scheduling", "[scheduler]") bool isThreads = execMode == faabric::BatchExecuteRequest::THREADS; // Set up a dummy snapshot if necessary - faabric::util::SnapshotData snapshot; faabric::snapshot::SnapshotRegistry& snapRegistry = faabric::snapshot::getSnapshotRegistry(); std::unique_ptr snapshotDataAllocation; if (!expectedSnapshot.empty()) { - snapshot.size = 1234; - snapshotDataAllocation = std::make_unique(snapshot.size); - snapshot.data = snapshotDataAllocation.get(); - - snapRegistry.takeSnapshot(expectedSnapshot, snapshot); + auto snap = std::make_shared(1234); + snapRegistry.registerSnapshot(expectedSnapshot, snap); } // Mock everything @@ -302,10 +298,13 @@ TEST_CASE_METHOD(SlowExecutorFixture, "Test batch scheduling", "[scheduler]") REQUIRE(snapshotPushes.empty()); } else { REQUIRE(snapshotPushes.size() == 1); + + auto snapshot = snapRegistry.getSnapshot(expectedSnapshot); + auto pushedSnapshot = snapshotPushes.at(0); REQUIRE(pushedSnapshot.first == otherHost); - REQUIRE(pushedSnapshot.second.size == snapshot.size); - REQUIRE(pushedSnapshot.second.data == snapshot.data); + REQUIRE(pushedSnapshot.second->getSize() == snapshot->getSize()); + REQUIRE(pushedSnapshot.second->getDataPtr() == snapshot->getDataPtr()); } // Check the executor counts on this host @@ -418,16 +417,13 @@ TEST_CASE_METHOD(SlowExecutorFixture, SECTION("Functions") { execMode = faabric::BatchExecuteRequest::FUNCTIONS; } // Set up snapshot if necessary - faabric::util::SnapshotData snapshot; faabric::snapshot::SnapshotRegistry& snapRegistry = faabric::snapshot::getSnapshotRegistry(); - std::unique_ptr snapshotDataAllocation; + size_t snapSize = 1234; if (!expectedSnapshot.empty()) { - snapshot.size = 1234; - snapshotDataAllocation = std::make_unique(snapshot.size); - snapshot.data = snapshotDataAllocation.get(); - snapRegistry.takeSnapshot(expectedSnapshot, snapshot); + auto snap = std::make_shared(snapSize); + snapRegistry.registerSnapshot(expectedSnapshot, snap); } // Set up this host with very low resources @@ -799,15 +795,11 @@ TEST_CASE_METHOD(SlowExecutorFixture, faabric::Message msg = faabric::util::messageFactory("foo", "bar"); msg.set_masterhost("otherHost"); - std::vector diffs; int returnValue = 123; std::string snapshotKey; - SECTION("Without diffs") - { - // Set the thread result - sch.setThreadResult(msg, returnValue); - } + // Set the thread result + sch.setThreadResult(msg, returnValue); auto actualResults = faabric::snapshot::getThreadResults(); @@ -862,6 +854,9 @@ TEST_CASE_METHOD(DummyExecutorFixture, std::string thisHost = conf.endpointHost; std::string otherHost = "foobar"; + faabric::transport::PointToPointBroker& broker = + faabric::transport::getPointToPointBroker(); + sch.addHostToGlobalSet(otherHost); // Set resources for this host @@ -882,6 +877,7 @@ TEST_CASE_METHOD(DummyExecutorFixture, int appId = firstMsg.appid(); int groupId = 0; + int groupSize = 10; bool forceLocal = false; bool expectMappingsSent = false; @@ -919,6 +915,13 @@ TEST_CASE_METHOD(DummyExecutorFixture, } } + // Set up the group + if (groupId > 0) { + faabric::transport::PointToPointGroup::addGroup( + appId, groupId, groupSize); + } + + // Build expectation std::vector expectedHosts = { thisHost, thisHost, otherHost, otherHost }; @@ -950,8 +953,6 @@ TEST_CASE_METHOD(DummyExecutorFixture, checkSchedulingDecisionEquality(expectedDecision, actualDecision); // Check mappings set up locally or not - faabric::transport::PointToPointBroker& broker = - faabric::transport::getPointToPointBroker(); std::set registeredIdxs = broker.getIdxsRegisteredForGroup(groupId); if (expectMappingsSent) { REQUIRE(registeredIdxs.size() == 4); @@ -969,5 +970,15 @@ TEST_CASE_METHOD(DummyExecutorFixture, } else { REQUIRE(sentMappings.empty()); } + + // Wait for the functions on this host to complete + for (int i = 0; i < expectedHosts.size(); i++) { + if (expectedHosts.at(i) != thisHost) { + continue; + } + + uint32_t messageId = req->mutable_messages()->at(i).id(); + sch.getFunctionResult(messageId, 10000); + } } } diff --git a/tests/test/snapshot/test_snapshot_client_server.cpp b/tests/test/snapshot/test_snapshot_client_server.cpp index 7a9a55d99..25abd5545 100644 --- a/tests/test/snapshot/test_snapshot_client_server.cpp +++ b/tests/test/snapshot/test_snapshot_client_server.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include #include #include @@ -73,42 +74,31 @@ TEST_CASE_METHOD(SnapshotClientServerFixture, // Prepare some snapshot data std::string snapKeyA = "foo"; std::string snapKeyB = "bar"; - SnapshotData snapA; - SnapshotData snapB; size_t snapSizeA = 1024; size_t snapSizeB = 500; - snapA.size = snapSizeA; - snapB.size = snapSizeB; std::vector dataA(snapSizeA, 1); std::vector dataB(snapSizeB, 2); - snapA.data = dataA.data(); - snapB.data = dataB.data(); + auto snapA = std::make_shared(dataA); + auto snapB = std::make_shared(dataB); - // One request with no group - int appId = 111; - int groupIdA = 0; - int groupIdB = 123; - - setUpFunctionGroup(appId, groupIdB); + REQUIRE(reg.getSnapshotCount() == 0); - // Send the message - cli.pushSnapshot(snapKeyA, groupIdA, snapA); - cli.pushSnapshot(snapKeyB, groupIdB, snapB); + // Send the messages + cli.pushSnapshot(snapKeyA, snapA); + cli.pushSnapshot(snapKeyB, snapB); // Check snapshots created in registry REQUIRE(reg.getSnapshotCount() == 2); const auto actualA = reg.getSnapshot(snapKeyA); const auto actualB = reg.getSnapshot(snapKeyB); - REQUIRE(actualA->size == snapA.size); - REQUIRE(actualB->size == snapB.size); + REQUIRE(actualA->getSize() == snapA->getSize()); + REQUIRE(actualB->getSize() == snapB->getSize()); - std::vector actualDataA(actualA->data, - actualA->data + dataA.size()); - std::vector actualDataB(actualB->data, - actualB->data + dataB.size()); + std::vector actualDataA = actualA->getDataCopy(); + std::vector actualDataB = actualB->getDataCopy(); REQUIRE(actualDataA == dataA); REQUIRE(actualDataB == dataB); @@ -116,11 +106,12 @@ TEST_CASE_METHOD(SnapshotClientServerFixture, void checkDiffsApplied(const uint8_t* snapBase, std::vector diffs) { - for (const auto& d : diffs) { - std::vector actual(snapBase + d.offset, - snapBase + d.offset + d.size); + for (auto& d : diffs) { + std::vector actual(snapBase + d.getOffset(), + snapBase + d.getOffset() + + d.getData().size()); - std::vector expected(d.data, d.data + d.size); + std::vector expected(d.getData().begin(), d.getData().end()); REQUIRE(actual == expected); } @@ -132,59 +123,87 @@ TEST_CASE_METHOD(SnapshotClientServerFixture, { std::string thisHost = getSystemConfig().endpointHost; - // One request with no group, another with a group we must initialise - int appId = 111; - int groupIdA = 0; - int groupIdB = 234; + // Set up a snapshot that's got enough memory to expand into + std::string snapKey = std::to_string(generateGid()); + size_t initialSnapSize = 5 * HOST_PAGE_SIZE; + size_t expandedSnapSize = 10 * HOST_PAGE_SIZE; - setUpFunctionGroup(appId, groupIdB); + auto snap = + std::make_shared(initialSnapSize, expandedSnapSize); - // Set up a snapshot - std::string snapKey = std::to_string(generateGid()); - SnapshotData snap = takeSnapshot(snapKey, 5, true); + // Set up the snapshot + reg.registerSnapshot(snapKey, snap); - // Set up some diffs + // Set up some diffs for the initial request + uint32_t offsetA1 = 5; + uint32_t offsetA2 = 2 * HOST_PAGE_SIZE; std::vector diffDataA1 = { 0, 1, 2, 3 }; std::vector diffDataA2 = { 4, 5, 6 }; - std::vector diffDataB = { 7, 7, 8, 8, 8 }; - - std::vector diffsA; - std::vector diffsB; - size_t originalDiffsApplied = server.diffsApplied(); + REQUIRE(snap->getQueuedDiffsCount() == 0); SnapshotDiff diffA1(SnapshotDataType::Raw, SnapshotMergeOperation::Overwrite, - 5, - diffDataA1.data(), - diffDataA1.size()); + offsetA1, + diffDataA1); SnapshotDiff diffA2(SnapshotDataType::Raw, SnapshotMergeOperation::Overwrite, - 2 * HOST_PAGE_SIZE, - diffDataA2.data(), - diffDataA2.size()); + offsetA2, + diffDataA2); - diffsA = { diffA1, diffA2 }; - cli.pushSnapshotDiffs(snapKey, groupIdA, diffsA); + std::vector diffsA = { diffA1, diffA2 }; + cli.pushSnapshotDiffs(snapKey, false, diffsA); + REQUIRE(snap->getQueuedDiffsCount() == 2); - SnapshotDiff diffB(SnapshotDataType::Raw, - SnapshotMergeOperation::Overwrite, - 3 * HOST_PAGE_SIZE, - diffDataB.data(), - diffDataB.size()); - diffsB = { diffB }; - cli.pushSnapshotDiffs(snapKey, groupIdB, diffsB); + // Submit some more diffs, some larger than the original snapshot (to check + // it gets extended) + uint32_t offsetB1 = 3 * HOST_PAGE_SIZE; + uint32_t offsetB2 = initialSnapSize + 10; + uint32_t offsetB3 = initialSnapSize + (3 * HOST_PAGE_SIZE); - // Ensure the right number of diffs is applied - // Also acts as a memory barrier for TSan - REQUIRE(server.diffsApplied() == originalDiffsApplied + 3); + std::vector diffDataB1 = { 7, 7, 8, 8, 8 }; + std::vector diffDataB2 = { 5, 5, 5, 5 }; + std::vector diffDataB3 = { 1, 1, 2, 2, 3, 3, 4, 4 }; + + SnapshotDiff diffB1(SnapshotDataType::Raw, + SnapshotMergeOperation::Overwrite, + offsetB1, + diffDataB1); + + SnapshotDiff diffB2(SnapshotDataType::Raw, + SnapshotMergeOperation::Overwrite, + offsetB2, + diffDataB2); + + SnapshotDiff diffB3(SnapshotDataType::Raw, + SnapshotMergeOperation::Overwrite, + offsetB3, + diffDataB3); - // Check changes have been applied - checkDiffsApplied(snap.data, diffsA); - checkDiffsApplied(snap.data, diffsB); + std::vector diffsB = { diffB1, diffB2, diffB3 }; - deallocatePages(snap.data, 5); + bool force = false; + SECTION("Force") { force = true; } + + SECTION("Don't force") { force = false; } + + // Make the request + cli.pushSnapshotDiffs(snapKey, force, diffsB); + + if (force) { + // Check nothing queued + REQUIRE(snap->getQueuedDiffsCount() == 0); + } else { + // Check and write queued diffs + REQUIRE(snap->getQueuedDiffsCount() == 5); + + snap->writeQueuedDiffs(); + } + + // Check diffs have been applied + checkDiffsApplied(snap->getDataPtr(), diffsA); + checkDiffsApplied(snap->getDataPtr(), diffsB); } TEST_CASE_METHOD(SnapshotClientServerFixture, @@ -193,7 +212,9 @@ TEST_CASE_METHOD(SnapshotClientServerFixture, { // Set up a snapshot std::string snapKey = std::to_string(generateGid()); - SnapshotData snap = takeSnapshot(snapKey, 5, false); + int snapSize = 5 * HOST_PAGE_SIZE; + auto snap = std::make_shared(snapSize); + reg.registerSnapshot(snapKey, snap); // Set up a couple of ints in the snapshot int offsetA1 = 8; @@ -201,10 +222,8 @@ TEST_CASE_METHOD(SnapshotClientServerFixture, int baseA1 = 25; int baseA2 = 60; - int* basePtrA1 = (int*)(snap.data + offsetA1); - int* basePtrA2 = (int*)(snap.data + offsetA2); - *basePtrA1 = baseA1; - *basePtrA2 = baseA2; + snap->copyInData({ BYTES(&baseA1), sizeof(int) }, offsetA1); + snap->copyInData({ BYTES(&baseA2), sizeof(int) }, offsetA2); // Set up some diffs with different merge operations int diffIntA1 = 123; @@ -215,36 +234,27 @@ TEST_CASE_METHOD(SnapshotClientServerFixture, std::vector diffs; - SnapshotDiff diffA1(SnapshotDataType::Raw, - SnapshotMergeOperation::Overwrite, - offsetA1, - intDataA1.data(), - intDataA1.size()); - diffA1.operation = SnapshotMergeOperation::Sum; - diffA1.dataType = SnapshotDataType::Int; + SnapshotDiff diffA1( + SnapshotDataType::Int, SnapshotMergeOperation::Sum, offsetA1, intDataA1); - SnapshotDiff diffA2(SnapshotDataType::Raw, - SnapshotMergeOperation::Overwrite, - offsetA2, - intDataA2.data(), - intDataA2.size()); - diffA2.operation = SnapshotMergeOperation::Sum; - diffA2.dataType = SnapshotDataType::Int; + SnapshotDiff diffA2( + SnapshotDataType::Int, SnapshotMergeOperation::Sum, offsetA2, intDataA2); - size_t originalDiffsApplied = server.diffsApplied(); + size_t originalDiffsApplied = snap->getQueuedDiffsCount(); diffs = { diffA1, diffA2 }; - cli.pushSnapshotDiffs(snapKey, 0, diffs); + cli.pushSnapshotDiffs(snapKey, false, diffs); // Ensure the right number of diffs is applied - // Also acts as a memory barrier for TSan - REQUIRE(server.diffsApplied() == originalDiffsApplied + 2); - - // Check diffs have been applied according to the merge operations - REQUIRE(*basePtrA1 == baseA1 + diffIntA1); - REQUIRE(*basePtrA2 == baseA2 + diffIntA2); - - deallocatePages(snap.data, 5); + REQUIRE(snap->getQueuedDiffsCount() == originalDiffsApplied + 2); + + // Write and check diffs have been applied according to the merge operations + snap->writeQueuedDiffs(); + const uint8_t* rawSnapData = snap->getDataPtr(); + int actualA1 = faabric::util::unalignedRead(rawSnapData + offsetA1); + int actualA2 = faabric::util::unalignedRead(rawSnapData + offsetA2); + REQUIRE(actualA1 == baseA1 + diffIntA1); + REQUIRE(actualA2 == baseA2 + diffIntA2); } TEST_CASE_METHOD(SnapshotClientServerFixture, @@ -253,7 +263,7 @@ TEST_CASE_METHOD(SnapshotClientServerFixture, { // Set up a snapshot std::string snapKey = std::to_string(generateGid()); - SnapshotData snap = takeSnapshot(snapKey, 5, false); + auto snap = setUpSnapshot(snapKey, 5); int offset = 8; std::vector originalData; @@ -341,31 +351,23 @@ TEST_CASE_METHOD(SnapshotClientServerFixture, } // Put original data in place - std::memcpy(snap.data + offset, originalData.data(), originalData.size()); + snap->copyInData(originalData, offset); - SnapshotDiff diff(SnapshotDataType::Raw, - SnapshotMergeOperation::Overwrite, - offset, - diffData.data(), - diffData.size()); - diff.operation = operation; - diff.dataType = dataType; + SnapshotDiff diff(dataType, operation, offset, diffData); - size_t originalDiffsApplied = server.diffsApplied(); + size_t originalDiffsApplied = snap->getQueuedDiffsCount(); std::vector diffs = { diff }; - cli.pushSnapshotDiffs(snapKey, 0, diffs); + cli.pushSnapshotDiffs(snapKey, false, diffs); // Ensure the right number of diffs is applied - // Also acts as a memory barrier for TSan - REQUIRE(server.diffsApplied() == originalDiffsApplied + 1); + REQUIRE(snap->getQueuedDiffsCount() == originalDiffsApplied + 1); - // Check data is as expected - std::vector actualData(snap.data + offset, - snap.data + offset + expectedData.size()); + // Apply and check data is as expected + snap->writeQueuedDiffs(); + std::vector actualData = + snap->getDataCopy(offset, expectedData.size()); REQUIRE(actualData == expectedData); - - deallocatePages(snap.data, 5); } TEST_CASE_METHOD(SnapshotClientServerFixture, diff --git a/tests/test/snapshot/test_snapshot_diffs.cpp b/tests/test/snapshot/test_snapshot_diffs.cpp index 8a3987d9e..05ac925f8 100644 --- a/tests/test/snapshot/test_snapshot_diffs.cpp +++ b/tests/test/snapshot/test_snapshot_diffs.cpp @@ -14,11 +14,12 @@ void checkSnapshotDiff(int offset, std::vector data, SnapshotDiff& actual) { - REQUIRE(offset == actual.offset); - REQUIRE(actual.size > 0); - REQUIRE(actual.data != nullptr); + REQUIRE(offset == actual.getOffset()); + REQUIRE(!actual.getData().empty()); + REQUIRE(actual.getData().data() != nullptr); - std::vector actualData(actual.data, actual.data + actual.size); + std::vector actualData(actual.getData().begin(), + actual.getData().end()); REQUIRE(data == actualData); } @@ -28,13 +29,16 @@ TEST_CASE_METHOD(SnapshotTestFixture, { std::string snapKey = "foobar123"; int snapPages = 5; - SnapshotData snap = takeSnapshot(snapKey, snapPages, true); + + size_t snapSize = snapPages * faabric::util::HOST_PAGE_SIZE; + auto snap = std::make_shared(snapSize); + reg.registerSnapshot(snapKey, snap); int sharedMemPages = 8; size_t sharedMemSize = sharedMemPages * HOST_PAGE_SIZE; - uint8_t* sharedMem = allocatePages(sharedMemPages); + MemoryRegion sharedMem = allocateSharedMemory(sharedMemSize); - reg.mapSnapshot(snapKey, sharedMem); + reg.mapSnapshot(snapKey, sharedMem.get()); // Make various changes sharedMem[0] = 1; @@ -44,7 +48,7 @@ TEST_CASE_METHOD(SnapshotTestFixture, // Check there are no diffs std::vector changeDiffs = - snap.getChangeDiffs(sharedMem, sharedMemSize); + MemoryView({ sharedMem.get(), sharedMemSize }).diffWithSnapshot(snap); REQUIRE(changeDiffs.empty()); } @@ -53,15 +57,18 @@ TEST_CASE_METHOD(SnapshotTestFixture, "Test snapshot diffs", "[snapshot]") std::string snapKey = "foobar123"; int snapPages = 5; size_t snapSize = snapPages * HOST_PAGE_SIZE; - SnapshotData snap = takeSnapshot(snapKey, snapPages, true); + + auto snap = std::make_shared(snapSize); + reg.registerSnapshot(snapKey, snap); // Make shared memory larger than original snapshot int sharedMemPages = 8; size_t sharedMemSize = sharedMemPages * HOST_PAGE_SIZE; - uint8_t* sharedMem = allocatePages(sharedMemPages); + MemoryRegion sharedMem = + allocateSharedMemory(sharedMemPages * HOST_PAGE_SIZE); // Map the snapshot to the start of the memory - reg.mapSnapshot(snapKey, sharedMem); + reg.mapSnapshot(snapKey, sharedMem.get()); // Reset dirty tracking faabric::util::resetDirtyTracking(); @@ -69,51 +76,51 @@ TEST_CASE_METHOD(SnapshotTestFixture, "Test snapshot diffs", "[snapshot]") // Single change, single merge region std::vector dataA = { 1, 2, 3, 4 }; int offsetA = HOST_PAGE_SIZE; - std::memcpy(sharedMem + offsetA, dataA.data(), dataA.size()); + std::memcpy(sharedMem.get() + offsetA, dataA.data(), dataA.size()); - snap.addMergeRegion(offsetA, - dataA.size(), - SnapshotDataType::Raw, - SnapshotMergeOperation::Overwrite); + snap->addMergeRegion(offsetA, + dataA.size(), + SnapshotDataType::Raw, + SnapshotMergeOperation::Overwrite); // NOTE - deliberately add merge regions out of order // Diff starting in merge region and overlapping the end std::vector dataC = { 7, 6, 5, 4, 3, 2, 1 }; std::vector expectedDataC = { 7, 6, 5, 4 }; int offsetC = 2 * HOST_PAGE_SIZE; - std::memcpy(sharedMem + offsetC, dataC.data(), dataC.size()); + std::memcpy(sharedMem.get() + offsetC, dataC.data(), dataC.size()); int regionOffsetC = offsetC - 3; - snap.addMergeRegion(regionOffsetC, - dataC.size(), - SnapshotDataType::Raw, - SnapshotMergeOperation::Overwrite); + snap->addMergeRegion(regionOffsetC, + dataC.size(), + SnapshotDataType::Raw, + SnapshotMergeOperation::Overwrite); // Two changes in single merge region std::vector dataB1 = { 4, 5, 6 }; std::vector dataB2 = { 7, 6, 5 }; int offsetB1 = HOST_PAGE_SIZE + 10; int offsetB2 = HOST_PAGE_SIZE + 16; - std::memcpy(sharedMem + offsetB1, dataB1.data(), dataB1.size()); - std::memcpy(sharedMem + offsetB2, dataB2.data(), dataB2.size()); + std::memcpy(sharedMem.get() + offsetB1, dataB1.data(), dataB1.size()); + std::memcpy(sharedMem.get() + offsetB2, dataB2.data(), dataB2.size()); - snap.addMergeRegion(offsetB1, - (offsetB2 - offsetB1) + dataB2.size() + 10, - SnapshotDataType::Raw, - SnapshotMergeOperation::Overwrite); + snap->addMergeRegion(offsetB1, + (offsetB2 - offsetB1) + dataB2.size() + 10, + SnapshotDataType::Raw, + SnapshotMergeOperation::Overwrite); // Merge region within a change std::vector dataD = { 1, 1, 2, 2, 3, 3, 4 }; std::vector expectedDataD = { 2, 2, 3 }; int offsetD = 3 * HOST_PAGE_SIZE - dataD.size(); - std::memcpy(sharedMem + offsetD, dataD.data(), dataD.size()); + std::memcpy(sharedMem.get() + offsetD, dataD.data(), dataD.size()); int regionOffsetD = offsetD + 2; int regionSizeD = dataD.size() - 4; - snap.addMergeRegion(regionOffsetD, - regionSizeD, - SnapshotDataType::Raw, - SnapshotMergeOperation::Overwrite); + snap->addMergeRegion(regionOffsetD, + regionSizeD, + SnapshotDataType::Raw, + SnapshotMergeOperation::Overwrite); // Write some data to the region that exceeds the size of the original, then // add a merge region larger than it. Anything outside the original snapshot @@ -121,34 +128,33 @@ TEST_CASE_METHOD(SnapshotTestFixture, "Test snapshot diffs", "[snapshot]") std::vector dataExtra = { 2, 2, 2 }; std::vector expectedDataExtra = { 0, 0, 2, 2, 2, 0, 0 }; int extraOffset = snapSize + HOST_PAGE_SIZE + 10; - std::memcpy(sharedMem + extraOffset, dataExtra.data(), dataExtra.size()); + std::memcpy( + sharedMem.get() + extraOffset, dataExtra.data(), dataExtra.size()); int extraRegionOffset = extraOffset - 2; int extraRegionSize = dataExtra.size() + 4; - snap.addMergeRegion(extraRegionOffset, - extraRegionSize, - SnapshotDataType::Raw, - SnapshotMergeOperation::Overwrite); + snap->addMergeRegion(extraRegionOffset, + extraRegionSize, + SnapshotDataType::Raw, + SnapshotMergeOperation::Overwrite); - // Include an offset which doesn't change the data, but will register a - // dirty page + // Include an offset which doesn't change the data.get(), but will register + // a dirty page std::vector dataNoChange = { 0, 0, 0 }; int offsetNoChange = 4 * HOST_PAGE_SIZE - 10; - std::memcpy( - sharedMem + offsetNoChange, dataNoChange.data(), dataNoChange.size()); - - // Check original has no dirty pages - REQUIRE(snap.getDirtyPages().empty()); + std::memcpy(sharedMem.get() + offsetNoChange, + dataNoChange.data(), + dataNoChange.size()); // Check shared memory does have dirty pages (including the non-change) std::vector sharedDirtyPages = - getDirtyPageNumbers(sharedMem, sharedMemPages); + getDirtyPageNumbers(sharedMem.get(), sharedMemPages); std::vector expected = { 1, 2, 3, 6 }; REQUIRE(sharedDirtyPages == expected); // Check we have the right number of diffs std::vector changeDiffs = - snap.getChangeDiffs(sharedMem, sharedMemSize); + MemoryView({ sharedMem.get(), sharedMemSize }).diffWithSnapshot(snap); REQUIRE(changeDiffs.size() == 6); diff --git a/tests/test/snapshot/test_snapshot_registry.cpp b/tests/test/snapshot/test_snapshot_registry.cpp index 2ae4e14cb..f919618d8 100644 --- a/tests/test/snapshot/test_snapshot_registry.cpp +++ b/tests/test/snapshot/test_snapshot_registry.cpp @@ -1,6 +1,7 @@ -#include "faabric_utils.h" #include +#include "faabric_utils.h" + #include #include @@ -25,70 +26,63 @@ TEST_CASE_METHOD(SnapshotTestFixture, REQUIRE(!reg.snapshotExists(keyB)); REQUIRE(!reg.snapshotExists(keyC)); - SnapshotData snapA = takeSnapshot(keyA, 1, true); - SnapshotData snapB = takeSnapshot(keyB, 2, false); + auto snapA = setUpSnapshot(keyA, 1); + auto snapB = setUpSnapshot(keyB, 2); REQUIRE(reg.snapshotExists(keyA)); REQUIRE(reg.snapshotExists(keyB)); REQUIRE(!reg.snapshotExists(keyC)); REQUIRE(reg.getSnapshotCount() == 2); - SnapshotData snapC = takeSnapshot(keyC, 3, true); + auto snapC = setUpSnapshot(keyC, 3); REQUIRE(reg.snapshotExists(keyA)); REQUIRE(reg.snapshotExists(keyB)); REQUIRE(reg.snapshotExists(keyC)); REQUIRE(reg.getSnapshotCount() == 3); - // Add some random bits of data to the vectors + // Add some random bits of data to the snapshots for (int i = 0; i < HOST_PAGE_SIZE - 10; i += 50) { - snapA.data[i] = i; - snapB.data[i + 1] = i; - snapC.data[i + 2] = i; + snapA->copyInData({ BYTES(&i), sizeof(int) }, i); + snapB->copyInData({ BYTES(&i), sizeof(int) }, i + 1); + snapC->copyInData({ BYTES(&i), sizeof(int) }, i + 2); } // Take snapshots again with updated data - reg.takeSnapshot(keyA, snapA); - reg.takeSnapshot(keyB, snapB, false); - reg.takeSnapshot(keyC, snapC); + reg.registerSnapshot(keyA, snapA); + reg.registerSnapshot(keyB, snapB); + reg.registerSnapshot(keyC, snapC); auto actualA = reg.getSnapshot(keyA); auto actualB = reg.getSnapshot(keyB); auto actualC = reg.getSnapshot(keyC); - REQUIRE(actualA->size == snapA.size); - REQUIRE(actualB->size == snapB.size); - REQUIRE(actualC->size == snapC.size); + REQUIRE(actualA->getSize() == snapA->getSize()); + REQUIRE(actualB->getSize() == snapB->getSize()); + REQUIRE(actualC->getSize() == snapC->getSize()); // Pointer equality here is good enough - REQUIRE(actualA->data == snapA.data); - REQUIRE(actualB->data == snapB.data); - REQUIRE(actualC->data == snapC.data); - - REQUIRE(actualA->fd > 0); - REQUIRE(actualB->fd == 0); - REQUIRE(actualC->fd > 0); + REQUIRE(actualA->getDataPtr() == snapA->getDataPtr()); + REQUIRE(actualB->getDataPtr() == snapB->getDataPtr()); + REQUIRE(actualC->getDataPtr() == snapC->getDataPtr()); // Create regions onto which we will map the snapshots - uint8_t* actualDataA = allocatePages(1); - uint8_t* actualDataB = allocatePages(2); - uint8_t* actualDataC = allocatePages(3); - - // Check those that are mappable are mapped - reg.mapSnapshot(keyA, actualDataA); - reg.mapSnapshot(keyC, actualDataC); + MemoryRegion actualDataA = allocateSharedMemory(1 * HOST_PAGE_SIZE); + MemoryRegion actualDataB = allocateSharedMemory(2 * HOST_PAGE_SIZE); + MemoryRegion actualDataC = allocateSharedMemory(3 * HOST_PAGE_SIZE); - // Check error when mapping an unmappable snapshot - REQUIRE_THROWS(reg.mapSnapshot(keyB, actualDataB)); + // Map two of them + reg.mapSnapshot(keyA, actualDataA.get()); + reg.mapSnapshot(keyC, actualDataC.get()); // Here we need to check the actual data after mapping - std::vector vecDataA(snapA.data, snapA.data + HOST_PAGE_SIZE); - std::vector vecActualDataA(actualDataA, - actualDataA + HOST_PAGE_SIZE); - std::vector vecDataC(snapC.data, - snapC.data + (3 * HOST_PAGE_SIZE)); - std::vector vecActualDataC(actualDataC, - actualDataC + (3 * HOST_PAGE_SIZE)); + std::vector vecDataA = snapA->getDataCopy(); + std::vector vecActualDataA(actualDataA.get(), + actualDataA.get() + HOST_PAGE_SIZE); + + std::vector vecDataC = snapC->getDataCopy(); + std::vector vecActualDataC( + actualDataC.get(), actualDataC.get() + (3 * HOST_PAGE_SIZE)); REQUIRE(vecActualDataA == vecDataA); REQUIRE(vecActualDataC == vecDataC); @@ -107,14 +101,10 @@ TEST_CASE_METHOD(SnapshotTestFixture, REQUIRE(!reg.snapshotExists(keyB)); REQUIRE(!reg.snapshotExists(keyC)); REQUIRE(reg.getSnapshotCount() == 0); - - deallocatePages(actualDataA, 1); - deallocatePages(actualDataB, 2); - deallocatePages(actualDataC, 3); } TEST_CASE_METHOD(SnapshotTestFixture, - "Test set snapshot if not exists", + "Test register snapshot if not exists", "[snapshot]") { REQUIRE(reg.getSnapshotCount() == 0); @@ -126,40 +116,33 @@ TEST_CASE_METHOD(SnapshotTestFixture, REQUIRE(!reg.snapshotExists(keyB)); // Take one of the snapshots - SnapshotData snapBefore = takeSnapshot(keyA, 1, true); + auto snapBefore = setUpSnapshot(keyA, 1); REQUIRE(reg.snapshotExists(keyA)); REQUIRE(!reg.snapshotExists(keyB)); REQUIRE(reg.getSnapshotCount() == 1); - // Set up some different data - std::vector otherDataA(snapBefore.size + 10, 1); - std::vector otherDataB(snapBefore.size + 5, 2); - - SnapshotData snapUpdateA; - snapUpdateA.data = otherDataA.data(); - snapUpdateA.size = otherDataA.size(); - - SnapshotData snapUpdateB; - snapUpdateB.data = otherDataB.data(); - snapUpdateB.size = otherDataB.size(); + auto otherSnap = + std::make_shared(snapBefore->getSize() + 10); + std::vector otherData(snapBefore->getSize() + 10, 1); + otherSnap->copyInData(otherData); // Check existing snapshot is not overwritten - reg.takeSnapshotIfNotExists(keyA, snapUpdateA, true); + reg.registerSnapshotIfNotExists(keyA, otherSnap); auto snapAfterA = reg.getSnapshot(keyA); - REQUIRE(snapAfterA->data == snapBefore.data); - REQUIRE(snapAfterA->size == snapBefore.size); + REQUIRE(snapAfterA->getDataPtr() == snapBefore->getDataPtr()); + REQUIRE(snapAfterA->getSize() == snapBefore->getSize()); // Check new snapshot is still created - reg.takeSnapshotIfNotExists(keyB, snapUpdateB, true); + reg.registerSnapshotIfNotExists(keyB, otherSnap); REQUIRE(reg.snapshotExists(keyA)); REQUIRE(reg.snapshotExists(keyB)); REQUIRE(reg.getSnapshotCount() == 2); auto snapAfterB = reg.getSnapshot(keyB); - REQUIRE(snapAfterB->data == otherDataB.data()); - REQUIRE(snapAfterB->size == otherDataB.size()); + REQUIRE(snapAfterB->getDataPtr() == otherSnap->getDataPtr()); + REQUIRE(snapAfterB->getSize() == otherSnap->getSize()); } TEST_CASE_METHOD(SnapshotTestFixture, diff --git a/tests/test/util/test_memory.cpp b/tests/test/util/test_memory.cpp index 298396ae7..d49c3684b 100644 --- a/tests/test/util/test_memory.cpp +++ b/tests/test/util/test_memory.cpp @@ -1,6 +1,9 @@ #include + #include #include + +#include #include #include @@ -352,4 +355,91 @@ TEST_CASE("Test dirty region checking", "[util]") REQUIRE(actual.at(i).second == expected.at(i).second); } } + +TEST_CASE("Test allocating and claiming memory", "[util]") +{ + // Allocate some virtual memory + size_t vMemSize = 100 * HOST_PAGE_SIZE; + MemoryRegion vMem = allocateVirtualMemory(vMemSize); + + size_t sizeA = 10 * HOST_PAGE_SIZE; + claimVirtualMemory({ vMem.get(), sizeA }); + + // Write something to the new memory + vMem[10] = 1; + vMem[5 * HOST_PAGE_SIZE] = 2; + vMem[6 * HOST_PAGE_SIZE + 10] = 3; + + size_t sizeB = 5 * HOST_PAGE_SIZE; + claimVirtualMemory({ vMem.get() + sizeA, sizeB }); + + // Write something to the new memory + vMem[sizeA + 10] = 4; + vMem[sizeA + 3 * HOST_PAGE_SIZE] = 5; + vMem[sizeA + 4 * HOST_PAGE_SIZE + 10] = 6; + + // Check all edits still there + REQUIRE(vMem[10] == 1); + REQUIRE(vMem[5 * HOST_PAGE_SIZE] == 2); + REQUIRE(vMem[6 * HOST_PAGE_SIZE + 10] == 3); + REQUIRE(vMem[sizeA + 10] == 4); + REQUIRE(vMem[sizeA + 3 * HOST_PAGE_SIZE] == 5); + REQUIRE(vMem[sizeA + 4 * HOST_PAGE_SIZE + 10] == 6); +} + +TEST_CASE("Test mapping memory", "[util]") +{ + size_t vMemSize = 100 * HOST_PAGE_SIZE; + MemoryRegion vMem = allocateVirtualMemory(vMemSize); + + // Set up some data in memory + std::vector chunk(10 * HOST_PAGE_SIZE, 3); + claimVirtualMemory({ vMem.get(), chunk.size() }); + std::memcpy(vMem.get(), chunk.data(), chunk.size()); + + // Write this to a file descriptor + int fd = createFd(chunk.size(), "foobar"); + writeToFd(fd, 0, { vMem.get(), chunk.size() }); + + // Map some new memory to this fd + MemoryRegion memA = allocateSharedMemory(chunk.size()); + mapMemoryPrivate({ memA.get(), chunk.size() }, fd); + + std::vector memAData(memA.get(), memA.get() + chunk.size()); + REQUIRE(memAData == chunk); + + // Extend the memory and copy some new data in + std::vector chunkB(5 * HOST_PAGE_SIZE, 4); + claimVirtualMemory({ vMem.get() + chunk.size(), chunkB.size() }); + std::memcpy(vMem.get() + chunk.size(), chunkB.data(), chunkB.size()); + + // Append the data to the fd + appendDataToFd(fd, { chunkB.data(), chunkB.size() }); + + // Map a region to both chunks + MemoryRegion memB = allocateSharedMemory(chunk.size() + chunkB.size()); + mapMemoryPrivate({ memB.get(), chunk.size() + chunkB.size() }, fd); + + // Check region now contains both bits of data + std::vector memBData(memB.get(), + memB.get() + chunk.size() + chunkB.size()); + std::vector expected; + expected.insert(expected.end(), chunk.begin(), chunk.end()); + expected.insert(expected.end(), chunkB.begin(), chunkB.end()); + + REQUIRE(memBData == expected); +} + +TEST_CASE("Test mapping memory fails with invalid fd", "[util]") +{ + size_t memSize = 10 * HOST_PAGE_SIZE; + MemoryRegion sharedMem = allocateSharedMemory(memSize); + + int fd = 0; + SECTION("Zero fd") { fd = 0; } + + SECTION("Negative fd") { fd = -2; } + + REQUIRE_THROWS(mapMemoryPrivate({ sharedMem.get(), memSize }, fd)); +} } diff --git a/tests/test/util/test_snapshot.cpp b/tests/test/util/test_snapshot.cpp index ddad687e1..2fcf9d169 100644 --- a/tests/test/util/test_snapshot.cpp +++ b/tests/test/util/test_snapshot.cpp @@ -1,5 +1,6 @@ #include +#include "faabric/snapshot/SnapshotRegistry.h" #include "faabric_utils.h" #include "fixtures.h" @@ -16,29 +17,22 @@ class SnapshotMergeTestFixture : public SnapshotTestFixture { public: SnapshotMergeTestFixture() = default; + ~SnapshotMergeTestFixture() = default; protected: - std::string snapKey; - faabric::util::SnapshotData snap; - - uint8_t* setUpSnapshot(int snapPages, int sharedMemPages) - { - snapKey = "foobar123"; - snap.size = snapPages * faabric::util::HOST_PAGE_SIZE; - snap.data = allocatePages(snapPages); + std::string snapKey = "foobar123"; - // Take the snapshot - reg.takeSnapshot(snapKey, snap, true); + std::shared_ptr setUpSnapshot(int snapPages, + int sharedMemPages) - // Map the snapshot - uint8_t* sharedMem = allocatePages(sharedMemPages); - reg.mapSnapshot(snapKey, sharedMem); + { + auto snapData = + std::make_shared(snapPages * HOST_PAGE_SIZE); - // Reset dirty tracking - faabric::util::resetDirtyTracking(); + reg.registerSnapshot(snapKey, snapData); - return sharedMem; + return snapData; } void checkDiffs(std::vector& actualDiffs, @@ -47,22 +41,266 @@ class SnapshotMergeTestFixture : public SnapshotTestFixture REQUIRE(actualDiffs.size() == expectedDiffs.size()); for (int i = 0; i < actualDiffs.size(); i++) { - SnapshotDiff actualDiff = actualDiffs.at(i); - SnapshotDiff expectedDiff = expectedDiffs.at(i); + SnapshotDiff& actualDiff = actualDiffs.at(i); + SnapshotDiff& expectedDiff = expectedDiffs.at(i); - REQUIRE(actualDiff.operation == expectedDiff.operation); - REQUIRE(actualDiff.dataType == expectedDiff.dataType); - REQUIRE(actualDiff.offset == expectedDiff.offset); + REQUIRE(actualDiff.getOperation() == expectedDiff.getOperation()); + REQUIRE(actualDiff.getDataType() == expectedDiff.getDataType()); + REQUIRE(actualDiff.getOffset() == expectedDiff.getOffset()); - std::vector actualData(actualDiff.data, - actualDiff.data + actualDiff.size); - std::vector expectedData( - expectedDiff.data, expectedDiff.data + expectedDiff.size); + std::vector actualData = actualDiff.getDataCopy(); + std::vector expectedData = expectedDiff.getDataCopy(); REQUIRE(actualData == expectedData); } } }; +TEST_CASE_METHOD(SnapshotMergeTestFixture, + "Test snapshot diff operations", + "[snapshot][util]") +{ + std::vector dataA(100, 1); + + std::vector dataB(2 * HOST_PAGE_SIZE, 2); + MemoryRegion memB = allocateSharedMemory(dataB.size()); + std::memcpy(memB.get(), dataB.data(), dataB.size()); + + std::vector dataC(sizeof(int), 3); + + uint32_t offsetA = 10; + uint32_t offsetB = 5 * HOST_PAGE_SIZE; + uint32_t offsetC = 10 * HOST_PAGE_SIZE; + + SnapshotDiff diffA( + SnapshotDataType::Raw, SnapshotMergeOperation::Overwrite, offsetA, dataA); + + SnapshotDiff diffB(SnapshotDataType::Raw, + SnapshotMergeOperation::Overwrite, + offsetB, + std::span(memB.get(), dataB.size())); + + SnapshotDiff diffC( + SnapshotDataType::Int, SnapshotMergeOperation::Sum, offsetC, dataC); + + REQUIRE(diffA.getOffset() == offsetA); + REQUIRE(diffB.getOffset() == offsetB); + REQUIRE(diffC.getOffset() == offsetC); + + REQUIRE(diffA.getDataType() == SnapshotDataType::Raw); + REQUIRE(diffB.getDataType() == SnapshotDataType::Raw); + REQUIRE(diffC.getDataType() == SnapshotDataType::Int); + + REQUIRE(diffA.getOperation() == SnapshotMergeOperation::Overwrite); + REQUIRE(diffB.getOperation() == SnapshotMergeOperation::Overwrite); + REQUIRE(diffC.getOperation() == SnapshotMergeOperation::Sum); + + REQUIRE(diffA.getDataCopy() == dataA); + REQUIRE(diffB.getDataCopy() == dataB); + REQUIRE(diffC.getDataCopy() == dataC); +} + +TEST_CASE_METHOD(SnapshotMergeTestFixture, + "Test clear merge regions", + "[snapshot][util]") +{ + int snapPages = 20; + setUpSnapshot(snapPages, snapPages); + auto snap = reg.getSnapshot(snapKey); + + int nRegions = 10; + for (int i = 0; i < nRegions; i++) { + snap->addMergeRegion(i * HOST_PAGE_SIZE, + sizeof(int), + SnapshotDataType::Int, + SnapshotMergeOperation::Sum); + } + + REQUIRE(snap->getMergeRegions().size() == nRegions); + + snap->clearMergeRegions(); + + REQUIRE(snap->getMergeRegions().empty()); +} + +TEST_CASE_METHOD(SnapshotMergeTestFixture, + "Test mapping snapshot memory", + "[snapshot][util]") +{ + int snapPages = 10; + auto snap = std::make_shared(snapPages * HOST_PAGE_SIZE); + + // Put some data into the snapshot + std::vector dataA(100, 2); + std::vector dataB(300, 4); + + snap->copyInData(dataA, 2 * HOST_PAGE_SIZE); + snap->copyInData(dataB, 5 * HOST_PAGE_SIZE + 2); + + // Record the snap memory + std::vector actualSnapMem = snap->getDataCopy(); + + // Set up shared memory + int sharedMemPages = 20; + MemoryRegion sharedMem = + allocateSharedMemory(sharedMemPages * HOST_PAGE_SIZE); + + // Check it's zeroed + std::vector expectedInitial(snap->getSize(), 0); + std::vector actualSharedMemBefore( + sharedMem.get(), sharedMem.get() + snap->getSize()); + REQUIRE(actualSharedMemBefore == expectedInitial); + + // Map the snapshot and check again + snap->mapToMemory(sharedMem.get()); + std::vector actualSharedMemAfter( + sharedMem.get(), sharedMem.get() + snap->getSize()); + REQUIRE(actualSharedMemAfter == actualSnapMem); +} + +TEST_CASE_METHOD(SnapshotMergeTestFixture, + "Test mapping editing and remapping memory", + "[snapshot][util]") +{ + int snapPages = 4; + size_t snapSize = snapPages * HOST_PAGE_SIZE; + auto snap = std::make_shared(snapSize); + + std::vector initialData(100, 2); + std::vector dataA(150, 3); + std::vector dataB(200, 4); + + // Deliberately use offsets on the same page to check copy-on-write mappings + uint32_t initialOffset = 0; + uint32_t offsetA = HOST_PAGE_SIZE; + uint32_t offsetB = HOST_PAGE_SIZE + dataA.size() + 1; + + // Set up some initial data + snap->copyInData(initialData, initialOffset); + std::vector expectedSnapMem = snap->getDataCopy(); + + // Set up two shared mem regions + MemoryRegion sharedMemA = allocateSharedMemory(snapSize); + MemoryRegion sharedMemB = allocateSharedMemory(snapSize); + + // Map the snapshot and both regions reflect the change + snap->mapToMemory(sharedMemA.get()); + snap->mapToMemory(sharedMemB.get()); + + REQUIRE(std::vector(sharedMemA.get(), sharedMemA.get() + snapSize) == + expectedSnapMem); + REQUIRE(std::vector(sharedMemB.get(), sharedMemB.get() + snapSize) == + expectedSnapMem); + + // Reset dirty tracking + faabric::util::resetDirtyTracking(); + + // Make different edits to both mapped regions, check they are not + // propagated back to the snapshot + std::memcpy(sharedMemA.get() + offsetA, dataA.data(), dataA.size()); + std::memcpy(sharedMemB.get() + offsetB, dataB.data(), dataB.size()); + + std::vector actualSnapMem = snap->getDataCopy(); + REQUIRE(actualSnapMem == expectedSnapMem); + + // Set two separate merge regions to cover both changes + snap->addMergeRegion(offsetA, + dataA.size(), + SnapshotDataType::Raw, + SnapshotMergeOperation::Overwrite); + + snap->addMergeRegion(offsetB, + dataB.size(), + SnapshotDataType::Raw, + SnapshotMergeOperation::Overwrite); + + // Apply diffs from both snapshots + std::vector diffsA = + MemoryView({ sharedMemA.get(), snapSize }).diffWithSnapshot(snap); + std::vector diffsB = + MemoryView({ sharedMemB.get(), snapSize }).diffWithSnapshot(snap); + + REQUIRE(diffsA.size() == 1); + SnapshotDiff& diffA = diffsA.front(); + REQUIRE(diffA.getData().size() == dataA.size()); + REQUIRE(diffA.getOffset() == offsetA); + + REQUIRE(diffsB.size() == 1); + SnapshotDiff& diffB = diffsB.front(); + REQUIRE(diffB.getData().size() == dataB.size()); + REQUIRE(diffB.getOffset() == offsetB); + + snap->queueDiffs(diffsA); + REQUIRE(snap->getQueuedDiffsCount() == diffsA.size()); + + snap->queueDiffs(diffsB); + REQUIRE(snap->getQueuedDiffsCount() == diffsA.size() + diffsB.size()); + + snap->writeQueuedDiffs(); + + // Make sure snapshot now includes both + std::memcpy(expectedSnapMem.data() + offsetA, dataA.data(), dataA.size()); + std::memcpy(expectedSnapMem.data() + offsetB, dataB.data(), dataB.size()); + + actualSnapMem = snap->getDataCopy(); + REQUIRE(actualSnapMem == expectedSnapMem); +} + +TEST_CASE_METHOD(SnapshotMergeTestFixture, + "Test growing snapshots", + "[snapshot][util]") +{ + // Set up the snapshot + int originalPages = 10; + size_t originalSize = originalPages * HOST_PAGE_SIZE; + int expandedPages = 20; + size_t expandedSize = expandedPages * HOST_PAGE_SIZE; + + std::shared_ptr snap = + std::make_shared(originalSize, expandedSize); + + // Take the snapsho at the original size + reg.registerSnapshot(snapKey, snap); + + // Put some data into the snapshot + std::vector dataA(100, 2); + std::vector dataB(300, 4); + + snap->copyInData(dataA, 2 * HOST_PAGE_SIZE); + snap->copyInData(dataB, (5 * HOST_PAGE_SIZE) + 2); + + std::vector originalData = snap->getDataCopy(); + REQUIRE(originalData.size() == originalSize); + + // Map to some other region of memory large enough for the extended version + MemoryRegion sharedMem = allocateSharedMemory(expandedSize); + snap->mapToMemory(sharedMem.get()); + + // Add some data to the extended region. Check the snapshot extends to fit + std::vector dataC(300, 5); + std::vector dataD(200, 6); + uint32_t extendedOffsetA = (originalPages + 3) * HOST_PAGE_SIZE; + uint32_t extendedOffsetB = (originalPages + 5) * HOST_PAGE_SIZE; + + snap->copyInData(dataC, extendedOffsetA); + size_t expectedSizeA = extendedOffsetA + dataC.size(); + REQUIRE(snap->getSize() == expectedSizeA); + + snap->copyInData(dataD, extendedOffsetB); + size_t expectedSizeB = extendedOffsetB + dataD.size(); + REQUIRE(snap->getSize() == expectedSizeB); + + // Remap to shared memory + snap->mapToMemory(sharedMem.get()); + + // Check mapped region matches + std::vector actualData = snap->getDataCopy(); + std::vector actualSharedMem(sharedMem.get(), + sharedMem.get() + snap->getSize()); + + REQUIRE(actualSharedMem.size() == actualData.size()); + REQUIRE(actualSharedMem == actualData); +} + TEST_CASE_METHOD(SnapshotMergeTestFixture, "Detailed test snapshot merge regions with ints", "[snapshot][util]") @@ -78,32 +316,27 @@ TEST_CASE_METHOD(SnapshotMergeTestFixture, int finalValueB = 650123; int sumValueB = 350123; - faabric::util::SnapshotData snap; - snap.size = snapPages * faabric::util::HOST_PAGE_SIZE; - snap.data = allocatePages(snapPages); + size_t memSize = snapPages * HOST_PAGE_SIZE; + auto snap = std::make_shared(memSize); // Set up some integers in the snapshot int intAOffset = HOST_PAGE_SIZE + (10 * sizeof(int32_t)); int intBOffset = (2 * HOST_PAGE_SIZE) + (20 * sizeof(int32_t)); - int* intAOriginal = (int*)(snap.data + intAOffset); - int* intBOriginal = (int*)(snap.data + intBOffset); // Set the original values - *intAOriginal = originalValueA; - *intBOriginal = originalValueB; + snap->copyInData({ BYTES(&originalValueA), sizeof(int) }, intAOffset); + snap->copyInData({ BYTES(&originalValueB), sizeof(int) }, intBOffset); // Take the snapshot - reg.takeSnapshot(snapKey, snap, true); + reg.registerSnapshot(snapKey, snap); // Map the snapshot to some memory - size_t sharedMemSize = snapPages * HOST_PAGE_SIZE; - uint8_t* sharedMem = allocatePages(snapPages); - - reg.mapSnapshot(snapKey, sharedMem); + MemoryRegion sharedMem = allocateSharedMemory(memSize); + reg.mapSnapshot(snapKey, sharedMem.get()); // Check mapping works - int* intA = (int*)(sharedMem + intAOffset); - int* intB = (int*)(sharedMem + intBOffset); + int* intA = (int*)(sharedMem.get() + intAOffset); + int* intB = (int*)(sharedMem.get() + intBOffset); REQUIRE(*intA == originalValueA); REQUIRE(*intB == originalValueB); @@ -113,15 +346,15 @@ TEST_CASE_METHOD(SnapshotMergeTestFixture, // Set up the merge regions, deliberately do the one at higher offsets first // to check the ordering - snap.addMergeRegion(intBOffset, - sizeof(int), - SnapshotDataType::Int, - SnapshotMergeOperation::Sum); + snap->addMergeRegion(intBOffset, + sizeof(int), + SnapshotDataType::Int, + SnapshotMergeOperation::Sum); - snap.addMergeRegion(intAOffset, - sizeof(int), - SnapshotDataType::Int, - SnapshotMergeOperation::Sum); + snap->addMergeRegion(intAOffset, + sizeof(int), + SnapshotDataType::Int, + SnapshotMergeOperation::Sum); // Modify both values and some other data *intA = finalValueA; @@ -129,40 +362,42 @@ TEST_CASE_METHOD(SnapshotMergeTestFixture, std::vector otherData(100, 5); int otherOffset = (3 * HOST_PAGE_SIZE) + 5; - std::memcpy(sharedMem + otherOffset, otherData.data(), otherData.size()); + std::memcpy( + sharedMem.get() + otherOffset, otherData.data(), otherData.size()); // Get the snapshot diffs std::vector actualDiffs = - snap.getChangeDiffs(sharedMem, sharedMemSize); + MemoryView({ sharedMem.get(), memSize }).diffWithSnapshot(snap); // Check original hasn't changed - REQUIRE(*intAOriginal == originalValueA); - REQUIRE(*intBOriginal == originalValueB); + const uint8_t* rawSnapData = snap->getDataPtr(); + int actualA = *(int*)(rawSnapData + intAOffset); + int actualB = *(int*)(rawSnapData + intBOffset); + REQUIRE(actualA == originalValueA); + REQUIRE(actualB == originalValueB); // Check diffs themselves REQUIRE(actualDiffs.size() == 2); - SnapshotDiff diffA = actualDiffs.at(0); - SnapshotDiff diffB = actualDiffs.at(1); + SnapshotDiff& diffA = actualDiffs.at(0); + SnapshotDiff& diffB = actualDiffs.at(1); - REQUIRE(diffA.offset == intAOffset); - REQUIRE(diffB.offset == intBOffset); + REQUIRE(diffA.getOffset() == intAOffset); + REQUIRE(diffB.getOffset() == intBOffset); - REQUIRE(diffA.operation == SnapshotMergeOperation::Sum); - REQUIRE(diffB.operation == SnapshotMergeOperation::Sum); + REQUIRE(diffA.getOperation() == SnapshotMergeOperation::Sum); + REQUIRE(diffB.getOperation() == SnapshotMergeOperation::Sum); - REQUIRE(diffA.dataType == SnapshotDataType::Int); - REQUIRE(diffB.dataType == SnapshotDataType::Int); + REQUIRE(diffA.getDataType() == SnapshotDataType::Int); + REQUIRE(diffB.getDataType() == SnapshotDataType::Int); - REQUIRE(diffA.size == sizeof(int32_t)); - REQUIRE(diffB.size == sizeof(int32_t)); + REQUIRE(diffA.getData().size() == sizeof(int32_t)); + REQUIRE(diffB.getData().size() == sizeof(int32_t)); // Check that original values have been subtracted from final values for // sums - REQUIRE(*(int*)diffA.data == sumValueA); - REQUIRE(*(int*)diffB.data == sumValueB); - - deallocatePages(snap.data, snapPages); + REQUIRE(*(int*)diffA.getData().data() == sumValueA); + REQUIRE(*(int*)diffB.getData().data() == sumValueB); } TEST_CASE_METHOD(SnapshotMergeTestFixture, @@ -200,77 +435,72 @@ TEST_CASE_METHOD(SnapshotMergeTestFixture, int sumD = 1; uint32_t offsetD = snapSize - sizeof(int32_t); - faabric::util::SnapshotData snap; - snap.size = snapSize; - snap.data = allocatePages(snapPages); + std::shared_ptr snap = + std::make_shared(snapPages * HOST_PAGE_SIZE); // Set up original values - *(int*)(snap.data + offsetA) = originalA; - *(int*)(snap.data + offsetB) = originalB; - *(int*)(snap.data + offsetC) = originalC; - *(int*)(snap.data + offsetD) = originalD; + snap->copyInData({ BYTES(&originalA), sizeof(int) }, offsetA); + snap->copyInData({ BYTES(&originalB), sizeof(int) }, offsetB); + snap->copyInData({ BYTES(&originalC), sizeof(int) }, offsetC); + snap->copyInData({ BYTES(&originalD), sizeof(int) }, offsetD); // Take the snapshot - reg.takeSnapshot(snapKey, snap, true); + reg.registerSnapshot(snapKey, snap); // Map the snapshot to some memory size_t sharedMemSize = snapPages * HOST_PAGE_SIZE; - uint8_t* sharedMem = allocatePages(snapPages); + MemoryRegion sharedMem = allocateSharedMemory(snapPages * HOST_PAGE_SIZE); - reg.mapSnapshot(snapKey, sharedMem); + reg.mapSnapshot(snapKey, sharedMem.get()); // Reset dirty tracking faabric::util::resetDirtyTracking(); // Set up the merge regions - snap.addMergeRegion(offsetA, - sizeof(int), - SnapshotDataType::Int, - SnapshotMergeOperation::Subtract); + snap->addMergeRegion(offsetA, + sizeof(int), + SnapshotDataType::Int, + SnapshotMergeOperation::Subtract); - snap.addMergeRegion( + snap->addMergeRegion( offsetB, sizeof(int), SnapshotDataType::Int, SnapshotMergeOperation::Sum); - snap.addMergeRegion(offsetC, - sizeof(int), - SnapshotDataType::Int, - SnapshotMergeOperation::Subtract); + snap->addMergeRegion(offsetC, + sizeof(int), + SnapshotDataType::Int, + SnapshotMergeOperation::Subtract); - snap.addMergeRegion( + snap->addMergeRegion( offsetD, sizeof(int), SnapshotDataType::Int, SnapshotMergeOperation::Sum); // Set final values - *(int*)(sharedMem + offsetA) = finalA; - *(int*)(sharedMem + offsetB) = finalB; - *(int*)(sharedMem + offsetC) = finalC; - *(int*)(sharedMem + offsetD) = finalD; + *(int*)(sharedMem.get() + offsetA) = finalA; + *(int*)(sharedMem.get() + offsetB) = finalB; + *(int*)(sharedMem.get() + offsetC) = finalC; + *(int*)(sharedMem.get() + offsetD) = finalD; // Check the diffs std::vector expectedDiffs = { { SnapshotDataType::Int, SnapshotMergeOperation::Subtract, offsetA, - BYTES(&subA), - sizeof(int32_t) }, + { BYTES(&subA), sizeof(int32_t) } }, { SnapshotDataType::Int, SnapshotMergeOperation::Sum, offsetB, - BYTES(&sumB), - sizeof(int32_t) }, + { BYTES(&sumB), sizeof(int32_t) } }, { SnapshotDataType::Int, SnapshotMergeOperation::Subtract, offsetC, - BYTES(&subC), - sizeof(int32_t) }, + { BYTES(&subC), sizeof(int32_t) } }, { SnapshotDataType::Int, SnapshotMergeOperation::Sum, offsetD, - BYTES(&sumD), - sizeof(int32_t) }, + { BYTES(&sumD), sizeof(int32_t) } }, }; std::vector actualDiffs = - snap.getChangeDiffs(sharedMem, sharedMemSize); + MemoryView({ sharedMem.get(), sharedMemSize }).diffWithSnapshot(snap); REQUIRE(actualDiffs.size() == 4); checkDiffs(actualDiffs, expectedDiffs); @@ -285,9 +515,8 @@ TEST_CASE_METHOD(SnapshotMergeTestFixture, uint32_t offset = HOST_PAGE_SIZE + (10 * sizeof(int32_t)); - faabric::util::SnapshotData snap; - snap.size = snapPages * faabric::util::HOST_PAGE_SIZE; - snap.data = allocatePages(snapPages); + std::shared_ptr snap = + std::make_shared(snapPages * HOST_PAGE_SIZE); std::vector originalData; std::vector updatedData; @@ -384,41 +613,40 @@ TEST_CASE_METHOD(SnapshotMergeTestFixture, } // Write the original data into place - std::memcpy(snap.data + offset, originalData.data(), originalData.size()); + snap->copyInData(originalData, offset); - // Take the snapshot - reg.takeSnapshot(snapKey, snap, true); + // Register the snap + reg.registerSnapshot(snapKey, snap); // Map the snapshot to some memory size_t sharedMemSize = snapPages * HOST_PAGE_SIZE; - uint8_t* sharedMem = allocatePages(snapPages); + MemoryRegion sharedMem = allocateSharedMemory(snapPages * HOST_PAGE_SIZE); - reg.mapSnapshot(snapKey, sharedMem); + reg.mapSnapshot(snapKey, sharedMem.get()); // Reset dirty tracking faabric::util::resetDirtyTracking(); // Set up the merge region - snap.addMergeRegion(offset, regionLength, dataType, operation); + snap->addMergeRegion(offset, regionLength, dataType, operation); // Modify the value - std::memcpy(sharedMem + offset, updatedData.data(), updatedData.size()); + std::memcpy( + sharedMem.get() + offset, updatedData.data(), updatedData.size()); // Get the snapshot diffs std::vector actualDiffs = - snap.getChangeDiffs(sharedMem, sharedMemSize); + MemoryView({ sharedMem.get(), sharedMemSize }).diffWithSnapshot(snap); // Check diff REQUIRE(actualDiffs.size() == 1); std::vector expectedDiffs = { { dataType, operation, offset, - expectedData.data(), - expectedData.size() } }; + { expectedData.data(), + expectedData.size() } } }; checkDiffs(actualDiffs, expectedDiffs); - - deallocatePages(snap.data, snapPages); } TEST_CASE_METHOD(SnapshotMergeTestFixture, @@ -429,9 +657,8 @@ TEST_CASE_METHOD(SnapshotMergeTestFixture, int snapPages = 3; int offset = HOST_PAGE_SIZE + (2 * sizeof(int32_t)); - faabric::util::SnapshotData snap; - snap.size = snapPages * faabric::util::HOST_PAGE_SIZE; - snap.data = allocatePages(snapPages); + std::shared_ptr snap = + std::make_shared(snapPages * HOST_PAGE_SIZE); faabric::util::SnapshotDataType dataType = faabric::util::SnapshotDataType::Raw; @@ -458,34 +685,86 @@ TEST_CASE_METHOD(SnapshotMergeTestFixture, } // Take the snapshot - reg.takeSnapshot(snapKey, snap, true); + reg.registerSnapshot(snapKey, snap); // Map the snapshot size_t sharedMemSize = snapPages * HOST_PAGE_SIZE; - uint8_t* sharedMem = allocatePages(snapPages); - reg.mapSnapshot(snapKey, sharedMem); + MemoryRegion sharedMem = allocateSharedMemory(snapPages * HOST_PAGE_SIZE); + reg.mapSnapshot(snapKey, sharedMem.get()); // Reset dirty tracking faabric::util::resetDirtyTracking(); // Set up the merge region - snap.addMergeRegion(offset, dataLength, dataType, operation); + snap->addMergeRegion(offset, dataLength, dataType, operation); // Modify the value std::vector bytes(dataLength, 3); - std::memcpy(sharedMem + offset, bytes.data(), bytes.size()); + std::memcpy(sharedMem.get() + offset, bytes.data(), bytes.size()); // Check getting diffs throws an exception bool failed = false; try { - snap.getChangeDiffs(sharedMem, sharedMemSize); + MemoryView({ sharedMem.get(), sharedMemSize }).diffWithSnapshot(snap); } catch (std::runtime_error& ex) { failed = true; REQUIRE(ex.what() == expectedMsg); } REQUIRE(failed); - deallocatePages(snap.data, snapPages); +} + +TEST_CASE_METHOD(SnapshotMergeTestFixture, + "Test diffing snapshot memory", + "[snapshot][util]") +{ + // Set up a snapshot + int snapPages = 4; + size_t snapSize = snapPages * HOST_PAGE_SIZE; + + std::shared_ptr snap = + std::make_shared(snapSize); + reg.registerSnapshot(snapKey, snap); + + // Check memory is zeroed initially + std::vector zeroes(snapSize, 0); + REQUIRE(snap->getDataCopy() == zeroes); + + // Check memory is page-aligned + faabric::util::isPageAligned((const void*)snap->getDataPtr()); + + // Reset dirty tracking + faabric::util::resetDirtyTracking(); + + // Update the snapshot + std::vector dataA = { 0, 1, 2, 3 }; + std::vector dataB = { 3, 4, 5 }; + uint32_t offsetA = 0; + uint32_t offsetB = 2 * HOST_PAGE_SIZE + 1; + + snap->copyInData(dataA, offsetA); + snap->copyInData(dataB, offsetB); + + // Check we get the expected diffs + std::vector expectedDiffs = + MemoryView({ snap->getDataPtr(), snap->getSize() }).getDirtyRegions(); + + REQUIRE(expectedDiffs.size() == 2); + + SnapshotDiff& diffA = expectedDiffs.at(0); + SnapshotDiff& diffB = expectedDiffs.at(1); + + REQUIRE(diffA.getData().size() == HOST_PAGE_SIZE); + REQUIRE(diffB.getData().size() == HOST_PAGE_SIZE); + + std::vector actualA = { diffA.getData().begin(), + diffA.getData().begin() + dataA.size() }; + std::vector actualB = { + diffB.getData().begin() + 1, diffB.getData().begin() + 1 + dataB.size() + }; + + REQUIRE(actualA == dataA); + REQUIRE(actualB == dataB); } TEST_CASE_METHOD(SnapshotMergeTestFixture, @@ -494,57 +773,63 @@ TEST_CASE_METHOD(SnapshotMergeTestFixture, { int snapPages = 3; size_t sharedMemSize = snapPages * HOST_PAGE_SIZE; - uint8_t* sharedMem = setUpSnapshot(snapPages, snapPages); + + std::shared_ptr snap = + std::make_shared(snapPages * HOST_PAGE_SIZE); + reg.registerSnapshot(snapKey, snap); + + // Map the snapshot + MemoryRegion sharedMem = allocateSharedMemory(sharedMemSize); + reg.mapSnapshot(snapKey, sharedMem.get()); + + // Reset dirty tracking + faabric::util::resetDirtyTracking(); // Add some tightly-packed changes uint32_t offsetA = 0; std::vector dataA(10, 1); - std::memcpy(sharedMem + offsetA, dataA.data(), dataA.size()); + std::memcpy(sharedMem.get() + offsetA, dataA.data(), dataA.size()); uint32_t offsetB = dataA.size() + 1; std::vector dataB(2, 1); - std::memcpy(sharedMem + offsetB, dataB.data(), dataB.size()); + std::memcpy(sharedMem.get() + offsetB, dataB.data(), dataB.size()); uint32_t offsetC = offsetB + 3; std::vector dataC(1, 1); - std::memcpy(sharedMem + offsetC, dataC.data(), dataC.size()); + std::memcpy(sharedMem.get() + offsetC, dataC.data(), dataC.size()); uint32_t offsetD = offsetC + 2; std::vector dataD(1, 1); - std::memcpy(sharedMem + offsetD, dataD.data(), dataD.size()); + std::memcpy(sharedMem.get() + offsetD, dataD.data(), dataD.size()); std::vector expectedDiffs = { { SnapshotDataType::Raw, SnapshotMergeOperation::Overwrite, offsetA, - dataA.data(), - dataA.size() }, + { dataA.data(), dataA.size() } }, { SnapshotDataType::Raw, SnapshotMergeOperation::Overwrite, offsetB, - dataB.data(), - dataB.size() }, + { dataB.data(), dataB.size() } }, { SnapshotDataType::Raw, SnapshotMergeOperation::Overwrite, offsetC, - dataC.data(), - dataC.size() }, + { dataC.data(), dataC.size() } }, { SnapshotDataType::Raw, SnapshotMergeOperation::Overwrite, offsetD, - dataD.data(), - dataD.size() }, + { dataD.data(), dataD.size() } }, }; // Add a single merge region for all the changes - snap.addMergeRegion(0, - offsetD + dataD.size() + 20, - SnapshotDataType::Raw, - SnapshotMergeOperation::Overwrite); + snap->addMergeRegion(0, + offsetD + dataD.size() + 20, + SnapshotDataType::Raw, + SnapshotMergeOperation::Overwrite); // Check number of diffs std::vector actualDiffs = - snap.getChangeDiffs(sharedMem, sharedMemSize); + MemoryView({ sharedMem.get(), sharedMemSize }).diffWithSnapshot(snap); checkDiffs(actualDiffs, expectedDiffs); } @@ -555,65 +840,74 @@ TEST_CASE_METHOD(SnapshotMergeTestFixture, { int snapPages = 6; size_t sharedMemSize = snapPages * HOST_PAGE_SIZE; - uint8_t* sharedMem = setUpSnapshot(snapPages, snapPages); + + std::shared_ptr snap = + std::make_shared(snapPages * HOST_PAGE_SIZE); + reg.registerSnapshot(snapKey, snap); + + // Map the snapshot + MemoryRegion sharedMem = allocateSharedMemory(sharedMemSize); + reg.mapSnapshot(snapKey, sharedMem.get()); + + // Reset dirty tracking + faabric::util::resetDirtyTracking(); // Add a couple of merge regions on each page, which should be skipped as // they won't overlap any changes for (int i = 0; i < snapPages; i++) { // Overwrite int skippedOverwriteOffset = i * HOST_PAGE_SIZE; - snap.addMergeRegion(skippedOverwriteOffset, - 10, - faabric::util::SnapshotDataType::Raw, - faabric::util::SnapshotMergeOperation::Overwrite); + snap->addMergeRegion(skippedOverwriteOffset, + 10, + faabric::util::SnapshotDataType::Raw, + faabric::util::SnapshotMergeOperation::Overwrite); // Sum int skippedSumOffset = ((i + 1) * HOST_PAGE_SIZE) - (2 * sizeof(int32_t)); - snap.addMergeRegion(skippedSumOffset, - sizeof(int32_t), - faabric::util::SnapshotDataType::Int, - faabric::util::SnapshotMergeOperation::Sum); + snap->addMergeRegion(skippedSumOffset, + sizeof(int32_t), + faabric::util::SnapshotDataType::Int, + faabric::util::SnapshotMergeOperation::Sum); } // Add an overwrite region that should take effect uint32_t overwriteAOffset = (2 * HOST_PAGE_SIZE) + 20; - snap.addMergeRegion(overwriteAOffset, - 20, - faabric::util::SnapshotDataType::Raw, - faabric::util::SnapshotMergeOperation::Overwrite); + snap->addMergeRegion(overwriteAOffset, + 20, + faabric::util::SnapshotDataType::Raw, + faabric::util::SnapshotMergeOperation::Overwrite); std::vector overwriteData(10, 1); - std::memcpy( - sharedMem + overwriteAOffset, overwriteData.data(), overwriteData.size()); + std::memcpy(sharedMem.get() + overwriteAOffset, + overwriteData.data(), + overwriteData.size()); // Add a sum region and data that should also take effect uint32_t sumOffset = (4 * HOST_PAGE_SIZE) + 100; int sumValue = 333; int sumOriginal = 111; int sumExpected = 222; - snap.addMergeRegion(sumOffset, - sizeof(int32_t), - faabric::util::SnapshotDataType::Int, - faabric::util::SnapshotMergeOperation::Sum); - *(int*)(snap.data + sumOffset) = sumOriginal; - *(int*)(sharedMem + sumOffset) = sumValue; + snap->addMergeRegion(sumOffset, + sizeof(int32_t), + faabric::util::SnapshotDataType::Int, + faabric::util::SnapshotMergeOperation::Sum); + snap->copyInData({ BYTES(&sumOriginal), sizeof(int) }, sumOffset); + *(int*)(sharedMem.get() + sumOffset) = sumValue; // Check diffs std::vector expectedDiffs = { { faabric::util::SnapshotDataType::Raw, faabric::util::SnapshotMergeOperation::Overwrite, overwriteAOffset, - BYTES(overwriteData.data()), - overwriteData.size() }, + { BYTES(overwriteData.data()), overwriteData.size() } }, { faabric::util::SnapshotDataType::Int, faabric::util::SnapshotMergeOperation::Sum, sumOffset, - BYTES(&sumExpected), - sizeof(int32_t) }, + { BYTES(&sumExpected), sizeof(int32_t) } }, }; std::vector actualDiffs = - snap.getChangeDiffs(sharedMem, sharedMemSize); + MemoryView({ sharedMem.get(), sharedMemSize }).diffWithSnapshot(snap); checkDiffs(actualDiffs, expectedDiffs); } @@ -626,33 +920,197 @@ TEST_CASE_METHOD(SnapshotMergeTestFixture, int sharedMemPages = 10; size_t sharedMemSize = sharedMemPages * HOST_PAGE_SIZE; - uint8_t* sharedMem = setUpSnapshot(snapPages, sharedMemPages); + std::shared_ptr snap = + std::make_shared(snapPages * HOST_PAGE_SIZE); + reg.registerSnapshot(snapKey, snap); + + // Map the snapshot + MemoryRegion sharedMem = allocateSharedMemory(sharedMemSize); + reg.mapSnapshot(snapKey, sharedMem.get()); + faabric::util::resetDirtyTracking(); // Make an edit somewhere in the extended memory, outside the original // snapshot - uint32_t diffPageStart = 8 * HOST_PAGE_SIZE; + uint32_t diffPageStart = (snapPages + 2) * HOST_PAGE_SIZE; uint32_t diffOffset = diffPageStart + 100; std::vector diffData(120, 2); - std::memcpy(sharedMem + diffOffset, diffData.data(), diffData.size()); + std::memcpy(sharedMem.get() + diffOffset, diffData.data(), diffData.size()); // Add a merge region from near end of original snapshot upwards - snap.addMergeRegion(snap.size - 120, - 0, - faabric::util::SnapshotDataType::Raw, - faabric::util::SnapshotMergeOperation::Overwrite); + snap->addMergeRegion(snap->getSize() - 120, + 0, + faabric::util::SnapshotDataType::Raw, + faabric::util::SnapshotMergeOperation::Overwrite); std::vector actualDiffs = - snap.getChangeDiffs(sharedMem, sharedMemSize); + MemoryView({ sharedMem.get(), sharedMemSize }).diffWithSnapshot(snap); // Make sure the whole page containing the diff is included std::vector expectedDiffs = { { faabric::util::SnapshotDataType::Raw, faabric::util::SnapshotMergeOperation::Overwrite, diffPageStart, - sharedMem + diffPageStart, - (size_t)HOST_PAGE_SIZE }, + { sharedMem.get() + diffPageStart, (size_t)HOST_PAGE_SIZE } }, }; checkDiffs(actualDiffs, expectedDiffs); } + +TEST_CASE("Test snapshot data constructors", "[snapshot][util]") +{ + std::vector data(2 * HOST_PAGE_SIZE, 3); + + // Add known subsection + uint32_t chunkOffset = 120; + std::vector chunk(100, 4); + ::memcpy(data.data() + chunkOffset, chunk.data(), chunk.size()); + + size_t expectedMaxSize = data.size(); + + std::shared_ptr snap = nullptr; + SECTION("From size") + { + SECTION("No max") + { + snap = std::make_shared(data.size()); + } + + SECTION("Zero max") + { + snap = std::make_shared(data.size(), 0); + } + + SECTION("With max") + { + expectedMaxSize = data.size() + 123; + snap = std::make_shared(data.size(), expectedMaxSize); + } + + snap->copyInData(data, 0); + } + + SECTION("From span") + { + snap = std::make_shared( + std::span(data.data(), data.size())); + } + + SECTION("From vector") { snap = std::make_shared(data); } + + REQUIRE(snap->getSize() == data.size()); + REQUIRE(snap->getMaxSize() == expectedMaxSize); + + std::vector actualCopy = snap->getDataCopy(); + REQUIRE(actualCopy == data); + + std::vector actualChunk = + snap->getDataCopy(chunkOffset, chunk.size()); + REQUIRE(actualChunk == chunk); + + const std::vector actualConst( + snap->getDataPtr(), snap->getDataPtr() + snap->getSize()); + REQUIRE(actualConst == data); +} + +TEST_CASE("Test snapshot mapped memory diffs", "[snapshot][util]") +{ + int nSnapPages = 5; + size_t snapSize = nSnapPages * HOST_PAGE_SIZE; + + auto snap = std::make_shared(snapSize); + + std::vector dataA(150, 3); + std::vector dataB(200, 4); + std::vector dataC(250, 5); + + uint32_t offsetA = 0; + uint32_t offsetB = HOST_PAGE_SIZE; + uint32_t offsetC = HOST_PAGE_SIZE + 1 + dataB.size(); + + // Add merge regions + snap->addMergeRegion(offsetA, + dataA.size(), + SnapshotDataType::Raw, + SnapshotMergeOperation::Overwrite); + + snap->addMergeRegion(offsetB, + dataB.size(), + SnapshotDataType::Raw, + SnapshotMergeOperation::Overwrite); + + snap->addMergeRegion(offsetC, + dataC.size(), + SnapshotDataType::Raw, + SnapshotMergeOperation::Overwrite); + + // Write data to snapshot + snap->copyInData(dataA); + + // Map some memory + MemoryRegion memA = allocateSharedMemory(snapSize); + snap->mapToMemory(memA.get()); + + faabric::util::resetDirtyTracking(); + + std::vector actualSnap = snap->getDataCopy(); + std::vector actualA(memA.get(), memA.get() + snapSize); + REQUIRE(actualSnap == actualA); + + // Write data to snapshot + snap->copyInData(dataB, offsetB); + + // Write data to memory + std::memcpy(memA.get() + offsetC, dataC.data(), dataC.size()); + + // Check diffs from memory vs snapshot + std::vector actualDiffs = + MemoryView({ memA.get(), snapSize }).diffWithSnapshot(snap); + REQUIRE(actualDiffs.size() == 1); + + SnapshotDiff& actualDiff = actualDiffs.at(0); + REQUIRE(actualDiff.getData().size() == dataC.size()); + REQUIRE(actualDiff.getOffset() == offsetC); + + // Apply diffs from memory to the snapshot + snap->queueDiffs(actualDiffs); + snap->writeQueuedDiffs(); + + // Check snapshot now shows modified page + std::vector snapDirtyRegions = + MemoryView({ snap->getDataPtr(), snap->getSize() }).getDirtyRegions(); + + REQUIRE(snapDirtyRegions.size() == 1); + SnapshotDiff& snapDirtyRegion = snapDirtyRegions.at(0); + REQUIRE(snapDirtyRegion.getOffset() == HOST_PAGE_SIZE); + + // Check modified data includes both updates + std::vector dirtyRegionData = snapDirtyRegion.getDataCopy(); + REQUIRE(dirtyRegionData.size() == HOST_PAGE_SIZE); + + std::vector expectedDirtyRegionData(HOST_PAGE_SIZE, 0); + std::memcpy(expectedDirtyRegionData.data() + (offsetB - HOST_PAGE_SIZE), + dataB.data(), + dataB.size()); + std::memcpy(expectedDirtyRegionData.data() + (offsetC - HOST_PAGE_SIZE), + dataC.data(), + dataC.size()); + + REQUIRE(dirtyRegionData == expectedDirtyRegionData); + + // Map more memory from the snapshot, check it contains all updates + MemoryRegion memB = allocateSharedMemory(snapSize); + snap->mapToMemory(memB.get()); + std::vector expectedFinal(snapSize, 0); + std::memcpy(expectedFinal.data() + offsetA, dataA.data(), dataA.size()); + std::memcpy(expectedFinal.data() + offsetB, dataB.data(), dataB.size()); + std::memcpy(expectedFinal.data() + offsetC, dataC.data(), dataC.size()); + + std::vector actualMemB(memB.get(), memB.get() + snapSize); + REQUIRE(actualMemB == expectedFinal); + + // Remap first memory and check this also contains all updates + snap->mapToMemory(memA.get()); + std::vector remappedMemA(memB.get(), memB.get() + snapSize); + REQUIRE(remappedMemA == expectedFinal); +} } diff --git a/tests/utils/fixtures.h b/tests/utils/fixtures.h index c75a67963..86d64916a 100644 --- a/tests/utils/fixtures.h +++ b/tests/utils/fixtures.h @@ -113,40 +113,20 @@ class SnapshotTestFixture reg.clear(); } - uint8_t* allocatePages(int nPages) + std::shared_ptr setUpSnapshot( + const std::string& snapKey, + int nPages) { - return (uint8_t*)mmap(nullptr, - nPages * faabric::util::HOST_PAGE_SIZE, - PROT_WRITE, - MAP_SHARED | MAP_ANONYMOUS, - -1, - 0); - } + size_t snapSize = nPages * faabric::util::HOST_PAGE_SIZE; + auto snapData = std::make_shared(snapSize); + reg.registerSnapshot(snapKey, snapData); - void deallocatePages(uint8_t* base, int nPages) - { - munmap(base, nPages * faabric::util::HOST_PAGE_SIZE); - } - - faabric::util::SnapshotData takeSnapshot(const std::string& snapKey, - int nPages, - bool locallyRestorable) - { - faabric::util::SnapshotData snap; - uint8_t* data = allocatePages(nPages); - - snap.size = nPages * faabric::util::HOST_PAGE_SIZE; - snap.data = data; - - reg.takeSnapshot(snapKey, snap, locallyRestorable); - - return snap; + return snapData; } void removeSnapshot(const std::string& key, int nPages) { auto snap = reg.getSnapshot(key); - deallocatePages(snap->data, nPages); reg.deleteSnapshot(key); } @@ -318,19 +298,14 @@ class TestExecutor final : public faabric::scheduler::Executor public: TestExecutor(faabric::Message& msg); - ~TestExecutor(); - - uint8_t* dummyMemory = nullptr; - + faabric::util::MemoryRegion dummyMemory = nullptr; size_t dummyMemorySize = 0; - void postFinish() override; - void reset(faabric::Message& msg) override; void restore(faabric::Message& msg) override; - faabric::util::SnapshotData snapshot() override; + faabric::util::MemoryView getMemoryView() override; int32_t executeTask( int threadPoolIdx,