Repository: spark
Updated Branches:
  refs/heads/master e2773996b -> 9543fc0e0


[SPARK-20224][SS] Updated docs for streaming dropDuplicates and 
mapGroupsWithState

## What changes were proposed in this pull request?

- Fixed bug in Java API not passing timeout conf to scala API
- Updated markdown docs
- Updated scala docs
- Added scala and Java example

## How was this patch tested?
Manually ran examples.

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

Closes #17539 from tdas/SPARK-20224.


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

Branch: refs/heads/master
Commit: 9543fc0e08a21680961689ea772441c49fcd52ee
Parents: e277399
Author: Tathagata Das <tathagata.das1...@gmail.com>
Authored: Wed Apr 5 16:03:04 2017 -0700
Committer: Tathagata Das <tathagata.das1...@gmail.com>
Committed: Wed Apr 5 16:03:04 2017 -0700

----------------------------------------------------------------------
 docs/structured-streaming-programming-guide.md  |  98 ++++++-
 .../streaming/JavaStructuredSessionization.java | 255 +++++++++++++++++++
 .../streaming/StructuredSessionization.scala    | 151 +++++++++++
 .../spark/sql/KeyValueGroupedDataset.scala      |   2 +-
 .../apache/spark/sql/streaming/GroupState.scala |  15 +-
 5 files changed, 509 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/9543fc0e/docs/structured-streaming-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/structured-streaming-programming-guide.md 
b/docs/structured-streaming-programming-guide.md
index b5cf9f1..37a1d61 100644
--- a/docs/structured-streaming-programming-guide.md
+++ b/docs/structured-streaming-programming-guide.md
@@ -1,6 +1,6 @@
 ---
 layout: global
-displayTitle: Structured Streaming Programming Guide [Alpha]
+displayTitle: Structured Streaming Programming Guide [Experimental]
 title: Structured Streaming Programming Guide
 ---
 
@@ -871,6 +871,65 @@ streamingDf.join(staticDf, "type", "right_join")  # right 
outer join with a stat
 </div>
 </div>
 
+### Streaming Deduplication
+You can deduplicate records in data streams using a unique identifier in the 
events. This is exactly same as deduplication on static using a unique 
identifier column. The query will store the necessary amount of data from 
previous records such that it can filter duplicate records. Similar to 
aggregations, you can use deduplication with or without watermarking.
+
+- *With watermark* - If there is a upper bound on how late a duplicate record 
may arrive, then you can define a watermark on a event time column and 
deduplicate using both the guid and the event time columns. The query will use 
the watermark to remove old state data from past records that are not expected 
to get any duplicates any more. This bounds the amount of the state the query 
has to maintain.
+
+- *Without watermark* - Since there are no bounds on when a duplicate record 
may arrive, the query stores the data from all the past records as state.
+
+<div class="codetabs">
+<div data-lang="scala"  markdown="1">
+
+{% highlight scala %}
+val streamingDf = spark.readStream. ...  // columns: guid, eventTime, ...
+
+// Without watermark using guid column
+streamingDf.dropDuplicates("guid")
+
+// With watermark using guid and eventTime columns
+streamingDf
+  .withWatermark("eventTime", "10 seconds")
+  .dropDuplicates("guid", "eventTime")
+{% endhighlight %}
+
+</div>
+<div data-lang="java"  markdown="1">
+
+{% highlight java %}
+Dataset<Row> streamingDf = spark.readStream. ...;  // columns: guid, 
eventTime, ...
+
+// Without watermark using guid column
+streamingDf.dropDuplicates("guid");
+
+// With watermark using guid and eventTime columns
+streamingDf
+  .withWatermark("eventTime", "10 seconds")
+  .dropDuplicates("guid", "eventTime");
+{% endhighlight %}
+
+
+</div>
+<div data-lang="python"  markdown="1">
+
+{% highlight python %}
+streamingDf = spark.readStream. ...
+
+// Without watermark using guid column
+streamingDf.dropDuplicates("guid")
+
+// With watermark using guid and eventTime columns
+streamingDf \
+  .withWatermark("eventTime", "10 seconds") \
+  .dropDuplicates("guid", "eventTime")
+{% endhighlight %}
+
+</div>
+</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)).
 
