spark git commit: [SPARK-23064][DOCS][SS] Added documentation for stream-stream joins

2018-01-17 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 9783aea2c -> 050c1e24e


[SPARK-23064][DOCS][SS] Added documentation for stream-stream joins

## What changes were proposed in this pull request?
Added documentation for stream-stream joins

![image](https://user-images.githubusercontent.com/663212/35018744-e999895a-fad7-11e7-9d6a-8c7a73e6eb9c.png)

![image](https://user-images.githubusercontent.com/663212/35018775-157eb464-fad8-11e7-879e-47a2fcbd8690.png)

![image](https://user-images.githubusercontent.com/663212/35018784-27791a24-fad8-11e7-98f4-7ff246f62a74.png)

![image](https://user-images.githubusercontent.com/663212/35018791-36a80334-fad8-11e7-9791-f85efa7c6ba2.png)

## How was this patch tested?

N/a

Author: Tathagata Das 

Closes #20255 from tdas/join-docs.

(cherry picked from commit 1002bd6b23ff78a010ca259ea76988ef4c478c6e)
Signed-off-by: Shixiong Zhu 


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

Branch: refs/heads/branch-2.3
Commit: 050c1e24e506ff224bcf4e3e458e57fbd216765c
Parents: 9783aea
Author: Tathagata Das 
Authored: Wed Jan 17 16:41:43 2018 -0800
Committer: Shixiong Zhu 
Committed: Wed Jan 17 16:41:49 2018 -0800

--
 docs/structured-streaming-programming-guide.md | 338 +++-
 1 file changed, 326 insertions(+), 12 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/050c1e24/docs/structured-streaming-programming-guide.md
--
diff --git a/docs/structured-streaming-programming-guide.md 
b/docs/structured-streaming-programming-guide.md
index de13e28..1779a42 100644
--- a/docs/structured-streaming-programming-guide.md
+++ b/docs/structured-streaming-programming-guide.md
@@ -1051,7 +1051,19 @@ output mode.
 
 
 ### Join Operations
-Streaming DataFrames can be joined with static DataFrames to create new 
streaming DataFrames. Here are a few examples.
+Structured Streaming supports joining a streaming Dataset/DataFrame with a 
static Dataset/DataFrame
+as well as another streaming Dataset/DataFrame. The result of the streaming 
join is generated
+incrementally, similar to the results of streaming aggregations in the 
previous section. In this
+section we will explore what type of joins (i.e. inner, outer, etc.) are 
supported in the above
+cases. Note that in all the supported join types, the result of the join with 
a streaming
+Dataset/DataFrame will be the exactly the same as if it was with a static 
Dataset/DataFrame
+containing the same data in the stream.
+
+
+ Stream-static joins
+
+Since the introduction in Spark 2.0, Structured Streaming has supported joins 
(inner join and some
+type of outer joins) between a streaming and a static DataFrame/Dataset. Here 
is a simple example.
 
 
 
@@ -1089,6 +1101,300 @@ streamingDf.join(staticDf, "type", "right_join")  # 
right outer join with a stat
 
 
 
+Note that stream-static joins are not stateful, so no state management is 
necessary.
+However, a few types of stream-static outer joins are not yet supported.
+These are listed at the [end of this Join 
section](#support-matrix-for-joins-in-streaming-queries).
+
+ Stream-stream Joins
+In Spark 2.3, we have added support for stream-stream joins, that is, you can 
join two streaming
+Datasets/DataFrames. The challenge of generating join results between two data 
streams is that,
+at any point of time, the view of the dataset is incomplete for both sides of 
the join making
+it much harder to find matches between inputs. Any row received from one input 
stream can match
+with any future, yet-to-be-received row from the other input stream. Hence, 
for both the input
+streams, we buffer past input as streaming state, so that we can match every 
future input with
+past input and accordingly generate joined results. Furthermore, similar to 
streaming aggregations,
+we automatically handle late, out-of-order data and can limit the state using 
watermarks.
+Let’s discuss the different types of supported stream-stream joins and how 
to use them.
+
+# Inner Joins with optional Watermarking
+Inner joins on any kind of columns along with any kind of join conditions are 
supported.
+However, as the stream runs, the size of streaming state will keep growing 
indefinitely as
+*all* past input must be saved as the any new input can match with any input 
from the past.
+To avoid unbounded state, you have to define additional join conditions such 
that indefinitely
+old inputs cannot match with future inputs and therefore can be cleared from 
the state.
+In other words, you will have to do the following additional steps in the join.
+
+1. Define waterm

spark git commit: [SPARK-23064][DOCS][SS] Added documentation for stream-stream joins

2018-01-17 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/master bac0d661a -> 1002bd6b2


[SPARK-23064][DOCS][SS] Added documentation for stream-stream joins

## What changes were proposed in this pull request?
Added documentation for stream-stream joins

![image](https://user-images.githubusercontent.com/663212/35018744-e999895a-fad7-11e7-9d6a-8c7a73e6eb9c.png)

![image](https://user-images.githubusercontent.com/663212/35018775-157eb464-fad8-11e7-879e-47a2fcbd8690.png)

![image](https://user-images.githubusercontent.com/663212/35018784-27791a24-fad8-11e7-98f4-7ff246f62a74.png)

![image](https://user-images.githubusercontent.com/663212/35018791-36a80334-fad8-11e7-9791-f85efa7c6ba2.png)

## How was this patch tested?

N/a

Author: Tathagata Das 

Closes #20255 from tdas/join-docs.


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

Branch: refs/heads/master
Commit: 1002bd6b23ff78a010ca259ea76988ef4c478c6e
Parents: bac0d66
Author: Tathagata Das 
Authored: Wed Jan 17 16:41:43 2018 -0800
Committer: Shixiong Zhu 
Committed: Wed Jan 17 16:41:43 2018 -0800

--
 docs/structured-streaming-programming-guide.md | 338 +++-
 1 file changed, 326 insertions(+), 12 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/1002bd6b/docs/structured-streaming-programming-guide.md
--
diff --git a/docs/structured-streaming-programming-guide.md 
b/docs/structured-streaming-programming-guide.md
index de13e28..1779a42 100644
--- a/docs/structured-streaming-programming-guide.md
+++ b/docs/structured-streaming-programming-guide.md
@@ -1051,7 +1051,19 @@ output mode.
 
 
 ### Join Operations
-Streaming DataFrames can be joined with static DataFrames to create new 
streaming DataFrames. Here are a few examples.
+Structured Streaming supports joining a streaming Dataset/DataFrame with a 
static Dataset/DataFrame
+as well as another streaming Dataset/DataFrame. The result of the streaming 
join is generated
+incrementally, similar to the results of streaming aggregations in the 
previous section. In this
+section we will explore what type of joins (i.e. inner, outer, etc.) are 
supported in the above
+cases. Note that in all the supported join types, the result of the join with 
a streaming
+Dataset/DataFrame will be the exactly the same as if it was with a static 
Dataset/DataFrame
+containing the same data in the stream.
+
+
+ Stream-static joins
+
+Since the introduction in Spark 2.0, Structured Streaming has supported joins 
(inner join and some
+type of outer joins) between a streaming and a static DataFrame/Dataset. Here 
is a simple example.
 
 
 
@@ -1089,6 +1101,300 @@ streamingDf.join(staticDf, "type", "right_join")  # 
right outer join with a stat
 
 
 
+Note that stream-static joins are not stateful, so no state management is 
necessary.
+However, a few types of stream-static outer joins are not yet supported.
+These are listed at the [end of this Join 
section](#support-matrix-for-joins-in-streaming-queries).
+
+ Stream-stream Joins
+In Spark 2.3, we have added support for stream-stream joins, that is, you can 
join two streaming
+Datasets/DataFrames. The challenge of generating join results between two data 
streams is that,
+at any point of time, the view of the dataset is incomplete for both sides of 
the join making
+it much harder to find matches between inputs. Any row received from one input 
stream can match
+with any future, yet-to-be-received row from the other input stream. Hence, 
for both the input
+streams, we buffer past input as streaming state, so that we can match every 
future input with
+past input and accordingly generate joined results. Furthermore, similar to 
streaming aggregations,
+we automatically handle late, out-of-order data and can limit the state using 
watermarks.
+Let’s discuss the different types of supported stream-stream joins and how 
to use them.
+
+# Inner Joins with optional Watermarking
+Inner joins on any kind of columns along with any kind of join conditions are 
supported.
+However, as the stream runs, the size of streaming state will keep growing 
indefinitely as
+*all* past input must be saved as the any new input can match with any input 
from the past.
+To avoid unbounded state, you have to define additional join conditions such 
that indefinitely
+old inputs cannot match with future inputs and therefore can be cleared from 
the state.
+In other words, you will have to do the following additional steps in the join.
+
+1. Define watermark delays on both inputs such that the engine knows how 
delayed the input can be
+(similar to streaming a