[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-05 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/spark/pull/15102


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-05 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r82030620
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 ---
@@ -0,0 +1,396 @@
+/*
+ * 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

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-05 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r82030589
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 ---
@@ -0,0 +1,396 @@
+/*
+ * 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

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-05 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r82030575
  
--- Diff: 
external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala
 ---
@@ -0,0 +1,422 @@
+/*
+ * 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.concurrent.atomic.AtomicInteger
+
+import scala.util.Random
+
+import org.apache.kafka.clients.producer.RecordMetadata
+import org.scalatest.BeforeAndAfter
+import org.scalatest.time.SpanSugar._
+
+import org.apache.spark.sql.execution.streaming._
+import org.apache.spark.sql.streaming.StreamTest
+import org.apache.spark.sql.test.SharedSQLContext
+
+
+abstract class KafkaSourceTest extends StreamTest with SharedSQLContext {
+
+  protected var testUtils: KafkaTestUtils = _
+
+  override val streamingTimeout = 30.seconds
+
+  override def beforeAll(): Unit = {
+super.beforeAll()
+testUtils = new KafkaTestUtils
+testUtils.setup()
+  }
+
+  override def afterAll(): Unit = {
+if (testUtils != null) {
+  testUtils.teardown()
+  testUtils = null
+  super.afterAll()
+}
+  }
+
+  protected def makeSureGetOffsetCalled = AssertOnQuery { q =>
+// Because KafkaSource's initialPartitionOffsets is set lazily, we 
need to make sure
+// its "getOffset" is called before pushing any data. Otherwise, 
because of the race contion,
+// we don't know which data should be fetched when `startingOffset` is 
latest.
+q.processAllAvailable()
+true
+  }
+
+  /**
+   * Add data to Kafka.
+   *
+   * `topicAction` can be used to run actions for each topic before 
inserting data.
+   */
+  case class AddKafkaData(topics: Set[String], data: Int*)
+(implicit ensureDataInMultiplePartition: Boolean = false,
+  concurrent: Boolean = false,
+  message: String = "",
+  topicAction: (String, Option[Int]) => Unit = (_, _) => {}) extends 
AddData {
+
+override def addData(query: Option[StreamExecution]): (Source, Offset) 
= {
+  if (query.get.isActive) {
+// Make sure no Spark job is running when deleting a topic
+query.get.processAllAvailable()
+  }
+
+  val existingTopics = testUtils.getAllTopicsAndPartitionSize().toMap
+  val newTopics = topics.diff(existingTopics.keySet)
+  for (newTopic <- newTopics) {
+topicAction(newTopic, None)
+  }
+  for (existingTopicPartitions <- existingTopics) {
+topicAction(existingTopicPartitions._1, 
Some(existingTopicPartitions._2))
+  }
+
+  // Read all topics again in case some topics are delete.
+  val allTopics = testUtils.getAllTopicsAndPartitionSize().toMap.keys
+  require(
+query.nonEmpty,
+"Cannot add data when there is no query for finding the active 
kafka source")
+
+  val sources = query.get.logicalPlan.collect {
+case StreamingExecutionRelation(source, _) if 
source.isInstanceOf[KafkaSource] =>
+  source.asInstanceOf[KafkaSource]
+  }
+  if (sources.isEmpty) {
+throw new Exception(
+  "Could not find Kafka source in the StreamExecution logical plan 
to add data to")
+  } else if (sources.size > 1) {
+throw new Exception(
+  "Could not select the Kafka source in the StreamExecution 
logical plan as there" +
+"are multiple Kafka sources:\n\t" + sources.mkString("\n\t"))
+  }
+  val kafkaSource = sources.head
+  val topic = topics.toSeq(Random.nextInt(topics.size))
+  val sentMetadata = testUtils.sendMessages(topic, data.map { 
_.toString }.toArray)
+
+  def metadataToStr(m: (String, RecordMetadata)): String = {
+s"Sent ${m._1} to partition ${m._2.partition()}, offset 

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-05 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r82030607
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 ---
@@ -0,0 +1,396 @@
+/*
+ * 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

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-05 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r82030626
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 ---
@@ -0,0 +1,396 @@
+/*
+ * 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

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-05 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r82030551
  
--- Diff: 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
+
+
+
+
+// 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)]
+
+
+
+
+// Subscribe to 1 topic
+Dataset 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 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 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)")
+
+
+
+
+# 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)")
+
+
+
+
+Each row in the source has the following schema:
+
+ColumnType
+
+  key
+  binary
+
+
+  value
+  binary
+
+
+  topic
+  string
+
+
+  partition
+  int
+
+
+  offset
+  long
+
+
+  timestamp
+  long
+
+
+  timestampType
+  int
+
+
+
+The following options should be set for the Kafka source.
+
+
+Optionvaluemeaning
+
+  subscribe
+  A comma-separated list of topics
+  The topic list to subscribe. Only one of "subscribe" and 
"subscribePattern" options can be
+  specified for Kafka source.
+
+
+  subscribePattern
+  Java regex string
+  The pattern used to subscribe the topic. Only one of "subscribe" and 
"subscribePattern"
+  options can be specified for Kafka source.

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-05 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r82030561
  
--- Diff: 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
+
+
+
+
+// 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)]
+
+
+
+
+// Subscribe to 1 topic
+Dataset 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 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 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)")
+
+
+
+
+# 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)")
+
+
+
+
+Each row in the source has the following schema:
+
+ColumnType
+
+  key
+  binary
+
+
+  value
+  binary
+
+
+  topic
+  string
+
+
+  partition
+  int
+
+
+  offset
+  long
+
+
+  timestamp
+  long
+
+
+  timestampType
+  int
+
+
+
+The following options should be set for the Kafka source.
--- End diff --

done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, 

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-05 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r82030538
  
--- Diff: 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
+
+
+
+
+// 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)]
+
+
+
+
+// Subscribe to 1 topic
+Dataset 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 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 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)")
+
+
+
+
+# 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)")
+
+
+
+
+Each row in the source has the following schema:
+
+ColumnType
+
+  key
+  binary
+
+
+  value
+  binary
+
+
+  topic
+  string
+
+
+  partition
+  int
+
+
+  offset
+  long
+
+
+  timestamp
+  long
+
+
+  timestampType
+  int
+
+
+
+The following options should be set for the Kafka source.
+
+
+Optionvaluemeaning
+
+  subscribe
+  A comma-separated list of topics
+  The topic list to subscribe. Only one of "subscribe" and 
"subscribePattern" options can be
+  specified for Kafka source.
+
+
+  subscribePattern
+  Java regex string
+  The pattern used to subscribe the topic. Only one of "subscribe" and 
"subscribePattern"
+  options can be specified for Kafka source.

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-05 Thread tdas
Github user tdas commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81930531
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 ---
@@ -0,0 +1,396 @@
+/*
+ * 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
   

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-05 Thread tdas
Github user tdas commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81933155
  
--- Diff: 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
+
+
+
+
+// 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)]
+
+
+
+
+// Subscribe to 1 topic
+Dataset 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 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 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)")
+
+
+
+
+# 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)")
+
+
+
+
+Each row in the source has the following schema:
+
+ColumnType
+
+  key
+  binary
+
+
+  value
+  binary
+
+
+  topic
+  string
+
+
+  partition
+  int
+
+
+  offset
+  long
+
+
+  timestamp
+  long
+
+
+  timestampType
+  int
+
+
+
+The following options should be set for the Kafka source.
+
+
+Optionvaluemeaning
+
+  subscribe
+  A comma-separated list of topics
+  The topic list to subscribe. Only one of "subscribe" and 
"subscribePattern" options can be
+  specified for Kafka source.
+
+
+  subscribePattern
+  Java regex string
+  The pattern used to subscribe the topic. Only one of "subscribe" and 
"subscribePattern"
+  options can be specified for Kafka source.
+
  

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-05 Thread tdas
Github user tdas commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81933327
  
--- Diff: 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
+
+
+
+
+// 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)]
+
+
+
+
+// Subscribe to 1 topic
+Dataset 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 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 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)")
+
+
+
+
+# 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)")
+
+
+
+
+Each row in the source has the following schema:
+
+ColumnType
+
+  key
+  binary
+
+
+  value
+  binary
+
+
+  topic
+  string
+
+
+  partition
+  int
+
+
+  offset
+  long
+
+
+  timestamp
+  long
+
+
+  timestampType
+  int
+
+
+
+The following options should be set for the Kafka source.
+
+
+Optionvaluemeaning
+
+  subscribe
+  A comma-separated list of topics
+  The topic list to subscribe. Only one of "subscribe" and 
"subscribePattern" options can be
+  specified for Kafka source.
+
+
+  subscribePattern
+  Java regex string
+  The pattern used to subscribe the topic. Only one of "subscribe" and 
"subscribePattern"
+  options can be specified for Kafka source.
+
  

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-05 Thread tdas
Github user tdas commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81932096
  
--- Diff: 
external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala
 ---
@@ -0,0 +1,422 @@
+/*
+ * 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.concurrent.atomic.AtomicInteger
+
+import scala.util.Random
+
+import org.apache.kafka.clients.producer.RecordMetadata
+import org.scalatest.BeforeAndAfter
+import org.scalatest.time.SpanSugar._
+
+import org.apache.spark.sql.execution.streaming._
+import org.apache.spark.sql.streaming.StreamTest
+import org.apache.spark.sql.test.SharedSQLContext
+
+
+abstract class KafkaSourceTest extends StreamTest with SharedSQLContext {
+
+  protected var testUtils: KafkaTestUtils = _
+
+  override val streamingTimeout = 30.seconds
+
+  override def beforeAll(): Unit = {
+super.beforeAll()
+testUtils = new KafkaTestUtils
+testUtils.setup()
+  }
+
+  override def afterAll(): Unit = {
+if (testUtils != null) {
+  testUtils.teardown()
+  testUtils = null
+  super.afterAll()
+}
+  }
+
+  protected def makeSureGetOffsetCalled = AssertOnQuery { q =>
+// Because KafkaSource's initialPartitionOffsets is set lazily, we 
need to make sure
+// its "getOffset" is called before pushing any data. Otherwise, 
because of the race contion,
+// we don't know which data should be fetched when `startingOffset` is 
latest.
+q.processAllAvailable()
+true
+  }
+
+  /**
+   * Add data to Kafka.
+   *
+   * `topicAction` can be used to run actions for each topic before 
inserting data.
+   */
+  case class AddKafkaData(topics: Set[String], data: Int*)
+(implicit ensureDataInMultiplePartition: Boolean = false,
+  concurrent: Boolean = false,
+  message: String = "",
+  topicAction: (String, Option[Int]) => Unit = (_, _) => {}) extends 
AddData {
+
+override def addData(query: Option[StreamExecution]): (Source, Offset) 
= {
+  if (query.get.isActive) {
+// Make sure no Spark job is running when deleting a topic
+query.get.processAllAvailable()
+  }
+
+  val existingTopics = testUtils.getAllTopicsAndPartitionSize().toMap
+  val newTopics = topics.diff(existingTopics.keySet)
+  for (newTopic <- newTopics) {
+topicAction(newTopic, None)
+  }
+  for (existingTopicPartitions <- existingTopics) {
+topicAction(existingTopicPartitions._1, 
Some(existingTopicPartitions._2))
+  }
+
+  // Read all topics again in case some topics are delete.
+  val allTopics = testUtils.getAllTopicsAndPartitionSize().toMap.keys
+  require(
+query.nonEmpty,
+"Cannot add data when there is no query for finding the active 
kafka source")
+
+  val sources = query.get.logicalPlan.collect {
+case StreamingExecutionRelation(source, _) if 
source.isInstanceOf[KafkaSource] =>
+  source.asInstanceOf[KafkaSource]
+  }
+  if (sources.isEmpty) {
+throw new Exception(
+  "Could not find Kafka source in the StreamExecution logical plan 
to add data to")
+  } else if (sources.size > 1) {
+throw new Exception(
+  "Could not select the Kafka source in the StreamExecution 
logical plan as there" +
+"are multiple Kafka sources:\n\t" + sources.mkString("\n\t"))
+  }
+  val kafkaSource = sources.head
+  val topic = topics.toSeq(Random.nextInt(topics.size))
+  val sentMetadata = testUtils.sendMessages(topic, data.map { 
_.toString }.toArray)
+
+  def metadataToStr(m: (String, RecordMetadata)): String = {
+s"Sent ${m._1} to partition ${m._2.partition()}, offset 
${m._2.offset()}"
  

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-05 Thread tdas
Github user tdas commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81927094
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 ---
@@ -0,0 +1,396 @@
+/*
+ * 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
   

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-05 Thread tdas
Github user tdas commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81930482
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 ---
@@ -0,0 +1,396 @@
+/*
+ * 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
   

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-05 Thread tdas
Github user tdas commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81926721
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 ---
@@ -0,0 +1,396 @@
+/*
+ * 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
   

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-05 Thread tdas
Github user tdas commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81933097
  
--- Diff: 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
+
+
+
+
+// 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)]
+
+
+
+
+// Subscribe to 1 topic
+Dataset 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 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 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)")
+
+
+
+
+# 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)")
+
+
+
+
+Each row in the source has the following schema:
+
+ColumnType
+
+  key
+  binary
+
+
+  value
+  binary
+
+
+  topic
+  string
+
+
+  partition
+  int
+
+
+  offset
+  long
+
+
+  timestamp
+  long
+
+
+  timestampType
+  int
+
+
+
+The following options should be set for the Kafka source.
--- End diff --

nit: should --> must be


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-04 Thread tdas
Github user tdas commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81878432
  
--- Diff: docs/structured-streaming-programming-guide.md ---
@@ -1126,6 +1126,12 @@ aggDF \
 
 
 
+## Advanced Sources
--- End diff --

Cool.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-04 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81878267
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 ---
@@ -0,0 +1,396 @@
+/*
+ * 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

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-04 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81878240
  
--- Diff: 
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
+

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-04 Thread tdas
Github user tdas commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81875588
  
--- Diff: 
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
+

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-04 Thread tdas
Github user tdas commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81875168
  
--- Diff: docs/structured-streaming-programming-guide.md ---
@@ -1126,6 +1126,12 @@ aggDF \
 
 
 
+## Advanced Sources
--- End diff --

Discussed offline. Best to move it to the same location where the File and 
socket sources are discussed. Insert a subheading called Input Sources to 
mirror the subheading Output Sinks.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-04 Thread tdas
Github user tdas commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81873756
  
--- Diff: docs/structured-streaming-programming-guide.md ---
@@ -1126,6 +1126,12 @@ aggDF \
 
 
 
+## Advanced Sources
--- End diff --

Well there it was explicitly categories as inbuilt sources and advanced 
sources before this section. Here that is not there. So the term "advanced" 
does not make sense.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-04 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81873748
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 ---
@@ -0,0 +1,396 @@
+/*
+ * 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

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-04 Thread tdas
Github user tdas commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81873638
  
--- Diff: docs/structured-streaming-kafka-integration.md ---
@@ -0,0 +1,231 @@
+---
+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. 
It provides simple parallelism,
+1:1 correspondence between Kafka partitions and Spark partitions. The 
source will cache the Kafka
+consumer in executors and try the best to schedule the same Kafka topic 
partition to the same executor.
+
+### 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
+
+
+
+
+// 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)]
+
+
+
+
+// Subscribe to 1 topic
+Dataset 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 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 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)")
+
+
+
+
+# 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)")
+
+
+
+
+Each row in the source has the following schema:
+
+ColumnType
+
+  key
+  binary
+
+
+  value
+  binary
+
+
+  topic
+  string
+
+
+  partition
+  int
+
+
+  offset
+  long
+
+
+  timestamp
+  long
+
+
+  timestampType
+  int
+
+
+
+Right now, the Kafka source has the following Spark's specific options.
+
+
+Optionvaluedefaultmeaning
+
+  startingOffset
+  ["earliest", "latest"]
+  "latest"
+  The start point when a query is started, either "earliest" 

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-04 Thread tdas
Github user tdas commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81873392
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 ---
@@ -0,0 +1,396 @@
+/*
+ * 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
   

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-04 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81862040
  
--- Diff: docs/structured-streaming-kafka-integration.md ---
@@ -0,0 +1,231 @@
+---
+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. 
It provides simple parallelism,
+1:1 correspondence between Kafka partitions and Spark partitions. The 
source will cache the Kafka
+consumer in executors and try the best to schedule the same Kafka topic 
partition to the same executor.
+
+### 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
+
+
+
+
+// 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)]
+
+
+
+
+// Subscribe to 1 topic
+Dataset 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 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 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)")
+
+
+
+
+# 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)")
+
+
+
+
+Each row in the source has the following schema:
+
+ColumnType
+
+  key
+  binary
+
+
+  value
+  binary
+
+
+  topic
+  string
+
+
+  partition
+  int
+
+
+  offset
+  long
+
+
+  timestamp
+  long
+
+
+  timestampType
+  int
+
+
+
+Right now, the Kafka source has the following Spark's specific options.
+
+
+Optionvaluedefaultmeaning
+
+  startingOffset
+  ["earliest", "latest"]
+  "latest"
+  The start point when a query is started, either "earliest" 

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-04 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81861936
  
--- Diff: docs/structured-streaming-kafka-integration.md ---
@@ -0,0 +1,231 @@
+---
+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. 
It provides simple parallelism,
+1:1 correspondence between Kafka partitions and Spark partitions. The 
source will cache the Kafka
+consumer in executors and try the best to schedule the same Kafka topic 
partition to the same executor.
+
+### 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
+
+
+
+
+// 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)]
+
+
+
+
+// Subscribe to 1 topic
+Dataset 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 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 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)")
+
+
+
+
+# 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)")
+
+
+
+
+Each row in the source has the following schema:
+
+ColumnType
+
+  key
+  binary
+
+
+  value
+  binary
+
+
+  topic
+  string
+
+
+  partition
+  int
+
+
+  offset
+  long
+
+
+  timestamp
+  long
+
+
+  timestampType
+  int
+
+
+
+Right now, the Kafka source has the following Spark's specific options.
+
+
+Optionvaluedefaultmeaning
+
+  startingOffset
+  ["earliest", "latest"]
+  "latest"
+  The start point when a query is started, either "earliest" 

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-04 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81861960
  
--- Diff: docs/structured-streaming-kafka-integration.md ---
@@ -0,0 +1,231 @@
+---
+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. 
It provides simple parallelism,
+1:1 correspondence between Kafka partitions and Spark partitions. The 
source will cache the Kafka
+consumer in executors and try the best to schedule the same Kafka topic 
partition to the same executor.
+
+### 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
+
+
+
+
+// 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)]
+
+
+
+
+// Subscribe to 1 topic
+Dataset 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 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 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)")
+
+
+
+
+# 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)")
+
+
+
+
+Each row in the source has the following schema:
+
+ColumnType
+
+  key
+  binary
+
+
+  value
+  binary
+
+
+  topic
+  string
+
+
+  partition
+  int
+
+
+  offset
+  long
+
+
+  timestamp
+  long
+
+
+  timestampType
+  int
+
+
+
+Right now, the Kafka source has the following Spark's specific options.
+
+
--- End diff --

> I would split this into two tables, one that must be specified 
(subscribe/subscribePattern, and kafka.bootstrap...) and then optional 

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-04 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81861641
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala ---
@@ -469,29 +469,49 @@ trait StreamTest extends QueryTest with 
SharedSQLContext with Timeouts {
 }
   }
 
+
+  /**
+   * Creates a stress test that randomly starts/stops/adds data/checks the 
result.
+   *
+   * @param ds a dataframe that executes + 1 on a stream of integers, 
returning the result
+   * @param addData an add data action that adds the given numbers to the 
stream, encoding them
+   *as needed
+   * @param iterations the iteration number
+   */
+  def runStressTest(
+ds: Dataset[Int],
+addData: Seq[Int] => StreamAction,
+iterations: Int = 100): Unit = {
+runStressTest(ds, Seq.empty, (data, running) => addData(data), 
iterations)
+  }
+
   /**
* Creates a stress test that randomly starts/stops/adds data/checks the 
result.
*
-   * @param ds a dataframe that executes + 1 on a stream of integers, 
returning the result.
-   * @param addData and add data action that adds the given numbers to the 
stream, encoding them
+   * @param ds a dataframe that executes + 1 on a stream of integers, 
returning the result
+   * @param preparedActions actions need to run before starting the stress 
test.
--- End diff --

> nit: its kind of odd for this to be past tense.

fixed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-04 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81861633
  
--- Diff: 
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
+

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-04 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81857716
  
--- Diff: docs/structured-streaming-kafka-integration.md ---
@@ -0,0 +1,231 @@
+---
+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. 
It provides simple parallelism,
+1:1 correspondence between Kafka partitions and Spark partitions. The 
source will cache the Kafka
+consumer in executors and try the best to schedule the same Kafka topic 
partition to the same executor.
+
+### 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
+
+
+
+
+// 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)]
+
+
+
+
+// Subscribe to 1 topic
+Dataset 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 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 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)")
+
+
+
+
+# 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)")
+
+
+
+
+Each row in the source has the following schema:
+
+ColumnType
+
+  key
+  binary
+
+
+  value
+  binary
+
+
+  topic
+  string
+
+
+  partition
+  int
+
+
+  offset
+  long
+
+
+  timestamp
+  long
+
+
+  timestampType
+  int
+
+
+
+Right now, the Kafka source has the following Spark's specific options.
+
+
+Optionvaluedefaultmeaning
+
+  startingOffset
+  ["earliest", "latest"]
+  "latest"
+  The start point when a query is started, either "earliest" 

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-04 Thread tdas
Github user tdas commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81851594
  
--- Diff: docs/structured-streaming-kafka-integration.md ---
@@ -0,0 +1,231 @@
+---
+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. 
It provides simple parallelism,
+1:1 correspondence between Kafka partitions and Spark partitions. The 
source will cache the Kafka
+consumer in executors and try the best to schedule the same Kafka topic 
partition to the same executor.
+
+### 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
+
+
+
+
+// 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)]
+
+
+
+
+// Subscribe to 1 topic
+Dataset 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 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 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)")
+
+
+
+
+# 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)")
+
+
+
+
+Each row in the source has the following schema:
+
+ColumnType
+
+  key
+  binary
+
+
+  value
+  binary
+
+
+  topic
+  string
+
+
+  partition
+  int
+
+
+  offset
+  long
+
+
+  timestamp
+  long
+
+
+  timestampType
+  int
+
+
+
+Right now, the Kafka source has the following Spark's specific options.
+
+
+Optionvaluedefaultmeaning
+
+  startingOffset
+  ["earliest", "latest"]
+  "latest"
+  The start point when a query is started, either "earliest" 

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-04 Thread tdas
Github user tdas commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81851557
  
--- Diff: docs/structured-streaming-kafka-integration.md ---
@@ -0,0 +1,231 @@
+---
+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. 
It provides simple parallelism,
+1:1 correspondence between Kafka partitions and Spark partitions. The 
source will cache the Kafka
+consumer in executors and try the best to schedule the same Kafka topic 
partition to the same executor.
+
+### 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
+
+
+
+
+// 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)]
+
+
+
+
+// Subscribe to 1 topic
+Dataset 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 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 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)")
+
+
+
+
+# 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)")
+
+
+
+
+Each row in the source has the following schema:
+
+ColumnType
+
+  key
+  binary
+
+
+  value
+  binary
+
+
+  topic
+  string
+
+
+  partition
+  int
+
+
+  offset
+  long
+
+
+  timestamp
+  long
+
+
+  timestampType
+  int
+
+
+
+Right now, the Kafka source has the following Spark's specific options.
+
+
+Optionvaluedefaultmeaning
+
+  startingOffset
+  ["earliest", "latest"]
+  "latest"
+  The start point when a query is started, either "earliest" 

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-04 Thread tdas
Github user tdas commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81851459
  
--- Diff: docs/structured-streaming-kafka-integration.md ---
@@ -0,0 +1,231 @@
+---
+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. 
It provides simple parallelism,
+1:1 correspondence between Kafka partitions and Spark partitions. The 
source will cache the Kafka
+consumer in executors and try the best to schedule the same Kafka topic 
partition to the same executor.
+
+### 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
+
+
+
+
+// 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)]
+
+
+
+
+// Subscribe to 1 topic
+Dataset 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 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 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)")
+
+
+
+
+# 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)")
+
+
+
+
+Each row in the source has the following schema:
+
+ColumnType
+
+  key
+  binary
+
+
+  value
+  binary
+
+
+  topic
+  string
+
+
+  partition
+  int
+
+
+  offset
+  long
+
+
+  timestamp
+  long
+
+
+  timestampType
+  int
+
+
+
+Right now, the Kafka source has the following Spark's specific options.
+
+
+Optionvaluedefaultmeaning
+
+  startingOffset
+  ["earliest", "latest"]
+  "latest"
+  The start point when a query is started, either "earliest" 

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-04 Thread tdas
Github user tdas commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81851399
  
--- Diff: docs/structured-streaming-kafka-integration.md ---
@@ -0,0 +1,231 @@
+---
+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. 
It provides simple parallelism,
+1:1 correspondence between Kafka partitions and Spark partitions. The 
source will cache the Kafka
+consumer in executors and try the best to schedule the same Kafka topic 
partition to the same executor.
+
+### 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
+
+
+
+
+// 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)]
+
+
+
+
+// Subscribe to 1 topic
+Dataset 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 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 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)")
+
+
+
+
+# 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)")
+
+
+
+
+Each row in the source has the following schema:
+
+ColumnType
+
+  key
+  binary
+
+
+  value
+  binary
+
+
+  topic
+  string
+
+
+  partition
+  int
+
+
+  offset
+  long
+
+
+  timestamp
+  long
+
+
+  timestampType
+  int
+
+
+
+Right now, the Kafka source has the following Spark's specific options.
+
+
+Optionvaluedefaultmeaning
+
+  startingOffset
+  ["earliest", "latest"]
+  "latest"
+  The start point when a query is started, either "earliest" 

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-04 Thread tdas
Github user tdas commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81851039
  
--- Diff: docs/structured-streaming-kafka-integration.md ---
@@ -0,0 +1,231 @@
+---
+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. 
It provides simple parallelism,
+1:1 correspondence between Kafka partitions and Spark partitions. The 
source will cache the Kafka
+consumer in executors and try the best to schedule the same Kafka topic 
partition to the same executor.
+
+### 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
+
+
+
+
+// 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)]
+
+
+
+
+// Subscribe to 1 topic
+Dataset 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 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 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)")
+
+
+
+
+# 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)")
+
+
+
+
+Each row in the source has the following schema:
+
+ColumnType
+
+  key
+  binary
+
+
+  value
+  binary
+
+
+  topic
+  string
+
+
+  partition
+  int
+
+
+  offset
+  long
+
+
+  timestamp
+  long
+
+
+  timestampType
+  int
+
+
+
+Right now, the Kafka source has the following Spark's specific options.
+
+
--- End diff --

I would split this into two tables, one that must be specified 
(subscribe/subscribePattern, and kafka.bootstrap...) and then optional params.



[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-04 Thread tdas
Github user tdas commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81850342
  
--- Diff: docs/structured-streaming-kafka-integration.md ---
@@ -0,0 +1,231 @@
+---
+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. 
It provides simple parallelism,
+1:1 correspondence between Kafka partitions and Spark partitions. The 
source will cache the Kafka
+consumer in executors and try the best to schedule the same Kafka topic 
partition to the same executor.
+
+### 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
+
+
+
+
+// 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)]
+
+
+
+
+// Subscribe to 1 topic
+Dataset 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 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 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)")
+
+
+
+
+# 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)")
+
+
+
+
+Each row in the source has the following schema:
+
+ColumnType
+
+  key
+  binary
+
+
+  value
+  binary
+
+
+  topic
+  string
+
+
+  partition
+  int
+
+
+  offset
+  long
+
+
+  timestamp
+  long
+
+
+  timestampType
+  int
+
+
+
+Right now, the Kafka source has the following Spark's specific options.
+
+
+Optionvaluedefaultmeaning
+
+  startingOffset
+  ["earliest", "latest"]
+  "latest"
+  The start point when a query is started, either "earliest" 

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-04 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81844299
  
--- Diff: docs/structured-streaming-kafka-integration.md ---
@@ -0,0 +1,231 @@
+---
+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. 
It provides simple parallelism,
+1:1 correspondence between Kafka partitions and Spark partitions. The 
source will cache the Kafka
+consumer in executors and try the best to schedule the same Kafka topic 
partition to the same executor.
+
+### 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
+
+
+
+
+// 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)]
+
+
+
+
+// Subscribe to 1 topic
+Dataset 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 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 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)")
+
+
+
+
+# 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)")
+
+
+
+
+Each row in the source has the following schema:
+
+ColumnType
+
+  key
+  binary
+
+
+  value
+  binary
+
+
+  topic
+  string
+
+
+  partition
+  int
+
+
+  offset
+  long
+
+
+  timestamp
+  long
+
+
+  timestampType
+  int
+
+
+
+Right now, the Kafka source has the following Spark's specific options.
+
+
+Optionvaluedefaultmeaning
+
+  startingOffset
+  ["earliest", "latest"]
+  "latest"
+  The start point when a query is started, either "earliest" 

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-04 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81842503
  
--- Diff: docs/structured-streaming-kafka-integration.md ---
@@ -0,0 +1,231 @@
+---
+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. 
It provides simple parallelism,
+1:1 correspondence between Kafka partitions and Spark partitions. The 
source will cache the Kafka
+consumer in executors and try the best to schedule the same Kafka topic 
partition to the same executor.
+
+### 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
+
+
+
+
+// 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)]
+
+
+
+
+// Subscribe to 1 topic
+Dataset 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 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 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)")
+
+
+
+
+# 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)")
+
+
+
+
+Each row in the source has the following schema:
+
+ColumnType
+
+  key
+  binary
+
+
+  value
+  binary
+
+
+  topic
+  string
+
+
+  partition
+  int
+
+
+  offset
+  long
+
+
+  timestamp
+  long
+
+
+  timestampType
+  int
+
+
+
+Right now, the Kafka source has the following Spark's specific options.
+
+
+Optionvaluedefaultmeaning
+
+  startingOffset
+  ["earliest", "latest"]
+  "latest"
+  The start point when a query is started, either "earliest" 

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-04 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81840127
  
--- Diff: docs/structured-streaming-kafka-integration.md ---
@@ -0,0 +1,231 @@
+---
+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. 
It provides simple parallelism,
+1:1 correspondence between Kafka partitions and Spark partitions. The 
source will cache the Kafka
+consumer in executors and try the best to schedule the same Kafka topic 
partition to the same executor.
+
+### 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
+
+
+
+
+// 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)]
+
+
+
+
+// Subscribe to 1 topic
+Dataset 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 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 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)")
+
+
+
+
+# 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)")
+
+
+
+
+Each row in the source has the following schema:
+
+ColumnType
+
+  key
+  binary
+
+
+  value
+  binary
+
+
+  topic
+  string
+
+
+  partition
+  int
+
+
+  offset
+  long
+
+
+  timestamp
+  long
+
+
+  timestampType
+  int
+
+
+
+Right now, the Kafka source has the following Spark's specific options.
+
+
+Optionvaluedefaultmeaning
+
+  startingOffset
+  ["earliest", "latest"]
+  "latest"
+  The start point when a query is started, either "earliest" 

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-04 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81839555
  
--- Diff: docs/structured-streaming-kafka-integration.md ---
@@ -0,0 +1,231 @@
+---
+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. 
It provides simple parallelism,
+1:1 correspondence between Kafka partitions and Spark partitions. The 
source will cache the Kafka
+consumer in executors and try the best to schedule the same Kafka topic 
partition to the same executor.
+
+### 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
+
+
+
+
+// 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)]
+
+
+
+
+// Subscribe to 1 topic
+Dataset 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 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 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)")
+
+
+
+
+# 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)")
+
+
+
+
+Each row in the source has the following schema:
+
+ColumnType
+
+  key
+  binary
+
+
+  value
+  binary
+
+
+  topic
+  string
+
+
+  partition
+  int
+
+
+  offset
+  long
+
+
+  timestamp
+  long
+
+
+  timestampType
+  int
+
+
+
+Right now, the Kafka source has the following Spark's specific options.
+
+
+Optionvaluedefaultmeaning
+
+  startingOffset
+  ["earliest", "latest"]
+  "latest"
+  The start point when a query is started, either "earliest" 

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-04 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81834388
  
--- Diff: docs/structured-streaming-kafka-integration.md ---
@@ -0,0 +1,231 @@
+---
+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. 
It provides simple parallelism,
+1:1 correspondence between Kafka partitions and Spark partitions. The 
source will cache the Kafka
+consumer in executors and try the best to schedule the same Kafka topic 
partition to the same executor.
+
+### 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
+
+
+
+
+// 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)]
+
+
+
+
+// Subscribe to 1 topic
+Dataset 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 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 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)")
+
+
+
+
+# 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)")
+
+
+
+
+Each row in the source has the following schema:
+
+ColumnType
+
+  key
+  binary
+
+
+  value
+  binary
+
+
+  topic
+  string
+
+
+  partition
+  int
+
+
+  offset
+  long
+
+
+  timestamp
+  long
+
+
+  timestampType
+  int
+
+
+
+Right now, the Kafka source has the following Spark's specific options.
+
+
+Optionvaluedefaultmeaning
+
+  startingOffset
+  ["earliest", "latest"]
+  "latest"
+  The start point when a query is started, either "earliest" 

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-04 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81836478
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala
 ---
@@ -530,3 +530,8 @@ object StreamExecution {
 
   def nextId: Long = _nextId.getAndIncrement()
 }
+
+/**
+ * A special thread to run the stream query.
--- End diff --

You might comment that we actually check for this class elsewhere.  This 
part is not super clear to me.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-04 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81836317
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala ---
@@ -469,29 +469,49 @@ trait StreamTest extends QueryTest with 
SharedSQLContext with Timeouts {
 }
   }
 
+
+  /**
+   * Creates a stress test that randomly starts/stops/adds data/checks the 
result.
+   *
+   * @param ds a dataframe that executes + 1 on a stream of integers, 
returning the result
+   * @param addData an add data action that adds the given numbers to the 
stream, encoding them
+   *as needed
+   * @param iterations the iteration number
+   */
+  def runStressTest(
+ds: Dataset[Int],
+addData: Seq[Int] => StreamAction,
+iterations: Int = 100): Unit = {
+runStressTest(ds, Seq.empty, (data, running) => addData(data), 
iterations)
+  }
+
   /**
* Creates a stress test that randomly starts/stops/adds data/checks the 
result.
*
-   * @param ds a dataframe that executes + 1 on a stream of integers, 
returning the result.
-   * @param addData and add data action that adds the given numbers to the 
stream, encoding them
+   * @param ds a dataframe that executes + 1 on a stream of integers, 
returning the result
+   * @param preparedActions actions need to run before starting the stress 
test.
--- End diff --

nit: its kind of odd for this to be past tense.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-04 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81833814
  
--- Diff: docs/structured-streaming-kafka-integration.md ---
@@ -0,0 +1,231 @@
+---
+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. 
It provides simple parallelism,
+1:1 correspondence between Kafka partitions and Spark partitions. The 
source will cache the Kafka
+consumer in executors and try the best to schedule the same Kafka topic 
partition to the same executor.
--- End diff --

I would consider dropping all of these implementation details from the 
docs.  I think its possible we will change them.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-04 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81835883
  
--- Diff: 
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
+

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-03 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81663920
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 ---
@@ -0,0 +1,366 @@
+/*
+ * 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._
+
+/**
+ * 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.
+ */
+private[kafka010] case class KafkaSource(
+sqlContext: SQLContext,
+consumerStrategy: ConsumerStrategy,
+executorKafkaParams: ju.Map[String, Object],
+sourceOptions: Map[String, String])
+  extends Source with Logging {
+
+  private val consumer = consumerStrategy.createConsumer()
+  private val sc = sqlContext.sparkContext
+  private val initialPartitionOffsets = fetchPartitionOffsets(seekToLatest 
= false)
+  logInfo(s"Initial offsets: $initialPartitionOffsets")
+
+  override def schema: StructType = KafkaSource.kafkaSchema
+
+  /** Returns the maximum available offset for this source. */
+  

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-03 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81661726
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala
 ---
@@ -0,0 +1,155 @@
+/*
+ * 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,
--- End diff --

Added group id into logs.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-03 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81661667
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 ---
@@ -0,0 +1,446 @@
+/*
+ * 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 org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, 
KafkaConsumer}
+import 
org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener
+import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.serialization.ByteArrayDeserializer
+
+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.sources.{DataSourceRegister, 
StreamSourceProvider}
+import org.apache.spark.sql.types._
+import org.apache.spark.SparkContext
+
+/**
+ * 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
--- End diff --

> Still cuts off. I'd consider removing this abstraction completely.

Fixed the format


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-03 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81661642
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 ---
@@ -0,0 +1,366 @@
+/*
+ * 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._
+
+/**
+ * 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.
+ */
+private[kafka010] case class KafkaSource(
+sqlContext: SQLContext,
+consumerStrategy: ConsumerStrategy,
+executorKafkaParams: ju.Map[String, Object],
+sourceOptions: Map[String, String])
+  extends Source with Logging {
+
+  private val consumer = consumerStrategy.createConsumer()
+  private val sc = sqlContext.sparkContext
+  private val initialPartitionOffsets = fetchPartitionOffsets(seekToLatest 
= false)
+  logInfo(s"Initial offsets: $initialPartitionOffsets")
+
+  override def schema: StructType = KafkaSource.kafkaSchema
+
+  /** Returns the maximum available offset for this source. */
+  

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-03 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81658824
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala
 ---
@@ -0,0 +1,163 @@
+/*
+ * 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, 
OffsetOutOfRangeException}
+import org.apache.kafka.common.TopicPartition
+
+import org.apache.spark.{Partition, SparkContext, TaskContext}
+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.
+ *
+ * Note that this is a simplified version of the 
org.apache.spark.streaming.kafka010.KafkaRDD.
+ *
+ * @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])
+  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
+
--- End diff --

> Why missing countApprox?

Added


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-03 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81648912
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 ---
@@ -0,0 +1,446 @@
+/*
--- End diff --

Moved KafkaSourceOffset to a new file


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-03 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81648834
  
--- Diff: docs/structured-streaming-kafka-integration.md ---
@@ -0,0 +1,185 @@
+---
+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. 
It provides simple parallelism,
+1:1 correspondence between Kafka partitions and Spark partitions.
+
+### 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
+
+
+
+
+// Subscribe to 1 topic
+spark
+  .readStream
+  .format("kafka")
+  .option("kafka.bootstrap.servers", "host1:port1,host2:port2")
+  .option("subscribe", "topic1")
+  .load()
+
+// Subscribe to multiple topics
+spark
+  .readStream
+  .format("kafka")
+  .option("kafka.bootstrap.servers", "host1:port1,host2:port2")
+  .option("subscribe", "topic1,topic2")
+  .load()
+
+// Subscribe to a pattern
+spark
+  .readStream
+  .format("kafka")
+  .option("kafka.bootstrap.servers", "host1:port1,host2:port2")
+  .option("subscribePattern", "topic.*")
+  .load()
+
+
+
+
+// Subscribe to 1 topic
+spark
+  .readStream()
+  .format("kafka")
+  .option("kafka.bootstrap.servers", "host1:port1,host2:port2")
+  .option("subscribe", "topic1")
+  .load()
+
+// Subscribe to multiple topics
+spark
+  .readStream()
+  .format("kafka")
+  .option("kafka.bootstrap.servers", "host1:port1,host2:port2")
+  .option("subscribe", "topic1,topic2")
+  .load()
+
+// Subscribe to a pattern
+spark
+  .readStream()
+  .format("kafka")
+  .option("kafka.bootstrap.servers", "host1:port1,host2:port2")
+  .option("subscribePattern", "topic.*")
+  .load()
+
+
+
+
+# Subscribe to 1 topic
+spark
+  .readStream()
+  .format("kafka")
+  .option("kafka.bootstrap.servers", "host1:port1,host2:port2")
+  .option("subscribe", "topic1")
+  .load()
+
+# Subscribe to multiple topics
+spark
+  .readStream
+  .format("kafka")
+  .option("kafka.bootstrap.servers", "host1:port1,host2:port2")
+  .option("subscribe", "topic1,topic2")
+  .load()
+
+# Subscribe to a pattern
+spark
+  .readStream()
+  .format("kafka")
+  .option("kafka.bootstrap.servers", "host1:port1,host2:port2")
+  .option("subscribePattern", "topic.*")
+  .load()
+
+
+
+
+Each row in the source has the following schema:
+
+ColumnType
+
+  key
+  binary
+
+
+  value
+  binary
--- End diff --

Added an example to convert key/value to strings.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-03 Thread koeninger
Github user koeninger commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81623277
  
--- Diff: docs/structured-streaming-kafka-integration.md ---
@@ -0,0 +1,185 @@
+---
+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. 
It provides simple parallelism,
+1:1 correspondence between Kafka partitions and Spark partitions.
+
+### 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
+
+
+
+
+// Subscribe to 1 topic
+spark
+  .readStream
+  .format("kafka")
+  .option("kafka.bootstrap.servers", "host1:port1,host2:port2")
+  .option("subscribe", "topic1")
+  .load()
+
+// Subscribe to multiple topics
+spark
+  .readStream
+  .format("kafka")
+  .option("kafka.bootstrap.servers", "host1:port1,host2:port2")
+  .option("subscribe", "topic1,topic2")
+  .load()
+
+// Subscribe to a pattern
+spark
+  .readStream
+  .format("kafka")
+  .option("kafka.bootstrap.servers", "host1:port1,host2:port2")
+  .option("subscribePattern", "topic.*")
+  .load()
+
+
+
+
+// Subscribe to 1 topic
+spark
+  .readStream()
+  .format("kafka")
+  .option("kafka.bootstrap.servers", "host1:port1,host2:port2")
+  .option("subscribe", "topic1")
+  .load()
+
+// Subscribe to multiple topics
+spark
+  .readStream()
+  .format("kafka")
+  .option("kafka.bootstrap.servers", "host1:port1,host2:port2")
+  .option("subscribe", "topic1,topic2")
+  .load()
+
+// Subscribe to a pattern
+spark
+  .readStream()
+  .format("kafka")
+  .option("kafka.bootstrap.servers", "host1:port1,host2:port2")
+  .option("subscribePattern", "topic.*")
+  .load()
+
+
+
+
+# Subscribe to 1 topic
+spark
+  .readStream()
+  .format("kafka")
+  .option("kafka.bootstrap.servers", "host1:port1,host2:port2")
+  .option("subscribe", "topic1")
+  .load()
+
+# Subscribe to multiple topics
+spark
+  .readStream
+  .format("kafka")
+  .option("kafka.bootstrap.servers", "host1:port1,host2:port2")
+  .option("subscribe", "topic1,topic2")
+  .load()
+
+# Subscribe to a pattern
+spark
+  .readStream()
+  .format("kafka")
+  .option("kafka.bootstrap.servers", "host1:port1,host2:port2")
+  .option("subscribePattern", "topic.*")
+  .load()
+
+
+
+
+Each row in the source has the following schema:
+
+ColumnType
+
+  key
+  binary
+
+
+  value
+  binary
--- End diff --

This should probably include an example of how to select key / value in the 
desired format.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-10-03 Thread koeninger
Github user koeninger commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r81622511
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala
 ---
@@ -0,0 +1,281 @@
+/*
+ * 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
+

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-27 Thread koeninger
Github user koeninger commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80793355
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala
 ---
@@ -0,0 +1,155 @@
+/*
+ * 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,
--- End diff --

Except you also removed it from any logging, which is going to make it hard 
to tie to a particular stream if you have multiple.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-27 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80749159
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 ---
@@ -0,0 +1,366 @@
+/*
+ * 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._
+
+/**
+ * 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.
+ */
+private[kafka010] case class KafkaSource(
+sqlContext: SQLContext,
+consumerStrategy: ConsumerStrategy,
+executorKafkaParams: ju.Map[String, Object],
+sourceOptions: Map[String, String])
+  extends Source with Logging {
+
+  private val consumer = consumerStrategy.createConsumer()
+  private val sc = sqlContext.sparkContext
+  private val initialPartitionOffsets = fetchPartitionOffsets(seekToLatest 
= false)
+  logInfo(s"Initial offsets: $initialPartitionOffsets")
+
+  override def schema: StructType = KafkaSource.kafkaSchema
+
+  /** Returns the maximum available offset for this source. */
+  

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-27 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80747895
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala
 ---
@@ -0,0 +1,155 @@
+/*
+ * 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,
--- End diff --

`groupId` can be read via `kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG)`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread koeninger
Github user koeninger commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80617904
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 ---
@@ -0,0 +1,344 @@
+/*
+ * 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 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._
+
+/**
+ * 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.
+ */
+private[kafka010] case class KafkaSource(
+sqlContext: SQLContext,
+consumerStrategy: ConsumerStrategy,
+executorKafkaParams: ju.Map[String, Object],
+sourceOptions: Map[String, String])
+  extends Source with Logging {
+
+  private val consumer = consumerStrategy.createConsumer()
+  private val sc = sqlContext.sparkContext
+  private val initialPartitionOffsets = fetchPartitionOffsets(seekToLatest 
= false)
+  logInfo(s"Initial offsets: $initialPartitionOffsets")
+
+  override def schema: StructType = KafkaSource.kafkaSchema
+
+  /** Returns the maximum available offset for this source. */
+  override def getOffset: Option[Offset] = {
+val offset = KafkaSourceOffset(fetchPartitionOffsets(seekToLatest = 
true))
+logInfo(s"GetOffset: $offset")
+Some(offset)
+  }
+
+  /** Returns the data that is between 

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread koeninger
Github user koeninger commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80617145
  
--- Diff: external/kafka-0-10-sql/pom.xml ---
@@ -0,0 +1,82 @@
+
+
+
+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;>
+  4.0.0
+  
+org.apache.spark
+spark-parent_2.11
+2.1.0-SNAPSHOT
+../../pom.xml
+  
+
+  org.apache.spark
+  spark-sql-kafka-0-10_2.11
+  
+sql-kafka-0-10
+  
+  jar
+  Spark Integration for Kafka 0.10
--- End diff --

This has the same name as the DStream, should probably be changed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread koeninger
Github user koeninger commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80617036
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala
 ---
@@ -0,0 +1,163 @@
+/*
+ * 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, 
OffsetOutOfRangeException}
+import org.apache.kafka.common.TopicPartition
+
+import org.apache.spark.{Partition, SparkContext, TaskContext}
+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.
+ *
+ * Note that this is a simplified version of the 
org.apache.spark.streaming.kafka010.KafkaRDD.
+ *
+ * @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])
+  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 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(
+  

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread koeninger
Github user koeninger commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80616878
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala
 ---
@@ -0,0 +1,163 @@
+/*
+ * 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, 
OffsetOutOfRangeException}
+import org.apache.kafka.common.TopicPartition
+
+import org.apache.spark.{Partition, SparkContext, TaskContext}
+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.
+ *
+ * Note that this is a simplified version of the 
org.apache.spark.streaming.kafka010.KafkaRDD.
+ *
+ * @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])
+  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 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(
+  

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread koeninger
Github user koeninger commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80616386
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala
 ---
@@ -0,0 +1,163 @@
+/*
+ * 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, 
OffsetOutOfRangeException}
+import org.apache.kafka.common.TopicPartition
+
+import org.apache.spark.{Partition, SparkContext, TaskContext}
+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.
+ *
+ * Note that this is a simplified version of the 
org.apache.spark.streaming.kafka010.KafkaRDD.
+ *
+ * @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])
+  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
+
--- End diff --

Why missing countApprox?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread koeninger
Github user koeninger commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80616098
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala
 ---
@@ -0,0 +1,163 @@
+/*
+ * 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, 
OffsetOutOfRangeException}
+import org.apache.kafka.common.TopicPartition
+
+import org.apache.spark.{Partition, SparkContext, TaskContext}
+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.
+ *
+ * Note that this is a simplified version of the 
org.apache.spark.streaming.kafka010.KafkaRDD.
--- End diff --

Why is this the only place that has this note, when much of the code in 
other classes (e.g. cached consumer) is taken from that as well?  Not saying 
it's necessary to have it everywhere, just seems inconsistent


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread koeninger
Github user koeninger commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80615842
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala
 ---
@@ -0,0 +1,263 @@
+/*
+ * 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) = {
+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
+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 
unnecessaribly
+.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 =
+  

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread koeninger
Github user koeninger commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80615307
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala
 ---
@@ -0,0 +1,263 @@
+/*
+ * 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) = {
+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
+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 
unnecessaribly
+.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 =
+  

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread koeninger
Github user koeninger commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80614899
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala
 ---
@@ -0,0 +1,263 @@
+/*
+ * 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) = {
+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
+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 
unnecessaribly
+.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 =
+  

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread koeninger
Github user koeninger commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80614481
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala
 ---
@@ -0,0 +1,263 @@
+/*
+ * 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) = {
+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
+val uniqueGroupId = 
s"spark-kafka-source-${UUID.randomUUID}-${metadataPath.hashCode}"
--- End diff --

@marmbrus the new consumer shouldn't be using ZK for consumer groups, so 
this shouldn't leak state into ZK.  I quickly verified this with a spark job 
that had auto commit turned off.  It will show up in the list of consumer 
groups though, and a senseless name may make it hard for people that want to do 
monitoring via kafka tools.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread koeninger
Github user koeninger commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80613358
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala
 ---
@@ -0,0 +1,263 @@
+/*
+ * 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) = {
+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
+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 
unnecessaribly
+.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 =
+  

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread koeninger
Github user koeninger commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80613155
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala
 ---
@@ -0,0 +1,263 @@
+/*
+ * 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) = {
+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
+val uniqueGroupId = 
s"spark-kafka-source-${UUID.randomUUID}-${metadataPath.hashCode}"
--- End diff --

At the very least put the app name or something in there so that people 
have some hope of knowing what's going on.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread koeninger
Github user koeninger commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80612809
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 ---
@@ -0,0 +1,344 @@
+/*
+ * 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 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._
+
+/**
+ * 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.
+ */
+private[kafka010] case class KafkaSource(
+sqlContext: SQLContext,
+consumerStrategy: ConsumerStrategy,
+executorKafkaParams: ju.Map[String, Object],
+sourceOptions: Map[String, String])
+  extends Source with Logging {
+
+  private val consumer = consumerStrategy.createConsumer()
+  private val sc = sqlContext.sparkContext
+  private val initialPartitionOffsets = fetchPartitionOffsets(seekToLatest 
= false)
+  logInfo(s"Initial offsets: $initialPartitionOffsets")
+
+  override def schema: StructType = KafkaSource.kafkaSchema
+
+  /** Returns the maximum available offset for this source. */
+  override def getOffset: Option[Offset] = {
+val offset = KafkaSourceOffset(fetchPartitionOffsets(seekToLatest = 
true))
+logInfo(s"GetOffset: $offset")
+Some(offset)
+  }
+
+  /** Returns the data that is between 

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread koeninger
Github user koeninger commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80612619
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 ---
@@ -0,0 +1,366 @@
+/*
+ * 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._
+
+/**
+ * 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.
+ */
+private[kafka010] case class KafkaSource(
+sqlContext: SQLContext,
+consumerStrategy: ConsumerStrategy,
+executorKafkaParams: ju.Map[String, Object],
+sourceOptions: Map[String, String])
+  extends Source with Logging {
+
+  private val consumer = consumerStrategy.createConsumer()
+  private val sc = sqlContext.sparkContext
+  private val initialPartitionOffsets = fetchPartitionOffsets(seekToLatest 
= false)
+  logInfo(s"Initial offsets: $initialPartitionOffsets")
+
+  override def schema: StructType = KafkaSource.kafkaSchema
+
+  /** Returns the maximum available offset for this source. */
+  

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread koeninger
Github user koeninger commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80612293
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 ---
@@ -0,0 +1,366 @@
+/*
+ * 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._
+
+/**
+ * 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.
+ */
+private[kafka010] case class KafkaSource(
+sqlContext: SQLContext,
+consumerStrategy: ConsumerStrategy,
+executorKafkaParams: ju.Map[String, Object],
+sourceOptions: Map[String, String])
+  extends Source with Logging {
+
+  private val consumer = consumerStrategy.createConsumer()
+  private val sc = sqlContext.sparkContext
+  private val initialPartitionOffsets = fetchPartitionOffsets(seekToLatest 
= false)
+  logInfo(s"Initial offsets: $initialPartitionOffsets")
+
+  override def schema: StructType = KafkaSource.kafkaSchema
+
+  /** Returns the maximum available offset for this source. */
+  

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread koeninger
Github user koeninger commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80611556
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 ---
@@ -0,0 +1,366 @@
+/*
+ * 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._
+
+/**
+ * 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.
+ */
+private[kafka010] case class KafkaSource(
+sqlContext: SQLContext,
+consumerStrategy: ConsumerStrategy,
+executorKafkaParams: ju.Map[String, Object],
+sourceOptions: Map[String, String])
+  extends Source with Logging {
+
+  private val consumer = consumerStrategy.createConsumer()
+  private val sc = sqlContext.sparkContext
+  private val initialPartitionOffsets = fetchPartitionOffsets(seekToLatest 
= false)
+  logInfo(s"Initial offsets: $initialPartitionOffsets")
+
+  override def schema: StructType = KafkaSource.kafkaSchema
+
+  /** Returns the maximum available offset for this source. */
+  

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread koeninger
Github user koeninger commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80611465
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 ---
@@ -0,0 +1,366 @@
+/*
+ * 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._
+
+/**
+ * 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.
+ */
+private[kafka010] case class KafkaSource(
+sqlContext: SQLContext,
+consumerStrategy: ConsumerStrategy,
+executorKafkaParams: ju.Map[String, Object],
+sourceOptions: Map[String, String])
+  extends Source with Logging {
+
+  private val consumer = consumerStrategy.createConsumer()
+  private val sc = sqlContext.sparkContext
+  private val initialPartitionOffsets = fetchPartitionOffsets(seekToLatest 
= false)
+  logInfo(s"Initial offsets: $initialPartitionOffsets")
+
+  override def schema: StructType = KafkaSource.kafkaSchema
+
+  /** Returns the maximum available offset for this source. */
+  

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread koeninger
Github user koeninger commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80610814
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 ---
@@ -0,0 +1,366 @@
+/*
+ * 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._
+
+/**
+ * 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.
+ */
+private[kafka010] case class KafkaSource(
+sqlContext: SQLContext,
+consumerStrategy: ConsumerStrategy,
+executorKafkaParams: ju.Map[String, Object],
+sourceOptions: Map[String, String])
+  extends Source with Logging {
+
+  private val consumer = consumerStrategy.createConsumer()
+  private val sc = sqlContext.sparkContext
+  private val initialPartitionOffsets = fetchPartitionOffsets(seekToLatest 
= false)
+  logInfo(s"Initial offsets: $initialPartitionOffsets")
+
+  override def schema: StructType = KafkaSource.kafkaSchema
+
+  /** Returns the maximum available offset for this source. */
+  

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread koeninger
Github user koeninger commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80610748
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 ---
@@ -0,0 +1,366 @@
+/*
+ * 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._
+
+/**
+ * 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.
+ */
+private[kafka010] case class KafkaSource(
+sqlContext: SQLContext,
+consumerStrategy: ConsumerStrategy,
+executorKafkaParams: ju.Map[String, Object],
+sourceOptions: Map[String, String])
+  extends Source with Logging {
+
+  private val consumer = consumerStrategy.createConsumer()
+  private val sc = sqlContext.sparkContext
+  private val initialPartitionOffsets = fetchPartitionOffsets(seekToLatest 
= false)
+  logInfo(s"Initial offsets: $initialPartitionOffsets")
+
+  override def schema: StructType = KafkaSource.kafkaSchema
+
+  /** Returns the maximum available offset for this source. */
+  

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread koeninger
Github user koeninger commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80610562
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 ---
@@ -0,0 +1,366 @@
+/*
+ * 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._
+
+/**
+ * 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.
+ */
+private[kafka010] case class KafkaSource(
+sqlContext: SQLContext,
+consumerStrategy: ConsumerStrategy,
+executorKafkaParams: ju.Map[String, Object],
+sourceOptions: Map[String, String])
+  extends Source with Logging {
+
+  private val consumer = consumerStrategy.createConsumer()
+  private val sc = sqlContext.sparkContext
+  private val initialPartitionOffsets = fetchPartitionOffsets(seekToLatest 
= false)
+  logInfo(s"Initial offsets: $initialPartitionOffsets")
+
+  override def schema: StructType = KafkaSource.kafkaSchema
+
+  /** Returns the maximum available offset for this source. */
+  

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread koeninger
Github user koeninger commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80610210
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 ---
@@ -0,0 +1,366 @@
+/*
+ * 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._
+
+/**
+ * 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.
+ */
+private[kafka010] case class KafkaSource(
+sqlContext: SQLContext,
+consumerStrategy: ConsumerStrategy,
+executorKafkaParams: ju.Map[String, Object],
+sourceOptions: Map[String, String])
+  extends Source with Logging {
+
+  private val consumer = consumerStrategy.createConsumer()
+  private val sc = sqlContext.sparkContext
+  private val initialPartitionOffsets = fetchPartitionOffsets(seekToLatest 
= false)
+  logInfo(s"Initial offsets: $initialPartitionOffsets")
+
+  override def schema: StructType = KafkaSource.kafkaSchema
+
+  /** Returns the maximum available offset for this source. */
+  

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread koeninger
Github user koeninger commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80609930
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 ---
@@ -0,0 +1,366 @@
+/*
+ * 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._
+
+/**
+ * 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.
+ */
+private[kafka010] case class KafkaSource(
+sqlContext: SQLContext,
+consumerStrategy: ConsumerStrategy,
+executorKafkaParams: ju.Map[String, Object],
+sourceOptions: Map[String, String])
+  extends Source with Logging {
+
+  private val consumer = consumerStrategy.createConsumer()
+  private val sc = sqlContext.sparkContext
+  private val initialPartitionOffsets = fetchPartitionOffsets(seekToLatest 
= false)
+  logInfo(s"Initial offsets: $initialPartitionOffsets")
+
+  override def schema: StructType = KafkaSource.kafkaSchema
+
+  /** Returns the maximum available offset for this source. */
+  

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread koeninger
Github user koeninger commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80609718
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 ---
@@ -0,0 +1,366 @@
+/*
+ * 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._
+
+/**
+ * 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.
+ */
+private[kafka010] case class KafkaSource(
+sqlContext: SQLContext,
+consumerStrategy: ConsumerStrategy,
+executorKafkaParams: ju.Map[String, Object],
+sourceOptions: Map[String, String])
+  extends Source with Logging {
+
+  private val consumer = consumerStrategy.createConsumer()
+  private val sc = sqlContext.sparkContext
+  private val initialPartitionOffsets = fetchPartitionOffsets(seekToLatest 
= false)
+  logInfo(s"Initial offsets: $initialPartitionOffsets")
+
+  override def schema: StructType = KafkaSource.kafkaSchema
+
+  /** Returns the maximum available offset for this source. */
+  

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread koeninger
Github user koeninger commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80609252
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 ---
@@ -0,0 +1,366 @@
+/*
+ * 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._
+
+/**
+ * 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.
+ */
+private[kafka010] case class KafkaSource(
+sqlContext: SQLContext,
+consumerStrategy: ConsumerStrategy,
+executorKafkaParams: ju.Map[String, Object],
+sourceOptions: Map[String, String])
+  extends Source with Logging {
+
+  private val consumer = consumerStrategy.createConsumer()
+  private val sc = sqlContext.sparkContext
+  private val initialPartitionOffsets = fetchPartitionOffsets(seekToLatest 
= false)
--- End diff --

So if I was just reading this, I'd expect the initial offsets to always 
start at the earliest offset, since the argument seekToLatest was false.


---
If your project is set up for it, 

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread koeninger
Github user koeninger commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80609099
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 ---
@@ -0,0 +1,366 @@
+/*
+ * 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._
+
+/**
+ * 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.
+ */
+private[kafka010] case class KafkaSource(
+sqlContext: SQLContext,
+consumerStrategy: ConsumerStrategy,
+executorKafkaParams: ju.Map[String, Object],
+sourceOptions: Map[String, String])
+  extends Source with Logging {
+
+  private val consumer = consumerStrategy.createConsumer()
+  private val sc = sqlContext.sparkContext
+  private val initialPartitionOffsets = fetchPartitionOffsets(seekToLatest 
= false)
+  logInfo(s"Initial offsets: $initialPartitionOffsets")
+
+  override def schema: StructType = KafkaSource.kafkaSchema
+
+  /** Returns the maximum available offset for this source. */
+  

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread koeninger
Github user koeninger commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80607976
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala
 ---
@@ -0,0 +1,155 @@
+/*
+ * 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 consumer = {
+val c = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams)
+val tps = new ju.ArrayList[TopicPartition]()
+tps.add(topicPartition)
+c.assign(tps)
+c
+  }
+
+  /**
+   * Timeout for polls to the consumer. Since the data should be already 
available, the poll
+   * should get the data immediately, and the timeout value should not 
matter as long as it is
+   * generous and does not cause timeout when there are not issues.
+   */
+  private val pollTimeoutMs = 60 * 1000
+
+  /** 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): ConsumerRecord[Array[Byte], Array[Byte]] = {
+logDebug(s"Get $topicPartition nextOffset $nextOffsetInFetchedData 
requested $offset")
+if (offset != nextOffsetInFetchedData) {
+  logInfo(s"Initial fetch for $topicPartition $offset")
+  seek(offset)
+  poll()
+}
+
+if (!fetchedData.hasNext()) { poll() }
+assert(fetchedData.hasNext(),
+  s"Failed to get records for $topicPartition $offset after polling 
for $pollTimeoutMs")
+var record = fetchedData.next()
+
+if (record.offset != offset) {
+  logInfo(s"Buffer miss for $topicPartition $offset")
+  seek(offset)
+  poll()
+  assert(fetchedData.hasNext(),
+s"Failed to get records for $topicPartition $offset after polling 
for $pollTimeoutMs")
+  record = fetchedData.next()
+  assert(record.offset == offset,
+s"Got wrong record for $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 $topicPartition $offset")
+consumer.seek(topicPartition, offset)
+  }
+
+  private def poll(): Unit = {
+val p = consumer.poll(pollTimeoutMs)
--- End diff --

Ok, I see what's going on here now.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread koeninger
Github user koeninger commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80607882
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala
 ---
@@ -0,0 +1,155 @@
+/*
+ * 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 consumer = {
+val c = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams)
+val tps = new ju.ArrayList[TopicPartition]()
+tps.add(topicPartition)
+c.assign(tps)
+c
+  }
+
+  /**
+   * Timeout for polls to the consumer. Since the data should be already 
available, the poll
+   * should get the data immediately, and the timeout value should not 
matter as long as it is
+   * generous and does not cause timeout when there are not issues.
+   */
+  private val pollTimeoutMs = 60 * 1000
+
+  /** 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): ConsumerRecord[Array[Byte], Array[Byte]] = {
+logDebug(s"Get $topicPartition nextOffset $nextOffsetInFetchedData 
requested $offset")
+if (offset != nextOffsetInFetchedData) {
+  logInfo(s"Initial fetch for $topicPartition $offset")
+  seek(offset)
+  poll()
+}
+
+if (!fetchedData.hasNext()) { poll() }
+assert(fetchedData.hasNext(),
+  s"Failed to get records for $topicPartition $offset after polling 
for $pollTimeoutMs")
+var record = fetchedData.next()
+
+if (record.offset != offset) {
+  logInfo(s"Buffer miss for $topicPartition $offset")
+  seek(offset)
+  poll()
--- End diff --

Nor is the timeout used here


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread koeninger
Github user koeninger commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80607795
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala
 ---
@@ -0,0 +1,155 @@
+/*
+ * 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 consumer = {
+val c = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams)
+val tps = new ju.ArrayList[TopicPartition]()
+tps.add(topicPartition)
+c.assign(tps)
+c
+  }
+
+  /**
+   * Timeout for polls to the consumer. Since the data should be already 
available, the poll
+   * should get the data immediately, and the timeout value should not 
matter as long as it is
+   * generous and does not cause timeout when there are not issues.
+   */
+  private val pollTimeoutMs = 60 * 1000
+
+  /** 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): ConsumerRecord[Array[Byte], Array[Byte]] = {
+logDebug(s"Get $topicPartition nextOffset $nextOffsetInFetchedData 
requested $offset")
+if (offset != nextOffsetInFetchedData) {
+  logInfo(s"Initial fetch for $topicPartition $offset")
+  seek(offset)
+  poll()
+}
+
+if (!fetchedData.hasNext()) { poll() }
--- End diff --

pollTimeoutMs was defined above, but isn't used here


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread koeninger
Github user koeninger commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80607406
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala
 ---
@@ -0,0 +1,155 @@
+/*
+ * 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,
--- End diff --

Why did this remove groupId as compared to the DStreams version?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread koeninger
Github user koeninger commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80606631
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 ---
@@ -149,6 +160,14 @@ private[kafka010] case class KafkaSource(
 Some(sortedExecutors(positiveMod(tp.hashCode, numExecutors)))
   } else None
   KafkaSourceRDDOffsetRange(tp, fromOffset, untilOffset, preferredLoc)
+}.filter { range =>
+  if (range.untilOffset < range.fromOffset) {
+logWarning(s"Partition ${range.topicPartition} was deleted and 
then added, " +
--- End diff --

I'd change this to error level and specify what the offsets in question 
were.  I also wouldn't assume that a partition was deleted and then added, this 
could be caused by other things.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread koeninger
Github user koeninger commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80605915
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 ---
@@ -0,0 +1,344 @@
+/*
+ * 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 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._
+
+/**
+ * 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.
+ */
+private[kafka010] case class KafkaSource(
+sqlContext: SQLContext,
+consumerStrategy: ConsumerStrategy,
+executorKafkaParams: ju.Map[String, Object],
+sourceOptions: Map[String, String])
+  extends Source with Logging {
+
+  private val consumer = consumerStrategy.createConsumer()
+  private val sc = sqlContext.sparkContext
+  private val initialPartitionOffsets = fetchPartitionOffsets(seekToLatest 
= false)
+  logInfo(s"Initial offsets: $initialPartitionOffsets")
+
+  override def schema: StructType = KafkaSource.kafkaSchema
+
+  /** Returns the maximum available offset for this source. */
+  override def getOffset: Option[Offset] = {
+val offset = KafkaSourceOffset(fetchPartitionOffsets(seekToLatest = 
true))
+logInfo(s"GetOffset: $offset")
+Some(offset)
+  }
+
+  /** Returns the data that is between 

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80599802
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 ---
@@ -0,0 +1,344 @@
+/*
+ * 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 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._
+
+/**
+ * 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.
+ */
+private[kafka010] case class KafkaSource(
+sqlContext: SQLContext,
+consumerStrategy: ConsumerStrategy,
+executorKafkaParams: ju.Map[String, Object],
+sourceOptions: Map[String, String])
+  extends Source with Logging {
+
+  private val consumer = consumerStrategy.createConsumer()
+  private val sc = sqlContext.sparkContext
+  private val initialPartitionOffsets = fetchPartitionOffsets(seekToLatest 
= false)
+  logInfo(s"Initial offsets: $initialPartitionOffsets")
+
+  override def schema: StructType = KafkaSource.kafkaSchema
+
+  /** Returns the maximum available offset for this source. */
+  override def getOffset: Option[Offset] = {
+val offset = KafkaSourceOffset(fetchPartitionOffsets(seekToLatest = 
true))
+logInfo(s"GetOffset: $offset")
+Some(offset)
+  }
+
+  /** Returns the data that is between 

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread tdas
Github user tdas commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80599360
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 ---
@@ -0,0 +1,344 @@
+/*
+ * 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 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._
+
+/**
+ * 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.
+ */
+private[kafka010] case class KafkaSource(
+sqlContext: SQLContext,
+consumerStrategy: ConsumerStrategy,
+executorKafkaParams: ju.Map[String, Object],
+sourceOptions: Map[String, String])
+  extends Source with Logging {
+
+  private val consumer = consumerStrategy.createConsumer()
+  private val sc = sqlContext.sparkContext
+  private val initialPartitionOffsets = fetchPartitionOffsets(seekToLatest 
= false)
+  logInfo(s"Initial offsets: $initialPartitionOffsets")
+
+  override def schema: StructType = KafkaSource.kafkaSchema
+
+  /** Returns the maximum available offset for this source. */
+  override def getOffset: Option[Offset] = {
+val offset = KafkaSourceOffset(fetchPartitionOffsets(seekToLatest = 
true))
+logInfo(s"GetOffset: $offset")
+Some(offset)
+  }
+
+  /** Returns the data that is between the 

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread tdas
Github user tdas commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80599175
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 ---
@@ -0,0 +1,344 @@
+/*
+ * 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 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._
+
+/**
+ * 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.
+ */
+private[kafka010] case class KafkaSource(
+sqlContext: SQLContext,
+consumerStrategy: ConsumerStrategy,
+executorKafkaParams: ju.Map[String, Object],
+sourceOptions: Map[String, String])
+  extends Source with Logging {
+
+  private val consumer = consumerStrategy.createConsumer()
+  private val sc = sqlContext.sparkContext
+  private val initialPartitionOffsets = fetchPartitionOffsets(seekToLatest 
= false)
+  logInfo(s"Initial offsets: $initialPartitionOffsets")
+
+  override def schema: StructType = KafkaSource.kafkaSchema
+
+  /** Returns the maximum available offset for this source. */
+  override def getOffset: Option[Offset] = {
+val offset = KafkaSourceOffset(fetchPartitionOffsets(seekToLatest = 
true))
+logInfo(s"GetOffset: $offset")
+Some(offset)
+  }
+
+  /** Returns the data that is between the 

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread tdas
Github user tdas commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80599004
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 ---
@@ -0,0 +1,344 @@
+/*
+ * 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 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._
+
+/**
+ * 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.
+ */
+private[kafka010] case class KafkaSource(
+sqlContext: SQLContext,
+consumerStrategy: ConsumerStrategy,
+executorKafkaParams: ju.Map[String, Object],
+sourceOptions: Map[String, String])
+  extends Source with Logging {
+
+  private val consumer = consumerStrategy.createConsumer()
+  private val sc = sqlContext.sparkContext
+  private val initialPartitionOffsets = fetchPartitionOffsets(seekToLatest 
= false)
+  logInfo(s"Initial offsets: $initialPartitionOffsets")
+
+  override def schema: StructType = KafkaSource.kafkaSchema
+
+  /** Returns the maximum available offset for this source. */
+  override def getOffset: Option[Offset] = {
+val offset = KafkaSourceOffset(fetchPartitionOffsets(seekToLatest = 
true))
+logInfo(s"GetOffset: $offset")
+Some(offset)
+  }
+
+  /** Returns the data that is between the 

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80567253
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 ---
@@ -0,0 +1,344 @@
+/*
+ * 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 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._
+
+/**
+ * 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.
+ */
+private[kafka010] case class KafkaSource(
+sqlContext: SQLContext,
+consumerStrategy: ConsumerStrategy,
+executorKafkaParams: ju.Map[String, Object],
+sourceOptions: Map[String, String])
+  extends Source with Logging {
+
+  private val consumer = consumerStrategy.createConsumer()
+  private val sc = sqlContext.sparkContext
+  private val initialPartitionOffsets = fetchPartitionOffsets(seekToLatest 
= false)
+  logInfo(s"Initial offsets: $initialPartitionOffsets")
+
+  override def schema: StructType = KafkaSource.kafkaSchema
+
+  /** Returns the maximum available offset for this source. */
+  override def getOffset: Option[Offset] = {
+val offset = KafkaSourceOffset(fetchPartitionOffsets(seekToLatest = 
true))
+logInfo(s"GetOffset: $offset")
+Some(offset)
+  }
+
+  /** Returns the data that is between 

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80564114
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala
 ---
@@ -0,0 +1,263 @@
+/*
+ * 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) = {
+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
+val uniqueGroupId = 
s"spark-kafka-source-${UUID.randomUUID}-${metadataPath.hashCode}"
--- End diff --

Comment why we are picking something random.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80563908
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala
 ---
@@ -0,0 +1,263 @@
+/*
+ * 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) = {
+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
+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 
unnecessaribly
--- End diff --

nit: spelling 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80568269
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 ---
@@ -0,0 +1,344 @@
+/*
+ * 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 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._
+
+/**
+ * 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.
+ */
+private[kafka010] case class KafkaSource(
+sqlContext: SQLContext,
+consumerStrategy: ConsumerStrategy,
+executorKafkaParams: ju.Map[String, Object],
+sourceOptions: Map[String, String])
+  extends Source with Logging {
+
+  private val consumer = consumerStrategy.createConsumer()
+  private val sc = sqlContext.sparkContext
+  private val initialPartitionOffsets = fetchPartitionOffsets(seekToLatest 
= false)
+  logInfo(s"Initial offsets: $initialPartitionOffsets")
+
+  override def schema: StructType = KafkaSource.kafkaSchema
+
+  /** Returns the maximum available offset for this source. */
+  override def getOffset: Option[Offset] = {
+val offset = KafkaSourceOffset(fetchPartitionOffsets(seekToLatest = 
true))
+logInfo(s"GetOffset: $offset")
+Some(offset)
+  }
+
+  /** Returns the data that is between 

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80568479
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 ---
@@ -0,0 +1,344 @@
+/*
+ * 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 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._
+
+/**
+ * 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.
+ */
+private[kafka010] case class KafkaSource(
+sqlContext: SQLContext,
+consumerStrategy: ConsumerStrategy,
+executorKafkaParams: ju.Map[String, Object],
+sourceOptions: Map[String, String])
+  extends Source with Logging {
+
+  private val consumer = consumerStrategy.createConsumer()
+  private val sc = sqlContext.sparkContext
+  private val initialPartitionOffsets = fetchPartitionOffsets(seekToLatest 
= false)
+  logInfo(s"Initial offsets: $initialPartitionOffsets")
+
+  override def schema: StructType = KafkaSource.kafkaSchema
+
+  /** Returns the maximum available offset for this source. */
+  override def getOffset: Option[Offset] = {
+val offset = KafkaSourceOffset(fetchPartitionOffsets(seekToLatest = 
true))
+logInfo(s"GetOffset: $offset")
+Some(offset)
+  }
+
+  /** Returns the data that is between 

[GitHub] spark pull request #15102: [SPARK-17346][SQL] Add Kafka source for Structure...

2016-09-26 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/15102#discussion_r80568553
  
--- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala
 ---
@@ -0,0 +1,263 @@
+/*
+ * 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],
--- End diff --

Should we error if they specify a schema, rather than just ignore it?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



  1   2   >