You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by td...@apache.org on 2016/10/05 23:45:56 UTC

[2/2] spark git commit: [SPARK-17346][SQL] Add Kafka source for Structured Streaming

[SPARK-17346][SQL] Add Kafka source for Structured Streaming

## What changes were proposed in this pull request?

This PR adds a new project ` external/kafka-0-10-sql` for Structured Streaming Kafka source.

It's based on the design doc: https://docs.google.com/document/d/19t2rWe51x7tq2e5AOfrsM9qb8_m7BRuv9fel9i0PqR8/edit?usp=sharing

tdas did most of work and part of them was inspired by koeninger's work.

### Introduction

The Kafka source is a structured streaming data source to poll data from Kafka. The schema of reading data is as follows:

Column | Type
---- | ----
key | binary
value | binary
topic | string
partition | int
offset | long
timestamp | long
timestampType | int

The source can deal with deleting topics. However, the user should make sure there is no Spark job processing the data when deleting a topic.

### Configuration

The user can use `DataStreamReader.option` to set the following configurations.

Kafka Source's options | value | default | meaning
------ | ------- | ------ | -----
startingOffset | ["earliest", "latest"] | "latest" | The start point when a query is started, either "earliest" which is from the earliest offset, or "latest" which is just from the latest offset. Note: This only applies when a new Streaming query is started, and that resuming will always pick up from where the query left off.
failOnDataLost | [true, false] | true | Whether to fail the query when it's possible that data is lost (e.g., topics are deleted, or offsets are out of range). This may be a false alarm. You can disable it when it doesn't work as you expected.
subscribe | A comma-separated list of topics | (none) | The topic list to subscribe. Only one of "subscribe" and "subscribeParttern" options can be specified for Kafka source.
subscribePattern | Java regex string | (none) | The pattern used to subscribe the topic. Only one of "subscribe" and "subscribeParttern" options can be specified for Kafka source.
kafka.consumer.poll.timeoutMs | long | 512 | The timeout in milliseconds to poll data from Kafka in executors
fetchOffset.numRetries | int | 3 | Number of times to retry before giving up fatch Kafka latest offsets.
fetchOffset.retryIntervalMs | long | 10 | milliseconds to wait before retrying to fetch Kafka offsets

Kafka's own configurations can be set via `DataStreamReader.option` with `kafka.` prefix, e.g, `stream.option("kafka.bootstrap.servers", "host:port")`

### Usage

* Subscribe to 1 topic
```Scala
spark
  .readStream
  .format("kafka")
  .option("kafka.bootstrap.servers", "host:port")
  .option("subscribe", "topic1")
  .load()
```

* Subscribe to multiple topics
```Scala
spark
  .readStream
  .format("kafka")
  .option("kafka.bootstrap.servers", "host:port")
  .option("subscribe", "topic1,topic2")
  .load()
```

* Subscribe to a pattern
```Scala
spark
  .readStream
  .format("kafka")
  .option("kafka.bootstrap.servers", "host:port")
  .option("subscribePattern", "topic.*")
  .load()
```

## How was this patch tested?

The new unit tests.

Author: Shixiong Zhu <sh...@databricks.com>
Author: Tathagata Das <ta...@gmail.com>
Author: Shixiong Zhu <zs...@gmail.com>
Author: cody koeninger <co...@koeninger.org>

Closes #15102 from zsxwing/kafka-source.


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

Branch: refs/heads/master
Commit: 9293734d35eb3d6e4fd4ebb86f54dd5d3a35e6db
Parents: 5fd54b9
Author: Shixiong Zhu <sh...@databricks.com>
Authored: Wed Oct 5 16:45:45 2016 -0700
Committer: Tathagata Das <ta...@gmail.com>
Committed: Wed Oct 5 16:45:45 2016 -0700

----------------------------------------------------------------------
 .../spark/util/UninterruptibleThread.scala      |   7 -
 dev/run-tests.py                                |   2 +-
 dev/sparktestsupport/modules.py                 |  12 +
 docs/structured-streaming-kafka-integration.md  | 239 +++++++++++
 docs/structured-streaming-programming-guide.md  |   7 +-
 external/kafka-0-10-sql/pom.xml                 |  82 ++++
 ....apache.spark.sql.sources.DataSourceRegister |   1 +
 .../sql/kafka010/CachedKafkaConsumer.scala      | 152 +++++++
 .../apache/spark/sql/kafka010/KafkaSource.scala | 399 +++++++++++++++++
 .../spark/sql/kafka010/KafkaSourceOffset.scala  |  54 +++
 .../sql/kafka010/KafkaSourceProvider.scala      | 282 ++++++++++++
 .../spark/sql/kafka010/KafkaSourceRDD.scala     | 148 +++++++
 .../apache/spark/sql/kafka010/package-info.java |  21 +
 .../src/test/resources/log4j.properties         |  28 ++
 .../sql/kafka010/KafkaSourceOffsetSuite.scala   |  39 ++
 .../spark/sql/kafka010/KafkaSourceSuite.scala   | 424 +++++++++++++++++++
 .../spark/sql/kafka010/KafkaTestUtils.scala     | 339 +++++++++++++++
 pom.xml                                         |   1 +
 project/SparkBuild.scala                        |   6 +-
 .../execution/streaming/StreamExecution.scala   |   8 +-
 .../apache/spark/sql/streaming/StreamTest.scala |  40 +-
 21 files changed, 2268 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/9293734d/core/src/main/scala/org/apache/spark/util/UninterruptibleThread.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/util/UninterruptibleThread.scala b/core/src/main/scala/org/apache/spark/util/UninterruptibleThread.scala
index 4dcf951..f0b68f0 100644
--- a/core/src/main/scala/org/apache/spark/util/UninterruptibleThread.scala
+++ b/core/src/main/scala/org/apache/spark/util/UninterruptibleThread.scala
@@ -90,13 +90,6 @@ private[spark] class UninterruptibleThread(name: String) extends Thread(name) {
   }
 
   /**
-   * Tests whether `interrupt()` has been called.
-   */
-  override def isInterrupted: Boolean = {
-    super.isInterrupted || uninterruptibleLock.synchronized { shouldInterruptThread }
-  }
-
-  /**
    * Interrupt `this` thread if possible. If `this` is in the uninterruptible status, it won't be
    * interrupted until it enters into the interruptible status.
    */

http://git-wip-us.apache.org/repos/asf/spark/blob/9293734d/dev/run-tests.py
----------------------------------------------------------------------
diff --git a/dev/run-tests.py b/dev/run-tests.py
index ae4b530..5d661f5 100755
--- a/dev/run-tests.py
+++ b/dev/run-tests.py
@@ -110,7 +110,7 @@ def determine_modules_to_test(changed_modules):
     ['graphx', 'examples']
     >>> x = [x.name for x in determine_modules_to_test([modules.sql])]
     >>> x # doctest: +NORMALIZE_WHITESPACE
