You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by se...@apache.org on 2017/01/16 10:49:07 UTC

flink git commit: [FLINK-5457] [docs] Add documentation for asynchronous I/O

Repository: flink
Updated Branches:
  refs/heads/master e2ba042c1 -> fb3761b57


[FLINK-5457] [docs] Add documentation for asynchronous I/O


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/fb3761b5
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/fb3761b5
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/fb3761b5

Branch: refs/heads/master
Commit: fb3761b578d6a7d956ca26a8be2ee88b3fbf8c46
Parents: e2ba042
Author: Stephan Ewen <se...@apache.org>
Authored: Sun Jan 15 19:41:36 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Jan 16 11:47:57 2017 +0100

----------------------------------------------------------------------
 .gitignore                 |   1 +
 docs/dev/stream/asyncio.md | 226 ++++++++++++++++++++++++++-
 docs/fig/async_io.svg      | 337 ++++++++++++++++++++++++++++++++++++++++
 3 files changed, 562 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/fb3761b5/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index 1b9c64e..9012d0a 100644
--- a/.gitignore
+++ b/.gitignore
@@ -29,5 +29,6 @@ out/
 /docs/.rubydeps
 /docs/ruby2/.bundle
 /docs/ruby2/.rubydeps
+/docs/.jekyll-metadata
 *.ipr
 *.iws

http://git-wip-us.apache.org/repos/asf/flink/blob/fb3761b5/docs/dev/stream/asyncio.md
----------------------------------------------------------------------
diff --git a/docs/dev/stream/asyncio.md b/docs/dev/stream/asyncio.md
index 2d0867a..abc0b24 100644
--- a/docs/dev/stream/asyncio.md
+++ b/docs/dev/stream/asyncio.md
@@ -1,5 +1,6 @@
 ---
-title: "Async I/O for External Data Access"
+title: "Asynchronous I/O for External Data Access"
+nav-title: "Async I/O"
 nav-parent_id: streaming
 nav-pos: 60
 ---
@@ -25,4 +26,225 @@ under the License.
 * ToC
 {:toc}
 
