From b3229aa655cb7a94a9d59dadd2943f949960f4c4 Mon Sep 17 00:00:00 2001 From: Simon Shillaker <554768+Shillaker@users.noreply.github.com> Date: Fri, 7 Jan 2022 13:02:24 +0100 Subject: [PATCH] Dirty tracking performance improvements (#210) * Start on userfault experiment * Tidy up * Failing test for mapped regions * Destroy uffd tests * Refactor to suit configurable diffing * Fixed tests for soft dirty PTEs * Snapshot self-dirty tracking * Working tests * Tidy up when executor does dirty tracking * Special-case reinitialisation * Remove restartTracking * Remove duplicated function names * Handle dirty regions * Move gap filling back again * Update to using OffsetMemoryRegions * Remove debug logging * Tidy up * Renaming * Missing comments * Add test for multi-threaded segfault handling * Remove dirty checks from memory test * Fixing up tests * Refactor overwrite diff logic * Fix test * Formatting * Add main thread snapshot handling to faabric * Incorporate size change into main thread snapshot * Simplify logic around snapshots in executor * Fixing a couple of tests * Fixing tests * Small tidy-ups * Fix up unit tests * Fix up a couple of distributed tests * Moved thread scheduling and decision caching into executor * Fixing dist tests * Remove core dump * Formatting * Revert trace logging in DC file * Fix data race in tests * Tidy up and docs * Formatting * Guard against empty memory * More logging * Specify merge regions when spawning threads * Small logging fix * Attempt to speed up diffing * Tighter diffing loops * Merge diffing and regions * Remove OffsetMemoryRegion * Avoid vector * More dirty tracking * Fix up SDPTE * Switch from shared to full lock * Add test for dirty tracking config field * Remove await thread results funciton --- dist-test/dev_server.sh | 4 +- include/faabric/scheduler/Scheduler.h | 39 +- include/faabric/util/bytes.h | 16 + include/faabric/util/config.h | 3 + include/faabric/util/crash.h | 14 +- include/faabric/util/dirty.h | 124 +++++ include/faabric/util/func.h | 7 + include/faabric/util/memory.h | 22 +- include/faabric/util/snapshot.h | 54 ++- src/scheduler/Executor.cpp | 497 ++++++++++++++++---- src/scheduler/Scheduler.cpp | 44 +- src/snapshot/SnapshotRegistry.cpp | 11 +- src/snapshot/SnapshotServer.cpp | 7 +- src/util/CMakeLists.txt | 1 + src/util/bytes.cpp | 56 +++ src/util/config.cpp | 3 + src/util/crash.cpp | 29 +- src/util/dirty.cpp | 382 +++++++++++++++ src/util/func.cpp | 9 + src/util/logging.cpp | 2 +- src/util/memory.cpp | 163 ++----- src/util/snapshot.cpp | 275 +++++------ tests/dist/DistTestExecutor.cpp | 44 +- tests/dist/DistTestExecutor.h | 13 +- tests/dist/scheduler/functions.cpp | 169 +++---- tests/dist/scheduler/test_snapshots.cpp | 58 ++- tests/dist/scheduler/test_threads.cpp | 20 +- tests/dist/transport/functions.cpp | 8 +- tests/test/scheduler/test_executor.cpp | 154 +++--- tests/test/scheduler/test_scheduler.cpp | 181 ++++--- tests/test/snapshot/test_snapshot_diffs.cpp | 77 +-- tests/test/util/test_bytes.cpp | 84 ++++ tests/test/util/test_config.cpp | 8 + tests/test/util/test_dirty.cpp | 356 ++++++++++++++ tests/test/util/test_memory.cpp | 196 +++----- tests/test/util/test_snapshot.cpp | 437 +++++++++-------- tests/utils/fixtures.h | 39 +- 37 files changed, 2478 insertions(+), 1128 deletions(-) create mode 100644 include/faabric/util/dirty.h create mode 100644 src/util/dirty.cpp create mode 100644 tests/test/util/test_dirty.cpp diff --git a/dist-test/dev_server.sh b/dist-test/dev_server.sh index 0c87340ea..5a5aa38dd 100755 --- a/dist-test/dev_server.sh +++ b/dist-test/dev_server.sh @@ -13,12 +13,14 @@ elif [[ "$1" == "restart" ]]; then docker-compose restart dist-test-server elif [[ "$1" == "stop" ]]; then docker-compose stop dist-test-server +elif [[ "$1" == "rm" ]]; then + docker-compose rm dist-test-server else echo "Unrecognised argument: $1" echo "" echo "Usage:" echo "" - echo "./dist-test/dev_server.sh [restart|stop]" + echo "./dist-test/dev_server.sh [restart|stop|rm]" exit 1 fi diff --git a/include/faabric/scheduler/Scheduler.h b/include/faabric/scheduler/Scheduler.h index e588ccb1d..58b84a623 100644 --- a/include/faabric/scheduler/Scheduler.h +++ b/include/faabric/scheduler/Scheduler.h @@ -5,8 +5,10 @@ #include #include #include +#include #include #include +#include #include #include #include @@ -32,13 +34,11 @@ class ExecutorTask ExecutorTask(int messageIndexIn, std::shared_ptr reqIn, std::shared_ptr> batchCounterIn, - bool needsSnapshotSyncIn, bool skipResetIn); std::shared_ptr req; std::shared_ptr> batchCounter; int messageIndex = 0; - bool needsSnapshotSync = false; bool skipReset = false; }; @@ -51,6 +51,10 @@ class Executor virtual ~Executor() = default; + std::vector> executeThreads( + std::shared_ptr req, + const std::vector& mergeRegions); + void executeTasks(std::vector msgIdxs, std::shared_ptr req); @@ -69,20 +73,42 @@ class Executor void releaseClaim(); - virtual faabric::util::MemoryView getMemoryView(); + std::shared_ptr getMainThreadSnapshot( + faabric::Message& msg, + bool createIfNotExists = false); protected: - virtual void restore(faabric::Message& msg); + virtual void restore(const std::string& snapshotKey); virtual void postFinish(); + virtual std::span getMemoryView(); + + virtual void setMemorySize(size_t newSize); + faabric::Message boundMessage; + Scheduler& sch; + + faabric::snapshot::SnapshotRegistry& reg; + + faabric::util::DirtyTracker& tracker; + uint32_t threadPoolSize = 0; private: std::atomic claimed = false; + // ---- Application threads ---- + std::shared_mutex threadExecutionMutex; + std::unordered_map cachedGroupIds; + std::unordered_map> + cachedDecisionHosts; + std::vector> dirtyRegions; + + void deleteMainThreadSnapshot(const faabric::Message& msg); + + // ---- Function execution thread pool ---- std::mutex threadsMutex; std::vector> threadPoolThreads; std::vector> deadThreads; @@ -93,6 +119,10 @@ class Executor void threadPoolThread(int threadPoolIdx); }; +Executor* getExecutingExecutor(); + +void setExecutingExecutor(Executor* exec); + class Scheduler { public: @@ -138,6 +168,7 @@ class Scheduler void pushSnapshotDiffs( const faabric::Message& msg, + const std::string& snapshotKey, const std::vector& diffs); void setThreadResultLocally(uint32_t msgId, int32_t returnValue); diff --git a/include/faabric/util/bytes.h b/include/faabric/util/bytes.h index 620fb8f2d..a1d7cbc9d 100644 --- a/include/faabric/util/bytes.h +++ b/include/faabric/util/bytes.h @@ -1,6 +1,8 @@ #pragma once #include +#include +#include #include #include #include @@ -25,6 +27,20 @@ int safeCopyToBuffer(const uint8_t* dataIn, uint8_t* buffer, int bufferLen); +/* + * Returns a list of pairs of for any bytes differing between + * the two arrays. + */ +std::vector> diffArrayRegions( + std::span a, + std::span b); + +/* + * Returns a list of flags marking which bytes differ between the two arrays. + */ +std::vector diffArrays(std::span a, + std::span b); + template T unalignedRead(const uint8_t* bytes) { diff --git a/include/faabric/util/config.h b/include/faabric/util/config.h index cb79681c8..25106cd1b 100644 --- a/include/faabric/util/config.h +++ b/include/faabric/util/config.h @@ -50,6 +50,9 @@ class SystemConfig int snapshotServerThreads; int pointToPointServerThreads; + // Dirty tracking + std::string dirtyTrackingMode; + SystemConfig(); void print(); diff --git a/include/faabric/util/crash.h b/include/faabric/util/crash.h index a36cd3e09..a366af9d9 100644 --- a/include/faabric/util/crash.h +++ b/include/faabric/util/crash.h @@ -1,6 +1,18 @@ namespace faabric::util { -void setUpCrashHandler(); +/* + * Sets up crash handling. By default covers a number of signals that would + * otherwise cause a crash. Signal argument can be provided to reinstating crash + * handling for a specific signal after it's been used elsewhere in the + * application (e.g. for dirty tracking). + */ +void setUpCrashHandler(int sig = -1); + +/* + * Prints the stack trace for a given signal. Only to be called in signal + * handlers. + */ +void handleCrash(int sig); } diff --git a/include/faabric/util/dirty.h b/include/faabric/util/dirty.h new file mode 100644 index 000000000..afcb2d692 --- /dev/null +++ b/include/faabric/util/dirty.h @@ -0,0 +1,124 @@ +#pragma once + +#include +#include +#include + +#include +#include +#include + +#define CLEAR_REFS "/proc/self/clear_refs" +#define PAGEMAP "/proc/self/pagemap" + +#define PAGEMAP_ENTRY_BYTES sizeof(uint64_t) +#define PAGEMAP_SOFT_DIRTY (1Ull << 55) + +namespace faabric::util { + +/* + * Interface to all dirty page tracking. Implementation-specific boilerplate + * held in subclasses. + */ +class DirtyTracker +{ + public: + virtual void clearAll() = 0; + + virtual void reinitialise() = 0; + + virtual void startTracking(std::span region) = 0; + + virtual void stopTracking(std::span region) = 0; + + virtual std::vector> getDirtyOffsets( + std::span region) = 0; + + virtual void startThreadLocalTracking(std::span region) = 0; + + virtual void stopThreadLocalTracking(std::span region) = 0; + + virtual std::vector> + getThreadLocalDirtyOffsets(std::span region) = 0; + + virtual std::vector> getBothDirtyOffsets( + std::span region) = 0; +}; + +/* + * Dirty tracking implementation using soft-dirty PTEs + * https://www.kernel.org/doc/html/latest/admin-guide/mm/soft-dirty.html + */ +class SoftPTEDirtyTracker final : public DirtyTracker +{ + public: + SoftPTEDirtyTracker(); + + ~SoftPTEDirtyTracker(); + + void clearAll() override; + + void reinitialise() override; + + void startTracking(std::span region) override; + + void stopTracking(std::span region) override; + + std::vector> getDirtyOffsets( + std::span region) override; + + void startThreadLocalTracking(std::span region) override; + + void stopThreadLocalTracking(std::span region) override; + + std::vector> getThreadLocalDirtyOffsets( + std::span region) override; + + std::vector> getBothDirtyOffsets( + std::span region) override; + + private: + FILE* clearRefsFile = nullptr; + + FILE* pagemapFile = nullptr; +}; + +/* + * Dirty tracking implementation using mprotect to make pages read-only and + * use segfaults resulting from writes to mark them as dirty. + */ +class SegfaultDirtyTracker final : public DirtyTracker +{ + public: + SegfaultDirtyTracker(); + + void clearAll() override; + + void reinitialise() override; + + void startTracking(std::span region) override; + + void stopTracking(std::span region) override; + + std::vector> getDirtyOffsets( + std::span region) override; + + void startThreadLocalTracking(std::span region) override; + + void stopThreadLocalTracking(std::span region) override; + + std::vector> getThreadLocalDirtyOffsets( + std::span region) override; + + std::vector> getBothDirtyOffsets( + std::span region) override; + + // Signal handler for the resulting segfaults + static void handler(int sig, siginfo_t* info, void* ucontext) noexcept; + + private: + void setUpSignalHandler(); +}; + +DirtyTracker& getDirtyTracker(); +} diff --git a/include/faabric/util/func.h b/include/faabric/util/func.h index 3d055c7e5..b4fa1ba09 100644 --- a/include/faabric/util/func.h +++ b/include/faabric/util/func.h @@ -37,4 +37,11 @@ std::string statusKeyFromMessageId(unsigned int mid); std::vector messageToBytes(const faabric::Message& msg); std::vector getArgvForMessage(const faabric::Message& msg); + +/* + * Gets the key for the main thread snapshot for the given message. Result will + * be the same on all hosts. + */ +std::string getMainThreadSnapshotKey(const faabric::Message& msg); + } diff --git a/include/faabric/util/memory.h b/include/faabric/util/memory.h index 59000ca1f..d5511a2bc 100644 --- a/include/faabric/util/memory.h +++ b/include/faabric/util/memory.h @@ -10,6 +10,17 @@ namespace faabric::util { +/* + * Dedupes a list of dirty regions specified by offset and length + */ +std::vector> dedupeMemoryRegions( + std::vector>& regions); + +/* + * Typedef used to enforce RAII on mmapped memory regions + */ +typedef std::unique_ptr> MemoryRegion; + // ------------------------- // Alignment // ------------------------- @@ -36,20 +47,9 @@ size_t alignOffsetDown(size_t offset); AlignedChunk getPageAlignedChunk(long offset, long length); -// ------------------------- -// Dirty pages -// ------------------------- -void resetDirtyTracking(); - -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 allocatePrivateMemory(size_t size); diff --git a/include/faabric/util/snapshot.h b/include/faabric/util/snapshot.h index 43b50a5cf..c19cbfd2f 100644 --- a/include/faabric/util/snapshot.h +++ b/include/faabric/util/snapshot.h @@ -8,6 +8,7 @@ #include #include +#include #include #include #include @@ -79,16 +80,21 @@ class SnapshotMergeRegion void addDiffs(std::vector& diffs, std::span originalData, - std::span updatedData, - std::pair dirtyRange); + std::span updatedData, + std::pair dirtyRegion); private: void addOverwriteDiff(std::vector& diffs, std::span original, - std::span updated, - std::pair dirtyRange); + std::span updatedData, + std::pair dirtyRegion); }; +/* + * Calculates a diff value that can later be merged into the master copy of the + * given snapshot. It will be used on remote hosts to calculate the diffs that + * are to be sent back to the master host. + */ template inline bool calculateDiffValue(const uint8_t* original, uint8_t* updated, @@ -138,6 +144,10 @@ inline bool calculateDiffValue(const uint8_t* original, return true; } +/* + * Applies a diff value to the master copy of a snapshot, where the diff has + * been calculated based on a change made to another copy of the same snapshot. + */ template inline T applyDiffValue(const uint8_t* original, const uint8_t* diff, @@ -221,6 +231,20 @@ class SnapshotData size_t getMaxSize() const { return maxSize; } + // Returns a list of changes that have been made to the snapshot since the + // last time the list was cleared. + std::vector getTrackedChanges(); + + // Clears the list of tracked changes. + void clearTrackedChanges(); + + // Returns the list of changes in the given dirty regions versus their + // original value in the snapshot, based on the merge regions set on this + // snapshot. + std::vector diffWithDirtyRegions( + std::span updated, + std::vector> dirtyRegions); + private: size_t size = 0; size_t maxSize = 0; @@ -233,6 +257,8 @@ class SnapshotData std::vector queuedDiffs; + std::vector> trackedChanges; + // Note - we care about the order of this map, as we iterate through it // in order of offsets std::map mergeRegions; @@ -244,26 +270,6 @@ class SnapshotData 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); std::string snapshotMergeOpStr(SnapshotMergeOperation op); diff --git a/src/scheduler/Executor.cpp b/src/scheduler/Executor.cpp index 1c9c00a03..7576c6deb 100644 --- a/src/scheduler/Executor.cpp +++ b/src/scheduler/Executor.cpp @@ -5,35 +5,52 @@ #include #include #include +#include #include #include #include #include +#include #include #include #include #include +#include +#include #include #define POOL_SHUTDOWN -1 namespace faabric::scheduler { +static thread_local Executor* executingExecutor = nullptr; + +Executor* getExecutingExecutor() +{ + return executingExecutor; +} + +void setExecutingExecutor(Executor* exec) +{ + executingExecutor = exec; +} + ExecutorTask::ExecutorTask(int messageIndexIn, std::shared_ptr reqIn, std::shared_ptr> batchCounterIn, - bool needsSnapshotSyncIn, bool skipResetIn) : req(std::move(reqIn)) , batchCounter(std::move(batchCounterIn)) , messageIndex(messageIndexIn) - , needsSnapshotSync(needsSnapshotSyncIn) , skipReset(skipResetIn) {} // TODO - avoid the copy of the message here? Executor::Executor(faabric::Message& msg) : boundMessage(msg) + , sch(getScheduler()) + , reg(faabric::snapshot::getSnapshotRegistry()) + , tracker(faabric::util::getDirtyTracker()) , threadPoolSize(faabric::util::getUsableCores()) , threadPoolThreads(threadPoolSize) , threadTaskQueues(threadPoolSize) @@ -62,7 +79,7 @@ void Executor::finish() // Send a kill message SPDLOG_TRACE("Executor {} killing thread pool {}", id, i); threadTaskQueues[i].enqueue( - ExecutorTask(POOL_SHUTDOWN, nullptr, nullptr, false, false)); + ExecutorTask(POOL_SHUTDOWN, nullptr, nullptr, false)); faabric::util::UniqueLock threadsLock(threadsMutex); // Copy shared_ptr to avoid racing @@ -99,6 +116,189 @@ void Executor::finish() deadThreads.clear(); } +std::vector> Executor::executeThreads( + std::shared_ptr req, + const std::vector& mergeRegions) +{ + SPDLOG_DEBUG("Executor {} executing {} threads", id, req->messages_size()); + + faabric::Message& msg = req->mutable_messages()->at(0); + std::string snapshotKey = faabric::util::getMainThreadSnapshotKey(msg); + std::string funcStr = faabric::util::funcToString(msg, false); + + std::shared_ptr snap = nullptr; + bool exists = false; + { + faabric::util::SharedLock lock(threadExecutionMutex); + exists = reg.snapshotExists(snapshotKey); + } + + if (!exists) { + faabric::util::FullLock lock(threadExecutionMutex); + if (!reg.snapshotExists(snapshotKey)) { + SPDLOG_DEBUG( + "Creating main thread snapshot: {} for {}", snapshotKey, funcStr); + + snap = + std::make_shared(getMemoryView()); + reg.registerSnapshot(snapshotKey, snap); + } else { + exists = true; + } + } + + if (exists) { + SPDLOG_DEBUG( + "Main thread snapshot exists: {} for {}", snapshotKey, funcStr); + + // Get main snapshot + snap = reg.getSnapshot(snapshotKey); + std::span memView = getMemoryView(); + + // Get dirty regions since last batch of threads + tracker.stopTracking(memView); + tracker.stopThreadLocalTracking(memView); + + std::vector> dirtyRegions = + tracker.getBothDirtyOffsets(memView); + + // Apply changes to snapshot + snap->fillGapsWithOverwriteRegions(); + std::vector updates = + snap->diffWithDirtyRegions(memView, dirtyRegions); + + if (updates.empty()) { + SPDLOG_TRACE( + "No updates to main thread snapshot for {} from {} dirty regions", + faabric::util::funcToString(msg, false), + dirtyRegions.size()); + } else { + SPDLOG_DEBUG("Updating main thread snapshot for {} with {} diffs", + faabric::util::funcToString(msg, false), + updates.size()); + snap->queueDiffs(updates); + snap->writeQueuedDiffs(); + } + + snap->clearMergeRegions(); + } + + // Now we have to apply the merge regions for this parallel section + for (const auto& mr : mergeRegions) { + snap->addMergeRegion( + mr.offset, mr.length, mr.dataType, mr.operation, true); + } + + // TODO - here the main thread will wait, so technically frees up a slot + // that could be used. + std::string cacheKey = + std::to_string(msg.appid()) + "_" + std::to_string(req->messages_size()); + bool hasCachedDecision = false; + { + faabric::util::SharedLock lock(threadExecutionMutex); + hasCachedDecision = + cachedDecisionHosts.find(cacheKey) != cachedDecisionHosts.end(); + } + + if (!hasCachedDecision) { + faabric::util::FullLock lock(threadExecutionMutex); + if (cachedDecisionHosts.find(cacheKey) == cachedDecisionHosts.end()) { + // Set up a new group + int groupId = faabric::util::generateGid(); + for (auto& m : *req->mutable_messages()) { + m.set_groupid(groupId); + m.set_groupsize(req->messages_size()); + } + + // Invoke the functions + faabric::util::SchedulingDecision decision = sch.callFunctions(req); + + // Cache the decision for next time + SPDLOG_DEBUG( + "No cached decision for {} x {}/{}, caching group {}, hosts: {}", + req->messages().size(), + msg.user(), + msg.function(), + groupId, + faabric::util::vectorToString(decision.hosts)); + + cachedGroupIds[cacheKey] = groupId; + cachedDecisionHosts[cacheKey] = decision.hosts; + } else { + hasCachedDecision = true; + } + } + + if (hasCachedDecision) { + // Get the cached group ID and hosts + int groupId = cachedGroupIds[cacheKey]; + std::vector hosts = cachedDecisionHosts[cacheKey]; + + // Sanity check we've got something the right size + if (hosts.size() != req->messages().size()) { + SPDLOG_ERROR("Cached decision for {}/{} has {} hosts, expected {}", + msg.user(), + msg.function(), + hosts.size(), + req->messages().size()); + + throw std::runtime_error( + "Cached threads scheduling decision invalid"); + } + + // Create the scheduling hint + faabric::util::SchedulingDecision hint(msg.appid(), groupId); + for (int i = 0; i < hosts.size(); i++) { + // Reuse the group id + faabric::Message& m = req->mutable_messages()->at(i); + m.set_groupid(groupId); + m.set_groupsize(req->messages_size()); + + // Add to the decision + hint.addMessage(hosts.at(i), m); + } + + SPDLOG_DEBUG("Using cached decision for {}/{} {}, group {}", + msg.user(), + msg.function(), + msg.appid(), + hint.groupId); + + // Invoke the functions + sch.callFunctions(req, hint); + } + + // Await all child threads + std::vector> results; + results.reserve(req->messages_size()); + for (int i = 0; i < req->messages_size(); i++) { + uint32_t messageId = req->messages().at(i).id(); + + int result = sch.awaitThreadResult(messageId); + results.emplace_back(messageId, result); + } + + SPDLOG_DEBUG( + "Executor {} got results for {} threads", id, req->messages_size()); + + // Write queued changes to snapshot + snap->writeQueuedDiffs(); + + // Set memory size to fit new snapshot + setMemorySize(snap->getSize()); + + // Remap the memory + std::span memView = getMemoryView(); + snap->mapToMemory(memView); + + // Start tracking again + memView = getMemoryView(); + tracker.startTracking(memView); + tracker.startThreadLocalTracking(memView); + + return results; +} + void Executor::executeTasks(std::vector msgIdxs, std::shared_ptr req) { @@ -109,64 +309,74 @@ void Executor::executeTasks(std::vector msgIdxs, req->messages_size(), funcStr); - // Note that this lock is specific to this executor, so will only block when - // multiple threads are trying to schedule tasks. - // This will only happen when child threads of the same function are - // competing, hence is rare so we can afford to be conservative here. + // Note that this lock is specific to this executor, so will only block + // when multiple threads are trying to schedule tasks. This will only + // happen when child threads of the same function are competing to + // schedule more threads, hence is rare so we can afford to be + // conservative here. faabric::util::UniqueLock lock(threadsMutex); faabric::Message& firstMsg = req->mutable_messages()->at(0); - std::string snapshotKey = firstMsg.snapshotkey(); std::string thisHost = faabric::util::getSystemConfig().endpointHost; bool isMaster = firstMsg.masterhost() == thisHost; bool isThreads = req->type() == faabric::BatchExecuteRequest::THREADS; - bool isSnapshot = !snapshotKey.empty(); - // Restore if we have a snapshot - if (isSnapshot) { - SPDLOG_DEBUG("Restoring {} from snapshot {}", funcStr, snapshotKey); - restore(firstMsg); - } + if (isThreads) { + // Check we get a valid memory view + std::span memView = getMemoryView(); + if (memView.empty()) { + SPDLOG_ERROR("Can't execute threads for {}, empty memory view", + funcStr); + throw std::runtime_error("Empty memory view for threaded function"); + } - // Reset dirty page tracking if we're executing threads. - // Note this must be done after the restore has happened. - bool needsSnapshotSync = false; - if (isThreads && isSnapshot) { - faabric::util::resetDirtyTracking(); - needsSnapshotSync = true; + // Restore threads from main thread snapshot + std::string snapKey = faabric::util::getMainThreadSnapshotKey(firstMsg); + SPDLOG_DEBUG( + "Restoring thread of {} from snapshot {}", funcStr, snapKey); + restore(snapKey); + + // Get updated memory view and start global tracking of memory + memView = getMemoryView(); + tracker.startTracking(memView); + } else if (!firstMsg.snapshotkey().empty()) { + // Restore from snapshot if provided + std::string snapshotKey = firstMsg.snapshotkey(); + SPDLOG_DEBUG("Restoring {} from snapshot {}", funcStr, snapshotKey); + restore(snapshotKey); } // 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 happens for - // threads, as they will be restored from their respective snapshot on the - // next execution. + // 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 - // we start overloading + // one-to-one with thread pool threads. Only once the pool is exhausted + // do we start overloading for (int msgIdx : msgIdxs) { const faabric::Message& msg = req->messages().at(msgIdx); int threadPoolIdx = -1; if (availablePoolThreads.empty()) { // Here all threads are still executing, so we have to overload. - // If any tasks are blocking we risk a deadlock, and can no longer - // guarantee the application will finish. - // In general if we're on the master host and this is a thread, we - // should avoid the zeroth and first pool threads as they are likely - // to be the main thread and the zeroth in the communication group, + // If any tasks are blocking we risk a deadlock, and can no + // longer guarantee the application will finish. In general if + // we're on the master host and this is a thread, we should + // avoid the zeroth and first pool threads as they are likely to + // be the main thread and the zeroth in the communication group, // so will be blocking. if (isThreads && isMaster) { if (threadPoolSize <= 2) { - SPDLOG_ERROR( - "Insufficient pool threads ({}) to overload {} idx {}", - threadPoolSize, - funcStr, - msg.appidx()); + SPDLOG_ERROR("Insufficient pool threads ({}) to " + "overload {} idx {}", + threadPoolSize, + funcStr, + msg.appidx()); throw std::runtime_error("Insufficient pool threads"); } @@ -190,8 +400,8 @@ void Executor::executeTasks(std::vector msgIdxs, } // Enqueue the task - threadTaskQueues[threadPoolIdx].enqueue(ExecutorTask( - msgIdx, req, batchCounter, needsSnapshotSync, skipReset)); + threadTaskQueues[threadPoolIdx].enqueue( + ExecutorTask(msgIdx, req, batchCounter, skipReset)); // Lazily create the thread if (threadPoolThreads.at(threadPoolIdx) == nullptr) { @@ -201,11 +411,58 @@ void Executor::executeTasks(std::vector msgIdxs, } } +std::shared_ptr Executor::getMainThreadSnapshot( + faabric::Message& msg, + bool createIfNotExists) +{ + std::string snapshotKey = faabric::util::getMainThreadSnapshotKey(msg); + bool exists = false; + { + faabric::util::SharedLock lock(threadExecutionMutex); + exists = reg.snapshotExists(snapshotKey); + } + + if (!exists && createIfNotExists) { + faabric::util::FullLock lock(threadExecutionMutex); + if (!reg.snapshotExists(snapshotKey)) { + SPDLOG_DEBUG("Creating main thread snapshot: {} for {}", + snapshotKey, + faabric::util::funcToString(msg, false)); + + std::shared_ptr snap = + std::make_shared(getMemoryView()); + reg.registerSnapshot(snapshotKey, snap); + } else { + return reg.getSnapshot(snapshotKey); + } + } else if (!exists) { + SPDLOG_ERROR("No main thread snapshot {}", snapshotKey); + throw std::runtime_error("No main thread snapshot"); + } + + return reg.getSnapshot(snapshotKey); +} + +void Executor::deleteMainThreadSnapshot(const faabric::Message& msg) +{ + std::string snapshotKey = faabric::util::getMainThreadSnapshotKey(msg); + + if (reg.snapshotExists(snapshotKey)) { + SPDLOG_DEBUG("Deleting main thread snapshot for {}", + faabric::util::funcToString(msg, false)); + + // Broadcast the deletion + sch.broadcastSnapshotDelete(msg, snapshotKey); + + // Delete locally + reg.deleteSnapshot(snapshotKey); + } +} + void Executor::threadPoolThread(int threadPoolIdx) { SPDLOG_DEBUG("Thread pool thread {}:{} starting up", id, threadPoolIdx); - auto& sch = faabric::scheduler::getScheduler(); faabric::transport::PointToPointBroker& broker = faabric::transport::getPointToPointBroker(); const auto& conf = faabric::util::getSystemConfig(); @@ -220,8 +477,7 @@ void Executor::threadPoolThread(int threadPoolIdx) try { task = threadTaskQueues[threadPoolIdx].dequeue(conf.boundTimeout); } catch (faabric::util::QueueTimeoutException& ex) { - // If the thread has had no messages, it needs to - // remove itself + // If the thread has had no messages, it needs to remove itself SPDLOG_TRACE("Thread {}:{} got no messages in timeout {}ms", id, threadPoolIdx, @@ -242,6 +498,15 @@ void Executor::threadPoolThread(int threadPoolIdx) faabric::Message& msg = task.req->mutable_messages()->at(task.messageIndex); + // Start dirty tracking if executing threads + bool isThreads = + task.req->type() == faabric::BatchExecuteRequest::THREADS; + if (isThreads) { + // If tracking is thread local, start here as it will happen for + // each thread + tracker.startThreadLocalTracking(getMemoryView()); + } + // Check ptp group std::shared_ptr group = nullptr; if (msg.groupid() > 0) { @@ -250,8 +515,6 @@ void Executor::threadPoolThread(int threadPoolIdx) } bool isMaster = msg.masterhost() == conf.endpointHost; - bool isThreads = - task.req->type() == faabric::BatchExecuteRequest::THREADS; SPDLOG_TRACE("Thread {}:{} executing task {} ({}, thread={}, group={})", id, threadPoolIdx, @@ -260,6 +523,10 @@ void Executor::threadPoolThread(int threadPoolIdx) isThreads, msg.groupid()); + // Set executing executor + setExecutingExecutor(this); + + // Execute the task int32_t returnValue; try { returnValue = @@ -273,6 +540,22 @@ void Executor::threadPoolThread(int threadPoolIdx) msg.set_outputdata(errorMessage); } + // Handle thread-local diffing for every thread + if (isThreads) { + // Stop dirty tracking + std::span memView = getMemoryView(); + tracker.stopThreadLocalTracking(memView); + + // Add this thread's changes to executor-wide list of dirty regions + auto thisThreadDirtyRegions = + tracker.getThreadLocalDirtyOffsets(memView); + + faabric::util::FullLock lock(threadExecutionMutex); + dirtyRegions.insert(dirtyRegions.end(), + thisThreadDirtyRegions.begin(), + thisThreadDirtyRegions.end()); + } + // Set the return value msg.set_returnvalue(returnValue); @@ -289,48 +572,78 @@ void Executor::threadPoolThread(int threadPoolIdx) oldTaskCount - 1); // Handle snapshot diffs _before_ we reset the executor - faabric::util::MemoryView funcMemory = getMemoryView(); - if (!funcMemory.getData().empty() && isLastInBatch && - task.needsSnapshotSync) { - auto snap = faabric::snapshot::getSnapshotRegistry().getSnapshot( - msg.snapshotkey()); - - SPDLOG_TRACE("Diffing memory with pre-execution snapshot for {}", - msg.snapshotkey()); + if (isLastInBatch && isThreads) { + // Stop non-thread-local tracking as we're the last in the batch + std::span memView = getMemoryView(); + tracker.stopTracking(memView); + + // Add non-thread-local dirty regions + { + faabric::util::FullLock lock(threadExecutionMutex); + std::vector> r = + tracker.getDirtyOffsets(memView); + + dirtyRegions.insert(dirtyRegions.end(), r.begin(), r.end()); + } - // Fill gaps with overwrites + // Fill snapshot gaps with overwrite regions first + std::string mainThreadSnapKey = + faabric::util::getMainThreadSnapshotKey(msg); + auto snap = reg.getSnapshot(mainThreadSnapKey); snap->fillGapsWithOverwriteRegions(); - // Work out the diffs - std::vector diffs = - funcMemory.diffWithSnapshot(snap); - - // On master we queue the diffs locally directly, on a remote host - // we push them back to master - if (isMaster) { - SPDLOG_DEBUG("Queueing {} diffs for {} to snapshot {} on " - "master (group {})", - diffs.size(), - faabric::util::funcToString(msg, false), - msg.snapshotkey(), - msg.groupid()); - - snap->queueDiffs(diffs); - } else { - sch.pushSnapshotDiffs(msg, diffs); + // Compare snapshot with all dirty regions for this executor + std::vector diffs; + { + // Do the diffing + faabric::util::FullLock lock(threadExecutionMutex); + diffs = snap->diffWithDirtyRegions(memView, dirtyRegions); + dirtyRegions.clear(); } - // Reset dirty page tracking - faabric::util::resetDirtyTracking(); + if (diffs.empty()) { + SPDLOG_DEBUG("No diffs for {}", mainThreadSnapKey); + } else { + SPDLOG_DEBUG( + "Queueing {} diffs for {} to snapshot {} (group {})", + diffs.size(), + faabric::util::funcToString(msg, false), + mainThreadSnapKey, + msg.groupid()); + + // On master we queue the diffs locally directly, on a remote + // host we push them back to master + if (isMaster) { + snap->queueDiffs(diffs); + } else if (isLastInBatch) { + sch.pushSnapshotDiffs(msg, mainThreadSnapKey, diffs); + } + } - // Clear merge regions - SPDLOG_DEBUG("Clearing merge regions for {}", msg.snapshotkey()); + // If last in batch on this host, clear the merge regions + SPDLOG_DEBUG("Clearing merge regions for {}", mainThreadSnapKey); snap->clearMergeRegions(); } - // If this batch is finished, reset the executor and release its claim. - // Note that we have to release the claim _after_ resetting, otherwise - // the executor won't be ready for reuse. + // If this is not a threads request and last in its batch, it may be + // the main function in a threaded application, in which case we + // want to stop any tracking and delete the main thread snapshot + if (!isThreads && isLastInBatch) { + // Stop tracking memory + std::span memView = getMemoryView(); + if (!memView.empty()) { + tracker.stopTracking(memView); + tracker.stopThreadLocalTracking(memView); + + // Delete the main thread snapshot (implicitly does nothing if + // doesn't exist) + deleteMainThreadSnapshot(msg); + } + } + + // If this batch is finished, reset the executor and release its + // claim. Note that we have to release the claim _after_ resetting, + // otherwise the executor won't be ready for reuse if (isLastInBatch) { if (task.skipReset) { SPDLOG_TRACE("Skipping reset for {}", @@ -349,14 +662,15 @@ void Executor::threadPoolThread(int threadPoolIdx) } // Vacate the slot occupied by this task. This must be done after - // releasing the claim on this executor, otherwise the scheduler may try - // to schedule another function and be unable to reuse this executor. + // releasing the claim on this executor, otherwise the scheduler may + // try to schedule another function and be unable to reuse this + // executor. sch.vacateSlot(); - // Finally set the result of the task, this will allow anything waiting - // on its result to continue execution, therefore must be done once the - // executor has been reset, otherwise the executor may not be reused for - // a repeat invocation. + // Finally set the result of the task, this will allow anything + // waiting on its result to continue execution, therefore must be + // done once the executor has been reset, otherwise the executor may + // not be reused for a repeat invocation. if (isThreads) { // Set non-final thread result sch.setThreadResult(msg, returnValue); @@ -370,8 +684,8 @@ void Executor::threadPoolThread(int threadPoolIdx) SPDLOG_DEBUG( "Shutting down thread pool thread {}:{}", id, threadPoolIdx); - // Note - we have to keep a record of dead threads so we can join them - // all when the executor shuts down + // Note - we have to keep a record of dead threads so we can join + // them all when the executor shuts down bool isFinished = true; { faabric::util::UniqueLock threadsLock(threadsMutex); @@ -379,6 +693,12 @@ void Executor::threadPoolThread(int threadPoolIdx) threadPoolThreads.at(threadPoolIdx); deadThreads.emplace_back(thisThread); + // Make sure we're definitely not still tracking changes + std::span memView = getMemoryView(); + if (!memView.empty()) { + tracker.stopTracking(memView); + } + // Set this thread to nullptr threadPoolThreads.at(threadPoolIdx) = nullptr; @@ -430,14 +750,19 @@ void Executor::postFinish() {} void Executor::reset(faabric::Message& msg) {} -faabric::util::MemoryView Executor::getMemoryView() +std::span Executor::getMemoryView() { SPDLOG_WARN("Executor for {} has not implemented memory view method", faabric::util::funcToString(boundMessage, false)); - return faabric::util::MemoryView(); + return {}; +} + +void Executor::setMemorySize(size_t newSize) +{ + SPDLOG_WARN("Executor has not implemented set memory size method"); } -void Executor::restore(faabric::Message& msg) +void Executor::restore(const std::string& snapshotKey) { SPDLOG_WARN("Executor has not implemented restore method"); } diff --git a/src/scheduler/Scheduler.cpp b/src/scheduler/Scheduler.cpp index 23a5d30ed..6dfcbb072 100644 --- a/src/scheduler/Scheduler.cpp +++ b/src/scheduler/Scheduler.cpp @@ -493,20 +493,34 @@ faabric::util::SchedulingDecision Scheduler::doCallFunctions( // *all* hosts, regardless of whether they will be executing functions. // This greatly simplifies the reasoning about which hosts hold which // diffs. - std::string snapshotKey = firstMsg.snapshotkey(); + + std::string snapshotKey; + if (isThreads) { + if (!firstMsg.snapshotkey().empty()) { + SPDLOG_ERROR("{} should not provide snapshot key for {} threads", + funcStr, + req->messages().size()); + + std::runtime_error("Should not provide snapshot key for threads"); + } + + snapshotKey = faabric::util::getMainThreadSnapshotKey(firstMsg); + } else { + snapshotKey = firstMsg.snapshotkey(); + } + if (!snapshotKey.empty()) { + auto snap = + faabric::snapshot::getSnapshotRegistry().getSnapshot(snapshotKey); + for (const auto& host : getFunctionRegisteredHosts(firstMsg, false)) { SnapshotClient& c = getSnapshotClient(host); - 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 so, just push the diffs that have occurred in this main thread if (pushedSnapshotsMap[snapshotKey].contains(host)) { - MemoryView snapMemView({ snap->getDataPtr(), snap->getSize() }); - std::vector snapshotDiffs = - snapMemView.getDirtyRegions(); + snap->getTrackedChanges(); c.pushSnapshotUpdate(snapshotKey, snap, snapshotDiffs); } else { @@ -514,10 +528,10 @@ faabric::util::SchedulingDecision Scheduler::doCallFunctions( pushedSnapshotsMap[snapshotKey].insert(host); } } - } - // Now reset the dirty page tracking just before we start executing - faabric::util::resetDirtyTracking(); + // Now reset the tracking on the snapshot before we start executing + snap->clearTrackedChanges(); + } // ------------------------------------------- // EXECUTION @@ -611,7 +625,6 @@ faabric::util::SchedulingDecision Scheduler::doCallFunctions( // ------------------------------------------- // REMOTE EXECTUION // ------------------------------------------- - SPDLOG_DEBUG("Scheduling {}/{} calls to {} on {}", thisHostIdxs.size(), nMessages, @@ -900,6 +913,7 @@ void Scheduler::setThreadResult(const faabric::Message& msg, void Scheduler::pushSnapshotDiffs( const faabric::Message& msg, + const std::string& snapshotKey, const std::vector& diffs) { if (diffs.empty()) { @@ -907,21 +921,21 @@ void Scheduler::pushSnapshotDiffs( } 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); + snapshotKey); throw std::runtime_error("Cannot push snapshot diffs on master"); } SnapshotClient& c = getSnapshotClient(msg.masterhost()); - c.pushSnapshotDiffs(snapKey, diffs); + c.pushSnapshotDiffs(snapshotKey, diffs); } void Scheduler::setThreadResultLocally(uint32_t msgId, int32_t returnValue) { - faabric::util::SharedLock lock(mx); + faabric::util::FullLock lock(mx); SPDLOG_DEBUG("Setting result for thread {} to {}", msgId, returnValue); threadResults.at(msgId).set_value(returnValue); } diff --git a/src/snapshot/SnapshotRegistry.cpp b/src/snapshot/SnapshotRegistry.cpp index 50b69990f..98ca319fa 100644 --- a/src/snapshot/SnapshotRegistry.cpp +++ b/src/snapshot/SnapshotRegistry.cpp @@ -12,7 +12,6 @@ namespace faabric::snapshot { std::shared_ptr SnapshotRegistry::getSnapshot( const std::string& key) { - PROF_START(GetSnapshot) faabric::util::SharedLock lock(snapshotsMx); if (key.empty()) { @@ -20,12 +19,11 @@ std::shared_ptr SnapshotRegistry::getSnapshot( throw std::runtime_error("Getting snapshot with empty key"); } - if (snapshotMap.count(key) == 0) { + if (snapshotMap.find(key) == snapshotMap.end()) { SPDLOG_ERROR("Snapshot for {} does not exist", key); throw std::runtime_error("Snapshot doesn't exist"); } - PROF_END(GetSnapshot) return snapshotMap[key]; } @@ -41,17 +39,15 @@ void SnapshotRegistry::registerSnapshot( { faabric::util::FullLock lock(snapshotsMx); - SPDLOG_TRACE("Registering snapshot {} size {}", key, data->getSize()); + SPDLOG_DEBUG("Registering snapshot {} size {}", key, data->getSize()); snapshotMap.insert_or_assign(key, std::move(data)); - - // Reset dirty tracking - faabric::util::resetDirtyTracking(); } void SnapshotRegistry::deleteSnapshot(const std::string& key) { faabric::util::FullLock lock(snapshotsMx); + SPDLOG_DEBUG("Deleting snapshot {}", key); snapshotMap.erase(key); } @@ -70,6 +66,7 @@ SnapshotRegistry& getSnapshotRegistry() void SnapshotRegistry::clear() { faabric::util::FullLock lock(snapshotsMx); + SPDLOG_DEBUG("Deleting all snapshots"); snapshotMap.clear(); } } diff --git a/src/snapshot/SnapshotServer.cpp b/src/snapshot/SnapshotServer.cpp index d53392f09..3eaedcaf6 100644 --- a/src/snapshot/SnapshotServer.cpp +++ b/src/snapshot/SnapshotServer.cpp @@ -99,6 +99,8 @@ std::unique_ptr SnapshotServer::recvPushSnapshot( // Register snapshot reg.registerSnapshot(snapKey, snap); + snap->clearTrackedChanges(); + // Send response return std::make_unique(); } @@ -123,7 +125,7 @@ SnapshotServer::recvPushSnapshotDiffs(const uint8_t* buffer, size_t bufferSize) flatbuffers::GetRoot(buffer); SPDLOG_DEBUG( - "Applying {} diffs to snapshot {}", r->diffs()->size(), r->key()->str()); + "Queueing {} diffs for snapshot {}", r->diffs()->size(), r->key()->str()); // Get the snapshot faabric::snapshot::SnapshotRegistry& reg = @@ -146,6 +148,9 @@ SnapshotServer::recvPushSnapshotDiffs(const uint8_t* buffer, size_t bufferSize) // Write diffs and set merge regions if necessary if (r->force()) { + SPDLOG_DEBUG("Forcing write queued diffs to snapshot {}", + r->key()->str()); + // Write queued diffs snap->writeQueuedDiffs(); diff --git a/src/util/CMakeLists.txt b/src/util/CMakeLists.txt index 7c7a40a29..e891fd668 100644 --- a/src/util/CMakeLists.txt +++ b/src/util/CMakeLists.txt @@ -6,6 +6,7 @@ faabric_lib(util clock.cpp crash.cpp delta.cpp + dirty.cpp environment.cpp exec_graph.cpp files.cpp diff --git a/src/util/bytes.cpp b/src/util/bytes.cpp index 43666b242..49bbdf2a8 100644 --- a/src/util/bytes.cpp +++ b/src/util/bytes.cpp @@ -1,6 +1,9 @@ #include +#include +#include #include +#include #include namespace faabric::util { @@ -90,4 +93,57 @@ std::string formatByteArrayToIntString(const std::vector& bytes) return ss.str(); } + +// This function is called in a tight loop over large regions of data so +// make sure it stays efficient. +std::vector> diffArrayRegions( + std::span a, + std::span b) +{ + PROF_START(ByteArrayDiff) + std::vector> regions; + + // Iterate through diffs and work out start and finish offsets of each dirty + // region + uint32_t diffStart = 0; + bool diffInProgress = false; + for (uint32_t i = 0; i < a.size(); i++) { + bool dirty = a.data()[i] != b.data()[i]; + if (dirty && !diffInProgress) { + // Starts at this byte + diffInProgress = true; + diffStart = i; + } else if (!dirty && diffInProgress) { + // Finished on byte before + diffInProgress = false; + regions.emplace_back(diffStart, i - diffStart); + } + } + + // If we finish with a diff in progress, add it + if (diffInProgress) { + regions.emplace_back(diffStart, a.size() - diffStart); + } + + PROF_END(ByteArrayDiff) + return regions; +} + +std::vector diffArrays(std::span a, + std::span b) +{ + if (a.size() != b.size()) { + SPDLOG_ERROR( + "Cannot diff arrays of different sizes {} != {}", a.size(), b.size()); + throw std::runtime_error("Cannot diff arrays of different sizes"); + } + + std::vector diffs(a.size(), false); + for (int i = 0; i < a.size(); i++) { + diffs[i] = a.data()[i] != b.data()[i]; + } + + return diffs; +} + } diff --git a/src/util/config.cpp b/src/util/config.cpp index 0260cecae..56ac5a831 100644 --- a/src/util/config.cpp +++ b/src/util/config.cpp @@ -68,6 +68,9 @@ void SystemConfig::initialise() this->getSystemConfIntParam("SNAPSHOT_SERVER_THREADS", "2"); pointToPointServerThreads = this->getSystemConfIntParam("POINT_TO_POINT_SERVER_THREADS", "2"); + + // Dirty tracking + dirtyTrackingMode = getEnvVar("DIRTY_TRACKING_MODE", "segfault"); } int SystemConfig::getSystemConfIntParam(const char* name, diff --git a/src/util/crash.cpp b/src/util/crash.cpp index 9a6e97f17..999994813 100644 --- a/src/util/crash.cpp +++ b/src/util/crash.cpp @@ -14,6 +14,13 @@ constexpr int TEST_SIGNAL = 12341234; // Must be async-signal-safe - don't call allocating functions void crashHandler(int sig) noexcept +{ + faabric::util::handleCrash(sig); +} + +namespace faabric::util { + +void handleCrash(int sig) { std::array stackPtrs; size_t filledStacks = backtrace(stackPtrs.data(), stackPtrs.size()); @@ -28,18 +35,22 @@ void crashHandler(int sig) noexcept raise(sig); exit(1); } - return; } -namespace faabric::util { - -void setUpCrashHandler() +void setUpCrashHandler(int sig) { - fputs("Testing crash handler backtrace:\n", stderr); - fflush(stderr); - crashHandler(TEST_SIGNAL); - SPDLOG_INFO("Installing crash handler"); - for (auto signo : { SIGSEGV, SIGABRT, SIGILL, SIGFPE }) { + std::vector sigs; + if (sig >= 0) { + sigs = { sig }; + } else { + fputs("Testing crash handler backtrace:\n", stderr); + fflush(stderr); + crashHandler(TEST_SIGNAL); + SPDLOG_INFO("Installing crash handler"); + sigs = { SIGSEGV, SIGABRT, SIGILL, SIGFPE }; + } + + for (auto signo : sigs) { if (signal(signo, &crashHandler) == SIG_ERR) { SPDLOG_WARN("Couldn't install handler for signal {}", signo); } else { diff --git a/src/util/dirty.cpp b/src/util/dirty.cpp new file mode 100644 index 000000000..3e34295ef --- /dev/null +++ b/src/util/dirty.cpp @@ -0,0 +1,382 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include + +namespace faabric::util { + +DirtyTracker& getDirtyTracker() +{ + static SoftPTEDirtyTracker softpte; + static SegfaultDirtyTracker sigseg; + + std::string trackMode = faabric::util::getSystemConfig().dirtyTrackingMode; + if (trackMode == "softpte") { + return softpte; + } else if (trackMode == "segfault") { + sigseg.reinitialise(); + return sigseg; + } else { + throw std::runtime_error("Unrecognised dirty tracking mode"); + } +} + +// ---------------------------------- +// Soft dirty PTE +// ---------------------------------- + +SoftPTEDirtyTracker::SoftPTEDirtyTracker() +{ + clearRefsFile = ::fopen(CLEAR_REFS, "w"); + if (clearRefsFile == nullptr) { + SPDLOG_ERROR("Could not open clear_refs ({})", strerror(errno)); + throw std::runtime_error("Could not open clear_refs"); + } + + pagemapFile = ::fopen(PAGEMAP, "rb"); + if (pagemapFile == nullptr) { + SPDLOG_ERROR("Could not open pagemap ({})", strerror(errno)); + throw std::runtime_error("Could not open pagemap"); + } + + // Disable buffering, we want to repeatedly read updates to the same file + setbuf(pagemapFile, nullptr); +} + +SoftPTEDirtyTracker::~SoftPTEDirtyTracker() +{ + ::fclose(clearRefsFile); + ::fclose(pagemapFile); +} + +void SoftPTEDirtyTracker::clearAll() +{ + PROF_START(ClearSoftPTE) + // Write 4 to the file to reset and start tracking + // https://www.kernel.org/doc/html/v5.4/admin-guide/mm/soft-dirty.html + char value[] = "4"; + size_t nWritten = ::fwrite(value, sizeof(char), 1, clearRefsFile); + + if (nWritten != 1) { + SPDLOG_ERROR("Failed to write to clear_refs ({})", nWritten); + ::fclose(clearRefsFile); + throw std::runtime_error("Failed to write to clear_refs"); + } + + ::rewind(clearRefsFile); + PROF_END(ClearSoftPTE) +} + +void SoftPTEDirtyTracker::startTracking(std::span region) +{ + clearAll(); +} + +void SoftPTEDirtyTracker::startThreadLocalTracking(std::span region) +{ + // Do nothing +} + +void SoftPTEDirtyTracker::stopTracking(std::span region) +{ + // Do nothing +} + +void SoftPTEDirtyTracker::stopThreadLocalTracking(std::span region) +{ + // Do nothing +} + +std::vector> SoftPTEDirtyTracker::getDirtyOffsets( + std::span region) +{ + PROF_START(GetDirtyRegions) + + int nPages = getRequiredHostPages(region.size()); + uint8_t* ptr = region.data(); + uintptr_t vptr = (uintptr_t)ptr; + + // Work out offset for this pointer in the pagemap + off_t offset = (vptr / HOST_PAGE_SIZE) * PAGEMAP_ENTRY_BYTES; + + // Skip to location of this page (note that fd is open in binary mode) + int r = ::fseek(pagemapFile, offset, 0); + if (r != 0) { + SPDLOG_ERROR("Could not seek pagemap to {}, returned {}", offset, r); + throw std::runtime_error("Could not seek pagemap"); + } + + // Read the entries + std::vector entries(nPages, 0); + int nRead = + ::fread(entries.data(), PAGEMAP_ENTRY_BYTES, nPages, pagemapFile); + if (nRead != nPages) { + SPDLOG_ERROR("Could not read pagemap ({} != {})", nRead, nPages); + throw std::runtime_error("Could not read pagemap"); + } + + // Iterate through the pagemap entries to work out which are dirty + std::vector> regions; + std::vector dirtyPages; + bool isRegionInProgress = false; + int pageStart = 0; + for (int i = 0; i < nPages; i++) { + bool isDirty = entries.at(i) & PAGEMAP_SOFT_DIRTY; + + if (isDirty && !isRegionInProgress) { + isRegionInProgress = true; + pageStart = i; + } else if (!isDirty && isRegionInProgress) { + isRegionInProgress = false; + regions.emplace_back(pageStart * HOST_PAGE_SIZE, + (i - pageStart) * HOST_PAGE_SIZE); + } + } + + if (isRegionInProgress) { + regions.emplace_back(pageStart * HOST_PAGE_SIZE, + (nPages - pageStart) * HOST_PAGE_SIZE); + } + + SPDLOG_TRACE( + "Out of {} pages, found {} dirty regions", nPages, regions.size()); + + PROF_END(GetDirtyRegions) + return regions; +} + +std::vector> +SoftPTEDirtyTracker::getBothDirtyOffsets(std::span region) +{ + return getDirtyOffsets(region); +} + +std::vector> +SoftPTEDirtyTracker::getThreadLocalDirtyOffsets(std::span region) +{ + return {}; +} + +void SoftPTEDirtyTracker::reinitialise() {} + +// ------------------------------ +// Segfaults +// ------------------------------ + +class ThreadTrackingData +{ + public: + ThreadTrackingData() = default; + + ThreadTrackingData(std::span region) + : regionBase(region.data()) + , regionTop(region.data() + region.size()) + , nPages(faabric::util::getRequiredHostPages(region.size())) + , pageFlags(nPages, '0') + {} + + void markDirtyPage(void* addr) + { + ptrdiff_t offset = ((uint8_t*)addr) - regionBase; + long pageNum = offset / HOST_PAGE_SIZE; + pageFlags[pageNum] = '1'; + } + + std::vector> getDirtyRegions() + { + PROF_START(GetDirtyRegions) + std::vector> dirty; + if (regionBase == nullptr) { + return dirty; + } + + uint32_t diffPageStart = 0; + bool diffInProgress = false; + for (int i = 0; i < nPages; i++) { + bool isDirty = pageFlags[i] == '1'; + if (isDirty && !diffInProgress) { + diffInProgress = true; + diffPageStart = i; + } else if (!isDirty && diffInProgress) { + diffInProgress = false; + dirty.emplace_back(diffPageStart * HOST_PAGE_SIZE, + (i - diffPageStart) * HOST_PAGE_SIZE); + } + } + + if (diffInProgress) { + dirty.emplace_back(diffPageStart * HOST_PAGE_SIZE, + (nPages - diffPageStart) * HOST_PAGE_SIZE); + } + + PROF_END(GetDirtyRegions) + return dirty; + } + + bool isInitialised() { return regionTop != nullptr; } + + private: + uint8_t* regionBase = nullptr; + uint8_t* regionTop = nullptr; + int nPages = 0; + + // Note - std::vector here seems to worsen performance by >4x + std::vector pageFlags; +}; + +static thread_local ThreadTrackingData tracking; + +SegfaultDirtyTracker::SegfaultDirtyTracker() +{ + setUpSignalHandler(); +} + +void SegfaultDirtyTracker::setUpSignalHandler() +{ + // See sigaction docs + // https://www.man7.org/linux/man-pages/man2/sigaction.2.html + struct sigaction sa; + sa.sa_flags = SA_SIGINFO | SA_NODEFER; + + sigemptyset(&sa.sa_mask); + sigaddset(&sa.sa_mask, SIGSEGV); + + sa.sa_sigaction = SegfaultDirtyTracker::handler; + if (sigaction(SIGSEGV, &sa, NULL) == -1) { + throw std::runtime_error("Failed sigaction"); + } + + SPDLOG_TRACE("Set up dirty tracking signal handler"); +} + +void SegfaultDirtyTracker::handler(int sig, + siginfo_t* info, + void* ucontext) noexcept +{ + void* faultAddr = info->si_addr; + + if (!tracking.isInitialised()) { + // Unexpected segfault, treat as normal + faabric::util::handleCrash(sig); + } + + tracking.markDirtyPage(faultAddr); + + // Align down to nearest page boundary + uintptr_t addr = (uintptr_t)faultAddr; + addr &= -HOST_PAGE_SIZE; + auto* alignedAddr = (void*)addr; + + // Remove write protection from page + if (::mprotect(alignedAddr, HOST_PAGE_SIZE, PROT_READ | PROT_WRITE) == -1) { + SPDLOG_ERROR("WARNING: mprotect failed to unset read-only"); + } +} + +void SegfaultDirtyTracker::clearAll() +{ + tracking = ThreadTrackingData(); +} + +void SegfaultDirtyTracker::startThreadLocalTracking(std::span region) +{ + if (region.empty() || region.data() == nullptr) { + return; + } + + SPDLOG_TRACE("Starting thread-local tracking on region size {}", + region.size()); + tracking = ThreadTrackingData(region); +} + +void SegfaultDirtyTracker::startTracking(std::span region) +{ + SPDLOG_TRACE("Starting tracking on region size {}", region.size()); + + if (region.empty() || region.data() == nullptr) { + return; + } + + PROF_START(MprotectStart) + + // Note that here we want to mark the memory read-only, this is to ensure + // that only writes are counted as dirtying a page. + if (::mprotect(region.data(), region.size(), PROT_READ) == -1) { + SPDLOG_ERROR("Failed to start tracking with mprotect: {} ({})", + errno, + strerror(errno)); + throw std::runtime_error("Failed mprotect to start tracking"); + } + + PROF_END(MprotectStart) +} + +void SegfaultDirtyTracker::stopTracking(std::span region) +{ + if (region.empty() || region.data() == nullptr) { + return; + } + + SPDLOG_TRACE("Stopping tracking on region size {}", region.size()); + + PROF_START(MprotectEnd) + + if (::mprotect(region.data(), region.size(), PROT_READ | PROT_WRITE) == + -1) { + SPDLOG_ERROR("Failed to stop tracking with mprotect: {} ({})", + errno, + strerror(errno)); + throw std::runtime_error("Failed mprotect to stop tracking"); + } + + PROF_END(MprotectEnd) +} + +void SegfaultDirtyTracker::stopThreadLocalTracking(std::span region) +{ + // Do nothing - need to preserve thread-local data for getting dirty regions + SPDLOG_TRACE("Stopping thread-local tracking on region size {}", + region.size()); +} + +void SegfaultDirtyTracker::reinitialise() +{ + if (faabric::util::isTestMode()) { + // This is a hack because catch changes the segfault signal handler + // between test cases, so we have to reinisiatlise + setUpSignalHandler(); + } +} + +std::vector> +SegfaultDirtyTracker::getThreadLocalDirtyOffsets(std::span region) +{ + return tracking.getDirtyRegions(); +} + +std::vector> +SegfaultDirtyTracker::getDirtyOffsets(std::span region) +{ + return {}; +} + +std::vector> +SegfaultDirtyTracker::getBothDirtyOffsets(std::span region) +{ + return getThreadLocalDirtyOffsets(region); +} +} diff --git a/src/util/func.cpp b/src/util/func.cpp index af00c5475..67036cbb6 100644 --- a/src/util/func.cpp +++ b/src/util/func.cpp @@ -177,4 +177,13 @@ std::vector getArgvForMessage(const faabric::Message& msg) return argv; } + +std::string getMainThreadSnapshotKey(const faabric::Message& msg) +{ + std::string funcStr = faabric::util::funcToString(msg, false); + assert(msg.appid() > 0); + + std::string snapshotKey = funcStr + "_" + std::to_string(msg.appid()); + return snapshotKey; +} } diff --git a/src/util/logging.cpp b/src/util/logging.cpp index 2387c10d9..0b894d58d 100644 --- a/src/util/logging.cpp +++ b/src/util/logging.cpp @@ -6,7 +6,7 @@ namespace faabric::util { void initLogging() { // Docs: https://github.com/gabime/spdlog/wiki/3.-Custom-formatting - spdlog::set_pattern("[%H:%M:%S:%e] [%t] [%l] %v"); + spdlog::set_pattern("%^[%H:%M:%S] [%t] [%L]%$ %v"); // Set the current level for the level for the default logger (note that the // minimum log level is determined in the header). diff --git a/src/util/memory.cpp b/src/util/memory.cpp index bbcd6b1c9..3756b54fb 100644 --- a/src/util/memory.cpp +++ b/src/util/memory.cpp @@ -10,14 +10,42 @@ #include #include -#define CLEAR_REFS "/proc/self/clear_refs" -#define PAGEMAP "/proc/self/pagemap" +namespace faabric::util { -#define PAGEMAP_ENTRY_BYTES 8 -#define PAGEMAP_SOFT_DIRTY (1Ull << 55) +std::vector> dedupeMemoryRegions( + std::vector>& regions) +{ + if (regions.empty()) { + return {}; + } -namespace faabric::util { + std::vector> deduped; + + // Sort in place + std::sort( + std::begin(regions), + std::end(regions), + [](std::pair& a, std::pair& b) { + return a.first < b.first; + }); + + deduped.push_back(regions.front()); + uint32_t lastOffset = regions.front().first; + for (int i = 1; i < regions.size(); i++) { + const auto& r = regions.at(i); + assert(r.first >= lastOffset); + + if (r.first > lastOffset) { + deduped.push_back(r); + lastOffset = r.first; + } else if (deduped.back().second < r.second) { + deduped.pop_back(); + deduped.push_back(r); + } + } + return deduped; +} // ------------------------- // Alignment // ------------------------- @@ -77,131 +105,6 @@ AlignedChunk getPageAlignedChunk(long offset, long length) return c; } -// ------------------------- -// Dirty page tracking -// ------------------------- - -class ClearRefsWrapper -{ - public: - ClearRefsWrapper() - { - f = ::fopen(CLEAR_REFS, "w"); - if (f == nullptr) { - SPDLOG_ERROR("Could not open clear_refs ({})", strerror(errno)); - throw std::runtime_error("Could not open clear_refs"); - } - } - - ~ClearRefsWrapper() { ::fclose(f); } - - void reset() - { - // Write 4 to the file to track from now on - // https://www.kernel.org/doc/html/v5.4/admin-guide/mm/soft-dirty.html - char value[] = "4"; - size_t nWritten = ::fwrite(value, sizeof(char), 1, f); - - if (nWritten != 1) { - SPDLOG_ERROR("Failed to write to clear_refs ({})", nWritten); - ::fclose(f); - throw std::runtime_error("Failed to write to clear_refs"); - } - - ::rewind(f); - } - - private: - FILE* f = nullptr; -}; - -void resetDirtyTracking() -{ - static ClearRefsWrapper wrap; - - PROF_START(ResetDirty) - - SPDLOG_DEBUG("Resetting dirty tracking"); - wrap.reset(); - - PROF_END(ResetDirty) -} - -std::vector readPagemapEntries(uintptr_t ptr, int nEntries) -{ - // Work out offset for this pointer in the pagemap - off_t offset = (ptr / getpagesize()) * PAGEMAP_ENTRY_BYTES; - - // Open the pagemap - FILE* fd = fopen(PAGEMAP, "rb"); - if (fd == nullptr) { - SPDLOG_ERROR("Could not open pagemap ({})", strerror(errno)); - throw std::runtime_error("Could not open pagemap"); - } - - // Skip to location of this page - int r = fseek(fd, offset, SEEK_SET); - if (r < 0) { - SPDLOG_ERROR("Could not seek pagemap ({})", r); - throw std::runtime_error("Could not seek pagemap"); - } - - // Read the entries - std::vector entries(nEntries, 0); - int nRead = fread(entries.data(), PAGEMAP_ENTRY_BYTES, nEntries, fd); - if (nRead != nEntries) { - SPDLOG_ERROR("Could not read pagemap ({} != {})", nRead, nEntries); - throw std::runtime_error("Could not read pagemap"); - } - - fclose(fd); - - return entries; -} - -std::vector getDirtyPageNumbers(const uint8_t* ptr, int nPages) -{ - uintptr_t vptr = (uintptr_t)ptr; - - // Get the pagemap entries - std::vector entries = readPagemapEntries(vptr, nPages); - - // Iterate through to get boolean flags - std::vector pageNumbers; - for (int i = 0; i < nPages; i++) { - if (entries.at(i) & PAGEMAP_SOFT_DIRTY) { - pageNumbers.emplace_back(i); - } - } - - return pageNumbers; -} - -std::vector> getDirtyRegions(const uint8_t* ptr, - int nPages) -{ - std::vector dirtyPages = getDirtyPageNumbers(ptr, nPages); - - // Add a new region for each page, unless the one before it was also dirty, - // in which case we merge them - std::vector> regions; - for (int p = 0; p < dirtyPages.size(); p++) { - int thisPageNum = dirtyPages.at(p); - uint32_t thisPageStart = thisPageNum * HOST_PAGE_SIZE; - uint32_t thisPageEnd = thisPageStart + HOST_PAGE_SIZE; - - if (p > 0 && dirtyPages.at(p - 1) == thisPageNum - 1) { - // Previous page was also dirty, just update last region - regions.back().second = thisPageEnd; - } else { - // Previous page wasn't dirty, add new region - regions.emplace_back(thisPageStart, thisPageEnd); - } - } - - return regions; -} - // ------------------------- // Allocation // ------------------------- diff --git a/src/util/snapshot.cpp b/src/util/snapshot.cpp index 8b8bf378e..5018c6fb5 100644 --- a/src/util/snapshot.cpp +++ b/src/util/snapshot.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -82,20 +83,20 @@ 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) { + size_t regionEnd = offset + buffer.size(); + if (regionEnd > size) { + if (regionEnd > maxSize) { SPDLOG_ERROR( - "Copying snapshot data over max: {} > {}", newSize, maxSize); + "Copying snapshot data over max: {} > {}", regionEnd, maxSize); throw std::runtime_error("Copying snapshot data over max"); } - claimVirtualMemory({ data.get(), newSize }); - size = newSize; + claimVirtualMemory({ data.get(), regionEnd }); + size = regionEnd; // Update fd if (fd > 0) { - resizeFd(fd, newSize); + resizeFd(fd, regionEnd); } // Remap data @@ -105,6 +106,9 @@ void SnapshotData::writeData(std::span buffer, uint32_t offset) // Copy in new data uint8_t* copyTarget = validatedOffsetPtr(offset); ::memcpy(copyTarget, buffer.data(), buffer.size()); + + // Record the change + trackedChanges.emplace_back(offset, regionEnd); } const uint8_t* SnapshotData::getDataPtr(uint32_t offset) @@ -254,9 +258,6 @@ void SnapshotData::mapToMemory(std::span target) faabric::util::mapMemoryPrivate(target, fd); - // Reset dirty tracking otherwise whole mapped region is marked dirty - faabric::util::resetDirtyTracking(); - PROF_END(MapSnapshot) } @@ -394,81 +395,65 @@ void SnapshotData::writeQueuedDiffs() PROF_END(WriteQueuedDiffs) } -MemoryView::MemoryView(std::span dataIn) - : data(dataIn) -{} - -std::vector MemoryView::getDirtyRegions() +void SnapshotData::clearTrackedChanges() { - PROF_START(GetDirtyRegions) - if (data.empty()) { - return {}; - } + faabric::util::FullLock lock(snapMx); + trackedChanges.clear(); +} - // Get dirty regions - int nPages = getRequiredHostPages(data.size()); - std::vector dirtyPageNumbers = - getDirtyPageNumbers(data.data(), nPages); +std::vector SnapshotData::getTrackedChanges() +{ + faabric::util::SharedLock lock(snapMx); - SPDLOG_DEBUG( - "Memory view has {}/{} dirty pages", dirtyPageNumbers.size(), nPages); + std::vector diffs; + if (trackedChanges.empty()) { + return diffs; + } - std::vector> regions = - faabric::util::getDirtyRegions(data.data(), nPages); + std::span d(data.get(), size); // Convert to snapshot diffs - std::vector diffs; - diffs.reserve(regions.size()); - for (auto [regionBegin, regionEnd] : regions) { - SPDLOG_TRACE("Memory view dirty {}-{}", regionBegin, regionEnd); + diffs.reserve(trackedChanges.size()); + for (auto [regionBegin, regionEnd] : trackedChanges) { + SPDLOG_TRACE("Snapshot dirty {}-{}", regionBegin, regionEnd); diffs.emplace_back(SnapshotDataType::Raw, SnapshotMergeOperation::Overwrite, regionBegin, - data.subspan(regionBegin, regionEnd - regionBegin)); + d.subspan(regionBegin, regionEnd - regionBegin)); } - PROF_END(GetDirtyRegions) return diffs; } -std::vector MemoryView::diffWithSnapshot( - std::shared_ptr snap) +std::vector SnapshotData::diffWithDirtyRegions( + std::span updated, + std::vector> dirtyRegions) { + faabric::util::SharedLock lock(snapMx); + PROF_START(DiffWithSnapshot) - std::vector diffs; - std::map mergeRegions = - snap->getMergeRegions(); - if (mergeRegions.empty()) { - SPDLOG_DEBUG("No merge regions set, thus no diffs"); + std::vector diffs; + + if (mergeRegions.empty() || dirtyRegions.empty()) { 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 + SPDLOG_TRACE("Diffing {} merge regions vs {} dirty regions", + mergeRegions.size(), + dirtyRegions.size()); + + // First dedupe the memory regions + auto dedupedRegions = dedupeMemoryRegions(dirtyRegions); + + // Iterate through merge regions, allow them to add diffs based on the dirty + // regions + std::span original(data.get(), size); 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, - dirtyRegion); + faabric::util::SnapshotMergeRegion& mr = mrPair.second; + + // Add the diffs for each dirty region + for (auto& dirtyRegion : dedupedRegions) { + mr.addDiffs(diffs, original, updated, dirtyRegion); } } @@ -539,97 +524,67 @@ void SnapshotMergeRegion::addOverwriteDiff( std::vector& diffs, std::span original, std::span updated, - std::pair dirtyRange) + std::pair dirtyRegion) { - auto operation = SnapshotMergeOperation::Overwrite; - - // Work out bounds of region we're checking - uint32_t checkStart = std::max(dirtyRange.first, offset); - - // Here we need to make sure we don't overrun the original or the updated - // data - uint32_t checkEnd; - if (length == 0) { - checkEnd = dirtyRange.second; - } else { - checkEnd = std::min(dirtyRange.second, offset + length); + // In this function we have two possibilities: + // - Dirty region overlaps original and we can compare + // - Dirty region is past original, in which case we need to extend + + // First we calculate where the dirty region starts and ends. If the merge + // region ends before the dirty region, we take that as the limit. If the + // merge region is zero length, we take the whole dirty region. + uint32_t dirtyRegionStart = std::max(dirtyRegion.first, offset); + uint32_t dirtyRegionEnd = dirtyRegion.first + dirtyRegion.second; + if (length > 0) { + dirtyRegionEnd = std::min(dirtyRegionEnd, offset + length); } - - // If the region is outside the original data, automatically add a diff for - // the whole region - if (checkStart >= original.size()) { - SPDLOG_TRACE("Single extension {} overwrite diff at {}-{}", - snapshotDataTypeStr(dataType), - checkStart, - checkEnd - checkStart); - diffs.emplace_back(dataType, - operation, - checkStart, - updated.subspan(checkStart, checkEnd - checkStart)); - return; - } - - bool diffInProgress = false; - uint32_t diffStart = 0; - for (uint32_t b = checkStart; b <= checkEnd; b++) { - // If this byte is outside the original region, everything from here on - // is dirty, so we can add a single region to go from here to the end - if (b >= original.size()) { - if (!diffInProgress) { - diffStart = b; - } - - uint32_t diffLength = checkEnd - diffStart; - - SPDLOG_TRACE("Extension {} overwrite diff at {}-{}", - snapshotDataTypeStr(dataType), - diffStart, - diffStart + diffLength); - - diffs.emplace_back(dataType, - operation, - diffStart, - updated.subspan(diffStart, diffLength)); - return; - } - - bool isDirtyByte = (*(original.data() + b) != *(updated.data() + b)); - if (isDirtyByte && !diffInProgress) { - // Diff starts here if it's different and diff - // not in progress - diffInProgress = true; - diffStart = b; - } else if (!isDirtyByte && diffInProgress) { - // Diff ends if it's not different and diff is - // in progress - uint32_t diffLength = b - diffStart; - SPDLOG_TRACE("Found {} overwrite diff at {}-{}", + assert(dirtyRegionStart < dirtyRegionEnd); + + // Overlap with original data + if (original.size() > dirtyRegionStart) { + // Work out the end of the overlap + uint32_t overlapEnd = + std::min(dirtyRegionEnd, original.size()); + uint32_t overlapLen = overlapEnd - dirtyRegionStart; + + // Get the subsections of both the original data and dirty region to + // compare + std::span originalSub = + original.subspan(dirtyRegionStart, overlapLen); + + std::span dirtySub = + updated.subspan(dirtyRegionStart, overlapLen); + + std::vector> regions = + diffArrayRegions(originalSub, dirtySub); + + // Iterate through and build diffs + for (auto [start, len] : regions) { + uint32_t diffStart = dirtyRegionStart + start; + SPDLOG_TRACE("Adding {} overwrite diff at {}-{}", snapshotDataTypeStr(dataType), diffStart, - diffStart + diffLength); + diffStart + len); - diffInProgress = false; - diffs.emplace_back(dataType, - operation, - diffStart, - updated.subspan(diffStart, diffLength)); + diffs.emplace_back( + dataType, operation, diffStart, updated.subspan(diffStart, len)); } } - // If we've reached the end of this region with a diff - // in progress, we need to close it off - if (diffInProgress) { - uint32_t finalDiffLength = checkEnd - diffStart; - SPDLOG_TRACE("Adding {} {} diff at {}-{} (end of region)", + // Extension past original data + if (dirtyRegionEnd > original.size()) { + uint32_t diffStart = + std::max(original.size(), dirtyRegionStart); + uint32_t diffLength = dirtyRegionEnd - diffStart; + + SPDLOG_TRACE("Adding extension {} overwrite diff at {}-{}", snapshotDataTypeStr(dataType), - snapshotMergeOpStr(operation), diffStart, - diffStart + finalDiffLength); - + diffStart + diffLength); diffs.emplace_back(dataType, operation, diffStart, - updated.subspan(diffStart, finalDiffLength)); + updated.subspan(diffStart, diffLength)); } } @@ -645,15 +600,19 @@ SnapshotMergeRegion::SnapshotMergeRegion(uint32_t offsetIn, void SnapshotMergeRegion::addDiffs(std::vector& diffs, std::span originalData, - std::span updatedData, - std::pair dirtyRange) + std::span updatedData, + std::pair dirtyRegion) { // If the region has zero length, it signifies that it goes to the - // end of the memory, so we go all the way to the end of the dirty region. - // For all other regions, we just check if the dirty range is within the - // merge region. - bool isInRange = (dirtyRange.second > offset) && - ((length == 0) || (dirtyRange.first < offset + length)); + // end of the memory, so we go all the way to the end of the dirty + // region. For all other regions, we just check if the dirty range is + // within the merge region. + + uint32_t dirtyRangeStart = dirtyRegion.first; + uint32_t dirtyRangeEnd = dirtyRegion.first + dirtyRegion.second; + + bool isInRange = (dirtyRangeEnd > offset) && + ((length == 0) || (dirtyRangeStart < offset + length)); if (!isInRange) { SPDLOG_TRACE("{} {} merge region {}-{} not in dirty region {}-{}", @@ -661,8 +620,8 @@ void SnapshotMergeRegion::addDiffs(std::vector& diffs, snapshotMergeOpStr(operation), offset, offset + length, - dirtyRange.first, - dirtyRange.second); + dirtyRangeStart, + dirtyRangeEnd); return; } @@ -672,12 +631,12 @@ void SnapshotMergeRegion::addDiffs(std::vector& diffs, snapshotMergeOpStr(operation), offset, offset + length, - dirtyRange.first, - dirtyRange.second, + dirtyRangeStart, + dirtyRangeEnd, originalData.size()); if (operation == SnapshotMergeOperation::Overwrite) { - addOverwriteDiff(diffs, originalData, updatedData, dirtyRange); + addOverwriteDiff(diffs, originalData, updatedData, dirtyRegion); return; } @@ -686,11 +645,11 @@ void SnapshotMergeRegion::addDiffs(std::vector& diffs, } if (originalData.size() < offset) { - throw std::runtime_error( - "Do not support non-overwrite operations outside original snapshot"); + throw std::runtime_error("Do not support non-overwrite operations " + "outside original snapshot"); } - uint8_t* updated = (uint8_t*)updatedData.data() + offset; + uint8_t* updated = updatedData.data() + offset; const uint8_t* original = originalData.data() + offset; bool changed = false; diff --git a/tests/dist/DistTestExecutor.cpp b/tests/dist/DistTestExecutor.cpp index c7f337a40..b1fbb69c5 100644 --- a/tests/dist/DistTestExecutor.cpp +++ b/tests/dist/DistTestExecutor.cpp @@ -38,7 +38,9 @@ ExecutorFunction getDistTestExecutorCallback(const faabric::Message& msg) DistTestExecutor::DistTestExecutor(faabric::Message& msg) : Executor(msg) -{} +{ + setUpDummyMemory(dummyMemorySize); +} DistTestExecutor::~DistTestExecutor() {} @@ -60,24 +62,34 @@ void DistTestExecutor::reset(faabric::Message& msg) faabric::util::funcToString(msg, false)); } -void DistTestExecutor::restore(faabric::Message& msg) +void DistTestExecutor::restore(const std::string& snapshotKey) { - SPDLOG_DEBUG("Dist test executor restoring for {}", - faabric::util::funcToString(msg, false)); - - faabric::snapshot::SnapshotRegistry& reg = - faabric::snapshot::getSnapshotRegistry(); + SPDLOG_DEBUG("Dist test executor restoring from {}", snapshotKey); - auto snap = reg.getSnapshot(msg.snapshotkey()); - - setUpDummyMemory(snap->getSize()); + if (dummyMemory.get() == nullptr) { + SPDLOG_ERROR("No memory for dist test executor to restore {}", + snapshotKey); + throw std::runtime_error("No memory to restore dist test executor"); + } + auto snap = reg.getSnapshot(snapshotKey); snap->mapToMemory({ dummyMemory.get(), dummyMemorySize }); } -faabric::util::MemoryView DistTestExecutor::getMemoryView() +std::span DistTestExecutor::getMemoryView() { - return faabric::util::MemoryView({ dummyMemory.get(), dummyMemorySize }); + return { dummyMemory.get(), dummyMemorySize }; +} + +void DistTestExecutor::setMemorySize(size_t newSize) +{ + if (newSize != dummyMemorySize) { + SPDLOG_ERROR("DistTestExecutor cannot change memory size ({} != {})", + newSize, + dummyMemorySize); + throw std::runtime_error( + "DistTestExecutor does not support changing memory size"); + } } std::span DistTestExecutor::getDummyMemory() @@ -87,11 +99,9 @@ std::span DistTestExecutor::getDummyMemory() void DistTestExecutor::setUpDummyMemory(size_t memSize) { - if (dummyMemory.get() == nullptr) { - SPDLOG_DEBUG("Dist test executor initialising memory size {}", memSize); - dummyMemory = faabric::util::allocatePrivateMemory(memSize); - dummyMemorySize = memSize; - } + SPDLOG_DEBUG("Dist test executor initialising memory size {}", memSize); + dummyMemory = faabric::util::allocatePrivateMemory(memSize); + dummyMemorySize = memSize; } std::shared_ptr DistTestExecutorFactory::createExecutor( diff --git a/tests/dist/DistTestExecutor.h b/tests/dist/DistTestExecutor.h index 45a1fc749..d0882d118 100644 --- a/tests/dist/DistTestExecutor.h +++ b/tests/dist/DistTestExecutor.h @@ -7,6 +7,8 @@ namespace tests { +#define DIST_TEST_EXECUTOR_MEMORY_SIZE (30 * faabric::util::HOST_PAGE_SIZE) + class DistTestExecutor final : public faabric::scheduler::Executor { public: @@ -21,18 +23,21 @@ class DistTestExecutor final : public faabric::scheduler::Executor void reset(faabric::Message& msg) override; - void restore(faabric::Message& msg) override; + void restore(const std::string& snapshotKey) override; - faabric::util::MemoryView getMemoryView() override; + std::span getMemoryView() override; std::span getDummyMemory(); - void setUpDummyMemory(size_t memSize); + protected: + void setMemorySize(size_t newSize) override; private: faabric::util::MemoryRegion dummyMemory = nullptr; - size_t dummyMemorySize = 0; + size_t dummyMemorySize = DIST_TEST_EXECUTOR_MEMORY_SIZE; + + void setUpDummyMemory(size_t memSize); }; class DistTestExecutorFactory : public faabric::scheduler::ExecutorFactory diff --git a/tests/dist/scheduler/functions.cpp b/tests/dist/scheduler/functions.cpp index 23765fd0d..daa06a6fc 100644 --- a/tests/dist/scheduler/functions.cpp +++ b/tests/dist/scheduler/functions.cpp @@ -65,23 +65,15 @@ int handleFakeDiffsFunction(tests::DistTestExecutor* exec, faabric::Message& msg = req->mutable_messages()->at(msgIdx); std::string msgInput = msg.inputdata(); - std::string snapshotKey = msg.snapshotkey(); - - faabric::snapshot::SnapshotRegistry& reg = - faabric::snapshot::getSnapshotRegistry(); - - auto originalSnap = reg.getSnapshot(snapshotKey); - std::vector inputBytes = faabric::util::stringToBytes(msgInput); + std::vector otherData = { 1, 2, 3, 4 }; // Modify the executor's memory - std::vector keyBytes = faabric::util::stringToBytes(snapshotKey); - int offsetA = 10; - int offsetB = 100; + int offsetB = HOST_PAGE_SIZE + 10; std::memcpy(exec->getDummyMemory().data() + offsetA, - keyBytes.data(), - keyBytes.size()); + otherData.data(), + otherData.size()); std::memcpy(exec->getDummyMemory().data() + offsetB, inputBytes.data(), inputBytes.size()); @@ -97,70 +89,49 @@ int handleFakeDiffsThreadedFunction( { bool isThread = req->type() == faabric::BatchExecuteRequest::THREADS; faabric::Message& msg = req->mutable_messages()->at(msgIdx); - std::string snapshotKey = "fake-diffs-threaded-snap"; + std::string snapshotKey = getMainThreadSnapshotKey(msg); std::string msgInput = msg.inputdata(); - faabric::snapshot::SnapshotRegistry& reg = - faabric::snapshot::getSnapshotRegistry(); - - // This function creates a snapshot, then spawns some child threads that - // will modify the shared memory. It then awaits the results and checks that - // the modifications are synced back to the original host. + // This function spawns some child threads that will modify the shared + // memory. It then awaits the results and checks that the modifications are + // synced back to the original host. if (!isThread) { int nThreads = std::stoi(msgInput); - // Set up the snapshot - size_t snapSize = (nThreads * 4) * faabric::util::HOST_PAGE_SIZE; - auto snap = std::make_shared(snapSize); - reg.registerSnapshot(snapshotKey, snap); - 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_appid(msg.appid()); m.set_appidx(i); m.set_inputdata(std::string("thread_" + std::to_string(i))); - m.set_snapshotkey(snapshotKey); // Make a small modification to a page that will also be edited by // the child thread to make sure it's not overwritten std::vector localChange(3, i); int offset = 2 * i * faabric::util::HOST_PAGE_SIZE; - snap->copyInData(localChange, offset); + std::memcpy(exec->getDummyMemory().data() + offset, + localChange.data(), + localChange.size()); } - // Dispatch the message, expecting them all to execute on other hosts - std::string thisHost = faabric::util::getSystemConfig().endpointHost; - faabric::scheduler::Scheduler& sch = faabric::scheduler::getScheduler(); - - std::vector executedHosts = sch.callFunctions(req).hosts; + // Dispatch the message + std::vector> results = + exec->executeThreads(req, {}); - bool rightHosts = true; - for (auto& h : executedHosts) { - if (h == thisHost) { - SPDLOG_ERROR("Expected child threads to be executed on other " - "hosts (this host {}, actual host {})", - thisHost, - h); - rightHosts = false; + // Check results + for (auto [mid, res] : results) { + if (res != 0) { + SPDLOG_ERROR( + "Thread diffs test thread {} failed with value {}", mid, res); + throw std::runtime_error("Thread diffs check failed"); } } - if (!rightHosts) { - return 111; - } - - // Wait for the threads - for (auto& m : req->messages()) { - sch.awaitThreadResult(m.id()); - } - - // Write queued diffs - snap->writeQueuedDiffs(); - // Check changes have been applied + auto snap = exec->getMainThreadSnapshot(msg); bool success = true; for (int i = 0; i < nThreads; i++) { // Check local modifications @@ -199,6 +170,15 @@ int handleFakeDiffsThreadedFunction( int regionOffset = 2 * idx * faabric::util::HOST_PAGE_SIZE; int changeOffset = regionOffset + 10; + if (regionOffset > exec->getDummyMemory().size()) { + SPDLOG_ERROR( + "Dummy memory not large enough for function {} ({} > {})", + faabric::util::funcToString(msg, false), + regionOffset, + exec->getDummyMemory().size()); + throw std::runtime_error("Dummy memory not large enough"); + } + // Get the input data std::vector inputBytes = faabric::util::stringToBytes(msgInput); @@ -224,15 +204,10 @@ int handleReductionFunction(tests::DistTestExecutor* exec, 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 @@ -245,11 +220,8 @@ int handleReductionFunction(tests::DistTestExecutor* exec, // 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) { - // Set up snapshot - std::string snapKey = "dist-reduction-" + std::to_string(generateGid()); - std::shared_ptr snap = - std::make_shared(snapSize); - reg.registerSnapshot(snapKey, snap); + // Get the main thread snapshot, creating if doesn't already exist + auto snap = exec->getMainThreadSnapshot(msg, true); // Perform operations in a loop for (int r = 0; r < nRepeats; r++) { @@ -259,59 +231,37 @@ int handleReductionFunction(tests::DistTestExecutor* exec, 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_appid(msg.appid()); m.set_appidx(i); + m.set_groupidx(i); } // Set merge regions - snap->addMergeRegion(reductionAOffset, - sizeof(int32_t), - SnapshotDataType::Int, - SnapshotMergeOperation::Sum, - true); - - snap->addMergeRegion(reductionBOffset, - sizeof(int32_t), - SnapshotDataType::Int, - SnapshotMergeOperation::Sum, - true); - - // 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) { + std::vector mergeRegions = { + { reductionAOffset, + sizeof(int32_t), + SnapshotDataType::Int, + SnapshotMergeOperation::Sum }, + + { reductionBOffset, + sizeof(int32_t), + SnapshotDataType::Int, + SnapshotMergeOperation::Sum } + }; + + // Execute the threads + std::vector> results = + exec->executeThreads(req, mergeRegions); + + // Check thread results + for (auto [mid, res] : results) { + if (res != 0) { SPDLOG_ERROR( "Distributed reduction test thread {} failed: {}", - m.id(), - thisRes); + mid, + res); return 1; } @@ -319,12 +269,6 @@ int handleReductionFunction(tests::DistTestExecutor* exec, SPDLOG_DEBUG("Reduce test threads finished"); - // Write queued snapshot diffs - snap->writeQueuedDiffs(); - - // Remap memory to snapshot - snap->mapToMemory(exec->getDummyMemory()); - uint8_t* reductionAPtr = exec->getDummyMemory().data() + reductionAOffset; uint8_t* reductionBPtr = @@ -382,6 +326,7 @@ int handleReductionFunction(tests::DistTestExecutor* exec, uint8_t* thisArrayPtr = arrayPtr + (sizeof(int32_t) * thisIdx); // Lock group locally while doing reduction + int groupId = msg.groupid(); std::shared_ptr group = faabric::transport::PointToPointGroup::getGroup(groupId); group->localLock(); diff --git a/tests/dist/scheduler/test_snapshots.cpp b/tests/dist/scheduler/test_snapshots.cpp index cd93df8ad..9dba92a4b 100644 --- a/tests/dist/scheduler/test_snapshots.cpp +++ b/tests/dist/scheduler/test_snapshots.cpp @@ -21,28 +21,29 @@ namespace tests { TEST_CASE_METHOD(DistTestsFixture, "Check snapshots sent back from worker are queued", - "[snapshots]") + "[snapshots][threads]") { std::string user = "snapshots"; std::string function = "fake-diffs"; - std::string snapshotKey = "dist-snap-check"; std::vector inputData = { 0, 1, 2, 3, 4, 5, 6 }; - // Set up snapshot - size_t snapSize = 2 * faabric::util::HOST_PAGE_SIZE; - auto snap = std::make_shared(snapSize); - - reg.registerSnapshot(snapshotKey, snap); - // Set up the message std::shared_ptr req = faabric::util::batchExecFactory(user, function, 1); req->set_type(faabric::BatchExecuteRequest::THREADS); // Set up some input data - faabric::Message& m = req->mutable_messages()->at(0); - m.set_inputdata(inputData.data(), inputData.size()); - m.set_snapshotkey(snapshotKey); + faabric::Message& msg = req->mutable_messages()->at(0); + msg.set_inputdata(inputData.data(), inputData.size()); + + // Set up the main thread snapshot + faabric::snapshot::SnapshotRegistry& reg = + faabric::snapshot::getSnapshotRegistry(); + + size_t snapSize = DIST_TEST_EXECUTOR_MEMORY_SIZE; + std::string snapshotKey = faabric::util::getMainThreadSnapshotKey(msg); + auto snap = std::make_shared(snapSize); + reg.registerSnapshot(snapshotKey, snap); // Force the function to be executed remotely faabric::HostResources res; @@ -54,31 +55,33 @@ TEST_CASE_METHOD(DistTestsFixture, std::vector executedHosts = decision.hosts; REQUIRE(expectedHosts == executedHosts); - int actualResult = sch.awaitThreadResult(m.id()); + int actualResult = sch.awaitThreadResult(msg.id()); REQUIRE(actualResult == 123); // 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(); + size_t expectedOffsetA = 10; + size_t expectedOffsetB = faabric::util::HOST_PAGE_SIZE + 10; + std::vector expectedA = { 1, 2, 3, 4 }; + std::vector expectedB = inputData; + + size_t sizeA = expectedA.size(); + size_t sizeB = expectedB.size(); - const uint8_t* startA = snap->getDataPtr() + 10; - const uint8_t* startB = snap->getDataPtr() + 100; + const uint8_t* startA = snap->getDataPtr() + expectedOffsetA; + const uint8_t* startB = snap->getDataPtr() + expectedOffsetB; std::vector actualA(startA, startA + sizeA); std::vector actualB(startB, startB + sizeB); - std::vector expectedA = faabric::util::stringToBytes(snapshotKey); - std::vector expectedB = inputData; - REQUIRE(actualA == expectedA); REQUIRE(actualB == expectedB); } TEST_CASE_METHOD(DistTestsFixture, "Check snapshot diffs sent back from child threads", - "[snapshots]") + "[snapshots][threads]") { std::string user = "snapshots"; std::string function = "fake-diffs-threaded"; @@ -86,8 +89,9 @@ TEST_CASE_METHOD(DistTestsFixture, std::shared_ptr req = faabric::util::batchExecFactory(user, function, 1); - faabric::Message& m = req->mutable_messages()->at(0); - m.set_inputdata(std::to_string(nThreads)); + + faabric::Message& msg = req->mutable_messages()->at(0); + msg.set_inputdata(std::to_string(nThreads)); // Force the function itself to be executed on this host, but its child // threads on another host @@ -100,18 +104,20 @@ TEST_CASE_METHOD(DistTestsFixture, std::vector executedHosts = decision.hosts; REQUIRE(expectedHosts == executedHosts); - faabric::Message actualResult = sch.getFunctionResult(m.id(), 10000); + faabric::Message actualResult = sch.getFunctionResult(msg.id(), 10000); REQUIRE(actualResult.returnvalue() == 333); } -TEST_CASE_METHOD(DistTestsFixture, "Check repeated reduction", "[snapshots]") +TEST_CASE_METHOD(DistTestsFixture, + "Check repeated reduction", + "[snapshots][threads]") { 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); + faabric::Message& msg = req->mutable_messages()->at(0); // Main function and one thread execute on this host, others on another faabric::HostResources res; @@ -123,7 +129,7 @@ TEST_CASE_METHOD(DistTestsFixture, "Check repeated reduction", "[snapshots]") std::vector executedHosts = decision.hosts; REQUIRE(expectedHosts == executedHosts); - faabric::Message actualResult = sch.getFunctionResult(m.id(), 10000); + faabric::Message actualResult = sch.getFunctionResult(msg.id(), 10000); REQUIRE(actualResult.returnvalue() == 0); } } diff --git a/tests/dist/scheduler/test_threads.cpp b/tests/dist/scheduler/test_threads.cpp index 258be0a70..8849b9c94 100644 --- a/tests/dist/scheduler/test_threads.cpp +++ b/tests/dist/scheduler/test_threads.cpp @@ -19,7 +19,7 @@ namespace tests { TEST_CASE_METHOD(DistTestsFixture, "Test executing threads on multiple hosts", - "[threads]") + "[snapshots][threads]") { // Set up this host's resources int nLocalSlots = 2; @@ -28,13 +28,6 @@ TEST_CASE_METHOD(DistTestsFixture, res.set_slots(nLocalSlots); sch.setThisHostResources(res); - // Set up a snapshot - size_t snapshotSize = 5 * faabric::util::HOST_PAGE_SIZE; - auto snap = std::make_shared(snapshotSize); - - std::string snapKey = std::to_string(faabric::util::generateGid()); - reg.registerSnapshot(snapKey, snap); - // Set up the message std::shared_ptr req = faabric::util::batchExecFactory("threads", "simple", nThreads); @@ -43,9 +36,18 @@ TEST_CASE_METHOD(DistTestsFixture, for (int i = 0; i < nThreads; i++) { faabric::Message& m = req->mutable_messages()->at(i); m.set_appidx(i); - m.set_snapshotkey(snapKey); } + // Set up main thread snapshot + faabric::snapshot::SnapshotRegistry& reg = + faabric::snapshot::getSnapshotRegistry(); + faabric::Message& msg = req->mutable_messages()->at(0); + + std::string snapshotKey = faabric::util::getMainThreadSnapshotKey(msg); + auto snap = std::make_shared( + DIST_TEST_EXECUTOR_MEMORY_SIZE); + reg.registerSnapshot(snapshotKey, snap); + // Call the functions sch.callFunctions(req); diff --git a/tests/dist/transport/functions.cpp b/tests/dist/transport/functions.cpp index 09c329963..c0f046198 100644 --- a/tests/dist/transport/functions.cpp +++ b/tests/dist/transport/functions.cpp @@ -1,16 +1,16 @@ #include #include "DistTestExecutor.h" -#include "faabric/scheduler/Scheduler.h" -#include "faabric/util/func.h" -#include "faabric/util/gids.h" -#include "faabric/util/scheduling.h" #include "faabric_utils.h" #include "init.h" #include +#include #include #include +#include +#include +#include #include using namespace faabric::transport; diff --git a/tests/test/scheduler/test_executor.cpp b/tests/test/scheduler/test_executor.cpp index 3cda020ec..3af1a230b 100644 --- a/tests/test/scheduler/test_executor.cpp +++ b/tests/test/scheduler/test_executor.cpp @@ -12,6 +12,7 @@ #include #include #include +#include #include #include #include @@ -31,7 +32,9 @@ std::atomic resetCount = 0; TestExecutor::TestExecutor(faabric::Message& msg) : Executor(msg) -{} +{ + setUpDummyMemory(dummyMemorySize); +} void TestExecutor::reset(faabric::Message& msg) { @@ -39,23 +42,29 @@ void TestExecutor::reset(faabric::Message& msg) resetCount += 1; } -void TestExecutor::restore(faabric::Message& msg) +void TestExecutor::setUpDummyMemory(size_t memSize) +{ + SPDLOG_DEBUG("TestExecutor initialising memory size {}", memSize); + dummyMemory = faabric::util::allocatePrivateMemory(memSize); + dummyMemorySize = memSize; +} + +void TestExecutor::restore(const std::string& snapshotKey) { - SPDLOG_DEBUG("Restoring TestExecutor"); restoreCount += 1; - faabric::snapshot::SnapshotRegistry& reg = - faabric::snapshot::getSnapshotRegistry(); - auto snap = reg.getSnapshot(msg.snapshotkey()); + auto snap = reg.getSnapshot(snapshotKey); + if (dummyMemory == nullptr) { + throw std::runtime_error( + "Attempting to restore test executor with no memory set up"); + } - dummyMemorySize = snap->getSize(); - dummyMemory = faabric::util::allocatePrivateMemory(snap->getSize()); snap->mapToMemory({ dummyMemory.get(), dummyMemorySize }); } -faabric::util::MemoryView TestExecutor::getMemoryView() +std::span TestExecutor::getMemoryView() { - return faabric::util::MemoryView({ dummyMemory.get(), dummyMemorySize }); + return { dummyMemory.get(), dummyMemorySize }; } int32_t TestExecutor::executeTask( @@ -89,40 +98,26 @@ int32_t TestExecutor::executeTask( faabric::util::batchExecFactory("dummy", "thread-check", nThreads); chainedReq->set_type(faabric::BatchExecuteRequest::THREADS); - // Create a dummy snapshot - std::string snapKey = funcStr + "-snap"; - faabric::snapshot::SnapshotRegistry& reg = - faabric::snapshot::getSnapshotRegistry(); - 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); - m.set_snapshotkey(snapKey); + m.set_appid(msg.appid()); m.set_appidx(i + 1); } // Call the threads - Scheduler& sch = getScheduler(); - sch.callFunctions(chainedReq); + std::vector> results = + executeThreads(chainedReq, {}); // Await the results - for (const auto& msg : chainedReq->messages()) { - uint32_t mid = msg.id(); - int threadRes = sch.awaitThreadResult(mid); - - if (threadRes != mid / 100) { + for (auto [mid, result] : results) { + if (result != mid / 100) { SPDLOG_ERROR("TestExecutor got invalid thread result, {} != {}", - threadRes, + result, mid / 100); return 1; } } - // Delete the snapshot - reg.deleteSnapshot(snapKey); - SPDLOG_TRACE("TestExecutor got {} thread results", chainedReq->messages_size()); return 0; @@ -175,9 +170,6 @@ int32_t TestExecutor::executeTask( // Modify a page of the dummy memory uint8_t pageIdx = threadPoolIdx; - auto snapData = faabric::snapshot::getSnapshotRegistry().getSnapshot( - msg.snapshotkey()); - // Set up the data. // Note, avoid writing a zero here as the memory is already zeroed hence // it's not a change @@ -192,6 +184,11 @@ int32_t TestExecutor::executeTask( offset, offset + data.size()); + if (dummyMemorySize < offset + data.size()) { + throw std::runtime_error( + "TestExecutor memory not large enough for test"); + } + ::memcpy(dummyMemory.get() + offset, data.data(), data.size()); } @@ -234,9 +231,6 @@ class TestExecutorFixture std::make_shared(); setExecutorFactory(fac); - dummySnap = setUpSnapshot(snapshotKey, snapshotNPages); - faabric::util::resetDirtyTracking(); - restoreCount = 0; resetCount = 0; } @@ -244,16 +238,24 @@ class TestExecutorFixture ~TestExecutorFixture() = default; protected: - std::string snapshotKey = "foobar"; int snapshotNPages = 10; size_t snapshotSize = snapshotNPages * faabric::util::HOST_PAGE_SIZE; - std::shared_ptr dummySnap; + MemoryRegion dummyMemory; std::vector executeWithTestExecutor( std::shared_ptr req, bool forceLocal) { + // Create the main thread snapshot if we're executing threads directly + if (req->type() == faabric::BatchExecuteRequest::THREADS) { + faabric::Message& msg = req->mutable_messages()->at(0); + std::string snapKey = faabric::util::getMainThreadSnapshotKey(msg); + auto snap = + std::make_shared(snapshotSize); + reg.registerSnapshot(snapKey, snap); + } + conf.overrideCpuCount = 10; conf.boundTimeout = SHORT_TEST_TIMEOUT_MS; faabric::util::SchedulingTopologyHint topologyHint = @@ -297,7 +299,7 @@ TEST_CASE_METHOD(TestExecutorFixture, // Set the bound timeout to something short so the test runs fast conf.boundTimeout = 100; - int numRepeats = 20; + int numRepeats = 10; for (int i = 0; i < numRepeats; i++) { std::shared_ptr req = faabric::util::batchExecFactory("dummy", "simple", 1); @@ -356,7 +358,6 @@ TEST_CASE_METHOD(TestExecutorFixture, std::vector messageIds; for (int i = 0; i < nThreads; i++) { faabric::Message& msg = req->mutable_messages()->at(i); - msg.set_snapshotkey(snapshotKey); msg.set_appidx(i); messageIds.emplace_back(req->messages().at(i).id()); @@ -452,7 +453,6 @@ TEST_CASE_METHOD(TestExecutorFixture, std::vector messageIds; for (int i = 0; i < nThreads; i++) { faabric::Message& msg = req->mutable_messages()->at(i); - msg.set_snapshotkey(snapshotKey); msg.set_masterhost(otherHost); messageIds.emplace_back(msg.id()); @@ -517,7 +517,6 @@ TEST_CASE_METHOD(TestExecutorFixture, "Test erroring thread", "[executor]") faabric::Message& msg = req->mutable_messages()->at(0); req->set_type(faabric::BatchExecuteRequest::THREADS); - msg.set_snapshotkey(snapshotKey); executeWithTestExecutor(req, false); int32_t res = sch.awaitThreadResult(msg.id()); @@ -630,6 +629,10 @@ TEST_CASE_METHOD(TestExecutorFixture, "[executor]") { int nThreads = 4; + + // Sanity check memory size + REQUIRE(TEST_EXECUTOR_DEFAULT_MEMORY_SIZE > nThreads * HOST_PAGE_SIZE); + std::shared_ptr req = faabric::util::batchExecFactory("dummy", "snap-check", nThreads); req->set_type(faabric::BatchExecuteRequest::THREADS); @@ -637,11 +640,13 @@ TEST_CASE_METHOD(TestExecutorFixture, faabric::util::setMockMode(true); std::string otherHost = "other"; + std::string mainThreadSnapshotKey = + faabric::util::getMainThreadSnapshotKey(req->mutable_messages()->at(0)); + // Set up some messages executing with a different master host std::vector messageIds; for (int i = 0; i < nThreads; i++) { faabric::Message& msg = req->mutable_messages()->at(i); - msg.set_snapshotkey(snapshotKey); msg.set_masterhost(otherHost); msg.set_appidx(i); @@ -682,7 +687,7 @@ TEST_CASE_METHOD(TestExecutorFixture, } // Check no merge regions left on the snapshot - REQUIRE(reg.getSnapshot(snapshotKey)->getMergeRegions().empty()); + REQUIRE(reg.getSnapshot(mainThreadSnapshotKey)->getMergeRegions().empty()); } TEST_CASE_METHOD(TestExecutorFixture, @@ -716,7 +721,6 @@ TEST_CASE_METHOD(TestExecutorFixture, std::vector messageIds; for (int i = 0; i < nThreads; i++) { faabric::Message& msg = req->mutable_messages()->at(i); - msg.set_snapshotkey(snapshotKey); messageIds.emplace_back(msg.id()); } @@ -752,23 +756,29 @@ TEST_CASE_METHOD(TestExecutorFixture, faabric::snapshot::clearMockSnapshotRequests(); // Update the snapshot and check we get expected diffs - auto snap = reg.getSnapshot(snapshotKey); + std::string mainThreadSnapshotKey = + faabric::util::getMainThreadSnapshotKey(msg); + auto snap = reg.getSnapshot(mainThreadSnapshotKey); 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 = - faabric::util::MemoryView({ snap->getDataPtr(), snap->getSize() }) - .getDirtyRegions(); + snap->getTrackedChanges(); REQUIRE(expectedDiffs.size() == 2); - // Set up another function + // Set up another function, make sure they have the same app ID std::shared_ptr reqB = faabric::util::batchExecFactory("dummy", "blah", 1); reqB->set_type(faabric::BatchExecuteRequest::THREADS); - reqB->mutable_messages()->at(0).set_snapshotkey(snapshotKey); + + faabric::Message& msgB = reqB->mutable_messages()->at(0); + msgB.set_appid(msg.appid()); + + REQUIRE(faabric::util::getMainThreadSnapshotKey(msgB) == + mainThreadSnapshotKey); // Invoke the function std::vector expectedHostsB = { thisHost }; @@ -777,7 +787,7 @@ TEST_CASE_METHOD(TestExecutorFixture, REQUIRE(actualHostsB == expectedHostsB); // Wait for it to finish locally - sch.awaitThreadResult(reqB->mutable_messages()->at(0).id()); + sch.awaitThreadResult(msgB.id()); // Check the full snapshot hasn't been pushed REQUIRE(faabric::snapshot::getSnapshotPushes().empty()); @@ -798,7 +808,7 @@ TEST_CASE_METHOD(TestExecutorFixture, } TEST_CASE_METHOD(TestExecutorFixture, - "Test reset not called on threads", + "Test reset called for functions not threads", "[executor]") { faabric::util::setMockMode(true); @@ -812,40 +822,46 @@ TEST_CASE_METHOD(TestExecutorFixture, int expectedResets = 1; - SECTION("Non-master threads") - { - hostOverride = "foobar"; - nMessages = 3; - requestType = faabric::BatchExecuteRequest::THREADS; - expectedResets = 0; - } - - SECTION("Master threads") + SECTION("Threads") { requestType = faabric::BatchExecuteRequest::THREADS; nMessages = 3; expectedResets = 0; } - SECTION("Non-master function") { hostOverride = "foobar"; } - - SECTION("Master function") {} + SECTION("Function") {} std::shared_ptr req = faabric::util::batchExecFactory("dummy", "blah", nMessages); req->set_type(requestType); - std::vector mids; + faabric::Message& msg = req->mutable_messages()->at(0); + + if (requestType == faabric::BatchExecuteRequest::THREADS) { + // Set up main thread snapshot + size_t snapSize = TEST_EXECUTOR_DEFAULT_MEMORY_SIZE; + auto snap = std::make_shared(snapSize); + std::string snapKey = getMainThreadSnapshotKey(msg); + + reg.registerSnapshot(snapKey, snap); + } + for (auto& m : *req->mutable_messages()) { m.set_masterhost(hostOverride); - mids.push_back(m.id()); } // Call functions and force to execute locally sch.callFunctions(req, faabric::util::SchedulingTopologyHint::FORCE_LOCAL); - // As we're faking a non-master execution results will be sent back to - // the fake master so we can't wait on them, thus have to sleep - REQUIRE_RETRY({}, resetCount == expectedResets); + // Await execution + for (auto& m : *req->mutable_messages()) { + if (requestType == faabric::BatchExecuteRequest::THREADS) { + sch.awaitThreadResult(m.id()); + } else { + sch.getFunctionResult(m.id(), 2000); + } + } + + REQUIRE(resetCount == expectedResets); } } diff --git a/tests/test/scheduler/test_scheduler.cpp b/tests/test/scheduler/test_scheduler.cpp index 8cc3b23f9..be67f9afe 100644 --- a/tests/test/scheduler/test_scheduler.cpp +++ b/tests/test/scheduler/test_scheduler.cpp @@ -1,6 +1,7 @@ #include #include "DummyExecutorFactory.h" +#include "faabric/util/snapshot.h" #include "faabric_utils.h" #include "fixtures.h" @@ -29,7 +30,9 @@ class SlowExecutor final : public Executor public: SlowExecutor(faabric::Message& msg) : Executor(msg) - {} + { + setUpDummyMemory(dummyMemorySize); + } ~SlowExecutor() {} @@ -44,6 +47,22 @@ class SlowExecutor final : public Executor SLEEP_MS(SHORT_TEST_TIMEOUT_MS); return 0; } + + std::span getMemoryView() override + { + return { dummyMemory.get(), dummyMemorySize }; + } + + void setUpDummyMemory(size_t memSize) + { + SPDLOG_DEBUG("Slow test executor initialising memory size {}", memSize); + dummyMemory = faabric::util::allocatePrivateMemory(memSize); + dummyMemorySize = memSize; + } + + private: + faabric::util::MemoryRegion dummyMemory = nullptr; + size_t dummyMemorySize = 2 * faabric::util::HOST_PAGE_SIZE; }; class SlowExecutorFactory : public ExecutorFactory @@ -59,6 +78,7 @@ class SlowExecutorFixture : public RedisTestFixture , public SchedulerTestFixture , public ConfTestFixture + , public SnapshotTestFixture { public: SlowExecutorFixture() @@ -195,10 +215,21 @@ TEST_CASE_METHOD(SlowExecutorFixture, "Test batch scheduling", "[scheduler]") faabric::util::SystemConfig& conf = faabric::util::getSystemConfig(); conf.overrideCpuCount = thisCores; + int nCallsOne = 10; + int nCallsTwo = 20; + + std::shared_ptr reqOne = + faabric::util::batchExecFactory("foo", "bar", nCallsOne); + const faabric::Message firstMsg = reqOne->messages().at(0); + int appId = firstMsg.appid(); + + size_t snapSize = 2 * faabric::util::HOST_PAGE_SIZE; + auto snap = std::make_shared(snapSize); + SECTION("Threads") { execMode = faabric::BatchExecuteRequest::THREADS; - expectedSnapshot = "threadSnap"; + expectedSnapshot = faabric::util::getMainThreadSnapshotKey(firstMsg); expectedSubType = 123; expectedContextData = "thread context"; @@ -219,30 +250,13 @@ TEST_CASE_METHOD(SlowExecutorFixture, "Test batch scheduling", "[scheduler]") expectedSnapshot = ""; } - bool isThreads = execMode == faabric::BatchExecuteRequest::THREADS; - - // Set up a dummy snapshot if necessary - faabric::snapshot::SnapshotRegistry& snapRegistry = - faabric::snapshot::getSnapshotRegistry(); - - std::unique_ptr snapshotDataAllocation; - std::vector snapshotMergeRegions; + // Set up the snapshot if (!expectedSnapshot.empty()) { - auto snap = std::make_shared(1234); - - snap->addMergeRegion(123, - 1234, - faabric::util::SnapshotDataType::Int, - faabric::util::SnapshotMergeOperation::Sum); - - snap->addMergeRegion(345, - 3456, - faabric::util::SnapshotDataType::Raw, - faabric::util::SnapshotMergeOperation::Overwrite); - - snapRegistry.registerSnapshot(expectedSnapshot, snap); + reg.registerSnapshot(expectedSnapshot, snap); } + bool isThreads = execMode == faabric::BatchExecuteRequest::THREADS; + // Mock everything faabric::util::setMockMode(true); @@ -252,9 +266,7 @@ TEST_CASE_METHOD(SlowExecutorFixture, "Test batch scheduling", "[scheduler]") std::string otherHost = "beta"; sch.addHostToGlobalSet(otherHost); - int nCallsOne = 10; - int nCallsTwo = 5; - int otherCores = 11; + int otherCores = 15; int nCallsOffloadedOne = nCallsOne - thisCores; faabric::HostResources thisResources; @@ -268,37 +280,45 @@ TEST_CASE_METHOD(SlowExecutorFixture, "Test batch scheduling", "[scheduler]") faabric::scheduler::queueResourceResponse(otherHost, otherResources); // Set up the messages - std::shared_ptr reqOne = - faabric::util::batchExecFactory("foo", "bar", nCallsOne); - reqOne->set_type(execMode); - reqOne->set_subtype(expectedSubType); - reqOne->set_contextdata(expectedContextData); - - const faabric::Message firstMsg = reqOne->messages().at(0); faabric::util::SchedulingDecision expectedDecisionOne(firstMsg.appid(), firstMsg.groupid()); for (int i = 0; i < nCallsOne; i++) { // Set snapshot key faabric::Message& msg = reqOne->mutable_messages()->at(i); - msg.set_snapshotkey(expectedSnapshot); + + if (!isThreads) { + msg.set_snapshotkey(expectedSnapshot); + } // Set app index msg.set_appidx(i); // Expect this host to handle up to its number of cores - bool isThisHost = i < thisCores; - if (isThisHost) { - expectedDecisionOne.addMessage(thisHost, msg); - } else { - expectedDecisionOne.addMessage(otherHost, msg); - } + std::string host = i < thisCores ? thisHost : otherHost; + expectedDecisionOne.addMessage(host, msg); } // Schedule the functions + reqOne->set_type(execMode); + reqOne->set_subtype(expectedSubType); + reqOne->set_contextdata(expectedContextData); + faabric::util::SchedulingDecision actualDecisionOne = sch.callFunctions(reqOne); + + // Check decision is as expected checkSchedulingDecisionEquality(actualDecisionOne, expectedDecisionOne); + // Await the results + for (int i = 0; i < thisCores; i++) { + faabric::Message& m = reqOne->mutable_messages()->at(i); + if (isThreads) { + sch.awaitThreadResult(m.id()); + } else { + sch.getFunctionResult(m.id(), 10000); + } + } + // Check resource requests have been made to other host auto resRequestsOne = faabric::scheduler::getResourceRequests(); REQUIRE(resRequestsOne.size() == 1); @@ -311,14 +331,12 @@ TEST_CASE_METHOD(SlowExecutorFixture, "Test batch scheduling", "[scheduler]") } else { REQUIRE(snapshotPushes.size() == 1); - auto snapshot = snapRegistry.getSnapshot(expectedSnapshot); + auto snapshot = reg.getSnapshot(expectedSnapshot); auto pushedSnapshot = snapshotPushes.at(0); REQUIRE(pushedSnapshot.first == otherHost); REQUIRE(pushedSnapshot.second->getSize() == snapshot->getSize()); REQUIRE(pushedSnapshot.second->getDataPtr() == snapshot->getDataPtr()); - REQUIRE(pushedSnapshot.second->getMergeRegions().size() == - snapshot->getMergeRegions().size()); } // Check the executor counts on this host @@ -333,7 +351,7 @@ TEST_CASE_METHOD(SlowExecutorFixture, "Test batch scheduling", "[scheduler]") } REQUIRE(res.slots() == thisCores); - REQUIRE(res.usedslots() == thisCores); + REQUIRE(res.usedslots() == 0); // Check the number of messages executed locally and remotely REQUIRE(sch.getRecordedMessagesLocal().size() == thisCores); @@ -356,16 +374,25 @@ TEST_CASE_METHOD(SlowExecutorFixture, "Test batch scheduling", "[scheduler]") // Set up resource response again faabric::scheduler::queueResourceResponse(otherHost, otherResources); - // Now schedule a second batch and check they're all sent to the other host + // Now schedule a second batch and check the decision std::shared_ptr reqTwo = faabric::util::batchExecFactory("foo", "bar", nCallsTwo); + const faabric::Message& firstMsg2 = reqTwo->messages().at(0); - faabric::util::SchedulingDecision expectedDecisionTwo(firstMsg2.appid(), + faabric::util::SchedulingDecision expectedDecisionTwo(appId, firstMsg2.groupid()); for (int i = 0; i < nCallsTwo; i++) { faabric::Message& msg = reqTwo->mutable_messages()->at(i); - msg.set_snapshotkey(expectedSnapshot); - expectedDecisionTwo.addMessage(otherHost, msg); + + msg.set_appid(appId); + msg.set_appidx(i); + + if (!isThreads) { + msg.set_snapshotkey(expectedSnapshot); + } + + std::string host = i < thisCores ? thisHost : otherHost; + expectedDecisionTwo.addMessage(host, msg); } // Create the batch request @@ -375,18 +402,28 @@ TEST_CASE_METHOD(SlowExecutorFixture, "Test batch scheduling", "[scheduler]") faabric::util::SchedulingDecision actualDecisionTwo = sch.callFunctions(reqTwo); + // Check scheduling decision + checkSchedulingDecisionEquality(actualDecisionTwo, expectedDecisionTwo); + + // Await the results + for (int i = 0; i < thisCores; i++) { + faabric::Message& m = reqTwo->mutable_messages()->at(i); + if (isThreads) { + sch.awaitThreadResult(m.id()); + } else { + sch.getFunctionResult(m.id(), 10000); + } + } + // Check resource request made again auto resRequestsTwo = faabric::scheduler::getResourceRequests(); REQUIRE(resRequestsTwo.size() == 1); REQUIRE(resRequestsTwo.at(0).first == otherHost); - // Check scheduling decision - checkSchedulingDecisionEquality(actualDecisionTwo, expectedDecisionTwo); - // Check no other functions have been scheduled on this host - REQUIRE(sch.getRecordedMessagesLocal().size() == thisCores); + REQUIRE(sch.getRecordedMessagesLocal().size() == (2 * thisCores)); REQUIRE(sch.getRecordedMessagesShared().size() == - nCallsOffloadedOne + nCallsTwo); + (nCallsOne + nCallsTwo) - (2 * thisCores)); if (isThreads) { REQUIRE(sch.getFunctionExecutorCount(m) == 1); @@ -401,7 +438,7 @@ TEST_CASE_METHOD(SlowExecutorFixture, "Test batch scheduling", "[scheduler]") REQUIRE(pTwo.first == otherHost); // Check the request to the other host - REQUIRE(pTwo.second->messages_size() == nCallsTwo); + REQUIRE(pTwo.second->messages_size() == nCallsTwo - thisCores); } TEST_CASE_METHOD(SlowExecutorFixture, @@ -416,10 +453,16 @@ TEST_CASE_METHOD(SlowExecutorFixture, faabric::BatchExecuteRequest::BatchExecuteType execMode; std::string expectedSnapshot; + // Submit more calls than we have capacity for + int nCalls = 10; + std::shared_ptr req = + faabric::util::batchExecFactory("foo", "bar", nCalls); + SECTION("Threads") { execMode = faabric::BatchExecuteRequest::THREADS; - expectedSnapshot = "threadSnap"; + expectedSnapshot = + faabric::util::getMainThreadSnapshotKey(req->messages().at(0)); } SECTION("Processes") @@ -430,14 +473,10 @@ TEST_CASE_METHOD(SlowExecutorFixture, SECTION("Functions") { execMode = faabric::BatchExecuteRequest::FUNCTIONS; } - // Set up snapshot if necessary - faabric::snapshot::SnapshotRegistry& snapRegistry = - faabric::snapshot::getSnapshotRegistry(); - size_t snapSize = 1234; if (!expectedSnapshot.empty()) { auto snap = std::make_shared(snapSize); - snapRegistry.registerSnapshot(expectedSnapshot, snap); + reg.registerSnapshot(expectedSnapshot, snap); } // Set up this host with very low resources @@ -454,22 +493,23 @@ TEST_CASE_METHOD(SlowExecutorFixture, resOther.set_slots(2); faabric::scheduler::queueResourceResponse(otherHost, resOther); - // Submit more calls than we have capacity for - int nCalls = 10; - std::shared_ptr req = - faabric::util::batchExecFactory("foo", "bar", nCalls); + // Make the request req->set_type(execMode); - const faabric::Message firstMsg = req->messages().at(0); faabric::util::SchedulingDecision expectedDecision(firstMsg.appid(), firstMsg.groupid()); + std::vector mids; for (int i = 0; i < nCalls; i++) { faabric::Message& msg = req->mutable_messages()->at(i); - msg.set_snapshotkey(expectedSnapshot); + + if (req->type() != faabric::BatchExecuteRequest::THREADS) { + msg.set_snapshotkey(expectedSnapshot); + } if (i == 1 || i == 2) { expectedDecision.addMessage(otherHost, msg); } else { + mids.emplace_back(msg.id()); expectedDecision.addMessage(thisHost, msg); } } @@ -488,6 +528,15 @@ TEST_CASE_METHOD(SlowExecutorFixture, } REQUIRE(sch.getFunctionExecutorCount(firstMsg) == expectedExecutors); + + // Await results + for (const auto& mid : mids) { + if (execMode == faabric::BatchExecuteRequest::THREADS) { + sch.awaitThreadResult(mid); + } else { + sch.getFunctionResult(mid, 10000); + } + } } TEST_CASE_METHOD(SlowExecutorFixture, "Test unregistering host", "[scheduler]") diff --git a/tests/test/snapshot/test_snapshot_diffs.cpp b/tests/test/snapshot/test_snapshot_diffs.cpp index 066b01ad3..c673ce3ab 100644 --- a/tests/test/snapshot/test_snapshot_diffs.cpp +++ b/tests/test/snapshot/test_snapshot_diffs.cpp @@ -3,6 +3,7 @@ #include "faabric_utils.h" #include +#include #include using namespace faabric::snapshot; @@ -34,25 +35,36 @@ TEST_CASE_METHOD(SnapshotTestFixture, auto snap = std::make_shared(snapSize); reg.registerSnapshot(snapKey, snap); - int sharedMemPages = 8; - size_t sharedMemSize = sharedMemPages * HOST_PAGE_SIZE; - MemoryRegion sharedMem = allocatePrivateMemory(sharedMemSize); + int memPages = 8; + size_t memSize = memPages * HOST_PAGE_SIZE; + MemoryRegion mem = allocatePrivateMemory(memSize); + std::span memView(mem.get(), memSize); // Check we can write to shared mem - sharedMem[0] = 1; + mem[0] = 1; // Map to the snapshot - snap->mapToMemory({ sharedMem.get(), snapSize }); + snap->mapToMemory({ mem.get(), snapSize }); + + // Track changes + tracker.startTracking(memView); + tracker.startThreadLocalTracking(memView); // Make various changes - sharedMem[0] = 1; - sharedMem[2 * HOST_PAGE_SIZE] = 1; - sharedMem[3 * HOST_PAGE_SIZE + 10] = 1; - sharedMem[8 * HOST_PAGE_SIZE - 20] = 1; + mem[0] = 1; + mem[2 * HOST_PAGE_SIZE] = 1; + mem[3 * HOST_PAGE_SIZE + 10] = 1; + mem[8 * HOST_PAGE_SIZE - 20] = 1; + + tracker.stopTracking(memView); + tracker.stopThreadLocalTracking(memView); + + // Check there are no diffs even though we have dirty regions + auto dirtyRegions = tracker.getBothDirtyOffsets(memView); + REQUIRE(!dirtyRegions.empty()); - // Check there are no diffs std::vector changeDiffs = - MemoryView({ sharedMem.get(), sharedMemSize }).diffWithSnapshot(snap); + snap->diffWithDirtyRegions(memView, dirtyRegions); REQUIRE(changeDiffs.empty()); } @@ -66,20 +78,24 @@ TEST_CASE_METHOD(SnapshotTestFixture, "Test snapshot diffs", "[snapshot]") reg.registerSnapshot(snapKey, snap); // Make shared memory larger than original snapshot - int sharedMemPages = 8; - size_t sharedMemSize = sharedMemPages * HOST_PAGE_SIZE; - MemoryRegion sharedMem = allocatePrivateMemory(sharedMemSize); + int memPages = 8; + size_t memSize = memPages * HOST_PAGE_SIZE; + MemoryRegion mem = allocatePrivateMemory(memSize); + std::span memView(mem.get(), memSize); // Map the snapshot to the start of the memory - snap->mapToMemory({ sharedMem.get(), snapSize }); + snap->mapToMemory({ mem.get(), snapSize }); // Reset dirty tracking - faabric::util::resetDirtyTracking(); + faabric::util::DirtyTracker& tracker = faabric::util::getDirtyTracker(); + tracker.clearAll(); + tracker.startTracking(memView); + tracker.startThreadLocalTracking(memView); // Single change, single merge region std::vector dataA = { 1, 2, 3, 4 }; int offsetA = HOST_PAGE_SIZE; - std::memcpy(sharedMem.get() + offsetA, dataA.data(), dataA.size()); + std::memcpy(mem.get() + offsetA, dataA.data(), dataA.size()); snap->addMergeRegion(offsetA, dataA.size(), @@ -91,7 +107,7 @@ TEST_CASE_METHOD(SnapshotTestFixture, "Test snapshot diffs", "[snapshot]") 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.get() + offsetC, dataC.data(), dataC.size()); + std::memcpy(mem.get() + offsetC, dataC.data(), dataC.size()); int regionOffsetC = offsetC - 3; snap->addMergeRegion(regionOffsetC, @@ -104,8 +120,8 @@ TEST_CASE_METHOD(SnapshotTestFixture, "Test snapshot diffs", "[snapshot]") std::vector dataB2 = { 7, 6, 5 }; int offsetB1 = HOST_PAGE_SIZE + 10; int offsetB2 = HOST_PAGE_SIZE + 16; - std::memcpy(sharedMem.get() + offsetB1, dataB1.data(), dataB1.size()); - std::memcpy(sharedMem.get() + offsetB2, dataB2.data(), dataB2.size()); + std::memcpy(mem.get() + offsetB1, dataB1.data(), dataB1.size()); + std::memcpy(mem.get() + offsetB2, dataB2.data(), dataB2.size()); snap->addMergeRegion(offsetB1, (offsetB2 - offsetB1) + dataB2.size() + 10, @@ -116,7 +132,7 @@ TEST_CASE_METHOD(SnapshotTestFixture, "Test snapshot diffs", "[snapshot]") 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.get() + offsetD, dataD.data(), dataD.size()); + std::memcpy(mem.get() + offsetD, dataD.data(), dataD.size()); int regionOffsetD = offsetD + 2; int regionSizeD = dataD.size() - 4; @@ -131,8 +147,7 @@ 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.get() + extraOffset, dataExtra.data(), dataExtra.size()); + std::memcpy(mem.get() + extraOffset, dataExtra.data(), dataExtra.size()); int extraRegionOffset = extraOffset - 2; int extraRegionSize = dataExtra.size() + 4; @@ -145,19 +160,17 @@ TEST_CASE_METHOD(SnapshotTestFixture, "Test snapshot diffs", "[snapshot]") // a dirty page std::vector dataNoChange = { 0, 0, 0 }; int offsetNoChange = 4 * HOST_PAGE_SIZE - 10; - std::memcpy(sharedMem.get() + offsetNoChange, - dataNoChange.data(), - dataNoChange.size()); + std::memcpy( + mem.get() + offsetNoChange, dataNoChange.data(), dataNoChange.size()); - // Check shared memory does have dirty pages (including the non-change) - std::vector sharedDirtyPages = - getDirtyPageNumbers(sharedMem.get(), sharedMemPages); - std::vector expected = { 1, 2, 3, 6 }; - REQUIRE(sharedDirtyPages == expected); + // Stop tracking + tracker.stopTracking(memView); + tracker.stopThreadLocalTracking(memView); // Check we have the right number of diffs + auto dirtyRegions = tracker.getBothDirtyOffsets(memView); std::vector changeDiffs = - MemoryView({ sharedMem.get(), sharedMemSize }).diffWithSnapshot(snap); + snap->diffWithDirtyRegions(memView, dirtyRegions); REQUIRE(changeDiffs.size() == 6); diff --git a/tests/test/util/test_bytes.cpp b/tests/test/util/test_bytes.cpp index 20d549dff..c9123d24a 100644 --- a/tests/test/util/test_bytes.cpp +++ b/tests/test/util/test_bytes.cpp @@ -158,4 +158,88 @@ TEST_CASE("Test format byte array to string", "[util]") REQUIRE(formatByteArrayToIntString(bytesIn) == expectedString); } + +TEST_CASE("Test diffing byte array regions", "[util]") +{ + std::vector a; + std::vector b; + std::vector> expected; + + SECTION("Equal") + { + a = { 0, 1, 2, 3 }; + b = { 0, 1, 2, 3 }; + } + + SECTION("Empty") {} + + SECTION("Not equal") + { + a = { 0, 0, 2, 2, 3, 3, 4, 4, 5, 5 }; + b = { 0, 1, 1, 2, 3, 6, 6, 6, 5, 5 }; + expected = { + { 1, 2 }, + { 5, 3 }, + }; + } + + SECTION("Single length") + { + a = { 0, 1, 2, 3, 4 }; + b = { 0, 1, 3, 3, 4 }; + expected = { { 2, 1 } }; + } + + SECTION("Difference at start") + { + a = { 0, 1, 2, 3, 4, 5, 6 }; + b = { 1, 2, 3, 3, 3, 4, 6 }; + expected = { { 0, 3 }, { 4, 2 } }; + } + + SECTION("Difference at end") + { + a = { 0, 1, 2, 3, 4, 5, 6 }; + b = { 0, 1, 1, 3, 3, 4, 5 }; + expected = { { 2, 1 }, { 4, 3 } }; + } + + std::vector> actual = + diffArrayRegions({ a.data(), a.size() }, { b.data(), b.size() }); + + REQUIRE(actual.size() == expected.size()); + for (int i = 0; i < actual.size(); i++) { + REQUIRE(actual.at(i).first == expected.at(i).first); + REQUIRE(actual.at(i).second == expected.at(i).second); + } +} + +TEST_CASE("Test diffing byte arrays", "[util]") +{ + std::vector a; + std::vector b; + std::vector expected; + + SECTION("Equal") + { + a = { 0, 1, 2, 3 }; + b = { 0, 1, 2, 3 }; + expected = std::vector(a.size(), false); + } + + SECTION("Empty") {} + + SECTION("Not equal") + { + a = { 0, 0, 2, 2, 3, 3, 4, 4, 5, 5 }; + b = { 0, 1, 1, 2, 3, 6, 6, 6, 5, 5 }; + expected = { false, true, true, false, false, + true, true, true, false, false }; + } + + std::vector actual = + diffArrays({ a.data(), a.size() }, { b.data(), b.size() }); + + REQUIRE(actual == expected); +} } diff --git a/tests/test/util/test_config.cpp b/tests/test/util/test_config.cpp index 8927f135e..2eb3b6b14 100644 --- a/tests/test/util/test_config.cpp +++ b/tests/test/util/test_config.cpp @@ -29,6 +29,8 @@ TEST_CASE("Test default system config initialisation", "[util]") REQUIRE(conf.defaultMpiWorldSize == 5); REQUIRE(conf.mpiBasePort == 10800); + + REQUIRE(conf.dirtyTrackingMode == "segfault"); } TEST_CASE("Test overriding system config initialisation", "[util]") @@ -59,6 +61,8 @@ TEST_CASE("Test overriding system config initialisation", "[util]") std::string mpiSize = setEnvVar("DEFAULT_MPI_WORLD_SIZE", "2468"); std::string mpiPort = setEnvVar("MPI_BASE_PORT", "9999"); + std::string dirtyMode = setEnvVar("DIRTY_TRACKING_MODE", "dummy-track"); + // Create new conf for test SystemConfig conf; @@ -85,6 +89,8 @@ TEST_CASE("Test overriding system config initialisation", "[util]") REQUIRE(conf.defaultMpiWorldSize == 2468); REQUIRE(conf.mpiBasePort == 9999); + REQUIRE(conf.dirtyTrackingMode == "dummy-track"); + // Be careful with host type setEnvVar("LOG_LEVEL", logLevel); setEnvVar("LOG_FILE", logFile); @@ -110,6 +116,8 @@ TEST_CASE("Test overriding system config initialisation", "[util]") setEnvVar("DEFAULT_MPI_WORLD_SIZE", mpiSize); setEnvVar("MPI_BASE_PORT", mpiPort); + + setEnvVar("DIRTY_TRACKING_MODE", dirtyMode); } } diff --git a/tests/test/util/test_dirty.cpp b/tests/test/util/test_dirty.cpp new file mode 100644 index 000000000..2f8812089 --- /dev/null +++ b/tests/test/util/test_dirty.cpp @@ -0,0 +1,356 @@ +#include + +#include "fixtures.h" + +#include +#include + +#include +#include +#include + +using namespace faabric::util; + +namespace tests { + +class DirtyConfTestFixture + : public ConfTestFixture + , public DirtyTrackingTestFixture +{ + public: + DirtyConfTestFixture() = default; + ~DirtyConfTestFixture() = default; +}; + +TEST_CASE_METHOD(DirtyConfTestFixture, + "Test dirty page checking", + "[util][dirty]") +{ + SECTION("Soft dirty PTEs") { conf.dirtyTrackingMode = "softpte"; } + + SECTION("Segfaults") { conf.dirtyTrackingMode = "segfault"; } + + DirtyTracker& tracker = getDirtyTracker(); + + // Create several pages of memory + int nPages = 6; + size_t memSize = HOST_PAGE_SIZE * nPages; + MemoryRegion memPtr = allocatePrivateMemory(memSize); + std::span memView(memPtr.get(), memSize); + + tracker.clearAll(); + + std::vector> actual = + tracker.getBothDirtyOffsets(memView); + REQUIRE(actual.empty()); + + tracker.startTracking(memView); + tracker.startThreadLocalTracking(memView); + + // Dirty two of the pages + uint8_t* pageZero = memPtr.get(); + uint8_t* pageOne = pageZero + HOST_PAGE_SIZE; + uint8_t* pageThree = pageOne + (2 * HOST_PAGE_SIZE); + + pageOne[10] = 1; + pageThree[123] = 4; + + std::vector> expected = { + std::pair(HOST_PAGE_SIZE, HOST_PAGE_SIZE), + std::pair(3 * HOST_PAGE_SIZE, HOST_PAGE_SIZE) + }; + + actual = tracker.getBothDirtyOffsets(memView); + REQUIRE(actual == expected); + + // And another + uint8_t* pageFive = pageThree + (2 * HOST_PAGE_SIZE); + pageFive[99] = 3; + + expected.emplace_back(5 * HOST_PAGE_SIZE, HOST_PAGE_SIZE); + actual = tracker.getBothDirtyOffsets(memView); + REQUIRE(actual == expected); + + // Reset + tracker.stopTracking(memView); + tracker.stopThreadLocalTracking(memView); + tracker.startTracking(memView); + tracker.startThreadLocalTracking(memView); + + actual = tracker.getBothDirtyOffsets(memView); + REQUIRE(actual.empty()); + + // Check the data hasn't changed + REQUIRE(pageOne[10] == 1); + REQUIRE(pageThree[123] == 4); + REQUIRE(pageFive[99] == 3); + + // Set some other data + uint8_t* pageFour = pageThree + HOST_PAGE_SIZE; + pageThree[100] = 2; + pageFour[22] = 5; + + // As pages are adjacent we get a single region + expected = { + std::pair(3 * HOST_PAGE_SIZE, 2 * HOST_PAGE_SIZE), + }; + actual = tracker.getBothDirtyOffsets(memView); + REQUIRE(actual == expected); + + // Final reset and check + tracker.stopTracking(memView); + tracker.stopThreadLocalTracking(memView); + + tracker.startTracking(memView); + tracker.startThreadLocalTracking(memView); + actual = tracker.getBothDirtyOffsets(memView); + REQUIRE(actual.empty()); + + tracker.stopTracking(memView); + tracker.stopThreadLocalTracking(memView); +} + +TEST_CASE_METHOD(DirtyConfTestFixture, + "Test dirty region checking", + "[util][dirty]") +{ + SECTION("Segfaults") { conf.dirtyTrackingMode = "segfault"; } + + SECTION("Soft PTEs") { conf.dirtyTrackingMode = "softpte"; } + + tracker = getDirtyTracker(); + + int nPages = 15; + size_t memSize = HOST_PAGE_SIZE * nPages; + MemoryRegion mem = allocateSharedMemory(memSize); + std::span memView(mem.get(), memSize); + + DirtyTracker& tracker = getDirtyTracker(); + tracker.clearAll(); + + std::vector> actual = + tracker.getBothDirtyOffsets({ mem.get(), memSize }); + REQUIRE(actual.empty()); + + tracker.startTracking(memView); + tracker.startThreadLocalTracking(memView); + + // Dirty some pages, some adjacent + uint8_t* pageZero = mem.get(); + uint8_t* pageOne = pageZero + HOST_PAGE_SIZE; + uint8_t* pageThree = pageZero + (3 * HOST_PAGE_SIZE); + uint8_t* pageFour = pageZero + (4 * HOST_PAGE_SIZE); + uint8_t* pageSeven = pageZero + (7 * HOST_PAGE_SIZE); + uint8_t* pageNine = pageZero + (9 * HOST_PAGE_SIZE); + + // Set some byte within each page + pageZero[1] = 1; + pageOne[11] = 1; + pageThree[33] = 1; + pageFour[44] = 1; + pageSeven[77] = 1; + pageNine[99] = 1; + + tracker.stopTracking({ mem.get(), memSize }); + tracker.stopThreadLocalTracking({ mem.get(), memSize }); + + // Expect adjacent regions to be merged + std::vector> expected = { + std::pair(0, 2 * HOST_PAGE_SIZE), + std::pair(3 * HOST_PAGE_SIZE, 2 * HOST_PAGE_SIZE), + std::pair(7 * HOST_PAGE_SIZE, HOST_PAGE_SIZE), + std::pair(9 * HOST_PAGE_SIZE, HOST_PAGE_SIZE) + }; + + actual = tracker.getBothDirtyOffsets({ mem.get(), memSize }); + + REQUIRE(actual.size() == expected.size()); + + REQUIRE(actual == expected); +} + +TEST_CASE_METHOD(DirtyConfTestFixture, + "Test segfault tracking", + "[util][dirty]") +{ + conf.dirtyTrackingMode = "segfault"; + tracker = getDirtyTracker(); + + size_t memSize = 10 * HOST_PAGE_SIZE; + std::vector expectedData(memSize, 5); + + MemoryRegion mem = allocatePrivateMemory(memSize); + + std::span memView(mem.get(), memSize); + + SECTION("Standard alloc") + { + // Copy expected data into memory + std::memcpy(mem.get(), expectedData.data(), memSize); + } + + SECTION("Mapped from fd") + { + // Create a file descriptor holding expected data + int fd = createFd(memSize, "foobar"); + writeToFd(fd, 0, expectedData); + + // Map the memory + mapMemoryPrivate(memView, fd); + } + + // Check memory to start with + std::vector actualMemBefore(mem.get(), mem.get() + memSize); + REQUIRE(actualMemBefore == expectedData); + + // Start tracking + tracker.startTracking(memView); + tracker.startThreadLocalTracking(memView); + + // Make a change on one page + size_t offsetA = 0; + mem[offsetA] = 3; + expectedData[offsetA] = 3; + + // Make two changes on adjacent page + size_t offsetB1 = HOST_PAGE_SIZE + 10; + size_t offsetB2 = HOST_PAGE_SIZE + 50; + mem[offsetB1] = 4; + mem[offsetB2] = 2; + expectedData[offsetB1] = 4; + expectedData[offsetB2] = 2; + + // Change another page + size_t offsetC = (5 * HOST_PAGE_SIZE) + 10; + mem[offsetC] = 6; + expectedData[offsetC] = 6; + + // Just read from another (should not cause a diff) + int readValue = mem[4 * HOST_PAGE_SIZE + 5]; + REQUIRE(readValue == 5); + + // Check writes have propagated to the actual memory + std::vector actualMemAfter(mem.get(), mem.get() + memSize); + REQUIRE(actualMemAfter == expectedData); + + // Get dirty regions + std::vector> actualDirty = + tracker.getBothDirtyOffsets(memView); + + // Check dirty regions + REQUIRE(actualDirty.size() == 2); + + std::vector> expectedDirty = { + { 0, 2 * HOST_PAGE_SIZE }, + { (uint32_t)(5 * HOST_PAGE_SIZE), HOST_PAGE_SIZE } + }; + + REQUIRE(actualDirty == expectedDirty); + + tracker.stopTracking(memView); + tracker.stopThreadLocalTracking(memView); +} + +TEST_CASE_METHOD(DirtyConfTestFixture, + "Test multi-threaded segfault tracking", + "[util][dirty]") +{ + // Here we want to check that faults triggered in a given thread are caught + // by that thread, and so we can safely just to thread-local diff tracking. + conf.dirtyTrackingMode = "segfault"; + tracker = getDirtyTracker(); + + int nLoops = 20; + + // Deliberately cause contention + int nThreads = 100; + size_t memSize = 2 * nThreads * HOST_PAGE_SIZE; + + MemoryRegion mem = allocatePrivateMemory(memSize); + std::span memView(mem.get(), memSize); + + for (int loop = 0; loop < nLoops; loop++) { + std::vector>> success; + success.resize(nThreads); + + // Start global tracking + tracker.startTracking(memView); + + std::vector threads; + threads.reserve(nThreads); + for (int i = 0; i < nThreads; i++) { + threads.emplace_back([this, &success, &memView, i, loop] { + success.at(i) = std::make_shared>(); + + // Start thread-local tracking + tracker.startThreadLocalTracking(memView); + + // Modify a couple of pages specific to this thread + size_t pageOffset = i * 2 * HOST_PAGE_SIZE; + uint8_t* pageOne = memView.data() + pageOffset; + uint8_t* pageTwo = memView.data() + pageOffset + HOST_PAGE_SIZE; + + pageOne[20] = 3; + pageOne[250] = 5; + pageOne[HOST_PAGE_SIZE - 20] = 6; + + pageTwo[35] = 2; + pageTwo[HOST_PAGE_SIZE - 100] = 3; + + tracker.stopThreadLocalTracking(memView); + + // Check we get the right number of dirty regions + std::vector> regions = + tracker.getThreadLocalDirtyOffsets(memView); + if (regions.size() != 1) { + SPDLOG_ERROR("Segfault thread {} failed on loop {}. Got {} " + "regions instead of {}", + i, + loop, + regions.size(), + 1); + return; + } + + std::vector> expected = { + std::pair(pageOffset, + 2 * HOST_PAGE_SIZE), + }; + + if (regions != expected) { + SPDLOG_ERROR( + "Segfault thread {} failed on loop {}. Regions not equal", + i, + loop); + success.at(i)->store(false); + } else { + success.at(i)->store(true); + } + }); + } + + for (auto& t : threads) { + if (t.joinable()) { + t.join(); + } + } + + // Stop tracking + tracker.stopTracking(memView); + + // Check no global offsets + REQUIRE(tracker.getDirtyOffsets(memView).empty()); + + bool thisLoopSuccess = true; + for (int i = 0; i < nThreads; i++) { + if (!success.at(i)->load()) { + SPDLOG_ERROR( + "Segfault thread test thread {} on loop {} failed", i, loop); + thisLoopSuccess = false; + } + } + + REQUIRE(thisLoopSuccess); + } +} +} diff --git a/tests/test/util/test_memory.cpp b/tests/test/util/test_memory.cpp index d23ffe9cd..a7bc97905 100644 --- a/tests/test/util/test_memory.cpp +++ b/tests/test/util/test_memory.cpp @@ -1,5 +1,8 @@ #include +#include "fixtures.h" + +#include #include #include @@ -11,7 +14,59 @@ using namespace faabric::util; namespace tests { -TEST_CASE("Test rounding down offsets to page size", "[memory]") +TEST_CASE("Test dedupe memory regions", "[util][memory]") +{ + std::vector> input; + std::vector> expected; + + uint32_t offsetA = 0; + uint32_t offsetB = 10; + + uint32_t sizeA = 2; + uint32_t sizeB = 3; + uint32_t sizeC = 4; + + SECTION("Empty") {} + + SECTION("Nothing to do") + { + input = { { offsetA, sizeA } }; + expected = input; + } + + SECTION("Equal on the same offset") + { + input = { + { offsetB, sizeB }, + { offsetA, sizeA }, + { offsetA, sizeA }, + }; + expected = { + { offsetA, sizeA }, + { offsetB, sizeB }, + }; + } + + SECTION("Longer on the same offset") + { + input = { + { offsetB, sizeB }, + { offsetA, sizeA }, + { offsetA, sizeC }, + { offsetA, sizeB }, + }; + expected = { + { offsetA, sizeC }, + { offsetB, sizeB }, + }; + } + + std::vector> actual = + dedupeMemoryRegions(input); + REQUIRE(actual == expected); +} + +TEST_CASE("Test rounding down offsets to page size", "[util][memory]") { REQUIRE(faabric::util::alignOffsetDown(2 * faabric::util::HOST_PAGE_SIZE) == 2 * faabric::util::HOST_PAGE_SIZE); @@ -30,7 +85,7 @@ TEST_CASE("Test rounding down offsets to page size", "[memory]") 866 * faabric::util::HOST_PAGE_SIZE); } -TEST_CASE("Check CoW memory mapping", "[memory]") +TEST_CASE("Check CoW memory mapping", "[util][memory]") { size_t memSize = getpagesize(); @@ -96,7 +151,7 @@ TEST_CASE("Check CoW memory mapping", "[memory]") munmap(regionBVoid, 3 * memSize); } -TEST_CASE("Check shared memory mapping", "[memory]") +TEST_CASE("Check shared memory mapping", "[util][memory]") { int pageSize = getpagesize(); size_t memSize = 4 * pageSize; @@ -183,7 +238,7 @@ TEST_CASE("Check shared memory mapping", "[memory]") munmap(regionBVoid, memSize); } -TEST_CASE("Test small aligned memory chunk", "[util]") +TEST_CASE("Test small aligned memory chunk", "[util][memory]") { AlignedChunk actual = getPageAlignedChunk(0, 10); @@ -196,7 +251,7 @@ TEST_CASE("Test small aligned memory chunk", "[util]") REQUIRE(actual.offsetRemainder == 0); } -TEST_CASE("Test aligned memory chunks near page boundaries", "[util]") +TEST_CASE("Test aligned memory chunks near page boundaries", "[util][memory]") { long originalOffset = 2 * faabric::util::HOST_PAGE_SIZE - 1; long originalLength = 3; @@ -211,7 +266,7 @@ TEST_CASE("Test aligned memory chunks near page boundaries", "[util]") REQUIRE(actual.offsetRemainder == faabric::util::HOST_PAGE_SIZE - 1); } -TEST_CASE("Test large offset memory chunk", "[util]") +TEST_CASE("Test large offset memory chunk", "[util][memory]") { long originalOffset = 2 * faabric::util::HOST_PAGE_SIZE + 33; long originalLength = 5 * faabric::util::HOST_PAGE_SIZE + 123; @@ -226,7 +281,7 @@ TEST_CASE("Test large offset memory chunk", "[util]") REQUIRE(actual.offsetRemainder == 33); } -TEST_CASE("Test already aligned memory chunk", "[util]") +TEST_CASE("Test already aligned memory chunk", "[util][memory]") { long originalOffset = 10 * faabric::util::HOST_PAGE_SIZE; long originalLength = 5 * faabric::util::HOST_PAGE_SIZE; @@ -241,122 +296,7 @@ TEST_CASE("Test already aligned memory chunk", "[util]") REQUIRE(actual.offsetRemainder == 0); } -TEST_CASE("Test dirty page checking", "[util]") -{ - // Create several pages of memory - int nPages = 6; - size_t memSize = faabric::util::HOST_PAGE_SIZE * nPages; - auto* sharedMemory = (uint8_t*)mmap( - nullptr, memSize, PROT_WRITE, MAP_SHARED | MAP_ANONYMOUS, -1, 0); - - if (sharedMemory == nullptr) { - FAIL("Could not provision memory"); - } - - resetDirtyTracking(); - - std::vector actual = - faabric::util::getDirtyPageNumbers(sharedMemory, nPages); - REQUIRE(actual.empty()); - - // Dirty two of the pages - uint8_t* pageZero = sharedMemory; - uint8_t* pageOne = pageZero + faabric::util::HOST_PAGE_SIZE; - uint8_t* pageThree = pageOne + (2 * faabric::util::HOST_PAGE_SIZE); - - pageOne[10] = 1; - pageThree[123] = 4; - - std::vector expected = { 1, 3 }; - actual = faabric::util::getDirtyPageNumbers(sharedMemory, nPages); - REQUIRE(actual == expected); - - // And another - uint8_t* pageFive = pageThree + (2 * faabric::util::HOST_PAGE_SIZE); - pageFive[99] = 3; - - expected = { 1, 3, 5 }; - actual = faabric::util::getDirtyPageNumbers(sharedMemory, nPages); - REQUIRE(actual == expected); - - // Reset - resetDirtyTracking(); - - actual = faabric::util::getDirtyPageNumbers(sharedMemory, nPages); - REQUIRE(actual.empty()); - - // Check the data hasn't changed - REQUIRE(pageOne[10] == 1); - REQUIRE(pageThree[123] == 4); - REQUIRE(pageFive[99] == 3); - - // Set some other data - uint8_t* pageFour = pageThree + faabric::util::HOST_PAGE_SIZE; - pageThree[100] = 2; - pageFour[22] = 5; - - expected = { 3, 4 }; - actual = faabric::util::getDirtyPageNumbers(sharedMemory, nPages); - REQUIRE(actual == expected); - - // Final reset and check - resetDirtyTracking(); - actual = faabric::util::getDirtyPageNumbers(sharedMemory, nPages); - REQUIRE(actual.empty()); - - munmap(sharedMemory, memSize); -} - -TEST_CASE("Test dirty region checking", "[util]") -{ - int nPages = 15; - size_t memSize = HOST_PAGE_SIZE * nPages; - auto* sharedMemory = (uint8_t*)mmap( - nullptr, memSize, PROT_WRITE, MAP_SHARED | MAP_ANONYMOUS, -1, 0); - - if (sharedMemory == nullptr) { - FAIL("Could not provision memory"); - } - - resetDirtyTracking(); - - std::vector> actual = - faabric::util::getDirtyRegions(sharedMemory, nPages); - REQUIRE(actual.empty()); - - // Dirty some pages, some adjacent - uint8_t* pageZero = sharedMemory; - uint8_t* pageOne = pageZero + HOST_PAGE_SIZE; - uint8_t* pageThree = pageZero + (3 * HOST_PAGE_SIZE); - uint8_t* pageFour = pageZero + (4 * HOST_PAGE_SIZE); - uint8_t* pageSeven = pageZero + (7 * HOST_PAGE_SIZE); - uint8_t* pageNine = pageZero + (9 * HOST_PAGE_SIZE); - - // Set some byte within each page - pageZero[1] = 1; - pageOne[11] = 1; - pageThree[33] = 1; - pageFour[44] = 1; - pageSeven[77] = 1; - pageNine[99] = 1; - - // Expect adjacent regions to be merged - std::vector> expected = { - { 0, 2 * HOST_PAGE_SIZE }, - { 3 * HOST_PAGE_SIZE, 5 * HOST_PAGE_SIZE }, - { 7 * HOST_PAGE_SIZE, 8 * HOST_PAGE_SIZE }, - { 9 * HOST_PAGE_SIZE, 10 * HOST_PAGE_SIZE }, - }; - - actual = faabric::util::getDirtyRegions(sharedMemory, nPages); - REQUIRE(actual.size() == expected.size()); - for (int i = 0; i < actual.size(); i++) { - REQUIRE(actual.at(i).first == expected.at(i).first); - REQUIRE(actual.at(i).second == expected.at(i).second); - } -} - -TEST_CASE("Test allocating and claiming memory", "[util]") +TEST_CASE("Test allocating and claiming memory", "[util][memory]") { // Allocate some virtual memory size_t vMemSize = 100 * HOST_PAGE_SIZE; @@ -387,7 +327,7 @@ TEST_CASE("Test allocating and claiming memory", "[util]") REQUIRE(vMem[sizeA + 4 * HOST_PAGE_SIZE + 10] == 6); } -TEST_CASE("Test mapping memory", "[util]") +TEST_CASE("Test mapping memory", "[util][memory]") { size_t vMemSize = 100 * HOST_PAGE_SIZE; MemoryRegion vMem = allocateVirtualMemory(vMemSize); @@ -430,20 +370,20 @@ TEST_CASE("Test mapping memory", "[util]") REQUIRE(memBData == expected); } -TEST_CASE("Test mapping memory fails with invalid fd", "[util]") +TEST_CASE("Test mapping memory fails with invalid fd", "[util][memory]") { size_t memSize = 10 * HOST_PAGE_SIZE; - MemoryRegion sharedMem = allocatePrivateMemory(memSize); + MemoryRegion mem = allocatePrivateMemory(memSize); int fd = 0; SECTION("Zero fd") { fd = 0; } SECTION("Negative fd") { fd = -2; } - REQUIRE_THROWS(mapMemoryPrivate({ sharedMem.get(), memSize }, fd)); + REQUIRE_THROWS(mapMemoryPrivate({ mem.get(), memSize }, fd)); } -TEST_CASE("Test remapping memory", "[util]") +TEST_CASE("Test remapping memory", "[util][memory]") { // Set up some data size_t dataSize = 10 * HOST_PAGE_SIZE; diff --git a/tests/test/util/test_snapshot.cpp b/tests/test/util/test_snapshot.cpp index 837d0606b..3d507c3d8 100644 --- a/tests/test/util/test_snapshot.cpp +++ b/tests/test/util/test_snapshot.cpp @@ -5,6 +5,7 @@ #include #include +#include #include #include #include @@ -13,7 +14,9 @@ using namespace faabric::util; namespace tests { -class SnapshotMergeTestFixture : public SnapshotTestFixture +class SnapshotMergeTestFixture + : public ConfTestFixture + , public SnapshotTestFixture { public: SnapshotMergeTestFixture() = default; @@ -23,8 +26,7 @@ class SnapshotMergeTestFixture : public SnapshotTestFixture protected: std::string snapKey = "foobar123"; - std::shared_ptr setUpSnapshot(int snapPages, - int sharedMemPages) + std::shared_ptr setUpSnapshot(int snapPages, int memPages) { auto snapData = @@ -140,20 +142,20 @@ TEST_CASE_METHOD(SnapshotMergeTestFixture, std::vector actualSnapMem = snap->getDataCopy(); // Set up shared memory - int sharedMemPages = 20; - size_t sharedMemSize = sharedMemPages * HOST_PAGE_SIZE; - MemoryRegion sharedMem = allocatePrivateMemory(sharedMemSize); + int memPages = 20; + size_t memSize = memPages * HOST_PAGE_SIZE; + MemoryRegion mem = allocatePrivateMemory(memSize); // Check it's zeroed std::vector expectedInitial(snap->getSize(), 0); - std::vector actualSharedMemBefore( - sharedMem.get(), sharedMem.get() + snap->getSize()); + std::vector actualSharedMemBefore(mem.get(), + mem.get() + snap->getSize()); REQUIRE(actualSharedMemBefore == expectedInitial); // Map the snapshot and check again - snap->mapToMemory({ sharedMem.get(), snap->getSize() }); - std::vector actualSharedMemAfter( - sharedMem.get(), sharedMem.get() + snap->getSize()); + snap->mapToMemory({ mem.get(), snap->getSize() }); + std::vector actualSharedMemAfter(mem.get(), + mem.get() + snap->getSize()); REQUIRE(actualSharedMemAfter == actualSnapMem); } @@ -161,6 +163,10 @@ TEST_CASE_METHOD(SnapshotMergeTestFixture, "Test mapping editing and remapping memory", "[snapshot][util]") { + SECTION("Soft PTEs") { conf.dirtyTrackingMode = "softpte"; } + + SECTION("Segfaults") { conf.dirtyTrackingMode = "segfault"; } + int snapPages = 4; size_t snapSize = snapPages * HOST_PAGE_SIZE; auto snap = std::make_shared(snapSize); @@ -179,70 +185,25 @@ TEST_CASE_METHOD(SnapshotMergeTestFixture, std::vector expectedSnapMem = snap->getDataCopy(); // Set up two shared mem regions - MemoryRegion sharedMemA = allocatePrivateMemory(snapSize); - MemoryRegion sharedMemB = allocatePrivateMemory(snapSize); + MemoryRegion memA = allocatePrivateMemory(snapSize); + MemoryRegion memB = allocatePrivateMemory(snapSize); // Map the snapshot and both regions reflect the change - snap->mapToMemory({ sharedMemA.get(), snapSize }); - snap->mapToMemory({ sharedMemB.get(), snapSize }); - - REQUIRE(std::vector(sharedMemA.get(), sharedMemA.get() + snapSize) == - expectedSnapMem); - REQUIRE(std::vector(sharedMemB.get(), sharedMemB.get() + snapSize) == - expectedSnapMem); + std::span memViewA = { memA.get(), snapSize }; + std::span memViewB = { memB.get(), snapSize }; + snap->mapToMemory(memViewA); + snap->mapToMemory(memViewB); - // Reset dirty tracking - faabric::util::resetDirtyTracking(); + REQUIRE(std::vector(memA.get(), memA.get() + snapSize) == expectedSnapMem); + REQUIRE(std::vector(memB.get(), memB.get() + snapSize) == expectedSnapMem); // 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::memcpy(memA.get() + offsetA, dataA.data(), dataA.size()); + std::memcpy(memB.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, @@ -272,8 +233,8 @@ TEST_CASE_METHOD(SnapshotMergeTestFixture, REQUIRE(originalData.size() == originalSize); // Map to some other region of memory large enough for the extended version - MemoryRegion sharedMem = allocatePrivateMemory(expandedSize); - snap->mapToMemory({ sharedMem.get(), originalSize }); + MemoryRegion mem = allocatePrivateMemory(expandedSize); + snap->mapToMemory({ mem.get(), originalSize }); // Add some data to the extended region. Check the snapshot extends to fit std::vector dataC(300, 5); @@ -290,12 +251,12 @@ TEST_CASE_METHOD(SnapshotMergeTestFixture, REQUIRE(snap->getSize() == expectedSizeB); // Remap to shared memory - snap->mapToMemory({ sharedMem.get(), snap->getSize() }); + snap->mapToMemory({ mem.get(), snap->getSize() }); // Check mapped region matches std::vector actualData = snap->getDataCopy(); - std::vector actualSharedMem(sharedMem.get(), - sharedMem.get() + snap->getSize()); + std::vector actualSharedMem(mem.get(), + mem.get() + snap->getSize()); REQUIRE(actualSharedMem.size() == actualData.size()); REQUIRE(actualSharedMem == actualData); @@ -331,18 +292,21 @@ TEST_CASE_METHOD(SnapshotMergeTestFixture, reg.registerSnapshot(snapKey, snap); // Map the snapshot to some memory - MemoryRegion sharedMem = allocatePrivateMemory(memSize); - snap->mapToMemory({ sharedMem.get(), memSize }); + MemoryRegion mem = allocatePrivateMemory(memSize); + std::span memView({ mem.get(), memSize }); + snap->mapToMemory(memView); // Check mapping works - int* intA = (int*)(sharedMem.get() + intAOffset); - int* intB = (int*)(sharedMem.get() + intBOffset); + int* intA = (int*)(mem.get() + intAOffset); + int* intB = (int*)(mem.get() + intBOffset); REQUIRE(*intA == originalValueA); REQUIRE(*intB == originalValueB); // Reset dirty tracking to get a clean start - faabric::util::resetDirtyTracking(); + tracker.clearAll(); + tracker.startTracking(memView); + tracker.startThreadLocalTracking(memView); // Set up the merge regions, deliberately do the one at higher offsets first // to check the ordering @@ -362,12 +326,14 @@ TEST_CASE_METHOD(SnapshotMergeTestFixture, std::vector otherData(100, 5); int otherOffset = (3 * HOST_PAGE_SIZE) + 5; - std::memcpy( - sharedMem.get() + otherOffset, otherData.data(), otherData.size()); + std::memcpy(mem.get() + otherOffset, otherData.data(), otherData.size()); // Get the snapshot diffs + tracker.stopTracking(memView); + tracker.stopThreadLocalTracking(memView); + auto dirtyRegions = tracker.getBothDirtyOffsets(memView); std::vector actualDiffs = - MemoryView({ sharedMem.get(), memSize }).diffWithSnapshot(snap); + snap->diffWithDirtyRegions(memView, dirtyRegions); // Check original hasn't changed const uint8_t* rawSnapData = snap->getDataPtr(); @@ -448,13 +414,15 @@ TEST_CASE_METHOD(SnapshotMergeTestFixture, reg.registerSnapshot(snapKey, snap); // Map the snapshot to some memory - size_t sharedMemSize = snapPages * HOST_PAGE_SIZE; - MemoryRegion sharedMem = allocatePrivateMemory(sharedMemSize); - - snap->mapToMemory({ sharedMem.get(), sharedMemSize }); + size_t memSize = snapPages * HOST_PAGE_SIZE; + MemoryRegion mem = allocatePrivateMemory(memSize); + std::span memView(mem.get(), memSize); + snap->mapToMemory(memView); // Reset dirty tracking - faabric::util::resetDirtyTracking(); + tracker.clearAll(); + tracker.startTracking(memView); + tracker.startThreadLocalTracking(memView); // Set up the merge regions snap->addMergeRegion(offsetA, @@ -474,10 +442,10 @@ TEST_CASE_METHOD(SnapshotMergeTestFixture, offsetD, sizeof(int), SnapshotDataType::Int, SnapshotMergeOperation::Sum); // Set final values - *(int*)(sharedMem.get() + offsetA) = finalA; - *(int*)(sharedMem.get() + offsetB) = finalB; - *(int*)(sharedMem.get() + offsetC) = finalC; - *(int*)(sharedMem.get() + offsetD) = finalD; + *(int*)(mem.get() + offsetA) = finalA; + *(int*)(mem.get() + offsetB) = finalB; + *(int*)(mem.get() + offsetC) = finalC; + *(int*)(mem.get() + offsetD) = finalD; // Check the diffs std::vector expectedDiffs = { @@ -499,8 +467,11 @@ TEST_CASE_METHOD(SnapshotMergeTestFixture, { BYTES(&sumD), sizeof(int32_t) } }, }; + tracker.stopTracking(memView); + tracker.stopThreadLocalTracking(memView); + auto dirtyRegions = tracker.getBothDirtyOffsets(memView); std::vector actualDiffs = - MemoryView({ sharedMem.get(), sharedMemSize }).diffWithSnapshot(snap); + snap->diffWithDirtyRegions(memView, dirtyRegions); REQUIRE(actualDiffs.size() == 4); checkDiffs(actualDiffs, expectedDiffs); @@ -518,6 +489,10 @@ TEST_CASE_METHOD(SnapshotMergeTestFixture, std::shared_ptr snap = std::make_shared(snapPages * HOST_PAGE_SIZE); + size_t memSize = snapPages * HOST_PAGE_SIZE; + MemoryRegion mem = allocatePrivateMemory(memSize); + std::span memView(mem.get(), memSize); + std::vector originalData; std::vector updatedData; std::vector expectedData; @@ -838,24 +813,25 @@ TEST_CASE_METHOD(SnapshotMergeTestFixture, reg.registerSnapshot(snapKey, snap); // Map the snapshot to some memory - size_t sharedMemSize = snapPages * HOST_PAGE_SIZE; - MemoryRegion sharedMem = allocatePrivateMemory(sharedMemSize); - - snap->mapToMemory({ sharedMem.get(), sharedMemSize }); + snap->mapToMemory(memView); // Reset dirty tracking - faabric::util::resetDirtyTracking(); + tracker.clearAll(); + tracker.startTracking(memView); + tracker.startThreadLocalTracking(memView); // Set up the merge region snap->addMergeRegion(offset, regionLength, dataType, operation); // Modify the value - std::memcpy( - sharedMem.get() + offset, updatedData.data(), updatedData.size()); + std::memcpy(mem.get() + offset, updatedData.data(), updatedData.size()); // Get the snapshot diffs + tracker.stopTracking(memView); + tracker.stopThreadLocalTracking(memView); + auto dirtyRegions = tracker.getBothDirtyOffsets(memView); std::vector actualDiffs = - MemoryView({ sharedMem.get(), sharedMemSize }).diffWithSnapshot(snap); + snap->diffWithDirtyRegions(memView, dirtyRegions); if (expectNoDiff) { REQUIRE(actualDiffs.empty()); @@ -912,24 +888,30 @@ TEST_CASE_METHOD(SnapshotMergeTestFixture, reg.registerSnapshot(snapKey, snap); // Map the snapshot - size_t sharedMemSize = snapPages * HOST_PAGE_SIZE; - MemoryRegion sharedMem = allocatePrivateMemory(sharedMemSize); - snap->mapToMemory({ sharedMem.get(), sharedMemSize }); + size_t memSize = snapPages * HOST_PAGE_SIZE; + MemoryRegion mem = allocatePrivateMemory(memSize); + std::span memView(mem.get(), memSize); + snap->mapToMemory(memView); // Reset dirty tracking - faabric::util::resetDirtyTracking(); + tracker.clearAll(); + tracker.startTracking(memView); + tracker.startThreadLocalTracking(memView); // Set up the merge region snap->addMergeRegion(offset, dataLength, dataType, operation); // Modify the value std::vector bytes(dataLength, 3); - std::memcpy(sharedMem.get() + offset, bytes.data(), bytes.size()); + std::memcpy(mem.get() + offset, bytes.data(), bytes.size()); // Check getting diffs throws an exception + tracker.stopTracking(memView); + tracker.stopThreadLocalTracking(memView); + auto dirtyRegions = tracker.getBothDirtyOffsets(memView); bool failed = false; try { - MemoryView({ sharedMem.get(), sharedMemSize }).diffWithSnapshot(snap); + snap->diffWithDirtyRegions(memView, dirtyRegions); } catch (std::runtime_error& ex) { failed = true; REQUIRE(ex.what() == expectedMsg); @@ -958,7 +940,7 @@ TEST_CASE_METHOD(SnapshotMergeTestFixture, faabric::util::isPageAligned((const void*)snap->getDataPtr()); // Reset dirty tracking - faabric::util::resetDirtyTracking(); + tracker.clearAll(); // Update the snapshot std::vector dataA = { 0, 1, 2, 3 }; @@ -970,25 +952,18 @@ TEST_CASE_METHOD(SnapshotMergeTestFixture, snap->copyInData(dataB, offsetB); // Check we get the expected diffs - std::vector expectedDiffs = - MemoryView({ snap->getDataPtr(), snap->getSize() }).getDirtyRegions(); + std::vector expectedDiffs = snap->getTrackedChanges(); 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(diffA.getData().size() == dataA.size()); + REQUIRE(diffB.getData().size() == dataB.size()); - REQUIRE(actualA == dataA); - REQUIRE(actualB == dataB); + REQUIRE(diffA.getDataCopy() == dataA); + REQUIRE(diffB.getDataCopy() == dataB); } TEST_CASE_METHOD(SnapshotMergeTestFixture, @@ -996,35 +971,38 @@ TEST_CASE_METHOD(SnapshotMergeTestFixture, "[snapshot][util]") { int snapPages = 3; - size_t sharedMemSize = snapPages * HOST_PAGE_SIZE; + size_t memSize = snapPages * HOST_PAGE_SIZE; std::shared_ptr snap = std::make_shared(snapPages * HOST_PAGE_SIZE); reg.registerSnapshot(snapKey, snap); // Map the snapshot - MemoryRegion sharedMem = allocatePrivateMemory(sharedMemSize); - snap->mapToMemory({ sharedMem.get(), sharedMemSize }); + MemoryRegion mem = allocatePrivateMemory(memSize); + std::span memView(mem.get(), memSize); + snap->mapToMemory(memView); // Reset dirty tracking - faabric::util::resetDirtyTracking(); + tracker.clearAll(); + tracker.startTracking(memView); + tracker.startThreadLocalTracking(memView); // Add some tightly-packed changes uint32_t offsetA = 0; std::vector dataA(10, 1); - std::memcpy(sharedMem.get() + offsetA, dataA.data(), dataA.size()); + std::memcpy(mem.get() + offsetA, dataA.data(), dataA.size()); uint32_t offsetB = dataA.size() + 1; std::vector dataB(2, 1); - std::memcpy(sharedMem.get() + offsetB, dataB.data(), dataB.size()); + std::memcpy(mem.get() + offsetB, dataB.data(), dataB.size()); uint32_t offsetC = offsetB + 3; std::vector dataC(1, 1); - std::memcpy(sharedMem.get() + offsetC, dataC.data(), dataC.size()); + std::memcpy(mem.get() + offsetC, dataC.data(), dataC.size()); uint32_t offsetD = offsetC + 2; std::vector dataD(1, 1); - std::memcpy(sharedMem.get() + offsetD, dataD.data(), dataD.size()); + std::memcpy(mem.get() + offsetD, dataD.data(), dataD.size()); std::vector expectedDiffs = { { SnapshotDataType::Raw, @@ -1052,8 +1030,11 @@ TEST_CASE_METHOD(SnapshotMergeTestFixture, SnapshotMergeOperation::Overwrite); // Check number of diffs + tracker.stopTracking(memView); + tracker.stopThreadLocalTracking(memView); + auto dirtyRegions = tracker.getBothDirtyOffsets(memView); std::vector actualDiffs = - MemoryView({ sharedMem.get(), sharedMemSize }).diffWithSnapshot(snap); + snap->diffWithDirtyRegions(memView, dirtyRegions); checkDiffs(actualDiffs, expectedDiffs); } @@ -1187,18 +1168,21 @@ TEST_CASE_METHOD(SnapshotMergeTestFixture, "[snapshot][util]") { int snapPages = 6; - size_t sharedMemSize = snapPages * HOST_PAGE_SIZE; + size_t memSize = snapPages * HOST_PAGE_SIZE; std::shared_ptr snap = std::make_shared(snapPages * HOST_PAGE_SIZE); reg.registerSnapshot(snapKey, snap); // Map the snapshot - MemoryRegion sharedMem = allocatePrivateMemory(sharedMemSize); - snap->mapToMemory({ sharedMem.get(), sharedMemSize }); + MemoryRegion mem = allocatePrivateMemory(memSize); + std::span memView(mem.get(), memSize); + snap->mapToMemory(memView); // Reset dirty tracking - faabric::util::resetDirtyTracking(); + tracker.clearAll(); + tracker.startTracking(memView); + tracker.startThreadLocalTracking(memView); // Add a couple of merge regions on each page, which should be skipped as // they won't overlap any changes @@ -1225,22 +1209,24 @@ TEST_CASE_METHOD(SnapshotMergeTestFixture, 20, faabric::util::SnapshotDataType::Raw, faabric::util::SnapshotMergeOperation::Overwrite); + std::vector overwriteData(10, 1); - std::memcpy(sharedMem.get() + overwriteAOffset, - overwriteData.data(), - overwriteData.size()); + std::memcpy( + mem.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); + snap->copyInData({ BYTES(&sumOriginal), sizeof(int) }, sumOffset); - *(int*)(sharedMem.get() + sumOffset) = sumValue; + *(int*)(mem.get() + sumOffset) = sumValue; // Check diffs std::vector expectedDiffs = { @@ -1254,8 +1240,12 @@ TEST_CASE_METHOD(SnapshotMergeTestFixture, { BYTES(&sumExpected), sizeof(int32_t) } }, }; + tracker.stopTracking(memView); + tracker.stopThreadLocalTracking(memView); + auto dirtyRegions = tracker.getBothDirtyOffsets(memView); + std::vector actualDiffs = - MemoryView({ sharedMem.get(), sharedMemSize }).diffWithSnapshot(snap); + snap->diffWithDirtyRegions(memView, dirtyRegions); checkDiffs(actualDiffs, expectedDiffs); } @@ -1265,40 +1255,57 @@ TEST_CASE_METHOD(SnapshotMergeTestFixture, "[snapshot][util]") { int snapPages = 6; - int sharedMemPages = 10; + int memPages = 10; size_t snapSize = snapPages * HOST_PAGE_SIZE; - size_t sharedMemSize = sharedMemPages * HOST_PAGE_SIZE; + size_t memSize = memPages * HOST_PAGE_SIZE; std::shared_ptr snap = std::make_shared(snapSize); reg.registerSnapshot(snapKey, snap); // Map the snapshot - MemoryRegion sharedMem = allocatePrivateMemory(sharedMemSize); - snap->mapToMemory({ sharedMem.get(), snapSize }); - faabric::util::resetDirtyTracking(); + MemoryRegion mem = allocatePrivateMemory(memSize); + std::span memView(mem.get(), memSize); + + // Map only the size of the snapshot + snap->mapToMemory({ mem.get(), snapSize }); + + tracker.clearAll(); + tracker.startTracking(memView); + tracker.startThreadLocalTracking(memView); uint32_t changeStartPage = 0; uint32_t changeOffset = 0; uint32_t mergeRegionStart = snapSize; size_t changeLength = 123; - uint32_t expectedDiffStart = 0; - uint32_t expectedDiffSize = 0; - // When memory has changed at or past the end of the original data, the diff // will start at the end of the original data and round up to the next page // boundary. If the change starts before the end, it will start at the // beginning of the change and continue into the page boundary past the // original data. - SECTION("Change at end of original data, overlapping merge region") + std::vector expectedDiffs; + + std::vector zeroedPage(HOST_PAGE_SIZE, 0); + std::vector diffData(changeLength, 2); + + SECTION("Change on first page past end of original data, overlapping merge " + "region") { changeStartPage = snapSize; changeOffset = changeStartPage + 100; mergeRegionStart = snapSize; - expectedDiffStart = changeStartPage; - expectedDiffSize = HOST_PAGE_SIZE; + + diffData = std::vector(100, 2); + std::vector expectedData = zeroedPage; + std::memset(expectedData.data() + 100, 2, 100); + + // Diff should be the page after the end of the original data + expectedDiffs = { { faabric::util::SnapshotDataType::Raw, + faabric::util::SnapshotMergeOperation::Overwrite, + (uint32_t)snapSize, + expectedData } }; } SECTION("Change and merge region aligned at end of original data") @@ -1306,38 +1313,66 @@ TEST_CASE_METHOD(SnapshotMergeTestFixture, changeStartPage = snapSize; changeOffset = changeStartPage; mergeRegionStart = snapSize; - expectedDiffStart = changeStartPage; - expectedDiffSize = HOST_PAGE_SIZE; + + diffData = std::vector(100, 2); + std::vector expectedData = zeroedPage; + std::memset(expectedData.data(), 2, 100); + + // Diff should be the page after the end of the original data + expectedDiffs = { { faabric::util::SnapshotDataType::Raw, + faabric::util::SnapshotMergeOperation::Overwrite, + (uint32_t)snapSize, + expectedData } }; } - SECTION("Change after end of original data, overlapping merge region") + SECTION("Change and merge region after end of original data") { changeStartPage = (snapPages + 2) * HOST_PAGE_SIZE; changeOffset = changeStartPage + 100; mergeRegionStart = changeStartPage; - expectedDiffStart = changeStartPage; - expectedDiffSize = HOST_PAGE_SIZE; + + diffData = std::vector(100, 2); + std::vector expectedData = zeroedPage; + std::memset(expectedData.data() + 100, 2, 100); + + // Diff should be the page after the end of the original data, + // containing the change (and not those in between) + expectedDiffs = { { faabric::util::SnapshotDataType::Raw, + faabric::util::SnapshotMergeOperation::Overwrite, + changeStartPage, + expectedData } }; } - SECTION("Merge region and change crossing end of original data") + SECTION("Merge region and change both crossing end of original data") { - // Merge region starts before diff changeStartPage = (snapPages - 1) * HOST_PAGE_SIZE; changeOffset = changeStartPage + 100; mergeRegionStart = (snapPages - 2) * HOST_PAGE_SIZE; // Change goes from inside original data to overshoot the end - changeLength = 2 * HOST_PAGE_SIZE; - - // Diff will cover from the start of the change to round up to the - // nearest page in the overshoot region. - expectedDiffStart = changeOffset; - expectedDiffSize = changeLength + (HOST_PAGE_SIZE - 100); + size_t dataSize = 2 * HOST_PAGE_SIZE; + size_t overlapSize = HOST_PAGE_SIZE - 100; + size_t overshootSize = dataSize - overlapSize; + + diffData = std::vector(dataSize, 2); + + // One diff will cover the overlap with last part of original data, and + // another will be rounded up to the nearest page for the extension + std::vector expectedDataTwo(2 * HOST_PAGE_SIZE, 0); + std::memset(expectedDataTwo.data(), 2, overshootSize); + + expectedDiffs = { { faabric::util::SnapshotDataType::Raw, + faabric::util::SnapshotMergeOperation::Overwrite, + changeOffset, + std::vector(HOST_PAGE_SIZE - 100, 2) }, + { faabric::util::SnapshotDataType::Raw, + faabric::util::SnapshotMergeOperation::Overwrite, + (uint32_t)snapSize, + expectedDataTwo } }; } - std::vector diffData(changeLength, 2); - std::memcpy( - sharedMem.get() + changeOffset, diffData.data(), diffData.size()); + // Copy in the changed data + std::memcpy(mem.get() + changeOffset, diffData.data(), diffData.size()); // Add a merge region snap->addMergeRegion(mergeRegionStart, @@ -1345,16 +1380,12 @@ TEST_CASE_METHOD(SnapshotMergeTestFixture, faabric::util::SnapshotDataType::Raw, faabric::util::SnapshotMergeOperation::Overwrite); - std::vector actualDiffs = - MemoryView({ sharedMem.get(), sharedMemSize }).diffWithSnapshot(snap); + tracker.stopTracking(memView); + tracker.stopThreadLocalTracking(memView); + auto dirtyRegions = tracker.getBothDirtyOffsets(memView); - // Set up expected diff - std::vector expectedDiffs = { - { faabric::util::SnapshotDataType::Raw, - faabric::util::SnapshotMergeOperation::Overwrite, - expectedDiffStart, - { sharedMem.get() + expectedDiffStart, expectedDiffSize } }, - }; + std::vector actualDiffs = + snap->diffWithDirtyRegions(memView, dirtyRegions); checkDiffs(actualDiffs, expectedDiffs); } @@ -1415,7 +1446,9 @@ TEST_CASE("Test snapshot data constructors", "[snapshot][util]") REQUIRE(actualConst == data); } -TEST_CASE("Test snapshot mapped memory diffs", "[snapshot][util]") +TEST_CASE_METHOD(DirtyTrackingTestFixture, + "Test snapshot mapped memory diffs", + "[snapshot][util]") { int nSnapPages = 5; size_t snapSize = nSnapPages * HOST_PAGE_SIZE; @@ -1446,28 +1479,40 @@ TEST_CASE("Test snapshot mapped memory diffs", "[snapshot][util]") SnapshotDataType::Raw, SnapshotMergeOperation::Overwrite); - // Write data to snapshot + // Write some initial data to snapshot snap->copyInData(dataA); // Map some memory MemoryRegion memA = allocatePrivateMemory(snapSize); - snap->mapToMemory({ memA.get(), snapSize }); + std::span memViewA(memA.get(), snapSize); + snap->mapToMemory(memViewA); - faabric::util::resetDirtyTracking(); + // Clear tracking + tracker.clearAll(); + snap->clearTrackedChanges(); std::vector actualSnap = snap->getDataCopy(); - std::vector actualA(memA.get(), memA.get() + snapSize); - REQUIRE(actualSnap == actualA); + std::vector actualMemA(memA.get(), memA.get() + snapSize); + REQUIRE(actualSnap == actualMemA); - // Write data to snapshot + // Start dirty tracking + tracker.startTracking(memViewA); + tracker.startThreadLocalTracking(memViewA); + + // Write some data to the snapshot snap->copyInData(dataB, offsetB); - // Write data to memory + // Write data to the mapped memory std::memcpy(memA.get() + offsetC, dataC.data(), dataC.size()); - // Check diffs from memory vs snapshot + // Check diff of snapshot with memory only includes the change made to the + // memory itself + tracker.stopTracking(memViewA); + tracker.stopThreadLocalTracking(memViewA); + auto dirtyRegions = tracker.getBothDirtyOffsets(memViewA); + std::vector actualDiffs = - MemoryView({ memA.get(), snapSize }).diffWithSnapshot(snap); + snap->diffWithDirtyRegions(memViewA, dirtyRegions); REQUIRE(actualDiffs.size() == 1); SnapshotDiff& actualDiff = actualDiffs.at(0); @@ -1478,27 +1523,23 @@ TEST_CASE("Test snapshot mapped memory diffs", "[snapshot][util]") snap->queueDiffs(actualDiffs); snap->writeQueuedDiffs(); - // Check snapshot now shows modified page - std::vector snapDirtyRegions = - MemoryView({ snap->getDataPtr(), snap->getSize() }).getDirtyRegions(); + // Check snapshot now shows both diffs + std::vector snapDirtyRegions = snap->getTrackedChanges(); - REQUIRE(snapDirtyRegions.size() == 1); - SnapshotDiff& snapDirtyRegion = snapDirtyRegions.at(0); - REQUIRE(snapDirtyRegion.getOffset() == HOST_PAGE_SIZE); + REQUIRE(snapDirtyRegions.size() == 2); + + SnapshotDiff& diffB = snapDirtyRegions.at(0); + SnapshotDiff& diffC = snapDirtyRegions.at(1); + + REQUIRE(diffB.getOffset() == offsetB); + REQUIRE(diffC.getOffset() == offsetC); // 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); + std::vector diffDataB = diffB.getDataCopy(); + std::vector diffDataC = diffC.getDataCopy(); + + REQUIRE(diffDataB == dataB); + REQUIRE(diffDataC == dataC); // Map more memory from the snapshot, check it contains all updates MemoryRegion memB = allocatePrivateMemory(snapSize); diff --git a/tests/utils/fixtures.h b/tests/utils/fixtures.h index 86d64916a..b72ea04d0 100644 --- a/tests/utils/fixtures.h +++ b/tests/utils/fixtures.h @@ -13,6 +13,7 @@ #include #include #include +#include #include #include #include @@ -61,14 +62,25 @@ class StateTestFixture } }; -class SchedulerTestFixture +class DirtyTrackingTestFixture +{ + public: + DirtyTrackingTestFixture() + : tracker(faabric::util::getDirtyTracker()) + {} + + ~DirtyTrackingTestFixture() { tracker.clearAll(); } + + protected: + faabric::util::DirtyTracker& tracker; +}; + +class SchedulerTestFixture : public DirtyTrackingTestFixture { public: SchedulerTestFixture() : sch(faabric::scheduler::getScheduler()) { - faabric::util::resetDirtyTracking(); - faabric::util::setMockMode(false); faabric::util::setTestMode(true); @@ -89,29 +101,22 @@ class SchedulerTestFixture sch.shutdown(); sch.addHostToGlobalSet(); - - faabric::util::resetDirtyTracking(); }; protected: faabric::scheduler::Scheduler& sch; }; -class SnapshotTestFixture +class SnapshotTestFixture : public DirtyTrackingTestFixture { public: SnapshotTestFixture() : reg(faabric::snapshot::getSnapshotRegistry()) { - faabric::util::resetDirtyTracking(); reg.clear(); } - ~SnapshotTestFixture() - { - faabric::util::resetDirtyTracking(); - reg.clear(); - } + ~SnapshotTestFixture() { reg.clear(); } std::shared_ptr setUpSnapshot( const std::string& snapKey, @@ -293,19 +298,23 @@ class PointToPointClientServerFixture faabric::transport::PointToPointServer server; }; +#define TEST_EXECUTOR_DEFAULT_MEMORY_SIZE (10 * faabric::util::HOST_PAGE_SIZE) + class TestExecutor final : public faabric::scheduler::Executor { public: TestExecutor(faabric::Message& msg); faabric::util::MemoryRegion dummyMemory = nullptr; - size_t dummyMemorySize = 0; + size_t dummyMemorySize = TEST_EXECUTOR_DEFAULT_MEMORY_SIZE; void reset(faabric::Message& msg) override; - void restore(faabric::Message& msg) override; + void restore(const std::string& snapshotKey) override; + + std::span getMemoryView() override; - faabric::util::MemoryView getMemoryView() override; + void setUpDummyMemory(size_t memSize); int32_t executeTask( int threadPoolIdx,