You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by florianschmidt1994 <gi...@git.apache.org> on 2018/07/24 16:23:51 UTC

[GitHub] flink pull request #6407: [FLINK-8478][docs] Add documentation page for diff...

GitHub user florianschmidt1994 opened a pull request:

    https://github.com/apache/flink/pull/6407

    [FLINK-8478][docs] Add documentation page for different datastream joins

    ## What is the purpose of the change
    
    Add a documentation page under Application Development / Streaming / Joining that describes
    1. The different types of window joins in the DataStream API
    2. The newly introduced interval join in the DataStream API
    
    ## Brief change log
      - Added a new docs page
      - Added images to describe common scenarios
    
    ## Verifying this change
    
    Built the documentation with `./build_docs.sh` and it looks like expected
    
    ## Does this pull request potentially affect one of the following parts:
    
      - Dependencies (does it add or upgrade a dependency): no
      - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: no
      - The serializers: no
      - The runtime per-record code paths (performance sensitive): no
      - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: no
      - The S3 file system connector: no
    
    ## Documentation
    
      - Does this pull request introduce a new feature? no
      - If yes, how is the feature documented? not applicable
    
    For those that just want to have a "quick look" I attached a screenshot
    
    ![](https://i.imgur.com/c78WuD7.jpg)


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/florianschmidt1994/flink flink-8478-add-docs-for-joins

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/6407.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #6407
    
----
commit d2800ff33af179dd32876020d27994b9dc6579aa
Author: Florian Schmidt <fl...@...>
Date:   2018-07-24T16:14:50Z

    [FLINK-8478] Add documentation page for different datastream joins

----


---

[GitHub] flink issue #6407: [FLINK-8478][docs] Add documentation page for different d...

Posted by florianschmidt1994 <gi...@git.apache.org>.
Github user florianschmidt1994 commented on the issue:

    https://github.com/apache/flink/pull/6407
  
    Maybe @twalthr you can have a look at this?


---

[GitHub] flink pull request #6407: [FLINK-9877][docs] Add documentation page for diff...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/6407#discussion_r205097411
  
    --- Diff: docs/dev/stream/operators/joining.md ---
    @@ -0,0 +1,286 @@
    +---
    +title: "Joining"
    +nav-id: streaming_joins
    +nav-show_overview: true
    +nav-parent_id: streaming
    +nav-pos: 10
    +---
    +<!--
    +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 will join 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 a union of both streams. This is especially important for session window joins, which will be demonstrated below.
    +
    +The joined elements are then passed to a user-defined `JoinFunction` or `FlatJoinFunction` where the user can perform transformations on the joined elements.
    +
    +The general usage always looks like the followning:
    +
    +```java
    +stream.join(otherStream)
    +    .where(<KeySelector>)
    +    .equalTo(<KeySelector>)
    +    .window(<WindowAssigner>)
    +    .apply(<JoinFunction>)
    +```
    +
    +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 nine as their timestamp.
    +
    +In the following section we are going to give an overview over how different kinds of windows can be used for a window join and what the results of those joins would look like using examplary scenarios.
    +
    +## Tumbling Window
    +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 the user-defined function. 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!
    +
    +### Example
    +<img src="{{ site.baseurl }}/fig/tumbling-window-join.svg" class="center" style="width: 80%;" />
    +
    +In our example we are defining 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 user-defined function. You can also see how in the tumbling window `[6,7]` nothing is emitted because no elements from the green stream exist to be joined with the orange elements ⑥ and ⑦.
    +
    +<div class="codetabs" markdown="1">
    +<div data-lang="java" markdown="1">
    +
    +```java
    --- End diff --
    
    please use { % highlight java % } syntax


---