-**TDB**
+This page explains the use of Flink's API for asynchronous I/O with external data stores.
+For users not familiar with asynchronous or event-driven programming, an article about Futures and
+event-driven programming may be useful preparation.
+
+Note: Details about the design and implementation of the asynchronous I/O utility can be found in the proposal and design document
+[FLIP-12: Asynchronous I/O Design and Implementation](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=65870673).
+
+
+## The need for Asynchronous I/O Operations
+
+When interacting with external systems (for example when enriching stream events with data stored in a database), one needs to take care
+that communication delay with the external system does not dominate the streaming application's total work.
+
+Naively accessing data in the external database, for example in a `MapFunction`, typically means **synchronous** interaction:
+A request is sent to the database and the `MapFunction` waits until the response has been received. In many cases, this waiting
+makes up the vast majority of the function's time.
+
+Asynchronous interaction with the database means that a single parallel function instance can handle many requests concurrently and
+receive the responses concurrently. That way, the waiting time can be overlayed with sending other requests and
+receiving responses. At the very least, the waiting time is amortized over multiple requests. This leads in most cased to much higher
+streaming throughput.
+
+<img src="../../fig/async_io.svg" class="center" width="50%" />
+
+*Note:* Improving throughput by just scaling the `MapFunction` to a very high parallelism is in some cases possible as well, but usually
+comes at a very high resource cost: Having many more parallel MapFunction instances means more tasks, threads, Flink-internal network
+connections, network connections to the database, buffers, and general internal bookkeeping overhead.
+
+
+## Prerequisites
+
+As illustrated in the section above, implementing proper asynchronous I/O to a database (or key/value store) requires a client
+to that database that supports asynchronous requests. Many popular databases offer such a client.
+
+In the absence of such a client, one can try and turn a synchronous client into a limited concurrent client by creating
+multiple clients and handling the synchronous calls with a thread pool. However, this approach is usually less
+efficient than a proper asynchronous client.
+
+
+## Async I/O API
+
+Flink's Async I/O API allows users to use asynchronous request clients with data streams. The API handles the integration with
+data streams, well as handling order, event time, fault tolerance, etc.
+
+Assuming one has an asynchronous client for the target database, three parts are needed to implement a stream transformation
+with asynchronous I/O against the database:
+
+  - An implementation of `AsyncFunction` that dispatches the requests
+  - A *callback* that takes the result of the operation and hands it to the `AsyncCollector`
+  - Applying the async I/O operation on a DataStream as a transformation
+
+The following code example illustrates the basic pattern:
+
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+{% highlight java %}
+// This example implements the asynchronous request and callback with Futures that have the
+// interface of Java 8's futures (which is the same one followed by Flink's Future)
+
+/**
+ * An implementation of the 'AsyncFunction' that sends requests and sets the callback.
+ */
+class AsyncDatabaseRequest extends RichAsyncFunction<String, Tuple2<String, String>> {
+
+    /** The database specific client that can issue concurrent requests with callbacks */
+    private transient DatabaseClient client;
+
+    @Override
+    public void open(Configuration parameters) throws Exception {
+        client = new DatabaseClient(host, post, credentials);
+    }
+
+    @Override
+    public void close() throws Exception {
+        client.close();
+    }
+
+    @Override
+    public void asyncInvoke(final String str, final AsyncCollector<Tuple2<String, String>> asyncCollector) throws Exception {
+
+        // issue the asynchronous request, receive a future for result
+        Future<String> resultFuture = client.query(str);
+
+        // set the callback to be executed once the request by the client is complete
+        // the callback simply forwards the result to the collector
+        resultFuture.thenAccept( (String result) -> {
+
+            asyncCollector.collect(Collections.singleton(new Tuple2<>(str, result)));
+         
+        });
+    }
+}
+
+// create the original stream
+DataStream<String> stream = ...;
+
+// apply the async I/O transformation
+DataStream<Tuple2<String, String>> resultStream =
+    AsyncDataStream.unorderedWait(stream, new AsyncDatabaseRequest(), 1000, TimeUnit.MILLISECONDS, 100);
+
+{% endhighlight %}
+</div>
+<div data-lang="scala" markdown="1">
+{% highlight scala %}
+/**
+ * An implementation of the 'AsyncFunction' that sends requests and sets the callback.
+ */
+class AsyncDatabaseRequest extends AsyncFunction[String, (String, String)] {
+
+    /** The database specific client that can issue concurrent requests with callbacks */
+    lazy val client: DatabaseClient = new DatabaseClient(host, post, credentials)
+
+    /** The context used for the future callbacks */
+    implicit lazy val executor: ExecutionContext = ExecutionContext.fromExecutor(Executors.directExecutor()))
+
+
+    override def asyncInvoke(str: String, asyncCollector: AsyncCollector[(String, String)]): Unit = {
+
+        // issue the asynchronous request, receive a future for the result
+        val resultFuture: Future[String] = client.query(str)
+
+        // set the callback to be executed once the request by the client is complete
+        // the callback simply forwards the result to the collector
+        resultFuture.onSuccess {
+            case result: String => asyncCollector.collect(Collections.singleton((str, result)));
+        })
+    }
+}
+
+// create the original stream
+val stream: DataStream[String] = ...
+
+// apply the async I/O transformation
+val resultStream: DataStream[(String, String)] =
+    AsyncDataStream.unorderedWait(stream, new AsyncDatabaseRequest(), 1000, TimeUnit.MILLISECONDS, 100)
+
+{% endhighlight %}
+</div>
+</div>
+
+The following two parameters control the asynchronous operations:
+
+  - **Timeout**: The timeout defines how long an asynchronous request may take before it is considered failed. This parameter
+    guards against dead/failed requests.
+
+  - **Capacity**: This parameter defines how many asynchronous requests may be in progress at the same time.
+    Even though the async I/O approach leads typically to much better throughput, the operator can still be the bottleneck in
+    the streaming application. Limiting the number of concurrent requests ensures that the operator will not
+    accumulate an ever-growing backlog of pending requests, but that it will trigger backpressure once the capacity
+    is exhausted.
+
+
+### Order of Results
+
+The concurrent requests issued by the `AsyncFunction` frequently complete in some undefined order, based on which request finished first.
+To control in which order the resulting records are emitted, Flink offers two modes:
+
+  - **Unordered**: Result records are emitted as soon as the asynchronous request finishes.
+    The order of the records in the stream is different after the async I/O operator than before.
+    This mode has the lowest latency and lowest overhead, when used with *processing time* as the basic time characteristic.
+    Use `AsyncDataStream.unorderedWait(...)` for this mode.
+
+  - **Ordered**: In that case, the stream order is preserved. Result records are emitted in the same order as the asynchronous
+    requests are triggered (the order of the operators input records). To achieve that, the operator buffers a result record
+    until all its preceeding records are emitted (or timed out).
+    This usually introduces some amount of extra latency and some overhead in checkpointing, because records or results are maintained
+    in the checkpointed state for a longer time, compared to the unordered mode.
+    Use `AsyncDataStream.orderedWait(...)` for this mode.
+
+
+### Event Time
+
+When the streaming application works with [event time](../event_time.html), watermarks will be handled correctly by the
+asynchronous I/O operator. That means concretely the following for the two order modes:
+
+  - **Unordered**: Watermarks do not overtake records and vice versa, meaning watermarks establish an *order boundary*.
+    Records are emitted unordered only between watermarks.
+    A record occurring after a certain watermark will be emitted only after that watermark was emitted.
+    The watermark in turn will be emitted only after all result records from inputs before that watermark were emitted.
+
+    That means that in the presence of watermarks, the *unordered* mode introduces some of the same latency and management
+    overhead as the *ordered* mode does. The amount of that overhead depends on the watermark frequency.
+
+  - **Ordered**: Order of watermarks an records is preserved, just like order between records is preserved. There is no
+    significant change in overhead, compared to working with *processing time*.
+
+Please recall that *Ingestion Time* is a special case of *event time* with automatically generated watermarks that
+are based on the sources processing time.
+
+
+### Fault Tolerance Guarantees
+
+The asynchronous I/O operator offers full exactly-once fault tolerance guarantees. It stores the records for in-flight
+asynchronous requests in checkpoints and restores/re-triggers the requests when recovering from a failure.
+
+
+### Implementation Tips
+
+For implementations with *Futures* that have an *Executor* (or *ExecutionContext* in Scala) for callbacks, we suggets to use a `DirectExecutor`, because the
+callback typically does minimal work, and a `DirectExecutor` avoids an additional thread-to-thread handover overhead. The callback typically only hands
+the result to the `AsyncCollector`, which adds it to the output buffer. From there, the heavy logic that includes record emission and interaction
+with the checkpoint bookkeepting happens in a dedicated thread-pool anyways.
+
+A `DirectExecutor` can be obtained via `org.apache.flink.runtime.concurrent.Executors.directExecutor()` or
+`com.google.common.util.concurrent.MoreExecutors.directExecutor()`.
+
+
+### Caveat
+
+**The AsyncFunction is not called Multi-Threaded**
+
+A common confusion that we want to explicitly point out here is that the `AsyncFunction` is not called in a multi-threaded fashion.
+There exists only one instance of the `AsyncFunction` and it is called sequentially for each record in the respective partition
+of the stream. Unless the `asyncInvoke(...)` method returns fast and relies on a callback (by the client), it will not result in
+proper asynchronous I/O.
+
+For example, the following patterns result in a blocking `asyncInvoke(...)` functions and thus void the asynchronous behavior:
+
+  - Using a database client whose lookup/query method call blocks until the result has been received back
+
+  - Blocking/waiting on the future-type objects returned by an aynchronous client inside the `asyncInvoke(...)` method
+

