Repository: spark
Updated Branches:
  refs/heads/branch-2.2 4e53a4edd -> 576fd4c3a


[SPARK-21267][SS][DOCS] Update Structured Streaming Documentation

## What changes were proposed in this pull request?

Few changes to the Structured Streaming documentation
- Clarify that the entire stream input table is not materialized
- Add information for Ganglia
- Add Kafka Sink to the main docs
- Removed a couple of leftover experimental tags
- Added more associated reading material and talk videos.

In addition, https://github.com/apache/spark/pull/16856 broke the link to the 
RDD programming guide in several places while renaming the page. This PR fixes 
those sameeragarwal cloud-fan.
- Added a redirection to avoid breaking internal and possible external links.
- Removed unnecessary redirection pages that were there since the separate 
scala, java, and python programming guides were merged together in 2013 or 2014.

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, 
manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, 
remove this)

Please review http://spark.apache.org/contributing.html before opening a pull 
request.

Author: Tathagata Das <tathagata.das1...@gmail.com>

Closes #18485 from tdas/SPARK-21267.

(cherry picked from commit 0217dfd26f89133f146197359b556c9bf5aca172)
Signed-off-by: Shixiong Zhu <shixi...@databricks.com>


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

Branch: refs/heads/branch-2.2
Commit: 576fd4c3a67b4affc5ac50979e27ae929472f0d9
Parents: 4e53a4e
Author: Tathagata Das <tathagata.das1...@gmail.com>
Authored: Thu Jul 6 17:28:20 2017 -0700
Committer: Shixiong Zhu <shixi...@databricks.com>
Committed: Thu Jul 6 17:28:28 2017 -0700

----------------------------------------------------------------------
 docs/_layouts/global.html                       |   7 +-
 docs/index.md                                   |  13 +-
 docs/java-programming-guide.md                  |   7 -
 docs/programming-guide.md                       |   7 +
 docs/python-programming-guide.md                |   7 -
 docs/rdd-programming-guide.md                   |   2 +-
 docs/scala-programming-guide.md                 |   7 -
 docs/sql-programming-guide.md                   |  16 +-
 docs/structured-streaming-programming-guide.md  | 172 ++++++++++++++++---
 .../scala/org/apache/spark/sql/Dataset.scala    |   3 -
 10 files changed, 169 insertions(+), 72 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/576fd4c3/docs/_layouts/global.html
----------------------------------------------------------------------
diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html
index c00d0db..570483c 100755
--- a/docs/_layouts/global.html
+++ b/docs/_layouts/global.html
@@ -69,11 +69,10 @@
                             <a href="#" class="dropdown-toggle" 
data-toggle="dropdown">Programming Guides<b class="caret"></b></a>
                             <ul class="dropdown-menu">
                                 <li><a href="quick-start.html">Quick 
Start</a></li>
-                                <li><a href="programming-guide.html">Spark 
Programming Guide</a></li>
-                                <li class="divider"></li>
-                                <li><a 
href="streaming-programming-guide.html">Spark Streaming</a></li>
-                                <li><a 
href="sql-programming-guide.html">DataFrames, Datasets and SQL</a></li>
+                                <li><a href="rdd-programming-guide.html">RDDs, 
Accumulators, Broadcasts Vars</a></li>
+                                <li><a href="sql-programming-guide.html">SQL, 
DataFrames, and Datasets</a></li>
                                 <li><a 
href="structured-streaming-programming-guide.html">Structured Streaming</a></li>
+                                <li><a 
href="streaming-programming-guide.html">Spark Streaming (DStreams)</a></li>
                                 <li><a href="ml-guide.html">MLlib (Machine 
Learning)</a></li>
                                 <li><a 
href="graphx-programming-guide.html">GraphX (Graph Processing)</a></li>
                                 <li><a href="sparkr.html">SparkR (R on 
Spark)</a></li>

