Skip to content
Avi Kivity edited this page Feb 16, 2015 · 20 revisions

Seastar

Introduction

SeaStar is an event-driven framework allowing you to write non-blocking, asynchronous code in a relatively straightforward manner (once understood). Its APIs are based on futures.

Seastar utilizes the following concepts to achieve extreme performance:

  • Cooperative micro-task scheduler: instead of running threads, each core runs a cooperative task scheduler. Each task is typically very lightweight -- only running for as long as it takes to process the last I/O operation's result and to submit a new one.
  • Share-nothing SMP architecture: each cores run independently of other cores in an SMP system. Memory, data structures, and CPU time are not shared; instead, inter-core communication uses explicit message passing. A seastar core is often termed a shard.
  • Future based APIs: futures allow you to submit an I/O operation and to chain tasks to be executed on completion of the I/O operation. It is easy to run multiple I/O operations in parallel - for example, in response to a request coming from a TCP connection, you can issue multiple disk I/O requests, send messages to other cores on the same system, or send requests to other nodes in the cluster, wait for some or all of the results to complete, aggregate the results, and send a response.
  • Share-nothing TCP stack: while seastar can use the host operating system's TCP stack, it also provides its own high-performance TCP/IP stack built on top of the task scheduler and the share-nothing architecture. The stack provides zero-copy in both directions: you can process data directly from the TCP stack's buffers, and send the contents of your own data structures as part of a message without incurring a copy.
  • DMA-based storage APIs: as with the networking stack, seastar provides zero-copy storage APIs, allowing you to DMA your data to and from your storage devices.

Building

See Building Seastar and Running unit tests.

Futures and promises

A future is a result of a computation that may not be available yet. Examples include:

  • a data buffer that we are reading from the network
  • the expiration of a timer
  • the completion of a disk write
  • the result computation that requires the values from one or more other futures.

a promise is an object or function that provides you with a future, with the expectation that it will fulfill the future.

Promises and futures simplify asynchronous programming since they decouple the event producer (the promise) and the event consumer (whoever uses the future). Whether the promise is fulfilled before the future is consumed, or vice versa, does not change the outcome of the code.

Consuming a future

You consume a future by using its then() method, providing it with a callback (typically a lambda). For example, consider the following operation:

future<int> get();   // promises an int will be produced eventually
future<> put(int)    // promises to store an int

void f() {
    get().then([] (int value) {
        put(value + 1).then([] {
            std::cout << "value stored successfully\n";
        });
    });
}

Here, we initate a get() operation, requesting that when it completes, a put() operation will be scheduled with an incremented value. We also request that when the put() completes, some text will be printed out.

Chaining futures

If a then() lambda returns a future (call it x), then that then() will return a future (call it y) that will receive the same value. This removes the need for nesting lambda blocks; for example the code above could be rewritten as:

future<int> get();   // promises an int will be produced eventually
future<> put(int)    // promises to store an int

void f() {
    get().then([] (int value) {
        return put(value + 1);
    }).then([] {
        std::cout << "value stored successfully\n";
    });
}

Loops

Loops are achieved with a tail call; for example:

future<int> get();   // promises an int will be produced eventually
future<> put(int)    // promises to store an int

future<> loop_to(int end) {
    if (value == end) {
        return make_ready_future<>();
    }
    get().then([end] (int value) {
        return put(value + 1);
    }).then([end] {
        return loop_to(end);
    });
}

The make_ready_future() function returns a future that is already available --- corresponding to the loop termination condition, where no further I/O needs to take place.

Another way to compose an asynchronous action to form a loop is to use do_until():

#include <core/async-action.hh>

do_until([] { return _done; }, [] {
   return get().then([] (int value) {
        return put(value + 1);
   });
});

Infinite loop can be achieved with:

#include <core/async-action.hh>

keep_doing([] {
    return get().then([] (int value) {
        return put(value + 1);
    });
});

Under the hood

When the loop above runs, both then method calls execute immediately --- but without executing the bodies. What happens is the following:

  1. get() is called, initiates the I/O operation, and allocates a temporary structure (call it f1).
  2. The first then() call chains its body to f1 and allocates another temporary structure, f2.
  3. The second then() call chains its body to f2.

Again, all this runs immediately without waiting for anything.

After the I/O operation initiated by get() completes, it calls the continuation stored in f1, calls it, and frees f1. The continuation calls put(), which initiates the I/O operation required to perform the store, and allocates a temporary object f12, and chains some glue code to it.

After the I/O operation initiated by put() completes, it calls the continuation associated with f12, which simply tells it to call the continuation assoicated with f2. This continuation simply calls loop_to(). Both f12 and f2 are freed. loop_to() then calls get(), which starts the process all over again, allocating new versions of f1 and f2.

Handling exceptions

If a .then() clause throws an exception, the scheduler will catch it and cancel any dependent .then() clauses. If you want to trap the exception, add a .rescue() clause at the end:

future<buffer> receive();
request parse(buffer buf);
future<response> process(request req);
future<> send(response resp);

void f() {
    receive().then([] (buffer buf) {
        return process(parse(std::move(buf));
    }).then([] (response resp) {
        return send(std::move(resp));
    }).then([] {
        f();
    }).rescue([] (auto get_ex) {
        try {
            get_ex();
        } (catch std::exception& e) {
            // your handler goes here
        }
    });
}

When the get_ex variable is called as a function, it will rethrow the exception that aborted processing, and you can then apply any needed error handling. It is essentially a transformation of

buffer receive();
request parse(buffer buf);
response process(request req);
void send(response resp);

void f() {
    try {
        while (true) {
            auto req = parse(receive());
            auto resp = process(std::move(req));
            send(std::move(resp));
        }
    } catch (std::exception& e) {
        // your handler goes here
    }
}
Clone this wiki locally