+
 ### Unsupported Operations
 There are a few DataFrame/Dataset operations that are not supported with 
streaming DataFrames/Datasets. 
 Some of them are as follows.
@@ -891,7 +950,7 @@ Some of them are as follows.
 
     + Right outer join with a streaming Dataset on the left is not supported
 
-- Any kind of joins between two streaming Datasets are not yet supported.
+- Any kind of joins between two streaming Datasets is not yet supported.
 
 In addition, there are some Dataset methods that will not work on streaming 
Datasets. They are actions that will immediately run queries and return 
results, which does not make sense on a streaming Dataset. Rather, those 
functionalities can be done by explicitly starting a streaming query (see the 
next section regarding that).
 
@@ -952,13 +1011,6 @@ Here is the compatibility matrix.
     <th>Notes</th>        
   </tr>
   <tr>
-    <td colspan="2" style="vertical-align: middle;">Queries without 
aggregation</td>
-    <td style="vertical-align: middle;">Append, Update</td>
-    <td style="vertical-align: middle;">
-        Complete mode not supported as it is infeasible to keep all data in 
the Result Table.
-    </td>
-  </tr>
-  <tr>
     <td rowspan="2" style="vertical-align: middle;">Queries with 
aggregation</td>
     <td style="vertical-align: middle;">Aggregation on event-time with 
watermark</td>
     <td style="vertical-align: middle;">Append, Update, Complete</td>
@@ -987,6 +1039,33 @@ Here is the compatibility matrix.
     </td>  
   </tr>
   <tr>
+    <td colspan="2" style="vertical-align: middle;">Queries with 
<code>mapGroupsWithState</code></td>
+    <td style="vertical-align: middle;">Update</td>
+    <td style="vertical-align: middle;"></td>
+  </tr>
+  <tr>
+    <td rowspan="2" style="vertical-align: middle;">Queries with 
<code>flatMapGroupsWithState</code></td>
+    <td style="vertical-align: middle;">Append operation mode</td>
+    <td style="vertical-align: middle;">Append</td>
+    <td style="vertical-align: middle;">
+      Aggregations are allowed after <code>flatMapGroupsWithState</code>.
+    </td>
+  </tr>
+  <tr>
+    <td style="vertical-align: middle;">Update operation mode</td>
+    <td style="vertical-align: middle;">Update</td>
+    <td style="vertical-align: middle;">
+      Aggregations not allowed after <code>flatMapGroupsWithState</code>.
+    </td>
+  </tr>
+  <tr>
+    <td colspan="2" style="vertical-align: middle;">Other queries</td>
+    <td style="vertical-align: middle;">Append, Update</td>
+    <td style="vertical-align: middle;">
+      Complete mode not supported as it is infeasible to keep all unaggregated 
data in the Result Table.
+    </td>
+  </tr>
+  <tr>
     <td></td>
     <td></td>
     <td></td>
@@ -994,6 +1073,7 @@ Here is the compatibility matrix.
   </tr>
 </table>
 
+
 #### Output Sinks
 There are a few types of built-in output sinks.
 

http://git-wip-us.apache.org/repos/asf/spark/blob/9543fc0e/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredSessionization.java
----------------------------------------------------------------------
diff --git 
a/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredSessionization.java
 
