Skip to content

Commit

Permalink
Periodically Check For Migration Opportunities (#214)
Browse files Browse the repository at this point in the history
* proto: add field to set migration check period

* scheduler: add function migration thread that sleeps for a fixed amount of time, and a simple test.

* scheduler: add logic for check for migration opportunities method

* scheduler: start/stop migration thread and fix race condition when removing from in-flight map

* tests: add test for function migration thread and fix bug in thread shutdown

* util: add method to deep copy a faabric message + test

* function call server: add call to add a pending migration to remote hosts

* tests: add further testing

* mpi: add migration points, link with executor, and tests

* tests: fix data race

* pr: re-factor methods as suggested in the pr comments

* executor: throw exception to properly shutdown executing task

* executor: close the loop to alllow for function migration

* scheduler: add UNDERFULL scheduling topology hint

* Add migration exception, catch in executor

* Remove manual message copying

* Remove unnecessary migration APIs

* Make getMemoryView public

* scheduler: get functions to migrate properly

* cleanup

* mpi: use initLocalQueues() instead of repeating the logic, remove assertion from the method

* mpi: remove unused getMpiMockedPendingMigrations

* scheduler: remove unused function declarations

* scheduler: factor out method to start the function migration thread if necessary

* mpi: use a boolean flag to indicate that app has been migrated, and check for flag in barriers to remove used pendingMigrations in the scheduler

* proto: make topologyHint a message field, add test for json serialisation, remove unnecessary field to callFunctions, and re-factor necessary calls to callFunctions

Co-authored-by: Simon Shillaker <mail@simonshillaker.com>
  • Loading branch information
csegarragonz and Shillaker committed Feb 7, 2022
1 parent 5b67a98 commit 4f4628a
Show file tree
Hide file tree
Showing 24 changed files with 1,213 additions and 53 deletions.
3 changes: 2 additions & 1 deletion include/faabric/scheduler/FunctionCallApi.h
Original file line number Diff line number Diff line change
Expand Up @@ -7,6 +7,7 @@ enum FunctionCalls
ExecuteFunctions = 1,
Flush = 2,
Unregister = 3,
GetResources = 4
GetResources = 4,
PendingMigrations = 5
};
}
5 changes: 5 additions & 0 deletions include/faabric/scheduler/FunctionCallClient.h
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,9 @@ getBatchRequests();
std::vector<std::pair<std::string, faabric::EmptyRequest>>
getResourceRequests();

std::vector<std::pair<std::string, std::shared_ptr<faabric::PendingMigrations>>>
getPendingMigrationsRequests();

std::vector<std::pair<std::string, faabric::UnregisterRequest>>
getUnregisterRequests();

Expand All @@ -42,6 +45,8 @@ class FunctionCallClient : public faabric::transport::MessageEndpointClient

faabric::HostResources getResources();

void sendPendingMigrations(std::shared_ptr<faabric::PendingMigrations> req);

void executeFunctions(std::shared_ptr<faabric::BatchExecuteRequest> req);

void unregister(faabric::UnregisterRequest& req);
Expand Down
4 changes: 4 additions & 0 deletions include/faabric/scheduler/FunctionCallServer.h
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,10 @@ class FunctionCallServer final
const uint8_t* buffer,
size_t bufferSize);

std::unique_ptr<google::protobuf::Message> recvPendingMigrations(
const uint8_t* buffer,
size_t bufferSize);

void recvExecuteFunctions(const uint8_t* buffer, size_t bufferSize);

void recvUnregister(const uint8_t* buffer, size_t bufferSize);
Expand Down
26 changes: 26 additions & 0 deletions include/faabric/scheduler/FunctionMigrationThread.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,26 @@
#pragma once

#include <condition_variable>
#include <mutex>
#include <thread>

namespace faabric::scheduler {
// Start a background thread that, every wake up period, will check if there
// are migration opportunities for in-flight apps that have opted in to
// being checked for migrations.
class FunctionMigrationThread
{
public:
void start(int wakeUpPeriodSecondsIn);

void stop();

int wakeUpPeriodSeconds;

private:
std::unique_ptr<std::thread> workThread = nullptr;
std::mutex mx;
std::condition_variable mustStopCv;
std::atomic<bool> isShutdown;
};
}
9 changes: 9 additions & 0 deletions include/faabric/scheduler/MpiWorld.h
Original file line number Diff line number Diff line change
Expand Up @@ -207,6 +207,12 @@ class MpiWorld

void setMsgForRank(faabric::Message& msg);

/* Function Migration */

void prepareMigration(
int thisRank,
std::shared_ptr<faabric::PendingMigrations> pendingMigrations);

private:
int id = -1;
int size = -1;
Expand Down Expand Up @@ -283,5 +289,8 @@ class MpiWorld
MPI_Status* status,
faabric::MPIMessage::MPIMessageType messageType =
faabric::MPIMessage::NORMAL);