http://git-wip-us.apache.org/repos/asf/flink/blob/fb3761b5/docs/fig/async_io.svg
----------------------------------------------------------------------
diff --git a/docs/fig/async_io.svg b/docs/fig/async_io.svg
new file mode 100644
index 0000000..0dd5c5d
--- /dev/null
+++ b/docs/fig/async_io.svg
@@ -0,0 +1,337 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+
+<svg
+   xmlns:dc="http://purl.org/dc/elements/1.1/"
+   xmlns:cc="http://creativecommons.org/ns#"
+   xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#"
+   xmlns:svg="http://www.w3.org/2000/svg"
+   xmlns="http://www.w3.org/2000/svg"
+   version="1.1"
+   width="490.91293"
+   height="326.60162"
+   id="svg2">
+  <defs
+     id="defs4" />
+  <metadata
+     id="metadata7">
+    <rdf:RDF>
+      <cc:Work
+         rdf:about="">
+        <dc:format>image/svg+xml</dc:format>
+        <dc:type
+           rdf:resource="http://purl.org/dc/dcmitype/StillImage" />
+        <dc:title></dc:title>
+      </cc:Work>
+    </rdf:RDF>
+  </metadata>
+  <g
+     transform="translate(-223.11497,-374.77565)"
+     id="layer1">
+    <g
+       transform="translate(186.29955,360.58613)"
+       id="g2989">
+      <path
+         d="m 287.49975,36.027725 0,5.007498 -1.24719,0 0,-5.007498 1.24719,0 z m 0,8.758432 0,4.998121 -1.24719,0 0,-4.998121 1.24719,0 z m 0,8.749055 0,4.99812 -1.24719,0 0,-4.99812 1.24719,0 z m 0,8.749055 0,5.007497 -1.24719,0 0,-5.007497 1.24719,0 z m 0,8.758432 0,4.99812 -1.24719,0 0,-4.99812 1.24719,0 z m 0,8.749054 0,4.99812 -1.24719,0 0,-4.99812 1.24719,0 z m 0,8.749055 0,5.007497 -1.24719,0 0,-5.007497 1.24719,0 z m 0,8.758432 0,4.99812 -1.24719,0 0,-4.99812 1.24719,0 z m 0,8.74905 0,4.99812 -1.24719,0 0,-4.99812 1.24719,0 z m 0,8.74906 0,5.0075 -1.24719,0 0,-5.0075 1.24719,0 z m 0,8.75843 0,4.99812 -1.24719,0 0,-4.99812 1.24719,0 z m 0,8.74906 0,4.99812 -1.24719,0 0,-4.99812 1.24719,0 z m 0,8.74905 0,5.0075 -1.24719,0 0,-5.0075 1.24719,0 z m 0,8.75843 0,4.99812 -1.24719,0 0,-4.99812 1.24719,0 z m 0,8.74906 0,4.99812 -1.24719,0 0,-4.99812 1.24719,0 z m 0,8.74905 0,5.0075 -1.24719,0 0,-5.0075 1.24719,0 z m 0,8.75843 0,4.99812 -1.24719,0 0,-4.99812 1.24719,0 z m 0,8.74906 0,
 4.99812 -1.24719,0 0,-4.99812 1.24719,0 z m 0,8.74905 0,5.0075 -1.24719,0 0,-5.0075 1.24719,0 z m 0,8.75843 0,4.99812 -1.24719,0 0,-4.99812 1.24719,0 z m 0,8.74906 0,4.99812 -1.24719,0 0,-4.99812 1.24719,0 z m 0,8.74905 0,5.0075 -1.24719,0 0,-5.0075 1.24719,0 z m 0,8.75844 0,4.99812 -1.24719,0 0,-4.99812 1.24719,0 z m 0,8.74905 0,4.99812 -1.24719,0 0,-4.99812 1.24719,0 z m 0,8.74905 0,5.0075 -1.24719,0 0,-5.0075 1.24719,0 z m 0,8.75844 0,4.99812 -1.24719,0 0,-4.99812 1.24719,0 z m 0,8.74905 0,4.99812 -1.24719,0 0,-4.99812 1.24719,0 z m 0,8.74906 0,5.00749 -1.24719,0 0,-5.00749 1.24719,0 z m 0,8.75843 0,4.99812 -1.24719,0 0,-4.99812 1.24719,0 z m 0,8.74905 0,4.36984 -1.24719,0 0,-4.36984 1.24719,0 z"
