From 985cfa9625cdce900e187f8708a832c1262f14b5 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 13 Sep 2016 17:04:05 -0700 Subject: [PATCH 01/28] Kafka source project --- external/kafka-0-10-sql/pom.xml | 82 ++++ ...pache.spark.sql.sources.DataSourceRegister | 1 + .../sql/kafka010/CachedKafkaConsumer.scala | 186 ++++++++ .../spark/sql/kafka010/KafkaSource.scala | 436 ++++++++++++++++++ .../spark/sql/kafka010/KafkaSourceRDD.scala | 204 ++++++++ .../spark/sql/kafka010/package-info.java | 21 + .../src/test/resources/log4j.properties | 28 ++ .../sql/kafka010/KafkaSourceOffsetSuite.scala | 47 ++ .../spark/sql/kafka010/KafkaSourceSuite.scala | 227 +++++++++ .../spark/sql/kafka010/KafkaTestUtils.scala | 286 ++++++++++++ pom.xml | 1 + project/SparkBuild.scala | 4 +- 12 files changed, 1521 insertions(+), 2 deletions(-) create mode 100644 external/kafka-0-10-sql/pom.xml create mode 100644 external/kafka-0-10-sql/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister create mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala create mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala create mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala create mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/package-info.java create mode 100644 external/kafka-0-10-sql/src/test/resources/log4j.properties create mode 100644 external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala create mode 100644 external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala create mode 100644 external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml new file mode 100644 index 000000000000..b7ead0ebfa55 --- /dev/null +++ b/external/kafka-0-10-sql/pom.xml @@ -0,0 +1,82 @@ + + + + + 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 + http://spark.apache.org/ + + + + org.apache.spark + spark-sql_${scala.binary.version} + ${project.version} + provided + + + org.apache.spark + spark-sql_${scala.binary.version} + ${project.version} + test-jar + test + + + org.apache.kafka + kafka-clients + 0.10.0.1 + + + org.apache.kafka + kafka_${scala.binary.version} + 0.10.0.1 + test + + + net.sf.jopt-simple + jopt-simple + 3.2 + test + + + org.scalacheck + scalacheck_${scala.binary.version} + test + + + org.apache.spark + spark-tags_${scala.binary.version} + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + diff --git a/external/kafka-0-10-sql/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/external/kafka-0-10-sql/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister new file mode 100644 index 000000000000..2f9e9fc0396d --- /dev/null +++ b/external/kafka-0-10-sql/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister @@ -0,0 +1 @@ +org.apache.spark.sql.kafka010.KafkaSourceProvider diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala new file mode 100644 index 000000000000..df188dfec0ee --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala @@ -0,0 +1,186 @@ +/* + * 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.{KafkaException, TopicPartition} + +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. + */ +class CachedKafkaConsumer[K, V] private( + val groupId: String, + val topic: String, + val partition: Int, + val kafkaParams: ju.Map[String, Object]) extends Logging { + + assert(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG), + "groupId used for cache key must match the groupId in kafkaParams") + + val topicPartition = new TopicPartition(topic, partition) + + protected val consumer = { + val c = new KafkaConsumer[K, V](kafkaParams) + val tps = new ju.ArrayList[TopicPartition]() + tps.add(topicPartition) + c.assign(tps) + c + } + + // TODO if the buffer was kept around as a random-access structure, + // could possibly optimize re-calculating of an RDD in the same batch + protected var buffer = ju.Collections.emptyList[ConsumerRecord[K, V]]().iterator + protected var nextOffset = -2L + + def close(): Unit = consumer.close() + + /** + * 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, timeout: Long): ConsumerRecord[K, V] = { + logDebug(s"Get $groupId $topic $partition nextOffset $nextOffset requested $offset") + if (offset != nextOffset) { + logInfo(s"Initial fetch for $groupId $topic $partition $offset") + seek(offset) + poll(timeout) + } + + if (!buffer.hasNext()) { poll(timeout) } + assert(buffer.hasNext(), + s"Failed to get records for $groupId $topic $partition $offset after polling for $timeout") + var record = buffer.next() + + if (record.offset != offset) { + logInfo(s"Buffer miss for $groupId $topic $partition $offset") + seek(offset) + poll(timeout) + assert(buffer.hasNext(), + s"Failed to get records for $groupId $topic $partition $offset after polling for $timeout") + record = buffer.next() + assert(record.offset == offset, + s"Got wrong record for $groupId $topic $partition even after seeking to offset $offset") + } + + nextOffset = offset + 1 + record + } + + private def seek(offset: Long): Unit = { + logDebug(s"Seeking to $topicPartition $offset") + consumer.seek(topicPartition, offset) + } + + private def poll(timeout: Long): Unit = { + val p = consumer.poll(timeout) + val r = p.records(topicPartition) + logDebug(s"Polled ${p.partitions()} ${r.size}") + buffer = r.iterator + } + +} + +object CachedKafkaConsumer extends Logging { + + private case class CacheKey(groupId: String, topic: String, partition: Int) + + // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap + private var cache: ju.LinkedHashMap[CacheKey, CachedKafkaConsumer[_, _]] = null + + /** Must be called before get, once per JVM, to configure the cache. Further calls are ignored */ + def init( + initialCapacity: Int, + maxCapacity: Int, + loadFactor: Float): Unit = CachedKafkaConsumer.synchronized { + if (null == cache) { + logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor") + cache = new ju.LinkedHashMap[CacheKey, CachedKafkaConsumer[_, _]]( + initialCapacity, loadFactor, true) { + override def removeEldestEntry( + entry: ju.Map.Entry[CacheKey, CachedKafkaConsumer[_, _]]): Boolean = { + if (this.size > maxCapacity) { + try { + entry.getValue.consumer.close() + } catch { + case x: KafkaException => + logError("Error closing oldest Kafka consumer", x) + } + true + } else { + false + } + } + } + } + } + + /** + * Get a cached consumer for groupId, assigned to topic and partition. + * If matching consumer doesn't already exist, will be created using kafkaParams. + */ + def get[K, V]( + groupId: String, + topic: String, + partition: Int, + kafkaParams: ju.Map[String, Object]): CachedKafkaConsumer[K, V] = + CachedKafkaConsumer.synchronized { + val k = CacheKey(groupId, topic, partition) + val v = cache.get(k) + if (null == v) { + logInfo(s"Cache miss for $k") + logDebug(cache.keySet.toString) + val c = new CachedKafkaConsumer[K, V](groupId, topic, partition, kafkaParams) + cache.put(k, c) + c + } else { + // any given topicpartition should have a consistent key and value type + v.asInstanceOf[CachedKafkaConsumer[K, V]] + } + } + + /** + * Get a fresh new instance, unassociated with the global cache. + * Caller is responsible for closing + */ + def getUncached[K, V]( + groupId: String, + topic: String, + partition: Int, + kafkaParams: ju.Map[String, Object]): CachedKafkaConsumer[K, V] = + new CachedKafkaConsumer[K, V](groupId, topic, partition, kafkaParams) + + /** remove consumer for given groupId, topic, and partition, if it exists */ + def remove(groupId: String, topic: String, partition: Int): Unit = { + val k = CacheKey(groupId, topic, partition) + logInfo(s"Removing $k from cache") + val v = CachedKafkaConsumer.synchronized { + cache.remove(k) + } + if (null != v) { + v.close() + logInfo(s"Removed $k from cache") + } + } +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala new file mode 100644 index 000000000000..81483d86526c --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -0,0 +1,436 @@ +/* + * 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.catalyst.encoders.ExpressionEncoder +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.StructType +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 + * [[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[Array[Byte], Array[Byte]], + executorKafkaParams: ju.Map[String, Object], + sourceOptions: Map[String, String]) + extends Source with Logging { + + implicit private val encoder = ExpressionEncoder.tuple( + ExpressionEncoder[Array[Byte]](), ExpressionEncoder[Array[Byte]]()) + + @transient private val consumer = consumerStrategy.createConsumer() + @transient private val sc = sqlContext.sparkContext + @transient private val initialPartitionOffsets = fetchPartitionOffsets(seekToLatest = false) + logInfo(s"Initial offsets: " + initialPartitionOffsets) + + override def schema: StructType = encoder.schema + + /** 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 offsets [`start`, `end`), i.e. end is exclusive. + */ + override def getBatch(start: Option[Offset], end: Offset): DataFrame = { + logDebug(s"GetBatch called with start = $start, end = $end") + val untilPartitionOffsets = KafkaSourceOffset.getPartitionOffsets(end) + val fromPartitionOffsets = start match { + case Some(prevBatchEndOffset) => + KafkaSourceOffset.getPartitionOffsets(prevBatchEndOffset) + case None => + initialPartitionOffsets + } + + // Sort the partitions and current list of executors to consistently assign each partition + // to the executor. This allows cached KafkaConsumers in the executors to be re-used to + // read the same partition in every batch. + val topicPartitionOrdering = new Ordering[TopicPartition] { + override def compare(l: TopicPartition, r: TopicPartition): Int = { + implicitly[Ordering[(String, Long)]].compare( + (l.topic, l.partition), + (r.topic, r.partition)) + } + } + val sortedTopicPartitions = untilPartitionOffsets.keySet.toSeq.sorted(topicPartitionOrdering) + val sortedExecutors = getSortedExecutorList(sc) + val numExecutors = sortedExecutors.size + logDebug("Sorted executors: " + sortedExecutors.mkString(", ")) + val offsetRanges = sortedTopicPartitions.flatMap { tp => + fromPartitionOffsets.get(tp).map { fromOffset => + val untilOffset = untilPartitionOffsets(tp) + val preferredLoc = if (numExecutors > 0) { + Some(sortedExecutors(positiveMod(tp.hashCode, numExecutors))) + } else None + KafkaSourceRDD.OffsetRange(tp, fromOffset, untilOffset, preferredLoc) + } + }.toArray + + // Create a RDD that reads from Kafka and get the (key, value) pair as byte arrays. + val rdd = new KafkaSourceRDD[Array[Byte], Array[Byte]]( + sc, executorKafkaParams, offsetRanges, sourceOptions) + .map { r => (r.key, r.value) } + + logInfo("GetBatch: " + offsetRanges.sortBy(_.topicPartition.toString).mkString(", ")) + sqlContext.createDataset(rdd).toDF("key", "value") + } + + /** Stop this source and free any resources it has allocated. */ + override def stop(): Unit = synchronized { + consumer.close() + } + + override def toString(): String = s"KafkaSource[$consumerStrategy]" + + private def fetchPartitionOffsets(seekToLatest: Boolean): Map[TopicPartition, Long] = { + synchronized { + logTrace("\tPolling") + consumer.poll(0) + val partitions = consumer.assignment() + consumer.pause(partitions) + logDebug(s"\tPartitioned assigned to consumer: $partitions") + if (seekToLatest) { + consumer.seekToEnd(partitions) + logDebug("\tSeeked to the end") + } + logTrace("Getting positions") + val partitionToOffsets = partitions.asScala.map(p => p -> consumer.position(p)) + logDebug(s"Got positions $partitionToOffsets") + partitionToOffsets.toMap + } + } + + private def positiveMod(a: Long, b: Int): Int = ((a % b).toInt + b) % b +} + +/** Companion object for the [[KafkaSource]]. */ +private[kafka010] object KafkaSource { + + sealed trait ConsumerStrategy[K, V] { + def createConsumer(): Consumer[K, V] + } + + case class SubscribeStrategy[K, V](topics: Seq[String], kafkaParams: ju.Map[String, Object]) + extends ConsumerStrategy[K, V] { + override def createConsumer(): Consumer[K, V] = { + val consumer = new KafkaConsumer[K, V](kafkaParams) + consumer.subscribe(topics.asJava) + consumer.poll(0) + consumer + } + + override def toString: String = s"Subscribe[${topics.mkString(", ")}]" + } + + case class SubscribePatternStrategy[K, V]( + topicPattern: String, kafkaParams: ju.Map[String, Object]) + extends ConsumerStrategy[K, V] { + override def createConsumer(): Consumer[K, V] = { + val consumer = new KafkaConsumer[K, V](kafkaParams) + consumer.subscribe( + ju.regex.Pattern.compile(topicPattern), + new NoOpConsumerRebalanceListener()) + consumer.poll(0) + consumer + } + + override def toString: String = s"SubscribePattern[$topicPattern]" + } + + def getSortedExecutorList(sc: SparkContext): Array[String] = { + def compare(a: ExecutorCacheTaskLocation, b: ExecutorCacheTaskLocation): Boolean = { + if (a.host == b.host) { a.executorId > b.executorId } else { a.host > b.host } + } + + val bm = sc.env.blockManager + bm.master.getPeers(bm.blockManagerId).toArray + .map(x => ExecutorCacheTaskLocation(x.host, x.executorId)) + .sortWith(compare) + .map(_.toString) + } +} + + +/** An [[Offset]] for the [[KafkaSource]]. */ +private[kafka010] +case class KafkaSourceOffset(partitionToOffsets: Map[TopicPartition, Long]) extends Offset { + /** + * Returns a negative integer, zero, or a positive integer as this object is less than, equal to, + * or greater than the specified object. + */ + override def compareTo(other: Offset): Int = other match { + case KafkaSourceOffset(otherOffsets) => + val allTopicAndPartitions = (this.partitionToOffsets.keySet ++ otherOffsets.keySet).toSeq + + val comparisons = allTopicAndPartitions.map { tp => + (this.partitionToOffsets.get(tp), otherOffsets.get(tp)) match { + case (Some(a), Some(b)) => + if (a < b) { + -1 + } else if (a > b) { + 1 + } else { + 0 + } + case (None, _) => -1 + case (_, None) => 1 + } + } + val nonZeroSigns = comparisons.filter { _ != 0 }.toSet + nonZeroSigns.size match { + case 0 => 0 // if both empty or only 0s + case 1 => nonZeroSigns.head // if there are only (0s and 1s) or (0s and -1s) + case _ => // there are both 1s and -1s + throw new IllegalArgumentException( + s"Invalid comparison between non-linear histories: $this <=> $other") + } + + case _ => + throw new IllegalArgumentException(s"Cannot compare $this <=> $other") + } + + override def toString(): String = { + partitionToOffsets.toSeq.sortBy(_._1.toString).mkString("[", ", ", "]") + } +} + +/** Companion object of the [[KafkaSourceOffset]] */ +private[kafka010] object KafkaSourceOffset { + + def getPartitionOffsets(offset: Offset): Map[TopicPartition, Long] = { + offset match { + case o: KafkaSourceOffset => o.partitionToOffsets + case _ => + throw new IllegalArgumentException( + s"Invalid conversion from offset of ${offset.getClass} to KafkaSourceOffset") + } + } + + /** + * Returns [[KafkaSourceOffset]] from a variable sequence of (topic, partitionId, offset) + * tuples. + */ + def apply(offsetTuples: (String, Int, Long)*): KafkaSourceOffset = { + KafkaSourceOffset(offsetTuples.map { case(t, p, o) => (new TopicPartition(t, p), o) }.toMap) + } +} + + +/** + * 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 { + private val structType = new StructType().add("key", "binary").add("value", "binary") + private val strategyOptionNames = Set("subscribe", "subscribepattern") + + /** Class to conveniently update Kafka config params, while logging the changes */ + private case class ConfigUpdater(module: String, kafkaParams: Map[String, String]) { + private val map = new ju.HashMap[String, Object](kafkaParams.asJava) + def set(key: String, value: Object): this.type = { + map.put(key, value) + logInfo(s"$module: Set $key to $value, earlier value: ${kafkaParams.get(key).getOrElse("")}") + this + } + + def setIfUnset(key: String, value: Object): ConfigUpdater = { + if (!map.containsKey(key)) { + map.put(key, value) + logInfo(s"$module: Set $key to $value") + } + this + } + + def build(): ju.Map[String, Object] = map + } + + /** + * 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", structType) + } + + 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 kafkaParamsForStrategy = + ConfigUpdater("source", specifiedKafkaParams) + .set(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, deserClassName) + .set(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, deserClassName) + .build() + + val kafkaParamsForExecutors = + ConfigUpdater("source", specifiedKafkaParams) + .set(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, deserClassName) + .set(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, deserClassName) + + // So that consumers in executors never throw NoOffsetForPartitionException + .set(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none") + + // So that consumers in executors do not mess with user-specified group id + .set(ConsumerConfig.GROUP_ID_CONFIG, + "spark-executor-" + specifiedKafkaParams(ConsumerConfig.GROUP_ID_CONFIG)) + + // So that consumers in executors no keep committing offsets unnecessaribly + .set(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") + + // If buffer config is not set, it to reasonable value to work around + // buffer issues (see KAFKA-3135) + .setIfUnset(ConsumerConfig.RECEIVE_BUFFER_CONFIG, 65536: java.lang.Integer) + .build() + + val strategy = caseInsensitiveParams.find(x => strategyOptionNames.contains(x._1)).get match { + case ("subscribe", value) => + SubscribeStrategy[Array[Byte], Array[Byte]]( + value.split(",").map(_.trim()).filter(_.nonEmpty), + kafkaParamsForStrategy) + case ("subscribepattern", value) => + SubscribePatternStrategy[Array[Byte], Array[Byte]]( + value.trim(), + kafkaParamsForStrategy) + case _ => + // Should never reach here as we are already matching on + // matched strategy names + throw new IllegalArgumentException("Unknown option") + } + + new KafkaSource(sqlContext, strategy, kafkaParamsForExecutors, parameters) + } + + private def validateOptions(parameters: Map[String, String]): Unit = { + val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase, v) } + val specifiedStrategies = + caseInsensitiveParams.filter { case(k, _) => strategyOptionNames.contains(k) }.toSeq + if (specifiedStrategies.isEmpty) { + throw new IllegalArgumentException( + "One of the following options must be specified for Kafka source: " + + strategyOptionNames.mkString(", ") + ". See docs for more details.") + } else if (specifiedStrategies.size > 1) { + throw new IllegalArgumentException( + "Only one of the following options can be specified for Kafka source: " + + strategyOptionNames.mkString(", ") + ". See docs for more details.") + } + + val strategy = caseInsensitiveParams.find(x => strategyOptionNames.contains(x._1)).get match { + case ("subscribe", value) => + val topics = value.split(",").map(_.trim).filter(_.nonEmpty) + if (topics.isEmpty) { + throw new IllegalArgumentException( + "No topics to subscribe to as specified value for option " + + s"'subscribe' is '$value'") + } + case ("subscribepattern", value) => + val pattern = caseInsensitiveParams("subscribepattern").trim() + if (pattern.isEmpty) { + throw new IllegalArgumentException( + "Pattern to subscribe is empty as specified value for option " + + s"'subscribePattern' is '$value'") + } + case _ => + // Should never reach here as we are already matching on + // matched strategy names + throw new IllegalArgumentException("Unknown option") + } + + if (!caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG}")) { + throw new IllegalArgumentException( + "Option 'kafka.bootstrap.servers' must be specified for configuring Kafka consumer") + } + + if (!caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.GROUP_ID_CONFIG}")) { + throw new IllegalArgumentException( + "Option 'kafka.group.id' must be specified for configuring Kafka consumer") + } + } + + override def shortName(): String = "kafka" +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala new file mode 100644 index 000000000000..249d124aff4f --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala @@ -0,0 +1,204 @@ +/* + * 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 scala.util.Try + +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord} +import org.apache.kafka.common.TopicPartition + +import org.apache.spark.{Partition, SparkContext, TaskContext} +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.kafka010.KafkaSourceRDD._ +import org.apache.spark.storage.StorageLevel + + +private[kafka010] +case class KafkaSourceRDDPartition(index: Int, offsetRange: OffsetRange) 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 + * @param sourceOptions Options provided through the source + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + */ +private[kafka010] class KafkaSourceRDD[K, V]( + sc: SparkContext, + executorKafkaParams: ju.Map[String, Object], + offsetRanges: Seq[OffsetRange], + sourceOptions: Map[String, String]) extends RDD[ConsumerRecord[K, V]](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[K, V]] = { + val nonEmptyPartitions = + this.partitions.map(_.asInstanceOf[KafkaSourceRDDPartition]).filter(_.offsetRange.size > 0) + + if (num < 1 || nonEmptyPartitions.isEmpty) { + return new Array[ConsumerRecord[K, V]](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[K, V]] + val res = context.runJob( + this, + (tc: TaskContext, it: Iterator[ConsumerRecord[K, V]]) => + it.take(parts(tc.partitionId)).toArray, parts.keys.toArray + ) + res.foreach(buf ++= _) + buf.toArray + } + + override def compute(thePart: Partition, context: TaskContext): Iterator[ConsumerRecord[K, V]] = { + val range = thePart.asInstanceOf[KafkaSourceRDDPartition].offsetRange + assert( + range.fromOffset <= range.untilOffset, + s"Beginning offset ${range.fromOffset} is after the ending offset ${range.untilOffset} " + + s"for topic ${range.topic} partition ${range.partition}. " + + "You either provided an invalid fromOffset, or the Kafka topic has been damaged") + if (range.fromOffset == range.untilOffset) { + logInfo(s"Beginning offset ${range.fromOffset} is the same as ending offset " + + s"skipping ${range.topic} ${range.partition}") + Iterator.empty + } else { + new KafkaRDDIterator(range, executorKafkaParams, sourceOptions, context) + } + } +} + +private[kafka010] object KafkaSourceRDD { + + /** Offset range that one partition of the KafkaSourceRDD has to read */ + case class OffsetRange( + topicPartition: TopicPartition, + fromOffset: Long, + untilOffset: Long, + preferredLoc: Option[String]) { + def topic: String = topicPartition.topic + def partition: Int = topicPartition.partition + def size: Long = untilOffset - fromOffset + } + + /** + * An iterator that fetches messages directly from Kafka for the offsets in partition. + * Uses a cached consumer where possible to take advantage of prefetching + */ + private class KafkaRDDIterator[K, V]( + part: OffsetRange, + executorKafkaParams: ju.Map[String, Object], + options: Map[String, String], + context: TaskContext) extends Iterator[ConsumerRecord[K, V]] with Logging { + + logInfo(s"Computing topic ${part.topic}, partition ${part.partition} " + + s"offsets ${part.fromOffset} -> ${part.untilOffset}") + + val groupId = executorKafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG).asInstanceOf[String] + + // Time between polling for more data by the KafkaConsumer in the executor. This should not + // require much configuration as data should already be available in Kafka when the executors + // are polling. + private val pollTimeout = + getLong(options, "consumer.pollMs", 512) + + // Configurations for initializing the cache of KafkaConsumers. + private val cacheInitialCapacity = + getInt(options, "consumer.cache.initialCapacity", 16) + private val cacheMaxCapacity = + getInt(options, "consumer.cache.maxCapacity", 64) + private val cacheLoadFactor = + getDouble(options, "consumer.cache.loadFactor", 0.75).toFloat + + // Initialize the cache if not already done, and get a cached KafkaConsumer + CachedKafkaConsumer.init(cacheInitialCapacity, cacheMaxCapacity, cacheLoadFactor) + if (context.attemptNumber > 1) { + // Just in case the prior attempt failures were cache related + CachedKafkaConsumer.remove(groupId, part.topic, part.partition) + } + val consumer = + CachedKafkaConsumer.get[K, V](groupId, part.topic, part.partition, executorKafkaParams) + + var requestOffset = part.fromOffset + + override def hasNext(): Boolean = requestOffset < part.untilOffset + + override def next(): ConsumerRecord[K, V] = { + assert(hasNext(), "Can't call next() once untilOffset has been reached") + val r = consumer.get(requestOffset, pollTimeout) + requestOffset += 1 + r + } + } + + def getInt(options: Map[String, String], name: String, defaultValue: Int): Int = { + options.get(name).map { str => + Try(str.toInt).getOrElse { + throw new IllegalArgumentException("Option '$name' must be a integer") + } + }.getOrElse(defaultValue) + } + + def getDouble(options: Map[String, String], name: String, defaultValue: Double): Double = { + options.get(name).map { str => + Try(str.toDouble).getOrElse { + throw new IllegalArgumentException("Option '$name' must be a double") + } + }.getOrElse(defaultValue) + } + + def getLong(options: Map[String, String], name: String, defaultValue: Long): Long = { + options.get(name).map { str => + Try(str.toLong).getOrElse { + throw new IllegalArgumentException("Option '$name' must be a long") + } + }.getOrElse(defaultValue) + } +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/package-info.java b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/package-info.java new file mode 100644 index 000000000000..596f775c56db --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Structured Streaming Data Source for Kafka 0.10 + */ +package org.apache.spark.sql.kafka010; diff --git a/external/kafka-0-10-sql/src/test/resources/log4j.properties b/external/kafka-0-10-sql/src/test/resources/log4j.properties new file mode 100644 index 000000000000..75e3b53a093f --- /dev/null +++ b/external/kafka-0-10-sql/src/test/resources/log4j.properties @@ -0,0 +1,28 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Set everything to be logged to the file target/unit-tests.log +log4j.rootCategory=INFO, file +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=true +log4j.appender.file.file=target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +log4j.logger.org.spark-project.jetty=WARN + diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala new file mode 100644 index 000000000000..c6ec94e874c3 --- /dev/null +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.kafka010 + +import org.apache.spark.sql.streaming.OffsetSuite + +class KafkaSourceOffsetSuite extends OffsetSuite { + + compare( + one = KafkaSourceOffset(("t", 0, 1L)), + two = KafkaSourceOffset(("t", 0, 2L))) + + compare( + one = KafkaSourceOffset(("t", 0, 1L), ("t", 1, 0L)), + two = KafkaSourceOffset(("t", 0, 2L), ("t", 1, 1L))) + + compare( + one = KafkaSourceOffset(("t", 0, 1L), ("T", 0, 0L)), + two = KafkaSourceOffset(("t", 0, 2L), ("T", 0, 1L))) + + compare( + one = KafkaSourceOffset(("t", 0, 1L)), + two = KafkaSourceOffset(("t", 0, 2L), ("t", 1, 1L))) + + compareInvalid( + one = KafkaSourceOffset(("t", 1, 1L)), + two = KafkaSourceOffset(("t", 0, 2L))) + + compareInvalid( + one = KafkaSourceOffset(("t", 0, 1L)), + two = KafkaSourceOffset(("T", 0, 2L))) +} diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala new file mode 100644 index 000000000000..3cb4c48f4dbd --- /dev/null +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala @@ -0,0 +1,227 @@ +/* + * 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.time.SpanSugar._ + +import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.streaming.StreamTest +import org.apache.spark.sql.test.SharedSQLContext + + +class KafkaSourceSuite extends StreamTest with SharedSQLContext { + + import testImplicits._ + + private val topicId = new AtomicInteger(0) + private var testUtils: KafkaTestUtils = _ + + override val streamingTimeout = 10.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() + } + } + + test("subscribing topic by name from latest offsets") { + val topic = newTopic() + testFromLatestOffsets(topic, "subscribe" -> topic) + } + + test("subscribing topic by name from earliest offsets") { + val topic = newTopic() + testFromEarliestOffsets(topic, "subscribe" -> topic) + } + + test("subscribing topic by pattern from latest offsets") { + val topicPrefix = newTopic() + val topic = topicPrefix + "-suffix" + testFromLatestOffsets(topic, "subscribePattern" -> s"$topicPrefix-.*") + } + + test("subscribing topic by pattern from earliest offsets") { + val topicPrefix = newTopic() + val topic = topicPrefix + "-suffix" + testFromEarliestOffsets(topic, "subscribePattern" -> s"$topicPrefix-.*") + } + + test("stress test with multiple topics and partitions") { + val topics = (1 to 5).map(i => s"stress$i").toSet + topics.foreach { topic => + testUtils.createTopic(topic, partitions = Random.nextInt(5) + 1) + testUtils.sendMessages(topic, (101 to 105).map { _.toString }.toArray) + } + + // Create Kafka source that reads from latest offset + val kafka = + spark.readStream + .format(classOf[KafkaSourceProvider].getCanonicalName.stripSuffix("$")) + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.group.id", s"group-stress-test") + .option("subscribe", topics.mkString(",")) + .load() + .as[(Array[Byte], Array[Byte])] + + val mapped = kafka.map(kv => new String(kv._2).toInt + 1) + + runStressTest( + mapped, + d => AddKafkaData(topics, d: _*)(ensureDataInMultiplePartition = false), + iterations = 50) + } + + test("bad source options") { + def testBadOptions(options: (String, String)*)(expectedMsgs: String*): Unit = { + val ex = intercept[IllegalArgumentException] { + val reader = spark + .readStream + .format("kafka") + options.foreach { case (k, v) => reader.option(k, v) } + reader.load() + } + expectedMsgs.foreach { m => + assert(ex.getMessage.toLowerCase.contains(m.toLowerCase)) + } + } + + // No strategy specified + testBadOptions()("options must be specified", "subscribe", "subscribePattern") + + // Multiple strategies specified + testBadOptions("subscribe" -> "t", "subscribePattern" -> "t.*")( + "only one", "options can be specified") + + testBadOptions("subscribe" -> "")("no topics to subscribe") + testBadOptions("subscribePattern" -> "")("pattern to subscribe is empty") + } + + + private def newTopic(): String = s"topic-${topicId.getAndIncrement()}" + + private def testFromLatestOffsets(topic: String, options: (String, String)*): Unit = { + testUtils.createTopic(topic, partitions = 5) + testUtils.sendMessages(topic, Array("-1")) + require(testUtils.getLatestOffsets(Set(topic)).size === 5) + + val reader = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.group.id", s"group-$topic") + .option("kafka.auto.offset.reset", s"latest") + options.foreach { case (k, v) => reader.option(k, v) } + val kafka = reader.load().as[(Array[Byte], Array[Byte])] + val mapped = kafka.map(kv => new String(kv._2).toInt + 1) + + testStream(mapped)( + AddKafkaData(Set(topic), 1, 2, 3), + CheckAnswer(2, 3, 4), + StopStream, + StartStream(), + CheckAnswer(2, 3, 4), // Should get the data back on recovery + StopStream, + AddKafkaData(Set(topic), 4, 5, 6), // Add data when stream is stopped + StartStream(), + CheckAnswer(2, 3, 4, 5, 6, 7), // Should get the added data + AddKafkaData(Set(topic), 7, 8), + CheckAnswer(2, 3, 4, 5, 6, 7, 8, 9) + ) + } + + private def testFromEarliestOffsets(topic: String, options: (String, String)*): Unit = { + testUtils.createTopic(topic, partitions = 5) + testUtils.sendMessages(topic, (1 to 3).map { _.toString }.toArray) + require(testUtils.getLatestOffsets(Set(topic)).size === 5) + + val reader = spark.readStream + reader + .format(classOf[KafkaSourceProvider].getCanonicalName.stripSuffix("$")) + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.group.id", s"group-$topic") + .option("kafka.auto.offset.reset", s"earliest") + options.foreach { case (k, v) => reader.option(k, v) } + val kafka = reader.load().as[(Array[Byte], Array[Byte])] + val mapped = kafka.map(kv => new String(kv._2).toInt + 1) + + testStream(mapped)( + AddKafkaData(Set(topic), 4, 5, 6), // Add data when stream is stopped + CheckAnswer(2, 3, 4, 5, 6, 7), + StopStream, + StartStream(), + CheckAnswer(2, 3, 4, 5, 6, 7), + StopStream, + AddKafkaData(Set(topic), 7, 8), + StartStream(), + CheckAnswer(2, 3, 4, 5, 6, 7, 8, 9) + ) + } + + case class AddKafkaData(topics: Set[String], data: Int*) + (implicit ensureDataInMultiplePartition: Boolean = false) extends AddData { + + override def addData(query: Option[StreamExecution]): (Source, Offset) = { + 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()}" + } + // Verify that the test data gets inserted into multiple partitions + if (ensureDataInMultiplePartition) { + require( + sentMetadata.groupBy(_._2.partition).size > 1, + s"Added data does not test multiple partitions: ${sentMetadata.map(metadataToStr)}") + } + + val offset = KafkaSourceOffset(testUtils.getLatestOffsets(topics)) + logInfo(s"Added data, expected offset $offset") + (kafkaSource, offset) + } + } +} diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala new file mode 100644 index 000000000000..5d1f9fec6091 --- /dev/null +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala @@ -0,0 +1,286 @@ +/* + * 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.io.File +import java.lang.{Integer => JInt} +import java.net.InetSocketAddress +import java.util.{Map => JMap, Properties} +import java.util.concurrent.TimeUnit + +import scala.collection.JavaConverters._ +import scala.language.postfixOps +import scala.util.Random + +import kafka.admin.AdminUtils +import kafka.api.Request +import kafka.server.{KafkaConfig, KafkaServer} +import kafka.utils.ZkUtils +import org.apache.kafka.clients.consumer.KafkaConsumer +import org.apache.kafka.clients.producer._ +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.serialization.{StringDeserializer, StringSerializer} +import org.apache.zookeeper.server.{NIOServerCnxnFactory, ZooKeeperServer} +import org.scalatest.concurrent.Eventually._ +import org.scalatest.time.SpanSugar._ + +import org.apache.spark.internal.Logging +import org.apache.spark.util.Utils +import org.apache.spark.SparkConf + +/** + * This is a helper class for Kafka test suites. This has the functionality to set up + * and tear down local Kafka servers, and to push data using Kafka producers. + * + * The reason to put Kafka test utility class in src is to test Python related Kafka APIs. + */ +class KafkaTestUtils extends Logging { + + // Zookeeper related configurations + private val zkHost = "localhost" + private var zkPort: Int = 0 + private val zkConnectionTimeout = 60000 + private val zkSessionTimeout = 6000 + + private var zookeeper: EmbeddedZookeeper = _ + + private var zkUtils: ZkUtils = _ + + // Kafka broker related configurations + private val brokerHost = "localhost" + private var brokerPort = 0 + private var brokerConf: KafkaConfig = _ + + // Kafka broker server + private var server: KafkaServer = _ + + // Kafka producer + private var producer: Producer[String, String] = _ + + // Flag to test whether the system is correctly started + private var zkReady = false + private var brokerReady = false + + def zkAddress: String = { + assert(zkReady, "Zookeeper not setup yet or already torn down, cannot get zookeeper address") + s"$zkHost:$zkPort" + } + + def brokerAddress: String = { + assert(brokerReady, "Kafka not setup yet or already torn down, cannot get broker address") + s"$brokerHost:$brokerPort" + } + + def zookeeperClient: ZkUtils = { + assert(zkReady, "Zookeeper not setup yet or already torn down, cannot get zookeeper client") + Option(zkUtils).getOrElse( + throw new IllegalStateException("Zookeeper client is not yet initialized")) + } + + // Set up the Embedded Zookeeper server and get the proper Zookeeper port + private def setupEmbeddedZookeeper(): Unit = { + // Zookeeper server startup + zookeeper = new EmbeddedZookeeper(s"$zkHost:$zkPort") + // Get the actual zookeeper binding port + zkPort = zookeeper.actualPort + zkUtils = ZkUtils(s"$zkHost:$zkPort", zkSessionTimeout, zkConnectionTimeout, false) + zkReady = true + } + + // Set up the Embedded Kafka server + private def setupEmbeddedKafkaServer(): Unit = { + assert(zkReady, "Zookeeper should be set up beforehand") + + // Kafka broker startup + Utils.startServiceOnPort(brokerPort, port => { + brokerPort = port + brokerConf = new KafkaConfig(brokerConfiguration, doLog = false) + server = new KafkaServer(brokerConf) + server.startup() + brokerPort = server.boundPort() + (server, brokerPort) + }, new SparkConf(), "KafkaBroker") + + brokerReady = true + } + + /** setup the whole embedded servers, including Zookeeper and Kafka brokers */ + def setup(): Unit = { + setupEmbeddedZookeeper() + setupEmbeddedKafkaServer() + } + + /** Teardown the whole servers, including Kafka broker and Zookeeper */ + def teardown(): Unit = { + brokerReady = false + zkReady = false + + if (producer != null) { + producer.close() + producer = null + } + + if (server != null) { + server.shutdown() + server = null + } + + brokerConf.logDirs.foreach { f => Utils.deleteRecursively(new File(f)) } + + if (zkUtils != null) { + zkUtils.close() + zkUtils = null + } + + if (zookeeper != null) { + zookeeper.shutdown() + zookeeper = null + } + } + + /** Create a Kafka topic and wait until it is propagated to the whole cluster */ + def createTopic(topic: String, partitions: Int): Unit = { + AdminUtils.createTopic(zkUtils, topic, partitions, 1) + // wait until metadata is propagated + (0 until partitions).foreach { p => + waitUntilMetadataIsPropagated(topic, p) + } + } + + /** Create a Kafka topic and wait until it is propagated to the whole cluster */ + def createTopic(topic: String): Unit = { + createTopic(topic, 1) + } + + /** Java-friendly function for sending messages to the Kafka broker */ + def sendMessages(topic: String, messageToFreq: JMap[String, JInt]): Unit = { + sendMessages(topic, Map(messageToFreq.asScala.mapValues(_.intValue()).toSeq: _*)) + } + + /** Send the messages to the Kafka broker */ + def sendMessages(topic: String, messageToFreq: Map[String, Int]): Unit = { + val messages = messageToFreq.flatMap { case (s, freq) => Seq.fill(freq)(s) }.toArray + sendMessages(topic, messages) + } + + /** Send the array of messages to the Kafka broker */ + def sendMessages(topic: String, messages: Array[String]): Seq[(String, RecordMetadata)] = { + producer = new KafkaProducer[String, String](producerConfiguration) + val offsets = try { + messages.map { m => + val metadata = + producer.send(new ProducerRecord[String, String](topic, m)).get(10, TimeUnit.SECONDS) + logInfo(s"\tSent $m to partition ${metadata.partition}, offset ${metadata.offset}") + (m, metadata) + } + } finally { + if (producer != null) { + producer.close() + producer = null + } + } + offsets + } + + def getLatestOffsets(topics: Set[String]): Map[TopicPartition, Long] = { + val kc = new KafkaConsumer[String, String](consumerConfiguration) + logInfo("Created consumer to get latest offsets") + kc.subscribe(topics.asJavaCollection) + kc.poll(0) + val partitions = kc.assignment() + kc.pause(partitions) + kc.seekToEnd(partitions) + val offsets = partitions.asScala.map(p => p -> kc.position(p)).toMap + kc.close() + logInfo("Closed consumer to get latest offsets") + offsets + } + + private def brokerConfiguration: Properties = { + val props = new Properties() + props.put("broker.id", "0") + props.put("host.name", "localhost") + props.put("advertised.host.name", "localhost") + props.put("port", brokerPort.toString) + props.put("log.dir", Utils.createTempDir().getAbsolutePath) + props.put("zookeeper.connect", zkAddress) + props.put("log.flush.interval.messages", "1") + props.put("replica.socket.timeout.ms", "1500") + props + } + + private def producerConfiguration: Properties = { + val props = new Properties() + props.put("bootstrap.servers", brokerAddress) + props.put("value.serializer", classOf[StringSerializer].getName) + props.put("key.serializer", classOf[StringSerializer].getName) + // wait for all in-sync replicas to ack sends + props.put("acks", "all") + props + } + + private def consumerConfiguration: Properties = { + val props = new Properties() + props.put("bootstrap.servers", brokerAddress) + props.put("group.id", "group-KafkaTestUtils-" + Random.nextInt) + props.put("value.deserializer", classOf[StringDeserializer].getName) + props.put("key.deserializer", classOf[StringDeserializer].getName) + props.put("enable.auto.commit", "false") + props + } + + private def waitUntilMetadataIsPropagated(topic: String, partition: Int): Unit = { + def isPropagated = server.apis.metadataCache.getPartitionInfo(topic, partition) match { + case Some(partitionState) => + val leaderAndInSyncReplicas = partitionState.leaderIsrAndControllerEpoch.leaderAndIsr + + zkUtils.getLeaderForPartition(topic, partition).isDefined && + Request.isValidBrokerId(leaderAndInSyncReplicas.leader) && + leaderAndInSyncReplicas.isr.size >= 1 + + case _ => + false + } + eventually(timeout(10.seconds)) { + assert(isPropagated, s"Partition [$topic, $partition] metadata not propagated after timeout") + } + } + + private class EmbeddedZookeeper(val zkConnect: String) { + val snapshotDir = Utils.createTempDir() + val logDir = Utils.createTempDir() + + val zookeeper = new ZooKeeperServer(snapshotDir, logDir, 500) + val (ip, port) = { + val splits = zkConnect.split(":") + (splits(0), splits(1).toInt) + } + val factory = new NIOServerCnxnFactory() + factory.configure(new InetSocketAddress(ip, port), 16) + factory.startup(zookeeper) + + val actualPort = factory.getLocalPort + + def shutdown() { + factory.shutdown() + Utils.deleteRecursively(snapshotDir) + Utils.deleteRecursively(logDir) + } + } +} + diff --git a/pom.xml b/pom.xml index 3b3ad39b4757..ec22580f1e9e 100644 --- a/pom.xml +++ b/pom.xml @@ -111,6 +111,7 @@ external/kafka-0-8-assembly external/kafka-0-10 external/kafka-0-10-assembly + external/kafka-0-10-sql diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index a39c93e9574f..03778fe676c7 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -39,8 +39,8 @@ object BuildCommons { private val buildLocation = file(".").getAbsoluteFile.getParentFile - val sqlProjects@Seq(catalyst, sql, hive, hiveThriftServer) = Seq( - "catalyst", "sql", "hive", "hive-thriftserver" + val sqlProjects@Seq(catalyst, sql, hive, hiveThriftServer, sqlKafka010) = Seq( + "catalyst", "sql", "hive", "hive-thriftserver", "sql-kafka-0-10" ).map(ProjectRef(buildLocation, _)) val streamingProjects@Seq( From c4db857d745e8814892c9ef8076d8752f6cb4e19 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Wed, 14 Sep 2016 14:21:02 -0700 Subject: [PATCH 02/28] Add more columns inspired by Cody's work --- .../spark/sql/kafka010/KafkaSource.scala | 32 ++++++++++++------- .../spark/sql/kafka010/KafkaSourceRDD.scala | 8 ++--- .../spark/sql/kafka010/KafkaSourceSuite.scala | 5 +-- 3 files changed, 28 insertions(+), 17 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index 81483d86526c..3693896e0d12 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -29,11 +29,10 @@ 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.catalyst.encoders.ExpressionEncoder 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.StructType +import org.apache.spark.sql.types._ import org.apache.spark.SparkContext /** @@ -78,15 +77,12 @@ private[kafka010] case class KafkaSource( sourceOptions: Map[String, String]) extends Source with Logging { - implicit private val encoder = ExpressionEncoder.tuple( - ExpressionEncoder[Array[Byte]](), ExpressionEncoder[Array[Byte]]()) - @transient private val consumer = consumerStrategy.createConsumer() @transient private val sc = sqlContext.sparkContext @transient private val initialPartitionOffsets = fetchPartitionOffsets(seekToLatest = false) logInfo(s"Initial offsets: " + initialPartitionOffsets) - override def schema: StructType = encoder.schema + override def schema: StructType = KafkaSource.kafkaSchema /** Returns the maximum available offset for this source. */ override def getOffset: Option[Offset] = { @@ -134,11 +130,13 @@ private[kafka010] case class KafkaSource( // Create a RDD that reads from Kafka and get the (key, value) pair as byte arrays. val rdd = new KafkaSourceRDD[Array[Byte], Array[Byte]]( - sc, executorKafkaParams, offsetRanges, sourceOptions) - .map { r => (r.key, r.value) } + sc, executorKafkaParams, offsetRanges, sourceOptions).map { cr => + Row(cr.checksum, cr.key, cr.offset, cr.partition, cr.serializedKeySize, + cr.serializedValueSize, cr.timestamp, cr.timestampType.id, cr.topic, cr.value) + } logInfo("GetBatch: " + offsetRanges.sortBy(_.topicPartition.toString).mkString(", ")) - sqlContext.createDataset(rdd).toDF("key", "value") + sqlContext.createDataFrame(rdd, schema) } /** Stop this source and free any resources it has allocated. */ @@ -172,6 +170,19 @@ private[kafka010] case class KafkaSource( /** Companion object for the [[KafkaSource]]. */ private[kafka010] object KafkaSource { + def kafkaSchema: StructType = StructType(Seq( + StructField("checksum", LongType), + StructField("key", BinaryType), + StructField("offset", LongType), + StructField("partition", IntegerType), + StructField("serializedKeySize", IntegerType), + StructField("serializedValueSize", IntegerType), + StructField("timestamp", LongType), + StructField("timestampType", IntegerType), + StructField("topic", StringType), + StructField("value", BinaryType) + )) + sealed trait ConsumerStrategy[K, V] { def createConsumer(): Consumer[K, V] } @@ -289,7 +300,6 @@ private[kafka010] object KafkaSourceOffset { */ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider with DataSourceRegister with Logging { - private val structType = new StructType().add("key", "binary").add("value", "binary") private val strategyOptionNames = Set("subscribe", "subscribepattern") /** Class to conveniently update Kafka config params, while logging the changes */ @@ -322,7 +332,7 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider providerName: String, parameters: Map[String, String]): (String, StructType) = { validateOptions(parameters) - ("kafka", structType) + ("kafka", KafkaSource.kafkaSchema) } override def createSource( diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala index 249d124aff4f..0cd8534b05e9 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala @@ -25,7 +25,7 @@ import scala.util.Try import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord} import org.apache.kafka.common.TopicPartition -import org.apache.spark.{Partition, SparkContext, TaskContext} +import org.apache.spark.{Partition, SparkContext, SparkEnv, TaskContext} import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.kafka010.KafkaSourceRDD._ @@ -151,11 +151,11 @@ private[kafka010] object KafkaSourceRDD { // Configurations for initializing the cache of KafkaConsumers. private val cacheInitialCapacity = - getInt(options, "consumer.cache.initialCapacity", 16) + SparkEnv.get.conf.getInt("kafka.consumer.cache.initialCapacity", 16) private val cacheMaxCapacity = - getInt(options, "consumer.cache.maxCapacity", 64) + SparkEnv.get.conf.getInt("kafka.consumer.cache.maxCapacity", 64) private val cacheLoadFactor = - getDouble(options, "consumer.cache.loadFactor", 0.75).toFloat + SparkEnv.get.conf.getDouble("kafka.consumer.cache.loadFactor", 0.75).toFloat // Initialize the cache if not already done, and get a cached KafkaConsumer CachedKafkaConsumer.init(cacheInitialCapacity, cacheMaxCapacity, cacheLoadFactor) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala index 3cb4c48f4dbd..61b07a0b0c63 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala @@ -89,6 +89,7 @@ class KafkaSourceSuite extends StreamTest with SharedSQLContext { .option("kafka.group.id", s"group-stress-test") .option("subscribe", topics.mkString(",")) .load() + .select("key", "value") .as[(Array[Byte], Array[Byte])] val mapped = kafka.map(kv => new String(kv._2).toInt + 1) @@ -139,7 +140,7 @@ class KafkaSourceSuite extends StreamTest with SharedSQLContext { .option("kafka.group.id", s"group-$topic") .option("kafka.auto.offset.reset", s"latest") options.foreach { case (k, v) => reader.option(k, v) } - val kafka = reader.load().as[(Array[Byte], Array[Byte])] + val kafka = reader.load().select("key", "value").as[(Array[Byte], Array[Byte])] val mapped = kafka.map(kv => new String(kv._2).toInt + 1) testStream(mapped)( @@ -169,7 +170,7 @@ class KafkaSourceSuite extends StreamTest with SharedSQLContext { .option("kafka.group.id", s"group-$topic") .option("kafka.auto.offset.reset", s"earliest") options.foreach { case (k, v) => reader.option(k, v) } - val kafka = reader.load().as[(Array[Byte], Array[Byte])] + val kafka = reader.load().select("key", "value").as[(Array[Byte], Array[Byte])] val mapped = kafka.map(kv => new String(kv._2).toInt + 1) testStream(mapped)( From d3c0754c9b5c9aa2c29b0d3ca0b28ddae041b054 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Wed, 14 Sep 2016 14:52:21 -0700 Subject: [PATCH 03/28] Fix the build --- dev/run-tests.py | 2 +- dev/sparktestsupport/modules.py | 12 ++++++++++++ project/SparkBuild.scala | 2 +- 3 files changed, 14 insertions(+), 2 deletions(-) diff --git a/dev/run-tests.py b/dev/run-tests.py index ae4b5306fc5c..5d661f5f1a1c 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -110,7 +110,7 @@ def determine_modules_to_test(changed_modules): ['graphx', 'examples'] >>> x = [x.name for x in determine_modules_to_test([modules.sql])] >>> x # doctest: +NORMALIZE_WHITESPACE - ['sql', 'hive', 'mllib', 'examples', 'hive-thriftserver', + ['sql', 'hive', 'mllib', 'sql-kafka-0-10', 'examples', 'hive-thriftserver', 'pyspark-sql', 'sparkr', 'pyspark-mllib', 'pyspark-ml'] """ modules_to_test = set() diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index 050cdf043757..5f14683d9a52 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -158,6 +158,18 @@ def __hash__(self): ) +sql_kafka = Module( + name="sql-kafka-0-10", + dependencies=[sql], + source_file_regexes=[ + "external/kafka-0-10-sql", + ], + sbt_test_goals=[ + "sql-kafka-0-10/test", + ] +) + + sketch = Module( name="sketch", dependencies=[tags], diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 03778fe676c7..c687d2fb8dbf 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -353,7 +353,7 @@ object SparkBuild extends PomBuild { val mimaProjects = allProjects.filterNot { x => Seq( spark, hive, hiveThriftServer, catalyst, repl, networkCommon, networkShuffle, networkYarn, - unsafe, tags + unsafe, tags, sqlKafka010 ).contains(x) } From f5c57f51f675002298c833edb486451642735221 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Mon, 19 Sep 2016 22:49:09 -0700 Subject: [PATCH 04/28] Support add partitions --- .../spark/sql/kafka010/KafkaSource.scala | 19 ++++++++++--------- .../spark/sql/kafka010/KafkaSourceSuite.scala | 16 ++++++++++++++-- .../spark/sql/kafka010/KafkaTestUtils.scala | 9 +++++++++ 3 files changed, 33 insertions(+), 11 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index 3693896e0d12..602d5c000261 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -87,7 +87,7 @@ private[kafka010] case class KafkaSource( /** Returns the maximum available offset for this source. */ override def getOffset: Option[Offset] = { val offset = KafkaSourceOffset(fetchPartitionOffsets(seekToLatest = true)) - logInfo(s"GetOffset: $offset") + logDebug(s"GetOffset: $offset") Some(offset) } @@ -118,14 +118,15 @@ private[kafka010] case class KafkaSource( val sortedExecutors = getSortedExecutorList(sc) val numExecutors = sortedExecutors.size logDebug("Sorted executors: " + sortedExecutors.mkString(", ")) - val offsetRanges = sortedTopicPartitions.flatMap { tp => - fromPartitionOffsets.get(tp).map { fromOffset => - val untilOffset = untilPartitionOffsets(tp) - val preferredLoc = if (numExecutors > 0) { - Some(sortedExecutors(positiveMod(tp.hashCode, numExecutors))) - } else None - KafkaSourceRDD.OffsetRange(tp, fromOffset, untilOffset, preferredLoc) - } + val offsetRanges = sortedTopicPartitions.map { tp => + // If fromPartitionOffsets doesn't contain tp, then it's a new partition. + // So use 0 as the start offset. + val fromOffset = fromPartitionOffsets.get(tp).getOrElse(0L) + val untilOffset = untilPartitionOffsets(tp) + val preferredLoc = if (numExecutors > 0) { + Some(sortedExecutors(positiveMod(tp.hashCode, numExecutors))) + } else None + KafkaSourceRDD.OffsetRange(tp, fromOffset, untilOffset, preferredLoc) }.toArray // Create a RDD that reads from Kafka and get the (key, value) pair as byte arrays. diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala index 61b07a0b0c63..36475e7cbbcb 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala @@ -139,6 +139,7 @@ class KafkaSourceSuite extends StreamTest with SharedSQLContext { .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.group.id", s"group-$topic") .option("kafka.auto.offset.reset", s"latest") + .option("kafka.metadata.max.age.ms", "1") options.foreach { case (k, v) => reader.option(k, v) } val kafka = reader.load().select("key", "value").as[(Array[Byte], Array[Byte])] val mapped = kafka.map(kv => new String(kv._2).toInt + 1) @@ -154,7 +155,12 @@ class KafkaSourceSuite extends StreamTest with SharedSQLContext { StartStream(), CheckAnswer(2, 3, 4, 5, 6, 7), // Should get the added data AddKafkaData(Set(topic), 7, 8), - CheckAnswer(2, 3, 4, 5, 6, 7, 8, 9) + CheckAnswer(2, 3, 4, 5, 6, 7, 8, 9), + AssertOnQuery("Add partitions") { query: StreamExecution => + testUtils.addPartitions(topic, 10) + }, + AddKafkaData(Set(topic), 9, 10, 11, 12, 13, 14, 15, 16), + CheckAnswer(2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17) ) } @@ -169,6 +175,7 @@ class KafkaSourceSuite extends StreamTest with SharedSQLContext { .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.group.id", s"group-$topic") .option("kafka.auto.offset.reset", s"earliest") + .option("kafka.metadata.max.age.ms", "1") options.foreach { case (k, v) => reader.option(k, v) } val kafka = reader.load().select("key", "value").as[(Array[Byte], Array[Byte])] val mapped = kafka.map(kv => new String(kv._2).toInt + 1) @@ -182,7 +189,12 @@ class KafkaSourceSuite extends StreamTest with SharedSQLContext { StopStream, AddKafkaData(Set(topic), 7, 8), StartStream(), - CheckAnswer(2, 3, 4, 5, 6, 7, 8, 9) + CheckAnswer(2, 3, 4, 5, 6, 7, 8, 9), + AssertOnQuery("Add partitions") { query: StreamExecution => + testUtils.addPartitions(topic, 10) + }, + AddKafkaData(Set(topic), 9, 10, 11, 12, 13, 14, 15, 16), + CheckAnswer(2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17) ) } diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala index 5d1f9fec6091..c0f154e43535 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala @@ -167,6 +167,15 @@ class KafkaTestUtils extends Logging { createTopic(topic, 1) } + /** Add new paritions to a Kafka topic */ + def addPartitions(topic: String, partitions: Int): Unit = { + AdminUtils.addPartitions(zkUtils, topic, partitions) + // wait until metadata is propagated + (0 until partitions).foreach { p => + waitUntilMetadataIsPropagated(topic, p) + } + } + /** Java-friendly function for sending messages to the Kafka broker */ def sendMessages(topic: String, messageToFreq: JMap[String, JInt]): Unit = { sendMessages(topic, Map(messageToFreq.asScala.mapValues(_.intValue()).toSeq: _*)) From b64d1046a13ce3b03b2598f13dd2d1aeb1219048 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Mon, 19 Sep 2016 23:21:26 -0700 Subject: [PATCH 05/28] Fix compile error --- .../scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala index 36475e7cbbcb..bacd9b0409a9 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala @@ -158,6 +158,7 @@ class KafkaSourceSuite extends StreamTest with SharedSQLContext { CheckAnswer(2, 3, 4, 5, 6, 7, 8, 9), AssertOnQuery("Add partitions") { query: StreamExecution => testUtils.addPartitions(topic, 10) + true }, AddKafkaData(Set(topic), 9, 10, 11, 12, 13, 14, 15, 16), CheckAnswer(2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17) @@ -192,6 +193,7 @@ class KafkaSourceSuite extends StreamTest with SharedSQLContext { CheckAnswer(2, 3, 4, 5, 6, 7, 8, 9), AssertOnQuery("Add partitions") { query: StreamExecution => testUtils.addPartitions(topic, 10) + true }, AddKafkaData(Set(topic), 9, 10, 11, 12, 13, 14, 15, 16), CheckAnswer(2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17) From a3d0a2b436b14e134b971e7d23815adf78d4a329 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Tue, 20 Sep 2016 10:23:37 -0700 Subject: [PATCH 06/28] Get rid of type parameters --- .../sql/kafka010/CachedKafkaConsumer.scala | 29 ++++++++++--------- .../spark/sql/kafka010/KafkaSourceRDD.scala | 25 +++++++++------- 2 files changed, 29 insertions(+), 25 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala index df188dfec0ee..93fa105dc576 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala @@ -30,7 +30,7 @@ import org.apache.spark.internal.Logging * Underlying consumer is not threadsafe, so neither is this, * but processing the same topicpartition and group id in multiple threads is usually bad anyway. */ -class CachedKafkaConsumer[K, V] private( +class CachedKafkaConsumer private( val groupId: String, val topic: String, val partition: Int, @@ -42,7 +42,7 @@ class CachedKafkaConsumer[K, V] private( val topicPartition = new TopicPartition(topic, partition) protected val consumer = { - val c = new KafkaConsumer[K, V](kafkaParams) + val c = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams) val tps = new ju.ArrayList[TopicPartition]() tps.add(topicPartition) c.assign(tps) @@ -51,7 +51,8 @@ class CachedKafkaConsumer[K, V] private( // TODO if the buffer was kept around as a random-access structure, // could possibly optimize re-calculating of an RDD in the same batch - protected var buffer = ju.Collections.emptyList[ConsumerRecord[K, V]]().iterator + protected var buffer = + ju.Collections.emptyList[ConsumerRecord[Array[Byte], Array[Byte]]]().iterator protected var nextOffset = -2L def close(): Unit = consumer.close() @@ -60,7 +61,7 @@ class CachedKafkaConsumer[K, V] private( * 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, timeout: Long): ConsumerRecord[K, V] = { + def get(offset: Long, timeout: Long): ConsumerRecord[Array[Byte], Array[Byte]] = { logDebug(s"Get $groupId $topic $partition nextOffset $nextOffset requested $offset") if (offset != nextOffset) { logInfo(s"Initial fetch for $groupId $topic $partition $offset") @@ -107,7 +108,7 @@ object CachedKafkaConsumer extends Logging { private case class CacheKey(groupId: String, topic: String, partition: Int) // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap - private var cache: ju.LinkedHashMap[CacheKey, CachedKafkaConsumer[_, _]] = null + private var cache: ju.LinkedHashMap[CacheKey, CachedKafkaConsumer] = null /** Must be called before get, once per JVM, to configure the cache. Further calls are ignored */ def init( @@ -116,10 +117,10 @@ object CachedKafkaConsumer extends Logging { loadFactor: Float): Unit = CachedKafkaConsumer.synchronized { if (null == cache) { logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor") - cache = new ju.LinkedHashMap[CacheKey, CachedKafkaConsumer[_, _]]( + cache = new ju.LinkedHashMap[CacheKey, CachedKafkaConsumer]( initialCapacity, loadFactor, true) { override def removeEldestEntry( - entry: ju.Map.Entry[CacheKey, CachedKafkaConsumer[_, _]]): Boolean = { + entry: ju.Map.Entry[CacheKey, CachedKafkaConsumer]): Boolean = { if (this.size > maxCapacity) { try { entry.getValue.consumer.close() @@ -140,23 +141,23 @@ object CachedKafkaConsumer extends Logging { * Get a cached consumer for groupId, assigned to topic and partition. * If matching consumer doesn't already exist, will be created using kafkaParams. */ - def get[K, V]( + def get( groupId: String, topic: String, partition: Int, - kafkaParams: ju.Map[String, Object]): CachedKafkaConsumer[K, V] = + kafkaParams: ju.Map[String, Object]): CachedKafkaConsumer = CachedKafkaConsumer.synchronized { val k = CacheKey(groupId, topic, partition) val v = cache.get(k) if (null == v) { logInfo(s"Cache miss for $k") logDebug(cache.keySet.toString) - val c = new CachedKafkaConsumer[K, V](groupId, topic, partition, kafkaParams) + val c = new CachedKafkaConsumer(groupId, topic, partition, kafkaParams) cache.put(k, c) c } else { // any given topicpartition should have a consistent key and value type - v.asInstanceOf[CachedKafkaConsumer[K, V]] + v.asInstanceOf[CachedKafkaConsumer] } } @@ -164,12 +165,12 @@ object CachedKafkaConsumer extends Logging { * Get a fresh new instance, unassociated with the global cache. * Caller is responsible for closing */ - def getUncached[K, V]( + def getUncached( groupId: String, topic: String, partition: Int, - kafkaParams: ju.Map[String, Object]): CachedKafkaConsumer[K, V] = - new CachedKafkaConsumer[K, V](groupId, topic, partition, kafkaParams) + kafkaParams: ju.Map[String, Object]): CachedKafkaConsumer = + new CachedKafkaConsumer(groupId, topic, partition, kafkaParams) /** remove consumer for given groupId, topic, and partition, if it exists */ def remove(groupId: String, topic: String, partition: Int): Unit = { diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala index 0cd8534b05e9..218b58aeb6d6 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala @@ -49,11 +49,11 @@ case class KafkaSourceRDDPartition(index: Int, offsetRange: OffsetRange) extends * @tparam K type of Kafka message key * @tparam V type of Kafka message value */ -private[kafka010] class KafkaSourceRDD[K, V]( +private[kafka010] class KafkaSourceRDD( sc: SparkContext, executorKafkaParams: ju.Map[String, Object], offsetRanges: Seq[OffsetRange], - sourceOptions: Map[String, String]) extends RDD[ConsumerRecord[K, V]](sc, Nil) { + sourceOptions: Map[String, String]) extends RDD[ConsumerRecord[Array[Byte], Array[Byte]]](sc, Nil) { override def persist(newLevel: StorageLevel): this.type = { logError("Kafka ConsumerRecord is not serializable. " + @@ -69,12 +69,12 @@ private[kafka010] class KafkaSourceRDD[K, V]( override def isEmpty(): Boolean = count == 0L - override def take(num: Int): Array[ConsumerRecord[K, V]] = { + 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[K, V]](0) + return new Array[ConsumerRecord[Array[Byte], Array[Byte]]](0) } // Determine in advance how many messages need to be taken from each partition @@ -88,17 +88,19 @@ private[kafka010] class KafkaSourceRDD[K, V]( } } - val buf = new ArrayBuffer[ConsumerRecord[K, V]] + val buf = new ArrayBuffer[ConsumerRecord[Array[Byte], Array[Byte]]] val res = context.runJob( this, - (tc: TaskContext, it: Iterator[ConsumerRecord[K, V]]) => + (tc: TaskContext, it: Iterator[ConsumerRecord[Array[Byte], Array[Byte]]]) => it.take(parts(tc.partitionId)).toArray, parts.keys.toArray ) res.foreach(buf ++= _) buf.toArray } - override def compute(thePart: Partition, context: TaskContext): Iterator[ConsumerRecord[K, V]] = { + override def compute( + thePart: Partition, + context: TaskContext): Iterator[ConsumerRecord[Array[Byte], Array[Byte]]] = { val range = thePart.asInstanceOf[KafkaSourceRDDPartition].offsetRange assert( range.fromOffset <= range.untilOffset, @@ -132,11 +134,12 @@ private[kafka010] object KafkaSourceRDD { * An iterator that fetches messages directly from Kafka for the offsets in partition. * Uses a cached consumer where possible to take advantage of prefetching */ - private class KafkaRDDIterator[K, V]( + private class KafkaRDDIterator( part: OffsetRange, executorKafkaParams: ju.Map[String, Object], options: Map[String, String], - context: TaskContext) extends Iterator[ConsumerRecord[K, V]] with Logging { + context: TaskContext) + extends Iterator[ConsumerRecord[Array[Byte], Array[Byte]]] with Logging { logInfo(s"Computing topic ${part.topic}, partition ${part.partition} " + s"offsets ${part.fromOffset} -> ${part.untilOffset}") @@ -164,13 +167,13 @@ private[kafka010] object KafkaSourceRDD { CachedKafkaConsumer.remove(groupId, part.topic, part.partition) } val consumer = - CachedKafkaConsumer.get[K, V](groupId, part.topic, part.partition, executorKafkaParams) + CachedKafkaConsumer.get(groupId, part.topic, part.partition, executorKafkaParams) var requestOffset = part.fromOffset override def hasNext(): Boolean = requestOffset < part.untilOffset - override def next(): ConsumerRecord[K, V] = { + override def next(): ConsumerRecord[Array[Byte], Array[Byte]] = { assert(hasNext(), "Can't call next() once untilOffset has been reached") val r = consumer.get(requestOffset, pollTimeout) requestOffset += 1 From 6bc2994f98567127137aee54a5141d98337986c5 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Tue, 20 Sep 2016 15:33:09 -0700 Subject: [PATCH 07/28] Use Kafka API to fetch the earlies offsets of new partitions; Get rid of type parameters; Update stress tests to test adding partitions --- .../spark/sql/kafka010/KafkaSource.scala | 61 +++++++++++++------ .../spark/sql/kafka010/KafkaSourceRDD.scala | 24 +------- .../spark/sql/kafka010/KafkaSourceSuite.scala | 28 ++++++++- 3 files changed, 72 insertions(+), 41 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index 602d5c000261..eefcfaa53799 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -72,7 +72,7 @@ import org.apache.spark.SparkContext */ private[kafka010] case class KafkaSource( sqlContext: SQLContext, - consumerStrategy: ConsumerStrategy[Array[Byte], Array[Byte]], + consumerStrategy: ConsumerStrategy, executorKafkaParams: ju.Map[String, Object], sourceOptions: Map[String, String]) extends Source with Logging { @@ -104,6 +104,13 @@ private[kafka010] case class KafkaSource( initialPartitionOffsets } + val newPartitions = untilPartitionOffsets.keySet.diff(fromPartitionOffsets.keySet) + val newPartitionOffsets = if (newPartitions.nonEmpty) { + fetchNewPartitionEarliestOffsets(newPartitions.toSeq) + } else { + Map.empty[TopicPartition, Long] + } + // Sort the partitions and current list of executors to consistently assign each partition // to the executor. This allows cached KafkaConsumers in the executors to be re-used to // read the same partition in every batch. @@ -119,9 +126,13 @@ private[kafka010] case class KafkaSource( val numExecutors = sortedExecutors.size logDebug("Sorted executors: " + sortedExecutors.mkString(", ")) val offsetRanges = sortedTopicPartitions.map { tp => - // If fromPartitionOffsets doesn't contain tp, then it's a new partition. - // So use 0 as the start offset. - val fromOffset = fromPartitionOffsets.get(tp).getOrElse(0L) + val fromOffset = fromPartitionOffsets.get(tp).getOrElse { + newPartitionOffsets.getOrElse(tp, { + // This should not happen since newPartitionOffsets contains all paritions not in + // fromPartitionOffsets + throw new IllegalStateException(s"$tp doesn't have a offset") + }) + } val untilOffset = untilPartitionOffsets(tp) val preferredLoc = if (numExecutors > 0) { Some(sortedExecutors(positiveMod(tp.hashCode, numExecutors))) @@ -130,7 +141,7 @@ private[kafka010] case class KafkaSource( }.toArray // Create a RDD that reads from Kafka and get the (key, value) pair as byte arrays. - val rdd = new KafkaSourceRDD[Array[Byte], Array[Byte]]( + val rdd = new KafkaSourceRDD( sc, executorKafkaParams, offsetRanges, sourceOptions).map { cr => Row(cr.checksum, cr.key, cr.offset, cr.partition, cr.serializedKeySize, cr.serializedValueSize, cr.timestamp, cr.timestampType.id, cr.topic, cr.value) @@ -165,6 +176,21 @@ private[kafka010] case class KafkaSource( } } + private def fetchNewPartitionEarliestOffsets( + newPartitions: Seq[TopicPartition]): Map[TopicPartition, Long] = { + synchronized { + consumer.poll(0) + val partitions = consumer.assignment() + logDebug(s"\tPartitioned assigned to consumer: $partitions") + require(newPartitions.forall(tp => partitions.contains(tp)), + s"$partitions doesn't contain all new paritions: $newPartitions") + consumer.seekToBeginning(newPartitions.asJava) + val partitionToOffsets = newPartitions.map(p => p -> consumer.position(p)) + logDebug(s"Got earliest positions $partitionToOffsets") + partitionToOffsets.toMap + } + } + private def positiveMod(a: Long, b: Int): Int = ((a % b).toInt + b) % b } @@ -184,14 +210,14 @@ private[kafka010] object KafkaSource { StructField("value", BinaryType) )) - sealed trait ConsumerStrategy[K, V] { - def createConsumer(): Consumer[K, V] + sealed trait ConsumerStrategy { + def createConsumer(): Consumer[Array[Byte], Array[Byte]] } - case class SubscribeStrategy[K, V](topics: Seq[String], kafkaParams: ju.Map[String, Object]) - extends ConsumerStrategy[K, V] { - override def createConsumer(): Consumer[K, V] = { - val consumer = new KafkaConsumer[K, V](kafkaParams) + case class SubscribeStrategy(topics: Seq[String], kafkaParams: ju.Map[String, Object]) + extends ConsumerStrategy { + override def createConsumer(): Consumer[Array[Byte], Array[Byte]] = { + val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams) consumer.subscribe(topics.asJava) consumer.poll(0) consumer @@ -200,11 +226,11 @@ private[kafka010] object KafkaSource { override def toString: String = s"Subscribe[${topics.mkString(", ")}]" } - case class SubscribePatternStrategy[K, V]( + case class SubscribePatternStrategy( topicPattern: String, kafkaParams: ju.Map[String, Object]) - extends ConsumerStrategy[K, V] { - override def createConsumer(): Consumer[K, V] = { - val consumer = new KafkaConsumer[K, V](kafkaParams) + extends ConsumerStrategy { + override def createConsumer(): Consumer[Array[Byte], Array[Byte]] = { + val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams) consumer.subscribe( ju.regex.Pattern.compile(topicPattern), new NoOpConsumerRebalanceListener()) @@ -357,6 +383,7 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider ConfigUpdater("source", specifiedKafkaParams) .set(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, deserClassName) .set(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, deserClassName) + .set(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") .build() val kafkaParamsForExecutors = @@ -381,11 +408,11 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider val strategy = caseInsensitiveParams.find(x => strategyOptionNames.contains(x._1)).get match { case ("subscribe", value) => - SubscribeStrategy[Array[Byte], Array[Byte]]( + SubscribeStrategy( value.split(",").map(_.trim()).filter(_.nonEmpty), kafkaParamsForStrategy) case ("subscribepattern", value) => - SubscribePatternStrategy[Array[Byte], Array[Byte]]( + SubscribePatternStrategy( value.trim(), kafkaParamsForStrategy) case _ => diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala index 218b58aeb6d6..5b553260804f 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala @@ -46,14 +46,13 @@ case class KafkaSourceRDDPartition(index: Int, offsetRange: OffsetRange) extends * @param executorKafkaParams Kafka configuration for creating KafkaConsumer on the executors * @param offsetRanges Offset ranges that define the Kafka data belonging to this RDD * @param sourceOptions Options provided through the source - * @tparam K type of Kafka message key - * @tparam V type of Kafka message value */ private[kafka010] class KafkaSourceRDD( sc: SparkContext, executorKafkaParams: ju.Map[String, Object], offsetRanges: Seq[OffsetRange], - sourceOptions: Map[String, String]) extends RDD[ConsumerRecord[Array[Byte], Array[Byte]]](sc, Nil) { + sourceOptions: Map[String, String]) + extends RDD[ConsumerRecord[Array[Byte], Array[Byte]]](sc, Nil) { override def persist(newLevel: StorageLevel): this.type = { logError("Kafka ConsumerRecord is not serializable. " + @@ -149,8 +148,7 @@ private[kafka010] object KafkaSourceRDD { // Time between polling for more data by the KafkaConsumer in the executor. This should not // require much configuration as data should already be available in Kafka when the executors // are polling. - private val pollTimeout = - getLong(options, "consumer.pollMs", 512) + private val pollTimeout = getLong(options, "consumer.pollMs", 512) // Configurations for initializing the cache of KafkaConsumers. private val cacheInitialCapacity = @@ -181,22 +179,6 @@ private[kafka010] object KafkaSourceRDD { } } - def getInt(options: Map[String, String], name: String, defaultValue: Int): Int = { - options.get(name).map { str => - Try(str.toInt).getOrElse { - throw new IllegalArgumentException("Option '$name' must be a integer") - } - }.getOrElse(defaultValue) - } - - def getDouble(options: Map[String, String], name: String, defaultValue: Double): Double = { - options.get(name).map { str => - Try(str.toDouble).getOrElse { - throw new IllegalArgumentException("Option '$name' must be a double") - } - }.getOrElse(defaultValue) - } - def getLong(options: Map[String, String], name: String, defaultValue: Long): Long = { options.get(name).map { str => Try(str.toLong).getOrElse { diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala index bacd9b0409a9..dfa119f6968f 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala @@ -76,8 +76,12 @@ class KafkaSourceSuite extends StreamTest with SharedSQLContext { test("stress test with multiple topics and partitions") { val topics = (1 to 5).map(i => s"stress$i").toSet + var partitionRange = (1, 5) + def randomPartitions: Int = { + Random.nextInt(partitionRange._2 + 1 - partitionRange._1) + partitionRange._1 + } topics.foreach { topic => - testUtils.createTopic(topic, partitions = Random.nextInt(5) + 1) + testUtils.createTopic(topic, partitions = randomPartitions) testUtils.sendMessages(topic, (101 to 105).map { _.toString }.toArray) } @@ -88,6 +92,7 @@ class KafkaSourceSuite extends StreamTest with SharedSQLContext { .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.group.id", s"group-stress-test") .option("subscribe", topics.mkString(",")) + .option("kafka.metadata.max.age.ms", "1") .load() .select("key", "value") .as[(Array[Byte], Array[Byte])] @@ -96,7 +101,16 @@ class KafkaSourceSuite extends StreamTest with SharedSQLContext { runStressTest( mapped, - d => AddKafkaData(topics, d: _*)(ensureDataInMultiplePartition = false), + d => { + if (Random.nextInt(5) == 0) { + partitionRange = (partitionRange._1 + 5, partitionRange._2 + 5) + val addPartitions = topics.toSeq.map(_ => randomPartitions) + AddKafkaData(topics, d: _*)( + ensureDataInMultiplePartition = false, addPartitions = Some(addPartitions)) + } else { + AddKafkaData(topics, d: _*)(ensureDataInMultiplePartition = false) + } + }, iterations = 50) } @@ -201,9 +215,17 @@ class KafkaSourceSuite extends StreamTest with SharedSQLContext { } case class AddKafkaData(topics: Set[String], data: Int*) - (implicit ensureDataInMultiplePartition: Boolean = false) extends AddData { + (implicit ensureDataInMultiplePartition: Boolean = false, + addPartitions: Option[Seq[Int]] = None) extends AddData { override def addData(query: Option[StreamExecution]): (Source, Offset) = { + if (addPartitions.nonEmpty) { + require(topics.size == addPartitions.get.size, + s"$addPartitions should have the same size of $topics") + topics.zip(addPartitions.get).foreach { case (topic, partitions) => + testUtils.addPartitions(topic, partitions) + } + } require( query.nonEmpty, "Cannot add data when there is no query for finding the active kafka source") From 881b20678f84b321962085a1c9dea0a7387e21c6 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Tue, 20 Sep 2016 17:41:45 -0700 Subject: [PATCH 08/28] Stress test for adding new topics --- .../spark/sql/kafka010/KafkaSourceSuite.scala | 51 ++++++++++++++----- .../spark/sql/kafka010/KafkaTestUtils.scala | 4 ++ 2 files changed, 43 insertions(+), 12 deletions(-) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala index dfa119f6968f..bb210eb8a1b4 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala @@ -36,7 +36,7 @@ class KafkaSourceSuite extends StreamTest with SharedSQLContext { private val topicId = new AtomicInteger(0) private var testUtils: KafkaTestUtils = _ - override val streamingTimeout = 10.seconds + override val streamingTimeout = 30.seconds override def beforeAll(): Unit = { super.beforeAll() @@ -75,11 +75,20 @@ class KafkaSourceSuite extends StreamTest with SharedSQLContext { } test("stress test with multiple topics and partitions") { - val topics = (1 to 5).map(i => s"stress$i").toSet - var partitionRange = (1, 5) + val topicId = new AtomicInteger(1) + + def newStressTopic: String = s"stress${topicId.getAndIncrement()}" + + @volatile var topics = (1 to 5).map(_ => newStressTopic).toSet + + @volatile var partitionRange = (1, 5) + + def newPartitionRange: (Int, Int) = (partitionRange._1 + 5, partitionRange._2 + 5) + def randomPartitions: Int = { Random.nextInt(partitionRange._2 + 1 - partitionRange._1) + partitionRange._1 } + topics.foreach { topic => testUtils.createTopic(topic, partitions = randomPartitions) testUtils.sendMessages(topic, (101 to 105).map { _.toString }.toArray) @@ -91,8 +100,8 @@ class KafkaSourceSuite extends StreamTest with SharedSQLContext { .format(classOf[KafkaSourceProvider].getCanonicalName.stripSuffix("$")) .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.group.id", s"group-stress-test") - .option("subscribe", topics.mkString(",")) .option("kafka.metadata.max.age.ms", "1") + .option("subscribePattern", "stress.*") .load() .select("key", "value") .as[(Array[Byte], Array[Byte])] @@ -102,13 +111,20 @@ class KafkaSourceSuite extends StreamTest with SharedSQLContext { runStressTest( mapped, d => { - if (Random.nextInt(5) == 0) { - partitionRange = (partitionRange._1 + 5, partitionRange._2 + 5) - val addPartitions = topics.toSeq.map(_ => randomPartitions) - AddKafkaData(topics, d: _*)( - ensureDataInMultiplePartition = false, addPartitions = Some(addPartitions)) - } else { - AddKafkaData(topics, d: _*)(ensureDataInMultiplePartition = false) + Random.nextInt(5) match { + case 0 => + partitionRange = newPartitionRange + val addPartitions = topics.toSeq.map(_ => randomPartitions) + AddKafkaData(topics, d: _*)( + ensureDataInMultiplePartition = false, addPartitions = Some(addPartitions)) + case 1 => + topics = topics + newStressTopic + partitionRange = newPartitionRange + val addPartitions = topics.toSeq.map(_ => randomPartitions) + AddKafkaData(topics, d: _*)( + ensureDataInMultiplePartition = false, addPartitions = Some(addPartitions)) + case _ => + AddKafkaData(topics, d: _*)(ensureDataInMultiplePartition = false) } }, iterations = 50) @@ -214,16 +230,27 @@ class KafkaSourceSuite extends StreamTest with SharedSQLContext { ) } + /** + * Add data to Kafka. If any topic in `topics` does not exist, it will be created automatically. + * + * `addPartitions` is the new partition numbers of the topics. The caller should make sure using + * a bigger partition number. Otherwise, it will throw an exception. + */ case class AddKafkaData(topics: Set[String], data: Int*) (implicit ensureDataInMultiplePartition: Boolean = false, addPartitions: Option[Seq[Int]] = None) extends AddData { override def addData(query: Option[StreamExecution]): (Source, Offset) = { + val allTopics = testUtils.getAllTopics().toSet if (addPartitions.nonEmpty) { require(topics.size == addPartitions.get.size, s"$addPartitions should have the same size of $topics") topics.zip(addPartitions.get).foreach { case (topic, partitions) => - testUtils.addPartitions(topic, partitions) + if (allTopics.contains(topic)) { + testUtils.addPartitions(topic, partitions) + } else { + testUtils.createTopic(topic, partitions) + } } } require( diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala index c0f154e43535..c201da96b9a9 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala @@ -162,6 +162,10 @@ class KafkaTestUtils extends Logging { } } + def getAllTopics(): Seq[String] = { + zkUtils.getAllTopics() + } + /** Create a Kafka topic and wait until it is propagated to the whole cluster */ def createTopic(topic: String): Unit = { createTopic(topic, 1) From 8e86f9863dab12dc5043291ca696e7475637cb24 Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Thu, 22 Sep 2016 00:22:51 -0500 Subject: [PATCH 09/28] test case that shows why the current implementation is wrong from an end-user perspective --- .../spark/sql/kafka010/KafkaSourceSuite.scala | 32 +++++++++++++++ .../spark/sql/kafka010/KafkaTestUtils.scala | 41 ++++++++++++++++++- 2 files changed, 72 insertions(+), 1 deletion(-) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala index bb210eb8a1b4..b4d16d6420f6 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala @@ -155,6 +155,38 @@ class KafkaSourceSuite extends StreamTest with SharedSQLContext { testBadOptions("subscribePattern" -> "")("pattern to subscribe is empty") } + test("users will delete topics") { + val topicPrefix = newTopic() + val topic = topicPrefix + "-seems" + val topic2 = topicPrefix + "-bad" + testUtils.createTopic(topic, partitions = 5) + testUtils.sendMessages(topic, Array("-1")) + require(testUtils.getLatestOffsets(Set(topic)).size === 5) + + val reader = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.group.id", s"group-$topic") + .option("kafka.auto.offset.reset", s"latest") + .option("kafka.metadata.max.age.ms", "1") + .option("subscribePattern", s"$topicPrefix-.*") + + val kafka = reader.load().select("key", "value").as[(Array[Byte], Array[Byte])] + val mapped = kafka.map(kv => new String(kv._2).toInt + 1) + + testStream(mapped)( + AddKafkaData(Set(topic), 1, 2, 3), + CheckAnswer(2, 3, 4), + Assert { + testUtils.deleteTopic(topic, 5) + testUtils.createTopic(topic2, partitions = 5) + true + }, + AddKafkaData(Set(topic2), 4, 5, 6), + CheckAnswer(2, 3, 4, 5, 6, 7) + ) + } private def newTopic(): String = s"topic-${topicId.getAndIncrement()}" diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala index c201da96b9a9..ca613df0fe91 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala @@ -29,7 +29,8 @@ import scala.util.Random import kafka.admin.AdminUtils import kafka.api.Request -import kafka.server.{KafkaConfig, KafkaServer} +import kafka.common.TopicAndPartition +import kafka.server.{KafkaConfig, KafkaServer, OffsetCheckpoint} import kafka.utils.ZkUtils import org.apache.kafka.clients.consumer.KafkaConsumer import org.apache.kafka.clients.producer._ @@ -171,6 +172,12 @@ class KafkaTestUtils extends Logging { createTopic(topic, 1) } + /** Delete a Kafka topic and wait until it is propagated to the whole cluster */ + def deleteTopic(topic: String, partitions: Int): Unit = { + AdminUtils.deleteTopic(zkUtils, topic) + verifyTopicDeletion(zkUtils, topic, partitions, List(this.server)) + } + /** Add new paritions to a Kafka topic */ def addPartitions(topic: String, partitions: Int): Unit = { AdminUtils.addPartitions(zkUtils, topic, partitions) @@ -234,6 +241,7 @@ class KafkaTestUtils extends Logging { props.put("zookeeper.connect", zkAddress) props.put("log.flush.interval.messages", "1") props.put("replica.socket.timeout.ms", "1500") + props.put("delete.topic.enable", "true") props } @@ -257,6 +265,37 @@ class KafkaTestUtils extends Logging { props } + private def verifyTopicDeletion( + zkUtils: ZkUtils, + topic: String, + numPartitions: Int, + servers: Seq[KafkaServer]) { + import ZkUtils._ + val topicAndPartitions = (0 until numPartitions).map(TopicAndPartition(topic, _)) + def isDeleted(): Boolean = { + // wait until admin path for delete topic is deleted, signaling completion of topic deletion + val deletePath = !zkUtils.pathExists(getDeleteTopicPath(topic)) + val topicPath = !zkUtils.pathExists(getTopicPath(topic)) + // ensure that the topic-partition has been deleted from all brokers' replica managers + val replicaManager = servers.forall(server => topicAndPartitions.forall(tp => + server.replicaManager.getPartition(tp.topic, tp.partition) == None)) + // ensure that logs from all replicas are deleted if delete topic is marked successful + val logManager = servers.forall(server => topicAndPartitions.forall(tp => + server.getLogManager().getLog(tp).isEmpty)) + // ensure that topic is removed from all cleaner offsets + val cleaner = servers.forall(server => topicAndPartitions.forall { tp => + val checkpoints = server.getLogManager().logDirs.map { logDir => + new OffsetCheckpoint(new File(logDir, "cleaner-offset-checkpoint")).read() + } + checkpoints.forall(checkpointsPerLogDir => !checkpointsPerLogDir.contains(tp)) + }) + deletePath && topicPath && replicaManager && logManager && cleaner + } + eventually(timeout(10.seconds)) { + assert(isDeleted, s"$topic not deleted after timeout") + } + } + private def waitUntilMetadataIsPropagated(topic: String, partition: Int): Unit = { def isPropagated = server.apis.metadataCache.getPartitionInfo(topic, partition) match { case Some(partitionState) => From 4fe16c833c4c468d0e041f7f894c8236920c6cc3 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Fri, 23 Sep 2016 14:11:04 -0700 Subject: [PATCH 10/28] Refactored and simplified --- .../sql/kafka010/CachedKafkaConsumer.scala | 156 ++++----- .../spark/sql/kafka010/KafkaSource.scala | 267 ++++----------- .../sql/kafka010/KafkaSourceProvider.scala | 235 ++++++++++++++ .../spark/sql/kafka010/KafkaSourceRDD.scala | 115 ++----- .../sql/kafka010/KafkaSourceOffsetSuite.scala | 8 - .../spark/sql/kafka010/KafkaSourceSuite.scala | 307 ++++++++++-------- 6 files changed, 555 insertions(+), 533 deletions(-) create mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala index 93fa105dc576..131d0986b0db 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala @@ -20,8 +20,9 @@ package org.apache.spark.sql.kafka010 import java.{util => ju} import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer} -import org.apache.kafka.common.{KafkaException, TopicPartition} +import org.apache.kafka.common.TopicPartition +import org.apache.spark.{SparkEnv, SparkException, TaskContext} import org.apache.spark.internal.Logging @@ -30,16 +31,9 @@ import org.apache.spark.internal.Logging * Underlying consumer is not threadsafe, so neither is this, * but processing the same topicpartition and group id in multiple threads is usually bad anyway. */ -class CachedKafkaConsumer private( - val groupId: String, - val topic: String, - val partition: Int, - val kafkaParams: ju.Map[String, Object]) extends Logging { - - assert(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG), - "groupId used for cache key must match the groupId in kafkaParams") - - val topicPartition = new TopicPartition(topic, partition) +private[kafka010] case class CachedKafkaConsumer private( + topicPartition: TopicPartition, + kafkaParams: ju.Map[String, Object]) extends Logging { protected val consumer = { val c = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams) @@ -49,89 +43,82 @@ class CachedKafkaConsumer private( c } - // TODO if the buffer was kept around as a random-access structure, - // could possibly optimize re-calculating of an RDD in the same batch - protected var buffer = - ju.Collections.emptyList[ConsumerRecord[Array[Byte], Array[Byte]]]().iterator - protected var nextOffset = -2L + // Timeout for polls to the consumer can be small as the data is expected to be present in Kafka + private val pollTimeoutMs = 1000 - def close(): Unit = consumer.close() + // Iterator to the already fetch data + protected var buffer = ju.Collections.emptyIterator[ConsumerRecord[Array[Byte], Array[Byte]]] + protected var nextOffset = -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, timeout: Long): ConsumerRecord[Array[Byte], Array[Byte]] = { - logDebug(s"Get $groupId $topic $partition nextOffset $nextOffset requested $offset") + def get(offset: Long): ConsumerRecord[Array[Byte], Array[Byte]] = { + logDebug(s"Get $topicPartition nextOffset $nextOffset requested $offset") if (offset != nextOffset) { - logInfo(s"Initial fetch for $groupId $topic $partition $offset") + logInfo(s"Initial fetch for $topicPartition $offset") seek(offset) - poll(timeout) + poll() } - if (!buffer.hasNext()) { poll(timeout) } + if (!buffer.hasNext()) { poll() } assert(buffer.hasNext(), - s"Failed to get records for $groupId $topic $partition $offset after polling for $timeout") + s"Failed to get records for $topicPartition $offset after polling for $pollTimeoutMs") var record = buffer.next() if (record.offset != offset) { - logInfo(s"Buffer miss for $groupId $topic $partition $offset") + logInfo(s"Buffer miss for $topicPartition $offset") seek(offset) - poll(timeout) + poll() assert(buffer.hasNext(), - s"Failed to get records for $groupId $topic $partition $offset after polling for $timeout") + s"Failed to get records for $topicPartition $offset after polling for $pollTimeoutMs") record = buffer.next() assert(record.offset == offset, - s"Got wrong record for $groupId $topic $partition even after seeking to offset $offset") + s"Got wrong record for $topicPartition even after seeking to offset $offset") } nextOffset = offset + 1 record } + def close(): Unit = consumer.close() + private def seek(offset: Long): Unit = { logDebug(s"Seeking to $topicPartition $offset") consumer.seek(topicPartition, offset) } - private def poll(timeout: Long): Unit = { - val p = consumer.poll(timeout) + private def poll(): Unit = { + val p = consumer.poll(pollTimeoutMs) val r = p.records(topicPartition) logDebug(s"Polled ${p.partitions()} ${r.size}") buffer = r.iterator } - } -object CachedKafkaConsumer extends Logging { - - private case class CacheKey(groupId: String, topic: String, partition: Int) - - // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap - private var cache: ju.LinkedHashMap[CacheKey, CachedKafkaConsumer] = null - - /** Must be called before get, once per JVM, to configure the cache. Further calls are ignored */ - def init( - initialCapacity: Int, - maxCapacity: Int, - loadFactor: Float): Unit = CachedKafkaConsumer.synchronized { - if (null == cache) { - logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor") - cache = new ju.LinkedHashMap[CacheKey, CachedKafkaConsumer]( - initialCapacity, loadFactor, true) { - override def removeEldestEntry( - entry: ju.Map.Entry[CacheKey, CachedKafkaConsumer]): Boolean = { - if (this.size > maxCapacity) { - try { - entry.getValue.consumer.close() - } catch { - case x: KafkaException => - logError("Error closing oldest Kafka consumer", x) - } - true - } else { - false +private[kafka010] object CachedKafkaConsumer extends Logging { + + private case class CacheKey(groupId: String, topicPartition: TopicPartition) + + private lazy val cache = { + val conf = SparkEnv.get.conf + val capacity = conf.getInt("spark.sql.kafkaConsumerCache.capacity", 64) + new ju.LinkedHashMap[CacheKey, CachedKafkaConsumer](capacity, 0.75f, true) { + override def removeEldestEntry( + entry: ju.Map.Entry[CacheKey, CachedKafkaConsumer]): Boolean = { + if (this.size > capacity) { + logWarning(s"KafkaConsumer cache hitting max capacity of $capacity, " + + s"removing consumer for ${entry.getKey}") + try { + entry.getValue.close() + } catch { + case e: SparkException => + logError(s"Error closing earliest Kafka consumer for ${entry.getKey}", e) } + true + } else { + false } } } @@ -141,47 +128,24 @@ object CachedKafkaConsumer extends Logging { * Get a cached consumer for groupId, assigned to topic and partition. * If matching consumer doesn't already exist, will be created using kafkaParams. */ - def get( - groupId: String, + def getOrCreate( topic: String, partition: Int, - kafkaParams: ju.Map[String, Object]): CachedKafkaConsumer = - CachedKafkaConsumer.synchronized { - val k = CacheKey(groupId, topic, partition) - val v = cache.get(k) - if (null == v) { - logInfo(s"Cache miss for $k") - logDebug(cache.keySet.toString) - val c = new CachedKafkaConsumer(groupId, topic, partition, kafkaParams) - cache.put(k, c) - c - } else { - // any given topicpartition should have a consistent key and value type - v.asInstanceOf[CachedKafkaConsumer] + kafkaParams: ju.Map[String, Object]): CachedKafkaConsumer = synchronized { + val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG).asInstanceOf[String] + val topicPartition = new TopicPartition(topic, partition) + val key = CacheKey(groupId, topicPartition) + + // If this is reattempt at running the task, then invalidate cache and start with + // a new consumer + if (TaskContext.get != null && TaskContext.get.attemptNumber > 1) { + cache.remove(key) + new CachedKafkaConsumer(topicPartition, kafkaParams) + } else { + if (!cache.containsKey(key)) { + cache.put(key, new CachedKafkaConsumer(topicPartition, kafkaParams)) } - } - - /** - * Get a fresh new instance, unassociated with the global cache. - * Caller is responsible for closing - */ - def getUncached( - groupId: String, - topic: String, - partition: Int, - kafkaParams: ju.Map[String, Object]): CachedKafkaConsumer = - new CachedKafkaConsumer(groupId, topic, partition, kafkaParams) - - /** remove consumer for given groupId, topic, and partition, if it exists */ - def remove(groupId: String, topic: String, partition: Int): Unit = { - val k = CacheKey(groupId, topic, partition) - logInfo(s"Removing $k from cache") - val v = CachedKafkaConsumer.synchronized { - cache.remove(k) - } - if (null != v) { - v.close() - logInfo(s"Removed $k from cache") + cache.get(key) } } } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index eefcfaa53799..b674181491d3 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -21,19 +21,17 @@ import java.{util => ju} import scala.collection.JavaConverters._ -import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, KafkaConsumer} +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.kafka.common.serialization.ByteArrayDeserializer +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.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 @@ -77,17 +75,18 @@ private[kafka010] case class KafkaSource( sourceOptions: Map[String, String]) extends Source with Logging { - @transient private val consumer = consumerStrategy.createConsumer() - @transient private val sc = sqlContext.sparkContext - @transient private val initialPartitionOffsets = fetchPartitionOffsets(seekToLatest = false) - logInfo(s"Initial offsets: " + initialPartitionOffsets) + 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)) - logDebug(s"GetOffset: $offset") + logInfo(s"GetOffset: $offset") Some(offset) } @@ -95,7 +94,7 @@ private[kafka010] case class KafkaSource( * Returns the data that is between the offsets [`start`, `end`), i.e. end is exclusive. */ override def getBatch(start: Option[Offset], end: Offset): DataFrame = { - logDebug(s"GetBatch called with start = $start, end = $end") + logInfo(s"GetBatch called with start = $start, end = $end") val untilPartitionOffsets = KafkaSourceOffset.getPartitionOffsets(end) val fromPartitionOffsets = start match { case Some(prevBatchEndOffset) => @@ -123,8 +122,10 @@ private[kafka010] case class KafkaSource( } val sortedTopicPartitions = untilPartitionOffsets.keySet.toSeq.sorted(topicPartitionOrdering) val sortedExecutors = getSortedExecutorList(sc) - val numExecutors = sortedExecutors.size + logDebug("Sorted topicPartitions: " + sortedTopicPartitions.mkString(", ")) logDebug("Sorted executors: " + sortedExecutors.mkString(", ")) + + val numExecutors = sortedExecutors.size val offsetRanges = sortedTopicPartitions.map { tp => val fromOffset = fromPartitionOffsets.get(tp).getOrElse { newPartitionOffsets.getOrElse(tp, { @@ -137,17 +138,18 @@ private[kafka010] case class KafkaSource( val preferredLoc = if (numExecutors > 0) { Some(sortedExecutors(positiveMod(tp.hashCode, numExecutors))) } else None - KafkaSourceRDD.OffsetRange(tp, fromOffset, untilOffset, preferredLoc) + KafkaSourceRDDOffsetRange(tp, fromOffset, untilOffset, preferredLoc) }.toArray // Create a RDD that reads from Kafka and get the (key, value) pair as byte arrays. val rdd = new KafkaSourceRDD( - sc, executorKafkaParams, offsetRanges, sourceOptions).map { cr => + sc, executorKafkaParams, offsetRanges).map { cr => Row(cr.checksum, cr.key, cr.offset, cr.partition, cr.serializedKeySize, cr.serializedValueSize, cr.timestamp, cr.timestampType.id, cr.topic, cr.value) } - logInfo("GetBatch: " + offsetRanges.sortBy(_.topicPartition.toString).mkString(", ")) + logInfo("GetBatch generating RDD of offset range: " + + offsetRanges.sortBy(_.topicPartition.toString).mkString(", ")) sqlContext.createDataFrame(rdd, schema) } @@ -158,8 +160,12 @@ private[kafka010] case class KafkaSource( override def toString(): String = s"KafkaSource[$consumerStrategy]" + /** + * Fetch the offset of a partition, either the latest offsets or the current offsets in the + * KafkaConsumer. + */ private def fetchPartitionOffsets(seekToLatest: Boolean): Map[TopicPartition, Long] = { - synchronized { + val partitionOffsets = fetchOffsetWithRetry { logTrace("\tPolling") consumer.poll(0) val partitions = consumer.assignment() @@ -170,15 +176,16 @@ private[kafka010] case class KafkaSource( logDebug("\tSeeked to the end") } logTrace("Getting positions") - val partitionToOffsets = partitions.asScala.map(p => p -> consumer.position(p)) - logDebug(s"Got positions $partitionToOffsets") - partitionToOffsets.toMap + partitions.asScala.map(p => p -> consumer.position(p)).toMap } + logInfo(s"Got partition offsets: $partitionOffsets") + partitionOffsets } + /** Fetch the earliest offsets for newly discovered partitions */ private def fetchNewPartitionEarliestOffsets( newPartitions: Seq[TopicPartition]): Map[TopicPartition, Long] = { - synchronized { + val partitionOffsets = fetchOffsetWithRetry { consumer.poll(0) val partitions = consumer.assignment() logDebug(s"\tPartitioned assigned to consumer: $partitions") @@ -189,6 +196,33 @@ private[kafka010] case class KafkaSource( logDebug(s"Got earliest positions $partitionToOffsets") partitionToOffsets.toMap } + logDebug(s"Got offsets for new partitions: $partitionOffsets") + partitionOffsets + } + + /** Helper function that does multiple retries on the a body of code that returns offsets */ + private def fetchOffsetWithRetry( + body: => Map[TopicPartition, Long]): Map[TopicPartition, Long] = synchronized { + + var result: Option[Map[TopicPartition, Long]] = None + var attempt = 1 + var lastException: Exception = null + while (result.isEmpty && attempt < MAX_OFFSET_FETCH_ATTEMPTS) { + try { + result = Some(body) + } catch { + case e: Exception => + lastException = e + logWarning(s"Error in attempt $attempt getting Kafka offsets: ", e) + attempt += 1 + } + } + if (result.isEmpty) { + assert(attempt >= MAX_OFFSET_FETCH_ATTEMPTS) + assert(lastException != null) + throw lastException + } + result.get } private def positiveMod(a: Long, b: Int): Int = ((a % b).toInt + b) % b @@ -197,6 +231,8 @@ private[kafka010] case class KafkaSource( /** Companion object for the [[KafkaSource]]. */ private[kafka010] object KafkaSource { + val MAX_OFFSET_FETCH_ATTEMPTS = 3 + def kafkaSchema: StructType = StructType(Seq( StructField("checksum", LongType), StructField("key", BinaryType), @@ -257,46 +293,7 @@ private[kafka010] object KafkaSource { /** An [[Offset]] for the [[KafkaSource]]. */ private[kafka010] -case class KafkaSourceOffset(partitionToOffsets: Map[TopicPartition, Long]) extends Offset { - /** - * Returns a negative integer, zero, or a positive integer as this object is less than, equal to, - * or greater than the specified object. - */ - override def compareTo(other: Offset): Int = other match { - case KafkaSourceOffset(otherOffsets) => - val allTopicAndPartitions = (this.partitionToOffsets.keySet ++ otherOffsets.keySet).toSeq - - val comparisons = allTopicAndPartitions.map { tp => - (this.partitionToOffsets.get(tp), otherOffsets.get(tp)) match { - case (Some(a), Some(b)) => - if (a < b) { - -1 - } else if (a > b) { - 1 - } else { - 0 - } - case (None, _) => -1 - case (_, None) => 1 - } - } - val nonZeroSigns = comparisons.filter { _ != 0 }.toSet - nonZeroSigns.size match { - case 0 => 0 // if both empty or only 0s - case 1 => nonZeroSigns.head // if there are only (0s and 1s) or (0s and -1s) - case _ => // there are both 1s and -1s - throw new IllegalArgumentException( - s"Invalid comparison between non-linear histories: $this <=> $other") - } - - case _ => - throw new IllegalArgumentException(s"Cannot compare $this <=> $other") - } - - override def toString(): String = { - partitionToOffsets.toSeq.sortBy(_._1.toString).mkString("[", ", ", "]") - } -} +case class KafkaSourceOffset(partitionToOffsets: Map[TopicPartition, Long]) extends Offset /** Companion object of the [[KafkaSourceOffset]] */ private[kafka010] object KafkaSourceOffset { @@ -320,155 +317,3 @@ private[kafka010] object KafkaSourceOffset { } -/** - * 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 { - private val strategyOptionNames = Set("subscribe", "subscribepattern") - - /** Class to conveniently update Kafka config params, while logging the changes */ - private case class ConfigUpdater(module: String, kafkaParams: Map[String, String]) { - private val map = new ju.HashMap[String, Object](kafkaParams.asJava) - def set(key: String, value: Object): this.type = { - map.put(key, value) - logInfo(s"$module: Set $key to $value, earlier value: ${kafkaParams.get(key).getOrElse("")}") - this - } - - def setIfUnset(key: String, value: Object): ConfigUpdater = { - if (!map.containsKey(key)) { - map.put(key, value) - logInfo(s"$module: Set $key to $value") - } - this - } - - def build(): ju.Map[String, Object] = map - } - - /** - * 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 kafkaParamsForStrategy = - ConfigUpdater("source", specifiedKafkaParams) - .set(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, deserClassName) - .set(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, deserClassName) - .set(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") - .build() - - val kafkaParamsForExecutors = - ConfigUpdater("source", specifiedKafkaParams) - .set(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, deserClassName) - .set(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, deserClassName) - - // So that consumers in executors never throw NoOffsetForPartitionException - .set(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none") - - // So that consumers in executors do not mess with user-specified group id - .set(ConsumerConfig.GROUP_ID_CONFIG, - "spark-executor-" + specifiedKafkaParams(ConsumerConfig.GROUP_ID_CONFIG)) - - // So that consumers in executors no keep committing offsets unnecessaribly - .set(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") - - // If buffer config is not set, it to reasonable value to work around - // buffer issues (see KAFKA-3135) - .setIfUnset(ConsumerConfig.RECEIVE_BUFFER_CONFIG, 65536: java.lang.Integer) - .build() - - val strategy = caseInsensitiveParams.find(x => strategyOptionNames.contains(x._1)).get match { - case ("subscribe", value) => - SubscribeStrategy( - value.split(",").map(_.trim()).filter(_.nonEmpty), - kafkaParamsForStrategy) - case ("subscribepattern", value) => - SubscribePatternStrategy( - value.trim(), - kafkaParamsForStrategy) - case _ => - // Should never reach here as we are already matching on - // matched strategy names - throw new IllegalArgumentException("Unknown option") - } - - new KafkaSource(sqlContext, strategy, kafkaParamsForExecutors, parameters) - } - - private def validateOptions(parameters: Map[String, String]): Unit = { - val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase, v) } - val specifiedStrategies = - caseInsensitiveParams.filter { case(k, _) => strategyOptionNames.contains(k) }.toSeq - if (specifiedStrategies.isEmpty) { - throw new IllegalArgumentException( - "One of the following options must be specified for Kafka source: " - + strategyOptionNames.mkString(", ") + ". See docs for more details.") - } else if (specifiedStrategies.size > 1) { - throw new IllegalArgumentException( - "Only one of the following options can be specified for Kafka source: " - + strategyOptionNames.mkString(", ") + ". See docs for more details.") - } - - val strategy = caseInsensitiveParams.find(x => strategyOptionNames.contains(x._1)).get match { - case ("subscribe", value) => - val topics = value.split(",").map(_.trim).filter(_.nonEmpty) - if (topics.isEmpty) { - throw new IllegalArgumentException( - "No topics to subscribe to as specified value for option " + - s"'subscribe' is '$value'") - } - case ("subscribepattern", value) => - val pattern = caseInsensitiveParams("subscribepattern").trim() - if (pattern.isEmpty) { - throw new IllegalArgumentException( - "Pattern to subscribe is empty as specified value for option " + - s"'subscribePattern' is '$value'") - } - case _ => - // Should never reach here as we are already matching on - // matched strategy names - throw new IllegalArgumentException("Unknown option") - } - - if (!caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG}")) { - throw new IllegalArgumentException( - "Option 'kafka.bootstrap.servers' must be specified for configuring Kafka consumer") - } - - if (!caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.GROUP_ID_CONFIG}")) { - throw new IllegalArgumentException( - "Option 'kafka.group.id' must be specified for configuring Kafka consumer") - } - } - - override def shortName(): String = "kafka" -} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala new file mode 100644 index 000000000000..e0a9e055422a --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala @@ -0,0 +1,235 @@ +/* + * 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 { + private val strategyOptionNames = Set("subscribe", "subscribepattern") + + /** + * 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 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 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 = + ConfigUpdater("executor", specifiedKafkaParams) + .set(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, deserClassName) + .set(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, deserClassName) + + // So that consumers in executors never throw NoOffsetForPartitionException + .set(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none") + + // So that consumers in executors do not mess with any existing group id + .set(ConsumerConfig.GROUP_ID_CONFIG, s"$uniqueGroupId-executor") + + // So that consumers in executors does not commit offsets unnecessaribly + .set(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") + + // If buffer config is not set, set it to reasonable value to work around + // buffer issues (see KAFKA-3135) + .setIfUnset(ConsumerConfig.RECEIVE_BUFFER_CONFIG, 65536: java.lang.Integer) + .build() + + val strategy = caseInsensitiveParams.find(x => strategyOptionNames.contains(x._1)).get match { + case ("subscribe", value) => + SubscribeStrategy( + value.split(",").map(_.trim()).filter(_.nonEmpty), + kafkaParamsForStrategy) + case ("subscribepattern", value) => + SubscribePatternStrategy( + value.trim(), + kafkaParamsForStrategy) + case _ => + // Should never reach here as we are already matching on + // matched strategy names + throw new IllegalArgumentException("Unknown option") + } + + new KafkaSource(sqlContext, strategy, kafkaParamsForExecutors, parameters) + } + + private def validateOptions(parameters: Map[String, String]): Unit = { + val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase, v) } + val specifiedStrategies = + caseInsensitiveParams.filter { case (k, _) => strategyOptionNames.contains(k) }.toSeq + if (specifiedStrategies.isEmpty) { + throw new IllegalArgumentException( + "One of the following options must be specified for Kafka source: " + + strategyOptionNames.mkString(", ") + ". See docs for more details.") + } else if (specifiedStrategies.size > 1) { + throw new IllegalArgumentException( + "Only one of the following options can be specified for Kafka source: " + + strategyOptionNames.mkString(", ") + ". See docs for more details.") + } + + val strategy = caseInsensitiveParams.find(x => strategyOptionNames.contains(x._1)).get match { + case ("subscribe", value) => + val topics = value.split(",").map(_.trim).filter(_.nonEmpty) + if (topics.isEmpty) { + throw new IllegalArgumentException( + "No topics to subscribe to as specified value for option " + + s"'subscribe' is '$value'") + } + case ("subscribepattern", value) => + val pattern = caseInsensitiveParams("subscribepattern").trim() + if (pattern.isEmpty) { + throw new IllegalArgumentException( + "Pattern to subscribe is empty as specified value for option " + + s"'subscribePattern' is '$value'") + } + case _ => + // Should never reach here as we are already matching on + // matched strategy names + throw new IllegalArgumentException("Unknown option") + } + + if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.GROUP_ID_CONFIG}")) { + throw new IllegalArgumentException( + s"Kafka option '${ConsumerConfig.GROUP_ID_CONFIG}' is not supported as " + + s"user-specified consumer groups is not used to track offsets.") + } + + if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG}")) { + throw new IllegalArgumentException( + s"Kafka option '${ConsumerConfig.GROUP_ID_CONFIG}' is not supported as keys are " + + s"deserialized as byte arrays with ByteArrayDeserializer. Use Dataframe operations to " + + s"explicitly deserialize the keys.") + } + + if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG}")) + { + throw new IllegalArgumentException( + s"Kafka option '${ConsumerConfig.GROUP_ID_CONFIG}' is not supported as value are " + + s"deserialized as byte arrays with ByteArrayDeserializer. Use Dataframe operations to " + + s"explicitly deserialize the values.") + } + + if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG}")) { + val supportedValues = Set("earliest", "latest") + val value = caseInsensitiveParams(s"kafka.${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG}") + if (!supportedValues.contains(value)) { + throw new IllegalArgumentException(s"Value $value for Kafka option " + + s"'${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG}' is not supported") + } + } + + val otherUnsupportedConfigs = Seq( + ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, // committing correctly requires new APIs in Source + ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG) // interceptors can modify payload, so not safe + + otherUnsupportedConfigs.foreach { c => + if (caseInsensitiveParams.contains(s"kafka.$c")) { + throw new IllegalArgumentException(s"Kafka option '$c' is not supported") + } + } + + if (!caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG}")) { + throw new IllegalArgumentException( + s"Option 'kafka.${ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG}' must be specified for " + + s"configuring Kafka consumer") + } + } + + override def shortName(): String = "kafka" + + /** Class to conveniently update Kafka config params, while logging the changes */ + private case class ConfigUpdater(module: String, kafkaParams: Map[String, String]) { + private val map = new ju.HashMap[String, Object](kafkaParams.asJava) + + def set(key: String, value: Object): this.type = { + map.put(key, value) + logInfo(s"$module: Set $key to $value, earlier value: ${kafkaParams.get(key).getOrElse("")}") + this + } + + def setIfUnset(key: String, value: Object): ConfigUpdater = { + if (!map.containsKey(key)) { + map.put(key, value) + logInfo(s"$module: Set $key to $value") + } + this + } + + def build(): ju.Map[String, Object] = map + } +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala index 5b553260804f..6d984dde4621 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala @@ -20,20 +20,31 @@ package org.apache.spark.sql.kafka010 import java.{util => ju} import scala.collection.mutable.ArrayBuffer -import scala.util.Try -import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord} +import org.apache.kafka.clients.consumer.ConsumerRecord import org.apache.kafka.common.TopicPartition -import org.apache.spark.{Partition, SparkContext, SparkEnv, TaskContext} -import org.apache.spark.internal.Logging +import org.apache.spark.{Partition, SparkContext, TaskContext} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.kafka010.KafkaSourceRDD._ import org.apache.spark.storage.StorageLevel -private[kafka010] -case class KafkaSourceRDDPartition(index: Int, offsetRange: OffsetRange) extends Partition +/** 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. @@ -41,17 +52,15 @@ case class KafkaSourceRDDPartition(index: Int, offsetRange: OffsetRange) extends * 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]]. + * 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 - * @param sourceOptions Options provided through the source */ private[kafka010] class KafkaSourceRDD( sc: SparkContext, executorKafkaParams: ju.Map[String, Object], - offsetRanges: Seq[OffsetRange], - sourceOptions: Map[String, String]) + offsetRanges: Seq[KafkaSourceRDDOffsetRange]) extends RDD[ConsumerRecord[Array[Byte], Array[Byte]]](sc, Nil) { override def persist(newLevel: StorageLevel): this.type = { @@ -110,80 +119,24 @@ private[kafka010] class KafkaSourceRDD( logInfo(s"Beginning offset ${range.fromOffset} is the same as ending offset " + s"skipping ${range.topic} ${range.partition}") Iterator.empty - } else { - new KafkaRDDIterator(range, executorKafkaParams, sourceOptions, context) - } - } -} -private[kafka010] object KafkaSourceRDD { - - /** Offset range that one partition of the KafkaSourceRDD has to read */ - case class OffsetRange( - topicPartition: TopicPartition, - fromOffset: Long, - untilOffset: Long, - preferredLoc: Option[String]) { - def topic: String = topicPartition.topic - def partition: Int = topicPartition.partition - def size: Long = untilOffset - fromOffset - } - - /** - * An iterator that fetches messages directly from Kafka for the offsets in partition. - * Uses a cached consumer where possible to take advantage of prefetching - */ - private class KafkaRDDIterator( - part: OffsetRange, - executorKafkaParams: ju.Map[String, Object], - options: Map[String, String], - context: TaskContext) - extends Iterator[ConsumerRecord[Array[Byte], Array[Byte]]] with Logging { - - logInfo(s"Computing topic ${part.topic}, partition ${part.partition} " + - s"offsets ${part.fromOffset} -> ${part.untilOffset}") - - val groupId = executorKafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG).asInstanceOf[String] - - // Time between polling for more data by the KafkaConsumer in the executor. This should not - // require much configuration as data should already be available in Kafka when the executors - // are polling. - private val pollTimeout = getLong(options, "consumer.pollMs", 512) - - // Configurations for initializing the cache of KafkaConsumers. - private val cacheInitialCapacity = - SparkEnv.get.conf.getInt("kafka.consumer.cache.initialCapacity", 16) - private val cacheMaxCapacity = - SparkEnv.get.conf.getInt("kafka.consumer.cache.maxCapacity", 64) - private val cacheLoadFactor = - SparkEnv.get.conf.getDouble("kafka.consumer.cache.loadFactor", 0.75).toFloat - - // Initialize the cache if not already done, and get a cached KafkaConsumer - CachedKafkaConsumer.init(cacheInitialCapacity, cacheMaxCapacity, cacheLoadFactor) - if (context.attemptNumber > 1) { - // Just in case the prior attempt failures were cache related - CachedKafkaConsumer.remove(groupId, part.topic, part.partition) - } - val consumer = - CachedKafkaConsumer.get(groupId, part.topic, part.partition, executorKafkaParams) - - var requestOffset = part.fromOffset + } else { - override def hasNext(): Boolean = requestOffset < part.untilOffset + val consumer = CachedKafkaConsumer.getOrCreate( + range.topic, range.partition, executorKafkaParams) + var requestOffset = range.fromOffset - override def next(): ConsumerRecord[Array[Byte], Array[Byte]] = { - assert(hasNext(), "Can't call next() once untilOffset has been reached") - val r = consumer.get(requestOffset, pollTimeout) - requestOffset += 1 - r - } - } + logDebug(s"Creating iterator for $range") - def getLong(options: Map[String, String], name: String, defaultValue: Long): Long = { - options.get(name).map { str => - Try(str.toLong).getOrElse { - throw new IllegalArgumentException("Option '$name' must be a long") + new Iterator[ConsumerRecord[Array[Byte], Array[Byte]]]() { + override def hasNext(): Boolean = requestOffset < range.untilOffset + override def next(): ConsumerRecord[Array[Byte], Array[Byte]] = { + assert(hasNext(), "Can't call next() once untilOffset has been reached") + val r = consumer.get(requestOffset) + requestOffset += 1 + r + } } - }.getOrElse(defaultValue) + } } } diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala index c6ec94e874c3..7056a41b1751 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala @@ -36,12 +36,4 @@ class KafkaSourceOffsetSuite extends OffsetSuite { compare( one = KafkaSourceOffset(("t", 0, 1L)), two = KafkaSourceOffset(("t", 0, 2L), ("t", 1, 1L))) - - compareInvalid( - one = KafkaSourceOffset(("t", 1, 1L)), - two = KafkaSourceOffset(("t", 0, 2L))) - - compareInvalid( - one = KafkaSourceOffset(("t", 0, 1L)), - two = KafkaSourceOffset(("T", 0, 2L))) } diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala index b4d16d6420f6..6764c2233c24 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala @@ -29,12 +29,9 @@ import org.apache.spark.sql.streaming.StreamTest import org.apache.spark.sql.test.SharedSQLContext -class KafkaSourceSuite extends StreamTest with SharedSQLContext { +abstract class KafkaSourceTest extends StreamTest with SharedSQLContext { - import testImplicits._ - - private val topicId = new AtomicInteger(0) - private var testUtils: KafkaTestUtils = _ + protected var testUtils: KafkaTestUtils = _ override val streamingTimeout = 30.seconds @@ -52,6 +49,73 @@ class KafkaSourceSuite extends StreamTest with SharedSQLContext { } } + /** + * Add data to Kafka. If any topic in `topics` does not exist, it will be created automatically. + * + * `addPartitions` is the new partition numbers of the topics. The caller should make sure using + * a bigger partition number. Otherwise, it will throw an exception. + */ + case class AddKafkaData(topics: Set[String], data: Int*) + (implicit ensureDataInMultiplePartition: Boolean = false, + addPartitions: Option[Seq[Int]] = None) extends AddData { + + override def addData(query: Option[StreamExecution]): (Source, Offset) = { + val allTopics = testUtils.getAllTopics().toSet + if (addPartitions.nonEmpty) { + require(topics.size == addPartitions.get.size, + s"$addPartitions should have the same size of $topics") + topics.zip(addPartitions.get).foreach { case (topic, partitions) => + if (allTopics.contains(topic)) { + testUtils.addPartitions(topic, partitions) + } else { + testUtils.createTopic(topic, partitions) + } + } + } + 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()}" + } + // Verify that the test data gets inserted into multiple partitions + if (ensureDataInMultiplePartition) { + require( + sentMetadata.groupBy(_._2.partition).size > 1, + s"Added data does not test multiple partitions: ${sentMetadata.map(metadataToStr)}") + } + + val offset = KafkaSourceOffset(testUtils.getLatestOffsets(topics)) + logInfo(s"Added data, expected offset $offset") + (kafkaSource, offset) + } + } +} + + +class KafkaSourceSuite extends KafkaSourceTest { + + import testImplicits._ + + private val topicId = new AtomicInteger(0) + test("subscribing topic by name from latest offsets") { val topic = newTopic() testFromLatestOffsets(topic, "subscribe" -> topic) @@ -74,60 +138,36 @@ class KafkaSourceSuite extends StreamTest with SharedSQLContext { testFromEarliestOffsets(topic, "subscribePattern" -> s"$topicPrefix-.*") } - test("stress test with multiple topics and partitions") { - val topicId = new AtomicInteger(1) - - def newStressTopic: String = s"stress${topicId.getAndIncrement()}" - - @volatile var topics = (1 to 5).map(_ => newStressTopic).toSet - - @volatile var partitionRange = (1, 5) - - def newPartitionRange: (Int, Int) = (partitionRange._1 + 5, partitionRange._2 + 5) - - def randomPartitions: Int = { - Random.nextInt(partitionRange._2 + 1 - partitionRange._1) + partitionRange._1 - } - - topics.foreach { topic => - testUtils.createTopic(topic, partitions = randomPartitions) - testUtils.sendMessages(topic, (101 to 105).map { _.toString }.toArray) - } + test("subscribing topic by pattern with topic deletions") { + val topicPrefix = newTopic() + val topic = topicPrefix + "-seems" + val topic2 = topicPrefix + "-bad" + testUtils.createTopic(topic, partitions = 5) + testUtils.sendMessages(topic, Array("-1")) + require(testUtils.getLatestOffsets(Set(topic)).size === 5) - // Create Kafka source that reads from latest offset - val kafka = - spark.readStream - .format(classOf[KafkaSourceProvider].getCanonicalName.stripSuffix("$")) - .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("kafka.group.id", s"group-stress-test") - .option("kafka.metadata.max.age.ms", "1") - .option("subscribePattern", "stress.*") - .load() - .select("key", "value") - .as[(Array[Byte], Array[Byte])] + val reader = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.auto.offset.reset", s"latest") + .option("kafka.metadata.max.age.ms", "1") + .option("subscribePattern", s"$topicPrefix-.*") + val kafka = reader.load().select("key", "value").as[(Array[Byte], Array[Byte])] val mapped = kafka.map(kv => new String(kv._2).toInt + 1) - runStressTest( - mapped, - d => { - Random.nextInt(5) match { - case 0 => - partitionRange = newPartitionRange - val addPartitions = topics.toSeq.map(_ => randomPartitions) - AddKafkaData(topics, d: _*)( - ensureDataInMultiplePartition = false, addPartitions = Some(addPartitions)) - case 1 => - topics = topics + newStressTopic - partitionRange = newPartitionRange - val addPartitions = topics.toSeq.map(_ => randomPartitions) - AddKafkaData(topics, d: _*)( - ensureDataInMultiplePartition = false, addPartitions = Some(addPartitions)) - case _ => - AddKafkaData(topics, d: _*)(ensureDataInMultiplePartition = false) - } + testStream(mapped)( + AddKafkaData(Set(topic), 1, 2, 3), + CheckAnswer(2, 3, 4), + Assert { + testUtils.deleteTopic(topic, 5) + testUtils.createTopic(topic2, partitions = 5) + true }, - iterations = 50) + AddKafkaData(Set(topic2), 4, 5, 6), + CheckAnswer(2, 3, 4, 5, 6, 7) + ) } test("bad source options") { @@ -155,37 +195,29 @@ class KafkaSourceSuite extends StreamTest with SharedSQLContext { testBadOptions("subscribePattern" -> "")("pattern to subscribe is empty") } - test("users will delete topics") { - val topicPrefix = newTopic() - val topic = topicPrefix + "-seems" - val topic2 = topicPrefix + "-bad" - testUtils.createTopic(topic, partitions = 5) - testUtils.sendMessages(topic, Array("-1")) - require(testUtils.getLatestOffsets(Set(topic)).size === 5) - - val reader = spark - .readStream - .format("kafka") - .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("kafka.group.id", s"group-$topic") - .option("kafka.auto.offset.reset", s"latest") - .option("kafka.metadata.max.age.ms", "1") - .option("subscribePattern", s"$topicPrefix-.*") + test("unsupported kafka configs") { + def testUnsupportedConfig(key: String, value: String = "someValue"): Unit = { + val ex = intercept[IllegalArgumentException] { + val reader = spark + .readStream + .format("kafka") + .option("subscribe", "topic") + .option("kafka.bootstrap.servers", "somehost") + .option(s"$key", value) + reader.load() + } + assert(ex.getMessage.toLowerCase.contains("not supported")) + } - val kafka = reader.load().select("key", "value").as[(Array[Byte], Array[Byte])] - val mapped = kafka.map(kv => new String(kv._2).toInt + 1) + testUnsupportedConfig("kafka.group.id") + testUnsupportedConfig("kafka.enable.auto.commit") + testUnsupportedConfig("kafka.interceptor.classes") + testUnsupportedConfig("kafka.key.deserializer") + testUnsupportedConfig("kafka.value.deserializer") - testStream(mapped)( - AddKafkaData(Set(topic), 1, 2, 3), - CheckAnswer(2, 3, 4), - Assert { - testUtils.deleteTopic(topic, 5) - testUtils.createTopic(topic2, partitions = 5) - true - }, - AddKafkaData(Set(topic2), 4, 5, 6), - CheckAnswer(2, 3, 4, 5, 6, 7) - ) + // only earliest and latest is supported + testUnsupportedConfig("kafka.auto.offset.reset", "none") + testUnsupportedConfig("kafka.auto.offset.reset", "someValue") } private def newTopic(): String = s"topic-${topicId.getAndIncrement()}" @@ -199,7 +231,6 @@ class KafkaSourceSuite extends StreamTest with SharedSQLContext { .readStream .format("kafka") .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("kafka.group.id", s"group-$topic") .option("kafka.auto.offset.reset", s"latest") .option("kafka.metadata.max.age.ms", "1") options.foreach { case (k, v) => reader.option(k, v) } @@ -236,7 +267,6 @@ class KafkaSourceSuite extends StreamTest with SharedSQLContext { reader .format(classOf[KafkaSourceProvider].getCanonicalName.stripSuffix("$")) .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("kafka.group.id", s"group-$topic") .option("kafka.auto.offset.reset", s"earliest") .option("kafka.metadata.max.age.ms", "1") options.foreach { case (k, v) => reader.option(k, v) } @@ -261,63 +291,66 @@ class KafkaSourceSuite extends StreamTest with SharedSQLContext { CheckAnswer(2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17) ) } +} - /** - * Add data to Kafka. If any topic in `topics` does not exist, it will be created automatically. - * - * `addPartitions` is the new partition numbers of the topics. The caller should make sure using - * a bigger partition number. Otherwise, it will throw an exception. - */ - case class AddKafkaData(topics: Set[String], data: Int*) - (implicit ensureDataInMultiplePartition: Boolean = false, - addPartitions: Option[Seq[Int]] = None) extends AddData { - override def addData(query: Option[StreamExecution]): (Source, Offset) = { - val allTopics = testUtils.getAllTopics().toSet - if (addPartitions.nonEmpty) { - require(topics.size == addPartitions.get.size, - s"$addPartitions should have the same size of $topics") - topics.zip(addPartitions.get).foreach { case (topic, partitions) => - if (allTopics.contains(topic)) { - testUtils.addPartitions(topic, partitions) - } else { - testUtils.createTopic(topic, partitions) - } - } - } - require( - query.nonEmpty, - "Cannot add data when there is no query for finding the active kafka source") +class KafkaSourceStressSuite extends KafkaSourceTest { - 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) + import testImplicits._ - def metadataToStr(m: (String, RecordMetadata)): String = { - s"Sent ${m._1} to partition ${m._2.partition()}, offset ${m._2.offset()}" - } - // Verify that the test data gets inserted into multiple partitions - if (ensureDataInMultiplePartition) { - require( - sentMetadata.groupBy(_._2.partition).size > 1, - s"Added data does not test multiple partitions: ${sentMetadata.map(metadataToStr)}") - } + val topicId = new AtomicInteger(1) - val offset = KafkaSourceOffset(testUtils.getLatestOffsets(topics)) - logInfo(s"Added data, expected offset $offset") - (kafkaSource, offset) + @volatile var topics = (1 to 5).map(_ => newStressTopic).toSet + + @volatile var partitionRange = (1, 5) + + + test("stress test with multiple topics and partitions") { + topics.foreach { topic => + testUtils.createTopic(topic, partitions = randomPartitions) + testUtils.sendMessages(topic, (101 to 105).map { _.toString }.toArray) } + + // Create Kafka source that reads from latest offset + val kafka = + spark.readStream + .format(classOf[KafkaSourceProvider].getCanonicalName.stripSuffix("$")) + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + .option("subscribePattern", "stress.*") + .load() + .select("key", "value") + .as[(Array[Byte], Array[Byte])] + + val mapped = kafka.map(kv => new String(kv._2).toInt + 1) + + runStressTest( + mapped, + d => { + Random.nextInt(5) match { + case 0 => + partitionRange = newPartitionRange + val addPartitions = topics.toSeq.map(_ => randomPartitions) + AddKafkaData(topics, d: _*)( + ensureDataInMultiplePartition = false, addPartitions = Some(addPartitions)) + case 1 => + topics = topics + newStressTopic + partitionRange = newPartitionRange + val addPartitions = topics.toSeq.map(_ => randomPartitions) + AddKafkaData(topics, d: _*)( + ensureDataInMultiplePartition = false, addPartitions = Some(addPartitions)) + case _ => + AddKafkaData(topics, d: _*)(ensureDataInMultiplePartition = false) + } + }, + iterations = 50) + } + + def newStressTopic: String = s"stress${topicId.getAndIncrement()}" + + def newPartitionRange: (Int, Int) = (partitionRange._1 + 5, partitionRange._2 + 5) + + def randomPartitions: Int = { + Random.nextInt(partitionRange._2 + 1 - partitionRange._1) + partitionRange._1 } } From e1671525d142b249749fcd58d5d3bc325fc46c71 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Fri, 23 Sep 2016 15:48:27 -0700 Subject: [PATCH 11/28] Added source option startingOffset --- .../sql/kafka010/CachedKafkaConsumer.scala | 32 ++++---- .../spark/sql/kafka010/KafkaSource.scala | 73 +++++++++-------- .../sql/kafka010/KafkaSourceProvider.scala | 81 +++++++++++++------ .../spark/sql/kafka010/KafkaSourceSuite.scala | 6 +- 4 files changed, 114 insertions(+), 78 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala index 131d0986b0db..92e7933f8f25 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala @@ -43,42 +43,46 @@ private[kafka010] case class CachedKafkaConsumer private( c } - // Timeout for polls to the consumer can be small as the data is expected to be present in Kafka - private val pollTimeoutMs = 1000 + /** + * 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 - protected var buffer = ju.Collections.emptyIterator[ConsumerRecord[Array[Byte], Array[Byte]]] - protected var nextOffset = -2L + /** Iterator to the already fetch data */ + protected var fetchedData = ju.Collections.emptyIterator[ConsumerRecord[Array[Byte], Array[Byte]]] + protected 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 $nextOffset requested $offset") - if (offset != nextOffset) { + logDebug(s"Get $topicPartition nextOffset $nextOffsetInFetchedData requested $offset") + if (offset != nextOffsetInFetchedData) { logInfo(s"Initial fetch for $topicPartition $offset") seek(offset) poll() } - if (!buffer.hasNext()) { poll() } - assert(buffer.hasNext(), + if (!fetchedData.hasNext()) { poll() } + assert(fetchedData.hasNext(), s"Failed to get records for $topicPartition $offset after polling for $pollTimeoutMs") - var record = buffer.next() + var record = fetchedData.next() if (record.offset != offset) { logInfo(s"Buffer miss for $topicPartition $offset") seek(offset) poll() - assert(buffer.hasNext(), + assert(fetchedData.hasNext(), s"Failed to get records for $topicPartition $offset after polling for $pollTimeoutMs") - record = buffer.next() + record = fetchedData.next() assert(record.offset == offset, s"Got wrong record for $topicPartition even after seeking to offset $offset") } - nextOffset = offset + 1 + nextOffsetInFetchedData = offset + 1 record } @@ -93,7 +97,7 @@ private[kafka010] case class CachedKafkaConsumer private( val p = consumer.poll(pollTimeoutMs) val r = p.records(topicPartition) logDebug(s"Polled ${p.partitions()} ${r.size}") - buffer = r.iterator + fetchedData = r.iterator } } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index b674181491d3..8c79e454a216 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -57,7 +57,7 @@ import org.apache.spark.sql.types._ * 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]]. + * 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 @@ -78,7 +78,6 @@ private[kafka010] case class KafkaSource( 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 @@ -90,9 +89,7 @@ private[kafka010] case class KafkaSource( Some(offset) } - /** - * Returns the data that is between the offsets [`start`, `end`), i.e. end is exclusive. - */ + /** Returns the data that is between the offsets [`start`, `end`), i.e. end is exclusive. */ override def getBatch(start: Option[Offset], end: Offset): DataFrame = { logInfo(s"GetBatch called with start = $start, end = $end") val untilPartitionOffsets = KafkaSourceOffset.getPartitionOffsets(end) @@ -103,6 +100,7 @@ private[kafka010] case class KafkaSource( initialPartitionOffsets } + // Find the new partitions, and get their earliest offsets val newPartitions = untilPartitionOffsets.keySet.diff(fromPartitionOffsets.keySet) val newPartitionOffsets = if (newPartitions.nonEmpty) { fetchNewPartitionEarliestOffsets(newPartitions.toSeq) @@ -121,11 +119,12 @@ private[kafka010] case class KafkaSource( } } val sortedTopicPartitions = untilPartitionOffsets.keySet.toSeq.sorted(topicPartitionOrdering) - val sortedExecutors = getSortedExecutorList(sc) logDebug("Sorted topicPartitions: " + sortedTopicPartitions.mkString(", ")) + + val sortedExecutors = getSortedExecutorList(sc) + val numExecutors = sortedExecutors.length logDebug("Sorted executors: " + sortedExecutors.mkString(", ")) - val numExecutors = sortedExecutors.size val offsetRanges = sortedTopicPartitions.map { tp => val fromOffset = fromPartitionOffsets.get(tp).getOrElse { newPartitionOffsets.getOrElse(tp, { @@ -164,44 +163,48 @@ private[kafka010] case class KafkaSource( * Fetch the offset of a partition, either the latest offsets or the current offsets in the * KafkaConsumer. */ - private def fetchPartitionOffsets(seekToLatest: Boolean): Map[TopicPartition, Long] = { - val partitionOffsets = fetchOffsetWithRetry { - logTrace("\tPolling") - consumer.poll(0) - val partitions = consumer.assignment() - consumer.pause(partitions) - logDebug(s"\tPartitioned assigned to consumer: $partitions") - if (seekToLatest) { - consumer.seekToEnd(partitions) - logDebug("\tSeeked to the end") - } - logTrace("Getting positions") - partitions.asScala.map(p => p -> consumer.position(p)).toMap + private def fetchPartitionOffsets( + seekToLatest: Boolean): Map[TopicPartition, Long] = withRetries { + + // Poll to get the latest assigned partitions + logTrace("\tPolling") + consumer.poll(0) + val partitions = consumer.assignment() + consumer.pause(partitions) + logDebug(s"\tPartitioned assigned to consumer: $partitions") + + // Get the current or latest offset of each partition + if (seekToLatest) { + consumer.seekToEnd(partitions) + logDebug("\tSeeked to the end") } + logTrace("Getting positions") + val partitionOffsets = partitions.asScala.map(p => p -> consumer.position(p)).toMap logInfo(s"Got partition offsets: $partitionOffsets") partitionOffsets } /** Fetch the earliest offsets for newly discovered partitions */ private def fetchNewPartitionEarliestOffsets( - newPartitions: Seq[TopicPartition]): Map[TopicPartition, Long] = { - val partitionOffsets = fetchOffsetWithRetry { - consumer.poll(0) - val partitions = consumer.assignment() - logDebug(s"\tPartitioned assigned to consumer: $partitions") - require(newPartitions.forall(tp => partitions.contains(tp)), - s"$partitions doesn't contain all new paritions: $newPartitions") - consumer.seekToBeginning(newPartitions.asJava) - val partitionToOffsets = newPartitions.map(p => p -> consumer.position(p)) - logDebug(s"Got earliest positions $partitionToOffsets") - partitionToOffsets.toMap - } - logDebug(s"Got offsets for new partitions: $partitionOffsets") - partitionOffsets + newPartitions: Seq[TopicPartition]): Map[TopicPartition, Long] = withRetries { + + // Poll to get the latest assigned partitions + logTrace("\tPolling") + consumer.poll(0) + val partitions = consumer.assignment() + logDebug(s"\tPartitioned assigned to consumer: $partitions") + require(newPartitions.forall(tp => partitions.contains(tp)), + s"$partitions doesn't contain all new paritions: $newPartitions") + + // Get the earliest offset of each partition + consumer.seekToBeginning(newPartitions.asJava) + val partitionToOffsets = newPartitions.map(p => p -> consumer.position(p)).toMap + logInfo(s"Got offsets for new partitions: $partitionToOffsets") + partitionToOffsets } /** Helper function that does multiple retries on the a body of code that returns offsets */ - private def fetchOffsetWithRetry( + private def withRetries( body: => Map[TopicPartition, Long]): Map[TopicPartition, Long] = synchronized { var result: Option[Map[TopicPartition, Long]] = None diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala index e0a9e055422a..3de060ad1ee4 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala @@ -38,29 +38,29 @@ import org.apache.spark.sql.types.StructType * missing options even before the query is started. */ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider -with DataSourceRegister with Logging { - private val strategyOptionNames = Set("subscribe", "subscribepattern") + 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) = { + 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) + 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 @@ -72,6 +72,11 @@ with DataSourceRegister with Logging { 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) @@ -80,6 +85,9 @@ with DataSourceRegister with Logging { // 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") @@ -110,7 +118,7 @@ with DataSourceRegister with Logging { .setIfUnset(ConsumerConfig.RECEIVE_BUFFER_CONFIG, 65536: java.lang.Integer) .build() - val strategy = caseInsensitiveParams.find(x => strategyOptionNames.contains(x._1)).get match { + val strategy = caseInsensitiveParams.find(x => STRATEGY_OPTION_KEYS.contains(x._1)).get match { case ("subscribe", value) => SubscribeStrategy( value.split(",").map(_.trim()).filter(_.nonEmpty), @@ -129,20 +137,23 @@ with DataSourceRegister with Logging { } private def validateOptions(parameters: Map[String, String]): Unit = { + + // Validate source options + val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase, v) } val specifiedStrategies = - caseInsensitiveParams.filter { case (k, _) => strategyOptionNames.contains(k) }.toSeq + caseInsensitiveParams.filter { case (k, _) => STRATEGY_OPTION_KEYS.contains(k) }.toSeq if (specifiedStrategies.isEmpty) { throw new IllegalArgumentException( "One of the following options must be specified for Kafka source: " - + strategyOptionNames.mkString(", ") + ". See docs for more details.") + + STRATEGY_OPTION_KEYS.mkString(", ") + ". See the docs for more details.") } else if (specifiedStrategies.size > 1) { throw new IllegalArgumentException( "Only one of the following options can be specified for Kafka source: " - + strategyOptionNames.mkString(", ") + ". See docs for more details.") + + STRATEGY_OPTION_KEYS.mkString(", ") + ". See the docs for more details.") } - val strategy = caseInsensitiveParams.find(x => strategyOptionNames.contains(x._1)).get match { + val strategy = caseInsensitiveParams.find(x => STRATEGY_OPTION_KEYS.contains(x._1)).get match { case ("subscribe", value) => val topics = value.split(",").map(_.trim).filter(_.nonEmpty) if (topics.isEmpty) { @@ -163,12 +174,32 @@ with DataSourceRegister with Logging { throw new IllegalArgumentException("Unknown option") } + caseInsensitiveParams.get(STARTING_OFFSET_OPTION_KEY) match { + case Some(pos) if !STARTING_OFFSET_OPTION_VALUES.contains(pos.trim.toLowerCase) => + throw new IllegalArgumentException( + s"Illegal value '$pos' for option '$STARTING_OFFSET_OPTION_KEY', " + + s"acceptable values are: ${STARTING_OFFSET_OPTION_VALUES.mkString(", ")}") + case _ => + } + + // Validate user-specified Kafka options + if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.GROUP_ID_CONFIG}")) { throw new IllegalArgumentException( s"Kafka option '${ConsumerConfig.GROUP_ID_CONFIG}' is not supported as " + s"user-specified consumer groups is not used to track offsets.") } + if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG}")) { + throw new IllegalArgumentException( + s""" + |Kafka option '${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG}' is not supported. + |Instead set the source option '$STARTING_OFFSET_OPTION_KEY' to 'earliest' or 'latest' to + |specify where to start. This will ensure that no data is missed when when new + |topics/partitions are dynamically subscribed. See the docs for more details. + """.stripMargin) + } + if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG}")) { throw new IllegalArgumentException( s"Kafka option '${ConsumerConfig.GROUP_ID_CONFIG}' is not supported as keys are " + @@ -184,15 +215,6 @@ with DataSourceRegister with Logging { s"explicitly deserialize the values.") } - if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG}")) { - val supportedValues = Set("earliest", "latest") - val value = caseInsensitiveParams(s"kafka.${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG}") - if (!supportedValues.contains(value)) { - throw new IllegalArgumentException(s"Value $value for Kafka option " + - s"'${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG}' is not supported") - } - } - val otherUnsupportedConfigs = Seq( ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, // committing correctly requires new APIs in Source ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG) // interceptors can modify payload, so not safe @@ -233,3 +255,10 @@ with DataSourceRegister with Logging { def build(): ju.Map[String, Object] = map } } + +private[kafka010] object KafkaSourceProvider { + private val STRATEGY_OPTION_KEYS = Set("subscribe", "subscribepattern") + private val STARTING_OFFSET_OPTION_KEY = "startingoffset" + private val STARTING_OFFSET_OPTION_VALUES = Set("earliest", "latest") + +} diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala index 6764c2233c24..eb0c66ddd24a 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala @@ -150,7 +150,6 @@ class KafkaSourceSuite extends KafkaSourceTest { .readStream .format("kafka") .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("kafka.auto.offset.reset", s"latest") .option("kafka.metadata.max.age.ms", "1") .option("subscribePattern", s"$topicPrefix-.*") @@ -210,6 +209,7 @@ class KafkaSourceSuite extends KafkaSourceTest { } testUnsupportedConfig("kafka.group.id") + testUnsupportedConfig("kafka.auto.offset.reset") testUnsupportedConfig("kafka.enable.auto.commit") testUnsupportedConfig("kafka.interceptor.classes") testUnsupportedConfig("kafka.key.deserializer") @@ -230,8 +230,8 @@ class KafkaSourceSuite extends KafkaSourceTest { val reader = spark .readStream .format("kafka") + .option("startingOffset", s"latest") .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("kafka.auto.offset.reset", s"latest") .option("kafka.metadata.max.age.ms", "1") options.foreach { case (k, v) => reader.option(k, v) } val kafka = reader.load().select("key", "value").as[(Array[Byte], Array[Byte])] @@ -266,8 +266,8 @@ class KafkaSourceSuite extends KafkaSourceTest { val reader = spark.readStream reader .format(classOf[KafkaSourceProvider].getCanonicalName.stripSuffix("$")) + .option("startingOffset", s"earliest") .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("kafka.auto.offset.reset", s"earliest") .option("kafka.metadata.max.age.ms", "1") options.foreach { case (k, v) => reader.option(k, v) } val kafka = reader.load().select("key", "value").as[(Array[Byte], Array[Byte])] From 608b8c39eb1ba9b7db0a1634795b54c30fc655be Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Fri, 23 Sep 2016 17:22:28 -0700 Subject: [PATCH 12/28] Addressed zsxwing's comments --- .../sql/kafka010/CachedKafkaConsumer.scala | 8 ++-- .../spark/sql/kafka010/KafkaSource.scala | 38 +++++++++++++++---- 2 files changed, 34 insertions(+), 12 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala index 92e7933f8f25..d8bc1ef634f1 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala @@ -35,7 +35,7 @@ private[kafka010] case class CachedKafkaConsumer private( topicPartition: TopicPartition, kafkaParams: ju.Map[String, Object]) extends Logging { - protected val consumer = { + private val consumer = { val c = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams) val tps = new ju.ArrayList[TopicPartition]() tps.add(topicPartition) @@ -51,8 +51,8 @@ private[kafka010] case class CachedKafkaConsumer private( private val pollTimeoutMs = 60 * 1000 /** Iterator to the already fetch data */ - protected var fetchedData = ju.Collections.emptyIterator[ConsumerRecord[Array[Byte], Array[Byte]]] - protected var nextOffsetInFetchedData = -2L + 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. @@ -86,7 +86,7 @@ private[kafka010] case class CachedKafkaConsumer private( record } - def close(): Unit = consumer.close() + private def close(): Unit = consumer.close() private def seek(offset: Long): Unit = { logDebug(s"Seeking to $topicPartition $offset") diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index 8c79e454a216..d7024917da0b 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -107,6 +107,13 @@ private[kafka010] case class KafkaSource( } else { Map.empty[TopicPartition, Long] } + logInfo(s"Partitions added: $newPartitionOffsets") + newPartitionOffsets.filter(_._2 != 0).foreach { case (p, o) => + logWarning(s"Added partition $p starts from $o instead of 0, some data may have been missed") + } + + val deletedPartitions = fromPartitionOffsets.keySet.diff(untilPartitionOffsets.keySet) + logWarning(s"Partitions removed: $deletedPartitions, some data may have been missed") // Sort the partitions and current list of executors to consistently assign each partition // to the executor. This allows cached KafkaConsumers in the executors to be re-used to @@ -118,6 +125,9 @@ private[kafka010] case class KafkaSource( (r.topic, r.partition)) } } + + // Use the until partitions to calculate offset ranges to ignore partitions that have + // been deleted val sortedTopicPartitions = untilPartitionOffsets.keySet.toSeq.sorted(topicPartitionOrdering) logDebug("Sorted topicPartitions: " + sortedTopicPartitions.mkString(", ")) @@ -125,12 +135,13 @@ private[kafka010] case class KafkaSource( val numExecutors = sortedExecutors.length logDebug("Sorted executors: " + sortedExecutors.mkString(", ")) + // Calculate offset ranges val offsetRanges = sortedTopicPartitions.map { tp => val fromOffset = fromPartitionOffsets.get(tp).getOrElse { newPartitionOffsets.getOrElse(tp, { - // This should not happen since newPartitionOffsets contains all paritions not in + // This should not happen since newPartitionOffsets contains all partitions not in // fromPartitionOffsets - throw new IllegalStateException(s"$tp doesn't have a offset") + throw new IllegalStateException(s"$tp doesn't have a from offset") }) } val untilOffset = untilPartitionOffsets(tp) @@ -180,7 +191,7 @@ private[kafka010] case class KafkaSource( } logTrace("Getting positions") val partitionOffsets = partitions.asScala.map(p => p -> consumer.position(p)).toMap - logInfo(s"Got partition offsets: $partitionOffsets") + logDebug(s"Got offsets for partition : $partitionOffsets") partitionOffsets } @@ -199,18 +210,22 @@ private[kafka010] case class KafkaSource( // Get the earliest offset of each partition consumer.seekToBeginning(newPartitions.asJava) val partitionToOffsets = newPartitions.map(p => p -> consumer.position(p)).toMap - logInfo(s"Got offsets for new partitions: $partitionToOffsets") + logDebug(s"Got offsets for new partitions: $partitionToOffsets") partitionToOffsets } - /** Helper function that does multiple retries on the a body of code that returns offsets */ + /** + * Helper function that does multiple retries on the a body of code that returns offsets. + * Retries are needed to handle transient failures. For e.g. race conditions between getting + * assignment and getting position while topics/partitions are deleted can cause NPEs. + */ private def withRetries( body: => Map[TopicPartition, Long]): Map[TopicPartition, Long] = synchronized { var result: Option[Map[TopicPartition, Long]] = None var attempt = 1 var lastException: Exception = null - while (result.isEmpty && attempt < MAX_OFFSET_FETCH_ATTEMPTS) { + while (result.isEmpty && attempt <= MAX_OFFSET_FETCH_ATTEMPTS) { try { result = Some(body) } catch { @@ -218,10 +233,11 @@ private[kafka010] case class KafkaSource( lastException = e logWarning(s"Error in attempt $attempt getting Kafka offsets: ", e) attempt += 1 + Thread.sleep(OFFSET_FETCH_ATTEMPT_INTERVAL_MS) } } if (result.isEmpty) { - assert(attempt >= MAX_OFFSET_FETCH_ATTEMPTS) + assert(attempt > MAX_OFFSET_FETCH_ATTEMPTS) assert(lastException != null) throw lastException } @@ -231,10 +247,12 @@ private[kafka010] case class KafkaSource( private def positiveMod(a: Long, b: Int): Int = ((a % b).toInt + b) % b } + /** Companion object for the [[KafkaSource]]. */ private[kafka010] object KafkaSource { val MAX_OFFSET_FETCH_ATTEMPTS = 3 + val OFFSET_FETCH_ATTEMPT_INTERVAL_MS = 10 def kafkaSchema: StructType = StructType(Seq( StructField("checksum", LongType), @@ -296,7 +314,11 @@ private[kafka010] object KafkaSource { /** An [[Offset]] for the [[KafkaSource]]. */ private[kafka010] -case class KafkaSourceOffset(partitionToOffsets: Map[TopicPartition, Long]) extends Offset +case class KafkaSourceOffset(partitionToOffsets: Map[TopicPartition, Long]) extends Offset { + override def toString(): String = { + partitionToOffsets.toSeq.sortBy(_._1.toString).mkString("[", ", ", "]") + } +} /** Companion object of the [[KafkaSourceOffset]] */ private[kafka010] object KafkaSourceOffset { From 3ee83051e2ef68d85e34fe8d2928a6194d086124 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Sun, 25 Sep 2016 23:07:17 -0700 Subject: [PATCH 13/28] Removed line --- .../org/apache/spark/sql/kafka010/KafkaSourceProvider.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala index 3de060ad1ee4..9e10fbc9c35e 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala @@ -260,5 +260,4 @@ private[kafka010] object KafkaSourceProvider { private val STRATEGY_OPTION_KEYS = Set("subscribe", "subscribepattern") private val STARTING_OFFSET_OPTION_KEY = "startingoffset" private val STARTING_OFFSET_OPTION_VALUES = Set("earliest", "latest") - } From 852f607a4253af67d2b425527fa0b87ad20aa953 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Mon, 26 Sep 2016 17:35:35 -0700 Subject: [PATCH 14/28] Fix corner cases and update stress test --- .../spark/sql/kafka010/KafkaSource.scala | 42 ++++-- .../spark/sql/kafka010/KafkaSourceRDD.scala | 29 +++- .../spark/sql/kafka010/KafkaSourceSuite.scala | 130 ++++++++++++------ .../spark/sql/kafka010/KafkaTestUtils.scala | 7 +- .../spark/sql/streaming/StreamTest.scala | 45 ++++-- 5 files changed, 185 insertions(+), 68 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index d7024917da0b..4c6ec4570a95 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -20,6 +20,7 @@ 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 @@ -67,6 +68,9 @@ import org.apache.spark.sql.types._ * 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, @@ -85,7 +89,7 @@ private[kafka010] case class KafkaSource( /** Returns the maximum available offset for this source. */ override def getOffset: Option[Offset] = { val offset = KafkaSourceOffset(fetchPartitionOffsets(seekToLatest = true)) - logInfo(s"GetOffset: $offset") + logDebug(s"GetOffset: ${offset.partitionToOffsets.toSeq.map(_.toString).sorted}") Some(offset) } @@ -107,6 +111,11 @@ private[kafka010] case class KafkaSource( } else { Map.empty[TopicPartition, Long] } + if (newPartitionOffsets.keySet != newPartitions) { + // We cannot get from offsets for some partitions. It means they got deleted. + val deletedPartitions = newPartitions.diff(newPartitionOffsets.keySet) + logWarning(s"Partitions removed: ${deletedPartitions}, some data may have been missed") + } logInfo(s"Partitions added: $newPartitionOffsets") newPartitionOffsets.filter(_._2 != 0).foreach { case (p, o) => logWarning(s"Added partition $p starts from $o instead of 0, some data may have been missed") @@ -128,7 +137,9 @@ private[kafka010] case class KafkaSource( // Use the until partitions to calculate offset ranges to ignore partitions that have // been deleted - val sortedTopicPartitions = untilPartitionOffsets.keySet.toSeq.sorted(topicPartitionOrdering) + val sortedTopicPartitions = untilPartitionOffsets.keySet.filter { tp => + newPartitionOffsets.contains(tp) || fromPartitionOffsets.contains(tp) + }.toSeq.sorted(topicPartitionOrdering) logDebug("Sorted topicPartitions: " + sortedTopicPartitions.mkString(", ")) val sortedExecutors = getSortedExecutorList(sc) @@ -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, " + + "some data may have been missed") + false + } else { + true + } }.toArray // Create a RDD that reads from Kafka and get the (key, value) pair as byte arrays. @@ -175,7 +194,7 @@ private[kafka010] case class KafkaSource( * KafkaConsumer. */ private def fetchPartitionOffsets( - seekToLatest: Boolean): Map[TopicPartition, Long] = withRetries { + seekToLatest: Boolean): Map[TopicPartition, Long] = { // Poll to get the latest assigned partitions logTrace("\tPolling") @@ -197,19 +216,21 @@ private[kafka010] case class KafkaSource( /** Fetch the earliest offsets for newly discovered partitions */ private def fetchNewPartitionEarliestOffsets( - newPartitions: Seq[TopicPartition]): Map[TopicPartition, Long] = withRetries { + newPartitions: Seq[TopicPartition]): Map[TopicPartition, Long] = { // Poll to get the latest assigned partitions logTrace("\tPolling") consumer.poll(0) val partitions = consumer.assignment() logDebug(s"\tPartitioned assigned to consumer: $partitions") - require(newPartitions.forall(tp => partitions.contains(tp)), - s"$partitions doesn't contain all new paritions: $newPartitions") // Get the earliest offset of each partition - consumer.seekToBeginning(newPartitions.asJava) - val partitionToOffsets = newPartitions.map(p => p -> consumer.position(p)).toMap + consumer.seekToBeginning(partitions) + val partitionToOffsets = newPartitions.filter { p => + // When deleting topics happen at the same time, some partitions may not be in `partitions`. + // So we need to ignore them + partitions.contains(p) + }.map(p => p -> consumer.position(p)).toMap logDebug(s"Got offsets for new partitions: $partitionToOffsets") partitionToOffsets } @@ -224,13 +245,14 @@ private[kafka010] case class KafkaSource( var result: Option[Map[TopicPartition, Long]] = None var attempt = 1 - var lastException: Exception = null + var lastException: Throwable = null while (result.isEmpty && attempt <= MAX_OFFSET_FETCH_ATTEMPTS) { try { result = Some(body) } catch { - case e: Exception => + case NonFatal(e) => lastException = e + e.printStackTrace() logWarning(s"Error in attempt $attempt getting Kafka offsets: ", e) attempt += 1 Thread.sleep(OFFSET_FETCH_ATTEMPT_INTERVAL_MS) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala index 6d984dde4621..4778a0d8b1b5 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala @@ -21,7 +21,7 @@ import java.{util => ju} import scala.collection.mutable.ArrayBuffer -import org.apache.kafka.clients.consumer.ConsumerRecord +import org.apache.kafka.clients.consumer.{ConsumerRecord, OffsetOutOfRangeException} import org.apache.kafka.common.TopicPartition import org.apache.spark.{Partition, SparkContext, TaskContext} @@ -129,11 +129,32 @@ private[kafka010] class KafkaSourceRDD( logDebug(s"Creating iterator for $range") new Iterator[ConsumerRecord[Array[Byte], Array[Byte]]]() { - override def hasNext(): Boolean = requestOffset < range.untilOffset + + private var prefetch: ConsumerRecord[Array[Byte], Array[Byte]] = _ + + private def fetchNext(): ConsumerRecord[Array[Byte], Array[Byte]] = { + try { + val r = consumer.get(requestOffset) + requestOffset += 1 + r + } catch { + case e: OffsetOutOfRangeException => + logWarning(s"${range.topicPartition} was deleted, some data may have been missed") + null + } + } + + override def hasNext(): Boolean = { + if (prefetch == null && requestOffset < range.untilOffset) { + prefetch = fetchNext() + } + prefetch != null + } + override def next(): ConsumerRecord[Array[Byte], Array[Byte]] = { assert(hasNext(), "Can't call next() once untilOffset has been reached") - val r = consumer.get(requestOffset) - requestOffset += 1 + val r = prefetch + prefetch = null r } } diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala index eb0c66ddd24a..2458e0397cac 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala @@ -22,6 +22,7 @@ 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._ @@ -50,28 +51,28 @@ abstract class KafkaSourceTest extends StreamTest with SharedSQLContext { } /** - * Add data to Kafka. If any topic in `topics` does not exist, it will be created automatically. + * Add data to Kafka. * - * `addPartitions` is the new partition numbers of the topics. The caller should make sure using - * a bigger partition number. Otherwise, it will throw an exception. + * `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, - addPartitions: Option[Seq[Int]] = None) extends AddData { + concurrent: Boolean = false, + message: String = "", + topicAction: (String, Option[Int]) => Unit = (_, _) => {}) extends AddData { override def addData(query: Option[StreamExecution]): (Source, Offset) = { - val allTopics = testUtils.getAllTopics().toSet - if (addPartitions.nonEmpty) { - require(topics.size == addPartitions.get.size, - s"$addPartitions should have the same size of $topics") - topics.zip(addPartitions.get).foreach { case (topic, partitions) => - if (allTopics.contains(topic)) { - testUtils.addPartitions(topic, partitions) - } else { - testUtils.createTopic(topic, partitions) - } - } + 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") @@ -106,6 +107,9 @@ abstract class KafkaSourceTest extends StreamTest with SharedSQLContext { logInfo(s"Added data, expected offset $offset") (kafkaSource, offset) } + + override def toString: String = + s"AddKafkaData(topics = $topics, data = $data, message = $message)" } } @@ -116,6 +120,26 @@ class KafkaSourceSuite extends KafkaSourceTest { private val topicId = new AtomicInteger(0) + test("cannot stop Kafka stream") { + val topic = newTopic() + testUtils.createTopic(newTopic(), partitions = 5) + testUtils.sendMessages(topic, (101 to 105).map { _.toString }.toArray) + + val reader = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + .option("subscribePattern", s"topic-.*") + + val kafka = reader.load().select("key", "value").as[(Array[Byte], Array[Byte])] + val mapped = kafka.map(kv => new String(kv._2).toInt + 1) + + testStream(mapped)( + StopStream + ) + } + test("subscribing topic by name from latest offsets") { val topic = newTopic() testFromLatestOffsets(topic, "subscribe" -> topic) @@ -160,7 +184,7 @@ class KafkaSourceSuite extends KafkaSourceTest { AddKafkaData(Set(topic), 1, 2, 3), CheckAnswer(2, 3, 4), Assert { - testUtils.deleteTopic(topic, 5) + testUtils.deleteTopic(topic) testUtils.createTopic(topic2, partitions = 5) true }, @@ -294,20 +318,29 @@ class KafkaSourceSuite extends KafkaSourceTest { } -class KafkaSourceStressSuite extends KafkaSourceTest { +class KafkaSourceStressSuite extends KafkaSourceTest with BeforeAndAfter { import testImplicits._ val topicId = new AtomicInteger(1) - @volatile var topics = (1 to 5).map(_ => newStressTopic).toSet + @volatile var topics: Seq[String] = (1 to 5).map(_ => newStressTopic) - @volatile var partitionRange = (1, 5) + def newStressTopic: String = s"stress${topicId.getAndIncrement()}" + private def nextInt(start: Int, end: Int): Int = { + start + Random.nextInt(start + end - 1) + } - test("stress test with multiple topics and partitions") { + after { + for (topic <- testUtils.getAllTopicsAndPartitionSize().toMap.keys) { + testUtils.deleteTopic(topic) + } + } + + private def stressTest(checkAnswer: Boolean): Unit = { topics.foreach { topic => - testUtils.createTopic(topic, partitions = randomPartitions) + testUtils.createTopic(topic, partitions = nextInt(1, 6)) testUtils.sendMessages(topic, (101 to 105).map { _.toString }.toArray) } @@ -326,31 +359,48 @@ class KafkaSourceStressSuite extends KafkaSourceTest { runStressTest( mapped, - d => { + (d, running) => { Random.nextInt(5) match { - case 0 => - partitionRange = newPartitionRange - val addPartitions = topics.toSeq.map(_ => randomPartitions) - AddKafkaData(topics, d: _*)( - ensureDataInMultiplePartition = false, addPartitions = Some(addPartitions)) - case 1 => - topics = topics + newStressTopic - partitionRange = newPartitionRange - val addPartitions = topics.toSeq.map(_ => randomPartitions) - AddKafkaData(topics, d: _*)( - ensureDataInMultiplePartition = false, addPartitions = Some(addPartitions)) - case _ => - AddKafkaData(topics, d: _*)(ensureDataInMultiplePartition = false) + case 0 => // Add a new topic + topics = topics ++ Seq(newStressTopic) + AddKafkaData(topics.toSet, d: _*)(message = s"Add topic $newStressTopic", + topicAction = (topic, partition) => { + if (partition.isEmpty) { + testUtils.createTopic(topic, partitions = nextInt(1, 6)) + } + }) + case 1 if !checkAnswer || running => + // Only delete a topic when the query is running. Otherwise, we may lost data and + // cannot check the correctness. + val deletedTopic = topics(Random.nextInt(topics.size)) + if (deletedTopic != topics.head) { + topics = topics.filterNot(_ == deletedTopic) + } + AddKafkaData(topics.toSet, d: _*)(message = s"Delete topic $deletedTopic", + topicAction = (topic, partition) => { + // Never remove the first topic to make sure we have at least one topic + if (topic == deletedTopic && deletedTopic != topics.head) { + testUtils.deleteTopic(deletedTopic) + } + }) + case 2 => // Add new partitions + AddKafkaData(topics.toSet, d: _*)(message = "Add partitiosn", + topicAction = (topic, partition) => { + testUtils.addPartitions(topic, partition.get + nextInt(1, 6)) + }) + case _ => // Just add new data + AddKafkaData(topics.toSet, d: _*) } }, + checkAnswer = checkAnswer, iterations = 50) } - def newStressTopic: String = s"stress${topicId.getAndIncrement()}" - - def newPartitionRange: (Int, Int) = (partitionRange._1 + 5, partitionRange._2 + 5) + test("stress test with multiple topics and partitions") { + stressTest(checkAnswer = true) + } - def randomPartitions: Int = { - Random.nextInt(partitionRange._2 + 1 - partitionRange._1) + partitionRange._1 + test("don't crash when adding and deleting partitions concurrently") { + stressTest(checkAnswer = false) } } diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala index ca613df0fe91..3eb8a737ba4c 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala @@ -163,8 +163,8 @@ class KafkaTestUtils extends Logging { } } - def getAllTopics(): Seq[String] = { - zkUtils.getAllTopics() + def getAllTopicsAndPartitionSize(): Seq[(String, Int)] = { + zkUtils.getPartitionsForTopics(zkUtils.getAllTopics()).mapValues(_.size).toSeq } /** Create a Kafka topic and wait until it is propagated to the whole cluster */ @@ -173,7 +173,8 @@ class KafkaTestUtils extends Logging { } /** Delete a Kafka topic and wait until it is propagated to the whole cluster */ - def deleteTopic(topic: String, partitions: Int): Unit = { + def deleteTopic(topic: String): Unit = { + val partitions = zkUtils.getPartitionsForTopics(Seq(topic))(topic).size AdminUtils.deleteTopic(zkUtils, topic) verifyTopicDeletion(zkUtils, topic, partitions, List(this.server)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index 6c5b170d9c7c..d7c00c308064 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -50,11 +50,11 @@ import org.apache.spark.util.{Clock, ManualClock, SystemClock, Utils} * * {{{ * val inputData = MemoryStream[Int] - val mapped = inputData.toDS().map(_ + 1) - - testStream(mapped)( - AddData(inputData, 1, 2, 3), - CheckAnswer(2, 3, 4)) + * val mapped = inputData.toDS().map(_ + 1) + ** + *testStream(mapped)( + *AddData(inputData, 1, 2, 3), + *CheckAnswer(2, 3, 4)) * }}} * * Note that while we do sleep to allow the other thread to progress without spinning, @@ -469,29 +469,52 @@ 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, (data, running) => addData(data), true, 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 addData an add data action that adds the given numbers to the stream, encoding them * as needed + * @param checkAnswer should add `CheckAnswer` to the test. + * @param iterations the iteration number */ def runStressTest( ds: Dataset[Int], - addData: Seq[Int] => StreamAction, - iterations: Int = 100): Unit = { + addData: (Seq[Int], Boolean) => StreamAction, + checkAnswer: Boolean, + iterations: Int): Unit = { implicit val intEncoder = ExpressionEncoder[Int]() var dataPos = 0 var running = true val actions = new ArrayBuffer[StreamAction]() - def addCheck() = { actions += CheckAnswer(1 to dataPos: _*) } + def addCheck() = { + if (checkAnswer) { + actions += CheckAnswer(1 to dataPos: _*) + } + } def addRandomData() = { val numItems = Random.nextInt(10) val data = dataPos until (dataPos + numItems) dataPos += numItems - actions += addData(data) + actions += addData(data, running) } (1 to iterations).foreach { i => From 59a93a561235d4bb0db04b3be6c0325e695df7e2 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Tue, 27 Sep 2016 17:34:22 -0700 Subject: [PATCH 15/28] Workaround for KAFKA-1894 and address some commments --- .../spark/util/UninterruptibleThread.scala | 2 +- external/kafka-0-10-sql/pom.xml | 2 +- .../spark/sql/kafka010/KafkaSource.scala | 162 +++++++++--------- .../sql/kafka010/KafkaSourceOffset.scala | 54 ++++++ .../sql/kafka010/KafkaSourceProvider.scala | 3 +- .../spark/sql/kafka010/KafkaSourceRDD.scala | 27 +-- .../spark/sql/kafka010/KafkaSourceSuite.scala | 14 ++ .../execution/streaming/StreamExecution.scala | 7 +- .../spark/sql/streaming/StreamTest.scala | 14 +- 9 files changed, 174 insertions(+), 111 deletions(-) create mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala diff --git a/core/src/main/scala/org/apache/spark/util/UninterruptibleThread.scala b/core/src/main/scala/org/apache/spark/util/UninterruptibleThread.scala index 4dcf95177aa7..3b262b89562d 100644 --- a/core/src/main/scala/org/apache/spark/util/UninterruptibleThread.scala +++ b/core/src/main/scala/org/apache/spark/util/UninterruptibleThread.scala @@ -93,7 +93,7 @@ private[spark] class UninterruptibleThread(name: String) extends Thread(name) { * Tests whether `interrupt()` has been called. */ override def isInterrupted: Boolean = { - super.isInterrupted || uninterruptibleLock.synchronized { shouldInterruptThread } + super.isInterrupted } /** diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index b7ead0ebfa55..b96445a11f85 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -31,7 +31,7 @@ sql-kafka-0-10 jar - Spark Integration for Kafka 0.10 + Kafka 0.10 Source for Structured Streaming http://spark.apache.org/ diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index 4c6ec4570a95..770b9e63a5d2 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -33,6 +33,7 @@ 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 @@ -71,23 +72,41 @@ import org.apache.spark.sql.types._ * * 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]) + sourceOptions: Map[String, String], + autoOffsetResetValue: 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") + + /** + * A KafkaConsumer used in the driver to query the latest Kafka offsets. This only queries the + * offsets and never commits them. + */ + private val consumer = consumerStrategy.createConsumer() + + /** Lazy set initialPartitionOffsets to only call `KafkaConsumer.poll` in StreamExecutionThread */ + private lazy val initialPartitionOffsets = { + val offsets = fetchPartitionOffsets(seekToLatest = false) + logInfo(s"Initial offsets: $offsets") + offsets + } override def schema: StructType = KafkaSource.kafkaSchema /** Returns the maximum available offset for this source. */ override def getOffset: Option[Offset] = { + // Make sure initialPartitionOffsets is set + initialPartitionOffsets + val offset = KafkaSourceOffset(fetchPartitionOffsets(seekToLatest = true)) logDebug(s"GetOffset: ${offset.partitionToOffsets.toSeq.map(_.toString).sorted}") Some(offset) @@ -95,6 +114,9 @@ private[kafka010] case class KafkaSource( /** Returns the data that is between the offsets [`start`, `end`), i.e. end is exclusive. */ override def getBatch(start: Option[Offset], end: Offset): DataFrame = { + // Make sure initialPartitionOffsets is set + initialPartitionOffsets + logInfo(s"GetBatch called with start = $start, end = $end") val untilPartitionOffsets = KafkaSourceOffset.getPartitionOffsets(end) val fromPartitionOffsets = start match { @@ -157,7 +179,7 @@ private[kafka010] case class KafkaSource( } val untilOffset = untilPartitionOffsets(tp) val preferredLoc = if (numExecutors > 0) { - Some(sortedExecutors(positiveMod(tp.hashCode, numExecutors))) + Some(sortedExecutors(floorMod(tp.hashCode, numExecutors))) } else None KafkaSourceRDDOffsetRange(tp, fromOffset, untilOffset, preferredLoc) }.filter { range => @@ -173,8 +195,7 @@ private[kafka010] case class KafkaSource( // Create a RDD that reads from Kafka and get the (key, value) pair as byte arrays. val rdd = new KafkaSourceRDD( sc, executorKafkaParams, offsetRanges).map { cr => - Row(cr.checksum, cr.key, cr.offset, cr.partition, cr.serializedKeySize, - cr.serializedValueSize, cr.timestamp, cr.timestampType.id, cr.topic, cr.value) + Row(cr.key, cr.value, cr.topic, cr.partition, cr.offset, cr.timestamp, cr.timestampType.id) } logInfo("GetBatch generating RDD of offset range: " + @@ -194,10 +215,11 @@ private[kafka010] case class KafkaSource( * KafkaConsumer. */ private def fetchPartitionOffsets( - seekToLatest: Boolean): Map[TopicPartition, Long] = { - + seekToLatest: Boolean): Map[TopicPartition, Long] = withRetries { // Poll to get the latest assigned partitions logTrace("\tPolling") + // Make sure `KafkaConsumer.poll` won't be interrupted (KAFKA-1894) + assert(Thread.currentThread().isInstanceOf[StreamExecutionThread]) consumer.poll(0) val partitions = consumer.assignment() consumer.pause(partitions) @@ -216,10 +238,11 @@ private[kafka010] case class KafkaSource( /** Fetch the earliest offsets for newly discovered partitions */ private def fetchNewPartitionEarliestOffsets( - newPartitions: Seq[TopicPartition]): Map[TopicPartition, Long] = { - + newPartitions: Seq[TopicPartition]): Map[TopicPartition, Long] = withRetries { // Poll to get the latest assigned partitions logTrace("\tPolling") + // Make sure `KafkaConsumer.poll` won't be interrupted (KAFKA-1894) + assert(Thread.currentThread().isInstanceOf[StreamExecutionThread]) consumer.poll(0) val partitions = consumer.assignment() logDebug(s"\tPartitioned assigned to consumer: $partitions") @@ -239,34 +262,54 @@ private[kafka010] case class KafkaSource( * Helper function that does multiple retries on the a body of code that returns offsets. * Retries are needed to handle transient failures. For e.g. race conditions between getting * assignment and getting position while topics/partitions are deleted can cause NPEs. + * + * This method also makes sure `body` won't be interrupted to workaround a potential issue in + * `KafkaConsumer.poll`. (KAFKA-1894) */ - private def withRetries( - body: => Map[TopicPartition, Long]): Map[TopicPartition, Long] = synchronized { - - var result: Option[Map[TopicPartition, Long]] = None - var attempt = 1 - var lastException: Throwable = null - while (result.isEmpty && attempt <= MAX_OFFSET_FETCH_ATTEMPTS) { - try { - result = Some(body) - } catch { - case NonFatal(e) => - lastException = e - e.printStackTrace() - logWarning(s"Error in attempt $attempt getting Kafka offsets: ", e) - attempt += 1 - Thread.sleep(OFFSET_FETCH_ATTEMPT_INTERVAL_MS) + private def withRetries(body: => Map[TopicPartition, Long]): Map[TopicPartition, Long] = { + synchronized { + var result: Option[Map[TopicPartition, Long]] = None + var attempt = 1 + var lastException: Throwable = null + while (result.isEmpty && attempt <= MAX_OFFSET_FETCH_ATTEMPTS + && !Thread.currentThread().isInterrupted) { + Thread.currentThread match { + case ut: UninterruptibleThread => + // "KafkaConsumer.poll" may hang forever if the thread is interrupted (E.g., the query + // is stopped)(KAFKA-1894). Hence, we just make sure we don't interrupt it. + // + // If the broker addresses are wrong, or Kafka cluster is down, "KafkaConsumer.poll" may + // hang forever as well. This cannot be resolved in KafkaSource until Kafka fixes the + // issue. + ut.runUninterruptibly { + try { + result = Some(body) + } catch { + case NonFatal(e) => + lastException = e + logWarning(s"Error in attempt $attempt getting Kafka offsets: ", e) + attempt += 1 + Thread.sleep(OFFSET_FETCH_ATTEMPT_INTERVAL_MS) + } + } + case _ => + throw new IllegalStateException( + "Kafka APIs must be executed on a o.a.spark.util.UninterruptibleThread") + } } + if (Thread.interrupted()) { + throw new InterruptedException() + } + if (result.isEmpty) { + assert(attempt > MAX_OFFSET_FETCH_ATTEMPTS) + assert(lastException != null) + throw lastException + } + result.get } - if (result.isEmpty) { - assert(attempt > MAX_OFFSET_FETCH_ATTEMPTS) - assert(lastException != null) - throw lastException - } - result.get } - private def positiveMod(a: Long, b: Int): Int = ((a % b).toInt + b) % b + private def floorMod(a: Long, b: Int): Int = ((a % b).toInt + b) % b } @@ -277,16 +320,13 @@ private[kafka010] object KafkaSource { val OFFSET_FETCH_ATTEMPT_INTERVAL_MS = 10 def kafkaSchema: StructType = StructType(Seq( - StructField("checksum", LongType), StructField("key", BinaryType), - StructField("offset", LongType), + StructField("value", BinaryType), + StructField("topic", StringType), StructField("partition", IntegerType), - StructField("serializedKeySize", IntegerType), - StructField("serializedValueSize", IntegerType), + StructField("offset", LongType), StructField("timestamp", LongType), - StructField("timestampType", IntegerType), - StructField("topic", StringType), - StructField("value", BinaryType) + StructField("timestampType", IntegerType) )) sealed trait ConsumerStrategy { @@ -298,7 +338,6 @@ private[kafka010] object KafkaSource { override def createConsumer(): Consumer[Array[Byte], Array[Byte]] = { val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams) consumer.subscribe(topics.asJava) - consumer.poll(0) consumer } @@ -313,54 +352,21 @@ private[kafka010] object KafkaSource { consumer.subscribe( ju.regex.Pattern.compile(topicPattern), new NoOpConsumerRebalanceListener()) - consumer.poll(0) consumer } override def toString: String = s"SubscribePattern[$topicPattern]" } - def getSortedExecutorList(sc: SparkContext): Array[String] = { - def compare(a: ExecutorCacheTaskLocation, b: ExecutorCacheTaskLocation): Boolean = { - if (a.host == b.host) { a.executorId > b.executorId } else { a.host > b.host } - } - + private def getSortedExecutorList(sc: SparkContext): Array[String] = { val bm = sc.env.blockManager bm.master.getPeers(bm.blockManagerId).toArray .map(x => ExecutorCacheTaskLocation(x.host, x.executorId)) .sortWith(compare) .map(_.toString) } -} - -/** An [[Offset]] for the [[KafkaSource]]. */ -private[kafka010] -case class KafkaSourceOffset(partitionToOffsets: Map[TopicPartition, Long]) extends Offset { - override def toString(): String = { - partitionToOffsets.toSeq.sortBy(_._1.toString).mkString("[", ", ", "]") + private def compare(a: ExecutorCacheTaskLocation, b: ExecutorCacheTaskLocation): Boolean = { + if (a.host == b.host) { a.executorId > b.executorId } else { a.host > b.host } } } - -/** Companion object of the [[KafkaSourceOffset]] */ -private[kafka010] object KafkaSourceOffset { - - def getPartitionOffsets(offset: Offset): Map[TopicPartition, Long] = { - offset match { - case o: KafkaSourceOffset => o.partitionToOffsets - case _ => - throw new IllegalArgumentException( - s"Invalid conversion from offset of ${offset.getClass} to KafkaSourceOffset") - } - } - - /** - * Returns [[KafkaSourceOffset]] from a variable sequence of (topic, partitionId, offset) - * tuples. - */ - def apply(offsetTuples: (String, Int, Long)*): KafkaSourceOffset = { - KafkaSourceOffset(offsetTuples.map { case(t, p, o) => (new TopicPartition(t, p), o) }.toMap) - } -} - - diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala new file mode 100644 index 000000000000..b5ade982515f --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.kafka010 + +import org.apache.kafka.common.TopicPartition + +import org.apache.spark.sql.execution.streaming.Offset + +/** + * An [[Offset]] for the [[KafkaSource]]. This one tracks all partitions of subscribed topics and + * their offsets. + */ +private[kafka010] +case class KafkaSourceOffset(partitionToOffsets: Map[TopicPartition, Long]) extends Offset { + override def toString(): String = { + partitionToOffsets.toSeq.sortBy(_._1.toString).mkString("[", ", ", "]") + } +} + +/** Companion object of the [[KafkaSourceOffset]] */ +private[kafka010] object KafkaSourceOffset { + + def getPartitionOffsets(offset: Offset): Map[TopicPartition, Long] = { + offset match { + case o: KafkaSourceOffset => o.partitionToOffsets + case _ => + throw new IllegalArgumentException( + s"Invalid conversion from offset of ${offset.getClass} to KafkaSourceOffset") + } + } + + /** + * Returns [[KafkaSourceOffset]] from a variable sequence of (topic, partitionId, offset) + * tuples. + */ + def apply(offsetTuples: (String, Int, Long)*): KafkaSourceOffset = { + KafkaSourceOffset(offsetTuples.map { case(t, p, o) => (new TopicPartition(t, p), o) }.toMap) + } +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala index 9e10fbc9c35e..73e29ef2ef51 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala @@ -50,6 +50,7 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider 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) } @@ -133,7 +134,7 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider throw new IllegalArgumentException("Unknown option") } - new KafkaSource(sqlContext, strategy, kafkaParamsForExecutors, parameters) + new KafkaSource(sqlContext, strategy, kafkaParamsForExecutors, parameters, autoOffsetResetValue) } private def validateOptions(parameters: Map[String, String]): Unit = { diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala index 4778a0d8b1b5..62a3ffb81c3f 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala @@ -129,32 +129,11 @@ private[kafka010] class KafkaSourceRDD( logDebug(s"Creating iterator for $range") new Iterator[ConsumerRecord[Array[Byte], Array[Byte]]]() { - - private var prefetch: ConsumerRecord[Array[Byte], Array[Byte]] = _ - - private def fetchNext(): ConsumerRecord[Array[Byte], Array[Byte]] = { - try { - val r = consumer.get(requestOffset) - requestOffset += 1 - r - } catch { - case e: OffsetOutOfRangeException => - logWarning(s"${range.topicPartition} was deleted, some data may have been missed") - null - } - } - - override def hasNext(): Boolean = { - if (prefetch == null && requestOffset < range.untilOffset) { - prefetch = fetchNext() - } - prefetch != null - } - + override def hasNext(): Boolean = requestOffset < range.untilOffset override def next(): ConsumerRecord[Array[Byte], Array[Byte]] = { assert(hasNext(), "Can't call next() once untilOffset has been reached") - val r = prefetch - prefetch = null + val r = consumer.get(requestOffset) + requestOffset += 1 r } } diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala index 2458e0397cac..b65314b9873d 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala @@ -62,6 +62,10 @@ abstract class KafkaSourceTest extends StreamTest with SharedSQLContext { topicAction: (String, Option[Int]) => Unit = (_, _) => {}) extends AddData { override def addData(query: Option[StreamExecution]): (Source, Offset) = { + if (query.get.isActive) { + query.get.processAllAvailable() + } + val existingTopics = testUtils.getAllTopicsAndPartitionSize().toMap val newTopics = topics.diff(existingTopics.keySet) for (newTopic <- newTopics) { @@ -181,6 +185,10 @@ class KafkaSourceSuite extends KafkaSourceTest { val mapped = kafka.map(kv => new String(kv._2).toInt + 1) testStream(mapped)( + AssertOnQuery { q => + q.processAllAvailable() + true + }, AddKafkaData(Set(topic), 1, 2, 3), CheckAnswer(2, 3, 4), Assert { @@ -262,6 +270,12 @@ class KafkaSourceSuite extends KafkaSourceTest { val mapped = kafka.map(kv => new String(kv._2).toInt + 1) testStream(mapped)( + AssertOnQuery { q => + // Because KafkaSource's initialPartitionOffsets is set lazily, we need to make sure + // its "getOffset" is called before pushing any data. + q.processAllAvailable() + true + }, AddKafkaData(Set(topic), 1, 2, 3), CheckAnswer(2, 3, 4), StopStream, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 9825f19b86a5..4c5f8a9dc923 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -116,7 +116,7 @@ class StreamExecution( * [[HDFSMetadataLog]]. See SPARK-14131 for more details. */ val microBatchThread = - new UninterruptibleThread(s"stream execution thread for $name") { + new StreamExecutionThread(s"stream execution thread for $name") { override def run(): Unit = { // To fix call site like "run at :0", we bridge the call site from the caller // thread to this micro batch thread @@ -530,3 +530,8 @@ object StreamExecution { def nextId: Long = _nextId.getAndIncrement() } + +/** + * A special thread to run the stream query. + */ +abstract class StreamExecutionThread(name: String) extends UninterruptibleThread(name) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index d7c00c308064..b74e76caeb44 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -50,11 +50,11 @@ import org.apache.spark.util.{Clock, ManualClock, SystemClock, Utils} * * {{{ * val inputData = MemoryStream[Int] - * val mapped = inputData.toDS().map(_ + 1) - ** - *testStream(mapped)( - *AddData(inputData, 1, 2, 3), - *CheckAnswer(2, 3, 4)) + * val mapped = inputData.toDS().map(_ + 1) + * + * testStream(mapped)( + * AddData(inputData, 1, 2, 3), + * CheckAnswer(2, 3, 4)) * }}} * * Note that while we do sleep to allow the other thread to progress without spinning, @@ -503,6 +503,10 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { var dataPos = 0 var running = true val actions = new ArrayBuffer[StreamAction]() + actions += AssertOnQuery { q => + q.processAllAvailable() + true + } def addCheck() = { if (checkAnswer) { From 9d95d52cd1cb9ef83efaaadc5d4e9a5dc3e1c843 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Wed, 28 Sep 2016 11:26:25 -0700 Subject: [PATCH 16/28] Address more --- .../spark/util/UninterruptibleThread.scala | 7 -- .../spark/sql/kafka010/KafkaSource.scala | 87 ++++++++++++------- .../sql/kafka010/KafkaSourceProvider.scala | 26 ++++-- .../spark/sql/kafka010/KafkaSourceRDD.scala | 11 ++- .../spark/sql/kafka010/KafkaSourceSuite.scala | 35 +++----- .../spark/sql/streaming/StreamTest.scala | 17 ++-- 6 files changed, 101 insertions(+), 82 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/UninterruptibleThread.scala b/core/src/main/scala/org/apache/spark/util/UninterruptibleThread.scala index 3b262b89562d..f0b68f0cb7e2 100644 --- a/core/src/main/scala/org/apache/spark/util/UninterruptibleThread.scala +++ b/core/src/main/scala/org/apache/spark/util/UninterruptibleThread.scala @@ -89,13 +89,6 @@ private[spark] class UninterruptibleThread(name: String) extends Thread(name) { } } - /** - * Tests whether `interrupt()` has been called. - */ - override def isInterrupted: Boolean = { - super.isInterrupted - } - /** * Interrupt `this` thread if possible. If `this` is in the uninterruptible status, it won't be * interrupted until it enters into the interruptible status. diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index 770b9e63a5d2..34b969388484 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -82,7 +82,7 @@ private[kafka010] case class KafkaSource( consumerStrategy: ConsumerStrategy, executorKafkaParams: ju.Map[String, Object], sourceOptions: Map[String, String], - autoOffsetResetValue: String) + failOnCorruptMetadata: Boolean) extends Source with Logging { private val sc = sqlContext.sparkContext @@ -93,9 +93,13 @@ private[kafka010] case class KafkaSource( */ private val consumer = consumerStrategy.createConsumer() - /** Lazy set initialPartitionOffsets to only call `KafkaConsumer.poll` in StreamExecutionThread */ + /** + * Lazy set initialPartitionOffsets to make sure only call `KafkaConsumer.poll` in + * StreamExecutionThread. Otherwise, interrupting a thread running `KafkaConsumer.poll` may hang + * forever (KAFKA-1894). + */ private lazy val initialPartitionOffsets = { - val offsets = fetchPartitionOffsets(seekToLatest = false) + val offsets = fetchPartitionOffsets(seekToEnd = false) logInfo(s"Initial offsets: $offsets") offsets } @@ -107,7 +111,7 @@ private[kafka010] case class KafkaSource( // Make sure initialPartitionOffsets is set initialPartitionOffsets - val offset = KafkaSourceOffset(fetchPartitionOffsets(seekToLatest = true)) + val offset = KafkaSourceOffset(fetchPartitionOffsets(seekToEnd = true)) logDebug(s"GetOffset: ${offset.partitionToOffsets.toSeq.map(_.toString).sorted}") Some(offset) } @@ -136,30 +140,23 @@ private[kafka010] case class KafkaSource( if (newPartitionOffsets.keySet != newPartitions) { // We cannot get from offsets for some partitions. It means they got deleted. val deletedPartitions = newPartitions.diff(newPartitionOffsets.keySet) - logWarning(s"Partitions removed: ${deletedPartitions}, some data may have been missed") + reportCorruptMetadata( + s"Cannot find earliest offsets of ${deletedPartitions}. Some data may have been missed") } logInfo(s"Partitions added: $newPartitionOffsets") newPartitionOffsets.filter(_._2 != 0).foreach { case (p, o) => - logWarning(s"Added partition $p starts from $o instead of 0, some data may have been missed") + reportCorruptMetadata( + s"Added partition $p starts from $o instead of 0. Some data may have been missed") } val deletedPartitions = fromPartitionOffsets.keySet.diff(untilPartitionOffsets.keySet) - logWarning(s"Partitions removed: $deletedPartitions, some data may have been missed") - - // Sort the partitions and current list of executors to consistently assign each partition - // to the executor. This allows cached KafkaConsumers in the executors to be re-used to - // read the same partition in every batch. - val topicPartitionOrdering = new Ordering[TopicPartition] { - override def compare(l: TopicPartition, r: TopicPartition): Int = { - implicitly[Ordering[(String, Long)]].compare( - (l.topic, l.partition), - (r.topic, r.partition)) - } - } + // TODO should this one ever throw an exception? + reportCorruptMetadata(s"$deletedPartitions are removed. Some data may have been missed") // Use the until partitions to calculate offset ranges to ignore partitions that have // been deleted val sortedTopicPartitions = untilPartitionOffsets.keySet.filter { tp => + // Ignore partitions that we don't know the from offsets. newPartitionOffsets.contains(tp) || fromPartitionOffsets.contains(tp) }.toSeq.sorted(topicPartitionOrdering) logDebug("Sorted topicPartitions: " + sortedTopicPartitions.mkString(", ")) @@ -211,38 +208,38 @@ private[kafka010] case class KafkaSource( override def toString(): String = s"KafkaSource[$consumerStrategy]" /** - * Fetch the offset of a partition, either the latest offsets or the current offsets in the - * KafkaConsumer. + * Fetch the offset of a partition, either seek to the latest offsets or use the current offsets + * in the consumer. */ private def fetchPartitionOffsets( - seekToLatest: Boolean): Map[TopicPartition, Long] = withRetries { - // Poll to get the latest assigned partitions - logTrace("\tPolling") + seekToEnd: Boolean): Map[TopicPartition, Long] = withRetriesWithoutInterrupt { // Make sure `KafkaConsumer.poll` won't be interrupted (KAFKA-1894) assert(Thread.currentThread().isInstanceOf[StreamExecutionThread]) + // Poll to get the latest assigned partitions consumer.poll(0) val partitions = consumer.assignment() consumer.pause(partitions) - logDebug(s"\tPartitioned assigned to consumer: $partitions") + logDebug(s"Partitioned assigned to consumer: $partitions") // Get the current or latest offset of each partition - if (seekToLatest) { + if (seekToEnd) { consumer.seekToEnd(partitions) - logDebug("\tSeeked to the end") + logDebug("Seeked to the end") } - logTrace("Getting positions") val partitionOffsets = partitions.asScala.map(p => p -> consumer.position(p)).toMap logDebug(s"Got offsets for partition : $partitionOffsets") partitionOffsets } - /** Fetch the earliest offsets for newly discovered partitions */ + /** + * Fetch the earliest offsets for newly discovered partitions. The return results may not contain + * some partitions if they are deleted. + */ private def fetchNewPartitionEarliestOffsets( - newPartitions: Seq[TopicPartition]): Map[TopicPartition, Long] = withRetries { - // Poll to get the latest assigned partitions - logTrace("\tPolling") + newPartitions: Seq[TopicPartition]): Map[TopicPartition, Long] = withRetriesWithoutInterrupt { // Make sure `KafkaConsumer.poll` won't be interrupted (KAFKA-1894) assert(Thread.currentThread().isInstanceOf[StreamExecutionThread]) + // Poll to get the latest assigned partitions consumer.poll(0) val partitions = consumer.assignment() logDebug(s"\tPartitioned assigned to consumer: $partitions") @@ -266,7 +263,8 @@ private[kafka010] case class KafkaSource( * This method also makes sure `body` won't be interrupted to workaround a potential issue in * `KafkaConsumer.poll`. (KAFKA-1894) */ - private def withRetries(body: => Map[TopicPartition, Long]): Map[TopicPartition, Long] = { + private def withRetriesWithoutInterrupt( + body: => Map[TopicPartition, Long]): Map[TopicPartition, Long] = { synchronized { var result: Option[Map[TopicPartition, Long]] = None var attempt = 1 @@ -309,7 +307,17 @@ private[kafka010] case class KafkaSource( } } - private def floorMod(a: Long, b: Int): Int = ((a % b).toInt + b) % b + /** + * If `failOnCorruptMetadata` is true, this method will throw an `IllegalStateException`. + * Otherwise, just log a warning. + */ + private def reportCorruptMetadata(message: String): Unit = { + if (failOnCorruptMetadata) { + throw new IllegalStateException(message) + } else { + logWarning(message) + } + } } @@ -369,4 +377,17 @@ private[kafka010] object KafkaSource { private def compare(a: ExecutorCacheTaskLocation, b: ExecutorCacheTaskLocation): Boolean = { if (a.host == b.host) { a.executorId > b.executorId } else { a.host > b.host } } + + // Sort the partitions and current list of executors to consistently assign each partition + // to the executor. This allows cached KafkaConsumers in the executors to be re-used to + // read the same partition in every batch. + private val topicPartitionOrdering = new Ordering[TopicPartition] { + override def compare(l: TopicPartition, r: TopicPartition): Int = { + implicitly[Ordering[(String, Long)]].compare( + (l.topic, l.partition), + (r.topic, r.partition)) + } + } + + private def floorMod(a: Long, b: Int): Int = ((a % b).toInt + b) % b } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala index 73e29ef2ef51..f92a7132ae01 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala @@ -38,7 +38,8 @@ import org.apache.spark.sql.types.StructType * missing options even before the query is started. */ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider - with DataSourceRegister with Logging { + with DataSourceRegister with Logging { + import KafkaSourceProvider._ /** @@ -105,13 +106,13 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider .set(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, deserClassName) .set(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, deserClassName) - // So that consumers in executors never throw NoOffsetForPartitionException + // Make sure executors do only what the driver tells them. .set(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none") // So that consumers in executors do not mess with any existing group id .set(ConsumerConfig.GROUP_ID_CONFIG, s"$uniqueGroupId-executor") - // So that consumers in executors does not commit offsets unnecessaribly + // So that consumers in executors does not commit offsets unnecessarily .set(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") // If buffer config is not set, set it to reasonable value to work around @@ -134,7 +135,15 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider throw new IllegalArgumentException("Unknown option") } - new KafkaSource(sqlContext, strategy, kafkaParamsForExecutors, parameters, autoOffsetResetValue) + val failOnCorruptMetadata = + caseInsensitiveParams.getOrElse(FAIL_ON_CORRUPT_METADATA_OPTION_KEY, "false").toBoolean + + new KafkaSource( + sqlContext, + strategy, + kafkaParamsForExecutors, + parameters, + failOnCorruptMetadata) } private def validateOptions(parameters: Map[String, String]): Unit = { @@ -196,8 +205,12 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider s""" |Kafka option '${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG}' is not supported. |Instead set the source option '$STARTING_OFFSET_OPTION_KEY' to 'earliest' or 'latest' to - |specify where to start. This will ensure that no data is missed when when new - |topics/partitions are dynamically subscribed. See the docs for more details. + |specify where to start. Structured Streaming manages which offsets are consumed + |internally, rather than rely on the kafka Consumer to do it. This will ensure that no + |data is missed when when new topics/partitions are dynamically subscribed. Note that + |'$STARTING_OFFSET_OPTION_KEY' only applies when a new Streaming query is started, and + |that resuming will always pick up from where the query left off. See the docs for more + |details. """.stripMargin) } @@ -261,4 +274,5 @@ private[kafka010] object KafkaSourceProvider { private val STRATEGY_OPTION_KEYS = Set("subscribe", "subscribepattern") private val STARTING_OFFSET_OPTION_KEY = "startingoffset" private val STARTING_OFFSET_OPTION_VALUES = Set("earliest", "latest") + private val FAIL_ON_CORRUPT_METADATA_OPTION_KEY = "failoncorruptmetadata" } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala index 62a3ffb81c3f..b7af9c8a152e 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala @@ -21,10 +21,11 @@ import java.{util => ju} import scala.collection.mutable.ArrayBuffer -import org.apache.kafka.clients.consumer.{ConsumerRecord, OffsetOutOfRangeException} +import org.apache.kafka.clients.consumer.ConsumerRecord import org.apache.kafka.common.TopicPartition import org.apache.spark.{Partition, SparkContext, TaskContext} +import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel @@ -52,8 +53,7 @@ private[kafka010] case class KafkaSourceRDDPartition( * 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 sc the [[SparkContext]] * @param executorKafkaParams Kafka configuration for creating KafkaConsumer on the executors * @param offsetRanges Offset ranges that define the Kafka data belonging to this RDD */ @@ -75,6 +75,11 @@ private[kafka010] class KafkaSourceRDD( override def count(): Long = offsetRanges.map(_.size).sum + override def countApprox(timeout: Long, confidence: Double): PartialResult[BoundedDouble] = { + val c = count + new PartialResult(new BoundedDouble(c, 1.0, c, c), true) + } + override def isEmpty(): Boolean = count == 0L override def take(num: Int): Array[ConsumerRecord[Array[Byte], Array[Byte]]] = { diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala index b65314b9873d..7bae42e21271 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala @@ -50,6 +50,14 @@ abstract class KafkaSourceTest extends StreamTest with SharedSQLContext { } } + 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. * @@ -63,6 +71,7 @@ abstract class KafkaSourceTest extends StreamTest with SharedSQLContext { 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() } @@ -185,10 +194,7 @@ class KafkaSourceSuite extends KafkaSourceTest { val mapped = kafka.map(kv => new String(kv._2).toInt + 1) testStream(mapped)( - AssertOnQuery { q => - q.processAllAvailable() - true - }, + makeSureGetOffsetCalled, AddKafkaData(Set(topic), 1, 2, 3), CheckAnswer(2, 3, 4), Assert { @@ -270,12 +276,7 @@ class KafkaSourceSuite extends KafkaSourceTest { val mapped = kafka.map(kv => new String(kv._2).toInt + 1) testStream(mapped)( - AssertOnQuery { q => - // Because KafkaSource's initialPartitionOffsets is set lazily, we need to make sure - // its "getOffset" is called before pushing any data. - q.processAllAvailable() - true - }, + makeSureGetOffsetCalled, AddKafkaData(Set(topic), 1, 2, 3), CheckAnswer(2, 3, 4), StopStream, @@ -352,7 +353,7 @@ class KafkaSourceStressSuite extends KafkaSourceTest with BeforeAndAfter { } } - private def stressTest(checkAnswer: Boolean): Unit = { + test("stress test with multiple topics and partitions") { topics.foreach { topic => testUtils.createTopic(topic, partitions = nextInt(1, 6)) testUtils.sendMessages(topic, (101 to 105).map { _.toString }.toArray) @@ -373,6 +374,7 @@ class KafkaSourceStressSuite extends KafkaSourceTest with BeforeAndAfter { runStressTest( mapped, + Seq(makeSureGetOffsetCalled), (d, running) => { Random.nextInt(5) match { case 0 => // Add a new topic @@ -383,7 +385,7 @@ class KafkaSourceStressSuite extends KafkaSourceTest with BeforeAndAfter { testUtils.createTopic(topic, partitions = nextInt(1, 6)) } }) - case 1 if !checkAnswer || running => + case 1 if running => // Only delete a topic when the query is running. Otherwise, we may lost data and // cannot check the correctness. val deletedTopic = topics(Random.nextInt(topics.size)) @@ -406,15 +408,6 @@ class KafkaSourceStressSuite extends KafkaSourceTest with BeforeAndAfter { AddKafkaData(topics.toSet, d: _*) } }, - checkAnswer = checkAnswer, iterations = 50) } - - test("stress test with multiple topics and partitions") { - stressTest(checkAnswer = true) - } - - test("don't crash when adding and deleting partitions concurrently") { - stressTest(checkAnswer = false) - } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index b74e76caeb44..46b00b5a884c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -482,37 +482,30 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { ds: Dataset[Int], addData: Seq[Int] => StreamAction, iterations: Int = 100): Unit = { - runStressTest(ds, (data, running) => addData(data), true, iterations) + 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 preparedActions actions need to run before starting the stress test. * @param addData an add data action that adds the given numbers to the stream, encoding them * as needed - * @param checkAnswer should add `CheckAnswer` to the test. * @param iterations the iteration number */ def runStressTest( ds: Dataset[Int], + preparedActions: Seq[StreamAction], addData: (Seq[Int], Boolean) => StreamAction, - checkAnswer: Boolean, iterations: Int): Unit = { implicit val intEncoder = ExpressionEncoder[Int]() var dataPos = 0 var running = true val actions = new ArrayBuffer[StreamAction]() - actions += AssertOnQuery { q => - q.processAllAvailable() - true - } + actions ++= preparedActions - def addCheck() = { - if (checkAnswer) { - actions += CheckAnswer(1 to dataPos: _*) - } - } + def addCheck() = { actions += CheckAnswer(1 to dataPos: _*) } def addRandomData() = { val numItems = Random.nextInt(10) From e88306267e372101a685492afcfa652408a83109 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Mon, 3 Oct 2016 10:42:51 -0700 Subject: [PATCH 17/28] Address more --- .../scala/org/apache/spark/sql/kafka010/KafkaSource.scala | 4 ++-- .../org/apache/spark/sql/kafka010/KafkaSourceProvider.scala | 5 ++++- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index 34b969388484..fb4224616f63 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -181,8 +181,8 @@ private[kafka010] case class KafkaSource( KafkaSourceRDDOffsetRange(tp, fromOffset, untilOffset, preferredLoc) }.filter { range => if (range.untilOffset < range.fromOffset) { - logWarning(s"Partition ${range.topicPartition} was deleted and then added, " + - "some data may have been missed") + reportCorruptMetadata(s"Partition ${range.topicPartition}'s offset was changed from " + + s"${range.fromOffset} to ${range.untilOffset}, some data may have been missed") false } else { true diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala index f92a7132ae01..afedf39ff631 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala @@ -72,6 +72,9 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider .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 { @@ -90,7 +93,7 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider // 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 + // 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 From d1545329673e889e93b0e4859248cebff8b488c1 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Mon, 3 Oct 2016 12:06:43 -0700 Subject: [PATCH 18/28] Add programming guide --- .../structured-streaming-kafka-integration.md | 185 ++++++++++++++++++ .../structured-streaming-programming-guide.md | 6 + 2 files changed, 191 insertions(+) create mode 100644 docs/structured-streaming-kafka-integration.md diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md new file mode 100644 index 000000000000..e12c1afe3a8d --- /dev/null +++ b/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
keybinary
valuebinary
topicstring
partitionint
offsetlong
timestamplong
timestampTypeint
+ +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" which is from the earliest offset, + or "latest" which is just from the latest offset. Note: This only applies when a new Streaming q + uery is started, and that resuming will always pick up from where the query left off.
failOnCorruptMetadata[true, false]trueWhether to fail the query when metadata is corrupt (e.g., topics are deleted), which can cause + data lost.
subscribeA comma-separated list of topics(none)The topic list to subscribe. Only one of "subscribe" and "subscribeParttern" options can be + specified for Kafka source.
subscribePartternJava regex string(none)The pattern used to subscribe the topic. Only one of "subscribe" and "subscribeParttern" + options can be specified for Kafka source.
+ +Kafka's own configurations can be set via `DataStreamReader.option` with `kafka.` prefix, e.g, +`stream.option("kafka.bootstrap.servers", "host:port")`. For possible kafkaParams, see +[Kafka consumer config docs](http://kafka.apache.org/documentation.html#newconsumerconfigs). + +### Deploying + +As with any Spark applications, `spark-submit` is used to launch your application. `spark-sql-kafka-0-10_{{site.SCALA_BINARY_VERSION}}` +and its dependencies can be directly added to `spark-submit` using `--packages`, such as, + + ./bin/spark-submit --packages org.apache.spark:spark-sql-kafka-0-10_{{site.SCALA_BINARY_VERSION}}:{{site.SPARK_VERSION_SHORT}} ... + +See [Application Submission Guide](submitting-applications.html) for more details about submitting +applications with external dependencies. diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md index c7ed3b04bced..8b8f776a4b6c 100644 --- a/docs/structured-streaming-programming-guide.md +++ b/docs/structured-streaming-programming-guide.md @@ -1126,6 +1126,12 @@ aggDF \ +## Advanced Sources + +Structured Streaming supports the following advanced sources: + +- **Kafka:** Structured Streaming {{site.SPARK_VERSION_SHORT}} is compatible with Kafka broker versions 0.10.0 or higher. See the [Kafka Integration Guide](structured-streaming-kafka-integration.html) for more details. + # Where to go from here - Examples: See and run the [Scala]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples/sql/streaming)/[Java]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/java/org/apache/spark/examples/sql/streaming)/[Python]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/python/sql/streaming) From 77208d1611810f5c6afb5ba63911cadb1794c863 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Mon, 3 Oct 2016 14:27:05 -0700 Subject: [PATCH 19/28] Adddress; Fix a corner case when the first batch fails; set failOnCorruptMetadata's default value to true --- .../structured-streaming-kafka-integration.md | 25 +++++++++++++++---- .../spark/sql/kafka010/KafkaSource.scala | 14 +++++++---- .../sql/kafka010/KafkaSourceProvider.scala | 15 +++++------ .../spark/sql/kafka010/KafkaSourceSuite.scala | 1 + 4 files changed, 38 insertions(+), 17 deletions(-) diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md index e12c1afe3a8d..f8a1d1d676a5 100644 --- a/docs/structured-streaming-kafka-integration.md +++ b/docs/structured-streaming-kafka-integration.md @@ -151,21 +151,21 @@ Right now, the Kafka source has the following Spark's specific options. failOnCorruptMetadata [true, false] true - Whether to fail the query when metadata is corrupt (e.g., topics are deleted), which can cause - data lost. + Whether to fail the query when metadata is corrupt (e.g., topics are deleted, or offsets are + out of range), which may lost data. subscribe A comma-separated list of topics (none) - The topic list to subscribe. Only one of "subscribe" and "subscribeParttern" options can be + The topic list to subscribe. Only one of "subscribe" and "subscribePattern" options can be specified for Kafka source. - subscribeParttern + subscribePattern Java regex string (none) - The pattern used to subscribe the topic. Only one of "subscribe" and "subscribeParttern" + The pattern used to subscribe the topic. Only one of "subscribe" and "subscribePattern" options can be specified for Kafka source. @@ -174,6 +174,21 @@ Kafka's own configurations can be set via `DataStreamReader.option` with `kafka. `stream.option("kafka.bootstrap.servers", "host:port")`. For possible kafkaParams, see [Kafka consumer config docs](http://kafka.apache.org/documentation.html#newconsumerconfigs). +Note that the following Kafka params cannot be set and the Kafka source will throw an exception: +- **group.id**: Kafka source will create a unique group id for each query automatically. +- **auto.offset.reset**: Set the source option `startingOffset` to `earliest` or `latest` to specify + where to start instead. Structured Streaming manages which offsets are consumed internally, rather + than rely on the kafka Consumer to do it. This will ensure that no data is missed when when new + topics/partitions are dynamically subscribed. Note that `startingOffset` only applies when a new + Streaming query is started, and that resuming will always pick up from where the query left off. +- **key.deserializer**: Keys are always deserialized as byte arrays with ByteArrayDeserializer. Use + Dataframe operations to explicitly deserialize the keys. +- **value.deserializer**: Values are always deserialized as byte arrays with ByteArrayDeserializer. + Use Dataframe operations to explicitly deserialize the values. +- **enable.auto.commit**: Kafka source doesn't commit any offset. +- **interceptor.classes**: Kafka source always read keys and values as byte arrays. It's not safe to + use ConsumerInterceptor as it may break the query. + ### Deploying As with any Spark applications, `spark-submit` is used to launch your application. `spark-sql-kafka-0-10_{{site.SCALA_BINARY_VERSION}}` diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index fb4224616f63..f05ff605ba5e 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -82,6 +82,7 @@ private[kafka010] case class KafkaSource( consumerStrategy: ConsumerStrategy, executorKafkaParams: ju.Map[String, Object], sourceOptions: Map[String, String], + metadataPath: String, failOnCorruptMetadata: Boolean) extends Source with Logging { @@ -99,9 +100,13 @@ private[kafka010] case class KafkaSource( * forever (KAFKA-1894). */ private lazy val initialPartitionOffsets = { - val offsets = fetchPartitionOffsets(seekToEnd = false) - logInfo(s"Initial offsets: $offsets") - offsets + val metadataLog = new HDFSMetadataLog[KafkaSourceOffset](sqlContext.sparkSession, metadataPath) + metadataLog.get(0).getOrElse { + val offsets = KafkaSourceOffset(fetchPartitionOffsets(seekToEnd = false)) + metadataLog.add(0, offsets) + logInfo(s"Initial offsets: $offsets") + offsets + }.partitionToOffsets } override def schema: StructType = KafkaSource.kafkaSchema @@ -150,8 +155,7 @@ private[kafka010] case class KafkaSource( } val deletedPartitions = fromPartitionOffsets.keySet.diff(untilPartitionOffsets.keySet) - // TODO should this one ever throw an exception? - reportCorruptMetadata(s"$deletedPartitions are removed. Some data may have been missed") + logWarning(s"$deletedPartitions are gone. Some data may have been missed") // Use the until partitions to calculate offset ranges to ignore partitions that have // been deleted diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala index afedf39ff631..729108064dd1 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala @@ -139,13 +139,14 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider } val failOnCorruptMetadata = - caseInsensitiveParams.getOrElse(FAIL_ON_CORRUPT_METADATA_OPTION_KEY, "false").toBoolean + caseInsensitiveParams.getOrElse(FAIL_ON_CORRUPT_METADATA_OPTION_KEY, "true").toBoolean new KafkaSource( sqlContext, strategy, kafkaParamsForExecutors, parameters, + metadataPath, failOnCorruptMetadata) } @@ -219,17 +220,17 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG}")) { throw new IllegalArgumentException( - s"Kafka option '${ConsumerConfig.GROUP_ID_CONFIG}' is not supported as keys are " + - s"deserialized as byte arrays with ByteArrayDeserializer. Use Dataframe operations to " + - s"explicitly deserialize the keys.") + s"Kafka option '${ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG}' is not supported as keys " + + "are deserialized as byte arrays with ByteArrayDeserializer. Use Dataframe operations " + + "to explicitly deserialize the keys.") } if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG}")) { throw new IllegalArgumentException( - s"Kafka option '${ConsumerConfig.GROUP_ID_CONFIG}' is not supported as value are " + - s"deserialized as byte arrays with ByteArrayDeserializer. Use Dataframe operations to " + - s"explicitly deserialize the values.") + s"Kafka option '${ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG}' is not supported as " + + "value are deserialized as byte arrays with ByteArrayDeserializer. Use Dataframe " + + "operations to explicitly deserialize the values.") } val otherUnsupportedConfigs = Seq( diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala index 7bae42e21271..bfa7b60258a0 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala @@ -366,6 +366,7 @@ class KafkaSourceStressSuite extends KafkaSourceTest with BeforeAndAfter { .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.metadata.max.age.ms", "1") .option("subscribePattern", "stress.*") + .option("failOnCorruptMetadata", "false") .load() .select("key", "value") .as[(Array[Byte], Array[Byte])] From ccadd81d08f56e45bbe5970656960578ee291bb5 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Mon, 3 Oct 2016 14:53:28 -0700 Subject: [PATCH 20/28] Add example about how to convert key and value to strings --- .../structured-streaming-kafka-integration.md | 30 +++++++++++++------ .../spark/sql/kafka010/KafkaSourceSuite.scala | 30 ++++++++++++------- 2 files changed, 40 insertions(+), 20 deletions(-) diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md index f8a1d1d676a5..6c0f428613d7 100644 --- a/docs/structured-streaming-kafka-integration.md +++ b/docs/structured-streaming-kafka-integration.md @@ -22,82 +22,94 @@ application. See the [Deploying](#deploying) subsection below.
// Subscribe to 1 topic - spark + 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 - spark + 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 - spark + 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 - spark + 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 - spark + 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 - spark + 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 - spark + 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 - spark + 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 - spark + 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)")
diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala index bfa7b60258a0..1c92d40619fe 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala @@ -145,8 +145,10 @@ class KafkaSourceSuite extends KafkaSourceTest { .option("kafka.metadata.max.age.ms", "1") .option("subscribePattern", s"topic-.*") - val kafka = reader.load().select("key", "value").as[(Array[Byte], Array[Byte])] - val mapped = kafka.map(kv => new String(kv._2).toInt + 1) + val kafka = reader.load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + val mapped = kafka.map(kv => kv._2.toInt + 1) testStream(mapped)( StopStream @@ -190,8 +192,10 @@ class KafkaSourceSuite extends KafkaSourceTest { .option("kafka.metadata.max.age.ms", "1") .option("subscribePattern", s"$topicPrefix-.*") - val kafka = reader.load().select("key", "value").as[(Array[Byte], Array[Byte])] - val mapped = kafka.map(kv => new String(kv._2).toInt + 1) + val kafka = reader.load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + val mapped = kafka.map(kv => kv._2.toInt + 1) testStream(mapped)( makeSureGetOffsetCalled, @@ -272,8 +276,10 @@ class KafkaSourceSuite extends KafkaSourceTest { .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.metadata.max.age.ms", "1") options.foreach { case (k, v) => reader.option(k, v) } - val kafka = reader.load().select("key", "value").as[(Array[Byte], Array[Byte])] - val mapped = kafka.map(kv => new String(kv._2).toInt + 1) + val kafka = reader.load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + val mapped = kafka.map(kv => kv._2.toInt + 1) testStream(mapped)( makeSureGetOffsetCalled, @@ -309,8 +315,10 @@ class KafkaSourceSuite extends KafkaSourceTest { .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.metadata.max.age.ms", "1") options.foreach { case (k, v) => reader.option(k, v) } - val kafka = reader.load().select("key", "value").as[(Array[Byte], Array[Byte])] - val mapped = kafka.map(kv => new String(kv._2).toInt + 1) + val kafka = reader.load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + val mapped = kafka.map(kv => kv._2.toInt + 1) testStream(mapped)( AddKafkaData(Set(topic), 4, 5, 6), // Add data when stream is stopped @@ -368,10 +376,10 @@ class KafkaSourceStressSuite extends KafkaSourceTest with BeforeAndAfter { .option("subscribePattern", "stress.*") .option("failOnCorruptMetadata", "false") .load() - .select("key", "value") - .as[(Array[Byte], Array[Byte])] + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] - val mapped = kafka.map(kv => new String(kv._2).toInt + 1) + val mapped = kafka.map(kv => kv._2.toInt + 1) runStressTest( mapped, From 7ff10599fdadcbdd2515b3216d35307e906de184 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Mon, 3 Oct 2016 16:25:16 -0700 Subject: [PATCH 21/28] Address rest comments --- .../structured-streaming-kafka-integration.md | 18 ++++++++++ .../sql/kafka010/CachedKafkaConsumer.scala | 35 +++++++++---------- .../spark/sql/kafka010/KafkaSource.scala | 32 ++++++++++------- .../spark/sql/kafka010/KafkaSourceRDD.scala | 5 +-- 4 files changed, 57 insertions(+), 33 deletions(-) diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md index 6c0f428613d7..ae794796d983 100644 --- a/docs/structured-streaming-kafka-integration.md +++ b/docs/structured-streaming-kafka-integration.md @@ -180,6 +180,24 @@ Right now, the Kafka source has the following Spark's specific options. The pattern used to subscribe the topic. Only one of "subscribe" and "subscribePattern" options can be specified for Kafka source. + + kafka.consumer.poll.timeoutMs + long + 512 + The timeout in milliseconds to poll data from Kafka in executors. + + + fetchOffset.numRetries + int + 3 + Number of times to retry before giving up fatch Kafka latest offsets. + + + fetchOffset.retry.intervalMs + long + 10 + milliseconds to wait before retrying to fetch Kafka offsets + Kafka's own configurations can be set via `DataStreamReader.option` with `kafka.` prefix, e.g, diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala index d8bc1ef634f1..3b5a96534f9b 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala @@ -35,6 +35,8 @@ private[kafka010] case class CachedKafkaConsumer private( topicPartition: TopicPartition, kafkaParams: ju.Map[String, Object]) extends Logging { + private val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG).asInstanceOf[String] + private val consumer = { val c = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams) val tps = new ju.ArrayList[TopicPartition]() @@ -43,13 +45,6 @@ private[kafka010] case class CachedKafkaConsumer private( 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 @@ -58,28 +53,30 @@ private[kafka010] case class CachedKafkaConsumer private( * 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") + def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[Array[Byte], Array[Byte]] = { + logDebug(s"Get $groupId $topicPartition nextOffset $nextOffsetInFetchedData requested $offset") if (offset != nextOffsetInFetchedData) { logInfo(s"Initial fetch for $topicPartition $offset") seek(offset) - poll() + poll(pollTimeoutMs) } - if (!fetchedData.hasNext()) { poll() } + if (!fetchedData.hasNext()) { poll(pollTimeoutMs) } assert(fetchedData.hasNext(), - s"Failed to get records for $topicPartition $offset after polling for $pollTimeoutMs") + s"Failed to get records for $groupId $topicPartition $offset " + + s"after polling for $pollTimeoutMs") var record = fetchedData.next() if (record.offset != offset) { - logInfo(s"Buffer miss for $topicPartition $offset") + logInfo(s"Buffer miss for $groupId $topicPartition $offset") seek(offset) - poll() + poll(pollTimeoutMs) assert(fetchedData.hasNext(), - s"Failed to get records for $topicPartition $offset after polling for $pollTimeoutMs") + s"Failed to get records for $groupId $topicPartition $offset " + + s"after polling for $pollTimeoutMs") record = fetchedData.next() assert(record.offset == offset, - s"Got wrong record for $topicPartition even after seeking to offset $offset") + s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset") } nextOffsetInFetchedData = offset + 1 @@ -89,14 +86,14 @@ private[kafka010] case class CachedKafkaConsumer private( private def close(): Unit = consumer.close() private def seek(offset: Long): Unit = { - logDebug(s"Seeking to $topicPartition $offset") + logDebug(s"Seeking to $groupId $topicPartition $offset") consumer.seek(topicPartition, offset) } - private def poll(): Unit = { + private def poll(pollTimeoutMs: Long): Unit = { val p = consumer.poll(pollTimeoutMs) val r = p.records(topicPartition) - logDebug(s"Polled ${p.partitions()} ${r.size}") + logDebug(s"Polled $groupId ${p.partitions()} ${r.size}") fetchedData = r.iterator } } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index f05ff605ba5e..bbebe642bb1f 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -47,10 +47,9 @@ import org.apache.spark.util.UninterruptibleThread * * - 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. + * 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. * @@ -88,6 +87,14 @@ private[kafka010] case class KafkaSource( private val sc = sqlContext.sparkContext + private val pollTimeoutMs = sourceOptions.getOrElse("kafka.consumer.poll.timeoutMs", "512").toLong + + private val maxOffsetFetchAttempts = + sourceOptions.getOrElse("fetchOffset.numRetries", "3").toInt + + private val offsetFetchAttemptIntervalMs = + sourceOptions.getOrElse("fetchOffset.retry.intervalMs", "10").toLong + /** * A KafkaConsumer used in the driver to query the latest Kafka offsets. This only queries the * offsets and never commits them. @@ -121,7 +128,11 @@ private[kafka010] case class KafkaSource( Some(offset) } - /** Returns the data that is between the offsets [`start`, `end`), i.e. end is exclusive. */ + /** + * Returns the data that is between the offsets + * [`start.get.partitionToOffsets`, `end.partitionToOffsets`), i.e. end.partitionToOffsets is + * exclusive. + */ override def getBatch(start: Option[Offset], end: Offset): DataFrame = { // Make sure initialPartitionOffsets is set initialPartitionOffsets @@ -195,7 +206,7 @@ private[kafka010] case class KafkaSource( // Create a RDD that reads from Kafka and get the (key, value) pair as byte arrays. val rdd = new KafkaSourceRDD( - sc, executorKafkaParams, offsetRanges).map { cr => + sc, executorKafkaParams, offsetRanges, pollTimeoutMs).map { cr => Row(cr.key, cr.value, cr.topic, cr.partition, cr.offset, cr.timestamp, cr.timestampType.id) } @@ -273,7 +284,7 @@ private[kafka010] case class KafkaSource( var result: Option[Map[TopicPartition, Long]] = None var attempt = 1 var lastException: Throwable = null - while (result.isEmpty && attempt <= MAX_OFFSET_FETCH_ATTEMPTS + while (result.isEmpty && attempt <= maxOffsetFetchAttempts && !Thread.currentThread().isInterrupted) { Thread.currentThread match { case ut: UninterruptibleThread => @@ -291,7 +302,7 @@ private[kafka010] case class KafkaSource( lastException = e logWarning(s"Error in attempt $attempt getting Kafka offsets: ", e) attempt += 1 - Thread.sleep(OFFSET_FETCH_ATTEMPT_INTERVAL_MS) + Thread.sleep(offsetFetchAttemptIntervalMs) } } case _ => @@ -303,7 +314,7 @@ private[kafka010] case class KafkaSource( throw new InterruptedException() } if (result.isEmpty) { - assert(attempt > MAX_OFFSET_FETCH_ATTEMPTS) + assert(attempt > maxOffsetFetchAttempts) assert(lastException != null) throw lastException } @@ -328,9 +339,6 @@ private[kafka010] case class KafkaSource( /** Companion object for the [[KafkaSource]]. */ private[kafka010] object KafkaSource { - val MAX_OFFSET_FETCH_ATTEMPTS = 3 - val OFFSET_FETCH_ATTEMPT_INTERVAL_MS = 10 - def kafkaSchema: StructType = StructType(Seq( StructField("key", BinaryType), StructField("value", BinaryType), diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala index b7af9c8a152e..496af7e39aba 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala @@ -60,7 +60,8 @@ private[kafka010] case class KafkaSourceRDDPartition( private[kafka010] class KafkaSourceRDD( sc: SparkContext, executorKafkaParams: ju.Map[String, Object], - offsetRanges: Seq[KafkaSourceRDDOffsetRange]) + offsetRanges: Seq[KafkaSourceRDDOffsetRange], + pollTimeoutMs: Long) extends RDD[ConsumerRecord[Array[Byte], Array[Byte]]](sc, Nil) { override def persist(newLevel: StorageLevel): this.type = { @@ -137,7 +138,7 @@ private[kafka010] class KafkaSourceRDD( override def hasNext(): Boolean = requestOffset < range.untilOffset override def next(): ConsumerRecord[Array[Byte], Array[Byte]] = { assert(hasNext(), "Can't call next() once untilOffset has been reached") - val r = consumer.get(requestOffset) + val r = consumer.get(requestOffset, pollTimeoutMs) requestOffset += 1 r } From a6c4970ace1df46e2d65c2cc8a606f3736454d35 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Mon, 3 Oct 2016 16:46:24 -0700 Subject: [PATCH 22/28] Remove the sorting --- .../structured-streaming-kafka-integration.md | 3 ++- .../spark/sql/kafka010/KafkaSource.scala | 21 ++++++------------- 2 files changed, 8 insertions(+), 16 deletions(-) diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md index ae794796d983..355c783e500a 100644 --- a/docs/structured-streaming-kafka-integration.md +++ b/docs/structured-streaming-kafka-integration.md @@ -4,7 +4,8 @@ title: Structured Streaming + Kafka Integration Guide (Kafka broker version 0.10 --- 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. +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: diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index bbebe642bb1f..b46ecd5ed69b 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -170,18 +170,18 @@ private[kafka010] case class KafkaSource( // Use the until partitions to calculate offset ranges to ignore partitions that have // been deleted - val sortedTopicPartitions = untilPartitionOffsets.keySet.filter { tp => + val topicPartitions = untilPartitionOffsets.keySet.filter { tp => // Ignore partitions that we don't know the from offsets. newPartitionOffsets.contains(tp) || fromPartitionOffsets.contains(tp) - }.toSeq.sorted(topicPartitionOrdering) - logDebug("Sorted topicPartitions: " + sortedTopicPartitions.mkString(", ")) + }.toSeq + logDebug("TopicPartitions: " + topicPartitions.mkString(", ")) val sortedExecutors = getSortedExecutorList(sc) val numExecutors = sortedExecutors.length logDebug("Sorted executors: " + sortedExecutors.mkString(", ")) // Calculate offset ranges - val offsetRanges = sortedTopicPartitions.map { tp => + val offsetRanges = topicPartitions.map { tp => val fromOffset = fromPartitionOffsets.get(tp).getOrElse { newPartitionOffsets.getOrElse(tp, { // This should not happen since newPartitionOffsets contains all partitions not in @@ -191,6 +191,8 @@ private[kafka010] case class KafkaSource( } val untilOffset = untilPartitionOffsets(tp) val preferredLoc = if (numExecutors > 0) { + // This allows cached KafkaConsumers in the executors to be re-used to read the same + // partition in every batch. Some(sortedExecutors(floorMod(tp.hashCode, numExecutors))) } else None KafkaSourceRDDOffsetRange(tp, fromOffset, untilOffset, preferredLoc) @@ -390,16 +392,5 @@ private[kafka010] object KafkaSource { if (a.host == b.host) { a.executorId > b.executorId } else { a.host > b.host } } - // Sort the partitions and current list of executors to consistently assign each partition - // to the executor. This allows cached KafkaConsumers in the executors to be re-used to - // read the same partition in every batch. - private val topicPartitionOrdering = new Ordering[TopicPartition] { - override def compare(l: TopicPartition, r: TopicPartition): Int = { - implicitly[Ordering[(String, Long)]].compare( - (l.topic, l.partition), - (r.topic, r.partition)) - } - } - private def floorMod(a: Long, b: Int): Int = ((a % b).toInt + b) % b } From 9e9fef33ef26913daddf80ee63d2d189e3c9f8f8 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Tue, 4 Oct 2016 13:42:04 -0700 Subject: [PATCH 23/28] Address nits --- .../structured-streaming-kafka-integration.md | 52 +++++++++++-------- .../spark/sql/kafka010/KafkaSource.scala | 2 +- .../sql/kafka010/KafkaSourceProvider.scala | 2 +- .../execution/streaming/StreamExecution.scala | 3 +- .../spark/sql/streaming/StreamTest.scala | 6 +-- 5 files changed, 37 insertions(+), 28 deletions(-) diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md index 355c783e500a..5af5b43308e3 100644 --- a/docs/structured-streaming-kafka-integration.md +++ b/docs/structured-streaming-kafka-integration.md @@ -3,9 +3,7 @@ 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. +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: @@ -148,7 +146,30 @@ Each row in the source has the following schema: -Right now, the Kafka source has the following Spark's specific options. +The following options should be set for the Kafka source. + + + + + + + + + + + + + + + + + + +
Optionvaluemeaning
subscribeA comma-separated list of topicsThe topic list to subscribe. Only one of "subscribe" and "subscribePattern" options can be + specified for Kafka source.
subscribePatternJava regex stringThe pattern used to subscribe the topic. Only one of "subscribe" and "subscribePattern" + options can be specified for Kafka source.
kafka.bootstrap.serversA comma-separated list of host:portThe Kafka "bootstrap.servers" configuration.
+ +The rest configurations are optional: @@ -161,25 +182,12 @@ Right now, the Kafka source has the following Spark's specific options. uery is started, and that resuming will always pick up from where the query left off. - + - - - - - - - - - - - - - + @@ -194,7 +202,7 @@ Right now, the Kafka source has the following Spark's specific options. - + diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index b46ecd5ed69b..8f85738fd198 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -93,7 +93,7 @@ private[kafka010] case class KafkaSource( sourceOptions.getOrElse("fetchOffset.numRetries", "3").toInt private val offsetFetchAttemptIntervalMs = - sourceOptions.getOrElse("fetchOffset.retry.intervalMs", "10").toLong + sourceOptions.getOrElse("fetchOffset.retryIntervalMs", "10").toLong /** * A KafkaConsumer used in the driver to query the latest Kafka offsets. This only queries the diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala index 729108064dd1..fd71dce524fd 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala @@ -210,7 +210,7 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider |Kafka option '${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG}' is not supported. |Instead set the source option '$STARTING_OFFSET_OPTION_KEY' to 'earliest' or 'latest' to |specify where to start. Structured Streaming manages which offsets are consumed - |internally, rather than rely on the kafka Consumer to do it. This will ensure that no + |internally, rather than relying on the kafka Consumer to do it. This will ensure that no |data is missed when when new topics/partitions are dynamically subscribed. Note that |'$STARTING_OFFSET_OPTION_KEY' only applies when a new Streaming query is started, and |that resuming will always pick up from where the query left off. See the docs for more diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 4c5f8a9dc923..b3a0d6ad0bd4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -532,6 +532,7 @@ object StreamExecution { } /** - * A special thread to run the stream query. + * A special thread to run the stream query. Some codes require to run in the StreamExecutionThread + * and will use `classOf[StreamExecutionThread]` to check. */ abstract class StreamExecutionThread(name: String) extends UninterruptibleThread(name) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index 46b00b5a884c..3138467a0988 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -489,21 +489,21 @@ 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 preparedActions actions need to run before starting the stress test. + * @param prepareActions actions need to run before starting the stress test. * @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], - preparedActions: Seq[StreamAction], + prepareActions: Seq[StreamAction], addData: (Seq[Int], Boolean) => StreamAction, iterations: Int): Unit = { implicit val intEncoder = ExpressionEncoder[Int]() var dataPos = 0 var running = true val actions = new ArrayBuffer[StreamAction]() - actions ++= preparedActions + actions ++= prepareActions def addCheck() = { actions += CheckAnswer(1 to dataPos: _*) } From d50a05eb703acb841b85b04e2b52e958778e6ab1 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Tue, 4 Oct 2016 14:46:26 -0700 Subject: [PATCH 24/28] Address --- .../apache/spark/sql/kafka010/KafkaSource.scala | 14 +++++++------- .../spark/sql/kafka010/KafkaSourceProvider.scala | 8 ++++---- .../spark/sql/kafka010/KafkaSourceSuite.scala | 2 +- 3 files changed, 12 insertions(+), 12 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index 8f85738fd198..8352bdfc8ffc 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -82,7 +82,7 @@ private[kafka010] case class KafkaSource( executorKafkaParams: ju.Map[String, Object], sourceOptions: Map[String, String], metadataPath: String, - failOnCorruptMetadata: Boolean) + failOnDataLoss: Boolean) extends Source with Logging { private val sc = sqlContext.sparkContext @@ -156,12 +156,12 @@ private[kafka010] case class KafkaSource( if (newPartitionOffsets.keySet != newPartitions) { // We cannot get from offsets for some partitions. It means they got deleted. val deletedPartitions = newPartitions.diff(newPartitionOffsets.keySet) - reportCorruptMetadata( + reportDataLoss( s"Cannot find earliest offsets of ${deletedPartitions}. Some data may have been missed") } logInfo(s"Partitions added: $newPartitionOffsets") newPartitionOffsets.filter(_._2 != 0).foreach { case (p, o) => - reportCorruptMetadata( + reportDataLoss( s"Added partition $p starts from $o instead of 0. Some data may have been missed") } @@ -198,7 +198,7 @@ private[kafka010] case class KafkaSource( KafkaSourceRDDOffsetRange(tp, fromOffset, untilOffset, preferredLoc) }.filter { range => if (range.untilOffset < range.fromOffset) { - reportCorruptMetadata(s"Partition ${range.topicPartition}'s offset was changed from " + + reportDataLoss(s"Partition ${range.topicPartition}'s offset was changed from " + s"${range.fromOffset} to ${range.untilOffset}, some data may have been missed") false } else { @@ -325,11 +325,11 @@ private[kafka010] case class KafkaSource( } /** - * If `failOnCorruptMetadata` is true, this method will throw an `IllegalStateException`. + * If `failOnDataLoss` is true, this method will throw an `IllegalStateException`. * Otherwise, just log a warning. */ - private def reportCorruptMetadata(message: String): Unit = { - if (failOnCorruptMetadata) { + private def reportDataLoss(message: String): Unit = { + if (failOnDataLoss) { throw new IllegalStateException(message) } else { logWarning(message) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala index fd71dce524fd..86d3431d75ce 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala @@ -138,8 +138,8 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider throw new IllegalArgumentException("Unknown option") } - val failOnCorruptMetadata = - caseInsensitiveParams.getOrElse(FAIL_ON_CORRUPT_METADATA_OPTION_KEY, "true").toBoolean + val failOnDataLoss = + caseInsensitiveParams.getOrElse(FAIL_ON_DATA_LOSS_OPTION_KEY, "true").toBoolean new KafkaSource( sqlContext, @@ -147,7 +147,7 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider kafkaParamsForExecutors, parameters, metadataPath, - failOnCorruptMetadata) + failOnDataLoss) } private def validateOptions(parameters: Map[String, String]): Unit = { @@ -278,5 +278,5 @@ private[kafka010] object KafkaSourceProvider { private val STRATEGY_OPTION_KEYS = Set("subscribe", "subscribepattern") private val STARTING_OFFSET_OPTION_KEY = "startingoffset" private val STARTING_OFFSET_OPTION_VALUES = Set("earliest", "latest") - private val FAIL_ON_CORRUPT_METADATA_OPTION_KEY = "failoncorruptmetadata" + private val FAIL_ON_DATA_LOSS_OPTION_KEY = "failondataloss" } diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala index 1c92d40619fe..20b8ea3eee57 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala @@ -374,7 +374,7 @@ class KafkaSourceStressSuite extends KafkaSourceTest with BeforeAndAfter { .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.metadata.max.age.ms", "1") .option("subscribePattern", "stress.*") - .option("failOnCorruptMetadata", "false") + .option("failOnDataLoss", "false") .load() .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") .as[(String, String)] From 4316906ab19556d297ded2bc38af85bb81a2e91b Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Tue, 4 Oct 2016 16:47:25 -0700 Subject: [PATCH 25/28] Address more --- docs/structured-streaming-kafka-integration.md | 2 +- docs/structured-streaming-programming-guide.md | 13 ++++++------- .../org/apache/spark/sql/kafka010/KafkaSource.scala | 2 +- .../spark/sql/kafka010/KafkaSourceProvider.scala | 2 +- 4 files changed, 9 insertions(+), 10 deletions(-) diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md index 5af5b43308e3..9e86a3ee4932 100644 --- a/docs/structured-streaming-kafka-integration.md +++ b/docs/structured-streaming-kafka-integration.md @@ -190,7 +190,7 @@ The rest configurations are optional: as you expected. - + diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md index 8b8f776a4b6c..8e49a3d0f1c7 100644 --- a/docs/structured-streaming-programming-guide.md +++ b/docs/structured-streaming-programming-guide.md @@ -418,10 +418,15 @@ Since Spark 2.0, DataFrames and Datasets can represent static, bounded data, as Streaming DataFrames can be created through the `DataStreamReader` interface ([Scala](api/scala/index.html#org.apache.spark.sql.streaming.DataStreamReader)/ [Java](api/java/org/apache/spark/sql/streaming/DataStreamReader.html)/ -[Python](api/python/pyspark.sql.html#pyspark.sql.streaming.DataStreamReader) docs) returned by `SparkSession.readStream()`. Similar to the read interface for creating static DataFrame, you can specify the details of the source – data format, schema, options, etc. In Spark 2.0, there are a few built-in sources. +[Python](api/python/pyspark.sql.html#pyspark.sql.streaming.DataStreamReader) docs) returned by `SparkSession.readStream()`. Similar to the read interface for creating static DataFrame, you can specify the details of the source – data format, schema, options, etc. + +#### Data Sources +In Spark 2.0, there are a few built-in sources. - **File source** - Reads files written in a directory as a stream of data. Supported file formats are text, csv, json, parquet. See the docs of the DataStreamReader interface for a more up-to-date list, and supported options for each file format. Note that the files must be atomically placed in the given directory, which in most file systems, can be achieved by file move operations. + - **Kafka source** - Poll data from Kafka. It's compatible with Kafka broker versions 0.10.0 or higher. See the [Kafka Integration Guide](structured-streaming-kafka-integration.html) for more details. + - **Socket source (for testing)** - Reads UTF8 text data from a socket connection. The listening server socket is at the driver. Note that this should be used only for testing as this does not provide end-to-end fault-tolerance guarantees. Here are some examples. @@ -1126,12 +1131,6 @@ aggDF \ -## Advanced Sources - -Structured Streaming supports the following advanced sources: - -- **Kafka:** Structured Streaming {{site.SPARK_VERSION_SHORT}} is compatible with Kafka broker versions 0.10.0 or higher. See the [Kafka Integration Guide](structured-streaming-kafka-integration.html) for more details. - # Where to go from here - Examples: See and run the [Scala]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples/sql/streaming)/[Java]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/java/org/apache/spark/examples/sql/streaming)/[Python]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/python/sql/streaming) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index 8352bdfc8ffc..346709c234dc 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -87,7 +87,7 @@ private[kafka010] case class KafkaSource( private val sc = sqlContext.sparkContext - private val pollTimeoutMs = sourceOptions.getOrElse("kafka.consumer.poll.timeoutMs", "512").toLong + private val pollTimeoutMs = sourceOptions.getOrElse("kafkaConsumer.pollTimeoutMs", "512").toLong private val maxOffsetFetchAttempts = sourceOptions.getOrElse("fetchOffset.numRetries", "3").toInt diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala index 86d3431d75ce..067e3f99dd32 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala @@ -210,7 +210,7 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider |Kafka option '${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG}' is not supported. |Instead set the source option '$STARTING_OFFSET_OPTION_KEY' to 'earliest' or 'latest' to |specify where to start. Structured Streaming manages which offsets are consumed - |internally, rather than relying on the kafka Consumer to do it. This will ensure that no + |internally, rather than relying on the kafkaConsumer to do it. This will ensure that no |data is missed when when new topics/partitions are dynamically subscribed. Note that |'$STARTING_OFFSET_OPTION_KEY' only applies when a new Streaming query is started, and |that resuming will always pick up from where the query left off. See the docs for more From d9d848ca455d659e39ceadf8c2b8d50867b85962 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Wed, 5 Oct 2016 10:40:38 -0700 Subject: [PATCH 26/28] Address nits --- docs/structured-streaming-kafka-integration.md | 8 ++++---- .../org/apache/spark/sql/kafka010/KafkaSource.scala | 12 ++++++------ .../spark/sql/kafka010/KafkaSourceProvider.scala | 4 ++-- .../apache/spark/sql/kafka010/KafkaSourceSuite.scala | 3 ++- 4 files changed, 14 insertions(+), 13 deletions(-) diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md index 9e86a3ee4932..668489addf82 100644 --- a/docs/structured-streaming-kafka-integration.md +++ b/docs/structured-streaming-kafka-integration.md @@ -146,7 +146,7 @@ Each row in the source has the following schema:
Optionvaluedefaultmeaning
failOnCorruptMetadatafailOnDataLoss [true, false] trueWhether to fail the query when metadata is corrupt (e.g., topics are deleted, or offsets are - out of range), which may lost data.
subscribeA comma-separated list of topics(none)The topic list to subscribe. Only one of "subscribe" and "subscribePattern" options can be - specified for Kafka source.
subscribePatternJava regex string(none)The pattern used to subscribe the topic. Only one of "subscribe" and "subscribePattern" - options can be specified for Kafka source.Whether to fail the query when it's possible that data is lost (e.g., topics are deleted, or + offsets are out of range). This may be a false alarm. You can disable it when it doesn't work + as you expected.
kafka.consumer.poll.timeoutMsNumber of times to retry before giving up fatch Kafka latest offsets.
fetchOffset.retry.intervalMsfetchOffset.retryIntervalMs long 10 milliseconds to wait before retrying to fetch Kafka offsets
kafka.consumer.poll.timeoutMskafkaConsumer.pollTimeoutMs long 512 The timeout in milliseconds to poll data from Kafka in executors.
-The following options should be set for the Kafka source. +The following options must be set for the Kafka source. @@ -169,7 +169,7 @@ The following options should be set for the Kafka source.
Optionvaluemeaning
-The rest configurations are optional: +The following configurations are optional: @@ -221,9 +221,9 @@ Note that the following Kafka params cannot be set and the Kafka source will thr topics/partitions are dynamically subscribed. Note that `startingOffset` only applies when a new Streaming query is started, and that resuming will always pick up from where the query left off. - **key.deserializer**: Keys are always deserialized as byte arrays with ByteArrayDeserializer. Use - Dataframe operations to explicitly deserialize the keys. + DataFrame operations to explicitly deserialize the keys. - **value.deserializer**: Values are always deserialized as byte arrays with ByteArrayDeserializer. - Use Dataframe operations to explicitly deserialize the values. + Use DataFrame operations to explicitly deserialize the values. - **enable.auto.commit**: Kafka source doesn't commit any offset. - **interceptor.classes**: Kafka source always read keys and values as byte arrays. It's not safe to use ConsumerInterceptor as it may break the query. diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index 346709c234dc..77b8e604a4b5 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -102,9 +102,9 @@ private[kafka010] case class KafkaSource( private val consumer = consumerStrategy.createConsumer() /** - * Lazy set initialPartitionOffsets to make sure only call `KafkaConsumer.poll` in - * StreamExecutionThread. Otherwise, interrupting a thread running `KafkaConsumer.poll` may hang - * forever (KAFKA-1894). + * Lazily initialize `initialPartitionOffsets` to make sure that `KafkaConsumer.poll` is only + * called in StreamExecutionThread. Otherwise, interrupting a thread while running + * `KafkaConsumer.poll` may hang forever (KAFKA-1894). */ private lazy val initialPartitionOffsets = { val metadataLog = new HDFSMetadataLog[KafkaSourceOffset](sqlContext.sparkSession, metadataPath) @@ -120,7 +120,7 @@ private[kafka010] case class KafkaSource( /** Returns the maximum available offset for this source. */ override def getOffset: Option[Offset] = { - // Make sure initialPartitionOffsets is set + // Make sure initialPartitionOffsets is initialized initialPartitionOffsets val offset = KafkaSourceOffset(fetchPartitionOffsets(seekToEnd = true)) @@ -134,7 +134,7 @@ private[kafka010] case class KafkaSource( * exclusive. */ override def getBatch(start: Option[Offset], end: Offset): DataFrame = { - // Make sure initialPartitionOffsets is set + // Make sure initialPartitionOffsets is initialized initialPartitionOffsets logInfo(s"GetBatch called with start = $start, end = $end") @@ -249,7 +249,7 @@ private[kafka010] case class KafkaSource( } /** - * Fetch the earliest offsets for newly discovered partitions. The return results may not contain + * Fetch the earliest offsets for newly discovered partitions. The return result may not contain * some partitions if they are deleted. */ private def fetchNewPartitionEarliestOffsets( diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala index 067e3f99dd32..1b0a2fe955d0 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala @@ -221,7 +221,7 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG}")) { throw new IllegalArgumentException( s"Kafka option '${ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG}' is not supported as keys " - + "are deserialized as byte arrays with ByteArrayDeserializer. Use Dataframe operations " + + "are deserialized as byte arrays with ByteArrayDeserializer. Use DataFrame operations " + "to explicitly deserialize the keys.") } @@ -229,7 +229,7 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider { throw new IllegalArgumentException( s"Kafka option '${ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG}' is not supported as " - + "value are deserialized as byte arrays with ByteArrayDeserializer. Use Dataframe " + + "value are deserialized as byte arrays with ByteArrayDeserializer. Use DataFrame " + "operations to explicitly deserialize the values.") } diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala index 20b8ea3eee57..66b74ca917d8 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala @@ -257,9 +257,10 @@ class KafkaSourceSuite extends KafkaSourceTest { testUnsupportedConfig("kafka.key.deserializer") testUnsupportedConfig("kafka.value.deserializer") - // only earliest and latest is supported testUnsupportedConfig("kafka.auto.offset.reset", "none") testUnsupportedConfig("kafka.auto.offset.reset", "someValue") + testUnsupportedConfig("kafka.auto.offset.reset", "earliest") + testUnsupportedConfig("kafka.auto.offset.reset", "latest") } private def newTopic(): String = s"topic-${topicId.getAndIncrement()}" From 7d658f1004375bbc49c60ac5091b56da34f04da2 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Wed, 5 Oct 2016 10:48:16 -0700 Subject: [PATCH 27/28] Address one more nit --- .../scala/org/apache/spark/sql/kafka010/KafkaSource.scala | 4 +++- .../org/apache/spark/sql/kafka010/KafkaSourceSuite.scala | 1 + 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index 77b8e604a4b5..7afb048b7a2b 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -166,7 +166,9 @@ private[kafka010] case class KafkaSource( } val deletedPartitions = fromPartitionOffsets.keySet.diff(untilPartitionOffsets.keySet) - logWarning(s"$deletedPartitions are gone. Some data may have been missed") + if (deletedPartitions.nonEmpty) { + reportDataLoss(s"$deletedPartitions are gone. Some data may have been missed") + } // Use the until partitions to calculate offset ranges to ignore partitions that have // been deleted diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala index 66b74ca917d8..64bf50305802 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala @@ -191,6 +191,7 @@ class KafkaSourceSuite extends KafkaSourceTest { .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.metadata.max.age.ms", "1") .option("subscribePattern", s"$topicPrefix-.*") + .option("failOnDataLoss", "false") val kafka = reader.load() .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") From 4754125d041ebdf2286bb015770d58351779622d Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Wed, 5 Oct 2016 11:46:45 -0700 Subject: [PATCH 28/28] Add an instruction for failOnDataLoss --- .../main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index 7afb048b7a2b..1be70db87497 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -332,7 +332,8 @@ private[kafka010] case class KafkaSource( */ private def reportDataLoss(message: String): Unit = { if (failOnDataLoss) { - throw new IllegalStateException(message) + throw new IllegalStateException(message + + ". Set the source option 'failOnDataLoss' to 'false' if you want to ignore these checks.") } else { logWarning(message) }
Optionvaluedefaultmeaning