http://git-wip-us.apache.org/repos/asf/spark/blob/576fd4c3/docs/index.md
----------------------------------------------------------------------
diff --git a/docs/index.md b/docs/index.md
index a757fa0..51641c9 100644
--- a/docs/index.md
+++ b/docs/index.md
@@ -88,13 +88,12 @@ options for deployment:
 **Programming Guides:**
 
 * [Quick Start](quick-start.html): a quick introduction to the Spark API; 
start here!
-* [Spark Programming Guide](programming-guide.html): detailed overview of Spark
-  in all supported languages (Scala, Java, Python, R)
-* Modules built on Spark:
-  * [Spark Streaming](streaming-programming-guide.html): processing real-time 
data streams
-  * [Spark SQL, Datasets, and DataFrames](sql-programming-guide.html): support 
for structured data and relational queries
-  * [MLlib](ml-guide.html): built-in machine learning library
-  * [GraphX](graphx-programming-guide.html): Spark's new API for graph 
processing
+* [RDD Programming Guide](programming-guide.html): overview of Spark basics - 
RDDs (core but old API), accumulators, and broadcast variables  
+* [Spark SQL, Datasets, and DataFrames](sql-programming-guide.html): 
processing structured data with relational queries (newer API than RDDs)
+* [Structured Streaming](structured-streaming-programming-guide.html): 
processing structured data streams with relation queries (using Datasets and 
DataFrames, newer API than DStreams)
+* [Spark Streaming](streaming-programming-guide.html): processing data streams 
using DStreams (old API)
+* [MLlib](ml-guide.html): applying machine learning algorithms
+* [GraphX](graphx-programming-guide.html): processing graphs 
 
 **API Docs:**
 

http://git-wip-us.apache.org/repos/asf/spark/blob/576fd4c3/docs/java-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/java-programming-guide.md b/docs/java-programming-guide.md
deleted file mode 100644
index bb53958..0000000
--- a/docs/java-programming-guide.md
+++ /dev/null
@@ -1,7 +0,0 @@
----
-layout: global
-title: Java Programming Guide
-redirect: programming-guide.html
----
-
-This document has been merged into the [Spark programming 
guide](programming-guide.html).

http://git-wip-us.apache.org/repos/asf/spark/blob/576fd4c3/docs/programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/programming-guide.md b/docs/programming-guide.md
new file mode 100644
index 0000000..f8b8f74
--- /dev/null
+++ b/docs/programming-guide.md
@@ -0,0 +1,7 @@
+---
+layout: global
+title: Spark Programming Guide
+redirect: rdd-programming-guide.html
+---
+
+This document has moved [here](rdd-programming-guide.html).

http://git-wip-us.apache.org/repos/asf/spark/blob/576fd4c3/docs/python-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md
deleted file mode 100644
index 68f04b5..0000000
--- a/docs/python-programming-guide.md
+++ /dev/null
@@ -1,7 +0,0 @@
----
-layout: global
-title: Python Programming Guide
-redirect: programming-guide.html
----
-
-This document has been merged into the [Spark programming 
guide](programming-guide.html).

http://git-wip-us.apache.org/repos/asf/spark/blob/576fd4c3/docs/rdd-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/rdd-programming-guide.md b/docs/rdd-programming-guide.md
index f7cfd5f..d021b73 100644
--- a/docs/rdd-programming-guide.md
+++ b/docs/rdd-programming-guide.md
@@ -1,6 +1,6 @@
 ---
 layout: global
-title: Spark Programming Guide
+title: RDD Programming Guide
 description: Spark SPARK_VERSION_SHORT programming guide in Java, Scala and 
Python
 ---
 

http://git-wip-us.apache.org/repos/asf/spark/blob/576fd4c3/docs/scala-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md
deleted file mode 100644
index 69ceb63..0000000
--- a/docs/scala-programming-guide.md
+++ /dev/null
@@ -1,7 +0,0 @@
----
-layout: global
-title: Spark Programming Guide
-redirect: programming-guide.html
----
-
-This document has moved [here](programming-guide.html).

