You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by al...@apache.org on 2018/08/01 08:03:54 UTC

[flink] branch master updated: [FLINK-9877][docs] Add documentation page for different datastream joins

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

aljoscha pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
     new 81c445f  [FLINK-9877][docs] Add documentation page for different datastream joins
81c445f is described below

commit 81c445ffa0058d186287def8d5cec96795ec0d6a
Author: Florian Schmidt <fl...@icloud.com>
AuthorDate: Tue Jul 24 18:14:50 2018 +0200

    [FLINK-9877][docs] Add documentation page for different datastream joins
---
 docs/dev/stream/operators/joining.md | 282 +++++++++++++++++++++++++++++++++++
 docs/fig/interval-join.svg           | 147 ++++++++++++++++++
 docs/fig/session-window-join.svg     | 125 ++++++++++++++++
 docs/fig/sliding-window-join.svg     | 113 ++++++++++++++
 docs/fig/tumbling-window-join.svg    | 131 ++++++++++++++++
 5 files changed, 798 insertions(+)

diff --git a/docs/dev/stream/operators/joining.md b/docs/dev/stream/operators/joining.md
new file mode 100644
index 0000000..b95aadd
--- /dev/null
+++ b/docs/dev/stream/operators/joining.md
@@ -0,0 +1,282 @@
+---
+title: "Joining"
+nav-id: streaming_joins
+nav-show_overview: true
+nav-parent_id: streaming_operators
+nav-pos: 11
+---
+<!--
+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.
+-->
+
+* toc
+{:toc}
+
+# Window Join
+A window join joins the elements of two streams that share a common key and lie in the same window. These windows can be defined by using a [window assigner]({{ site.baseurl}}/dev/stream/operators/windows.html#window-assigners) and are evaluated on elements from both of the streams.
+
+The elements from both sides are then passed to a user-defined `JoinFunction` or `FlatJoinFunction` where the user can emit results that meet the join criteria.
+
+The general usage can be summarized as follows:
+
+{% highlight java %}
+stream.join(otherStream)
+    .where(<KeySelector>)
+    .equalTo(<KeySelector>)
+    .window(<WindowAssigner>)
+    .apply(<JoinFunction>)
+{% endhighlight %}
+
+Some notes on semantics:
+- The creation of pairwise combinations of elements of the two streams behaves like an inner-join, meaning elements from one stream will not be emitted if they don't have a corresponding element from the other stream to be joined with.
+- Those elements that do get joined will have as their timestamp the largest timestamp that still lies in the respective window. For example a window with `[5, 10)` as its boundaries would result in the joined elements having 9 as their timestamp.
+
+In the following section we are going to give an overview over how different kinds of window joins behave using some exemplary scenarios.
+
+## Tumbling Window Join
+When performing a tumbling window join, all elements with a common key and a common tumbling window are joined as pairwise combinations and passed on to a `JoinFunction` or `FlatJoinFunction`. Because this behaves like an inner join, elements of one stream that do not have elements from another stream in their tumbling window are not emitted!
+
+<img src="{{ site.baseurl }}/fig/tumbling-window-join.svg" class="center" style="width: 80%;" />
+
+As illustrated in the figure, we define a tumbling window with the size of 2 milliseconds, which results in windows of the form `[0,1], [2,3], ...`. The image shows the pairwise combinations of all elements in each window which will be passed on to the `JoinFunction`. Note that in the tumbling window `[6,7]` nothing is emitted because no elements exist in the green stream to be joined with the orange elements ⑥ and ⑦.
+
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+{% highlight java %}
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows;
+import org.apache.flink.streaming.api.windowing.time.Time;
+ 
+...
+
+DataStream<Integer> orangeStream = ...
+DataStream<Integer> greenStream = ...
+
+orangeStream.join(greenStream)
+    .where(<KeySelector>)
+    .equalTo(<KeySelector>)
+    .window(TumblingEventTimeWindows.of(Time.seconds(2)))
+    .apply (new JoinFunction<Integer, Integer, String> (){
+        @Override
+        public String join(Integer first, Integer second) {
+            return first + "," + second;
+        }
+    });
+{% endhighlight %}
+</div>
+<div data-lang="scala" markdown="1">
+
+{% highlight scala %}
+import org.apache.flink.streaming.api.windowing.assigners.SlidingEventTimeWindows;
+import org.apache.flink.streaming.api.windowing.time.Time;
+
+...
+
+val orangeStream: DataStream[Integer] = ...
+val greenStream: DataStream[Integer] = ...
+
+orangeStream.join(greenStream)
+    .where(elem => /* select key */)
+    .equalTo(elem => /* select key */)
+    .window(TumblingEventTimeWindows.of(Time.milliseconds(2)))
+    .apply { (e1, e2) => e1 + "," + e2 }
+{% endhighlight %}
+
+</div>
+</div>
+
+## Sliding Window Join
+When performing a sliding window join, all elements with a common key and common sliding window are joined are pairwise combinations and passed on to the `JoinFunction` or `FlatJoinFunction`. Elements of one stream that do not have elements from the other stream in the current sliding window are not emitted! Note that some elements might be joined in one sliding window but not in another!
+
+<img src="{{ site.baseurl }}/fig/sliding-window-join.svg" class="center" style="width: 80%;" />
+
+In this example we are using sliding windows with a size of two milliseconds and slide them by one millisecond, resulting in the sliding windows `[-1, 0],[0,1],[1,2],[2,3], …`.<!-- TODO: Can -1 actually exist?--> The joined elements below the x-axis are the ones that are passed to the `JoinFunction` for each sliding window. Here you can also see how for example the orange ② is joined with the green ③ in the window `[2,3]`, but is not joined with anything in the window `[1,2]`.
+
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+
+{% highlight java %}
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.streaming.api.windowing.assigners.SlidingEventTimeWindows;
+import org.apache.flink.streaming.api.windowing.time.Time;
+
+...
+
+DataStream<Integer> orangeStream = ...
+DataStream<Integer> greenStream = ...
+
+orangeStream.join(greenStream)
+    .where(<KeySelector>)
+    .equalTo(<KeySelector>)
+    .window(SlidingEventTimeWindows.of(Time.milliseconds(2) /* size */, Time.milliseconds(1) /* slide */))
+    .apply (new JoinFunction<Integer, Integer, String> (){
+        @Override
+        public String join(Integer first, Integer second) {
+            return first + "," + second;
+        }
+    });
+{% endhighlight %}
+</div>
+<div data-lang="scala" markdown="1">
+
+{% highlight scala %}
+import org.apache.flink.streaming.api.windowing.assigners.SlidingEventTimeWindows;
+import org.apache.flink.streaming.api.windowing.time.Time;
+
+...
+
+val orangeStream: DataStream[Integer] = ...
+val greenStream: DataStream[Integer] = ...
+
+orangeStream.join(greenStream)
+    .where(elem => /* select key */)
+    .equalTo(elem => /* select key */)
+    .window(SlidingEventTimeWindows.of(Time.milliseconds(2) /* size */, Time.milliseconds(1) /* slide */))
+    .apply { (e1, e2) => e1 + "," + e2 }
+{% endhighlight %}
+</div>
+</div>
+
+## Session Window Join
+When performing a session window join, all elements with the same key that when _"combined"_ fulfill the session criteria are joined in pairwise combinations and passed on to the `JoinFunction` or `FlatJoinFunction`. Again this performs an inner join, so if there is a session window that only contains elements from one stream, no output will be emitted!
+
+<img src="{{ site.baseurl }}/fig/session-window-join.svg" class="center" style="width: 80%;" />
+
+Here we define a session window join where each session is divided by a gap of at least 1ms. There are three sessions, and in the first two sessions the joined elements from both streams are passed to the `JoinFunction`. In the third session there are no elements in the green stream, so ⑧ and ⑨ are not joined!
+
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+
+{% highlight java %}
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.streaming.api.windowing.assigners.EventTimeSessionWindows;
+import org.apache.flink.streaming.api.windowing.time.Time;
+ 
+...
+
+DataStream<Integer> orangeStream = ...
+DataStream<Integer> greenStream = ...
+
+orangeStream.join(greenStream)
+    .where(<KeySelector>)
+    .equalTo(<KeySelector>)
+    .window(EventTimeSessionWindows.withGap(Time.milliseconds(1)))
+    .apply (new JoinFunction<Integer, Integer, String> (){
+        @Override
+        public String join(Integer first, Integer second) {
+            return first + "," + second;
+        }
+    });
+{% endhighlight %}
+</div>
+<div data-lang="scala" markdown="1">
+
+{% highlight scala %}
+import org.apache.flink.streaming.api.windowing.assigners.EventTimeSessionWindows;
+import org.apache.flink.streaming.api.windowing.time.Time;
+ 
+...
+
+val orangeStream: DataStream[Integer] = ...
+val greenStream: DataStream[Integer] = ...
+
+orangeStream.join(greenStream)
+    .where(elem => /* select key */)
+    .equalTo(elem => /* select key */)
+    .window(EventTimeSessionWindows.withGap(Time.milliseconds(1)))
+    .apply { (e1, e2) => e1 + "," + e2 }
+{% endhighlight %}
+
+</div>
+</div>
+
+# Interval Join
+The interval join joins elements of two streams (we'll call them A & B for now) with a common key and where elements of stream B have timestamps that lie in a relative time interval to timestamps of elements in stream A.
+
+This can also be expressed more formally as
+`b.timestamp ∈ [a.timestamp + lowerBound; a.timestamp + upperBound]` or 
+`a.timestamp + lowerBound <= b.timestamp <= a.timestamp + upperBound`
+
+where a and b are elements of A and B that share a common key. Both the lower and upper bound can be either negative or positive as long as as the lower bound is always smaller or equal to the upper bound. The interval join currently only performs inner joins.
+
+When a pair of elements are passed to the `ProcessJoinFunction`, they will be assigned with the larger timestamp (which can be accessed via the `ProcessJoinFunction.Context`) of the two elements.
+
+<span class="label label-info">Note</span> The interval join currently only supports event time.
+
+<img src="{{ site.baseurl }}/fig/interval-join.svg" class="center" style="width: 80%;" />
+
+In the example above, we join two streams 'orange' and 'green' with a lower bound of -2 milliseconds and an upper bound of +1 millisecond. Be default, these boundaries are inclusive, but `.lowerBoundExclusive()` and `.upperBoundExclusive` can be applied to change the behaviour.
+
+Using the more formal notation again this will translate to 
+
+`orangeElem.ts + lowerBound <= greenElem.ts <= orangeElem.ts + upperBound`
+
+as indicated by the triangles.
+
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+
+{% highlight java %}
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.streaming.api.functions.co.ProcessJoinFunction;
+import org.apache.flink.streaming.api.windowing.time.Time;
+
+...
+
+DataStream<Integer> orangeStream = ...
+DataStream<Integer> greenStream = ...
+
+orangeStream
+    .keyBy(<KeySelector>)
+    .intervalJoin(greenStream.keyBy(<KeySelector>))
+    .between(Time.milliseconds(-2), Time.milliseconds(1))
+    .process (new ProcessJoinFunction<Integer, Integer, String(){
+
+        @Override
+        public void processElement(Integer left, Integer right, Context ctx, Collector<String> out) {
+            out.collect(first + "," + second);
+        }
+    });
+{% endhighlight %}
+
+</div>
+<div data-lang="scala" markdown="1">
+
+{% highlight scala %}
+import org.apache.flink.streaming.api.functions.co.ProcessJoinFunction;
+import org.apache.flink.streaming.api.windowing.time.Time;
+
+...
+
+val orangeStream: DataStream[Integer] = ...
+val greenStream: DataStream[Integer] = ...
+
+orangeStream
+    .keyBy(elem => /* select key */)
+    .intervalJoin(greenStream.keyBy(elem => /* select key */))
+    .between(Time.milliseconds(-2), Time.milliseconds(1))
+    .process(new ProcessJoinFunction[Integer, Integer, String] {
+        override def processElement(left: Integer, right: Integer, ctx: ProcessJoinFunction[Integer, Integer, String]#Context, out: Collector[String]): Unit = {
+         out.collect(left + "," + right); 
+        }
+      });
+    });
+{% endhighlight %}
+
+</div>
+</div>
diff --git a/docs/fig/interval-join.svg b/docs/fig/interval-join.svg
new file mode 100644
index 0000000..25b5932
--- /dev/null
+++ b/docs/fig/interval-join.svg
@@ -0,0 +1,147 @@
+<?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.
+-->
+
+<svg width="1000px" height="350px" viewBox="0 0 1000 350" version="1.1" xmlns="http://www.w3.org/2000/svg" xmlns:xlink="http://www.w3.org/1999/xlink" style="background: #FFFFFF;">
+    <!-- Generator: Sketch 48.2 (47327) - http://www.bohemiancoding.com/sketch -->
+    <title>Interval Join (Selected)</title>
+    <desc>Created with Sketch.</desc>
+    <defs></defs>
+    <g id="Page-1" stroke="none" stroke-width="1" fill="none" fill-rule="evenodd">
+        <g id="Interval-Join-(Selected)">
+            <text id="time" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                <tspan x="882" y="262">time</tspan>
+            </text>
+            <g id="Group-3" transform="translate(116.000000, 229.000000)" stroke="#979797" stroke-linecap="square">
+                <path d="M0.5,5.5 L781.5,5.5" id="Line" stroke-width="2"></path>
+                <path id="Line-decoration-1" d="M781.5,5.5 L770.7,2.5 L770.7,8.5 L781.5,5.5 Z" stroke-width="2"></path>
+                <path d="M132.5,0.5 L132.5,10.5" id="Line-3"></path>
+                <path d="M31.5,0.5 L31.5,10.5" id="Line-3-Copy"></path>
+                <path d="M234.5,0.5 L234.5,10.5" id="Line-3-Copy"></path>
+                <path d="M334.5,0.5 L334.5,10.5" id="Line-3-Copy"></path>
+                <path d="M435.5,0.5 L435.5,10.5" id="Line-3-Copy"></path>
+                <path d="M536.5,0.5 L536.5,10.5" id="Line-3-Copy"></path>
+                <path d="M638.5,0.5 L638.5,10.5" id="Line-3-Copy"></path>
+                <path d="M739.5,0.5 L739.5,10.5" id="Line-3-Copy"></path>
+            </g>
+            <g id="Group-3-Copy" transform="translate(116.000000, 93.000000)" stroke="#979797" stroke-linecap="square">
+                <path d="M0.5,5.5 L781.5,5.5" id="Line" stroke-width="2"></path>
+                <path id="Line-decoration-1" d="M781.5,5.5 L770.7,2.5 L770.7,8.5 L781.5,5.5 Z" stroke-width="2"></path>
+                <path d="M132.5,0.5 L132.5,10.5" id="Line-3"></path>
+                <path d="M31.5,0.5 L31.5,10.5" id="Line-3-Copy"></path>
+                <path d="M234.5,0.5 L234.5,10.5" id="Line-3-Copy"></path>
+                <path d="M334.5,0.5 L334.5,10.5" id="Line-3-Copy"></path>
+                <path d="M435.5,0.5 L435.5,10.5" id="Line-3-Copy"></path>
+                <path d="M536.5,0.5 L536.5,10.5" id="Line-3-Copy"></path>
+                <path d="M638.5,0.5 L638.5,10.5" id="Line-3-Copy"></path>
+                <path d="M739.5,0.5 L739.5,10.5" id="Line-3-Copy"></path>
+            </g>
+            <g id="StreamRecord-Copy-7" transform="translate(740.000000, 85.000000)">
+                <circle id="Oval" stroke="#979797" fill="#7ED321" cx="12.5" cy="12.5" r="12.5"></circle>
+                <text id="6" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                    <tspan x="8" y="18">6</tspan>
+                </text>
+            </g>
+            <g id="StreamRecord-Copy-7" transform="translate(842.000000, 86.000000)">
+                <circle id="Oval" stroke="#979797" fill="#7ED321" cx="12.5" cy="12.5" r="12.5"></circle>
+                <text id="7" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                    <tspan x="8" y="18">7</tspan>
+                </text>
+            </g>
+            <g id="StreamRecord-Copy" transform="translate(435.000000, 222.000000)">
+                <circle id="Oval" stroke="#979797" fill="#F5A623" cx="12.5" cy="12.5" r="12.5"></circle>
+                <text id="3" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                    <tspan x="8" y="18">3</tspan>
+                </text>
+            </g>
+            <g id="StreamRecord-Copy-2" transform="translate(540.000000, 222.000000)">
+                <circle id="Oval" stroke="#979797" fill="#F5A623" cx="12.5" cy="12.5" r="12.5"></circle>
+                <text id="4" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                    <tspan x="8" y="18">4</tspan>
+                </text>
+            </g>
+            <g id="StreamRecord-Copy-3" transform="translate(640.000000, 222.000000)">
+                <circle id="Oval" stroke="#979797" fill="#F5A623" cx="12.5" cy="12.5" r="12.5"></circle>
+                <text id="5" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                    <tspan x="8" y="18">5</tspan>
+                </text>
+            </g>
+            <g id="StreamRecord-Copy-5" transform="translate(842.000000, 222.000000)">
+                <circle id="Oval" stroke="#979797" fill="#F5A623" cx="12.5" cy="12.5" r="12.5"></circle>
+                <text id="7" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                    <tspan x="8" y="18">7</tspan>
+                </text>
+            </g>
+            <g id="StreamRecord-Copy-6" transform="translate(136.000000, 222.000000)">
+                <circle id="Oval" stroke="#979797" fill="#F5A623" cx="12.5" cy="12.5" r="12.5"></circle>
+                <text id="0" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                    <tspan x="8" y="18">0</tspan>
+                </text>
+            </g>
+            <text id="0,0-0,1" font-family="AndaleMono, Andale Mono" font-size="20" font-weight="normal" fill="#9B9B9B">
+                <tspan x="131" y="276">0,0</tspan>
+                <tspan x="131" y="298">0,1</tspan>
+            </text>
+            <text id="2,0-2,1" font-family="AndaleMono, Andale Mono" font-size="20" font-weight="normal" fill="#34323B">
+                <tspan x="334" y="276">2,0</tspan>
+                <tspan x="334" y="298">2,1</tspan>
+            </text>
+            <text id="3,1" font-family="AndaleMono, Andale Mono" font-size="20" font-weight="normal" fill="#9B9B9B">
+                <tspan x="429" y="276">3,1</tspan>
+            </text>
+            <text id="5,6" font-family="AndaleMono, Andale Mono" font-size="20" font-weight="normal" fill="#9B9B9B">
+                <tspan x="634" y="276">5,6</tspan>
+            </text>
+            <text id="7,6-7,7" font-family="AndaleMono, Andale Mono" font-size="20" font-weight="normal" fill="#9B9B9B">
+                <tspan x="836" y="276">7,6</tspan>
+                <tspan x="836" y="298">7,7</tspan>
+            </text>
+            <polygon id="Path-3" fill-opacity="0.100000001" fill="#9B9B9B" points="-51 99.8380503 151.566279 235.968947 251.677716 99.8380503"></polygon>
+            <polygon id="Path-3" fill-opacity="0.100000001" fill="#9B9B9B" points="246 99.8380503 448.566279 235.968947 548.677716 99.8380503"></polygon>
+            <polygon id="Path-3" fill-opacity="0.100000001" fill="#9B9B9B" points="350 99.8380503 552.566279 235.968947 652.677716 99.8380503"></polygon>
+            <polygon id="Path-3" fill-opacity="0.100000001" fill="#9B9B9B" points="448 99.8380503 650.566279 235.968947 750.677716 99.8380503"></polygon>
+            <polygon id="Path-3" fill-opacity="0.100000001" fill="#9B9B9B" points="651 99.8380503 853.566279 235.968947 953.677716 99.8380503"></polygon>
+            <polygon id="Path-3" fill-opacity="0.5" fill="#F8E71C" points="148.284508 99.8380503 350.850787 235.968947 450.962224 99.8380503"></polygon>
+            <g id="StreamRecord-Copy-7" transform="translate(235.000000, 85.000000)">
+                <circle id="Oval" stroke="#979797" fill="#7ED321" cx="12.5" cy="12.5" r="12.5"></circle>
+                <text id="1" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                    <tspan x="8" y="18">1</tspan>
+                </text>
+            </g>
+            <text id="lower-bound" transform="translate(246.800863, 159.474217) rotate(34.000000) translate(-246.800863, -159.474217) " font-family="AndaleMono, Andale Mono" font-size="14" font-weight="normal" fill="#34323B">
+                <tspan x="200.300863" y="164.474217">lower bound</tspan>
+            </text>
+            <text id="upper-bound" transform="translate(398.500000, 156.000000) rotate(-53.000000) translate(-398.500000, -156.000000) " font-family="AndaleMono, Andale Mono" font-size="14" font-weight="normal" fill="#34323B">
+                <tspan x="352" y="161">upper bound</tspan>
+            </text>
+            <g id="StreamRecord" transform="translate(338.000000, 222.000000)">
+                <circle id="Oval" stroke="#979797" fill="#F5A623" cx="12.5" cy="12.5" r="12.5"></circle>
+                <text id="2" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                    <tspan x="8" y="18">2</tspan>
+                </text>
+            </g>
+            <g id="StreamRecord-Copy-6" transform="translate(135.000000, 86.000000)">
+                <circle id="Oval" stroke="#979797" fill="#7ED321" cx="12.5" cy="12.5" r="12.5"></circle>
+                <text id="0" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                    <tspan x="8" y="18">0</tspan>
+                </text>
+            </g>
+        </g>
+    </g>
+</svg>
diff --git a/docs/fig/session-window-join.svg b/docs/fig/session-window-join.svg
new file mode 100644
index 0000000..8fb3293
--- /dev/null
+++ b/docs/fig/session-window-join.svg
@@ -0,0 +1,125 @@
+<?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.
+-->
+
+<svg width="1000px" height="350px" viewBox="0 0 1000 350" version="1.1" xmlns="http://www.w3.org/2000/svg" xmlns:xlink="http://www.w3.org/1999/xlink" style="background: #FFFFFF;">
+    <!-- Generator: Sketch 48.2 (47327) - http://www.bohemiancoding.com/sketch -->
+    <title>Session Window Join</title>
+    <desc>Created with Sketch.</desc>
+    <defs>
+        <rect id="path-1" x="386" y="52" width="37" height="113"></rect>
+        <rect id="path-2" x="701" y="52" width="37" height="113"></rect>
+    </defs>
+    <g id="Page-1" stroke="none" stroke-width="1" fill="none" fill-rule="evenodd">
+        <g id="Session-Window-Join">
+            <path d="M58.5,196.5 L941.5,196.5" id="Line" stroke="#979797" stroke-linecap="square"></path>
+            <path id="Line-decoration-1" d="M941.5,196.5 L930.7,193.5 L930.7,199.5 L941.5,196.5 Z" stroke="#979797" stroke-linecap="square"></path>
+            <text id="time" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                <tspan x="926" y="227">time</tspan>
+            </text>
+            <g id="StreamRecord" transform="translate(307.000000, 138.000000)">
+                <circle id="Oval" stroke="#979797" fill="#F5A623" cx="12.5" cy="12.5" r="12.5"></circle>
+                <text id="2" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                    <tspan x="8" y="18">2</tspan>
+                </text>
+            </g>
+            <g id="StreamRecord-Copy-3" transform="translate(547.000000, 138.000000)">
+                <circle id="Oval" stroke="#979797" fill="#F5A623" cx="12.5" cy="12.5" r="12.5"></circle>
+                <text id="5" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                    <tspan x="8" y="18">5</tspan>
+                </text>
+            </g>
+            <g id="StreamRecord-Copy-4" transform="translate(627.000000, 138.000000)">
+                <circle id="Oval" stroke="#979797" fill="#F5A623" cx="12.5" cy="12.5" r="12.5"></circle>
+                <text id="6" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                    <tspan x="8" y="18">6</tspan>
+                </text>
+            </g>
+            <g id="StreamRecord-Copy-4" transform="translate(787.000000, 138.000000)">
+                <circle id="Oval" stroke="#979797" fill="#F5A623" cx="12.5" cy="12.5" r="12.5"></circle>
+                <text id="6" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                    <tspan x="8" y="18">6</tspan>
+                </text>
+            </g>
+            <g id="StreamRecord-Copy-10" transform="translate(787.000000, 138.000000)">
+                <circle id="Oval" stroke="#979797" fill="#F5A623" cx="12.5" cy="12.5" r="12.5"></circle>
+                <text id="8" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                    <tspan x="8" y="18">8</tspan>
+                </text>
+            </g>
+            <g id="StreamRecord-Copy-9" transform="translate(867.000000, 138.000000)">
+                <circle id="Oval" stroke="#979797" fill="#F5A623" cx="12.5" cy="12.5" r="12.5"></circle>
+                <text id="9" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                    <tspan x="8" y="18">9</tspan>
+                </text>
+            </g>
+            <g id="StreamRecord-Copy-7" transform="translate(227.000000, 138.000000)">
+                <circle id="Oval" stroke="#979797" fill="#F5A623" cx="12.5" cy="12.5" r="12.5"></circle>
+                <text id="1" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                    <tspan x="8" y="18">1</tspan>
+                </text>
+            </g>
+            <g id="StreamRecord-Copy" transform="translate(547.000000, 68.000000)">
+                <circle id="Oval" stroke="#979797" fill="#7ED321" cx="12.5" cy="12.5" r="12.5"></circle>
+                <text id="5" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                    <tspan x="8" y="18">5</tspan>
+                </text>
+            </g>
+            <g id="StreamRecord-Copy-2" transform="translate(467.000000, 68.000000)">
+                <circle id="Oval" stroke="#979797" fill="#7ED321" cx="12.5" cy="12.5" r="12.5"></circle>
+                <text id="4" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                    <tspan x="8" y="18">4</tspan>
+                </text>
+            </g>
+            <g id="StreamRecord-Copy-6" transform="translate(148.000000, 68.000000)">
+                <circle id="Oval" stroke="#979797" fill="#7ED321" cx="12.5" cy="12.5" r="12.5"></circle>
+                <text id="0" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                    <tspan x="8" y="18">0</tspan>
+                </text>
+            </g>
+            <polyline id="Path-2" stroke="#979797" points="133 163 133 50 361 50 361 163"></polyline>
+            <polyline id="Path-2" stroke="#979797" points="449 163 449 50 677 50 677 163"></polyline>
+            <polyline id="Path-2" stroke="#979797" points="758 163 758 50 926 50 926 163"></polyline>
+            <g id="Rectangle-3">
+                <use fill="#D8D8D8" fill-rule="evenodd" xlink:href="#path-1"></use>
+                <rect stroke="#979797" stroke-width="1" x="386.5" y="52.5" width="36" height="112"></rect>
+            </g>
+            <g id="Rectangle-3">
+                <use fill="#D8D8D8" fill-rule="evenodd" xlink:href="#path-2"></use>
+                <rect stroke="#979797" stroke-width="1" x="701.5" y="52.5" width="36" height="112"></rect>
+            </g>
+            <text id="GAP" transform="translate(406.000000, 108.500000) rotate(-90.000000) translate(-406.000000, -108.500000) " font-family="AndaleMono, Andale Mono" font-size="18" font-weight="normal" fill="#000000">
+                <tspan x="389.5" y="114.5">GAP</tspan>
+            </text>
+            <text id="GAP" transform="translate(719.000000, 108.500000) rotate(-90.000000) translate(-719.000000, -108.500000) " font-family="AndaleMono, Andale Mono" font-size="18" font-weight="normal" fill="#000000">
+                <tspan x="702.5" y="114.5">GAP</tspan>
+            </text>
+            <text id="1,0-2,0" font-family="AndaleMono, Andale Mono" font-size="20" font-weight="normal" fill="#34323B">
+                <tspan x="227" y="234">1,0</tspan>
+                <tspan x="227" y="256">2,0</tspan>
+            </text>
+            <text id="5,4-5,5-6,4-6,5" font-family="AndaleMono, Andale Mono" font-size="20" font-weight="normal" fill="#34323B">
+                <tspan x="545" y="230">5,4</tspan>
+                <tspan x="545" y="252">5,5</tspan>
+                <tspan x="545" y="274">6,4</tspan>
+                <tspan x="545" y="296">6,5</tspan>
+            </text>
+        </g>
+    </g>
+</svg>
diff --git a/docs/fig/sliding-window-join.svg b/docs/fig/sliding-window-join.svg
new file mode 100644
index 0000000..f544e0d
--- /dev/null
+++ b/docs/fig/sliding-window-join.svg
@@ -0,0 +1,113 @@
+<?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.
+-->
+
+<svg width="1000px" height="350px" viewBox="0 0 1000 350" version="1.1" xmlns="http://www.w3.org/2000/svg" xmlns:xlink="http://www.w3.org/1999/xlink" style="background: #FFFFFF;">
+    <!-- Generator: Sketch 48.2 (47327) - http://www.bohemiancoding.com/sketch -->
+    <title>Sliding Window Join</title>
+    <desc>Created with Sketch.</desc>
+    <defs></defs>
+    <g id="Page-1" stroke="none" stroke-width="1" fill="none" fill-rule="evenodd">
+        <g id="Sliding-Window-Join">
+            <path d="M101.5,215.5 L882.5,215.5" id="Line" stroke="#979797" stroke-linecap="square"></path>
+            <path id="Line-decoration-1" d="M882.5,215.5 L871.7,212.5 L871.7,218.5 L882.5,215.5 Z" stroke="#979797" stroke-linecap="square"></path>
+            <text id="time" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                <tspan x="867" y="242">time</tspan>
+            </text>
+            <path d="M544,53 L544,163.8 L650.44697,163.8 L650.44697,53 L544,53 Z" id="Rectangle-Copy" stroke="#BD10E0" stroke-width="2"></path>
+            <path d="M490,46 L490,170.9 L596.44697,170.9 L596.44697,46 L490,46 Z" id="Rectangle-Copy-2" stroke="#3F8FAE" stroke-width="2"></path>
+            <path d="M381,46 L381,170.9 L487.44697,170.9 L487.44697,46 L381,46 Z" id="Rectangle-Copy-3" stroke="#50E3C2" stroke-width="2"></path>
+            <path d="M327,53 L327,163.8 L433.44697,163.8 L433.44697,53 L327,53 Z" id="Rectangle-Copy-4" stroke="#F5A623" stroke-width="2"></path>
+            <path d="M435,39 L435,178 L541.44697,178 L541.44697,39 L435,39 Z" id="Rectangle-Copy-5" stroke="#417505" stroke-width="2"></path>
+            <path d="M273.98992,60 L273.010183,156.7 L379.44697,156.7 L379.44697,60 L273.98992,60 Z" id="Rectangle" stroke="#D0021B" stroke-width="2"></path>
+            <rect id="Rectangle-2" fill="#FFFFFF" x="257" y="70" width="409" height="78.96"></rect>
+            <g id="StreamRecord" transform="translate(463.000000, 124.000000)">
+                <ellipse id="Oval" stroke="#979797" fill="#F5A623" cx="9.68276515" cy="9.5" rx="9.68276515" ry="9.5"></ellipse>
+                <text id="2" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                    <tspan x="6.1969697" y="15">2</tspan>
+                </text>
+            </g>
+            <g id="StreamRecord-Copy" transform="translate(510.000000, 124.000000)">
+                <ellipse id="Oval" stroke="#979797" fill="#F5A623" cx="9.68276515" cy="9.5" rx="9.68276515" ry="9.5"></ellipse>
+                <text id="3" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                    <tspan x="6.1969697" y="15">3</tspan>
+                </text>
+            </g>
+            <g id="StreamRecord-Copy-2" transform="translate(564.000000, 124.000000)">
+                <ellipse id="Oval" stroke="#979797" fill="#F5A623" cx="9.68276515" cy="9.5" rx="9.68276515" ry="9.5"></ellipse>
+                <text id="4" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                    <tspan x="6.1969697" y="15">4</tspan>
+                </text>
+            </g>
+            <g id="StreamRecord-Copy-7" transform="translate(401.000000, 124.000000)">
+                <ellipse id="Oval" stroke="#979797" fill="#F5A623" cx="9.68276515" cy="9.5" rx="9.68276515" ry="9.5"></ellipse>
+                <text id="1" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                    <tspan x="6.1969697" y="15">1</tspan>
+                </text>
+            </g>
+            <g id="StreamRecord-Copy" transform="translate(507.000000, 75.000000)">
+                <ellipse id="Oval" stroke="#979797" fill="#7ED321" cx="9.68276515" cy="9.5" rx="9.68276515" ry="9.5"></ellipse>
+                <text id="3" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                    <tspan x="6.1969697" y="15">3</tspan>
+                </text>
+            </g>
+            <g id="StreamRecord-Copy-2" transform="translate(563.000000, 75.000000)">
+                <ellipse id="Oval" stroke="#979797" fill="#7ED321" cx="9.68276515" cy="9.5" rx="9.68276515" ry="9.5"></ellipse>
+                <text id="4" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                    <tspan x="6.1969697" y="15">4</tspan>
+                </text>
+            </g>
+            <g id="StreamRecord-Copy-6" transform="translate(340.000000, 124.000000)">
+                <ellipse id="Oval" stroke="#979797" fill="#F5A623" cx="9.68276515" cy="9.5" rx="9.68276515" ry="9.5"></ellipse>
+                <text id="0" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                    <tspan x="6.1969697" y="15">0</tspan>
+                </text>
+            </g>
+            <g id="StreamRecord-Copy-6" transform="translate(340.000000, 75.000000)">
+                <ellipse id="Oval" stroke="#979797" fill="#7ED321" cx="9.68276515" cy="9.5" rx="9.68276515" ry="9.5"></ellipse>
+                <text id="0" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                    <tspan x="6.1969697" y="15">0</tspan>
+                </text>
+            </g>
+            <text id="0,0" font-family="AndaleMono, Andale Mono" font-size="20" font-weight="normal" fill="#D0021B">
+                <tspan x="307" y="242">0,0</tspan>
+            </text>
+            <text id="0,0-1,0" font-family="AndaleMono, Andale Mono" font-size="20" font-weight="normal" fill="#F5A623">
+                <tspan x="362" y="242">0,0</tspan>
+                <tspan x="362" y="264">1,0</tspan>
+            </text>
+            <text id="2,3-3,3" font-family="AndaleMono, Andale Mono" font-size="20" font-weight="normal" fill="#417505">
+                <tspan x="470" y="242">2,3</tspan>
+                <tspan x="470" y="264">3,3</tspan>
+            </text>
+            <text id="3,3-3,4-4,3-4,4" font-family="AndaleMono, Andale Mono" font-size="20" font-weight="normal" fill="#3F8FAE">
+                <tspan x="524" y="242">3,3</tspan>
+                <tspan x="524" y="264">3,4</tspan>
+                <tspan x="524" y="286">4,3</tspan>
+                <tspan x="524" y="308">4,4</tspan>
+            </text>
+            <text id="4,4" font-family="AndaleMono, Andale Mono" font-size="20" font-weight="normal" fill="#9013FE">
+                <tspan x="579" y="242">4,4</tspan>
+            </text>
+            <text id="-" font-family="AndaleMono, Andale Mono" font-size="20" font-weight="normal" fill="#50E3C2">
+                <tspan x="428" y="242">-</tspan>
+            </text>
+        </g>
+    </g>
+</svg>
diff --git a/docs/fig/tumbling-window-join.svg b/docs/fig/tumbling-window-join.svg
new file mode 100644
index 0000000..9efb58c
--- /dev/null
+++ b/docs/fig/tumbling-window-join.svg
@@ -0,0 +1,131 @@
+<?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.
+-->
+
+<svg width="1000px" height="350px" viewBox="0 0 1000 350" version="1.1" xmlns="http://www.w3.org/2000/svg" xmlns:xlink="http://www.w3.org/1999/xlink" style="background: #FFFFFF;">
+    <!-- Generator: Sketch 48.2 (47327) - http://www.bohemiancoding.com/sketch -->
+    <title>Tumbling Window Join</title>
+    <desc>Created with Sketch.</desc>
+    <defs></defs>
+    <g id="Page-1" stroke="none" stroke-width="1" fill="none" fill-rule="evenodd">
+        <g id="Tumbling-Window-Join">
+            <text id="time" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                <tspan x="859" y="210">time</tspan>
+            </text>
+            <g id="Group-3" transform="translate(199.000000, 48.000000)">
+                <g id="Group-2" transform="translate(17.000000, 88.000000)">
+                    <g id="StreamRecord" transform="translate(159.714286, 0.000000)">
+                        <circle id="Oval" stroke="#979797" fill="#F5A623" cx="12.5" cy="12.5" r="12.5"></circle>
+                        <text id="2" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                            <tspan x="8" y="18">2</tspan>
+                        </text>
+                    </g>
+                    <g id="StreamRecord-Copy" transform="translate(239.571429, 0.000000)">
+                        <circle id="Oval" stroke="#979797" fill="#F5A623" cx="12.5" cy="12.5" r="12.5"></circle>
+                        <text id="3" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                            <tspan x="8" y="18">3</tspan>
+                        </text>
+                    </g>
+                    <g id="StreamRecord-Copy-2" transform="translate(319.428571, 0.000000)">
+                        <circle id="Oval" stroke="#979797" fill="#F5A623" cx="12.5" cy="12.5" r="12.5"></circle>
+                        <text id="4" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                            <tspan x="8" y="18">4</tspan>
+                        </text>
+                    </g>
+                    <g id="StreamRecord-Copy-3" transform="translate(399.285714, 0.000000)">
+                        <circle id="Oval" stroke="#979797" fill="#F5A623" cx="12.5" cy="12.5" r="12.5"></circle>
+                        <text id="5" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                            <tspan x="8" y="18">5</tspan>
+                        </text>
+                    </g>
+                    <g id="StreamRecord-Copy-4" transform="translate(479.142857, 0.000000)">
+                        <circle id="Oval" stroke="#979797" fill="#F5A623" cx="12.5" cy="12.5" r="12.5"></circle>
+                        <text id="6" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                            <tspan x="8" y="18">6</tspan>
+                        </text>
+                    </g>
+                    <g id="StreamRecord-Copy-5" transform="translate(559.000000, 0.000000)">
+                        <circle id="Oval" stroke="#979797" fill="#F5A623" cx="12.5" cy="12.5" r="12.5"></circle>
+                        <text id="7" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                            <tspan x="8" y="18">7</tspan>
+                        </text>
+                    </g>
+                    <g id="StreamRecord-Copy-6">
+                        <circle id="Oval" stroke="#979797" fill="#F5A623" cx="12.5" cy="12.5" r="12.5"></circle>
+                        <text id="0" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                            <tspan x="8" y="18">0</tspan>
+                        </text>
+                    </g>
+                    <g id="StreamRecord-Copy-7" transform="translate(79.857143, 0.000000)">
+                        <circle id="Oval" stroke="#979797" fill="#F5A623" cx="12.5" cy="12.5" r="12.5"></circle>
+                        <text id="1" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                            <tspan x="8" y="18">1</tspan>
+                        </text>
+                    </g>
+                </g>
+                <g id="Group" transform="translate(17.000000, 18.000000)">
+                    <g id="StreamRecord-Copy" transform="translate(239.571429, 0.000000)">
+                        <circle id="Oval" stroke="#979797" fill="#7ED321" cx="12.5" cy="12.5" r="12.5"></circle>
+                        <text id="3" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                            <tspan x="8" y="18">3</tspan>
+                        </text>
+                    </g>
+                    <g id="StreamRecord-Copy-2" transform="translate(319.428571, 0.000000)">
+                        <circle id="Oval" stroke="#979797" fill="#7ED321" cx="12.5" cy="12.5" r="12.5"></circle>
+                        <text id="4" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                            <tspan x="8" y="18">4</tspan>
+                        </text>
+                    </g>
+                    <g id="StreamRecord-Copy-6">
+                        <circle id="Oval" stroke="#979797" fill="#7ED321" cx="12.5" cy="12.5" r="12.5"></circle>
+                        <text id="0" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                            <tspan x="8" y="18">0</tspan>
+                        </text>
+                    </g>
+                    <g id="StreamRecord-Copy-7" transform="translate(79.857143, 0.000000)">
+                        <circle id="Oval" stroke="#979797" fill="#7ED321" cx="12.5" cy="12.5" r="12.5"></circle>
+                        <text id="1" font-family="Roboto-Regular, Roboto" font-size="16" font-weight="normal" fill="#34323B">
+                            <tspan x="8" y="18">1</tspan>
+                        </text>
+                    </g>
+                </g>
+                <polyline id="Path-2" stroke="#979797" points="0 113 0 0 136 0 136 113"></polyline>
+                <polyline id="Path-2-Copy" stroke="#979797" points="161 113 161 0 297 0 297 113"></polyline>
+                <polyline id="Path-2-Copy-2" stroke="#979797" points="325 113 325 0 461 0 461 113"></polyline>
+                <polyline id="Path-2-Copy-3" stroke="#979797" points="483 113 483 0 619 0 619 113"></polyline>
+            </g>
+            <text id="0,0-0,1-1,0-1,1" font-family="AndaleMono, Andale Mono" font-size="20" font-weight="normal" fill="#34323B">
+                <tspan x="238" y="232">0,0</tspan>
+                <tspan x="238" y="254">0,1</tspan>
+                <tspan x="238" y="276">1,0</tspan>
+                <tspan x="238" y="298">1,1</tspan>
+            </text>
+            <text id="2,3-3,3" font-family="AndaleMono, Andale Mono" font-size="20" font-weight="normal" fill="#34323B">
+                <tspan x="402" y="232">2,3</tspan>
+                <tspan x="402" y="254">3,3</tspan>
+            </text>
+            <text id="4,4-5,4" font-family="AndaleMono, Andale Mono" font-size="20" font-weight="normal" fill="#34323B">
+                <tspan x="559" y="232">4,4</tspan>
+                <tspan x="559" y="254">5,4</tspan>
+            </text>
+            <path d="M93.5,180.5 L874.5,180.5" id="Line" stroke="#979797" stroke-linecap="square"></path>
+            <path id="Line-decoration-1" d="M874.5,180.5 L863.7,177.5 L863.7,183.5 L874.5,180.5 Z" stroke="#979797" stroke-linecap="square"></path>
+        </g>
+    </g>
+</svg>