/* Function migration */
bool hasBeenMigrated = false;
};
}
59 changes: 47 additions & 12 deletions include/faabric/scheduler/Scheduler.h
Original file line number Diff line number Diff line change
Expand Up @@ -3,6 +3,7 @@
#include <faabric/proto/faabric.pb.h>
#include <faabric/scheduler/ExecGraph.h>
#include <faabric/scheduler/FunctionCallClient.h>
#include <faabric/scheduler/FunctionMigrationThread.h>
#include <faabric/scheduler/InMemoryMessageQueue.h>
#include <faabric/snapshot/SnapshotClient.h>
#include <faabric/snapshot/SnapshotRegistry.h>
Expand All @@ -22,6 +23,10 @@

namespace faabric::scheduler {

typedef std::pair<std::shared_ptr<BatchExecuteRequest>,
std::shared_ptr<faabric::util::SchedulingDecision>>
InFlightPair;

class Scheduler;

Scheduler& getScheduler();
Expand Down Expand Up @@ -77,13 +82,13 @@ class Executor
faabric::Message& msg,
bool createIfNotExists = false);

virtual std::span<uint8_t> getMemoryView();

protected:
virtual void restore(const std::string& snapshotKey);

virtual void postFinish();

virtual std::span<uint8_t> getMemoryView();

virtual void setMemorySize(size_t newSize);

faabric::Message boundMessage;
Expand Down Expand Up @@ -131,9 +136,7 @@ class Scheduler
void callFunction(faabric::Message& msg, bool forceLocal = false);

faabric::util::SchedulingDecision callFunctions(
std::shared_ptr<faabric::BatchExecuteRequest> req,
faabric::util::SchedulingTopologyHint =
faabric::util::SchedulingTopologyHint::NORMAL);
std::shared_ptr<faabric::BatchExecuteRequest> req);

faabric::util::SchedulingDecision callFunctions(
std::shared_ptr<faabric::BatchExecuteRequest> req,
Expand Down Expand Up @@ -220,6 +223,27 @@ class Scheduler

ExecGraph getFunctionExecGraph(unsigned int msgId);

// ----------------------------------
// Function Migration
// ----------------------------------
void checkForMigrationOpportunities();

std::shared_ptr<faabric::PendingMigrations> getPendingAppMigrations(
uint32_t appId);

void addPendingMigration(std::shared_ptr<faabric::PendingMigrations> msg);

void removePendingMigration(uint32_t appId);

// ----------------------------------
// Clients
// ----------------------------------
faabric::scheduler::FunctionCallClient& getFunctionCallClient(
const std::string& otherHost);

faabric::snapshot::SnapshotClient& getSnapshotClient(
const std::string& otherHost);

private:
std::string thisHost;

Expand All @@ -244,13 +268,6 @@ class Scheduler

std::mutex localResultsMutex;

// ---- Clients ----
faabric::scheduler::FunctionCallClient& getFunctionCallClient(
const std::string& otherHost);

faabric::snapshot::SnapshotClient& getSnapshotClient(
const std::string& otherHost);

// ---- Host resources and hosts ----
faabric::HostResources thisHostResources;
std::atomic<int32_t> thisHostUsedSlots = 0;
Expand Down Expand Up @@ -290,6 +307,24 @@ class Scheduler

// ---- Point-to-point ----
faabric::transport::PointToPointBroker& broker;

// ---- Function migration ----
FunctionMigrationThread functionMigrationThread;
std::unordered_map<uint32_t, InFlightPair> inFlightRequests;
std::unordered_map<uint32_t, std::shared_ptr<faabric::PendingMigrations>>
pendingMigrations;

std::vector<std::shared_ptr<faabric::PendingMigrations>>
doCheckForMigrationOpportunities(
faabric::util::MigrationStrategy migrationStrategy =
faabric::util::MigrationStrategy::BIN_PACK);

void broadcastPendingMigrations(
std::shared_ptr<faabric::PendingMigrations> pendingMigrations);

void doStartFunctionMigrationThread(
std::shared_ptr<faabric::BatchExecuteRequest> req,
faabric::util::SchedulingDecision& decision);
};

}
8 changes: 8 additions & 0 deletions include/faabric/util/func.h
Original file line number Diff line number Diff line change
Expand Up @@ -7,6 +7,14 @@