http://git-wip-us.apache.org/repos/asf/spark/blob/576fd4c3/docs/sql-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md
index 8e722ae..b5eca76 100644
--- a/docs/sql-programming-guide.md
+++ b/docs/sql-programming-guide.md
@@ -392,41 +392,31 @@ While those functions are designed for DataFrames, Spark 
SQL also has type-safe
 Moreover, users are not limited to the predefined aggregate functions and can 
create their own.
 
 ### Untyped User-Defined Aggregate Functions
-
-<div class="codetabs">
-
-<div data-lang="scala"  markdown="1">
-
 Users have to extend the 
[UserDefinedAggregateFunction](api/scala/index.html#org.apache.spark.sql.expressions.UserDefinedAggregateFunction)
 abstract class to implement a custom untyped aggregate function. For example, 
a user-defined average
 can look like:
 
+<div class="codetabs">
+<div data-lang="scala"  markdown="1">
 {% include_example untyped_custom_aggregation 
scala/org/apache/spark/examples/sql/UserDefinedUntypedAggregation.scala%}
 </div>
-
 <div data-lang="java"  markdown="1">
-
 {% include_example untyped_custom_aggregation 
java/org/apache/spark/examples/sql/JavaUserDefinedUntypedAggregation.java%}
 </div>
-
 </div>
 
 ### Type-Safe User-Defined Aggregate Functions
 
 User-defined aggregations for strongly typed Datasets revolve around the 
[Aggregator](api/scala/index.html#org.apache.spark.sql.expressions.Aggregator) 
abstract class.
 For example, a type-safe user-defined average can look like:
-<div class="codetabs">
 
+<div class="codetabs">
 <div data-lang="scala"  markdown="1">
-
 {% include_example typed_custom_aggregation 
scala/org/apache/spark/examples/sql/UserDefinedTypedAggregation.scala%}
 </div>
-
 <div data-lang="java"  markdown="1">
-
 {% include_example typed_custom_aggregation 
java/org/apache/spark/examples/sql/JavaUserDefinedTypedAggregation.java%}
 </div>
-
 </div>
 
 # Data Sources

http://git-wip-us.apache.org/repos/asf/spark/blob/576fd4c3/docs/structured-streaming-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/structured-streaming-programming-guide.md 
b/docs/structured-streaming-programming-guide.md
index d478042..3bc377c 100644
--- a/docs/structured-streaming-programming-guide.md
+++ b/docs/structured-streaming-programming-guide.md
@@ -15,7 +15,7 @@ In this guide, we are going to walk you through the 
programming model and the AP
 # Quick Example
 Let’s say you want to maintain a running word count of text data received 
from a data server listening on a TCP socket. Let’s see how you can express 
this using Structured Streaming. You can see the full code in
 
[Scala]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCount.scala)/[Java]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredNetworkWordCount.java)/[Python]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/python/sql/streaming/structured_network_wordcount.py)/[R]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/r/streaming/structured_network_wordcount.R).
-And if you [download Spark](http://spark.apache.org/downloads.html), you can 
directly run the example. In any case, let’s walk through the example 
step-by-step and understand how it works. First, we have to import the 
necessary classes and create a local SparkSession, the starting point of all 
functionalities related to Spark.
+And if you [download Spark](http://spark.apache.org/downloads.html), you can 
directly [run the example](index.html#running-the-examples-and-shell). In any 
case, let’s walk through the example step-by-step and understand how it 
works. First, we have to import the necessary classes and create a local 
SparkSession, the starting point of all functionalities related to Spark.
 
 <div class="codetabs">
 <div data-lang="scala"  markdown="1">
@@ -450,7 +450,12 @@ running counts with the new data to compute updated 
counts, as shown below.
 
 ![Model](img/structured-streaming-example-model.png)
 
-This model is significantly different from many other stream processing 
+**Note that Structured Streaming does not materialize the entire table**. It 
reads the latest
+available data from the streaming data source, processes it incrementally to 
update the result,
+and then discards the source data. It only keeps around the minimal 
intermediate *state* data as
+required to update the result (e.g. intermediate counts in the earlier 
example).
+
+This model is significantly different from many other stream processing
 engines. Many streaming systems require the user to maintain running 
 aggregations themselves, thus having to reason about fault-tolerance, and 
 data consistency (at-least-once, or at-most-once, or exactly-once). In this 
@@ -486,11 +491,11 @@ Streaming DataFrames can be created through the 
`DataStreamReader` interface
 returned by `SparkSession.readStream()`. In [R](api/R/read.stream.html), with 
the `read.stream()` method. Similar to the read interface for creating static 
DataFrame, you can specify the details of the source – data format, schema, 
options, etc.
 
 #### Input Sources
-In Spark 2.0, there are a few built-in sources.
+There are a few built-in sources.
 
   - **File source** - Reads files written in a directory as a stream of data. 
Supported file formats are text, csv, json, parquet. See the docs of the 
DataStreamReader interface for a more up-to-date list, and supported options 
for each file format. Note that the files must be atomically placed in the 
given directory, which in most file systems, can be achieved by file move 
operations.
 
-  - **Kafka source** - Poll data from Kafka. It's compatible with Kafka broker 
versions 0.10.0 or higher. See the [Kafka Integration 
Guide](structured-streaming-kafka-integration.html) for more details.
+  - **Kafka source** - Reads data from Kafka. It's compatible with Kafka 
broker versions 0.10.0 or higher. See the [Kafka Integration 
Guide](structured-streaming-kafka-integration.html) for more details.
 
   - **Socket source (for testing)** - Reads UTF8 text data from a socket 
connection. The listening server socket is at the driver. Note that this should 
be used only for testing as this does not provide end-to-end fault-tolerance 
guarantees. 
 
@@ -517,17 +522,18 @@ Here are the details of all the sources in Spark.
         <br/>
         <code>fileNameOnly</code>: whether to check new files based on only 
the filename instead of on the full path (default: false). With this set to 
`true`, the following files would be considered as the same file, because their 
filenames, "dataset.txt", are the same:
         <br/>
-        · "file:///dataset.txt"<br/>
-        · "s3://a/dataset.txt"<br/>
-        · "s3n://a/b/dataset.txt"<br/>
-        · "s3a://a/b/c/dataset.txt"<br/>
-        <br/>
-
-        <br/>
+        "file:///dataset.txt"<br/>
+        "s3://a/dataset.txt"<br/>
+        "s3n://a/b/dataset.txt"<br/>
+        "s3a://a/b/c/dataset.txt"<br/>
+        <br/><br/>
         For file-format-specific options, see the related methods in 
<code>DataStreamReader</code>
         (<a 
href="api/scala/index.html#org.apache.spark.sql.streaming.DataStreamReader">Scala</a>/<a
 
href="api/java/org/apache/spark/sql/streaming/DataStreamReader.html">Java</a>/<a
 
href="api/python/pyspark.sql.html#pyspark.sql.streaming.DataStreamReader">Python</a>/<a
         href="api/R/read.stream.html">R</a>).
-        E.g. for "parquet" format options see 
<code>DataStreamReader.parquet()</code></td>
+        E.g. for "parquet" format options see 
<code>DataStreamReader.parquet()</code>.
+        <br/><br/>
+        In addition, there are session configurations that affect certain 
file-formats. See the <a href="sql-programming-guide.html">SQL Programming 
Guide</a> for more details. E.g., for "parquet", see <a 
href="sql-programming-guide.html#configuration">Parquet configuration</a> 
section.
+        </td>
     <td>Yes</td>
     <td>Supports glob paths, but does not support multiple comma-separated 
paths/globs.</td>
   </tr>
@@ -758,6 +764,60 @@ count(groupBy(df, "deviceType"))
 </div>
 </div>
 
+You can also register a streaming DataFrame/Dataset as a temporary view and 
then apply SQL commands on it.
+
+<div class="codetabs">
+<div data-lang="scala"  markdown="1">
+{% highlight scala %}
+df.createOrReplaceTempView("updates")
+spark.sql("select count(*) from updates")  // returns another streaming DF
+{% endhighlight %}
+</div>
+<div data-lang="java"  markdown="1">  
+{% highlight java %}
+df.createOrReplaceTempView("updates");
+spark.sql("select count(*) from updates");  // returns another streaming DF
+{% endhighlight %}
+</div>
+<div data-lang="python"  markdown="1">  
+{% highlight python %}
+df.createOrReplaceTempView("updates")
+spark.sql("select count(*) from updates")  # returns another streaming DF
+{% endhighlight %}
+</div>
+<div data-lang="r"  markdown="1">
+{% highlight r %}
+createOrReplaceTempView(df, "updates")
+sql("select count(*) from updates")
+{% endhighlight %}
+</div>
+</div>
+
+Note, you can identify whether a DataFrame/Dataset has streaming data or not 
by using `df.isStreaming`.
+
+<div class="codetabs">
+<div data-lang="scala"  markdown="1">
+{% highlight scala %}
+df.isStreaming
+{% endhighlight %}
+</div>
+<div data-lang="java"  markdown="1">
+{% highlight java %}
+df.isStreaming()
+{% endhighlight %}
+</div>
+<div data-lang="python"  markdown="1">
+{% highlight python %}
+df.isStreaming()
+{% endhighlight %}
+</div>
+<div data-lang="r"  markdown="1">
+{% highlight bash %}
+Not available.
+{% endhighlight %}
+</div>
+</div>
+
 ### Window Operations on Event Time
 Aggregations over a sliding event-time window are straightforward with 
Structured Streaming and are very similar to grouped aggregations. In a grouped 
aggregation, aggregate values (e.g. counts) are maintained for each unique 
value in the user-specified grouping column. In case of window-based 
aggregations, aggregate values are maintained for each window the event-time of 
a row falls into. Let's understand this with an illustration. 
 
@@ -1043,7 +1103,7 @@ streamingDf \
 </div>
 
 ### Arbitrary Stateful Operations
-Many uscases require more advanced stateful operations than aggregations. For 
example, in many usecases, you have to track sessions from data streams of 
events. For doing such sessionization, you will have to save arbitrary types of 
data as state, and perform arbitrary operations on the state using the data 
stream events in every trigger. Since Spark 2.2, this can be done using the 
operation `mapGroupsWithState` and the more powerful operation 
`flatMapGroupsWithState`. Both operations allow you to apply user-defined code 
on grouped Datasets to update user-defined state. For more concrete details, 
take a look at the API documentation 
([Scala](api/scala/index.html#org.apache.spark.sql.streaming.GroupState)/[Java](api/java/org/apache/spark/sql/streaming/GroupState.html))
 and the examples 
([Scala]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredSessionization.scala)/[Java]({{site.SPARK_GITHUB_URL}}/bl
 
ob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredSessionization.java)).
 
+Many usecases require more advanced stateful operations than aggregations. For 
example, in many usecases, you have to track sessions from data streams of 
events. For doing such sessionization, you will have to save arbitrary types of 
data as state, and perform arbitrary operations on the state using the data 
stream events in every trigger. Since Spark 2.2, this can be done using the 
operation `mapGroupsWithState` and the more powerful operation 
`flatMapGroupsWithState`. Both operations allow you to apply user-defined code 
on grouped Datasets to update user-defined state. For more concrete details, 
take a look at the API documentation 
([Scala](api/scala/index.html#org.apache.spark.sql.streaming.GroupState)/[Java](api/java/org/apache/spark/sql/streaming/GroupState.html))
 and the examples 
([Scala]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredSessionization.scala)/[Java]({{site.SPARK_GITHUB_URL}}/b
 
lob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredSessionization.java)).
 
 ### Unsupported Operations
 There are a few DataFrame/Dataset operations that are not supported with 
streaming DataFrames/Datasets. 
@@ -1201,6 +1261,16 @@ writeStream
     .start()
 {% endhighlight %}
 
+- **Kafka sink** - Stores the output to one or more topics in Kafka.
+
+{% highlight scala %}
+writeStream
+    .format("kafka")
+    .option("kafka.bootstrap.servers", "host1:port1,host2:port2")
+    .option("topic", "updates")
+    .start()
+{% endhighlight %}
+
 - **Foreach sink** - Runs arbitrary computation on the records in the output. 
See later in the section for more details.
 
 {% highlight scala %}
@@ -1253,12 +1323,19 @@ Here are the details of all the sinks in Spark.
         href="api/R/write.stream.html">R</a>).
         E.g. for "parquet" format options see 
<code>DataFrameWriter.parquet()</code>
     </td>
-    <td>Yes</td>
+    <td>Yes (exactly-once)</td>
     <td>Supports writes to partitioned tables. Partitioning by time may be 
useful.</td>
   </tr>
   <tr>
+    <td><b>Kafka Sink</b></td>
+    <td>Append, Update, Complete</td>
+    <td>See the <a href="structured-streaming-kafka-integration.html">Kafka 
Integration Guide</a></td>
+    <td>Yes (at-least-once)</td>
+    <td>More details in the <a 
href="structured-streaming-kafka-integration.html">Kafka Integration 
Guide</a></td>
+  </tr>
+  <tr>
     <td><b>Foreach Sink</b></td>
-    <td>Append, Update, Compelete</td>
+    <td>Append, Update, Complete</td>
     <td>None</td>
     <td>Depends on ForeachWriter implementation</td>
     <td>More details in the <a href="#using-foreach">next section</a></td>
@@ -1624,10 +1701,9 @@ Not available in R.
 
 
 ## Monitoring Streaming Queries
-There are two APIs for monitoring and debugging active queries - 
-interactively and asynchronously.
+There are multiple ways to monitor active streaming queries. You can either 
push metrics to external systems using Spark's Dropwizard Metrics support, or 
access them programmatically.
 
-### Interactive APIs
+### Reading Metrics Interactively
 
 You can directly get the current status and metrics of an active query using 
 `streamingQuery.lastProgress()` and `streamingQuery.status()`. 
@@ -1857,7 +1933,7 @@ Will print something like the following.
 </div>
 </div>
 
-### Asynchronous API
+### Reporting Metrics programmatically using Asynchronous APIs
 
 You can also asynchronously monitor all queries associated with a
 `SparkSession` by attaching a `StreamingQueryListener`
@@ -1922,6 +1998,41 @@ Not available in R.
 </div>
 </div>
 
+### Reporting Metrics using Dropwizard 
+Spark supports reporting metrics using the [Dropwizard 
Library](monitoring.html#metrics). To enable metrics of Structured Streaming 
queries to be reported as well, you have to explicitly enable the configuration 
`spark.sql.streaming.metricsEnabled` in the SparkSession. 
+
+<div class="codetabs">
+<div data-lang="scala"  markdown="1">
+{% highlight scala %}
+spark.conf.set("spark.sql.streaming.metricsEnabled", "true")
+// or
+spark.sql("SET spark.sql.streaming.metricsEnabled=true")
+{% endhighlight %}
+</div>
+<div data-lang="java"  markdown="1">  
+{% highlight java %}
+spark.conf().set("spark.sql.streaming.metricsEnabled", "true");
+// or
+spark.sql("SET spark.sql.streaming.metricsEnabled=true");
+{% endhighlight %}
+</div>
+<div data-lang="python"  markdown="1">  
+{% highlight python %}
+spark.conf.set("spark.sql.streaming.metricsEnabled", "true")
+# or
+spark.sql("SET spark.sql.streaming.metricsEnabled=true")
+{% endhighlight %}
+</div>
+<div data-lang="r"  markdown="1">
+{% highlight r %}
+sql("SET spark.sql.streaming.metricsEnabled=true")
+{% endhighlight %}
+</div>
+</div>
+
+
+All queries started in the SparkSession after this configuration has been 
enabled will report metrics through Dropwizard to whatever 
[sinks](monitoring.html#metrics) have been configured (e.g. Ganglia, Graphite, 
JMX, etc.).
+
 ## Recovering from Failures with Checkpointing 
 In case of a failure or intentional shutdown, you can recover the previous 
progress and state of a previous query, and continue where it left off. This is 
done using checkpointing and write ahead logs. You can configure a query with a 
checkpoint location, and the query will save all the progress information (i.e. 
range of offsets processed in each trigger) and the running aggregates (e.g. 
word counts in the [quick example](#quick-example)) to the checkpoint location. 
This checkpoint location has to be a path in an HDFS compatible file system, 
and can be set as an option in the DataStreamWriter when [starting a 
query](#starting-streaming-queries).
 
@@ -1971,8 +2082,23 @@ write.stream(aggDF, "memory", outputMode = "complete", 
checkpointLocation = "pat
 </div>
 </div>
 
-# Where to go from here
-- Examples: See and run the
-[Scala]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples/sql/streaming)/[Java]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/java/org/apache/spark/examples/sql/streaming)/[Python]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/python/sql/streaming)/[R]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/r/streaming)
-examples.
+# Additional Information
+
+**Further Reading**
+
+- See and run the
+  
[Scala]({{site.SPARK_GITHUB_URL}}/tree/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/scala/org/apache/spark/examples/sql/streaming)/[Java]({{site.SPARK_GITHUB_URL}}/tree/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/java/org/apache/spark/examples/sql/streaming)/[Python]({{site.SPARK_GITHUB_URL}}/tree/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/python/sql/streaming)/[R]({{site.SPARK_GITHUB_URL}}/tree/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/r/streaming)
+  examples.
+    - [Instructions](index.html#running-the-examples-and-shell) on how to run 
Spark examples
+- Read about integrating with Kafka in the [Structured Streaming Kafka 
Integration Guide](structured-streaming-kafka-integration.html)
+- Read more details about using DataFrames/Datasets in the [Spark SQL 
Programming Guide](sql-programming-guide.html)
+- Third-party Blog Posts
+    - [Real-time Streaming ETL with Structured Streaming in Apache Spark 2.1 
(Databricks 
Blog)](https://databricks.com/blog/2017/01/19/real-time-streaming-etl-structured-streaming-apache-spark-2-1.html)
+    - [Real-Time End-to-End Integration with Apache Kafka in Apache Spark’s 
Structured Streaming (Databricks 
Blog)](https://databricks.com/blog/2017/04/04/real-time-end-to-end-integration-with-apache-kafka-in-apache-sparks-structured-streaming.html)
+    - [Event-time Aggregation and Watermarking in Apache Spark’s Structured 
Streaming (Databricks 
Blog)](https://databricks.com/blog/2017/05/08/event-time-aggregation-watermarking-apache-sparks-structured-streaming.html)
+
+**Talks**
+
+- Spark Summit 2017 Talk - [Easy, Scalable, Fault-tolerant Stream Processing 
with Structured Streaming in Apache 
Spark](https://spark-summit.org/2017/events/easy-scalable-fault-tolerant-stream-processing-with-structured-streaming-in-apache-spark/)
 - Spark Summit 2016 Talk - [A Deep Dive into Structured 
Streaming](https://spark-summit.org/2016/events/a-deep-dive-into-structured-streaming/)
+

http://git-wip-us.apache.org/repos/asf/spark/blob/576fd4c3/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
index 3658890..65ce77f 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
@@ -484,7 +484,6 @@ class Dataset[T] private[sql](
    * @group streaming
    * @since 2.0.0
    */
-  @Experimental
   @InterfaceStability.Evolving
   def isStreaming: Boolean = logicalPlan.isStreaming
 
@@ -545,7 +544,6 @@ class Dataset[T] private[sql](
   }
 
   /**
-   * :: Experimental ::
    * Defines an event time watermark for this [[Dataset]]. A watermark tracks 
a point in time
    * before which we assume no more late data is going to arrive.
    *
@@ -569,7 +567,6 @@ class Dataset[T] private[sql](
    * @group streaming
    * @since 2.1.0
    */
-  @Experimental
   @InterfaceStability.Evolving
   // We only accept an existing column name, not a derived column here as a 
watermark that is
   // defined on a derived column cannot referenced elsewhere in the plan.


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org

Reply via email to