-    ['sql', 'hive', 'mllib', 'examples', 'hive-thriftserver',
+    ['sql', 'hive', 'mllib', 'sql-kafka-0-10', 'examples', 'hive-thriftserver',
      'pyspark-sql', 'sparkr', 'pyspark-mllib', 'pyspark-ml']
     """
     modules_to_test = set()

http://git-wip-us.apache.org/repos/asf/spark/blob/9293734d/dev/sparktestsupport/modules.py
----------------------------------------------------------------------
diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py
index 050cdf0..5f14683 100644
--- a/dev/sparktestsupport/modules.py
+++ b/dev/sparktestsupport/modules.py
@@ -158,6 +158,18 @@ hive_thriftserver = Module(
 )
 
 
+sql_kafka = Module(
+    name="sql-kafka-0-10",
+    dependencies=[sql],
+    source_file_regexes=[
+        "external/kafka-0-10-sql",
+    ],
+    sbt_test_goals=[
+        "sql-kafka-0-10/test",
+    ]
+)
+
+
 sketch = Module(
     name="sketch",
     dependencies=[tags],

http://git-wip-us.apache.org/repos/asf/spark/blob/9293734d/docs/structured-streaming-kafka-integration.md
----------------------------------------------------------------------
diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md
new file mode 100644
index 0000000..668489a
--- /dev/null
+++ b/docs/structured-streaming-kafka-integration.md
@@ -0,0 +1,239 @@
+---
+layout: global
+title: Structured Streaming + Kafka Integration Guide (Kafka broker version 0.10.0 or higher)
+---
+
+Structured Streaming integration for Kafka 0.10 to poll data from Kafka.
+
+### Linking
+For Scala/Java applications using SBT/Maven project definitions, link your application with the following artifact:
+
+    groupId = org.apache.spark
+    artifactId = spark-sql-kafka-0-10_{{site.SCALA_BINARY_VERSION}}
+    version = {{site.SPARK_VERSION_SHORT}}
+
+For Python applications, you need to add this above library and its dependencies when deploying your
+application. See the [Deploying](#deploying) subsection below.
+
+### Creating a Kafka Source Stream
+
+<div class="codetabs">
+<div data-lang="scala" markdown="1">
+
+    // Subscribe to 1 topic
+    val ds1 = spark
+      .readStream
+      .format("kafka")
+      .option("kafka.bootstrap.servers", "host1:port1,host2:port2")
+      .option("subscribe", "topic1")
+      .load()
+    ds1.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
+      .as[(String, String)]
+
+    // Subscribe to multiple topics
+    val ds2 = spark
+      .readStream
+      .format("kafka")
+      .option("kafka.bootstrap.servers", "host1:port1,host2:port2")
+      .option("subscribe", "topic1,topic2")
+      .load()
+    ds2.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
+      .as[(String, String)]
+
+    // Subscribe to a pattern
+    val ds3 = spark
+      .readStream
+      .format("kafka")
+      .option("kafka.bootstrap.servers", "host1:port1,host2:port2")
+      .option("subscribePattern", "topic.*")
+      .load()
+    ds3.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
+      .as[(String, String)]
+
+</div>
+<div data-lang="java" markdown="1">
+
+    // Subscribe to 1 topic
+    Dataset<Row> ds1 = spark
+      .readStream()
+      .format("kafka")
+      .option("kafka.bootstrap.servers", "host1:port1,host2:port2")
+      .option("subscribe", "topic1")
+      .load()
+    ds1.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
+
+    // Subscribe to multiple topics
+    Dataset<Row> ds2 = spark
+      .readStream()
+      .format("kafka")
+      .option("kafka.bootstrap.servers", "host1:port1,host2:port2")
+      .option("subscribe", "topic1,topic2")
+      .load()
+    ds2.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
+
+    // Subscribe to a pattern
+    Dataset<Row> ds3 = spark
+      .readStream()
+      .format("kafka")
+      .option("kafka.bootstrap.servers", "host1:port1,host2:port2")
+      .option("subscribePattern", "topic.*")
+      .load()
+    ds3.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
+
+</div>
+<div data-lang="python" markdown="1">
+
+    # Subscribe to 1 topic
+    ds1 = spark
+      .readStream()
+      .format("kafka")
+      .option("kafka.bootstrap.servers", "host1:port1,host2:port2")
+      .option("subscribe", "topic1")
+      .load()
+    ds1.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
+
+    # Subscribe to multiple topics
+    ds2 = spark
+      .readStream
+      .format("kafka")
+      .option("kafka.bootstrap.servers", "host1:port1,host2:port2")
+      .option("subscribe", "topic1,topic2")
+      .load()
+    ds2.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
+
+    # Subscribe to a pattern
+    ds3 = spark
+      .readStream()
+      .format("kafka")
+      .option("kafka.bootstrap.servers", "host1:port1,host2:port2")
+      .option("subscribePattern", "topic.*")
+      .load()
+    ds3.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
+
+</div>
+</div>
+
+Each row in the source has the following schema:
+<table class="table">
+<tr><th>Column</th><th>Type</th></tr>
+<tr>
+  <td>key</td>
+  <td>binary</td>
+</tr>
+<tr>
+  <td>value</td>
+  <td>binary</td>
+</tr>
+<tr>
+  <td>topic</td>
+  <td>string</td>
+</tr>
+<tr>
+  <td>partition</td>
+  <td>int</td>
+</tr>
+<tr>
+  <td>offset</td>
+  <td>long</td>
+</tr>
+<tr>
+  <td>timestamp</td>
+  <td>long</td>
+</tr>
+<tr>
+  <td>timestampType</td>
+  <td>int</td>
+</tr>
+</table>
+
+The following options must be set for the Kafka source.
+
+<table class="table">
+<tr><th>Option</th><th>value</th><th>meaning</th></tr>
+<tr>
+  <td>subscribe</td>
+  <td>A comma-separated list of topics</td>
+  <td>The topic list to subscribe. Only one of "subscribe" and "subscribePattern" options can be
+  specified for Kafka source.</td>
+</tr>
+<tr>
+  <td>subscribePattern</td>
+  <td>Java regex string</td>
+  <td>The pattern used to subscribe the topic. Only one of "subscribe" and "subscribePattern"
+  options can be specified for Kafka source.</td>
+</tr>
+<tr>
+  <td>kafka.bootstrap.servers</td>
+  <td>A comma-separated list of host:port</td>
+  <td>The Kafka "bootstrap.servers" configuration.</td>
+</tr>
+</table>
+
+The following configurations are optional:
+
+<table class="table">
+<tr><th>Option</th><th>value</th><th>default</th><th>meaning</th></tr>
+<tr>
+  <td>startingOffset</td>
+  <td>["earliest", "latest"]</td>
+  <td>"latest"</td>
+  <td>The start point when a query is started, either "earliest" which is from the earliest offset, 
+  or "latest" which is just from the latest offset. Note: This only applies when a new Streaming q
+  uery is started, and that resuming will always pick up from where the query left off.</td>
+</tr>
+<tr>
+  <td>failOnDataLoss</td>
+  <td>[true, false]</td>
+  <td>true</td>
+  <td>Whether to fail the query when it's possible that data is lost (e.g., topics are deleted, or 
+  offsets are out of range). This may be a false alarm. You can disable it when it doesn't work
+  as you expected.</td>
+</tr>
+<tr>
+  <td>kafkaConsumer.pollTimeoutMs</td>
+  <td>long</td>
+  <td>512</td>
+  <td>The timeout in milliseconds to poll data from Kafka in executors.</td>
+</tr>
+<tr>
+  <td>fetchOffset.numRetries</td>
+  <td>int</td>
+  <td>3</td>
+  <td>Number of times to retry before giving up fatch Kafka latest offsets.</td>
+</tr>
+<tr>
+  <td>fetchOffset.retryIntervalMs</td>
+  <td>long</td>
+  <td>10</td>
+  <td>milliseconds to wait before retrying to fetch Kafka offsets</td>
+</tr>
+</table>
+
+Kafka's own configurations can be set via `DataStreamReader.option` with `kafka.` prefix, e.g, 
+`stream.option("kafka.bootstrap.servers", "host:port")`. For possible kafkaParams, see 
+[Kafka consumer config docs](http://kafka.apache.org/documentation.html#newconsumerconfigs).
+
+Note that the following Kafka params cannot be set and the Kafka source will throw an exception:
+- **group.id**: Kafka source will create a unique group id for each query automatically.
+- **auto.offset.reset**: Set the source option `startingOffset` to `earliest` or `latest` to specify
+ where to start instead. Structured Streaming manages which offsets are consumed internally, rather 
+ than rely on the kafka Consumer to do it. This will ensure that no data is missed when when new 
+ topics/partitions are dynamically subscribed. Note that `startingOffset` only applies when a new
+ Streaming query is started, and that resuming will always pick up from where the query left off.
+- **key.deserializer**: Keys are always deserialized as byte arrays with ByteArrayDeserializer. Use 
+ DataFrame operations to explicitly deserialize the keys.
+- **value.deserializer**: Values are always deserialized as byte arrays with ByteArrayDeserializer. 
+ Use DataFrame operations to explicitly deserialize the values.
+- **enable.auto.commit**: Kafka source doesn't commit any offset.
+- **interceptor.classes**: Kafka source always read keys and values as byte arrays. It's not safe to
+ use ConsumerInterceptor as it may break the query.
+
+### Deploying
+
+As with any Spark applications, `spark-submit` is used to launch your application. `spark-sql-kafka-0-10_{{site.SCALA_BINARY_VERSION}}`
+and its dependencies can be directly added to `spark-submit` using `--packages`, such as,
+
+    ./bin/spark-submit --packages org.apache.spark:spark-sql-kafka-0-10_{{site.SCALA_BINARY_VERSION}}:{{site.SPARK_VERSION_SHORT}} ...
+
+See [Application Submission Guide](submitting-applications.html) for more details about submitting
+applications with external dependencies.

http://git-wip-us.apache.org/repos/asf/spark/blob/9293734d/docs/structured-streaming-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md
index 2e6df94..173fd6e 100644
--- a/docs/structured-streaming-programming-guide.md
+++ b/docs/structured-streaming-programming-guide.md
@@ -418,10 +418,15 @@ Since Spark 2.0, DataFrames and Datasets can represent static, bounded data, as
 Streaming DataFrames can be created through the `DataStreamReader` interface 
 ([Scala](api/scala/index.html#org.apache.spark.sql.streaming.DataStreamReader)/
 [Java](api/java/org/apache/spark/sql/streaming/DataStreamReader.html)/
-[Python](api/python/pyspark.sql.html#pyspark.sql.streaming.DataStreamReader) docs) returned by `SparkSession.readStream()`. Similar to the read interface for creating static DataFrame, you can specify the details of the source \u2013 data format, schema, options, etc. In Spark 2.0, there are a few built-in sources.
+[Python](api/python/pyspark.sql.html#pyspark.sql.streaming.DataStreamReader) docs) returned by `SparkSession.readStream()`. Similar to the read interface for creating static DataFrame, you can specify the details of the source \u2013 data format, schema, options, etc.
+
+#### Data Sources
+In Spark 2.0, 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.
+
   - **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. 
 
 Here are some examples.

http://git-wip-us.apache.org/repos/asf/spark/blob/9293734d/external/kafka-0-10-sql/pom.xml
----------------------------------------------------------------------
diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml
new file mode 100644
index 0000000..b96445a
--- /dev/null
+++ b/external/kafka-0-10-sql/pom.xml
@@ -0,0 +1,82 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.spark</groupId>
+    <artifactId>spark-parent_2.11</artifactId>
+    <version>2.1.0-SNAPSHOT</version>
+    <relativePath>../../pom.xml</relativePath>
+  </parent>
+
+  <groupId>org.apache.spark</groupId>
+  <artifactId>spark-sql-kafka-0-10_2.11</artifactId>
+  <properties>
+    <sbt.project.name>sql-kafka-0-10</sbt.project.name>
+  </properties>
+  <packaging>jar</packaging>
+  <name>Kafka 0.10 Source for Structured Streaming</name>
+  <url>http://spark.apache.org/</url>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-sql_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-sql_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.kafka</groupId>
+      <artifactId>kafka-clients</artifactId>
+      <version>0.10.0.1</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.kafka</groupId>
+      <artifactId>kafka_${scala.binary.version}</artifactId>
+      <version>0.10.0.1</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>net.sf.jopt-simple</groupId>
+      <artifactId>jopt-simple</artifactId>
+      <version>3.2</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.scalacheck</groupId>
+      <artifactId>scalacheck_${scala.binary.version}</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-tags_${scala.binary.version}</artifactId>
+    </dependency>
+  </dependencies>
+  <build>
+    <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+    <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
+  </build>
+</project>

http://git-wip-us.apache.org/repos/asf/spark/blob/9293734d/external/kafka-0-10-sql/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
----------------------------------------------------------------------
diff --git a/external/kafka-0-10-sql/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/external/kafka-0-10-sql/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
new file mode 100644
index 0000000..2f9e9fc
--- /dev/null
+++ b/external/kafka-0-10-sql/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
@@ -0,0 +1 @@
+org.apache.spark.sql.kafka010.KafkaSourceProvider

http://git-wip-us.apache.org/repos/asf/spark/blob/9293734d/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala
----------------------------------------------------------------------
diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala
new file mode 100644
index 0000000..3b5a965
--- /dev/null
+++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala
@@ -0,0 +1,152 @@
+/*
+ * 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.sql.kafka010
+
+import java.{util => ju}
+
+import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer}
+import org.apache.kafka.common.TopicPartition
+
+import org.apache.spark.{SparkEnv, SparkException, TaskContext}
+import org.apache.spark.internal.Logging
+
+
+/**
+ * Consumer of single topicpartition, intended for cached reuse.
+ * Underlying consumer is not threadsafe, so neither is this,
+ * but processing the same topicpartition and group id in multiple threads is usually bad anyway.
+ */
+private[kafka010] case class CachedKafkaConsumer private(
+    topicPartition: TopicPartition,
+    kafkaParams: ju.Map[String, Object]) extends Logging {
+
+  private val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG).asInstanceOf[String]
+
+  private val consumer = {
+    val c = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams)
+    val tps = new ju.ArrayList[TopicPartition]()
+    tps.add(topicPartition)
+    c.assign(tps)
+    c
+  }
+
+  /** Iterator to the already fetch data */
+  private var fetchedData = ju.Collections.emptyIterator[ConsumerRecord[Array[Byte], Array[Byte]]]
+  private var nextOffsetInFetchedData = -2L
+
+  /**
+   * Get the record for the given offset, waiting up to timeout ms if IO is necessary.
+   * Sequential forward access will use buffers, but random access will be horribly inefficient.
+   */
+  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[Array[Byte], Array[Byte]] = {
+    logDebug(s"Get $groupId $topicPartition nextOffset $nextOffsetInFetchedData requested $offset")
+    if (offset != nextOffsetInFetchedData) {
+      logInfo(s"Initial fetch for $topicPartition $offset")
+      seek(offset)
+      poll(pollTimeoutMs)
+    }
+
+    if (!fetchedData.hasNext()) { poll(pollTimeoutMs) }
+    assert(fetchedData.hasNext(),
+      s"Failed to get records for $groupId $topicPartition $offset " +
+        s"after polling for $pollTimeoutMs")
+    var record = fetchedData.next()
+
+    if (record.offset != offset) {
+      logInfo(s"Buffer miss for $groupId $topicPartition $offset")
+      seek(offset)
+      poll(pollTimeoutMs)
+      assert(fetchedData.hasNext(),
+        s"Failed to get records for $groupId $topicPartition $offset " +
+          s"after polling for $pollTimeoutMs")
+      record = fetchedData.next()
+      assert(record.offset == offset,
+        s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset")
+    }
+
+    nextOffsetInFetchedData = offset + 1
+    record
+  }
+
+  private def close(): Unit = consumer.close()
+
+  private def seek(offset: Long): Unit = {
+    logDebug(s"Seeking to $groupId $topicPartition $offset")
+    consumer.seek(topicPartition, offset)
+  }
+
+  private def poll(pollTimeoutMs: Long): Unit = {
+    val p = consumer.poll(pollTimeoutMs)
+    val r = p.records(topicPartition)
+    logDebug(s"Polled $groupId ${p.partitions()}  ${r.size}")
+    fetchedData = r.iterator
+  }
+}
+
+private[kafka010] object CachedKafkaConsumer extends Logging {
+
+  private case class CacheKey(groupId: String, topicPartition: TopicPartition)
+
+  private lazy val cache = {
+    val conf = SparkEnv.get.conf
+    val capacity = conf.getInt("spark.sql.kafkaConsumerCache.capacity", 64)
+    new ju.LinkedHashMap[CacheKey, CachedKafkaConsumer](capacity, 0.75f, true) {
+      override def removeEldestEntry(
+        entry: ju.Map.Entry[CacheKey, CachedKafkaConsumer]): Boolean = {
+        if (this.size > capacity) {
+          logWarning(s"KafkaConsumer cache hitting max capacity of $capacity, " +
+            s"removing consumer for ${entry.getKey}")
+          try {
+            entry.getValue.close()
+          } catch {
+            case e: SparkException =>
+              logError(s"Error closing earliest Kafka consumer for ${entry.getKey}", e)
+          }
+          true
+        } else {
+          false
+        }
+      }
+    }
+  }
+
+  /**
+   * Get a cached consumer for groupId, assigned to topic and partition.
+   * If matching consumer doesn't already exist, will be created using kafkaParams.
+   */
+  def getOrCreate(
+      topic: String,
+      partition: Int,
+      kafkaParams: ju.Map[String, Object]): CachedKafkaConsumer = synchronized {
+    val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG).asInstanceOf[String]
+    val topicPartition = new TopicPartition(topic, partition)
+    val key = CacheKey(groupId, topicPartition)
+
+    // If this is reattempt at running the task, then invalidate cache and start with
+    // a new consumer
+    if (TaskContext.get != null && TaskContext.get.attemptNumber > 1) {
+      cache.remove(key)
+      new CachedKafkaConsumer(topicPartition, kafkaParams)
+    } else {
+      if (!cache.containsKey(key)) {
+        cache.put(key, new CachedKafkaConsumer(topicPartition, kafkaParams))
+      }
+      cache.get(key)
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/9293734d/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
----------------------------------------------------------------------
diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
new file mode 100644
index 0000000..1be70db
--- /dev/null
+++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
@@ -0,0 +1,399 @@
+/*
+ * 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.sql.kafka010
+
+import java.{util => ju}
+
+import scala.collection.JavaConverters._
+import scala.util.control.NonFatal
+
+import org.apache.kafka.clients.consumer.{Consumer, KafkaConsumer}
+import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener
+import org.apache.kafka.common.TopicPartition
+
+import org.apache.spark.SparkContext
+import org.apache.spark.internal.Logging
+import org.apache.spark.scheduler.ExecutorCacheTaskLocation
+import org.apache.spark.sql._
+import org.apache.spark.sql.execution.streaming._
+import org.apache.spark.sql.kafka010.KafkaSource._
+import org.apache.spark.sql.types._
+import org.apache.spark.util.UninterruptibleThread
+
+/**
+ * A [[Source]] that uses Kafka's own [[KafkaConsumer]] API to reads data from Kafka. The design
+ * for this source is as follows.
+ *
+ * - The [[KafkaSourceOffset]] is the custom [[Offset]] defined for this source that contains
+ *   a map of TopicPartition -> offset. Note that this offset is 1 + (available offset). For
+ *   example if the last record in a Kafka topic "t", partition 2 is offset 5, then
+ *   KafkaSourceOffset will contain TopicPartition("t", 2) -> 6. This is done keep it consistent
+ *   with the semantics of `KafkaConsumer.position()`.
+ *
+ * - The [[ConsumerStrategy]] class defines which Kafka topics and partitions should be read
+ *   by this source. These strategies directly correspond to the different consumption options
+ *   in . This class is designed to return a configured [[KafkaConsumer]] that is used by the
+ *   [[KafkaSource]] to query for the offsets. See the docs on
+ *   [[org.apache.spark.sql.kafka010.KafkaSource.ConsumerStrategy]] for more details.
+ *
+ * - The [[KafkaSource]] written to do the following.
+ *
+ *  - As soon as the source is created, the pre-configured KafkaConsumer returned by the
+ *    [[ConsumerStrategy]] is used to query the initial offsets that this source should
+ *    start reading from. This used to create the first batch.
+ *
+ *   - `getOffset()` uses the KafkaConsumer to query the latest available offsets, which are
+ *     returned as a [[KafkaSourceOffset]].
+ *
+ *   - `getBatch()` returns a DF that reads from the 'start offset' until the 'end offset' in
+ *     for each partition. The end offset is excluded to be consistent with the semantics of
+ *     [[KafkaSourceOffset]] and `KafkaConsumer.position()`.
+ *
+ *   - The DF returned is based on [[KafkaSourceRDD]] which is constructed such that the
+ *     data from Kafka topic + partition is consistently read by the same executors across
+ *     batches, and cached KafkaConsumers in the executors can be reused efficiently. See the
+ *     docs on [[KafkaSourceRDD]] for more details.
+ *
+ * Zero data lost is not guaranteed when topics are deleted. If zero data lost is critical, the user
+ * must make sure all messages in a topic have been processed when deleting a topic.
+ *
+ * There is a known issue caused by KAFKA-1894: the query using KafkaSource maybe cannot be stopped.
+ * To avoid this issue, you should make sure stopping the query before stopping the Kafka brokers
+ * and not use wrong broker addresses.
+ */
+private[kafka010] case class KafkaSource(
+    sqlContext: SQLContext,
+    consumerStrategy: ConsumerStrategy,
+    executorKafkaParams: ju.Map[String, Object],
+    sourceOptions: Map[String, String],
+    metadataPath: String,
+    failOnDataLoss: Boolean)
+  extends Source with Logging {
+
+  private val sc = sqlContext.sparkContext
+
+  private val pollTimeoutMs = sourceOptions.getOrElse("kafkaConsumer.pollTimeoutMs", "512").toLong
+
+  private val maxOffsetFetchAttempts =
+    sourceOptions.getOrElse("fetchOffset.numRetries", "3").toInt
+
+  private val offsetFetchAttemptIntervalMs =
+    sourceOptions.getOrElse("fetchOffset.retryIntervalMs", "10").toLong
+
+  /**
+   * A KafkaConsumer used in the driver to query the latest Kafka offsets. This only queries the
+   * offsets and never commits them.
+   */
+  private val consumer = consumerStrategy.createConsumer()
+
+  /**
+   * Lazily initialize `initialPartitionOffsets` to make sure that `KafkaConsumer.poll` is only
+   * called in StreamExecutionThread. Otherwise, interrupting a thread while running
+   * `KafkaConsumer.poll` may hang forever (KAFKA-1894).
+   */
+  private lazy val initialPartitionOffsets = {
+    val metadataLog = new HDFSMetadataLog[KafkaSourceOffset](sqlContext.sparkSession, metadataPath)
+    metadataLog.get(0).getOrElse {
+      val offsets = KafkaSourceOffset(fetchPartitionOffsets(seekToEnd = false))
+      metadataLog.add(0, offsets)
+      logInfo(s"Initial offsets: $offsets")
+      offsets
+    }.partitionToOffsets
+  }
+
+  override def schema: StructType = KafkaSource.kafkaSchema
+
+  /** Returns the maximum available offset for this source. */
+  override def getOffset: Option[Offset] = {
+    // Make sure initialPartitionOffsets is initialized
+    initialPartitionOffsets
+
+    val offset = KafkaSourceOffset(fetchPartitionOffsets(seekToEnd = true))
+    logDebug(s"GetOffset: ${offset.partitionToOffsets.toSeq.map(_.toString).sorted}")
+    Some(offset)
+  }
+
+  /**
+   * Returns the data that is between the offsets
+   * [`start.get.partitionToOffsets`, `end.partitionToOffsets`), i.e. end.partitionToOffsets is
+   * exclusive.
+   */
+  override def getBatch(start: Option[Offset], end: Offset): DataFrame = {
+    // Make sure initialPartitionOffsets is initialized
+    initialPartitionOffsets
+
+    logInfo(s"GetBatch called with start = $start, end = $end")
+    val untilPartitionOffsets = KafkaSourceOffset.getPartitionOffsets(end)
+    val fromPartitionOffsets = start match {
+      case Some(prevBatchEndOffset) =>
+        KafkaSourceOffset.getPartitionOffsets(prevBatchEndOffset)
+      case None =>
+        initialPartitionOffsets
+    }
+
+    // Find the new partitions, and get their earliest offsets
+    val newPartitions = untilPartitionOffsets.keySet.diff(fromPartitionOffsets.keySet)
+    val newPartitionOffsets = if (newPartitions.nonEmpty) {
+      fetchNewPartitionEarliestOffsets(newPartitions.toSeq)
+    } else {
+      Map.empty[TopicPartition, Long]
+    }
+    if (newPartitionOffsets.keySet != newPartitions) {
+      // We cannot get from offsets for some partitions. It means they got deleted.
+      val deletedPartitions = newPartitions.diff(newPartitionOffsets.keySet)
+      reportDataLoss(
+        s"Cannot find earliest offsets of ${deletedPartitions}. Some data may have been missed")
+    }
+    logInfo(s"Partitions added: $newPartitionOffsets")
+    newPartitionOffsets.filter(_._2 != 0).foreach { case (p, o) =>
+      reportDataLoss(
+        s"Added partition $p starts from $o instead of 0. Some data may have been missed")
+    }
+
+    val deletedPartitions = fromPartitionOffsets.keySet.diff(untilPartitionOffsets.keySet)
+    if (deletedPartitions.nonEmpty) {
+      reportDataLoss(s"$deletedPartitions are gone. Some data may have been missed")
+    }
+
+    // Use the until partitions to calculate offset ranges to ignore partitions that have
+    // been deleted
+    val topicPartitions = untilPartitionOffsets.keySet.filter { tp =>
+      // Ignore partitions that we don't know the from offsets.
+      newPartitionOffsets.contains(tp) || fromPartitionOffsets.contains(tp)
+    }.toSeq
+    logDebug("TopicPartitions: " + topicPartitions.mkString(", "))
+
+    val sortedExecutors = getSortedExecutorList(sc)
+    val numExecutors = sortedExecutors.length
+    logDebug("Sorted executors: " + sortedExecutors.mkString(", "))
+
+    // Calculate offset ranges
+    val offsetRanges = topicPartitions.map { tp =>
+      val fromOffset = fromPartitionOffsets.get(tp).getOrElse {
+        newPartitionOffsets.getOrElse(tp, {
+          // This should not happen since newPartitionOffsets contains all partitions not in
+          // fromPartitionOffsets
+          throw new IllegalStateException(s"$tp doesn't have a from offset")
+        })
+      }
+      val untilOffset = untilPartitionOffsets(tp)
+      val preferredLoc = if (numExecutors > 0) {
+        // This allows cached KafkaConsumers in the executors to be re-used to read the same
+        // partition in every batch.
+        Some(sortedExecutors(floorMod(tp.hashCode, numExecutors)))
+      } else None
+      KafkaSourceRDDOffsetRange(tp, fromOffset, untilOffset, preferredLoc)
+    }.filter { range =>
+      if (range.untilOffset < range.fromOffset) {
+        reportDataLoss(s"Partition ${range.topicPartition}'s offset was changed from " +
+          s"${range.fromOffset} to ${range.untilOffset}, some data may have been missed")
+        false
+      } else {
+        true
+      }
+    }.toArray
+
+    // Create a RDD that reads from Kafka and get the (key, value) pair as byte arrays.
+    val rdd = new KafkaSourceRDD(
+      sc, executorKafkaParams, offsetRanges, pollTimeoutMs).map { cr =>
+      Row(cr.key, cr.value, cr.topic, cr.partition, cr.offset, cr.timestamp, cr.timestampType.id)
+    }
+
+    logInfo("GetBatch generating RDD of offset range: " +
+      offsetRanges.sortBy(_.topicPartition.toString).mkString(", "))
+    sqlContext.createDataFrame(rdd, schema)
+  }
+
+  /** Stop this source and free any resources it has allocated. */
+  override def stop(): Unit = synchronized {
+    consumer.close()
+  }
+
+  override def toString(): String = s"KafkaSource[$consumerStrategy]"
+
+  /**
+   * Fetch the offset of a partition, either seek to the latest offsets or use the current offsets
+   * in the consumer.
+   */
+  private def fetchPartitionOffsets(
+      seekToEnd: Boolean): Map[TopicPartition, Long] = withRetriesWithoutInterrupt {
+    // Make sure `KafkaConsumer.poll` won't be interrupted (KAFKA-1894)
+    assert(Thread.currentThread().isInstanceOf[StreamExecutionThread])
+    // Poll to get the latest assigned partitions
+    consumer.poll(0)
+    val partitions = consumer.assignment()
+    consumer.pause(partitions)
+    logDebug(s"Partitioned assigned to consumer: $partitions")
+
+    // Get the current or latest offset of each partition
+    if (seekToEnd) {
+      consumer.seekToEnd(partitions)
+      logDebug("Seeked to the end")
+    }
+    val partitionOffsets = partitions.asScala.map(p => p -> consumer.position(p)).toMap
+    logDebug(s"Got offsets for partition : $partitionOffsets")
+    partitionOffsets
+  }
+
+  /**
+   * Fetch the earliest offsets for newly discovered partitions. The return result may not contain
+   * some partitions if they are deleted.
+   */
+  private def fetchNewPartitionEarliestOffsets(
+      newPartitions: Seq[TopicPartition]): Map[TopicPartition, Long] = withRetriesWithoutInterrupt {
+    // Make sure `KafkaConsumer.poll` won't be interrupted (KAFKA-1894)
+    assert(Thread.currentThread().isInstanceOf[StreamExecutionThread])
+    // Poll to get the latest assigned partitions
+    consumer.poll(0)
+    val partitions = consumer.assignment()
+    logDebug(s"\tPartitioned assigned to consumer: $partitions")
+
+    // Get the earliest offset of each partition
+    consumer.seekToBeginning(partitions)
+    val partitionToOffsets = newPartitions.filter { p =>
+      // When deleting topics happen at the same time, some partitions may not be in `partitions`.
+      // So we need to ignore them
+      partitions.contains(p)
+    }.map(p => p -> consumer.position(p)).toMap
+    logDebug(s"Got offsets for new partitions: $partitionToOffsets")
+    partitionToOffsets
+  }
+
+  /**
+   * Helper function that does multiple retries on the a body of code that returns offsets.
+   * Retries are needed to handle transient failures. For e.g. race conditions between getting
+   * assignment and getting position while topics/partitions are deleted can cause NPEs.
+   *
+   * This method also makes sure `body` won't be interrupted to workaround a potential issue in
+   * `KafkaConsumer.poll`. (KAFKA-1894)
+   */
+  private def withRetriesWithoutInterrupt(
+      body: => Map[TopicPartition, Long]): Map[TopicPartition, Long] = {
+    synchronized {
+      var result: Option[Map[TopicPartition, Long]] = None
+      var attempt = 1
+      var lastException: Throwable = null
+      while (result.isEmpty && attempt <= maxOffsetFetchAttempts
+        && !Thread.currentThread().isInterrupted) {
+        Thread.currentThread match {
+          case ut: UninterruptibleThread =>
+            // "KafkaConsumer.poll" may hang forever if the thread is interrupted (E.g., the query
+            // is stopped)(KAFKA-1894). Hence, we just make sure we don't interrupt it.
+            //
+            // If the broker addresses are wrong, or Kafka cluster is down, "KafkaConsumer.poll" may
+            // hang forever as well. This cannot be resolved in KafkaSource until Kafka fixes the
+            // issue.
+            ut.runUninterruptibly {
+              try {
+                result = Some(body)
+              } catch {
+                case NonFatal(e) =>
+                  lastException = e
+                  logWarning(s"Error in attempt $attempt getting Kafka offsets: ", e)
+                  attempt += 1
+                  Thread.sleep(offsetFetchAttemptIntervalMs)
+              }
+            }
+          case _ =>
+            throw new IllegalStateException(
+              "Kafka APIs must be executed on a o.a.spark.util.UninterruptibleThread")
+        }
+      }
+      if (Thread.interrupted()) {
+        throw new InterruptedException()
+      }
+      if (result.isEmpty) {
+        assert(attempt > maxOffsetFetchAttempts)
+        assert(lastException != null)
+        throw lastException
+      }
+      result.get
+    }
+  }
+
+  /**
+   * If `failOnDataLoss` is true, this method will throw an `IllegalStateException`.
+   * Otherwise, just log a warning.
+   */
+  private def reportDataLoss(message: String): Unit = {
+    if (failOnDataLoss) {
+      throw new IllegalStateException(message +
+        ". Set the source option 'failOnDataLoss' to 'false' if you want to ignore these checks.")
+    } else {
+      logWarning(message)
+    }
+  }
+}
+
+
+/** Companion object for the [[KafkaSource]]. */
+private[kafka010] object KafkaSource {
+
+  def kafkaSchema: StructType = StructType(Seq(
+    StructField("key", BinaryType),
+    StructField("value", BinaryType),
+    StructField("topic", StringType),
+    StructField("partition", IntegerType),
+    StructField("offset", LongType),
+    StructField("timestamp", LongType),
+    StructField("timestampType", IntegerType)
+  ))
+
+  sealed trait ConsumerStrategy {
+    def createConsumer(): Consumer[Array[Byte], Array[Byte]]
+  }
+
+  case class SubscribeStrategy(topics: Seq[String], kafkaParams: ju.Map[String, Object])
+    extends ConsumerStrategy {
+    override def createConsumer(): Consumer[Array[Byte], Array[Byte]] = {
+      val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams)
+      consumer.subscribe(topics.asJava)
+      consumer
+    }
+
+    override def toString: String = s"Subscribe[${topics.mkString(", ")}]"
+  }
+
+  case class SubscribePatternStrategy(
+    topicPattern: String, kafkaParams: ju.Map[String, Object])
+    extends ConsumerStrategy {
+    override def createConsumer(): Consumer[Array[Byte], Array[Byte]] = {
+      val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams)
+      consumer.subscribe(
+        ju.regex.Pattern.compile(topicPattern),
+        new NoOpConsumerRebalanceListener())
+      consumer
+    }
+
+    override def toString: String = s"SubscribePattern[$topicPattern]"
+  }
+
+  private def getSortedExecutorList(sc: SparkContext): Array[String] = {
+    val bm = sc.env.blockManager
+    bm.master.getPeers(bm.blockManagerId).toArray
+      .map(x => ExecutorCacheTaskLocation(x.host, x.executorId))
+      .sortWith(compare)
+      .map(_.toString)
+  }
+
+  private def compare(a: ExecutorCacheTaskLocation, b: ExecutorCacheTaskLocation): Boolean = {
+    if (a.host == b.host) { a.executorId > b.executorId } else { a.host > b.host }
+  }
+
+  private def floorMod(a: Long, b: Int): Int = ((a % b).toInt + b) % b
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/9293734d/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala
----------------------------------------------------------------------
diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala
new file mode 100644
index 0000000..b5ade98
--- /dev/null
+++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala
@@ -0,0 +1,54 @@
+/*
+ * 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.sql.kafka010
+
+import org.apache.kafka.common.TopicPartition
+
+import org.apache.spark.sql.execution.streaming.Offset
+
+/**
+ * An [[Offset]] for the [[KafkaSource]]. This one tracks all partitions of subscribed topics and
+ * their offsets.
+ */
+private[kafka010]
+case class KafkaSourceOffset(partitionToOffsets: Map[TopicPartition, Long]) extends Offset {
+  override def toString(): String = {
+    partitionToOffsets.toSeq.sortBy(_._1.toString).mkString("[", ", ", "]")
+  }
+}
+
+/** Companion object of the [[KafkaSourceOffset]] */
+private[kafka010] object KafkaSourceOffset {
+
+  def getPartitionOffsets(offset: Offset): Map[TopicPartition, Long] = {
+    offset match {
+      case o: KafkaSourceOffset => o.partitionToOffsets
+      case _ =>
+        throw new IllegalArgumentException(
+          s"Invalid conversion from offset of ${offset.getClass} to KafkaSourceOffset")
+    }
+  }
+
+  /**
+   * Returns [[KafkaSourceOffset]] from a variable sequence of (topic, partitionId, offset)
+   * tuples.
+   */
+  def apply(offsetTuples: (String, Int, Long)*): KafkaSourceOffset = {
+    KafkaSourceOffset(offsetTuples.map { case(t, p, o) => (new TopicPartition(t, p), o) }.toMap)
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/9293734d/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala
----------------------------------------------------------------------
diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala
new file mode 100644
index 0000000..1b0a2fe
--- /dev/null
+++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala
@@ -0,0 +1,282 @@
+/*
+ * 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.sql.kafka010
+
+import java.{util => ju}
+import java.util.UUID
+
+import scala.collection.JavaConverters._
+
+import org.apache.kafka.clients.consumer.ConsumerConfig
+import org.apache.kafka.common.serialization.ByteArrayDeserializer
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.SQLContext
+import org.apache.spark.sql.execution.streaming.Source
+import org.apache.spark.sql.kafka010.KafkaSource._
+import org.apache.spark.sql.sources.{DataSourceRegister, StreamSourceProvider}
+import org.apache.spark.sql.types.StructType
+
+/**
+ * The provider class for the [[KafkaSource]]. This provider is designed such that it throws
+ * IllegalArgumentException when the Kafka Dataset is created, so that it can catch
+ * missing options even before the query is started.
+ */
+private[kafka010] class KafkaSourceProvider extends StreamSourceProvider
+  with DataSourceRegister with Logging {
+
+  import KafkaSourceProvider._
+
+  /**
+   * Returns the name and schema of the source. In addition, it also verifies whether the options
+   * are correct and sufficient to create the [[KafkaSource]] when the query is started.
+   */
+  override def sourceSchema(
+      sqlContext: SQLContext,
+      schema: Option[StructType],
+      providerName: String,
+      parameters: Map[String, String]): (String, StructType) = {
+    require(schema.isEmpty, "Kafka source has a fixed schema and cannot be set with a custom one")
+    validateOptions(parameters)
+    ("kafka", KafkaSource.kafkaSchema)
+  }
+
+  override def createSource(
+      sqlContext: SQLContext,
+      metadataPath: String,
+      schema: Option[StructType],
+      providerName: String,
+      parameters: Map[String, String]): Source = {
+      validateOptions(parameters)
+    val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase, v) }
+    val specifiedKafkaParams =
+      parameters
+        .keySet
+        .filter(_.toLowerCase.startsWith("kafka."))
+        .map { k => k.drop(6).toString -> parameters(k) }
+        .toMap
+
+    val deserClassName = classOf[ByteArrayDeserializer].getName
+    // Each running query should use its own group id. Otherwise, the query may be only assigned
+    // partial data since Kafka will assign partitions to multiple consumers having the same group
+    // id. Hence, we should generate a unique id for each query.
+    val uniqueGroupId = s"spark-kafka-source-${UUID.randomUUID}-${metadataPath.hashCode}"
+
+    val autoOffsetResetValue = caseInsensitiveParams.get(STARTING_OFFSET_OPTION_KEY) match {
+      case Some(value) => value.trim()  // same values as those supported by auto.offset.reset
+      case None => "latest"
+    }
+
+    val kafkaParamsForStrategy =
+      ConfigUpdater("source", specifiedKafkaParams)
+        .set(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, deserClassName)
+        .set(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, deserClassName)
+
+        // So that consumers in Kafka source do not mess with any existing group id
+        .set(ConsumerConfig.GROUP_ID_CONFIG, s"$uniqueGroupId-driver")
+
+        // So that consumers can start from earliest or latest
+        .set(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, autoOffsetResetValue)
+
+        // So that consumers in the driver does not commit offsets unnecessarily
+        .set(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false")
+
+        // So that the driver does not pull too much data
+        .set(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, new java.lang.Integer(1))
+
+        // If buffer config is not set, set it to reasonable value to work around
+        // buffer issues (see KAFKA-3135)
+        .setIfUnset(ConsumerConfig.RECEIVE_BUFFER_CONFIG, 65536: java.lang.Integer)
+        .build()
+
+    val kafkaParamsForExecutors =
+      ConfigUpdater("executor", specifiedKafkaParams)
+        .set(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, deserClassName)
+        .set(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, deserClassName)
+
+        // Make sure executors do only what the driver tells them.
+        .set(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none")
+
+        // So that consumers in executors do not mess with any existing group id
+        .set(ConsumerConfig.GROUP_ID_CONFIG, s"$uniqueGroupId-executor")
+
+        // So that consumers in executors does not commit offsets unnecessarily
+        .set(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false")
+
+        // If buffer config is not set, set it to reasonable value to work around
+        // buffer issues (see KAFKA-3135)
+        .setIfUnset(ConsumerConfig.RECEIVE_BUFFER_CONFIG, 65536: java.lang.Integer)
+        .build()
+
+    val strategy = caseInsensitiveParams.find(x => STRATEGY_OPTION_KEYS.contains(x._1)).get match {
+      case ("subscribe", value) =>
+        SubscribeStrategy(
+          value.split(",").map(_.trim()).filter(_.nonEmpty),
+          kafkaParamsForStrategy)
+      case ("subscribepattern", value) =>
+        SubscribePatternStrategy(
+          value.trim(),
+          kafkaParamsForStrategy)
+      case _ =>
+        // Should never reach here as we are already matching on
+        // matched strategy names
+        throw new IllegalArgumentException("Unknown option")
+    }
+
+    val failOnDataLoss =
+      caseInsensitiveParams.getOrElse(FAIL_ON_DATA_LOSS_OPTION_KEY, "true").toBoolean
+
+    new KafkaSource(
+      sqlContext,
+      strategy,
+      kafkaParamsForExecutors,
+      parameters,
+      metadataPath,
+      failOnDataLoss)
+  }
+
+  private def validateOptions(parameters: Map[String, String]): Unit = {
+
+    // Validate source options
+
+    val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase, v) }
+    val specifiedStrategies =
+      caseInsensitiveParams.filter { case (k, _) => STRATEGY_OPTION_KEYS.contains(k) }.toSeq
+    if (specifiedStrategies.isEmpty) {
+      throw new IllegalArgumentException(
+        "One of the following options must be specified for Kafka source: "
+          + STRATEGY_OPTION_KEYS.mkString(", ") + ". See the docs for more details.")
+    } else if (specifiedStrategies.size > 1) {
+      throw new IllegalArgumentException(
+        "Only one of the following options can be specified for Kafka source: "
+          + STRATEGY_OPTION_KEYS.mkString(", ") + ". See the docs for more details.")
+    }
+
+    val strategy = caseInsensitiveParams.find(x => STRATEGY_OPTION_KEYS.contains(x._1)).get match {
+      case ("subscribe", value) =>
+        val topics = value.split(",").map(_.trim).filter(_.nonEmpty)
+        if (topics.isEmpty) {
+          throw new IllegalArgumentException(
+            "No topics to subscribe to as specified value for option " +
+              s"'subscribe' is '$value'")
+        }
+      case ("subscribepattern", value) =>
+        val pattern = caseInsensitiveParams("subscribepattern").trim()
+        if (pattern.isEmpty) {
+          throw new IllegalArgumentException(
+            "Pattern to subscribe is empty as specified value for option " +
+              s"'subscribePattern' is '$value'")
+        }
+      case _ =>
+        // Should never reach here as we are already matching on
+        // matched strategy names
+        throw new IllegalArgumentException("Unknown option")
+    }
+
+    caseInsensitiveParams.get(STARTING_OFFSET_OPTION_KEY) match {
+      case Some(pos) if !STARTING_OFFSET_OPTION_VALUES.contains(pos.trim.toLowerCase) =>
+        throw new IllegalArgumentException(
+          s"Illegal value '$pos' for option '$STARTING_OFFSET_OPTION_KEY', " +
+            s"acceptable values are: ${STARTING_OFFSET_OPTION_VALUES.mkString(", ")}")
+      case _ =>
+    }
+
+    // Validate user-specified Kafka options
+
+    if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.GROUP_ID_CONFIG}")) {
+      throw new IllegalArgumentException(
+        s"Kafka option '${ConsumerConfig.GROUP_ID_CONFIG}' is not supported as " +
+          s"user-specified consumer groups is not used to track offsets.")
+    }
+
+    if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG}")) {
+      throw new IllegalArgumentException(
+        s"""
+           |Kafka option '${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG}' is not supported.
+           |Instead set the source option '$STARTING_OFFSET_OPTION_KEY' to 'earliest' or 'latest' to
+           |specify where to start. Structured Streaming manages which offsets are consumed
+           |internally, rather than relying on the kafkaConsumer to do it. This will ensure that no
+           |data is missed when when new topics/partitions are dynamically subscribed. Note that
+           |'$STARTING_OFFSET_OPTION_KEY' only applies when a new Streaming query is started, and
+           |that resuming will always pick up from where the query left off. See the docs for more
+           |details.
+         """.stripMargin)
+    }
+
+    if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG}")) {
+      throw new IllegalArgumentException(
+        s"Kafka option '${ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG}' is not supported as keys "
+          + "are deserialized as byte arrays with ByteArrayDeserializer. Use DataFrame operations "
+          + "to explicitly deserialize the keys.")
+    }
+
+    if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG}"))
+    {
+      throw new IllegalArgumentException(
+        s"Kafka option '${ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG}' is not supported as "
+          + "value are deserialized as byte arrays with ByteArrayDeserializer. Use DataFrame "
+          + "operations to explicitly deserialize the values.")
+    }
+
+    val otherUnsupportedConfigs = Seq(
+      ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, // committing correctly requires new APIs in Source
+      ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG) // interceptors can modify payload, so not safe
+
+    otherUnsupportedConfigs.foreach { c =>
+      if (caseInsensitiveParams.contains(s"kafka.$c")) {
+        throw new IllegalArgumentException(s"Kafka option '$c' is not supported")
+      }
+    }
+
+    if (!caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG}")) {
+      throw new IllegalArgumentException(
+        s"Option 'kafka.${ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG}' must be specified for " +
+          s"configuring Kafka consumer")
+    }
+  }
+
+  override def shortName(): String = "kafka"
+
+  /** Class to conveniently update Kafka config params, while logging the changes */
+  private case class ConfigUpdater(module: String, kafkaParams: Map[String, String]) {
+    private val map = new ju.HashMap[String, Object](kafkaParams.asJava)
+
+    def set(key: String, value: Object): this.type = {
+      map.put(key, value)
+      logInfo(s"$module: Set $key to $value, earlier value: ${kafkaParams.get(key).getOrElse("")}")
+      this
+    }
+
+    def setIfUnset(key: String, value: Object): ConfigUpdater = {
+      if (!map.containsKey(key)) {
+        map.put(key, value)
+        logInfo(s"$module: Set $key to $value")
+      }
+      this
+    }
+
+    def build(): ju.Map[String, Object] = map
+  }
+}
+
+private[kafka010] object KafkaSourceProvider {
+  private val STRATEGY_OPTION_KEYS = Set("subscribe", "subscribepattern")
+  private val STARTING_OFFSET_OPTION_KEY = "startingoffset"
+  private val STARTING_OFFSET_OPTION_VALUES = Set("earliest", "latest")
+  private val FAIL_ON_DATA_LOSS_OPTION_KEY = "failondataloss"
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/9293734d/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala
----------------------------------------------------------------------
diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala
new file mode 100644
index 0000000..496af7e
--- /dev/null
+++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala
@@ -0,0 +1,148 @@
+/*
+ * 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.sql.kafka010
+
+import java.{util => ju}
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.kafka.clients.consumer.ConsumerRecord
+import org.apache.kafka.common.TopicPartition
+
+import org.apache.spark.{Partition, SparkContext, TaskContext}
+import org.apache.spark.partial.{BoundedDouble, PartialResult}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.storage.StorageLevel
+
+
+/** Offset range that one partition of the KafkaSourceRDD has to read */
+private[kafka010] case class KafkaSourceRDDOffsetRange(
+    topicPartition: TopicPartition,
+    fromOffset: Long,
+    untilOffset: Long,
+    preferredLoc: Option[String]) {
+  def topic: String = topicPartition.topic
+  def partition: Int = topicPartition.partition
+  def size: Long = untilOffset - fromOffset
+}
+
+
+/** Partition of the KafkaSourceRDD */
+private[kafka010] case class KafkaSourceRDDPartition(
+  index: Int, offsetRange: KafkaSourceRDDOffsetRange) extends Partition
+
+
+/**
+ * An RDD that reads data from Kafka based on offset ranges across multiple partitions.
+ * Additionally, it allows preferred locations to be set for each topic + partition, so that
+ * the [[KafkaSource]] can ensure the same executor always reads the same topic + partition
+ * and cached KafkaConsuemrs (see [[CachedKafkaConsumer]] can be used read data efficiently.
+ *
+ * @param sc the [[SparkContext]]
+ * @param executorKafkaParams Kafka configuration for creating KafkaConsumer on the executors
+ * @param offsetRanges Offset ranges that define the Kafka data belonging to this RDD
+ */
+private[kafka010] class KafkaSourceRDD(
+    sc: SparkContext,
+    executorKafkaParams: ju.Map[String, Object],
+    offsetRanges: Seq[KafkaSourceRDDOffsetRange],
+    pollTimeoutMs: Long)
+  extends RDD[ConsumerRecord[Array[Byte], Array[Byte]]](sc, Nil) {
+
+  override def persist(newLevel: StorageLevel): this.type = {
+    logError("Kafka ConsumerRecord is not serializable. " +
+      "Use .map to extract fields before calling .persist or .window")
+    super.persist(newLevel)
+  }
+
+  override def getPartitions: Array[Partition] = {
+    offsetRanges.zipWithIndex.map { case (o, i) => new KafkaSourceRDDPartition(i, o) }.toArray
+  }
+
+  override def count(): Long = offsetRanges.map(_.size).sum
+
+  override def countApprox(timeout: Long, confidence: Double): PartialResult[BoundedDouble] = {
+    val c = count
+    new PartialResult(new BoundedDouble(c, 1.0, c, c), true)
+  }
+
+  override def isEmpty(): Boolean = count == 0L
+
+  override def take(num: Int): Array[ConsumerRecord[Array[Byte], Array[Byte]]] = {
+    val nonEmptyPartitions =
+      this.partitions.map(_.asInstanceOf[KafkaSourceRDDPartition]).filter(_.offsetRange.size > 0)
+
+    if (num < 1 || nonEmptyPartitions.isEmpty) {
+      return new Array[ConsumerRecord[Array[Byte], Array[Byte]]](0)
+    }
+
+    // Determine in advance how many messages need to be taken from each partition
+    val parts = nonEmptyPartitions.foldLeft(Map[Int, Int]()) { (result, part) =>
+      val remain = num - result.values.sum
+      if (remain > 0) {
+        val taken = Math.min(remain, part.offsetRange.size)
+        result + (part.index -> taken.toInt)
+      } else {
+        result
+      }
+    }
+
+    val buf = new ArrayBuffer[ConsumerRecord[Array[Byte], Array[Byte]]]
+    val res = context.runJob(
+      this,
+      (tc: TaskContext, it: Iterator[ConsumerRecord[Array[Byte], Array[Byte]]]) =>
+      it.take(parts(tc.partitionId)).toArray, parts.keys.toArray
+    )
+    res.foreach(buf ++= _)
+    buf.toArray
+  }
+
+  override def compute(
+      thePart: Partition,
+      context: TaskContext): Iterator[ConsumerRecord[Array[Byte], Array[Byte]]] = {
+    val range = thePart.asInstanceOf[KafkaSourceRDDPartition].offsetRange
+    assert(
+      range.fromOffset <= range.untilOffset,
+      s"Beginning offset ${range.fromOffset} is after the ending offset ${range.untilOffset} " +
+        s"for topic ${range.topic} partition ${range.partition}. " +
+        "You either provided an invalid fromOffset, or the Kafka topic has been damaged")
+    if (range.fromOffset == range.untilOffset) {
+      logInfo(s"Beginning offset ${range.fromOffset} is the same as ending offset " +
+        s"skipping ${range.topic} ${range.partition}")
+      Iterator.empty
+
+    } else {
+
+      val consumer = CachedKafkaConsumer.getOrCreate(
+        range.topic, range.partition, executorKafkaParams)
+      var requestOffset = range.fromOffset
+
+      logDebug(s"Creating iterator for $range")
+
+      new Iterator[ConsumerRecord[Array[Byte], Array[Byte]]]() {
+        override def hasNext(): Boolean = requestOffset < range.untilOffset
+        override def next(): ConsumerRecord[Array[Byte], Array[Byte]] = {
+          assert(hasNext(), "Can't call next() once untilOffset has been reached")
+          val r = consumer.get(requestOffset, pollTimeoutMs)
+          requestOffset += 1
+          r
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/9293734d/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/package-info.java
----------------------------------------------------------------------
diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/package-info.java b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/package-info.java
new file mode 100644
index 0000000..596f775
--- /dev/null
+++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+
+/**
+ * Structured Streaming Data Source for Kafka 0.10
+ */
+package org.apache.spark.sql.kafka010;

http://git-wip-us.apache.org/repos/asf/spark/blob/9293734d/external/kafka-0-10-sql/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/external/kafka-0-10-sql/src/test/resources/log4j.properties b/external/kafka-0-10-sql/src/test/resources/log4j.properties
new file mode 100644
index 0000000..75e3b53
--- /dev/null
+++ b/external/kafka-0-10-sql/src/test/resources/log4j.properties
@@ -0,0 +1,28 @@
+#
+# 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.
+#
+
+# Set everything to be logged to the file target/unit-tests.log
+log4j.rootCategory=INFO, file
+log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file.append=true
+log4j.appender.file.file=target/unit-tests.log
+log4j.appender.file.layout=org.apache.log4j.PatternLayout
+log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n
+
+# Ignore messages below warning level from Jetty, because it's a bit verbose
+log4j.logger.org.spark-project.jetty=WARN
+

http://git-wip-us.apache.org/repos/asf/spark/blob/9293734d/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala
----------------------------------------------------------------------
diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala
new file mode 100644
index 0000000..7056a41
--- /dev/null
+++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala
@@ -0,0 +1,39 @@
+/*
+ * 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.sql.kafka010
+
+import org.apache.spark.sql.streaming.OffsetSuite
+
+class KafkaSourceOffsetSuite extends OffsetSuite {
+
+  compare(
+    one = KafkaSourceOffset(("t", 0, 1L)),
+    two = KafkaSourceOffset(("t", 0, 2L)))
+
+  compare(
+    one = KafkaSourceOffset(("t", 0, 1L), ("t", 1, 0L)),
+    two = KafkaSourceOffset(("t", 0, 2L), ("t", 1, 1L)))
+
+  compare(
+    one = KafkaSourceOffset(("t", 0, 1L), ("T", 0, 0L)),
+    two = KafkaSourceOffset(("t", 0, 2L), ("T", 0, 1L)))
+
+  compare(
+    one = KafkaSourceOffset(("t", 0, 1L)),
+    two = KafkaSourceOffset(("t", 0, 2L), ("t", 1, 1L)))
+}


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