diff --git a/.env b/.env index 6ec017429..e98e00321 100644 --- a/.env +++ b/.env @@ -1,4 +1,4 @@ -FAABRIC_VERSION=0.6.1 -FAABRIC_CLI_IMAGE=faasm.azurecr.io/faabric:0.6.1 +FAABRIC_VERSION=0.7.0 +FAABRIC_CLI_IMAGE=faasm.azurecr.io/faabric:0.7.0 COMPOSE_PROJECT_NAME=faabric-dev CONAN_CACHE_MOUNT_SOURCE=./conan-cache/ diff --git a/.github/workflows/tests.yml b/.github/workflows/tests.yml index edbd1e105..bfc5c9902 100644 --- a/.github/workflows/tests.yml +++ b/.github/workflows/tests.yml @@ -23,7 +23,7 @@ jobs: if: github.event.pull_request.draft == false runs-on: ubuntu-latest container: - image: faasm.azurecr.io/faabric:0.6.1 + image: faasm.azurecr.io/faabric:0.7.0 credentials: username: ${{ secrets.ACR_SERVICE_PRINCIPAL_ID }} password: ${{ secrets.ACR_SERVICE_PRINCIPAL_PASSWORD }} @@ -36,7 +36,7 @@ jobs: if: github.event.pull_request.draft == false runs-on: ubuntu-latest container: - image: faasm.azurecr.io/faabric:0.6.1 + image: faasm.azurecr.io/faabric:0.7.0 credentials: username: ${{ secrets.ACR_SERVICE_PRINCIPAL_ID }} password: ${{ secrets.ACR_SERVICE_PRINCIPAL_PASSWORD }} @@ -50,7 +50,7 @@ jobs: if: github.event.pull_request.draft == false runs-on: ubuntu-latest container: - image: faasm.azurecr.io/faabric:0.6.1 + image: faasm.azurecr.io/faabric:0.7.0 credentials: username: ${{ secrets.ACR_SERVICE_PRINCIPAL_ID }} password: ${{ secrets.ACR_SERVICE_PRINCIPAL_PASSWORD }} @@ -73,7 +73,7 @@ jobs: REDIS_QUEUE_HOST: redis REDIS_STATE_HOST: redis container: - image: faasm.azurecr.io/faabric:0.6.1 + image: faasm.azurecr.io/faabric:0.7.0 credentials: username: ${{ secrets.ACR_SERVICE_PRINCIPAL_ID }} password: ${{ secrets.ACR_SERVICE_PRINCIPAL_PASSWORD }} @@ -113,7 +113,7 @@ jobs: REDIS_QUEUE_HOST: redis REDIS_STATE_HOST: redis container: - image: faasm.azurecr.io/faabric:0.6.1 + image: faasm.azurecr.io/faabric:0.7.0 credentials: username: ${{ secrets.ACR_SERVICE_PRINCIPAL_ID }} password: ${{ secrets.ACR_SERVICE_PRINCIPAL_PASSWORD }} @@ -167,7 +167,7 @@ jobs: REDIS_QUEUE_HOST: redis REDIS_STATE_HOST: redis container: - image: faasm.azurecr.io/faabric:0.6.1 + image: faasm.azurecr.io/faabric:0.7.0 credentials: username: ${{ secrets.ACR_SERVICE_PRINCIPAL_ID }} password: ${{ secrets.ACR_SERVICE_PRINCIPAL_PASSWORD }} diff --git a/VERSION b/VERSION index ee6cdce3c..faef31a43 100644 --- a/VERSION +++ b/VERSION @@ -1 +1 @@ -0.6.1 +0.7.0 diff --git a/include/faabric/batch-scheduler/BatchScheduler.h b/include/faabric/batch-scheduler/BatchScheduler.h index 576178b8a..2db72a649 100644 --- a/include/faabric/batch-scheduler/BatchScheduler.h +++ b/include/faabric/batch-scheduler/BatchScheduler.h @@ -7,10 +7,11 @@ #define DO_NOT_MIGRATE -98 #define DO_NOT_MIGRATE_DECISION \ - SchedulingDecision(DO_NOT_MIGRATE, DO_NOT_MIGRATE) + faabric::batch_scheduler::SchedulingDecision(DO_NOT_MIGRATE, DO_NOT_MIGRATE) #define NOT_ENOUGH_SLOTS -99 #define NOT_ENOUGH_SLOTS_DECISION \ - SchedulingDecision(NOT_ENOUGH_SLOTS, NOT_ENOUGH_SLOTS) + faabric::batch_scheduler::SchedulingDecision(NOT_ENOUGH_SLOTS, \ + NOT_ENOUGH_SLOTS) namespace faabric::batch_scheduler { @@ -70,7 +71,7 @@ class BatchScheduler std::shared_ptr req); virtual std::shared_ptr makeSchedulingDecision( - const HostMap& hostMap, + HostMap& hostMap, const InFlightReqs& inFlightReqs, std::shared_ptr req) = 0; @@ -111,7 +112,7 @@ class BatchScheduler std::shared_ptr decisionB) = 0; virtual std::vector getSortedHosts( - const HostMap& hostMap, + HostMap& hostMap, const InFlightReqs& inFlightReqs, std::shared_ptr req, const DecisionType& decisionType) = 0; diff --git a/include/faabric/batch-scheduler/BinPackScheduler.h b/include/faabric/batch-scheduler/BinPackScheduler.h index ad7692c36..f9a8010cb 100644 --- a/include/faabric/batch-scheduler/BinPackScheduler.h +++ b/include/faabric/batch-scheduler/BinPackScheduler.h @@ -11,7 +11,7 @@ class BinPackScheduler final : public BatchScheduler { public: std::shared_ptr makeSchedulingDecision( - const HostMap& hostMap, + HostMap& hostMap, const InFlightReqs& inFlightReqs, std::shared_ptr req) override; @@ -21,7 +21,7 @@ class BinPackScheduler final : public BatchScheduler std::shared_ptr decisionB) override; std::vector getSortedHosts( - const HostMap& hostMap, + HostMap& hostMap, const InFlightReqs& inFlightReqs, std::shared_ptr req, const DecisionType& decisionType) override; diff --git a/include/faabric/batch-scheduler/SchedulingDecision.h b/include/faabric/batch-scheduler/SchedulingDecision.h index 5a1da6745..7bbaa3099 100644 --- a/include/faabric/batch-scheduler/SchedulingDecision.h +++ b/include/faabric/batch-scheduler/SchedulingDecision.h @@ -101,7 +101,7 @@ class SchedulingDecision std::set uniqueHosts(); - void print(); + void print(const std::string& logLevel = "debug"); }; } diff --git a/include/faabric/mpi/MpiWorld.h b/include/faabric/mpi/MpiWorld.h index 8ee9f967d..51f3963fe 100644 --- a/include/faabric/mpi/MpiWorld.h +++ b/include/faabric/mpi/MpiWorld.h @@ -201,9 +201,7 @@ class MpiWorld /* Function Migration */ - void prepareMigration( - int thisRank, - std::shared_ptr pendingMigrations); + void prepareMigration(int thisRank); private: int id = -1; @@ -267,8 +265,5 @@ class MpiWorld int count, MPI_Status* status, MPIMessage::MPIMessageType messageType = MPIMessage::NORMAL); - - /* Function migration */ - bool hasBeenMigrated = false; }; } diff --git a/include/faabric/planner/Planner.h b/include/faabric/planner/Planner.h index c96667746..79e9c5288 100644 --- a/include/faabric/planner/Planner.h +++ b/include/faabric/planner/Planner.h @@ -1,8 +1,10 @@ #pragma once +#include #include #include #include +#include #include @@ -31,7 +33,7 @@ class Planner void printConfig() const; // ---------- - // Util + // Util public API // ---------- bool reset(); @@ -64,6 +66,12 @@ class Planner std::shared_ptr getBatchResults( int32_t appId); + std::shared_ptr + getSchedulingDecision(std::shared_ptr req); + + std::shared_ptr callBatch( + std::shared_ptr req); + private: // There's a singleton instance of the planner running, but it must allow // concurrent requests @@ -72,12 +80,31 @@ class Planner PlannerState state; PlannerConfig config; + // Snapshot registry to distribute snapshots in THREADS requests + faabric::snapshot::SnapshotRegistry& snapshotRegistry; + + // ---------- + // Util private API + // ---------- + void flushHosts(); void flushExecutors(); + // ---------- + // Host membership private API + // ---------- + // Check if a host's registration timestamp has expired bool isHostExpired(std::shared_ptr host, long epochTimeMs = 0); + + // ---------- + // Request scheduling private API + // ---------- + + void dispatchSchedulingDecision( + std::shared_ptr req, + std::shared_ptr decision); }; Planner& getPlanner(); diff --git a/include/faabric/planner/PlannerApi.h b/include/faabric/planner/PlannerApi.h index cbd16f179..94b88e7a7 100644 --- a/include/faabric/planner/PlannerApi.h +++ b/include/faabric/planner/PlannerApi.h @@ -13,5 +13,7 @@ enum PlannerCalls // Scheduling calls SetMessageResult = 8, GetMessageResult = 9, + GetSchedulingDecision = 10, + CallBatch = 11, }; } diff --git a/include/faabric/planner/PlannerClient.h b/include/faabric/planner/PlannerClient.h index 0c04839aa..1aa59731a 100644 --- a/include/faabric/planner/PlannerClient.h +++ b/include/faabric/planner/PlannerClient.h @@ -1,6 +1,8 @@ #pragma once +#include #include +#include #include #include @@ -31,12 +33,14 @@ class KeepAliveThread : public faabric::util::PeriodicBackgroundThread }; /* - * Local state associated with the current host, used to cache results and - * avoid unnecessary interactions with the planner server. + * Local state associated with the current host, used to store useful state + * like cached results to unnecessary interactions with the planner server. */ struct PlannerCache { std::unordered_map plannerResults; + // Keeps track of the snapshots that have been pushed to the planner + std::set pushedSnapshots; }; /* @@ -79,10 +83,19 @@ class PlannerClient final : public faabric::transport::MessageEndpointClient faabric::Message getMessageResult(const faabric::Message& msg, int timeoutMs); + faabric::batch_scheduler::SchedulingDecision callFunctions( + std::shared_ptr req); + + faabric::batch_scheduler::SchedulingDecision getSchedulingDecision( + std::shared_ptr req); + private: std::mutex plannerCacheMx; PlannerCache cache; + // Snapshot client for the planner snapshot server + std::shared_ptr snapshotClient; + faabric::Message doGetMessageResult( std::shared_ptr msgPtr, int timeoutMs); diff --git a/include/faabric/planner/PlannerServer.h b/include/faabric/planner/PlannerServer.h index ed9df956c..6ce974aa7 100644 --- a/include/faabric/planner/PlannerServer.h +++ b/include/faabric/planner/PlannerServer.h @@ -34,6 +34,12 @@ class PlannerServer final : public faabric::transport::MessageEndpointServer std::unique_ptr recvGetMessageResult( std::span buffer); + std::unique_ptr recvGetSchedulingDecision( + std::span buffer); + + std::unique_ptr recvCallBatch( + std::span buffer); + private: faabric::planner::Planner& planner; }; diff --git a/include/faabric/planner/PlannerState.h b/include/faabric/planner/PlannerState.h index 37ef884a1..744a43d12 100644 --- a/include/faabric/planner/PlannerState.h +++ b/include/faabric/planner/PlannerState.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include @@ -23,5 +24,8 @@ struct PlannerState // Map holding the hosts that have registered interest in getting an app // result std::map> appResultWaiters; + + // Map keeping track of the requests that are in-flight + faabric::batch_scheduler::InFlightReqs inFlightReqs; }; } diff --git a/include/faabric/scheduler/FunctionCallApi.h b/include/faabric/scheduler/FunctionCallApi.h index 5879e8331..3e8ab45db 100644 --- a/include/faabric/scheduler/FunctionCallApi.h +++ b/include/faabric/scheduler/FunctionCallApi.h @@ -6,9 +6,6 @@ enum FunctionCalls NoFunctionCall = 0, ExecuteFunctions = 1, Flush = 2, - Unregister = 3, - GetResources = 4, - PendingMigrations = 5, - SetMessageResult = 6, + SetMessageResult = 3, }; } diff --git a/include/faabric/scheduler/FunctionCallClient.h b/include/faabric/scheduler/FunctionCallClient.h index 02fbfd14b..889c0cb51 100644 --- a/include/faabric/scheduler/FunctionCallClient.h +++ b/include/faabric/scheduler/FunctionCallClient.h @@ -20,21 +20,9 @@ std::vector< std::pair>> getBatchRequests(); -std::vector> -getResourceRequests(); - -std::vector>> -getPendingMigrationsRequests(); - -std::vector> -getUnregisterRequests(); - std::vector>> getMessageResults(); -void queueResourceResponse(const std::string& host, - faabric::HostResources& res); - void clearMockRequests(); // ----------------------------------- @@ -52,14 +40,8 @@ class FunctionCallClient : public faabric::transport::MessageEndpointClient void sendFlush(); - faabric::HostResources getResources(); - - void sendPendingMigrations(std::shared_ptr req); - void executeFunctions(std::shared_ptr req); - void unregister(faabric::UnregisterRequest& req); - void setMessageResult(std::shared_ptr msg); }; diff --git a/include/faabric/scheduler/FunctionCallServer.h b/include/faabric/scheduler/FunctionCallServer.h index db62a330c..ed5d2e422 100644 --- a/include/faabric/scheduler/FunctionCallServer.h +++ b/include/faabric/scheduler/FunctionCallServer.h @@ -23,16 +23,8 @@ class FunctionCallServer final std::unique_ptr recvFlush( std::span buffer); - std::unique_ptr recvGetResources( - std::span buffer); - - std::unique_ptr recvPendingMigrations( - std::span buffer); - void recvExecuteFunctions(std::span buffer); - void recvUnregister(std::span buffer); - void recvSetMessageResult(std::span buffer); }; } diff --git a/include/faabric/scheduler/Scheduler.h b/include/faabric/scheduler/Scheduler.h index 5e83c3492..97fdcd744 100644 --- a/include/faabric/scheduler/Scheduler.h +++ b/include/faabric/scheduler/Scheduler.h @@ -1,34 +1,20 @@ #pragma once -#include #include #include -#include -#include -#include #include #include #include -#include #include -#include -#include -#include #include #include -#include #include -#define AVAILABLE_HOST_SET "available_hosts" -#define MIGRATED_FUNCTION_RETURN_VALUE -99 +#define DEFAULT_THREAD_RESULT_TIMEOUT_MS 1000 namespace faabric::scheduler { -typedef std::pair, - std::shared_ptr> - InFlightPair; - class Scheduler; Scheduler& getScheduler(); @@ -157,17 +143,6 @@ class Executor void threadPoolThread(std::stop_token st, int threadPoolIdx); }; -/** - * Background thread that periodically checks if there are migration - * opportunities for in-flight apps that have opted in to being checked for - * migrations. - */ -class FunctionMigrationThread : public faabric::util::PeriodicBackgroundThread -{ - public: - void doWork() override; -}; - /** * Background thread that periodically checks to see if any executors have * become stale (i.e. not handled any requests in a given timeout). If any are @@ -186,17 +161,7 @@ class Scheduler ~Scheduler(); - faabric::batch_scheduler::SchedulingDecision makeSchedulingDecision( - std::shared_ptr req, - faabric::batch_scheduler::SchedulingTopologyHint topologyHint = - faabric::batch_scheduler::SchedulingTopologyHint::NONE); - - faabric::batch_scheduler::SchedulingDecision callFunctions( - std::shared_ptr req); - - faabric::batch_scheduler::SchedulingDecision callFunctions( - std::shared_ptr req, - faabric::batch_scheduler::SchedulingDecision& hint); + void executeBatch(std::shared_ptr req); void reset(); @@ -213,60 +178,37 @@ class Scheduler long getFunctionExecutorCount(const faabric::Message& msg); - int getFunctionRegisteredHostCount(const faabric::Message& msg); - - const std::set& getFunctionRegisteredHosts( - const std::string& user, - const std::string& function, - bool acquireLock = true); - void flushLocally(); // ---------------------------------- // Message results // ---------------------------------- - // TODO(planner-scheduler): move this method to the planner client once - // the planner controls scheduling void setFunctionResult(faabric::Message& msg); - void setThreadResult(const faabric::Message& msg, + void setThreadResult(faabric::Message& msg, int32_t returnValue, const std::string& key, const std::vector& diffs); - void setThreadResultLocally(uint32_t msgId, int32_t returnValue); - /** * Caches a message along with the thread result, to allow the thread result * to refer to data held in that message (i.e. snapshot diffs). The message * will be destroyed once the thread result is consumed. */ - void setThreadResultLocally(uint32_t msgId, + void setThreadResultLocally(uint32_t appId, + uint32_t msgId, int32_t returnValue, faabric::transport::Message& message); std::vector> awaitThreadResults( - std::shared_ptr req); - - int32_t awaitThreadResult(uint32_t messageId); - - void registerThread(uint32_t msgId); - - void deregisterThreads(std::shared_ptr req); - - void deregisterThread(uint32_t msgId); - - std::vector getRegisteredThreads(); + std::shared_ptr req, + int timeoutMs = DEFAULT_THREAD_RESULT_TIMEOUT_MS); size_t getCachedMessageCount(); - void vacateSlot(); - std::string getThisHost(); - std::set getAvailableHosts(); - void addHostToGlobalSet(); void addHostToGlobalSet( @@ -275,41 +217,21 @@ class Scheduler void removeHostFromGlobalSet(const std::string& host); - void removeRegisteredHost(const std::string& host, - const std::string& user, - const std::string& function); - - void addRegisteredHost(const std::string& host, - const std::string& user, - const std::string& function); - - faabric::HostResources getThisHostResources(); - void setThisHostResources(faabric::HostResources& res); // ---------------------------------- // Testing // ---------------------------------- - std::vector getRecordedMessagesAll(); - - std::vector getRecordedMessagesLocal(); - - std::vector> - getRecordedMessagesShared(); + std::vector getRecordedMessages(); void clearRecordedMessages(); // ---------------------------------- // Function Migration // ---------------------------------- - void checkForMigrationOpportunities(); - - std::shared_ptr getPendingAppMigrations( - uint32_t appId); - - void addPendingMigration(std::shared_ptr msg); - - void removePendingMigration(uint32_t appId); + std::shared_ptr checkForMigrationOpportunities( + faabric::Message& msg, + int overwriteNewGroupId = 0); private: std::string thisHost; @@ -327,74 +249,24 @@ class Scheduler // ---- Threads ---- faabric::snapshot::SnapshotRegistry& reg; - std::unordered_map> threadResults; std::unordered_map threadResultMessages; - std::unordered_map> pushedSnapshotsMap; - - // ---- Host resources and hosts ---- - faabric::HostResources thisHostResources; - std::atomic thisHostUsedSlots = 0; - - void updateHostResources(); - - faabric::HostResources getHostResources(const std::string& host); - // ---- Planner---- faabric::planner::KeepAliveThread keepAliveThread; // ---- Actual scheduling ---- SchedulerReaperThread reaperThread; - std::set availableHostsCache; - - std::unordered_map> registeredHosts; - - faabric::batch_scheduler::SchedulingDecision doSchedulingDecision( - std::shared_ptr req, - faabric::batch_scheduler::SchedulingTopologyHint topologyHint); - - faabric::batch_scheduler::SchedulingDecision doCallFunctions( - std::shared_ptr req, - faabric::batch_scheduler::SchedulingDecision& decision, - faabric::util::FullLock& lock, - faabric::batch_scheduler::SchedulingTopologyHint topologyHint); - std::shared_ptr claimExecutor( faabric::Message& msg, faabric::util::FullLock& schedulerLock); - std::vector getUnregisteredHosts(const std::string& user, - const std::string& function, - bool noCache = false); - // ---- Accounting and debugging ---- - std::vector recordedMessagesAll; - std::vector recordedMessagesLocal; - std::vector> - recordedMessagesShared; + std::vector recordedMessages; // ---- Point-to-point ---- faabric::transport::PointToPointBroker& broker; - - // ---- Function migration ---- - FunctionMigrationThread functionMigrationThread; - std::unordered_map inFlightRequests; - std::unordered_map> - pendingMigrations; - - std::vector> - doCheckForMigrationOpportunities( - faabric::batch_scheduler::MigrationStrategy migrationStrategy = - faabric::batch_scheduler::MigrationStrategy::BIN_PACK); - - void broadcastPendingMigrations( - std::shared_ptr pendingMigrations); - - void doStartFunctionMigrationThread( - std::shared_ptr req, - faabric::batch_scheduler::SchedulingDecision& decision); }; } diff --git a/include/faabric/snapshot/SnapshotClient.h b/include/faabric/snapshot/SnapshotClient.h index c67934650..d8bfc2e86 100644 --- a/include/faabric/snapshot/SnapshotClient.h +++ b/include/faabric/snapshot/SnapshotClient.h @@ -50,9 +50,11 @@ class SnapshotClient final : public faabric::transport::MessageEndpointClient const std::shared_ptr& data, const std::vector& diffs); - void deleteSnapshot(const std::string& key); + // TODO(thread-opt): + // void deleteSnapshot(const std::string& key); void pushThreadResult( + uint32_t appId, uint32_t messageId, int returnValue, const std::string& key, diff --git a/include/faabric/transport/PointToPointBroker.h b/include/faabric/transport/PointToPointBroker.h index f41dae1be..355fd9c31 100644 --- a/include/faabric/transport/PointToPointBroker.h +++ b/include/faabric/transport/PointToPointBroker.h @@ -98,6 +98,10 @@ class PointToPointBroker void setAndSendMappingsFromSchedulingDecision( const faabric::batch_scheduler::SchedulingDecision& decision); + void sendMappingsFromSchedulingDecision( + const faabric::batch_scheduler::SchedulingDecision& decision, + const std::set& hostList); + void waitForMappingsOnThisHost(int groupId); std::set getIdxsRegisteredForGroup(int groupId); @@ -132,6 +136,8 @@ class PointToPointBroker void resetThreadLocalCache(); + void postMigrationHook(int groupId, int groupIdx); + private: faabric::util::SystemConfig& conf; diff --git a/include/faabric/util/func.h b/include/faabric/util/func.h index 7408d8b18..e8ec5b93e 100644 --- a/include/faabric/util/func.h +++ b/include/faabric/util/func.h @@ -5,6 +5,8 @@ #include #include +#define MIGRATED_FUNCTION_RETURN_VALUE -99 + namespace faabric::util { class FunctionMigratedException : public faabric::util::FaabricException diff --git a/src/batch-scheduler/BinPackScheduler.cpp b/src/batch-scheduler/BinPackScheduler.cpp index 065ce7d57..e96e1f029 100644 --- a/src/batch-scheduler/BinPackScheduler.cpp +++ b/src/batch-scheduler/BinPackScheduler.cpp @@ -137,7 +137,7 @@ bool BinPackScheduler::isFirstDecisionBetter( } std::vector BinPackScheduler::getSortedHosts( - const HostMap& hostMap, + HostMap& hostMap, const InFlightReqs& inFlightReqs, std::shared_ptr req, const DecisionType& decisionType) @@ -252,7 +252,7 @@ std::vector BinPackScheduler::getSortedHosts( // and then starts filling bins from begining to end, until it runs out of // messages to schedule std::shared_ptr BinPackScheduler::makeSchedulingDecision( - const HostMap& hostMap, + HostMap& hostMap, const InFlightReqs& inFlightReqs, std::shared_ptr req) { diff --git a/src/batch-scheduler/SchedulingDecision.cpp b/src/batch-scheduler/SchedulingDecision.cpp index 4467732e3..dafd6f8d5 100644 --- a/src/batch-scheduler/SchedulingDecision.cpp +++ b/src/batch-scheduler/SchedulingDecision.cpp @@ -59,21 +59,35 @@ std::set SchedulingDecision::uniqueHosts() return std::set(hosts.begin(), hosts.end()); } -void SchedulingDecision::print() +void SchedulingDecision::print(const std::string& logLevel) { - SPDLOG_DEBUG("-------------- Decision for App: {} ----------------", appId); - SPDLOG_DEBUG("MsgId\tAppId\tGroupId\tGrIdx\tHostIp"); + std::string printedText; + printedText += fmt::format( + "-------------- Decision for App: {} ----------------\n", appId); + printedText += "MsgId\tAppId\tGroupId\tGrIdx\tHostIp\n"; // Modulo a big number so that we can get the UUIDs to fit within one tab int formatBase = 1e6; for (int i = 0; i < hosts.size(); i++) { - SPDLOG_DEBUG("{}\t{}\t{}\t{}\t{}", - messageIds.at(i) % formatBase, - appId % formatBase, - groupId % formatBase, - groupIdxs.at(i), - hosts.at(i)); + printedText += fmt::format("{}\t{}\t{}\t{}\t{}\n", + messageIds.at(i) % formatBase, + appId % formatBase, + groupId % formatBase, + groupIdxs.at(i), + hosts.at(i)); + } + printedText += fmt::format( + "------------- End Decision for App {} ---------------", appId); + + if (logLevel == "debug") { + SPDLOG_DEBUG(printedText); + } else if (logLevel == "info") { + SPDLOG_INFO(printedText); + } else if (logLevel == "warn") { + SPDLOG_WARN(printedText); + } else if (logLevel == "error") { + SPDLOG_ERROR(printedText); + } else { + SPDLOG_ERROR("Unrecognised log level: {}", logLevel); } - SPDLOG_DEBUG("------------- End Decision for App {} ---------------", - appId); } } diff --git a/src/flat/faabric.fbs b/src/flat/faabric.fbs index 9d4ae3546..592ff7bd7 100644 --- a/src/flat/faabric.fbs +++ b/src/flat/faabric.fbs @@ -30,6 +30,7 @@ table SnapshotUpdateRequest { } table ThreadResultRequest { + app_id: int; message_id:int; return_value:int; key:string; diff --git a/src/mpi/MpiWorld.cpp b/src/mpi/MpiWorld.cpp index 7b779b2b7..148690052 100644 --- a/src/mpi/MpiWorld.cpp +++ b/src/mpi/MpiWorld.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include #include #include @@ -134,40 +134,29 @@ void MpiWorld::create(faabric::Message& call, int newId, int newSize) call.set_mpirank(0); call.set_mpiworldid(id); call.set_mpiworldsize(size); - call.set_groupid(call.mpiworldid()); call.set_groupidx(call.mpirank()); call.set_appidx(call.mpirank()); - auto& sch = faabric::scheduler::getScheduler(); - // Dispatch all the chained calls. With the main being rank zero, we want // to spawn (size - 1) new functions starting with rank 1 std::shared_ptr req = faabric::util::batchExecFactory(user, function, size - 1); + faabric::util::updateBatchExecAppId(req, call.appid()); for (int i = 0; i < req->messages_size(); i++) { + // Update MPI-related fields faabric::Message& msg = req->mutable_messages()->at(i); - msg.set_appid(call.appid()); msg.set_ismpi(true); msg.set_mpiworldid(call.mpiworldid()); msg.set_mpirank(i + 1); msg.set_mpiworldsize(call.mpiworldsize()); - // Set group ids for remote messaging - msg.set_groupid(call.groupid()); + // Set group idxs for remote messaging msg.set_groupidx(msg.mpirank()); if (thisRankMsg != nullptr) { // Set message fields to allow for function migration msg.set_appid(thisRankMsg->appid()); msg.set_cmdline(thisRankMsg->cmdline()); msg.set_inputdata(thisRankMsg->inputdata()); - msg.set_migrationcheckperiod(thisRankMsg->migrationcheckperiod()); - - // To run migration experiments easily, we may want to propagate - // the UNDERFULL topology hint. In general however, we don't - // need to propagate this field - if (thisRankMsg->topologyhint() == "UNDERFULL") { - msg.set_topologyhint(thisRankMsg->topologyhint()); - } // Log chained functions to generate execution graphs if (thisRankMsg->recordexecgraph()) { @@ -178,10 +167,11 @@ void MpiWorld::create(faabric::Message& call, int newId, int newSize) } // As a result of the call to the scheduler, a point-to-point communcation - // group will have been created with id equal to the MPI world's id. + // group will have been created. We update our recorded message group id + // to use the new PTP group if (size > 1) { - faabric::batch_scheduler::SchedulingDecision decision = - sch.callFunctions(req); + auto decision = faabric::planner::getPlannerClient().callFunctions(req); + thisRankMsg->set_groupid(decision.groupId); assert(decision.hosts.size() == size - 1); } else { // If world has size one, create the communication group (of size one) @@ -298,7 +288,12 @@ void MpiWorld::initLocalRemoteLeaders() int groupId = thisRankMsg->groupid(); auto rankIds = broker.getIdxsRegisteredForGroup(groupId); if (rankIds.size() != size) { - SPDLOG_ERROR("rankIds != size ({} != {})", rankIds.size(), size); + SPDLOG_ERROR("{}:{}:{} rankIds != size ({} != {})", + thisRankMsg->appid(), + groupId, + thisRankMsg->groupidx(), + rankIds.size(), + size); throw std::runtime_error("MPI Group-World size mismatch!"); } assert(rankIds.size() == size); @@ -1408,18 +1403,6 @@ void MpiWorld::barrier(int thisRank) send(thisRank, 0, nullptr, MPI_INT, 0, MPIMessage::BARRIER_JOIN); } - if (thisRank == localLeader && hasBeenMigrated) { - hasBeenMigrated = false; - if (thisRankMsg != nullptr) { - faabric::scheduler::getScheduler().removePendingMigration( - thisRankMsg->appid()); - } else { - SPDLOG_ERROR("App has been migrated but rank ({}) message not set", - thisRank); - throw std::runtime_error("App migrated but rank message not set"); - } - } - // Rank 0 broadcasts that the barrier is done (the others block here) broadcast(0, thisRank, nullptr, MPI_INT, 0, MPIMessage::BARRIER_DONE); SPDLOG_TRACE("MPI - barrier done {}", thisRank); @@ -1601,9 +1584,7 @@ void MpiWorld::checkRanksRange(int sendRank, int recvRank) } } -void MpiWorld::prepareMigration( - int thisRank, - std::shared_ptr pendingMigrations) +void MpiWorld::prepareMigration(int thisRank) { // Check that there are no pending asynchronous messages to send and receive for (auto umb : unackedMessageBuffers) { @@ -1630,46 +1611,7 @@ void MpiWorld::prepareMigration( // Update local records if (thisRank == localLeader) { - for (int i = 0; i < pendingMigrations->migrations_size(); i++) { - auto m = pendingMigrations->mutable_migrations()->at(i); - assert(hostForRank.at(m.msg().mpirank()) == m.srchost()); - - // Update the host for this rank. We only update the positions of - // the to-be migrated ranks, avoiding race conditions with not- - // migrated ranks - hostForRank.at(m.msg().mpirank()) = m.dsthost(); - - // Update the ranks for host. This structure is used when doing - // collective communications by all ranks. At this point, all non- - // leader ranks will be hitting a barrier, for which they don't - // need the ranks for host map, therefore it is safe to modify it - if (m.dsthost() == thisHost && m.msg().mpirank() < localLeader) { - SPDLOG_WARN("Changing local leader {} -> {}", - localLeader, - m.msg().mpirank()); - localLeader = m.msg().mpirank(); - ranksForHost[m.dsthost()].insert( - ranksForHost[m.dsthost()].begin(), m.msg().mpirank()); - } - - ranksForHost[m.dsthost()].push_back(m.msg().mpirank()); - ranksForHost[m.srchost()].erase( - std::remove(ranksForHost[m.srchost()].begin(), - ranksForHost[m.srchost()].end(), - m.msg().mpirank()), - ranksForHost[m.srchost()].end()); - - if (ranksForHost[m.srchost()].empty()) { - ranksForHost.erase(m.srchost()); - } - - // This could be made more efficient as the broker method acquires - // a full lock every time - broker.updateHostForIdx(id, m.msg().mpirank(), m.dsthost()); - } - - // Set the migration flag - hasBeenMigrated = true; + initLocalRemoteLeaders(); // Add the necessary new local messaging queues initLocalQueues(); diff --git a/src/planner/Planner.cpp b/src/planner/Planner.cpp index 78148d921..f7a551d5d 100644 --- a/src/planner/Planner.cpp +++ b/src/planner/Planner.cpp @@ -1,10 +1,15 @@ +#include +#include #include #include #include +#include +#include #include #include #include #include +#include #include #include #include @@ -12,9 +17,59 @@ #include namespace faabric::planner { + +// ---------------------- +// Utility Functions +// ---------------------- + +static void claimHostSlots(std::shared_ptr host, int slotsToClaim = 1) +{ + host->set_usedslots(host->usedslots() + slotsToClaim); + assert(host->usedslots() <= host->slots()); +} + +static void releaseHostSlots(std::shared_ptr host, int slotsToRelease = 1) +{ + host->set_usedslots(host->usedslots() - slotsToRelease); + assert(host->usedslots() >= 0); +} + +static void printHostState(std::map> hostMap, + const std::string& logLevel = "debug") +{ + std::string printedText; + std::string header = "\n-------------- Host Map --------------"; + std::string subhead = "Ip\t\tSlots"; + std::string footer = "--------------------------------------"; + + printedText += header + "\n" + subhead + "\n"; + for (const auto& [ip, hostState] : hostMap) { + printedText += fmt::format( + "{}\t{}/{}\n", ip, hostState->usedslots(), hostState->slots()); + } + printedText += footer; + + if (logLevel == "debug") { + SPDLOG_DEBUG(printedText); + } else if (logLevel == "info") { + SPDLOG_INFO(printedText); + } else if (logLevel == "warn") { + SPDLOG_WARN(printedText); + } else if (logLevel == "error") { + SPDLOG_ERROR(printedText); + } else { + SPDLOG_ERROR("Unrecognised log level: {}", logLevel); + } +} + +// ---------------------- +// Planner +// ---------------------- + // Planner is used globally as a static variable. This constructor relies on // the fact that C++ static variable's initialisation is thread-safe Planner::Planner() + : snapshotRegistry(faabric::snapshot::getSnapshotRegistry()) { // Note that we don't initialise the config in a separate method to prevent // that method from being called elsewhere in the codebase (as it would be @@ -201,11 +256,22 @@ void Planner::setMessageResult(std::shared_ptr msg) faabric::util::FullLock lock(plannerMx); - SPDLOG_INFO("Planner setting message result (id: {}) for {}:{}:{}", - msg->id(), - msg->appid(), - msg->groupid(), - msg->groupidx()); + SPDLOG_DEBUG("Planner setting message result (id: {}) for {}:{}:{}", + msg->id(), + msg->appid(), + msg->groupid(), + msg->groupidx()); + + // Release the slot only once + assert(state.hostMap.contains(msg->executedhost())); + if (!state.appResults[appId].contains(msgId)) { + + // If the message has been migrated, we have already release the slot + // so we don't have to do it here again + if (msg->returnvalue() != MIGRATED_FUNCTION_RETURN_VALUE) { + releaseHostSlots(state.hostMap.at(msg->executedhost())); + } + } // Set the result state.appResults[appId][msgId] = msg; @@ -213,7 +279,7 @@ void Planner::setMessageResult(std::shared_ptr msg) // Dispatch an async message to all hosts that are waiting if (state.appResultWaiters.find(msgId) != state.appResultWaiters.end()) { for (const auto& host : state.appResultWaiters[msgId]) { - SPDLOG_INFO("Sending result to waiting host: {}", host); + SPDLOG_DEBUG("Sending result to waiting host: {}", host); faabric::scheduler::getFunctionCallClient(host)->setMessageResult( msg); } @@ -289,6 +355,302 @@ std::shared_ptr Planner::getBatchResults( return berStatus; } +std::shared_ptr +Planner::getSchedulingDecision(std::shared_ptr req) +{ + int appId = req->appid(); + + // Acquire a read lock to get the scheduling decision for the requested app + faabric::util::SharedLock lock(plannerMx); + + if (state.inFlightReqs.find(appId) == state.inFlightReqs.end()) { + return nullptr; + } + + return state.inFlightReqs.at(appId).second; +} + +static faabric::batch_scheduler::HostMap convertToBatchSchedHostMap( + std::map> hostMapIn) +{ + faabric::batch_scheduler::HostMap hostMap; + + for (const auto& [ip, host] : hostMapIn) { + hostMap[ip] = std::make_shared( + host->ip(), host->slots(), host->usedslots()); + } + + return hostMap; +} + +std::shared_ptr +Planner::callBatch(std::shared_ptr req) +{ + int appId = req->appid(); + + // Acquire a full lock to make the scheduling decision and update the + // in-filght map if necessary + faabric::util::FullLock lock(plannerMx); + + auto batchScheduler = faabric::batch_scheduler::getBatchScheduler(); + auto decisionType = + batchScheduler->getDecisionType(state.inFlightReqs, req); + + // Make a copy of the host-map state to make sure the scheduling process + // does not modify it + auto hostMapCopy = convertToBatchSchedHostMap(state.hostMap); + + // For a DIST_CHANGE decision (i.e. migration) we want to try to imrpove + // on the old decision (we don't care the one we send), so we make sure + // we are scheduling the same messages from the old request + if (decisionType == faabric::batch_scheduler::DecisionType::DIST_CHANGE) { + SPDLOG_INFO("App {} asked for migration opportunities", appId); + auto oldReq = state.inFlightReqs.at(appId).first; + req->clear_messages(); + for (const auto& msg : oldReq->messages()) { + *req->add_messages() = msg; + } + } + + auto decision = batchScheduler->makeSchedulingDecision( + hostMapCopy, state.inFlightReqs, req); + + // Handle failures to schedule work + if (*decision == NOT_ENOUGH_SLOTS_DECISION) { + SPDLOG_ERROR( + "Not enough free slots to schedule app: {} (requested: {})", + appId, + req->messages_size()); + printHostState(state.hostMap, "error"); + return decision; + } + + if (*decision == DO_NOT_MIGRATE_DECISION) { + SPDLOG_INFO("Decided to not migrate app: {}", appId); + return decision; + } + + // A scheduling decision will create a new PTP mapping and, as a + // consequence, a new group ID + int newGroupId = faabric::util::generateGid(); + decision->groupId = newGroupId; + faabric::util::updateBatchExecGroupId(req, newGroupId); + + // Given a scheduling decision, depending on the decision type, we want to: + // 1. Update the host-map to reflect the new host occupation + // 2. Update the in-flight map to include the new request + // 3. Send the PTP mappings to all the hosts involved + auto& broker = faabric::transport::getPointToPointBroker(); + switch (decisionType) { + case faabric::batch_scheduler::DecisionType::NEW: { + // 0. Log the decision in debug mode +#ifndef NDEBUG + decision->print(); +#endif + + // 1. For a scale change request, we only need to update the hosts + // with the new messages being scheduled + for (int i = 0; i < decision->hosts.size(); i++) { + claimHostSlots(state.hostMap.at(decision->hosts.at(i))); + } + + // 2. For a new decision, we just add it to the in-flight map + state.inFlightReqs[appId] = std::make_pair(req, decision); + + // 3. We send the mappings to all the hosts involved + broker.setAndSendMappingsFromSchedulingDecision(*decision); + + break; + } + case faabric::batch_scheduler::DecisionType::SCALE_CHANGE: { + // 1. For a scale change request, we only need to update the hosts + // with the _new_ messages being scheduled + for (int i = 0; i < decision->hosts.size(); i++) { + claimHostSlots(state.hostMap.at(decision->hosts.at(i))); + } + + // 2. For a scale change request, we want to update the BER with the + // _new_ messages we are adding + auto oldReq = state.inFlightReqs.at(appId).first; + auto oldDec = state.inFlightReqs.at(appId).second; + faabric::util::updateBatchExecGroupId(oldReq, newGroupId); + oldDec->groupId = newGroupId; + + for (int i = 0; i < req->messages_size(); i++) { + *oldReq->add_messages() = req->messages(i); + oldDec->addMessage(decision->hosts.at(i), req->messages(i)); + } + + // 2.5. Log the updated decision in debug mode +#ifndef NDEBUG + oldDec->print(); +#endif + + // 3. We want to send the mappings for the _updated_ decision, + // including _all_ the messages (not just the ones that are being + // added) + broker.setAndSendMappingsFromSchedulingDecision(*oldDec); + + break; + } + case faabric::batch_scheduler::DecisionType::DIST_CHANGE: { + auto oldReq = state.inFlightReqs.at(appId).first; + auto oldDec = state.inFlightReqs.at(appId).second; + + // 0. For the time being, when migrating we always print both + // decisions (old and new) + SPDLOG_INFO("Decided to migrate app {}!", appId); + SPDLOG_INFO("Old decision:"); + oldDec->print("info"); + SPDLOG_INFO("New decision:"); + decision->print("info"); + + // We want to let all hosts involved in the migration (not only + // those in the new decision) that we are gonna migrate. For the + // evicted hosts (those present in the old decision but not in the + // new one) we need to send the mappings manually + + // Work out the evicted host set (unfortunately, couldn't come up + // with a less verbose way to do it) + std::vector evictedHostsVec; + std::vector oldDecHosts = oldDec->hosts; + std::sort(oldDecHosts.begin(), oldDecHosts.end()); + std::vector newDecHosts = decision->hosts; + std::sort(newDecHosts.begin(), newDecHosts.end()); + std::set_difference(oldDecHosts.begin(), + oldDecHosts.end(), + newDecHosts.begin(), + newDecHosts.end(), + std::back_inserter(evictedHostsVec)); + std::set evictedHosts(evictedHostsVec.begin(), + evictedHostsVec.end()); + + // 1. We only need to update the hosts where both decisions differ + assert(decision->hosts.size() == oldDec->hosts.size()); + for (int i = 0; i < decision->hosts.size(); i++) { + if (decision->hosts.at(i) == oldDec->hosts.at(i)) { + continue; + } + + releaseHostSlots(state.hostMap.at(oldDec->hosts.at(i))); + claimHostSlots(state.hostMap.at(decision->hosts.at(i))); + } + + // 2. For a DIST_CHANGE request (migration), we want to replace the + // exsiting decision with the new one + faabric::util::updateBatchExecGroupId(oldReq, newGroupId); + state.inFlightReqs.at(appId) = std::make_pair(oldReq, decision); + + // 3. We want to sent the new scheduling decision to all the hosts + // involved in the migration (even the ones that are evicted) + broker.setAndSendMappingsFromSchedulingDecision(*decision); + broker.sendMappingsFromSchedulingDecision(*decision, evictedHosts); + + break; + } + default: { + SPDLOG_ERROR("Unrecognised decision type: {} (app: {})", + decisionType, + req->appid()); + throw std::runtime_error("Unrecognised decision type"); + } + } + + // Sanity-checks before actually dispatching functions for execution + assert(req->messages_size() == decision->hosts.size()); + assert(req->appid() == decision->appId); + assert(req->groupid() == decision->groupId); + + // Lastly, asynchronously dispatch the execute requests to the + // corresponding hosts if new functions need to be spawned (not if + // migrating) + // We may not need the lock here anymore, but we are eager to make the + // whole function atomic) + if (decisionType != faabric::batch_scheduler::DecisionType::DIST_CHANGE) { + dispatchSchedulingDecision(req, decision); + } + + return decision; +} + +void Planner::dispatchSchedulingDecision( + std::shared_ptr req, + std::shared_ptr decision) +{ + std::map> + hostRequests; + + assert(req->messages_size() == decision->hosts.size()); + + // First we build all the BatchExecuteRequests for all the different hosts. + // We need to keep a map as the hosts may not be contiguous in the decision + // (i.e. we may have (hostA, hostB, hostA) + for (int i = 0; i < req->messages_size(); i++) { + auto msg = req->messages().at(i); + + // Initialise the BER if it is not there + std::string thisHost = decision->hosts.at(i); + if (hostRequests.find(thisHost) == hostRequests.end()) { + hostRequests[thisHost] = faabric::util::batchExecFactory(); + hostRequests[thisHost]->set_appid(decision->appId); + hostRequests[thisHost]->set_groupid(decision->groupId); + hostRequests[thisHost]->set_user(msg.user()); + hostRequests[thisHost]->set_function(msg.function()); + hostRequests[thisHost]->set_snapshotkey(req->snapshotkey()); + hostRequests[thisHost]->set_type(req->type()); + hostRequests[thisHost]->set_subtype(req->subtype()); + hostRequests[thisHost]->set_contextdata(req->contextdata()); + + if (decision->isSingleHost()) { + hostRequests[thisHost]->set_singlehost(true); + } + } + + *hostRequests[thisHost]->add_messages() = msg; + } + + bool isThreads = req->type() == faabric::BatchExecuteRequest::THREADS; + bool isSingleHost = req->singlehost(); + if (isSingleHost && !decision->isSingleHost()) { + SPDLOG_ERROR( + "User provided single-host hint in BER, but decision is not!"); + } + + for (const auto& [hostIp, hostReq] : hostRequests) { + SPDLOG_DEBUG("Dispatching {} messages to host {} for execution", + hostReq->messages_size(), + hostIp); + assert(faabric::util::isBatchExecRequestValid(hostReq)); + + // In a THREADS request, before sending an execution request we need to + // push the main (caller) thread snapshot to all non-main hosts + if (isThreads && !isSingleHost) { + auto snapshotKey = + faabric::util::getMainThreadSnapshotKey(hostReq->messages(0)); + try { + auto snap = snapshotRegistry.getSnapshot(snapshotKey); + + // TODO(thread-opt): push only diffs + if (hostIp != req->messages(0).mainhost()) { + faabric::snapshot::getSnapshotClient(hostIp)->pushSnapshot( + snapshotKey, snap); + } + } catch (std::runtime_error& e) { + // Catch errors, but don't let them crash the planner. Let the + // worker crash instead + SPDLOG_ERROR("Snapshot {} not regsitered in planner!", + snapshotKey); + } + } + + faabric::scheduler::getFunctionCallClient(hostIp)->executeFunctions( + hostReq); + } + + SPDLOG_DEBUG("Finished dispatching {} messages for execution", + req->messages_size()); +} + Planner& getPlanner() { static Planner planner; diff --git a/src/planner/PlannerClient.cpp b/src/planner/PlannerClient.cpp index f4b30cdcc..1c7f57590 100644 --- a/src/planner/PlannerClient.cpp +++ b/src/planner/PlannerClient.cpp @@ -1,9 +1,13 @@ #include #include #include +#include +#include #include +#include #include #include +#include #include #include #include @@ -50,6 +54,7 @@ PlannerClient::PlannerClient(const std::string& plannerIp) : faabric::transport::MessageEndpointClient(plannerIp, PLANNER_ASYNC_PORT, PLANNER_SYNC_PORT) + , snapshotClient(faabric::snapshot::getSnapshotClient(plannerIp)) {} void PlannerClient::ping() @@ -254,6 +259,115 @@ faabric::Message PlannerClient::doGetMessageResult( } } +faabric::batch_scheduler::SchedulingDecision PlannerClient::callFunctions( + std::shared_ptr req) +{ + // ------------------------ + // THREADS + // ------------------------ + + // For threads, we need to indicate that keep track of the main host (i.e. + // the host from which we invoke threads) to gather the diffs after + // execution + bool isThreads = req->type() == faabric::BatchExecuteRequest::THREADS; + if (isThreads) { + for (int i = 0; i < req->messages_size(); i++) { + req->mutable_messages(i)->set_mainhost( + faabric::util::getSystemConfig().endpointHost); + } + } + + // ------------------------ + // SNAPSHOTS + // ------------------------ + + // If we set a snapshot key (mostly in a threaded execution) we send the + // snapshot to the planner, that will manage its lifecycle and distribution + // to other hosts. Given that we don't support nested threading, if we + // have a THREADS request here it means that we are being called from the + // main thread (which holds the main snapshot) + const std::string funcStr = + faabric::util::funcToString(req->messages(0), false); + auto& reg = faabric::snapshot::getSnapshotRegistry(); + + std::string snapshotKey; + const auto firstMsg = req->messages(0); + 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"); + } + + // To optimise for single-host shared memory, we can skip sending the + // snapshot to the planner by setting the singlehost flag + if (!req->singlehost()) { + snapshotKey = faabric::util::getMainThreadSnapshotKey(firstMsg); + } + } else { + // In a single-host setting we can skip sending the snapshots to the + // planner + if (!req->singlehost()) { + snapshotKey = req->messages(0).snapshotkey(); + } + } + + if (!snapshotKey.empty()) { + faabric::util::UniqueLock lock(plannerCacheMx); + auto snap = reg.getSnapshot(snapshotKey); + + // See if we've already pushed this snapshot to the planner once, + // if so, just push the diffs that have occurred in this main thread + if (cache.pushedSnapshots.contains(snapshotKey)) { + std::vector snapshotDiffs = + snap->getTrackedChanges(); + + snapshotClient->pushSnapshotUpdate( + snapshotKey, snap, snapshotDiffs); + } else { + snapshotClient->pushSnapshot(snapshotKey, snap); + cache.pushedSnapshots.insert(snapshotKey); + } + + // Now reset the tracking on the snapshot before we start executing + snap->clearTrackedChanges(); + } + + // ------------------------ + // EXECUTION REQUEST TO PLANNER + // ------------------------ + + faabric::PointToPointMappings response; + syncSend(PlannerCalls::CallBatch, req.get(), &response); + + auto decision = + faabric::batch_scheduler::SchedulingDecision::fromPointToPointMappings( + response); + + // The planner decision sets a group id for PTP communication. Make sure we + // propagate the group id to the messages in the request. The group idx + // is set when creating the request + faabric::util::updateBatchExecGroupId(req, decision.groupId); + + return decision; +} + +faabric::batch_scheduler::SchedulingDecision +PlannerClient::getSchedulingDecision( + std::shared_ptr req) +{ + faabric::PointToPointMappings response; + syncSend(PlannerCalls::GetSchedulingDecision, req.get(), &response); + + auto decision = + faabric::batch_scheduler::SchedulingDecision::fromPointToPointMappings( + response); + + return decision; +} + // ----------------------------------- // Static setter/getters // ----------------------------------- diff --git a/src/planner/PlannerEndpointHandler.cpp b/src/planner/PlannerEndpointHandler.cpp index 964f92170..e74ec1ada 100644 --- a/src/planner/PlannerEndpointHandler.cpp +++ b/src/planner/PlannerEndpointHandler.cpp @@ -12,11 +12,6 @@ namespace faabric::planner { using header = beast::http::field; -// TODO(schedule): this atomic variable is used to temporarily select which -// host to forward an execute request to. This is because the planner still -// does not schedule resources to hosts, just acts as a proxy. -static std::atomic nextHostIdx = 0; - void PlannerEndpointHandler::onRequest( faabric::endpoint::HttpRequestContext&& ctx, faabric::util::BeastHttpRequest&& request) @@ -179,27 +174,16 @@ void PlannerEndpointHandler::onRequest( response.body() = "Bad BatchExecRequest"; return ctx.sendFunction(std::move(response)); } - ber->set_comesfromplanner(true); - // Schedule and execute the BER - // TODO: make scheduling decision here - // FIXME: for the moment, just forward randomly to one node. Note - // that choosing the node randomly may yield to uneven load - // distributions - auto availableHosts = - faabric::planner::getPlanner().getAvailableHosts(); - if (availableHosts.empty()) { - SPDLOG_ERROR("Planner doesn't have any registered hosts to" - " schedule EXECUTE_BATCH request to!"); + // Execute the BER + auto decision = getPlanner().callBatch(ber); + + // Handle cases where the scheduling failed + if (*decision == NOT_ENOUGH_SLOTS_DECISION) { response.result(beast::http::status::internal_server_error); - response.body() = std::string("No available hosts"); + response.body() = "No available hosts"; return ctx.sendFunction(std::move(response)); } - // Note that hostIdx++ is an atomic increment - int hostIdx = nextHostIdx++ % availableHosts.size(); - faabric::scheduler::getFunctionCallClient( - availableHosts.at(hostIdx)->ip()) - ->executeFunctions(ber); // Prepare the response response.result(beast::http::status::ok); diff --git a/src/planner/PlannerServer.cpp b/src/planner/PlannerServer.cpp index cfc0bbe38..fa288e9b2 100644 --- a/src/planner/PlannerServer.cpp +++ b/src/planner/PlannerServer.cpp @@ -54,6 +54,12 @@ std::unique_ptr PlannerServer::doSyncRecv( case PlannerCalls::GetMessageResult: { return recvGetMessageResult(message.udata()); } + case PlannerCalls::GetSchedulingDecision: { + return recvGetSchedulingDecision(message.udata()); + } + case PlannerCalls::CallBatch: { + return recvCallBatch(message.udata()); + } default: { // If we don't recognise the header, let the client fail, but don't // crash the planner @@ -146,4 +152,55 @@ std::unique_ptr PlannerServer::recvGetMessageResult( return std::make_unique(*resultMsg); } + +std::unique_ptr +PlannerServer::recvGetSchedulingDecision(std::span buffer) +{ + PARSE_MSG(BatchExecuteRequest, buffer.data(), buffer.size()); + auto req = std::make_shared(parsedMsg); + + auto decision = planner.getSchedulingDecision(req); + + // If the app is not registered in-flight, return an empty mapping + if (decision == nullptr) { + return std::make_unique(); + } + + // Build PointToPointMappings from scheduling decision + faabric::PointToPointMappings mappings; + mappings.set_appid(decision->appId); + mappings.set_groupid(decision->groupId); + for (int i = 0; i < decision->hosts.size(); i++) { + auto* mapping = mappings.add_mappings(); + mapping->set_host(decision->hosts.at(i)); + mapping->set_messageid(decision->messageIds.at(i)); + mapping->set_appidx(decision->appIdxs.at(i)); + mapping->set_groupidx(decision->groupIdxs.at(i)); + } + + return std::make_unique(mappings); +} + +std::unique_ptr PlannerServer::recvCallBatch( + std::span buffer) +{ + PARSE_MSG(BatchExecuteRequest, buffer.data(), buffer.size()); + auto req = std::make_shared(parsedMsg); + + auto decision = planner.callBatch(req); + + // Build PointToPointMappings from scheduling decision + faabric::PointToPointMappings mappings; + mappings.set_appid(decision->appId); + mappings.set_groupid(decision->groupId); + for (int i = 0; i < decision->hosts.size(); i++) { + auto* mapping = mappings.add_mappings(); + mapping->set_host(decision->hosts.at(i)); + mapping->set_messageid(decision->messageIds.at(i)); + mapping->set_appidx(decision->appIdxs.at(i)); + mapping->set_groupidx(decision->groupIdxs.at(i)); + } + + return std::make_unique(mappings); +} } diff --git a/src/planner/planner_server.cpp b/src/planner/planner_server.cpp index 9e0608d18..55640ebff 100644 --- a/src/planner/planner_server.cpp +++ b/src/planner/planner_server.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -19,6 +20,11 @@ int main() // The RPC server starts in the background plannerServer.start(); + // Start also a snapshot server to synchronise snapshots + SPDLOG_INFO("Starting planner snapshot server"); + faabric::snapshot::SnapshotServer snapshotServer; + snapshotServer.start(); + // The faabric endpoint starts in the foreground SPDLOG_INFO("Starting planner endpoint"); // We get the port from the global config, but the number of threads from @@ -29,5 +35,9 @@ int main() std::make_shared()); endpoint.start(faabric::endpoint::EndpointMode::SIGNAL); + SPDLOG_INFO("Planner snapshot server shutting down"); + snapshotServer.stop(); + SPDLOG_INFO("Planner server shutting down"); + plannerServer.stop(); } diff --git a/src/proto/faabric.proto b/src/proto/faabric.proto index 4a665b5cf..5bbe6fab6 100644 --- a/src/proto/faabric.proto +++ b/src/proto/faabric.proto @@ -50,12 +50,6 @@ message BatchExecuteRequest { // Flag set by the scheduler when this batch is all executing on a single // host bool singleHost = 10; - - // TODO(planner-schedule): remove me - // Temporary flag to indicate the workers that the BER comes from the - // planner (i.e. proxy-ed through planner) and so it has not been scheduled - // yet. Whenever the planner does scheduling we will be able to remove this - bool comesFromPlanner = 11; } message BatchExecuteRequestStatus { @@ -77,12 +71,6 @@ message HostResources { int32 usedSlots = 2; } -message UnregisterRequest { - string host = 1; - string user = 2; - string function = 3; -} - message FunctionStatusResponse { enum FunctionStatus { OK = 0; @@ -150,12 +138,6 @@ message Message { repeated int32 chainedMsgIds = 36; map intExecGraphDetails = 37; map execGraphDetails = 38; - - // Function migration - int32 migrationCheckPeriod = 39 [json_name = "migration_check_period"]; - - // Scheduling - string topologyHint = 40 [json_name = "topology_hint"]; } // --------------------------------------------- @@ -239,15 +221,10 @@ message PointToPointMappings { // FUNCTION MIGRATIONS // --------------------------------------------- -message PendingMigrations { +message PendingMigration { int32 appId = 1; int32 groupId = 2; - - message PendingMigration { - Message msg = 1; - string srcHost = 2; - string dstHost = 3; - } - - repeated PendingMigration migrations = 3; + int32 groupIdx = 3; + string srcHost = 4; + string dstHost = 5; } diff --git a/src/runner/FaabricMain.cpp b/src/runner/FaabricMain.cpp index 33912d9ca..0a900364c 100644 --- a/src/runner/FaabricMain.cpp +++ b/src/runner/FaabricMain.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace faabric::runner { FaabricMain::FaabricMain( diff --git a/src/scheduler/Executor.cpp b/src/scheduler/Executor.cpp index 882f4144c..ef9720269 100644 --- a/src/scheduler/Executor.cpp +++ b/src/scheduler/Executor.cpp @@ -117,19 +117,7 @@ std::vector> Executor::executeThreads( SPDLOG_DEBUG("Executor {} executing {} threads", id, req->messages_size()); std::string funcStr = faabric::util::funcToString(req); - - // Set group ID, this will get overridden in there's a cached decision - int groupId = faabric::util::generateGid(); - for (auto& m : *req->mutable_messages()) { - m.set_groupid(groupId); - m.set_groupsize(req->messages_size()); - } - - // Get the scheduling decision - faabric::batch_scheduler::SchedulingDecision decision = - sch.makeSchedulingDecision( - req, faabric::batch_scheduler::SchedulingTopologyHint::CACHED); - bool isSingleHost = decision.isSingleHost(); + bool isSingleHost = req->singlehost(); // Do snapshotting if not on a single host faabric::Message& msg = req->mutable_messages()->at(0); @@ -151,7 +139,7 @@ std::vector> Executor::executeThreads( snap->diffWithDirtyRegions(memView, dirtyRegions); if (updates.empty()) { - SPDLOG_TRACE( + SPDLOG_DEBUG( "No updates to main thread snapshot for {} over {} pages", faabric::util::funcToString(msg, false), dirtyRegions.size()); @@ -174,9 +162,9 @@ std::vector> Executor::executeThreads( } // Invoke threads and await - sch.callFunctions(req, decision); - std::vector> results = - sch.awaitThreadResults(req); + auto decision = faabric::planner::getPlannerClient().callFunctions(req); + std::vector> results = sch.awaitThreadResults( + req, faabric::util::getSystemConfig().boundTimeout); // Perform snapshot updates if not on single host if (!isSingleHost) { @@ -196,9 +184,6 @@ std::vector> Executor::executeThreads( tracker->startThreadLocalTracking(memView); } - // Deregister the threads - sch.deregisterThreads(req); - return results; } @@ -206,12 +191,11 @@ void Executor::executeTasks(std::vector msgIdxs, std::shared_ptr req) { const std::string funcStr = faabric::util::funcToString(req); - SPDLOG_TRACE("{} executing {}/{} tasks of {} (single-host={})", + SPDLOG_TRACE("{} executing {}/{} tasks of {}", id, msgIdxs.size(), req->messages_size(), - funcStr, - req->singlehost()); + 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 @@ -262,11 +246,10 @@ void Executor::executeTasks(std::vector msgIdxs, SPDLOG_DEBUG("Restoring {} from snapshot {}", funcStr, snapshotKey); restore(snapshotKey); } else { - SPDLOG_TRACE("Not restoring {}. threads={}, key={}, single={}", + SPDLOG_TRACE("Not restoring {}. threads={}, key={}", funcStr, isThreads, - snapshotKey, - isSingleHost); + snapshotKey); } // Initialise batch counter @@ -371,6 +354,7 @@ std::shared_ptr Executor::getMainThreadSnapshot( return reg.getSnapshot(snapshotKey); } +/* TODO(thread-opt): currently we never delete snapshots void Executor::deleteMainThreadSnapshot(const faabric::Message& msg) { std::string snapshotKey = faabric::util::getMainThreadSnapshotKey(msg); @@ -386,6 +370,7 @@ void Executor::deleteMainThreadSnapshot(const faabric::Message& msg) reg.deleteSnapshot(snapshotKey); } } +*/ void Executor::threadPoolThread(std::stop_token st, int threadPoolIdx) { @@ -424,10 +409,9 @@ void Executor::threadPoolThread(std::stop_token st, int threadPoolIdx) task.req->mutable_messages()->at(task.messageIndex); // Start dirty tracking if executing threads across hosts - bool isSingleHost = task.req->singlehost(); bool isThreads = task.req->type() == faabric::BatchExecuteRequest::THREADS; - bool doDirtyTracking = isThreads && !isSingleHost; + bool doDirtyTracking = isThreads && !task.req->singlehost(); if (doDirtyTracking) { // If tracking is thread local, start here as it will happen for // each thread @@ -441,6 +425,16 @@ void Executor::threadPoolThread(std::stop_token st, int threadPoolIdx) faabric::transport::PointToPointGroup::getGroup(msg.groupid()); } + // If the to-be-executed message is a migrated message, we need to + // execute the post-migration hook to sync with non-migrated messages + // in the same group + bool isMigration = + task.req->type() == faabric::BatchExecuteRequest::MIGRATION; + if (isMigration) { + faabric::transport::getPointToPointBroker().postMigrationHook( + msg.groupid(), msg.groupidx()); + } + SPDLOG_TRACE("Thread {}:{} executing task {} ({}, thread={}, group={})", id, threadPoolIdx, @@ -583,7 +577,8 @@ void Executor::threadPoolThread(std::stop_token st, int threadPoolIdx) // Delete the main thread snapshot (implicitly does nothing if // doesn't exist) - deleteMainThreadSnapshot(msg); + // TODO(thread-opt): cleanup snapshots (from planner maybe?) + // deleteMainThreadSnapshot(msg); } } @@ -610,12 +605,6 @@ void Executor::threadPoolThread(std::stop_token st, int threadPoolIdx) availablePoolThreads.insert(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. - 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 diff --git a/src/scheduler/FunctionCallClient.cpp b/src/scheduler/FunctionCallClient.cpp index 8a43dc84e..1a6a34256 100644 --- a/src/scheduler/FunctionCallClient.cpp +++ b/src/scheduler/FunctionCallClient.cpp @@ -21,20 +21,6 @@ static std::vector< std::pair>> batchMessages; -static std::vector> - resourceRequests; - -static std::unordered_map> - queuedResourceResponses; - -static std::vector< - std::pair>> - pendingMigrationsRequests; - -static std::vector> - unregisterRequests; - static std::vector>> messageResults; @@ -58,26 +44,6 @@ getBatchRequests() return batchMessages; } -std::vector> getResourceRequests() -{ - faabric::util::UniqueLock lock(mockMutex); - return resourceRequests; -} - -std::vector>> -getPendingMigrationsRequests() -{ - faabric::util::UniqueLock lock(mockMutex); - return pendingMigrationsRequests; -} - -std::vector> -getUnregisterRequests() -{ - faabric::util::UniqueLock lock(mockMutex); - return unregisterRequests; -} - std::vector>> getMessageResults() { @@ -85,25 +51,11 @@ getMessageResults() return messageResults; } -void queueResourceResponse(const std::string& host, faabric::HostResources& res) -{ - faabric::util::UniqueLock lock(mockMutex); - queuedResourceResponses[host].enqueue(res); -} - void clearMockRequests() { faabric::util::UniqueLock lock(mockMutex); functionCalls.clear(); batchMessages.clear(); - resourceRequests.clear(); - pendingMigrationsRequests.clear(); - unregisterRequests.clear(); - - for (auto& p : queuedResourceResponses) { - p.second.reset(); - } - queuedResourceResponses.clear(); messageResults.clear(); } @@ -129,48 +81,6 @@ void FunctionCallClient::sendFlush() } } -faabric::HostResources FunctionCallClient::getResources() -{ - faabric::EmptyRequest request; - faabric::HostResources response; - - if (faabric::util::isMockMode()) { - faabric::util::UniqueLock lock(mockMutex); - - // Register the request - resourceRequests.emplace_back(host, request); - - // See if we have a queued response - if (queuedResourceResponses[host].size() > 0) { - response = queuedResourceResponses[host].dequeue(); - } - } else { - syncSend( - faabric::scheduler::FunctionCalls::GetResources, &request, &response); - } - - return response; -} - -// This function call is used by the main host of an application to let know -// other hosts running functions of the same application that a migration -// opportunity has been found. -void FunctionCallClient::sendPendingMigrations( - std::shared_ptr req) -{ - faabric::PendingMigrations request; - faabric::EmptyResponse response; - - if (faabric::util::isMockMode()) { - faabric::util::UniqueLock lock(mockMutex); - pendingMigrationsRequests.emplace_back(host, req); - } else { - syncSend(faabric::scheduler::FunctionCalls::PendingMigrations, - req.get(), - &response); - } -} - void FunctionCallClient::executeFunctions( const std::shared_ptr req) { @@ -183,16 +93,6 @@ void FunctionCallClient::executeFunctions( } } -void FunctionCallClient::unregister(faabric::UnregisterRequest& req) -{ - if (faabric::util::isMockMode()) { - faabric::util::UniqueLock lock(mockMutex); - unregisterRequests.emplace_back(host, req); - } else { - asyncSend(faabric::scheduler::FunctionCalls::Unregister, &req); - } -} - void FunctionCallClient::setMessageResult(std::shared_ptr msg) { if (faabric::util::isMockMode()) { diff --git a/src/scheduler/FunctionCallServer.cpp b/src/scheduler/FunctionCallServer.cpp index a603a3973..70b56890a 100644 --- a/src/scheduler/FunctionCallServer.cpp +++ b/src/scheduler/FunctionCallServer.cpp @@ -25,10 +25,6 @@ void FunctionCallServer::doAsyncRecv(transport::Message& message) recvExecuteFunctions(message.udata()); break; } - case faabric::scheduler::FunctionCalls::Unregister: { - recvUnregister(message.udata()); - break; - } case faabric::scheduler::FunctionCalls::SetMessageResult: { recvSetMessageResult(message.udata()); break; @@ -48,12 +44,6 @@ std::unique_ptr FunctionCallServer::doSyncRecv( case faabric::scheduler::FunctionCalls::Flush: { return recvFlush(message.udata()); } - case faabric::scheduler::FunctionCalls::GetResources: { - return recvGetResources(message.udata()); - } - case faabric::scheduler::FunctionCalls::PendingMigrations: { - return recvPendingMigrations(message.udata()); - } default: { throw std::runtime_error( fmt::format("Unrecognized sync call header: {}", header)); @@ -78,53 +68,15 @@ void FunctionCallServer::recvExecuteFunctions(std::span buffer) PARSE_MSG(faabric::BatchExecuteRequest, buffer.data(), buffer.size()) // This host has now been told to execute these functions no matter what - // TODO(planner-schedule): this if is only here because, temporarily, the - // planner doesn't take any scheduling decisions - if (!parsedMsg.comesfromplanner()) { - parsedMsg.mutable_messages()->at(0).set_topologyhint("FORCE_LOCAL"); - } else { - // This flags were set by the old endpoint, we temporarily set them here - parsedMsg.mutable_messages()->at(0).set_timestamp( + for (int i = 0; i < parsedMsg.messages_size(); i++) { + parsedMsg.mutable_messages()->at(i).set_timestamp( faabric::util::getGlobalClock().epochMillis()); - parsedMsg.mutable_messages()->at(0).set_mainhost( + parsedMsg.mutable_messages()->at(i).set_executedhost( faabric::util::getSystemConfig().endpointHost); } - scheduler.callFunctions( - std::make_shared(parsedMsg)); -} - -void FunctionCallServer::recvUnregister(std::span buffer) -{ - PARSE_MSG(faabric::UnregisterRequest, buffer.data(), buffer.size()) - - SPDLOG_DEBUG("Unregistering host {} for {}/{}", - parsedMsg.host(), - parsedMsg.user(), - parsedMsg.function()); - - // Remove the host from the warm set - scheduler.removeRegisteredHost( - parsedMsg.host(), parsedMsg.user(), parsedMsg.function()); -} -std::unique_ptr FunctionCallServer::recvGetResources( - std::span buffer) -{ - auto response = std::make_unique( - scheduler.getThisHostResources()); - return response; -} - -std::unique_ptr -FunctionCallServer::recvPendingMigrations(std::span buffer) -{ - PARSE_MSG(faabric::PendingMigrations, buffer.data(), buffer.size()); - - auto msgPtr = std::make_shared(parsedMsg); - - scheduler.addPendingMigration(msgPtr); - - return std::make_unique(); + scheduler.executeBatch( + std::make_shared(parsedMsg)); } void FunctionCallServer::recvSetMessageResult(std::span buffer) diff --git a/src/scheduler/Scheduler.cpp b/src/scheduler/Scheduler.cpp index 5526c931b..b4e8ddcdf 100644 --- a/src/scheduler/Scheduler.cpp +++ b/src/scheduler/Scheduler.cpp @@ -1,10 +1,7 @@ -#include -#include +#include #include #include #include -#include -#include #include #include #include @@ -12,25 +9,16 @@ #include #include #include +#include #include +#include #include #include -#include -#include -#include #include -#include #include -#include - -#include -#include -#include #include -#define FLUSH_TIMEOUT_MS 10000 - using namespace faabric::util; using namespace faabric::snapshot; @@ -48,10 +36,6 @@ Scheduler::Scheduler() , reg(faabric::snapshot::getSnapshotRegistry()) , broker(faabric::transport::getPointToPointBroker()) { - // Set up the initial resources - int cores = faabric::util::getUsableCores(); - thisHostResources.set_slots(cores); - // Start the reaper thread reaperThread.start(conf.reaperIntervalSeconds); } @@ -63,18 +47,6 @@ Scheduler::~Scheduler() } } -std::set Scheduler::getAvailableHosts() -{ - auto availableHosts = - faabric::planner::getPlannerClient().getAvailableHosts(); - std::set availableHostsIps; - for (const auto& host : availableHosts) { - availableHostsIps.insert(host.ip()); - } - - return availableHostsIps; -} - void Scheduler::addHostToGlobalSet( const std::string& hostIp, std::shared_ptr overwriteResources) @@ -143,9 +115,6 @@ void Scheduler::reset() SPDLOG_DEBUG("Resetting scheduler"); resetThreadLocalCache(); - // Stop the function migration thread - functionMigrationThread.stop(); - // Stop the reaper thread reaperThread.stop(); @@ -170,28 +139,11 @@ void Scheduler::reset() // Ensure host is set correctly thisHost = faabric::util::getSystemConfig().endpointHost; - // Reset resources - thisHostResources = faabric::HostResources(); - thisHostResources.set_slots(faabric::util::getUsableCores()); - thisHostResources.set_usedslots(0); - thisHostUsedSlots.store(0, std::memory_order_release); - // Reset scheduler state - availableHostsCache.clear(); - registeredHosts.clear(); - threadResults.clear(); threadResultMessages.clear(); - pushedSnapshotsMap.clear(); - - // Reset function migration tracking - inFlightRequests.clear(); - pendingMigrations.clear(); - // Records - recordedMessagesAll.clear(); - recordedMessagesLocal.clear(); - recordedMessagesShared.clear(); + recordedMessages.clear(); // Restart reaper thread reaperThread.start(conf.reaperIntervalSeconds); @@ -277,24 +229,6 @@ int Scheduler::reapStaleExecutors() auto removed = std::remove(execs.begin(), execs.end(), exec); execs.erase(removed, execs.end()); } - - // Unregister this host if no more executors remain on this host, and - // it's not the main - if (execs.empty()) { - SPDLOG_TRACE("No remaining executors for {}", key); - - bool isMaster = thisHost == mainHost; - if (!isMaster) { - faabric::UnregisterRequest req; - req.set_host(thisHost); - req.set_user(user); - req.set_function(function); - - getFunctionCallClient(mainHost)->unregister(req); - } - - keysToRemove.emplace_back(key); - } } // Remove and erase @@ -314,695 +248,71 @@ long Scheduler::getFunctionExecutorCount(const faabric::Message& msg) return executors[funcStr].size(); } -int Scheduler::getFunctionRegisteredHostCount(const faabric::Message& msg) -{ - faabric::util::SharedLock lock(mx); - return getFunctionRegisteredHosts(msg.user(), msg.function(), false).size(); -} - -const std::set& Scheduler::getFunctionRegisteredHosts( - const std::string& user, - const std::string& func, - bool acquireLock) -{ - faabric::util::SharedLock lock; - if (acquireLock) { - lock = faabric::util::SharedLock(mx); - } - std::string key = user + "/" + func; - return registeredHosts[key]; -} - -void Scheduler::removeRegisteredHost(const std::string& host, - const std::string& user, - const std::string& function) +void Scheduler::executeBatch(std::shared_ptr req) { faabric::util::FullLock lock(mx); - std::string key = user + "/" + function; - registeredHosts[key].erase(host); -} -void Scheduler::addRegisteredHost(const std::string& host, - const std::string& user, - const std::string& function) -{ - std::string key = user + "/" + function; - registeredHosts[key].insert(host); -} - -void Scheduler::vacateSlot() -{ - thisHostUsedSlots.fetch_sub(1, std::memory_order_acq_rel); -} - -faabric::batch_scheduler::SchedulingDecision Scheduler::callFunctions( - std::shared_ptr req) -{ - // We assume all the messages are for the same function and have the - // same main host - faabric::Message& firstMsg = req->mutable_messages()->at(0); - std::string mainHost = firstMsg.mainhost(); - - // Get topology hint from message - faabric::batch_scheduler::SchedulingTopologyHint topologyHint = - firstMsg.topologyhint().empty() - ? faabric::batch_scheduler::SchedulingTopologyHint::NONE - : faabric::batch_scheduler::strToTopologyHint.at( - firstMsg.topologyhint()); - - bool isForceLocal = - topologyHint == - faabric::batch_scheduler::SchedulingTopologyHint::FORCE_LOCAL; - - // If we're not the main host, we need to forward the request back to the - // main host. This will only happen if a nested batch execution happens. - if (!isForceLocal && mainHost != thisHost) { - std::string funcStr = faabric::util::funcToString(firstMsg, false); - SPDLOG_DEBUG("Forwarding {} back to main {}", funcStr, mainHost); - - getFunctionCallClient(mainHost)->executeFunctions(req); - faabric::batch_scheduler::SchedulingDecision decision( - firstMsg.appid(), firstMsg.groupid()); - decision.returnHost = mainHost; - return decision; - } - - faabric::util::FullLock lock(mx); - - faabric::batch_scheduler::SchedulingDecision decision = - doSchedulingDecision(req, topologyHint); - - // Pass decision as hint - return doCallFunctions(req, decision, lock, topologyHint); -} - -faabric::batch_scheduler::SchedulingDecision Scheduler::makeSchedulingDecision( - std::shared_ptr req, - faabric::batch_scheduler::SchedulingTopologyHint topologyHint) -{ - faabric::util::FullLock lock(mx); - - return doSchedulingDecision(req, topologyHint); -} - -faabric::batch_scheduler::SchedulingDecision Scheduler::doSchedulingDecision( - std::shared_ptr req, - faabric::batch_scheduler::SchedulingTopologyHint topologyHint) -{ + bool isThreads = req->type() == faabric::BatchExecuteRequest::THREADS; + auto funcStr = faabric::util::funcToString(req); int nMessages = req->messages_size(); - faabric::Message& firstMsg = req->mutable_messages()->at(0); - std::string funcStr = faabric::util::funcToString(firstMsg, false); - - // If topology hints are disabled, unset the provided topology hint - if (conf.noTopologyHints == "on" && - topologyHint != - faabric::batch_scheduler::SchedulingTopologyHint::NONE) { - SPDLOG_WARN("Ignoring topology hint passed to scheduler as hints are " - "disabled in the config"); - topologyHint = faabric::batch_scheduler::SchedulingTopologyHint::NONE; - } - - // If requesting a cached decision, look for it now - faabric::batch_scheduler::DecisionCache& decisionCache = - faabric::batch_scheduler::getSchedulingDecisionCache(); - if (topologyHint == - faabric::batch_scheduler::SchedulingTopologyHint::CACHED) { - std::shared_ptr - cachedDecision = decisionCache.getCachedDecision(req); - - if (cachedDecision != nullptr) { - int groupId = cachedDecision->getGroupId(); - SPDLOG_DEBUG("Using cached decision for {} {}, group {}", - funcStr, - firstMsg.appid(), - groupId); - - // Get the cached hosts - std::vector hosts = cachedDecision->getHosts(); - - // Create the scheduling decision - faabric::batch_scheduler::SchedulingDecision decision( - firstMsg.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 - decision.addMessage(hosts.at(i), m); - } - - return decision; - } - - SPDLOG_DEBUG("No cached decision found for {} x {} in app {}", - req->messages_size(), - funcStr, - firstMsg.appid()); - } - - std::vector hosts; - hosts.reserve(nMessages); - - if (topologyHint == - faabric::batch_scheduler::SchedulingTopologyHint::FORCE_LOCAL) { - // We're forced to execute locally here so we do all the messages - SPDLOG_TRACE("Scheduling {}/{} of {} locally (force local)", - nMessages, - nMessages, - funcStr); + // Records for tests - copy messages before execution to avoid races + if (faabric::util::isTestMode()) { for (int i = 0; i < nMessages; i++) { - hosts.push_back(thisHost); - } - } else { - // At this point we know we're the main host, and we've not been - // asked to force full local execution. - - // Work out how many we can handle locally - int slots = thisHostResources.slots(); - if (topologyHint == - faabric::batch_scheduler::SchedulingTopologyHint::UNDERFULL) { - slots = slots / 2; - } - - // Work out available cores, flooring at zero - int available = - slots - this->thisHostUsedSlots.load(std::memory_order_acquire); - available = std::max(available, 0); - - // Claim as many as we can - int nLocally = std::min(available, nMessages); - - // Add those that can be executed locally - SPDLOG_TRACE( - "Scheduling {}/{} of {} locally", nLocally, nMessages, funcStr); - for (int i = 0; i < nLocally; i++) { - hosts.push_back(thisHost); - } - - // If some are left, we need to distribute. - // First try and do so on already registered hosts. - int remainder = nMessages - nLocally; - if (remainder > 0) { - const std::set& thisRegisteredHosts = - getFunctionRegisteredHosts( - firstMsg.user(), firstMsg.function(), false); - - for (const auto& h : thisRegisteredHosts) { - // Work out resources on the remote host - faabric::HostResources r = getHostResources(h); - int available = r.slots() - r.usedslots(); - - // We need to floor at zero here in case the remote host is - // overloaded, in which case its used slots will be greater than - // its available slots. - available = std::max(0, available); - int nOnThisHost = std::min(available, remainder); - - // Under the NEVER_ALONE topology hint, we never choose a host - // unless we can schedule at least two requests in it. - if (topologyHint == faabric::batch_scheduler:: - SchedulingTopologyHint::NEVER_ALONE && - nOnThisHost < 2) { - continue; - } - - SPDLOG_TRACE("Scheduling {}/{} of {} on {} (registered)", - nOnThisHost, - nMessages, - funcStr, - h); - - for (int i = 0; i < nOnThisHost; i++) { - hosts.push_back(h); - } - - remainder -= nOnThisHost; - if (remainder <= 0) { - break; - } - } - } - - // Now schedule to unregistered hosts if there are messages left - if (remainder > 0) { - std::vector unregisteredHosts = - getUnregisteredHosts(firstMsg.user(), firstMsg.function()); - - for (const auto& h : unregisteredHosts) { - // Skip if this host - if (h == thisHost) { - continue; - } - - // Work out resources on the remote host - faabric::HostResources r = getHostResources(h); - int available = r.slots() - r.usedslots(); - - // We need to floor at zero here in case the remote host is - // overloaded, in which case its used slots will be greater than - // its available slots. - available = std::max(0, available); - int nOnThisHost = std::min(available, remainder); - - if (topologyHint == faabric::batch_scheduler:: - SchedulingTopologyHint::NEVER_ALONE && - nOnThisHost < 2) { - continue; - } - - SPDLOG_TRACE("Scheduling {}/{} of {} on {} (unregistered)", - nOnThisHost, - nMessages, - funcStr, - h); - - // Register the host if it's exected a function - if (nOnThisHost > 0) { - addRegisteredHost(h, firstMsg.user(), firstMsg.function()); - } - - for (int i = 0; i < nOnThisHost; i++) { - hosts.push_back(h); - } - - remainder -= nOnThisHost; - if (remainder <= 0) { - break; - } - } - } - - // At this point there's no more capacity in the system, so we - // just need to overload locally - if (remainder > 0) { - std::string overloadedHost = thisHost; - - // Under the NEVER_ALONE scheduling topology hint we want to - // overload the last host we assigned requests to. - if (topologyHint == faabric::batch_scheduler:: - SchedulingTopologyHint::NEVER_ALONE && - !hosts.empty()) { - overloadedHost = hosts.back(); - } - - SPDLOG_DEBUG("Overloading {}/{} {} {}", - remainder, - nMessages, - funcStr, - overloadedHost == thisHost - ? "locally" - : "to host " + overloadedHost); - - for (int i = 0; i < remainder; i++) { - hosts.push_back(overloadedHost); - } - } - } - - // Sanity check - if (hosts.size() != nMessages) { - SPDLOG_ERROR( - "Serious scheduling error: {} != {}", hosts.size(), nMessages); - - throw std::runtime_error("Not enough scheduled hosts for messages"); - } - - // Set up decision - faabric::batch_scheduler::SchedulingDecision decision(firstMsg.appid(), - firstMsg.groupid()); - for (int i = 0; i < hosts.size(); i++) { - decision.addMessage(hosts.at(i), req->messages().at(i)); - } - - // Cache decision for next time if necessary - if (topologyHint == - faabric::batch_scheduler::SchedulingTopologyHint::CACHED) { - decisionCache.addCachedDecision(req, decision); - } - - return decision; -} - -faabric::batch_scheduler::SchedulingDecision Scheduler::callFunctions( - std::shared_ptr req, - faabric::batch_scheduler::SchedulingDecision& hint) -{ - faabric::util::FullLock lock(mx); - return doCallFunctions( - req, hint, lock, faabric::batch_scheduler::SchedulingTopologyHint::NONE); -} - -faabric::batch_scheduler::SchedulingDecision Scheduler::doCallFunctions( - std::shared_ptr req, - faabric::batch_scheduler::SchedulingDecision& decision, - faabric::util::FullLock& lock, - faabric::batch_scheduler::SchedulingTopologyHint topologyHint) -{ - faabric::Message& firstMsg = req->mutable_messages()->at(0); - std::string funcStr = faabric::util::funcToString(firstMsg, false); - int nMessages = req->messages_size(); - bool isMaster = thisHost == firstMsg.mainhost(); - bool isMigration = req->type() == faabric::BatchExecuteRequest::MIGRATION; - - if (decision.hosts.size() != nMessages) { - SPDLOG_ERROR( - "Passed decision for {} with {} messages, but request has {}", - funcStr, - decision.hosts.size(), - nMessages); - throw std::runtime_error("Invalid scheduler hint for messages"); - } - - if (firstMsg.mainhost().empty()) { - SPDLOG_ERROR("Request {} has no main host", funcStr); - throw std::runtime_error("Message with no main host"); - } - - // Send out point-to-point mappings if necessary (unless being forced to - // execute locally, in which case they will be transmitted from the - // main) - bool isForceLocal = - topologyHint == - faabric::batch_scheduler::SchedulingTopologyHint::FORCE_LOCAL; - if (!isForceLocal && !isMigration && (firstMsg.groupid() > 0)) { - if (firstMsg.ismpi()) { - // If we are scheduling an MPI message, we want rank 0 to be in the - // group. However, rank 0 is the one calling this method to schedule - // the remaining worldSize - 1 functions. We can not change the - // scheduling decision, as this would affect the downstream method, - // but we can make a special copy just for the broker - auto decisionCopy = decision; - auto msgCopy = firstMsg; - msgCopy.set_groupidx(0); - decisionCopy.addMessage(thisHost, msgCopy); - broker.setAndSendMappingsFromSchedulingDecision(decisionCopy); - } else { - broker.setAndSendMappingsFromSchedulingDecision(decision); + recordedMessages.emplace_back(req->messages().at(i)); } } - // Record in-flight request if function desires to be migrated - if (!isMigration && firstMsg.migrationcheckperiod() > 0) { - doStartFunctionMigrationThread(req, decision); - } - - // We want to schedule things on this host _last_, otherwise functions may - // start executing before all messages have been dispatched, thus slowing - // the remaining scheduling. Therefore we want to create a list of unique - // hosts, with this host last. - std::vector orderedHosts; - bool isSingleHost = false; - { - std::set uniqueHosts(decision.hosts.begin(), - decision.hosts.end()); - bool hasFunctionsOnThisHost = uniqueHosts.contains(thisHost); - - // Mark the request as being single-host if necessary - if (conf.noSingleHostOptimisations == 0) { - std::set thisHostUniset = { thisHost }; - isSingleHost = (uniqueHosts == thisHostUniset) && isMaster; - req->set_singlehost(isSingleHost); - } - - if (hasFunctionsOnThisHost) { - uniqueHosts.erase(thisHost); - } - - orderedHosts = std::vector(uniqueHosts.begin(), uniqueHosts.end()); - - if (hasFunctionsOnThisHost) { - orderedHosts.push_back(thisHost); - } - } - - // ------------------------------------------- - // THREADS - // ------------------------------------------- - bool isThreads = req->type() == faabric::BatchExecuteRequest::THREADS; - - // Register thread results if necessary + // For threads we only need one executor, for anything else we want + // one Executor per function in flight. if (isThreads) { - for (const auto& m : req->messages()) { - registerThread(m.id()); + // Threads use the existing executor. We assume there's only + // one running at a time. + std::vector>& thisExecutors = + executors[funcStr]; + + std::shared_ptr e = nullptr; + if (thisExecutors.empty()) { + // Create executor if not exists + e = claimExecutor(*req->mutable_messages(0), lock); + } else if (thisExecutors.size() == 1) { + // Use existing executor if exists + e = thisExecutors.back(); + } else { + SPDLOG_ERROR("Found {} executors for threaded function {}", + thisExecutors.size(), + funcStr); + throw std::runtime_error( + "Expected only one executor for threaded function"); } - } - - // ------------------------------------------- - // SNAPSHOTS - // ------------------------------------------- - // Push out snapshot diffs to registered hosts. We have to do this to - // *all* hosts, regardless of whether they will be executing functions. - // This greatly simplifies the reasoning about which hosts hold which - // diffs. - - std::string snapshotKey; - if (isThreads) { - if (!firstMsg.snapshotkey().empty()) { - SPDLOG_ERROR("{} should not provide snapshot key for {} threads", - funcStr, - req->messages().size()); + assert(e != nullptr); - std::runtime_error("Should not provide snapshot key for threads"); - } - - if (!isSingleHost) { - snapshotKey = faabric::util::getMainThreadSnapshotKey(firstMsg); - } + // Execute the tasks + std::vector thisHostIdxs(req->messages_size()); + std::iota(thisHostIdxs.begin(), thisHostIdxs.end(), 0); + e->executeTasks(thisHostIdxs, req); } else { - snapshotKey = firstMsg.snapshotkey(); - } - - if (!snapshotKey.empty()) { - auto snap = reg.getSnapshot(snapshotKey); - - for (const auto& host : getFunctionRegisteredHosts( - firstMsg.user(), firstMsg.function(), false)) { - std::shared_ptr c = getSnapshotClient(host); - - // See if we've already pushed this snapshot to the given host, - // if so, just push the diffs that have occurred in this main thread - if (pushedSnapshotsMap[snapshotKey].contains(host)) { - std::vector snapshotDiffs = - snap->getTrackedChanges(); - - c->pushSnapshotUpdate(snapshotKey, snap, snapshotDiffs); - } else { - c->pushSnapshot(snapshotKey, snap); - pushedSnapshotsMap[snapshotKey].insert(host); - } - } - - // Now reset the tracking on the snapshot before we start executing - snap->clearTrackedChanges(); - } else if (!snapshotKey.empty() && isMigration && isForceLocal) { - // If we are executing a migrated function, we don't need to distribute - // the snapshot to other hosts, as this snapshot is specific to the - // to-be-restored function - auto snap = reg.getSnapshot(snapshotKey); - - // Now reset the tracking on the snapshot before we start executing - snap->clearTrackedChanges(); - } - - // ------------------------------------------- - // EXECUTION - // ------------------------------------------- - - // Records for tests - copy messages before execution to avoid racing on msg - size_t recordedMessagesOffset = recordedMessagesAll.size(); - if (faabric::util::isTestMode()) { + // Non-threads require one executor per task for (int i = 0; i < nMessages; i++) { - recordedMessagesAll.emplace_back(req->messages().at(i)); - } - } - - // Iterate through unique hosts and dispatch messages - for (const std::string& host : orderedHosts) { - // Work out which indexes are scheduled on this host - std::vector thisHostIdxs; - for (int i = 0; i < decision.hosts.size(); i++) { - if (decision.hosts.at(i) == host) { - thisHostIdxs.push_back(i); - } - } - - if (host == thisHost) { - // ------------------------------------------- - // LOCAL EXECTUION - // ------------------------------------------- - // For threads we only need one executor, for anything else we want - // one Executor per function in flight. - - if (thisHostIdxs.empty()) { - SPDLOG_DEBUG("Not scheduling any calls to {} out of {} locally", - funcStr, - nMessages); - continue; - } - - SPDLOG_DEBUG("Scheduling {}/{} calls to {} locally", - thisHostIdxs.size(), - nMessages, - funcStr); + faabric::Message& localMsg = req->mutable_messages()->at(i); - // Update slots - this->thisHostUsedSlots.fetch_add(thisHostIdxs.size(), - std::memory_order_acquire); - - if (isThreads) { - // Threads use the existing executor. We assume there's only - // one running at a time. - std::vector>& thisExecutors = - executors[funcStr]; - - std::shared_ptr e = nullptr; - if (thisExecutors.empty()) { - // Create executor if not exists - e = claimExecutor(firstMsg, lock); - } else if (thisExecutors.size() == 1) { - // Use existing executor if exists - e = thisExecutors.back(); - } else { - SPDLOG_ERROR("Found {} executors for threaded function {}", - thisExecutors.size(), - funcStr); - throw std::runtime_error( - "Expected only one executor for threaded function"); - } - - assert(e != nullptr); - - // Execute the tasks - e->executeTasks(thisHostIdxs, req); - } else { - // Non-threads require one executor per task - for (auto i : thisHostIdxs) { - faabric::Message& localMsg = req->mutable_messages()->at(i); - - std::shared_ptr e = claimExecutor(localMsg, lock); - e->executeTasks({ i }, req); - } - } - } else { - // ------------------------------------------- - // REMOTE EXECTUION - // ------------------------------------------- - SPDLOG_DEBUG("Scheduling {}/{} calls to {} on {}", - thisHostIdxs.size(), - nMessages, - funcStr, - host); - - // Set up new request - std::shared_ptr hostRequest = - faabric::util::batchExecFactory(); - hostRequest->set_snapshotkey(req->snapshotkey()); - hostRequest->set_type(req->type()); - hostRequest->set_subtype(req->subtype()); - hostRequest->set_contextdata(req->contextdata()); - - // Add messages - for (auto msgIdx : thisHostIdxs) { - auto* newMsg = hostRequest->add_messages(); - *newMsg = req->messages().at(msgIdx); - newMsg->set_executeslocally(false); - } - - // Dispatch the calls - getFunctionCallClient(host)->executeFunctions(hostRequest); + std::shared_ptr e = claimExecutor(localMsg, lock); + e->executeTasks({ i }, req); } } - - // Records for tests - if (faabric::util::isTestMode()) { - for (int i = 0; i < nMessages; i++) { - std::string executedHost = decision.hosts.at(i); - const faabric::Message& msg = - recordedMessagesAll.at(recordedMessagesOffset + i); - - // Log results if in test mode - if (executedHost.empty() || executedHost == thisHost) { - recordedMessagesLocal.emplace_back(msg); - } else { - recordedMessagesShared.emplace_back(executedHost, msg); - } - } - } - - return decision; -} - -std::vector Scheduler::getUnregisteredHosts( - const std::string& user, - const std::string& function, - bool noCache) -{ - // Load the list of available hosts - if (availableHostsCache.empty() || noCache) { - availableHostsCache = getAvailableHosts(); - } - - // At this point we know we need to enlist unregistered hosts - const std::set& thisRegisteredHosts = - getFunctionRegisteredHosts(user, function, false); - - std::vector unregisteredHosts; - - std::set_difference( - availableHostsCache.begin(), - availableHostsCache.end(), - thisRegisteredHosts.begin(), - thisRegisteredHosts.end(), - std::inserter(unregisteredHosts, unregisteredHosts.begin())); - - // If we've not got any, try again without caching - if (unregisteredHosts.empty() && !noCache) { - return getUnregisteredHosts(user, function, true); - } - - return unregisteredHosts; -} - -void Scheduler::broadcastSnapshotDelete(const faabric::Message& msg, - const std::string& snapshotKey) -{ - const std::set& thisRegisteredHosts = - getFunctionRegisteredHosts(msg.user(), msg.function(), false); - - for (auto host : thisRegisteredHosts) { - getSnapshotClient(host)->deleteSnapshot(snapshotKey); - } } void Scheduler::clearRecordedMessages() { faabric::util::FullLock lock(mx); - recordedMessagesAll.clear(); - recordedMessagesLocal.clear(); - recordedMessagesShared.clear(); + recordedMessages.clear(); } -std::vector Scheduler::getRecordedMessagesAll() +std::vector Scheduler::getRecordedMessages() { faabric::util::SharedLock lock(mx); - return recordedMessagesAll; -} - -std::vector Scheduler::getRecordedMessagesLocal() -{ - faabric::util::SharedLock lock(mx); - return recordedMessagesLocal; -} - -std::vector> -Scheduler::getRecordedMessagesShared() -{ - faabric::util::SharedLock lock(mx); - return recordedMessagesShared; + return recordedMessages; } std::shared_ptr Scheduler::claimExecutor( @@ -1069,33 +379,17 @@ void Scheduler::flushLocally() void Scheduler::setFunctionResult(faabric::Message& msg) { - // Record which host did the execution - msg.set_executedhost(faabric::util::getSystemConfig().endpointHost); - // Set finish timestamp msg.set_finishtimestamp(faabric::util::getGlobalClock().epochMillis()); - // Remove the app from in-flight map if still there, and this host is the - // main host for the message - if (msg.mainhost() == thisHost) { - removePendingMigration(msg.appid()); - } - // Let the planner know this function has finished execution. This will // wake any thread waiting on this result faabric::planner::getPlannerClient().setMessageResult( std::make_shared(msg)); } -void Scheduler::registerThread(uint32_t msgId) -{ - // Here we need to ensure the promise is registered locally so - // callers can start waiting - threadResults[msgId]; -} - void Scheduler::setThreadResult( - const faabric::Message& msg, + faabric::Message& msg, int32_t returnValue, const std::string& key, const std::vector& diffs) @@ -1118,122 +412,52 @@ void Scheduler::setThreadResult( // merging operation. snap->queueDiffs(diffs); } - - // Set thread result locally - setThreadResultLocally(msg.id(), returnValue); } else { // Push thread result and diffs together getSnapshotClient(msg.mainhost()) - ->pushThreadResult(msg.id(), returnValue, key, diffs); + ->pushThreadResult(msg.appid(), msg.id(), returnValue, key, diffs); } -} -void Scheduler::setThreadResultLocally(uint32_t msgId, int32_t returnValue) -{ - faabric::util::FullLock lock(mx); - SPDLOG_DEBUG("Setting result for thread {} to {}", msgId, returnValue); - threadResults.at(msgId).set_value(returnValue); + // Finally, set the message result in the planner + setFunctionResult(msg); } -void Scheduler::setThreadResultLocally(uint32_t msgId, +void Scheduler::setThreadResultLocally(uint32_t appId, + uint32_t msgId, int32_t returnValue, faabric::transport::Message& message) { - setThreadResultLocally(msgId, returnValue); - // Keep the message faabric::util::FullLock lock(mx); threadResultMessages.insert(std::make_pair(msgId, std::move(message))); } +// TODO(scheduler-cleanup): move method elsewhere std::vector> Scheduler::awaitThreadResults( - std::shared_ptr req) + std::shared_ptr req, + int timeoutMs) { 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 = awaitThreadResult(messageId); - results.emplace_back(messageId, result); + auto msgResult = faabric::planner::getPlannerClient().getMessageResult( + req->appid(), messageId, timeoutMs); + results.emplace_back(messageId, msgResult.returnvalue()); } return results; } -int32_t Scheduler::awaitThreadResult(uint32_t messageId) -{ - faabric::util::SharedLock lock(mx); - auto it = threadResults.find(messageId); - if (it == threadResults.end()) { - SPDLOG_ERROR("Thread {} not registered on this host", messageId); - throw std::runtime_error("Awaiting unregistered thread"); - } - lock.unlock(); - - return it->second.get_future().get(); -} - -void Scheduler::deregisterThreads( - std::shared_ptr req) -{ - faabric::util::FullLock eraseLock(mx); - for (auto m : req->messages()) { - threadResults.erase(m.id()); - threadResultMessages.erase(m.id()); - } -} - -void Scheduler::deregisterThread(uint32_t msgId) -{ - // Erase the cached message and thread result - faabric::util::FullLock eraseLock(mx); - threadResults.erase(msgId); - threadResultMessages.erase(msgId); -} - -std::vector Scheduler::getRegisteredThreads() -{ - faabric::util::SharedLock lock(mx); - - std::vector registeredIds; - for (auto const& p : threadResults) { - registeredIds.push_back(p.first); - } - - std::sort(registeredIds.begin(), registeredIds.end()); - - return registeredIds; -} - size_t Scheduler::getCachedMessageCount() { return threadResultMessages.size(); } -faabric::HostResources Scheduler::getThisHostResources() -{ - faabric::util::SharedLock lock(mx); - faabric::HostResources hostResources = thisHostResources; - hostResources.set_usedslots( - this->thisHostUsedSlots.load(std::memory_order_acquire)); - return hostResources; -} - void Scheduler::setThisHostResources(faabric::HostResources& res) { - // Update the planner (no lock required) addHostToGlobalSet(thisHost, std::make_shared(res)); - - faabric::util::FullLock lock(mx); - thisHostResources = res; - this->thisHostUsedSlots.store(res.usedslots(), std::memory_order_release); -} - -faabric::HostResources Scheduler::getHostResources(const std::string& host) -{ - SPDLOG_TRACE("Requesting resources from {}", host); - return getFunctionCallClient(host)->getResources(); } // -------------------------------------------- @@ -1250,256 +474,71 @@ std::string getChainedKey(unsigned int msgId) // MIGRATION // ---------------------------------------- -void FunctionMigrationThread::doWork() -{ - getScheduler().checkForMigrationOpportunities(); -} - -void Scheduler::checkForMigrationOpportunities() -{ - std::vector> - tmpPendingMigrations; - - { - // Acquire a shared lock to read from the in-flight requests map - faabric::util::SharedLock lock(mx); - - tmpPendingMigrations = doCheckForMigrationOpportunities(); - } - - // If we find migration opportunites - if (tmpPendingMigrations.size() > 0) { - // Acquire full lock to write to the pending migrations map - faabric::util::FullLock lock(mx); - - for (auto msgPtr : tmpPendingMigrations) { - // First, broadcast the pending migrations to other hosts - broadcastPendingMigrations(msgPtr); - // Second, update our local records - pendingMigrations[msgPtr->appid()] = std::move(msgPtr); +std::shared_ptr +Scheduler::checkForMigrationOpportunities(faabric::Message& msg, + int overwriteNewGroupId) +{ + int appId = msg.appid(); + int groupId = msg.groupid(); + int groupIdx = msg.groupidx(); + SPDLOG_DEBUG("Message {}:{}:{} checking for migration opportunities", + appId, + groupId, + groupIdx); + + // TODO: maybe we could move this into a broker-specific function? + int newGroupId = 0; + if (groupIdx == 0) { + // To check for migration opportunities, we request a scheduling + // decision for the same batch execute request, but setting the + // migration flag + auto req = + faabric::util::batchExecFactory(msg.user(), msg.function(), 1); + faabric::util::updateBatchExecAppId(req, msg.appid()); + faabric::util::updateBatchExecGroupId(req, msg.groupid()); + req->set_type(faabric::BatchExecuteRequest::MIGRATION); + auto decision = planner::getPlannerClient().callFunctions(req); + + // Update the group ID if we want to migrate + if (decision == DO_NOT_MIGRATE_DECISION) { + newGroupId = groupId; + } else { + newGroupId = decision.groupId; } - } -} -void Scheduler::broadcastPendingMigrations( - std::shared_ptr pendingMigrations) -{ - // Get all hosts for the to-be migrated app - auto msg = pendingMigrations->migrations().at(0).msg(); - const std::set& thisRegisteredHosts = - getFunctionRegisteredHosts(msg.user(), msg.function(), false); - - // Remove this host from the set - registeredHosts.erase(thisHost); - - // Send pending migrations to all involved hosts - for (auto& otherHost : thisRegisteredHosts) { - getFunctionCallClient(otherHost)->sendPendingMigrations( - pendingMigrations); + // Send the new group id to all the members of the group + auto groupIdxs = broker.getIdxsRegisteredForGroup(groupId); + groupIdxs.erase(0); + for (const auto& recvIdx : groupIdxs) { + broker.sendMessage( + groupId, 0, recvIdx, BYTES_CONST(&newGroupId), sizeof(int)); + } + } else if (overwriteNewGroupId == 0) { + std::vector bytes = broker.recvMessage(groupId, 0, groupIdx); + newGroupId = faabric::util::bytesToInt(bytes); + } else { + // In some settings, like tests, we already know the new group id, so + // we can set it here (and in fact, we need to do so when faking two + // hosts) + newGroupId = overwriteNewGroupId; } -} - -std::shared_ptr Scheduler::getPendingAppMigrations( - uint32_t appId) -{ - faabric::util::SharedLock lock(mx); - if (pendingMigrations.find(appId) == pendingMigrations.end()) { + bool appMustMigrate = newGroupId != groupId; + if (!appMustMigrate) { return nullptr; } - return pendingMigrations[appId]; -} - -void Scheduler::addPendingMigration( - std::shared_ptr pMigration) -{ - faabric::util::FullLock lock(mx); - - auto msg = pMigration->migrations().at(0).msg(); - if (pendingMigrations.find(msg.appid()) != pendingMigrations.end()) { - SPDLOG_ERROR("Received remote request to add a pending migration for " - "app {}, but already recorded another migration request" - " for the same app.", - msg.appid()); - throw std::runtime_error("Remote request for app already there"); - } - - pendingMigrations[msg.appid()] = pMigration; -} - -void Scheduler::removePendingMigration(uint32_t appId) -{ - faabric::util::FullLock lock(mx); + msg.set_groupid(newGroupId); + broker.waitForMappingsOnThisHost(newGroupId); + std::string newHost = broker.getHostForReceiver(newGroupId, groupIdx); - inFlightRequests.erase(appId); - pendingMigrations.erase(appId); -} + auto migration = std::make_shared(); + migration->set_appid(appId); + migration->set_groupid(newGroupId); + migration->set_groupidx(groupIdx); + migration->set_srchost(thisHost); + migration->set_dsthost(newHost); -std::vector> -Scheduler::doCheckForMigrationOpportunities( - faabric::batch_scheduler::MigrationStrategy migrationStrategy) -{ - std::vector> - pendingMigrationsVec; - - // For each in-flight request that has opted in to be migrated, - // check if there is an opportunity to migrate - for (const auto& app : inFlightRequests) { - auto req = app.second.first; - auto originalDecision = *app.second.second; - - // If we have already recorded a pending migration for this req, - // skip - if (getPendingAppMigrations(originalDecision.appId) != nullptr) { - SPDLOG_TRACE("Skipping app {} as migration opportunity has " - "already been recorded", - originalDecision.appId); - continue; - } - - faabric::PendingMigrations msg; - msg.set_appid(originalDecision.appId); - - if (migrationStrategy == - faabric::batch_scheduler::MigrationStrategy::BIN_PACK) { - // We assume the batch was originally scheduled using - // bin-packing, thus the scheduling decision has at the begining - // (left) the hosts with the most allocated requests, and at the - // end (right) the hosts with the fewest. To check for migration - // oportunities, we compare a pointer to the possible - // destination of the migration (left), with one to the possible - // source of the migration (right). NOTE - this is a slight - // simplification, but makes the code simpler. - auto left = originalDecision.hosts.begin(); - auto right = originalDecision.hosts.end() - 1; - faabric::HostResources r = (*left == thisHost) - ? getThisHostResources() - : getHostResources(*left); - auto nAvailable = [&r]() -> int { - return r.slots() - r.usedslots(); - }; - auto claimSlot = [&r]() { - int currentUsedSlots = r.usedslots(); - r.set_usedslots(currentUsedSlots + 1); - }; - while (left < right) { - // If both pointers point to the same host, no migration - // opportunity, and must check another possible source of - // the migration - if (*left == *right) { - --right; - continue; - } - - // If the left pointer (possible destination of the - // migration) is out of available resources, no migration - // opportunity, and must check another possible destination - // of migration - if (nAvailable() == 0) { - auto oldHost = *left; - ++left; - if (*left != oldHost) { - r = (*left == thisHost) ? getThisHostResources() - : getHostResources(*left); - } - continue; - } - - // If each pointer points to a request scheduled in a - // different host, and the possible destination has slots, - // there is a migration opportunity - auto* migration = msg.add_migrations(); - migration->set_srchost(*right); - migration->set_dsthost(*left); - - faabric::Message* msgPtr = - &(*(req->mutable_messages()->begin() + - std::distance(originalDecision.hosts.begin(), right))); - auto* migrationMsgPtr = migration->mutable_msg(); - *migrationMsgPtr = *msgPtr; - // Decrement by one the availability, and check for more - // possible sources of migration - claimSlot(); - --right; - } - } else { - SPDLOG_ERROR("Unrecognised migration strategy: {}", - migrationStrategy); - throw std::runtime_error("Unrecognised migration strategy."); - } - - if (msg.migrations_size() > 0) { - pendingMigrationsVec.emplace_back( - std::make_shared(msg)); - SPDLOG_DEBUG("Detected migration opportunity for app: {}", - msg.appid()); - } else { - SPDLOG_DEBUG("No migration opportunity detected for app: {}", - msg.appid()); - } - } - - return pendingMigrationsVec; -} - -// Start the function migration thread if necessary -// NOTE: ideally, instead of allowing the applications to specify a check -// period, we would have a default one (overwritable through an env. -// variable), and apps would just opt in/out of being migrated. We set -// the actual check period instead to ease with experiments. -void Scheduler::doStartFunctionMigrationThread( - std::shared_ptr req, - faabric::batch_scheduler::SchedulingDecision& decision) -{ - bool startMigrationThread = inFlightRequests.size() == 0; - faabric::Message& firstMsg = req->mutable_messages()->at(0); - - if (inFlightRequests.find(decision.appId) != inFlightRequests.end()) { - // MPI applications are made up of two different requests: the - // original one (with one message) and the second one (with - // world size - 1 messages) created during world creation time. - // Thus, to correctly track migration opportunities we must merge - // both. We append the batch request to the original one (instead - // of the other way around) not to affect the rest of this methods - // functionality. - if (firstMsg.ismpi()) { - startMigrationThread = false; - auto originalReq = inFlightRequests[decision.appId].first; - auto originalDecision = inFlightRequests[decision.appId].second; - assert(req->messages_size() == firstMsg.mpiworldsize() - 1); - for (int i = 0; i < firstMsg.mpiworldsize() - 1; i++) { - // Append message to original request - auto* newMsgPtr = originalReq->add_messages(); - *newMsgPtr = req->messages().at(i); - - // Append message to original decision - originalDecision->addMessage(decision.hosts.at(i), - req->messages().at(i)); - } - } else { - SPDLOG_ERROR("There is already an in-flight request for app {}", - firstMsg.appid()); - throw std::runtime_error("App already in-flight"); - } - } else { - auto decisionPtr = - std::make_shared( - decision); - inFlightRequests[decision.appId] = std::make_pair(req, decisionPtr); - } - - // Decide wether we have to start the migration thread or not - if (startMigrationThread) { - functionMigrationThread.start(firstMsg.migrationcheckperiod()); - } else if (firstMsg.migrationcheckperiod() != - functionMigrationThread.getIntervalSeconds()) { - SPDLOG_WARN("Ignoring migration check period for app {} as the" - "migration thread is already running with a different" - " check period (provided: {}, current: {})", - firstMsg.appid(), - firstMsg.migrationcheckperiod(), - functionMigrationThread.getIntervalSeconds()); - } + return migration; } } diff --git a/src/snapshot/SnapshotClient.cpp b/src/snapshot/SnapshotClient.cpp index 294d02a39..ca63dc72d 100644 --- a/src/snapshot/SnapshotClient.cpp +++ b/src/snapshot/SnapshotClient.cpp @@ -170,26 +170,8 @@ void SnapshotClient::pushSnapshotUpdate( } } -void SnapshotClient::deleteSnapshot(const std::string& key) -{ - if (faabric::util::isMockMode()) { - faabric::util::UniqueLock lock(mockMutex); - snapshotDeletes.emplace_back(host, key); - - } else { - SPDLOG_DEBUG("Deleting snapshot {} from {}", key, host); - - // TODO - avoid copying data here - flatbuffers::FlatBufferBuilder mb; - auto keyOffset = mb.CreateString(key); - auto requestOffset = CreateSnapshotDeleteRequest(mb, keyOffset); - mb.Finish(requestOffset); - - SEND_FB_MSG_ASYNC(SnapshotCalls::DeleteSnapshot, mb); - } -} - void SnapshotClient::pushThreadResult( + uint32_t appId, uint32_t messageId, int returnValue, const std::string& key, @@ -231,13 +213,49 @@ void SnapshotClient::pushThreadResult( auto diffsOffset = mb.CreateVector(diffsFbVector); requestOffset = CreateThreadResultRequest( - mb, messageId, returnValue, keyOffset, diffsOffset); + mb, appId, messageId, returnValue, keyOffset, diffsOffset); mb.Finish(requestOffset); SEND_FB_MSG(SnapshotCalls::ThreadResult, mb); } } +/* TODO(thread-opt): currently we don't delete snapshots in threads +void SnapshotClient::deleteSnapshot(const std::string& key) +{ + if (faabric::util::isMockMode()) { + faabric::util::UniqueLock lock(mockMutex); + snapshotDeletes.emplace_back(host, key); + + } else { + SPDLOG_DEBUG("Deleting snapshot {} from {}", key, host); + + // TODO - avoid copying data here + flatbuffers::FlatBufferBuilder mb; + auto keyOffset = mb.CreateString(key); + auto requestOffset = CreateSnapshotDeleteRequest(mb, keyOffset); + mb.Finish(requestOffset); + + SEND_FB_MSG_ASYNC(SnapshotCalls::DeleteSnapshot, mb); + } +} + +void SnapshotClient::broadcastSnapshotDelete(const std::string& snapshotKey) +{ + std::set hostsToSendDelete; + { + faabric::util::SharedLock lock(mx); + if (pushedSnapshotsMap.contains(snapshotKey)) { + hostsToSendDelete = pushedSnapshotsMap.at(snapshotKey); + } + } + + for (auto host : hostsToSendDelete) { + getSnapshotClient(host)->deleteSnapshot(snapshotKey); + } +} +*/ + // ----------------------------------- // Static setter/getters // ----------------------------------- diff --git a/src/snapshot/SnapshotServer.cpp b/src/snapshot/SnapshotServer.cpp index 6e6a4fdc7..c508195fd 100644 --- a/src/snapshot/SnapshotServer.cpp +++ b/src/snapshot/SnapshotServer.cpp @@ -135,7 +135,8 @@ std::unique_ptr SnapshotServer::recvThreadResult( // Because we don't take ownership of the data in the diffs, we must also // ensure that the underlying message is cached faabric::scheduler::Scheduler& sch = faabric::scheduler::getScheduler(); - sch.setThreadResultLocally(r->message_id(), r->return_value(), message); + sch.setThreadResultLocally( + r->app_id(), r->message_id(), r->return_value(), message); return std::make_unique(); } diff --git a/src/transport/PointToPointBroker.cpp b/src/transport/PointToPointBroker.cpp index c10897841..2e232d206 100644 --- a/src/transport/PointToPointBroker.cpp +++ b/src/transport/PointToPointBroker.cpp @@ -442,7 +442,14 @@ void PointToPointBroker::setAndSendMappingsFromSchedulingDecision( setUpLocalMappingsFromSchedulingDecision(decision); // Send out to other hosts - for (const auto& host : otherHosts) { + sendMappingsFromSchedulingDecision(decision, otherHosts); +} + +void PointToPointBroker::sendMappingsFromSchedulingDecision( + const faabric::batch_scheduler::SchedulingDecision& decision, + const std::set& hostList) +{ + for (const auto& host : hostList) { faabric::PointToPointMappings msg; msg.set_appid(decision.appId); msg.set_groupid(decision.groupId); @@ -851,6 +858,40 @@ void PointToPointBroker::resetThreadLocalCache() threadEndpoints.clear(); } +void PointToPointBroker::postMigrationHook(int groupId, int groupIdx) +{ + int postMigrationOkCode = 1337; + int recvCode = 0; + + // TODO: implement this as a broadcast in the PTP broker + int mainIdx = 0; + if (groupIdx == mainIdx) { + auto groupIdxs = getIdxsRegisteredForGroup(groupId); + groupIdxs.erase(mainIdx); + for (const auto& recvIdx : groupIdxs) { + sendMessage(groupId, + mainIdx, + recvIdx, + BYTES_CONST(&postMigrationOkCode), + sizeof(int)); + } + recvCode = postMigrationOkCode; + } else { + std::vector bytes = recvMessage(groupId, 0, groupIdx); + recvCode = faabric::util::bytesToInt(bytes); + } + + if (recvCode != postMigrationOkCode) { + SPDLOG_ERROR("Error in post-migration hook. {}:{} received code {}", + groupId, + groupIdx, + recvCode); + throw std::runtime_error("Error in post-migration hook"); + } + + SPDLOG_DEBUG("{}:{} exiting post-migration hook", groupId, groupIdx); +} + PointToPointBroker& getPointToPointBroker() { static PointToPointBroker broker; diff --git a/src/util/ExecGraph.cpp b/src/util/ExecGraph.cpp index 3281efc21..c4b756454 100644 --- a/src/util/ExecGraph.cpp +++ b/src/util/ExecGraph.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include @@ -7,8 +8,6 @@ #include #define EXEC_GRAPH_TIMEOUT_MS 1000 -// TODO: avoid this duplication -#define MIGRATED_FUNCTION_RETURN_VALUE -99 namespace faabric::util { diff --git a/src/util/batch.cpp b/src/util/batch.cpp index 7bf0cb935..c437d36d5 100644 --- a/src/util/batch.cpp +++ b/src/util/batch.cpp @@ -33,11 +33,15 @@ std::shared_ptr batchExecFactory( bool isBatchExecRequestValid(std::shared_ptr ber) { if (ber == nullptr) { + SPDLOG_ERROR("Ber points to null!"); return false; } // An empty BER (thus invalid) will have 0 messages and an id of 0 if (ber->messages_size() <= 0 && ber->appid() == 0) { + SPDLOG_ERROR("Invalid (uninitialised) BER (size: {} - app id: {})", + ber->messages_size(), + ber->appid()); return false; } @@ -47,6 +51,7 @@ bool isBatchExecRequestValid(std::shared_ptr ber) // If the user or func are empty, the BER is invalid if (user.empty() || func.empty()) { + SPDLOG_ERROR("Unset user ({}) or func ({}) in BER!", user, func); return false; } diff --git a/tests/dist/dist_test_fixtures.h b/tests/dist/dist_test_fixtures.h index 5695680fc..a46145ee4 100644 --- a/tests/dist/dist_test_fixtures.h +++ b/tests/dist/dist_test_fixtures.h @@ -45,7 +45,7 @@ class DistTestsFixture std::string getMasterIP() { return conf.endpointHost; } - private: + protected: std::string workerIP; std::string mainIP; }; @@ -53,7 +53,7 @@ class DistTestsFixture class MpiDistTestsFixture : public DistTestsFixture { public: - MpiDistTestsFixture() { SLEEP_MS(INTER_MPI_TEST_SLEEP); } + MpiDistTestsFixture() {} ~MpiDistTestsFixture() = default; @@ -62,17 +62,43 @@ class MpiDistTestsFixture : public DistTestsFixture int worldSize = 4; bool origIsMsgOrderingOn; - // The server has four slots, therefore by setting the number of local slots - // and the world size we are able to infer the expected scheduling decision - void setLocalSlots(int numSlots, int worldSizeIn = 0) + void updateLocalSlots(int newLocalSlots, int newUsedLocalSlots = 0) { - faabric::HostResources res; - res.set_slots(numSlots); - sch.setThisHostResources(res); + faabric::HostResources localRes; + localRes.set_slots(newLocalSlots); + localRes.set_usedslots(newUsedLocalSlots); + sch.setThisHostResources(localRes); + } + + void updateRemoteSlots(int newRemoteSlots, int newRemoteUsedSlots = 0) + { + faabric::HostResources remoteRes; + remoteRes.set_slots(newRemoteSlots); + remoteRes.set_usedslots(newRemoteUsedSlots); + sch.addHostToGlobalSet(workerIP, + std::make_shared(remoteRes)); + } + void setLocalSlots(int numLocalSlots, int worldSizeIn = 0) + { if (worldSizeIn > 0) { worldSize = worldSizeIn; } + int numRemoteSlots = worldSize - numLocalSlots; + + if (numLocalSlots == numRemoteSlots) { + updateLocalSlots(2 * numLocalSlots, numLocalSlots); + updateRemoteSlots(numRemoteSlots); + } else if (numLocalSlots > numRemoteSlots) { + updateLocalSlots(numLocalSlots); + updateRemoteSlots(numRemoteSlots); + } else { + SPDLOG_ERROR( + "Unfeasible MPI world slots config (local: {} - remote: {})", + numLocalSlots, + numRemoteSlots); + throw std::runtime_error("Unfeasible slots configuration"); + } } std::shared_ptr setRequest( @@ -153,7 +179,10 @@ class MpiDistTestsFixture : public DistTestsFixture { faabric::Message& msg = req->mutable_messages()->at(0); faabric::Message result = plannerCli.getMessageResult(msg, timeoutMs); - REQUIRE(result.returnvalue() == 0); + + if (result.returnvalue() != MIGRATED_FUNCTION_RETURN_VALUE) { + REQUIRE(result.returnvalue() == 0); + } SLEEP_MS(1000); auto execGraph = faabric::util::getFunctionExecGraph(msg); checkSchedulingFromExecGraph( diff --git a/tests/dist/mpi/mpi_native.cpp b/tests/dist/mpi/mpi_native.cpp index 19925cb96..9bd0a99f1 100644 --- a/tests/dist/mpi/mpi_native.cpp +++ b/tests/dist/mpi/mpi_native.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -53,10 +54,18 @@ int MPI_Init(int* argc, char*** argv) faabric::Message* call = getExecutingCall(); if (call->mpirank() <= 0) { - SPDLOG_TRACE("MPI - MPI_Init (create)"); + // If we are rank 0 and the world already exists, it means we are being + // migrated + if (getMpiWorldRegistry().worldExists(call->mpiworldid())) { + SPDLOG_TRACE("MPI - MPI_Init (join)"); - int worldId = executingContext.createWorld(*call); - call->set_mpiworldid(worldId); + executingContext.joinWorld(*call); + } else { + SPDLOG_TRACE("MPI - MPI_Init (create)"); + + int worldId = executingContext.createWorld(*call); + call->set_mpiworldid(worldId); + } } else { SPDLOG_TRACE("MPI - MPI_Init (join)"); executingContext.joinWorld(*call); @@ -768,28 +777,27 @@ void mpiMigrationPoint(int entrypointFuncArg) auto& sch = faabric::scheduler::getScheduler(); // Detect if there is a pending migration for the current app - auto pendingMigrations = sch.getPendingAppMigrations(call->appid()); - bool appMustMigrate = pendingMigrations != nullptr; + auto migration = sch.checkForMigrationOpportunities(*call); + bool appMustMigrate = migration != nullptr; // Detect if this particular function needs to be migrated or not bool funcMustMigrate = false; std::string hostToMigrateTo = "otherHost"; if (appMustMigrate) { - for (int i = 0; i < pendingMigrations->migrations_size(); i++) { - auto m = pendingMigrations->mutable_migrations()->at(i); - if (m.msg().id() == call->id()) { - funcMustMigrate = true; - hostToMigrateTo = m.dsthost(); - break; - } - } + funcMustMigrate = migration->srchost() != migration->dsthost(); + hostToMigrateTo = migration->dsthost(); } // Regardless if we have to individually migrate or not, we need to prepare // for the app migration - if (appMustMigrate && call->ismpi()) { - auto& mpiWorld = getMpiWorldRegistry().getWorld(call->mpiworldid()); - mpiWorld.prepareMigration(call->mpirank(), pendingMigrations); + if (appMustMigrate) { + // A migration yields a new distribution, hence a new PTP group + call->set_groupid(migration->groupid()); + + if (call->ismpi()) { + auto& mpiWorld = getMpiWorldRegistry().getWorld(call->mpiworldid()); + mpiWorld.prepareMigration(call->mpirank()); + } } // Do actual migration @@ -802,6 +810,8 @@ void mpiMigrationPoint(int entrypointFuncArg) std::shared_ptr req = faabric::util::batchExecFactory(call->user(), call->function(), 1); req->set_type(faabric::BatchExecuteRequest::MIGRATION); + faabric::util::updateBatchExecAppId(req, migration->appid()); + faabric::util::updateBatchExecGroupId(req, migration->groupid()); faabric::Message& msg = req->mutable_messages()->at(0); msg.set_inputdata(inputData.data(), inputData.size()); @@ -821,9 +831,7 @@ void mpiMigrationPoint(int entrypointFuncArg) ->pushSnapshot(snapKey, snap); msg.set_snapshotkey(snapKey); - // Propagate the id's and indices - msg.set_appid(call->appid()); - msg.set_groupid(call->groupid()); + // Propagate the group idx msg.set_groupidx(call->groupidx()); // If message is MPI, propagate the necessary MPI bits @@ -843,11 +851,8 @@ void mpiMigrationPoint(int entrypointFuncArg) faabric::util::getSystemConfig().endpointHost, hostToMigrateTo); - // Build decision and send - faabric::batch_scheduler::SchedulingDecision decision(msg.appid(), - msg.groupid()); - decision.addMessage(hostToMigrateTo, msg); - sch.callFunctions(req, decision); + faabric::scheduler::getFunctionCallClient(hostToMigrateTo) + ->executeFunctions(req); if (call->recordexecgraph()) { faabric::util::logChainedFunction(*call, msg); @@ -856,6 +861,12 @@ void mpiMigrationPoint(int entrypointFuncArg) // Throw an exception to be caught by the executor and terminate throw faabric::util::FunctionMigratedException("Migrating MPI rank"); } + + // Hit the post-migration hook if not migrated (but someone has) + if (appMustMigrate) { + faabric::transport::getPointToPointBroker().postMigrationHook( + call->groupid(), call->groupidx()); + } } int doAllToAll(int rank, int worldSize, int i) diff --git a/tests/dist/mpi/test_mpi_functions.cpp b/tests/dist/mpi/test_mpi_functions.cpp index 479db0e97..74b022eb5 100644 --- a/tests/dist/mpi/test_mpi_functions.cpp +++ b/tests/dist/mpi/test_mpi_functions.cpp @@ -16,7 +16,7 @@ TEST_CASE_METHOD(MpiDistTestsFixture, "Test MPI all gather", "[mpi]") auto req = setRequest("allgather"); // Call the functions - sch.callFunctions(req); + plannerCli.callFunctions(req); checkAllocationAndResult(req); } @@ -28,7 +28,7 @@ TEST_CASE_METHOD(MpiDistTestsFixture, "Test MPI all reduce", "[mpi]") auto req = setRequest("allreduce"); // Call the functions - sch.callFunctions(req); + plannerCli.callFunctions(req); checkAllocationAndResult(req); } @@ -40,16 +40,14 @@ TEST_CASE_METHOD(MpiDistTestsFixture, "Test MPI all to all", "[.][mpi]") auto req = setRequest("alltoall"); // Call the functions - sch.callFunctions(req); + plannerCli.callFunctions(req); checkAllocationAndResult(req); } -TEST_CASE_METHOD(MpiDistTestsFixture, - "Test MPI all to all many times", - "[.][mpi]") +TEST_CASE_METHOD(MpiDistTestsFixture, "Test MPI all to all many times", "[mpi]") { - int numRuns = 50; + int numRuns = 5; int oldNumLocalSlots = nLocalSlots; nLocalSlots = 4; int worldSize = 8; @@ -60,7 +58,7 @@ TEST_CASE_METHOD(MpiDistTestsFixture, auto req = setRequest("alltoall"); // Call the functions - sch.callFunctions(req); + plannerCli.callFunctions(req); checkAllocationAndResult(req); } @@ -77,7 +75,7 @@ TEST_CASE_METHOD(MpiDistTestsFixture, auto req = setRequest("alltoall-sleep"); // Call the functions - sch.callFunctions(req); + plannerCli.callFunctions(req); // Wait for extra time as the test will sleep for five seconds checkAllocationAndResult(req, 20000); @@ -90,7 +88,7 @@ TEST_CASE_METHOD(MpiDistTestsFixture, "Test MPI barrier", "[mpi]") auto req = setRequest("barrier"); // Call the functions - sch.callFunctions(req); + plannerCli.callFunctions(req); checkAllocationAndResult(req); } @@ -102,7 +100,7 @@ TEST_CASE_METHOD(MpiDistTestsFixture, "Test MPI broadcast", "[mpi]") auto req = setRequest("bcast"); // Call the functions - sch.callFunctions(req); + plannerCli.callFunctions(req); checkAllocationAndResult(req); } @@ -114,7 +112,7 @@ TEST_CASE_METHOD(MpiDistTestsFixture, "Test MPI cart create", "[mpi]") auto req = setRequest("cart-create"); // Call the functions - sch.callFunctions(req); + plannerCli.callFunctions(req); checkAllocationAndResult(req); } @@ -126,7 +124,7 @@ TEST_CASE_METHOD(MpiDistTestsFixture, "Test MPI cartesian", "[mpi]") auto req = setRequest("cartesian"); // Call the functions - sch.callFunctions(req); + plannerCli.callFunctions(req); checkAllocationAndResult(req); } @@ -138,7 +136,7 @@ TEST_CASE_METHOD(MpiDistTestsFixture, "Test MPI checks", "[mpi]") auto req = setRequest("checks"); // Call the functions - sch.callFunctions(req); + plannerCli.callFunctions(req); checkAllocationAndResult(req); } @@ -156,25 +154,50 @@ TEST_CASE_METHOD(MpiDistTestsFixture, "Test MPI function migration", "[mpi]") // Check very often for migration opportunities so that we detect it // right away - msg.set_migrationcheckperiod(1); msg.set_inputdata(std::to_string(NUM_MIGRATION_LOOPS)); // Call the functions - sch.callFunctions(req); + plannerCli.callFunctions(req); - // Sleep for a while to let the scheduler schedule the MPI calls + // Sleep for a while to let the planner schedule the MPI calls SLEEP_MS(500); - // Update the local slots so that a migration opportunity appears - int newLocalSlots = worldSize; - setLocalSlots(newLocalSlots, worldSize); + // Update the slots so that a migration opportunity appears. We update + // either the local or remote worlds to force the migration of one + // half of the ranks or the other one (we also want to keep some used slots + // to prevent the planner reaching an inconsistent state) + bool migratingMainRank; + + SECTION("Migrate main rank") + { + // Make more space remotely, so we migrate the first half of ranks + // (including the main rank) + migratingMainRank = true; + updateRemoteSlots(2 * worldSize, worldSize); + } + + SECTION("Don't migrate main rank") + { + // Make more space locally, so we migrate the second half of ranks + migratingMainRank = false; + updateLocalSlots(2 * worldSize, worldSize); + } // The current function migration approach breaks the execution graph, as // some messages are left dangling (deliberately) without return value std::vector hostsBeforeMigration = { getMasterIP(), getMasterIP(), getWorkerIP(), getWorkerIP() }; - std::vector hostsAfterMigration(worldSize, getMasterIP()); + std::vector hostsAfterMigration; + if (migratingMainRank) { + hostsAfterMigration = { + getWorkerIP(), getWorkerIP(), getWorkerIP(), getWorkerIP() + }; + } else { + hostsAfterMigration = { + getMasterIP(), getMasterIP(), getMasterIP(), getMasterIP() + }; + } checkAllocationAndResultMigration( req, hostsBeforeMigration, hostsAfterMigration, 15000); } @@ -186,7 +209,7 @@ TEST_CASE_METHOD(MpiDistTestsFixture, "Test MPI gather", "[mpi]") auto req = setRequest("gather"); // Call the functions - sch.callFunctions(req); + plannerCli.callFunctions(req); checkAllocationAndResult(req); } @@ -198,7 +221,7 @@ TEST_CASE_METHOD(MpiDistTestsFixture, "Test MPI hello world", "[mpi]") auto req = setRequest("hello-world"); // Call the functions - sch.callFunctions(req); + plannerCli.callFunctions(req); checkAllocationAndResult(req); } @@ -210,7 +233,7 @@ TEST_CASE_METHOD(MpiDistTestsFixture, "Test MPI async. send recv", "[mpi]") auto req = setRequest("isendrecv"); // Call the functions - sch.callFunctions(req); + plannerCli.callFunctions(req); checkAllocationAndResult(req); } @@ -222,7 +245,7 @@ TEST_CASE_METHOD(MpiDistTestsFixture, "Test MPI order", "[mpi]") auto req = setRequest("order"); // Call the functions - sch.callFunctions(req); + plannerCli.callFunctions(req); checkAllocationAndResult(req); } @@ -234,7 +257,7 @@ TEST_CASE_METHOD(MpiDistTestsFixture, "Test MPI reduce", "[mpi]") auto req = setRequest("reduce"); // Call the functions - sch.callFunctions(req); + plannerCli.callFunctions(req); checkAllocationAndResult(req); } @@ -246,7 +269,7 @@ TEST_CASE_METHOD(MpiDistTestsFixture, "Test MPI reduce many times", "[mpi]") auto req = setRequest("reduce-many"); // Call the functions - sch.callFunctions(req); + plannerCli.callFunctions(req); checkAllocationAndResult(req); } @@ -258,7 +281,7 @@ TEST_CASE_METHOD(MpiDistTestsFixture, "Test MPI scan", "[mpi]") auto req = setRequest("scan"); // Call the functions - sch.callFunctions(req); + plannerCli.callFunctions(req); checkAllocationAndResult(req); } @@ -270,7 +293,7 @@ TEST_CASE_METHOD(MpiDistTestsFixture, "Test MPI scatter", "[mpi]") auto req = setRequest("scatter"); // Call the functions - sch.callFunctions(req); + plannerCli.callFunctions(req); checkAllocationAndResult(req); } @@ -282,7 +305,7 @@ TEST_CASE_METHOD(MpiDistTestsFixture, "Test MPI send", "[mpi]") auto req = setRequest("send"); // Call the functions - sch.callFunctions(req); + plannerCli.callFunctions(req); checkAllocationAndResult(req); } @@ -296,7 +319,7 @@ TEST_CASE_METHOD(MpiDistTestsFixture, auto req = setRequest("send-sync-async"); // Call the functions - sch.callFunctions(req); + plannerCli.callFunctions(req); checkAllocationAndResult(req); } @@ -308,7 +331,7 @@ TEST_CASE_METHOD(MpiDistTestsFixture, "Test sending many MPI messages", "[mpi]") auto req = setRequest("send-many"); // Call the functions - sch.callFunctions(req); + plannerCli.callFunctions(req); checkAllocationAndResult(req); } @@ -320,7 +343,7 @@ TEST_CASE_METHOD(MpiDistTestsFixture, "Test MPI send-recv", "[mpi]") auto req = setRequest("sendrecv"); // Call the functions - sch.callFunctions(req); + plannerCli.callFunctions(req); checkAllocationAndResult(req); } @@ -332,7 +355,7 @@ TEST_CASE_METHOD(MpiDistTestsFixture, "Test MPI status", "[mpi]") auto req = setRequest("status"); // Call the functions - sch.callFunctions(req); + plannerCli.callFunctions(req); checkAllocationAndResult(req); } @@ -344,7 +367,7 @@ TEST_CASE_METHOD(MpiDistTestsFixture, "Test MPI types sizes", "[mpi]") auto req = setRequest("typesize"); // Call the functions - sch.callFunctions(req); + plannerCli.callFunctions(req); checkAllocationAndResult(req); } diff --git a/tests/dist/mpi/test_multiple_mpi_worlds.cpp b/tests/dist/mpi/test_multiple_mpi_worlds.cpp index 3773eea61..8077e941a 100644 --- a/tests/dist/mpi/test_multiple_mpi_worlds.cpp +++ b/tests/dist/mpi/test_multiple_mpi_worlds.cpp @@ -21,40 +21,32 @@ TEST_CASE_METHOD(MpiDistTestsFixture, // The first request will schedule two functions on each host setLocalSlots(2, worldSize); - sch.callFunctions(req1); + plannerCli.callFunctions(req1); // Sleep for a bit to allow for the scheduler to schedule all MPI ranks - SLEEP_MS(200); - - // Override the local slots so that the same scheduling decision as before - // is taken - setLocalSlots(2, worldSize); - sch.callFunctions(req2); - - checkAllocationAndResult(req1, 15000); - checkAllocationAndResult(req2, 15000); -} - -TEST_CASE_METHOD(MpiDistTestsFixture, - "Test concurrent MPI applications with different main host", - "[mpi]") -{ - // Prepare the first request (local): 2 ranks locally, 2 remotely - int worldSize = 4; - setLocalSlots(2, worldSize); - auto req1 = setRequest("alltoall-sleep"); - sch.callFunctions(req1); - - // Sleep for a bit to allow for the scheduler to schedule all MPI ranks - SLEEP_MS(200); - - // Prepare the second request (remote): 4 ranks remotely, 2 locally - int newWorldSize = 6; - setLocalSlots(2, newWorldSize); - auto req2 = setRequest("alltoall-sleep"); - // Request remote execution - faabric::scheduler::FunctionCallClient cli(getWorkerIP()); - cli.executeFunctions(req2); + SLEEP_MS(400); + + // Override the local and remote slots so that the same scheduling + // decision as before is taken (2 on each host). Given that both hosts + // have the same number of slots, the planner will pick as main host the + // one with more total slots + bool sameMainHost; + + SECTION("Same main host") + { + sameMainHost = true; + updateLocalSlots(2 * worldSize, 2 * worldSize - 2); + updateRemoteSlots(worldSize, 2); + } + + SECTION("Different main host") + { + sameMainHost = false; + updateLocalSlots(worldSize, 2); + updateRemoteSlots(2 * worldSize, 2 * worldSize - 2); + } + + plannerCli.callFunctions(req2); // Skip the automated exec graph check, and check manually bool skipExecGraphCheck = true; @@ -77,9 +69,14 @@ TEST_CASE_METHOD(MpiDistTestsFixture, // Check exec graph for second request auto execGraph2 = faabric::util::getFunctionExecGraph(req2->mutable_messages()->at(0)); - std::vector expectedHosts2 = { getWorkerIP(), getWorkerIP(), - getMasterIP(), getMasterIP(), - getWorkerIP(), getWorkerIP() }; + std::vector expectedHosts2; + if (sameMainHost) { + expectedHosts2 = expectedHosts1; + } else { + expectedHosts2 = { + getWorkerIP(), getWorkerIP(), getMasterIP(), getMasterIP() + }; + } REQUIRE(expectedHosts2 == faabric::util::getMpiRankHostsFromExecGraph(execGraph2)); } @@ -98,35 +95,114 @@ TEST_CASE_METHOD(MpiDistTestsFixture, auto req1 = setRequest("alltoall-sleep"); auto req2 = setRequest("migration"); auto& msg = req2->mutable_messages()->at(0); - msg.set_migrationcheckperiod(1); msg.set_inputdata(std::to_string(NUM_MIGRATION_LOOPS)); // The first request will schedule two functions on each host - sch.callFunctions(req1); - - // Sleep for a while so that: - // - When we schedule the second application the first one is already - // running - // - When the first application finishes, the function migration thread - // picks up a migration opportunity - // - The previous point happens before the second application has checked - // for migration opportunities internally - SLEEP_MS(5000); - - // The previous three points are likely to be out-of-sync in a GHA test so: - // - We sleep for a very long time (almost the duration of the first app) - // - Even though we don't need it, we overwrite the local slots in case in - // a GHA run we have slept so long that the first application has already - // finished - setLocalSlots(2, worldSize); - sch.callFunctions(req2); + plannerCli.callFunctions(req1); + + // Sleep for a while to let the application make some progress + SLEEP_MS(1000); + + // Update the slots to make space for another app. Same scheduling as + // the first one: two slots locally, two remotely + updateLocalSlots(2 * worldSize, 2 * worldSize - 2); + updateRemoteSlots(worldSize, 2); + + plannerCli.callFunctions(req2); + + // Sleep for a tiny bit for all MPI ranks to begin doing work + SLEEP_MS(200); + + // Update the slots to create a migration opportunity. We migrate two ranks + // from one host to the other, and we test migrating both ways + bool migratingMainRank; + + SECTION("Migrate main rank") + { + // Make more space remotely, so we migrate the first half of ranks + // (including the main rank) + migratingMainRank = true; + updateRemoteSlots(2 * worldSize, worldSize); + } + + SECTION("Don't migrate main rank") + { + // Make more space locally, so we migrate the second half of ranks + migratingMainRank = false; + updateLocalSlots(2 * worldSize, worldSize); + } checkAllocationAndResult(req1, 15000); std::vector hostsBeforeMigration = { getMasterIP(), getMasterIP(), getWorkerIP(), getWorkerIP() }; - std::vector hostsAfterMigration(worldSize, getMasterIP()); + std::vector hostsAfterMigration; + if (migratingMainRank) { + hostsAfterMigration = { + getWorkerIP(), getWorkerIP(), getWorkerIP(), getWorkerIP() + }; + } else { + hostsAfterMigration = { + getMasterIP(), getMasterIP(), getMasterIP(), getMasterIP() + }; + } checkAllocationAndResultMigration( req2, hostsBeforeMigration, hostsAfterMigration, 15000); } + +TEST_CASE_METHOD(MpiDistTestsFixture, + "Test migrating two MPI applications in parallel", + "[mpi]") +{ + // Set the slots for the first request: 2 locally and 2 remote + int worldSize = 4; + setLocalSlots(2, worldSize); + + // Prepare both requests: both will do work and check for migration + // opportunities + auto req1 = setRequest("migration"); + req1->mutable_messages(0)->set_inputdata( + std::to_string(NUM_MIGRATION_LOOPS)); + auto req2 = setRequest("migration"); + req2->mutable_messages(0)->set_inputdata( + std::to_string(NUM_MIGRATION_LOOPS)); + + // The first request will schedule two functions on each host + plannerCli.callFunctions(req1); + + // Sleep for a tiny bit for all MPI ranks to begin doing work + SLEEP_MS(100); + + // Update the slots to make space for another app. Same scheduling as + // the first one: two slots locally, two remotely + updateLocalSlots(2 * worldSize, 2 * worldSize - 2); + updateRemoteSlots(worldSize, 2); + plannerCli.callFunctions(req2); + + // Sleep for a tiny bit for all MPI ranks to begin doing work + SLEEP_MS(100); + + // Update the slots to create two migration opportunities. For each app, we + // migrate two ranks from one host to the other. The first app will + // see that there are two free slots in the remote host (given the update + // below), it will then migrate two ranks, freeing two other ranks in the + // local world. The second app will see those newly freed slots, and use + // them to migrate to. + updateLocalSlots(3 * worldSize, 3 * worldSize); + updateRemoteSlots(2 * worldSize, 2 * worldSize - 2); + + std::vector hostsBeforeMigration1 = { + getMasterIP(), getMasterIP(), getWorkerIP(), getWorkerIP() + }; + std::vector hostsAfterMigration1(worldSize, getWorkerIP()); + checkAllocationAndResultMigration( + req1, hostsBeforeMigration1, hostsAfterMigration1, 15000); + + std::vector hostsBeforeMigration2 = { + getMasterIP(), getMasterIP(), getWorkerIP(), getWorkerIP() + }; + std::vector hostsAfterMigration2(worldSize, getMasterIP()); + checkAllocationAndResultMigration( + req2, hostsBeforeMigration2, hostsAfterMigration2, 15000); +} } diff --git a/tests/dist/scheduler/functions.cpp b/tests/dist/scheduler/functions.cpp index 176e95360..aac7c7e2f 100644 --- a/tests/dist/scheduler/functions.cpp +++ b/tests/dist/scheduler/functions.cpp @@ -101,10 +101,10 @@ int handleFakeDiffsThreadedFunction( auto req = faabric::util::batchExecFactory(msg.user(), msg.function(), nThreads); req->set_type(faabric::BatchExecuteRequest::THREADS); + faabric::util::updateBatchExecAppId(req, msg.appid()); 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))); @@ -229,11 +229,11 @@ int handleReductionFunction(tests::DistTestExecutor* exec, auto req = faabric::util::batchExecFactory( msg.user(), msg.function(), nThreads); req->set_type(faabric::BatchExecuteRequest::THREADS); + faabric::util::updateBatchExecAppId(req, msg.appid()); + + // Set app/ group info for (int i = 0; i < nThreads; i++) { auto& m = req->mutable_messages()->at(i); - - // Set app/ group info - m.set_appid(msg.appid()); m.set_appidx(i); m.set_groupidx(i); } diff --git a/tests/dist/scheduler/test_exec_graph.cpp b/tests/dist/scheduler/test_exec_graph.cpp index 367ba8552..174839ad1 100644 --- a/tests/dist/scheduler/test_exec_graph.cpp +++ b/tests/dist/scheduler/test_exec_graph.cpp @@ -10,17 +10,19 @@ TEST_CASE_METHOD(DistTestsFixture, "Test generating the execution graph", "[funcs]") { - // Set up this host's resources - int nLocalSlots = 2; - int nFuncs = 4; - faabric::HostResources res; - res.set_slots(nLocalSlots); - sch.setThisHostResources(res); - // Retry the test a number of times to catch the race-condition where // we get the execution graph before all results have been published int numRetries = 10; for (int r = 0; r < numRetries; r++) { + // Set up both host's resources + int nLocalSlots = 2; + int nFuncs = 4; + faabric::HostResources res; + res.set_slots(nLocalSlots); + sch.setThisHostResources(res); + sch.addHostToGlobalSet(getWorkerIP(), + std::make_shared(res)); + // Set up the messages std::shared_ptr req = faabric::util::batchExecFactory("funcs", "simple", nFuncs); @@ -35,7 +37,7 @@ TEST_CASE_METHOD(DistTestsFixture, } // Call the functions - sch.callFunctions(req); + plannerCli.callFunctions(req); faabric::Message& m = req->mutable_messages()->at(0); diff --git a/tests/dist/scheduler/test_funcs.cpp b/tests/dist/scheduler/test_funcs.cpp index 1e876ba84..1c38a400b 100644 --- a/tests/dist/scheduler/test_funcs.cpp +++ b/tests/dist/scheduler/test_funcs.cpp @@ -17,15 +17,19 @@ TEST_CASE_METHOD(DistTestsFixture, "Test executing functions on multiple hosts", "[funcs]") { - // Set up this host's resources + std::string thisHost = conf.endpointHost; + std::string otherHost = getWorkerIP(); + + // Set up this host's resources (2 functions locally, 2 remotely) int nLocalSlots = 2; int nFuncs = 4; faabric::HostResources res; - res.set_slots(nLocalSlots); + res.set_slots(nFuncs); + res.set_usedslots(nLocalSlots); sch.setThisHostResources(res); - - std::string thisHost = conf.endpointHost; - std::string otherHost = getWorkerIP(); + res.set_slots(nLocalSlots); + res.set_usedslots(0); + sch.addHostToGlobalSet(otherHost, std::make_shared(res)); // Set up the messages std::shared_ptr req = @@ -41,8 +45,7 @@ TEST_CASE_METHOD(DistTestsFixture, expectedDecision.addMessage(otherHost, req->messages().at(3)); // Call the functions - faabric::batch_scheduler::SchedulingDecision actualDecision = - sch.callFunctions(req); + auto actualDecision = plannerCli.callFunctions(req); // Check decision is as expected checkSchedulingDecisionEquality(actualDecision, expectedDecision); @@ -51,11 +54,11 @@ TEST_CASE_METHOD(DistTestsFixture, for (int i = 0; i < nLocalSlots; i++) { faabric::Message& m = req->mutable_messages()->at(i); - plannerCli.getMessageResult(m, 1000); + auto resultMsg = plannerCli.getMessageResult(m, 1000); std::string expected = fmt::format("Function {} executed on host {}", m.id(), getMasterIP()); - REQUIRE(m.outputdata() == expected); + REQUIRE(resultMsg.outputdata() == expected); } // Check functions executed on the other host diff --git a/tests/dist/scheduler/test_hosts.cpp b/tests/dist/scheduler/test_hosts.cpp index 5d9308bd1..40f963ce8 100644 --- a/tests/dist/scheduler/test_hosts.cpp +++ b/tests/dist/scheduler/test_hosts.cpp @@ -12,7 +12,12 @@ TEST_CASE_METHOD(DistTestsFixture, "Test available hosts", "[scheduler]") auto& sch = faabric::scheduler::getScheduler(); sch.addHostToGlobalSet(); - std::set actual = sch.getAvailableHosts(); + // Check the available hosts + auto availableHosts = plannerCli.getAvailableHosts(); + std::set actual; + for (auto& host : availableHosts) { + actual.insert(host.ip()); + } std::set expected = { getMasterIP(), getWorkerIP() }; REQUIRE(actual == expected); diff --git a/tests/dist/scheduler/test_snapshots.cpp b/tests/dist/scheduler/test_snapshots.cpp index 95d5e5cfa..d44a81aaf 100644 --- a/tests/dist/scheduler/test_snapshots.cpp +++ b/tests/dist/scheduler/test_snapshots.cpp @@ -37,9 +37,7 @@ TEST_CASE_METHOD(DistTestsFixture, msg.set_inputdata(inputData.data(), inputData.size()); // Set up the main thread snapshot - faabric::snapshot::SnapshotRegistry& reg = - faabric::snapshot::getSnapshotRegistry(); - + auto& 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); @@ -47,17 +45,20 @@ TEST_CASE_METHOD(DistTestsFixture, // Force the function to be executed remotely faabric::HostResources res; - res.set_slots(0); + res.set_usedslots(1); + res.set_slots(1); sch.setThisHostResources(res); + res.set_usedslots(0); + res.set_slots(4); + sch.addHostToGlobalSet(getWorkerIP(), std::make_shared(res)); std::vector expectedHosts = { getWorkerIP() }; - faabric::batch_scheduler::SchedulingDecision decision = - sch.callFunctions(req); + auto decision = plannerCli.callFunctions(req); std::vector executedHosts = decision.hosts; REQUIRE(expectedHosts == executedHosts); - int actualResult = sch.awaitThreadResult(msg.id()); - REQUIRE(actualResult == 123); + auto msgResult = plannerCli.getMessageResult(req->appid(), msg.id(), 500); + REQUIRE(msgResult.returnvalue() == 123); // Write the diffs and check they've been applied REQUIRE(snap->getQueuedDiffsCount() == 2); @@ -94,15 +95,18 @@ TEST_CASE_METHOD(DistTestsFixture, 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 + // Executing one function that spawns three threads. We execute one locally + // and the two others remotely faabric::HostResources res; - res.set_slots(1); + res.set_slots(3); + res.set_usedslots(1); sch.setThisHostResources(res); + res.set_slots(2); + res.set_usedslots(0); + sch.addHostToGlobalSet(getWorkerIP(), std::make_shared(res)); std::vector expectedHosts = { getMasterIP() }; - faabric::batch_scheduler::SchedulingDecision decision = - sch.callFunctions(req); + auto decision = plannerCli.callFunctions(req); std::vector executedHosts = decision.hosts; REQUIRE(expectedHosts == executedHosts); @@ -121,14 +125,18 @@ TEST_CASE_METHOD(DistTestsFixture, faabric::util::batchExecFactory(user, function, 1); faabric::Message& msg = req->mutable_messages()->at(0); - // Main function and one thread execute on this host, others on another + // Main function will spawn 4 child threads in a loop. We execute two + // locally and two remotely faabric::HostResources res; res.set_slots(3); + res.set_usedslots(0); sch.setThisHostResources(res); + res.set_usedslots(0); + res.set_slots(2); + sch.addHostToGlobalSet(getWorkerIP(), std::make_shared(res)); std::vector expectedHosts = { getMasterIP() }; - faabric::batch_scheduler::SchedulingDecision decision = - sch.callFunctions(req); + auto decision = plannerCli.callFunctions(req); std::vector executedHosts = decision.hosts; REQUIRE(expectedHosts == executedHosts); diff --git a/tests/dist/scheduler/test_threads.cpp b/tests/dist/scheduler/test_threads.cpp index c1f3c2d1e..0d870f032 100644 --- a/tests/dist/scheduler/test_threads.cpp +++ b/tests/dist/scheduler/test_threads.cpp @@ -25,8 +25,14 @@ TEST_CASE_METHOD(DistTestsFixture, int nLocalSlots = 2; int nThreads = 4; faabric::HostResources res; - res.set_slots(nLocalSlots); + + // Set the resources so that the "main" threads are on the main worker + res.set_usedslots(nLocalSlots); + res.set_slots(nThreads); sch.setThisHostResources(res); + res.set_usedslots(0); + res.set_slots(nLocalSlots); + sch.addHostToGlobalSet(getWorkerIP(), std::make_shared(res)); // Set up the message std::shared_ptr req = @@ -49,20 +55,12 @@ TEST_CASE_METHOD(DistTestsFixture, reg.registerSnapshot(snapshotKey, snap); // Call the functions - sch.callFunctions(req); + plannerCli.callFunctions(req); // Check threads executed on this host - for (int i = 0; i < nLocalSlots; i++) { - faabric::Message& m = req->mutable_messages()->at(i); - int res = sch.awaitThreadResult(m.id()); - REQUIRE(res == m.id() / 2); - } - - // Check threads executed on the other host - for (int i = nLocalSlots; i < nThreads; i++) { - faabric::Message& m = req->mutable_messages()->at(i); - int res = sch.awaitThreadResult(m.id()); - REQUIRE(res == m.id() / 2); + auto results = sch.awaitThreadResults(req); + for (const auto& [mid, res] : results) { + REQUIRE(res == mid / 2); } } } diff --git a/tests/dist/transport/functions.cpp b/tests/dist/transport/functions.cpp index e00a4216e..8c99b05b2 100644 --- a/tests/dist/transport/functions.cpp +++ b/tests/dist/transport/functions.cpp @@ -130,31 +130,33 @@ int handleDistributedLock(tests::DistTestExecutor* exec, if (msg.function() == "lock") { int initialValue = 0; - int groupId = faabric::util::generateGid(); stateKv->set(BYTES(&initialValue)); + // Create a nested request child of the parent request (parent-child + // indicated by same app id) std::shared_ptr nestedReq = faabric::util::batchExecFactory("ptp", "lock-worker", nWorkers); + faabric::util::updateBatchExecAppId(nestedReq, req->appid()); for (int i = 0; i < nWorkers; i++) { faabric::Message& m = nestedReq->mutable_messages()->at(i); - m.set_groupid(groupId); m.set_groupidx(i); } - faabric::scheduler::Scheduler& sch = faabric::scheduler::getScheduler(); - auto decision = sch.callFunctions(nestedReq); + auto& plannerCli = faabric::planner::getPlannerClient(); + auto decision = plannerCli.callFunctions(nestedReq); // Await results bool success = true; for (const auto& msg : nestedReq->messages()) { - faabric::Message res = - faabric::planner::getPlannerClient().getMessageResult(msg, 30000); + faabric::Message res = plannerCli.getMessageResult(msg, 30000); if (res.returnvalue() != 0) { success = false; } } + // Pull to make sure we have the latest version + stateKv->pull(); int finalValue = *(int*)stateKv->get(); int expectedValue = nWorkers * nLoops; if (finalValue != expectedValue) { @@ -256,23 +258,19 @@ class DistributedCoordinationTestRunner auto& m = chainReq->mutable_messages()->at(i); // Set app index and group data - m.set_appid(msg.appid()); m.set_appidx(i); - - m.set_groupid(groupId); m.set_groupidx(i); m.set_groupsize(nChained); m.set_inputdata(msg.inputdata()); } - faabric::scheduler::Scheduler& sch = faabric::scheduler::getScheduler(); - sch.callFunctions(chainReq); + auto& plannerCli = faabric::planner::getPlannerClient(); + plannerCli.callFunctions(chainReq); bool success = true; for (const auto& m : chainReq->messages()) { - faabric::Message result = - faabric::planner::getPlannerClient().getMessageResult(m, 10000); + faabric::Message result = plannerCli.getMessageResult(m, 10000); if (result.returnvalue() != 0) { SPDLOG_ERROR("Distributed coordination check call failed: {}", m.id()); @@ -326,8 +324,6 @@ class DistributedCoordinationTestRunner faabric::state::State& state; std::vector stateKeys; - - int groupId = 123; }; int handleDistributedBarrier(tests::DistTestExecutor* exec, diff --git a/tests/dist/transport/test_coordination.cpp b/tests/dist/transport/test_coordination.cpp index c80bc4ee0..567c87889 100644 --- a/tests/dist/transport/test_coordination.cpp +++ b/tests/dist/transport/test_coordination.cpp @@ -14,17 +14,20 @@ namespace tests { TEST_CASE_METHOD(DistTestsFixture, "Test distributed lock", "[ptp][transport]") { - // Set up this host's resources - int nLocalSlots = 5; + // Set up the host resources. The distributed lock test will start 10 other + // functions (so we need 11 slots). We give each host 8 slots for an even + // distribution + int nSlotsPerHost = 8; faabric::HostResources res; - res.set_slots(nLocalSlots); + res.set_slots(nSlotsPerHost); sch.setThisHostResources(res); + sch.addHostToGlobalSet(getWorkerIP(), std::make_shared(res)); // Set up the request std::shared_ptr req = faabric::util::batchExecFactory("ptp", "lock", 1); - sch.callFunctions(req); + plannerCli.callFunctions(req); faabric::Message& m = req->mutable_messages()->at(0); faabric::Message result = plannerCli.getMessageResult(m, 30000); diff --git a/tests/dist/transport/test_point_to_point.cpp b/tests/dist/transport/test_point_to_point.cpp index 64d5276bd..fbcd82fea 100644 --- a/tests/dist/transport/test_point_to_point.cpp +++ b/tests/dist/transport/test_point_to_point.cpp @@ -19,7 +19,11 @@ class PointToPointDistTestFixture : public DistTestsFixture PointToPointDistTestFixture() { // Check the available hosts - std::set actualAvailable = sch.getAvailableHosts(); + auto availableHosts = plannerCli.getAvailableHosts(); + std::set actualAvailable; + for (auto& host : availableHosts) { + actualAvailable.insert(host.ip()); + } std::set expectedAvailable = { getMasterIP(), getWorkerIP() }; REQUIRE(actualAvailable == expectedAvailable); @@ -29,21 +33,38 @@ class PointToPointDistTestFixture : public DistTestsFixture void setSlotsAndNumFuncs(int nLocalSlotsIn, int nFuncsIn) { + int nRemoteSlots = nFuncsIn - nLocalSlotsIn; nLocalSlots = nLocalSlotsIn; nFuncs = nFuncsIn; - // Set local resources - faabric::HostResources res; - res.set_slots(nLocalSlots); - sch.setThisHostResources(res); + faabric::HostResources localRes; + std::shared_ptr remoteRes = + std::make_shared(); + + if (nLocalSlots == nRemoteSlots) { + localRes.set_slots(2 * nLocalSlots); + localRes.set_usedslots(nLocalSlots); + remoteRes->set_slots(nRemoteSlots); + } else if (nLocalSlots > nRemoteSlots) { + localRes.set_slots(nLocalSlots); + remoteRes->set_slots(nRemoteSlots); + } else { + SPDLOG_ERROR("Unfeasible PTP slots config (local: {} - remote: {})", + nLocalSlots, + nRemoteSlots); + throw std::runtime_error("Unfeasible slots configuration"); + } + + sch.setThisHostResources(localRes); + sch.addHostToGlobalSet(getWorkerIP(), remoteRes); } faabric::batch_scheduler::SchedulingDecision prepareRequestReturnDecision( std::shared_ptr req) { // Prepare expected decision - faabric::batch_scheduler::SchedulingDecision expectedDecision(appId, - groupId); + faabric::batch_scheduler::SchedulingDecision expectedDecision( + req->appid(), req->groupid()); std::vector expectedHosts(nFuncs, getWorkerIP()); for (int i = 0; i < nLocalSlots; i++) { expectedHosts.at(i) = getMasterIP(); @@ -53,9 +74,9 @@ class PointToPointDistTestFixture : public DistTestsFixture for (int i = 0; i < nFuncs; i++) { faabric::Message& msg = req->mutable_messages()->at(i); - msg.set_appid(appId); + msg.set_appid(req->appid()); msg.set_appidx(i); - msg.set_groupid(groupId); + msg.set_groupid(req->groupid()); msg.set_groupidx(i); // Add to expected decision @@ -83,9 +104,6 @@ class PointToPointDistTestFixture : public DistTestsFixture } protected: - int appId = 222; - int groupId = 333; - int nLocalSlots; int nFuncs; }; @@ -94,7 +112,7 @@ TEST_CASE_METHOD(PointToPointDistTestFixture, "Test point-to-point messaging on multiple hosts", "[ptp][transport]") { - setSlotsAndNumFuncs(1, 4); + setSlotsAndNumFuncs(2, 4); // Set up batch request and scheduling decision std::shared_ptr req = @@ -103,8 +121,7 @@ TEST_CASE_METHOD(PointToPointDistTestFixture, prepareRequestReturnDecision(req); // Call the functions - faabric::batch_scheduler::SchedulingDecision actualDecision = - sch.callFunctions(req); + auto actualDecision = plannerCli.callFunctions(req); // Check for equality checkReturnCodesAndSchedulingDecision( @@ -124,8 +141,7 @@ TEST_CASE_METHOD(PointToPointDistTestFixture, prepareRequestReturnDecision(req); // Call the functions - faabric::batch_scheduler::SchedulingDecision actualDecision = - sch.callFunctions(req); + auto actualDecision = plannerCli.callFunctions(req); // Check for equality checkReturnCodesAndSchedulingDecision( @@ -138,11 +154,13 @@ TEST_CASE_METHOD(DistTestsFixture, { // Set up this host's resources, force execution across hosts int nChainedFuncs = 4; - int nLocalSlots = 2; + int nLocalSlots = 3; faabric::HostResources res; res.set_slots(nLocalSlots); sch.setThisHostResources(res); + res.set_slots(2); + sch.addHostToGlobalSet(getWorkerIP(), std::make_shared(res)); std::string function; SECTION("Barrier") { function = "barrier"; } @@ -159,7 +177,8 @@ TEST_CASE_METHOD(DistTestsFixture, // Call the function std::vector expectedHosts = { getMasterIP() }; - std::vector executedHosts = sch.callFunctions(req).hosts; + std::vector executedHosts = + plannerCli.callFunctions(req).hosts; REQUIRE(expectedHosts == executedHosts); // Get result diff --git a/tests/test/mpi/test_mpi_context.cpp b/tests/test/mpi/test_mpi_context.cpp index d115cb87e..d548df28e 100644 --- a/tests/test/mpi/test_mpi_context.cpp +++ b/tests/test/mpi/test_mpi_context.cpp @@ -55,9 +55,16 @@ TEST_CASE_METHOD(MpiBaseTestFixture, "Check default world size is set", "[mpi]") // Set a new world size faabric::util::SystemConfig& conf = faabric::util::getSystemConfig(); int origSize = conf.defaultMpiWorldSize; - int defaultWorldSize = 12; + int defaultWorldSize = 3; conf.defaultMpiWorldSize = defaultWorldSize; + faabric::HostResources res; + res.set_usedslots(1); + res.set_slots(defaultWorldSize * 2); + sch.setThisHostResources(res); + + SLEEP_MS(200); + // Request different sizes int requestedWorldSize; SECTION("Under zero") { requestedWorldSize = -1; } @@ -79,6 +86,8 @@ TEST_CASE_METHOD(MpiBaseTestFixture, "Check default world size is set", "[mpi]") conf.defaultMpiWorldSize = origSize; world.destroy(); + + waitForMpiMessages(req, defaultWorldSize); } TEST_CASE_METHOD(MpiBaseTestFixture, "Check joining world", "[mpi]") @@ -86,18 +95,30 @@ TEST_CASE_METHOD(MpiBaseTestFixture, "Check joining world", "[mpi]") const std::string expectedHost = faabric::util::getSystemConfig().endpointHost; - faabric::Message msgA = faabric::util::messageFactory("mpi", "hellompi"); + auto reqA = faabric::util::batchExecFactory("mpi", "hellompi", 1); + auto& msgA = *reqA->mutable_messages(0); int worldSize = 6; msgA.set_mpiworldsize(worldSize); + msgA.set_recordexecgraph(true); + msgA.set_executedhost(expectedHost); + + // Call the request before creating the MPI world + plannerCli.callFunctions(reqA); // Use one context to create the world MpiContext cA; cA.createWorld(msgA); int worldId = cA.getWorldId(); - // Get one message formed by world creation + waitForMpiMessages(reqA, worldSize); Scheduler& sch = getScheduler(); - faabric::Message msgB = sch.getRecordedMessagesAll().at(0); + // Set the function result to have access to the chained messages + sch.setFunctionResult(msgA); + + auto chainedMsgs = faabric::util::getChainedFunctions(msgA); + REQUIRE(chainedMsgs.size() == worldSize - 1); + auto msgB = + plannerCli.getMessageResult(msgA.appid(), *chainedMsgs.begin(), 500); // Create another context and make sure it's not initialised MpiContext cB; diff --git a/tests/test/mpi/test_mpi_exec_graph.cpp b/tests/test/mpi/test_mpi_exec_graph.cpp index 3feacaccd..fb328aa00 100644 --- a/tests/test/mpi/test_mpi_exec_graph.cpp +++ b/tests/test/mpi/test_mpi_exec_graph.cpp @@ -13,7 +13,7 @@ using namespace faabric::mpi; namespace tests { /* 03/05/2022 - Disable MPI execution graph tests as Protobuf is giving * spurious errors in the distributed tests. See issue #262 - */ + * TEST_CASE_METHOD(MpiTestFixture, "Test tracing the number of MPI messages", "[.]") @@ -215,4 +215,5 @@ TEST_CASE_METHOD(MpiTestFixture, REQUIRE(msg.intexecgraphdetails().count(expectedKey) == 1); REQUIRE(msg.intexecgraphdetails().at(expectedKey) == msgCount); } +*/ } diff --git a/tests/test/mpi/test_mpi_world.cpp b/tests/test/mpi/test_mpi_world.cpp index c9ac35bf6..1609db856 100644 --- a/tests/test/mpi/test_mpi_world.cpp +++ b/tests/test/mpi/test_mpi_world.cpp @@ -21,18 +21,26 @@ TEST_CASE_METHOD(MpiBaseTestFixture, "Test world creation", "[mpi]") // Create the world MpiWorld world; world.create(msg, worldId, worldSize); + msg.set_ismpi(true); + msg.set_mpiworldid(worldId); + msg.set_mpirank(0); REQUIRE(world.getSize() == worldSize); REQUIRE(world.getId() == worldId); REQUIRE(world.getUser() == user); REQUIRE(world.getFunction() == func); + // Wait to make sure all messages are scheduled and dispatched + waitForMpiMessages(req, worldSize); + // Check that chained function calls are made as expected - std::vector actual = sch.getRecordedMessagesAll(); - REQUIRE(actual.size() == worldSize - 1); + std::vector actual = sch.getRecordedMessages(); + REQUIRE(actual.size() == worldSize); - for (int i = 0; i < worldSize - 1; i++) { - faabric::Message actualCall = actual.at(i); + // Check all messages but the first one, which we only modify during + // world creation (and not when we first call it) + for (int i = 1; i < worldSize - 1; i++) { + faabric::Message actualCall = actual.at(i + 1); REQUIRE(actualCall.user() == user); REQUIRE(actualCall.function() == func); REQUIRE(actualCall.ismpi()); @@ -60,8 +68,8 @@ TEST_CASE_METHOD(MpiBaseTestFixture, "Test creating world of size 1", "[mpi]") REQUIRE(world.getUser() == user); REQUIRE(world.getFunction() == func); - // Check no messages are sent - REQUIRE(sch.getRecordedMessagesAll().empty()); + // Check only one message is sent + REQUIRE(sch.getRecordedMessages().size() == worldSize); world.destroy(); } @@ -434,8 +442,9 @@ TEST_CASE_METHOD(MpiTestFixture, "Test recv with partial data", "[mpi]") REQUIRE(status.bytesSize == actualSize * sizeof(int)); } -// 30/12/21 - MPI_Probe is broken after the switch to single-producer, single- -// consumer fixed capacity queues. +/* + * 30/12/21 - MPI_Probe is broken after the switch to single-producer, single- + * consumer fixed capacity queues. TEST_CASE_METHOD(MpiTestFixture, "Test probe", "[.]") { // Send two messages of different sizes @@ -477,6 +486,7 @@ TEST_CASE_METHOD(MpiTestFixture, "Test probe", "[.]") auto* bufferB = bufferBAllocation.get(); world.recv(1, 2, BYTES(bufferB), MPI_INT, sizeB * sizeof(int), nullptr); } +*/ TEST_CASE_METHOD(MpiTestFixture, "Check sending to invalid rank", "[mpi]") { diff --git a/tests/test/mpi/test_multiple_mpi_worlds.cpp b/tests/test/mpi/test_multiple_mpi_worlds.cpp index ef8af00bc..da82690f7 100644 --- a/tests/test/mpi/test_multiple_mpi_worlds.cpp +++ b/tests/test/mpi/test_multiple_mpi_worlds.cpp @@ -12,13 +12,19 @@ class MultiWorldMpiTestFixture : public MpiBaseTestFixture public: MultiWorldMpiTestFixture() { - faabric::Message msgA = faabric::util::messageFactory(userA, funcA); - faabric::Message msgB = faabric::util::messageFactory(userB, funcB); + auto reqA = faabric::util::batchExecFactory(userA, funcA, 1); + auto reqB = faabric::util::batchExecFactory(userB, funcB, 1); + auto& msgA = *reqA->mutable_messages(0); + auto& msgB = *reqB->mutable_messages(0); msgA.set_mpiworldsize(worldSizeA); msgA.set_mpiworldid(worldIdA); msgB.set_mpiworldsize(worldSizeB); msgB.set_mpiworldid(worldIdB); + // Call the requests once to keep track of the first MPI message + plannerCli.callFunctions(reqA); + plannerCli.callFunctions(reqB); + worldA.create(msgA, worldIdA, worldSizeA); worldB.create(msgB, worldIdB, worldSizeB); } @@ -51,9 +57,12 @@ TEST_CASE_METHOD(MpiBaseTestFixture, "Test creating two MPI worlds", "[mpi]") std::string funcA = "funcA"; int worldIdA = 123; int worldSizeA = 3; - auto msgA = faabric::util::messageFactory(userA, funcA); + auto reqA = faabric::util::batchExecFactory(userA, funcA, 1); + auto& msgA = *reqA->mutable_messages(0); + msgA.set_ismpi(true); msgA.set_mpiworldid(worldIdA); msgA.set_mpiworldsize(worldSizeA); + plannerCli.callFunctions(reqA); worldA.create(msgA, worldIdA, worldSizeA); MpiWorld worldB; @@ -61,11 +70,18 @@ TEST_CASE_METHOD(MpiBaseTestFixture, "Test creating two MPI worlds", "[mpi]") std::string funcB = "funcB"; int worldIdB = 245; int worldSizeB = 6; - auto msgB = faabric::util::messageFactory(userB, funcB); + auto reqB = faabric::util::batchExecFactory(userB, funcB, 1); + auto& msgB = *reqB->mutable_messages(0); + msgB.set_ismpi(true); msgB.set_mpiworldid(worldIdB); msgB.set_mpiworldsize(worldSizeB); + plannerCli.callFunctions(reqB); worldB.create(msgB, worldIdB, worldSizeB); + // Wait to make sure all messages are delivered + waitForMpiMessages(reqA, worldSizeA); + waitForMpiMessages(reqB, worldSizeB); + // Check getters on worlds REQUIRE(worldA.getSize() == worldSizeA); REQUIRE(worldA.getId() == worldIdA); @@ -77,29 +93,43 @@ TEST_CASE_METHOD(MpiBaseTestFixture, "Test creating two MPI worlds", "[mpi]") REQUIRE(worldB.getFunction() == funcB); // Check that chained function calls are made as expected - std::vector actual = sch.getRecordedMessagesAll(); - int expectedMsgCount = worldSizeA + worldSizeB - 2; + auto actual = sch.getRecordedMessages(); + // The first recorded message is sent as part of the test fixture, so we + // remove it + actual.erase(actual.begin()); + int expectedMsgCount = worldSizeA + worldSizeB; REQUIRE(actual.size() == expectedMsgCount); - for (int i = 0; i < expectedMsgCount; i++) { - faabric::Message actualCall = actual.at(i); - if (i < worldSizeA - 1) { - REQUIRE(actualCall.user() == userA); - REQUIRE(actualCall.function() == funcA); - REQUIRE(actualCall.ismpi()); - REQUIRE(actualCall.mpiworldid() == worldIdA); - REQUIRE(actualCall.mpirank() == i + 1); - REQUIRE(actualCall.mpiworldsize() == worldSizeA); + // Sort the messages by world and by rank so that we don't have races + // between messages from different ranks + std::vector worldAMsg(worldSizeA); + std::vector worldBMsg(worldSizeB); + for (const auto& msg : actual) { + if (msg.mpiworldid() == worldIdA) { + worldAMsg.at(msg.mpirank()) = msg; } else { - REQUIRE(actualCall.user() == userB); - REQUIRE(actualCall.function() == funcB); - REQUIRE(actualCall.ismpi()); - REQUIRE(actualCall.mpiworldid() == worldIdB); - REQUIRE(actualCall.mpirank() == i + 2 - worldSizeA); - REQUIRE(actualCall.mpiworldsize() == worldSizeB); + worldBMsg.at(msg.mpirank()) = msg; } } + for (int i = 0; i < worldSizeA; i++) { + REQUIRE(worldAMsg.at(i).user() == userA); + REQUIRE(worldAMsg.at(i).function() == funcA); + REQUIRE(worldAMsg.at(i).ismpi()); + REQUIRE(worldAMsg.at(i).mpiworldid() == worldIdA); + REQUIRE(worldAMsg.at(i).mpirank() == i); + REQUIRE(worldAMsg.at(i).mpiworldsize() == worldSizeA); + } + + for (int i = 0; i < worldSizeB; i++) { + REQUIRE(worldBMsg.at(i).user() == userB); + REQUIRE(worldBMsg.at(i).function() == funcB); + REQUIRE(worldBMsg.at(i).ismpi()); + REQUIRE(worldBMsg.at(i).mpiworldid() == worldIdB); + REQUIRE(worldBMsg.at(i).mpirank() == i); + REQUIRE(worldBMsg.at(i).mpiworldsize() == worldSizeB); + } + // Check that this host is registered as the main const std::string actualHostA = worldA.getHostForRank(0); const std::string actualHostB = worldB.getHostForRank(0); diff --git a/tests/test/planner/test_planner_client_server.cpp b/tests/test/planner/test_planner_client_server.cpp index f75ba9a1b..1b159e57c 100644 --- a/tests/test/planner/test_planner_client_server.cpp +++ b/tests/test/planner/test_planner_client_server.cpp @@ -96,9 +96,20 @@ TEST_CASE_METHOD(PlannerClientServerFixture, "[planner]") { faabric::util::setMockMode(true); + auto msgPtr = std::make_shared( faabric::util::messageFactory("foo", "bar")); + // Register a host with the planner so that we can set/get results from it + std::string hostIp = "foo"; + Host thisHost; + thisHost.set_ip(hostIp); + thisHost.set_usedslots(2); + thisHost.set_slots(12); + auto regReq = std::make_shared(); + *regReq->mutable_host() = thisHost; + plannerCli.registerHost(regReq); + // If we try to get the message result before setting it first, nothing // happens (note that we need to pass a 0 timeout to not block) auto resultMsg = @@ -109,6 +120,7 @@ TEST_CASE_METHOD(PlannerClientServerFixture, // from the mocked requests) int expectedReturnValue = 1337; msgPtr->set_returnvalue(expectedReturnValue); + msgPtr->set_executedhost(hostIp); plannerCli.setMessageResult(msgPtr); SLEEP_MS(500); @@ -130,4 +142,87 @@ TEST_CASE_METHOD(PlannerClientServerFixture, // if mocked) faabric::scheduler::getScheduler().shutdown(); } + +class PlannerClientServerExecTestFixture + : public PlannerClientServerFixture + , public FunctionCallClientServerFixture +{ + public: + PlannerClientServerExecTestFixture() + : sch(faabric::scheduler::getScheduler()) + { + sch.shutdown(); + sch.addHostToGlobalSet(); + + std::shared_ptr fac = + std::make_shared(); + faabric::scheduler::setExecutorFactory(fac); + } + + ~PlannerClientServerExecTestFixture() + { + sch.shutdown(); + sch.addHostToGlobalSet(); + } + + protected: + faabric::scheduler::Scheduler& sch; +}; + +TEST_CASE_METHOD(PlannerClientServerExecTestFixture, + "Test executing a batch of functions", + "[planner]") +{ + int nFuncs = 4; + faabric::HostResources res; + res.set_slots(nFuncs); + sch.setThisHostResources(res); + + auto req = faabric::util::batchExecFactory("foo", "bar", nFuncs); + + auto decision = plannerCli.callFunctions(req); + + for (auto mid : decision.messageIds) { + auto resultMsg = plannerCli.getMessageResult(decision.appId, mid, 500); + REQUIRE(resultMsg.returnvalue() == 0); + } +} + +TEST_CASE_METHOD(PlannerClientServerExecTestFixture, + "Test getting the scheduling decision from the planner client", + "[planner]") +{ + int nFuncs = 4; + faabric::HostResources res; + res.set_slots(nFuncs); + sch.setThisHostResources(res); + + auto req = faabric::util::batchExecFactory("foo", "bar", nFuncs); + + auto decision = plannerCli.callFunctions(req); + + bool appExists; + + SECTION("App not registered") + { + appExists = false; + faabric::util::updateBatchExecAppId(req, 1337); + } + + SECTION("App registered") { appExists = true; } + + auto actualDecision = plannerCli.getSchedulingDecision(req); + + if (appExists) { + checkSchedulingDecisionEquality(decision, actualDecision); + } else { + faabric::batch_scheduler::SchedulingDecision emptyDecision(0, 0); + checkSchedulingDecisionEquality(emptyDecision, actualDecision); + } + + for (auto mid : decision.messageIds) { + auto resultMsg = plannerCli.getMessageResult(decision.appId, mid, 500); + REQUIRE(resultMsg.returnvalue() == 0); + } +} } diff --git a/tests/test/planner/test_planner_endpoint.cpp b/tests/test/planner/test_planner_endpoint.cpp index ae9041454..bcfb0117c 100644 --- a/tests/test/planner/test_planner_endpoint.cpp +++ b/tests/test/planner/test_planner_endpoint.cpp @@ -322,7 +322,7 @@ TEST_CASE_METHOD(PlannerEndpointExecTestFixture, msg.set_payloadjson(faabric::util::messageToJson(ber->messages(0))); // Call a function first, and wait for the result - sch.callFunctions(ber); + plannerCli.callFunctions(ber); auto resultMsg = getPlannerClient().getMessageResult(appId, msgId, 1000); SECTION("Success") diff --git a/tests/test/runner/test_main.cpp b/tests/test/runner/test_main.cpp index 193555d50..bce0361e3 100644 --- a/tests/test/runner/test_main.cpp +++ b/tests/test/runner/test_main.cpp @@ -51,15 +51,13 @@ TEST_CASE_METHOD(MainRunnerTestFixture, "Test main runner", "[runner]") req->mutable_messages()->end(), [&msgIds](auto msg) { msgIds.push_back(msg.id()); }); - auto& sch = faabric::scheduler::getScheduler(); - sch.callFunctions(req); + plannerCli.callFunctions(req); for (auto msgId : msgIds) { std::string expected = fmt::format("DummyExecutor executed {}", msgId); faabric::Message res = - faabric::planner::getPlannerClient().getMessageResult( - appId, msgId, SHORT_TEST_TIMEOUT_MS); + plannerCli.getMessageResult(appId, msgId, SHORT_TEST_TIMEOUT_MS); REQUIRE(res.outputdata() == expected); } } diff --git a/tests/test/scheduler/test_executor.cpp b/tests/test/scheduler/test_executor.cpp index 4038b5302..d770fcf44 100644 --- a/tests/test/scheduler/test_executor.cpp +++ b/tests/test/scheduler/test_executor.cpp @@ -9,7 +9,6 @@ #include #include #include -#include #include #include #include @@ -102,24 +101,34 @@ int32_t TestExecutor::executeTask( std::shared_ptr chainedReq = faabric::util::batchExecFactory("dummy", "thread-check", nThreads); + faabric::util::updateBatchExecAppId(chainedReq, reqOrig->appid()); chainedReq->set_type(faabric::BatchExecuteRequest::THREADS); for (int i = 0; i < chainedReq->messages_size(); i++) { faabric::Message& m = chainedReq->mutable_messages()->at(i); - m.set_appid(msg.appid()); m.set_appidx(i + 1); } // Call the threads - std::vector> results = - executeThreads(chainedReq, {}); + if (reqOrig->singlehost()) { + chainedReq->set_singlehost(true); + for (int i = 0; i < chainedReq->messages_size(); i++) { + chainedReq->mutable_messages(i)->set_executedhost( + faabric::util::getSystemConfig().endpointHost); + } + sch.executeBatch(chainedReq); + } else { + auto& plannerCli = faabric::planner::getPlannerClient(); + plannerCli.callFunctions(chainedReq); + } // Await the results - for (auto [mid, result] : results) { - if (result != mid / 100) { + auto results = sch.awaitThreadResults(chainedReq); + for (const auto& [mid, result] : results) { + if (result != (mid / 100)) { SPDLOG_ERROR("TestExecutor got invalid thread result, {} != {}", result, - mid / 100); + msg.id() / 100); return 1; } } @@ -146,10 +155,9 @@ int32_t TestExecutor::executeTask( std::shared_ptr reqOther = faabric::util::batchExecFactory("dummy", "chain-check-b", 1); - Scheduler& sch = getScheduler(); - sch.callFunctions(reqThis); - sch.callFunctions(reqOther); auto& plannerCli = faabric::planner::getPlannerClient(); + plannerCli.callFunctions(reqThis); + plannerCli.callFunctions(reqOther); for (const auto& m : reqThis->messages()) { faabric::Message res = @@ -197,6 +205,7 @@ int32_t TestExecutor::executeTask( } ::memcpy(dummyMemory.get() + offset, data.data(), data.size()); + return 0; } if (msg.function() == "echo") { @@ -226,10 +235,6 @@ int32_t TestExecutor::executeTask( } if (msg.function() == "single-host") { - if (reqOrig->singlehost()) { - return 10; - } - return 20; } @@ -293,6 +298,12 @@ class TestExecutorFixture std::make_shared(); setExecutorFactory(fac); + // Give enough resources for the tests + faabric::HostResources thisResources; + thisResources.set_slots(20); + thisResources.set_usedslots(1); + sch.setThisHostResources(thisResources); + restoreCount = 0; resetCount = 0; } @@ -305,31 +316,28 @@ class TestExecutorFixture MemoryRegion dummyMemory; - std::vector executeWithTestExecutorHint( - std::shared_ptr req, - faabric::batch_scheduler::SchedulingDecision hint) - { - initThreadSnapshot(req); - - conf.overrideCpuCount = 10; - conf.boundTimeout = SHORT_TEST_TIMEOUT_MS; - - return sch.callFunctions(req, hint).hosts; - } - std::vector executeWithTestExecutor( std::shared_ptr req, bool forceLocal) { initThreadSnapshot(req); - conf.overrideCpuCount = 10; - conf.boundTimeout = SHORT_TEST_TIMEOUT_MS; + // If we are indicated to force local execution, we directly call the + // scheduler method if (forceLocal) { - req->mutable_messages()->at(0).set_topologyhint("FORCE_LOCAL"); + // Set the executed host if executing locally (otherwise set in + // the function call server) + req->set_singlehost(true); + for (int i = 0; i < req->messages_size(); i++) { + req->mutable_messages(i)->set_executedhost(conf.endpointHost); + } + + sch.executeBatch(req); + return std::vector(req->messages_size(), + conf.endpointHost); } - return sch.callFunctions(req).hosts; + return plannerCli.callFunctions(req).hosts; } void initThreadSnapshot(std::shared_ptr req) @@ -424,66 +432,56 @@ TEST_CASE_METHOD(TestExecutorFixture, std::string thisHost = conf.endpointHost; std::string otherHost = "other"; + // Set resources + HostResources localHost; + localHost.set_slots(nThreads); + localHost.set_usedslots(nThreads); + sch.setThisHostResources(localHost); + + // Prepare request std::shared_ptr req = faabric::util::batchExecFactory("dummy", "blah", nThreads); req->set_type(faabric::BatchExecuteRequest::THREADS); - int nLocally = 0; - int nRemotely = 0; SECTION("Single host") { expectedRestoreCount = 0; - nLocally = nThreads; + req->set_singlehost(true); } - SECTION("Non-single host") - { - expectedRestoreCount = 1; - nLocally = nThreads - 2; - nRemotely = 2; - } + SECTION("Non-single host") { expectedRestoreCount = 1; } - // Set up a hint to force the scheduler to execute single host or not - faabric::batch_scheduler::SchedulingDecision hint(123, 345); std::vector expectedHosts; - for (int i = 0; i < nLocally; i++) { - expectedHosts.emplace_back(thisHost); - } - - for (int i = 0; i < nRemotely; i++) { - expectedHosts.emplace_back(otherHost); - } - for (int i = 0; i < nThreads; i++) { - hint.addMessage(expectedHosts.at(i), req->messages().at(i)); + expectedHosts.emplace_back(thisHost); } // Turn mock mode on to catch any cross-host messages setMockMode(true); // Execute the functions - std::vector actualHosts = - executeWithTestExecutorHint(req, hint); + initThreadSnapshot(req); + for (int i = 0; i < req->messages_size(); i++) { + req->mutable_messages(i)->set_executedhost(conf.endpointHost); + } + sch.executeBatch(req); + + // Sleep to avoid not finding the messages when we query (otherwise, the + // function call client/server will be mocked and we won't receive the + // callback with the message result) + SLEEP_MS(500); // Await the results on this host - for (int i = 0; i < nLocally; i++) { - uint32_t msgId = req->messages().at(i).id(); - int32_t result = sch.awaitThreadResult(msgId); - REQUIRE(result == msgId / 100); + auto results = sch.awaitThreadResults(req); + for (const auto& [mid, result] : results) { + REQUIRE(result == (mid / 100)); } // Check sent to other host if necessary auto batchRequests = getBatchRequests(); - if (nRemotely > 0) { - REQUIRE(batchRequests.size() == 1); - } // Check the hosts match up - REQUIRE(actualHosts == expectedHosts); REQUIRE(restoreCount == expectedRestoreCount); - - // Turn off mock mode - setMockMode(false); } TEST_CASE_METHOD(TestExecutorFixture, @@ -491,7 +489,6 @@ TEST_CASE_METHOD(TestExecutorFixture, "[executor]") { int nThreads = 0; - SECTION("Overloaded") { nThreads = 100; } SECTION("Underloaded") { nThreads = 10; } @@ -501,6 +498,12 @@ TEST_CASE_METHOD(TestExecutorFixture, conf.noSingleHostOptimisations = 1; } + // Set resources + HostResources localHost; + localHost.set_slots(nThreads); + localHost.set_usedslots(nThreads); + sch.setThisHostResources(localHost); + std::shared_ptr req = faabric::util::batchExecFactory("dummy", "blah", nThreads); req->set_type(faabric::BatchExecuteRequest::THREADS); @@ -509,19 +512,19 @@ TEST_CASE_METHOD(TestExecutorFixture, for (int i = 0; i < nThreads; i++) { faabric::Message& msg = req->mutable_messages()->at(i); msg.set_appidx(i); + msg.set_executedhost(faabric::util::getSystemConfig().endpointHost); messageIds.emplace_back(req->messages().at(i).id()); } - std::vector actualHosts = executeWithTestExecutor(req, false); + std::vector actualHosts = executeWithTestExecutor(req, true); std::vector expectedHosts(nThreads, conf.endpointHost); REQUIRE(actualHosts == expectedHosts); - for (int i = 0; i < nThreads; i++) { - uint32_t msgId = messageIds.at(i); - int32_t result = sch.awaitThreadResult(msgId); - REQUIRE(result == msgId / 100); + auto results = sch.awaitThreadResults(req); + for (const auto& [mid, res] : results) { + REQUIRE(res == mid / 100); } } @@ -532,20 +535,23 @@ TEST_CASE_METHOD(TestExecutorFixture, int nThreads; SECTION("Underloaded") { nThreads = 8; } - SECTION("Overloaded") { nThreads = 100; } - SECTION("Underloaded no single host optimisation") { nThreads = 10; conf.noSingleHostOptimisations = 1; } - std::shared_ptr req = - faabric::util::batchExecFactory("dummy", "thread-check", 1); + // Set resources + HostResources localHost; + localHost.set_slots(nThreads + 1); + localHost.set_usedslots(nThreads + 1); + sch.setThisHostResources(localHost); + + auto req = faabric::util::batchExecFactory("dummy", "thread-check", 1); faabric::Message msg = req->messages(0); - msg.set_inputdata(std::to_string(nThreads)); + req->mutable_messages(0)->set_inputdata(std::to_string(nThreads)); - std::vector actualHosts = executeWithTestExecutor(req, false); + std::vector actualHosts = executeWithTestExecutor(req, true); std::vector expectedHosts = { conf.endpointHost }; REQUIRE(actualHosts == expectedHosts); @@ -560,23 +566,22 @@ TEST_CASE_METHOD(TestExecutorFixture, // We really want to stress things here, but it's quite quick to run, so // don't be afraid to bump up the number of threads int nRepeats = 10; - int nThreads = 1000; - - std::shared_ptr fac = - std::make_shared(); - setExecutorFactory(fac); + int nThreads = 10; - conf.overrideCpuCount = 10; + conf.overrideCpuCount = (nThreads + 1) * nRepeats; conf.boundTimeout = LONG_TEST_TIMEOUT_MS; + faabric::HostResources hostResources; + hostResources.set_slots(conf.overrideCpuCount); + hostResources.set_usedslots(conf.overrideCpuCount); + sch.setThisHostResources(hostResources); for (int i = 0; i < nRepeats; i++) { std::shared_ptr req = faabric::util::batchExecFactory("dummy", "thread-check", 1); faabric::Message msg = req->messages(0); - msg.set_inputdata(std::to_string(nThreads)); + req->mutable_messages(0)->set_inputdata(std::to_string(nThreads)); - std::vector actualHosts = - executeWithTestExecutor(req, false); + auto actualHosts = executeWithTestExecutor(req, true); std::vector expectedHosts = { conf.endpointHost }; REQUIRE(actualHosts == expectedHosts); @@ -584,6 +589,12 @@ TEST_CASE_METHOD(TestExecutorFixture, plannerCli.getMessageResult(msg, LONG_TEST_TIMEOUT_MS); REQUIRE(res.returnvalue() == 0); + for (int mid : faabric::util::getChainedFunctions(msg)) { + auto chainedRes = plannerCli.getMessageResult( + msg.appid(), mid, LONG_TEST_TIMEOUT_MS); + REQUIRE(chainedRes.returnvalue() == 0); + } + sch.reset(); } } @@ -600,6 +611,12 @@ TEST_CASE_METHOD(TestExecutorFixture, faabric::util::batchExecFactory("dummy", "blah", nThreads); req->set_type(faabric::BatchExecuteRequest::THREADS); + // Set resources + HostResources localHost; + localHost.set_slots(nThreads); + localHost.set_usedslots(nThreads); + sch.setThisHostResources(localHost); + std::vector messageIds; for (int i = 0; i < nThreads; i++) { faabric::Message& msg = req->mutable_messages()->at(i); @@ -667,10 +684,17 @@ TEST_CASE_METHOD(TestExecutorFixture, "Test erroring thread", "[executor]") faabric::Message msg = req->messages(0); req->set_type(faabric::BatchExecuteRequest::THREADS); - executeWithTestExecutor(req, false); + // Set resources + HostResources localHost; + localHost.set_slots(10); + localHost.set_usedslots(10); + sch.setThisHostResources(localHost); + executeWithTestExecutor(req, true); - int32_t res = sch.awaitThreadResult(msg.id()); - REQUIRE(res == 1); + auto results = sch.awaitThreadResults(req); + for (const auto& [mid, res] : results) { + REQUIRE(res == 1); + } } TEST_CASE_METHOD(TestExecutorFixture, @@ -696,9 +720,9 @@ TEST_CASE_METHOD(TestExecutorFixture, conf.boundTimeout = SHORT_TEST_TIMEOUT_MS; // Execute all the functions - sch.callFunctions(reqA); - sch.callFunctions(reqB); - sch.callFunctions(reqC); + plannerCli.callFunctions(reqA); + plannerCli.callFunctions(reqB); + plannerCli.callFunctions(reqC); faabric::Message resA1 = plannerCli.getMessageResult(reqA->messages().at(0), SHORT_TEST_TIMEOUT_MS); @@ -764,6 +788,13 @@ TEST_CASE_METHOD(TestExecutorFixture, { auto req = faabric::util::batchExecFactory("foo", "bar"); faabric::Message msg = req->messages(0); + req->mutable_messages(0)->set_executedhost( + faabric::util::getSystemConfig().endpointHost); + + HostResources localHost; + localHost.set_slots(5); + localHost.set_usedslots(5); + sch.setThisHostResources(localHost); std::shared_ptr fac = faabric::scheduler::getExecutorFactory(); @@ -799,6 +830,12 @@ TEST_CASE_METHOD(TestExecutorFixture, { int nThreads = 4; + // Give enough resources for the tests + faabric::HostResources thisResources; + thisResources.set_slots(nThreads); + thisResources.set_usedslots(nThreads); + sch.setThisHostResources(thisResources); + SECTION("XOR diffs") { conf.diffingMode = "xor"; } SECTION("Bytewise diffs") { conf.diffingMode = "bytewise"; } @@ -821,12 +858,15 @@ TEST_CASE_METHOD(TestExecutorFixture, for (int i = 0; i < nThreads; i++) { faabric::Message& msg = req->mutable_messages()->at(i); msg.set_mainhost(otherHost); + msg.set_executedhost(faabric::util::getSystemConfig().endpointHost); msg.set_appidx(i); messageIds.emplace_back(msg.id()); } - executeWithTestExecutor(req, true); + // Execute directly calling the scheduler + initThreadSnapshot(req); + sch.executeBatch(req); // Results aren't set on this host as it's not the main, so we have to // wait @@ -887,6 +927,8 @@ TEST_CASE_METHOD(TestExecutorFixture, REQUIRE(reg.getSnapshot(mainThreadSnapshotKey)->getMergeRegions().empty()); } +/* TODO(thread-opt): currently, we push the full snapshot every time from the + * planner (and not diffs on subsequent executions) TEST_CASE_METHOD(TestExecutorFixture, "Test snapshot diffs pushed to workers after initial snapshot", "[executor]") @@ -930,11 +972,6 @@ TEST_CASE_METHOD(TestExecutorFixture, } } - // Check the other host is registered - std::set expectedRegistered = { otherHost }; - REQUIRE(sch.getFunctionRegisteredHosts(msg.user(), msg.function()) == - expectedRegistered); - // Check snapshot has been pushed auto pushes = faabric::snapshot::getSnapshotPushes(); REQUIRE(pushes.size() == 1); @@ -1004,15 +1041,12 @@ TEST_CASE_METHOD(TestExecutorFixture, expectedDiffs.at(i).getData().size()); } } +*/ TEST_CASE_METHOD(TestExecutorFixture, "Test reset called for functions not threads", "[executor]") { - faabric::util::setMockMode(true); - - conf.overrideCpuCount = 4; - std::string hostOverride = conf.endpointHost; int nMessages = 1; faabric::BatchExecuteRequest::BatchExecuteType requestType = @@ -1053,18 +1087,31 @@ TEST_CASE_METHOD(TestExecutorFixture, std::vector msgIds; for (auto& m : *req->mutable_messages()) { m.set_mainhost(hostOverride); + m.set_executedhost(hostOverride); msgIds.push_back(m.id()); } - // Call functions and force to execute locally - req->mutable_messages()->at(0).set_topologyhint("FORCE_LOCAL"); - sch.callFunctions(req); + // Set resources + HostResources localHost; + localHost.set_slots(20); + localHost.set_usedslots(10); + sch.setThisHostResources(localHost); + + // Call functions + if (requestType == faabric::BatchExecuteRequest::THREADS) { + // For threads, we call directly the scheduler method (instead of + // routing through the planner) to avoid sending/receiving snapshots + initThreadSnapshot(req); + sch.executeBatch(req); + } else { + plannerCli.callFunctions(req); + } // Await execution - for (auto msgId : msgIds) { - if (requestType == faabric::BatchExecuteRequest::THREADS) { - sch.awaitThreadResult(msgId); - } else { + if (requestType == faabric::BatchExecuteRequest::THREADS) { + sch.awaitThreadResults(req); + } else { + for (auto msgId : msgIds) { plannerCli.getMessageResult(appId, msgId, 2000); } } @@ -1086,35 +1133,22 @@ TEST_CASE_METHOD(TestExecutorFixture, int nMessages = singleHosts.size(); std::shared_ptr req = faabric::util::batchExecFactory("dummy", "single-host", nMessages); + int appId = req->messages(0).appid(); std::vector msgIds; - - int expectedResult = 0; - SECTION("Single host") { expectedResult = 10; } - - SECTION("Single host disabled in conf") - { - expectedResult = 20; - conf.noSingleHostOptimisations = 1; - } - - SECTION("Not single host") - { - expectedResult = 20; - singleHosts[1] = otherHost; - singleHosts[2] = otherHost; - } - - faabric::batch_scheduler::SchedulingDecision hint(123, 123); for (int i = 0; i < nMessages; i++) { - hint.addMessage(singleHosts[i], req->messages().at(i)); msgIds.push_back(req->messages(i).id()); } - // Mock mode to avoid requests sent across hosts - setMockMode(true); - executeWithTestExecutorHint(req, hint); - setMockMode(false); + // Give enough resources for the tests + faabric::HostResources thisResources; + thisResources.set_slots(nMessages + 1); + thisResources.set_usedslots(nMessages + 1); + sch.setThisHostResources(thisResources); + + int expectedResult = 20; + + executeWithTestExecutor(req, true); // Await results on this host for (int i = 0; i < nMessages; i++) { @@ -1137,7 +1171,7 @@ TEST_CASE_METHOD(TestExecutorFixture, faabric::util::batchExecFactory("dummy", "context-check", nMessages); int expectedResult = 123; - sch.callFunctions(req); + plannerCli.callFunctions(req); for (int i = 0; i < nMessages; i++) { faabric::Message res = @@ -1290,4 +1324,39 @@ TEST_CASE_METHOD(TestExecutorFixture, exec->reset(firstMsg); REQUIRE(exec->getChainedMessageIds().empty()); } + +TEST_CASE_METHOD(TestExecutorFixture, + "Test execute threads using top-level function in executor", + "[executor]") +{ + int nThreads = 5; + + // Set resources + HostResources localHost; + localHost.set_slots(2 * nThreads); + localHost.set_usedslots(nThreads); + sch.setThisHostResources(localHost); + + // Prepare request + std::shared_ptr req = + faabric::util::batchExecFactory("dummy", "blah", nThreads); + req->set_type(faabric::BatchExecuteRequest::THREADS); + // Set single-host to avoid any snapshot sending + req->set_singlehost(true); + + // Prepare executor + auto exec = std::make_shared(*req->mutable_messages(0)); + + // Execute directly calling the executor + auto results = exec->executeThreads(req, {}); + + // Check results + REQUIRE(results.size() == req->messages_size()); + for (const auto& [mid, res] : results) { + REQUIRE(res == (mid / 100)); + } + + // Shut down executor + exec->shutdown(); +} } diff --git a/tests/test/scheduler/test_executor_reaping.cpp b/tests/test/scheduler/test_executor_reaping.cpp index fe1106588..778ed09d7 100644 --- a/tests/test/scheduler/test_executor_reaping.cpp +++ b/tests/test/scheduler/test_executor_reaping.cpp @@ -13,11 +13,16 @@ namespace tests { class SchedulerReapingTestFixture : public SchedulerFixture + , public FunctionCallClientServerFixture , public ConfFixture { public: SchedulerReapingTestFixture() { + faabric::HostResources res; + res.set_slots(20); + sch.setThisHostResources(res); + std::shared_ptr fac = std::make_shared(); faabric::scheduler::setExecutorFactory(fac); @@ -59,7 +64,8 @@ TEST_CASE_METHOD(SchedulerReapingTestFixture, int nMsgs = 10; auto req = faabric::util::batchExecFactory("foo", "bar", nMsgs); faabric::Message& firstMsg = req->mutable_messages()->at(0); - sch.callFunctions(req); + plannerCli.callFunctions(req); + plannerCli.getMessageResult(firstMsg, 500); // Check executor count REQUIRE(sch.getFunctionExecutorCount(firstMsg) == nMsgs); diff --git a/tests/test/scheduler/test_function_client_server.cpp b/tests/test/scheduler/test_function_client_server.cpp index 00a5f4bf0..ff370048e 100644 --- a/tests/test/scheduler/test_function_client_server.cpp +++ b/tests/test/scheduler/test_function_client_server.cpp @@ -71,20 +71,20 @@ TEST_CASE_METHOD(FunctionClientServerTestFixture, auto msgA = reqA->messages(0); auto reqB = faabric::util::batchExecFactory("dummy", "bar", 1); auto msgB = reqB->messages(0); - sch.callFunctions(reqA); - sch.callFunctions(reqB); + plannerCli.callFunctions(reqA); + plannerCli.callFunctions(reqB); + + // Wait for functions to finish + plannerCli.getMessageResult(msgA, 2000); + plannerCli.getMessageResult(msgB, 2000); // Check messages passed - std::vector msgs = sch.getRecordedMessagesAll(); + std::vector msgs = sch.getRecordedMessages(); REQUIRE(msgs.size() == 2); REQUIRE(msgs.at(0).function() == "foo"); REQUIRE(msgs.at(1).function() == "bar"); sch.clearRecordedMessages(); - // Wait for functions to finish - plannerCli.getMessageResult(msgA, 2000); - plannerCli.getMessageResult(msgB, 2000); - // Check executors present REQUIRE(sch.getFunctionExecutorCount(msgA) == 1); REQUIRE(sch.getFunctionExecutorCount(msgB) == 1); @@ -112,6 +112,16 @@ TEST_CASE_METHOD(FunctionClientServerTestFixture, int nCalls = 30; std::shared_ptr req = faabric::util::batchExecFactory("foo", "bar", nCalls); + for (int i = 0; i < req->messages_size(); i++) { + req->mutable_messages(i)->set_executedhost( + faabric::util::getSystemConfig().endpointHost); + } + + // Set resources (used and normal, as we bypass the planner for scheduling) + HostResources localHost; + localHost.set_slots(nCalls); + localHost.set_usedslots(nCalls); + sch.setThisHostResources(localHost); // Make the request functionCallClient.executeFunctions(req); @@ -121,109 +131,8 @@ TEST_CASE_METHOD(FunctionClientServerTestFixture, plannerCli.getMessageResult(m, 5 * SHORT_TEST_TIMEOUT_MS); } - // Check no other hosts have been registered - faabric::Message m = req->messages().at(0); - REQUIRE(sch.getFunctionRegisteredHostCount(m) == 0); - // Check calls have been registered - REQUIRE(sch.getRecordedMessagesLocal().size() == nCalls); - REQUIRE(sch.getRecordedMessagesShared().empty()); -} - -TEST_CASE_METHOD(FunctionClientServerTestFixture, - "Test get resources request", - "[scheduler]") -{ - int expectedSlots; - int expectedUsedSlots; - - faabric::HostResources originalResources; - originalResources.set_slots(sch.getThisHostResources().slots()); - - SECTION("Override resources") - { - faabric::HostResources res; - - expectedSlots = 10; - expectedUsedSlots = 15; - - res.set_slots(expectedSlots); - res.set_usedslots(expectedUsedSlots); - - sch.setThisHostResources(res); - } - SECTION("Default resources") - { - expectedSlots = sch.getThisHostResources().slots(); - expectedUsedSlots = 0; - } - - // Make the request - faabric::HostResources resResponse = functionCallClient.getResources(); - - REQUIRE(resResponse.slots() == expectedSlots); - REQUIRE(resResponse.usedslots() == expectedUsedSlots); - - // Reset the host resources - sch.setThisHostResources(originalResources); -} - -TEST_CASE_METHOD(FunctionClientServerTestFixture, - "Test unregister request", - "[scheduler]") -{ - faabric::util::setMockMode(true); - std::string otherHost = "other"; - - faabric::HostResources originalResources; - originalResources.set_slots(sch.getThisHostResources().slots()); - - // Remove capacity from this host and add on other - faabric::HostResources thisResources; - faabric::HostResources otherResources; - thisResources.set_slots(0); - otherResources.set_slots(5); - - sch.setThisHostResources(thisResources); - faabric::scheduler::queueResourceResponse(otherHost, otherResources); - - // Request a function and check the other host is registered - auto funcReq = faabric::util::batchExecFactory("foo", "bar", 1); - auto& msg = *funcReq->mutable_messages(0); - sch.addHostToGlobalSet(otherHost); - sch.callFunctions(funcReq); - - REQUIRE(sch.getFunctionRegisteredHostCount(msg) == 1); - - faabric::util::setMockMode(false); - faabric::scheduler::clearMockRequests(); - - // Make the request with a host that's not registered - faabric::UnregisterRequest reqA; - reqA.set_host("foobar"); - reqA.set_user(msg.user()); - reqA.set_function(msg.function()); - - // Check that nothing's happened - functionCallServer.setRequestLatch(); - functionCallClient.unregister(reqA); - functionCallServer.awaitRequestLatch(); - REQUIRE(sch.getFunctionRegisteredHostCount(msg) == 1); - - // Make the request to unregister the actual host - faabric::UnregisterRequest reqB; - reqB.set_host(otherHost); - reqB.set_user(msg.user()); - reqB.set_function(msg.function()); - - functionCallServer.setRequestLatch(); - functionCallClient.unregister(reqB); - functionCallServer.awaitRequestLatch(); - - REQUIRE(sch.getFunctionRegisteredHostCount(msg) == 0); - - sch.setThisHostResources(originalResources); - faabric::scheduler::clearMockRequests(); + REQUIRE(sch.getRecordedMessages().size() == nCalls); } TEST_CASE_METHOD(FunctionClientServerTestFixture, diff --git a/tests/test/scheduler/test_function_migration.cpp b/tests/test/scheduler/test_function_migration.cpp index 66cdb1fe6..2220935a5 100644 --- a/tests/test/scheduler/test_function_migration.cpp +++ b/tests/test/scheduler/test_function_migration.cpp @@ -5,14 +5,15 @@ #include #include -#include #include #include using namespace faabric::scheduler; namespace tests { -class FunctionMigrationTestFixture : public SchedulerFixture +class FunctionMigrationTestFixture + : public SchedulerFixture + , public FunctionCallClientServerFixture { public: FunctionMigrationTestFixture() @@ -24,18 +25,9 @@ class FunctionMigrationTestFixture : public SchedulerFixture setExecutorFactory(fac); } - ~FunctionMigrationTestFixture() - { - faabric::util::setMockMode(false); - - // Remove all hosts from global set - for (const std::string& host : sch.getAvailableHosts()) { - sch.removeHostFromGlobalSet(host); - } - } + ~FunctionMigrationTestFixture() { faabric::util::setMockMode(false); } protected: - FunctionMigrationThread migrationThread; std::string mainHost = faabric::util::getSystemConfig().endpointHost; // Helper method to set the available hosts and slots per host prior to @@ -58,9 +50,9 @@ class FunctionMigrationTestFixture : public SchedulerFixture if (i == 0) { sch.setThisHostResources(resources); } else { - sch.addHostToGlobalSet(registeredHosts.at(i)); - faabric::scheduler::queueResourceResponse(registeredHosts.at(i), - resources); + sch.addHostToGlobalSet( + registeredHosts.at(i), + std::make_shared(resources)); } } } @@ -72,331 +64,89 @@ class FunctionMigrationTestFixture : public SchedulerFixture r.set_usedslots(usedSlots); sch.setThisHostResources(r); } - - std::shared_ptr - buildPendingMigrationsExpectation( - std::shared_ptr req, - std::vector hosts, - std::vector> migrations) - { - faabric::PendingMigrations expected; - expected.set_appid(req->messages().at(0).appid()); - - for (auto pair : migrations) { - auto* migration = expected.add_migrations(); - auto* migrationMsg = migration->mutable_msg(); - *migrationMsg = req->mutable_messages()->at(pair.first); - migration->set_srchost(hosts.at(pair.first)); - migration->set_dsthost(hosts.at(pair.second)); - } - - return std::make_shared(expected); - } - - void checkPendingMigrationsExpectation( - std::shared_ptr expectedMigrations, - std::shared_ptr actualMigrations, - std::vector hosts, - bool skipMsgIdCheck = false) - { - if (expectedMigrations == nullptr) { - REQUIRE(actualMigrations == expectedMigrations); - } else { - // Check actual migration matches expectation - REQUIRE(actualMigrations->appid() == expectedMigrations->appid()); - REQUIRE(actualMigrations->migrations_size() == - expectedMigrations->migrations_size()); - for (int i = 0; i < actualMigrations->migrations_size(); i++) { - auto actual = actualMigrations->mutable_migrations()->at(i); - auto expected = expectedMigrations->mutable_migrations()->at(i); - if (!skipMsgIdCheck) { - REQUIRE(actual.msg().id() == expected.msg().id()); - } - REQUIRE(actual.srchost() == expected.srchost()); - REQUIRE(actual.dsthost() == expected.dsthost()); - } - - // Check we have sent a message to all other hosts with the pending - // migration - auto pendingRequests = getPendingMigrationsRequests(); - REQUIRE(pendingRequests.size() == hosts.size() - 1); - for (auto& pendingReq : getPendingMigrationsRequests()) { - std::string host = pendingReq.first; - std::shared_ptr migration = - pendingReq.second; - auto it = std::find(hosts.begin(), hosts.end(), host); - REQUIRE(it != hosts.end()); - REQUIRE(migration == actualMigrations); - } - } - } }; TEST_CASE_METHOD(FunctionMigrationTestFixture, - "Test starting and stopping the function migration thread", + "Test we can detect migration opportunities", "[scheduler]") -{ - int wakeUpPeriodSeconds = 2; - migrationThread.start(wakeUpPeriodSeconds); - - SLEEP_MS(SHORT_TEST_TIMEOUT_MS); - - migrationThread.stop(); -} - -TEST_CASE_METHOD( - FunctionMigrationTestFixture, - "Test migration opportunities are only detected if set in the message", - "[scheduler]") { // First set resources before calling the functions: one will be allocated // locally, another one in the remote host - std::vector hosts = { mainHost, "hostA" }; + std::vector hosts = { mainHost, LOCALHOST }; std::vector slots = { 1, 1 }; std::vector usedSlots = { 0, 0 }; setHostResources(hosts, slots, usedSlots); // Second, prepare the request we will migrate in-flight. - // NOTE: the sleep function sleeps for a set timeout before returning. auto req = faabric::util::batchExecFactory("foo", "sleep", 2); - int timeToSleep = SHORT_TEST_TIMEOUT_MS; - req->mutable_messages()->at(0).set_inputdata(std::to_string(timeToSleep)); - uint32_t appId = req->messages().at(0).appid(); - uint32_t msgId = req->messages().at(0).id(); - - // Build expected pending migrations - std::shared_ptr expectedMigrations; - SECTION("Migration not enabled") { expectedMigrations = nullptr; } - - SECTION("Migration enabled") - { - // Set to a non-zero value so that migration is enabled - req->mutable_messages()->at(0).set_migrationcheckperiod(2); - - // Build expected migrations - std::vector> migrations = { { 1, 0 } }; - expectedMigrations = - buildPendingMigrationsExpectation(req, hosts, migrations); + for (int i = 0; i < req->messages_size(); i++) { + req->mutable_messages(i)->set_groupidx(i); } - auto decision = sch.callFunctions(req); + auto decision = plannerCli.callFunctions(req); // Update host resources so that a migration opportunity appears, but will // only be detected if migration check period is set. - updateLocalResources(2, 1); - - sch.checkForMigrationOpportunities(); - - auto actualMigrations = sch.getPendingAppMigrations(appId); - checkPendingMigrationsExpectation( - expectedMigrations, actualMigrations, hosts); - - faabric::Message res = - plannerCli.getMessageResult(appId, msgId, 2 * timeToSleep); - REQUIRE(res.returnvalue() == 0); - - // Check that after the result is set, the app can't be migrated no more - sch.checkForMigrationOpportunities(); - REQUIRE(sch.getPendingAppMigrations(appId) == nullptr); -} - -TEST_CASE_METHOD(FunctionMigrationTestFixture, - "Test checking for migration opportunities", - "[scheduler]") -{ - std::vector hosts = { mainHost, "hostA" }; - std::vector slots = { 1, 1 }; - std::vector usedSlots = { 0, 0 }; - setHostResources(hosts, slots, usedSlots); - - auto req = faabric::util::batchExecFactory("foo", "sleep", 2); - int timeToSleep = SHORT_TEST_TIMEOUT_MS; - req->mutable_messages()->at(0).set_inputdata(std::to_string(timeToSleep)); - uint32_t appId = req->messages().at(0).appid(); - uint32_t msgId = req->messages().at(0).id(); - - // By setting the check period to a non-zero value, we are effectively - // opting in to be considered for migration - req->mutable_messages()->at(0).set_migrationcheckperiod(2); - - auto decision = sch.callFunctions(req); - - std::shared_ptr expectedMigrations; - - // As we don't update the available resources, no migration opportunities - // will appear, even though we are checking for them - SECTION("Can not migrate") { expectedMigrations = nullptr; } - - SECTION("Can migrate") + bool mustMigrate; + SECTION("Must migrate") { - // Update host resources so that a migration opportunity appears + mustMigrate = true; updateLocalResources(2, 1); - - // Build expected migrations - std::vector> migrations = { { 1, 0 } }; - expectedMigrations = - buildPendingMigrationsExpectation(req, hosts, migrations); } - sch.checkForMigrationOpportunities(); - - auto actualMigrations = sch.getPendingAppMigrations(appId); - checkPendingMigrationsExpectation( - expectedMigrations, actualMigrations, hosts); - - faabric::Message res = - plannerCli.getMessageResult(appId, msgId, 2 * timeToSleep); - REQUIRE(res.returnvalue() == 0); - - // Check that after the result is set, the app can't be migrated no more - sch.checkForMigrationOpportunities(); - REQUIRE(sch.getPendingAppMigrations(appId) == nullptr); -} - -TEST_CASE_METHOD( - FunctionMigrationTestFixture, - "Test detecting migration opportunities for several messages and hosts", - "[scheduler]") -{ - // First set resources before calling the functions: one request will be - // allocated to each host - std::vector hosts = { mainHost, "hostA", "hostB", "hostC" }; - std::vector slots = { 1, 1, 1, 1 }; - std::vector usedSlots = { 0, 0, 0, 0 }; - setHostResources(hosts, slots, usedSlots); - - auto req = faabric::util::batchExecFactory("foo", "sleep", 4); - int timeToSleep = SHORT_TEST_TIMEOUT_MS; - req->mutable_messages()->at(0).set_inputdata(std::to_string(timeToSleep)); - uint32_t appId = req->messages().at(0).appid(); - uint32_t msgId = req->messages().at(0).id(); - - // Opt in to be considered for migration - req->mutable_messages()->at(0).set_migrationcheckperiod(2); - - auto decision = sch.callFunctions(req); - - // Set up expectations - std::shared_ptr expectedMigrations; - SECTION("Can not migrate") { expectedMigrations = nullptr; } - - SECTION("Can migrate") - { - // Update host resources so that two migration opportunities appear in - // different hosts. - std::vector newSlots = { 2, 2, 1, 1 }; - std::vector newUsedSlots = { 1, 1, 1, 1 }; - setHostResources(hosts, newSlots, newUsedSlots); - - // Build expected result: two migrations - std::vector> migrations = { { 3, 0 }, { 2, 1 } }; - expectedMigrations = - buildPendingMigrationsExpectation(req, hosts, migrations); + SECTION("Must not migrate") { mustMigrate = false; } + + // The group leader (message with index 0) will detect the migration, but + // does not have to migrate + auto migration0 = + sch.checkForMigrationOpportunities(*req->mutable_messages(0)); + if (mustMigrate) { + REQUIRE(migration0 != nullptr); + // App id is the same, but group id has changed as the distribution has + // changed + REQUIRE(migration0->appid() == decision.appId); + REQUIRE(migration0->groupid() != decision.groupId); + // Group idx 0 does not have to migrate + REQUIRE(migration0->groupidx() == 0); + REQUIRE(migration0->srchost() == migration0->dsthost()); + REQUIRE(decision.hosts.at(0) == migration0->dsthost()); + + // Group idx 1 must migrate. Note that we manually set the new group + // id, we only have to do this in the tests + auto migration1 = sch.checkForMigrationOpportunities( + *req->mutable_messages(1), migration0->groupid()); + REQUIRE(migration1->appid() == decision.appId); + REQUIRE(migration1->groupid() != decision.groupId); + // Group idx 0 does not have to migrate + REQUIRE(migration1->groupidx() == 1); + REQUIRE(migration1->dsthost() != decision.hosts.at(1)); + REQUIRE(migration1->dsthost() == mainHost); + } else { + REQUIRE(migration0 == nullptr); + auto migration1 = sch.checkForMigrationOpportunities( + *req->mutable_messages(1), decision.groupId); + REQUIRE(migration1 == nullptr); } - sch.checkForMigrationOpportunities(); - - auto actualMigrations = sch.getPendingAppMigrations(appId); - checkPendingMigrationsExpectation( - expectedMigrations, actualMigrations, hosts); - - faabric::Message res = - plannerCli.getMessageResult(appId, msgId, 2 * timeToSleep); - REQUIRE(res.returnvalue() == 0); - - // Check that after the result is set, the app can't be migrated no more - sch.checkForMigrationOpportunities(); - REQUIRE(sch.getPendingAppMigrations(appId) == nullptr); -} - -// TODO(flaky): fix test -/* -TEST_CASE_METHOD( - FunctionMigrationTestFixture, - "Test function migration thread detects migration opportunities", - "[scheduler][.]") -{ - std::vector hosts = { mainHost, "hostA" }; - std::vector slots = { 1, 1 }; - std::vector usedSlots = { 0, 0 }; - setHostResources(hosts, slots, usedSlots); - - auto req = faabric::util::batchExecFactory("foo", "sleep", 2); - int checkPeriodSecs = 1; - int timeToSleep = 4 * checkPeriodSecs * 1000; - req->mutable_messages()->at(0).set_inputdata(std::to_string(timeToSleep)); - uint32_t appId = req->messages().at(0).appid(); - uint32_t msgId = req->messages().at(0).id(); - - // Opt in to be migrated - req->mutable_messages()->at(0).set_migrationcheckperiod(checkPeriodSecs); - - auto decision = sch.callFunctions(req); - - std::shared_ptr expectedMigrations; - - // As we don't update the available resources, no migration opportunities - // will appear, even though we are checking for them - SECTION("Can not migrate") { expectedMigrations = nullptr; } - - SECTION("Can migrate") - { - // Update host resources so that a migration opportunity appears - updateLocalResources(2, 1); - - // Build expected migrations - std::vector> migrations = { { 1, 0 } }; - expectedMigrations = - buildPendingMigrationsExpectation(req, hosts, migrations); + // Set the executed host for correct accounting in the planner when + // setting the function result + req->mutable_messages(0)->set_executedhost(mainHost); + if (mustMigrate) { + req->mutable_messages(1)->set_executedhost(mainHost); + } else { + req->mutable_messages(1)->set_executedhost(LOCALHOST); } - // Instead of directly calling the scheduler function to check for migration - // opportunites, sleep for enough time (twice the check period) so that a - // migration is detected by the background thread. - SLEEP_MS(2 * checkPeriodSecs * 1000); - - auto actualMigrations = sch.getPendingAppMigrations(appId); - checkPendingMigrationsExpectation( - expectedMigrations, actualMigrations, hosts); - - faabric::Message res = sch.getFunctionResult(appId, msgId, 2 * timeToSleep); - REQUIRE(res.returnvalue() == 0); - - SLEEP_MS(100); - - // Check that after the result is set, the app can't be migrated no more - sch.checkForMigrationOpportunities(); - REQUIRE(sch.getPendingAppMigrations(appId) == nullptr); -} -*/ - -TEST_CASE_METHOD(FunctionMigrationTestFixture, - "Test adding and removing pending migrations manually", - "[scheduler]") -{ - auto req = faabric::util::batchExecFactory("foo", "sleep", 2); - uint32_t appId = req->messages().at(0).appid(); - std::vector hosts = { mainHost, "hostA" }; - std::vector> migrations = { { 1, 0 } }; - auto expectedMigrations = - buildPendingMigrationsExpectation(req, hosts, migrations); - - // Add migration manually - REQUIRE(sch.getPendingAppMigrations(appId) == nullptr); - sch.addPendingMigration(expectedMigrations); - REQUIRE(sch.getPendingAppMigrations(appId) == expectedMigrations); - - // Remove migration manually - sch.removePendingMigration(appId); - REQUIRE(sch.getPendingAppMigrations(appId) == nullptr); + sch.setFunctionResult(*req->mutable_messages(0)); + sch.setFunctionResult(*req->mutable_messages(1)); } TEST_CASE_METHOD(FunctionMigrationTestFixture, - "Test MPI function migration points", + "Test MPI migration opportunities", "[scheduler]") { // Set up host resources - std::vector hosts = { mainHost, "hostA" }; + std::vector hosts = { mainHost, LOCALHOST }; std::vector slots = { 2, 2 }; std::vector usedSlots = { 0, 0 }; setHostResources(hosts, slots, usedSlots); @@ -416,57 +166,74 @@ TEST_CASE_METHOD(FunctionMigrationTestFixture, firstMsg->set_ismpi(true); firstMsg->set_mpiworldsize(worldSize); firstMsg->set_mpiworldid(worldId); - firstMsg->set_migrationcheckperiod(checkPeriodSecs); - uint32_t appId = req->messages().at(0).appid(); - uint32_t msgId = req->messages().at(0).id(); // Call function that wil just sleep - auto decision = sch.callFunctions(req); + auto decision = plannerCli.callFunctions(req); // Manually create the world, and trigger a second function invocation in // the remote host faabric::mpi::MpiWorld world; - // Note that we deliberately pass a copy of the message. The `world.create` - // method modifies the passed message, which can race with the thread pool - // thread executing the message. Note that, normally, the thread pool - // thread _would_ be calling world.create itself, thus not racing - auto firstMsgCopy = req->messages(0); - world.create(firstMsgCopy, worldId, worldSize); + world.create(*firstMsg, worldId, worldSize); + + // TODO: check the original MPI scheduling + + // Get the group ID _after_ we create the world (it is only assigned then) + int appId = firstMsg->appid(); + int groupId = firstMsg->groupid(); // Update host resources so that a migration opportunity appears - updateLocalResources(4, 2); - - // Build expected migrations - std::shared_ptr expectedMigrations; - - // We need to add to the original request the ones that will be - // chained by MPI (this is only needed to build the expectation). - // NOTE: we do it in a copy of the original request, as otherwise TSAN - // complains about a data race. - auto reqCopy = faabric::util::batchExecFactory("mpi", "sleep", worldSize); - for (int i = 0; i < worldSize; i++) { - reqCopy->mutable_messages(i)->set_appid(firstMsg->appid()); + bool mustMigrate; + SECTION("Must migrate") + { + mustMigrate = true; + updateLocalResources(4, 2); + } + + SECTION("Must not migrate") { mustMigrate = false; } + + // The group leader (message with index 0) will detect the migration, but + // does not have to migrate + auto migration0 = + sch.checkForMigrationOpportunities(*req->mutable_messages(0)); + int newGroupId; + if (mustMigrate) { + REQUIRE(migration0 != nullptr); + // App id is the same, but group id has changed as the distribution has + // changed + REQUIRE(migration0->appid() == decision.appId); + REQUIRE(migration0->groupid() != decision.groupId); + // Group idx 0 does not have to migrate + REQUIRE(migration0->groupidx() == 0); + REQUIRE(migration0->srchost() == migration0->dsthost()); + REQUIRE(decision.hosts.at(0) == migration0->dsthost()); + newGroupId = migration0->groupid(); + } else { + REQUIRE(migration0 == nullptr); + newGroupId = groupId; } - std::vector> migrations = { { 1, 0 }, { 1, 0 } }; - expectedMigrations = - buildPendingMigrationsExpectation(reqCopy, hosts, migrations); - - // Instead of directly calling the scheduler function to check for migration - // opportunites, sleep for enough time (twice the check period) so that a - // migration is detected by the background thread. - SLEEP_MS(2 * checkPeriodSecs * 1000); - - // When checking that a migration has taken place in MPI, we skip the msg - // id check. Part of the request is built by the runtime, and therefore - // we don't have access to the actual messages scheduled. - auto actualMigrations = sch.getPendingAppMigrations(appId); - checkPendingMigrationsExpectation( - expectedMigrations, actualMigrations, hosts, true); - - faabric::Message res = - plannerCli.getMessageResult(appId, msgId, 2 * timeToSleep); - REQUIRE(res.returnvalue() == 0); + // Ideally, we checkMigrationOpportunities with each thread's message. + // Unfortunately, for MPI, the ranks >= 1 messages' are created + // automatically, as part of a scale change. Thus, in the tests we fake + // these messages (we just need to set the app id, group id, and group idx) + for (int i = 1; i < worldSize; i++) { + faabric::Message msg; + msg.set_appid(appId); + msg.set_groupid(groupId); + msg.set_groupidx(i); + auto expectedMigration = + sch.checkForMigrationOpportunities(msg, newGroupId); + if (mustMigrate) { + REQUIRE(expectedMigration != nullptr); + REQUIRE(expectedMigration->appid() == appId); + REQUIRE(expectedMigration->groupid() != groupId); + REQUIRE(expectedMigration->groupidx() == i); + // All ranks migrate to the mainHost + REQUIRE(expectedMigration->dsthost() == mainHost); + } else { + REQUIRE(expectedMigration == nullptr); + } + } // Clean up world.destroy(); diff --git a/tests/test/scheduler/test_scheduler.cpp b/tests/test/scheduler/test_scheduler.cpp index 4621956d4..d83a3888b 100644 --- a/tests/test/scheduler/test_scheduler.cpp +++ b/tests/test/scheduler/test_scheduler.cpp @@ -65,6 +65,11 @@ class SlowExecutor final : public Executor private: faabric::util::MemoryRegion dummyMemory = nullptr; size_t dummyMemorySize = 2 * faabric::util::HOST_PAGE_SIZE; + + void setMemorySize(size_t newSize) override + { + SPDLOG_DEBUG("Setting dummy memory size"); + } }; class SlowExecutorFactory : public ExecutorFactory @@ -124,13 +129,7 @@ TEST_CASE_METHOD(SlowExecutorTestFixture, "Test scheduler clear-up", "[scheduler]") { - faabric::util::setMockMode(true); - std::string thisHost = conf.endpointHost; - std::string otherHost = "other"; - std::set expectedHosts = { otherHost }; - - sch.addHostToGlobalSet(otherHost); // Set resources int nCores = 5; @@ -138,75 +137,24 @@ TEST_CASE_METHOD(SlowExecutorTestFixture, res.set_slots(nCores); sch.setThisHostResources(res); - // Set resources for other host too - faabric::scheduler::queueResourceResponse(otherHost, res); - // Set request - int nCalls = nCores + 1; + int nCalls = nCores; auto req = faabric::util::batchExecFactory("blah", "foo", nCalls); auto msg = req->messages(0); // Initial checks REQUIRE(sch.getFunctionExecutorCount(msg) == 0); - REQUIRE(sch.getFunctionRegisteredHostCount(msg) == 0); - REQUIRE(sch.getFunctionRegisteredHosts(msg.user(), msg.function()).empty()); - - faabric::HostResources resCheck = sch.getThisHostResources(); - REQUIRE(resCheck.slots() == nCores); - REQUIRE(resCheck.usedslots() == 0); - REQUIRE(sch.getThisHostResources().slots() == nCores); // Make calls with one extra that should be sent to the other host - sch.callFunctions(req); + plannerCli.callFunctions(req); + plannerCli.getMessageResult(msg, 500); REQUIRE(sch.getFunctionExecutorCount(msg) == nCores); - REQUIRE(sch.getFunctionRegisteredHostCount(msg) == 1); - REQUIRE(sch.getFunctionRegisteredHosts(msg.user(), msg.function()) == - expectedHosts); - - resCheck = sch.getThisHostResources(); - REQUIRE(resCheck.slots() == nCores); - REQUIRE(resCheck.usedslots() == nCores); sch.reset(); // Check scheduler has been cleared REQUIRE(sch.getFunctionExecutorCount(msg) == 0); - REQUIRE(sch.getFunctionRegisteredHostCount(msg) == 0); - REQUIRE(sch.getFunctionRegisteredHosts(msg.user(), msg.function()).empty()); - - resCheck = sch.getThisHostResources(); - int actualCores = faabric::util::getUsableCores(); - REQUIRE(resCheck.slots() == actualCores); - REQUIRE(resCheck.usedslots() == 0); -} - -TEST_CASE_METHOD(SlowExecutorTestFixture, - "Test scheduler available hosts", - "[scheduler]") -{ - // Set up some available hosts - std::string thisHost = faabric::util::getSystemConfig().endpointHost; - std::string hostA = "hostA"; - std::string hostB = "hostB"; - std::string hostC = "hostC"; - - sch.addHostToGlobalSet(hostA); - sch.addHostToGlobalSet(hostB); - sch.addHostToGlobalSet(hostC); - - std::set expectedHosts = { thisHost, hostA, hostB, hostC }; - std::set actualHosts = sch.getAvailableHosts(); - - REQUIRE(actualHosts == expectedHosts); - - sch.removeHostFromGlobalSet(hostB); - sch.removeHostFromGlobalSet(hostC); - - expectedHosts = { thisHost, hostA }; - actualHosts = sch.getAvailableHosts(); - - REQUIRE(actualHosts == expectedHosts); } TEST_CASE_METHOD(SlowExecutorTestFixture, @@ -218,12 +166,13 @@ TEST_CASE_METHOD(SlowExecutorTestFixture, int32_t expectedSubType; std::string expectedContextData; - int thisCores = 5; - faabric::util::SystemConfig& conf = faabric::util::getSystemConfig(); - conf.overrideCpuCount = thisCores; + int thisCores = 20; + faabric::HostResources thisResources; + thisResources.set_slots(thisCores); + sch.setThisHostResources(thisResources); - int nCallsOne = 10; - int nCallsTwo = 20; + int nCallsOne = 5; + int nCallsTwo = 10; std::shared_ptr reqOne = faabric::util::batchExecFactory("foo", "bar", nCallsOne); @@ -264,28 +213,8 @@ TEST_CASE_METHOD(SlowExecutorTestFixture, bool isThreads = execMode == faabric::BatchExecuteRequest::THREADS; - // Mock everything - faabric::util::setMockMode(true); - std::string thisHost = conf.endpointHost; - // Set up another host - std::string otherHost = "beta"; - sch.addHostToGlobalSet(otherHost); - - int otherCores = 15; - int nCallsOffloadedOne = nCallsOne - thisCores; - - faabric::HostResources thisResources; - thisResources.set_slots(thisCores); - - faabric::HostResources otherResources; - otherResources.set_slots(otherCores); - - // Prepare resource response for other host - sch.setThisHostResources(thisResources); - faabric::scheduler::queueResourceResponse(otherHost, otherResources); - // Set up the messages std::vector reqOneMsgIds; faabric::batch_scheduler::SchedulingDecision expectedDecisionOne( @@ -301,9 +230,7 @@ TEST_CASE_METHOD(SlowExecutorTestFixture, // Set app index msg.set_appidx(i); - // Expect this host to handle up to its number of cores - std::string host = i < thisCores ? thisHost : otherHost; - expectedDecisionOne.addMessage(host, msg); + expectedDecisionOne.addMessage(thisHost, msg); reqOneMsgIds.push_back(msg.id()); } @@ -313,96 +240,55 @@ TEST_CASE_METHOD(SlowExecutorTestFixture, reqOne->set_subtype(expectedSubType); reqOne->set_contextdata(expectedContextData); - faabric::batch_scheduler::SchedulingDecision actualDecisionOne = - sch.callFunctions(reqOne); + // Set the singlehost flag to avoid sending snapshots to the planner + reqOne->set_singlehost(true); + + auto actualDecisionOne = plannerCli.callFunctions(reqOne); // Check decision is as expected checkSchedulingDecisionEquality(actualDecisionOne, expectedDecisionOne); // Await the results - for (int i = 0; i < thisCores; i++) { - if (isThreads) { - sch.awaitThreadResult(reqOneMsgIds.at(i)); - } else { - plannerCli.getMessageResult(appId, reqOneMsgIds.at(i), 10000); - } - } - - // Check resource requests have been made to other host - auto resRequestsOne = faabric::scheduler::getResourceRequests(); - REQUIRE(resRequestsOne.size() == 1); - REQUIRE(resRequestsOne.at(0).first == otherHost); - - // Check snapshots have been pushed - auto snapshotPushes = faabric::snapshot::getSnapshotPushes(); - if (expectedSnapshot.empty()) { - REQUIRE(snapshotPushes.empty()); + if (isThreads) { + sch.awaitThreadResults(reqOne); } else { - REQUIRE(snapshotPushes.size() == 1); - - 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()); + for (int i = 0; i < nCallsOne; i++) { + plannerCli.getMessageResult(appId, reqOneMsgIds.at(i), 500); + } } // Check the executor counts on this host faabric::Message m = reqOne->messages().at(0); - faabric::HostResources res = sch.getThisHostResources(); if (isThreads) { // For threads we expect only one executor REQUIRE(sch.getFunctionExecutorCount(m) == 1); } else { // For functions we expect one per core - REQUIRE(sch.getFunctionExecutorCount(m) == thisCores); + REQUIRE(sch.getFunctionExecutorCount(m) == nCallsOne); } - REQUIRE(res.slots() == thisCores); - REQUIRE(res.usedslots() == 0); - // Check the number of messages executed locally and remotely - REQUIRE(sch.getRecordedMessagesLocal().size() == thisCores); - REQUIRE(sch.getRecordedMessagesShared().size() == nCallsOffloadedOne); - - // Check the message is dispatched to the other host - auto batchRequestsOne = faabric::scheduler::getBatchRequests(); - REQUIRE(batchRequestsOne.size() == 1); - - auto batchRequestOne = batchRequestsOne.at(0); - REQUIRE(batchRequestOne.first == otherHost); - REQUIRE(batchRequestOne.second->messages_size() == nCallsOffloadedOne); - REQUIRE(batchRequestOne.second->type() == execMode); - REQUIRE(batchRequestOne.second->subtype() == expectedSubType); - REQUIRE(batchRequestOne.second->contextdata() == expectedContextData); - - // Clear mocks - faabric::scheduler::clearMockRequests(); - - // Set up resource response again - faabric::scheduler::queueResourceResponse(otherHost, otherResources); + REQUIRE(sch.getRecordedMessages().size() == nCallsOne); // Now schedule a second batch and check the decision std::shared_ptr reqTwo = faabric::util::batchExecFactory("foo", "bar", nCallsTwo); + int appId2 = reqTwo->appid(); std::vector reqTwoMsgIds; const faabric::Message& firstMsg2 = reqTwo->messages().at(0); faabric::batch_scheduler::SchedulingDecision expectedDecisionTwo( - appId, firstMsg2.groupid()); + appId2, firstMsg2.groupid()); for (int i = 0; i < nCallsTwo; i++) { faabric::Message& msg = reqTwo->mutable_messages()->at(i); - 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); + expectedDecisionTwo.addMessage(thisHost, msg); reqTwoMsgIds.push_back(msg.id()); } @@ -410,187 +296,32 @@ TEST_CASE_METHOD(SlowExecutorTestFixture, // Create the batch request reqTwo->set_type(execMode); + // Set the singlehost flag to avoid sending snapshots to the planner + reqTwo->set_singlehost(true); + // Schedule the functions - faabric::batch_scheduler::SchedulingDecision actualDecisionTwo = - sch.callFunctions(reqTwo); + auto actualDecisionTwo = plannerCli.callFunctions(reqTwo); // Check scheduling decision checkSchedulingDecisionEquality(actualDecisionTwo, expectedDecisionTwo); // Await the results - for (int i = 0; i < thisCores; i++) { - if (isThreads) { - sch.awaitThreadResult(reqTwoMsgIds.at(i)); - } else { - plannerCli.getMessageResult(appId, reqTwoMsgIds.at(i), 10000); + if (isThreads) { + sch.awaitThreadResults(reqTwo); + } else { + for (int i = 0; i < nCallsTwo; i++) { + plannerCli.getMessageResult(appId2, reqTwoMsgIds.at(i), 10000); } } - // Check resource request made again - auto resRequestsTwo = faabric::scheduler::getResourceRequests(); - REQUIRE(resRequestsTwo.size() == 1); - REQUIRE(resRequestsTwo.at(0).first == otherHost); - // Check no other functions have been scheduled on this host - REQUIRE(sch.getRecordedMessagesLocal().size() == (2 * thisCores)); - REQUIRE(sch.getRecordedMessagesShared().size() == - (nCallsOne + nCallsTwo) - (2 * thisCores)); + REQUIRE(sch.getRecordedMessages().size() == nCallsOne + nCallsTwo); if (isThreads) { REQUIRE(sch.getFunctionExecutorCount(m) == 1); } else { - REQUIRE(sch.getFunctionExecutorCount(m) == thisCores); - } - - // Check the second message is dispatched to the other host - auto batchRequestsTwo = faabric::scheduler::getBatchRequests(); - REQUIRE(batchRequestsTwo.size() == 1); - auto pTwo = batchRequestsTwo.at(0); - REQUIRE(pTwo.first == otherHost); - - // Check the request to the other host - REQUIRE(pTwo.second->messages_size() == nCallsTwo - thisCores); -} - -TEST_CASE_METHOD(SlowExecutorTestFixture, - "Test overloaded scheduler", - "[scheduler]") -{ - faabric::util::SystemConfig& conf = faabric::util::getSystemConfig(); - conf.overrideCpuCount = 5; - - faabric::util::setMockMode(true); - - 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 = - faabric::util::getMainThreadSnapshotKey(req->messages().at(0)); - } - - SECTION("Processes") - { - execMode = faabric::BatchExecuteRequest::PROCESSES; - expectedSnapshot = "procSnap"; - } - - SECTION("Functions") { execMode = faabric::BatchExecuteRequest::FUNCTIONS; } - - size_t snapSize = 1234; - if (!expectedSnapshot.empty()) { - auto snap = std::make_shared(snapSize); - reg.registerSnapshot(expectedSnapshot, snap); - } - - // Set up this host with very low resources - std::string thisHost = sch.getThisHost(); - int nCores = 1; - faabric::HostResources res; - res.set_slots(nCores); - sch.setThisHostResources(res); - - // Set up another host with insufficient resources - std::string otherHost = "other"; - sch.addHostToGlobalSet(otherHost); - faabric::HostResources resOther; - resOther.set_slots(2); - faabric::scheduler::queueResourceResponse(otherHost, resOther); - - // Make the request - req->set_type(execMode); - const faabric::Message firstMsg = req->messages().at(0); - faabric::batch_scheduler::SchedulingDecision expectedDecision( - firstMsg.appid(), firstMsg.groupid()); - std::vector msgToWait; - for (int i = 0; i < nCalls; i++) { - faabric::Message& msg = req->mutable_messages()->at(i); - - if (req->type() != faabric::BatchExecuteRequest::THREADS) { - msg.set_snapshotkey(expectedSnapshot); - } - - if (i == 1 || i == 2) { - expectedDecision.addMessage(otherHost, msg); - } else { - msgToWait.emplace_back(msg); - expectedDecision.addMessage(thisHost, msg); - } - } - - // Submit the request - faabric::batch_scheduler::SchedulingDecision decision = - sch.callFunctions(req); - checkSchedulingDecisionEquality(decision, expectedDecision); - - // Check status of local queueing - int expectedLocalCalls = nCalls - 2; - int expectedExecutors; - if (execMode == faabric::BatchExecuteRequest::THREADS) { - expectedExecutors = 1; - } else { - expectedExecutors = expectedLocalCalls; + REQUIRE(sch.getFunctionExecutorCount(m) == nCallsTwo); } - - REQUIRE(sch.getFunctionExecutorCount(firstMsg) == expectedExecutors); - - // Await results - for (const auto& msg : msgToWait) { - if (execMode == faabric::BatchExecuteRequest::THREADS) { - sch.awaitThreadResult(msg.id()); - } else { - plannerCli.getMessageResult(msg, 10000); - } - } -} - -TEST_CASE_METHOD(SlowExecutorTestFixture, - "Test unregistering host", - "[scheduler]") -{ - faabric::util::setMockMode(true); - - std::string thisHost = faabric::util::getSystemConfig().endpointHost; - std::string otherHost = "foobar"; - sch.addHostToGlobalSet(otherHost); - - int nCores = 5; - faabric::HostResources res; - res.set_slots(nCores); - sch.setThisHostResources(res); - - // Set up capacity for other host - faabric::scheduler::queueResourceResponse(otherHost, res); - - std::shared_ptr req = - faabric::util::batchExecFactory("foo", "bar", nCores + 1); - sch.callFunctions(req); - faabric::Message msg = req->messages().at(0); - - // Check other host is added - const std::set& expectedHosts = { otherHost }; - REQUIRE(sch.getFunctionRegisteredHosts(msg.user(), msg.function()) == - expectedHosts); - REQUIRE(sch.getFunctionRegisteredHostCount(msg) == 1); - - // Remove host for another function and check host isn't removed - faabric::Message otherMsg = faabric::util::messageFactory("foo", "qux"); - sch.removeRegisteredHost(otherHost, otherMsg.user(), otherMsg.function()); - REQUIRE(sch.getFunctionRegisteredHosts(msg.user(), msg.function()) == - expectedHosts); - REQUIRE(sch.getFunctionRegisteredHostCount(msg) == 1); - - // Remove host - sch.removeRegisteredHost(otherHost, msg.user(), msg.function()); - REQUIRE(sch.getFunctionRegisteredHosts(msg.user(), msg.function()).empty()); - REQUIRE(sch.getFunctionRegisteredHostCount(msg) == 0); } TEST_CASE_METHOD(SlowExecutorTestFixture, "Check test mode", "[scheduler]") @@ -606,20 +337,25 @@ TEST_CASE_METHOD(SlowExecutorTestFixture, "Check test mode", "[scheduler]") { faabric::util::setTestMode(false); - sch.callFunctions(reqA); - REQUIRE(sch.getRecordedMessagesAll().empty()); + plannerCli.callFunctions(reqA); + plannerCli.getMessageResult(msgA, 500); + REQUIRE(sch.getRecordedMessages().empty()); } SECTION("Test mode") { faabric::util::setTestMode(true); - sch.callFunctions(reqA); - sch.callFunctions(reqB); - sch.callFunctions(reqC); + plannerCli.callFunctions(reqA); + plannerCli.callFunctions(reqB); + plannerCli.callFunctions(reqC); + + plannerCli.getMessageResult(msgA, 500); + plannerCli.getMessageResult(msgB, 500); + plannerCli.getMessageResult(msgC, 500); std::vector expectedIds = { msgA.id(), msgB.id(), msgC.id() }; - std::vector actual = sch.getRecordedMessagesAll(); + std::vector actual = sch.getRecordedMessages(); REQUIRE(actual.size() == expectedIds.size()); for (int i = 0; i < expectedIds.size(); i++) { @@ -636,15 +372,24 @@ TEST_CASE_METHOD(SlowExecutorTestFixture, std::string funcName = "my func"; std::string userName = "some user"; std::string inputData = "blahblah"; - faabric::Message call = faabric::util::messageFactory(userName, funcName); - call.set_inputdata(inputData); + auto req = faabric::util::batchExecFactory(userName, funcName, 1); + auto& firstMsg = *req->mutable_messages(0); + firstMsg.set_inputdata(inputData); + firstMsg.set_executedhost(faabric::util::getSystemConfig().endpointHost); + + // If we want to set a function result, the planner must see at least one + // slot, and at least one used slot in this host + faabric::HostResources res; + res.set_slots(1); + res.set_usedslots(1); + sch.setThisHostResources(res); - sch.setFunctionResult(call); + sch.setFunctionResult(firstMsg); // Check retrieval method gets the same call out again - faabric::Message actualCall2 = plannerCli.getMessageResult(call, 1); + faabric::Message resultMsg = plannerCli.getMessageResult(firstMsg, 1); - checkMessageEquality(call, actualCall2); + checkMessageEquality(firstMsg, resultMsg); } TEST_CASE_METHOD(SlowExecutorTestFixture, @@ -653,14 +398,16 @@ TEST_CASE_METHOD(SlowExecutorTestFixture, { int nWaiters = 10; int nWaiterMessages = 4; - conf.overrideCpuCount = nWaiters * nWaiterMessages; + int nCores = nWaiters * nWaiterMessages; + faabric::HostResources res; + res.set_slots(nCores); + sch.setThisHostResources(res); std::vector waiterThreads; // Create waiters that will submit messages and await their results for (int i = 0; i < nWaiters; i++) { waiterThreads.emplace_back([nWaiterMessages] { - Scheduler& sch = scheduler::getScheduler(); auto& plannerCli = faabric::planner::getPlannerClient(); std::shared_ptr req = @@ -672,7 +419,7 @@ TEST_CASE_METHOD(SlowExecutorTestFixture, [&msgIds](auto msg) { msgIds.push_back(msg.id()); }); // Invoke and await - sch.callFunctions(req); + plannerCli.callFunctions(req); for (auto msgId : msgIds) { plannerCli.getMessageResult(appId, msgId, 5000); } @@ -696,6 +443,13 @@ TEST_CASE_METHOD(SlowExecutorTestFixture, faabric::Message_MessageType expectedType; std::string expectedHost = faabric::util::getSystemConfig().endpointHost; + // If we want to set a function result, the planner must see at least one + // slot, and at least one used slot in this host + faabric::HostResources res; + res.set_slots(1); + res.set_usedslots(1); + sch.setThisHostResources(res); + faabric::Message msg; SECTION("Running") { @@ -712,6 +466,7 @@ TEST_CASE_METHOD(SlowExecutorTestFixture, expectedOutput = "I have failed"; msg.set_outputdata(expectedOutput); msg.set_returnvalue(1); + msg.set_executedhost(expectedHost); sch.setFunctionResult(msg); expectedReturnValue = 1; @@ -725,6 +480,7 @@ TEST_CASE_METHOD(SlowExecutorTestFixture, expectedOutput = "I have succeeded"; msg.set_outputdata(expectedOutput); msg.set_returnvalue(0); + msg.set_executedhost(expectedHost); sch.setFunctionResult(msg); expectedReturnValue = 0; @@ -750,8 +506,16 @@ TEST_CASE_METHOD(SlowExecutorTestFixture, faabric::Message& chainedMsgB = *ber->mutable_messages(2); faabric::Message& chainedMsgC = *ber->mutable_messages(3); + // If we want to set a function result, the planner must see at least one + // slot, and at least one used slot in this host + faabric::HostResources res; + res.set_slots(8); + res.set_usedslots(4); + sch.setThisHostResources(res); + // We need to set the function result in order to get the chained // functions. We can do so multiple times + msg.set_executedhost(faabric::util::getSystemConfig().endpointHost); sch.setFunctionResult(msg); // Check empty initially @@ -780,30 +544,7 @@ TEST_CASE_METHOD(SlowExecutorTestFixture, REQUIRE(faabric::util::getChainedFunctions(msg) == expected); } -TEST_CASE_METHOD(SlowExecutorTestFixture, - "Test non-main batch request returned to main", - "[scheduler]") -{ - faabric::util::setMockMode(true); - - std::string otherHost = "other"; - - std::shared_ptr req = - faabric::util::batchExecFactory("blah", "foo", 1); - req->mutable_messages()->at(0).set_mainhost(otherHost); - - faabric::batch_scheduler::SchedulingDecision decision = - sch.callFunctions(req); - REQUIRE(decision.hosts.empty()); - REQUIRE(decision.returnHost == otherHost); - - // Check forwarded to main - auto actualReqs = faabric::scheduler::getBatchRequests(); - REQUIRE(actualReqs.size() == 1); - REQUIRE(actualReqs.at(0).first == otherHost); - REQUIRE(actualReqs.at(0).second->appid() == req->appid()); -} - +/* TODO(thread-opt): we don't delete snapshots yet TEST_CASE_METHOD(SlowExecutorTestFixture, "Test broadcast snapshot deletion", "[scheduler]") @@ -836,10 +577,10 @@ TEST_CASE_METHOD(SlowExecutorTestFixture, std::shared_ptr req = faabric::util::batchExecFactory("foo", "bar", nRequests); - sch.callFunctions(req); + plannerCli.callFunctions(req); // Check other hosts are added - REQUIRE(sch.getFunctionRegisteredHostCount(msg) == 2); + // REQUIRE(sch.getFunctionRegisteredHostCount(msg) == 2); std::set expectedHosts = sch.getFunctionRegisteredHosts(msg.user(), msg.function()); @@ -857,6 +598,7 @@ TEST_CASE_METHOD(SlowExecutorTestFixture, REQUIRE(actualDeleteRequests == expectedDeleteRequests); } +*/ TEST_CASE_METHOD(SlowExecutorTestFixture, "Test set thread results on remote host", @@ -866,6 +608,14 @@ TEST_CASE_METHOD(SlowExecutorTestFixture, faabric::Message msg = faabric::util::messageFactory("foo", "bar"); msg.set_mainhost("otherHost"); + msg.set_executedhost(faabric::util::getSystemConfig().endpointHost); + + // If we want to set a function result, the planner must see at least one + // slot, and at least one used slot in this host + faabric::HostResources res; + res.set_slots(1); + res.set_usedslots(1); + sch.setThisHostResources(res); // Set the thread result int returnValue = 123; @@ -913,7 +663,7 @@ TEST_CASE_METHOD(DummyExecutorTestFixture, "Test executor reuse", "[scheduler]") faabric::Message msgB = reqB->mutable_messages()->at(0); // Execute a couple of functions - sch.callFunctions(reqA); + plannerCli.callFunctions(reqA); for (auto msgId : reqAMsgIds) { faabric::Message res = plannerCli.getMessageResult( msgA.appid(), msgId, SHORT_TEST_TIMEOUT_MS); @@ -924,7 +674,7 @@ TEST_CASE_METHOD(DummyExecutorTestFixture, "Test executor reuse", "[scheduler]") REQUIRE(sch.getFunctionExecutorCount(msgA) == 2); // Execute a couple more functions - sch.callFunctions(reqB); + plannerCli.callFunctions(reqB); for (auto msgId : reqBMsgIds) { faabric::Message res = plannerCli.getMessageResult( msgB.appid(), msgId, SHORT_TEST_TIMEOUT_MS); @@ -937,19 +687,17 @@ TEST_CASE_METHOD(DummyExecutorTestFixture, "Test executor reuse", "[scheduler]") } TEST_CASE_METHOD(DummyExecutorTestFixture, - "Test point-to-point mappings sent from scheduler", + "Test point-to-point mappings are sent", "[scheduler]") { faabric::util::setMockMode(true); std::string thisHost = conf.endpointHost; - std::string otherHost = "foobar"; + std::string otherHost = LOCALHOST; faabric::transport::PointToPointBroker& broker = faabric::transport::getPointToPointBroker(); - sch.addHostToGlobalSet(otherHost); - // Set resources for this host int nSlotsThisHost = 2; faabric::HostResources resourcesThisHost; @@ -957,68 +705,29 @@ TEST_CASE_METHOD(DummyExecutorTestFixture, sch.setThisHostResources(resourcesThisHost); // Set resources for other host - int nSlotsOtherHost = 5; + int nSlotsOtherHost = 2; faabric::HostResources resourcesOtherHost; resourcesOtherHost.set_slots(nSlotsOtherHost); - faabric::scheduler::queueResourceResponse(otherHost, resourcesOtherHost); + sch.addHostToGlobalSet(otherHost, + std::make_shared(resourcesOtherHost)); // Set up request - auto req = faabric::util::batchExecFactory("foo", "bar", 4); + int numMessages = 4; + auto req = faabric::util::batchExecFactory("foo", "bar", numMessages); + for (int i = 0; i < numMessages; i++) { + req->mutable_messages(i)->set_groupidx(i); + } faabric::Message& firstMsg = req->mutable_messages()->at(0); - int appId = firstMsg.appid(); - int groupId = 0; - int groupSize = 10; - bool forceLocal = false; - bool expectMappingsSent = false; - - SECTION("No group ID") - { - groupId = 0; - SECTION("Force local") - { - forceLocal = true; - expectMappingsSent = false; - } - - SECTION("No force local") - { - forceLocal = false; - expectMappingsSent = false; - } - } - - SECTION("With group ID") - { - groupId = 123; - - SECTION("Force local") - { - forceLocal = true; - expectMappingsSent = false; - } - - SECTION("No force local") - { - forceLocal = false; - expectMappingsSent = true; - } - } - - // Set up the group - if (groupId > 0) { - faabric::transport::PointToPointGroup::addGroup( - appId, groupId, groupSize); - } + // Schedule and check decision + auto actualDecision = plannerCli.callFunctions(req); + int groupId = actualDecision.groupId; // Build expectation std::vector expectedHosts = { thisHost, thisHost, otherHost, otherHost }; - if (forceLocal) { - expectedHosts = { thisHost, thisHost, thisHost, thisHost }; - } faabric::batch_scheduler::SchedulingDecision expectedDecision(appId, groupId); @@ -1034,65 +743,16 @@ TEST_CASE_METHOD(DummyExecutorTestFixture, msgIds.push_back(m.id()); } - if (forceLocal) { - req->mutable_messages()->at(0).set_topologyhint("FORCE_LOCAL"); - } - - // Schedule and check decision - faabric::batch_scheduler::SchedulingDecision actualDecision = - sch.callFunctions(req); checkSchedulingDecisionEquality(expectedDecision, actualDecision); // Check mappings set up locally or not std::set registeredIdxs = broker.getIdxsRegisteredForGroup(groupId); - if (expectMappingsSent) { - REQUIRE(registeredIdxs.size() == 4); - } else { - REQUIRE(registeredIdxs.empty()); - } - - // Check mappings sent or not - std::vector> - sentMappings = faabric::transport::getSentMappings(); - - if (expectMappingsSent) { - REQUIRE(sentMappings.size() == 1); - REQUIRE(sentMappings.at(0).first == otherHost); - } else { - REQUIRE(sentMappings.empty()); - } - - // Wait for the functions on this host to complete - for (int i = 0; i < expectedHosts.size(); i++) { - if (expectedHosts.at(i) != thisHost) { - continue; - } - - plannerCli.getMessageResult(appId, msgIds.at(i), 10000); - } -} - -TEST_CASE_METHOD(DummyExecutorTestFixture, - "Test scheduler register and deregister threads", - "[scheduler]") -{ - uint32_t msgIdA = 123; - uint32_t msgIdB = 124; - - // Check empty initially - REQUIRE(sch.getRegisteredThreads().empty()); - - // Register a couple and check they're listed - sch.registerThread(msgIdA); - sch.registerThread(msgIdB); - - std::vector expected = { msgIdA, msgIdB }; - REQUIRE(sch.getRegisteredThreads() == expected); + REQUIRE(registeredIdxs.size() == 4); - // Deregister and check - sch.deregisterThread(msgIdB); - expected = { msgIdA }; - REQUIRE(sch.getRegisteredThreads() == expected); + // Check mappings sent + auto sentMappings = faabric::transport::getSentMappings(); + REQUIRE(sentMappings.size() == 1); + REQUIRE(sentMappings.at(0).first == otherHost); } TEST_CASE_METHOD(DummyExecutorTestFixture, @@ -1120,11 +780,11 @@ TEST_CASE_METHOD(DummyExecutorTestFixture, msgData[2] = 3; // Register a thread + int appId = 1; uint32_t msgId = 123; - sch.registerThread(msgId); // Set result along with the message to cache - sch.setThreadResultLocally(msgId, 0, msg); + sch.setThreadResultLocally(appId, msgId, 0, msg); } // Now check that it's cached diff --git a/tests/test/scheduler/test_scheduling_decisions.cpp b/tests/test/scheduler/test_scheduling_decisions.cpp deleted file mode 100644 index 5b73de504..000000000 --- a/tests/test/scheduler/test_scheduling_decisions.cpp +++ /dev/null @@ -1,502 +0,0 @@ -#include - -#include "fixtures.h" - -#include -#include - -using namespace faabric::scheduler; - -namespace tests { - -// TODO(planner-schedule): remove this file as this is already tested in -// tests/test/batch-scheduler - -class SchedulingDecisionTestFixture : public SchedulerFixture -{ - public: - SchedulingDecisionTestFixture() - { - faabric::util::setMockMode(true); - - std::shared_ptr fac = - std::make_shared(); - setExecutorFactory(fac); - } - - ~SchedulingDecisionTestFixture() { faabric::util::setMockMode(false); } - - protected: - std::string mainHost = faabric::util::getSystemConfig().endpointHost; - - // Helper struct to configure one scheduling decision - struct SchedulingConfig - { - std::vector hosts; - std::vector slots; - std::vector used; - int numReqs; - faabric::batch_scheduler::SchedulingTopologyHint topologyHint; - std::vector expectedHosts; - }; - - // We test the scheduling decision twice: the first one will follow the - // unregistered hosts path, the second one the registerd hosts one. - void testActualSchedulingDecision( - std::shared_ptr req, - const SchedulingConfig& config) - { - // Set resources for all hosts - setHostResources(config.hosts, config.slots, config.used); - - // The first time we run the batch request, we will follow the - // unregistered hosts path - faabric::batch_scheduler::SchedulingDecision actualDecision = - sch.makeSchedulingDecision(req, config.topologyHint); - REQUIRE(actualDecision.hosts == config.expectedHosts); - - // Reestablish host resources - setHostResources(config.hosts, config.slots, config.used); - - // Create a new request, preserving the app ID to ensure a repeat - // request uses the registered hosts - auto repeatReq = - faabric::util::batchExecFactory("foo", "baz", req->messages_size()); - for (int i = 0; i < req->messages_size(); i++) { - faabric::Message& msg = repeatReq->mutable_messages()->at(i); - msg.set_appid(actualDecision.appId); - } - - // The second time we run the batch request, we will follow - // the registered hosts path - actualDecision = - sch.makeSchedulingDecision(repeatReq, config.topologyHint); - REQUIRE(actualDecision.hosts == config.expectedHosts); - } -}; - -TEST_CASE_METHOD(SchedulingDecisionTestFixture, - "Test basic scheduling decision", - "[scheduler]") -{ - SchedulingConfig config = { - .hosts = { mainHost, "hostA" }, - .slots = { 1, 1 }, - .used = { 0, 0 }, - .numReqs = 2, - .topologyHint = faabric::batch_scheduler::SchedulingTopologyHint::NONE, - .expectedHosts = { mainHost, "hostA" }, - }; - - auto req = faabric::util::batchExecFactory("foo", "bar", config.numReqs); - - testActualSchedulingDecision(req, config); -} - -TEST_CASE_METHOD(SchedulingDecisionTestFixture, - "Test scheduling decision on hosts with load", - "[scheduler]") -{ - - SchedulingConfig config = { - .hosts = { mainHost, "hostA", "hostB" }, - .slots = { 4, 4, 4 }, - .used = { 0, 0, 0 }, - .topologyHint = faabric::batch_scheduler::SchedulingTopologyHint::NONE, - }; - - SECTION("Capacity on all hosts") - { - config.used = { 2, 3, 2 }; - config.numReqs = 5; - config.expectedHosts = { - mainHost, mainHost, "hostA", "hostB", "hostB" - }; - } - - SECTION("Non-main host overloaded") - { - config.used = { 2, 6, 2 }; - config.numReqs = 4; - config.expectedHosts = { mainHost, mainHost, "hostB", "hostB" }; - } - - SECTION("Non-main host overloaded, insufficient capacity") - { - config.used = { 3, 6, 2 }; - config.numReqs = 5; - config.expectedHosts = { - mainHost, "hostB", "hostB", mainHost, mainHost, - }; - } - - SECTION("Non-main host overloaded, main overloaded") - { - config.used = { 6, 6, 2 }; - config.numReqs = 5; - config.expectedHosts = { - "hostB", "hostB", mainHost, mainHost, mainHost, - }; - } - - auto req = faabric::util::batchExecFactory("foo", "bar", config.numReqs); - - testActualSchedulingDecision(req, config); -} - -TEST_CASE_METHOD(SchedulingDecisionTestFixture, - "Test overloading all resources defaults to main", - "[scheduler]") -{ - SchedulingConfig config = { - .hosts = { mainHost, "hostA" }, - .slots = { 1, 1 }, - .used = { 0, 0 }, - .numReqs = 3, - .topologyHint = faabric::batch_scheduler::SchedulingTopologyHint::NONE, - .expectedHosts = { mainHost, "hostA", mainHost }, - }; - - auto req = faabric::util::batchExecFactory("foo", "bar", config.numReqs); - - testActualSchedulingDecision(req, config); -} - -TEST_CASE_METHOD(SchedulingDecisionTestFixture, - "Test force local forces executing at main", - "[scheduler]") -{ - SchedulingConfig config = { - .hosts = { mainHost, "hostA" }, - .slots = { 1, 1 }, - .used = { 0, 0 }, - .numReqs = 2, - .topologyHint = - faabric::batch_scheduler::SchedulingTopologyHint::FORCE_LOCAL, - .expectedHosts = { mainHost, "hostA" }, - }; - - auto req = faabric::util::batchExecFactory("foo", "bar", config.numReqs); - - SECTION("Force local off") - { - config.topologyHint = - faabric::batch_scheduler::SchedulingTopologyHint::NONE, - config.expectedHosts = { mainHost, "hostA" }; - } - - SECTION("Force local on") - { - config.topologyHint = - faabric::batch_scheduler::SchedulingTopologyHint::FORCE_LOCAL, - config.expectedHosts = { mainHost, mainHost }; - } - - testActualSchedulingDecision(req, config); -} - -TEST_CASE_METHOD(SchedulingDecisionTestFixture, - "Test scheduling hints can be disabled through the config", - "[scheduler]") -{ - SchedulingConfig config = { - .hosts = { mainHost, "hostA" }, - .slots = { 1, 1 }, - .used = { 0, 0 }, - .numReqs = 2, - .topologyHint = - faabric::batch_scheduler::SchedulingTopologyHint::FORCE_LOCAL, - .expectedHosts = { mainHost, "hostA" }, - }; - - auto req = faabric::util::batchExecFactory("foo", "bar", config.numReqs); - auto& faabricConf = faabric::util::getSystemConfig(); - - SECTION("Config. variable set") - { - faabricConf.noTopologyHints = "on"; - config.expectedHosts = { mainHost, "hostA" }; - } - - SECTION("Config. variable not set") - { - config.expectedHosts = { mainHost, mainHost }; - } - - testActualSchedulingDecision(req, config); - - faabricConf.reset(); -} - -TEST_CASE_METHOD(SchedulingDecisionTestFixture, - "Test main running out of resources", - "[scheduler]") -{ - SchedulingConfig config = { - .hosts = { mainHost, "hostA" }, - .slots = { 0, 2 }, - .used = { 0, 0 }, - .numReqs = 2, - .topologyHint = faabric::batch_scheduler::SchedulingTopologyHint::NONE, - .expectedHosts = { "hostA", "hostA" }, - }; - - auto req = faabric::util::batchExecFactory("foo", "bar", config.numReqs); - - testActualSchedulingDecision(req, config); -} - -TEST_CASE_METHOD(SchedulingDecisionTestFixture, - "Test scheduling decision skips fully occupied worker hosts", - "[scheduler]") -{ - SchedulingConfig config = { - .hosts = { mainHost, "hostA", "hostB" }, - .slots = { 2, 0, 2 }, - .used = { 0, 0, 0 }, - .numReqs = 4, - .topologyHint = faabric::batch_scheduler::SchedulingTopologyHint::NONE, - .expectedHosts = { mainHost, mainHost, "hostB", "hostB" }, - }; - - auto req = faabric::util::batchExecFactory("foo", "bar", config.numReqs); - - SECTION("No topology hint") - { - config.topologyHint = - faabric::batch_scheduler::SchedulingTopologyHint::NONE; - } - - SECTION("Never alone topology hint") - { - config.topologyHint = - faabric::batch_scheduler::SchedulingTopologyHint::NEVER_ALONE; - } - - testActualSchedulingDecision(req, config); -} - -TEST_CASE_METHOD(SchedulingDecisionTestFixture, - "Test scheduling decision with many requests", - "[scheduler]") -{ - SchedulingConfig config = { - .hosts = { mainHost, "hostA", "hostB", "hostC" }, - .slots = { 0, 0, 0, 0 }, - .used = { 0, 0, 0, 0 }, - .numReqs = 8, - .topologyHint = faabric::batch_scheduler::SchedulingTopologyHint::NONE, - .expectedHosts = { mainHost, mainHost, mainHost, mainHost }, - }; - - auto req = faabric::util::batchExecFactory("foo", "bar", config.numReqs); - - SECTION("Even slot distribution across hosts") - { - config.slots = { 2, 2, 2, 2 }; - config.expectedHosts = { mainHost, mainHost, "hostA", "hostA", - "hostB", "hostB", "hostC", "hostC" }; - } - - SECTION("Uneven slot ditribution across hosts") - { - config.slots = { 3, 2, 2, 1 }; - config.expectedHosts = { mainHost, mainHost, mainHost, "hostA", - "hostA", "hostB", "hostB", "hostC" }; - } - - SECTION("Very uneven slot distribution across hosts") - { - config.slots = { 1, 0, 0, 0 }; - config.expectedHosts = { mainHost, mainHost, mainHost, mainHost, - mainHost, mainHost, mainHost, mainHost }; - } - - SECTION("Decreasing to one and increasing slot distribution") - { - config.slots = { 2, 2, 1, 2 }; - - SECTION("No topology hint") - { - config.topologyHint = - faabric::batch_scheduler::SchedulingTopologyHint::NONE; - config.expectedHosts = { mainHost, mainHost, "hostA", "hostA", - "hostB", "hostC", "hostC", mainHost }; - } - - SECTION("Never alone topology hint") - { - config.topologyHint = - faabric::batch_scheduler::SchedulingTopologyHint::NEVER_ALONE; - config.expectedHosts = { mainHost, mainHost, "hostA", "hostA", - "hostC", "hostC", "hostC", "hostC" }; - } - } - - testActualSchedulingDecision(req, config); -} - -TEST_CASE_METHOD(SchedulingDecisionTestFixture, - "Test sticky pairs scheduling topology hint", - "[scheduler]") -{ - SchedulingConfig config = { - .hosts = { mainHost, "hostA" }, - .slots = { 1, 1 }, - .used = { 0, 0 }, - .numReqs = 2, - .topologyHint = - faabric::batch_scheduler::SchedulingTopologyHint::NEVER_ALONE, - .expectedHosts = { mainHost, "hostA" }, - }; - - std::shared_ptr req; - - SECTION("Test with hint we only schedule to new hosts pairs of requests") - { - config.expectedHosts = { mainHost, mainHost }; - req = faabric::util::batchExecFactory("foo", "bar", config.numReqs); - } - - SECTION("Test with hint we may overload remote hosts") - { - config.hosts = { mainHost, "hostA", "hostB" }; - config.numReqs = 5; - config.slots = { 2, 2, 1 }; - config.used = { 0, 0, 0 }; - config.expectedHosts = { - mainHost, mainHost, "hostA", "hostA", "hostA" - }; - req = faabric::util::batchExecFactory("foo", "bar", config.numReqs); - } - - SECTION( - "Test with hint we still overload correctly if running out of slots") - { - config.hosts = { mainHost, "hostA" }; - config.numReqs = 5; - config.slots = { 2, 2 }; - config.used = { 0, 0 }; - config.expectedHosts = { - mainHost, mainHost, "hostA", "hostA", "hostA" - }; - req = faabric::util::batchExecFactory("foo", "bar", config.numReqs); - } - - SECTION("Test hint with uneven slot distribution") - { - config.hosts = { mainHost, "hostA" }; - config.numReqs = 5; - config.slots = { 2, 3 }; - config.used = { 0, 0 }; - config.expectedHosts = { - mainHost, mainHost, "hostA", "hostA", "hostA" - }; - req = faabric::util::batchExecFactory("foo", "bar", config.numReqs); - } - - SECTION("Test hint with uneven slot distribution and overload") - { - config.hosts = { mainHost, "hostA", "hostB" }; - config.numReqs = 6; - config.slots = { 2, 3, 1 }; - config.used = { 0, 0, 0 }; - config.expectedHosts = { mainHost, mainHost, "hostA", - "hostA", "hostA", "hostA" }; - req = faabric::util::batchExecFactory("foo", "bar", config.numReqs); - } - - testActualSchedulingDecision(req, config); -} - -TEST_CASE_METHOD(SchedulingDecisionTestFixture, - "Test underfull scheduling topology hint", - "[scheduler]") -{ - SchedulingConfig config = { - .hosts = { mainHost, "hostA" }, - .slots = { 2, 2 }, - .used = { 0, 0 }, - .numReqs = 2, - .topologyHint = - faabric::batch_scheduler::SchedulingTopologyHint::UNDERFULL, - .expectedHosts = { mainHost, "hostA" }, - }; - - std::shared_ptr req; - - SECTION("Test hint's basic functionality") - { - req = faabric::util::batchExecFactory("foo", "bar", config.numReqs); - } - - SECTION("Test hint does not affect other hosts") - { - config.numReqs = 3; - config.expectedHosts = { mainHost, "hostA", "hostA" }; - req = faabric::util::batchExecFactory("foo", "bar", config.numReqs); - } - - SECTION("Test with hint we still overload to main") - { - config.numReqs = 4; - config.expectedHosts = { mainHost, "hostA", "hostA", mainHost }; - req = faabric::util::batchExecFactory("foo", "bar", config.numReqs); - } - - testActualSchedulingDecision(req, config); -} - -TEST_CASE_METHOD(SchedulingDecisionTestFixture, - "Test cached scheduling topology hint", - "[scheduler]") -{ - // Set up a basic scenario - SchedulingConfig config = { - .hosts = { mainHost, "hostA" }, - .slots = { 2, 2 }, - .used = { 0, 0 }, - .numReqs = 4, - .topologyHint = - faabric::batch_scheduler::SchedulingTopologyHint::CACHED, - .expectedHosts = { mainHost, mainHost, "hostA", "hostA" }, - }; - - auto req = faabric::util::batchExecFactory("foo", "bar", config.numReqs); - int appId = req->messages().at(0).appid(); - testActualSchedulingDecision(req, config); - - // Now change the setup so that another decision would do something - // different - SchedulingConfig hitConfig = { - .hosts = { mainHost, "hostA" }, - .slots = { 0, 10 }, - .used = { 0, 0 }, - .numReqs = 4, - .topologyHint = - faabric::batch_scheduler::SchedulingTopologyHint::CACHED, - .expectedHosts = { mainHost, mainHost, "hostA", "hostA" }, - }; - - // Note we must preserve the app ID to get a cached decision - auto hitReq = faabric::util::batchExecFactory("foo", "bar", config.numReqs); - for (int i = 0; i < hitReq->messages_size(); i++) { - hitReq->mutable_messages()->at(i).set_appid(appId); - } - testActualSchedulingDecision(hitReq, hitConfig); - - // Change app ID and confirm we get new decision - SchedulingConfig missConfig = { - .hosts = { mainHost, "hostA" }, - .slots = { 0, 10 }, - .used = { 0, 0 }, - .numReqs = 4, - .topologyHint = - faabric::batch_scheduler::SchedulingTopologyHint::CACHED, - .expectedHosts = { "hostA", "hostA", "hostA", "hostA" }, - }; - auto missReq = - faabric::util::batchExecFactory("foo", "bar", config.numReqs); - testActualSchedulingDecision(missReq, missConfig); -} -} diff --git a/tests/test/snapshot/test_snapshot_client_server.cpp b/tests/test/snapshot/test_snapshot_client_server.cpp index 5c39cae2c..c2f7d7d6b 100644 --- a/tests/test/snapshot/test_snapshot_client_server.cpp +++ b/tests/test/snapshot/test_snapshot_client_server.cpp @@ -286,10 +286,10 @@ TEST_CASE_METHOD(SnapshotClientServerTestFixture, SnapshotDataType::Int, SnapshotMergeOperation::Sum, offsetA2, intDataA2); // Push diffs with result for a fake thread + int appId = 111; int msgId = 345; - sch.registerThread(msgId); diffs = { diffA1, diffA2 }; - snapshotClient.pushThreadResult(msgId, 0, snapKey, diffs); + snapshotClient.pushThreadResult(appId, msgId, 0, snapKey, diffs); // Write and check diffs have been applied according to the merge operations snap->writeQueuedDiffs(); @@ -401,10 +401,10 @@ TEST_CASE_METHOD(SnapshotClientServerTestFixture, SnapshotDiff diff(dataType, operation, offset, diffData); // Push diffs for a fake thread + int appId = 777; int msgId = 123; - sch.registerThread(msgId); std::vector diffs = { diff }; - snapshotClient.pushThreadResult(msgId, 0, snapKey, diffs); + snapshotClient.pushThreadResult(appId, msgId, 0, snapKey, diffs); // Ensure the right number of diffs is applied REQUIRE(snap->getQueuedDiffsCount() == 1); @@ -416,37 +416,55 @@ TEST_CASE_METHOD(SnapshotClientServerTestFixture, std::vector actualData = snap->getDataCopy(offset, expectedData.size()); REQUIRE(actualData == expectedData); - - sch.deregisterThread(msgId); } TEST_CASE_METHOD(SnapshotClientServerTestFixture, "Test set thread result", "[snapshot]") { - // Register threads on this host + int appIdA = 7; + int appIdB = 8; int threadIdA = 123; int threadIdB = 345; int returnValueA = 88; int returnValueB = 99; - sch.registerThread(threadIdA); - sch.registerThread(threadIdB); - snapshotClient.pushThreadResult(threadIdA, returnValueA, "", {}); - snapshotClient.pushThreadResult(threadIdB, returnValueB, "", {}); + // If we want to set a function result, the planner must see at least one + // slot, and at least one used slot in this host + faabric::HostResources res; + res.set_slots(2); + res.set_usedslots(2); + sch.setThisHostResources(res); + + snapshotClient.pushThreadResult(appIdA, threadIdA, returnValueA, "", {}); + snapshotClient.pushThreadResult(appIdB, threadIdB, returnValueB, "", {}); + + // Set tmp function results too (so that they are accessible) + Message msgA; + msgA.set_appid(appIdA); + msgA.set_id(threadIdA); + msgA.set_returnvalue(returnValueA); + msgA.set_executedhost(faabric::util::getSystemConfig().endpointHost); + sch.setFunctionResult(msgA); + Message msgB; + msgB.set_appid(appIdB); + msgB.set_id(threadIdB); + msgB.set_returnvalue(returnValueB); + msgB.set_executedhost(faabric::util::getSystemConfig().endpointHost); + sch.setFunctionResult(msgB); int rA = 0; int rB = 0; // Set up two threads to await the results - std::jthread tA([&rA, threadIdA] { - faabric::scheduler::Scheduler& sch = faabric::scheduler::getScheduler(); - rA = sch.awaitThreadResult(threadIdA); + std::jthread tA([&rA, appIdA, threadIdA] { + auto& plannerCli = faabric::planner::getPlannerClient(); + rA = plannerCli.getMessageResult(appIdA, threadIdA, 500).returnvalue(); }); - std::jthread tB([&rB, threadIdB] { - faabric::scheduler::Scheduler& sch = faabric::scheduler::getScheduler(); - rB = sch.awaitThreadResult(threadIdB); + std::jthread tB([&rB, appIdB, threadIdB] { + auto& plannerCli = faabric::planner::getPlannerClient(); + rB = plannerCli.getMessageResult(appIdB, threadIdB, 500).returnvalue(); }); if (tA.joinable()) { diff --git a/tests/test/util/test_exec_graph.cpp b/tests/test/util/test_exec_graph.cpp index bd9a0bb2e..bb39b079e 100644 --- a/tests/test/util/test_exec_graph.cpp +++ b/tests/test/util/test_exec_graph.cpp @@ -21,14 +21,30 @@ class ExecGraphTestFixture TEST_CASE_METHOD(ExecGraphTestFixture, "Test execution graph", "[util]") { - auto ber = faabric::util::batchExecFactory("demo", "echo", 7); + int nMsg = 7; + auto thisHost = faabric::util::getSystemConfig().endpointHost; + auto ber = faabric::util::batchExecFactory("demo", "echo", nMsg); faabric::Message msgA = *ber->mutable_messages(0); + msgA.set_executedhost(thisHost); faabric::Message msgB1 = *ber->mutable_messages(1); + msgB1.set_executedhost(thisHost); faabric::Message msgB2 = *ber->mutable_messages(2); + msgB2.set_executedhost(thisHost); faabric::Message msgC1 = *ber->mutable_messages(3); + msgC1.set_executedhost(thisHost); faabric::Message msgC2 = *ber->mutable_messages(4); + msgC2.set_executedhost(thisHost); faabric::Message msgC3 = *ber->mutable_messages(5); + msgC3.set_executedhost(thisHost); faabric::Message msgD = *ber->mutable_messages(6); + msgD.set_executedhost(thisHost); + + // If we want to set a function result, the planner must see as many used + // slots as results we are setting + faabric::HostResources res; + res.set_slots(nMsg); + res.set_usedslots(nMsg); + sch.setThisHostResources(res); // Set up chaining relationships logChainedFunction(msgA, msgB1); @@ -116,9 +132,9 @@ TEST_CASE_METHOD(ExecGraphTestFixture, TEST_CASE_METHOD(MpiBaseTestFixture, "Test MPI execution graph", "[scheduler]") { faabric::mpi::MpiWorld world; - msg.set_appid(1337); msg.set_ismpi(true); msg.set_recordexecgraph(true); + auto thisHost = faabric::util::getSystemConfig().endpointHost; // Build the message vector to reconstruct the graph std::vector messages(worldSize); @@ -129,8 +145,7 @@ TEST_CASE_METHOD(MpiBaseTestFixture, "Test MPI execution graph", "[scheduler]") messages.at(rank).set_finishtimestamp(0); messages.at(rank).set_resultkey(""); messages.at(rank).set_statuskey(""); - messages.at(rank).set_executedhost( - faabric::util::getSystemConfig().endpointHost); + messages.at(rank).set_executedhost(thisHost); messages.at(rank).set_ismpi(true); messages.at(rank).set_mpiworldid(worldId); messages.at(rank).set_mpirank(rank); @@ -138,10 +153,10 @@ TEST_CASE_METHOD(MpiBaseTestFixture, "Test MPI execution graph", "[scheduler]") messages.at(rank).set_recordexecgraph(true); } - world.create(msg, worldId, worldSize); + // First call the original message + plannerCli.callFunctions(req); - // Update the result for the main message - sch.setFunctionResult(msg); + world.create(msg, worldId, worldSize); // Build expected graph ExecGraphNode nodeB1 = { .msg = messages.at(1) }; @@ -154,8 +169,16 @@ TEST_CASE_METHOD(MpiBaseTestFixture, "Test MPI execution graph", "[scheduler]") ExecGraph expected{ .rootNode = nodeA }; + // The MPI base fixture uses the DummyExecutor, which immediately sets + // the function result. We want to overwrite said function result with the + // chained calls (logged as part of MpiWorld::create) thus we sleep enough + // to let the dummy executor set the result, to make sure we can overwrite + // it here + SLEEP_MS(500); + msg.set_executedhost(thisHost); + sch.setFunctionResult(msg); + // Wait for the MPI messages to finish - auto& plannerCli = faabric::planner::getPlannerClient(); plannerCli.getMessageResult(msg, 2000); for (const auto& id : faabric::util::getChainedFunctions(msg)) { plannerCli.getMessageResult(msg.appid(), id, 2000); diff --git a/tests/test/util/test_json.cpp b/tests/test/util/test_json.cpp index 9f4103690..5debe8a5f 100644 --- a/tests/test/util/test_json.cpp +++ b/tests/test/util/test_json.cpp @@ -38,10 +38,6 @@ class JsonTestFixture auto& intMap = *msg.mutable_intexecgraphdetails(); intMap["foo"] = 0; - msg.set_migrationcheckperiod(33); - - msg.set_topologyhint("TEST_TOPOLOGY_HINT"); - msg.set_inputdata("foo bar"); faabric::util::setMessageId(msg); @@ -77,17 +73,11 @@ TEST_CASE_METHOD(JsonTestFixture, "Test JSON contains required keys", "[util]") // We consume the generated JSON files from a variety of places, so this // test ensures that the keywords we use elsewhere are generated as part // of the serialisation process - std::vector requiredKeys = { "input_data", - "python", - "py_user", - "py_func", - "mpi", - "mpi_world_size", - "record_exec_graph", - "migration_check_period", - "start_ts", - "finish_ts", - "topology_hint" }; + std::vector requiredKeys = { + "input_data", "python", "py_user", + "py_func", "mpi", "mpi_world_size", + "record_exec_graph", "start_ts", "finish_ts", + }; std::string jsonString = faabric::util::messageToJson(msg); for (const auto& key : requiredKeys) { diff --git a/tests/utils/fixtures.h b/tests/utils/fixtures.h index 109f33551..022e06630 100644 --- a/tests/utils/fixtures.h +++ b/tests/utils/fixtures.h @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -169,40 +170,6 @@ class SchedulerFixture faabric::util::getDirtyTracker()->clearAll(); }; - // Helper method to set the available hosts and slots per host prior to - // making a scheduling decision - void setHostResources(std::vector hosts, - std::vector slotsPerHost, - std::vector usedPerHost) - { - if (hosts.size() != slotsPerHost.size() || - hosts.size() != usedPerHost.size()) { - SPDLOG_ERROR("Must provide one value for slots and used per host"); - throw std::runtime_error( - "Not providing one value per slot and used per host"); - } - - sch.clearRecordedMessages(); - faabric::scheduler::clearMockRequests(); - - for (int i = 0; i < hosts.size(); i++) { - faabric::HostResources resources; - resources.set_slots(slotsPerHost.at(i)); - resources.set_usedslots(usedPerHost.at(i)); - - sch.addHostToGlobalSet(hosts.at(i)); - - // If setting resources for the main host, update the scheduler. - // Otherwise, queue the resource response - if (i == 0) { - sch.setThisHostResources(resources); - } else { - faabric::scheduler::queueResourceResponse(hosts.at(i), - resources); - } - } - } - protected: faabric::scheduler::Scheduler& sch; }; @@ -429,7 +396,8 @@ class MpiBaseTestFixture , func("hellompi") , worldId(123) , worldSize(5) - , msg(faabric::util::messageFactory(user, func)) + , req(faabric::util::batchExecFactory(user, func, 1)) + , msg(*req->mutable_messages(0)) { std::shared_ptr fac = std::make_shared(); @@ -437,14 +405,21 @@ class MpiBaseTestFixture msg.set_mpiworldid(worldId); msg.set_mpiworldsize(worldSize); + + // Make enough space in this host to run MPI functions + faabric::HostResources res; + res.set_slots(2 * worldSize); + sch.setThisHostResources(res); + + // Call the request, so that we have the original message recorded + // in the planner + plannerCli.callFunctions(req); } ~MpiBaseTestFixture() { - // TODO - without this sleep, we sometimes clear the PTP broker before - // all the executor threads have been set up, and when trying to query - // for the comm. group we throw a runtime error. - SLEEP_MS(200); + // Make sure we get the message result to avoid data races + plannerCli.getMessageResult(msg, 500); } protected: @@ -453,7 +428,49 @@ class MpiBaseTestFixture int worldId; int worldSize; - faabric::Message msg; + std::shared_ptr req; + faabric::Message& msg; + + // This method waits for all MPI messages to be scheduled. In MPI, + // (worldSize - 1) messages are scheduled after calling MpiWorld::create. + // Thus, it is hard when this second batch has already started executing + void waitForMpiMessages( + std::shared_ptr reqIn = nullptr, + int expectedWorldSize = 0) const + { + if (reqIn == nullptr) { + reqIn = req; + } + + if (expectedWorldSize == 0) { + expectedWorldSize = worldSize; + } + + int maxRetries = 5; + int numRetries = 0; + auto decision = plannerCli.getSchedulingDecision(reqIn); + while (decision.messageIds.size() != expectedWorldSize) { + if (numRetries >= maxRetries) { + SPDLOG_ERROR( + "Timed-out waiting for MPI messages to be scheduled ({}/{})", + decision.messageIds.size(), + expectedWorldSize); + throw std::runtime_error("Timed-out waiting for MPI messges"); + } + + SPDLOG_DEBUG("Waiting for MPI messages to be scheduled ({}/{})", + decision.messageIds.size(), + expectedWorldSize); + SLEEP_MS(200); + + numRetries += 1; + decision = plannerCli.getSchedulingDecision(reqIn); + } + + for (auto mid : decision.messageIds) { + plannerCli.getMessageResult(decision.appId, mid, 500); + } + } }; class MpiTestFixture : public MpiBaseTestFixture @@ -508,10 +525,8 @@ class RemoteMpiTestFixture : public MpiBaseTestFixture // Set up the other world and add it to the global set of hosts faabric::HostResources otherResources; otherResources.set_slots(ranksOtherWorld); - sch.addHostToGlobalSet(otherHost); - - // Queue the resource response for this other host - faabric::scheduler::queueResourceResponse(otherHost, otherResources); + sch.addHostToGlobalSet( + otherHost, std::make_shared(otherResources)); } protected: diff --git a/tests/utils/message_utils.cpp b/tests/utils/message_utils.cpp index cd9c7a082..206c9e5aa 100644 --- a/tests/utils/message_utils.cpp +++ b/tests/utils/message_utils.cpp @@ -42,7 +42,5 @@ void checkMessageEquality(const faabric::Message& msgA, checkMessageMapEquality(msgA.execgraphdetails(), msgB.execgraphdetails()); checkMessageMapEquality(msgA.intexecgraphdetails(), msgB.intexecgraphdetails()); - - REQUIRE(msgA.migrationcheckperiod() == msgB.migrationcheckperiod()); } }