b/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredSessionization.java
new file mode 100644
index 0000000..da3a5df
--- /dev/null
+++ 
b/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredSessionization.java
@@ -0,0 +1,255 @@
+/*
+ * 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.
+ */
+package org.apache.spark.examples.sql.streaming;
+
+import org.apache.spark.api.java.function.FlatMapFunction;
+import org.apache.spark.api.java.function.MapFunction;
+import org.apache.spark.api.java.function.MapGroupsWithStateFunction;
+import org.apache.spark.sql.*;
+import org.apache.spark.sql.streaming.GroupState;
+import org.apache.spark.sql.streaming.GroupStateTimeout;
+import org.apache.spark.sql.streaming.StreamingQuery;
+
+import java.io.Serializable;
+import java.sql.Timestamp;
+import java.util.*;
+
+import scala.Tuple2;
+
+/**
+ * Counts words in UTF8 encoded, '\n' delimited text received from the network.
+ * <p>
+ * Usage: JavaStructuredNetworkWordCount <hostname> <port>
+ * <hostname> and <port> describe the TCP server that Structured Streaming
+ * would connect to receive data.
+ * <p>
+ * To run this on your local machine, you need to first run a Netcat server
+ * `$ nc -lk 9999`
+ * and then run the example
+ * `$ bin/run-example sql.streaming.JavaStructuredSessionization
+ * localhost 9999`
+ */
+public final class JavaStructuredSessionization {
+
+  public static void main(String[] args) throws Exception {
+    if (args.length < 2) {
+      System.err.println("Usage: JavaStructuredSessionization <hostname> 
<port>");
+      System.exit(1);
+    }
+
+    String host = args[0];
+    int port = Integer.parseInt(args[1]);
+
+    SparkSession spark = SparkSession
+        .builder()
+        .appName("JavaStructuredSessionization")
+        .getOrCreate();
+
+    // Create DataFrame representing the stream of input lines from connection 
to host:port
+    Dataset<Row> lines = spark
+        .readStream()
+        .format("socket")
+        .option("host", host)
+        .option("port", port)
+        .option("includeTimestamp", true)
+        .load();
+
+    FlatMapFunction<LineWithTimestamp, Event> linesToEvents =
+      new FlatMapFunction<LineWithTimestamp, Event>() {
+        @Override
+        public Iterator<Event> call(LineWithTimestamp lineWithTimestamp) 
throws Exception {
+          ArrayList<Event> eventList = new ArrayList<Event>();
+          for (String word : lineWithTimestamp.getLine().split(" ")) {
+            eventList.add(new Event(word, lineWithTimestamp.getTimestamp()));
+          }
+          System.out.println(
+              "Number of events from " + lineWithTimestamp.getLine() + " = " + 
eventList.size());
+          return eventList.iterator();
+        }
+      };
+
+    // Split the lines into words, treat words as sessionId of events
+    Dataset<Event> events = lines
+        .withColumnRenamed("value", "line")
+        .as(Encoders.bean(LineWithTimestamp.class))
+        .flatMap(linesToEvents, Encoders.bean(Event.class));
+
+    // Sessionize the events. Track number of events, start and end timestamps 
of session, and
+    // and report session updates.
+    //
+    // Step 1: Define the state update function
+    MapGroupsWithStateFunction<String, Event, SessionInfo, SessionUpdate> 
stateUpdateFunc =
+      new MapGroupsWithStateFunction<String, Event, SessionInfo, 
SessionUpdate>() {
+        @Override public SessionUpdate call(
+            String sessionId, Iterator<Event> events, GroupState<SessionInfo> 
state)
+              throws Exception {
+          // If timed out, then remove session and send final update
+          if (state.hasTimedOut()) {
+            SessionUpdate finalUpdate = new SessionUpdate(
+                sessionId, state.get().getDurationMs(), 
state.get().getNumEvents(), true);
+            state.remove();
+            return finalUpdate;
+
+          } else {
+            // Find max and min timestamps in events
+            long maxTimestampMs = Long.MIN_VALUE;
+            long minTimestampMs = Long.MAX_VALUE;
+            int numNewEvents = 0;
+            while (events.hasNext()) {
+              Event e = events.next();
+              long timestampMs = e.getTimestamp().getTime();
+              maxTimestampMs = Math.max(timestampMs, maxTimestampMs);
+              minTimestampMs = Math.min(timestampMs, minTimestampMs);
+              numNewEvents += 1;
+            }
+            SessionInfo updatedSession = new SessionInfo();
+
+            // Update start and end timestamps in session
+            if (state.exists()) {
+              SessionInfo oldSession = state.get();
+              updatedSession.setNumEvents(oldSession.numEvents + numNewEvents);
+              updatedSession.setStartTimestampMs(oldSession.startTimestampMs);
+              
updatedSession.setEndTimestampMs(Math.max(oldSession.endTimestampMs, 
maxTimestampMs));
+            } else {
+              updatedSession.setNumEvents(numNewEvents);
+              updatedSession.setStartTimestampMs(minTimestampMs);
+              updatedSession.setEndTimestampMs(maxTimestampMs);
+            }
+            state.update(updatedSession);
+            // Set timeout such that the session will be expired if no data 
received for 10 seconds
+            state.setTimeoutDuration("10 seconds");
+            return new SessionUpdate(
+                sessionId, state.get().getDurationMs(), 
state.get().getNumEvents(), false);
+          }
+        }
+      };
+
+    // Step 2: Apply the state update function to the events streaming Dataset 
grouped by sessionId
+    Dataset<SessionUpdate> sessionUpdates = events
+        .groupByKey(
+            new MapFunction<Event, String>() {
+              @Override public String call(Event event) throws Exception {
+                return event.getSessionId();
+              }
+            }, Encoders.STRING())
+        .mapGroupsWithState(
+            stateUpdateFunc,
+            Encoders.bean(SessionInfo.class),
+            Encoders.bean(SessionUpdate.class),
+            GroupStateTimeout.ProcessingTimeTimeout());
+
+    // Start running the query that prints the session updates to the console
+    StreamingQuery query = sessionUpdates
+        .writeStream()
+        .outputMode("update")
+        .format("console")
+        .start();
+
+    query.awaitTermination();
+  }
+
+  /**
+   * User-defined data type representing the raw lines with timestamps.
+   */
+  public static class LineWithTimestamp implements Serializable {
+    private String line;
+    private Timestamp timestamp;
+
+    public Timestamp getTimestamp() { return timestamp; }
+    public void setTimestamp(Timestamp timestamp) { this.timestamp = 
timestamp; }
+
+    public String getLine() { return line; }
+    public void setLine(String sessionId) { this.line = sessionId; }
+  }
+
+  /**
+   * User-defined data type representing the input events
+   */
+  public static class Event implements Serializable {
+    private String sessionId;
+    private Timestamp timestamp;
+
+    public Event() { }
+    public Event(String sessionId, Timestamp timestamp) {
+      this.sessionId = sessionId;
+      this.timestamp = timestamp;
+    }
+
+    public Timestamp getTimestamp() { return timestamp; }
+    public void setTimestamp(Timestamp timestamp) { this.timestamp = 
timestamp; }
+
+    public String getSessionId() { return sessionId; }
+    public void setSessionId(String sessionId) { this.sessionId = sessionId; }
+  }
+
+  /**
+   * User-defined data type for storing a session information as state in 
mapGroupsWithState.
+   */
+  public static class SessionInfo implements Serializable {
+    private int numEvents = 0;
+    private long startTimestampMs = -1;
+    private long endTimestampMs = -1;
+
+    public int getNumEvents() { return numEvents; }
+    public void setNumEvents(int numEvents) { this.numEvents = numEvents; }
+
+    public long getStartTimestampMs() { return startTimestampMs; }
+    public void setStartTimestampMs(long startTimestampMs) {
+      this.startTimestampMs = startTimestampMs;
+    }
+
+    public long getEndTimestampMs() { return endTimestampMs; }
+    public void setEndTimestampMs(long endTimestampMs) { this.endTimestampMs = 
endTimestampMs; }
+
+    public long getDurationMs() { return endTimestampMs - startTimestampMs; }
+    @Override public String toString() {
+      return "SessionInfo(numEvents = " + numEvents +
+          ", timestamps = " + startTimestampMs + " to " + endTimestampMs + ")";
+    }
+  }
+
+  /**
+   * User-defined data type representing the update information returned by 
mapGroupsWithState.
+   */
+  public static class SessionUpdate implements Serializable {
+    private String id;
+    private long durationMs;
+    private int numEvents;
+    private boolean expired;
+
+    public SessionUpdate() { }
+
+    public SessionUpdate(String id, long durationMs, int numEvents, boolean 
expired) {
+      this.id = id;
+      this.durationMs = durationMs;
+      this.numEvents = numEvents;
+      this.expired = expired;
+    }
+
+    public String getId() { return id; }
+    public void setId(String id) { this.id = id; }
+
+    public long getDurationMs() { return durationMs; }
+    public void setDurationMs(long durationMs) { this.durationMs = durationMs; 
}
+
+    public int getNumEvents() { return numEvents; }
+    public void setNumEvents(int numEvents) { this.numEvents = numEvents; }
+
+    public boolean isExpired() { return expired; }
+    public void setExpired(boolean expired) { this.expired = expired; }
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/9543fc0e/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredSessionization.scala
----------------------------------------------------------------------
diff --git 
a/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredSessionization.scala
 
b/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredSessionization.scala
new file mode 100644
index 0000000..2ce792c
--- /dev/null
+++ 
b/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredSessionization.scala
@@ -0,0 +1,151 @@
+/*
+ * 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.
+ */
+
+// scalastyle:off println
+package org.apache.spark.examples.sql.streaming
+
+import java.sql.Timestamp
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.streaming._
+
+
+/**
+ * Counts words in UTF8 encoded, '\n' delimited text received from the network.
+ *
+ * Usage: MapGroupsWithState <hostname> <port>
+ * <hostname> and <port> describe the TCP server that Structured Streaming
+ * would connect to receive data.
+ *
+ * To run this on your local machine, you need to first run a Netcat server
+ * `$ nc -lk 9999`
+ * and then run the example
+ * `$ bin/run-example sql.streaming.StructuredNetworkWordCount
+ * localhost 9999`
+ */
+object StructuredSessionization {
+
+  def main(args: Array[String]): Unit = {
+    if (args.length < 2) {
+      System.err.println("Usage: StructuredNetworkWordCount <hostname> <port>")
+      System.exit(1)
+    }
+
+    val host = args(0)
+    val port = args(1).toInt
+
+    val spark = SparkSession
+      .builder
+      .appName("StructuredSessionization")
+      .getOrCreate()
+
+    import spark.implicits._
+
+    // Create DataFrame representing the stream of input lines from connection 
to host:port
+    val lines = spark.readStream
+      .format("socket")
+      .option("host", host)
+      .option("port", port)
+      .option("includeTimestamp", true)
+      .load()
+
+    // Split the lines into words, treat words as sessionId of events
+    val events = lines
+      .as[(String, Timestamp)]
+      .flatMap { case (line, timestamp) =>
+        line.split(" ").map(word => Event(sessionId = word, timestamp))
+      }
+
+    // Sessionize the events. Track number of events, start and end timestamps 
of session, and
+    // and report session updates.
+    val sessionUpdates = events
+      .groupByKey(event => event.sessionId)
+      .mapGroupsWithState[SessionInfo, 
SessionUpdate](GroupStateTimeout.ProcessingTimeTimeout) {
+
+        case (sessionId: String, events: Iterator[Event], state: 
GroupState[SessionInfo]) =>
+
+          // If timed out, then remove session and send final update
+          if (state.hasTimedOut) {
+            val finalUpdate =
+              SessionUpdate(sessionId, state.get.durationMs, 
state.get.numEvents, expired = true)
+            state.remove()
+            finalUpdate
+          } else {
+            // Update start and end timestamps in session
+            val timestamps = events.map(_.timestamp.getTime).toSeq
+            val updatedSession = if (state.exists) {
+              val oldSession = state.get
+              SessionInfo(
+                oldSession.numEvents + timestamps.size,
+                oldSession.startTimestampMs,
+                math.max(oldSession.endTimestampMs, timestamps.max))
+            } else {
+              SessionInfo(timestamps.size, timestamps.min, timestamps.max)
+            }
+            state.update(updatedSession)
+
+            // Set timeout such that the session will be expired if no data 
received for 10 seconds
+            state.setTimeoutDuration("10 seconds")
+            SessionUpdate(sessionId, state.get.durationMs, 
state.get.numEvents, expired = false)
+          }
+      }
+
+    // Start running the query that prints the session updates to the console
+    val query = sessionUpdates
+      .writeStream
+      .outputMode("update")
+      .format("console")
+      .start()
+
+    query.awaitTermination()
+  }
+}
+/** User-defined data type representing the input events */
+case class Event(sessionId: String, timestamp: Timestamp)
+
+/**
+ * User-defined data type for storing a session information as state in 
mapGroupsWithState.
+ *
+ * @param numEvents        total number of events received in the session
+ * @param startTimestampMs timestamp of first event received in the session 
when it started
+ * @param endTimestampMs   timestamp of last event received in the session 
before it expired
+ */
+case class SessionInfo(
+    numEvents: Int,
+    startTimestampMs: Long,
+    endTimestampMs: Long) {
+
+  /** Duration of the session, between the first and last events */
+  def durationMs: Long = endTimestampMs - startTimestampMs
+}
+
+/**
+ * User-defined data type representing the update information returned by 
mapGroupsWithState.
+ *
+ * @param id          Id of the session
+ * @param durationMs  Duration the session was active, that is, from first 
event to its expiry
+ * @param numEvents   Number of events received by the session while it was 
active
+ * @param expired     Is the session active or expired
+ */
+case class SessionUpdate(
+    id: String,
+    durationMs: Long,
+    numEvents: Int,
+    expired: Boolean)
+
+// scalastyle:on println
+

http://git-wip-us.apache.org/repos/asf/spark/blob/9543fc0e/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala
index 022c2f5..cb42e9e 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala
@@ -347,7 +347,7 @@ class KeyValueGroupedDataset[K, V] private[sql](
       stateEncoder: Encoder[S],
       outputEncoder: Encoder[U],
       timeoutConf: GroupStateTimeout): Dataset[U] = {
-    mapGroupsWithState[S, U](
+    mapGroupsWithState[S, U](timeoutConf)(
       (key: K, it: Iterator[V], s: GroupState[S]) => func.call(key, it.asJava, 
s)
     )(stateEncoder, outputEncoder)
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/9543fc0e/sql/core/src/main/scala/org/apache/spark/sql/streaming/GroupState.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/streaming/GroupState.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/streaming/GroupState.scala
index 15df906..c659ac7 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/GroupState.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/GroupState.scala
@@ -34,7 +34,7 @@ import 
org.apache.spark.sql.catalyst.plans.logical.LogicalGroupState
  * `Dataset.groupByKey()`) while maintaining user-defined per-group state 
between invocations.
  * For a static batch Dataset, the function will be invoked once per group. 
For a streaming
  * Dataset, the function will be invoked for each group repeatedly in every 
trigger.
- * That is, in every batch of the `streaming.StreamingQuery`,
+ * That is, in every batch of the `StreamingQuery`,
  * the function will be invoked once for each group that has data in the 
trigger. Furthermore,
  * if timeout is set, then the function will invoked on timed out groups (more 
detail below).
  *
@@ -42,12 +42,23 @@ import 
org.apache.spark.sql.catalyst.plans.logical.LogicalGroupState
  *  - The key of the group.
  *  - An iterator containing all the values for this group.
  *  - A user-defined state object set by previous invocations of the given 
function.
+ *
  * In case of a batch Dataset, there is only one invocation and state object 
will be empty as
  * there is no prior state. Essentially, for batch Datasets, 
`[map/flatMap]GroupsWithState`
  * is equivalent to `[map/flatMap]Groups` and any updates to the state and/or 
timeouts have
  * no effect.
  *
- * Important points to note about the function.
+ * The major difference between `mapGroupsWithState` and 
`flatMapGroupsWithState` is that the
+ * former allows the function to return one and only one record, whereas the 
latter
+ * allows the function to return any number of records (including no records). 
Furthermore, the
+ * `flatMapGroupsWithState` is associated with an operation output mode, which 
can be either
+ * `Append` or `Update`. Semantically, this defines whether the output records 
of one trigger
+ * is effectively replacing the previously output records (from previous 
triggers) or is appending
+ * to the list of previously output records. Essentially, this defines how the 
Result Table (refer
+ * to the semantics in the programming guide) is updated, and allows us to 
reason about the
+ * semantics of later operations.
+ *
+ * Important points to note about the function (both mapGroupsWithState and 
flatMapGroupsWithState).
  *  - In a trigger, the function will be called only the groups present in the 
batch. So do not
  *    assume that the function will be called in every trigger for every group 
that has state.
  *  - There is no guaranteed ordering of values in the iterator in the 
function, neither with


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

Reply via email to