Skip to content

Commit

Permalink
[SPARK-20224][SS] Updated docs for streaming dropDuplicates and mapGr…
Browse files Browse the repository at this point in the history
…oupsWithState

## 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 <[email protected]>

Closes apache#17539 from tdas/SPARK-20224.
  • Loading branch information
tdas committed Apr 5, 2017
1 parent e277399 commit 9543fc0
Show file tree
Hide file tree
Showing 5 changed files with 509 additions and 12 deletions.
98 changes: 89 additions & 9 deletions docs/structured-streaming-programming-guide.md
Original file line number Diff line number Diff line change
@@ -1,6 +1,6 @@
---
layout: global
displayTitle: Structured Streaming Programming Guide [Alpha]
displayTitle: Structured Streaming Programming Guide [Experimental]
title: Structured Streaming Programming Guide
---

Expand Down Expand Up @@ -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}}/blob/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.
Expand All @@ -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).

Expand Down Expand Up @@ -951,13 +1010,6 @@ Here is the compatibility matrix.
<th>Supported Output Modes</th>
<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>
Expand Down Expand Up @@ -986,6 +1038,33 @@ Here is the compatibility matrix.
this mode.
</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>
Expand All @@ -994,6 +1073,7 @@ Here is the compatibility matrix.
</tr>
</table>


#### Output Sinks
There are a few types of built-in output sinks.

Expand Down
Original file line number Diff line number Diff line change
@@ -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; }
}
}
Loading

0 comments on commit 9543fc0

Please sign in to comment.