namespace faabric::util {

class FunctionMigratedException : public faabric::util::FaabricException
{
public:
explicit FunctionMigratedException(std::string message)
: FaabricException(std::move(message))
{}
};

std::string funcToString(const faabric::Message& msg, bool includeId);

std::string funcToString(
Expand Down
38 changes: 37 additions & 1 deletion include/faabric/util/scheduling.h
Original file line number Diff line number Diff line change
Expand Up @@ -2,6 +2,7 @@

#include <cstdint>
#include <string>
#include <unordered_map>
#include <vector>

#include <faabric/proto/faabric.pb.h>
Expand Down Expand Up @@ -46,12 +47,47 @@ class SchedulingDecision
// requests in a batch.
// - NORMAL: bin-packs requests to slots in hosts starting from the master
// host, and overloadds the master if it runs out of resources.
// - FORCE_LOCAL: force local execution irrespective of the available
// resources.
// - NEVER_ALONE: never allocates a single (non-master) request to a host
// without other requests of the batch.
// - UNDERFULL: schedule up to 50% of the master hosts' capacity to force
// migration opportunities to appear.
enum SchedulingTopologyHint
{
NORMAL,
FORCE_LOCAL,
NEVER_ALONE
NEVER_ALONE,
UNDERFULL,
};

// Map to convert input strings to scheduling topology hints and the other way
// around
const std::unordered_map<std::string, SchedulingTopologyHint>
strToTopologyHint = {
{ "NORMAL", SchedulingTopologyHint::NORMAL },
{ "FORCE_LOCAL", SchedulingTopologyHint::FORCE_LOCAL },
{ "NEVER_ALONE", SchedulingTopologyHint::NEVER_ALONE },
{ "UNDERFULL", SchedulingTopologyHint::UNDERFULL },
};

const std::unordered_map<SchedulingTopologyHint, std::string>
topologyHintToStr = {
{ SchedulingTopologyHint::NORMAL, "NORMAL" },
{ SchedulingTopologyHint::FORCE_LOCAL, "FORCE_LOCAL" },
{ SchedulingTopologyHint::NEVER_ALONE, "NEVER_ALONE" },
{ SchedulingTopologyHint::UNDERFULL, "UNDERFULL" },
};

// Migration strategies help the scheduler decide wether the scheduling decision
// for a batch request could be changed with the new set of available resources.
// - BIN_PACK: sort hosts by the number of functions from the batch they are
// running. Bin-pack batches in increasing order to hosts in
// decreasing order.
// - EMPTY_HOSTS: pack batches in increasing order to empty hosts.
enum MigrationStrategy
{
BIN_PACK,
EMPTY_HOSTS
};
}
24 changes: 24 additions & 0 deletions src/proto/faabric.proto
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ message BatchExecuteRequest {
FUNCTIONS = 0;
THREADS = 1;
PROCESSES = 2;
MIGRATION = 3;
}

BatchExecuteType type = 2;
Expand Down Expand Up @@ -164,6 +165,12 @@ message Message {
bool recordExecGraph = 41;
map<string, int32> intExecGraphDetails = 42;
map<string, string> execGraphDetails = 43;

// Function migration
int32 migrationCheckPeriod = 44;

// Scheduling
string topologyHint = 45;
}

// ---------------------------------------------
Expand Down Expand Up @@ -242,3 +249,20 @@ message PointToPointMappings {

repeated PointToPointMapping mappings = 3;
}

// ---------------------------------------------
// FUNCTION MIGRATIONS
// ---------------------------------------------

message PendingMigrations {
int32 appId = 1;
int32 groupId = 2;

message PendingMigration {
Message msg = 1;
string srcHost = 2;
string dstHost = 3;
}

repeated PendingMigration migrations = 3;
}
1 change: 1 addition & 0 deletions src/scheduler/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,7 @@ faabric_lib(scheduler
Executor.cpp
FunctionCallClient.cpp
FunctionCallServer.cpp
FunctionMigrationThread.cpp
MpiContext.cpp
MpiMessageBuffer.cpp
MpiWorld.cpp
Expand Down
26 changes: 26 additions & 0 deletions src/scheduler/Executor.cpp
Original file line number Diff line number Diff line change
@@ -1,4 +1,5 @@
#include <faabric/proto/faabric.pb.h>
#include <faabric/scheduler/MpiWorldRegistry.h>
#include <faabric/scheduler/Scheduler.h>
#include <faabric/snapshot/SnapshotRegistry.h>
#include <faabric/state/State.h>
Expand Down Expand Up @@ -528,9 +529,28 @@ void Executor::threadPoolThread(int threadPoolIdx)

// Execute the task
int32_t returnValue;
bool migrated = false;
try {
returnValue =
executeTask(threadPoolIdx, task.messageIndex, task.req);
} catch (const faabric::util::FunctionMigratedException& ex) {
SPDLOG_DEBUG(
"Task {} migrated, shutting down executor {}", msg.id(), id);

// Note that when a task has been migrated, we need to perform all
// the normal executor shutdown, but we must NOT set the result for
// the call.
migrated = true;
selfShutdown = true;
returnValue = -99;

// MPI migration
if (msg.ismpi()) {
auto& mpiWorld =
faabric::scheduler::getMpiWorldRegistry().getWorld(
msg.mpiworldid());
mpiWorld.destroy();
}
} catch (const std::exception& ex) {
returnValue = 1;

Expand Down Expand Up @@ -667,6 +687,12 @@ void Executor::threadPoolThread(int threadPoolIdx)
// executor.
sch.vacateSlot();

// If the function has been migrated, we drop out here and shut down the
// executor
if (migrated) {
break;
}

// 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
Expand Down
Loading

0 comments on commit 4f4628a

Please sign in to comment.