+         id="path2991"
+         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:#000000;stroke-width:0.00937734px;stroke-linecap:butt;stroke-linejoin:round;stroke-opacity:1;stroke-dasharray:none" />
+      <path
+         d="m 315.47234,50.731388 0,21.399082 20.16127,0 0,-21.399082 -20.16127,0 z"
+         id="path2993"
+         style="fill:#5b9bd5;fill-opacity:1;fill-rule:evenodd;stroke:none" />
+      <text
+         x="321.22928"
+         y="67.309143"
+         id="text2995"
+         xml:space="preserve"
+         style="font-size:15.00373745px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">a</text>
+      <path
+         d="m 315.47234,73.855899 0,21.249044 20.16127,0 0,-21.249044 -20.16127,0 z"
+         id="path2997"
+         style="fill:#5b9bd5;fill-opacity:1;fill-rule:evenodd;stroke:none" />
+      <text
+         x="321.07925"
+         y="90.391762"
+         id="text2999"
+         xml:space="preserve"
+         style="font-size:15.00373745px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">b</text>
+      <path
+         d="m 315.47234,96.830373 0,21.417837 20.16127,0 0,-21.417837 -20.16127,0 z"
+         id="path3001"
+         style="fill:#5b9bd5;fill-opacity:1;fill-rule:evenodd;stroke:none" />
+      <text
+         x="321.82944"
+         y="113.47438"
+         id="text3003"
+         xml:space="preserve"
+         style="font-size:15.00373745px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">c</text>
+      <path
+         d="m 315.47234,119.95488 0,21.41784 20.16127,0 0,-21.41784 -20.16127,0 z"
+         id="path3005"
+         style="fill:#5b9bd5;fill-opacity:1;fill-rule:evenodd;stroke:none" />
+      <text
+         x="321.07925"
+         y="136.55701"
+         id="text3007"
+         xml:space="preserve"
+         style="font-size:15.00373745px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">d</text>
+      <path
+         d="m 315.47234,189.81604 0,21.2678 20.16127,0 0,-21.2678 -20.16127,0 z"
+         id="path3009"
+         style="fill:#548235;fill-opacity:1;fill-rule:evenodd;stroke:none" />
+      <text
+         x="321.07925"
+         y="206.38158"
+         id="text3011"
+         xml:space="preserve"
+         style="font-size:15.00373745px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">b</text>
+      <path
+         d="m 315.47234,212.79051 0,21.41784 20.16127,0 0,-21.41784 -20.16127,0 z"
+         id="path3013"
+         style="fill:#548235;fill-opacity:1;fill-rule:evenodd;stroke:none" />
+      <text
+         x="321.82944"
+         y="229.4642"
+         id="text3015"
+         xml:space="preserve"
+         style="font-size:15.00373745px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">c</text>
+      <path
+         d="m 315.47234,235.93378 0,21.39908 20.16127,0 0,-21.39908 -20.16127,0 z"
+         id="path3017"
+         style="fill:#548235;fill-opacity:1;fill-rule:evenodd;stroke:none" />
+      <text
+         x="321.07925"
+         y="252.54683"
+         id="text3019"
+         xml:space="preserve"
+         style="font-size:15.00373745px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">d</text>
+      <path
+         d="m 48.855921,319.54211 0,21.24904 20.161273,0 0,-21.24904 -20.161273,0 z"
+         id="path3021"
+         style="fill:#5b9bd5;fill-opacity:1;fill-rule:evenodd;stroke:none" />
+      <text
+         x="54.577576"
+         y="336.11618"
+         id="text3023"
+         xml:space="preserve"
+         style="font-size:15.00373745px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">x</text>
+      <path
+         d="m 527.10006,152.40047 c 0,4.68866 -17.04799,8.51462 -38.05323,8.51462 -21.02398,0 -38.07198,-3.82596 -38.07198,-8.51462"
+         id="path3025"
+         style="fill:none;stroke:#000000;stroke-width:1.25656307px;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+      <path
+         d="m 450.97485,152.40047 c 0,-4.70743 17.048,-8.53338 38.07198,-8.53338 21.00524,0 38.05323,3.82595 38.05323,8.53338 l 0,34.05848 c 0,4.70742 -17.04799,8.51462 -38.05323,8.51462 -21.02398,0 -38.07198,-3.8072 -38.07198,-8.51462 z"
+         id="path3027"
+         style="fill:none;stroke:#000000;stroke-width:1.25656307px;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+      <path
+         d="m 36.815422,50.722011 0,21.408459 20.31131,0 0,-21.408459 -20.31131,0 z"
+         id="path3029"
+         style="fill:#5b9bd5;fill-opacity:1;fill-rule:evenodd;stroke:none" />
+      <text
+         x="42.636166"
+         y="67.309143"
+         id="text3031"
+         xml:space="preserve"
+         style="font-size:15.00373745px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">a</text>
+      <path
+         d="m 36.815422,73.855899 0,67.516821 20.31131,0 0,-67.516821 -20.31131,0 z"
+         id="path3033"
+         style="fill:#c55a11;fill-opacity:1;fill-rule:evenodd;stroke:none" />
+      <path
+         d="m 36.815422,143.39822 0,21.25843 20.31131,0 0,-21.25843 -20.31131,0 z"
+         id="path3035"
+         style="fill:#548235;fill-opacity:1;fill-rule:evenodd;stroke:none" />
+      <text
+         x="42.636166"
+         y="159.92798"
+         id="text3037"
+         xml:space="preserve"
+         style="font-size:15.00373745px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">a</text>
+      <path
+         d="m 248.27435,152.39109 c 0,4.69804 -17.03862,8.51462 -38.06261,8.51462 -21.01461,0 -38.05323,-3.81658 -38.05323,-8.51462"
+         id="path3039"
+         style="fill:none;stroke:#000000;stroke-width:1.24718571px;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+      <path
+         d="m 172.15851,152.39109 c 0,-4.70742 17.03862,-8.524 38.05323,-8.524 21.02399,0 38.06261,3.81658 38.06261,8.524 l 0,34.06786 c 0,4.70742 -17.03862,8.51462 -38.06261,8.51462 -21.01461,0 -38.05323,-3.8072 -38.05323,-8.51462 z"
+         id="path3041"
+         style="fill:none;stroke:#000000;stroke-width:1.24718571px;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+      <path
+         d="m 36.815422,166.3727 0,21.41783 20.31131,0 0,-21.41783 -20.31131,0 z"
+         id="path3043"
+         style="fill:#5b9bd5;fill-opacity:1;fill-rule:evenodd;stroke:none" />
+      <text
+         x="42.48613"
+         y="183.0106"
+         id="text3045"
+         xml:space="preserve"
+         style="font-size:15.00373745px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">b</text>
+      <path
+         d="m 36.815422,189.50659 0,67.51682 20.31131,0 0,-67.51682 -20.31131,0 z"
+         id="path3047"
+         style="fill:#c55a11;fill-opacity:1;fill-rule:evenodd;stroke:none" />
+      <path
+         d="m 36.815422,259.05829 0,21.24904 20.31131,0 0,-21.24904 -20.31131,0 z"
+         id="path3049"
+         style="fill:#548235;fill-opacity:1;fill-rule:evenodd;stroke:none" />
+      <text
+         x="42.48613"
+         y="275.62946"
+         id="text3051"
+         xml:space="preserve"
+         style="font-size:15.00373745px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">b</text>
+      <path
+         d="m 315.47234,166.3727 0,21.41783 20.16127,0 0,-21.41783 -20.16127,0 z"
+         id="path3053"
+         style="fill:#548235;fill-opacity:1;fill-rule:evenodd;stroke:none" />
+      <text
+         x="321.22928"
+         y="183.01012"
+         id="text3055"
+         xml:space="preserve"
+         style="font-size:15.00373745px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">a</text>
+      <path
+         d="M 335.98995,60.840157 449.64327,136.459 448.94934,137.50926 335.29603,61.871664 z m 110.05242,68.079463 4.29482,8.75843 -9.73367,-0.5814 c -0.35634,-0.0188 -0.61891,-0.31883 -0.60015,-0.65641 0.0187,-0.35634 0.31883,-0.6189 0.65641,-0.58139 l 8.68341,0.50637 -0.60015,0.90023 -3.82595,-7.80195 c -0.15004,-0.31883 -0.0187,-0.69392 0.28132,-0.84396 0.31883,-0.15004 0.69392,-0.0188 0.84396,0.30008 z"
+         id="path3057"
+         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:#000000;stroke-width:0.01875467px;stroke-linecap:butt;stroke-linejoin:round;stroke-opacity:1;stroke-dasharray:none" />
+      <path
+         d="m 162.30293,159.99611 -103.966522,-5.27944 0.06564,-1.24719 103.966532,5.27007 z m -97.008539,-0.62828 -8.167659,-5.33571 8.664658,-4.48236 c 0.309452,-0.15942 0.684546,-0.0375 0.843961,0.26256 0.159414,0.30945 0.03751,0.68455 -0.262566,0.84396 l -7.717548,3.99475 0.05627,-1.0784 7.267435,4.74494 c 0.290697,0.18754 0.375093,0.58139 0.178169,0.87209 -0.187546,0.28132 -0.572017,0.36571 -0.862715,0.17817 z"
+         id="path3059"
+         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:#000000;stroke-width:0.00937734px;stroke-linecap:butt;stroke-linejoin:round;stroke-opacity:1;stroke-dasharray:none" />
+      <path
+         d="m 57.154864,177.70052 105.992026,-3.3102 -0.0375,-1.25656 -106.001403,3.31957 z m 98.940276,1.2003 8.27081,-5.17629 -8.57089,-4.65116 c -0.30945,-0.16879 -0.68454,-0.0563 -0.85334,0.25319 -0.15941,0.30007 -0.0469,0.68454 0.25319,0.84396 l 7.63315,4.14478 -0.0281,-1.08777 -7.36121,4.61365 c -0.30007,0.17817 -0.38447,0.57202 -0.19692,0.86271 0.17817,0.2907 0.56264,0.3751 0.85334,0.19693 z"
+         id="path3061"
+         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:#000000;stroke-width:0.00937734px;stroke-linecap:butt;stroke-linejoin:round;stroke-opacity:1;stroke-dasharray:none" />
+      <path
+         d="m 163.98148,183.86143 -106.273353,84.52731 0.778318,0.97524 106.273355,-84.52731 z m -103.319493,76.69723 -3.535255,9.08664 9.658656,-1.39722 c 0.337584,-0.0469 0.581395,-0.36572 0.525131,-0.7033 -0.04689,-0.34696 -0.365716,-0.5814 -0.7033,-0.53451 l -8.599018,1.24718 0.675168,0.84396 3.141408,-8.09264 c 0.131283,-0.31883 -0.02813,-0.68454 -0.346961,-0.80645 -0.328207,-0.13128 -0.684546,0.0281 -0.815829,0.35634 z"
+         id="path3063"
+         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:#000000;stroke-width:0.00937734px;stroke-linecap:butt;stroke-linejoin:round;stroke-opacity:1;stroke-dasharray:none" />
+      <text
+         x="104.91452"
+         y="29.350618"
+         id="text3065"
+         xml:space="preserve"
+         style="font-size:19.95497131px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">Sync. I/O</text>
+      <text
+         x="387.67728"
+         y="29.350618"
+         id="text3067"
+         xml:space="preserve"
+         style="font-size:19.95497131px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">Async</text>
+      <text
+         x="446.49194"
+         y="29.350618"
+         id="text3069"
+         xml:space="preserve"
+         style="font-size:19.95497131px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">. I/O</text>
+      <path
+         d="m 335.93369,83.927158 110.5963,58.870912 -0.60015,1.08778 -110.57755,-58.852166 z m 106.26397,51.706632 5.13878,8.28956 -9.75243,0.3751 c -0.35634,0.0188 -0.63766,-0.24381 -0.65641,-0.60015 -0.0188,-0.33759 0.26257,-0.63766 0.60015,-0.63766 l 8.68341,-0.33758 -0.50637,0.93773 -4.57614,-7.37059 c -0.1688,-0.30007 -0.0938,-0.69392 0.2063,-0.86271 0.30007,-0.18755 0.67517,-0.0938 0.86271,0.2063 z"
+         id="path3071"
+         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:#000000;stroke-width:0.01875467px;stroke-linecap:butt;stroke-linejoin:round;stroke-opacity:1;stroke-dasharray:none" />
+      <path
+         d="m 335.87743,107.03291 110.54003,45.70514 -0.46886,1.14404 -110.54004,-45.68639 z m 105.53254,38.99097 5.92647,7.76443 -9.67741,1.31283 c -0.33758,0.0563 -0.65641,-0.18755 -0.69392,-0.52513 -0.0563,-0.33759 0.18755,-0.65641 0.52513,-0.71268 l 8.60839,-1.18154 -0.4126,1.01275 -5.27006,-6.92047 c -0.2063,-0.26257 -0.15004,-0.65642 0.13128,-0.88147 0.26257,-0.20631 0.65641,-0.15004 0.86272,0.13128 z"
+         id="path3073"
+         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:#000000;stroke-width:0.01875467px;stroke-linecap:butt;stroke-linejoin:round;stroke-opacity:1;stroke-dasharray:none" />
+      <path
+         d="m 335.80241,129.98863 108.51453,29.68865 -0.33758,1.2003 -108.49578,-29.68865 z m 102.70058,23.64965 6.84546,6.95798 -9.4336,2.51312 c -0.33759,0.0938 -0.67517,-0.0938 -0.76894,-0.43135 -0.0938,-0.33759 0.11252,-0.67517 0.45011,-0.76894 l 8.38334,-2.25057 -0.28132,1.05027 -6.07652,-6.2078 c -0.24381,-0.24381 -0.24381,-0.63766 0,-0.88147 0.24381,-0.24381 0.63766,-0.22506 0.88147,0.0188 z"
+         id="path3075"
+         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:#000000;stroke-width:0.01875467px;stroke-linecap:butt;stroke-linejoin:round;stroke-opacity:1;stroke-dasharray:none" />
+      <path
+         d="m 440.77231,172.01785 -103.91964,4.44486 0.0563,1.23781 103.91964,-4.44486 z m -96.92415,-0.15003 -8.21455,5.27006 8.62715,4.55738 c 0.31883,0.15004 0.69393,0.0375 0.86272,-0.26256 0.15004,-0.31883 0.0375,-0.69392 -0.26257,-0.84396 l -7.68941,-4.05101 0.0375,1.06902 7.31432,-4.68867 c 0.30007,-0.18755 0.37509,-0.5814 0.18755,-0.86272 -0.18755,-0.30007 -0.56264,-0.37509 -0.86272,-0.18754 z"
+         id="path3077"
+         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:#000000;stroke-width:0.01875467px;stroke-linecap:butt;stroke-linejoin:round;stroke-opacity:1;stroke-dasharray:none" />
+      <path
+         d="m 440.65978,175.76879 -103.95715,23.85594 0.28132,1.21906 103.95715,-23.85595 z m -97.89939,18.06075 -7.12678,6.67666 9.32108,2.90698 c 0.31883,0.11252 0.67517,-0.075 0.78769,-0.41261 0.0938,-0.31883 -0.0938,-0.67517 -0.4126,-0.78769 l -8.28956,-2.58815 0.24381,1.06902 6.32032,-5.94523 c 0.26257,-0.24381 0.26257,-0.63766 0.0375,-0.88147 -0.24381,-0.26257 -0.63766,-0.28132 -0.88147,-0.0375 z"
+         id="path3079"
+         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:#000000;stroke-width:0.01875467px;stroke-linecap:butt;stroke-linejoin:round;stroke-opacity:1;stroke-dasharray:none" />
+      <path
+         d="m 440.56601,180.34493 -104.01342,42.23552 0.46887,1.14404 104.01341,-42.23553 z m -98.96841,35.5401 -5.96399,7.72693 9.65866,1.38784 c 0.35634,0.0563 0.65641,-0.18754 0.71268,-0.52513 0.0563,-0.33758 -0.18755,-0.65641 -0.52513,-0.71268 l -8.6084,-1.21905 0.41261,0.994 5.30757,-6.88297 c 0.2063,-0.26256 0.15004,-0.65641 -0.11253,-0.88147 -0.28132,-0.2063 -0.67517,-0.15003 -0.88147,0.11253 z"
+         id="path3081"
+         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:#000000;stroke-width:0.01875467px;stroke-linecap:butt;stroke-linejoin:round;stroke-opacity:1;stroke-dasharray:none" />
+      <path
+         d="m 440.47223,183.19564 -104.08843,62.26551 0.63766,1.06902 104.10719,-62.26552 z m -100.13119,54.89492 -4.70743,8.53338 9.77119,-0.0938 c 0.33758,-0.0188 0.6189,-0.30008 0.60015,-0.63766 0,-0.33759 -0.28132,-0.61891 -0.61891,-0.61891 l -8.68341,0.0938 0.54389,0.93773 4.20104,-7.6144 c 0.1688,-0.30007 0.0563,-0.67516 -0.24381,-0.84396 -0.30007,-0.16879 -0.69392,-0.0563 -0.86271,0.24381 z"
+         id="path3083"
+         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:#000000;stroke-width:0.01875467px;stroke-linecap:butt;stroke-linejoin:round;stroke-opacity:1;stroke-dasharray:none" />
+      <path
+         d="m 57.529957,60.868289 106.273353,86.459041 -0.7877,0.97524 L 56.732884,61.834154 z m 103.403883,78.600831 3.43211,9.12415 -9.6399,-1.50038 c -0.33759,-0.0563 -0.57202,-0.37509 -0.51576,-0.71267 0.0469,-0.34696 0.36572,-0.57202 0.71268,-0.52513 l 8.58026,1.34096 -0.68454,0.83458 -3.05701,-8.12077 c -0.12191,-0.32821 0.0469,-0.68455 0.36571,-0.80646 0.32821,-0.1219 0.68455,0.0375 0.80645,0.36572 z"
+         id="path3085"
+         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:#000000;stroke-width:0.00937734px;stroke-linecap:butt;stroke-linejoin:round;stroke-opacity:1;stroke-dasharray:none" />
+      <text
+         x="178.40282"
+         y="180.86819"
+         id="text3087"
+         xml:space="preserve"
+         style="font-size:13.80343914px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">database</text>
+      <text
+         x="457.8309"
+         y="180.86819"
+         id="text3089"
+         xml:space="preserve"
+         style="font-size:13.80343914px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">database</text>
+      <path
+         d="m 315.47234,142.92936 0,21.2678 20.16127,0 0,-21.2678 -20.16127,0 z"
+         id="path3091"
+         style="fill:#c55a11;fill-opacity:1;fill-rule:evenodd;stroke:none" />
+      <text
+         x="78.076439"
+         y="335.54126"
+         id="text3093"
+         xml:space="preserve"
+         style="font-size:13.80343914px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">sendRequest</text>
+      <text
+         x="166.74852"
+         y="335.54126"
+         id="text3095"
+         xml:space="preserve"
+         style="font-size:13.80343914px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">(x)</text>
+      <path
+         d="m 242.32912,319.54211 0,21.24904 20.16127,0 0,-21.24904 -20.16127,0 z"
+         id="path3097"
+         style="fill:#548235;fill-opacity:1;fill-rule:evenodd;stroke:none" />
+      <text
+         x="248.05853"
+         y="336.11618"
+         id="text3099"
+         xml:space="preserve"
+         style="font-size:15.00373745px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">x</text>
+      <text
+         x="271.55743"
+         y="335.54126"
+         id="text3101"
+         xml:space="preserve"
+         style="font-size:13.80343914px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">receiveResponse</text>
+      <text
+         x="387.23621"
+         y="335.54126"
+         id="text3103"
+         xml:space="preserve"
+         style="font-size:13.80343914px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">(x)</text>
+      <path
+         d="m 457.23891,319.54211 0,21.24904 20.16127,0 0,-21.24904 -20.16127,0 z"
+         id="path3105"
+         style="fill:#c55a11;fill-opacity:1;fill-rule:evenodd;stroke:none" />
+      <text
+         x="486.51855"
+         y="335.54126"
+         id="text3107"
+         xml:space="preserve"
+         style="font-size:13.80343914px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">wait</text>
+    </g>
+  </g>
+</svg>