You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by zh...@apache.org on 2020/06/29 14:12:00 UTC

[flink] branch release-1.11 updated (87eb508 -> 44900b8)

This is an automated email from the ASF dual-hosted git repository.

zhijiang pushed a change to branch release-1.11
in repository https://gitbox.apache.org/repos/asf/flink.git.


    from 87eb508  [FLINK-17920][python][docs] Add the Python example of the Interval Join (#12779)
     new 1e1c81b  [FLINK-18064][python] Adding unaligned checkpoint config options.
     new fc1ad10  [hotfix][conf] Fix javadoc of CheckpointConfig#isUnalignedCheckpointsEnabled.
     new d54fea5  [hotfix][docs] Fix broken link in metrics.md.
     new 5afb3a6  [FLINK-18064][docs] Added unaligned checkpointing to docs.
     new 44900b8  [hotfix][docs] Replace/fix links in checkpointing documents.

The 5 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../execution_checkpointing_configuration.html     |    2 +-
 docs/concepts/stateful-stream-processing.md        |   58 +-
 docs/dev/stream/state/checkpointing.md             |   23 +-
 docs/fig/stream_aligning.svg                       | 1221 ++++++--------------
 docs/fig/stream_unaligning.svg                     |  383 ++++++
 docs/monitoring/metrics.md                         |    2 +-
 docs/ops/state/checkpoints.md                      |   49 +-
 docs/ops/state/large_state_tuning.md               |    2 +
 .../pyflink/datastream/checkpoint_config.py        |   39 +
 .../datastream/tests/test_check_point_config.py    |   16 +
 .../api/environment/CheckpointConfig.java          |   21 +-
 .../environment/ExecutionCheckpointingOptions.java |    5 +-
 12 files changed, 945 insertions(+), 876 deletions(-)
 create mode 100644 docs/fig/stream_unaligning.svg


[flink] 01/05: [FLINK-18064][python] Adding unaligned checkpoint config options.

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhijiang pushed a commit to branch release-1.11
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 1e1c81b7089b07a4c0ab5c7881a6617b5b27ac57
Author: Arvid Heise <ar...@ververica.com>
AuthorDate: Thu Jun 18 09:51:12 2020 +0200

    [FLINK-18064][python] Adding unaligned checkpoint config options.
---
 .../pyflink/datastream/checkpoint_config.py        | 39 ++++++++++++++++++++++
 .../datastream/tests/test_check_point_config.py    | 16 +++++++++
 2 files changed, 55 insertions(+)

diff --git a/flink-python/pyflink/datastream/checkpoint_config.py b/flink-python/pyflink/datastream/checkpoint_config.py
index 3812c39..084939b 100644
--- a/flink-python/pyflink/datastream/checkpoint_config.py
+++ b/flink-python/pyflink/datastream/checkpoint_config.py
@@ -268,6 +268,45 @@ class CheckpointConfig(object):
         cleanup_mode = self._j_checkpoint_config.getExternalizedCheckpointCleanup()
         return ExternalizedCheckpointCleanup._from_j_externalized_checkpoint_cleanup(cleanup_mode)
 
+    def is_unaligned_checkpoints_enabled(self):
+        """
+        Returns whether unaligned checkpoints are enabled.
+
+        :return: ``True`` if unaligned checkpoints are enabled.
+        """
+        return self._j_checkpoint_config.isUnalignedCheckpointsEnabled()
+
+    def enable_unaligned_checkpoints(self, enabled=True):
+        """
+        Enables unaligned checkpoints, which greatly reduce checkpointing times under backpressure.
+
+        Unaligned checkpoints contain data stored in buffers as part of the checkpoint state, which
+        allows checkpoint barriers to overtake these buffers. Thus, the checkpoint duration becomes
+        independent of the current throughput as checkpoint barriers are effectively not embedded
+        into the stream of data anymore.
+
+        Unaligned checkpoints can only be enabled if :func:`get_checkpointing_mode` is
+        :data:`CheckpointingMode.EXACTLY_ONCE`.
+
+        :param enabled: ``True`` if a checkpoints should be taken in unaligned mode.
+        """
+        self._j_checkpoint_config.enableUnalignedCheckpoints(enabled)
+
+    def disable_unaligned_checkpoints(self):
+        """
+        Enables unaligned checkpoints, which greatly reduce checkpointing times under backpressure
+        (experimental).
+
+        Unaligned checkpoints contain data stored in buffers as part of the checkpoint state, which
+        allows checkpoint barriers to overtake these buffers. Thus, the checkpoint duration becomes
+        independent of the current throughput as checkpoint barriers are effectively not embedded
+        into the stream of data anymore.
+
+        Unaligned checkpoints can only be enabled if :func:`get_checkpointing_mode` is
+        :data:`CheckpointingMode.EXACTLY_ONCE`.
+        """
+        self.enable_unaligned_checkpoints(False)
+
 
 class ExternalizedCheckpointCleanup(object):
     """
diff --git a/flink-python/pyflink/datastream/tests/test_check_point_config.py b/flink-python/pyflink/datastream/tests/test_check_point_config.py
index 4a10bb7..67b22d2 100644
--- a/flink-python/pyflink/datastream/tests/test_check_point_config.py
+++ b/flink-python/pyflink/datastream/tests/test_check_point_config.py
@@ -135,3 +135,19 @@ class CheckpointConfigTests(PyFlinkTestCase):
         self.checkpoint_config.set_prefer_checkpoint_for_recovery(True)
 
         self.assertTrue(self.checkpoint_config.is_prefer_checkpoint_for_recovery())
+
+    def test_is_unaligned_checkpointing_enabled(self):
+
+        self.assertFalse(self.checkpoint_config.is_unaligned_checkpoints_enabled())
+
+        self.checkpoint_config.enable_unaligned_checkpoints()
+
+        self.assertTrue(self.checkpoint_config.is_unaligned_checkpoints_enabled())
+
+        self.checkpoint_config.disable_unaligned_checkpoints()
+
+        self.assertFalse(self.checkpoint_config.is_unaligned_checkpoints_enabled())
+
+        self.checkpoint_config.enable_unaligned_checkpoints(True)
+
+        self.assertTrue(self.checkpoint_config.is_unaligned_checkpoints_enabled())


[flink] 02/05: [hotfix][conf] Fix javadoc of CheckpointConfig#isUnalignedCheckpointsEnabled.

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhijiang pushed a commit to branch release-1.11
in repository https://gitbox.apache.org/repos/asf/flink.git

commit fc1ad10f81c11385ef9ea88d6dbd7684067714ef
Author: Arvid Heise <ar...@ververica.com>
AuthorDate: Mon Jun 22 13:27:01 2020 +0200

    [hotfix][conf] Fix javadoc of CheckpointConfig#isUnalignedCheckpointsEnabled.
---
 .../streaming/api/environment/CheckpointConfig.java | 21 ++++++++++++++++++---
 1 file changed, 18 insertions(+), 3 deletions(-)

diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/CheckpointConfig.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/CheckpointConfig.java
index 7644004..29190b0 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/CheckpointConfig.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/CheckpointConfig.java
@@ -382,7 +382,7 @@ public class CheckpointConfig implements java.io.Serializable {
 	}
 
 	/**
-	 * Enables unaligned checkpoints, which greatly reduces checkpointing times under backpressure.
+	 * Enables unaligned checkpoints, which greatly reduce checkpointing times under backpressure.
 	 *
 	 * <p>Unaligned checkpoints contain data stored in buffers as part of the checkpoint state, which allows
 	 * checkpoint barriers to overtake these buffers. Thus, the checkpoint duration becomes independent of the
@@ -399,9 +399,24 @@ public class CheckpointConfig implements java.io.Serializable {
 	}
 
 	/**
-	 * Returns whether checkpoints should be persisted externally.
+	 * Enables unaligned checkpoints, which greatly reduce checkpointing times under backpressure.
 	 *
-	 * @return <code>true</code> if checkpoints should be externalized.
+	 * <p>Unaligned checkpoints contain data stored in buffers as part of the checkpoint state, which allows
+	 * checkpoint barriers to overtake these buffers. Thus, the checkpoint duration becomes independent of the
+	 * current throughput as checkpoint barriers are effectively not embedded into the stream of data anymore.
+	 *
+	 * <p>Unaligned checkpoints can only be enabled if {@link #checkpointingMode} is
+	 * {@link CheckpointingMode#EXACTLY_ONCE}.
+	 */
+	@PublicEvolving
+	public void enableUnalignedCheckpoints() {
+		enableUnalignedCheckpoints(true);
+	}
+
+	/**
+	 * Returns whether unaligned checkpoints are enabled.
+	 *
+	 * @return <code>true</code> if unaligned checkpoints are enabled.
 	 */
 	@PublicEvolving
 	public boolean isUnalignedCheckpointsEnabled() {


[flink] 05/05: [hotfix][docs] Replace/fix links in checkpointing documents.

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhijiang pushed a commit to branch release-1.11
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 44900b804d857f0679121207b3c65b2a1db644ae
Author: Arvid Heise <ar...@ververica.com>
AuthorDate: Fri Jun 19 14:42:52 2020 +0200

    [hotfix][docs] Replace/fix links in checkpointing documents.
---
 docs/concepts/stateful-stream-processing.md |  2 +-
 docs/dev/stream/state/checkpointing.md      | 12 ++++++------
 docs/ops/state/checkpoints.md               |  2 +-
 3 files changed, 8 insertions(+), 8 deletions(-)

diff --git a/docs/concepts/stateful-stream-processing.md b/docs/concepts/stateful-stream-processing.md
index b95f483..45c916b 100644
--- a/docs/concepts/stateful-stream-processing.md
+++ b/docs/concepts/stateful-stream-processing.md
@@ -365,6 +365,6 @@ programs, with minor exceptions:
 
   - The DataSet API introduces special synchronized (superstep-based)
     iterations, which are only possible on bounded streams. For details, check
-    out the [iteration docs]({{ site.baseurl }}/dev/batch/iterations.html).
+    out the [iteration docs]({% link dev/batch/iterations.md %}).
 
 {% top %}
diff --git a/docs/dev/stream/state/checkpointing.md b/docs/dev/stream/state/checkpointing.md
index 147f4a1..43dc4b9 100644
--- a/docs/dev/stream/state/checkpointing.md
+++ b/docs/dev/stream/state/checkpointing.md
@@ -32,7 +32,7 @@ any type of more elaborate operation.
 In order to make state fault tolerant, Flink needs to **checkpoint** the state. Checkpoints allow Flink to recover state and positions
 in the streams to give the application the same semantics as a failure-free execution.
 
-The [documentation on streaming fault tolerance]({{ site.baseurl }}/learn-flink/fault_tolerance.html) describes in detail the technique behind Flink's streaming fault tolerance mechanism.
+The [documentation on streaming fault tolerance]({% link learn-flink/fault_tolerance.md %}) describes in detail the technique behind Flink's streaming fault tolerance mechanism.
 
 
 ## Prerequisites
@@ -72,7 +72,7 @@ Other parameters for checkpointing include:
 
     This option cannot be used when a minimum time between checkpoints is defined.
 
-  - *externalized checkpoints*: You can configure periodic checkpoints to be persisted externally. Externalized checkpoints write their meta data out to persistent storage and are *not* automatically cleaned up when the job fails. This way, you will have a checkpoint around to resume from if your job fails. There are more details in the [deployment notes on externalized checkpoints]({{ site.baseurl }}/ops/state/checkpoints.html#externalized-checkpoints).
+  - *externalized checkpoints*: You can configure periodic checkpoints to be persisted externally. Externalized checkpoints write their meta data out to persistent storage and are *not* automatically cleaned up when the job fails. This way, you will have a checkpoint around to resume from if your job fails. There are more details in the [deployment notes on externalized checkpoints]({% link ops/state/checkpoints.md %}#externalized-checkpoints).
 
   - *fail/continue task on checkpoint errors*: This determines if a task will be failed if an error occurs in the execution of the task's checkpoint procedure. This is the default behaviour. Alternatively, when this is disabled, the task will simply decline the checkpoint to the checkpoint coordinator and continue running.
 
@@ -175,7 +175,7 @@ env.get_checkpoint_config().enable_unaligned_checkpoints()
 
 ### Related Config Options
 
-Some more parameters and/or defaults may be set via `conf/flink-conf.yaml` (see [configuration]({{ site.baseurl }}/ops/config.html) for a full guide):
+Some more parameters and/or defaults may be set via `conf/flink-conf.yaml` (see [configuration]({% link ops/config.md %}) for a full guide):
 
 {% include generated/checkpointing_configuration.html %}
 
@@ -184,7 +184,7 @@ Some more parameters and/or defaults may be set via `conf/flink-conf.yaml` (see
 
 ## Selecting a State Backend
 
-Flink's [checkpointing mechanism]({{ site.baseurl }}/learn-flink/fault_tolerance.html) stores consistent snapshots
+Flink's [checkpointing mechanism]({% link learn-flink/fault_tolerance.md %}) stores consistent snapshots
 of all the state in timers and stateful operators, including connectors, windows, and any [user-defined state](state.html).
 Where the checkpoints are stored (e.g., JobManager memory, file system, database) depends on the configured
 **State Backend**. 
@@ -192,7 +192,7 @@ Where the checkpoints are stored (e.g., JobManager memory, file system, database
 By default, state is kept in memory in the TaskManagers and checkpoints are stored in memory in the JobManager. For proper persistence of large state,
 Flink supports various approaches for storing and checkpointing state in other state backends. The choice of state backend can be configured via `StreamExecutionEnvironment.setStateBackend(…)`.
 
-See [state backends]({{ site.baseurl }}/ops/state/state_backends.html) for more details on the available state backends and options for job-wide and cluster-wide configuration.
+See [state backends]({% link ops/state/state_backends.md %}) for more details on the available state backends and options for job-wide and cluster-wide configuration.
 
 
 ## State Checkpoints in Iterative Jobs
@@ -207,7 +207,7 @@ Please note that records in flight in the loop edges (and the state changes asso
 ## Restart Strategies
 
 Flink supports different restart strategies which control how the jobs are restarted in case of a failure. For more 
-information, see [Restart Strategies]({{ site.baseurl }}/dev/restart_strategies.html).
+information, see [Restart Strategies]({% link dev/task_failure_recovery.md %}).
 
 {% top %}
 
diff --git a/docs/ops/state/checkpoints.md b/docs/ops/state/checkpoints.md
index 9264231..c9d4f08 100644
--- a/docs/ops/state/checkpoints.md
+++ b/docs/ops/state/checkpoints.md
@@ -32,7 +32,7 @@ Checkpoints make state in Flink fault tolerant by allowing state and the
 corresponding stream positions to be recovered, thereby giving the application
 the same semantics as a failure-free execution.
 
-See [Checkpointing]({{ site.baseurl }}/dev/stream/state/checkpointing.html) for how to enable and
+See [Checkpointing]({% link dev/stream/state/checkpointing.md %}) for how to enable and
 configure checkpoints for your program.
 
 ## Retained Checkpoints


[flink] 03/05: [hotfix][docs] Fix broken link in metrics.md.

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhijiang pushed a commit to branch release-1.11
in repository https://gitbox.apache.org/repos/asf/flink.git

commit d54fea58993270f32f14401548dbf24918af96c0
Author: Arvid Heise <ar...@ververica.com>
AuthorDate: Fri Jun 19 13:03:53 2020 +0200

    [hotfix][docs] Fix broken link in metrics.md.
---
 docs/monitoring/metrics.md | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/docs/monitoring/metrics.md b/docs/monitoring/metrics.md
index b602936..9709960 100644
--- a/docs/monitoring/metrics.md
+++ b/docs/monitoring/metrics.md
@@ -586,7 +586,7 @@ metrics.reporter.my_other_reporter.port: 10000
 {% endhighlight %}
 
 **Important:** The jar containing the reporter must be accessible when Flink is started. Reporters that support the
- `factory.class` property can be loaded as [plugins]({{ site.baseurl }}/ops/plugins). Otherwise the jar must be placed
+ `factory.class` property can be loaded as [plugins]({% link ops/plugins.md %}). Otherwise the jar must be placed
  in the /lib folder. Reporters that are shipped with Flink (i.e., all reporters documented on this page) are available
  by default.
 


[flink] 04/05: [FLINK-18064][docs] Added unaligned checkpointing to docs.

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhijiang pushed a commit to branch release-1.11
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 5afb3a68ccb370a9f77649f89f0fcb22dc02fc06
Author: Arvid Heise <ar...@ververica.com>
AuthorDate: Thu Jun 18 10:10:11 2020 +0200

    [FLINK-18064][docs] Added unaligned checkpointing to docs.
    
    It's split into 3 parts to simulate the description of aligned checkpointing:
    - It's added on conceptual level in stateful-stream-processing.md with new/revised pics. It's written in a way that it could survive 1.12 without change.
    - A small change to dev/stream/state/checkpointing.md to show how it is enabled programmatically in Java/Scala/Python. Might need to be extended for 1.12 when new options become available (depending whether they can be programmatically changed or not).
    - A larger discussion in ops/state/checkpoints.md which includes the current limitations and a small glimpse into the next steps (will be in much more detail in blog post). This part needs to be largely rewritten for 1.12+ to reflect the new options.
---
 .../execution_checkpointing_configuration.html     |    2 +-
 docs/concepts/stateful-stream-processing.md        |   56 +-
 docs/dev/stream/state/checkpointing.md             |   11 +
 docs/fig/stream_aligning.svg                       | 1221 ++++++--------------
 docs/fig/stream_unaligning.svg                     |  383 ++++++
 docs/ops/state/checkpoints.md                      |   47 +
 docs/ops/state/large_state_tuning.md               |    2 +
 .../environment/ExecutionCheckpointingOptions.java |    5 +-
 8 files changed, 863 insertions(+), 864 deletions(-)

diff --git a/docs/_includes/generated/execution_checkpointing_configuration.html b/docs/_includes/generated/execution_checkpointing_configuration.html
index 506fa9d..94219b1 100644
--- a/docs/_includes/generated/execution_checkpointing_configuration.html
+++ b/docs/_includes/generated/execution_checkpointing_configuration.html
@@ -60,7 +60,7 @@
             <td><h5>execution.checkpointing.unaligned</h5></td>
             <td style="word-wrap: break-word;">false</td>
             <td>Boolean</td>
-            <td>Enables unaligned checkpoints, which greatly reduce checkpointing times under backpressure.<br /><br />Unaligned checkpoints contain data stored in buffers as part of the checkpoint state, which allows checkpoint barriers to overtake these buffers. Thus, the checkpoint duration becomes independent of the current throughput as checkpoint barriers are effectively not embedded into the stream of data anymore.<br /><br />Unaligned checkpoints can only be enabled if <span mark [...]
+            <td>Enables unaligned checkpoints, which greatly reduce checkpointing times under backpressure.<br /><br />Unaligned checkpoints contain data stored in buffers as part of the checkpoint state, which allows checkpoint barriers to overtake these buffers. Thus, the checkpoint duration becomes independent of the current throughput as checkpoint barriers are effectively not embedded into the stream of data anymore.<br /><br />Unaligned checkpoints can only be enabled if <span mark [...]
         </tr>
     </tbody>
 </table>
diff --git a/docs/concepts/stateful-stream-processing.md b/docs/concepts/stateful-stream-processing.md
index 112171a..b95f483 100644
--- a/docs/concepts/stateful-stream-processing.md
+++ b/docs/concepts/stateful-stream-processing.md
@@ -140,6 +140,8 @@ Keep in mind that everything to do with checkpointing can be done
 asynchronously. The checkpoint barriers don't travel in lock step and
 operations can asynchronously snapshot their state.
 
+Since Flink 1.11, checkpoints can be taken with or without alignment. In this 
+section, we describe aligned checkpoints first.
 
 #### Barriers
 
@@ -189,14 +191,16 @@ streams on the snapshot barriers. The figure above illustrates this:
     received the barrier *n* from the other inputs as well. Otherwise, it would
     mix records that belong to snapshot *n* and with records that belong to
     snapshot *n+1*.
-  - Streams that report barrier *n* are temporarily set aside. Records that are
-    received from these streams are not processed, but put into an input
-    buffer.
   - Once the last stream has received barrier *n*, the operator emits all
     pending outgoing records, and then emits snapshot *n* barriers itself.
-  - After that, it resumes processing records from all input streams,
+  - It snapshots the state and resumes processing records from all input streams,
     processing records from the input buffers before processing the records
     from the streams.
+  - Finally, the operator writes the state asynchronously to the state backend.
+  
+Note that the alignment is needed for all operators with multiple inputs and for 
+operators after a shuffle when they consume output streams of multiple upstream 
+subtasks.
 
 #### Snapshotting Operator State
 
@@ -206,7 +210,7 @@ snapshots as well.
 Operators snapshot their state at the point in time when they have received all
 snapshot barriers from their input streams, and before emitting the barriers to
 their output streams. At that point, all updates to the state from records
-before the barriers will have been made, and no updates that depend on records
+before the barriers have been made, and no updates that depend on records
 from after the barriers have been applied. Because the state of a snapshot may
 be large, it is stored in a configurable *[state backend]({{ site.baseurl }}{%
 link ops/state/state_backends.md %})*. By default, this is the JobManager's
@@ -242,6 +246,48 @@ updates to that state.
 See [Restart Strategies]({% link dev/task_failure_recovery.md
 %}#restart-strategies) for more information.
 
+### Unaligned Checkpointing
+
+Starting with Flink 1.11, checkpointing can also be performed unaligned.
+The basic idea is that checkpoints can overtake all in-flight data as long as 
+the in-flight data becomes part of the operator state.
+
+Note that this approach is actually closer to the [Chandy-Lamport algorithm
+](http://research.microsoft.com/en-us/um/people/lamport/pubs/chandy.pdf), but
+Flink still inserts the barrier in the sources to avoid overloading the
+checkpoint coordinator.
+
+<div style="text-align: center">
+  <img src="{% link fig/stream_unaligning.svg %}" alt="Unaligned checkpointing" style="width:100%; padding-top:10px; padding-bottom:10px;" />
+</div>
+
+The figure depicts how an operator handles unaligned checkpoint barriers:
+
+- The operator reacts on the first barrier that is stored in its input buffers.
+- It immediately forwards the barrier to the downstream operator by adding it 
+  to the end of the output buffers.
+- The operator marks all overtaken records to be stored asynchronously and 
+  creates a snapshot of its own state.
+ 
+Consequently, the operator only briefly stops the processing of input to mark
+the buffers, forwards the barrier, and creates the snapshot of the other state.
+  
+Unaligned checkpointing ensures that barriers are arriving at the sink as fast 
+as possible. It's especially suited for applications with at least one slow 
+moving data path, where alignment times can reach hours. However, since it's
+adding additional I/O pressure, it doesn't help when the I/O to the state 
+backends is the bottleneck. See the more in-depth discussion in 
+[ops]({% link ops/state/checkpoints.md %}#unaligned-checkpoints)
+for other limitations.
+
+Note that savepoints will always be aligned.
+
+#### Unaligned Recovery
+
+Operators first recover the in-flight data before starting processing any data
+from upstream operators in unaligned checkpointing. Aside from that, it 
+performs the same steps as during [recovery of aligned checkpoints](#recovery).
+
 ### State Backends
 
 The exact data structures in which the key/values indexes are stored depends on
diff --git a/docs/dev/stream/state/checkpointing.md b/docs/dev/stream/state/checkpointing.md
index 496b540..147f4a1 100644
--- a/docs/dev/stream/state/checkpointing.md
+++ b/docs/dev/stream/state/checkpointing.md
@@ -78,6 +78,8 @@ Other parameters for checkpointing include:
 
   - *prefer checkpoint for recovery*: This determines if a job will fallback to latest checkpoint even when there are more recent savepoints available to potentially reduce recovery time.
 
+  - *unaligned checkpoints*: You can enable [unaligned checkpoints]({% link ops/state/checkpoints.md %}#unaligned-checkpoints) to greatly reduce checkpointing times under backpressure. Only works for exactly-once checkpoints and with number of concurrent checkpoints of 1.
+
 <div class="codetabs" markdown="1">
 <div data-lang="java" markdown="1">
 {% highlight java %}
@@ -105,6 +107,9 @@ env.getCheckpointConfig().enableExternalizedCheckpoints(ExternalizedCheckpointCl
 
 // allow job recovery fallback to checkpoint when there is a more recent savepoint
 env.getCheckpointConfig().setPreferCheckpointForRecovery(true);
+
+// enables the experimental unaligned checkpoints
+env.getCheckpointConfig.enableUnalignedCheckpoints();
 {% endhighlight %}
 </div>
 <div data-lang="scala" markdown="1">
@@ -130,6 +135,9 @@ env.getCheckpointConfig.setFailTasksOnCheckpointingErrors(false)
 
 // allow only one checkpoint to be in progress at the same time
 env.getCheckpointConfig.setMaxConcurrentCheckpoints(1)
+
+// enables the experimental unaligned checkpoints
+env.getCheckpointConfig.enableUnalignedCheckpoints()
 {% endhighlight %}
 </div>
 <div data-lang="python" markdown="1">
@@ -158,6 +166,9 @@ env.get_checkpoint_config().enable_externalized_checkpoints(ExternalizedCheckpoi
 
 # allow job recovery fallback to checkpoint when there is a more recent savepoint
 env.get_checkpoint_config().set_prefer_checkpoint_for_recovery(True)
+
+// enables the experimental unaligned checkpoints
+env.get_checkpoint_config().enable_unaligned_checkpoints()
 {% endhighlight %}
 </div>
 </div>
diff --git a/docs/fig/stream_aligning.svg b/docs/fig/stream_aligning.svg
index de25435..2192117 100644
--- a/docs/fig/stream_aligning.svg
+++ b/docs/fig/stream_aligning.svg
@@ -1,4 +1,4 @@
-<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<?xml version="1.0" encoding="UTF-8"?>
 <!--
 Licensed to the Apache Software Foundation (ASF) under one
 or more contributor license agreements.  See the NOTICE file
@@ -18,860 +18,369 @@ 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="1238.8232"
-   height="221.09566"
-   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(307.98305,-396.10006)"
-     id="layer1">
-    <g
-       transform="translate(-313.94491,396.17508)"
-       id="g2989">
-      <path
-         d="m 150.43123,97.139825 c 0,-28.47897 23.13388,-51.575349 51.65974,-51.575349 28.52586,0 51.65037,23.096379 51.65037,51.575349 0,28.488345 -23.12451,51.575345 -51.65037,51.575345 -28.52586,0 -51.65974,-23.087 -51.65974,-51.575345"
-         id="path2991"
-         style="fill:#8a3142;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="M 23.368322,-0.07501869 162.13414,64.047206 151.37834,87.312377 12.612517,23.190152 z"
-         id="path2993"
-         style="fill:#e4eaf4;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 156.39521,63.156359 2.52251,1.16279 -9.41485,20.376951 -2.53188,-1.172167 z"
-         id="path2995"
-         style="fill:#6e7277;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 143.23881,63.944055 7.9426,3.666538 -4.18229,9.058507 -7.9426,-3.666538 z"
-         id="path2997"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 145.30182,74.193483 -3.11327,-1.434732 0.27194,-0.590772 1.2003,0.553263 1.77232,-3.844708 -1.19093,-0.553263 0.24382,-0.525131 c 0.15941,0.07502 0.33758,0.14066 0.5345,0.196924 0.19693,0.05626 0.35634,0.0844 0.47825,0.07502 0.15003,-0.0094 0.28132,-0.04689 0.39385,-0.112528 0.11252,-0.06564 0.21567,-0.187547 0.30007,-0.346962 l 0.60015,0.281321 -2.39122,5.176289 1.17217,0.534508 z"
-         id="path2999"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <path
-         d="m 127.49426,56.667242 7.95198,3.666539 -4.19167,9.067884 -7.9426,-3.675916 z"
-         id="path3001"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 129.88548,67.066708 -3.89159,-1.791071 0.37509,-0.806451 c 0.3751,-0.112528 0.75019,-0.215679 1.13466,-0.31883 0.37509,-0.112528 0.73143,-0.225056 1.07839,-0.346961 0.71268,-0.243811 1.22843,-0.468867 1.53789,-0.665791 0.30945,-0.196924 0.54388,-0.459489 0.69392,-0.787696 0.14066,-0.300075 0.15004,-0.581395 0.0281,-0.834583 -0.1219,-0.262565 -0.35634,-0.468867 -0.7033,-0.637659 -0.23443,-0.103151 -0.50638,-0.178169 -0.81583,-0.225056 -0.30945,-0.04689 -0.63766,-0.03751 -0.9 [...]
-         id="path3003"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <path
-         d="m 109.69608,48.443318 7.9426,3.666539 -4.18229,9.058507 -7.95198,-3.666539 z"
-         id="path3005"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 112.85624,55.842037 c 0.075,0.168792 0.11253,0.356338 0.11253,0.56264 0,0.196924 -0.0563,0.431357 -0.17817,0.7033 -0.12191,0.253188 -0.28132,0.478244 -0.47825,0.647036 -0.19692,0.17817 -0.42198,0.309452 -0.66579,0.393848 -0.28132,0.0844 -0.56264,0.112528 -0.86271,0.0844 -0.30008,-0.03751 -0.60953,-0.131282 -0.91898,-0.271942 -0.31883,-0.150038 -0.6189,-0.328207 -0.89085,-0.543886 -0.27194,-0.225056 -0.48762,-0.42198 -0.64703,-0.600149 l 0.37509,-0.806451 0.0563,0.01875 c 0. [...]
-         id="path3007"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <path
-         d="m 79.969923,34.705521 7.951981,3.675916 -4.19167,9.058506 -7.942603,-3.675915 z"
-         id="path3009"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 83.242613,43.548349 -0.853338,-0.393848 -0.750187,1.622279 -0.740809,-0.346961 0.750187,-1.612902 -2.756937,-1.275318 0.412603,-0.88147 4.276065,-1.959863 0.712677,0.328207 -1.622279,3.516501 0.853338,0.393848 -0.28132,0.609527 z m -1.312827,-1.350336 1.200299,-2.588145 -3.422728,1.556638 2.222429,1.031507 z"
-         id="path3011"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <path
-         d="m 48.696507,20.255046 7.942603,3.666538 -4.182292,9.058507 -7.951981,-3.666538 z"
-         id="path3013"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 51.875423,28.807177 c -0.121905,0.271942 -0.290697,0.496998 -0.496998,0.7033 -0.215679,0.196924 -0.440735,0.337584 -0.684546,0.431357 -0.271943,0.09377 -0.553263,0.131283 -0.853337,0.103151 -0.300075,-0.01875 -0.618905,-0.103151 -0.956489,-0.262565 -0.318829,-0.14066 -0.600149,-0.31883 -0.862715,-0.515754 -0.262565,-0.196924 -0.468866,-0.393848 -0.628281,-0.581395 l 0.375093,-0.815828 0.05626,0.02813 c 0.150038,0.234433 0.346962,0.459489 0.581395,0.684546 0.243811,0.225056  [...]
-         id="path3015"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <path
-         d="m 35.821424,14.300438 7.942604,3.675915 -4.182292,9.058507 -7.942604,-3.675916 z"
-         id="path3017"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 39.150378,22.8807 c -0.262565,0.581395 -0.675168,0.975243 -1.237808,1.16279 -0.553263,0.187547 -1.106526,0.159415 -1.669166,-0.103151 -0.28132,-0.131283 -0.515753,-0.290697 -0.7033,-0.487621 -0.196924,-0.187547 -0.337584,-0.412603 -0.440735,-0.665791 -0.121905,-0.31883 -0.159415,-0.684546 -0.112528,-1.087771 0.03751,-0.412603 0.178169,-0.862715 0.403226,-1.359714 0.234433,-0.506376 0.506376,-0.928356 0.797073,-1.275318 0.290698,-0.337584 0.618904,-0.609526 1.003375,-0.80645 [...]
-         id="path3019"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <path
-         d="M 7.5487556,164.28155 148.8277,104.0603 158.91772,127.74745 17.638769,187.9687 z"
-         id="path3021"
-         style="fill:#e4eaf4;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 93.116948,129.59479 2.578767,-1.10653 8.842825,20.75204 -2.57876,1.09715 z"
-         id="path3023"
-         style="fill:#6e7277;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 136.01826,118.07004 8.08326,-3.45086 3.93849,9.2273 -8.09265,3.44148 z"
-         id="path3025"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 145.04864,123.41512 -0.73144,0.30945 -0.19692,-0.45949 c -0.0469,0.0656 -0.10315,0.1688 -0.18755,0.2907 -0.075,0.13128 -0.15004,0.24381 -0.23443,0.32821 -0.10315,0.11253 -0.21568,0.22505 -0.35634,0.33758 -0.15004,0.10315 -0.32821,0.2063 -0.53451,0.2907 -0.39385,0.16879 -0.78769,0.18755 -1.17217,0.0375 -0.38447,-0.14066 -0.65641,-0.4126 -0.83458,-0.81583 -0.14066,-0.33758 -0.17817,-0.62828 -0.13128,-0.90022 0.0563,-0.26257 0.18755,-0.50638 0.40322,-0.74081 0.21568,-0.22506 0 [...]
-         id="path3027"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <path
-         d="m 119.98302,124.89674 8.09264,-3.45086 3.9291,9.2273 -8.08326,3.45086 z"
-         id="path3029"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 128.45075,127.86936 c 0.15942,0.36571 0.25319,0.71267 0.27194,1.05026 0.0188,0.33758 -0.0188,0.63766 -0.10315,0.9096 -0.10315,0.2907 -0.23443,0.53451 -0.4126,0.73143 -0.17817,0.19692 -0.40323,0.34696 -0.66579,0.45949 -0.25319,0.11253 -0.47825,0.16879 -0.68455,0.18755 -0.21568,0.0281 -0.43135,0.0187 -0.64703,0 l 0.0375,0.22505 -0.69393,0.2907 -2.58814,-6.07651 0.73143,-0.31883 0.92836,2.17554 c 0.13128,-0.26257 0.29069,-0.48762 0.47824,-0.69392 0.18755,-0.2063 0.42198,-0.375 [...]
-         id="path3031"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <path
-         d="m 101.866,132.62366 8.08327,-3.45085 3.93848,9.22729 -8.09264,3.45086 z"
-         id="path3033"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 110.98077,137.61241 c -0.19692,0.22505 -0.39385,0.4126 -0.58139,0.57202 -0.19693,0.15941 -0.4126,0.29069 -0.66579,0.40322 -0.30945,0.13128 -0.61891,0.2063 -0.92836,0.22506 -0.30007,0.0187 -0.58139,-0.0188 -0.85334,-0.13129 -0.27194,-0.10315 -0.51575,-0.28132 -0.74081,-0.52513 -0.22505,-0.23443 -0.42198,-0.55326 -0.59077,-0.93773 -0.30007,-0.71268 -0.34696,-1.35034 -0.1219,-1.93173 0.21568,-0.57202 0.64703,-0.994 1.29407,-1.26594 0.24381,-0.11253 0.50637,-0.17817 0.77832,-0. [...]
-         id="path3035"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <path
-         d="m 71.605339,145.5175 8.083263,-3.44148 3.938482,9.21792 -8.092641,3.45086 z"
-         id="path3037"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 80.710732,150.83445 -0.731432,0.31883 -0.196924,-0.45949 c -0.131283,0.27194 -0.290698,0.50638 -0.478244,0.71268 -0.187547,0.19692 -0.412603,0.34696 -0.684546,0.46887 -0.515753,0.21567 -1.012752,0.19692 -1.490996,-0.075 -0.478244,-0.27195 -0.872093,-0.75957 -1.172167,-1.47224 -0.159415,-0.3751 -0.243811,-0.72206 -0.262566,-1.05964 -0.01875,-0.33759 0.01875,-0.63766 0.112528,-0.91898 0.09377,-0.27195 0.234434,-0.51576 0.42198,-0.72206 0.196924,-0.19692 0.412603,-0.35634 0.66 [...]
-         id="path3039"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <path
-         d="m 39.759905,159.09588 8.083264,-3.45085 3.938481,9.22729 -8.092641,3.45086 z"
-         id="path3041"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 48.22764,162.19978 -3.216427,1.36909 c 0.112528,0.26257 0.253188,0.48763 0.421981,0.64704 0.159414,0.16879 0.346961,0.28132 0.534508,0.34696 0.196924,0.0656 0.393848,0.0938 0.609527,0.075 0.206301,-0.009 0.431357,-0.0656 0.656413,-0.16879 0.290698,-0.12191 0.572018,-0.30945 0.815828,-0.55326 0.253188,-0.25319 0.412603,-0.45949 0.496999,-0.62829 l 0.03751,-0.0188 0.337585,0.80645 c -0.196924,0.2063 -0.412603,0.40323 -0.637659,0.5814 -0.234434,0.17816 -0.487622,0.3282 -0.7689 [...]
-         id="path3043"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <path
-         d="m 26.650389,164.68478 8.092641,-3.45086 3.929104,9.2273 -8.092641,3.44148 z"
-         id="path3045"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 33.355185,164.6754 -0.03751,0.0188 c -0.09377,0.009 -0.206301,0.0281 -0.346961,0.0657 -0.14066,0.0281 -0.262566,0.0656 -0.356339,0.10315 -0.318829,0.14066 -0.515753,0.30007 -0.600149,0.50637 -0.0844,0.19693 -0.04689,0.47825 0.112528,0.84396 l 0.05626,0.15004 1.331581,-0.56264 0.262566,0.6189 -1.30345,0.55327 1.594147,3.74155 -0.731432,0.31883 -1.594147,-3.75093 -0.496999,0.21568 -0.262565,-0.61891 0.496998,-0.21567 -0.06564,-0.14067 c -0.225056,-0.51575 -0.262565,-0.97524 - [...]
-         id="path3047"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <text
-         x="126.48644"
-         y="217.39491"
-         id="text3049"
-         xml:space="preserve"
-         style="font-size:17.55437279px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">begin aligning</text>
-      <path
-         d="M 46.633493,3.0288796 147.84308,50.51571 146.65216,53.066345 45.433194,5.579515 46.633493,3.0288796 z m 101.134567,44.3454224 5.84208,7.408096 -9.4336,0.234433 3.59152,-7.642529 z"
-         id="path3051"
-         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 38.306418,188.70013 105.110562,-44.84242 -1.09715,-2.58814 -105.119938,44.84242 1.106526,2.58814 z m 104.923012,-41.70101 6.10465,-7.19242 -9.41485,-0.57201 3.3102,7.76443 z"
-         id="path3053"
-         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="5.2541633"
-         y="98.157814"
-         id="text3055"
-         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">c</text>
-      <text
-         x="12.455957"
-         y="98.157814"
-         id="text3057"
-         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">heckpoint</text>
-      <text
-         x="12.30592"
-         y="114.66193"
-         id="text3059"
-         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">barrier </text>
-      <text
-         x="63.768742"
-         y="114.66193"
-         id="text3061"
-         xml:space="preserve"
-         style="font-size:13.80343914px;font-style:italic;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">n</text>
-      <path
-         d="M 85.68072,95.930149 148.36821,84.780496 148.15253,83.552065 85.465041,94.69234 z m 63.0157,-9.302318 c 1.35971,-0.234433 2.26931,-1.537883 2.0255,-2.897597 -0.24381,-1.359713 -1.53788,-2.269315 -2.89759,-2.025504 -1.35972,0.24381 -2.26932,1.537883 -2.02551,2.897597 0.24381,1.359713 1.53789,2.269315 2.8976,2.025504 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" />
-      <path
-         d="m 80.129337,109.72421 14.769305,18.93284 -0.984621,0.76894 -14.769304,-18.93284 z m 16.250924,17.77943 c 0.84396,1.08777 0.656413,2.66316 -0.431358,3.50712 -1.097148,0.85334 -2.663163,0.65642 -3.516501,-0.43135 -0.84396,-1.08778 -0.656413,-2.66317 0.431358,-3.50713 1.097148,-0.85334 2.663163,-0.65641 3.516501,0.43136 z"
-         id="path3065"
-         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 252.64419,84.170969 0,25.628261 48.61211,0 0,-25.628261 -48.61211,0 z"
-         id="path3067"
-         style="fill:#e4eaf4;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 284.3771,91.98229 0,10.00562 8.74905,0 0,-10.00562 -8.74905,0 z"
-         id="path3069"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <text
-         x="286.25586"
-         y="100.50925"
-         id="text3071"
-         xml:space="preserve"
-         style="font-size:8.70216751px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">x</text>
-      <path
-         d="m 270.15168,91.98229 0,10.00562 8.74905,0 0,-10.00562 -8.74905,0 z"
-         id="path3073"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <text
-         x="272.00772"
-         y="100.50934"
-         id="text3075"
-         xml:space="preserve"
-         style="font-size:8.70216751px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">y</text>
-      <path
-         d="m 253.75072,77.288004 42.5731,0 0,2.813201 -42.5731,0 0,-2.813201 z m 41.1665,-2.8132 8.4396,4.219801 -8.4396,4.219801 0,-8.439602 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" />
-      <text
-         x="166.75418"
-         y="123.98857"
-         id="text3079"
-         xml:space="preserve"
-         style="font-size:17.55437279px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">operator</text>
-      <path
-         d="m 455.04461,97.149202 c 0,-28.488347 23.087,-51.575348 51.57535,-51.575348 28.48835,0 51.57535,23.087001 51.57535,51.575348 0,28.488348 -23.087,51.575348 -51.57535,51.575348 -28.48835,0 -51.57535,-23.087 -51.57535,-51.575348"
-         id="path3081"
-         style="fill:#8a3142;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="M 327.86918,-0.07501869 466.635,64.047206 455.88857,87.321754 317.12275,23.19953 z"
-         id="path3083"
-         style="fill:#e4eaf4;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 384.47078,34.714898 7.95198,3.675916 -4.18229,9.058507 -7.95198,-3.675916 z"
-         id="path3085"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 389.14069,40.228772 -4.85746,3.69467 -0.8252,-0.393848 5.10127,-3.788443 -3.26331,-1.500374 0.30007,-0.675168 3.93848,1.819203 z"
-         id="path3087"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <path
-         d="m 353.20674,20.255046 7.93323,3.675916 -4.18229,9.058507 -7.95198,-3.675916 z"
-         id="path3089"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 356.37628,29.069742 c -0.24381,0.506376 -0.6189,0.825206 -1.16279,0.975243 -0.54388,0.150037 -1.10652,0.09377 -1.68792,-0.187547 -0.6189,-0.28132 -1.05026,-0.675168 -1.25656,-1.162789 -0.22506,-0.487622 -0.22506,-0.993998 0.0188,-1.48162 0.13129,-0.318829 0.3751,-0.56264 0.67517,-0.750186 0.30008,-0.168792 0.65642,-0.243811 1.06902,-0.243811 l 0,-0.01875 c -0.22506,-0.318829 -0.3751,-0.600149 -0.43136,-0.862715 -0.0563,-0.262565 -0.0188,-0.543885 0.11253,-0.84396 0.2063,-0. [...]
-         id="path3091"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <path
-         d="m 340.32228,14.309815 7.95198,3.675916 -4.18229,9.058506 -7.95198,-3.675915 z"
-         id="path3093"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 344.18575,21.492854 c -0.22506,0.506377 -0.48763,0.918979 -0.80646,1.275318 -0.30007,0.356339 -0.6189,0.637659 -0.99399,0.806451 -0.3751,0.187547 -0.76894,0.28132 -1.2003,0.28132 -0.41261,0 -0.86272,-0.112528 -1.35034,-0.337584 -0.13128,-0.05626 -0.24381,-0.131283 -0.35634,-0.187547 -0.11253,-0.07502 -0.2063,-0.150037 -0.28132,-0.225056 l 0.33759,-0.731432 0.0375,0.01875 c 0.0563,0.07502 0.13128,0.150037 0.26256,0.24381 0.11253,0.112528 0.26257,0.187547 0.43136,0.262566 0.5 [...]
-         id="path3095"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <path
-         d="m 312.05899,164.29093 141.27895,-60.22125 10.09001,23.68715 -141.27894,60.22125 z"
-         id="path3097"
-         style="fill:#e4eaf4;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 433.19542,113.80335 2.58814,-1.08777 8.83346,20.74267 -2.56939,1.08777 z"
-         id="path3099"
-         style="fill:#6e7277;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 440.5285,118.07942 8.08326,-3.45086 3.93848,9.20854 -8.10202,3.45086 z"
-         id="path3101"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 449.64327,123.06816 c -0.2063,0.22506 -0.39385,0.4126 -0.5814,0.56264 -0.2063,0.16879 -0.4126,0.30007 -0.67517,0.39385 -0.30007,0.15004 -0.6189,0.22505 -0.91897,0.24381 -0.30008,0.0187 -0.5814,-0.0375 -0.86272,-0.13128 -0.26256,-0.11253 -0.50638,-0.28132 -0.73143,-0.52514 -0.22506,-0.24381 -0.43136,-0.56264 -0.60015,-0.93773 -0.30008,-0.71268 -0.33759,-1.36909 -0.11253,-1.93173 0.2063,-0.58139 0.63766,-0.994 1.29407,-1.27532 0.24381,-0.11253 0.50638,-0.16879 0.76894,-0.2063 [...]
-         id="path3103"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <path
-         d="m 424.49325,124.90612 8.08327,-3.45086 3.93848,9.2273 -8.08327,3.45086 z"
-         id="path3105"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 433.60802,130.21369 -0.73143,0.31883 -0.2063,-0.45011 c -0.13128,0.26256 -0.28132,0.50637 -0.46887,0.69392 -0.18754,0.2063 -0.4126,0.35634 -0.69392,0.46887 -0.52513,0.22505 -1.01275,0.2063 -1.50037,-0.0563 -0.46887,-0.28132 -0.86272,-0.76894 -1.16279,-1.48162 -0.15004,-0.37509 -0.24381,-0.71267 -0.26257,-1.05026 -0.0188,-0.33758 0.0188,-0.65641 0.11253,-0.91898 0.0938,-0.28132 0.24381,-0.52513 0.43136,-0.73143 0.18754,-0.2063 0.4126,-0.35634 0.65641,-0.46887 0.22506,-0.0938 [...]
-         id="path3107"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <path
-         d="m 406.35748,132.63304 8.10202,-3.45086 3.91973,9.2273 -8.08326,3.45086 z"
-         id="path3109"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 414.8346,135.72756 -3.20705,1.36909 c 0.11252,0.26257 0.24381,0.48763 0.4126,0.65642 0.16879,0.16879 0.33758,0.28132 0.54389,0.33758 0.18754,0.075 0.39384,0.0938 0.60014,0.075 0.20631,0 0.43136,-0.0563 0.65642,-0.16879 0.30007,-0.11253 0.56264,-0.30008 0.8252,-0.54389 0.24381,-0.26256 0.41261,-0.46886 0.48763,-0.63766 l 0.0375,-0.0187 0.33759,0.80645 c -0.18755,0.2063 -0.4126,0.4126 -0.63766,0.5814 -0.22506,0.18754 -0.48762,0.33758 -0.76894,0.45011 -0.73143,0.31883 -1.36909 [...]
-         id="path3111"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <path
-         d="m 376.1062,145.5175 8.08326,-3.4321 3.93848,9.20854 -8.08326,3.45086 z"
-         id="path3113"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 382.82037,145.5175 -0.0375,0.0188 c -0.0938,0.0188 -0.2063,0.0375 -0.35634,0.0563 -0.13128,0.0375 -0.26256,0.075 -0.35634,0.11253 -0.31883,0.13128 -0.50637,0.30007 -0.60015,0.50637 -0.075,0.20631 -0.0375,0.48763 0.11253,0.84397 l 0.0563,0.15003 1.33158,-0.56264 0.26256,0.61891 -1.29407,0.54388 1.59415,3.75094 -0.73143,0.31883 -1.61291,-3.75094 -0.48762,0.2063 -0.26256,-0.60015 0.48762,-0.22505 -0.0563,-0.13129 c -0.22505,-0.52513 -0.26256,-0.97524 -0.11252,-1.36909 0.13128, [...]
-         id="path3115"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <path
-         d="m 344.26076,159.09588 8.08327,-3.45085 3.93848,9.22729 -8.08326,3.45086 z"
-         id="path3117"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 353.16923,163.91584 c 0.30008,0.75018 0.3751,1.35033 0.18755,1.85671 -0.18755,0.48762 -0.63766,0.88147 -1.33158,1.18154 -0.24381,0.0938 -0.46887,0.16879 -0.69392,0.22506 -0.24382,0.075 -0.48763,0.11253 -0.71268,0.15004 l -0.31883,-0.75019 0.0375,-0.0188 c 0.13128,0 0.35634,-0.0375 0.63766,-0.075 0.30007,-0.0375 0.56264,-0.11253 0.8252,-0.22506 0.24381,-0.11253 0.45011,-0.22506 0.5814,-0.35634 0.15003,-0.13128 0.22505,-0.26256 0.28132,-0.4126 0.0375,-0.15004 0.0563,-0.30008  [...]
-         id="path3119"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <path
-         d="m 331.15125,164.68478 8.08326,-3.45086 3.93848,9.2273 -8.08326,3.45086 z"
-         id="path3121"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 340.37855,169.95484 -0.73144,0.31883 -1.06901,-2.49437 c -0.075,-0.18755 -0.16879,-0.3751 -0.28132,-0.54389 -0.0938,-0.16879 -0.18755,-0.28132 -0.30008,-0.35634 -0.11252,-0.075 -0.24381,-0.13128 -0.37509,-0.13128 -0.15004,0 -0.31883,0.0375 -0.52513,0.13128 -0.18755,0.075 -0.37509,0.22506 -0.56264,0.41261 -0.16879,0.18754 -0.33759,0.4126 -0.46887,0.65641 l 1.38785,3.24456 -0.73143,0.31883 -2.58815,-6.07652 0.73143,-0.31883 0.93774,2.1943 c 0.15003,-0.28132 0.31883,-0.52513 0 [...]
-         id="path3123"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <text
-         x="458.11777"
-         y="217.39491"
-         id="text3125"
-         xml:space="preserve"
-         style="font-size:17.55437279px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">aligning</text>
-      <path
-         d="m 351.08746,3.0382569 101.21897,47.4868301 -1.2003,2.550635 -101.21896,-47.4868297 1.20029,-2.5506354 z m 101.1252,44.3360451 5.85146,7.408096 -9.4336,0.24381 3.58214,-7.651906 z"
-         id="path3127"
-         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 342.76039,188.70951 105.11994,-44.84242 -1.10653,-2.58814 -105.11994,44.84242 1.10653,2.58814 z m 104.93239,-41.71039 6.09527,-7.18304 -9.41485,-0.58139 3.31958,7.76443 z"
-         id="path3129"
-         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 557.27633,84.170969 0,25.656391 67.17924,0 0,-25.656391 -67.17924,0 z"
-         id="path3131"
-         style="fill:#e4eaf4;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 588.82169,91.991667 0,9.996243 8.90847,0 0,-9.996243 -8.90847,0 z"
-         id="path3133"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <text
-         x="590.67731"
-         y="100.50925"
-         id="text3135"
-         xml:space="preserve"
-         style="font-size:8.70216751px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">a</text>
-      <path
-         d="m 574.60565,91.991667 0,9.996243 8.75843,0 0,-9.996243 -8.75843,0 z"
-         id="path3137"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <text
-         x="576.35101"
-         y="100.50934"
-         id="text3139"
-         xml:space="preserve"
-         style="font-size:8.70216751px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">b</text>
-      <path
-         d="m 558.21407,77.306759 42.5731,0 0,2.813201 -42.5731,0 0,-2.813201 z m 41.14775,-2.813201 8.4396,4.201047 -8.4396,4.238556 0,-8.439603 z"
-         id="path3141"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:#000000;stroke-width:0.03750934px;stroke-linecap:butt;stroke-linejoin:round;stroke-opacity:1;stroke-dasharray:none" />
-      <text
-         x="471.25378"
-         y="123.98857"
-         id="text3143"
-         xml:space="preserve"
-         style="font-size:17.55437279px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">operator</text>
-      <path
-         d="m 480.04459,70.104965 0,19.542368 57.05171,0 0,-19.542368 -57.05171,0 z"
-         id="path3145"
-         style="fill:#b8bec6;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 497.56146,74.94367 0,10.014995 8.73967,0 0,-10.014995 -8.73967,0 z"
-         id="path3147"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <text
-         x="499.1666"
-         y="83.519676"
-         id="text3149"
-         xml:space="preserve"
-         style="font-size:8.70216751px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">2</text>
-      <path
-         d="m 483.32666,74.94367 0,10.014995 8.75843,0 0,-10.014995 -8.75843,0 z"
-         id="path3151"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <text
-         x="484.91843"
-         y="83.519775"
-         id="text3153"
-         xml:space="preserve"
-         style="font-size:8.70216751px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">3</text>
-      <path
-         d="m 510.83976,74.793633 0,9.99624 8.75843,0 0,-9.99624 -8.75843,0 z"
-         id="path3155"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <text
-         x="512.43573"
-         y="83.345718"
-         id="text3157"
-         xml:space="preserve"
-         style="font-size:8.70216751px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">1</text>
-      <path
-         d="m 446.64252,63.146981 7.93323,3.675916 -4.1823,9.058507 -7.93322,-3.657161 z"
-         id="path3159"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 449.90583,71.999187 -0.86271,-0.393848 -0.73143,1.612901 -0.75019,-0.337584 0.75019,-1.612901 -2.75694,-1.275318 0.4126,-0.88147 4.27607,-1.950486 0.71267,0.31883 -1.6129,3.507124 0.84396,0.393848 -0.28132,0.618904 z m -1.31282,-1.350337 1.20029,-2.588144 -3.4321,1.556637 2.23181,1.031507 z"
-         id="path3161"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <path
-         d="m 415.35973,48.705884 7.95198,3.675916 -4.20105,9.058506 -7.93323,-3.675915 z"
-         id="path3163"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 418.54802,57.258014 c -0.13128,0.262566 -0.30007,0.506377 -0.50638,0.693923 -0.2063,0.206302 -0.43135,0.356339 -0.69392,0.431358 -0.26256,0.09377 -0.54388,0.131282 -0.84396,0.112528 -0.30007,-0.01875 -0.6189,-0.112528 -0.95649,-0.262566 -0.31883,-0.150037 -0.60015,-0.318829 -0.86271,-0.52513 -0.26257,-0.187547 -0.46887,-0.393849 -0.61891,-0.562641 l 0.3751,-0.825205 0.0563,0.03751 c 0.13128,0.225056 0.33759,0.450112 0.5814,0.675168 0.22505,0.225056 0.48762,0.393848 0.76894, [...]
-         id="path3165"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <path
-         d="m 402.49402,42.760653 7.93323,3.657161 -4.18229,9.058507 -7.95199,-3.657162 z"
-         id="path3167"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 405.8136,51.331538 c -0.26257,0.581395 -0.67517,0.975243 -1.23781,1.16279 -0.54389,0.187546 -1.10653,0.150037 -1.66917,-0.09377 -0.28132,-0.131282 -0.50637,-0.300074 -0.71267,-0.487621 -0.18755,-0.206302 -0.33759,-0.431358 -0.43136,-0.675168 -0.11253,-0.31883 -0.15004,-0.675169 -0.11253,-1.087771 0.0375,-0.412603 0.16879,-0.862715 0.4126,-1.350337 0.22506,-0.525131 0.48763,-0.937733 0.7877,-1.294072 0.28132,-0.337584 0.6189,-0.60015 1.01275,-0.806451 0.35634,-0.168792 0.750 [...]
-         id="path3169"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <text
-         x="486.90817"
-         y="30.892836"
-         id="text3171"
-         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">input buffer</text>
-      <path
-         d="m 525.2246,37.077987 2.86946,34.733653 -1.23781,0.09377 -2.88822,-34.714898 z m 4.74493,34.564861 c 0.11253,1.387846 -0.90023,2.588145 -2.28807,2.700673 -1.36909,0.112528 -2.58815,-0.900224 -2.70067,-2.28807 -0.11253,-1.369091 0.91897,-2.56939 2.28807,-2.681918 1.38784,-0.131283 2.58814,0.900224 2.70067,2.269315 z"
-         id="path3173"
-         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 767.17862,97.149202 c 0,-28.469592 23.06825,-51.575348 51.57535,-51.575348 28.46959,0 51.57535,23.105756 51.57535,51.575348 0,28.507098 -23.10576,51.575348 -51.57535,51.575348 -28.5071,0 -51.57535,-23.06825 -51.57535,-51.575348"
-         id="path3175"
-         style="fill:#8a3142;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="M 640.05945,-0.07501869 778.80652,64.06596 768.07885,87.321754 629.29427,23.218284 z"
-         id="path3177"
-         style="fill:#e4eaf4;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 724.75555,48.274526 7.95199,3.675916 -4.20105,9.039752 -7.95198,-3.638406 z"
-         id="path3179"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 729.40671,53.7884 -4.8387,3.675916 -0.82521,-0.375094 5.10127,-3.788443 -3.26331,-1.500374 0.30008,-0.675168 3.93848,1.800448 z"
-         id="path3181"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <path
-         d="m 693.47276,33.833429 7.95198,3.675916 -4.20104,9.039752 -7.91448,-3.675916 z"
-         id="path3183"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 696.66106,42.648125 c -0.22506,0.487621 -0.63766,0.825205 -1.16279,0.975243 -0.52513,0.150037 -1.08777,0.07502 -1.68792,-0.187547 -0.63766,-0.300075 -1.05027,-0.675168 -1.27532,-1.16279 -0.22506,-0.487621 -0.22506,-1.012752 0.0375,-1.500374 0.15003,-0.337584 0.37509,-0.56264 0.67517,-0.750186 0.30007,-0.150038 0.63765,-0.262566 1.05026,-0.225056 l 0,-0.03751 c -0.22506,-0.300075 -0.3751,-0.600149 -0.41261,-0.862715 -0.075,-0.262565 -0.0375,-0.525131 0.11253,-0.825205 0.1875 [...]
-         id="path3185"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <path
-         d="m 680.60706,27.869443 7.95198,3.675916 -4.20105,9.039752 -7.95198,-3.638407 z"
-         id="path3187"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 684.47052,35.071237 c -0.22506,0.487622 -0.48762,0.900224 -0.7877,1.275318 -0.30007,0.337584 -0.63766,0.600149 -1.01275,0.787696 -0.37509,0.187547 -0.7877,0.300075 -1.2003,0.300075 -0.4126,-0.03751 -0.86271,-0.150038 -1.35034,-0.337584 -0.15003,-0.07502 -0.26256,-0.150038 -0.37509,-0.225056 -0.11253,-0.03751 -0.18755,-0.112528 -0.26256,-0.187547 l 0.33758,-0.750187 0.0375,0.03751 c 0.0375,0.03751 0.15004,0.150038 0.26256,0.225056 0.11253,0.112528 0.26257,0.187547 0.41261,0. [...]
-         id="path3189"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <path
-         d="m 624.23051,164.29093 141.2977,-60.2025 10.09002,23.6684 -141.2977,60.24 z"
-         id="path3191"
-         style="fill:#e4eaf4;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 768.82903,104.80111 2.58815,-1.08777 8.8147,20.74267 -2.55064,1.08777 z"
-         id="path3193"
-         style="fill:#6e7277;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 752.39994,118.26696 8.06451,-3.45086 3.93848,9.2273 -8.10202,3.41335 z"
-         id="path3195"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 761.4772,123.55578 -0.71267,0.33758 -0.18755,-0.48762 c -0.15004,0.30008 -0.30008,0.52513 -0.48762,0.71268 -0.18755,0.22506 -0.41261,0.37509 -0.67517,0.48762 -0.52513,0.22506 -1.01275,0.18755 -1.50037,-0.075 -0.48763,-0.26256 -0.86272,-0.75018 -1.16279,-1.46286 -0.18755,-0.37509 -0.26257,-0.75019 -0.26257,-1.08777 -0.0375,-0.30008 0,-0.63766 0.11253,-0.90023 0.075,-0.26256 0.22505,-0.52513 0.4126,-0.71267 0.18755,-0.22506 0.4126,-0.3751 0.67517,-0.48762 0.22506,-0.075 0.450 [...]
-         id="path3197"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <path
-         d="m 734.24542,125.95638 8.10202,-3.41335 3.93848,9.18979 -8.10202,3.45086 z"
-         id="path3199"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 742.72253,129.06965 -3.2258,1.38785 c 0.11253,0.26257 0.26256,0.48762 0.45011,0.63766 0.15004,0.15004 0.33758,0.30007 0.52513,0.33758 0.18755,0.075 0.37509,0.11253 0.60015,0.075 0.22506,0 0.45011,-0.075 0.67517,-0.15004 0.26256,-0.15003 0.56264,-0.30007 0.78769,-0.56264 0.26257,-0.26256 0.45012,-0.45011 0.52513,-0.63765 l 0.0375,0 0.33759,0.78769 c -0.22506,0.22506 -0.4126,0.4126 -0.63766,0.60015 -0.26257,0.18755 -0.48762,0.33759 -0.7877,0.45011 -0.71267,0.30008 -1.38784,0. [...]
-         id="path3201"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <path
-         d="m 704.01289,138.85959 8.06451,-3.45086 3.93848,9.2273 -8.10202,3.45086 z"
-         id="path3203"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 710.68955,138.85959 -0.0375,0.0375 c -0.075,0 -0.18755,0 -0.33758,0.0375 -0.15004,0.0375 -0.26257,0.075 -0.33759,0.11253 -0.33758,0.15004 -0.52513,0.30007 -0.60015,0.52513 -0.11252,0.18755 -0.075,0.45011 0.11253,0.82521 l 0.0375,0.15003 1.35034,-0.56264 0.26256,0.63766 -1.31282,0.52513 1.57539,3.75094 -0.71268,0.33758 -1.6129,-3.75093 -0.48762,0.18754 -0.26257,-0.60015 0.48762,-0.22505 -0.0375,-0.15004 c -0.22505,-0.48762 -0.26256,-0.97524 -0.15003,-1.35034 0.15003,-0.37509 [...]
-         id="path3205"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <path
-         d="m 672.16745,152.43798 8.06451,-3.45086 3.93848,9.2273 -8.10201,3.45085 z"
-         id="path3207"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 681.05717,157.27668 c 0.30007,0.71268 0.37509,1.35034 0.18755,1.83796 -0.18755,0.48762 -0.63766,0.86271 -1.35034,1.16279 -0.22506,0.11253 -0.45011,0.18755 -0.67517,0.26256 -0.22506,0.0375 -0.48762,0.11253 -0.71268,0.15004 l -0.33758,-0.75018 0.0375,-0.0375 c 0.15004,0 0.37509,-0.0375 0.67517,-0.075 0.26256,-0.0375 0.56264,-0.11253 0.8252,-0.22506 0.22506,-0.11253 0.4126,-0.22506 0.56264,-0.33758 0.15004,-0.15004 0.22506,-0.30008 0.30008,-0.41261 0.0375,-0.15003 0.0375,-0.30 [...]
-         id="path3209"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <path
-         d="m 659.03918,158.02687 8.10202,-3.45086 3.93848,9.2273 -8.10202,3.45086 z"
-         id="path3211"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 668.26648,163.31569 -0.75018,0.30007 -1.05027,-2.47562 c -0.075,-0.22505 -0.18754,-0.37509 -0.26256,-0.56264 -0.11253,-0.15003 -0.22506,-0.30007 -0.30008,-0.33758 -0.11252,-0.11253 -0.26256,-0.15004 -0.4126,-0.15004 -0.15004,0 -0.30007,0.0375 -0.48762,0.15004 -0.22506,0.075 -0.4126,0.22506 -0.56264,0.4126 -0.18755,0.18755 -0.33759,0.41261 -0.48762,0.63766 l 1.38784,3.26331 -0.71267,0.30008 -2.58815,-6.07651 0.71268,-0.30008 0.93773,2.17554 c 0.15004,-0.26256 0.33759,-0.5251 [...]
-         id="path3213"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <path
-         d="m 663.35276,3.0382569 101.23772,47.4868301 -1.2003,2.550635 -101.23772,-47.4868297 1.2003,-2.5506354 z m 101.12519,44.3360451 5.85146,7.42685 -9.41485,0.225056 3.56339,-7.651906 z"
-         id="path3215"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:#000000;stroke-width:0.03750934px;stroke-linecap:butt;stroke-linejoin:round;stroke-opacity:1;stroke-dasharray:none" />
-      <path
-         d="m 654.87565,188.70951 105.10118,-44.82366 -1.08777,-2.58815 -105.10119,44.82367 1.08778,2.58814 z m 104.91363,-41.71039 6.11403,-7.16428 -9.41485,-0.60015 3.30082,7.76443 z"
-         id="path3217"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:#000000;stroke-width:0.03750934px;stroke-linecap:butt;stroke-linejoin:round;stroke-opacity:1;stroke-dasharray:none" />
-      <path
-         d="m 869.39159,84.170969 0,25.656391 69.99243,0 0,-25.656391 -69.99243,0 z"
-         id="path3219"
-         style="fill:#e4eaf4;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 901.08698,92.010422 0,9.977488 8.77719,0 0,-9.977488 -8.77719,0 z"
-         id="path3221"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <text
-         x="902.7738"
-         y="100.50934"
-         id="text3223"
-         xml:space="preserve"
-         style="font-size:8.70216751px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">b</text>
-      <path
-         d="m 886.87094,92.010422 0,9.977488 8.77719,0 0,-9.977488 -8.77719,0 z"
-         id="path3225"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <text
-         x="888.99445"
-         y="100.50944"
-         id="text3227"
-         xml:space="preserve"
-         style="font-size:8.70216751px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">c</text>
-      <path
-         d="m 870.32932,77.306759 42.57311,0 0,2.813201 -42.57311,0 0,-2.813201 z m 41.14775,-2.813201 8.4396,4.201047 -8.4396,4.238556 0,-8.439603 z"
-         id="path3229"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:#000000;stroke-width:0.03750934px;stroke-linecap:butt;stroke-linejoin:round;stroke-opacity:1;stroke-dasharray:none" />
-      <text
-         x="783.42834"
-         y="123.98867"
-         id="text3231"
-         xml:space="preserve"
-         style="font-size:17.55437279px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">operator</text>
-      <path
-         d="m 792.30988,70.104965 0,19.542368 57.05172,0 0,-19.542368 -57.05172,0 z"
-         id="path3233"
-         style="fill:#b8bec6;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 823.25509,74.94367 0,10.014995 8.77719,0 0,-10.014995 -8.77719,0 z"
-         id="path3235"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <text
-         x="824.888"
-         y="83.519775"
-         id="text3237"
-         xml:space="preserve"
-         style="font-size:8.70216751px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">2</text>
-      <path
-         d="m 809.03905,74.94367 0,10.014995 8.77719,0 0,-10.014995 -8.77719,0 z"
-         id="path3239"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <text
-         x="810.63989"
-         y="83.519875"
-         id="text3241"
-         xml:space="preserve"
-         style="font-size:8.70216751px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">3</text>
-      <path
-         d="m 836.57091,74.793633 0,10.014995 8.73968,0 0,-10.014995 -8.73968,0 z"
-         id="path3243"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <text
-         x="838.15717"
-         y="83.34581"
-         id="text3245"
-         xml:space="preserve"
-         style="font-size:8.70216751px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">1</text>
-      <path
-         d="m 755.62575,62.265512 7.95198,3.675916 -4.16354,9.039752 -7.95198,-3.638407 z"
-         id="path3247"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 758.81404,70.817642 c -0.11253,0.262566 -0.30008,0.487622 -0.48762,0.712678 -0.22506,0.187547 -0.45011,0.337584 -0.67517,0.412603 -0.30007,0.112528 -0.56264,0.150037 -0.86272,0.112528 -0.30007,0 -0.63765,-0.112528 -0.97524,-0.262566 -0.30007,-0.150037 -0.60015,-0.300074 -0.86271,-0.52513 -0.26257,-0.187547 -0.45012,-0.375094 -0.60015,-0.562641 l 0.37509,-0.825205 0.0375,0.03751 c 0.15004,0.225056 0.33758,0.450112 0.60015,0.675168 0.22506,0.225056 0.48762,0.412603 0.75019,0. [...]
-         id="path3249"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <path
-         d="m 742.76004,56.301526 7.95198,3.675916 -4.20105,9.077261 -7.91447,-3.675916 z"
-         id="path3251"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 746.09837,64.891166 c -0.26256,0.60015 -0.67517,0.975243 -1.23781,1.16279 -0.56264,0.187546 -1.12528,0.150037 -1.68792,-0.112528 -0.26256,-0.112528 -0.48762,-0.262566 -0.67517,-0.487622 -0.22505,-0.187547 -0.33758,-0.412603 -0.45011,-0.637659 -0.11253,-0.337584 -0.15003,-0.712677 -0.11253,-1.087771 0.0375,-0.412602 0.18755,-0.862715 0.41261,-1.387845 0.22505,-0.487622 0.48762,-0.900225 0.78769,-1.275318 0.30008,-0.337584 0.60015,-0.60015 1.01276,-0.787696 0.33758,-0.187547  [...]
-         id="path3253"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <text
-         x="794.7132"
-         y="31.091112"
-         id="text3255"
-         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">emit barrier </text>
-      <text
-         x="881.73486"
-         y="31.091112"
-         id="text3257"
-         xml:space="preserve"
-         style="font-size:13.80343914px;font-style:italic;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">n</text>
-      <path
-         d="m 862.63991,84.658591 0,22.505609 2.8132,0 0,-22.505609 -2.8132,0 z"
-         id="path3259"
-         style="fill:#6e7277;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 1071.0043,97.149202 c 0,-28.469592 23.0683,-51.575348 51.5754,-51.575348 28.4696,0 51.5753,23.105756 51.5753,51.575348 0,28.507098 -23.1057,51.575348 -51.5753,51.575348 -28.5071,0 -51.5754,-23.06825 -51.5754,-51.575348"
-         id="path3261"
-         style="fill:#8a3142;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="M 943.81013,-0.07501869 1082.5947,64.06596 1071.8295,87.321754 933.08245,23.218284 z"
-         id="path3263"
-         style="fill:#e4eaf4;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 1034.3202,51.162746 7.952,3.675916 -4.2011,9.077261 -7.9145,-3.675916 z"
-         id="path3265"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 1038.9713,56.714129 -4.8387,3.675916 -0.8252,-0.375094 5.1013,-3.788444 -3.2633,-1.537883 0.3001,-0.675168 3.9384,1.837958 z"
-         id="path3267"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <path
-         d="m 1003.0374,36.721648 7.952,3.675916 -4.2011,9.039752 -7.91446,-3.675916 z"
-         id="path3269"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 1006.2257,45.536344 c -0.2251,0.487622 -0.6377,0.825206 -1.1628,0.975243 -0.5251,0.150038 -1.0878,0.07502 -1.6879,-0.187547 -0.6377,-0.300074 -1.0503,-0.675168 -1.2754,-1.162789 -0.225,-0.487622 -0.1875,-0.975243 0.038,-1.500374 0.15,-0.300075 0.3751,-0.56264 0.6751,-0.712678 0.3001,-0.187546 0.6377,-0.262565 1.0503,-0.262565 l 0.037,-0.03751 c -0.2626,-0.300075 -0.4126,-0.60015 -0.4501,-0.862715 -0.075,-0.262566 -0.038,-0.525131 0.1125,-0.825206 0.1876,-0.412603 0.5627,-0. [...]
-         id="path3271"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <path
-         d="m 990.17168,30.757663 7.95198,3.675915 -4.20105,9.077262 -7.95198,-3.675916 z"
-         id="path3273"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 994.03514,37.959457 c -0.22506,0.487621 -0.48762,0.937733 -0.7877,1.275317 -0.30007,0.375094 -0.63766,0.637659 -1.01275,0.825206 -0.37509,0.187547 -0.75019,0.262565 -1.2003,0.262565 -0.4126,0 -0.86271,-0.112528 -1.35033,-0.337584 -0.11253,-0.07502 -0.26257,-0.112528 -0.3751,-0.187546 -0.11253,-0.07502 -0.18755,-0.150038 -0.26256,-0.225057 l 0.33758,-0.750186 0.0375,0.03751 c 0.0375,0.07502 0.15004,0.150037 0.26257,0.262565 0.11252,0.07502 0.26256,0.187547 0.4126,0.262566 0. [...]
-         id="path3275"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <path
-         d="m 927.98118,164.29093 141.29772,-60.2025 10.09,23.6684 -141.26019,60.24 z"
-         id="path3277"
-         style="fill:#e4eaf4;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 1057.501,117.44176 8.102,-3.45086 3.9384,9.2273 -8.102,3.45086 z"
-         id="path3279"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 1066.6157,122.76808 -0.7502,0.30008 -0.1875,-0.45011 c -0.1125,0.26256 -0.3001,0.52513 -0.4501,0.71267 -0.1876,0.18755 -0.4126,0.33759 -0.7127,0.48763 -0.5251,0.18754 -1.0128,0.18754 -1.5004,-0.075 -0.4501,-0.30008 -0.8627,-0.7877 -1.1628,-1.50038 -0.15,-0.37509 -0.225,-0.71267 -0.2625,-1.05026 0,-0.33758 0.037,-0.63766 0.1125,-0.90022 0.1125,-0.30008 0.2251,-0.52513 0.4501,-0.75019 0.1876,-0.18755 0.4126,-0.33758 0.6377,-0.45011 0.225,-0.11253 0.4501,-0.15004 0.6376,-0.187 [...]
-         id="path3281"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <path
-         d="m 1039.3839,125.16868 8.1021,-3.45086 3.9009,9.2273 -8.0645,3.45086 z"
-         id="path3283"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 1047.861,128.28196 -3.2258,1.35033 c 0.1126,0.30008 0.2626,0.48763 0.4126,0.67517 0.1876,0.15004 0.3376,0.26257 0.5627,0.33759 0.1875,0.075 0.3751,0.075 0.6001,0.075 0.1876,0 0.4126,-0.075 0.6377,-0.15004 0.3001,-0.15004 0.5626,-0.33759 0.8252,-0.56264 0.2626,-0.26257 0.4126,-0.48762 0.4876,-0.63766 l 0.037,0 0.3376,0.7877 c -0.1875,0.18754 -0.4126,0.4126 -0.6376,0.56264 -0.2251,0.18754 -0.4877,0.33758 -0.7502,0.48762 -0.7502,0.30007 -1.3879,0.33758 -1.9505,0.11253 -0.6002, [...]
-         id="path3285"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <path
-         d="m 1009.1139,138.0719 8.102,-3.45086 3.9385,9.2273 -8.102,3.45086 z"
-         id="path3287"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 1015.8281,138.0719 -0.037,0 c -0.075,0.0375 -0.1876,0.0375 -0.3376,0.075 -0.1501,0.0375 -0.2626,0.075 -0.3751,0.11252 -0.3001,0.11253 -0.5251,0.30008 -0.6002,0.48763 -0.075,0.22505 -0.038,0.48762 0.1126,0.86271 l 0.075,0.15004 1.3128,-0.56264 0.2626,0.60015 -1.2753,0.56264 1.5753,3.75093 -0.7126,0.30008 -1.6129,-3.75094 -0.4877,0.22506 -0.2625,-0.60015 0.4876,-0.22506 -0.075,-0.15003 c -0.2251,-0.52514 -0.2626,-0.97525 -0.1125,-1.35034 0.15,-0.4126 0.4501,-0.67517 0.9377,-0 [...]
-         id="path3289"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <path
-         d="m 977.26846,151.65028 8.10202,-3.45086 3.93848,9.2273 -8.10202,3.45086 z"
-         id="path3291"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 986.15818,156.45148 c 0.33758,0.75018 0.4126,1.38784 0.22505,1.87546 -0.18754,0.48762 -0.63766,0.86272 -1.35033,1.16279 -0.22506,0.11253 -0.45012,0.18755 -0.71268,0.22506 -0.22506,0.075 -0.45011,0.11253 -0.67517,0.15004 l -0.33758,-0.75019 0.0375,0 c 0.15003,0 0.37509,-0.0375 0.63765,-0.075 0.30008,-0.0375 0.56264,-0.11253 0.82521,-0.22506 0.26257,-0.11252 0.45011,-0.22505 0.60015,-0.37509 0.15004,-0.11253 0.22506,-0.26256 0.26257,-0.4126 0.0375,-0.11253 0.075,-0.26257 0.03 [...]
-         id="path3293"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <path
-         d="m 964.1777,157.23917 8.06451,-3.45086 3.93848,9.2273 -8.06451,3.45086 z"
-         id="path3295"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 973.405,162.49048 -0.75019,0.33758 -1.05026,-2.51312 c -0.11253,-0.18755 -0.18755,-0.37509 -0.30007,-0.52513 -0.075,-0.18755 -0.18755,-0.30008 -0.30008,-0.3751 -0.11253,-0.075 -0.22505,-0.11252 -0.37509,-0.11252 -0.15004,0 -0.30008,0.0375 -0.52513,0.11252 -0.18755,0.075 -0.3751,0.22506 -0.56264,0.41261 -0.15004,0.18754 -0.33759,0.4126 -0.45012,0.67517 l 1.38785,3.2258 -0.75019,0.33758 -2.58814,-6.07651 0.75019,-0.33758 0.93773,2.21305 c 0.15004,-0.30008 0.30007,-0.52513 0.5 [...]
-         id="path3297"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <path
-         d="m 967.02841,3.0382569 101.20019,47.4868301 -1.2003,2.550635 -101.20019,-47.4868297 1.2003,-2.5506354 z m 101.12519,44.3360451 5.8515,7.42685 -9.4149,0.225056 3.5634,-7.651906 z"
-         id="path3299"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:#000000;stroke-width:0.03750934px;stroke-linecap:butt;stroke-linejoin:round;stroke-opacity:1;stroke-dasharray:none" />
-      <path
-         d="m 958.70134,188.70951 105.10116,-44.82366 -1.0878,-2.58815 -105.10113,44.82367 1.08777,2.58814 z m 104.91366,-41.71039 6.114,-7.16428 -9.4148,-0.60015 3.3008,7.76443 z"
-         id="path3301"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:#000000;stroke-width:0.03750934px;stroke-linecap:butt;stroke-linejoin:round;stroke-opacity:1;stroke-dasharray:none" />
-      <path
-         d="m 1173.2173,84.170969 0,25.656391 71.5678,0 0,-25.656391 -71.5678,0 z"
-         id="path3303"
-         style="fill:#e4eaf4;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 1232.482,91.672838 0,10.052502 8.7022,0 0,-10.052502 -8.7022,0 z"
-         id="path3305"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <text
-         x="1234.6501"
-         y="100.24077"
-         id="text3307"
-         xml:space="preserve"
-         style="font-size:8.70216751px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">c</text>
-      <path
-         d="m 1174.1925,77.344268 42.5356,0 0,2.775692 -42.5356,0 0,-2.775692 z m 41.1103,-2.85071 8.4771,4.201047 -8.4771,4.276065 0,-8.477112 z"
-         id="path3309"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:#000000;stroke-width:0.07501869px;stroke-linecap:butt;stroke-linejoin:round;stroke-opacity:1;stroke-dasharray:none" />
-      <text
-         x="1087.1899"
-         y="123.98876"
-         id="text3311"
-         xml:space="preserve"
-         style="font-size:17.55437279px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">operator</text>
-      <path
-         d="m 1095.9855,70.104965 0,19.542368 57.0518,0 0,-19.542368 -57.0518,0 z"
-         id="path3313"
-         style="fill:#b8bec6;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 1195.7229,91.822875 0,10.014995 8.8897,0 0,-10.014995 -8.8897,0 z"
-         id="path3315"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <text
-         x="1197.3989"
-         y="100.41473"
-         id="text3317"
-         xml:space="preserve"
-         style="font-size:8.70216751px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">2</text>
-      <path
-         d="m 1181.4693,91.822875 0,10.014995 8.7772,0 0,-10.014995 -8.7772,0 z"
-         id="path3319"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <text
-         x="1183.1508"
-         y="100.41483"
-         id="text3321"
-         xml:space="preserve"
-         style="font-size:8.70216751px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">3</text>
-      <path
-         d="m 1209.0012,91.672838 0,10.014992 8.7397,0 0,-10.014992 -8.7397,0 z"
-         id="path3323"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <text
-         x="1210.668"
-         y="100.24077"
-         id="text3325"
-         xml:space="preserve"
-         style="font-size:8.70216751px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">1</text>
-      <path
-         d="m 1065.1904,65.153731 7.9519,3.675916 -4.1635,9.077262 -7.952,-3.675916 z"
-         id="path3327"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 1068.3787,73.705862 c -0.1126,0.300075 -0.2626,0.525131 -0.4877,0.712678 -0.225,0.187546 -0.4501,0.337584 -0.6751,0.450112 -0.3001,0.07502 -0.5627,0.112528 -0.8627,0.112528 -0.3001,-0.03751 -0.6377,-0.112528 -0.9753,-0.300075 -0.3001,-0.112528 -0.6001,-0.300075 -0.8627,-0.487622 -0.2626,-0.225056 -0.4501,-0.412602 -0.6002,-0.600149 l 0.3751,-0.787696 0.038,0 c 0.15,0.225056 0.3375,0.450112 0.6001,0.675168 0.2251,0.225056 0.4876,0.412603 0.7502,0.525131 0.1875,0.112528 0.375 [...]
-         id="path3329"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <path
-         d="m 1052.3247,59.227255 7.9519,3.675916 -4.201,9.039752 -7.9145,-3.675916 z"
-         id="path3331"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 1055.663,67.779386 c -0.2626,0.600149 -0.6752,0.975243 -1.2378,1.162789 -0.5627,0.225056 -1.1253,0.187547 -1.6504,-0.07502 -0.3001,-0.150038 -0.5252,-0.300075 -0.7127,-0.487622 -0.1876,-0.187547 -0.3376,-0.412603 -0.4501,-0.675168 -0.1125,-0.337584 -0.1501,-0.675168 -0.1125,-1.087771 0.038,-0.412603 0.1875,-0.862715 0.4126,-1.350337 0.225,-0.52513 0.4876,-0.937733 0.7877,-1.275317 0.3,-0.337584 0.6376,-0.637659 1.0127,-0.825206 0.3376,-0.187547 0.7502,-0.262565 1.1628,-0.26 [...]
-         id="path3333"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <text
-         x="1102.8444"
-         y="30.893032"
-         id="text3335"
-         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">input buffer</text>
-      <path
-         d="m 1141.1468,37.096742 2.8882,34.733653 -1.2378,0.07502 -2.8882,-34.696143 z m 4.7637,34.546106 c 0.1125,1.387846 -0.9002,2.588145 -2.2881,2.700673 -1.3878,0.112528 -2.5881,-0.900224 -2.7007,-2.28807 -0.1125,-1.350337 0.9003,-2.550636 2.2881,-2.663164 1.3878,-0.150037 2.5881,0.900225 2.7007,2.250561 z"
-         id="path3337"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:#000000;stroke-width:0.03750934px;stroke-linecap:butt;stroke-linejoin:round;stroke-opacity:1;stroke-dasharray:none" />
-      <path
-         d="m 1226.0304,87.171717 0,22.505603 2.8507,0 0,-22.505603 -2.8507,0 z"
-         id="path3339"
-         style="fill:#6e7277;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <text
-         x="1078.7684"
-         y="217.39491"
-         id="text3341"
-         xml:space="preserve"
-         style="font-size:17.55437279px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">continue</text>
-      <text
-         x="743.72571"
-         y="217.39491"
-         id="text3343"
-         xml:space="preserve"
-         style="font-size:17.55437279px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">checkpoint</text>
-      <path
-         d="m 797.1861,74.94367 0,10.014995 8.73968,0 0,-10.014995 -8.73968,0 z"
-         id="path3345"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <text
-         x="798.82666"
-         y="83.519875"
-         id="text3347"
-         xml:space="preserve"
-         style="font-size:8.70216751px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">4</text>
-      <path
-         d="m 860.46436,34.733653 4.08852,54.201003 -1.23781,0.07502 -4.08852,-54.163494 z m 5.96399,54.050965 c 0.11253,1.387846 -0.93773,2.588145 -2.28807,2.700673 -1.38785,0.07502 -2.58815,-0.937733 -2.70067,-2.325579 -0.11253,-1.350336 0.93773,-2.550636 2.32558,-2.663164 1.35033,-0.112528 2.55063,0.900225 2.66316,2.28807 z"
-         id="path3349"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:#000000;stroke-width:0.03750934px;stroke-linecap:butt;stroke-linejoin:round;stroke-opacity:1;stroke-dasharray:none" />
-      <path
-         d="m 1099.8865,74.94367 0,10.014995 8.9272,0 0,-10.014995 -8.9272,0 z"
-         id="path3351"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <text
-         x="1101.621"
-         y="83.519875"
-         id="text3353"
-         xml:space="preserve"
-         style="font-size:8.70216751px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">4</text>
-      <path
-         d="m 948.46129,164.32844 8.10201,-3.45086 3.93849,9.2273 -8.10202,3.45086 z"
-         id="path3355"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <path
-         d="m 954.08769,165.11613 -0.82521,0.33759 -0.30007,-0.75019 0.8252,-0.37509 0.30008,0.78769 z m 2.13803,5.10128 -0.75019,0.30007 -1.83795,-4.35108 0.71267,-0.30008 1.87547,4.35109 z"
-         id="path3357"
-         style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:none" />
-      <path
-         d="m 915.49057,92.010422 0,9.977488 8.73968,0 0,-9.977488 -8.73968,0 z"
-         id="path3359"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <text
-         x="917.28796"
-         y="100.50934"
-         id="text3361"
-         xml:space="preserve"
-         style="font-size:8.70216751px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">a</text>
-      <path
-         d="m 605.40082,91.991667 0,9.996243 8.73968,0 0,-9.996243 -8.73968,0 z"
-         id="path3363"
-         style="fill:#f5a030;fill-opacity:1;fill-rule:evenodd;stroke:none" />
-      <text
-         x="607.2179"
-         y="100.50934"
-         id="text3365"
-         xml:space="preserve"
-         style="font-size:8.70216751px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">y</text>
+<!DOCTYPE svg
+        PUBLIC '-//W3C//DTD SVG 1.1//EN'
+        'http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd'>
+<svg content="&lt;mxfile modified=&quot;2020-06-19T07:12:00.833Z&quot; host=&quot;app.diagrams.net&quot; agent=&quot;5.0 (Macintosh)&quot; etag=&quot;05p1r_BUnWSHJGdrPjBS&quot; version=&quot;13.2.6&quot; type=&quot;google&quot;&gt;&lt;diagram id=&quot;n4wjEhElwpPeuiAcmKNk&quot; name=&quot;Page-1&quot;&gt;7V1Lk5s4EP41c5wpS+Jhjsk8sluVVJLKIbt7STEg2yTY8mKc2Pn1C0bygBob7MFIMjt72FhgGb5+qLvVH9yQ+/nmXeIvZx9YSOMbPAo3N+ThBmM8tqzsf/nIlo8Q4hYj0yQKizH0MvAl+k354IiPrqOQrionpozFabSsDgZssaBBWhnzk4T9qp42YX [...]
+  <defs>
+    <linearGradient id="mx-gradient-ffffff-1-ffffff00-1-e-0" x1="0%" x2="100%" y1="0%" y2="0%">
+      <stop offset="0%" style="stop-color:#FFFFFF"/>
+      <stop offset="100%" style="stop-color:#FFFFFF" stop-opacity="0"/>
+    </linearGradient>
+  </defs>
+  <g>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="980" y="140"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="989.5" y="168.5">h</text>
     </g>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="540" y="140"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="549.5" y="168.5">h</text>
+    </g>
+    <path d="M -20 90 C -20 123.14 6.86 150 40 150 C 73.14 150 100 123.14 100 90 Z" fill="#f5a030" pointer-events="all" stroke="#000000" stroke-miterlimit="10" transform="rotate(-90,40,120)"/>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="60" y="60"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="69.5" y="88.5">6</text>
+    </g>
+    <ellipse cx="280" cy="120" fill="#f5a030" pointer-events="all" rx="60" ry="60" stroke="#000000"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="279.5" y="128.5">Operator</text>
+    </g>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="80" y="60"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="89.5" y="88.5">5</text>
+    </g>
+    <path d="M 120 80 L 163.63 80" fill="none" pointer-events="stroke" stroke="#000000" stroke-miterlimit="10"/>
+    <path d="M 168.88 80 L 161.88 83.5 L 163.63 80 L 161.88 76.5 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="100" y="60"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="109.5" y="88.5">4</text>
+    </g>
+    <rect fill="#e6e6e6" fill-opacity="0.5" height="40" pointer-events="all" stroke="#000000" width="20" x="170" y="60"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="179.5" y="88.5">3</text>
+    </g>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="190" y="60"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="199.5" y="88.5">2</text>
+    </g>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="210" y="60"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="219.5" y="88.5">1</text>
+    </g>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="60" y="140"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="69.5" y="168.5">f</text>
+    </g>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="80" y="140"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="89.5" y="168.5">e</text>
+    </g>
+    <path d="M 120 160 L 163.63 160" fill="none" pointer-events="stroke" stroke="#000000" stroke-miterlimit="10"/>
+    <path d="M 168.88 160 L 161.88 163.5 L 163.63 160 L 161.88 156.5 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="100" y="140"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="109.5" y="168.5">d</text>
+    </g>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="170" y="140"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="179.5" y="168.5">c</text>
+    </g>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="190" y="140"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="199.5" y="168.5">b</text>
+    </g>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="210" y="140"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="219.5" y="168.5">a</text>
+    </g>
+    <path d="M 229 110 L 229 50" fill="none" pointer-events="stroke" stroke="#8a3142" stroke-miterlimit="10" stroke-width="3"/>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="340" y="100"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="349.5" y="128.5">z</text>
+    </g>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="360" y="100"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="369.5" y="128.5">y</text>
+    </g>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="380" y="100"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="389.5" y="128.5">x</text>
+    </g>
+    <rect fill="none" height="40" pointer-events="all" stroke="none" width="110" x="145" y="0"/>
+    <g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+      <text x="199.5" y="17">Checkpoint</text>
+      <text x="199.5" y="35">barrier</text>
+    </g>
+    <rect fill="none" height="20" pointer-events="all" stroke="none" width="40" x="190" y="210"/>
+    <g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+      <text x="209.5" y="217">Input</text>
+      <text x="209.5" y="235">buffers</text>
+    </g>
+    <path d="M 210 200 L 202.85 185.7" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+    <path d="M 200.5 181 L 206.76 185.7 L 202.85 185.7 L 200.5 188.83 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+    <path d="M 210 200 L 217.15 185.7" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+    <path d="M 219.5 181 L 219.5 188.83 L 217.15 185.7 L 213.24 185.7 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+    <path d="M 220 40 L 225.5 45.5" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+    <path d="M 229.21 49.21 L 221.78 46.73 L 225.5 45.5 L 226.73 41.78 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+    <path d="M 358.75 210 L 350.79 146.32" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+    <path d="M 350.14 141.11 L 354.48 147.62 L 350.79 146.32 L 347.53 148.49 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+    <path d="M 361 210 L 367.37 146.34" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+    <path d="M 367.89 141.11 L 370.67 148.43 L 367.37 146.34 L 363.71 147.73 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+    <rect fill="none" height="20" pointer-events="all" stroke="none" width="40" x="340" y="210"/>
+    <g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+      <text x="359.5" y="217">Output</text>
+      <text x="359.5" y="235">buffers</text>
+    </g>
+    <path d="M 78.06 210 L 73.44 186.25" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+    <path d="M 72.44 181.1 L 77.21 187.3 L 73.44 186.25 L 70.34 188.64 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+    <path d="M 81.88 210 L 86.33 186.26" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+    <path d="M 87.29 181.1 L 89.44 188.62 L 86.33 186.26 L 82.56 187.33 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+    <rect fill="none" height="20" pointer-events="all" stroke="none" width="40" x="60" y="210"/>
+    <g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+      <text x="79.5" y="217">Output</text>
+      <text x="79.5" y="235">buffers</text>
+    </g>
+    <path d="M 100 190 L 100 130" fill="none" pointer-events="stroke" stroke="#8a3142" stroke-miterlimit="10" stroke-width="3"/>
+    <rect fill="none" height="40" pointer-events="all" stroke="none" width="200" x="110" y="250"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="209.5" y="278.5">Begin alignment</text>
+    </g>
+    <path d="M 181.82 40 L 104.28 125.29" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+    <path d="M 100.75 129.17 L 102.87 121.64 L 104.28 125.29 L 108.05 126.35 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+    <path d="M 420 90 C 420 123.14 446.86 150 480 150 C 513.14 150 540 123.14 540 90 Z" fill="#f5a030" pointer-events="all" stroke="#000000" stroke-miterlimit="10" transform="rotate(-90,480,120)"/>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="500" y="60"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="509.5" y="88.5">6</text>
+    </g>
+    <ellipse cx="720" cy="120" fill="#f5a030" pointer-events="all" rx="60" ry="60" stroke="#000000"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="719.5" y="128.5">Operator</text>
+    </g>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="520" y="60"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="529.5" y="88.5">5</text>
+    </g>
+    <path d="M 560 80 L 603.63 80" fill="none" pointer-events="stroke" stroke="#000000" stroke-miterlimit="10"/>
+    <path d="M 608.88 80 L 601.88 83.5 L 603.63 80 L 601.88 76.5 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="540" y="60"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="549.5" y="88.5">4</text>
+    </g>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="610" y="60"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="619.5" y="88.5">3</text>
+    </g>
+    <path d="M 650 85 L 660.64 90.32" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+    <path d="M 665.33 92.67 L 657.51 92.67 L 660.64 90.32 L 660.64 86.41 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="630" y="60"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="639.5" y="88.5">2</text>
+    </g>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="650" y="60"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="659.5" y="88.5">1</text>
+    </g>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="500" y="140"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="509.5" y="168.5">j</text>
+    </g>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="520" y="140"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="529.5" y="168.5">i</text>
+    </g>
+    <path d="M 560 160 L 603.63 160" fill="none" pointer-events="stroke" stroke="#000000" stroke-miterlimit="10"/>
+    <path d="M 608.88 160 L 601.88 163.5 L 603.63 160 L 601.88 156.5 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="610" y="140"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="619.5" y="168.5">g</text>
+    </g>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="630" y="140"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="639.5" y="168.5">f</text>
+    </g>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="650" y="140"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="659.5" y="168.5">e</text>
+    </g>
+    <path d="M 669 110 L 669 50" fill="none" pointer-events="stroke" stroke="#8a3142" stroke-miterlimit="10" stroke-width="3"/>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="780" y="100"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="789.5" y="128.5">d</text>
+    </g>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="800" y="100"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="809.5" y="128.5">c</text>
+    </g>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="820" y="100"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="829.5" y="128.5">b</text>
+    </g>
+    <rect fill="none" height="40" pointer-events="all" stroke="none" width="110" x="585" y="0"/>
+    <g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+      <text x="639.5" y="17">Checkpoint</text>
+      <text x="639.5" y="35">barrier</text>
+    </g>
+    <rect fill="none" height="20" pointer-events="all" stroke="none" width="40" x="630" y="210"/>
+    <g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+      <text x="649.5" y="217">Input</text>
+      <text x="649.5" y="235">buffers</text>
+    </g>
+    <path d="M 650 200 L 642.85 185.7" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+    <path d="M 640.5 181 L 646.76 185.7 L 642.85 185.7 L 640.5 188.83 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+    <path d="M 650 200 L 657.15 185.7" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+    <path d="M 659.5 181 L 659.5 188.83 L 657.15 185.7 L 653.24 185.7 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+    <path d="M 660 40 L 665.5 45.5" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+    <path d="M 669.21 49.21 L 661.78 46.73 L 665.5 45.5 L 666.73 41.78 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+    <path d="M 798.75 210 L 790.79 146.32" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+    <path d="M 790.14 141.11 L 794.48 147.62 L 790.79 146.32 L 787.53 148.49 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+    <path d="M 801 210 L 807.37 146.34" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+    <path d="M 807.89 141.11 L 810.67 148.43 L 807.37 146.34 L 803.71 147.73 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+    <rect fill="none" height="20" pointer-events="all" stroke="none" width="40" x="780" y="210"/>
+    <g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+      <text x="799.5" y="217">Output</text>
+      <text x="799.5" y="235">buffers</text>
+    </g>
+    <path d="M 518.06 210 L 513.44 186.25" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+    <path d="M 512.44 181.1 L 517.21 187.3 L 513.44 186.25 L 510.34 188.64 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+    <path d="M 521.88 210 L 526.33 186.26" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+    <path d="M 527.29 181.1 L 529.44 188.62 L 526.33 186.26 L 522.56 187.33 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+    <rect fill="none" height="20" pointer-events="all" stroke="none" width="40" x="500" y="210"/>
+    <g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+      <text x="519.5" y="217">Output</text>
+      <text x="519.5" y="235">buffers</text>
+    </g>
+    <rect fill="none" height="40" pointer-events="all" stroke="none" width="200" x="550" y="250"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="649.5" y="278.5">End alignment</text>
+    </g>
+    <path d="M 645 40 L 668.46 133.82" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+    <path d="M 669.73 138.92 L 664.64 132.97 L 668.46 133.82 L 671.43 131.28 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+    <path d="M 670 190 L 670 130" fill="none" pointer-events="stroke" stroke="#8a3142" stroke-miterlimit="10" stroke-width="3"/>
+    <path d="M 860 90 C 860 123.14 886.86 150 920 150 C 953.14 150 980 123.14 980 90 Z" fill="#f5a030" pointer-events="all" stroke="#000000" stroke-miterlimit="10" transform="rotate(-90,920,120)"/>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="940" y="60"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="949.5" y="88.5">6</text>
+    </g>
+    <ellipse cx="1160" cy="120" fill="#f5a030" pointer-events="all" rx="60" ry="60" stroke="#000000"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="1159.5" y="128.5">Operator</text>
+    </g>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="960" y="60"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="969.5" y="88.5">5</text>
+    </g>
+    <path d="M 1000 80 L 1043.63 80" fill="none" pointer-events="stroke" stroke="#000000" stroke-miterlimit="10"/>
+    <path d="M 1048.88 80 L 1041.88 83.5 L 1043.63 80 L 1041.88 76.5 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="980" y="60"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="989.5" y="88.5">4</text>
+    </g>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="1050" y="60"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="1059.5" y="88.5">3</text>
+    </g>
+    <path d="M 1090 85 L 1100.64 90.32" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+    <path d="M 1105.33 92.67 L 1097.51 92.67 L 1100.64 90.32 L 1100.64 86.41 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="1070" y="60"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="1079.5" y="88.5">2</text>
+    </g>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="1090" y="60"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="1099.5" y="88.5">1</text>
+    </g>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="940" y="140"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="949.5" y="168.5">j</text>
+    </g>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="960" y="140"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="969.5" y="168.5">i</text>
+    </g>
+    <path d="M 1000 160 L 1043.63 160" fill="none" pointer-events="stroke" stroke="#000000" stroke-miterlimit="10"/>
+    <path d="M 1048.88 160 L 1041.88 163.5 L 1043.63 160 L 1041.88 156.5 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="1050" y="140"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="1059.5" y="168.5">g</text>
+    </g>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="1070" y="140"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="1079.5" y="168.5">f</text>
+    </g>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="1090" y="140"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="1099.5" y="168.5">e</text>
+    </g>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="1220" y="100"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="1229.5" y="128.5">d</text>
+    </g>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="1240" y="100"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="1249.5" y="128.5">c</text>
+    </g>
+    <rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="1260" y="100"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="1269.5" y="128.5">b</text>
+    </g>
+    <rect fill="none" height="40" pointer-events="all" stroke="none" width="110" x="1170" y="0"/>
+    <g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+      <text x="1224.5" y="17">Checkpoint</text>
+      <text x="1224.5" y="35">barrier</text>
+    </g>
+    <rect fill="none" height="20" pointer-events="all" stroke="none" width="40" x="1070" y="210"/>
+    <g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+      <text x="1089.5" y="217">Input</text>
+      <text x="1089.5" y="235">buffers</text>
+    </g>
+    <path d="M 1090 200 L 1082.85 185.7" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+    <path d="M 1080.5 181 L 1086.76 185.7 L 1082.85 185.7 L 1080.5 188.83 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+    <path d="M 1090 200 L 1097.15 185.7" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+    <path d="M 1099.5 181 L 1099.5 188.83 L 1097.15 185.7 L 1093.24 185.7 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+    <path d="M 1223.57 40 L 1220.45 83.65" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+    <path d="M 1220.08 88.88 L 1217.09 81.65 L 1220.45 83.65 L 1224.07 82.15 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+    <path d="M 958.06 210 L 953.44 186.25" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+    <path d="M 952.44 181.1 L 957.21 187.3 L 953.44 186.25 L 950.34 188.64 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+    <path d="M 961.88 210 L 966.33 186.26" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+    <path d="M 967.29 181.1 L 969.44 188.62 L 966.33 186.26 L 962.56 187.33 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+    <rect fill="none" height="20" pointer-events="all" stroke="none" width="40" x="940" y="210"/>
+    <g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+      <text x="959.5" y="217">Output</text>
+      <text x="959.5" y="235">buffers</text>
+    </g>
+    <rect fill="none" height="40" pointer-events="all" stroke="none" width="200" x="990" y="250"/>
+    <g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+      <text x="1089.5" y="278.5">Checkpoint</text>
+    </g>
+    <path d="M 1219 150 L 1219 90" fill="none" pointer-events="stroke" stroke="#8a3142" stroke-miterlimit="10" stroke-width="3"/>
+    <path d="M 1240 212 C 1240 196 1310 196 1310 212 L 1310 248 C 1310 264 1240 264 1240 248 Z" fill="#70ad47" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+    <path d="M 1240 212 C 1240 224 1310 224 1310 212" fill="none" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+    <g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+      <text x="1274.5" y="236">State</text>
+      <text x="1274.5" y="254">backend</text>
+    </g>
+    <path d="M 250 146 C 250 138 310 138 310 146 L 310 164 C 310 172 250 172 250 164 Z" fill="#70ad47" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+    <path d="M 250 146 C 250 152 310 152 310 146" fill="none" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+    <g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+      <text x="279.5" y="165.5">state</text>
+    </g>
+    <path d="M 690 146 C 690 138 750 138 750 146 L 750 164 C 750 172 690 172 690 164 Z" fill="#70ad47" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+    <path d="M 690 146 C 690 152 750 152 750 146" fill="none" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+    <g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+      <text x="719.5" y="165.5">state</text>
+    </g>
+    <path d="M 1183 170 L 1233.1 202.67" fill="none" pointer-events="stroke" stroke="#724591" stroke-dasharray="6 2" stroke-miterlimit="10" stroke-width="2"/>
+    <path d="M 1238.13 205.95 L 1229.24 204.93 L 1233.1 202.67 L 1233.61 198.23 Z" fill="#724591" pointer-events="all" stroke="#724591" stroke-miterlimit="10" stroke-width="2"/>
+    <path d="M 1130 146 C 1130 138 1190 138 1190 146 L 1190 164 C 1190 172 1130 172 1130 164 Z" fill="#70ad47" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+    <path d="M 1130 146 C 1130 152 1190 152 1190 146" fill="none" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+    <g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+      <text x="1159.5" y="165.5">state</text>
+    </g>
+    <rect fill="url(#mx-gradient-ffffff-1-ffffff00-1-e-0)" height="190" pointer-events="all" stroke="none" width="170" x="0" y="50"/>
+    <path d="M 420 285 L 420 5" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+    <path d="M 860 285 L 860 5" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+    <rect fill="url(#mx-gradient-ffffff-1-ffffff00-1-e-0)" height="190" pointer-events="all" stroke="none" width="170" x="440" y="50"/>
+    <rect fill="url(#mx-gradient-ffffff-1-ffffff00-1-e-0)" height="190" pointer-events="all" stroke="none" width="170" x="880" y="50"/>
   </g>
-</svg>
+</svg>
\ No newline at end of file
diff --git a/docs/fig/stream_unaligning.svg b/docs/fig/stream_unaligning.svg
new file mode 100644
index 0000000..8a8b26f
--- /dev/null
+++ b/docs/fig/stream_unaligning.svg
@@ -0,0 +1,383 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+
+<!DOCTYPE svg
+  PUBLIC '-//W3C//DTD SVG 1.1//EN'
+  'http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd'>
+<svg content="&lt;mxfile modified=&quot;2020-06-19T07:13:49.823Z&quot; host=&quot;app.diagrams.net&quot; agent=&quot;5.0 (Macintosh)&quot; etag=&quot;L5GzBTQtblpbB8QD-aNV&quot; version=&quot;13.2.6&quot; type=&quot;google&quot;&gt;&lt;diagram id=&quot;n4wjEhElwpPeuiAcmKNk&quot; name=&quot;Page-1&quot;&gt;5V1Rc9o4EP41eUwGSTbGj03a5DqTXrm2N7325UbBBnxxEGecltyvPznIgLUCC2pbEtCHxsI29rer1e7qW+mC3Dwt7zI6n35gUZxe4F60vCBvLzDGA8/j/xUtL6KF9PCqZZIl0aoNbRo+J//ForEnWp+TKF5UTswZS/NkXm0csdksHuWVNppl7Gf1tD [...]
+	<defs>
+		<linearGradient id="mx-gradient-ffffff-1-ffffff00-1-e-0" x1="0%" x2="100%" y1="0%" y2="0%">
+			<stop offset="0%" style="stop-color:#FFFFFF"/>
+			<stop offset="100%" style="stop-color:#FFFFFF" stop-opacity="0"/>
+		</linearGradient>
+	</defs>
+	<g>
+		<path d="M 1000 162.46 L 1232 219.44" fill="none" pointer-events="stroke" stroke="#724591" stroke-dasharray="6 2" stroke-miterlimit="10" stroke-width="2"/>
+		<path d="M 1237.83 220.87 L 1229.11 222.85 L 1232 219.44 L 1231.01 215.08 Z" fill="#724591" pointer-events="all" stroke="#724591" stroke-miterlimit="10" stroke-width="2"/>
+		<rect fill="#70ad47" height="40" pointer-events="all" stroke="#000000" width="20" x="980" y="140"/>
+		<g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+			<text x="989.5" y="166">d</text>
+		</g>
+		<rect fill="#70ad47" height="40" pointer-events="all" stroke="#000000" width="20" x="540" y="140"/>
+		<g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+			<text x="549.5" y="166">d</text>
+		</g>
+		<path d="M -20 90 C -20 123.14 6.86 150 40 150 C 73.14 150 100 123.14 100 90 Z" fill="#f5a030" pointer-events="all" stroke="#000000" stroke-miterlimit="10" transform="rotate(-90,40,120)"/>
+		<rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="60" y="60"/>
+		<g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+			<text x="69.5" y="88.5">6</text>
+		</g>
+		<ellipse cx="280" cy="120" fill="#f5a030" pointer-events="all" rx="60" ry="60" stroke="#000000"/>
+		<g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+			<text x="279.5" y="128.5">Operator</text>
+		</g>
+		<rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="80" y="60"/>
+		<g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+			<text x="89.5" y="88.5">5</text>
+		</g>
+		<path d="M 120 80 L 163.63 80" fill="none" pointer-events="stroke" stroke="#000000" stroke-miterlimit="10"/>
+		<path d="M 168.88 80 L 161.88 83.5 L 163.63 80 L 161.88 76.5 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+		<rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="100" y="60"/>
+		<g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+			<text x="109.5" y="88.5">4</text>
+		</g>
+		<rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="170" y="60"/>
+		<g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+			<text x="179.5" y="88.5">3</text>
+		</g>
+		<rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="190" y="60"/>
+		<g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+			<text x="199.5" y="88.5">2</text>
+		</g>
+		<rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="210" y="60"/>
+		<g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+			<text x="219.5" y="88.5">1</text>
+		</g>
+		<rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="60" y="140"/>
+		<g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+			<text x="69.5" y="168.5">f</text>
+		</g>
+		<rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="80" y="140"/>
+		<g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+			<text x="89.5" y="168.5">e</text>
+		</g>
+		<path d="M 120 160 L 163.63 160" fill="none" pointer-events="stroke" stroke="#000000" stroke-miterlimit="10"/>
+		<path d="M 168.88 160 L 161.88 163.5 L 163.63 160 L 161.88 156.5 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+		<rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="100" y="140"/>
+		<g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+			<text x="109.5" y="168.5">d</text>
+		</g>
+		<rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="170" y="140"/>
+		<g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+			<text x="179.5" y="168.5">c</text>
+		</g>
+		<rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="190" y="140"/>
+		<g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+			<text x="199.5" y="168.5">b</text>
+		</g>
+		<rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="210" y="140"/>
+		<g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+			<text x="219.5" y="168.5">a</text>
+		</g>
+		<path d="M 170 110 L 170 50" fill="none" pointer-events="stroke" stroke="#000000" stroke-miterlimit="10" stroke-width="3"/>
+		<rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="340" y="100"/>
+		<g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+			<text x="349.5" y="128.5">z</text>
+		</g>
+		<rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="360" y="100"/>
+		<g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+			<text x="369.5" y="128.5">y</text>
+		</g>
+		<rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="380" y="100"/>
+		<g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+			<text x="389.5" y="128.5">x</text>
+		</g>
+		<rect fill="none" height="40" pointer-events="all" stroke="none" width="110" x="80" y="0"/>
+		<g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+			<text x="134.5" y="17">Checkpoint</text>
+			<text x="134.5" y="35">barrier</text>
+		</g>
+		<rect fill="none" height="20" pointer-events="all" stroke="none" width="40" x="190" y="210"/>
+		<g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+			<text x="209.5" y="217">Input</text>
+			<text x="209.5" y="235">buffers</text>
+		</g>
+		<path d="M 210 200 L 202.85 185.7" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+		<path d="M 200.5 181 L 206.76 185.7 L 202.85 185.7 L 200.5 188.83 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+		<path d="M 210 200 L 217.15 185.7" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+		<path d="M 219.5 181 L 219.5 188.83 L 217.15 185.7 L 213.24 185.7 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+		<path d="M 158.33 40 L 165.17 45.86" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+		<path d="M 169.15 49.27 L 161.56 47.37 L 165.17 45.86 L 166.11 42.06 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+		<path d="M 358.75 210 L 350.79 146.32" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+		<path d="M 350.14 141.11 L 354.48 147.62 L 350.79 146.32 L 347.53 148.49 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+		<path d="M 361 210 L 367.37 146.34" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+		<path d="M 367.89 141.11 L 370.67 148.43 L 367.37 146.34 L 363.71 147.73 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+		<rect fill="none" height="20" pointer-events="all" stroke="none" width="40" x="340" y="210"/>
+		<g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+			<text x="359.5" y="217">Output</text>
+			<text x="359.5" y="235">buffers</text>
+		</g>
+		<path d="M 78.06 210 L 73.44 186.25" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+		<path d="M 72.44 181.1 L 77.21 187.3 L 73.44 186.25 L 70.34 188.64 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+		<path d="M 81.88 210 L 86.33 186.26" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+		<path d="M 87.29 181.1 L 89.44 188.62 L 86.33 186.26 L 82.56 187.33 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+		<rect fill="none" height="20" pointer-events="all" stroke="none" width="40" x="60" y="210"/>
+		<g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+			<text x="79.5" y="217">Output</text>
+			<text x="79.5" y="235">buffers</text>
+		</g>
+		<path d="M 100 190 L 100 130" fill="none" pointer-events="stroke" stroke="#000000" stroke-miterlimit="10" stroke-width="3"/>
+		<rect fill="none" height="40" pointer-events="all" stroke="none" width="200" x="90" y="250"/>
+		<g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+			<text x="189.5" y="278.5">On first barrier</text>
+		</g>
+		<rect fill="url(#mx-gradient-ffffff-1-ffffff00-1-e-0)" height="190" pointer-events="all" stroke="none" width="170" x="0" y="50"/>
+		<path d="M 128.64 40 L 101.93 123.93" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+		<path d="M 100.34 128.93 L 99.13 121.2 L 101.93 123.93 L 105.8 123.33 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+		<path d="M 420 90 C 420 123.14 446.86 150 480 150 C 513.14 150 540 123.14 540 90 Z" fill="#f5a030" pointer-events="all" stroke="#000000" stroke-miterlimit="10" transform="rotate(-90,480,120)"/>
+		<rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="500" y="60"/>
+		<g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+			<text x="509.5" y="88.5">6</text>
+		</g>
+		<ellipse cx="720" cy="120" fill="#f5a030" pointer-events="all" rx="60" ry="60" stroke="#000000"/>
+		<g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+			<text x="719.5" y="128.5">Operator</text>
+		</g>
+		<rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="520" y="60"/>
+		<g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+			<text x="529.5" y="88.5">5</text>
+		</g>
+		<path d="M 560 80 L 603.63 80" fill="none" pointer-events="stroke" stroke="#000000" stroke-miterlimit="10"/>
+		<path d="M 608.88 80 L 601.88 83.5 L 603.63 80 L 601.88 76.5 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+		<rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="540" y="60"/>
+		<g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+			<text x="549.5" y="88.5">4</text>
+		</g>
+		<rect fill="#70ad47" height="40" pointer-events="all" stroke="#000000" width="20" x="610" y="60"/>
+		<g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+			<text x="619.5" y="86">3</text>
+		</g>
+		<path d="M 650 85 L 660.64 90.32" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+		<path d="M 665.33 92.67 L 657.51 92.67 L 660.64 90.32 L 660.64 86.41 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+		<rect fill="#70ad47" height="40" pointer-events="all" stroke="#000000" width="20" x="630" y="60"/>
+		<g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+			<text x="639.5" y="86">2</text>
+		</g>
+		<rect fill="#70ad47" height="40" pointer-events="all" stroke="#000000" width="20" x="650" y="60"/>
+		<g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+			<text x="659.5" y="86">1</text>
+		</g>
+		<rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="500" y="140"/>
+		<g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+			<text x="509.5" y="168.5">f</text>
+		</g>
+		<rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="520" y="140"/>
+		<g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+			<text x="529.5" y="168.5">e</text>
+		</g>
+		<path d="M 560 160 L 603.63 160" fill="none" pointer-events="stroke" stroke="#000000" stroke-miterlimit="10"/>
+		<path d="M 608.88 160 L 601.88 163.5 L 603.63 160 L 601.88 156.5 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+		<rect fill="#70ad47" height="40" pointer-events="all" stroke="#000000" width="20" x="610" y="140"/>
+		<g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+			<text x="619.5" y="166">c</text>
+		</g>
+		<rect fill="#70ad47" height="40" pointer-events="all" stroke="#000000" width="20" x="630" y="140"/>
+		<g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+			<text x="639.5" y="166">b</text>
+		</g>
+		<rect fill="#70ad47" height="40" pointer-events="all" stroke="#000000" width="20" x="650" y="140"/>
+		<g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+			<text x="659.5" y="166">a</text>
+		</g>
+		<rect fill="#70ad47" height="40" pointer-events="all" stroke="#000000" width="20" x="780" y="100"/>
+		<g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+			<text x="789.5" y="126">z</text>
+		</g>
+		<rect fill="#70ad47" height="40" pointer-events="all" stroke="#000000" width="20" x="800" y="100"/>
+		<g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+			<text x="809.5" y="126">y</text>
+		</g>
+		<rect fill="#70ad47" height="40" pointer-events="all" stroke="#000000" width="20" x="820" y="100"/>
+		<g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+			<text x="829.5" y="126">x</text>
+		</g>
+		<rect fill="none" height="40" pointer-events="all" stroke="none" width="110" x="775" y="0"/>
+		<g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+			<text x="829.5" y="17">Checkpoint</text>
+			<text x="829.5" y="35">barrier</text>
+		</g>
+		<rect fill="none" height="20" pointer-events="all" stroke="none" width="40" x="630" y="210"/>
+		<g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+			<text x="649.5" y="217">Input</text>
+			<text x="649.5" y="235">buffers</text>
+		</g>
+		<path d="M 650 200 L 642.85 185.7" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+		<path d="M 640.5 181 L 646.76 185.7 L 642.85 185.7 L 640.5 188.83 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+		<path d="M 650 200 L 657.15 185.7" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+		<path d="M 659.5 181 L 659.5 188.83 L 657.15 185.7 L 653.24 185.7 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+		<path d="M 798.75 210 L 790.79 146.32" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+		<path d="M 790.14 141.11 L 794.48 147.62 L 790.79 146.32 L 787.53 148.49 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+		<path d="M 801 210 L 807.37 146.34" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+		<path d="M 807.89 141.11 L 810.67 148.43 L 807.37 146.34 L 803.71 147.73 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+		<rect fill="none" height="20" pointer-events="all" stroke="none" width="40" x="780" y="210"/>
+		<g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+			<text x="799.5" y="217">Output</text>
+			<text x="799.5" y="235">buffers</text>
+		</g>
+		<path d="M 518.06 210 L 513.44 186.25" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+		<path d="M 512.44 181.1 L 517.21 187.3 L 513.44 186.25 L 510.34 188.64 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+		<path d="M 521.88 210 L 526.33 186.26" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+		<path d="M 527.29 181.1 L 529.44 188.62 L 526.33 186.26 L 522.56 187.33 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+		<rect fill="none" height="20" pointer-events="all" stroke="none" width="40" x="500" y="210"/>
+		<g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+			<text x="519.5" y="217">Output</text>
+			<text x="519.5" y="235">buffers</text>
+		</g>
+		<rect fill="none" height="40" pointer-events="all" stroke="none" width="290" x="495" y="250"/>
+		<g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+			<text x="639.5" y="278.5">Tag buffers and forward barrier</text>
+		</g>
+		<path d="M 832.86 40 L 839.1 83.7" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+		<path d="M 839.84 88.89 L 835.39 82.46 L 839.1 83.7 L 842.32 81.47 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+		<path d="M 860 90 C 860 123.14 886.86 150 920 150 C 953.14 150 980 123.14 980 90 Z" fill="#f5a030" pointer-events="all" stroke="#000000" stroke-miterlimit="10" transform="rotate(-90,920,120)"/>
+		<rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="940" y="60"/>
+		<g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+			<text x="949.5" y="88.5">6</text>
+		</g>
+		<ellipse cx="1160" cy="120" fill="#f5a030" pointer-events="all" rx="60" ry="60" stroke="#000000"/>
+		<g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+			<text x="1159.5" y="128.5">Operator</text>
+		</g>
+		<rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="960" y="60"/>
+		<g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+			<text x="969.5" y="88.5">5</text>
+		</g>
+		<path d="M 1000 80 L 1043.63 80" fill="none" pointer-events="stroke" stroke="#000000" stroke-miterlimit="10"/>
+		<path d="M 1048.88 80 L 1041.88 83.5 L 1043.63 80 L 1041.88 76.5 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+		<rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="980" y="60"/>
+		<g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+			<text x="989.5" y="88.5">4</text>
+		</g>
+		<rect fill="#70ad47" height="40" pointer-events="all" stroke="#000000" width="20" x="1050" y="60"/>
+		<g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+			<text x="1059.5" y="86">3</text>
+		</g>
+		<path d="M 1090 85 L 1100.64 90.32" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+		<path d="M 1105.33 92.67 L 1097.51 92.67 L 1100.64 90.32 L 1100.64 86.41 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+		<rect fill="#70ad47" height="40" pointer-events="all" stroke="#000000" width="20" x="1070" y="60"/>
+		<g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+			<text x="1079.5" y="86">2</text>
+		</g>
+		<rect fill="#70ad47" height="40" pointer-events="all" stroke="#000000" width="20" x="1090" y="60"/>
+		<g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+			<text x="1099.5" y="86">1</text>
+		</g>
+		<rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="940" y="140"/>
+		<g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+			<text x="949.5" y="168.5">f</text>
+		</g>
+		<rect fill="#e6e6e6" height="40" pointer-events="all" stroke="#000000" width="20" x="960" y="140"/>
+		<g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+			<text x="969.5" y="168.5">e</text>
+		</g>
+		<path d="M 1000 160 L 1043.63 160" fill="none" pointer-events="stroke" stroke="#000000" stroke-miterlimit="10"/>
+		<path d="M 1048.88 160 L 1041.88 163.5 L 1043.63 160 L 1041.88 156.5 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+		<rect fill="#70ad47" height="40" pointer-events="all" stroke="#000000" width="20" x="1050" y="140"/>
+		<g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+			<text x="1059.5" y="166">c</text>
+		</g>
+		<rect fill="#70ad47" height="40" pointer-events="all" stroke="#000000" width="20" x="1070" y="140"/>
+		<g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+			<text x="1079.5" y="166">b</text>
+		</g>
+		<rect fill="#70ad47" height="40" pointer-events="all" stroke="#000000" width="20" x="1090" y="140"/>
+		<g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+			<text x="1099.5" y="166">a</text>
+		</g>
+		<rect fill="#70ad47" height="40" pointer-events="all" stroke="#000000" width="20" x="1220" y="100"/>
+		<g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+			<text x="1229.5" y="126">z</text>
+		</g>
+		<rect fill="#70ad47" height="40" pointer-events="all" stroke="#000000" width="20" x="1240" y="100"/>
+		<g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+			<text x="1249.5" y="126">y</text>
+		</g>
+		<rect fill="#70ad47" height="40" pointer-events="all" stroke="#000000" width="20" x="1260" y="100"/>
+		<g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+			<text x="1269.5" y="126">x</text>
+		</g>
+		<rect fill="none" height="20" pointer-events="all" stroke="none" width="40" x="1070" y="210"/>
+		<g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+			<text x="1089.5" y="217">Input</text>
+			<text x="1089.5" y="235">buffers</text>
+		</g>
+		<path d="M 1090 200 L 1082.85 185.7" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+		<path d="M 1080.5 181 L 1086.76 185.7 L 1082.85 185.7 L 1080.5 188.83 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+		<path d="M 1090 200 L 1097.15 185.7" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+		<path d="M 1099.5 181 L 1099.5 188.83 L 1097.15 185.7 L 1093.24 185.7 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+		<path d="M 958.06 210 L 953.44 186.25" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+		<path d="M 952.44 181.1 L 957.21 187.3 L 953.44 186.25 L 950.34 188.64 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+		<path d="M 961.88 210 L 966.33 186.26" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+		<path d="M 967.29 181.1 L 969.44 188.62 L 966.33 186.26 L 962.56 187.33 Z" fill="#000000" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+		<rect fill="none" height="20" pointer-events="all" stroke="none" width="40" x="940" y="210"/>
+		<g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+			<text x="959.5" y="217">Output</text>
+			<text x="959.5" y="235">buffers</text>
+		</g>
+		<rect fill="none" height="40" pointer-events="all" stroke="none" width="200" x="1000" y="250"/>
+		<g fill="#000000" font-family="Helvetica" font-size="20px" text-anchor="middle">
+			<text x="1099.5" y="278.5">Checkpoint</text>
+		</g>
+		<path d="M 1240 212 C 1240 196 1310 196 1310 212 L 1310 248 C 1310 264 1240 264 1240 248 Z" fill="#70ad47" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+		<path d="M 1240 212 C 1240 224 1310 224 1310 212" fill="none" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+		<g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+			<text x="1274.5" y="236">State </text>
+			<text x="1274.5" y="254">backend</text>
+		</g>
+		<path d="M 250 146 C 250 138 310 138 310 146 L 310 164 C 310 172 250 172 250 164 Z" fill="#70ad47" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+		<path d="M 250 146 C 250 152 310 152 310 146" fill="none" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+		<g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+			<text x="279.5" y="165.5">state</text>
+		</g>
+		<path d="M 690 146 C 690 138 750 138 750 146 L 750 164 C 750 172 690 172 690 164 Z" fill="#70ad47" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+		<path d="M 690 146 C 690 152 750 152 750 146" fill="none" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+		<g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+			<text x="719.5" y="165.5">state</text>
+		</g>
+		<path d="M 1183 170 L 1233.1 202.67" fill="none" pointer-events="stroke" stroke="#724591" stroke-dasharray="6 2" stroke-miterlimit="10" stroke-width="2"/>
+		<path d="M 1238.13 205.95 L 1229.24 204.93 L 1233.1 202.67 L 1233.61 198.23 Z" fill="#724591" pointer-events="all" stroke="#724591" stroke-miterlimit="10" stroke-width="2"/>
+		<path d="M 840 150 L 840 90" fill="none" pointer-events="stroke" stroke="#000000" stroke-miterlimit="10" stroke-width="3"/>
+		<path d="M 1254.55 140 L 1266.36 191.97" fill="none" pointer-events="stroke" stroke="#724591" stroke-dasharray="6 2" stroke-miterlimit="10" stroke-width="2"/>
+		<path d="M 1267.69 197.82 L 1262.01 190.91 L 1266.36 191.97 L 1269.81 189.13 Z" fill="#724591" pointer-events="all" stroke="#724591" stroke-miterlimit="10" stroke-width="2"/>
+		<path d="M 1110 88.57 L 1233.75 194.64" fill="none" pointer-events="stroke" stroke="#724591" stroke-dasharray="6 2" stroke-miterlimit="10" stroke-width="2"/>
+		<path d="M 1238.3 198.54 L 1229.63 196.38 L 1233.75 194.64 L 1234.83 190.3 Z" fill="#724591" pointer-events="all" stroke="#724591" stroke-miterlimit="10" stroke-width="2"/>
+		<path d="M 420 285 L 420 5" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+		<path d="M 870 285 L 870 5" fill="none" pointer-events="stroke" stroke="#000000" stroke-dasharray="3 3" stroke-miterlimit="10"/>
+		<rect fill="url(#mx-gradient-ffffff-1-ffffff00-1-e-0)" height="190" pointer-events="all" stroke="none" width="170" x="440" y="50"/>
+		<rect fill="url(#mx-gradient-ffffff-1-ffffff00-1-e-0)" height="190" pointer-events="all" stroke="none" width="170" x="880" y="50"/>
+		<path d="M 1110 164 L 1232.35 212.94" fill="none" pointer-events="stroke" stroke="#724591" stroke-dasharray="6 2" stroke-miterlimit="10" stroke-width="2"/>
+		<path d="M 1237.92 215.17 L 1229.01 215.91 L 1232.35 212.94 L 1231.98 208.48 Z" fill="#724591" pointer-events="all" stroke="#724591" stroke-miterlimit="10" stroke-width="2"/>
+		<path d="M 1130 146 C 1130 138 1190 138 1190 146 L 1190 164 C 1190 172 1130 172 1130 164 Z" fill="#70ad47" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+		<path d="M 1130 146 C 1130 152 1190 152 1190 146" fill="none" pointer-events="all" stroke="#000000" stroke-miterlimit="10"/>
+		<g fill="#000000" font-family="Helvetica" font-size="15px" text-anchor="middle">
+			<text x="1159.5" y="165.5">state</text>
+		</g>
+	</g>
+</svg>
\ No newline at end of file
diff --git a/docs/ops/state/checkpoints.md b/docs/ops/state/checkpoints.md
index cf54b07..9264231 100644
--- a/docs/ops/state/checkpoints.md
+++ b/docs/ops/state/checkpoints.md
@@ -113,4 +113,51 @@ above).
 $ bin/flink run -s :checkpointMetaDataPath [:runArgs]
 {% endhighlight %}
 
+### Unaligned checkpoints
+
+Starting with Flink 1.11, checkpoints can be unaligned. 
+[Unaligned checkpoints]({% link concepts/stateful-stream-processing.md
+%}#unaligned-checkpointing) contain in-flight data (i.e., data stored in
+buffers) as part of the checkpoint state, which allows checkpoint barriers to
+overtake these buffers. Thus, the checkpoint duration becomes independent of the
+current throughput as checkpoint barriers are effectively not embedded into 
+the stream of data anymore.
+
+You should use unaligned checkpoints if your checkpointing durations are very
+high due to backpressure. Then, checkpointing time becomes mostly
+independent of the end-to-end latency. Be aware unaligned checkpointing
+adds to I/O to the state backends, so you shouldn't use it when the I/O to
+the state backend is actually the bottleneck during checkpointing.
+
+Note that unaligned checkpoints is a brand-new feature that currently has the
+following limitations:
+
+- You cannot rescale or change job graph with from unaligned checkpoints. You 
+  have to take a savepoint before rescaling. Savepoints are always aligned 
+  independent of the alignment setting of checkpoints.
+- Flink currently does not support concurrent unaligned checkpoints. However, 
+  due to the more predictable and shorter checkpointing times, concurrent 
+  checkpoints might not be needed at all. However, savepoints can also not 
+  happen concurrently to unaligned checkpoints, so they will take slightly 
+  longer.
+- Unaligned checkpoints break with an implicit guarantee in respect to 
+  watermarks during recovery:
+
+Currently, Flink generates the watermark as a first step of recovery instead of 
+storing the latest watermark in the operators to ease rescaling. In unaligned 
+checkpoints, that means on recovery, **Flink generates watermarks after it 
+restores in-flight data**. If your pipeline uses an **operator that applies the
+latest watermark on each record**, it will produce **different results** than 
+for aligned checkpoints. If your operator depends on the latest watermark being
+always available, then the workaround is to store the watermark in the operator 
+state. To support rescaling, watermarks should be stored per key-group in a 
+union-state. We most likely will implement this approach as a general solution 
+(didn't make it into Flink 1.11.0).
+
+In the upcoming release(s), Flink will address these limitations and will
+provide a fine-grained way to trigger unaligned checkpoints only for the 
+in-flight data that moves slowly with timeout mechanism. These options will
+decrease the pressure on I/O in the state backends and eventually allow
+unaligned checkpoints to become the default checkpointing. 
+
 {% top %}
diff --git a/docs/ops/state/large_state_tuning.md b/docs/ops/state/large_state_tuning.md
index ee9e93c..55bf003 100644
--- a/docs/ops/state/large_state_tuning.md
+++ b/docs/ops/state/large_state_tuning.md
@@ -278,6 +278,8 @@ that the task-local state is an in-memory consisting of heap objects, and not st
 Task-local recovery is *deactivated by default* and can be activated through Flink's configuration with the key `state.backend.local-recovery` as specified
 in `CheckpointingOptions.LOCAL_RECOVERY`. The value for this setting can either be *true* to enable or *false* (default) to disable local recovery.
 
+Note that [unaligned checkpoints]({% link ops/state/checkpoints.md %}#unaligned-checkpoints) currently do not support task-local recovery.
+
 ### Details on task-local recovery for different state backends
 
 ***Limitation**: Currently, task-local recovery only covers keyed state backends. Keyed state is typically by far the largest part of the state. In the near future, we will
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/ExecutionCheckpointingOptions.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/ExecutionCheckpointingOptions.java
index 08da029..fcfe264 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/ExecutionCheckpointingOptions.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/ExecutionCheckpointingOptions.java
@@ -138,8 +138,9 @@ public class ExecutionCheckpointingOptions {
 					"the stream of data anymore.")
 				.linebreak()
 				.linebreak()
-				.text("Unaligned checkpoints can only be enabled if %s is %s",
+				.text("Unaligned checkpoints can only be enabled if %s is %s and if %s is 1",
 					TextElement.code(CHECKPOINTING_MODE.key()),
-					TextElement.code(CheckpointingMode.EXACTLY_ONCE.toString()))
+					TextElement.code(CheckpointingMode.EXACTLY_ONCE.toString()),
+					TextElement.code(MAX_CONCURRENT_CHECKPOINTS.key()))
 				.build());
 }