From d371758538f659cbcf604e591110665cfca4f216 Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Thu, 19 Jan 2017 17:53:03 -0800 Subject: [PATCH 01/40] add kafka relation and refactor kafka source --- ...artingOffsets.scala => KafkaOffsets.scala} | 8 +- .../spark/sql/kafka010/KafkaReader.scala | 286 +++++++++++++++ .../spark/sql/kafka010/KafkaRelation.scala | 103 ++++++ .../spark/sql/kafka010/KafkaSource.scala | 333 ++---------------- .../sql/kafka010/KafkaSourceProvider.scala | 12 +- .../spark/sql/kafka010/KafkaUtils.scala | 38 ++ 6 files changed, 475 insertions(+), 305 deletions(-) rename external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/{StartingOffsets.scala => KafkaOffsets.scala} (79%) create mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaReader.scala create mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala create mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaUtils.scala diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/StartingOffsets.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsets.scala similarity index 79% rename from external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/StartingOffsets.scala rename to external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsets.scala index 83959e597171..5ac206604608 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/StartingOffsets.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsets.scala @@ -22,11 +22,11 @@ import org.apache.kafka.common.TopicPartition /* * Values that can be specified for config startingOffsets */ -private[kafka010] sealed trait StartingOffsets +private[kafka010] sealed trait KafkaOffsets -private[kafka010] case object EarliestOffsets extends StartingOffsets +private[kafka010] case object EarliestOffsets extends KafkaOffsets -private[kafka010] case object LatestOffsets extends StartingOffsets +private[kafka010] case object LatestOffsets extends KafkaOffsets private[kafka010] case class SpecificOffsets( - partitionOffsets: Map[TopicPartition, Long]) extends StartingOffsets + partitionOffsets: Map[TopicPartition, Long]) extends KafkaOffsets diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaReader.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaReader.scala new file mode 100644 index 000000000000..93190e252272 --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaReader.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.{util => ju} + +import scala.collection.JavaConverters._ +import scala.util.control.NonFatal + +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.spark.internal.Logging +import org.apache.spark.sql.kafka010.KafkaReader.ConsumerStrategy +import org.apache.spark.sql.types._ +import org.apache.spark.util.UninterruptibleThread + +/** + * This class uses Kafka's own [[KafkaConsumer]] API to read data offsets from Kafka. + * + * - 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.KafkaReader.ConsumerStrategy]] for more details. + */ +private[kafka010] class KafkaReader( + consumerStrategy: ConsumerStrategy, + driverKafkaParams: ju.Map[String, Object], + readerOptions: Map[String, String], + driverGroupIdPrefix: String) + extends Logging { + + /** + * A KafkaConsumer used in the driver to query the latest Kafka offsets. This only queries the + * offsets and never commits them. + */ + protected var consumer = createConsumer() + + private val maxOffsetFetchAttempts = + readerOptions.getOrElse("fetchOffset.numRetries", "3").toInt + + private val offsetFetchAttemptIntervalMs = + readerOptions.getOrElse("fetchOffset.retryIntervalMs", "1000").toLong + + private var groupId: String = null + + private var nextId = 0 + + private def nextGroupId(): String = { + groupId = driverGroupIdPrefix + "-" + nextId + nextId += 1 + groupId + } + + override def toString(): String = consumerStrategy.toString + + def close(): Unit = consumer.close() + + /** + * Set consumer position to specified offsets, making sure all assignments are set. + */ + def fetchSpecificStartingOffsets( + partitionOffsets: Map[TopicPartition, Long]): Map[TopicPartition, Long] = + withRetriesWithoutInterrupt { + // Poll to get the latest assigned partitions + consumer.poll(0) + val partitions = consumer.assignment() + consumer.pause(partitions) + assert(partitions.asScala == partitionOffsets.keySet, + "If startingOffsets contains specific offsets, you must specify all TopicPartitions.\n" + + "Use -1 for latest, -2 for earliest, if you don't care.\n" + + s"Specified: ${partitionOffsets.keySet} Assigned: ${partitions.asScala}") + logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $partitionOffsets") + + partitionOffsets.foreach { + case (tp, -1) => consumer.seekToEnd(ju.Arrays.asList(tp)) + case (tp, -2) => consumer.seekToBeginning(ju.Arrays.asList(tp)) + case (tp, off) => consumer.seek(tp, off) + } + partitionOffsets.map { + case (tp, _) => tp -> consumer.position(tp) + } + } + + /** + * Fetch the earliest offsets of partitions. + */ + def fetchEarliestOffsets(): Map[TopicPartition, Long] = withRetriesWithoutInterrupt { + // Poll to get the latest assigned partitions + consumer.poll(0) + val partitions = consumer.assignment() + consumer.pause(partitions) + logDebug(s"Partitions assigned to consumer: $partitions. Seeking to the beginning") + + consumer.seekToBeginning(partitions) + val partitionOffsets = partitions.asScala.map(p => p -> consumer.position(p)).toMap + logDebug(s"Got earliest offsets for partition : $partitionOffsets") + partitionOffsets + } + + /** + * Fetch the latest offset of partitions. + */ + def fetchLatestOffsets(): Map[TopicPartition, Long] = withRetriesWithoutInterrupt { + // Poll to get the latest assigned partitions + consumer.poll(0) + val partitions = consumer.assignment() + consumer.pause(partitions) + logDebug(s"Partitions assigned to consumer: $partitions. Seeking to the end.") + + consumer.seekToEnd(partitions) + val partitionOffsets = partitions.asScala.map(p => p -> consumer.position(p)).toMap + logDebug(s"Got latest offsets for partition : $partitionOffsets") + partitionOffsets + } + + /** + * Fetch the earliest offsets for newly discovered partitions. The return result may not contain + * some partitions if they are deleted. + */ + def fetchNewPartitionEarliestOffsets( + newPartitions: Seq[TopicPartition]): Map[TopicPartition, Long] = { + if (newPartitions.isEmpty) { + Map.empty[TopicPartition, Long] + } else { + withRetriesWithoutInterrupt { + // Poll to get the latest assigned partitions + consumer.poll(0) + val partitions = consumer.assignment() + consumer.pause(partitions) + logDebug(s"\tPartitions assigned to consumer: $partitions") + + // Get the earliest offset of each partition + consumer.seekToBeginning(partitions) + val partitionOffsets = 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 earliest offsets for new partitions: $partitionOffsets") + partitionOffsets + } + } + } + + /** + * Helper function that does multiple retries on the a body of code that returns offsets. + * Retries are needed to handle transient failures. For e.g. race conditions between getting + * assignment and getting position while topics/partitions are deleted can cause NPEs. + * + * This method also makes sure `body` won't be interrupted to workaround a potential issue in + * `KafkaConsumer.poll`. (KAFKA-1894) + */ + private def withRetriesWithoutInterrupt( + body: => Map[TopicPartition, Long]): Map[TopicPartition, Long] = { + // Make sure `KafkaConsumer.poll` won't be interrupted (KAFKA-1894) + assert(Thread.currentThread().isInstanceOf[UninterruptibleThread]) + + synchronized { + var result: Option[Map[TopicPartition, Long]] = None + var attempt = 1 + var lastException: Throwable = null + while (result.isEmpty && attempt <= maxOffsetFetchAttempts + && !Thread.currentThread().isInterrupted) { + Thread.currentThread match { + case ut: UninterruptibleThread => + // "KafkaConsumer.poll" may hang forever if the thread is interrupted (E.g., the query + // is stopped)(KAFKA-1894). Hence, we just make sure we don't interrupt it. + // + // If the broker addresses are wrong, or Kafka cluster is down, "KafkaConsumer.poll" may + // hang forever as well. This cannot be resolved in KafkaSource until Kafka fixes the + // issue. + ut.runUninterruptibly { + try { + result = Some(body) + } catch { + case NonFatal(e) => + lastException = e + logWarning(s"Error in attempt $attempt getting Kafka offsets: ", e) + attempt += 1 + Thread.sleep(offsetFetchAttemptIntervalMs) + resetConsumer() + } + } + case _ => + throw new IllegalStateException( + "Kafka APIs must be executed on a o.a.spark.util.UninterruptibleThread") + } + } + if (Thread.interrupted()) { + throw new InterruptedException() + } + if (result.isEmpty) { + assert(attempt > maxOffsetFetchAttempts) + assert(lastException != null) + throw lastException + } + result.get + } + } + + /** + * Create a consumer using the new generated group id. We always use a new consumer to avoid + * just using a broken consumer to retry on Kafka errors, which likely will fail again. + */ + private def createConsumer(): Consumer[Array[Byte], Array[Byte]] = synchronized { + val newKafkaParams = new ju.HashMap[String, Object](driverKafkaParams) + newKafkaParams.put(ConsumerConfig.GROUP_ID_CONFIG, nextGroupId()) + consumerStrategy.createConsumer(newKafkaParams) + } + + private def resetConsumer(): Unit = synchronized { + consumer.close() + consumer = createConsumer() + } +} + +private[kafka010] object KafkaReader { + + def kafkaSchema: StructType = StructType(Seq( + StructField("key", BinaryType), + StructField("value", BinaryType), + StructField("topic", StringType), + StructField("partition", IntegerType), + StructField("offset", LongType), + StructField("timestamp", TimestampType), + StructField("timestampType", IntegerType) + )) + + sealed trait ConsumerStrategy { + def createConsumer(kafkaParams: ju.Map[String, Object]): Consumer[Array[Byte], Array[Byte]] + } + + case class AssignStrategy(partitions: Array[TopicPartition]) extends ConsumerStrategy { + override def createConsumer( + kafkaParams: ju.Map[String, Object]): Consumer[Array[Byte], Array[Byte]] = { + val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams) + consumer.assign(ju.Arrays.asList(partitions: _*)) + consumer + } + + override def toString: String = s"Assign[${partitions.mkString(", ")}]" + } + + case class SubscribeStrategy(topics: Seq[String]) extends ConsumerStrategy { + override def createConsumer( + kafkaParams: ju.Map[String, Object]): Consumer[Array[Byte], Array[Byte]] = { + val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams) + consumer.subscribe(topics.asJava) + consumer + } + + override def toString: String = s"Subscribe[${topics.mkString(", ")}]" + } + + case class SubscribePatternStrategy(topicPattern: String) + extends ConsumerStrategy { + override def createConsumer( + kafkaParams: ju.Map[String, Object]): Consumer[Array[Byte], Array[Byte]] = { + val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams) + consumer.subscribe( + ju.regex.Pattern.compile(topicPattern), + new NoOpConsumerRebalanceListener()) + consumer + } + + override def toString: String = s"SubscribePattern[$topicPattern]" + } +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala new file mode 100644 index 000000000000..7be88b995daa --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala @@ -0,0 +1,103 @@ +/* + * 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.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.sources.{BaseRelation, TableScan} +import org.apache.spark.sql.types.StructType +import org.apache.spark.unsafe.types.UTF8String + + +private[kafka010] class KafkaRelation( + override val sqlContext: SQLContext, + kafkaReader: KafkaReader, + executorKafkaParams: ju.Map[String, Object], + sourceOptions: Map[String, String], + startingOffsets: Option[KafkaOffsets] = Some(EarliestOffsets), + endingOffsets: Option[KafkaOffsets] = Some(LatestOffsets), + failOnDataLoss: Boolean) + extends BaseRelation + with TableScan + with Logging { + + private val pollTimeoutMs = sourceOptions.getOrElse( + "kafkaConsumer.pollTimeoutMs", + sqlContext.sparkContext.conf.getTimeAsMs("spark.network.timeout", "120s").toString + ).toLong + + override def schema: StructType = KafkaReader.kafkaSchema + + override def buildScan(): RDD[Row] = { + // Leverage the KafkaReader to obtain the relevant partition offsets + val fromPartitionOffsets = getPartitionOffsets(startingOffsets.getOrElse(EarliestOffsets)) + val untilPartitionOffsets = getPartitionOffsets(endingOffsets.getOrElse(LatestOffsets)) + // Obtain topicPartitions in both from and until partition offset, ignoring + // topic partitions that were added and/or deleted between the two above calls. + val topicPartitions = fromPartitionOffsets.keySet.intersect(untilPartitionOffsets.keySet) + + + val sortedExecutors = KafkaUtils.getSortedExecutorList(sqlContext.sparkContext) + val numExecutors = sortedExecutors.length + logDebug("Sorted executors: " + sortedExecutors.mkString(", ")) + + // Calculate offset ranges + val offsetRanges = topicPartitions.map { tp => + val fromOffset = fromPartitionOffsets.get(tp).getOrElse { + // This should not happen since topicPartitions contains all partitions not in + // fromPartitionOffsets + throw new IllegalStateException(s"$tp doesn't have a from offset") + } + val untilOffset = untilPartitionOffsets(tp) + val preferredLoc = if (numExecutors > 0) { + // This allows cached KafkaConsumers in the executors to be re-used to read the same + // partition in every batch. + Some(sortedExecutors(KafkaUtils.floorMod(tp.hashCode, numExecutors))) + } else None + KafkaSourceRDDOffsetRange(tp, fromOffset, untilOffset, preferredLoc) + }.toArray + + logInfo("GetBatch generating RDD of offset range: " + + offsetRanges.sortBy(_.topicPartition.toString).mkString(", ")) + + // Create an RDD that reads from Kafka and get the (key, value) pair as byte arrays. + val rdd = new KafkaSourceRDD( + sqlContext.sparkContext, executorKafkaParams, offsetRanges, pollTimeoutMs, failOnDataLoss).map { cr => + InternalRow( + cr.key, + cr.value, + UTF8String.fromString(cr.topic), + cr.partition, + cr.offset, + DateTimeUtils.fromJavaTimestamp(new java.sql.Timestamp(cr.timestamp)), + cr.timestampType.id) + } + sqlContext.internalCreateDataFrame(rdd, schema).rdd + } + + private def getPartitionOffsets(kafkaOffsets: KafkaOffsets) = kafkaOffsets match { + case EarliestOffsets => kafkaReader.fetchEarliestOffsets() + case LatestOffsets => kafkaReader.fetchLatestOffsets() + case SpecificOffsets(p) => kafkaReader.fetchSpecificStartingOffsets(p) + } +} 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 43b8d9d6d7ee..1fb1ebe4b7d2 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,16 +21,10 @@ import java.{util => ju} import java.io._ import java.nio.charset.StandardCharsets -import scala.collection.JavaConverters._ -import scala.util.control.NonFatal - -import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, KafkaConsumer, OffsetOutOfRangeException} -import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener +import org.apache.kafka.clients.consumer.KafkaConsumer import org.apache.kafka.common.TopicPartition -import org.apache.spark.SparkContext import org.apache.spark.internal.Logging -import org.apache.spark.scheduler.ExecutorCacheTaskLocation import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.util.DateTimeUtils @@ -38,11 +32,9 @@ import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.kafka010.KafkaSource._ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String -import org.apache.spark.util.UninterruptibleThread /** - * A [[Source]] that uses Kafka's own [[KafkaConsumer]] API to reads data from Kafka. The design - * for this source is as follows. + * A [[Source]] that reads data from Kafka using the following design. * * - 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 @@ -50,19 +42,13 @@ import org.apache.spark.util.UninterruptibleThread * 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. + * - As soon as the source is created, the pre-configured [[KafkaReader]] + * is used to query the initial offsets that this source should + * start reading from. This is used to create the first batch. * - * - `getOffset()` uses the KafkaConsumer to query the latest available offsets, which are + * - `getOffset()` uses the [[KafkaReader]] 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 @@ -82,15 +68,13 @@ import org.apache.spark.util.UninterruptibleThread * and not use wrong broker addresses. */ private[kafka010] class KafkaSource( - sqlContext: SQLContext, - consumerStrategy: ConsumerStrategy, - driverKafkaParams: ju.Map[String, Object], - executorKafkaParams: ju.Map[String, Object], - sourceOptions: Map[String, String], - metadataPath: String, - startingOffsets: StartingOffsets, - failOnDataLoss: Boolean, - driverGroupIdPrefix: String) + sqlContext: SQLContext, + kafkaReader: KafkaReader, + executorKafkaParams: ju.Map[String, Object], + sourceOptions: Map[String, String], + metadataPath: String, + startingOffsets: KafkaOffsets, + failOnDataLoss: Boolean) extends Source with Logging { private val sc = sqlContext.sparkContext @@ -100,41 +84,9 @@ private[kafka010] class KafkaSource( sc.conf.getTimeAsMs("spark.network.timeout", "120s").toString ).toLong - private val maxOffsetFetchAttempts = - sourceOptions.getOrElse("fetchOffset.numRetries", "3").toInt - - private val offsetFetchAttemptIntervalMs = - sourceOptions.getOrElse("fetchOffset.retryIntervalMs", "1000").toLong - private val maxOffsetsPerTrigger = sourceOptions.get("maxOffsetsPerTrigger").map(_.toLong) - private var groupId: String = null - - private var nextId = 0 - - private def nextGroupId(): String = { - groupId = driverGroupIdPrefix + "-" + nextId - nextId += 1 - groupId - } - - /** - * A KafkaConsumer used in the driver to query the latest Kafka offsets. This only queries the - * offsets and never commits them. - */ - private var consumer: Consumer[Array[Byte], Array[Byte]] = createConsumer() - - /** - * Create a consumer using the new generated group id. We always use a new consumer to avoid - * just using a broken consumer to retry on Kafka errors, which likely will fail again. - */ - private def createConsumer(): Consumer[Array[Byte], Array[Byte]] = synchronized { - val newKafkaParams = new ju.HashMap[String, Object](driverKafkaParams) - newKafkaParams.put(ConsumerConfig.GROUP_ID_CONFIG, nextGroupId()) - consumerStrategy.createConsumer(newKafkaParams) - } - /** * Lazily initialize `initialPartitionOffsets` to make sure that `KafkaConsumer.poll` is only * called in StreamExecutionThread. Otherwise, interrupting a thread while running @@ -159,9 +111,9 @@ private[kafka010] class KafkaSource( metadataLog.get(0).getOrElse { val offsets = startingOffsets match { - case EarliestOffsets => KafkaSourceOffset(fetchEarliestOffsets()) - case LatestOffsets => KafkaSourceOffset(fetchLatestOffsets()) - case SpecificOffsets(p) => KafkaSourceOffset(fetchSpecificStartingOffsets(p)) + case EarliestOffsets => KafkaSourceOffset(kafkaReader.fetchEarliestOffsets()) + case LatestOffsets => KafkaSourceOffset(kafkaReader.fetchLatestOffsets()) + case SpecificOffsets(p) => fetchAndVerify(p) } metadataLog.add(0, offsets) logInfo(s"Initial offsets: $offsets") @@ -169,16 +121,30 @@ private[kafka010] class KafkaSource( }.partitionToOffsets } + private def fetchAndVerify(specificOffsets: Map[TopicPartition, Long]) = { + val result = kafkaReader.fetchSpecificStartingOffsets(specificOffsets) + specificOffsets.foreach { + case (tp, off) if off != -1 && off != -2 => + if (result(tp) != off) { + reportDataLoss( + s"startingOffsets for $tp was $off but consumer reset to ${result(tp)}") + } + case _ => + // no real way to check that beginning or end is reasonable + } + KafkaSourceOffset(result) + } + private var currentPartitionOffsets: Option[Map[TopicPartition, Long]] = None - override def schema: StructType = KafkaSource.kafkaSchema + override def schema: StructType = KafkaReader.kafkaSchema /** Returns the maximum available offset for this source. */ override def getOffset: Option[Offset] = { // Make sure initialPartitionOffsets is initialized initialPartitionOffsets - val latest = fetchLatestOffsets() + val latest = kafkaReader.fetchLatestOffsets() val offsets = maxOffsetsPerTrigger match { case None => latest @@ -193,17 +159,12 @@ private[kafka010] class KafkaSource( Some(KafkaSourceOffset(offsets)) } - private def resetConsumer(): Unit = synchronized { - consumer.close() - consumer = createConsumer() - } - /** Proportionally distribute limit number of offsets among topicpartitions */ private def rateLimit( limit: Long, from: Map[TopicPartition, Long], until: Map[TopicPartition, Long]): Map[TopicPartition, Long] = { - val fromNew = fetchNewPartitionEarliestOffsets(until.keySet.diff(from.keySet).toSeq) + val fromNew = kafkaReader.fetchNewPartitionEarliestOffsets(until.keySet.diff(from.keySet).toSeq) val sizes = until.flatMap { case (tp, end) => // If begin isn't defined, something's wrong, but let alert logic in getBatch handle it @@ -253,7 +214,7 @@ private[kafka010] class KafkaSource( // Find the new partitions, and get their earliest offsets val newPartitions = untilPartitionOffsets.keySet.diff(fromPartitionOffsets.keySet) - val newPartitionOffsets = fetchNewPartitionEarliestOffsets(newPartitions.toSeq) + val newPartitionOffsets = kafkaReader.fetchNewPartitionEarliestOffsets(newPartitions.toSeq) if (newPartitionOffsets.keySet != newPartitions) { // We cannot get from offsets for some partitions. It means they got deleted. val deletedPartitions = newPartitions.diff(newPartitionOffsets.keySet) @@ -279,7 +240,7 @@ private[kafka010] class KafkaSource( }.toSeq logDebug("TopicPartitions: " + topicPartitions.mkString(", ")) - val sortedExecutors = getSortedExecutorList(sc) + val sortedExecutors = KafkaUtils.getSortedExecutorList(sc) val numExecutors = sortedExecutors.length logDebug("Sorted executors: " + sortedExecutors.mkString(", ")) @@ -296,7 +257,7 @@ private[kafka010] class KafkaSource( 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))) + Some(sortedExecutors(KafkaUtils.floorMod(tp.hashCode, numExecutors))) } else None KafkaSourceRDDOffsetRange(tp, fromOffset, untilOffset, preferredLoc) }.filter { range => @@ -335,163 +296,10 @@ private[kafka010] class KafkaSource( /** Stop this source and free any resources it has allocated. */ override def stop(): Unit = synchronized { - consumer.close() + kafkaReader.close() } - override def toString(): String = s"KafkaSource[$consumerStrategy]" - - /** - * Set consumer position to specified offsets, making sure all assignments are set. - */ - private def fetchSpecificStartingOffsets( - partitionOffsets: Map[TopicPartition, Long]): Map[TopicPartition, Long] = { - val result = withRetriesWithoutInterrupt { - // Poll to get the latest assigned partitions - consumer.poll(0) - val partitions = consumer.assignment() - consumer.pause(partitions) - assert(partitions.asScala == partitionOffsets.keySet, - "If startingOffsets contains specific offsets, you must specify all TopicPartitions.\n" + - "Use -1 for latest, -2 for earliest, if you don't care.\n" + - s"Specified: ${partitionOffsets.keySet} Assigned: ${partitions.asScala}") - logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $partitionOffsets") - - partitionOffsets.foreach { - case (tp, -1) => consumer.seekToEnd(ju.Arrays.asList(tp)) - case (tp, -2) => consumer.seekToBeginning(ju.Arrays.asList(tp)) - case (tp, off) => consumer.seek(tp, off) - } - partitionOffsets.map { - case (tp, _) => tp -> consumer.position(tp) - } - } - partitionOffsets.foreach { - case (tp, off) if off != -1 && off != -2 => - if (result(tp) != off) { - reportDataLoss( - s"startingOffsets for $tp was $off but consumer reset to ${result(tp)}") - } - case _ => - // no real way to check that beginning or end is reasonable - } - result - } - - /** - * Fetch the earliest offsets of partitions. - */ - private def fetchEarliestOffsets(): Map[TopicPartition, Long] = withRetriesWithoutInterrupt { - // Poll to get the latest assigned partitions - consumer.poll(0) - val partitions = consumer.assignment() - consumer.pause(partitions) - logDebug(s"Partitions assigned to consumer: $partitions. Seeking to the beginning") - - consumer.seekToBeginning(partitions) - val partitionOffsets = partitions.asScala.map(p => p -> consumer.position(p)).toMap - logDebug(s"Got earliest offsets for partition : $partitionOffsets") - partitionOffsets - } - - /** - * Fetch the latest offset of partitions. - */ - private def fetchLatestOffsets(): Map[TopicPartition, Long] = withRetriesWithoutInterrupt { - // Poll to get the latest assigned partitions - consumer.poll(0) - val partitions = consumer.assignment() - consumer.pause(partitions) - logDebug(s"Partitions assigned to consumer: $partitions. Seeking to the end.") - - consumer.seekToEnd(partitions) - val partitionOffsets = partitions.asScala.map(p => p -> consumer.position(p)).toMap - logDebug(s"Got latest offsets for partition : $partitionOffsets") - partitionOffsets - } - - /** - * Fetch the earliest offsets for newly discovered partitions. The return result may not contain - * some partitions if they are deleted. - */ - private def fetchNewPartitionEarliestOffsets( - newPartitions: Seq[TopicPartition]): Map[TopicPartition, Long] = - if (newPartitions.isEmpty) { - Map.empty[TopicPartition, Long] - } else { - withRetriesWithoutInterrupt { - // Poll to get the latest assigned partitions - consumer.poll(0) - val partitions = consumer.assignment() - consumer.pause(partitions) - logDebug(s"\tPartitions assigned to consumer: $partitions") - - // Get the earliest offset of each partition - consumer.seekToBeginning(partitions) - val partitionOffsets = 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 earliest offsets for new partitions: $partitionOffsets") - partitionOffsets - } - } - - /** - * Helper function that does multiple retries on the a body of code that returns offsets. - * Retries are needed to handle transient failures. For e.g. race conditions between getting - * assignment and getting position while topics/partitions are deleted can cause NPEs. - * - * This method also makes sure `body` won't be interrupted to workaround a potential issue in - * `KafkaConsumer.poll`. (KAFKA-1894) - */ - private def withRetriesWithoutInterrupt( - body: => Map[TopicPartition, Long]): Map[TopicPartition, Long] = { - // Make sure `KafkaConsumer.poll` won't be interrupted (KAFKA-1894) - assert(Thread.currentThread().isInstanceOf[StreamExecutionThread]) - - synchronized { - var result: Option[Map[TopicPartition, Long]] = None - var attempt = 1 - var lastException: Throwable = null - while (result.isEmpty && attempt <= maxOffsetFetchAttempts - && !Thread.currentThread().isInterrupted) { - Thread.currentThread match { - case ut: UninterruptibleThread => - // "KafkaConsumer.poll" may hang forever if the thread is interrupted (E.g., the query - // is stopped)(KAFKA-1894). Hence, we just make sure we don't interrupt it. - // - // If the broker addresses are wrong, or Kafka cluster is down, "KafkaConsumer.poll" may - // hang forever as well. This cannot be resolved in KafkaSource until Kafka fixes the - // issue. - ut.runUninterruptibly { - try { - result = Some(body) - } catch { - case NonFatal(e) => - lastException = e - logWarning(s"Error in attempt $attempt getting Kafka offsets: ", e) - attempt += 1 - Thread.sleep(offsetFetchAttemptIntervalMs) - resetConsumer() - } - } - case _ => - throw new IllegalStateException( - "Kafka APIs must be executed on a o.a.spark.util.UninterruptibleThread") - } - } - if (Thread.interrupted()) { - throw new InterruptedException() - } - if (result.isEmpty) { - assert(attempt > maxOffsetFetchAttempts) - assert(lastException != null) - throw lastException - } - result.get - } - } + override def toString(): String = s"KafkaSource[$kafkaReader]" /** * If `failOnDataLoss` is true, this method will throw an `IllegalStateException`. @@ -506,7 +314,6 @@ private[kafka010] class KafkaSource( } } - /** Companion object for the [[KafkaSource]]. */ private[kafka010] object KafkaSource { @@ -525,68 +332,4 @@ private[kafka010] object KafkaSource { | topic was processed. If you don't want your streaming query to fail on such cases, set the | source option "failOnDataLoss" to "false". """.stripMargin - - def kafkaSchema: StructType = StructType(Seq( - StructField("key", BinaryType), - StructField("value", BinaryType), - StructField("topic", StringType), - StructField("partition", IntegerType), - StructField("offset", LongType), - StructField("timestamp", TimestampType), - StructField("timestampType", IntegerType) - )) - - sealed trait ConsumerStrategy { - def createConsumer(kafkaParams: ju.Map[String, Object]): Consumer[Array[Byte], Array[Byte]] - } - - case class AssignStrategy(partitions: Array[TopicPartition]) extends ConsumerStrategy { - override def createConsumer( - kafkaParams: ju.Map[String, Object]): Consumer[Array[Byte], Array[Byte]] = { - val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams) - consumer.assign(ju.Arrays.asList(partitions: _*)) - consumer - } - - override def toString: String = s"Assign[${partitions.mkString(", ")}]" - } - - case class SubscribeStrategy(topics: Seq[String]) extends ConsumerStrategy { - override def createConsumer( - kafkaParams: ju.Map[String, Object]): Consumer[Array[Byte], Array[Byte]] = { - val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams) - consumer.subscribe(topics.asJava) - consumer - } - - override def toString: String = s"Subscribe[${topics.mkString(", ")}]" - } - - case class SubscribePatternStrategy(topicPattern: String) - extends ConsumerStrategy { - override def createConsumer( - kafkaParams: ju.Map[String, Object]): Consumer[Array[Byte], Array[Byte]] = { - val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams) - consumer.subscribe( - ju.regex.Pattern.compile(topicPattern), - new NoOpConsumerRebalanceListener()) - consumer - } - - override def toString: String = s"SubscribePattern[$topicPattern]" - } - - private def getSortedExecutorList(sc: SparkContext): Array[String] = { - val bm = sc.env.blockManager - bm.master.getPeers(bm.blockManagerId).toArray - .map(x => ExecutorCacheTaskLocation(x.host, x.executorId)) - .sortWith(compare) - .map(_.toString) - } - - private def compare(a: ExecutorCacheTaskLocation, b: ExecutorCacheTaskLocation): Boolean = { - if (a.host == b.host) { a.executorId > b.executorId } else { a.host > b.host } - } - - private def floorMod(a: Long, b: Int): Int = ((a % b).toInt + b) % b } 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 ff9965b854c6..5e8d794ad1e6 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 @@ -28,7 +28,7 @@ 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.kafka010.KafkaReader._ import org.apache.spark.sql.sources.{DataSourceRegister, StreamSourceProvider} import org.apache.spark.sql.types.StructType @@ -53,7 +53,7 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider 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) + ("kafka", KafkaReader.kafkaSchema) } override def createSource( @@ -139,17 +139,17 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider val failOnDataLoss = caseInsensitiveParams.getOrElse(FAIL_ON_DATA_LOSS_OPTION_KEY, "true").toBoolean + val kafkaReader = new KafkaReader(strategy, kafkaParamsForDriver, parameters, + driverGroupIdPrefix = s"$uniqueGroupId-driver") new KafkaSource( sqlContext, - strategy, - kafkaParamsForDriver, + kafkaReader, kafkaParamsForExecutors, parameters, metadataPath, startingOffsets, - failOnDataLoss, - driverGroupIdPrefix = s"$uniqueGroupId-driver") + failOnDataLoss) } private def validateOptions(parameters: Map[String, String]): Unit = { diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaUtils.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaUtils.scala new file mode 100644 index 000000000000..78bdb66655f4 --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaUtils.scala @@ -0,0 +1,38 @@ +/* + * 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.SparkContext +import org.apache.spark.scheduler.ExecutorCacheTaskLocation + +private[kafka010] object KafkaUtils { + + def getSortedExecutorList(sc: SparkContext): Array[String] = { + val bm = sc.env.blockManager + bm.master.getPeers(bm.blockManagerId).toArray + .map(x => ExecutorCacheTaskLocation(x.host, x.executorId)) + .sortWith(compare) + .map(_.toString) + } + + private def compare(a: ExecutorCacheTaskLocation, b: ExecutorCacheTaskLocation): Boolean = { + if (a.host == b.host) { a.executorId > b.executorId } else { a.host > b.host } + } + + def floorMod(a: Long, b: Int): Int = ((a % b).toInt + b) % b +} From b6c3055f2c21050cb9e203213651e02779d724bc Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Thu, 19 Jan 2017 18:29:27 -0800 Subject: [PATCH 02/40] update --- .../apache/spark/sql/kafka010/KafkaRelation.scala | 11 ++++++----- .../apache/spark/sql/kafka010/KafkaSource.scala | 14 +++++++------- 2 files changed, 13 insertions(+), 12 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala index 7be88b995daa..15a948ec4a76 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala @@ -34,12 +34,13 @@ private[kafka010] class KafkaRelation( kafkaReader: KafkaReader, executorKafkaParams: ju.Map[String, Object], sourceOptions: Map[String, String], + failOnDataLoss: Boolean, startingOffsets: Option[KafkaOffsets] = Some(EarliestOffsets), - endingOffsets: Option[KafkaOffsets] = Some(LatestOffsets), - failOnDataLoss: Boolean) - extends BaseRelation - with TableScan - with Logging { + endingOffsets: Option[KafkaOffsets] = Some(LatestOffsets)) + extends BaseRelation with TableScan with Logging { + + require(startingOffsets.get != LatestOffsets, + "Start offset not allowed to be set to latests offsets.") private val pollTimeoutMs = sourceOptions.getOrElse( "kafkaConsumer.pollTimeoutMs", 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 1fb1ebe4b7d2..74a4441ac696 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 @@ -68,13 +68,13 @@ import org.apache.spark.unsafe.types.UTF8String * and not use wrong broker addresses. */ private[kafka010] class KafkaSource( - sqlContext: SQLContext, - kafkaReader: KafkaReader, - executorKafkaParams: ju.Map[String, Object], - sourceOptions: Map[String, String], - metadataPath: String, - startingOffsets: KafkaOffsets, - failOnDataLoss: Boolean) + sqlContext: SQLContext, + kafkaReader: KafkaReader, + executorKafkaParams: ju.Map[String, Object], + sourceOptions: Map[String, String], + metadataPath: String, + startingOffsets: KafkaOffsets, + failOnDataLoss: Boolean) extends Source with Logging { private val sc = sqlContext.sparkContext From 4c81812e93907219af69dd57e0f4f5ebaf92b262 Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Fri, 20 Jan 2017 10:24:07 -0800 Subject: [PATCH 03/40] update --- ...pache.spark.sql.sources.DataSourceRegister | 1 + .../sql/kafka010/KafkaConfigOptions.scala | 248 ++++++++++++++++++ ...kaReader.scala => KafkaOffsetReader.scala} | 8 +- .../spark/sql/kafka010/KafkaRelation.scala | 23 +- .../sql/kafka010/KafkaRelationProvider.scala | 60 +++++ .../spark/sql/kafka010/KafkaSource.scala | 8 +- .../sql/kafka010/KafkaSourceProvider.scala | 223 +--------------- 7 files changed, 338 insertions(+), 233 deletions(-) create mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaConfigOptions.scala rename external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/{KafkaReader.scala => KafkaOffsetReader.scala} (97%) create mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelationProvider.scala 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 index 2f9e9fc0396d..ad0f72ff4ade 100644 --- 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 @@ -1 +1,2 @@ org.apache.spark.sql.kafka010.KafkaSourceProvider +org.apache.spark.sql.kafka010.KafkaRelationProvider diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaConfigOptions.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaConfigOptions.scala new file mode 100644 index 000000000000..86c1463e3de7 --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaConfigOptions.scala @@ -0,0 +1,248 @@ +/* + * 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.ConsumerConfig +import org.apache.kafka.common.serialization.ByteArrayDeserializer + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.kafka010.KafkaOffsetReader.{AssignStrategy, SubscribePatternStrategy, SubscribeStrategy} + +private[kafka010] class KafkaConfigOptions(parameters: Map[String, String], uniqueGroupId: String) + extends Logging { + import KafkaConfigOptions._ + validateOptions(parameters) + private val deserClassName = classOf[ByteArrayDeserializer].getName + + private val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase, v) } + private val specifiedKafkaParams = + parameters + .keySet + .filter(_.toLowerCase.startsWith("kafka.")) + .map { k => k.drop(6).toString -> parameters(k) } + .toMap + + val startingStreamOffsets = + caseInsensitiveParams.get(STARTING_OFFSETS_OPTION_KEY).map(_.trim.toLowerCase) match { + case Some("latest") => LatestOffsets + case Some("earliest") => EarliestOffsets + case Some(json) => SpecificOffsets(JsonUtils.partitionOffsets(json)) + case None => LatestOffsets + } + + val startingRelationOffsets = + caseInsensitiveParams.get(STARTING_OFFSETS_OPTION_KEY).map(_.trim.toLowerCase) match { + case Some("latest") => + throw new IllegalArgumentException("Starting relation offset can't be latest.") + case Some("earliest") => EarliestOffsets + case Some(json) => SpecificOffsets(JsonUtils.partitionOffsets(json)) + case None => EarliestOffsets + } + + val endingRelationOffsets = + caseInsensitiveParams.get(ENDING_OFFSETS_OPTION_KEY).map(_.trim.toLowerCase) match { + case Some("latest") => LatestOffsets + case Some("earliest") => + throw new IllegalArgumentException("Ending relation offset can't be earliest.") + case Some(json) => SpecificOffsets(JsonUtils.partitionOffsets(json)) + case None => LatestOffsets + } + + val kafkaParamsForDriver = + ConfigUpdater("source", specifiedKafkaParams) + .set(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, deserClassName) + .set(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, deserClassName) + + // Set to "earliest" to avoid exceptions. However, KafkaSource will fetch the initial + // offsets by itself instead of counting on KafkaConsumer. + .set(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") + + // So that consumers in the driver does not commit offsets unnecessarily + .set(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") + + // So that the driver does not pull too much data + .set(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, new java.lang.Integer(1)) + + // If buffer config is not set, set it to reasonable value to work around + // buffer issues (see KAFKA-3135) + .setIfUnset(ConsumerConfig.RECEIVE_BUFFER_CONFIG, 65536: java.lang.Integer) + .build() + + val kafkaParamsForExecutors = + ConfigUpdater("executor", specifiedKafkaParams) + .set(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, deserClassName) + .set(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, deserClassName) + + // Make sure executors do only what the driver tells them. + .set(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none") + + // So that consumers in executors do not mess with any existing group id + .set(ConsumerConfig.GROUP_ID_CONFIG, s"$uniqueGroupId-executor") + + // So that consumers in executors does not commit offsets unnecessarily + .set(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") + + // If buffer config is not set, set it to reasonable value to work around + // buffer issues (see KAFKA-3135) + .setIfUnset(ConsumerConfig.RECEIVE_BUFFER_CONFIG, 65536: java.lang.Integer) + .build() + + val strategy = caseInsensitiveParams.find(x => STRATEGY_OPTION_KEYS.contains(x._1)).get match { + case ("assign", value) => + AssignStrategy(JsonUtils.partitions(value)) + case ("subscribe", value) => + SubscribeStrategy(value.split(",").map(_.trim()).filter(_.nonEmpty)) + case ("subscribepattern", value) => + SubscribePatternStrategy(value.trim()) + case _ => + // Should never reach here as we are already matching on + // matched strategy names + throw new IllegalArgumentException("Unknown option") + } + + val failOnDataLoss = + caseInsensitiveParams.getOrElse(FAIL_ON_DATA_LOSS_OPTION_KEY, "true").toBoolean + private def validateOptions(parameters: Map[String, String]): Unit = { + // Validate source options + val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase, v) } + val specifiedStrategies = + caseInsensitiveParams.filter { case (k, _) => STRATEGY_OPTION_KEYS.contains(k) }.toSeq + if (specifiedStrategies.isEmpty) { + throw new IllegalArgumentException( + "One of the following options must be specified for Kafka source: " + + STRATEGY_OPTION_KEYS.mkString(", ") + ". See the docs for more details.") + } else if (specifiedStrategies.size > 1) { + throw new IllegalArgumentException( + "Only one of the following options can be specified for Kafka source: " + + STRATEGY_OPTION_KEYS.mkString(", ") + ". See the docs for more details.") + } + + val strategy = caseInsensitiveParams.find(x => STRATEGY_OPTION_KEYS.contains(x._1)).get match { + case ("assign", value) => + if (!value.trim.startsWith("{")) { + throw new IllegalArgumentException( + "No topicpartitions to assign as specified value for option " + + s"'assign' is '$value'") + } + + 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") + } + + // 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_OFFSETS_OPTION_KEY' to 'earliest' or 'latest' + |to specify where to start. Structured Streaming manages which offsets are consumed + |internally, rather than relying on the kafkaConsumer to do it. This will ensure that no + |data is missed when new topics/partitions are dynamically subscribed. Note that + |'$STARTING_OFFSETS_OPTION_KEY' only applies when a new Streaming query is started, and + |that resuming will always pick up from where the query left off. See the docs for more + |details. + """.stripMargin) + } + + if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG}")) { + throw new IllegalArgumentException( + s"Kafka option '${ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG}' is not supported as keys " + + "are deserialized as byte arrays with ByteArrayDeserializer. Use DataFrame operations " + + "to explicitly deserialize the keys.") + } + + if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG}")) + { + throw new IllegalArgumentException( + s"Kafka option '${ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG}' is not supported as " + + "value are deserialized as byte arrays with ByteArrayDeserializer. Use DataFrame " + + "operations to explicitly deserialize the values.") + } + + val otherUnsupportedConfigs = Seq( + ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, // committing correctly requires new APIs in Source + ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG) // interceptors can modify payload, so not safe + + otherUnsupportedConfigs.foreach { c => + if (caseInsensitiveParams.contains(s"kafka.$c")) { + throw new IllegalArgumentException(s"Kafka option '$c' is not supported") + } + } + + if (!caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG}")) { + throw new IllegalArgumentException( + s"Option 'kafka.${ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG}' must be specified for " + + s"configuring Kafka consumer") + } + } + + /** Class to conveniently update Kafka config params, while logging the changes */ + private case class ConfigUpdater(module: String, kafkaParams: Map[String, String]) { + private val map = new ju.HashMap[String, Object](kafkaParams.asJava) + + def set(key: String, value: Object): this.type = { + map.put(key, value) + logInfo(s"$module: Set $key to $value, earlier value: ${kafkaParams.get(key).getOrElse("")}") + this + } + + def setIfUnset(key: String, value: Object): ConfigUpdater = { + if (!map.containsKey(key)) { + map.put(key, value) + logInfo(s"$module: Set $key to $value") + } + this + } + + def build(): ju.Map[String, Object] = map + } +} + +private[kafka010] object KafkaConfigOptions { + private val STRATEGY_OPTION_KEYS = Set("subscribe", "subscribepattern", "assign") + private val STARTING_OFFSETS_OPTION_KEY = "startingoffsets" + private val ENDING_OFFSETS_OPTION_KEY = "endingoffsets" + private val FAIL_ON_DATA_LOSS_OPTION_KEY = "failondataloss" +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaReader.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala similarity index 97% rename from external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaReader.scala rename to external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala index 93190e252272..ecee47ae1fda 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala @@ -27,7 +27,7 @@ import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener import org.apache.kafka.common.TopicPartition import org.apache.spark.internal.Logging -import org.apache.spark.sql.kafka010.KafkaReader.ConsumerStrategy +import org.apache.spark.sql.kafka010.KafkaOffsetReader.ConsumerStrategy import org.apache.spark.sql.types._ import org.apache.spark.util.UninterruptibleThread @@ -38,9 +38,9 @@ import org.apache.spark.util.UninterruptibleThread * 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.KafkaReader.ConsumerStrategy]] for more details. + * [[org.apache.spark.sql.kafka010.KafkaOffsetReader.ConsumerStrategy]] for more details. */ -private[kafka010] class KafkaReader( +private[kafka010] class KafkaOffsetReader( consumerStrategy: ConsumerStrategy, driverKafkaParams: ju.Map[String, Object], readerOptions: Map[String, String], @@ -232,7 +232,7 @@ private[kafka010] class KafkaReader( } } -private[kafka010] object KafkaReader { +private[kafka010] object KafkaOffsetReader { def kafkaSchema: StructType = StructType(Seq( StructField("key", BinaryType), diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala index 15a948ec4a76..c4ade2476e66 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala @@ -21,9 +21,9 @@ import java.{util => ju} import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{Row, SQLContext} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.{Row, SQLContext} import org.apache.spark.sql.sources.{BaseRelation, TableScan} import org.apache.spark.sql.types.StructType import org.apache.spark.unsafe.types.UTF8String @@ -31,28 +31,30 @@ import org.apache.spark.unsafe.types.UTF8String private[kafka010] class KafkaRelation( override val sqlContext: SQLContext, - kafkaReader: KafkaReader, + kafkaReader: KafkaOffsetReader, executorKafkaParams: ju.Map[String, Object], sourceOptions: Map[String, String], failOnDataLoss: Boolean, - startingOffsets: Option[KafkaOffsets] = Some(EarliestOffsets), - endingOffsets: Option[KafkaOffsets] = Some(LatestOffsets)) + startingOffsets: KafkaOffsets, + endingOffsets: KafkaOffsets) extends BaseRelation with TableScan with Logging { - require(startingOffsets.get != LatestOffsets, - "Start offset not allowed to be set to latests offsets.") + require(startingOffsets != LatestOffsets, + "Starting offset not allowed to be set to latest offsets.") + require(endingOffsets != EarliestOffsets, + "Ending offset not allowed to be set to earliest offsets.") private val pollTimeoutMs = sourceOptions.getOrElse( "kafkaConsumer.pollTimeoutMs", sqlContext.sparkContext.conf.getTimeAsMs("spark.network.timeout", "120s").toString ).toLong - override def schema: StructType = KafkaReader.kafkaSchema + override def schema: StructType = KafkaOffsetReader.kafkaSchema override def buildScan(): RDD[Row] = { // Leverage the KafkaReader to obtain the relevant partition offsets - val fromPartitionOffsets = getPartitionOffsets(startingOffsets.getOrElse(EarliestOffsets)) - val untilPartitionOffsets = getPartitionOffsets(endingOffsets.getOrElse(LatestOffsets)) + val fromPartitionOffsets = getPartitionOffsets(startingOffsets) + val untilPartitionOffsets = getPartitionOffsets(endingOffsets) // Obtain topicPartitions in both from and until partition offset, ignoring // topic partitions that were added and/or deleted between the two above calls. val topicPartitions = fromPartitionOffsets.keySet.intersect(untilPartitionOffsets.keySet) @@ -83,7 +85,8 @@ private[kafka010] class KafkaRelation( // Create an RDD that reads from Kafka and get the (key, value) pair as byte arrays. val rdd = new KafkaSourceRDD( - sqlContext.sparkContext, executorKafkaParams, offsetRanges, pollTimeoutMs, failOnDataLoss).map { cr => + sqlContext.sparkContext, executorKafkaParams, offsetRanges, + pollTimeoutMs, failOnDataLoss).map { cr => InternalRow( cr.key, cr.value, diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelationProvider.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelationProvider.scala new file mode 100644 index 000000000000..f55bf98adbc1 --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelationProvider.scala @@ -0,0 +1,60 @@ +/* + * 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.UUID + +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.sources.{BaseRelation, DataSourceRegister, RelationProvider} + +class KafkaRelationProvider extends RelationProvider with DataSourceRegister { + + /** + * The string that represents the format that this data source provider uses. This is + * overridden by children to provide a nice alias for the data source. For example: + */ + override def shortName(): String = "kafka" + + /** + * Returns a new base relation with the given parameters. + * + * @note The parameters' keywords are case insensitive and this insensitivity is enforced + * by the Map that is passed to the function. + */ + override def createRelation( + sqlContext: SQLContext, + parameters: Map[String, String]): BaseRelation = { + // 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-relation-${UUID.randomUUID}" + val kafkaConfigOptions = new KafkaConfigOptions(parameters, uniqueGroupId) + val kafkaOffsetReader = new KafkaOffsetReader(kafkaConfigOptions.strategy, + kafkaConfigOptions.kafkaParamsForDriver, parameters, + driverGroupIdPrefix = s"$uniqueGroupId-driver") + + new KafkaRelation( + sqlContext, + kafkaOffsetReader, + kafkaConfigOptions.kafkaParamsForExecutors, + parameters, + kafkaConfigOptions.failOnDataLoss, + kafkaConfigOptions.startingRelationOffsets, + kafkaConfigOptions.endingRelationOffsets) + } +} 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 74a4441ac696..80842a568622 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 @@ -44,11 +44,11 @@ import org.apache.spark.unsafe.types.UTF8String * * - The [[KafkaSource]] written to do the following. * - * - As soon as the source is created, the pre-configured [[KafkaReader]] + * - As soon as the source is created, the pre-configured [[KafkaOffsetReader]] * is used to query the initial offsets that this source should * start reading from. This is used to create the first batch. * - * - `getOffset()` uses the [[KafkaReader]] to query the latest available offsets, which are + * - `getOffset()` uses the [[KafkaOffsetReader]] 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 @@ -69,7 +69,7 @@ import org.apache.spark.unsafe.types.UTF8String */ private[kafka010] class KafkaSource( sqlContext: SQLContext, - kafkaReader: KafkaReader, + kafkaReader: KafkaOffsetReader, executorKafkaParams: ju.Map[String, Object], sourceOptions: Map[String, String], metadataPath: String, @@ -137,7 +137,7 @@ private[kafka010] class KafkaSource( private var currentPartitionOffsets: Option[Map[TopicPartition, Long]] = None - override def schema: StructType = KafkaReader.kafkaSchema + override def schema: StructType = KafkaOffsetReader.kafkaSchema /** Returns the maximum available offset for this source. */ override def getOffset: Option[Offset] = { 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 5e8d794ad1e6..7e0ea9257acc 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 @@ -17,18 +17,11 @@ 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.KafkaReader._ import org.apache.spark.sql.sources.{DataSourceRegister, StreamSourceProvider} import org.apache.spark.sql.types.StructType @@ -40,8 +33,6 @@ import org.apache.spark.sql.types.StructType private[kafka010] class KafkaSourceProvider extends StreamSourceProvider with DataSourceRegister with Logging { - import KafkaSourceProvider._ - /** * Returns the name and schema of the source. In addition, it also verifies whether the options * are correct and sufficient to create the [[KafkaSource]] when the query is started. @@ -52,8 +43,7 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider 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", KafkaReader.kafkaSchema) + (shortName(), KafkaOffsetReader.kafkaSchema) } override def createSource( @@ -62,221 +52,24 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider schema: Option[StructType], providerName: String, parameters: Map[String, String]): Source = { - validateOptions(parameters) - val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase, v) } - val specifiedKafkaParams = - parameters - .keySet - .filter(_.toLowerCase.startsWith("kafka.")) - .map { k => k.drop(6).toString -> parameters(k) } - .toMap - - val deserClassName = classOf[ByteArrayDeserializer].getName // Each running query should use its own group id. Otherwise, the query may be only assigned // partial data since Kafka will assign partitions to multiple consumers having the same group // id. Hence, we should generate a unique id for each query. val uniqueGroupId = s"spark-kafka-source-${UUID.randomUUID}-${metadataPath.hashCode}" - - val startingOffsets = - caseInsensitiveParams.get(STARTING_OFFSETS_OPTION_KEY).map(_.trim.toLowerCase) match { - case Some("latest") => LatestOffsets - case Some("earliest") => EarliestOffsets - case Some(json) => SpecificOffsets(JsonUtils.partitionOffsets(json)) - case None => LatestOffsets - } - - val kafkaParamsForDriver = - ConfigUpdater("source", specifiedKafkaParams) - .set(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, deserClassName) - .set(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, deserClassName) - - // Set to "earliest" to avoid exceptions. However, KafkaSource will fetch the initial - // offsets by itself instead of counting on KafkaConsumer. - .set(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") - - // So that consumers in the driver does not commit offsets unnecessarily - .set(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") - - // So that the driver does not pull too much data - .set(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, new java.lang.Integer(1)) - - // If buffer config is not set, set it to reasonable value to work around - // buffer issues (see KAFKA-3135) - .setIfUnset(ConsumerConfig.RECEIVE_BUFFER_CONFIG, 65536: java.lang.Integer) - .build() - - val kafkaParamsForExecutors = - ConfigUpdater("executor", specifiedKafkaParams) - .set(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, deserClassName) - .set(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, deserClassName) - - // Make sure executors do only what the driver tells them. - .set(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none") - - // So that consumers in executors do not mess with any existing group id - .set(ConsumerConfig.GROUP_ID_CONFIG, s"$uniqueGroupId-executor") - - // So that consumers in executors does not commit offsets unnecessarily - .set(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") - - // If buffer config is not set, set it to reasonable value to work around - // buffer issues (see KAFKA-3135) - .setIfUnset(ConsumerConfig.RECEIVE_BUFFER_CONFIG, 65536: java.lang.Integer) - .build() - - val strategy = caseInsensitiveParams.find(x => STRATEGY_OPTION_KEYS.contains(x._1)).get match { - case ("assign", value) => - AssignStrategy(JsonUtils.partitions(value)) - case ("subscribe", value) => - SubscribeStrategy(value.split(",").map(_.trim()).filter(_.nonEmpty)) - case ("subscribepattern", value) => - SubscribePatternStrategy(value.trim()) - case _ => - // Should never reach here as we are already matching on - // matched strategy names - throw new IllegalArgumentException("Unknown option") - } - - val failOnDataLoss = - caseInsensitiveParams.getOrElse(FAIL_ON_DATA_LOSS_OPTION_KEY, "true").toBoolean - val kafkaReader = new KafkaReader(strategy, kafkaParamsForDriver, parameters, + val kafkaConfigOptions = new KafkaConfigOptions(parameters, uniqueGroupId) + val kafkaOffsetReader = new KafkaOffsetReader(kafkaConfigOptions.strategy, + kafkaConfigOptions.kafkaParamsForDriver, parameters, driverGroupIdPrefix = s"$uniqueGroupId-driver") new KafkaSource( sqlContext, - kafkaReader, - kafkaParamsForExecutors, + kafkaOffsetReader, + kafkaConfigOptions.kafkaParamsForExecutors, parameters, metadataPath, - startingOffsets, - failOnDataLoss) - } - - private def validateOptions(parameters: Map[String, String]): Unit = { - - // Validate source options - - val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase, v) } - val specifiedStrategies = - caseInsensitiveParams.filter { case (k, _) => STRATEGY_OPTION_KEYS.contains(k) }.toSeq - if (specifiedStrategies.isEmpty) { - throw new IllegalArgumentException( - "One of the following options must be specified for Kafka source: " - + STRATEGY_OPTION_KEYS.mkString(", ") + ". See the docs for more details.") - } else if (specifiedStrategies.size > 1) { - throw new IllegalArgumentException( - "Only one of the following options can be specified for Kafka source: " - + STRATEGY_OPTION_KEYS.mkString(", ") + ". See the docs for more details.") - } - - val strategy = caseInsensitiveParams.find(x => STRATEGY_OPTION_KEYS.contains(x._1)).get match { - case ("assign", value) => - if (!value.trim.startsWith("{")) { - throw new IllegalArgumentException( - "No topicpartitions to assign as specified value for option " + - s"'assign' is '$value'") - } - - 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") - } - - // 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_OFFSETS_OPTION_KEY' to 'earliest' or 'latest' - |to specify where to start. Structured Streaming manages which offsets are consumed - |internally, rather than relying on the kafkaConsumer to do it. This will ensure that no - |data is missed when new topics/partitions are dynamically subscribed. Note that - |'$STARTING_OFFSETS_OPTION_KEY' only applies when a new Streaming query is started, and - |that resuming will always pick up from where the query left off. See the docs for more - |details. - """.stripMargin) - } - - if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG}")) { - throw new IllegalArgumentException( - s"Kafka option '${ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG}' is not supported as keys " - + "are deserialized as byte arrays with ByteArrayDeserializer. Use DataFrame operations " - + "to explicitly deserialize the keys.") - } - - if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG}")) - { - throw new IllegalArgumentException( - s"Kafka option '${ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG}' is not supported as " - + "value are deserialized as byte arrays with ByteArrayDeserializer. Use DataFrame " - + "operations to explicitly deserialize the values.") - } - - val otherUnsupportedConfigs = Seq( - ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, // committing correctly requires new APIs in Source - ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG) // interceptors can modify payload, so not safe - - otherUnsupportedConfigs.foreach { c => - if (caseInsensitiveParams.contains(s"kafka.$c")) { - throw new IllegalArgumentException(s"Kafka option '$c' is not supported") - } - } - - if (!caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG}")) { - throw new IllegalArgumentException( - s"Option 'kafka.${ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG}' must be specified for " + - s"configuring Kafka consumer") - } + kafkaConfigOptions.startingStreamOffsets, + kafkaConfigOptions.failOnDataLoss) } override def shortName(): String = "kafka" - - /** Class to conveniently update Kafka config params, while logging the changes */ - private case class ConfigUpdater(module: String, kafkaParams: Map[String, String]) { - private val map = new ju.HashMap[String, Object](kafkaParams.asJava) - - def set(key: String, value: Object): this.type = { - map.put(key, value) - logInfo(s"$module: Set $key to $value, earlier value: ${kafkaParams.get(key).getOrElse("")}") - this - } - - def setIfUnset(key: String, value: Object): ConfigUpdater = { - if (!map.containsKey(key)) { - map.put(key, value) - logInfo(s"$module: Set $key to $value") - } - this - } - - def build(): ju.Map[String, Object] = map - } -} - -private[kafka010] object KafkaSourceProvider { - private val STRATEGY_OPTION_KEYS = Set("subscribe", "subscribepattern", "assign") - private val STARTING_OFFSETS_OPTION_KEY = "startingoffsets" - private val FAIL_ON_DATA_LOSS_OPTION_KEY = "failondataloss" } From ab02a4c631f9fc0ecd8528d85c61fe3c5de64040 Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Fri, 20 Jan 2017 11:53:24 -0800 Subject: [PATCH 04/40] single kafka provider for both stream and batch --- ...pache.spark.sql.sources.DataSourceRegister | 3 +- ...onfigOptions.scala => KafkaProvider.scala} | 176 ++++++++++++++---- .../spark/sql/kafka010/KafkaRelation.scala | 3 + .../sql/kafka010/KafkaRelationProvider.scala | 60 ------ .../sql/kafka010/KafkaSourceProvider.scala | 75 -------- .../sql/kafka010/KafkaRelationSuite.scala | 77 ++++++++ .../spark/sql/kafka010/KafkaSourceSuite.scala | 4 +- 7 files changed, 219 insertions(+), 179 deletions(-) rename external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/{KafkaConfigOptions.scala => KafkaProvider.scala} (62%) delete mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelationProvider.scala delete mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala create mode 100644 external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala 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 index ad0f72ff4ade..106a211d6ae4 100644 --- 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 @@ -1,2 +1 @@ -org.apache.spark.sql.kafka010.KafkaSourceProvider -org.apache.spark.sql.kafka010.KafkaRelationProvider +org.apache.spark.sql.kafka010.KafkaProvider diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaConfigOptions.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaProvider.scala similarity index 62% rename from external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaConfigOptions.scala rename to external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaProvider.scala index 86c1463e3de7..07f65d2f092d 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaConfigOptions.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaProvider.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.kafka010 import java.{util => ju} +import java.util.UUID import scala.collection.JavaConverters._ @@ -25,49 +26,140 @@ 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.KafkaOffsetReader.{AssignStrategy, SubscribePatternStrategy, SubscribeStrategy} +import org.apache.spark.sql.sources._ +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 KafkaProvider extends DataSourceRegister with StreamSourceProvider + with RelationProvider with Logging { + import KafkaProvider._ -private[kafka010] class KafkaConfigOptions(parameters: Map[String, String], uniqueGroupId: String) - extends Logging { - import KafkaConfigOptions._ - validateOptions(parameters) private val deserClassName = classOf[ByteArrayDeserializer].getName - private val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase, v) } - private val specifiedKafkaParams = - parameters - .keySet - .filter(_.toLowerCase.startsWith("kafka.")) - .map { k => k.drop(6).toString -> parameters(k) } - .toMap - - val startingStreamOffsets = - caseInsensitiveParams.get(STARTING_OFFSETS_OPTION_KEY).map(_.trim.toLowerCase) match { - case Some("latest") => LatestOffsets - case Some("earliest") => EarliestOffsets - case Some(json) => SpecificOffsets(JsonUtils.partitionOffsets(json)) - case None => LatestOffsets - } + override def shortName(): String = "kafka" + + /** + * 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) + require(schema.isEmpty, "Kafka source has a fixed schema and cannot be set with a custom one") + (shortName(), KafkaOffsetReader.kafkaSchema) + } - val startingRelationOffsets = - caseInsensitiveParams.get(STARTING_OFFSETS_OPTION_KEY).map(_.trim.toLowerCase) match { - case Some("latest") => - throw new IllegalArgumentException("Starting relation offset can't be latest.") - case Some("earliest") => EarliestOffsets - case Some(json) => SpecificOffsets(JsonUtils.partitionOffsets(json)) - case None => EarliestOffsets - } + override def createSource( + sqlContext: SQLContext, + metadataPath: String, + schema: Option[StructType], + providerName: String, + parameters: Map[String, String]): Source = { + validateOptions(parameters) + // 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 endingRelationOffsets = - caseInsensitiveParams.get(ENDING_OFFSETS_OPTION_KEY).map(_.trim.toLowerCase) match { - case Some("latest") => LatestOffsets - case Some("earliest") => - throw new IllegalArgumentException("Ending relation offset can't be earliest.") - case Some(json) => SpecificOffsets(JsonUtils.partitionOffsets(json)) - case None => LatestOffsets - } + 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 startingStreamOffsets = + caseInsensitiveParams.get(STARTING_OFFSETS_OPTION_KEY).map(_.trim.toLowerCase) match { + case Some("latest") => LatestOffsets + case Some("earliest") => EarliestOffsets + case Some(json) => SpecificOffsets(JsonUtils.partitionOffsets(json)) + case None => LatestOffsets + } + + val kafkaOffsetReader = new KafkaOffsetReader( + strategy(caseInsensitiveParams), + kafkaParamsForDriver(specifiedKafkaParams), + parameters, + driverGroupIdPrefix = s"$uniqueGroupId-driver") + + new KafkaSource( + sqlContext, + kafkaOffsetReader, + kafkaParamsForExecutors(specifiedKafkaParams, uniqueGroupId), + parameters, + metadataPath, + startingStreamOffsets, + failOnDataLoss(caseInsensitiveParams)) + } - val kafkaParamsForDriver = + /** + * Returns a new base relation with the given parameters. + * + * @note The parameters' keywords are case insensitive and this insensitivity is enforced + * by the Map that is passed to the function. + */ + override def createRelation( + sqlContext: SQLContext, + parameters: Map[String, String]): BaseRelation = { + validateOptions(parameters) + // 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-relation-${UUID.randomUUID}" + 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 startingRelationOffsets = + caseInsensitiveParams.get(STARTING_OFFSETS_OPTION_KEY).map(_.trim.toLowerCase) match { + case Some("latest") => + throw new IllegalArgumentException("Starting relation offset can't be latest.") + case Some("earliest") => EarliestOffsets + case Some(json) => SpecificOffsets(JsonUtils.partitionOffsets(json)) + case None => EarliestOffsets + } + + val endingRelationOffsets = + caseInsensitiveParams.get(ENDING_OFFSETS_OPTION_KEY).map(_.trim.toLowerCase) match { + case Some("latest") => LatestOffsets + case Some("earliest") => + throw new IllegalArgumentException("Ending relation offset can't be earliest.") + case Some(json) => SpecificOffsets(JsonUtils.partitionOffsets(json)) + case None => LatestOffsets + } + + val kafkaOffsetReader = new KafkaOffsetReader( + strategy(caseInsensitiveParams), + kafkaParamsForDriver(specifiedKafkaParams), + parameters, + driverGroupIdPrefix = s"$uniqueGroupId-driver") + + new KafkaRelation( + sqlContext, + kafkaOffsetReader, + kafkaParamsForExecutors(specifiedKafkaParams, uniqueGroupId), + parameters, + failOnDataLoss(caseInsensitiveParams), + startingRelationOffsets, + endingRelationOffsets) + } + + private def kafkaParamsForDriver(specifiedKafkaParams: Map[String, String]) = ConfigUpdater("source", specifiedKafkaParams) .set(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, deserClassName) .set(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, deserClassName) @@ -87,7 +179,8 @@ private[kafka010] class KafkaConfigOptions(parameters: Map[String, String], uniq .setIfUnset(ConsumerConfig.RECEIVE_BUFFER_CONFIG, 65536: java.lang.Integer) .build() - val kafkaParamsForExecutors = + private def kafkaParamsForExecutors( + specifiedKafkaParams: Map[String, String], uniqueGroupId: String) = ConfigUpdater("executor", specifiedKafkaParams) .set(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, deserClassName) .set(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, deserClassName) @@ -106,7 +199,8 @@ private[kafka010] class KafkaConfigOptions(parameters: Map[String, String], uniq .setIfUnset(ConsumerConfig.RECEIVE_BUFFER_CONFIG, 65536: java.lang.Integer) .build() - val strategy = caseInsensitiveParams.find(x => STRATEGY_OPTION_KEYS.contains(x._1)).get match { + private def strategy(caseInsensitiveParams: Map[String, String]) = + caseInsensitiveParams.find(x => STRATEGY_OPTION_KEYS.contains(x._1)).get match { case ("assign", value) => AssignStrategy(JsonUtils.partitions(value)) case ("subscribe", value) => @@ -119,8 +213,10 @@ private[kafka010] class KafkaConfigOptions(parameters: Map[String, String], uniq throw new IllegalArgumentException("Unknown option") } - val failOnDataLoss = + private def failOnDataLoss(caseInsensitiveParams: Map[String, String]) = caseInsensitiveParams.getOrElse(FAIL_ON_DATA_LOSS_OPTION_KEY, "true").toBoolean + + private def validateOptions(parameters: Map[String, String]): Unit = { // Validate source options val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase, v) } @@ -240,7 +336,7 @@ private[kafka010] class KafkaConfigOptions(parameters: Map[String, String], uniq } } -private[kafka010] object KafkaConfigOptions { +private[kafka010] object KafkaProvider { private val STRATEGY_OPTION_KEYS = Set("subscribe", "subscribepattern", "assign") private val STARTING_OFFSETS_OPTION_KEY = "startingoffsets" private val ENDING_OFFSETS_OPTION_KEY = "endingoffsets" diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala index c4ade2476e66..c20be9e82e7d 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala @@ -44,6 +44,9 @@ private[kafka010] class KafkaRelation( require(endingOffsets != EarliestOffsets, "Ending offset not allowed to be set to earliest offsets.") + if (sourceOptions.get("maxOffsetsPerTrigger").isDefined) + logWarning("maxOffsetsPerTrigger option ignored in batch mode.") + private val pollTimeoutMs = sourceOptions.getOrElse( "kafkaConsumer.pollTimeoutMs", sqlContext.sparkContext.conf.getTimeAsMs("spark.network.timeout", "120s").toString diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelationProvider.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelationProvider.scala deleted file mode 100644 index f55bf98adbc1..000000000000 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelationProvider.scala +++ /dev/null @@ -1,60 +0,0 @@ -/* - * 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.UUID - -import org.apache.spark.sql.SQLContext -import org.apache.spark.sql.sources.{BaseRelation, DataSourceRegister, RelationProvider} - -class KafkaRelationProvider extends RelationProvider with DataSourceRegister { - - /** - * The string that represents the format that this data source provider uses. This is - * overridden by children to provide a nice alias for the data source. For example: - */ - override def shortName(): String = "kafka" - - /** - * Returns a new base relation with the given parameters. - * - * @note The parameters' keywords are case insensitive and this insensitivity is enforced - * by the Map that is passed to the function. - */ - override def createRelation( - sqlContext: SQLContext, - parameters: Map[String, String]): BaseRelation = { - // 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-relation-${UUID.randomUUID}" - val kafkaConfigOptions = new KafkaConfigOptions(parameters, uniqueGroupId) - val kafkaOffsetReader = new KafkaOffsetReader(kafkaConfigOptions.strategy, - kafkaConfigOptions.kafkaParamsForDriver, parameters, - driverGroupIdPrefix = s"$uniqueGroupId-driver") - - new KafkaRelation( - sqlContext, - kafkaOffsetReader, - kafkaConfigOptions.kafkaParamsForExecutors, - parameters, - kafkaConfigOptions.failOnDataLoss, - kafkaConfigOptions.startingRelationOffsets, - kafkaConfigOptions.endingRelationOffsets) - } -} 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 deleted file mode 100644 index 7e0ea9257acc..000000000000 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala +++ /dev/null @@ -1,75 +0,0 @@ -/* - * 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.UUID - -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.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 { - - /** - * Returns the name and schema of the source. In addition, it also verifies whether the options - * are correct and sufficient to create the [[KafkaSource]] when the query is started. - */ - override def sourceSchema( - sqlContext: SQLContext, - schema: Option[StructType], - providerName: String, - parameters: Map[String, String]): (String, StructType) = { - require(schema.isEmpty, "Kafka source has a fixed schema and cannot be set with a custom one") - (shortName(), KafkaOffsetReader.kafkaSchema) - } - - override def createSource( - sqlContext: SQLContext, - metadataPath: String, - schema: Option[StructType], - providerName: String, - parameters: Map[String, String]): Source = { - // 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 kafkaConfigOptions = new KafkaConfigOptions(parameters, uniqueGroupId) - val kafkaOffsetReader = new KafkaOffsetReader(kafkaConfigOptions.strategy, - kafkaConfigOptions.kafkaParamsForDriver, parameters, - driverGroupIdPrefix = s"$uniqueGroupId-driver") - - new KafkaSource( - sqlContext, - kafkaOffsetReader, - kafkaConfigOptions.kafkaParamsForExecutors, - parameters, - metadataPath, - kafkaConfigOptions.startingStreamOffsets, - kafkaConfigOptions.failOnDataLoss) - } - - override def shortName(): String = "kafka" -} diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala new file mode 100644 index 000000000000..08cddb111b58 --- /dev/null +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala @@ -0,0 +1,77 @@ +/* + * 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 org.apache.kafka.common.TopicPartition +import org.scalatest.BeforeAndAfter + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.test.SharedSQLContext + +class KafkaRelationSuite extends SparkFunSuite with BeforeAndAfter with SharedSQLContext { + + import testImplicits._ + + private val topicId = new AtomicInteger(0) + + private var testUtils: KafkaTestUtils = _ + + private def newTopic(): String = s"topic-${topicId.getAndIncrement()}" + + private def assignString(topic: String, partitions: Iterable[Int]): String = { + JsonUtils.partitions(partitions.map(p => new TopicPartition(topic, p))) + } + + 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("maxOffsetsPerTrigger") { + val topic = newTopic() + testUtils.createTopic(topic, partitions = 3) + testUtils.sendMessages(topic, (100 to 200).map(_.toString).toArray, Some(0)) + testUtils.sendMessages(topic, (10 to 20).map(_.toString).toArray, Some(1)) + testUtils.sendMessages(topic, Array("1"), Some(2)) + + val reader = spark + .read + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("subscribe", topic) + .option("startingOffsets", "earliest") + .option("endingOffsets", "latest") + + val kafka = reader.load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + val mapped: org.apache.spark.sql.Dataset[_] = kafka.map(kv => kv._2.toInt) + mapped.collect().foreach(println) + } +} 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 544fbc5ec36a..35a05c7031be 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 @@ -689,7 +689,7 @@ class KafkaSourceSuite extends KafkaSourceTest { val reader = spark.readStream reader - .format(classOf[KafkaSourceProvider].getCanonicalName.stripSuffix("$")) + .format(classOf[KafkaProvider].getCanonicalName.stripSuffix("$")) .option("startingOffsets", s"earliest") .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.metadata.max.age.ms", "1") @@ -751,7 +751,7 @@ class KafkaSourceStressSuite extends KafkaSourceTest { // Create Kafka source that reads from latest offset val kafka = spark.readStream - .format(classOf[KafkaSourceProvider].getCanonicalName.stripSuffix("$")) + .format(classOf[KafkaProvider].getCanonicalName.stripSuffix("$")) .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.metadata.max.age.ms", "1") .option("subscribePattern", "stress.*") From e6b57edb0958649062749cb9a0f7cda74f0b2829 Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Mon, 23 Jan 2017 16:32:56 -0800 Subject: [PATCH 05/40] added uninterruptible thread version of kafka offset reader --- .../sql/kafka010/KafkaOffsetReader.scala | 96 ++++++++++++++++++- .../spark/sql/kafka010/KafkaProvider.scala | 64 +++++++++++-- .../spark/sql/kafka010/KafkaRelation.scala | 3 - .../sql/kafka010/KafkaRelationSuite.scala | 50 ++++++++-- .../spark/sql/kafka010/KafkaSourceSuite.scala | 3 + 5 files changed, 191 insertions(+), 25 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala index ecee47ae1fda..b2ea85b13dd0 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala @@ -18,8 +18,11 @@ package org.apache.spark.sql.kafka010 import java.{util => ju} +import java.util.concurrent.{Executor, LinkedBlockingQueue} import scala.collection.JavaConverters._ +import scala.concurrent.{ExecutionContext, Future} +import scala.concurrent.duration.Duration import scala.util.control.NonFatal import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, KafkaConsumer} @@ -29,7 +32,23 @@ import org.apache.kafka.common.TopicPartition import org.apache.spark.internal.Logging import org.apache.spark.sql.kafka010.KafkaOffsetReader.ConsumerStrategy import org.apache.spark.sql.types._ -import org.apache.spark.util.UninterruptibleThread +import org.apache.spark.util.{ThreadUtils, UninterruptibleThread} + + +private[kafka010] trait KafkaOffsetReader { + + def close() + + def fetchSpecificStartingOffsets( + partitionOffsets: Map[TopicPartition, Long]): Map[TopicPartition, Long] + + def fetchEarliestOffsets(): Map[TopicPartition, Long] + + def fetchLatestOffsets(): Map[TopicPartition, Long] + + def fetchNewPartitionEarliestOffsets( + newPartitions: Seq[TopicPartition]): Map[TopicPartition, Long] +} /** * This class uses Kafka's own [[KafkaConsumer]] API to read data offsets from Kafka. @@ -40,12 +59,12 @@ import org.apache.spark.util.UninterruptibleThread * [[KafkaSource]] to query for the offsets. See the docs on * [[org.apache.spark.sql.kafka010.KafkaOffsetReader.ConsumerStrategy]] for more details. */ -private[kafka010] class KafkaOffsetReader( +private[kafka010] class KafkaOffsetReaderImpl( consumerStrategy: ConsumerStrategy, driverKafkaParams: ju.Map[String, Object], readerOptions: Map[String, String], driverGroupIdPrefix: String) - extends Logging { + extends KafkaOffsetReader with Logging { /** * A KafkaConsumer used in the driver to query the latest Kafka offsets. This only queries the @@ -232,6 +251,77 @@ private[kafka010] class KafkaOffsetReader( } } +/** + * The Kafka Consumer must be called in an UninterruptibleThread. This naturally occurs + * in Spark Streaming, but not in Spark SQL, which will use this call to communicate + * with Kafak for obtaining offsets. + * + * @param kafkaOffsetReader Basically in instance of [[KafkaOffsetReaderImpl]] that + * this class wraps and executes in an [[UninterruptibleThread]] + */ +private[kafka010] class UninterruptibleKafkaOffsetReader(kafkaOffsetReader: KafkaOffsetReader) + extends KafkaOffsetReader with Logging { + + private class KafkaOffsetReaderThread extends UninterruptibleThread("Kafka Offset Reader") { + override def run(): Unit = { + while (this.isInterrupted == false) { + val runnable = queue.take() + runnable.run() + } + } + } + private val readerThread = new KafkaOffsetReaderThread + + private val queue = new LinkedBlockingQueue[Runnable]() + + private val execContext = ExecutionContext.fromExecutor(new Executor { + override def execute(runnable: Runnable): Unit = { + if (readerThread.isAlive == false) readerThread.start() + queue.add(runnable) + } + }) + + + override def close(): Unit = { + kafkaOffsetReader.close() + readerThread.interrupt() + queue.add(new Runnable() { + override def run(): Unit = { } + }) + } + + override def fetchSpecificStartingOffsets( + partitionOffsets: Map[TopicPartition, Long]): Map[TopicPartition, Long] = { + val future = Future { + kafkaOffsetReader.fetchSpecificStartingOffsets(partitionOffsets) + }(execContext) + ThreadUtils.awaitResult(future, Duration.Inf) + } + + override def fetchEarliestOffsets(): Map[TopicPartition, Long] = { + val future = Future { + kafkaOffsetReader.fetchEarliestOffsets() + }(execContext) + ThreadUtils.awaitResult(future, Duration.Inf) + } + + override def fetchLatestOffsets(): Map[TopicPartition, Long] = { + val future = Future { + kafkaOffsetReader.fetchLatestOffsets() + }(execContext) + ThreadUtils.awaitResult(future, Duration.Inf) + } + + override def fetchNewPartitionEarliestOffsets( + newPartitions: Seq[TopicPartition]): Map[TopicPartition, Long] = { + val future = Future { + kafkaOffsetReader.fetchNewPartitionEarliestOffsets(newPartitions) + }(execContext) + ThreadUtils.awaitResult(future, Duration.Inf) + + } +} + private[kafka010] object KafkaOffsetReader { def kafkaSchema: StructType = StructType(Seq( diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaProvider.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaProvider.scala index 07f65d2f092d..926d00797de0 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaProvider.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaProvider.scala @@ -41,6 +41,11 @@ private[kafka010] class KafkaProvider extends DataSourceRegister with StreamSour with RelationProvider with Logging { import KafkaProvider._ + // Used to check parameters for different source modes + private sealed trait Mode + private case object Batch extends Mode + private case object Stream extends Mode + private val deserClassName = classOf[ByteArrayDeserializer].getName override def shortName(): String = "kafka" @@ -54,7 +59,7 @@ private[kafka010] class KafkaProvider extends DataSourceRegister with StreamSour schema: Option[StructType], providerName: String, parameters: Map[String, String]): (String, StructType) = { - validateOptions(parameters) + validateOptions(parameters, Stream) require(schema.isEmpty, "Kafka source has a fixed schema and cannot be set with a custom one") (shortName(), KafkaOffsetReader.kafkaSchema) } @@ -65,7 +70,7 @@ private[kafka010] class KafkaProvider extends DataSourceRegister with StreamSour schema: Option[StructType], providerName: String, parameters: Map[String, String]): Source = { - validateOptions(parameters) + validateOptions(parameters, Stream) // 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. @@ -87,7 +92,7 @@ private[kafka010] class KafkaProvider extends DataSourceRegister with StreamSour case None => LatestOffsets } - val kafkaOffsetReader = new KafkaOffsetReader( + val kafkaOffsetReader = new KafkaOffsetReaderImpl( strategy(caseInsensitiveParams), kafkaParamsForDriver(specifiedKafkaParams), parameters, @@ -112,7 +117,7 @@ private[kafka010] class KafkaProvider extends DataSourceRegister with StreamSour override def createRelation( sqlContext: SQLContext, parameters: Map[String, String]): BaseRelation = { - validateOptions(parameters) + validateOptions(parameters, Batch) // 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. @@ -127,8 +132,6 @@ private[kafka010] class KafkaProvider extends DataSourceRegister with StreamSour val startingRelationOffsets = caseInsensitiveParams.get(STARTING_OFFSETS_OPTION_KEY).map(_.trim.toLowerCase) match { - case Some("latest") => - throw new IllegalArgumentException("Starting relation offset can't be latest.") case Some("earliest") => EarliestOffsets case Some(json) => SpecificOffsets(JsonUtils.partitionOffsets(json)) case None => EarliestOffsets @@ -137,17 +140,16 @@ private[kafka010] class KafkaProvider extends DataSourceRegister with StreamSour val endingRelationOffsets = caseInsensitiveParams.get(ENDING_OFFSETS_OPTION_KEY).map(_.trim.toLowerCase) match { case Some("latest") => LatestOffsets - case Some("earliest") => - throw new IllegalArgumentException("Ending relation offset can't be earliest.") case Some(json) => SpecificOffsets(JsonUtils.partitionOffsets(json)) case None => LatestOffsets } - val kafkaOffsetReader = new KafkaOffsetReader( + val kafkaOffsetReaderImpl = new KafkaOffsetReaderImpl( strategy(caseInsensitiveParams), kafkaParamsForDriver(specifiedKafkaParams), parameters, driverGroupIdPrefix = s"$uniqueGroupId-driver") + val kafkaOffsetReader = new UninterruptibleKafkaOffsetReader(kafkaOffsetReaderImpl) new KafkaRelation( sqlContext, @@ -217,11 +219,17 @@ private[kafka010] class KafkaProvider extends DataSourceRegister with StreamSour caseInsensitiveParams.getOrElse(FAIL_ON_DATA_LOSS_OPTION_KEY, "true").toBoolean - private def validateOptions(parameters: Map[String, String]): Unit = { + private def validateOptions(parameters: Map[String, String], mode: Mode): Unit = { // Validate source options val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase, v) } val specifiedStrategies = caseInsensitiveParams.filter { case (k, _) => STRATEGY_OPTION_KEYS.contains(k) }.toSeq + + mode match { + case Stream => validateStream(caseInsensitiveParams) + case Batch => validateBatch(caseInsensitiveParams) + } + if (specifiedStrategies.isEmpty) { throw new IllegalArgumentException( "One of the following options must be specified for Kafka source: " @@ -314,6 +322,42 @@ private[kafka010] class KafkaProvider extends DataSourceRegister with StreamSour } } + private def validateStream(caseInsensitiveParams: Map[String, String]) = { + caseInsensitiveParams.get(ENDING_OFFSETS_OPTION_KEY).map(_ => + throw new IllegalArgumentException("Ending offset not valid in stream mode")) + } + + private def validateBatch(caseInsensitiveParams: Map[String, String]) = { + caseInsensitiveParams.get(STARTING_OFFSETS_OPTION_KEY).map(_.trim.toLowerCase) match { + case Some("earliest") => // good to go + case Some("latest") => + throw new IllegalArgumentException("Starting relation offset can't be latest") + case Some(json) => (SpecificOffsets(JsonUtils.partitionOffsets(json))) + .partitionOffsets.foreach { + case (tp, off) if off == -1 => + throw new IllegalArgumentException(s"startingOffsets for $tp can't be latest") + } + case _ => // default to earliest + } + + caseInsensitiveParams.get(ENDING_OFFSETS_OPTION_KEY).map(_.trim.toLowerCase) match { + case Some("earliest") => + throw new IllegalArgumentException("Ending relation offset can't be earliest") + case Some("latest") => // good to go + case Some(json) => (SpecificOffsets(JsonUtils.partitionOffsets(json))) + .partitionOffsets.foreach { + case (tp, off) if off == -2 => + throw new IllegalArgumentException(s"ending offset for $tp can't be earliest") + } + case _ => // default to latest + } + + // Don't want to throw an error, but at least log a warning. + if (caseInsensitiveParams.get("maxoffsetspertrigger").isDefined) { + logWarning("maxOffsetsPerTrigger option ignored in batch mode") + } + } + /** 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) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala index c20be9e82e7d..c4ade2476e66 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala @@ -44,9 +44,6 @@ private[kafka010] class KafkaRelation( require(endingOffsets != EarliestOffsets, "Ending offset not allowed to be set to earliest offsets.") - if (sourceOptions.get("maxOffsetsPerTrigger").isDefined) - logWarning("maxOffsetsPerTrigger option ignored in batch mode.") - private val pollTimeoutMs = sourceOptions.getOrElse( "kafkaConsumer.pollTimeoutMs", sqlContext.sparkContext.conf.getTimeAsMs("spark.network.timeout", "120s").toString diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala index 08cddb111b58..0b85022a7441 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala @@ -53,12 +53,12 @@ class KafkaRelationSuite extends SparkFunSuite with BeforeAndAfter with SharedSQ } } - test("maxOffsetsPerTrigger") { + test("Test batch processing earliest to latest") { val topic = newTopic() testUtils.createTopic(topic, partitions = 3) - testUtils.sendMessages(topic, (100 to 200).map(_.toString).toArray, Some(0)) - testUtils.sendMessages(topic, (10 to 20).map(_.toString).toArray, Some(1)) - testUtils.sendMessages(topic, Array("1"), Some(2)) + testUtils.sendMessages(topic, (0 to 9).map(_.toString).toArray, Some(0)) + testUtils.sendMessages(topic, (10 to 19).map(_.toString).toArray, Some(1)) + testUtils.sendMessages(topic, Array("20"), Some(2)) val reader = spark .read @@ -67,11 +67,43 @@ class KafkaRelationSuite extends SparkFunSuite with BeforeAndAfter with SharedSQ .option("subscribe", topic) .option("startingOffsets", "earliest") .option("endingOffsets", "latest") + .load() - val kafka = reader.load() - .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") - .as[(String, String)] - val mapped: org.apache.spark.sql.Dataset[_] = kafka.map(kv => kv._2.toInt) - mapped.collect().foreach(println) + assert(reader.count() === 21) + + testUtils.sendMessages(topic, (21 to 29).map(_.toString).toArray, Some(2)) + assert(reader.count() === 30) + } + + test("bad source options") { + def testBadOptions(options: (String, String)*)(expectedMsgs: String*): Unit = { + val ex = intercept[IllegalArgumentException] { + val reader = spark + .read + .format("kafka") + options.foreach { case (k, v) => reader.option(k, v) } + reader.load() + } + expectedMsgs.foreach { m => + assert(ex.getMessage.toLowerCase.contains(m.toLowerCase)) + } + } + + // Specifying an ending offset + testBadOptions("startingOffsets" -> "latest")("Starting relation offset can't be latest") + + // 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" -> "t", "assign" -> """{"a":[0]}""")( + "only one", "options can be specified") + + testBadOptions("assign" -> "")("no topicpartitions to assign") + testBadOptions("subscribe" -> "")("no topics to subscribe") + testBadOptions("subscribePattern" -> "")("pattern to subscribe is empty") } } 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 35a05c7031be..cd6a21224b2c 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 @@ -384,6 +384,9 @@ class KafkaSourceSuite extends KafkaSourceTest { } } + // Specifying an ending offset + testBadOptions("endingOffsets" -> "latest")("Ending offset not valid in stream mode") + // No strategy specified testBadOptions()("options must be specified", "subscribe", "subscribePattern") From ff94ed803474448f6bb388f8933e6ec091fc24a1 Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Mon, 23 Jan 2017 16:44:31 -0800 Subject: [PATCH 06/40] added uninterruptible thread version of kafka offset reader --- .../spark/sql/kafka010/KafkaRelationSuite.scala | 17 +++++++++++++---- 1 file changed, 13 insertions(+), 4 deletions(-) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala index 0b85022a7441..5a94304c4d48 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala @@ -53,14 +53,15 @@ class KafkaRelationSuite extends SparkFunSuite with BeforeAndAfter with SharedSQ } } - test("Test batch processing earliest to latest") { + test("batch processing earliest to latest") { val topic = newTopic() testUtils.createTopic(topic, partitions = 3) testUtils.sendMessages(topic, (0 to 9).map(_.toString).toArray, Some(0)) testUtils.sendMessages(topic, (10 to 19).map(_.toString).toArray, Some(1)) testUtils.sendMessages(topic, Array("20"), Some(2)) - val reader = spark + // Specify explicit earliest and latest offset values + var reader = spark .read .format("kafka") .option("kafka.bootstrap.servers", testUtils.brokerAddress) @@ -68,11 +69,19 @@ class KafkaRelationSuite extends SparkFunSuite with BeforeAndAfter with SharedSQ .option("startingOffsets", "earliest") .option("endingOffsets", "latest") .load() - assert(reader.count() === 21) - testUtils.sendMessages(topic, (21 to 29).map(_.toString).toArray, Some(2)) assert(reader.count() === 30) + + + // Implicit offset values, should default to earliest and latest + reader = spark + .read + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("subscribe", topic) + .load() + assert(reader.count() === 30) } test("bad source options") { From f8fd34cf0c2da4f1b9c793e4e021c23a923a0285 Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Mon, 23 Jan 2017 17:13:16 -0800 Subject: [PATCH 07/40] update tests --- .../spark/sql/kafka010/KafkaProvider.scala | 8 +-- .../sql/kafka010/KafkaRelationSuite.scala | 51 ++++++++++++++++++- 2 files changed, 54 insertions(+), 5 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaProvider.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaProvider.scala index 926d00797de0..a8c5f4d47f27 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaProvider.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaProvider.scala @@ -324,30 +324,32 @@ private[kafka010] class KafkaProvider extends DataSourceRegister with StreamSour private def validateStream(caseInsensitiveParams: Map[String, String]) = { caseInsensitiveParams.get(ENDING_OFFSETS_OPTION_KEY).map(_ => - throw new IllegalArgumentException("Ending offset not valid in stream mode")) + throw new IllegalArgumentException("ending offset not valid in stream mode")) } private def validateBatch(caseInsensitiveParams: Map[String, String]) = { caseInsensitiveParams.get(STARTING_OFFSETS_OPTION_KEY).map(_.trim.toLowerCase) match { case Some("earliest") => // good to go case Some("latest") => - throw new IllegalArgumentException("Starting relation offset can't be latest") + throw new IllegalArgumentException("starting relation offset can't be latest") case Some(json) => (SpecificOffsets(JsonUtils.partitionOffsets(json))) .partitionOffsets.foreach { case (tp, off) if off == -1 => throw new IllegalArgumentException(s"startingOffsets for $tp can't be latest") + case _ => // ignore } case _ => // default to earliest } caseInsensitiveParams.get(ENDING_OFFSETS_OPTION_KEY).map(_.trim.toLowerCase) match { case Some("earliest") => - throw new IllegalArgumentException("Ending relation offset can't be earliest") + throw new IllegalArgumentException("ending relation offset can't be earliest") case Some("latest") => // good to go case Some(json) => (SpecificOffsets(JsonUtils.partitionOffsets(json))) .partitionOffsets.foreach { case (tp, off) if off == -2 => throw new IllegalArgumentException(s"ending offset for $tp can't be earliest") + case _ => // ignore } case _ => // default to latest } diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala index 5a94304c4d48..d013626d746d 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala @@ -82,6 +82,34 @@ class KafkaRelationSuite extends SparkFunSuite with BeforeAndAfter with SharedSQ .option("subscribe", topic) .load() assert(reader.count() === 30) + + // Test explicitly specified offsets + val startPartitionOffsets = Map( + new TopicPartition(topic, 0) -> -2L, + new TopicPartition(topic, 1) -> -2L, + new TopicPartition(topic, 2) -> 0L + ) + val startingOffsets = JsonUtils.partitionOffsets(startPartitionOffsets) + + val endPartitionOffsets = Map( + new TopicPartition(topic, 0) -> -1L, + new TopicPartition(topic, 1) -> -1L, + new TopicPartition(topic, 2) -> 10L + ) + val endingOffsets = JsonUtils.partitionOffsets(endPartitionOffsets) + reader = spark + .read + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("subscribe", topic) + .option("startingOffsets", startingOffsets) + .option("endingOffsets", endingOffsets) + .load() + assert(reader.count() === 30) + testUtils.sendMessages(topic, (30 to 39).map(_.toString).toArray, Some(2)) + assert(reader.count() === 30) // static offset partition 2, nothing should change + testUtils.sendMessages(topic, (30 to 39).map(_.toString).toArray, Some(1)) + assert(reader.count() === 40) // latest offset partition 1, should change } test("bad source options") { @@ -98,8 +126,27 @@ class KafkaRelationSuite extends SparkFunSuite with BeforeAndAfter with SharedSQ } } - // Specifying an ending offset - testBadOptions("startingOffsets" -> "latest")("Starting relation offset can't be latest") + // Specifying an ending offset as the starting point + testBadOptions("startingOffsets" -> "latest")("starting relation offset can't be latest") + + // Now do it with an explicit json offset + val startPartitionOffsets = Map( + new TopicPartition("t", 0) -> -1L // specify latest + ) + val startingOffsets = JsonUtils.partitionOffsets(startPartitionOffsets) + testBadOptions("subscribe" -> "t", "startingOffsets" -> startingOffsets)( + "startingoffsets for t-0 can't be latest") + + + // Make sure we catch ending offsets that indicate earliest + testBadOptions("endingOffsets" -> "earliest")("ending relation offset can't be earliest") + + val endPartitionOffsets = Map( + new TopicPartition("t", 0) -> -2L // specify earliest + ) + val endingOffsets = JsonUtils.partitionOffsets(endPartitionOffsets) + testBadOptions("subscribe" -> "t", "endingOffsets" -> endingOffsets)( + "ending offset for t-0 can't be earliest") // No strategy specified testBadOptions()("options must be specified", "subscribe", "subscribePattern") From 74d96fc9049a0a0fb6de6d011eb896b7d7c32b30 Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Tue, 24 Jan 2017 10:45:12 -0800 Subject: [PATCH 08/40] update comments --- .../apache/spark/sql/kafka010/KafkaRelationSuite.scala | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala index d013626d746d..ed2181129a66 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala @@ -71,6 +71,7 @@ class KafkaRelationSuite extends SparkFunSuite with BeforeAndAfter with SharedSQ .load() assert(reader.count() === 21) testUtils.sendMessages(topic, (21 to 29).map(_.toString).toArray, Some(2)) + // "latest" should late bind to the current (latest) offset assert(reader.count() === 30) @@ -81,20 +82,21 @@ class KafkaRelationSuite extends SparkFunSuite with BeforeAndAfter with SharedSQ .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("subscribe", topic) .load() + // Test that we default to "earliest" and "latest" assert(reader.count() === 30) // Test explicitly specified offsets val startPartitionOffsets = Map( - new TopicPartition(topic, 0) -> -2L, + new TopicPartition(topic, 0) -> -2L, // -2 => earliest new TopicPartition(topic, 1) -> -2L, - new TopicPartition(topic, 2) -> 0L + new TopicPartition(topic, 2) -> 0L // explicit earliest ) val startingOffsets = JsonUtils.partitionOffsets(startPartitionOffsets) val endPartitionOffsets = Map( - new TopicPartition(topic, 0) -> -1L, + new TopicPartition(topic, 0) -> -1L, // -1 => latest new TopicPartition(topic, 1) -> -1L, - new TopicPartition(topic, 2) -> 10L + new TopicPartition(topic, 2) -> 10L // explicit offset happens to = the latest ) val endingOffsets = JsonUtils.partitionOffsets(endPartitionOffsets) reader = spark From d31fc8104bf94c82bc6fa1c099def9ca16fec93a Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Tue, 24 Jan 2017 16:53:11 -0800 Subject: [PATCH 09/40] address comments from @zsxwing --- ...pache.spark.sql.sources.DataSourceRegister | 2 +- .../sql/kafka010/KafkaOffsetReader.scala | 64 +++++++-------- .../spark/sql/kafka010/KafkaRelation.scala | 22 ++--- .../spark/sql/kafka010/KafkaSource.scala | 2 +- ...ovider.scala => KafkaSourceProvider.scala} | 20 ++--- .../spark/sql/kafka010/KafkaSourceRDD.scala | 11 ++- .../sql/kafka010/KafkaRelationSuite.scala | 82 +++++++++++++++---- .../spark/sql/kafka010/KafkaSourceSuite.scala | 4 +- 8 files changed, 125 insertions(+), 82 deletions(-) rename external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/{KafkaProvider.scala => KafkaSourceProvider.scala} (98%) 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 index 106a211d6ae4..2f9e9fc0396d 100644 --- 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 @@ -1 +1 @@ -org.apache.spark.sql.kafka010.KafkaProvider +org.apache.spark.sql.kafka010.KafkaSourceProvider diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala index b2ea85b13dd0..b52d01801d2f 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.kafka010 import java.{util => ju} -import java.util.concurrent.{Executor, LinkedBlockingQueue} +import java.util.concurrent.{Executors, ThreadFactory} import scala.collection.JavaConverters._ import scala.concurrent.{ExecutionContext, Future} @@ -37,15 +37,31 @@ import org.apache.spark.util.{ThreadUtils, UninterruptibleThread} private[kafka010] trait KafkaOffsetReader { + /** + * Closes the connection to Kafka, and cleans up state. + */ def close() + /** + * Set consumer position to specified offsets, making sure all assignments are set. + */ def fetchSpecificStartingOffsets( partitionOffsets: Map[TopicPartition, Long]): Map[TopicPartition, Long] + /** + * Fetch the earliest offsets of partitions. + */ def fetchEarliestOffsets(): Map[TopicPartition, Long] + /** + * Fetch the latest offsets of partitions. + */ def fetchLatestOffsets(): Map[TopicPartition, Long] + /** + * Fetch the earliest offsets for newly discovered partitions. The return result may not contain + * some partitions if they are deleted. + */ def fetchNewPartitionEarliestOffsets( newPartitions: Seq[TopicPartition]): Map[TopicPartition, Long] } @@ -92,9 +108,6 @@ private[kafka010] class KafkaOffsetReaderImpl( def close(): Unit = consumer.close() - /** - * Set consumer position to specified offsets, making sure all assignments are set. - */ def fetchSpecificStartingOffsets( partitionOffsets: Map[TopicPartition, Long]): Map[TopicPartition, Long] = withRetriesWithoutInterrupt { @@ -118,9 +131,6 @@ private[kafka010] class KafkaOffsetReaderImpl( } } - /** - * Fetch the earliest offsets of partitions. - */ def fetchEarliestOffsets(): Map[TopicPartition, Long] = withRetriesWithoutInterrupt { // Poll to get the latest assigned partitions consumer.poll(0) @@ -134,9 +144,6 @@ private[kafka010] class KafkaOffsetReaderImpl( partitionOffsets } - /** - * Fetch the latest offset of partitions. - */ def fetchLatestOffsets(): Map[TopicPartition, Long] = withRetriesWithoutInterrupt { // Poll to get the latest assigned partitions consumer.poll(0) @@ -150,10 +157,6 @@ private[kafka010] class KafkaOffsetReaderImpl( partitionOffsets } - /** - * Fetch the earliest offsets for newly discovered partitions. The return result may not contain - * some partitions if they are deleted. - */ def fetchNewPartitionEarliestOffsets( newPartitions: Seq[TopicPartition]): Map[TopicPartition, Long] = { if (newPartitions.isEmpty) { @@ -253,7 +256,7 @@ private[kafka010] class KafkaOffsetReaderImpl( /** * The Kafka Consumer must be called in an UninterruptibleThread. This naturally occurs - * in Spark Streaming, but not in Spark SQL, which will use this call to communicate + * in Structured Streaming, but not in Spark SQL, which will use this call to communicate * with Kafak for obtaining offsets. * * @param kafkaOffsetReader Basically in instance of [[KafkaOffsetReaderImpl]] that @@ -262,32 +265,23 @@ private[kafka010] class KafkaOffsetReaderImpl( private[kafka010] class UninterruptibleKafkaOffsetReader(kafkaOffsetReader: KafkaOffsetReader) extends KafkaOffsetReader with Logging { - private class KafkaOffsetReaderThread extends UninterruptibleThread("Kafka Offset Reader") { - override def run(): Unit = { - while (this.isInterrupted == false) { - val runnable = queue.take() - runnable.run() + val kafkaReaderThread = Executors.newSingleThreadExecutor(new ThreadFactory { + override def newThread(r: Runnable): Thread = { + logInfo("NEW UNINTERRUPTIBLE THREAD KAFKA OFFSET") + val t = new UninterruptibleThread("Kafka Offset Reader") { + override def run(): Unit = { + r.run() + } } - } - } - private val readerThread = new KafkaOffsetReaderThread - - private val queue = new LinkedBlockingQueue[Runnable]() - - private val execContext = ExecutionContext.fromExecutor(new Executor { - override def execute(runnable: Runnable): Unit = { - if (readerThread.isAlive == false) readerThread.start() - queue.add(runnable) + t.setDaemon(true) + t } }) - + val execContext = ExecutionContext.fromExecutorService(kafkaReaderThread) override def close(): Unit = { kafkaOffsetReader.close() - readerThread.interrupt() - queue.add(new Runnable() { - override def run(): Unit = { } - }) + kafkaReaderThread.shutdownNow() } override def fetchSpecificStartingOffsets( diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala index c4ade2476e66..ebe8493da463 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala @@ -38,10 +38,9 @@ private[kafka010] class KafkaRelation( startingOffsets: KafkaOffsets, endingOffsets: KafkaOffsets) extends BaseRelation with TableScan with Logging { - - require(startingOffsets != LatestOffsets, + assert(startingOffsets != LatestOffsets, "Starting offset not allowed to be set to latest offsets.") - require(endingOffsets != EarliestOffsets, + assert(endingOffsets != EarliestOffsets, "Ending offset not allowed to be set to earliest offsets.") private val pollTimeoutMs = sourceOptions.getOrElse( @@ -57,27 +56,24 @@ private[kafka010] class KafkaRelation( val untilPartitionOffsets = getPartitionOffsets(endingOffsets) // Obtain topicPartitions in both from and until partition offset, ignoring // topic partitions that were added and/or deleted between the two above calls. - val topicPartitions = fromPartitionOffsets.keySet.intersect(untilPartitionOffsets.keySet) - + if (fromPartitionOffsets.keySet.size != untilPartitionOffsets.keySet.size) { + throw new IllegalStateException("Kafka return different topic partitions " + + "for starting and ending offsets") + } val sortedExecutors = KafkaUtils.getSortedExecutorList(sqlContext.sparkContext) val numExecutors = sortedExecutors.length logDebug("Sorted executors: " + sortedExecutors.mkString(", ")) // Calculate offset ranges - val offsetRanges = topicPartitions.map { tp => + val offsetRanges = untilPartitionOffsets.keySet.map { tp => val fromOffset = fromPartitionOffsets.get(tp).getOrElse { // This should not happen since topicPartitions contains all partitions not in // fromPartitionOffsets throw new IllegalStateException(s"$tp doesn't have a from offset") } val untilOffset = untilPartitionOffsets(tp) - val preferredLoc = if (numExecutors > 0) { - // This allows cached KafkaConsumers in the executors to be re-used to read the same - // partition in every batch. - Some(sortedExecutors(KafkaUtils.floorMod(tp.hashCode, numExecutors))) - } else None - KafkaSourceRDDOffsetRange(tp, fromOffset, untilOffset, preferredLoc) + KafkaSourceRDDOffsetRange(tp, fromOffset, untilOffset, None) }.toArray logInfo("GetBatch generating RDD of offset range: " + @@ -86,7 +82,7 @@ private[kafka010] class KafkaRelation( // Create an RDD that reads from Kafka and get the (key, value) pair as byte arrays. val rdd = new KafkaSourceRDD( sqlContext.sparkContext, executorKafkaParams, offsetRanges, - pollTimeoutMs, failOnDataLoss).map { cr => + pollTimeoutMs, failOnDataLoss, false).map { cr => InternalRow( cr.key, cr.value, 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 9459f169efc7..15cad9f60ed1 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 @@ -271,7 +271,7 @@ private[kafka010] class KafkaSource( // Create an RDD that reads from Kafka and get the (key, value) pair as byte arrays. val rdd = new KafkaSourceRDD( - sc, executorKafkaParams, offsetRanges, pollTimeoutMs, failOnDataLoss).map { cr => + sc, executorKafkaParams, offsetRanges, pollTimeoutMs, failOnDataLoss, true).map { cr => InternalRow( cr.key, cr.value, diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaProvider.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala similarity index 98% rename from external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaProvider.scala rename to external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala index a8c5f4d47f27..54b888ada7a1 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaProvider.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala @@ -37,16 +37,9 @@ import org.apache.spark.sql.types.StructType * IllegalArgumentException when the Kafka Dataset is created, so that it can catch * missing options even before the query is started. */ -private[kafka010] class KafkaProvider extends DataSourceRegister with StreamSourceProvider +private[kafka010] class KafkaSourceProvider extends DataSourceRegister with StreamSourceProvider with RelationProvider with Logging { - import KafkaProvider._ - - // Used to check parameters for different source modes - private sealed trait Mode - private case object Batch extends Mode - private case object Stream extends Mode - - private val deserClassName = classOf[ByteArrayDeserializer].getName + import KafkaSourceProvider._ override def shortName(): String = "kafka" @@ -382,9 +375,16 @@ private[kafka010] class KafkaProvider extends DataSourceRegister with StreamSour } } -private[kafka010] object KafkaProvider { +private[kafka010] object KafkaSourceProvider { private val STRATEGY_OPTION_KEYS = Set("subscribe", "subscribepattern", "assign") private val STARTING_OFFSETS_OPTION_KEY = "startingoffsets" private val ENDING_OFFSETS_OPTION_KEY = "endingoffsets" private val FAIL_ON_DATA_LOSS_OPTION_KEY = "failondataloss" + + // Used to check parameters for different source modes + private sealed trait Mode + private case object Batch extends Mode + private case object Stream extends Mode + + private val deserClassName = classOf[ByteArrayDeserializer].getName } 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 244cd2c225bd..34b77baa3385 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 @@ -63,7 +63,8 @@ private[kafka010] class KafkaSourceRDD( executorKafkaParams: ju.Map[String, Object], offsetRanges: Seq[KafkaSourceRDDOffsetRange], pollTimeoutMs: Long, - failOnDataLoss: Boolean) + failOnDataLoss: Boolean, + cachedConsumer: Boolean) extends RDD[ConsumerRecord[Array[Byte], Array[Byte]]](sc, Nil) { override def persist(newLevel: StorageLevel): this.type = { @@ -134,8 +135,12 @@ private[kafka010] class KafkaSourceRDD( Iterator.empty } else { new NextIterator[ConsumerRecord[Array[Byte], Array[Byte]]]() { - val consumer = CachedKafkaConsumer.getOrCreate( - range.topic, range.partition, executorKafkaParams) + val consumer = if (cachedConsumer) { + CachedKafkaConsumer.getOrCreate( + range.topic, range.partition, executorKafkaParams) + } else { + CachedKafkaConsumer(range.topicPartition, executorKafkaParams) + } var requestOffset = range.fromOffset override def getNext(): ConsumerRecord[Array[Byte], Array[Byte]] = { diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala index ed2181129a66..d5bea3aed2fc 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala @@ -22,10 +22,10 @@ import java.util.concurrent.atomic.AtomicInteger import org.apache.kafka.common.TopicPartition import org.scalatest.BeforeAndAfter -import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.test.{SharedSQLContext, TestSparkSession} -class KafkaRelationSuite extends SparkFunSuite with BeforeAndAfter with SharedSQLContext { +class KafkaRelationSuite extends QueryTest with BeforeAndAfter with SharedSQLContext { import testImplicits._ @@ -53,7 +53,7 @@ class KafkaRelationSuite extends SparkFunSuite with BeforeAndAfter with SharedSQ } } - test("batch processing earliest to latest") { + test("explicit earliest to latest offsets") { val topic = newTopic() testUtils.createTopic(topic, partitions = 3) testUtils.sendMessages(topic, (0 to 9).map(_.toString).toArray, Some(0)) @@ -61,7 +61,7 @@ class KafkaRelationSuite extends SparkFunSuite with BeforeAndAfter with SharedSQ testUtils.sendMessages(topic, Array("20"), Some(2)) // Specify explicit earliest and latest offset values - var reader = spark + val reader = spark .read .format("kafka") .option("kafka.bootstrap.servers", testUtils.brokerAddress) @@ -69,21 +69,40 @@ class KafkaRelationSuite extends SparkFunSuite with BeforeAndAfter with SharedSQ .option("startingOffsets", "earliest") .option("endingOffsets", "latest") .load() - assert(reader.count() === 21) + var df = reader.selectExpr("CAST(value AS STRING)") + checkAnswer(df, (0 to 20).map(_.toString).toDF) + + // "latest" should late bind to the current (latest) offset in the reader testUtils.sendMessages(topic, (21 to 29).map(_.toString).toArray, Some(2)) - // "latest" should late bind to the current (latest) offset - assert(reader.count() === 30) + df = reader.selectExpr("CAST(value AS STRING)") + checkAnswer(df, (0 to 29).map(_.toString).toDF) + } + test("default starting and ending offsets") { + val topic = newTopic() + testUtils.createTopic(topic, partitions = 3) + testUtils.sendMessages(topic, (0 to 9).map(_.toString).toArray, Some(0)) + testUtils.sendMessages(topic, (10 to 19).map(_.toString).toArray, Some(1)) + testUtils.sendMessages(topic, Array("20"), Some(2)) // Implicit offset values, should default to earliest and latest - reader = spark + val df = spark .read .format("kafka") .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("subscribe", topic) .load() + .selectExpr("CAST(value AS STRING)") // Test that we default to "earliest" and "latest" - assert(reader.count() === 30) + checkAnswer(df, (0 to 20).map(_.toString).toDF) + } + + test("explicit offsets") { + val topic = newTopic() + testUtils.createTopic(topic, partitions = 3) + testUtils.sendMessages(topic, (0 to 9).map(_.toString).toArray, Some(0)) + testUtils.sendMessages(topic, (10 to 19).map(_.toString).toArray, Some(1)) + testUtils.sendMessages(topic, Array("20"), Some(2)) // Test explicitly specified offsets val startPartitionOffsets = Map( @@ -96,10 +115,10 @@ class KafkaRelationSuite extends SparkFunSuite with BeforeAndAfter with SharedSQ val endPartitionOffsets = Map( new TopicPartition(topic, 0) -> -1L, // -1 => latest new TopicPartition(topic, 1) -> -1L, - new TopicPartition(topic, 2) -> 10L // explicit offset happens to = the latest + new TopicPartition(topic, 2) -> 1L // explicit offset happens to = the latest ) val endingOffsets = JsonUtils.partitionOffsets(endPartitionOffsets) - reader = spark + val reader = spark .read .format("kafka") .option("kafka.bootstrap.servers", testUtils.brokerAddress) @@ -107,11 +126,40 @@ class KafkaRelationSuite extends SparkFunSuite with BeforeAndAfter with SharedSQ .option("startingOffsets", startingOffsets) .option("endingOffsets", endingOffsets) .load() - assert(reader.count() === 30) - testUtils.sendMessages(topic, (30 to 39).map(_.toString).toArray, Some(2)) - assert(reader.count() === 30) // static offset partition 2, nothing should change - testUtils.sendMessages(topic, (30 to 39).map(_.toString).toArray, Some(1)) - assert(reader.count() === 40) // latest offset partition 1, should change + var df = reader.selectExpr("CAST(value AS STRING)") + checkAnswer(df, (0 to 20).map(_.toString).toDF) + + // static offset partition 2, nothing should change + testUtils.sendMessages(topic, (31 to 39).map(_.toString).toArray, Some(2)) + df = reader.selectExpr("CAST(value AS STRING)") + checkAnswer(df, (0 to 20).map(_.toString).toDF) + + // latest offset partition 1, should change + testUtils.sendMessages(topic, (21 to 30).map(_.toString).toArray, Some(1)) + df = reader.selectExpr("CAST(value AS STRING)") + checkAnswer(df, (0 to 30).map(_.toString).toDF) + } + + test("reuse same dataframe in query") { + // This test ensures that we do not cache the Kafka Consumer in KafkaRelation + val topic = newTopic() + testUtils.createTopic(topic, partitions = 1) + testUtils.sendMessages(topic, (0 to 10).map(_.toString).toArray, Some(0)) + + // Ensure local[2] so that two tasks will execute the query on one partition + val testSession = new TestSparkSession(sparkContext) + // Specify explicit earliest and latest offset values + val reader = testSession + .read + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("subscribe", topic) + .option("startingOffsets", "earliest") + .option("endingOffsets", "latest") + .load() + var df = reader.selectExpr("CAST(value AS STRING)") + checkAnswer(df.union(df), + (0 to 10).map(_.toString).toDF.union((0 to 10).map(_.toString).toDF)) } test("bad source options") { 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 cd6a21224b2c..e20aa8f671c5 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 @@ -692,7 +692,7 @@ class KafkaSourceSuite extends KafkaSourceTest { val reader = spark.readStream reader - .format(classOf[KafkaProvider].getCanonicalName.stripSuffix("$")) + .format(classOf[KafkaSourceProvider].getCanonicalName.stripSuffix("$")) .option("startingOffsets", s"earliest") .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.metadata.max.age.ms", "1") @@ -754,7 +754,7 @@ class KafkaSourceStressSuite extends KafkaSourceTest { // Create Kafka source that reads from latest offset val kafka = spark.readStream - .format(classOf[KafkaProvider].getCanonicalName.stripSuffix("$")) + .format(classOf[KafkaSourceProvider].getCanonicalName.stripSuffix("$")) .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.metadata.max.age.ms", "1") .option("subscribePattern", "stress.*") From 1db1649201361bcede52997ec8c2f0610a55da8b Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Tue, 24 Jan 2017 17:04:06 -0800 Subject: [PATCH 10/40] update --- .../spark/sql/kafka010/CachedKafkaConsumer.scala | 5 +++-- .../org/apache/spark/sql/kafka010/KafkaSourceRDD.scala | 10 +++------- 2 files changed, 6 insertions(+), 9 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 3f396a7e6b69..924f8f40245d 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 @@ -334,14 +334,15 @@ private[kafka010] object CachedKafkaConsumer extends Logging { def getOrCreate( topic: String, partition: Int, - kafkaParams: ju.Map[String, Object]): CachedKafkaConsumer = synchronized { + kafkaParams: ju.Map[String, Object], + reuse: Boolean): 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) { + if (!reuse || TaskContext.get != null && TaskContext.get.attemptNumber > 1) { val removedConsumer = cache.remove(key) if (removedConsumer != null) { removedConsumer.close() 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 34b77baa3385..97ab6fbbb4cd 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 @@ -64,7 +64,7 @@ private[kafka010] class KafkaSourceRDD( offsetRanges: Seq[KafkaSourceRDDOffsetRange], pollTimeoutMs: Long, failOnDataLoss: Boolean, - cachedConsumer: Boolean) + reuseKafkaConsumer: Boolean) extends RDD[ConsumerRecord[Array[Byte], Array[Byte]]](sc, Nil) { override def persist(newLevel: StorageLevel): this.type = { @@ -135,12 +135,8 @@ private[kafka010] class KafkaSourceRDD( Iterator.empty } else { new NextIterator[ConsumerRecord[Array[Byte], Array[Byte]]]() { - val consumer = if (cachedConsumer) { - CachedKafkaConsumer.getOrCreate( - range.topic, range.partition, executorKafkaParams) - } else { - CachedKafkaConsumer(range.topicPartition, executorKafkaParams) - } + val consumer = CachedKafkaConsumer.getOrCreate( + range.topic, range.partition, executorKafkaParams, reuseKafkaConsumer) var requestOffset = range.fromOffset override def getNext(): ConsumerRecord[Array[Byte], Array[Byte]] = { From a5b02691ddbaef6c8092881ea15b582d9137be71 Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Thu, 26 Jan 2017 13:32:00 -0800 Subject: [PATCH 11/40] address comments from @zsxwing --- .../spark/sql/kafka010/KafkaOffsetReader.scala | 2 -- .../spark/sql/kafka010/KafkaRelation.scala | 16 +++++++++------- .../apache/spark/sql/kafka010/KafkaSource.scala | 3 ++- .../spark/sql/kafka010/KafkaRelationSuite.scala | 7 ++----- 4 files changed, 13 insertions(+), 15 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala index b52d01801d2f..74c1da5a198c 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala @@ -267,7 +267,6 @@ private[kafka010] class UninterruptibleKafkaOffsetReader(kafkaOffsetReader: Kafk val kafkaReaderThread = Executors.newSingleThreadExecutor(new ThreadFactory { override def newThread(r: Runnable): Thread = { - logInfo("NEW UNINTERRUPTIBLE THREAD KAFKA OFFSET") val t = new UninterruptibleThread("Kafka Offset Reader") { override def run(): Unit = { r.run() @@ -312,7 +311,6 @@ private[kafka010] class UninterruptibleKafkaOffsetReader(kafkaOffsetReader: Kafk kafkaOffsetReader.fetchNewPartitionEarliestOffsets(newPartitions) }(execContext) ThreadUtils.awaitResult(future, Duration.Inf) - } } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala index ebe8493da463..1e86d2adafbb 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.kafka010 import java.{util => ju} +import org.apache.kafka.common.TopicPartition + import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.{Row, SQLContext} @@ -57,14 +59,14 @@ private[kafka010] class KafkaRelation( // Obtain topicPartitions in both from and until partition offset, ignoring // topic partitions that were added and/or deleted between the two above calls. if (fromPartitionOffsets.keySet.size != untilPartitionOffsets.keySet.size) { - throw new IllegalStateException("Kafka return different topic partitions " + - "for starting and ending offsets") + implicit val topicOrdering: Ordering[TopicPartition] = Ordering.by(t => t.topic()) + val fromTopics = fromPartitionOffsets.keySet.toList.sorted.mkString(",") + val untilTopics = untilPartitionOffsets.keySet.toList.sorted.mkString(",") + throw new IllegalStateException("different topic partitions " + + s"for starting offsets topics[${fromTopics}] and " + + s"ending offsets topics[${untilTopics}]") } - val sortedExecutors = KafkaUtils.getSortedExecutorList(sqlContext.sparkContext) - val numExecutors = sortedExecutors.length - logDebug("Sorted executors: " + sortedExecutors.mkString(", ")) - // Calculate offset ranges val offsetRanges = untilPartitionOffsets.keySet.map { tp => val fromOffset = fromPartitionOffsets.get(tp).getOrElse { @@ -82,7 +84,7 @@ private[kafka010] class KafkaRelation( // Create an RDD that reads from Kafka and get the (key, value) pair as byte arrays. val rdd = new KafkaSourceRDD( sqlContext.sparkContext, executorKafkaParams, offsetRanges, - pollTimeoutMs, failOnDataLoss, false).map { cr => + pollTimeoutMs, failOnDataLoss, reuseKafkaConsumer = false).map { cr => InternalRow( cr.key, cr.value, 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 15cad9f60ed1..5e14cda4adbc 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 @@ -271,7 +271,8 @@ private[kafka010] class KafkaSource( // Create an RDD that reads from Kafka and get the (key, value) pair as byte arrays. val rdd = new KafkaSourceRDD( - sc, executorKafkaParams, offsetRanges, pollTimeoutMs, failOnDataLoss, true).map { cr => + sc, executorKafkaParams, offsetRanges, pollTimeoutMs, failOnDataLoss, + reuseKafkaConsumer = true).map { cr => InternalRow( cr.key, cr.value, diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala index d5bea3aed2fc..0c4fc3825c50 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala @@ -146,10 +146,8 @@ class KafkaRelationSuite extends QueryTest with BeforeAndAfter with SharedSQLCon testUtils.createTopic(topic, partitions = 1) testUtils.sendMessages(topic, (0 to 10).map(_.toString).toArray, Some(0)) - // Ensure local[2] so that two tasks will execute the query on one partition - val testSession = new TestSparkSession(sparkContext) // Specify explicit earliest and latest offset values - val reader = testSession + val reader = spark .read .format("kafka") .option("kafka.bootstrap.servers", testUtils.brokerAddress) @@ -158,8 +156,7 @@ class KafkaRelationSuite extends QueryTest with BeforeAndAfter with SharedSQLCon .option("endingOffsets", "latest") .load() var df = reader.selectExpr("CAST(value AS STRING)") - checkAnswer(df.union(df), - (0 to 10).map(_.toString).toDF.union((0 to 10).map(_.toString).toDF)) + checkAnswer(df.union(df), ((0 to 10) ++ (0 to 10)).map(_.toString).toDF) } test("bad source options") { From c08c01fd21fc53d3db3504a257ab6bb6115cd462 Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Fri, 27 Jan 2017 08:33:29 -0800 Subject: [PATCH 12/40] late binding offsets --- .../sql/kafka010/CachedKafkaConsumer.scala | 20 +++--- .../sql/kafka010/KafkaOffsetReader.scala | 27 ++++++-- .../spark/sql/kafka010/KafkaRelation.scala | 28 ++++++-- .../spark/sql/kafka010/KafkaSourceRDD.scala | 21 +++++- .../src/test/resources/log4j.properties | 1 + .../sql/kafka010/KafkaRelationSuite.scala | 65 +++++++++++++++---- .../spark/sql/kafka010/KafkaTestUtils.scala | 25 ++++++- 7 files changed, 155 insertions(+), 32 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 924f8f40245d..d8336590d5d3 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 @@ -42,7 +42,7 @@ private[kafka010] case class CachedKafkaConsumer private( private val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG).asInstanceOf[String] - private var consumer = createConsumer + var rawConsumer = createConsumer /** Iterator to the already fetch data */ private var fetchedData = ju.Collections.emptyIterator[ConsumerRecord[Array[Byte], Array[Byte]]] @@ -224,8 +224,8 @@ private[kafka010] case class CachedKafkaConsumer private( /** Create a new consumer and reset cached states */ private def resetConsumer(): Unit = { - consumer.close() - consumer = createConsumer + rawConsumer.close() + rawConsumer = createConsumer resetFetchedData() } @@ -271,15 +271,15 @@ private[kafka010] case class CachedKafkaConsumer private( } } - private def close(): Unit = consumer.close() + private def close(): Unit = rawConsumer.close() private def seek(offset: Long): Unit = { logDebug(s"Seeking to $groupId $topicPartition $offset") - consumer.seek(topicPartition, offset) + rawConsumer.seek(topicPartition, offset) } private def poll(pollTimeoutMs: Long): Unit = { - val p = consumer.poll(pollTimeoutMs) + val p = rawConsumer.poll(pollTimeoutMs) val r = p.records(topicPartition) logDebug(s"Polled $groupId ${p.partitions()} ${r.size}") fetchedData = r.iterator @@ -290,10 +290,10 @@ private[kafka010] case class CachedKafkaConsumer private( * and the latest offset. */ private def getAvailableOffsetRange(): (Long, Long) = { - consumer.seekToBeginning(Set(topicPartition).asJava) - val earliestOffset = consumer.position(topicPartition) - consumer.seekToEnd(Set(topicPartition).asJava) - val latestOffset = consumer.position(topicPartition) + rawConsumer.seekToBeginning(Set(topicPartition).asJava) + val earliestOffset = rawConsumer.position(topicPartition) + rawConsumer.seekToEnd(Set(topicPartition).asJava) + val latestOffset = rawConsumer.position(topicPartition) (earliestOffset, latestOffset) } } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala index 74c1da5a198c..b3b1e8c1b2b0 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala @@ -42,6 +42,11 @@ private[kafka010] trait KafkaOffsetReader { */ def close() + /** + * @return The Set of TopicPartitions for a given topic + */ + def fetchTopicPartitions(): Set[TopicPartition] + /** * Set consumer position to specified offsets, making sure all assignments are set. */ @@ -108,7 +113,14 @@ private[kafka010] class KafkaOffsetReaderImpl( def close(): Unit = consumer.close() - def fetchSpecificStartingOffsets( + override def fetchTopicPartitions(): Set[TopicPartition] = { + assert(Thread.currentThread().isInstanceOf[UninterruptibleThread]) + // Poll to get the latest assigned partitions + consumer.poll(0) + consumer.assignment().asScala.toSet + } + + override def fetchSpecificStartingOffsets( partitionOffsets: Map[TopicPartition, Long]): Map[TopicPartition, Long] = withRetriesWithoutInterrupt { // Poll to get the latest assigned partitions @@ -131,7 +143,7 @@ private[kafka010] class KafkaOffsetReaderImpl( } } - def fetchEarliestOffsets(): Map[TopicPartition, Long] = withRetriesWithoutInterrupt { + override def fetchEarliestOffsets(): Map[TopicPartition, Long] = withRetriesWithoutInterrupt { // Poll to get the latest assigned partitions consumer.poll(0) val partitions = consumer.assignment() @@ -144,7 +156,7 @@ private[kafka010] class KafkaOffsetReaderImpl( partitionOffsets } - def fetchLatestOffsets(): Map[TopicPartition, Long] = withRetriesWithoutInterrupt { + override def fetchLatestOffsets(): Map[TopicPartition, Long] = withRetriesWithoutInterrupt { // Poll to get the latest assigned partitions consumer.poll(0) val partitions = consumer.assignment() @@ -157,7 +169,7 @@ private[kafka010] class KafkaOffsetReaderImpl( partitionOffsets } - def fetchNewPartitionEarliestOffsets( + override def fetchNewPartitionEarliestOffsets( newPartitions: Seq[TopicPartition]): Map[TopicPartition, Long] = { if (newPartitions.isEmpty) { Map.empty[TopicPartition, Long] @@ -283,6 +295,13 @@ private[kafka010] class UninterruptibleKafkaOffsetReader(kafkaOffsetReader: Kafk kafkaReaderThread.shutdownNow() } + override def fetchTopicPartitions(): Set[TopicPartition] = { + val future = Future { + kafkaOffsetReader.fetchTopicPartitions() + }(execContext) + ThreadUtils.awaitResult(future, Duration.Inf) + } + override def fetchSpecificStartingOffsets( partitionOffsets: Map[TopicPartition, Long]): Map[TopicPartition, Long] = { val future = Future { diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala index 1e86d2adafbb..9727c346396e 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala @@ -58,7 +58,7 @@ private[kafka010] class KafkaRelation( val untilPartitionOffsets = getPartitionOffsets(endingOffsets) // Obtain topicPartitions in both from and until partition offset, ignoring // topic partitions that were added and/or deleted between the two above calls. - if (fromPartitionOffsets.keySet.size != untilPartitionOffsets.keySet.size) { + if (fromPartitionOffsets.keySet != untilPartitionOffsets.keySet) { implicit val topicOrdering: Ordering[TopicPartition] = Ordering.by(t => t.topic()) val fromTopics = fromPartitionOffsets.keySet.toList.sorted.mkString(",") val untilTopics = untilPartitionOffsets.keySet.toList.sorted.mkString(",") @@ -97,9 +97,27 @@ private[kafka010] class KafkaRelation( sqlContext.internalCreateDataFrame(rdd, schema).rdd } - private def getPartitionOffsets(kafkaOffsets: KafkaOffsets) = kafkaOffsets match { - case EarliestOffsets => kafkaReader.fetchEarliestOffsets() - case LatestOffsets => kafkaReader.fetchLatestOffsets() - case SpecificOffsets(p) => kafkaReader.fetchSpecificStartingOffsets(p) + private def getPartitionOffsets(kafkaOffsets: KafkaOffsets): Map[TopicPartition, Long] = { + def validateTopicPartitions(partitions: Set[TopicPartition], + partitionOffsets: Map[TopicPartition, Long]): Map[TopicPartition, Long] = { + assert(partitions == partitionOffsets.keySet, + "If startingOffsets contains specific offsets, you must specify all TopicPartitions.\n" + + "Use -1 for latest, -2 for earliest, if you don't care.\n" + + s"Specified: ${partitionOffsets.keySet} Assigned: ${partitions}") + logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $partitionOffsets") + partitionOffsets + } + val partitions = kafkaReader.fetchTopicPartitions() + // Obtain TopicPartition offsets with late binding support + kafkaOffsets match { + case EarliestOffsets => partitions.map { + case tp => tp -> -2L + }.toMap + case LatestOffsets => partitions.map { + case tp => tp -> -1L + }.toMap + case SpecificOffsets(partitionOffsets) => + validateTopicPartitions(partitions, partitionOffsets) + } } } 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 97ab6fbbb4cd..61bd12d6e232 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 @@ -123,7 +123,26 @@ private[kafka010] class KafkaSourceRDD( override def compute( thePart: Partition, context: TaskContext): Iterator[ConsumerRecord[Array[Byte], Array[Byte]]] = { - val range = thePart.asInstanceOf[KafkaSourceRDDPartition].offsetRange + var range = thePart.asInstanceOf[KafkaSourceRDDPartition].offsetRange + if (range.fromOffset < 0 || range.untilOffset < 0) { + // Late bind the offset range + val consumer = CachedKafkaConsumer.getOrCreate( + range.topic, range.partition, executorKafkaParams, reuseKafkaConsumer) + val fromOffset = if (range.fromOffset < 0) { + consumer.rawConsumer.seekToBeginning(ju.Arrays.asList(range.topicPartition)) + consumer.rawConsumer.position(range.topicPartition) + } else { + range.fromOffset + } + val untilOffset = if (range.untilOffset < 0) { + consumer.rawConsumer.seekToEnd(ju.Arrays.asList(range.topicPartition)) + consumer.rawConsumer.position(range.topicPartition) + } else { + range.untilOffset + } + range = KafkaSourceRDDOffsetRange(range.topicPartition, + fromOffset, untilOffset, range.preferredLoc) + } assert( range.fromOffset <= range.untilOffset, s"Beginning offset ${range.fromOffset} is after the ending offset ${range.untilOffset} " + diff --git a/external/kafka-0-10-sql/src/test/resources/log4j.properties b/external/kafka-0-10-sql/src/test/resources/log4j.properties index 75e3b53a093f..dee3dedd2ac5 100644 --- a/external/kafka-0-10-sql/src/test/resources/log4j.properties +++ b/external/kafka-0-10-sql/src/test/resources/log4j.properties @@ -25,4 +25,5 @@ log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{ # Ignore messages below warning level from Jetty, because it's a bit verbose log4j.logger.org.spark-project.jetty=WARN +log4j.logger.kafka.log=DEBUG diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala index 0c4fc3825c50..69169a239994 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala @@ -23,7 +23,7 @@ import org.apache.kafka.common.TopicPartition import org.scalatest.BeforeAndAfter import org.apache.spark.sql.QueryTest -import org.apache.spark.sql.test.{SharedSQLContext, TestSparkSession} +import org.apache.spark.sql.test.SharedSQLContext class KafkaRelationSuite extends QueryTest with BeforeAndAfter with SharedSQLContext { @@ -61,7 +61,7 @@ class KafkaRelationSuite extends QueryTest with BeforeAndAfter with SharedSQLCon testUtils.sendMessages(topic, Array("20"), Some(2)) // Specify explicit earliest and latest offset values - val reader = spark + val df = spark .read .format("kafka") .option("kafka.bootstrap.servers", testUtils.brokerAddress) @@ -69,12 +69,11 @@ class KafkaRelationSuite extends QueryTest with BeforeAndAfter with SharedSQLCon .option("startingOffsets", "earliest") .option("endingOffsets", "latest") .load() - var df = reader.selectExpr("CAST(value AS STRING)") + .selectExpr("CAST(value AS STRING)") checkAnswer(df, (0 to 20).map(_.toString).toDF) - // "latest" should late bind to the current (latest) offset in the reader + // "latest" should late bind to the current (latest) offset in the df testUtils.sendMessages(topic, (21 to 29).map(_.toString).toArray, Some(2)) - df = reader.selectExpr("CAST(value AS STRING)") checkAnswer(df, (0 to 29).map(_.toString).toDF) } @@ -118,7 +117,7 @@ class KafkaRelationSuite extends QueryTest with BeforeAndAfter with SharedSQLCon new TopicPartition(topic, 2) -> 1L // explicit offset happens to = the latest ) val endingOffsets = JsonUtils.partitionOffsets(endPartitionOffsets) - val reader = spark + val df = spark .read .format("kafka") .option("kafka.bootstrap.servers", testUtils.brokerAddress) @@ -126,17 +125,15 @@ class KafkaRelationSuite extends QueryTest with BeforeAndAfter with SharedSQLCon .option("startingOffsets", startingOffsets) .option("endingOffsets", endingOffsets) .load() - var df = reader.selectExpr("CAST(value AS STRING)") + .selectExpr("CAST(value as STRING)") checkAnswer(df, (0 to 20).map(_.toString).toDF) // static offset partition 2, nothing should change testUtils.sendMessages(topic, (31 to 39).map(_.toString).toArray, Some(2)) - df = reader.selectExpr("CAST(value AS STRING)") checkAnswer(df, (0 to 20).map(_.toString).toDF) // latest offset partition 1, should change testUtils.sendMessages(topic, (21 to 30).map(_.toString).toArray, Some(1)) - df = reader.selectExpr("CAST(value AS STRING)") checkAnswer(df, (0 to 30).map(_.toString).toDF) } @@ -147,7 +144,7 @@ class KafkaRelationSuite extends QueryTest with BeforeAndAfter with SharedSQLCon testUtils.sendMessages(topic, (0 to 10).map(_.toString).toArray, Some(0)) // Specify explicit earliest and latest offset values - val reader = spark + val df = spark .read .format("kafka") .option("kafka.bootstrap.servers", testUtils.brokerAddress) @@ -155,10 +152,56 @@ class KafkaRelationSuite extends QueryTest with BeforeAndAfter with SharedSQLCon .option("startingOffsets", "earliest") .option("endingOffsets", "latest") .load() - var df = reader.selectExpr("CAST(value AS STRING)") + .selectExpr("CAST(value AS STRING)") checkAnswer(df.union(df), ((0 to 10) ++ (0 to 10)).map(_.toString).toDF) } + test("test late binding start offsets") { + try { + // First, establish a new KafkaUtils instance that will clear + // all messages when cleanupLogs is called. + if (testUtils != null) { + testUtils.teardown() + testUtils = null + } + // The following settings will ensure that all log entries + // are removed following a call to cleanupLogs + val brokerProps = Map[String, Object]( + "log.retention.bytes" -> 1.asInstanceOf[AnyRef], // retain nothing + "log.retention.ms" -> 1.asInstanceOf[AnyRef] // no wait time + ) + testUtils = new KafkaTestUtils(withBrokerProps = Some(brokerProps)) + testUtils.setup() + + val topic = newTopic() + testUtils.createTopic(topic, partitions = 1) + testUtils.sendMessages(topic, (0 to 9).map(_.toString).toArray, Some(0)) + // Specify explicit earliest and latest offset values + val df = spark + .read + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("subscribe", topic) + .option("startingOffsets", "earliest") + .option("endingOffsets", "latest") + .load() + .selectExpr("CAST(value AS STRING)") + checkAnswer(df, (0 to 9).map(_.toString).toDF) + // Blow away current set of messages. + testUtils.cleanupLogs() + // Add some more data, but do not call cleanup + testUtils.sendMessages(topic, (10 to 19).map(_.toString).toArray, Some(0)) + // Ensure that we late bind to the new starting position + checkAnswer(df, (10 to 19).map(_.toString).toDF) + } finally { + if (testUtils != null) { + testUtils.teardown() + } + testUtils = new KafkaTestUtils + testUtils.setup() + } + } + test("bad source options") { def testBadOptions(options: (String, String)*)(expectedMsgs: String*): Unit = { val ex = intercept[IllegalArgumentException] { 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 7e60410c90d1..90cec165d2ba 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 @@ -50,7 +50,7 @@ import org.apache.spark.SparkConf * * The reason to put Kafka test utility class in src is to test Python related Kafka APIs. */ -class KafkaTestUtils extends Logging { +class KafkaTestUtils(withBrokerProps: Option[Map[String, Object]] = None) extends Logging { // Zookeeper related configurations private val zkHost = "localhost" @@ -249,6 +249,24 @@ class KafkaTestUtils extends Logging { offsets } + def cleanupLogs(): Unit = { + server.logManager.cleanupLogs() + } + + def getEarliestOffsets(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.seekToBeginning(partitions) + val offsets = partitions.asScala.map(p => p -> kc.position(p)).toMap + kc.close() + logInfo("Closed consumer to get latest offsets") + offsets + } + def getLatestOffsets(topics: Set[String]): Map[TopicPartition, Long] = { val kc = new KafkaConsumer[String, String](consumerConfiguration) logInfo("Created consumer to get latest offsets") @@ -274,6 +292,11 @@ class KafkaTestUtils extends Logging { props.put("log.flush.interval.messages", "1") props.put("replica.socket.timeout.ms", "1500") props.put("delete.topic.enable", "true") + withBrokerProps.map { p => + p.foreach { + case (key, value) => props.put(key, value) + } + } props } From 79d335e697ba5af3f02f524d39c16e61d2cc73d9 Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Fri, 27 Jan 2017 10:44:50 -0800 Subject: [PATCH 13/40] update to late binding logic --- .../spark/sql/kafka010/KafkaSourceRDD.scala | 36 +++++++++---------- 1 file changed, 17 insertions(+), 19 deletions(-) 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 61bd12d6e232..1d6e69843745 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 @@ -124,25 +124,6 @@ private[kafka010] class KafkaSourceRDD( thePart: Partition, context: TaskContext): Iterator[ConsumerRecord[Array[Byte], Array[Byte]]] = { var range = thePart.asInstanceOf[KafkaSourceRDDPartition].offsetRange - if (range.fromOffset < 0 || range.untilOffset < 0) { - // Late bind the offset range - val consumer = CachedKafkaConsumer.getOrCreate( - range.topic, range.partition, executorKafkaParams, reuseKafkaConsumer) - val fromOffset = if (range.fromOffset < 0) { - consumer.rawConsumer.seekToBeginning(ju.Arrays.asList(range.topicPartition)) - consumer.rawConsumer.position(range.topicPartition) - } else { - range.fromOffset - } - val untilOffset = if (range.untilOffset < 0) { - consumer.rawConsumer.seekToEnd(ju.Arrays.asList(range.topicPartition)) - consumer.rawConsumer.position(range.topicPartition) - } else { - range.untilOffset - } - range = KafkaSourceRDDOffsetRange(range.topicPartition, - fromOffset, untilOffset, range.preferredLoc) - } assert( range.fromOffset <= range.untilOffset, s"Beginning offset ${range.fromOffset} is after the ending offset ${range.untilOffset} " + @@ -156,6 +137,23 @@ private[kafka010] class KafkaSourceRDD( new NextIterator[ConsumerRecord[Array[Byte], Array[Byte]]]() { val consumer = CachedKafkaConsumer.getOrCreate( range.topic, range.partition, executorKafkaParams, reuseKafkaConsumer) + if (range.fromOffset < 0 || range.untilOffset < 0) { + // Late bind the offset range + val fromOffset = if (range.fromOffset < 0) { + consumer.rawConsumer.seekToBeginning(ju.Arrays.asList(range.topicPartition)) + consumer.rawConsumer.position(range.topicPartition) + } else { + range.fromOffset + } + val untilOffset = if (range.untilOffset < 0) { + consumer.rawConsumer.seekToEnd(ju.Arrays.asList(range.topicPartition)) + consumer.rawConsumer.position(range.topicPartition) + } else { + range.untilOffset + } + range = KafkaSourceRDDOffsetRange(range.topicPartition, + fromOffset, untilOffset, range.preferredLoc) + } var requestOffset = range.fromOffset override def getNext(): ConsumerRecord[Array[Byte], Array[Byte]] = { From 51291e36fad5f3511ce1d3afc17e2e714dffe2d3 Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Fri, 27 Jan 2017 12:21:36 -0800 Subject: [PATCH 14/40] remove kafka log4j debug --- external/kafka-0-10-sql/src/test/resources/log4j.properties | 1 - 1 file changed, 1 deletion(-) diff --git a/external/kafka-0-10-sql/src/test/resources/log4j.properties b/external/kafka-0-10-sql/src/test/resources/log4j.properties index dee3dedd2ac5..75e3b53a093f 100644 --- a/external/kafka-0-10-sql/src/test/resources/log4j.properties +++ b/external/kafka-0-10-sql/src/test/resources/log4j.properties @@ -25,5 +25,4 @@ log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{ # Ignore messages below warning level from Jetty, because it's a bit verbose log4j.logger.org.spark-project.jetty=WARN -log4j.logger.kafka.log=DEBUG From b597cf1bf2135659de1e25bb7d41aeeac26f87f9 Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Fri, 27 Jan 2017 12:22:33 -0800 Subject: [PATCH 15/40] remove kafka log4j debug --- external/kafka-0-10-sql/src/test/resources/log4j.properties | 1 - 1 file changed, 1 deletion(-) diff --git a/external/kafka-0-10-sql/src/test/resources/log4j.properties b/external/kafka-0-10-sql/src/test/resources/log4j.properties index dee3dedd2ac5..75e3b53a093f 100644 --- a/external/kafka-0-10-sql/src/test/resources/log4j.properties +++ b/external/kafka-0-10-sql/src/test/resources/log4j.properties @@ -25,5 +25,4 @@ log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{ # Ignore messages below warning level from Jetty, because it's a bit verbose log4j.logger.org.spark-project.jetty=WARN -log4j.logger.kafka.log=DEBUG From f5ae3012ec37be0c4a6208ba00438592cd3aa791 Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Tue, 31 Jan 2017 12:17:18 -0800 Subject: [PATCH 16/40] update --- .../spark/sql/kafka010/KafkaWriter.scala | 205 ++++++++++++++++++ 1 file changed, 205 insertions(+) create mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala new file mode 100644 index 000000000000..4f376b394fae --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala @@ -0,0 +1,205 @@ +/* + * 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.ListBuffer +import scala.concurrent.{Await, Future, blocking} +import scala.concurrent.duration._ + +import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord, RecordMetadata} + +import org.apache.spark.{SparkException, TaskContext} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.execution.datasources.FileFormatWriter._ +import org.apache.spark.sql.execution.datasources.PartitioningUtils +import org.apache.spark.sql.execution.{QueryExecution, SQLExecution} +import org.apache.spark.sql.types.{NullType, StringType} +import org.apache.spark.util.Utils + +object KafkaWriter extends Logging { + val TOPIC_ATTRIBUTE_NAME: String = "topic" + val KEY_ATTRIBUTE_NAME: String = "key" + val VALUE_ATTRIBUTE_NAME: String = "value" + + private case class TaskCommitMessage( + sparkStageId: Int, + sparkPartitionId: Int, + writeCommitted: Boolean) extends Serializable + + def write(sparkSession: SparkSession, + queryExecution: QueryExecution, + writerOptions: Map[String, String], + kafkaParameters: ju.Map[String, Object], + defaultTopic: Option[String] = None) = { + + val schema = queryExecution.logical.output + schema.find(p => p.name == TOPIC_ATTRIBUTE_NAME).getOrElse( + if (defaultTopic == None) { + throw new IllegalArgumentException(s"Default topic required when no " + + s"'$TOPIC_ATTRIBUTE_NAME' attribute is present") + } + ) + schema.find(p => p.name == KEY_ATTRIBUTE_NAME).getOrElse( + throw new IllegalArgumentException(s"Required attribute '$KEY_ATTRIBUTE_NAME' not found") + ) + schema.find(p => p.name == VALUE_ATTRIBUTE_NAME).getOrElse( + throw new IllegalArgumentException(s"Required attribute '$VALUE_ATTRIBUTE_NAME' not found") + ) + + val pollTimeoutMs = writerOptions.getOrElse( + "kafkaProducer.pollTimeoutMs", + sparkSession.sparkContext.conf.getTimeAsMs("spark.network.timeout", "120s").toString + ).toLong + + SQLExecution.withNewExecutionId(sparkSession, queryExecution) { + try { + val ret = sparkSession.sparkContext.runJob(queryExecution.toRdd, + (taskContext: TaskContext, iter: Iterator[InternalRow]) => { + executeTask( + iterator = iter, + producerConfiguration = kafkaParameters, + writerOptions = writerOptions, + pollTimeoutMs = pollTimeoutMs, + sparkStageId = taskContext.stageId(), + sparkPartitionId = taskContext.partitionId(), + sparkAttemptNumber = taskContext.attemptNumber(), + inputSchema = schema, + defaultTopic = defaultTopic) + }) + + // logInfo(s"Job ${job.getJobID} committed.") + } catch { + case cause: Throwable => + // logError(s"Aborting job ${job.getJobID}.", cause) + throw new SparkException("Job aborted.", cause) + } + } + } + + /** Writes data out in a single Spark task. */ + private def executeTask( + iterator: Iterator[InternalRow], + producerConfiguration: ju.Map[String, Object], + writerOptions: Map[String, String], + pollTimeoutMs: Long, + sparkStageId: Int, + sparkPartitionId: Int, + sparkAttemptNumber: Int, + inputSchema: Seq[Attribute], + defaultTopic: Option[String]): TaskCommitMessage = { + import scala.concurrent.ExecutionContext.Implicits.global + + val producer = new KafkaProducer[Array[Byte], Array[Byte]](producerConfiguration) + val topicExpression = inputSchema.find(p => p.name == TOPIC_ATTRIBUTE_NAME).getOrElse( + if (defaultTopic == None) { + throw new IllegalStateException()(s"Default topic required when no " + + s"'$TOPIC_ATTRIBUTE_NAME' attribute is present") + } else { + Literal(null, NullType) + } + ).map{c => + if (defaultTopic == None) { + c // return null if we can't fall back on a default value + } else { + // fall back on a default value in case we evaluate c to null + If(IsNull(c), Literal(defaultTopic.get, StringType), c) + }} + // Use to extract the topic from either the Row or default value + val getTopic = UnsafeProjection.create(topicExpression, inputSchema) + + val keyExpression = inputSchema.find(p => p.name == KEY_ATTRIBUTE_NAME).getOrElse( + throw new IllegalStateException(s"Required attribute '$KEY_ATTRIBUTE_NAME' not found") + ) + // Use to extract the key from a Row + val getKey = UnsafeProjection.create(Seq(keyExpression), inputSchema) + + val valueExpression = inputSchema.find(p => p.name == VALUE_ATTRIBUTE_NAME).getOrElse( + throw new IllegalStateException(s"Required attribute '$VALUE_ATTRIBUTE_NAME' not found") + ) + // Use to extract the value from a Row + val getValue = UnsafeProjection.create(Seq(valueExpression), inputSchema) + + try { + Utils.tryWithSafeFinallyAndFailureCallbacks(block = { + // Execute the task to write rows out and commit the task. + val outputPartitions = writeTask.execute(iterator) + writeTask.releaseResources() + (committer.commitTask(taskAttemptContext), outputPartitions) + })(catchBlock = { + // If there is an error, release resource and then abort the task + try { + writeTask.releaseResources() + } finally { + committer.abortTask(taskAttemptContext) + logError(s"Job $jobId aborted.") + } + }) + } catch { + case t: Throwable => + throw new SparkException("Task failed while writing rows", t) + } + var futures = ListBuffer.empty[Future[RecordMetadata]] + while (iterator.hasNext) { + val currentRow = iterator.next() + val topic = getTopic(currentRow).get(0, StringType).toString + val key = getKey(currentRow).getBytes + val value = getValue(currentRow).getBytes + val record = new ProducerRecord[Array[Byte], Array[Byte]](topic, key, value) + futures += Future[RecordMetadata] { + blocking { + producer.send(record).get() + } + } + } + val result = + Await.ready(Future.sequence(futures.toList), Duration.create(pollTimeoutMs, MILLISECONDS)) + if (result.isCompleted) { + TaskCommitMessage(sparkStageId, sparkPartitionId, writeCommitted = true) + } else { + TaskCommitMessage(sparkStageId, sparkPartitionId, writeCommitted = false) + } + } + + /** + * A simple trait for writing out data in a single Spark task, without any concerns about how + * to commit or abort tasks. Exceptions thrown by the implementation of this trait will + * automatically trigger task aborts. + */ + private class KafkaWriteTask( + producerConfiguration: ju.Map[String, Object], + writerOptions: Map[String, String], + inputSchema: Seq[Attribute], + defaultTopic: Option[String]) { + /** + * Writes data out to files, and then returns the list of partition strings written out. + * The list of partitions is sent back to the driver and used to update the catalog. + */ + def execute(iterator: Iterator[InternalRow]): List[Future[RecordMetadata]] { + + } + + def releaseResources(): Unit { + + } + } +} From 2487a7260ec11496aed0135a723e156376c6ff31 Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Tue, 31 Jan 2017 14:29:24 -0800 Subject: [PATCH 17/40] address comments from @zsxwing --- .../spark/sql/kafka010/KafkaOffsetReader.scala | 4 +++- .../apache/spark/sql/kafka010/KafkaRelation.scala | 4 ++-- .../org/apache/spark/sql/kafka010/KafkaSource.scala | 2 +- .../spark/sql/kafka010/KafkaSourceProvider.scala | 4 ++-- .../apache/spark/sql/kafka010/KafkaSourceRDD.scala | 4 ++++ .../org/apache/spark/sql/kafka010/KafkaUtils.scala | 4 ++++ .../apache/spark/sql/kafka010/KafkaTestUtils.scala | 12 ++++-------- 7 files changed, 20 insertions(+), 14 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala index b3b1e8c1b2b0..632cef08355d 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala @@ -117,7 +117,9 @@ private[kafka010] class KafkaOffsetReaderImpl( assert(Thread.currentThread().isInstanceOf[UninterruptibleThread]) // Poll to get the latest assigned partitions consumer.poll(0) - consumer.assignment().asScala.toSet + val partitions = consumer.assignment() + consumer.pause(partitions) + partitions.asScala.toSet } override def fetchSpecificStartingOffsets( diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala index 9727c346396e..c62e5b49cb06 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala @@ -111,10 +111,10 @@ private[kafka010] class KafkaRelation( // Obtain TopicPartition offsets with late binding support kafkaOffsets match { case EarliestOffsets => partitions.map { - case tp => tp -> -2L + case tp => tp -> KafkaUtils.EARLIEST }.toMap case LatestOffsets => partitions.map { - case tp => tp -> -1L + case tp => tp -> KafkaUtils.LATEST }.toMap case SpecificOffsets(partitionOffsets) => validateTopicPartitions(partitions, partitionOffsets) 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 5e14cda4adbc..7ee3c02903dc 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 @@ -123,7 +123,7 @@ private[kafka010] class KafkaSource( private def fetchAndVerify(specificOffsets: Map[TopicPartition, Long]) = { val result = kafkaReader.fetchSpecificStartingOffsets(specificOffsets) specificOffsets.foreach { - case (tp, off) if off != -1 && off != -2 => + case (tp, off) if off != KafkaUtils.LATEST && off != KafkaUtils.EARLIEST => if (result(tp) != off) { reportDataLoss( s"startingOffsets for $tp was $off but consumer reset to ${result(tp)}") 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 54b888ada7a1..83ddb12e3ae9 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 @@ -327,7 +327,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister with Stre throw new IllegalArgumentException("starting relation offset can't be latest") case Some(json) => (SpecificOffsets(JsonUtils.partitionOffsets(json))) .partitionOffsets.foreach { - case (tp, off) if off == -1 => + case (tp, off) if off == KafkaUtils.LATEST => throw new IllegalArgumentException(s"startingOffsets for $tp can't be latest") case _ => // ignore } @@ -340,7 +340,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister with Stre case Some("latest") => // good to go case Some(json) => (SpecificOffsets(JsonUtils.partitionOffsets(json))) .partitionOffsets.foreach { - case (tp, off) if off == -2 => + case (tp, off) if off == KafkaUtils.EARLIEST => throw new IllegalArgumentException(s"ending offset for $tp can't be earliest") case _ => // ignore } 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 1d6e69843745..11c9a8573fe5 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 @@ -140,12 +140,16 @@ private[kafka010] class KafkaSourceRDD( if (range.fromOffset < 0 || range.untilOffset < 0) { // Late bind the offset range val fromOffset = if (range.fromOffset < 0) { + assert(range.fromOffset == KafkaUtils.EARLIEST, + s"earliest offset does not equal ${KafkaUtils.EARLIEST}") consumer.rawConsumer.seekToBeginning(ju.Arrays.asList(range.topicPartition)) consumer.rawConsumer.position(range.topicPartition) } else { range.fromOffset } val untilOffset = if (range.untilOffset < 0) { + assert(range.fromOffset == KafkaUtils.LATEST, + s"latest offset does not equal ${KafkaUtils.LATEST}") consumer.rawConsumer.seekToEnd(ju.Arrays.asList(range.topicPartition)) consumer.rawConsumer.position(range.topicPartition) } else { diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaUtils.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaUtils.scala index 78bdb66655f4..a43a016aaa0c 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaUtils.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaUtils.scala @@ -22,6 +22,10 @@ import org.apache.spark.scheduler.ExecutorCacheTaskLocation private[kafka010] object KafkaUtils { + // Used to denote unbounded offset positions + val LATEST = -1L + val EARLIEST = -2L + def getSortedExecutorList(sc: SparkContext): Array[String] = { val bm = sc.env.blockManager bm.master.getPeers(bm.blockManagerId).toArray 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 90cec165d2ba..2ce2760b7f46 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 @@ -50,7 +50,7 @@ import org.apache.spark.SparkConf * * The reason to put Kafka test utility class in src is to test Python related Kafka APIs. */ -class KafkaTestUtils(withBrokerProps: Option[Map[String, Object]] = None) extends Logging { +class KafkaTestUtils(withBrokerProps: Map[String, Object] = Map.empty) extends Logging { // Zookeeper related configurations private val zkHost = "localhost" @@ -255,7 +255,7 @@ class KafkaTestUtils(withBrokerProps: Option[Map[String, Object]] = None) extend def getEarliestOffsets(topics: Set[String]): Map[TopicPartition, Long] = { val kc = new KafkaConsumer[String, String](consumerConfiguration) - logInfo("Created consumer to get latest offsets") + logInfo("Created consumer to get earliest offsets") kc.subscribe(topics.asJavaCollection) kc.poll(0) val partitions = kc.assignment() @@ -263,7 +263,7 @@ class KafkaTestUtils(withBrokerProps: Option[Map[String, Object]] = None) extend kc.seekToBeginning(partitions) val offsets = partitions.asScala.map(p => p -> kc.position(p)).toMap kc.close() - logInfo("Closed consumer to get latest offsets") + logInfo("Closed consumer to get earliest offsets") offsets } @@ -292,11 +292,7 @@ class KafkaTestUtils(withBrokerProps: Option[Map[String, Object]] = None) extend props.put("log.flush.interval.messages", "1") props.put("replica.socket.timeout.ms", "1500") props.put("delete.topic.enable", "true") - withBrokerProps.map { p => - p.foreach { - case (key, value) => props.put(key, value) - } - } + props.putAll(withBrokerProps.asJava) props } From 789d3afc0163b93aa0f859354221ea7e2374f74a Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Tue, 31 Jan 2017 15:12:43 -0800 Subject: [PATCH 18/40] update --- .../org/apache/spark/sql/kafka010/KafkaSourceRDD.scala | 6 +++--- .../org/apache/spark/sql/kafka010/KafkaRelationSuite.scala | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) 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 11c9a8573fe5..bad66c4aab59 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 @@ -141,15 +141,15 @@ private[kafka010] class KafkaSourceRDD( // Late bind the offset range val fromOffset = if (range.fromOffset < 0) { assert(range.fromOffset == KafkaUtils.EARLIEST, - s"earliest offset does not equal ${KafkaUtils.EARLIEST}") + s"earliest offset ${range.fromOffset} does not equal ${KafkaUtils.EARLIEST}") consumer.rawConsumer.seekToBeginning(ju.Arrays.asList(range.topicPartition)) consumer.rawConsumer.position(range.topicPartition) } else { range.fromOffset } val untilOffset = if (range.untilOffset < 0) { - assert(range.fromOffset == KafkaUtils.LATEST, - s"latest offset does not equal ${KafkaUtils.LATEST}") + assert(range.untilOffset == KafkaUtils.LATEST, + s"latest offset ${range.untilOffset} does not equal ${KafkaUtils.LATEST}") consumer.rawConsumer.seekToEnd(ju.Arrays.asList(range.topicPartition)) consumer.rawConsumer.position(range.topicPartition) } else { diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala index 69169a239994..a58c235af22f 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala @@ -170,7 +170,7 @@ class KafkaRelationSuite extends QueryTest with BeforeAndAfter with SharedSQLCon "log.retention.bytes" -> 1.asInstanceOf[AnyRef], // retain nothing "log.retention.ms" -> 1.asInstanceOf[AnyRef] // no wait time ) - testUtils = new KafkaTestUtils(withBrokerProps = Some(brokerProps)) + testUtils = new KafkaTestUtils(withBrokerProps = brokerProps) testUtils.setup() val topic = newTopic() From e74473b28b160633ab6545fae208981096fe367c Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Wed, 1 Feb 2017 16:58:01 -0800 Subject: [PATCH 19/40] update --- .../sql/kafka010/KafkaOffsetReader.scala | 11 -- .../spark/sql/kafka010/KafkaRelation.scala | 2 +- .../apache/spark/sql/kafka010/KafkaSink.scala | 41 +++++ .../spark/sql/kafka010/KafkaSource.scala | 2 +- .../sql/kafka010/KafkaSourceProvider.scala | 59 +++++++- .../spark/sql/kafka010/KafkaUtils.scala | 11 ++ .../spark/sql/kafka010/KafkaWriter.scala | 142 +++++++++--------- 7 files changed, 176 insertions(+), 92 deletions(-) create mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSink.scala diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala index 632cef08355d..b5b1c8bb49a2 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala @@ -31,7 +31,6 @@ import org.apache.kafka.common.TopicPartition import org.apache.spark.internal.Logging import org.apache.spark.sql.kafka010.KafkaOffsetReader.ConsumerStrategy -import org.apache.spark.sql.types._ import org.apache.spark.util.{ThreadUtils, UninterruptibleThread} @@ -337,16 +336,6 @@ private[kafka010] class UninterruptibleKafkaOffsetReader(kafkaOffsetReader: Kafk private[kafka010] object KafkaOffsetReader { - def kafkaSchema: StructType = StructType(Seq( - StructField("key", BinaryType), - StructField("value", BinaryType), - StructField("topic", StringType), - StructField("partition", IntegerType), - StructField("offset", LongType), - StructField("timestamp", TimestampType), - StructField("timestampType", IntegerType) - )) - sealed trait ConsumerStrategy { def createConsumer(kafkaParams: ju.Map[String, Object]): Consumer[Array[Byte], Array[Byte]] } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala index c62e5b49cb06..9348ba5c0ff7 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala @@ -50,7 +50,7 @@ private[kafka010] class KafkaRelation( sqlContext.sparkContext.conf.getTimeAsMs("spark.network.timeout", "120s").toString ).toLong - override def schema: StructType = KafkaOffsetReader.kafkaSchema + override def schema: StructType = KafkaUtils.kafkaSchema override def buildScan(): RDD[Row] = { // Leverage the KafkaReader to obtain the relevant partition offsets diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSink.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSink.scala new file mode 100644 index 000000000000..2726eafe05fa --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSink.scala @@ -0,0 +1,41 @@ +/* + * 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.spark.internal.Logging +import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.sql.execution.streaming.Sink + +private[kafka010] class KafkaSink( + sqlContext: SQLContext, + executorKafkaParams: ju.Map[String, Object], + defaultTopic: Option[String]) extends Sink with Logging { + var latestBatchId = -1L + + override def addBatch(batchId: Long, data: DataFrame): Unit = { + if (batchId <= latestBatchId) { + logInfo(s"Skipping already committed batch $batchId") + } else { + KafkaWriter.write(sqlContext.sparkSession, + data.queryExecution, executorKafkaParams, defaultTopic) + latestBatchId = batchId + } + } +} 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 7ee3c02903dc..5d38efd1de34 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 @@ -136,7 +136,7 @@ private[kafka010] class KafkaSource( private var currentPartitionOffsets: Option[Map[TopicPartition, Long]] = None - override def schema: StructType = KafkaOffsetReader.kafkaSchema + override def schema: StructType = KafkaUtils.kafkaSchema /** Returns the maximum available offset for this source. */ override def getOffset: Option[Offset] = { 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 83ddb12e3ae9..850433b0c637 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 @@ -26,10 +26,11 @@ 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._ +import org.apache.spark.sql.execution.streaming.{Sink, Source} import org.apache.spark.sql.kafka010.KafkaOffsetReader.{AssignStrategy, SubscribePatternStrategy, SubscribeStrategy} import org.apache.spark.sql.sources._ +import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.StructType /** @@ -37,8 +38,9 @@ import org.apache.spark.sql.types.StructType * 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 DataSourceRegister with StreamSourceProvider - with RelationProvider with Logging { +private[kafka010] class KafkaSourceProvider extends DataSourceRegister + with StreamSourceProvider with StreamSinkProvider + with RelationProvider with CreatableRelationProvider with Logging { import KafkaSourceProvider._ override def shortName(): String = "kafka" @@ -54,7 +56,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister with Stre parameters: Map[String, String]): (String, StructType) = { validateOptions(parameters, Stream) require(schema.isEmpty, "Kafka source has a fixed schema and cannot be set with a custom one") - (shortName(), KafkaOffsetReader.kafkaSchema) + (shortName(), KafkaUtils.kafkaSchema) } override def createSource( @@ -154,6 +156,52 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister with Stre endingRelationOffsets) } + override def createSink(sqlContext: SQLContext, + parameters: Map[String, String], + partitionColumns: Seq[String], + outputMode: OutputMode): Sink = { + + if (outputMode != OutputMode.Append()) { + throw new IllegalArgumentException(s"Kafka supports ${OutputMode.Append()} only") + } + val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase, v) } + val defaultTopic = caseInsensitiveParams.get(DEFAULT_TOPIC).map(_.trim.toLowerCase) + val specifiedKafkaParams = + parameters + .keySet + .filter(_.toLowerCase.startsWith("kafka.")) + .map { k => k.drop(6).toString -> parameters(k) } + .toMap + new KafkaSink(sqlContext, + new ju.HashMap[String, Object](specifiedKafkaParams.asJava), + defaultTopic) + } + + override def createRelation( + outerSQLContext: SQLContext, + mode: SaveMode, + parameters: Map[String, String], + data: DataFrame): BaseRelation = { + logInfo(s"Save mode = $mode") + val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase, v) } + val defaultTopic = caseInsensitiveParams.get(DEFAULT_TOPIC).map(_.trim.toLowerCase) + val specifiedKafkaParams = + parameters + .keySet + .filter(_.toLowerCase.startsWith("kafka.")) + .map { k => k.drop(6).toString -> parameters(k) } + .toMap + KafkaWriter.write(outerSQLContext.sparkSession, data.queryExecution, + new ju.HashMap[String, Object](specifiedKafkaParams.asJava), + defaultTopic) + + new BaseRelation { + override def sqlContext: SQLContext = outerSQLContext + + override def schema: StructType = KafkaUtils.kafkaSchema + } + } + private def kafkaParamsForDriver(specifiedKafkaParams: Map[String, String]) = ConfigUpdater("source", specifiedKafkaParams) .set(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, deserClassName) @@ -380,6 +428,7 @@ private[kafka010] object KafkaSourceProvider { private val STARTING_OFFSETS_OPTION_KEY = "startingoffsets" private val ENDING_OFFSETS_OPTION_KEY = "endingoffsets" private val FAIL_ON_DATA_LOSS_OPTION_KEY = "failondataloss" + private val DEFAULT_TOPIC = "defaulttopic" // Used to check parameters for different source modes private sealed trait Mode diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaUtils.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaUtils.scala index a43a016aaa0c..54b025de9ad6 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaUtils.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaUtils.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.kafka010 import org.apache.spark.SparkContext import org.apache.spark.scheduler.ExecutorCacheTaskLocation +import org.apache.spark.sql.types._ private[kafka010] object KafkaUtils { @@ -26,6 +27,16 @@ private[kafka010] object KafkaUtils { val LATEST = -1L val EARLIEST = -2L + def kafkaSchema: StructType = StructType(Seq( + StructField("key", BinaryType), + StructField("value", BinaryType), + StructField("topic", StringType), + StructField("partition", IntegerType), + StructField("offset", LongType), + StructField("timestamp", TimestampType), + StructField("timestampType", IntegerType) + )) + def getSortedExecutorList(sc: SparkContext): Array[String] = { val bm = sc.env.blockManager bm.master.getPeers(bm.blockManagerId).toArray diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala index 4f376b394fae..8bee8c9ddb3f 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala @@ -20,8 +20,7 @@ package org.apache.spark.sql.kafka010 import java.{util => ju} import scala.collection.mutable.ListBuffer -import scala.concurrent.{Await, Future, blocking} -import scala.concurrent.duration._ +import scala.concurrent.{blocking, Future} import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord, RecordMetadata} @@ -30,8 +29,6 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.execution.datasources.FileFormatWriter._ -import org.apache.spark.sql.execution.datasources.PartitioningUtils import org.apache.spark.sql.execution.{QueryExecution, SQLExecution} import org.apache.spark.sql.types.{NullType, StringType} import org.apache.spark.util.Utils @@ -48,9 +45,8 @@ object KafkaWriter extends Logging { def write(sparkSession: SparkSession, queryExecution: QueryExecution, - writerOptions: Map[String, String], kafkaParameters: ju.Map[String, Object], - defaultTopic: Option[String] = None) = { + defaultTopic: Option[String] = None): Unit = { val schema = queryExecution.logical.output schema.find(p => p.name == TOPIC_ATTRIBUTE_NAME).getOrElse( @@ -65,12 +61,6 @@ object KafkaWriter extends Logging { schema.find(p => p.name == VALUE_ATTRIBUTE_NAME).getOrElse( throw new IllegalArgumentException(s"Required attribute '$VALUE_ATTRIBUTE_NAME' not found") ) - - val pollTimeoutMs = writerOptions.getOrElse( - "kafkaProducer.pollTimeoutMs", - sparkSession.sparkContext.conf.getTimeAsMs("spark.network.timeout", "120s").toString - ).toLong - SQLExecution.withNewExecutionId(sparkSession, queryExecution) { try { val ret = sparkSession.sparkContext.runJob(queryExecution.toRdd, @@ -78,8 +68,6 @@ object KafkaWriter extends Logging { executeTask( iterator = iter, producerConfiguration = kafkaParameters, - writerOptions = writerOptions, - pollTimeoutMs = pollTimeoutMs, sparkStageId = taskContext.stageId(), sparkPartitionId = taskContext.partitionId(), sparkAttemptNumber = taskContext.attemptNumber(), @@ -100,19 +88,58 @@ object KafkaWriter extends Logging { private def executeTask( iterator: Iterator[InternalRow], producerConfiguration: ju.Map[String, Object], - writerOptions: Map[String, String], - pollTimeoutMs: Long, sparkStageId: Int, sparkPartitionId: Int, sparkAttemptNumber: Int, inputSchema: Seq[Attribute], defaultTopic: Option[String]): TaskCommitMessage = { - import scala.concurrent.ExecutionContext.Implicits.global + val writeTask = new KafkaWriteTask( + producerConfiguration, inputSchema, defaultTopic) + try { + Utils.tryWithSafeFinallyAndFailureCallbacks(block = { + // Execute the task to write rows out and commit the task. + writeTask.execute(iterator) + writeTask.releaseResources() + })(catchBlock = { + // If there is an error, release resource and then abort the task + try { + writeTask.releaseResources() + } finally { + logError(s"Stage $sparkStageId, task $sparkPartitionId aborted.") + } + }) + } catch { + case t: Throwable => + throw new SparkException("Task failed while writing rows", t) + } + + if (writeTask.failedWrites.size == 0) { + assert(writeTask.confirmedWrites == writeTask.producerRecordCount, + s"Confirmed writes ${writeTask.confirmedWrites} != " + + s"records written ${writeTask.producerRecordCount}") + TaskCommitMessage(sparkStageId, sparkPartitionId, writeCommitted = true) + } else { + TaskCommitMessage(sparkStageId, sparkPartitionId, writeCommitted = false) + } + } + + /** + * A simple trait for writing out data in a single Spark task, without any concerns about how + * to commit or abort tasks. Exceptions thrown by the implementation of this trait will + * automatically trigger task aborts. + */ + private class KafkaWriteTask( + producerConfiguration: ju.Map[String, Object], + inputSchema: Seq[Attribute], + defaultTopic: Option[String]) { + var producerRecordCount = 0 + var confirmedWrites = 0 + var failedWrites = ListBuffer.empty[Throwable] val producer = new KafkaProducer[Array[Byte], Array[Byte]](producerConfiguration) val topicExpression = inputSchema.find(p => p.name == TOPIC_ATTRIBUTE_NAME).getOrElse( if (defaultTopic == None) { - throw new IllegalStateException()(s"Default topic required when no " + + throw new IllegalStateException(s"Default topic required when no " + s"'$TOPIC_ATTRIBUTE_NAME' attribute is present") } else { Literal(null, NullType) @@ -139,67 +166,34 @@ object KafkaWriter extends Logging { // Use to extract the value from a Row val getValue = UnsafeProjection.create(Seq(valueExpression), inputSchema) - try { - Utils.tryWithSafeFinallyAndFailureCallbacks(block = { - // Execute the task to write rows out and commit the task. - val outputPartitions = writeTask.execute(iterator) - writeTask.releaseResources() - (committer.commitTask(taskAttemptContext), outputPartitions) - })(catchBlock = { - // If there is an error, release resource and then abort the task - try { - writeTask.releaseResources() - } finally { - committer.abortTask(taskAttemptContext) - logError(s"Job $jobId aborted.") + /** + * Writes key value data out to topics. + */ + def execute(iterator: Iterator[InternalRow]): Unit = { + import scala.concurrent.ExecutionContext.Implicits.global + while (iterator.hasNext) { + val currentRow = iterator.next() + val topic = getTopic(currentRow).get(0, StringType).toString + val key = getKey(currentRow).getBytes + val value = getValue(currentRow).getBytes + val record = new ProducerRecord[Array[Byte], Array[Byte]](topic, key, value) + val future = Future[RecordMetadata] { + blocking { + producer.send(record).get() + } } - }) - } catch { - case t: Throwable => - throw new SparkException("Task failed while writing rows", t) - } - var futures = ListBuffer.empty[Future[RecordMetadata]] - while (iterator.hasNext) { - val currentRow = iterator.next() - val topic = getTopic(currentRow).get(0, StringType).toString - val key = getKey(currentRow).getBytes - val value = getValue(currentRow).getBytes - val record = new ProducerRecord[Array[Byte], Array[Byte]](topic, key, value) - futures += Future[RecordMetadata] { - blocking { - producer.send(record).get() + future.onSuccess { + case rm => confirmedWrites += 1 + } + future.onFailure { + case e => failedWrites += e } } + producer.flush() } - val result = - Await.ready(Future.sequence(futures.toList), Duration.create(pollTimeoutMs, MILLISECONDS)) - if (result.isCompleted) { - TaskCommitMessage(sparkStageId, sparkPartitionId, writeCommitted = true) - } else { - TaskCommitMessage(sparkStageId, sparkPartitionId, writeCommitted = false) - } - } - - /** - * A simple trait for writing out data in a single Spark task, without any concerns about how - * to commit or abort tasks. Exceptions thrown by the implementation of this trait will - * automatically trigger task aborts. - */ - private class KafkaWriteTask( - producerConfiguration: ju.Map[String, Object], - writerOptions: Map[String, String], - inputSchema: Seq[Attribute], - defaultTopic: Option[String]) { - /** - * Writes data out to files, and then returns the list of partition strings written out. - * The list of partitions is sent back to the driver and used to update the catalog. - */ - def execute(iterator: Iterator[InternalRow]): List[Future[RecordMetadata]] { - - } - - def releaseResources(): Unit { + def releaseResources(): Unit = { + producer.close() } } } From 73df054e87eb755bc0b25ee15e7fdd501cd0c10c Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Thu, 2 Feb 2017 14:43:39 -0800 Subject: [PATCH 20/40] update --- .../spark/sql/kafka010/KafkaSinkSuite.scala | 66 +++++++++++++++++++ 1 file changed, 66 insertions(+) create mode 100644 external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala new file mode 100644 index 000000000000..f53bc7c9c6d8 --- /dev/null +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala @@ -0,0 +1,66 @@ +/* + * 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 org.scalatest.time.SpanSugar._ + +import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.streaming.StreamTest +import org.apache.spark.sql.test.SharedSQLContext + +class KafkaSinkSuite extends StreamTest with SharedSQLContext { + import testImplicits._ + + protected var testUtils: KafkaTestUtils = _ + + override val streamingTimeout = 30.seconds + + override def beforeAll(): Unit = { + super.beforeAll() + testUtils = new KafkaTestUtils + testUtils.setup() + } + + override def afterAll(): Unit = { + if (testUtils != null) { + testUtils.teardown() + testUtils = null + super.afterAll() + } + } + + private val topicId = new AtomicInteger(0) + + private def newTopic(): String = s"topic-${topicId.getAndIncrement()}" + + test("write to stream") { + val input = MemoryStream[Int] + val df = input.toDF().selectExpr("CAST(value as BYTE) key", "CAST(value as BYTE) value") + df.printSchema() + val query = df.writeStream + .format("memory") + .outputMode("append") + .queryName("memStream") + .start() + input.addData(1, 2, 3) + query.processAllAvailable() + } + +} From 5b48fc65ac08e8ed4a09edd0d346990d40d042e0 Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Thu, 2 Feb 2017 17:20:03 -0800 Subject: [PATCH 21/40] address comments from @tdas --- .../sql/kafka010/CachedKafkaConsumer.scala | 62 ++-- .../spark/sql/kafka010/ConsumerStrategy.scala | 83 +++++ .../spark/sql/kafka010/KafkaOffsets.scala | 23 +- .../spark/sql/kafka010/KafkaRelation.scala | 20 +- .../spark/sql/kafka010/KafkaSource.scala | 38 +- .../sql/kafka010/KafkaSourceProvider.scala | 64 ++-- .../spark/sql/kafka010/KafkaSourceRDD.scala | 55 +-- ... => KafkaTopicPartitionOffsetReader.scala} | 331 +++++++----------- .../spark/sql/kafka010/KafkaUtils.scala | 42 --- .../sql/kafka010/KafkaRelationSuite.scala | 87 ++--- .../spark/sql/kafka010/KafkaSourceSuite.scala | 2 +- 11 files changed, 388 insertions(+), 419 deletions(-) create mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/ConsumerStrategy.scala rename external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/{KafkaOffsetReader.scala => KafkaTopicPartitionOffsetReader.scala} (50%) delete mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaUtils.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 d8336590d5d3..663b5358b1ce 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 @@ -42,7 +42,7 @@ private[kafka010] case class CachedKafkaConsumer private( private val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG).asInstanceOf[String] - var rawConsumer = createConsumer + private var consumer = createConsumer /** Iterator to the already fetch data */ private var fetchedData = ju.Collections.emptyIterator[ConsumerRecord[Array[Byte], Array[Byte]]] @@ -57,6 +57,20 @@ private[kafka010] case class CachedKafkaConsumer private( c } + case class AvailableOffsetRange(earliest: Long, latest: Long) + + /** + * Return the available offset range of the current partition. It's a pair of the earliest offset + * and the latest offset. + */ + def getAvailableOffsetRange(): AvailableOffsetRange = { + consumer.seekToBeginning(Set(topicPartition).asJava) + val earliestOffset = consumer.position(topicPartition) + consumer.seekToEnd(Set(topicPartition).asJava) + val latestOffset = consumer.position(topicPartition) + AvailableOffsetRange(earliestOffset, latestOffset) + } + /** * Get the record for the given offset if available. Otherwise it will either throw error * (if failOnDataLoss = true), or return the next available offset within [offset, untilOffset), @@ -107,9 +121,9 @@ private[kafka010] case class CachedKafkaConsumer private( * `UNKNOWN_OFFSET`. */ private def getEarliestAvailableOffsetBetween(offset: Long, untilOffset: Long): Long = { - val (earliestOffset, latestOffset) = getAvailableOffsetRange() - logWarning(s"Some data may be lost. Recovering from the earliest offset: $earliestOffset") - if (offset >= latestOffset || earliestOffset >= untilOffset) { + val range = getAvailableOffsetRange() + logWarning(s"Some data may be lost. Recovering from the earliest offset: ${range.earliest}") + if (offset >= range.latest || range.earliest >= untilOffset) { // [offset, untilOffset) and [earliestOffset, latestOffset) have no overlap, // either // -------------------------------------------------------- @@ -124,13 +138,13 @@ private[kafka010] case class CachedKafkaConsumer private( // offset untilOffset earliestOffset latestOffset val warningMessage = s""" - |The current available offset range is [$earliestOffset, $latestOffset). + |The current available offset range is [${range.earliest}, ${range.latest}). | Offset ${offset} is out of range, and records in [$offset, $untilOffset) will be | skipped ${additionalMessage(failOnDataLoss = false)} """.stripMargin logWarning(warningMessage) UNKNOWN_OFFSET - } else if (offset >= earliestOffset) { + } else if (offset >= range.earliest) { // ----------------------------------------------------------------------------- // ^ ^ ^ ^ // | | | | @@ -149,12 +163,12 @@ private[kafka010] case class CachedKafkaConsumer private( // offset earliestOffset min(untilOffset,latestOffset) max(untilOffset, latestOffset) val warningMessage = s""" - |The current available offset range is [$earliestOffset, $latestOffset). - | Offset ${offset} is out of range, and records in [$offset, $earliestOffset) will be + |The current available offset range is [${range.earliest}, ${range.latest}). + | Offset ${offset} is out of range, and records in [$offset, ${range.earliest}) will be | skipped ${additionalMessage(failOnDataLoss = false)} """.stripMargin logWarning(warningMessage) - earliestOffset + range.earliest } } @@ -183,8 +197,8 @@ private[kafka010] case class CachedKafkaConsumer private( // - `offset` is out of range so that Kafka returns nothing. Just throw // `OffsetOutOfRangeException` to let the caller handle it. // - Cannot fetch any data before timeout. TimeoutException will be thrown. - val (earliestOffset, latestOffset) = getAvailableOffsetRange() - if (offset < earliestOffset || offset >= latestOffset) { + val range = getAvailableOffsetRange() + if (offset < range.earliest || offset >= range.latest) { throw new OffsetOutOfRangeException( Map(topicPartition -> java.lang.Long.valueOf(offset)).asJava) } else { @@ -224,8 +238,8 @@ private[kafka010] case class CachedKafkaConsumer private( /** Create a new consumer and reset cached states */ private def resetConsumer(): Unit = { - rawConsumer.close() - rawConsumer = createConsumer + consumer.close() + consumer = createConsumer resetFetchedData() } @@ -271,31 +285,19 @@ private[kafka010] case class CachedKafkaConsumer private( } } - private def close(): Unit = rawConsumer.close() + private def close(): Unit = consumer.close() private def seek(offset: Long): Unit = { logDebug(s"Seeking to $groupId $topicPartition $offset") - rawConsumer.seek(topicPartition, offset) + consumer.seek(topicPartition, offset) } private def poll(pollTimeoutMs: Long): Unit = { - val p = rawConsumer.poll(pollTimeoutMs) + val p = consumer.poll(pollTimeoutMs) val r = p.records(topicPartition) logDebug(s"Polled $groupId ${p.partitions()} ${r.size}") fetchedData = r.iterator } - - /** - * Return the available offset range of the current partition. It's a pair of the earliest offset - * and the latest offset. - */ - private def getAvailableOffsetRange(): (Long, Long) = { - rawConsumer.seekToBeginning(Set(topicPartition).asJava) - val earliestOffset = rawConsumer.position(topicPartition) - rawConsumer.seekToEnd(Set(topicPartition).asJava) - val latestOffset = rawConsumer.position(topicPartition) - (earliestOffset, latestOffset) - } } private[kafka010] object CachedKafkaConsumer extends Logging { @@ -335,14 +337,14 @@ private[kafka010] object CachedKafkaConsumer extends Logging { topic: String, partition: Int, kafkaParams: ju.Map[String, Object], - reuse: Boolean): CachedKafkaConsumer = synchronized { + reuseExistingIfPresent: Boolean): 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 (!reuse || TaskContext.get != null && TaskContext.get.attemptNumber > 1) { + if (!reuseExistingIfPresent || TaskContext.get != null && TaskContext.get.attemptNumber > 1) { val removedConsumer = cache.remove(key) if (removedConsumer != null) { removedConsumer.close() diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/ConsumerStrategy.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/ConsumerStrategy.scala new file mode 100644 index 000000000000..947202308c92 --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/ConsumerStrategy.scala @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.kafka010 + +import java.{util => ju} + +import scala.collection.JavaConverters._ + +import org.apache.kafka.clients.consumer.{Consumer, KafkaConsumer} +import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener +import org.apache.kafka.common.TopicPartition + +/** + * Subscribe allows you to subscribe to a fixed collection of topics. + * SubscribePattern allows you to use a regex to specify topics of interest. + * Note that unlike the 0.8 integration, * using Subscribe or SubscribePattern + * should respond to adding partitions during a running stream. + * Finally, Assign allows you to specify a fixed collection of partitions. + * All three strategies have overloaded constructors that allow you to specify + * the starting offset for a particular partition. + */ +sealed trait ConsumerStrategy { + def createConsumer(kafkaParams: ju.Map[String, Object]): Consumer[Array[Byte], Array[Byte]] +} + +/** + * Specify a fixed collection of partitions. + */ +case class AssignStrategy(partitions: Array[TopicPartition]) extends ConsumerStrategy { + override def createConsumer( + kafkaParams: ju.Map[String, Object]): Consumer[Array[Byte], Array[Byte]] = { + val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams) + consumer.assign(ju.Arrays.asList(partitions: _*)) + consumer + } + + override def toString: String = s"Assign[${partitions.mkString(", ")}]" +} + +/** + * Subscribe to a fixed collection of topics. + */ +case class SubscribeStrategy(topics: Seq[String]) extends ConsumerStrategy { + override def createConsumer( + kafkaParams: ju.Map[String, Object]): Consumer[Array[Byte], Array[Byte]] = { + val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams) + consumer.subscribe(topics.asJava) + consumer + } + + override def toString: String = s"Subscribe[${topics.mkString(", ")}]" +} + +/** + * Use a regex to specify topics of interest. + */ +case class SubscribePatternStrategy(topicPattern: String) extends ConsumerStrategy { + override def createConsumer( + kafkaParams: ju.Map[String, Object]): Consumer[Array[Byte], Array[Byte]] = { + val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams) + consumer.subscribe( + ju.regex.Pattern.compile(topicPattern), + new NoOpConsumerRebalanceListener()) + consumer + } + + override def toString: String = s"SubscribePattern[$topicPattern]" +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsets.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsets.scala index 5ac206604608..6c5676453b87 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsets.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsets.scala @@ -19,14 +19,31 @@ package org.apache.spark.sql.kafka010 import org.apache.kafka.common.TopicPartition -/* - * Values that can be specified for config startingOffsets +/** + * Values that can be specified to configure starting, + * ending, and specific offsets. */ private[kafka010] sealed trait KafkaOffsets +/** + * Bind to the earliest offsets in Kafka + */ private[kafka010] case object EarliestOffsets extends KafkaOffsets +/** + * Bind to the latest offsets in Kafka + */ private[kafka010] case object LatestOffsets extends KafkaOffsets +/** + * Bind to the specific offsets. A offset == -1 binds to the latest + * offset, and offset == -2 binds to the earliest offset. + */ private[kafka010] case class SpecificOffsets( - partitionOffsets: Map[TopicPartition, Long]) extends KafkaOffsets + partitionOffsets: Map[TopicPartition, Long]) extends KafkaOffsets + +private[kafka010] object KafkaOffsets { + // Used to denote unbounded offset positions + val LATEST = -1L + val EARLIEST = -2L +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala index c62e5b49cb06..b233ba03c20f 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala @@ -32,13 +32,13 @@ import org.apache.spark.unsafe.types.UTF8String private[kafka010] class KafkaRelation( - override val sqlContext: SQLContext, - kafkaReader: KafkaOffsetReader, - executorKafkaParams: ju.Map[String, Object], - sourceOptions: Map[String, String], - failOnDataLoss: Boolean, - startingOffsets: KafkaOffsets, - endingOffsets: KafkaOffsets) + override val sqlContext: SQLContext, + kafkaReader: KafkaTopicPartitionOffsetReader, + executorKafkaParams: ju.Map[String, Object], + sourceOptions: Map[String, String], + failOnDataLoss: Boolean, + startingOffsets: KafkaOffsets, + endingOffsets: KafkaOffsets) extends BaseRelation with TableScan with Logging { assert(startingOffsets != LatestOffsets, "Starting offset not allowed to be set to latest offsets.") @@ -50,7 +50,7 @@ private[kafka010] class KafkaRelation( sqlContext.sparkContext.conf.getTimeAsMs("spark.network.timeout", "120s").toString ).toLong - override def schema: StructType = KafkaOffsetReader.kafkaSchema + override def schema: StructType = KafkaTopicPartitionOffsetReader.kafkaSchema override def buildScan(): RDD[Row] = { // Leverage the KafkaReader to obtain the relevant partition offsets @@ -111,10 +111,10 @@ private[kafka010] class KafkaRelation( // Obtain TopicPartition offsets with late binding support kafkaOffsets match { case EarliestOffsets => partitions.map { - case tp => tp -> KafkaUtils.EARLIEST + case tp => tp -> KafkaOffsets.EARLIEST }.toMap case LatestOffsets => partitions.map { - case tp => tp -> KafkaUtils.LATEST + case tp => tp -> KafkaOffsets.LATEST }.toMap case SpecificOffsets(partitionOffsets) => validateTopicPartitions(partitions, partitionOffsets) 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 7ee3c02903dc..b954d4a0fa55 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 @@ -23,7 +23,9 @@ import java.nio.charset.StandardCharsets import org.apache.kafka.common.TopicPartition +import org.apache.spark.SparkContext import org.apache.spark.internal.Logging +import org.apache.spark.scheduler.ExecutorCacheTaskLocation import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.util.DateTimeUtils @@ -43,12 +45,12 @@ import org.apache.spark.unsafe.types.UTF8String * * - The [[KafkaSource]] written to do the following. * - * - As soon as the source is created, the pre-configured [[KafkaOffsetReader]] + * - As soon as the source is created, the pre-configured [[KafkaTopicPartitionOffsetReader]] * is used to query the initial offsets that this source should * start reading from. This is used to create the first batch. * - * - `getOffset()` uses the [[KafkaOffsetReader]] to query the latest available offsets, which are - * returned as a [[KafkaSourceOffset]]. + * - `getOffset()` uses the [[KafkaTopicPartitionOffsetReader]] 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 @@ -68,7 +70,7 @@ import org.apache.spark.unsafe.types.UTF8String */ private[kafka010] class KafkaSource( sqlContext: SQLContext, - kafkaReader: KafkaOffsetReader, + kafkaReader: KafkaTopicPartitionOffsetReader, executorKafkaParams: ju.Map[String, Object], sourceOptions: Map[String, String], metadataPath: String, @@ -121,9 +123,9 @@ private[kafka010] class KafkaSource( } private def fetchAndVerify(specificOffsets: Map[TopicPartition, Long]) = { - val result = kafkaReader.fetchSpecificStartingOffsets(specificOffsets) + val result = kafkaReader.fetchSpecificOffsets(specificOffsets) specificOffsets.foreach { - case (tp, off) if off != KafkaUtils.LATEST && off != KafkaUtils.EARLIEST => + case (tp, off) if off != KafkaOffsets.LATEST && off != KafkaOffsets.EARLIEST => if (result(tp) != off) { reportDataLoss( s"startingOffsets for $tp was $off but consumer reset to ${result(tp)}") @@ -136,7 +138,7 @@ private[kafka010] class KafkaSource( private var currentPartitionOffsets: Option[Map[TopicPartition, Long]] = None - override def schema: StructType = KafkaOffsetReader.kafkaSchema + override def schema: StructType = KafkaTopicPartitionOffsetReader.kafkaSchema /** Returns the maximum available offset for this source. */ override def getOffset: Option[Offset] = { @@ -163,7 +165,7 @@ private[kafka010] class KafkaSource( limit: Long, from: Map[TopicPartition, Long], until: Map[TopicPartition, Long]): Map[TopicPartition, Long] = { - val fromNew = kafkaReader.fetchNewPartitionEarliestOffsets(until.keySet.diff(from.keySet).toSeq) + val fromNew = kafkaReader.fetchEarliestOffsets(until.keySet.diff(from.keySet).toSeq) val sizes = until.flatMap { case (tp, end) => // If begin isn't defined, something's wrong, but let alert logic in getBatch handle it @@ -213,7 +215,7 @@ private[kafka010] class KafkaSource( // Find the new partitions, and get their earliest offsets val newPartitions = untilPartitionOffsets.keySet.diff(fromPartitionOffsets.keySet) - val newPartitionOffsets = kafkaReader.fetchNewPartitionEarliestOffsets(newPartitions.toSeq) + val newPartitionOffsets = kafkaReader.fetchEarliestOffsets(newPartitions.toSeq) if (newPartitionOffsets.keySet != newPartitions) { // We cannot get from offsets for some partitions. It means they got deleted. val deletedPartitions = newPartitions.diff(newPartitionOffsets.keySet) @@ -239,7 +241,7 @@ private[kafka010] class KafkaSource( }.toSeq logDebug("TopicPartitions: " + topicPartitions.mkString(", ")) - val sortedExecutors = KafkaUtils.getSortedExecutorList(sc) + val sortedExecutors = getSortedExecutorList(sc) val numExecutors = sortedExecutors.length logDebug("Sorted executors: " + sortedExecutors.mkString(", ")) @@ -256,7 +258,7 @@ private[kafka010] class KafkaSource( 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(KafkaUtils.floorMod(tp.hashCode, numExecutors))) + Some(sortedExecutors(floorMod(tp.hashCode, numExecutors))) } else None KafkaSourceRDDOffsetRange(tp, fromOffset, untilOffset, preferredLoc) }.filter { range => @@ -331,4 +333,18 @@ private[kafka010] object KafkaSource { | topic was processed. If you don't want your streaming query to fail on such cases, set the | source option "failOnDataLoss" to "false". """.stripMargin + + def getSortedExecutorList(sc: SparkContext): Array[String] = { + val bm = sc.env.blockManager + bm.master.getPeers(bm.blockManagerId).toArray + .map(x => ExecutorCacheTaskLocation(x.host, x.executorId)) + .sortWith(compare) + .map(_.toString) + } + + private def compare(a: ExecutorCacheTaskLocation, b: ExecutorCacheTaskLocation): Boolean = { + if (a.host == b.host) { a.executorId > b.executorId } else { a.host > b.host } + } + + 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 83ddb12e3ae9..858ead777f52 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 @@ -28,7 +28,6 @@ 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.KafkaOffsetReader.{AssignStrategy, SubscribePatternStrategy, SubscribeStrategy} import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.StructType @@ -52,9 +51,9 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister with Stre schema: Option[StructType], providerName: String, parameters: Map[String, String]): (String, StructType) = { - validateOptions(parameters, Stream) + validateStreamOptions(parameters) require(schema.isEmpty, "Kafka source has a fixed schema and cannot be set with a custom one") - (shortName(), KafkaOffsetReader.kafkaSchema) + (shortName(), KafkaTopicPartitionOffsetReader.kafkaSchema) } override def createSource( @@ -63,7 +62,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister with Stre schema: Option[StructType], providerName: String, parameters: Map[String, String]): Source = { - validateOptions(parameters, Stream) + validateStreamOptions(parameters) // 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. @@ -85,7 +84,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister with Stre case None => LatestOffsets } - val kafkaOffsetReader = new KafkaOffsetReaderImpl( + val kafkaOffsetReader = new KafkaTopicPartitionOffsetReader( strategy(caseInsensitiveParams), kafkaParamsForDriver(specifiedKafkaParams), parameters, @@ -108,9 +107,9 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister with Stre * by the Map that is passed to the function. */ override def createRelation( - sqlContext: SQLContext, - parameters: Map[String, String]): BaseRelation = { - validateOptions(parameters, Batch) + sqlContext: SQLContext, + parameters: Map[String, String]): BaseRelation = { + validateBatchOptions(parameters) // 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. @@ -137,12 +136,11 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister with Stre case None => LatestOffsets } - val kafkaOffsetReaderImpl = new KafkaOffsetReaderImpl( + val kafkaOffsetReader = new KafkaTopicPartitionOffsetReader( strategy(caseInsensitiveParams), kafkaParamsForDriver(specifiedKafkaParams), parameters, driverGroupIdPrefix = s"$uniqueGroupId-driver") - val kafkaOffsetReader = new UninterruptibleKafkaOffsetReader(kafkaOffsetReaderImpl) new KafkaRelation( sqlContext, @@ -175,7 +173,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister with Stre .build() private def kafkaParamsForExecutors( - specifiedKafkaParams: Map[String, String], uniqueGroupId: String) = + specifiedKafkaParams: Map[String, String], uniqueGroupId: String) = ConfigUpdater("executor", specifiedKafkaParams) .set(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, deserClassName) .set(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, deserClassName) @@ -195,7 +193,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister with Stre .build() private def strategy(caseInsensitiveParams: Map[String, String]) = - caseInsensitiveParams.find(x => STRATEGY_OPTION_KEYS.contains(x._1)).get match { + caseInsensitiveParams.find(x => STRATEGY_OPTION_KEYS.contains(x._1)).get match { case ("assign", value) => AssignStrategy(JsonUtils.partitions(value)) case ("subscribe", value) => @@ -211,18 +209,12 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister with Stre private def failOnDataLoss(caseInsensitiveParams: Map[String, String]) = caseInsensitiveParams.getOrElse(FAIL_ON_DATA_LOSS_OPTION_KEY, "true").toBoolean - - private def validateOptions(parameters: Map[String, String], mode: Mode): Unit = { + private def validateGeneralOptions(parameters: Map[String, String]): Unit = { // Validate source options val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase, v) } val specifiedStrategies = caseInsensitiveParams.filter { case (k, _) => STRATEGY_OPTION_KEYS.contains(k) }.toSeq - mode match { - case Stream => validateStream(caseInsensitiveParams) - case Batch => validateBatch(caseInsensitiveParams) - } - if (specifiedStrategies.isEmpty) { throw new IllegalArgumentException( "One of the following options must be specified for Kafka source: " @@ -315,20 +307,25 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister with Stre } } - private def validateStream(caseInsensitiveParams: Map[String, String]) = { + private def validateStreamOptions(caseInsensitiveParams: Map[String, String]) = { + // Stream specific options caseInsensitiveParams.get(ENDING_OFFSETS_OPTION_KEY).map(_ => - throw new IllegalArgumentException("ending offset not valid in stream mode")) + throw new IllegalArgumentException("ending offset not valid in streaming queries")) + validateGeneralOptions(caseInsensitiveParams) } - private def validateBatch(caseInsensitiveParams: Map[String, String]) = { + private def validateBatchOptions(caseInsensitiveParams: Map[String, String]) = { + // Batch specific options caseInsensitiveParams.get(STARTING_OFFSETS_OPTION_KEY).map(_.trim.toLowerCase) match { case Some("earliest") => // good to go case Some("latest") => - throw new IllegalArgumentException("starting relation offset can't be latest") + throw new IllegalArgumentException("starting offset can't be latest " + + "for batch queries on Kafka") case Some(json) => (SpecificOffsets(JsonUtils.partitionOffsets(json))) .partitionOffsets.foreach { - case (tp, off) if off == KafkaUtils.LATEST => - throw new IllegalArgumentException(s"startingOffsets for $tp can't be latest") + case (tp, off) if off == KafkaOffsets.LATEST => + throw new IllegalArgumentException(s"startingOffsets for $tp can't " + + "be latest for batch queries on Kafka") case _ => // ignore } case _ => // default to earliest @@ -336,20 +333,24 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister with Stre caseInsensitiveParams.get(ENDING_OFFSETS_OPTION_KEY).map(_.trim.toLowerCase) match { case Some("earliest") => - throw new IllegalArgumentException("ending relation offset can't be earliest") + throw new IllegalArgumentException("ending offset can't be earliest " + + "for batch queries on Kafka") case Some("latest") => // good to go case Some(json) => (SpecificOffsets(JsonUtils.partitionOffsets(json))) .partitionOffsets.foreach { - case (tp, off) if off == KafkaUtils.EARLIEST => - throw new IllegalArgumentException(s"ending offset for $tp can't be earliest") + case (tp, off) if off == KafkaOffsets.EARLIEST => + throw new IllegalArgumentException(s"ending offset for $tp can't be " + + "earliest for batch queries on Kafka") case _ => // ignore } case _ => // default to latest } + validateGeneralOptions(caseInsensitiveParams) + // Don't want to throw an error, but at least log a warning. if (caseInsensitiveParams.get("maxoffsetspertrigger").isDefined) { - logWarning("maxOffsetsPerTrigger option ignored in batch mode") + logWarning("maxOffsetsPerTrigger option ignored in batch queries") } } @@ -381,10 +382,5 @@ private[kafka010] object KafkaSourceProvider { private val ENDING_OFFSETS_OPTION_KEY = "endingoffsets" private val FAIL_ON_DATA_LOSS_OPTION_KEY = "failondataloss" - // Used to check parameters for different source modes - private sealed trait Mode - private case object Batch extends Mode - private case object Stream extends Mode - private val deserClassName = classOf[ByteArrayDeserializer].getName } 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 bad66c4aab59..7c789b691b99 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 @@ -123,7 +123,12 @@ private[kafka010] class KafkaSourceRDD( override def compute( thePart: Partition, context: TaskContext): Iterator[ConsumerRecord[Array[Byte], Array[Byte]]] = { - var range = thePart.asInstanceOf[KafkaSourceRDDPartition].offsetRange + val sourcePartition = thePart.asInstanceOf[KafkaSourceRDDPartition] + val topic = sourcePartition.offsetRange.topic + val partition = sourcePartition.offsetRange.partition + val consumer = CachedKafkaConsumer.getOrCreate( + topic, partition, executorKafkaParams, reuseKafkaConsumer) + val range = resolveRange(consumer, sourcePartition.offsetRange) assert( range.fromOffset <= range.untilOffset, s"Beginning offset ${range.fromOffset} is after the ending offset ${range.untilOffset} " + @@ -135,29 +140,6 @@ private[kafka010] class KafkaSourceRDD( Iterator.empty } else { new NextIterator[ConsumerRecord[Array[Byte], Array[Byte]]]() { - val consumer = CachedKafkaConsumer.getOrCreate( - range.topic, range.partition, executorKafkaParams, reuseKafkaConsumer) - if (range.fromOffset < 0 || range.untilOffset < 0) { - // Late bind the offset range - val fromOffset = if (range.fromOffset < 0) { - assert(range.fromOffset == KafkaUtils.EARLIEST, - s"earliest offset ${range.fromOffset} does not equal ${KafkaUtils.EARLIEST}") - consumer.rawConsumer.seekToBeginning(ju.Arrays.asList(range.topicPartition)) - consumer.rawConsumer.position(range.topicPartition) - } else { - range.fromOffset - } - val untilOffset = if (range.untilOffset < 0) { - assert(range.untilOffset == KafkaUtils.LATEST, - s"latest offset ${range.untilOffset} does not equal ${KafkaUtils.LATEST}") - consumer.rawConsumer.seekToEnd(ju.Arrays.asList(range.topicPartition)) - consumer.rawConsumer.position(range.topicPartition) - } else { - range.untilOffset - } - range = KafkaSourceRDDOffsetRange(range.topicPartition, - fromOffset, untilOffset, range.preferredLoc) - } var requestOffset = range.fromOffset override def getNext(): ConsumerRecord[Array[Byte], Array[Byte]] = { @@ -182,4 +164,29 @@ private[kafka010] class KafkaSourceRDD( } } } + + private def resolveRange(consumer: CachedKafkaConsumer, range: KafkaSourceRDDOffsetRange) = { + if (range.fromOffset < 0 || range.untilOffset < 0) { + // Late bind the offset range + val availableOffsetRange = consumer.getAvailableOffsetRange() + val fromOffset = if (range.fromOffset < 0) { + assert(range.fromOffset == KafkaOffsets.EARLIEST, + s"earliest offset ${range.fromOffset} does not equal ${KafkaOffsets.EARLIEST}") + availableOffsetRange.earliest + } else { + range.fromOffset + } + val untilOffset = if (range.untilOffset < 0) { + assert(range.untilOffset == KafkaOffsets.LATEST, + s"latest offset ${range.untilOffset} does not equal ${KafkaOffsets.LATEST}") + availableOffsetRange.latest + } else { + range.untilOffset + } + KafkaSourceRDDOffsetRange(range.topicPartition, + fromOffset, untilOffset, range.preferredLoc) + } else { + range + } + } } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaTopicPartitionOffsetReader.scala similarity index 50% rename from external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala rename to external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaTopicPartitionOffsetReader.scala index 632cef08355d..ef64f9ca77a3 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaTopicPartitionOffsetReader.scala @@ -26,66 +26,43 @@ import scala.concurrent.duration.Duration import scala.util.control.NonFatal 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.spark.internal.Logging -import org.apache.spark.sql.kafka010.KafkaOffsetReader.ConsumerStrategy import org.apache.spark.sql.types._ import org.apache.spark.util.{ThreadUtils, UninterruptibleThread} - -private[kafka010] trait KafkaOffsetReader { - - /** - * Closes the connection to Kafka, and cleans up state. - */ - def close() - - /** - * @return The Set of TopicPartitions for a given topic - */ - def fetchTopicPartitions(): Set[TopicPartition] - - /** - * Set consumer position to specified offsets, making sure all assignments are set. - */ - def fetchSpecificStartingOffsets( - partitionOffsets: Map[TopicPartition, Long]): Map[TopicPartition, Long] - - /** - * Fetch the earliest offsets of partitions. - */ - def fetchEarliestOffsets(): Map[TopicPartition, Long] - - /** - * Fetch the latest offsets of partitions. - */ - def fetchLatestOffsets(): Map[TopicPartition, Long] - - /** - * Fetch the earliest offsets for newly discovered partitions. The return result may not contain - * some partitions if they are deleted. - */ - def fetchNewPartitionEarliestOffsets( - newPartitions: Seq[TopicPartition]): Map[TopicPartition, Long] -} - /** * This class uses Kafka's own [[KafkaConsumer]] API to read data offsets from Kafka. + * 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.ConsumerStrategy]] + * for more details. * - * - 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.KafkaOffsetReader.ConsumerStrategy]] for more details. + * Note: This class is not ThreadSafe */ -private[kafka010] class KafkaOffsetReaderImpl( +private[kafka010] class KafkaTopicPartitionOffsetReader( consumerStrategy: ConsumerStrategy, driverKafkaParams: ju.Map[String, Object], readerOptions: Map[String, String], - driverGroupIdPrefix: String) - extends KafkaOffsetReader with Logging { + driverGroupIdPrefix: String) extends Logging { + /** + * Used to ensure execute fetch operations execute in an UninterruptibleThread + */ + val kafkaReaderThread = Executors.newSingleThreadExecutor(new ThreadFactory { + override def newThread(r: Runnable): Thread = { + val t = new UninterruptibleThread("Kafka Offset Reader") { + override def run(): Unit = { + r.run() + } + } + t.setDaemon(true) + t + } + }) + val execContext = ExecutionContext.fromExecutorService(kafkaReaderThread) /** * A KafkaConsumer used in the driver to query the latest Kafka offsets. This only queries the @@ -111,9 +88,18 @@ private[kafka010] class KafkaOffsetReaderImpl( override def toString(): String = consumerStrategy.toString - def close(): Unit = consumer.close() + /** + * Closes the connection to Kafka, and cleans up state. + */ + def close(): Unit = { + consumer.close() + kafkaReaderThread.shutdownNow() + } - override def fetchTopicPartitions(): Set[TopicPartition] = { + /** + * @return The Set of TopicPartitions for a given topic + */ + def fetchTopicPartitions(): Set[TopicPartition] = runUninterruptibly { assert(Thread.currentThread().isInstanceOf[UninterruptibleThread]) // Poll to get the latest assigned partitions consumer.poll(0) @@ -122,80 +108,112 @@ private[kafka010] class KafkaOffsetReaderImpl( partitions.asScala.toSet } - override def fetchSpecificStartingOffsets( + /** + * Resolves the specific offsets based on Kafka seek positions. + * This method resolves offset value -1 to the latest and -2 to the + * earliest Kafka seek position. + */ + def fetchSpecificOffsets( partitionOffsets: Map[TopicPartition, Long]): Map[TopicPartition, Long] = + runUninterruptibly { + withRetriesWithoutInterrupt { + // Poll to get the latest assigned partitions + consumer.poll(0) + val partitions = consumer.assignment() + consumer.pause(partitions) + assert(partitions.asScala == partitionOffsets.keySet, + "If startingOffsets contains specific offsets, you must specify all TopicPartitions.\n" + + "Use -1 for latest, -2 for earliest, if you don't care.\n" + + s"Specified: ${partitionOffsets.keySet} Assigned: ${partitions.asScala}") + logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $partitionOffsets") + + partitionOffsets.foreach { + case (tp, KafkaOffsets.LATEST) => consumer.seekToEnd(ju.Arrays.asList(tp)) + case (tp, KafkaOffsets.EARLIEST) => consumer.seekToBeginning(ju.Arrays.asList(tp)) + case (tp, off) => consumer.seek(tp, off) + } + partitionOffsets.map { + case (tp, _) => tp -> consumer.position(tp) + } + } + } + + /** + * Fetch the earliest offsets of partitions. + */ + def fetchEarliestOffsets(): Map[TopicPartition, Long] = runUninterruptibly { withRetriesWithoutInterrupt { // Poll to get the latest assigned partitions consumer.poll(0) val partitions = consumer.assignment() consumer.pause(partitions) - assert(partitions.asScala == partitionOffsets.keySet, - "If startingOffsets contains specific offsets, you must specify all TopicPartitions.\n" + - "Use -1 for latest, -2 for earliest, if you don't care.\n" + - s"Specified: ${partitionOffsets.keySet} Assigned: ${partitions.asScala}") - logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $partitionOffsets") + logDebug(s"Partitions assigned to consumer: $partitions. Seeking to the beginning") - partitionOffsets.foreach { - case (tp, -1) => consumer.seekToEnd(ju.Arrays.asList(tp)) - case (tp, -2) => consumer.seekToBeginning(ju.Arrays.asList(tp)) - case (tp, off) => consumer.seek(tp, off) - } - partitionOffsets.map { - case (tp, _) => tp -> consumer.position(tp) - } + consumer.seekToBeginning(partitions) + val partitionOffsets = partitions.asScala.map(p => p -> consumer.position(p)).toMap + logDebug(s"Got earliest offsets for partition : $partitionOffsets") + partitionOffsets } - - override def fetchEarliestOffsets(): Map[TopicPartition, Long] = withRetriesWithoutInterrupt { - // Poll to get the latest assigned partitions - consumer.poll(0) - val partitions = consumer.assignment() - consumer.pause(partitions) - logDebug(s"Partitions assigned to consumer: $partitions. Seeking to the beginning") - - consumer.seekToBeginning(partitions) - val partitionOffsets = partitions.asScala.map(p => p -> consumer.position(p)).toMap - logDebug(s"Got earliest offsets for partition : $partitionOffsets") - partitionOffsets } - override def fetchLatestOffsets(): Map[TopicPartition, Long] = withRetriesWithoutInterrupt { - // Poll to get the latest assigned partitions - consumer.poll(0) - val partitions = consumer.assignment() - consumer.pause(partitions) - logDebug(s"Partitions assigned to consumer: $partitions. Seeking to the end.") + /** + * Fetch the latest offsets of partitions. + */ + def fetchLatestOffsets(): Map[TopicPartition, Long] = runUninterruptibly { + withRetriesWithoutInterrupt { + // Poll to get the latest assigned partitions + consumer.poll(0) + val partitions = consumer.assignment() + consumer.pause(partitions) + logDebug(s"Partitions assigned to consumer: $partitions. Seeking to the end.") - consumer.seekToEnd(partitions) - val partitionOffsets = partitions.asScala.map(p => p -> consumer.position(p)).toMap - logDebug(s"Got latest offsets for partition : $partitionOffsets") - partitionOffsets + consumer.seekToEnd(partitions) + val partitionOffsets = partitions.asScala.map(p => p -> consumer.position(p)).toMap + logDebug(s"Got latest offsets for partition : $partitionOffsets") + partitionOffsets + } } - override def fetchNewPartitionEarliestOffsets( + /** + * Fetch the earliest offsets for specific partitions. + * The return result may not contain some partitions if they are deleted. + */ + def fetchEarliestOffsets( newPartitions: Seq[TopicPartition]): Map[TopicPartition, Long] = { if (newPartitions.isEmpty) { Map.empty[TopicPartition, Long] } else { - withRetriesWithoutInterrupt { - // Poll to get the latest assigned partitions - consumer.poll(0) - val partitions = consumer.assignment() - consumer.pause(partitions) - logDebug(s"\tPartitions assigned to consumer: $partitions") - - // Get the earliest offset of each partition - consumer.seekToBeginning(partitions) - val partitionOffsets = 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 earliest offsets for new partitions: $partitionOffsets") - partitionOffsets + runUninterruptibly { + withRetriesWithoutInterrupt { + // Poll to get the latest assigned partitions + consumer.poll(0) + val partitions = consumer.assignment() + consumer.pause(partitions) + logDebug(s"\tPartitions assigned to consumer: $partitions") + + // Get the earliest offset of each partition + consumer.seekToBeginning(partitions) + val partitionOffsets = 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 earliest offsets for new partitions: $partitionOffsets") + partitionOffsets + } } } } + private def runUninterruptibly[T](body: => T): T = { + if (!Thread.currentThread.isInstanceOf[UninterruptibleThread]) { + val future = Future { body }(execContext) + ThreadUtils.awaitResult(future, Duration.Inf) + } else { + body + } + } + /** * 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 @@ -268,74 +286,7 @@ private[kafka010] class KafkaOffsetReaderImpl( } } -/** - * The Kafka Consumer must be called in an UninterruptibleThread. This naturally occurs - * in Structured Streaming, but not in Spark SQL, which will use this call to communicate - * with Kafak for obtaining offsets. - * - * @param kafkaOffsetReader Basically in instance of [[KafkaOffsetReaderImpl]] that - * this class wraps and executes in an [[UninterruptibleThread]] - */ -private[kafka010] class UninterruptibleKafkaOffsetReader(kafkaOffsetReader: KafkaOffsetReader) - extends KafkaOffsetReader with Logging { - - val kafkaReaderThread = Executors.newSingleThreadExecutor(new ThreadFactory { - override def newThread(r: Runnable): Thread = { - val t = new UninterruptibleThread("Kafka Offset Reader") { - override def run(): Unit = { - r.run() - } - } - t.setDaemon(true) - t - } - }) - val execContext = ExecutionContext.fromExecutorService(kafkaReaderThread) - - override def close(): Unit = { - kafkaOffsetReader.close() - kafkaReaderThread.shutdownNow() - } - - override def fetchTopicPartitions(): Set[TopicPartition] = { - val future = Future { - kafkaOffsetReader.fetchTopicPartitions() - }(execContext) - ThreadUtils.awaitResult(future, Duration.Inf) - } - - override def fetchSpecificStartingOffsets( - partitionOffsets: Map[TopicPartition, Long]): Map[TopicPartition, Long] = { - val future = Future { - kafkaOffsetReader.fetchSpecificStartingOffsets(partitionOffsets) - }(execContext) - ThreadUtils.awaitResult(future, Duration.Inf) - } - - override def fetchEarliestOffsets(): Map[TopicPartition, Long] = { - val future = Future { - kafkaOffsetReader.fetchEarliestOffsets() - }(execContext) - ThreadUtils.awaitResult(future, Duration.Inf) - } - - override def fetchLatestOffsets(): Map[TopicPartition, Long] = { - val future = Future { - kafkaOffsetReader.fetchLatestOffsets() - }(execContext) - ThreadUtils.awaitResult(future, Duration.Inf) - } - - override def fetchNewPartitionEarliestOffsets( - newPartitions: Seq[TopicPartition]): Map[TopicPartition, Long] = { - val future = Future { - kafkaOffsetReader.fetchNewPartitionEarliestOffsets(newPartitions) - }(execContext) - ThreadUtils.awaitResult(future, Duration.Inf) - } -} - -private[kafka010] object KafkaOffsetReader { +private[kafka010] object KafkaTopicPartitionOffsetReader { def kafkaSchema: StructType = StructType(Seq( StructField("key", BinaryType), @@ -346,44 +297,4 @@ private[kafka010] object KafkaOffsetReader { StructField("timestamp", TimestampType), StructField("timestampType", IntegerType) )) - - sealed trait ConsumerStrategy { - def createConsumer(kafkaParams: ju.Map[String, Object]): Consumer[Array[Byte], Array[Byte]] - } - - case class AssignStrategy(partitions: Array[TopicPartition]) extends ConsumerStrategy { - override def createConsumer( - kafkaParams: ju.Map[String, Object]): Consumer[Array[Byte], Array[Byte]] = { - val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams) - consumer.assign(ju.Arrays.asList(partitions: _*)) - consumer - } - - override def toString: String = s"Assign[${partitions.mkString(", ")}]" - } - - case class SubscribeStrategy(topics: Seq[String]) extends ConsumerStrategy { - override def createConsumer( - kafkaParams: ju.Map[String, Object]): Consumer[Array[Byte], Array[Byte]] = { - val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams) - consumer.subscribe(topics.asJava) - consumer - } - - override def toString: String = s"Subscribe[${topics.mkString(", ")}]" - } - - case class SubscribePatternStrategy(topicPattern: String) - extends ConsumerStrategy { - override def createConsumer( - kafkaParams: ju.Map[String, Object]): Consumer[Array[Byte], Array[Byte]] = { - val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams) - consumer.subscribe( - ju.regex.Pattern.compile(topicPattern), - new NoOpConsumerRebalanceListener()) - consumer - } - - override def toString: String = s"SubscribePattern[$topicPattern]" - } } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaUtils.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaUtils.scala deleted file mode 100644 index a43a016aaa0c..000000000000 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaUtils.scala +++ /dev/null @@ -1,42 +0,0 @@ -/* - * 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.SparkContext -import org.apache.spark.scheduler.ExecutorCacheTaskLocation - -private[kafka010] object KafkaUtils { - - // Used to denote unbounded offset positions - val LATEST = -1L - val EARLIEST = -2L - - def getSortedExecutorList(sc: SparkContext): Array[String] = { - val bm = sc.env.blockManager - bm.master.getPeers(bm.blockManagerId).toArray - .map(x => ExecutorCacheTaskLocation(x.host, x.executorId)) - .sortWith(compare) - .map(_.toString) - } - - private def compare(a: ExecutorCacheTaskLocation, b: ExecutorCacheTaskLocation): Boolean = { - if (a.host == b.host) { a.executorId > b.executorId } else { a.host > b.host } - } - - def floorMod(a: Long, b: Int): Int = ((a % b).toInt + b) % b -} diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala index a58c235af22f..da1e8914b386 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala @@ -53,6 +53,20 @@ class KafkaRelationSuite extends QueryTest with BeforeAndAfter with SharedSQLCon } } + private def createDF(topic: String, + withOptions: Map[String, String] = Map.empty[String, String]) = { + val df = spark + .read + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("subscribe", topic) + withOptions.foreach { + case (key, value) => df.option(key, value) + } + df.load().selectExpr("CAST(value AS STRING)") + } + + test("explicit earliest to latest offsets") { val topic = newTopic() testUtils.createTopic(topic, partitions = 3) @@ -61,15 +75,8 @@ class KafkaRelationSuite extends QueryTest with BeforeAndAfter with SharedSQLCon testUtils.sendMessages(topic, Array("20"), Some(2)) // Specify explicit earliest and latest offset values - val df = spark - .read - .format("kafka") - .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("subscribe", topic) - .option("startingOffsets", "earliest") - .option("endingOffsets", "latest") - .load() - .selectExpr("CAST(value AS STRING)") + val df = createDF(topic, + withOptions = Map("startingOffsets" -> "earliest", "endingOffsets" -> "latest")) checkAnswer(df, (0 to 20).map(_.toString).toDF) // "latest" should late bind to the current (latest) offset in the df @@ -85,13 +92,7 @@ class KafkaRelationSuite extends QueryTest with BeforeAndAfter with SharedSQLCon testUtils.sendMessages(topic, Array("20"), Some(2)) // Implicit offset values, should default to earliest and latest - val df = spark - .read - .format("kafka") - .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("subscribe", topic) - .load() - .selectExpr("CAST(value AS STRING)") + val df = createDF(topic) // Test that we default to "earliest" and "latest" checkAnswer(df, (0 to 20).map(_.toString).toDF) } @@ -117,15 +118,8 @@ class KafkaRelationSuite extends QueryTest with BeforeAndAfter with SharedSQLCon new TopicPartition(topic, 2) -> 1L // explicit offset happens to = the latest ) val endingOffsets = JsonUtils.partitionOffsets(endPartitionOffsets) - val df = spark - .read - .format("kafka") - .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("subscribe", topic) - .option("startingOffsets", startingOffsets) - .option("endingOffsets", endingOffsets) - .load() - .selectExpr("CAST(value as STRING)") + val df = createDF(topic, + withOptions = Map("startingOffsets" -> startingOffsets, "endingOffsets" -> endingOffsets)) checkAnswer(df, (0 to 20).map(_.toString).toDF) // static offset partition 2, nothing should change @@ -144,15 +138,8 @@ class KafkaRelationSuite extends QueryTest with BeforeAndAfter with SharedSQLCon testUtils.sendMessages(topic, (0 to 10).map(_.toString).toArray, Some(0)) // Specify explicit earliest and latest offset values - val df = spark - .read - .format("kafka") - .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("subscribe", topic) - .option("startingOffsets", "earliest") - .option("endingOffsets", "latest") - .load() - .selectExpr("CAST(value AS STRING)") + val df = createDF(topic, + withOptions = Map("startingOffsets" -> "earliest", "endingOffsets" -> "latest")) checkAnswer(df.union(df), ((0 to 10) ++ (0 to 10)).map(_.toString).toDF) } @@ -177,15 +164,8 @@ class KafkaRelationSuite extends QueryTest with BeforeAndAfter with SharedSQLCon testUtils.createTopic(topic, partitions = 1) testUtils.sendMessages(topic, (0 to 9).map(_.toString).toArray, Some(0)) // Specify explicit earliest and latest offset values - val df = spark - .read - .format("kafka") - .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("subscribe", topic) - .option("startingOffsets", "earliest") - .option("endingOffsets", "latest") - .load() - .selectExpr("CAST(value AS STRING)") + val df = createDF(topic, + withOptions = Map("startingOffsets" -> "earliest", "endingOffsets" -> "latest")) checkAnswer(df, (0 to 9).map(_.toString).toDF) // Blow away current set of messages. testUtils.cleanupLogs() @@ -217,26 +197,25 @@ class KafkaRelationSuite extends QueryTest with BeforeAndAfter with SharedSQLCon } // Specifying an ending offset as the starting point - testBadOptions("startingOffsets" -> "latest")("starting relation offset can't be latest") + testBadOptions("startingOffsets" -> "latest")("starting offset can't be latest " + + "for batch queries on Kafka") - // Now do it with an explicit json offset - val startPartitionOffsets = Map( - new TopicPartition("t", 0) -> -1L // specify latest - ) + // Now do it with an explicit json start offset indicating latest + val startPartitionOffsets = Map( new TopicPartition("t", 0) -> -1L) val startingOffsets = JsonUtils.partitionOffsets(startPartitionOffsets) testBadOptions("subscribe" -> "t", "startingOffsets" -> startingOffsets)( - "startingoffsets for t-0 can't be latest") + "startingOffsets for t-0 can't be latest for batch queries on Kafka") // Make sure we catch ending offsets that indicate earliest - testBadOptions("endingOffsets" -> "earliest")("ending relation offset can't be earliest") + testBadOptions("endingOffsets" -> "earliest")("ending offset can't be earliest " + + "for batch queries on Kafka") - val endPartitionOffsets = Map( - new TopicPartition("t", 0) -> -2L // specify earliest - ) + // Make sure we catch ending offsets that indicating earliest + val endPartitionOffsets = Map(new TopicPartition("t", 0) -> -2L) val endingOffsets = JsonUtils.partitionOffsets(endPartitionOffsets) testBadOptions("subscribe" -> "t", "endingOffsets" -> endingOffsets)( - "ending offset for t-0 can't be earliest") + "ending offset for t-0 can't be earliest for batch queries on Kafka") // No strategy specified testBadOptions()("options must be specified", "subscribe", "subscribePattern") 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 e20aa8f671c5..211c8a5e73e4 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 @@ -385,7 +385,7 @@ class KafkaSourceSuite extends KafkaSourceTest { } // Specifying an ending offset - testBadOptions("endingOffsets" -> "latest")("Ending offset not valid in stream mode") + testBadOptions("endingOffsets" -> "latest")("Ending offset not valid in streaming queries") // No strategy specified testBadOptions()("options must be specified", "subscribe", "subscribePattern") From 57760094591439525d3a53d0c5845a0da7e9b8eb Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Fri, 3 Feb 2017 11:46:00 -0800 Subject: [PATCH 22/40] address feedback from @tdas and @sxwing --- .../sql/kafka010/CachedKafkaConsumer.scala | 61 +++++++++++++++---- .../spark/sql/kafka010/ConsumerStrategy.scala | 3 +- ...sets.scala => KafkaOffsetRangeLimit.scala} | 30 ++++----- ...etReader.scala => KafkaOffsetReader.scala} | 30 ++++++--- .../spark/sql/kafka010/KafkaRelation.scala | 27 ++++---- .../spark/sql/kafka010/KafkaSource.scala | 29 ++++----- .../sql/kafka010/KafkaSourceProvider.scala | 36 +++++------ .../spark/sql/kafka010/KafkaSourceRDD.scala | 40 +++++++++--- .../sql/kafka010/KafkaRelationSuite.scala | 45 +++++++------- 9 files changed, 187 insertions(+), 114 deletions(-) rename external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/{KafkaOffsets.scala => KafkaOffsetRangeLimit.scala} (53%) rename external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/{KafkaTopicPartitionOffsetReader.scala => KafkaOffsetReader.scala} (90%) 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 663b5358b1ce..735792236520 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 @@ -44,6 +44,9 @@ private[kafka010] case class CachedKafkaConsumer private( private var consumer = createConsumer + /** indicates whether this consumer is in use or not */ + private var inuse = true + /** Iterator to the already fetch data */ private var fetchedData = ju.Collections.emptyIterator[ConsumerRecord[Array[Byte], Array[Byte]]] private var nextOffsetInFetchedData = UNKNOWN_OFFSET @@ -138,7 +141,7 @@ private[kafka010] case class CachedKafkaConsumer private( // offset untilOffset earliestOffset latestOffset val warningMessage = s""" - |The current available offset range is [${range.earliest}, ${range.latest}). + |The current available offset range is $range. | Offset ${offset} is out of range, and records in [$offset, $untilOffset) will be | skipped ${additionalMessage(failOnDataLoss = false)} """.stripMargin @@ -163,7 +166,7 @@ private[kafka010] case class CachedKafkaConsumer private( // offset earliestOffset min(untilOffset,latestOffset) max(untilOffset, latestOffset) val warningMessage = s""" - |The current available offset range is [${range.earliest}, ${range.latest}). + |The current available offset range is $range. | Offset ${offset} is out of range, and records in [$offset, ${range.earliest}) will be | skipped ${additionalMessage(failOnDataLoss = false)} """.stripMargin @@ -312,7 +315,7 @@ private[kafka010] object CachedKafkaConsumer extends Logging { new ju.LinkedHashMap[CacheKey, CachedKafkaConsumer](capacity, 0.75f, true) { override def removeEldestEntry( entry: ju.Map.Entry[CacheKey, CachedKafkaConsumer]): Boolean = { - if (this.size > capacity) { + if (entry.getValue.inuse == false && this.size > capacity) { logWarning(s"KafkaConsumer cache hitting max capacity of $capacity, " + s"removing consumer for ${entry.getKey}") try { @@ -329,6 +332,39 @@ private[kafka010] object CachedKafkaConsumer extends Logging { } } + def releaseKafkaConsumer( + topic: String, + partition: Int, + kafkaParams: ju.Map[String, Object]): Unit = { + val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG).asInstanceOf[String] + val topicPartition = new TopicPartition(topic, partition) + val key = CacheKey(groupId, topicPartition) + + val consumer = cache.get(key) + if (consumer != null) { + consumer.inuse = false + } else { + logWarning(s"Attempting to release consumer that does not exist") + } + } + + /** + * Removes (and closes) the Kafka Consumer for the given topic, partition and group id. + */ + def removeKafkaConsumer( + topic: String, + partition: Int, + kafkaParams: ju.Map[String, Object]): Unit = { + val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG).asInstanceOf[String] + val topicPartition = new TopicPartition(topic, partition) + val key = CacheKey(groupId, topicPartition) + + val removedConsumer = cache.remove(key) + if (removedConsumer != null) { + removedConsumer.close() + } + } + /** * Get a cached consumer for groupId, assigned to topic and partition. * If matching consumer doesn't already exist, will be created using kafkaParams. @@ -336,25 +372,26 @@ private[kafka010] object CachedKafkaConsumer extends Logging { def getOrCreate( topic: String, partition: Int, - kafkaParams: ju.Map[String, Object], - reuseExistingIfPresent: Boolean): CachedKafkaConsumer = synchronized { + 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 (!reuseExistingIfPresent || TaskContext.get != null && TaskContext.get.attemptNumber > 1) { - val removedConsumer = cache.remove(key) - if (removedConsumer != null) { - removedConsumer.close() - } - new CachedKafkaConsumer(topicPartition, kafkaParams) + if (TaskContext.get != null && TaskContext.get.attemptNumber > 1) { + removeKafkaConsumer(topic, partition, kafkaParams) + val consumer = new CachedKafkaConsumer(topicPartition, kafkaParams) + consumer.inuse = true + cache.put(key, consumer) + consumer } else { if (!cache.containsKey(key)) { cache.put(key, new CachedKafkaConsumer(topicPartition, kafkaParams)) } - cache.get(key) + val consumer = cache.get(key) + consumer.inuse = true + consumer } } } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/ConsumerStrategy.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/ConsumerStrategy.scala index 947202308c92..66511b306541 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/ConsumerStrategy.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/ConsumerStrategy.scala @@ -28,13 +28,14 @@ import org.apache.kafka.common.TopicPartition /** * Subscribe allows you to subscribe to a fixed collection of topics. * SubscribePattern allows you to use a regex to specify topics of interest. - * Note that unlike the 0.8 integration, * using Subscribe or SubscribePattern + * Note that unlike the 0.8 integration, using Subscribe or SubscribePattern * should respond to adding partitions during a running stream. * Finally, Assign allows you to specify a fixed collection of partitions. * All three strategies have overloaded constructors that allow you to specify * the starting offset for a particular partition. */ sealed trait ConsumerStrategy { + /** Create a [[KafkaConsumer]] and subscribe to topics according to a desired strategy */ def createConsumer(kafkaParams: ju.Map[String, Object]): Consumer[Array[Byte], Array[Byte]] } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsets.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeLimit.scala similarity index 53% rename from external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsets.scala rename to external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeLimit.scala index 6c5676453b87..80a026f4f5d7 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsets.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeLimit.scala @@ -20,30 +20,32 @@ package org.apache.spark.sql.kafka010 import org.apache.kafka.common.TopicPartition /** - * Values that can be specified to configure starting, + * Objects that represent desired offset range limits for starting, * ending, and specific offsets. */ -private[kafka010] sealed trait KafkaOffsets +private[kafka010] sealed trait KafkaOffsetRangeLimit /** - * Bind to the earliest offsets in Kafka + * Represents the desire to bind to the earliest offsets in Kafka */ -private[kafka010] case object EarliestOffsets extends KafkaOffsets +private[kafka010] case object EarliestOffsetRangeLimit extends KafkaOffsetRangeLimit /** - * Bind to the latest offsets in Kafka + * Represents the desire to bind to the latest offsets in Kafka */ -private[kafka010] case object LatestOffsets extends KafkaOffsets +private[kafka010] case object LatestOffsetRangeLimit extends KafkaOffsetRangeLimit /** - * Bind to the specific offsets. A offset == -1 binds to the latest - * offset, and offset == -2 binds to the earliest offset. + * Represents the desire to bind to specific offsets. A offset == -1 binds to the + * latest offset, and offset == -2 binds to the earliest offset. */ -private[kafka010] case class SpecificOffsets( - partitionOffsets: Map[TopicPartition, Long]) extends KafkaOffsets +private[kafka010] case class SpecificOffsetRangeLimit( + partitionOffsets: Map[TopicPartition, Long]) extends KafkaOffsetRangeLimit -private[kafka010] object KafkaOffsets { - // Used to denote unbounded offset positions - val LATEST = -1L - val EARLIEST = -2L +private[kafka010] object KafkaOffsetRangeLimit { + /** + * Used to denote offset range limits that are resolved via Kafka + */ + val LATEST = -1L // indicates resolution to the latest offset + val EARLIEST = -2L // indicates resolution to the earliest offset } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaTopicPartitionOffsetReader.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala similarity index 90% rename from external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaTopicPartitionOffsetReader.scala rename to external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala index ef64f9ca77a3..6b2fb3c11255 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaTopicPartitionOffsetReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala @@ -43,7 +43,7 @@ import org.apache.spark.util.{ThreadUtils, UninterruptibleThread} * * Note: This class is not ThreadSafe */ -private[kafka010] class KafkaTopicPartitionOffsetReader( +private[kafka010] class KafkaOffsetReader( consumerStrategy: ConsumerStrategy, driverKafkaParams: ju.Map[String, Object], readerOptions: Map[String, String], @@ -128,8 +128,10 @@ private[kafka010] class KafkaTopicPartitionOffsetReader( logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $partitionOffsets") partitionOffsets.foreach { - case (tp, KafkaOffsets.LATEST) => consumer.seekToEnd(ju.Arrays.asList(tp)) - case (tp, KafkaOffsets.EARLIEST) => consumer.seekToBeginning(ju.Arrays.asList(tp)) + case (tp, KafkaOffsetRangeLimit.LATEST) => + consumer.seekToEnd(ju.Arrays.asList(tp)) + case (tp, KafkaOffsetRangeLimit.EARLIEST) => + consumer.seekToBeginning(ju.Arrays.asList(tp)) case (tp, off) => consumer.seek(tp, off) } partitionOffsets.map { @@ -139,7 +141,8 @@ private[kafka010] class KafkaTopicPartitionOffsetReader( } /** - * Fetch the earliest offsets of partitions. + * Fetch the earliest offsets for the topic partitions that are indicated + * in the [[ConsumerStrategy]]. */ def fetchEarliestOffsets(): Map[TopicPartition, Long] = runUninterruptibly { withRetriesWithoutInterrupt { @@ -157,7 +160,8 @@ private[kafka010] class KafkaTopicPartitionOffsetReader( } /** - * Fetch the latest offsets of partitions. + * Fetch the latest offsets for the topic partitions that are indicated + * in the [[ConsumerStrategy]]. */ def fetchLatestOffsets(): Map[TopicPartition, Long] = runUninterruptibly { withRetriesWithoutInterrupt { @@ -175,7 +179,7 @@ private[kafka010] class KafkaTopicPartitionOffsetReader( } /** - * Fetch the earliest offsets for specific partitions. + * Fetch the earliest offsets for specific topic partitions. * The return result may not contain some partitions if they are deleted. */ def fetchEarliestOffsets( @@ -205,9 +209,17 @@ private[kafka010] class KafkaTopicPartitionOffsetReader( } } + /** + * This method ensures that the closure is called in an [[UninterruptibleThread]]. + * This is required when communicating with the [[KafkaConsumer]]. In the case + * of streaming queries, we are already running in an [[UninterruptibleThread]], + * however for batch mode this is not the case. + */ private def runUninterruptibly[T](body: => T): T = { if (!Thread.currentThread.isInstanceOf[UninterruptibleThread]) { - val future = Future { body }(execContext) + val future = Future { + body + }(execContext) ThreadUtils.awaitResult(future, Duration.Inf) } else { body @@ -215,7 +227,7 @@ private[kafka010] class KafkaTopicPartitionOffsetReader( } /** - * Helper function that does multiple retries on the a body of code that returns offsets. + * Helper function that does multiple retries on 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. * @@ -286,7 +298,7 @@ private[kafka010] class KafkaTopicPartitionOffsetReader( } } -private[kafka010] object KafkaTopicPartitionOffsetReader { +private[kafka010] object KafkaOffsetReader { def kafkaSchema: StructType = StructType(Seq( StructField("key", BinaryType), diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala index b233ba03c20f..0e09ee191b09 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala @@ -33,16 +33,16 @@ import org.apache.spark.unsafe.types.UTF8String private[kafka010] class KafkaRelation( override val sqlContext: SQLContext, - kafkaReader: KafkaTopicPartitionOffsetReader, + kafkaReader: KafkaOffsetReader, executorKafkaParams: ju.Map[String, Object], sourceOptions: Map[String, String], failOnDataLoss: Boolean, - startingOffsets: KafkaOffsets, - endingOffsets: KafkaOffsets) - extends BaseRelation with TableScan with Logging { - assert(startingOffsets != LatestOffsets, + startingOffsets: KafkaOffsetRangeLimit, + endingOffsets: KafkaOffsetRangeLimit) + extends BaseRelation with TableScan with Logging { + assert(startingOffsets != LatestOffsetRangeLimit, "Starting offset not allowed to be set to latest offsets.") - assert(endingOffsets != EarliestOffsets, + assert(endingOffsets != EarliestOffsetRangeLimit, "Ending offset not allowed to be set to earliest offsets.") private val pollTimeoutMs = sourceOptions.getOrElse( @@ -50,7 +50,7 @@ private[kafka010] class KafkaRelation( sqlContext.sparkContext.conf.getTimeAsMs("spark.network.timeout", "120s").toString ).toLong - override def schema: StructType = KafkaTopicPartitionOffsetReader.kafkaSchema + override def schema: StructType = KafkaOffsetReader.kafkaSchema override def buildScan(): RDD[Row] = { // Leverage the KafkaReader to obtain the relevant partition offsets @@ -97,7 +97,8 @@ private[kafka010] class KafkaRelation( sqlContext.internalCreateDataFrame(rdd, schema).rdd } - private def getPartitionOffsets(kafkaOffsets: KafkaOffsets): Map[TopicPartition, Long] = { + private def getPartitionOffsets( + kafkaOffsets: KafkaOffsetRangeLimit): Map[TopicPartition, Long] = { def validateTopicPartitions(partitions: Set[TopicPartition], partitionOffsets: Map[TopicPartition, Long]): Map[TopicPartition, Long] = { assert(partitions == partitionOffsets.keySet, @@ -110,13 +111,13 @@ private[kafka010] class KafkaRelation( val partitions = kafkaReader.fetchTopicPartitions() // Obtain TopicPartition offsets with late binding support kafkaOffsets match { - case EarliestOffsets => partitions.map { - case tp => tp -> KafkaOffsets.EARLIEST + case EarliestOffsetRangeLimit => partitions.map { + case tp => tp -> KafkaOffsetRangeLimit.EARLIEST }.toMap - case LatestOffsets => partitions.map { - case tp => tp -> KafkaOffsets.LATEST + case LatestOffsetRangeLimit => partitions.map { + case tp => tp -> KafkaOffsetRangeLimit.LATEST }.toMap - case SpecificOffsets(partitionOffsets) => + case SpecificOffsetRangeLimit(partitionOffsets) => validateTopicPartitions(partitions, partitionOffsets) } } 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 b954d4a0fa55..02b23111af78 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 @@ -45,11 +45,11 @@ import org.apache.spark.unsafe.types.UTF8String * * - The [[KafkaSource]] written to do the following. * - * - As soon as the source is created, the pre-configured [[KafkaTopicPartitionOffsetReader]] + * - As soon as the source is created, the pre-configured [[KafkaOffsetReader]] * is used to query the initial offsets that this source should * start reading from. This is used to create the first batch. * - * - `getOffset()` uses the [[KafkaTopicPartitionOffsetReader]] to query the latest + * - `getOffset()` uses the [[KafkaOffsetReader]] 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 @@ -69,13 +69,13 @@ import org.apache.spark.unsafe.types.UTF8String * and not use wrong broker addresses. */ private[kafka010] class KafkaSource( - sqlContext: SQLContext, - kafkaReader: KafkaTopicPartitionOffsetReader, - executorKafkaParams: ju.Map[String, Object], - sourceOptions: Map[String, String], - metadataPath: String, - startingOffsets: KafkaOffsets, - failOnDataLoss: Boolean) + sqlContext: SQLContext, + kafkaReader: KafkaOffsetReader, + executorKafkaParams: ju.Map[String, Object], + sourceOptions: Map[String, String], + metadataPath: String, + startingOffsets: KafkaOffsetRangeLimit, + failOnDataLoss: Boolean) extends Source with Logging { private val sc = sqlContext.sparkContext @@ -112,9 +112,9 @@ private[kafka010] class KafkaSource( metadataLog.get(0).getOrElse { val offsets = startingOffsets match { - case EarliestOffsets => KafkaSourceOffset(kafkaReader.fetchEarliestOffsets()) - case LatestOffsets => KafkaSourceOffset(kafkaReader.fetchLatestOffsets()) - case SpecificOffsets(p) => fetchAndVerify(p) + case EarliestOffsetRangeLimit => KafkaSourceOffset(kafkaReader.fetchEarliestOffsets()) + case LatestOffsetRangeLimit => KafkaSourceOffset(kafkaReader.fetchLatestOffsets()) + case SpecificOffsetRangeLimit(p) => fetchAndVerify(p) } metadataLog.add(0, offsets) logInfo(s"Initial offsets: $offsets") @@ -125,7 +125,8 @@ private[kafka010] class KafkaSource( private def fetchAndVerify(specificOffsets: Map[TopicPartition, Long]) = { val result = kafkaReader.fetchSpecificOffsets(specificOffsets) specificOffsets.foreach { - case (tp, off) if off != KafkaOffsets.LATEST && off != KafkaOffsets.EARLIEST => + case (tp, off) if off != KafkaOffsetRangeLimit.LATEST && + off != KafkaOffsetRangeLimit.EARLIEST => if (result(tp) != off) { reportDataLoss( s"startingOffsets for $tp was $off but consumer reset to ${result(tp)}") @@ -138,7 +139,7 @@ private[kafka010] class KafkaSource( private var currentPartitionOffsets: Option[Map[TopicPartition, Long]] = None - override def schema: StructType = KafkaTopicPartitionOffsetReader.kafkaSchema + override def schema: StructType = KafkaOffsetReader.kafkaSchema /** Returns the maximum available offset for this source. */ override def getOffset: Option[Offset] = { 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 858ead777f52..6a7456719875 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 @@ -53,7 +53,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister with Stre parameters: Map[String, String]): (String, StructType) = { validateStreamOptions(parameters) require(schema.isEmpty, "Kafka source has a fixed schema and cannot be set with a custom one") - (shortName(), KafkaTopicPartitionOffsetReader.kafkaSchema) + (shortName(), KafkaOffsetReader.kafkaSchema) } override def createSource( @@ -78,13 +78,13 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister with Stre val startingStreamOffsets = caseInsensitiveParams.get(STARTING_OFFSETS_OPTION_KEY).map(_.trim.toLowerCase) match { - case Some("latest") => LatestOffsets - case Some("earliest") => EarliestOffsets - case Some(json) => SpecificOffsets(JsonUtils.partitionOffsets(json)) - case None => LatestOffsets + case Some("latest") => LatestOffsetRangeLimit + case Some("earliest") => EarliestOffsetRangeLimit + case Some(json) => SpecificOffsetRangeLimit(JsonUtils.partitionOffsets(json)) + case None => LatestOffsetRangeLimit } - val kafkaOffsetReader = new KafkaTopicPartitionOffsetReader( + val kafkaOffsetReader = new KafkaOffsetReader( strategy(caseInsensitiveParams), kafkaParamsForDriver(specifiedKafkaParams), parameters, @@ -109,7 +109,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister with Stre override def createRelation( sqlContext: SQLContext, parameters: Map[String, String]): BaseRelation = { - validateBatchOptions(parameters) + validateBatchOptions(parameters) // 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. @@ -124,19 +124,19 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister with Stre val startingRelationOffsets = caseInsensitiveParams.get(STARTING_OFFSETS_OPTION_KEY).map(_.trim.toLowerCase) match { - case Some("earliest") => EarliestOffsets - case Some(json) => SpecificOffsets(JsonUtils.partitionOffsets(json)) - case None => EarliestOffsets + case Some("earliest") => EarliestOffsetRangeLimit + case Some(json) => SpecificOffsetRangeLimit(JsonUtils.partitionOffsets(json)) + case None => EarliestOffsetRangeLimit } val endingRelationOffsets = caseInsensitiveParams.get(ENDING_OFFSETS_OPTION_KEY).map(_.trim.toLowerCase) match { - case Some("latest") => LatestOffsets - case Some(json) => SpecificOffsets(JsonUtils.partitionOffsets(json)) - case None => LatestOffsets + case Some("latest") => LatestOffsetRangeLimit + case Some(json) => SpecificOffsetRangeLimit(JsonUtils.partitionOffsets(json)) + case None => LatestOffsetRangeLimit } - val kafkaOffsetReader = new KafkaTopicPartitionOffsetReader( + val kafkaOffsetReader = new KafkaOffsetReader( strategy(caseInsensitiveParams), kafkaParamsForDriver(specifiedKafkaParams), parameters, @@ -321,9 +321,9 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister with Stre case Some("latest") => throw new IllegalArgumentException("starting offset can't be latest " + "for batch queries on Kafka") - case Some(json) => (SpecificOffsets(JsonUtils.partitionOffsets(json))) + case Some(json) => (SpecificOffsetRangeLimit(JsonUtils.partitionOffsets(json))) .partitionOffsets.foreach { - case (tp, off) if off == KafkaOffsets.LATEST => + case (tp, off) if off == KafkaOffsetRangeLimit.LATEST => throw new IllegalArgumentException(s"startingOffsets for $tp can't " + "be latest for batch queries on Kafka") case _ => // ignore @@ -336,9 +336,9 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister with Stre throw new IllegalArgumentException("ending offset can't be earliest " + "for batch queries on Kafka") case Some("latest") => // good to go - case Some(json) => (SpecificOffsets(JsonUtils.partitionOffsets(json))) + case Some(json) => (SpecificOffsetRangeLimit(JsonUtils.partitionOffsets(json))) .partitionOffsets.foreach { - case (tp, off) if off == KafkaOffsets.EARLIEST => + case (tp, off) if off == KafkaOffsetRangeLimit.EARLIEST => throw new IllegalArgumentException(s"ending offset for $tp can't be " + "earliest for batch queries on Kafka") case _ => // ignore 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 7c789b691b99..6fb3473eb75f 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.{ConsumerConfig, ConsumerRecord} import org.apache.kafka.common.TopicPartition import org.apache.spark.{Partition, SparkContext, TaskContext} @@ -125,9 +125,16 @@ private[kafka010] class KafkaSourceRDD( context: TaskContext): Iterator[ConsumerRecord[Array[Byte], Array[Byte]]] = { val sourcePartition = thePart.asInstanceOf[KafkaSourceRDDPartition] val topic = sourcePartition.offsetRange.topic - val partition = sourcePartition.offsetRange.partition - val consumer = CachedKafkaConsumer.getOrCreate( - topic, partition, executorKafkaParams, reuseKafkaConsumer) + if (!reuseKafkaConsumer) { + // if we can't reuse CachedKafkaConsumers, let's reset the groupId to something unique + // to each task (i.e., append the task's unique partition id), because we will have + // multiple tasks (e.g., in the case of union) reading from the same topic partitions + val old = executorKafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG).asInstanceOf[String] + val id = TaskContext.getPartitionId() + executorKafkaParams.put(ConsumerConfig.GROUP_ID_CONFIG, old + "-" + id) + } + val kafkaPartition = sourcePartition.offsetRange.partition + val consumer = CachedKafkaConsumer.getOrCreate(topic, kafkaPartition, executorKafkaParams) val range = resolveRange(consumer, sourcePartition.offsetRange) assert( range.fromOffset <= range.untilOffset, @@ -139,7 +146,7 @@ private[kafka010] class KafkaSourceRDD( s"skipping ${range.topic} ${range.partition}") Iterator.empty } else { - new NextIterator[ConsumerRecord[Array[Byte], Array[Byte]]]() { + val underlying = new NextIterator[ConsumerRecord[Array[Byte], Array[Byte]]]() { var requestOffset = range.fromOffset override def getNext(): ConsumerRecord[Array[Byte], Array[Byte]] = { @@ -160,8 +167,21 @@ private[kafka010] class KafkaSourceRDD( } } - override protected def close(): Unit = {} + override protected def close(): Unit = { + if (!reuseKafkaConsumer) { + // Don't forget to close non-reuse KafkaConsumers. You may take down your cluster! + CachedKafkaConsumer.removeKafkaConsumer(topic, kafkaPartition, executorKafkaParams) + } else { + // Indicate that we're no longer using this consumer + CachedKafkaConsumer.releaseKafkaConsumer(topic, kafkaPartition, executorKafkaParams) + } + } + } + // Release consumer, either by removing it or indicating we're no longer using it + context.addTaskCompletionListener { _ => + underlying.closeIfNeeded() } + underlying } } @@ -170,15 +190,15 @@ private[kafka010] class KafkaSourceRDD( // Late bind the offset range val availableOffsetRange = consumer.getAvailableOffsetRange() val fromOffset = if (range.fromOffset < 0) { - assert(range.fromOffset == KafkaOffsets.EARLIEST, - s"earliest offset ${range.fromOffset} does not equal ${KafkaOffsets.EARLIEST}") + assert(range.fromOffset == KafkaOffsetRangeLimit.EARLIEST, + s"earliest offset ${range.fromOffset} does not equal ${KafkaOffsetRangeLimit.EARLIEST}") availableOffsetRange.earliest } else { range.fromOffset } val untilOffset = if (range.untilOffset < 0) { - assert(range.untilOffset == KafkaOffsets.LATEST, - s"latest offset ${range.untilOffset} does not equal ${KafkaOffsets.LATEST}") + assert(range.untilOffset == KafkaOffsetRangeLimit.LATEST, + s"latest offset ${range.untilOffset} does not equal ${KafkaOffsetRangeLimit.LATEST}") availableOffsetRange.latest } else { range.untilOffset diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala index da1e8914b386..673d60ff6f87 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala @@ -53,12 +53,15 @@ class KafkaRelationSuite extends QueryTest with BeforeAndAfter with SharedSQLCon } } - private def createDF(topic: String, - withOptions: Map[String, String] = Map.empty[String, String]) = { + private def createDF( + topic: String, + withOptions: Map[String, String] = Map.empty[String, String], + brokerAddress: Option[String] = None) = { val df = spark .read .format("kafka") - .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.bootstrap.servers", + brokerAddress.getOrElse(testUtils.brokerAddress)) .option("subscribe", topic) withOptions.foreach { case (key, value) => df.option(key, value) @@ -144,45 +147,41 @@ class KafkaRelationSuite extends QueryTest with BeforeAndAfter with SharedSQLCon } test("test late binding start offsets") { + var kafkaUtils: KafkaTestUtils = null try { - // First, establish a new KafkaUtils instance that will clear - // all messages when cleanupLogs is called. - if (testUtils != null) { - testUtils.teardown() - testUtils = null - } - // The following settings will ensure that all log entries - // are removed following a call to cleanupLogs + /** + * The following settings will ensure that all log entries + * are removed following a call to cleanupLogs + */ val brokerProps = Map[String, Object]( "log.retention.bytes" -> 1.asInstanceOf[AnyRef], // retain nothing "log.retention.ms" -> 1.asInstanceOf[AnyRef] // no wait time ) - testUtils = new KafkaTestUtils(withBrokerProps = brokerProps) - testUtils.setup() + kafkaUtils = new KafkaTestUtils(withBrokerProps = brokerProps) + kafkaUtils.setup() val topic = newTopic() - testUtils.createTopic(topic, partitions = 1) - testUtils.sendMessages(topic, (0 to 9).map(_.toString).toArray, Some(0)) + kafkaUtils.createTopic(topic, partitions = 1) + kafkaUtils.sendMessages(topic, (0 to 9).map(_.toString).toArray, Some(0)) // Specify explicit earliest and latest offset values val df = createDF(topic, - withOptions = Map("startingOffsets" -> "earliest", "endingOffsets" -> "latest")) + withOptions = Map("startingOffsets" -> "earliest", "endingOffsets" -> "latest"), + Some(kafkaUtils.brokerAddress)) checkAnswer(df, (0 to 9).map(_.toString).toDF) // Blow away current set of messages. - testUtils.cleanupLogs() + kafkaUtils.cleanupLogs() // Add some more data, but do not call cleanup - testUtils.sendMessages(topic, (10 to 19).map(_.toString).toArray, Some(0)) + kafkaUtils.sendMessages(topic, (10 to 19).map(_.toString).toArray, Some(0)) // Ensure that we late bind to the new starting position checkAnswer(df, (10 to 19).map(_.toString).toDF) } finally { - if (testUtils != null) { - testUtils.teardown() + if (kafkaUtils != null) { + kafkaUtils.teardown() } - testUtils = new KafkaTestUtils - testUtils.setup() } } - test("bad source options") { + test("bad batch query options") { def testBadOptions(options: (String, String)*)(expectedMsgs: String*): Unit = { val ex = intercept[IllegalArgumentException] { val reader = spark From 3c4eecf3cd529c45c7b7a1bf053c7ea92e4caa50 Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Fri, 3 Feb 2017 15:16:03 -0800 Subject: [PATCH 23/40] update --- .../spark/sql/kafka010/KafkaRelation.scala | 16 +++--- .../spark/sql/kafka010/KafkaSource.scala | 2 +- .../sql/kafka010/KafkaSourceProvider.scala | 14 ++--- .../spark/sql/kafka010/KafkaWriter.scala | 9 ++-- .../spark/sql/kafka010/KafkaSinkSuite.scala | 53 ++++++++++++++----- 5 files changed, 64 insertions(+), 30 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala index b47373a354a1..f180bbad6e36 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala @@ -32,13 +32,13 @@ import org.apache.spark.unsafe.types.UTF8String private[kafka010] class KafkaRelation( - override val sqlContext: SQLContext, - kafkaReader: KafkaOffsetReader, - executorKafkaParams: ju.Map[String, Object], - sourceOptions: Map[String, String], - failOnDataLoss: Boolean, - startingOffsets: KafkaOffsetRangeLimit, - endingOffsets: KafkaOffsetRangeLimit) + override val sqlContext: SQLContext, + kafkaReader: KafkaOffsetReader, + executorKafkaParams: ju.Map[String, Object], + sourceOptions: Map[String, String], + failOnDataLoss: Boolean, + startingOffsets: KafkaOffsetRangeLimit, + endingOffsets: KafkaOffsetRangeLimit) extends BaseRelation with TableScan with Logging { assert(startingOffsets != LatestOffsetRangeLimit, "Starting offset not allowed to be set to latest offsets.") @@ -50,7 +50,7 @@ private[kafka010] class KafkaRelation( sqlContext.sparkContext.conf.getTimeAsMs("spark.network.timeout", "120s").toString ).toLong - override def schema: StructType = KafkaUtils.kafkaSchema + override def schema: StructType = KafkaOffsetReader.kafkaSchema override def buildScan(): RDD[Row] = { // Leverage the KafkaReader to obtain the relevant partition offsets 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 b7f97e6dd56b..02b23111af78 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 @@ -139,7 +139,7 @@ private[kafka010] class KafkaSource( private var currentPartitionOffsets: Option[Map[TopicPartition, Long]] = None - override def schema: StructType = KafkaUtils.kafkaSchema + override def schema: StructType = KafkaOffsetReader.kafkaSchema /** Returns the maximum available offset for this source. */ override def getOffset: Option[Offset] = { 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 22d57ceeed26..d0b478adc104 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 @@ -23,10 +23,10 @@ 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.kafka.common.serialization.{ByteArrayDeserializer, BytesSerializer} import org.apache.spark.internal.Logging -import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode} +import org.apache.spark.sql.{DataFrame, SaveMode, SQLContext} import org.apache.spark.sql.execution.streaming.{Sink, Source} import org.apache.spark.sql.sources._ import org.apache.spark.sql.streaming.OutputMode @@ -55,7 +55,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister parameters: Map[String, String]): (String, StructType) = { validateStreamOptions(parameters) require(schema.isEmpty, "Kafka source has a fixed schema and cannot be set with a custom one") - (shortName(), KafkaUtils.kafkaSchema) + (shortName(), KafkaOffsetReader.kafkaSchema) } override def createSource( @@ -169,7 +169,8 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister .keySet .filter(_.toLowerCase.startsWith("kafka.")) .map { k => k.drop(6).toString -> parameters(k) } - .toMap + .toMap + ("value.serializer" -> classOf[BytesSerializer].getName, + "key.serializer" -> classOf[BytesSerializer].getName) new KafkaSink(sqlContext, new ju.HashMap[String, Object](specifiedKafkaParams.asJava), defaultTopic) @@ -188,7 +189,8 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister .keySet .filter(_.toLowerCase.startsWith("kafka.")) .map { k => k.drop(6).toString -> parameters(k) } - .toMap + .toMap + ("value.serializer" -> classOf[BytesSerializer].getName, + "key.serializer" -> classOf[BytesSerializer].getName) KafkaWriter.write(outerSQLContext.sparkSession, data.queryExecution, new ju.HashMap[String, Object](specifiedKafkaParams.asJava), defaultTopic) @@ -196,7 +198,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister new BaseRelation { override def sqlContext: SQLContext = outerSQLContext - override def schema: StructType = KafkaUtils.kafkaSchema + override def schema: StructType = KafkaOffsetReader.kafkaSchema } } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala index 8bee8c9ddb3f..4c83cd6773ea 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.kafka010 import java.{util => ju} +import java.nio.ByteBuffer import scala.collection.mutable.ListBuffer import scala.concurrent.{blocking, Future} @@ -30,7 +31,8 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution.{QueryExecution, SQLExecution} -import org.apache.spark.sql.types.{NullType, StringType} +import org.apache.spark.sql.types.StringType +import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.Utils object KafkaWriter extends Logging { @@ -142,14 +144,14 @@ object KafkaWriter extends Logging { throw new IllegalStateException(s"Default topic required when no " + s"'$TOPIC_ATTRIBUTE_NAME' attribute is present") } else { - Literal(null, NullType) + Literal(null, StringType) } ).map{c => if (defaultTopic == None) { c // return null if we can't fall back on a default value } else { // fall back on a default value in case we evaluate c to null - If(IsNull(c), Literal(defaultTopic.get, StringType), c) + If(IsNull(c), Literal(UTF8String.fromString(defaultTopic.get), StringType), c) }} // Use to extract the topic from either the Row or default value val getTopic = UnsafeProjection.create(topicExpression, inputSchema) @@ -176,6 +178,7 @@ object KafkaWriter extends Logging { val topic = getTopic(currentRow).get(0, StringType).toString val key = getKey(currentRow).getBytes val value = getValue(currentRow).getBytes + println(s"topic $topic, key ${ByteBuffer.wrap(key).getInt}, value $value") val record = new ProducerRecord[Array[Byte], Array[Byte]](topic, key, value) val future = Future[RecordMetadata] { blocking { diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala index f53bc7c9c6d8..5e50dd5b0094 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala @@ -17,12 +17,13 @@ package org.apache.spark.sql.kafka010 +import java.nio.ByteBuffer import java.util.concurrent.atomic.AtomicInteger import org.scalatest.time.SpanSugar._ import org.apache.spark.sql.execution.streaming.MemoryStream -import org.apache.spark.sql.streaming.StreamTest +import org.apache.spark.sql.streaming.{OutputMode, StreamTest} import org.apache.spark.sql.test.SharedSQLContext class KafkaSinkSuite extends StreamTest with SharedSQLContext { @@ -51,16 +52,44 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext { private def newTopic(): String = s"topic-${topicId.getAndIncrement()}" test("write to stream") { - val input = MemoryStream[Int] - val df = input.toDF().selectExpr("CAST(value as BYTE) key", "CAST(value as BYTE) value") - df.printSchema() - val query = df.writeStream - .format("memory") - .outputMode("append") - .queryName("memStream") - .start() - input.addData(1, 2, 3) - query.processAllAvailable() - } + withTempDir { checkpointDir => + val input = MemoryStream[Int] + val topic = newTopic() + + spark.udf.register("intCast", (s: Array[Byte]) => ByteBuffer.wrap(s).getInt()) + spark.udf.register("byteCast", (i: Int) => ByteBuffer.allocate(4).putInt(i).array()) + + val writer = input.toDF() + .selectExpr("byteCast(value) key", "byteCast(value) value") + .writeStream + .format("kafka") + .option("checkpointLocation", checkpointDir.getCanonicalPath) + .outputMode(OutputMode.Append) + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("defaultTopic", topic) + .queryName("kafkaStream") + .start() + + // Create Kafka source that reads from earliest to latest offset + val reader = spark.readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + .option("subscribe", topic) + .option("failOnDataLoss", "false") + .load() + .selectExpr("intCast(key)", "intCast(value)") + .as[(Int, Int)] + input.addData(1, 2, 3) + writer.processAllAvailable() + + testStream(reader)( + AddData(input, 1, 2, 3), // Add data when stream is stopped + CheckAnswer((1, 1), (2, 2), (3, 3)), + StopStream + ) + + } + } } From 3c6a52b2442c7f15eeb75ae5b3fdeba4ba004dfe Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Fri, 17 Feb 2017 15:06:57 -0800 Subject: [PATCH 24/40] update --- .../spark/sql/kafka010/KafkaWriter.scala | 7 +++++-- .../spark/sql/kafka010/KafkaSinkSuite.scala | 20 +++++++++---------- 2 files changed, 14 insertions(+), 13 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala index 4c83cd6773ea..5b81996e9b14 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution.{QueryExecution, SQLExecution} -import org.apache.spark.sql.types.StringType +import org.apache.spark.sql.types.{ByteType, DataTypes, StringType} import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.Utils @@ -138,6 +138,8 @@ object KafkaWriter extends Logging { var confirmedWrites = 0 var failedWrites = ListBuffer.empty[Throwable] + println(s"INPUT SCHEMA ${inputSchema.toString()}") + val producer = new KafkaProducer[Array[Byte], Array[Byte]](producerConfiguration) val topicExpression = inputSchema.find(p => p.name == TOPIC_ATTRIBUTE_NAME).getOrElse( if (defaultTopic == None) { @@ -178,7 +180,8 @@ object KafkaWriter extends Logging { val topic = getTopic(currentRow).get(0, StringType).toString val key = getKey(currentRow).getBytes val value = getValue(currentRow).getBytes - println(s"topic $topic, key ${ByteBuffer.wrap(key).getInt}, value $value") + println(s"topic $topic, key ${new String(key)}, " + + s"value ${new String(value)}") val record = new ProducerRecord[Array[Byte], Array[Byte]](topic, key, value) val future = Future[RecordMetadata] { blocking { diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala index 5e50dd5b0094..8e53b0b078c7 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala @@ -53,14 +53,11 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext { test("write to stream") { withTempDir { checkpointDir => - val input = MemoryStream[Int] + val input = MemoryStream[String] val topic = newTopic() - spark.udf.register("intCast", (s: Array[Byte]) => ByteBuffer.wrap(s).getInt()) - spark.udf.register("byteCast", (i: Int) => ByteBuffer.allocate(4).putInt(i).array()) - - val writer = input.toDF() - .selectExpr("byteCast(value) key", "byteCast(value) value") + val writer = input.toDF.map(s => s.get(0).toString.getBytes()).toDF("value") + .selectExpr("value as key", "value as value") .writeStream .format("kafka") .option("checkpointLocation", checkpointDir.getCanonicalPath) @@ -75,18 +72,19 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext { .format("kafka") .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.metadata.max.age.ms", "1") + .option("startingOffsets", "earliest") .option("subscribe", topic) .option("failOnDataLoss", "false") .load() - .selectExpr("intCast(key)", "intCast(value)") - .as[(Int, Int)] + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] - input.addData(1, 2, 3) + input.addData("1", "2", "3") writer.processAllAvailable() testStream(reader)( - AddData(input, 1, 2, 3), // Add data when stream is stopped - CheckAnswer((1, 1), (2, 2), (3, 3)), + AddData(input, "1", "2", "3"), + CheckAnswer(("1", "1"), ("2", "2"), ("3", "3")), StopStream ) From 8ba33a7b1e970880aa3881bcaaef91cbe60ca62c Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Wed, 22 Feb 2017 18:32:04 -0800 Subject: [PATCH 25/40] update --- .../sql/kafka010/KafkaSourceProvider.scala | 3 - .../spark/sql/kafka010/KafkaWriter.scala | 105 +++++++++++------- .../spark/sql/kafka010/KafkaSinkSuite.scala | 45 ++++++-- 3 files changed, 98 insertions(+), 55 deletions(-) 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 d374d454ba7d..d8a0d12da72f 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 @@ -159,9 +159,6 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister parameters: Map[String, String], partitionColumns: Seq[String], outputMode: OutputMode): Sink = { - if (outputMode != OutputMode.Append()) { - throw new IllegalArgumentException(s"Kafka supports ${OutputMode.Append()} only") - } val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase, v) } val defaultTopic = caseInsensitiveParams.get(DEFAULT_TOPIC).map(_.trim.toLowerCase) val specifiedKafkaParams = diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala index 5b81996e9b14..2c82aaa1655c 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala @@ -18,12 +18,12 @@ package org.apache.spark.sql.kafka010 import java.{util => ju} -import java.nio.ByteBuffer import scala.collection.mutable.ListBuffer import scala.concurrent.{blocking, Future} import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord, RecordMetadata} +import org.apache.kafka.common.serialization.{ByteArraySerializer, StringSerializer} import org.apache.spark.{SparkException, TaskContext} import org.apache.spark.internal.Logging @@ -31,7 +31,7 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution.{QueryExecution, SQLExecution} -import org.apache.spark.sql.types.{ByteType, DataTypes, StringType} +import org.apache.spark.sql.types.{BinaryType, StringType} import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.Utils @@ -55,14 +55,30 @@ object KafkaWriter extends Logging { if (defaultTopic == None) { throw new IllegalArgumentException(s"Default topic required when no " + s"'$TOPIC_ATTRIBUTE_NAME' attribute is present") + } else { + Literal(defaultTopic.get, StringType) } - ) + ).dataType match { + case StringType => // good + case _ => + throw new IllegalArgumentException(s"Topic type must be a String") + } schema.find(p => p.name == KEY_ATTRIBUTE_NAME).getOrElse( - throw new IllegalArgumentException(s"Required attribute '$KEY_ATTRIBUTE_NAME' not found") - ) - schema.find(p => p.name == VALUE_ATTRIBUTE_NAME).getOrElse( + Literal(null, StringType) + ).dataType match { + case StringType | BinaryType => // good + case _ => + throw new IllegalArgumentException(s"$KEY_ATTRIBUTE_NAME attribute type " + + s"must be a String or BinaryType") + } + val valueField = schema.find(p => p.name == VALUE_ATTRIBUTE_NAME).getOrElse( throw new IllegalArgumentException(s"Required attribute '$VALUE_ATTRIBUTE_NAME' not found") - ) + ).dataType match { + case StringType | BinaryType => // good + case _ => + throw new IllegalArgumentException(s"$VALUE_ATTRIBUTE_NAME attribute type " + + s"must be a String or BinaryType") + } SQLExecution.withNewExecutionId(sparkSession, queryExecution) { try { val ret = sparkSession.sparkContext.runJob(queryExecution.toRdd, @@ -88,13 +104,13 @@ object KafkaWriter extends Logging { /** Writes data out in a single Spark task. */ private def executeTask( - iterator: Iterator[InternalRow], - producerConfiguration: ju.Map[String, Object], - sparkStageId: Int, - sparkPartitionId: Int, - sparkAttemptNumber: Int, - inputSchema: Seq[Attribute], - defaultTopic: Option[String]): TaskCommitMessage = { + iterator: Iterator[InternalRow], + producerConfiguration: ju.Map[String, Object], + sparkStageId: Int, + sparkPartitionId: Int, + sparkAttemptNumber: Int, + inputSchema: Seq[Attribute], + defaultTopic: Option[String]): TaskCommitMessage = { val writeTask = new KafkaWriteTask( producerConfiguration, inputSchema, defaultTopic) try { @@ -116,9 +132,6 @@ object KafkaWriter extends Logging { } if (writeTask.failedWrites.size == 0) { - assert(writeTask.confirmedWrites == writeTask.producerRecordCount, - s"Confirmed writes ${writeTask.confirmedWrites} != " + - s"records written ${writeTask.producerRecordCount}") TaskCommitMessage(sparkStageId, sparkPartitionId, writeCommitted = true) } else { TaskCommitMessage(sparkStageId, sparkPartitionId, writeCommitted = false) @@ -131,16 +144,11 @@ object KafkaWriter extends Logging { * automatically trigger task aborts. */ private class KafkaWriteTask( - producerConfiguration: ju.Map[String, Object], - inputSchema: Seq[Attribute], - defaultTopic: Option[String]) { - var producerRecordCount = 0 + producerConfiguration: ju.Map[String, Object], + inputSchema: Seq[Attribute], + defaultTopic: Option[String]) { var confirmedWrites = 0 var failedWrites = ListBuffer.empty[Throwable] - - println(s"INPUT SCHEMA ${inputSchema.toString()}") - - val producer = new KafkaProducer[Array[Byte], Array[Byte]](producerConfiguration) val topicExpression = inputSchema.find(p => p.name == TOPIC_ATTRIBUTE_NAME).getOrElse( if (defaultTopic == None) { throw new IllegalStateException(s"Default topic required when no " + @@ -154,21 +162,31 @@ object KafkaWriter extends Logging { } else { // fall back on a default value in case we evaluate c to null If(IsNull(c), Literal(UTF8String.fromString(defaultTopic.get), StringType), c) - }} - // Use to extract the topic from either the Row or default value - val getTopic = UnsafeProjection.create(topicExpression, inputSchema) - + } + } val keyExpression = inputSchema.find(p => p.name == KEY_ATTRIBUTE_NAME).getOrElse( - throw new IllegalStateException(s"Required attribute '$KEY_ATTRIBUTE_NAME' not found") + Literal(null, BinaryType) ) - // Use to extract the key from a Row - val getKey = UnsafeProjection.create(Seq(keyExpression), inputSchema) - + keyExpression.dataType match { + case StringType | BinaryType => // good + case t => + throw new IllegalStateException(s"$KEY_ATTRIBUTE_NAME attribute unsupported type $t") + } val valueExpression = inputSchema.find(p => p.name == VALUE_ATTRIBUTE_NAME).getOrElse( throw new IllegalStateException(s"Required attribute '$VALUE_ATTRIBUTE_NAME' not found") ) - // Use to extract the value from a Row - val getValue = UnsafeProjection.create(Seq(valueExpression), inputSchema) + valueExpression.dataType match { + case StringType | BinaryType => // good + case t => + throw new IllegalStateException(s"$VALUE_ATTRIBUTE_NAME attribute unsupported type $t") + } + val projection = UnsafeProjection.create(topicExpression ++ + Seq(Cast(keyExpression, BinaryType), Cast(valueExpression, BinaryType)), inputSchema) + + // Create a Kafka Producer + producerConfiguration.put("key.serializer", classOf[ByteArraySerializer].getName) + producerConfiguration.put("value.serializer", classOf[ByteArraySerializer].getName) + val producer = new KafkaProducer[Any, Any](producerConfiguration) /** * Writes key value data out to topics. @@ -177,25 +195,28 @@ object KafkaWriter extends Logging { import scala.concurrent.ExecutionContext.Implicits.global while (iterator.hasNext) { val currentRow = iterator.next() - val topic = getTopic(currentRow).get(0, StringType).toString - val key = getKey(currentRow).getBytes - val value = getValue(currentRow).getBytes - println(s"topic $topic, key ${new String(key)}, " + - s"value ${new String(value)}") - val record = new ProducerRecord[Array[Byte], Array[Byte]](topic, key, value) + val projectedRow = projection(currentRow) + val topic = projectedRow.get(0, StringType).toString + val key = projectedRow.get(1, BinaryType) + val value = projectedRow.get(2, BinaryType) + // TODO: check for null value topic, which can happen when + // we have a topic field and no default + val record = new ProducerRecord[Any, Any](topic, key, value) val future = Future[RecordMetadata] { blocking { producer.send(record).get() } } future.onSuccess { - case rm => confirmedWrites += 1 + case _ => confirmedWrites += 1 } future.onFailure { case e => failedWrites += e } + scala.concurrent.Await.ready(future, scala.concurrent.duration.Duration.Inf) } producer.flush() + println(s"Confirmed writes $confirmedWrites, Failures ${failedWrites.mkString}") } def releaseResources(): Unit = { diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala index 8e53b0b078c7..17d46124ef5a 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala @@ -17,18 +17,26 @@ package org.apache.spark.sql.kafka010 -import java.nio.ByteBuffer import java.util.concurrent.atomic.AtomicInteger import org.scalatest.time.SpanSugar._ import org.apache.spark.sql.execution.streaming.MemoryStream -import org.apache.spark.sql.streaming.{OutputMode, StreamTest} +import org.apache.spark.sql.streaming.{OutputMode, ProcessingTime, StreamingQuery, StreamTest} import org.apache.spark.sql.test.SharedSQLContext class KafkaSinkSuite extends StreamTest with SharedSQLContext { import testImplicits._ + case class AddMoreData(ms: MemoryStream[String], q: StreamingQuery, + values: String*) extends ExternalAction { + override def runAction(): Unit = { + ms.addData(values) + q.processAllAvailable() + Thread.sleep(5000) // wait for data to appear in Kafka + } + } + protected var testUtils: KafkaTestUtils = _ override val streamingTimeout = 30.seconds @@ -56,8 +64,7 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext { val input = MemoryStream[String] val topic = newTopic() - val writer = input.toDF.map(s => s.get(0).toString.getBytes()).toDF("value") - .selectExpr("value as key", "value as value") + val writer = input.toDF().toDF("value") .writeStream .format("kafka") .option("checkpointLocation", checkpointDir.getCanonicalPath) @@ -76,18 +83,36 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext { .option("subscribe", topic) .option("failOnDataLoss", "false") .load() + val kafka = reader .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") .as[(String, String)] + val mapped: org.apache.spark.sql.Dataset[_] = kafka.map(kv => kv._2.trim.toInt) - input.addData("1", "2", "3") - writer.processAllAvailable() + testStream(mapped, outputMode = OutputMode.Append)( + StartStream(ProcessingTime(0)), + AddMoreData(input, writer, "1", "2", "3", "4", "5"), - testStream(reader)( - AddData(input, "1", "2", "3"), - CheckAnswer(("1", "1"), ("2", "2"), ("3", "3")), + CheckAnswer(1, 2, 3, 4, 5), + AddMoreData(input, writer, "6", "7", "8", "9", "10"), + CheckAnswer(1, 2, 3, 4, 5, 6, 7, 8, 9, 10), StopStream ) - } } + + test("write batch to kafka") { + + val topic = newTopic() + val df = spark + .sparkContext + .parallelize(Seq("1", "2", "3", "4", "5")) + .map(v => (topic, v)) + .toDF("topic", "value") + + df.write + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("defaultTopic", topic) + .save() + } } From 68a2a18b836ec5c23d8741268023e8f77d4c26f1 Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Thu, 23 Feb 2017 13:09:57 -0800 Subject: [PATCH 26/40] update --- .../spark/sql/kafka010/KafkaWriter.scala | 103 ++++++++++++++---- .../spark/sql/kafka010/KafkaSinkSuite.scala | 49 ++++++++- 2 files changed, 128 insertions(+), 24 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala index 2c82aaa1655c..e5b00737c134 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala @@ -19,11 +19,11 @@ package org.apache.spark.sql.kafka010 import java.{util => ju} +import scala.collection.mutable import scala.collection.mutable.ListBuffer -import scala.concurrent.{blocking, Future} -import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord, RecordMetadata} -import org.apache.kafka.common.serialization.{ByteArraySerializer, StringSerializer} +import org.apache.kafka.clients.producer.{Callback, KafkaProducer, ProducerRecord, RecordMetadata} +import org.apache.kafka.common.serialization.ByteArraySerializer import org.apache.spark.{SparkException, TaskContext} import org.apache.spark.internal.Logging @@ -39,6 +39,12 @@ object KafkaWriter extends Logging { val TOPIC_ATTRIBUTE_NAME: String = "topic" val KEY_ATTRIBUTE_NAME: String = "key" val VALUE_ATTRIBUTE_NAME: String = "value" + val MAX_OUTSTANDING_WRITES: String = "maxOutstandingWrites" + val DEFAULT_MAX_OUTSTANDING_WRITES: Int = 1000 + val WAIT_FOR_CONFIRMED_WRITE_MS = "waitForConfirmedWriteMs" + val DEFAULT_WAIT_FOR_CONFIRMED_WRITE_MS = 1000 + val WAIT_FOR_CONFIRMED_WRITE_RETRIES = "waitForConfirmedWriteRetries" + val DEFAULT_WAIT_FOR_CONFIRMED_WRITE_RETRIES = 3 private case class TaskCommitMessage( sparkStageId: Int, @@ -138,6 +144,23 @@ object KafkaWriter extends Logging { } } + /** + * Used to reference an outstanding write. Kafka will call onCompletion when + * the write has been confirmed, after which we remove it from the outstanding + * write set. + * @param uuid used to differentiate writes + * @param outstandingWriteSet to remove itself from after write is confirmed + */ + private case class KafkaCallback(uuid: ju.UUID)( + val outstandingWriteSet: mutable.HashSet[KafkaCallback]) extends Callback { + override def onCompletion(recordMetadata: RecordMetadata, e: Exception): Unit = { + outstandingWriteSet.synchronized { + outstandingWriteSet.remove(this) + outstandingWriteSet.notifyAll() + } + } + } + /** * A simple trait for writing out data in a single Spark task, without any concerns about how * to commit or abort tasks. Exceptions thrown by the implementation of this trait will @@ -147,7 +170,25 @@ object KafkaWriter extends Logging { producerConfiguration: ju.Map[String, Object], inputSchema: Seq[Attribute], defaultTopic: Option[String]) { - var confirmedWrites = 0 + val waitForConfirmedWriteMs = + if (producerConfiguration.containsKey(WAIT_FOR_CONFIRMED_WRITE_MS)) { + producerConfiguration.get(WAIT_FOR_CONFIRMED_WRITE_MS).asInstanceOf[Int] + } else { + DEFAULT_WAIT_FOR_CONFIRMED_WRITE_MS + } + val waitForConfirmedWriteRetries = + if (producerConfiguration.containsKey(WAIT_FOR_CONFIRMED_WRITE_RETRIES)) { + producerConfiguration.get(WAIT_FOR_CONFIRMED_WRITE_RETRIES).asInstanceOf[Int] + } else { + DEFAULT_WAIT_FOR_CONFIRMED_WRITE_RETRIES + } + val maxOutstandingWrites = + if (producerConfiguration.containsKey(MAX_OUTSTANDING_WRITES)) { + producerConfiguration.get(MAX_OUTSTANDING_WRITES).asInstanceOf[Int] + } else { + DEFAULT_MAX_OUTSTANDING_WRITES + } + val outstandingWriteSet = new mutable.HashSet[KafkaCallback] var failedWrites = ListBuffer.empty[Throwable] val topicExpression = inputSchema.find(p => p.name == TOPIC_ATTRIBUTE_NAME).getOrElse( if (defaultTopic == None) { @@ -186,40 +227,58 @@ object KafkaWriter extends Logging { // Create a Kafka Producer producerConfiguration.put("key.serializer", classOf[ByteArraySerializer].getName) producerConfiguration.put("value.serializer", classOf[ByteArraySerializer].getName) - val producer = new KafkaProducer[Any, Any](producerConfiguration) + val producer = new KafkaProducer[Array[Byte], Array[Byte]](producerConfiguration) /** * Writes key value data out to topics. */ def execute(iterator: Iterator[InternalRow]): Unit = { - import scala.concurrent.ExecutionContext.Implicits.global while (iterator.hasNext) { val currentRow = iterator.next() val projectedRow = projection(currentRow) val topic = projectedRow.get(0, StringType).toString - val key = projectedRow.get(1, BinaryType) - val value = projectedRow.get(2, BinaryType) - // TODO: check for null value topic, which can happen when - // we have a topic field and no default - val record = new ProducerRecord[Any, Any](topic, key, value) - val future = Future[RecordMetadata] { - blocking { - producer.send(record).get() + val key = projectedRow.get(1, BinaryType).asInstanceOf[Array[Byte]] + val value = projectedRow.get(2, BinaryType).asInstanceOf[Array[Byte]] + /* TODO: check for null value topic, which can happen when + * we have a topic field and no default */ + val record = new ProducerRecord[Array[Byte], Array[Byte]](topic, key, value) + val callback = outstandingWriteSet.synchronized { + var waitRetries = waitForConfirmedWriteRetries + while (outstandingWriteSet.size >= maxOutstandingWrites && waitRetries > 0) { + waitRetries -= 1 + outstandingWriteSet.wait(waitForConfirmedWriteMs) } + if (outstandingWriteSet.size >= maxOutstandingWrites) { + throw new SparkException(s"Outstanding Kafka writes not draining") + } + // okay we have head room now, create the callback and add to set + val callback = new KafkaCallback(ju.UUID.randomUUID())(outstandingWriteSet) + outstandingWriteSet.add(callback) + callback } - future.onSuccess { - case _ => confirmedWrites += 1 - } - future.onFailure { - case e => failedWrites += e - } - scala.concurrent.Await.ready(future, scala.concurrent.duration.Duration.Inf) + producer.send(record, callback) } producer.flush() - println(s"Confirmed writes $confirmedWrites, Failures ${failedWrites.mkString}") } def releaseResources(): Unit = { + /* Ensure that all writes are confirmed */ + outstandingWriteSet.synchronized { + var waitRetries = waitForConfirmedWriteRetries + while (outstandingWriteSet.size > 0 && waitRetries > 0) { + val currentSize = outstandingWriteSet.size + waitRetries -= 1 + outstandingWriteSet.wait(waitForConfirmedWriteMs) + if (currentSize < outstandingWriteSet.size) { + /* we made progress so let's keep retrying */ + waitRetries = waitForConfirmedWriteRetries + } + } + if (outstandingWriteSet.size > 0) { + throw new SparkException(s"Unable to confirm ${outstandingWriteSet.size} " + + s"record writes to Kafka") + } + } producer.close() } } diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala index 17d46124ef5a..9b316331f5b0 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala @@ -81,7 +81,7 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext { .option("kafka.metadata.max.age.ms", "1") .option("startingOffsets", "earliest") .option("subscribe", topic) - .option("failOnDataLoss", "false") + .option("failOnDataLoss", "true") .load() val kafka = reader .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") @@ -97,11 +97,56 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext { CheckAnswer(1, 2, 3, 4, 5, 6, 7, 8, 9, 10), StopStream ) + writer.stop() } } - test("write batch to kafka") { + test("write structured streaming aggregation") { + withTempDir { checkpointDir => + val input = MemoryStream[String] + val topic = newTopic() + + val writer = input.toDF() + .groupBy("value") + .count() + .selectExpr("CAST(value as STRING) key", "CAST(count as STRING) value") + .writeStream + .format("kafka") + .option("checkpointLocation", checkpointDir.getCanonicalPath) + .outputMode(OutputMode.Update) + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("defaultTopic", topic) + .queryName("kafkaAggStream") + .start() + + // Create Kafka source that reads from earliest to latest offset + val reader = spark.readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + .option("startingOffsets", "earliest") + .option("subscribe", topic) + .option("failOnDataLoss", "true") + .load() + val kafka = reader + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .selectExpr("CAST(key AS INT)", "CAST(value AS INT)") + .as[(Int, Int)] + testStream(kafka, outputMode = OutputMode.Update)( + StartStream(ProcessingTime(0)), + AddMoreData(input, writer, "1", "2", "2", "3", "3", "3"), + + CheckAnswer((1, 1), (2, 2), (3, 3)), + AddMoreData(input, writer, "1", "2", "3"), + CheckAnswer((1, 1), (2, 2), (3, 3), (1, 2), (2, 3), (3, 4)), + StopStream + ) + writer.stop() + } + } + + test("write batch to kafka") { val topic = newTopic() val df = spark .sparkContext From c8c38e192595ae7c6bb9cd8b0ed56efd33834412 Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Thu, 23 Feb 2017 13:39:19 -0800 Subject: [PATCH 27/40] update --- .../spark/sql/kafka010/KafkaWriter.scala | 25 +++---------------- 1 file changed, 3 insertions(+), 22 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala index e5b00737c134..9db57390bb76 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala @@ -43,8 +43,6 @@ object KafkaWriter extends Logging { val DEFAULT_MAX_OUTSTANDING_WRITES: Int = 1000 val WAIT_FOR_CONFIRMED_WRITE_MS = "waitForConfirmedWriteMs" val DEFAULT_WAIT_FOR_CONFIRMED_WRITE_MS = 1000 - val WAIT_FOR_CONFIRMED_WRITE_RETRIES = "waitForConfirmedWriteRetries" - val DEFAULT_WAIT_FOR_CONFIRMED_WRITE_RETRIES = 3 private case class TaskCommitMessage( sparkStageId: Int, @@ -176,12 +174,6 @@ object KafkaWriter extends Logging { } else { DEFAULT_WAIT_FOR_CONFIRMED_WRITE_MS } - val waitForConfirmedWriteRetries = - if (producerConfiguration.containsKey(WAIT_FOR_CONFIRMED_WRITE_RETRIES)) { - producerConfiguration.get(WAIT_FOR_CONFIRMED_WRITE_RETRIES).asInstanceOf[Int] - } else { - DEFAULT_WAIT_FOR_CONFIRMED_WRITE_RETRIES - } val maxOutstandingWrites = if (producerConfiguration.containsKey(MAX_OUTSTANDING_WRITES)) { producerConfiguration.get(MAX_OUTSTANDING_WRITES).asInstanceOf[Int] @@ -239,13 +231,9 @@ object KafkaWriter extends Logging { val topic = projectedRow.get(0, StringType).toString val key = projectedRow.get(1, BinaryType).asInstanceOf[Array[Byte]] val value = projectedRow.get(2, BinaryType).asInstanceOf[Array[Byte]] - /* TODO: check for null value topic, which can happen when - * we have a topic field and no default */ val record = new ProducerRecord[Array[Byte], Array[Byte]](topic, key, value) val callback = outstandingWriteSet.synchronized { - var waitRetries = waitForConfirmedWriteRetries - while (outstandingWriteSet.size >= maxOutstandingWrites && waitRetries > 0) { - waitRetries -= 1 + if (outstandingWriteSet.size >= maxOutstandingWrites) { outstandingWriteSet.wait(waitForConfirmedWriteMs) } if (outstandingWriteSet.size >= maxOutstandingWrites) { @@ -264,19 +252,12 @@ object KafkaWriter extends Logging { def releaseResources(): Unit = { /* Ensure that all writes are confirmed */ outstandingWriteSet.synchronized { - var waitRetries = waitForConfirmedWriteRetries - while (outstandingWriteSet.size > 0 && waitRetries > 0) { - val currentSize = outstandingWriteSet.size - waitRetries -= 1 + if (outstandingWriteSet.size > 0) { outstandingWriteSet.wait(waitForConfirmedWriteMs) - if (currentSize < outstandingWriteSet.size) { - /* we made progress so let's keep retrying */ - waitRetries = waitForConfirmedWriteRetries - } } if (outstandingWriteSet.size > 0) { throw new SparkException(s"Unable to confirm ${outstandingWriteSet.size} " + - s"record writes to Kafka") + s"record writes to Kafka.") } } producer.close() From 71f8de01217a16cbe5415fff4939aa0fae046d17 Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Thu, 23 Feb 2017 14:17:42 -0800 Subject: [PATCH 28/40] update --- .../scala/org/apache/spark/sql/kafka010/KafkaWriter.scala | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala index 9db57390bb76..31bcc3847f6b 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.kafka010 import java.{util => ju} +import java.util.concurrent.TimeUnit import scala.collection.mutable import scala.collection.mutable.ListBuffer @@ -250,17 +251,14 @@ object KafkaWriter extends Logging { } def releaseResources(): Unit = { + producer.close(waitForConfirmedWriteMs, TimeUnit.MILLISECONDS) /* Ensure that all writes are confirmed */ outstandingWriteSet.synchronized { - if (outstandingWriteSet.size > 0) { - outstandingWriteSet.wait(waitForConfirmedWriteMs) - } if (outstandingWriteSet.size > 0) { throw new SparkException(s"Unable to confirm ${outstandingWriteSet.size} " + s"record writes to Kafka.") } } - producer.close() } } } From c4c9395b3639c0dbb19d9658313ae84701227808 Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Thu, 23 Feb 2017 17:34:35 -0800 Subject: [PATCH 29/40] address comments from @tdas --- .../spark/sql/kafka010/KafkaSource.scala | 1 + .../sql/kafka010/KafkaSourceProvider.scala | 25 ++- .../spark/sql/kafka010/KafkaWriteTask.scala | 140 ++++++++++++ .../spark/sql/kafka010/KafkaWriter.scala | 209 +++--------------- .../spark/sql/kafka010/KafkaSinkSuite.scala | 125 ++++++++++- 5 files changed, 303 insertions(+), 197 deletions(-) create mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index 109b4d18deef..6f81e203b379 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 @@ -347,3 +347,4 @@ private[kafka010] object KafkaSource { if (a.host == b.host) { a.executorId > b.executorId } else { a.host > b.host } } } + 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 d8a0d12da72f..9318bb629b4a 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,8 +38,11 @@ import org.apache.spark.sql.types.StructType * missing options even before the query is started. */ private[kafka010] class KafkaSourceProvider extends DataSourceRegister - with StreamSourceProvider with StreamSinkProvider - with RelationProvider with CreatableRelationProvider with Logging { + with StreamSourceProvider + with StreamSinkProvider + with RelationProvider + with CreatableRelationProvider + with Logging { import KafkaSourceProvider._ override def shortName(): String = "kafka" @@ -160,7 +163,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister partitionColumns: Seq[String], outputMode: OutputMode): Sink = { val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase, v) } - val defaultTopic = caseInsensitiveParams.get(DEFAULT_TOPIC).map(_.trim.toLowerCase) + val defaultTopic = caseInsensitiveParams.get(DEFAULT_TOPIC_KEY).map(_.trim.toLowerCase) val specifiedKafkaParams = parameters .keySet @@ -178,9 +181,8 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister mode: SaveMode, parameters: Map[String, String], data: DataFrame): BaseRelation = { - logInfo(s"Save mode = $mode") val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase, v) } - val defaultTopic = caseInsensitiveParams.get(DEFAULT_TOPIC).map(_.trim.toLowerCase) + val defaultTopic = caseInsensitiveParams.get(DEFAULT_TOPIC_KEY).map(_.trim.toLowerCase) val specifiedKafkaParams = parameters .keySet @@ -194,7 +196,6 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister new BaseRelation { override def sqlContext: SQLContext = outerSQLContext - override def schema: StructType = KafkaOffsetReader.kafkaSchema } } @@ -424,11 +425,11 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister } private[kafka010] object KafkaSourceProvider { - private val STRATEGY_OPTION_KEYS = Set("subscribe", "subscribepattern", "assign") - private val STARTING_OFFSETS_OPTION_KEY = "startingoffsets" - private val ENDING_OFFSETS_OPTION_KEY = "endingoffsets" - private val FAIL_ON_DATA_LOSS_OPTION_KEY = "failondataloss" - private val DEFAULT_TOPIC = "defaulttopic" + val STRATEGY_OPTION_KEYS = Set("subscribe", "subscribepattern", "assign") + val STARTING_OFFSETS_OPTION_KEY = "startingoffsets" + val ENDING_OFFSETS_OPTION_KEY = "endingoffsets" + val FAIL_ON_DATA_LOSS_OPTION_KEY = "failondataloss" + val DEFAULT_TOPIC_KEY = "defaulttopic" - private val deserClassName = classOf[ByteArrayDeserializer].getName + val deserClassName = classOf[ByteArrayDeserializer].getName } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala new file mode 100644 index 000000000000..f71d0f200a26 --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala @@ -0,0 +1,140 @@ +/* + * 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.ListBuffer + +import org.apache.kafka.clients.producer._ +import org.apache.kafka.common.serialization.ByteArraySerializer + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, If, IsNull, Literal, UnsafeProjection} +import org.apache.spark.sql.types.{BinaryType, StringType} +import org.apache.spark.unsafe.types.UTF8String + +/** + * A simple trait for writing out data in a single Spark task, without any concerns about how + * to commit or abort tasks. Exceptions thrown by the implementation of this trait will + * automatically trigger task aborts. + */ +private[kafka010] class KafkaWriteTask( + producerConfiguration: ju.Map[String, Object], + inputSchema: Seq[Attribute], + defaultTopic: Option[String]) { + var failedWrites = ListBuffer.empty[Throwable] + val topicExpression = inputSchema.find(p => + p.name == KafkaWriter.TOPIC_ATTRIBUTE_NAME).getOrElse( + if (defaultTopic == None) { + throw new IllegalStateException(s"Default topic required when no " + + s"'${KafkaWriter.TOPIC_ATTRIBUTE_NAME}' attribute is present") + } else { + Literal(null, StringType) + } + ).map{c => + if (defaultTopic == None) { + c // return null if we can't fall back on a default value + } else { + // fall back on a default value in case we evaluate c to null + If(IsNull(c), Literal(UTF8String.fromString(defaultTopic.get), StringType), c) + } + } + val keyExpression = inputSchema.find(p => + p.name == KafkaWriter.KEY_ATTRIBUTE_NAME).getOrElse( + Literal(null, BinaryType) + ) + keyExpression.dataType match { + case StringType | BinaryType => // good + case t => + throw new IllegalStateException(s"${KafkaWriter.KEY_ATTRIBUTE_NAME} " + + s"attribute unsupported type $t") + } + val valueExpression = inputSchema.find(p => + p.name == KafkaWriter.VALUE_ATTRIBUTE_NAME).getOrElse( + throw new IllegalStateException(s"Required attribute " + + s"'${KafkaWriter.VALUE_ATTRIBUTE_NAME}' not found") + ) + valueExpression.dataType match { + case StringType | BinaryType => // good + case t => + throw new IllegalStateException(s"${KafkaWriter.VALUE_ATTRIBUTE_NAME} " + + s"attribute unsupported type $t") + } + val projection = UnsafeProjection.create(topicExpression ++ + Seq(Cast(keyExpression, BinaryType), Cast(valueExpression, BinaryType)), inputSchema) + + // Create a Kafka Producer + producerConfiguration.put("key.serializer", classOf[ByteArraySerializer].getName) + producerConfiguration.put("value.serializer", classOf[ByteArraySerializer].getName) + val producer = new KafkaProducer[Array[Byte], Array[Byte]](producerConfiguration) + + /** + * Writes key value data out to topics. + */ + def execute(iterator: Iterator[InternalRow]): Unit = { + while (iterator.hasNext) { + val currentRow = iterator.next() + val projectedRow = projection(currentRow) + val topic = projectedRow.get(0, StringType).toString + val key = projectedRow.get(1, BinaryType).asInstanceOf[Array[Byte]] + val value = projectedRow.get(2, BinaryType).asInstanceOf[Array[Byte]] + val record = new ProducerRecord[Array[Byte], Array[Byte]](topic, key, value) + val callback = new Callback() { + override def onCompletion(recordMetadata: RecordMetadata, e: Exception): Unit = { + failedWrites.synchronized { + if (e != null) { + failedWrites += e + } + } + } + } + try { + producer.send(record, callback) + } catch { + case _: BufferExhaustedException => flushAndSend(record, callback) + case t: Throwable => throw t + } finally { + failedWrites.synchronized { + if (failedWrites.size > 0) { + throw new IllegalStateException(failedWrites.remove(0)) + } + } + } + } + producer.flush() + } + + def close(): Unit = { + producer.close() + /* Ensure that all writes are confirmed */ + failedWrites.synchronized { + if (failedWrites.size > 0) { + throw new IllegalStateException(failedWrites.remove(0)) + } + } + } + + private def flushAndSend( + record: ProducerRecord[Array[Byte], Array[Byte]], + callback: Callback): Unit = { + producer.flush() + producer.send(record, callback) + } +} + diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala index 31bcc3847f6b..4b859152a735 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala @@ -19,11 +19,12 @@ package org.apache.spark.sql.kafka010 import java.{util => ju} import java.util.concurrent.TimeUnit +import javax.annotation.concurrent.GuardedBy import scala.collection.mutable import scala.collection.mutable.ListBuffer -import org.apache.kafka.clients.producer.{Callback, KafkaProducer, ProducerRecord, RecordMetadata} +import org.apache.kafka.clients.producer._ import org.apache.kafka.common.serialization.ByteArraySerializer import org.apache.spark.{SparkException, TaskContext} @@ -36,30 +37,22 @@ import org.apache.spark.sql.types.{BinaryType, StringType} import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.Utils -object KafkaWriter extends Logging { +private[kafka010] object KafkaWriter extends Logging { val TOPIC_ATTRIBUTE_NAME: String = "topic" val KEY_ATTRIBUTE_NAME: String = "key" val VALUE_ATTRIBUTE_NAME: String = "value" - val MAX_OUTSTANDING_WRITES: String = "maxOutstandingWrites" - val DEFAULT_MAX_OUTSTANDING_WRITES: Int = 1000 - val WAIT_FOR_CONFIRMED_WRITE_MS = "waitForConfirmedWriteMs" - val DEFAULT_WAIT_FOR_CONFIRMED_WRITE_MS = 1000 - - private case class TaskCommitMessage( - sparkStageId: Int, - sparkPartitionId: Int, - writeCommitted: Boolean) extends Serializable - - def write(sparkSession: SparkSession, - queryExecution: QueryExecution, - kafkaParameters: ju.Map[String, Object], - defaultTopic: Option[String] = None): Unit = { + def write( + sparkSession: SparkSession, + queryExecution: QueryExecution, + kafkaParameters: ju.Map[String, Object], + defaultTopic: Option[String] = None): Unit = { val schema = queryExecution.logical.output schema.find(p => p.name == TOPIC_ATTRIBUTE_NAME).getOrElse( if (defaultTopic == None) { throw new IllegalArgumentException(s"Default topic required when no " + - s"'$TOPIC_ATTRIBUTE_NAME' attribute is present") + s"'$TOPIC_ATTRIBUTE_NAME' attribute is present. Use the " + + s"${KafkaSourceProvider.DEFAULT_TOPIC_KEY} option for setting a default topic.") } else { Literal(defaultTopic.get, StringType) } @@ -85,25 +78,17 @@ object KafkaWriter extends Logging { s"must be a String or BinaryType") } SQLExecution.withNewExecutionId(sparkSession, queryExecution) { - try { - val ret = sparkSession.sparkContext.runJob(queryExecution.toRdd, - (taskContext: TaskContext, iter: Iterator[InternalRow]) => { - executeTask( - iterator = iter, - producerConfiguration = kafkaParameters, - sparkStageId = taskContext.stageId(), - sparkPartitionId = taskContext.partitionId(), - sparkAttemptNumber = taskContext.attemptNumber(), - inputSchema = schema, - defaultTopic = defaultTopic) - }) - - // logInfo(s"Job ${job.getJobID} committed.") - } catch { - case cause: Throwable => - // logError(s"Aborting job ${job.getJobID}.", cause) - throw new SparkException("Job aborted.", cause) - } + sparkSession.sparkContext.runJob(queryExecution.toRdd, + (taskContext: TaskContext, iter: Iterator[InternalRow]) => { + executeTask( + iterator = iter, + producerConfiguration = kafkaParameters, + sparkStageId = taskContext.stageId(), + sparkPartitionId = taskContext.partitionId(), + sparkAttemptNumber = taskContext.attemptNumber(), + inputSchema = schema, + defaultTopic = defaultTopic) + }) } } @@ -115,150 +100,16 @@ object KafkaWriter extends Logging { sparkPartitionId: Int, sparkAttemptNumber: Int, inputSchema: Seq[Attribute], - defaultTopic: Option[String]): TaskCommitMessage = { + defaultTopic: Option[String]): Unit = { val writeTask = new KafkaWriteTask( producerConfiguration, inputSchema, defaultTopic) - try { - Utils.tryWithSafeFinallyAndFailureCallbacks(block = { - // Execute the task to write rows out and commit the task. - writeTask.execute(iterator) - writeTask.releaseResources() - })(catchBlock = { - // If there is an error, release resource and then abort the task - try { - writeTask.releaseResources() - } finally { - logError(s"Stage $sparkStageId, task $sparkPartitionId aborted.") - } - }) - } catch { - case t: Throwable => - throw new SparkException("Task failed while writing rows", t) - } - - if (writeTask.failedWrites.size == 0) { - TaskCommitMessage(sparkStageId, sparkPartitionId, writeCommitted = true) - } else { - TaskCommitMessage(sparkStageId, sparkPartitionId, writeCommitted = false) - } - } - - /** - * Used to reference an outstanding write. Kafka will call onCompletion when - * the write has been confirmed, after which we remove it from the outstanding - * write set. - * @param uuid used to differentiate writes - * @param outstandingWriteSet to remove itself from after write is confirmed - */ - private case class KafkaCallback(uuid: ju.UUID)( - val outstandingWriteSet: mutable.HashSet[KafkaCallback]) extends Callback { - override def onCompletion(recordMetadata: RecordMetadata, e: Exception): Unit = { - outstandingWriteSet.synchronized { - outstandingWriteSet.remove(this) - outstandingWriteSet.notifyAll() - } - } - } - - /** - * A simple trait for writing out data in a single Spark task, without any concerns about how - * to commit or abort tasks. Exceptions thrown by the implementation of this trait will - * automatically trigger task aborts. - */ - private class KafkaWriteTask( - producerConfiguration: ju.Map[String, Object], - inputSchema: Seq[Attribute], - defaultTopic: Option[String]) { - val waitForConfirmedWriteMs = - if (producerConfiguration.containsKey(WAIT_FOR_CONFIRMED_WRITE_MS)) { - producerConfiguration.get(WAIT_FOR_CONFIRMED_WRITE_MS).asInstanceOf[Int] - } else { - DEFAULT_WAIT_FOR_CONFIRMED_WRITE_MS - } - val maxOutstandingWrites = - if (producerConfiguration.containsKey(MAX_OUTSTANDING_WRITES)) { - producerConfiguration.get(MAX_OUTSTANDING_WRITES).asInstanceOf[Int] - } else { - DEFAULT_MAX_OUTSTANDING_WRITES - } - val outstandingWriteSet = new mutable.HashSet[KafkaCallback] - var failedWrites = ListBuffer.empty[Throwable] - val topicExpression = inputSchema.find(p => p.name == TOPIC_ATTRIBUTE_NAME).getOrElse( - if (defaultTopic == None) { - throw new IllegalStateException(s"Default topic required when no " + - s"'$TOPIC_ATTRIBUTE_NAME' attribute is present") - } else { - Literal(null, StringType) - } - ).map{c => - if (defaultTopic == None) { - c // return null if we can't fall back on a default value - } else { - // fall back on a default value in case we evaluate c to null - If(IsNull(c), Literal(UTF8String.fromString(defaultTopic.get), StringType), c) - } - } - val keyExpression = inputSchema.find(p => p.name == KEY_ATTRIBUTE_NAME).getOrElse( - Literal(null, BinaryType) - ) - keyExpression.dataType match { - case StringType | BinaryType => // good - case t => - throw new IllegalStateException(s"$KEY_ATTRIBUTE_NAME attribute unsupported type $t") - } - val valueExpression = inputSchema.find(p => p.name == VALUE_ATTRIBUTE_NAME).getOrElse( - throw new IllegalStateException(s"Required attribute '$VALUE_ATTRIBUTE_NAME' not found") - ) - valueExpression.dataType match { - case StringType | BinaryType => // good - case t => - throw new IllegalStateException(s"$VALUE_ATTRIBUTE_NAME attribute unsupported type $t") - } - val projection = UnsafeProjection.create(topicExpression ++ - Seq(Cast(keyExpression, BinaryType), Cast(valueExpression, BinaryType)), inputSchema) - - // Create a Kafka Producer - producerConfiguration.put("key.serializer", classOf[ByteArraySerializer].getName) - producerConfiguration.put("value.serializer", classOf[ByteArraySerializer].getName) - val producer = new KafkaProducer[Array[Byte], Array[Byte]](producerConfiguration) - - /** - * Writes key value data out to topics. - */ - def execute(iterator: Iterator[InternalRow]): Unit = { - while (iterator.hasNext) { - val currentRow = iterator.next() - val projectedRow = projection(currentRow) - val topic = projectedRow.get(0, StringType).toString - val key = projectedRow.get(1, BinaryType).asInstanceOf[Array[Byte]] - val value = projectedRow.get(2, BinaryType).asInstanceOf[Array[Byte]] - val record = new ProducerRecord[Array[Byte], Array[Byte]](topic, key, value) - val callback = outstandingWriteSet.synchronized { - if (outstandingWriteSet.size >= maxOutstandingWrites) { - outstandingWriteSet.wait(waitForConfirmedWriteMs) - } - if (outstandingWriteSet.size >= maxOutstandingWrites) { - throw new SparkException(s"Outstanding Kafka writes not draining") - } - // okay we have head room now, create the callback and add to set - val callback = new KafkaCallback(ju.UUID.randomUUID())(outstandingWriteSet) - outstandingWriteSet.add(callback) - callback - } - producer.send(record, callback) - } - producer.flush() - } - - def releaseResources(): Unit = { - producer.close(waitForConfirmedWriteMs, TimeUnit.MILLISECONDS) - /* Ensure that all writes are confirmed */ - outstandingWriteSet.synchronized { - if (outstandingWriteSet.size > 0) { - throw new SparkException(s"Unable to confirm ${outstandingWriteSet.size} " + - s"record writes to Kafka.") - } - } - } + Utils.tryWithSafeFinallyAndFailureCallbacks(block = { + // Execute the task to write rows out and commit the task. + writeTask.execute(iterator) + })(catchBlock = { + logError(s"Stage $sparkStageId, task $sparkPartitionId aborted.") + }, finallyBlock = { + writeTask.close() + }) } } diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala index 9b316331f5b0..fd73310cd957 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala @@ -19,10 +19,11 @@ package org.apache.spark.sql.kafka010 import java.util.concurrent.atomic.AtomicInteger +import org.apache.kafka.common.errors.TimeoutException import org.scalatest.time.SpanSugar._ import org.apache.spark.sql.execution.streaming.MemoryStream -import org.apache.spark.sql.streaming.{OutputMode, ProcessingTime, StreamingQuery, StreamTest} +import org.apache.spark.sql.streaming._ import org.apache.spark.sql.test.SharedSQLContext class KafkaSinkSuite extends StreamTest with SharedSQLContext { @@ -43,7 +44,7 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext { override def beforeAll(): Unit = { super.beforeAll() - testUtils = new KafkaTestUtils + testUtils = new KafkaTestUtils(withBrokerProps = Map("auto.create.topics.enable" -> "false")) testUtils.setup() } @@ -59,18 +60,19 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext { private def newTopic(): String = s"topic-${topicId.getAndIncrement()}" - test("write to stream") { + test("write to stream with topic field") { withTempDir { checkpointDir => val input = MemoryStream[String] val topic = newTopic() + testUtils.createTopic(topic) - val writer = input.toDF().toDF("value") + val writer = input.toDF() + .selectExpr(s"'$topic' as topic", "value") .writeStream .format("kafka") .option("checkpointLocation", checkpointDir.getCanonicalPath) .outputMode(OutputMode.Append) .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("defaultTopic", topic) .queryName("kafkaStream") .start() @@ -101,10 +103,11 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext { } } - test("write structured streaming aggregation") { + test("write structured streaming aggregation w/o topic field, with default topic") { withTempDir { checkpointDir => val input = MemoryStream[String] val topic = newTopic() + testUtils.createTopic(topic) val writer = input.toDF() .groupBy("value") @@ -146,8 +149,118 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext { } } + test("write data with bad schema") { + withTempDir { checkpointDir => + val input = MemoryStream[String] + val topic = newTopic() + testUtils.createTopic(topic) + + /* No topic field or default topic */ + var writer: StreamingQuery = null + var ex = intercept[StreamingQueryException] { + writer = input.toDF() + .selectExpr("value as key", "value") + .writeStream + .format("kafka") + .option("checkpointLocation", checkpointDir.getCanonicalPath) + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .queryName("kafkaNoTopicFieldStream") + .start() + input.addData("1", "2", "3", "4", "5") + writer.processAllAvailable() + } + writer.stop() + assert(ex.getMessage + .toLowerCase + .contains("default topic required when no 'topic' attribute is present")) + + /* No value field */ + ex = intercept[StreamingQueryException] { + writer = input.toDF() + .selectExpr(s"'$topic' as topic", "value as key") + .writeStream + .format("kafka") + .option("checkpointLocation", checkpointDir.getCanonicalPath) + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .queryName("kafkaNoValueFieldStream") + .start() + input.addData("1", "2", "3", "4", "5") + writer.processAllAvailable() + } + writer.stop() + assert(ex.getMessage.toLowerCase.contains("required attribute 'value' not found")) + } + } + + test("write data with valid schema but wrong types") { + withTempDir { checkpointDir => + val input = MemoryStream[String] + val topic = newTopic() + testUtils.createTopic(topic) + + /* value field wrong type */ + var writer: StreamingQuery = null + var ex = intercept[StreamingQueryException] { + writer = input.toDF() + .selectExpr(s"'$topic' as topic", "CAST(value as INT) as value") + .writeStream + .format("kafka") + .option("checkpointLocation", checkpointDir.getCanonicalPath) + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .queryName("kafkaIntValueFieldStream") + .start() + input.addData("1", "2", "3", "4", "5") + writer.processAllAvailable() + } + writer.stop() + assert(ex.getMessage.toLowerCase.contains( + "value attribute type must be a string or binarytype")) + + /* key field wrong type */ + ex = intercept[StreamingQueryException] { + writer = input.toDF() + .selectExpr(s"'$topic' as topic", "CAST(value as INT) as key", "value") + .writeStream + .format("kafka") + .option("checkpointLocation", checkpointDir.getCanonicalPath) + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .queryName("kafkaIntValueFieldStream") + .start() + input.addData("1", "2", "3", "4", "5") + writer.processAllAvailable() + } + writer.stop() + assert(ex.getMessage.toLowerCase.contains( + "key attribute type must be a string or binarytype")) + } + } + + test("write to non-existing topic") { + withTempDir { checkpointDir => + val input = MemoryStream[String] + val topic = newTopic() + + var writer: StreamingQuery = null + val ex = intercept[StreamingQueryException] { + writer = input.toDF() + .writeStream + .format("kafka") + .option("checkpointLocation", checkpointDir.getCanonicalPath) + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("defaultTopic", topic) + .queryName("kafkaBadTopicStream") + .start() + input.addData("1", "2", "3", "4", "5") + writer.processAllAvailable() + } + writer.stop() + assert(ex.getMessage.toLowerCase.contains("job aborted")) + } + } + test("write batch to kafka") { val topic = newTopic() + testUtils.createTopic(topic) val df = spark .sparkContext .parallelize(Seq("1", "2", "3", "4", "5")) From 8f5da8b377ba711bb7eea4d2f69baed5a49ee3f6 Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Thu, 23 Feb 2017 17:37:33 -0800 Subject: [PATCH 30/40] update --- .../apache/spark/sql/kafka010/KafkaWriteTask.scala | 2 +- .../org/apache/spark/sql/kafka010/KafkaWriter.scala | 13 ++----------- 2 files changed, 3 insertions(+), 12 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala index f71d0f200a26..6af3fc3c1dba 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala @@ -31,7 +31,7 @@ import org.apache.spark.unsafe.types.UTF8String /** * A simple trait for writing out data in a single Spark task, without any concerns about how - * to commit or abort tasks. Exceptions thrown by the implementation of this trait will + * to commit or abort tasks. Exceptions thrown by the implementation of this class will * automatically trigger task aborts. */ private[kafka010] class KafkaWriteTask( diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala index 4b859152a735..3b6893e50e28 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala @@ -18,23 +18,14 @@ package org.apache.spark.sql.kafka010 import java.{util => ju} -import java.util.concurrent.TimeUnit -import javax.annotation.concurrent.GuardedBy -import scala.collection.mutable -import scala.collection.mutable.ListBuffer - -import org.apache.kafka.clients.producer._ -import org.apache.kafka.common.serialization.ByteArraySerializer - -import org.apache.spark.{SparkException, TaskContext} +import org.apache.spark.TaskContext import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution.{QueryExecution, SQLExecution} import org.apache.spark.sql.types.{BinaryType, StringType} -import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.Utils private[kafka010] object KafkaWriter extends Logging { @@ -69,7 +60,7 @@ private[kafka010] object KafkaWriter extends Logging { throw new IllegalArgumentException(s"$KEY_ATTRIBUTE_NAME attribute type " + s"must be a String or BinaryType") } - val valueField = schema.find(p => p.name == VALUE_ATTRIBUTE_NAME).getOrElse( + schema.find(p => p.name == VALUE_ATTRIBUTE_NAME).getOrElse( throw new IllegalArgumentException(s"Required attribute '$VALUE_ATTRIBUTE_NAME' not found") ).dataType match { case StringType | BinaryType => // good From c85b803774d0190129cb1a9d8493796a383fc1b8 Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Thu, 23 Feb 2017 19:38:15 -0800 Subject: [PATCH 31/40] update --- .../spark/sql/kafka010/KafkaWriteTask.scala | 42 ++++++------------- .../spark/sql/kafka010/KafkaSinkSuite.scala | 33 +++++++++++++++ 2 files changed, 45 insertions(+), 30 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala index 6af3fc3c1dba..cab3556d52cc 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala @@ -38,7 +38,8 @@ private[kafka010] class KafkaWriteTask( producerConfiguration: ju.Map[String, Object], inputSchema: Seq[Attribute], defaultTopic: Option[String]) { - var failedWrites = ListBuffer.empty[Throwable] + // var failedWrites = ListBuffer.empty[Throwable] + @volatile var failedWrite: Exception = null val topicExpression = inputSchema.find(p => p.name == KafkaWriter.TOPIC_ATTRIBUTE_NAME).getOrElse( if (defaultTopic == None) { @@ -88,7 +89,7 @@ private[kafka010] class KafkaWriteTask( * Writes key value data out to topics. */ def execute(iterator: Iterator[InternalRow]): Unit = { - while (iterator.hasNext) { + while (iterator.hasNext && failedWrite == null) { val currentRow = iterator.next() val projectedRow = projection(currentRow) val topic = projectedRow.get(0, StringType).toString @@ -97,44 +98,25 @@ private[kafka010] class KafkaWriteTask( val record = new ProducerRecord[Array[Byte], Array[Byte]](topic, key, value) val callback = new Callback() { override def onCompletion(recordMetadata: RecordMetadata, e: Exception): Unit = { - failedWrites.synchronized { - if (e != null) { - failedWrites += e - } - } - } - } - try { - producer.send(record, callback) - } catch { - case _: BufferExhaustedException => flushAndSend(record, callback) - case t: Throwable => throw t - } finally { - failedWrites.synchronized { - if (failedWrites.size > 0) { - throw new IllegalStateException(failedWrites.remove(0)) + if (failedWrite == null && e != null) { + failedWrite = e } } } + producer.send(record, callback) } - producer.flush() } def close(): Unit = { + checkForErrors() producer.close() - /* Ensure that all writes are confirmed */ - failedWrites.synchronized { - if (failedWrites.size > 0) { - throw new IllegalStateException(failedWrites.remove(0)) - } - } + checkForErrors() } - private def flushAndSend( - record: ProducerRecord[Array[Byte], Array[Byte]], - callback: Callback): Unit = { - producer.flush() - producer.send(record, callback) + private def checkForErrors() = { + if (failedWrite != null) { + throw failedWrite + } } } diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala index fd73310cd957..4e7c7e0d59ed 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala @@ -17,14 +17,18 @@ package org.apache.spark.sql.kafka010 +import java.nio.ByteBuffer import java.util.concurrent.atomic.AtomicInteger import org.apache.kafka.common.errors.TimeoutException import org.scalatest.time.SpanSugar._ +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, PrettyAttribute, SpecificInternalRow, UnsafeProjection} import org.apache.spark.sql.execution.streaming.MemoryStream import org.apache.spark.sql.streaming._ import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.{BinaryType, DataType} class KafkaSinkSuite extends StreamTest with SharedSQLContext { import testImplicits._ @@ -273,4 +277,33 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext { .option("defaultTopic", topic) .save() } + + test("write big data with small producer buffer") { + val topic = newTopic() + testUtils.createTopic(topic, 1) + val options = new java.util.HashMap[String, Object] + options.put("bootstrap.servers", testUtils.brokerAddress) + options.put("buffer.memory", "16384") + // options.put("linger.ms", "1000") + val inputSchema = Seq(AttributeReference("value", BinaryType)()) + val data = new Array[Byte](15000) + val writeTask = new KafkaWriteTask(options, inputSchema, Some(topic)) + writeTask.execute(new Iterator[InternalRow]() { + var count = 0 + override def hasNext: Boolean = count < 1 + + override def next(): InternalRow = { + count += 1 + val fieldTypes: Array[DataType] = Array(BinaryType) + val converter = UnsafeProjection.create(fieldTypes) + + val row = new SpecificInternalRow(fieldTypes) + row.update(0, data) + converter.apply(row) + } + }) + println("Calling close") + writeTask.close() + println("Close returns") + } } From 9d7a00dd9a614454b9cff3f24c6ca7c8151a8fcd Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Fri, 24 Feb 2017 08:20:33 -0800 Subject: [PATCH 32/40] update --- .../scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala index 4e7c7e0d59ed..2311121efd58 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala @@ -302,8 +302,6 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext { converter.apply(row) } }) - println("Calling close") writeTask.close() - println("Close returns") } } From 66fa01bac54c751e35bfca2c0e7dc4a860d3899b Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Fri, 24 Feb 2017 10:04:08 -0800 Subject: [PATCH 33/40] update --- .../spark/sql/kafka010/KafkaWriteTask.scala | 8 ++- .../spark/sql/kafka010/KafkaWriter.scala | 12 ++-- .../spark/sql/kafka010/KafkaSinkSuite.scala | 61 ++++++++++++------- 3 files changed, 51 insertions(+), 30 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala index cab3556d52cc..8bb9a822e6d3 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala @@ -92,10 +92,14 @@ private[kafka010] class KafkaWriteTask( while (iterator.hasNext && failedWrite == null) { val currentRow = iterator.next() val projectedRow = projection(currentRow) - val topic = projectedRow.get(0, StringType).toString + val topic = projectedRow.get(0, StringType) val key = projectedRow.get(1, BinaryType).asInstanceOf[Array[Byte]] val value = projectedRow.get(2, BinaryType).asInstanceOf[Array[Byte]] - val record = new ProducerRecord[Array[Byte], Array[Byte]](topic, key, value) + if (topic == null) { + throw new NullPointerException(s"null topic present in the data. Use the " + + s"${KafkaSourceProvider.DEFAULT_TOPIC_KEY} option for setting a default topic.") + } + val record = new ProducerRecord[Array[Byte], Array[Byte]](topic.toString, key, value) val callback = new Callback() { override def onCompletion(recordMetadata: RecordMetadata, e: Exception): Unit = { if (failedWrite == null && e != null) { diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala index 3b6893e50e28..bf8dbf843742 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala @@ -21,7 +21,7 @@ import java.{util => ju} import org.apache.spark.TaskContext import org.apache.spark.internal.Logging -import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution.{QueryExecution, SQLExecution} @@ -41,7 +41,7 @@ private[kafka010] object KafkaWriter extends Logging { val schema = queryExecution.logical.output schema.find(p => p.name == TOPIC_ATTRIBUTE_NAME).getOrElse( if (defaultTopic == None) { - throw new IllegalArgumentException(s"Default topic required when no " + + throw new AnalysisException(s"Default topic required when no " + s"'$TOPIC_ATTRIBUTE_NAME' attribute is present. Use the " + s"${KafkaSourceProvider.DEFAULT_TOPIC_KEY} option for setting a default topic.") } else { @@ -50,22 +50,22 @@ private[kafka010] object KafkaWriter extends Logging { ).dataType match { case StringType => // good case _ => - throw new IllegalArgumentException(s"Topic type must be a String") + throw new AnalysisException(s"Topic type must be a String") } schema.find(p => p.name == KEY_ATTRIBUTE_NAME).getOrElse( Literal(null, StringType) ).dataType match { case StringType | BinaryType => // good case _ => - throw new IllegalArgumentException(s"$KEY_ATTRIBUTE_NAME attribute type " + + throw new AnalysisException(s"$KEY_ATTRIBUTE_NAME attribute type " + s"must be a String or BinaryType") } schema.find(p => p.name == VALUE_ATTRIBUTE_NAME).getOrElse( - throw new IllegalArgumentException(s"Required attribute '$VALUE_ATTRIBUTE_NAME' not found") + throw new AnalysisException(s"Required attribute '$VALUE_ATTRIBUTE_NAME' not found") ).dataType match { case StringType | BinaryType => // good case _ => - throw new IllegalArgumentException(s"$VALUE_ATTRIBUTE_NAME attribute type " + + throw new AnalysisException(s"$VALUE_ATTRIBUTE_NAME attribute type " + s"must be a String or BinaryType") } SQLExecution.withNewExecutionId(sparkSession, queryExecution) { diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala index 2311121efd58..d7fec55fbfbd 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala @@ -23,6 +23,7 @@ import java.util.concurrent.atomic.AtomicInteger import org.apache.kafka.common.errors.TimeoutException import org.scalatest.time.SpanSugar._ +import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{AttributeReference, PrettyAttribute, SpecificInternalRow, UnsafeProjection} import org.apache.spark.sql.execution.streaming.MemoryStream @@ -278,30 +279,46 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext { .save() } + test("write batch with null topic field value, and no default topic") { + val df = spark + .sparkContext + .parallelize(Seq("1")) + .map(v => (null.asInstanceOf[String], v)) + .toDF("topic", "value") + + val ex = intercept[SparkException] { + df.write + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .save() + } + assert(ex.getMessage.toLowerCase.contains( + "null topic present in the data")) + } + test("write big data with small producer buffer") { val topic = newTopic() testUtils.createTopic(topic, 1) - val options = new java.util.HashMap[String, Object] - options.put("bootstrap.servers", testUtils.brokerAddress) - options.put("buffer.memory", "16384") - // options.put("linger.ms", "1000") - val inputSchema = Seq(AttributeReference("value", BinaryType)()) - val data = new Array[Byte](15000) - val writeTask = new KafkaWriteTask(options, inputSchema, Some(topic)) - writeTask.execute(new Iterator[InternalRow]() { - var count = 0 - override def hasNext: Boolean = count < 1 - - override def next(): InternalRow = { - count += 1 - val fieldTypes: Array[DataType] = Array(BinaryType) - val converter = UnsafeProjection.create(fieldTypes) - - val row = new SpecificInternalRow(fieldTypes) - row.update(0, data) - converter.apply(row) - } - }) - writeTask.close() + val options = new java.util.HashMap[String, Object] + options.put("bootstrap.servers", testUtils.brokerAddress) + options.put("buffer.memory", "16384") + val inputSchema = Seq(AttributeReference("value", BinaryType)()) + val data = new Array[Byte](15000) + val writeTask = new KafkaWriteTask(options, inputSchema, Some(topic)) + writeTask.execute(new Iterator[InternalRow]() { + var count = 0 + override def hasNext: Boolean = count < 1000 + + override def next(): InternalRow = { + count += 1 + val fieldTypes: Array[DataType] = Array(BinaryType) + val converter = UnsafeProjection.create(fieldTypes) + + val row = new SpecificInternalRow(fieldTypes) + row.update(0, data) + converter.apply(row) + } + }) + writeTask.close() } } From 129cfcd4620190f65b895abb6bd09ddead9fbcff Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Fri, 24 Feb 2017 10:06:35 -0800 Subject: [PATCH 34/40] update --- .../scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala index d7fec55fbfbd..0d7cd66e6250 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala @@ -301,9 +301,9 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext { testUtils.createTopic(topic, 1) val options = new java.util.HashMap[String, Object] options.put("bootstrap.servers", testUtils.brokerAddress) - options.put("buffer.memory", "16384") + options.put("buffer.memory", "16384") // min buffer size val inputSchema = Seq(AttributeReference("value", BinaryType)()) - val data = new Array[Byte](15000) + val data = new Array[Byte](15000) // large value val writeTask = new KafkaWriteTask(options, inputSchema, Some(topic)) writeTask.execute(new Iterator[InternalRow]() { var count = 0 From 67e3c0660bbd27b00e7f5fc8aebd047d02a18571 Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Fri, 24 Feb 2017 10:07:48 -0800 Subject: [PATCH 35/40] update --- .../scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala index 0d7cd66e6250..c20fea73605e 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala @@ -17,15 +17,13 @@ package org.apache.spark.sql.kafka010 -import java.nio.ByteBuffer import java.util.concurrent.atomic.AtomicInteger -import org.apache.kafka.common.errors.TimeoutException import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, PrettyAttribute, SpecificInternalRow, UnsafeProjection} +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, SpecificInternalRow, UnsafeProjection} import org.apache.spark.sql.execution.streaming.MemoryStream import org.apache.spark.sql.streaming._ import org.apache.spark.sql.test.SharedSQLContext From e6b6dc1035588dfdb8d4a16e2004ad3e0c838adb Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Fri, 24 Feb 2017 12:16:52 -0800 Subject: [PATCH 36/40] revise exceptions and topic option --- .../sql/kafka010/KafkaSourceProvider.scala | 15 ++- .../spark/sql/kafka010/KafkaWriteTask.scala | 39 +++---- .../spark/sql/kafka010/KafkaWriter.scala | 12 +- .../spark/sql/kafka010/KafkaSinkSuite.scala | 109 ++++++++++++++++-- 4 files changed, 133 insertions(+), 42 deletions(-) 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 9318bb629b4a..cd55b1fba341 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 @@ -26,7 +26,7 @@ import org.apache.kafka.clients.consumer.ConsumerConfig import org.apache.kafka.common.serialization.{ByteArrayDeserializer, BytesSerializer} import org.apache.spark.internal.Logging -import org.apache.spark.sql.{DataFrame, SaveMode, SQLContext} +import org.apache.spark.sql.{AnalysisException, DataFrame, SaveMode, SQLContext} import org.apache.spark.sql.execution.streaming.{Sink, Source} import org.apache.spark.sql.sources._ import org.apache.spark.sql.streaming.OutputMode @@ -163,7 +163,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister partitionColumns: Seq[String], outputMode: OutputMode): Sink = { val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase, v) } - val defaultTopic = caseInsensitiveParams.get(DEFAULT_TOPIC_KEY).map(_.trim.toLowerCase) + val defaultTopic = caseInsensitiveParams.get(TOPIC_OPTION_KEY).map(_.trim.toLowerCase) val specifiedKafkaParams = parameters .keySet @@ -181,8 +181,15 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister mode: SaveMode, parameters: Map[String, String], data: DataFrame): BaseRelation = { + mode match { + case SaveMode.Overwrite | SaveMode.Ignore => + throw new AnalysisException(s"save mode $mode not allowed for Kafka. " + + s"Allowable save modes are ${SaveMode.Append} and " + + s"${SaveMode.ErrorIfExists} (default).") + case _ => // good + } val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase, v) } - val defaultTopic = caseInsensitiveParams.get(DEFAULT_TOPIC_KEY).map(_.trim.toLowerCase) + val defaultTopic = caseInsensitiveParams.get(TOPIC_OPTION_KEY).map(_.trim.toLowerCase) val specifiedKafkaParams = parameters .keySet @@ -429,7 +436,7 @@ private[kafka010] object KafkaSourceProvider { val STARTING_OFFSETS_OPTION_KEY = "startingoffsets" val ENDING_OFFSETS_OPTION_KEY = "endingoffsets" val FAIL_ON_DATA_LOSS_OPTION_KEY = "failondataloss" - val DEFAULT_TOPIC_KEY = "defaulttopic" + val TOPIC_OPTION_KEY = "topic" val deserClassName = classOf[ByteArrayDeserializer].getName } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala index 8bb9a822e6d3..706b2ad11008 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala @@ -19,13 +19,11 @@ package org.apache.spark.sql.kafka010 import java.{util => ju} -import scala.collection.mutable.ListBuffer - import org.apache.kafka.clients.producer._ import org.apache.kafka.common.serialization.ByteArraySerializer import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, If, IsNull, Literal, UnsafeProjection} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Literal, UnsafeProjection} import org.apache.spark.sql.types.{BinaryType, StringType} import org.apache.spark.unsafe.types.UTF8String @@ -37,24 +35,18 @@ import org.apache.spark.unsafe.types.UTF8String private[kafka010] class KafkaWriteTask( producerConfiguration: ju.Map[String, Object], inputSchema: Seq[Attribute], - defaultTopic: Option[String]) { - // var failedWrites = ListBuffer.empty[Throwable] + topic: Option[String]) { + // used to synchronize with Kafka callbacks @volatile var failedWrite: Exception = null - val topicExpression = inputSchema.find(p => - p.name == KafkaWriter.TOPIC_ATTRIBUTE_NAME).getOrElse( - if (defaultTopic == None) { - throw new IllegalStateException(s"Default topic required when no " + - s"'${KafkaWriter.TOPIC_ATTRIBUTE_NAME}' attribute is present") - } else { - Literal(null, StringType) - } - ).map{c => - if (defaultTopic == None) { - c // return null if we can't fall back on a default value - } else { - // fall back on a default value in case we evaluate c to null - If(IsNull(c), Literal(UTF8String.fromString(defaultTopic.get), StringType), c) - } + val topicExpression = + if (topic.isDefined) { + // topic option overrides topic field + Literal(UTF8String.fromString(topic.get), StringType) + } else { + inputSchema.find(p => + p.name == KafkaWriter.TOPIC_ATTRIBUTE_NAME).getOrElse( + throw new IllegalStateException(s"topic option required when no " + + s"'${KafkaWriter.TOPIC_ATTRIBUTE_NAME}' attribute is present")) } val keyExpression = inputSchema.find(p => p.name == KafkaWriter.KEY_ATTRIBUTE_NAME).getOrElse( @@ -77,8 +69,9 @@ private[kafka010] class KafkaWriteTask( throw new IllegalStateException(s"${KafkaWriter.VALUE_ATTRIBUTE_NAME} " + s"attribute unsupported type $t") } - val projection = UnsafeProjection.create(topicExpression ++ - Seq(Cast(keyExpression, BinaryType), Cast(valueExpression, BinaryType)), inputSchema) + val projection = UnsafeProjection.create( + Seq(topicExpression, Cast(keyExpression, BinaryType), + Cast(valueExpression, BinaryType)), inputSchema) // Create a Kafka Producer producerConfiguration.put("key.serializer", classOf[ByteArraySerializer].getName) @@ -97,7 +90,7 @@ private[kafka010] class KafkaWriteTask( val value = projectedRow.get(2, BinaryType).asInstanceOf[Array[Byte]] if (topic == null) { throw new NullPointerException(s"null topic present in the data. Use the " + - s"${KafkaSourceProvider.DEFAULT_TOPIC_KEY} option for setting a default topic.") + s"${KafkaSourceProvider.TOPIC_OPTION_KEY} option for setting a default topic.") } val record = new ProducerRecord[Array[Byte], Array[Byte]](topic.toString, key, value) val callback = new Callback() { diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala index bf8dbf843742..f442ac0aa84c 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala @@ -37,15 +37,15 @@ private[kafka010] object KafkaWriter extends Logging { sparkSession: SparkSession, queryExecution: QueryExecution, kafkaParameters: ju.Map[String, Object], - defaultTopic: Option[String] = None): Unit = { + topic: Option[String] = None): Unit = { val schema = queryExecution.logical.output schema.find(p => p.name == TOPIC_ATTRIBUTE_NAME).getOrElse( - if (defaultTopic == None) { - throw new AnalysisException(s"Default topic required when no " + + if (topic == None) { + throw new AnalysisException(s"topic option required when no " + s"'$TOPIC_ATTRIBUTE_NAME' attribute is present. Use the " + - s"${KafkaSourceProvider.DEFAULT_TOPIC_KEY} option for setting a default topic.") + s"${KafkaSourceProvider.TOPIC_OPTION_KEY} option for setting a topic.") } else { - Literal(defaultTopic.get, StringType) + Literal(topic.get, StringType) } ).dataType match { case StringType => // good @@ -78,7 +78,7 @@ private[kafka010] object KafkaWriter extends Logging { sparkPartitionId = taskContext.partitionId(), sparkAttemptNumber = taskContext.attemptNumber(), inputSchema = schema, - defaultTopic = defaultTopic) + defaultTopic = topic) }) } } diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala index c20fea73605e..db555f2dd3ed 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala @@ -22,6 +22,7 @@ import java.util.concurrent.atomic.AtomicInteger import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkException +import org.apache.spark.sql.{AnalysisException, SaveMode} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{AttributeReference, SpecificInternalRow, UnsafeProjection} import org.apache.spark.sql.execution.streaming.MemoryStream @@ -32,7 +33,9 @@ import org.apache.spark.sql.types.{BinaryType, DataType} class KafkaSinkSuite extends StreamTest with SharedSQLContext { import testImplicits._ - case class AddMoreData(ms: MemoryStream[String], q: StreamingQuery, + case class AddMoreData( + ms: MemoryStream[String], + q: StreamingQuery, values: String*) extends ExternalAction { override def runAction(): Unit = { ms.addData(values) @@ -47,7 +50,8 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext { override def beforeAll(): Unit = { super.beforeAll() - testUtils = new KafkaTestUtils(withBrokerProps = Map("auto.create.topics.enable" -> "false")) + testUtils = new KafkaTestUtils( + withBrokerProps = Map("auto.create.topics.enable" -> "false")) testUtils.setup() } @@ -106,7 +110,7 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext { } } - test("write structured streaming aggregation w/o topic field, with default topic") { + test("write structured streaming aggregation w/o topic field, with topic option") { withTempDir { checkpointDir => val input = MemoryStream[String] val topic = newTopic() @@ -121,7 +125,7 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext { .option("checkpointLocation", checkpointDir.getCanonicalPath) .outputMode(OutputMode.Update) .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("defaultTopic", topic) + .option("topic", topic) .queryName("kafkaAggStream") .start() @@ -152,13 +156,68 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext { } } + test("write structured streaming aggregation with topic field and topic option") { + /* The purpose of this test is to ensure that the topic option + * overrides the topic field. We begin by writing some data that + * includes a topic field and value (e.g., 'foo') along with a topic + * option. Then when we read from the topic specified in the option + * we should see the data i.e., the data was written to the topic + * option, and not to the topic in the data e.g., foo + */ + withTempDir { checkpointDir => + val input = MemoryStream[String] + val topic = newTopic() + testUtils.createTopic(topic) + + val writer = input.toDF() + .groupBy("value") + .count() + .selectExpr("'foo' as topic", + "CAST(value as STRING) key", "CAST(count as STRING) value") + .writeStream + .format("kafka") + .option("checkpointLocation", checkpointDir.getCanonicalPath) + .outputMode(OutputMode.Update) + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("topic", topic) + .queryName("kafkaAggStream") + .start() + + // Create Kafka source that reads from earliest to latest offset + val reader = spark.readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + .option("startingOffsets", "earliest") + .option("subscribe", topic) + .option("failOnDataLoss", "true") + .load() + val kafka = reader + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .selectExpr("CAST(key AS INT)", "CAST(value AS INT)") + .as[(Int, Int)] + + testStream(kafka, outputMode = OutputMode.Update)( + StartStream(ProcessingTime(0)), + AddMoreData(input, writer, "1", "2", "2", "3", "3", "3"), + + CheckAnswer((1, 1), (2, 2), (3, 3)), + AddMoreData(input, writer, "1", "2", "3"), + CheckAnswer((1, 1), (2, 2), (3, 3), (1, 2), (2, 3), (3, 4)), + StopStream + ) + writer.stop() + } + } + + test("write data with bad schema") { withTempDir { checkpointDir => val input = MemoryStream[String] val topic = newTopic() testUtils.createTopic(topic) - /* No topic field or default topic */ + /* No topic field or topic option */ var writer: StreamingQuery = null var ex = intercept[StreamingQueryException] { writer = input.toDF() @@ -175,7 +234,7 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext { writer.stop() assert(ex.getMessage .toLowerCase - .contains("default topic required when no 'topic' attribute is present")) + .contains("topic option required when no 'topic' attribute is present")) /* No value field */ ex = intercept[StreamingQueryException] { @@ -250,7 +309,7 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext { .format("kafka") .option("checkpointLocation", checkpointDir.getCanonicalPath) .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("defaultTopic", topic) + .option("topic", topic) .queryName("kafkaBadTopicStream") .start() input.addData("1", "2", "3", "4", "5") @@ -273,11 +332,11 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext { df.write .format("kafka") .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("defaultTopic", topic) + .option("topic", topic) .save() } - test("write batch with null topic field value, and no default topic") { + test("write batch with null topic field value, and no topic option") { val df = spark .sparkContext .parallelize(Seq("1")) @@ -294,6 +353,38 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext { "null topic present in the data")) } + test("write batch unsupported save modes") { + val topic = newTopic() + testUtils.createTopic(topic) + val df = spark + .sparkContext + .parallelize(Seq("1")) + .map(v => (null.asInstanceOf[String], v)) + .toDF("topic", "value") + + // Test bad save mode Ignore + var ex = intercept[AnalysisException] { + df.write + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .mode(SaveMode.Ignore) + .save() + } + assert(ex.getMessage.toLowerCase.contains( + "save mode ignore not allowed for kafka")) + + // Test bad save mode Overwrite + ex = intercept[AnalysisException] { + df.write + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .mode(SaveMode.Overwrite) + .save() + } + assert(ex.getMessage.toLowerCase.contains( + "save mode overwrite not allowed for kafka")) + } + test("write big data with small producer buffer") { val topic = newTopic() testUtils.createTopic(topic, 1) From b48f173005cf784210f5dbc2fd7322b3ea8e0257 Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Tue, 28 Feb 2017 14:25:01 -0800 Subject: [PATCH 37/40] address comments from @tdas @zsxwing --- .../apache/spark/sql/kafka010/KafkaSink.scala | 10 +- .../spark/sql/kafka010/KafkaSource.scala | 2 +- .../sql/kafka010/KafkaSourceProvider.scala | 70 ++- .../spark/sql/kafka010/KafkaWriteTask.scala | 104 ++-- .../spark/sql/kafka010/KafkaWriter.scala | 65 +-- .../spark/sql/kafka010/KafkaSinkSuite.scala | 528 +++++++++--------- 6 files changed, 391 insertions(+), 388 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSink.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSink.scala index 2726eafe05fa..e1a0713cdc43 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSink.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSink.scala @@ -24,10 +24,12 @@ import org.apache.spark.sql.{DataFrame, SQLContext} import org.apache.spark.sql.execution.streaming.Sink private[kafka010] class KafkaSink( - sqlContext: SQLContext, - executorKafkaParams: ju.Map[String, Object], - defaultTopic: Option[String]) extends Sink with Logging { - var latestBatchId = -1L + sqlContext: SQLContext, + executorKafkaParams: ju.Map[String, Object], + defaultTopic: Option[String]) extends Sink with Logging { + @volatile private var latestBatchId = -1L + + override def toString(): String = "KafkaSink" override def addBatch(batchId: Long, data: DataFrame): Unit = { if (batchId <= latestBatchId) { 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 4611abd4b2d4..92b5d91ba435 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 @@ -364,5 +364,5 @@ 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 } } -} +} 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 cd55b1fba341..b371ee307081 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 @@ -162,18 +162,10 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister parameters: Map[String, String], partitionColumns: Seq[String], outputMode: OutputMode): Sink = { - val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase, v) } - val defaultTopic = caseInsensitiveParams.get(TOPIC_OPTION_KEY).map(_.trim.toLowerCase) - val specifiedKafkaParams = - parameters - .keySet - .filter(_.toLowerCase.startsWith("kafka.")) - .map { k => k.drop(6).toString -> parameters(k) } - .toMap + ("value.serializer" -> classOf[BytesSerializer].getName, - "key.serializer" -> classOf[BytesSerializer].getName) + val defaultTopic = parameters.get(TOPIC_OPTION_KEY).map(_.trim) + val specifiedKafkaParams = kafkaParamsForProducer(parameters) new KafkaSink(sqlContext, - new ju.HashMap[String, Object](specifiedKafkaParams.asJava), - defaultTopic) + new ju.HashMap[String, Object](specifiedKafkaParams.asJava), defaultTopic) } override def createRelation( @@ -183,30 +175,50 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister data: DataFrame): BaseRelation = { mode match { case SaveMode.Overwrite | SaveMode.Ignore => - throw new AnalysisException(s"save mode $mode not allowed for Kafka. " + - s"Allowable save modes are ${SaveMode.Append} and " + + throw new AnalysisException(s"Save mode $mode not allowed for Kafka. " + + s"Allowed save modes are ${SaveMode.Append} and " + s"${SaveMode.ErrorIfExists} (default).") case _ => // good } - val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase, v) } - val defaultTopic = caseInsensitiveParams.get(TOPIC_OPTION_KEY).map(_.trim.toLowerCase) - val specifiedKafkaParams = - parameters - .keySet - .filter(_.toLowerCase.startsWith("kafka.")) - .map { k => k.drop(6).toString -> parameters(k) } - .toMap + ("value.serializer" -> classOf[BytesSerializer].getName, - "key.serializer" -> classOf[BytesSerializer].getName) + val defaultTopic = parameters.get(TOPIC_OPTION_KEY).map(_.trim.toLowerCase) + val specifiedKafkaParams = kafkaParamsForProducer(parameters) KafkaWriter.write(outerSQLContext.sparkSession, data.queryExecution, - new ju.HashMap[String, Object](specifiedKafkaParams.asJava), - defaultTopic) + new ju.HashMap[String, Object](specifiedKafkaParams.asJava), defaultTopic) + /* This method is suppose to return a relation that reads the data that was written. + * We cannot support this for Kafka. Therefore, in order to make things consistent, + * we return an empty base relation. + */ new BaseRelation { override def sqlContext: SQLContext = outerSQLContext override def schema: StructType = KafkaOffsetReader.kafkaSchema } } + private def kafkaParamsForProducer(parameters: Map[String, String]): Map[String, String] = { + val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase, v) } + if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG}")) { + throw new IllegalArgumentException( + s"Kafka option '${ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG}' is not supported as keys " + + "are deserialized as byte arrays with ByteArrayDeserializer. Use DataFrame operations " + + "to explicitly deserialize the keys.") + } + + if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG}")) + { + throw new IllegalArgumentException( + s"Kafka option '${ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG}' is not supported as " + + "value are deserialized as byte arrays with ByteArrayDeserializer. Use DataFrame " + + "operations to explicitly deserialize the values.") + } + parameters + .keySet + .filter(_.toLowerCase.startsWith("kafka.")) + .map { k => k.drop(6).toString -> parameters(k) } + .toMap + (ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG -> classOf[BytesSerializer].getName, + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG -> classOf[BytesSerializer].getName) + } + private def kafkaParamsForDriver(specifiedKafkaParams: Map[String, String]) = ConfigUpdater("source", specifiedKafkaParams) .set(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, deserClassName) @@ -432,11 +444,11 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister } private[kafka010] object KafkaSourceProvider { - val STRATEGY_OPTION_KEYS = Set("subscribe", "subscribepattern", "assign") - val STARTING_OFFSETS_OPTION_KEY = "startingoffsets" - val ENDING_OFFSETS_OPTION_KEY = "endingoffsets" - val FAIL_ON_DATA_LOSS_OPTION_KEY = "failondataloss" + private val STRATEGY_OPTION_KEYS = Set("subscribe", "subscribepattern", "assign") + private val STARTING_OFFSETS_OPTION_KEY = "startingoffsets" + private val ENDING_OFFSETS_OPTION_KEY = "endingoffsets" + private val FAIL_ON_DATA_LOSS_OPTION_KEY = "failondataloss" val TOPIC_OPTION_KEY = "topic" - val deserClassName = classOf[ByteArrayDeserializer].getName + private val deserClassName = classOf[ByteArrayDeserializer].getName } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala index 706b2ad11008..13ab77dc4bda 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala @@ -19,13 +19,12 @@ package org.apache.spark.sql.kafka010 import java.{util => ju} -import org.apache.kafka.clients.producer._ +import org.apache.kafka.clients.producer.{KafkaProducer, _} import org.apache.kafka.common.serialization.ByteArraySerializer import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Literal, UnsafeProjection} import org.apache.spark.sql.types.{BinaryType, StringType} -import org.apache.spark.unsafe.types.UTF8String /** * A simple trait for writing out data in a single Spark task, without any concerns about how @@ -38,56 +37,24 @@ private[kafka010] class KafkaWriteTask( topic: Option[String]) { // used to synchronize with Kafka callbacks @volatile var failedWrite: Exception = null - val topicExpression = - if (topic.isDefined) { - // topic option overrides topic field - Literal(UTF8String.fromString(topic.get), StringType) - } else { - inputSchema.find(p => - p.name == KafkaWriter.TOPIC_ATTRIBUTE_NAME).getOrElse( - throw new IllegalStateException(s"topic option required when no " + - s"'${KafkaWriter.TOPIC_ATTRIBUTE_NAME}' attribute is present")) - } - val keyExpression = inputSchema.find(p => - p.name == KafkaWriter.KEY_ATTRIBUTE_NAME).getOrElse( - Literal(null, BinaryType) - ) - keyExpression.dataType match { - case StringType | BinaryType => // good - case t => - throw new IllegalStateException(s"${KafkaWriter.KEY_ATTRIBUTE_NAME} " + - s"attribute unsupported type $t") - } - val valueExpression = inputSchema.find(p => - p.name == KafkaWriter.VALUE_ATTRIBUTE_NAME).getOrElse( - throw new IllegalStateException(s"Required attribute " + - s"'${KafkaWriter.VALUE_ATTRIBUTE_NAME}' not found") - ) - valueExpression.dataType match { - case StringType | BinaryType => // good - case t => - throw new IllegalStateException(s"${KafkaWriter.VALUE_ATTRIBUTE_NAME} " + - s"attribute unsupported type $t") - } - val projection = UnsafeProjection.create( - Seq(topicExpression, Cast(keyExpression, BinaryType), - Cast(valueExpression, BinaryType)), inputSchema) - - // Create a Kafka Producer - producerConfiguration.put("key.serializer", classOf[ByteArraySerializer].getName) - producerConfiguration.put("value.serializer", classOf[ByteArraySerializer].getName) - val producer = new KafkaProducer[Array[Byte], Array[Byte]](producerConfiguration) + val projection = createProjection + var producer: KafkaProducer[Array[Byte], Array[Byte]] = _ /** * Writes key value data out to topics. */ def execute(iterator: Iterator[InternalRow]): Unit = { + producer = { + producerConfiguration.put("key.serializer", classOf[ByteArraySerializer].getName) + producerConfiguration.put("value.serializer", classOf[ByteArraySerializer].getName) + new KafkaProducer[Array[Byte], Array[Byte]](producerConfiguration) + } while (iterator.hasNext && failedWrite == null) { val currentRow = iterator.next() val projectedRow = projection(currentRow) - val topic = projectedRow.get(0, StringType) - val key = projectedRow.get(1, BinaryType).asInstanceOf[Array[Byte]] - val value = projectedRow.get(2, BinaryType).asInstanceOf[Array[Byte]] + val topic = projectedRow.getUTF8String(0) + val key = projectedRow.getBinary(1) + val value = projectedRow.getBinary(2) if (topic == null) { throw new NullPointerException(s"null topic present in the data. Use the " + s"${KafkaSourceProvider.TOPIC_OPTION_KEY} option for setting a default topic.") @@ -105,12 +72,53 @@ private[kafka010] class KafkaWriteTask( } def close(): Unit = { - checkForErrors() - producer.close() - checkForErrors() + if (producer != null) { + checkForErrors + producer.close() + checkForErrors + producer = null + } + } + + private def createProjection: UnsafeProjection = { + val topicExpression = topic.map(Literal(_)).orElse { + inputSchema.find(_.name == KafkaWriter.TOPIC_ATTRIBUTE_NAME) + }.getOrElse { + throw new IllegalStateException(s"topic option required when no " + + s"'${KafkaWriter.TOPIC_ATTRIBUTE_NAME}' attribute is present") + } + topicExpression.dataType match { + case StringType => // good + case t => + throw new IllegalStateException(s"${KafkaWriter.TOPIC_ATTRIBUTE_NAME} " + + s"attribute unsupported type $t. ${KafkaWriter.TOPIC_ATTRIBUTE_NAME} " + + s"must be a ${StringType}") + } + val keyExpression = inputSchema.find(_.name == KafkaWriter.KEY_ATTRIBUTE_NAME) + .getOrElse(Literal(null, BinaryType)) + keyExpression.dataType match { + case StringType | BinaryType => // good + case t => + throw new IllegalStateException(s"${KafkaWriter.KEY_ATTRIBUTE_NAME} " + + s"attribute unsupported type $t") + } + val valueExpression = inputSchema + .find(_.name == KafkaWriter.VALUE_ATTRIBUTE_NAME).getOrElse( + throw new IllegalStateException(s"Required attribute " + + s"'${KafkaWriter.VALUE_ATTRIBUTE_NAME}' not found") + ) + valueExpression.dataType match { + case StringType | BinaryType => // good + case t => + throw new IllegalStateException(s"${KafkaWriter.VALUE_ATTRIBUTE_NAME} " + + s"attribute unsupported type $t") + } + UnsafeProjection.create( + Seq(topicExpression, Cast(keyExpression, BinaryType), + Cast(valueExpression, BinaryType)), inputSchema) } - private def checkForErrors() = { + private def checkForErrors: Unit = { if (failedWrite != null) { throw failedWrite } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala index f442ac0aa84c..ee3f6c18abb6 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.kafka010 import java.{util => ju} -import org.apache.spark.TaskContext import org.apache.spark.internal.Logging import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.InternalRow @@ -28,18 +27,26 @@ import org.apache.spark.sql.execution.{QueryExecution, SQLExecution} import org.apache.spark.sql.types.{BinaryType, StringType} import org.apache.spark.util.Utils +/** + * The [[KafkaWriter]] class is used to write data from a batch query + * or structured streaming query, given by a [[QueryExecution]], to Kafka. + * The data is assumed to have a value column, and an optional topic and key + * columns. If the topic column is missing, then the topic must come from + * the 'topic' configuration option. If the key column is missing, then a + * null valued key field will be added to the + * [[org.apache.kafka.clients.producer.ProducerRecord]]. + */ private[kafka010] object KafkaWriter extends Logging { val TOPIC_ATTRIBUTE_NAME: String = "topic" val KEY_ATTRIBUTE_NAME: String = "key" val VALUE_ATTRIBUTE_NAME: String = "value" - def write( - sparkSession: SparkSession, + def validateQuery( queryExecution: QueryExecution, kafkaParameters: ju.Map[String, Object], topic: Option[String] = None): Unit = { val schema = queryExecution.logical.output - schema.find(p => p.name == TOPIC_ATTRIBUTE_NAME).getOrElse( + schema.find(_.name == TOPIC_ATTRIBUTE_NAME).getOrElse( if (topic == None) { throw new AnalysisException(s"topic option required when no " + s"'$TOPIC_ATTRIBUTE_NAME' attribute is present. Use the " + @@ -52,7 +59,7 @@ private[kafka010] object KafkaWriter extends Logging { case _ => throw new AnalysisException(s"Topic type must be a String") } - schema.find(p => p.name == KEY_ATTRIBUTE_NAME).getOrElse( + schema.find(_.name == KEY_ATTRIBUTE_NAME).getOrElse( Literal(null, StringType) ).dataType match { case StringType | BinaryType => // good @@ -60,7 +67,7 @@ private[kafka010] object KafkaWriter extends Logging { throw new AnalysisException(s"$KEY_ATTRIBUTE_NAME attribute type " + s"must be a String or BinaryType") } - schema.find(p => p.name == VALUE_ATTRIBUTE_NAME).getOrElse( + schema.find(_.name == VALUE_ATTRIBUTE_NAME).getOrElse( throw new AnalysisException(s"Required attribute '$VALUE_ATTRIBUTE_NAME' not found") ).dataType match { case StringType | BinaryType => // good @@ -68,39 +75,21 @@ private[kafka010] object KafkaWriter extends Logging { throw new AnalysisException(s"$VALUE_ATTRIBUTE_NAME attribute type " + s"must be a String or BinaryType") } - SQLExecution.withNewExecutionId(sparkSession, queryExecution) { - sparkSession.sparkContext.runJob(queryExecution.toRdd, - (taskContext: TaskContext, iter: Iterator[InternalRow]) => { - executeTask( - iterator = iter, - producerConfiguration = kafkaParameters, - sparkStageId = taskContext.stageId(), - sparkPartitionId = taskContext.partitionId(), - sparkAttemptNumber = taskContext.attemptNumber(), - inputSchema = schema, - defaultTopic = topic) - }) - } } - /** Writes data out in a single Spark task. */ - private def executeTask( - iterator: Iterator[InternalRow], - producerConfiguration: ju.Map[String, Object], - sparkStageId: Int, - sparkPartitionId: Int, - sparkAttemptNumber: Int, - inputSchema: Seq[Attribute], - defaultTopic: Option[String]): Unit = { - val writeTask = new KafkaWriteTask( - producerConfiguration, inputSchema, defaultTopic) - Utils.tryWithSafeFinallyAndFailureCallbacks(block = { - // Execute the task to write rows out and commit the task. - writeTask.execute(iterator) - })(catchBlock = { - logError(s"Stage $sparkStageId, task $sparkPartitionId aborted.") - }, finallyBlock = { - writeTask.close() - }) + def write( + sparkSession: SparkSession, + queryExecution: QueryExecution, + kafkaParameters: ju.Map[String, Object], + topic: Option[String] = None): Unit = { + val schema = queryExecution.logical.output + validateQuery(queryExecution, kafkaParameters, topic) + SQLExecution.withNewExecutionId(sparkSession, queryExecution) { + queryExecution.toRdd.foreachPartition { iter => + val writeTask = new KafkaWriteTask(kafkaParameters, schema, topic) + Utils.tryWithSafeFinally(block = writeTask.execute(iter))( + finallyBlock = writeTask.close()) + } + } } } diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala index db555f2dd3ed..daa7f6a4fd08 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala @@ -22,8 +22,7 @@ import java.util.concurrent.atomic.AtomicInteger import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkException -import org.apache.spark.sql.{AnalysisException, SaveMode} -import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, SpecificInternalRow, UnsafeProjection} import org.apache.spark.sql.execution.streaming.MemoryStream import org.apache.spark.sql.streaming._ @@ -33,17 +32,6 @@ import org.apache.spark.sql.types.{BinaryType, DataType} class KafkaSinkSuite extends StreamTest with SharedSQLContext { import testImplicits._ - case class AddMoreData( - ms: MemoryStream[String], - q: StreamingQuery, - values: String*) extends ExternalAction { - override def runAction(): Unit = { - ms.addData(values) - q.processAllAvailable() - Thread.sleep(5000) // wait for data to appear in Kafka - } - } - protected var testUtils: KafkaTestUtils = _ override val streamingTimeout = 30.seconds @@ -67,96 +55,157 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext { private def newTopic(): String = s"topic-${topicId.getAndIncrement()}" - test("write to stream with topic field") { - withTempDir { checkpointDir => - val input = MemoryStream[String] - val topic = newTopic() - testUtils.createTopic(topic) + private def createKafkaReader(topic: String): DataFrame = { + spark.read + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("startingOffsets", "earliest") + .option("endingOffsets", "latest") + .option("subscribe", topic) + .load() + } - val writer = input.toDF() - .selectExpr(s"'$topic' as topic", "value") - .writeStream + private def createKafkaWriter( + input: DataFrame, + withTopic: Option[String] = None, + withOutputMode: Option[OutputMode] = None, + withOptions: Option[Map[String, String]] = None) + (withSelectExpr: String*): StreamingQuery = { + var stream: DataStreamWriter[Row] = null + withTempDir { checkpointDir => + var df = input.toDF() + if (withSelectExpr.length > 0) { + df = df.selectExpr(withSelectExpr: _*) + } + stream = df.writeStream .format("kafka") .option("checkpointLocation", checkpointDir.getCanonicalPath) - .outputMode(OutputMode.Append) .option("kafka.bootstrap.servers", testUtils.brokerAddress) .queryName("kafkaStream") - .start() + withTopic.foreach(stream.option("topic", _)) + withOutputMode.foreach(stream.outputMode(_)) + withOptions.map(_.foreach(opt => stream.option(opt._1, opt._2))) + } + stream.start() + } - // Create Kafka source that reads from earliest to latest offset - val reader = spark.readStream + test("batch - write to kafka") { + val topic = newTopic() + testUtils.createTopic(topic) + val df = Seq("1", "2", "3", "4", "5").map(v => (topic, v)).toDF("topic", "value") + df.write + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("topic", topic) + .save() + } + + test("batch - null topic field value, and no topic option") { + val df = Seq[(String, String)](null.asInstanceOf[String] -> "1").toDF("topic", "value") + val ex = intercept[SparkException] { + df.write .format("kafka") .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("kafka.metadata.max.age.ms", "1") - .option("startingOffsets", "earliest") - .option("subscribe", topic) - .option("failOnDataLoss", "true") - .load() - val kafka = reader - .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") - .as[(String, String)] - val mapped: org.apache.spark.sql.Dataset[_] = kafka.map(kv => kv._2.trim.toInt) - - testStream(mapped, outputMode = OutputMode.Append)( - StartStream(ProcessingTime(0)), - AddMoreData(input, writer, "1", "2", "3", "4", "5"), - - CheckAnswer(1, 2, 3, 4, 5), - AddMoreData(input, writer, "6", "7", "8", "9", "10"), - CheckAnswer(1, 2, 3, 4, 5, 6, 7, 8, 9, 10), - StopStream - ) - writer.stop() + .save() } + assert(ex.getMessage.toLowerCase.contains( + "null topic present in the data")) } - test("write structured streaming aggregation w/o topic field, with topic option") { - withTempDir { checkpointDir => - val input = MemoryStream[String] - val topic = newTopic() - testUtils.createTopic(topic) - - val writer = input.toDF() - .groupBy("value") - .count() - .selectExpr("CAST(value as STRING) key", "CAST(count as STRING) value") - .writeStream + test("batch - unsupported save modes") { + val topic = newTopic() + testUtils.createTopic(topic) + val df = Seq[(String, String)](null.asInstanceOf[String] -> "1").toDF("topic", "value") + + // Test bad save mode Ignore + var ex = intercept[AnalysisException] { + df.write .format("kafka") - .option("checkpointLocation", checkpointDir.getCanonicalPath) - .outputMode(OutputMode.Update) .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("topic", topic) - .queryName("kafkaAggStream") - .start() + .mode(SaveMode.Ignore) + .save() + } + assert(ex.getMessage.toLowerCase.contains( + s"save mode ignore not allowed for kafka")) - // Create Kafka source that reads from earliest to latest offset - val reader = spark.readStream + // Test bad save mode Overwrite + ex = intercept[AnalysisException] { + df.write .format("kafka") .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("kafka.metadata.max.age.ms", "1") - .option("startingOffsets", "earliest") - .option("subscribe", topic) - .option("failOnDataLoss", "true") - .load() - val kafka = reader - .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") - .selectExpr("CAST(key AS INT)", "CAST(value AS INT)") - .as[(Int, Int)] - - testStream(kafka, outputMode = OutputMode.Update)( - StartStream(ProcessingTime(0)), - AddMoreData(input, writer, "1", "2", "2", "3", "3", "3"), - - CheckAnswer((1, 1), (2, 2), (3, 3)), - AddMoreData(input, writer, "1", "2", "3"), - CheckAnswer((1, 1), (2, 2), (3, 3), (1, 2), (2, 3), (3, 4)), - StopStream - ) + .mode(SaveMode.Overwrite) + .save() + } + assert(ex.getMessage.toLowerCase.contains( + s"save mode overwrite not allowed for kafka")) + } + + test("streaming - write to kafka with topic field") { + val input = MemoryStream[String] + val topic = newTopic() + testUtils.createTopic(topic) + + val writer = createKafkaWriter( + input.toDF(), + withTopic = None, + withOutputMode = Some(OutputMode.Append))( + withSelectExpr = s"'$topic' as topic", "value") + + val reader = createKafkaReader(topic) + .selectExpr("CAST(key as STRING) key", "CAST(value as STRING) value") + .selectExpr("CAST(key as INT) key", "CAST(value as INT) value") + .as[(Int, Int)] + .map(_._2) + + try { + input.addData("1", "2", "3", "4", "5") + failAfter(streamingTimeout) { + writer.processAllAvailable() + } + checkDatasetUnorderly(reader, 1, 2, 3, 4, 5) + input.addData("6", "7", "8", "9", "10") + failAfter(streamingTimeout) { + writer.processAllAvailable() + } + checkDatasetUnorderly(reader, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10) + } finally { writer.stop() } } - test("write structured streaming aggregation with topic field and topic option") { + test("streaming - write aggregation w/o topic field, with topic option") { + val input = MemoryStream[String] + val topic = newTopic() + testUtils.createTopic(topic) + + val writer = createKafkaWriter( + input.toDF().groupBy("value").count(), + withTopic = Some(topic), + withOutputMode = Some(OutputMode.Update()))( + withSelectExpr = "CAST(value as STRING) key", "CAST(count as STRING) value") + + val reader = createKafkaReader(topic) + .selectExpr("CAST(key as STRING) key", "CAST(value as STRING) value") + .selectExpr("CAST(key as INT) key", "CAST(value as INT) value") + .as[(Int, Int)] + + try { + input.addData("1", "2", "2", "3", "3", "3") + failAfter(streamingTimeout) { + writer.processAllAvailable() + } + checkDatasetUnorderly(reader, (1, 1), (2, 2), (3, 3)) + input.addData("1", "2", "3") + failAfter(streamingTimeout) { + writer.processAllAvailable() + } + checkDatasetUnorderly(reader, (1, 1), (2, 2), (3, 3), (1, 2), (2, 3), (3, 4)) + } finally { + writer.stop() + } + } + + test("streaming - aggregation with topic field and topic option") { /* The purpose of this test is to ensure that the topic option * overrides the topic field. We begin by writing some data that * includes a topic field and value (e.g., 'foo') along with a topic @@ -164,250 +213,193 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext { * we should see the data i.e., the data was written to the topic * option, and not to the topic in the data e.g., foo */ - withTempDir { checkpointDir => - val input = MemoryStream[String] - val topic = newTopic() - testUtils.createTopic(topic) - - val writer = input.toDF() - .groupBy("value") - .count() - .selectExpr("'foo' as topic", - "CAST(value as STRING) key", "CAST(count as STRING) value") - .writeStream - .format("kafka") - .option("checkpointLocation", checkpointDir.getCanonicalPath) - .outputMode(OutputMode.Update) - .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("topic", topic) - .queryName("kafkaAggStream") - .start() + val input = MemoryStream[String] + val topic = newTopic() + testUtils.createTopic(topic) - // Create Kafka source that reads from earliest to latest offset - val reader = spark.readStream - .format("kafka") - .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("kafka.metadata.max.age.ms", "1") - .option("startingOffsets", "earliest") - .option("subscribe", topic) - .option("failOnDataLoss", "true") - .load() - val kafka = reader - .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") - .selectExpr("CAST(key AS INT)", "CAST(value AS INT)") - .as[(Int, Int)] - - testStream(kafka, outputMode = OutputMode.Update)( - StartStream(ProcessingTime(0)), - AddMoreData(input, writer, "1", "2", "2", "3", "3", "3"), - - CheckAnswer((1, 1), (2, 2), (3, 3)), - AddMoreData(input, writer, "1", "2", "3"), - CheckAnswer((1, 1), (2, 2), (3, 3), (1, 2), (2, 3), (3, 4)), - StopStream - ) + val writer = createKafkaWriter( + input.toDF().groupBy("value").count(), + withTopic = Some(topic), + withOutputMode = Some(OutputMode.Update()))( + withSelectExpr = "'foo' as topic", + "CAST(value as STRING) key", "CAST(count as STRING) value") + + val reader = createKafkaReader(topic) + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .selectExpr("CAST(key AS INT)", "CAST(value AS INT)") + .as[(Int, Int)] + + try { + input.addData("1", "2", "2", "3", "3", "3") + failAfter(streamingTimeout) { + writer.processAllAvailable() + } + checkDatasetUnorderly(reader, (1, 1), (2, 2), (3, 3)) + input.addData("1", "2", "3") + failAfter(streamingTimeout) { + writer.processAllAvailable() + } + checkDatasetUnorderly(reader, (1, 1), (2, 2), (3, 3), (1, 2), (2, 3), (3, 4)) + } finally { writer.stop() } } - test("write data with bad schema") { - withTempDir { checkpointDir => - val input = MemoryStream[String] - val topic = newTopic() - testUtils.createTopic(topic) - - /* No topic field or topic option */ - var writer: StreamingQuery = null - var ex = intercept[StreamingQueryException] { - writer = input.toDF() - .selectExpr("value as key", "value") - .writeStream - .format("kafka") - .option("checkpointLocation", checkpointDir.getCanonicalPath) - .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .queryName("kafkaNoTopicFieldStream") - .start() + test("streaming - write data with bad schema") { + val input = MemoryStream[String] + val topic = newTopic() + testUtils.createTopic(topic) + + /* No topic field or topic option */ + var writer: StreamingQuery = null + var ex: Exception = null + try { + ex = intercept[StreamingQueryException] { + writer = createKafkaWriter(input.toDF())( + withSelectExpr = "value as key", "value" + ) input.addData("1", "2", "3", "4", "5") writer.processAllAvailable() } + } finally { writer.stop() - assert(ex.getMessage - .toLowerCase - .contains("topic option required when no 'topic' attribute is present")) + } + assert(ex.getMessage + .toLowerCase + .contains("topic option required when no 'topic' attribute is present")) + try { /* No value field */ ex = intercept[StreamingQueryException] { - writer = input.toDF() - .selectExpr(s"'$topic' as topic", "value as key") - .writeStream - .format("kafka") - .option("checkpointLocation", checkpointDir.getCanonicalPath) - .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .queryName("kafkaNoValueFieldStream") - .start() + writer = createKafkaWriter(input.toDF())( + withSelectExpr = s"'$topic' as topic", "value as key" + ) input.addData("1", "2", "3", "4", "5") writer.processAllAvailable() } + } finally { writer.stop() - assert(ex.getMessage.toLowerCase.contains("required attribute 'value' not found")) } + assert(ex.getMessage.toLowerCase.contains("required attribute 'value' not found")) } - test("write data with valid schema but wrong types") { - withTempDir { checkpointDir => - val input = MemoryStream[String] - val topic = newTopic() - testUtils.createTopic(topic) + test("streaming - write data with valid schema but wrong types") { + val input = MemoryStream[String] + val topic = newTopic() + testUtils.createTopic(topic) - /* value field wrong type */ - var writer: StreamingQuery = null - var ex = intercept[StreamingQueryException] { - writer = input.toDF() - .selectExpr(s"'$topic' as topic", "CAST(value as INT) as value") - .writeStream - .format("kafka") - .option("checkpointLocation", checkpointDir.getCanonicalPath) - .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .queryName("kafkaIntValueFieldStream") - .start() + var writer: StreamingQuery = null + var ex: Exception = null + try { + /* topic field wrong type */ + ex = intercept[StreamingQueryException] { + writer = createKafkaWriter(input.toDF())( + withSelectExpr = s"CAST('1' as INT) as topic", "value" + ) input.addData("1", "2", "3", "4", "5") writer.processAllAvailable() } + } finally { writer.stop() - assert(ex.getMessage.toLowerCase.contains( - "value attribute type must be a string or binarytype")) + } + assert(ex.getMessage.toLowerCase.contains("topic type must be a string")) - /* key field wrong type */ + try { + /* value field wrong type */ ex = intercept[StreamingQueryException] { - writer = input.toDF() - .selectExpr(s"'$topic' as topic", "CAST(value as INT) as key", "value") - .writeStream - .format("kafka") - .option("checkpointLocation", checkpointDir.getCanonicalPath) - .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .queryName("kafkaIntValueFieldStream") - .start() + writer = createKafkaWriter(input.toDF())( + withSelectExpr = s"'$topic' as topic", "CAST(value as INT) as value" + ) input.addData("1", "2", "3", "4", "5") writer.processAllAvailable() } + } finally { writer.stop() - assert(ex.getMessage.toLowerCase.contains( - "key attribute type must be a string or binarytype")) } - } + assert(ex.getMessage.toLowerCase.contains( + "value attribute type must be a string or binarytype")) - test("write to non-existing topic") { - withTempDir { checkpointDir => - val input = MemoryStream[String] - val topic = newTopic() - - var writer: StreamingQuery = null - val ex = intercept[StreamingQueryException] { - writer = input.toDF() - .writeStream - .format("kafka") - .option("checkpointLocation", checkpointDir.getCanonicalPath) - .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("topic", topic) - .queryName("kafkaBadTopicStream") - .start() + try { + ex = intercept[StreamingQueryException] { + /* key field wrong type */ + writer = createKafkaWriter(input.toDF())( + withSelectExpr = s"'$topic' as topic", "CAST(value as INT) as key", "value" + ) input.addData("1", "2", "3", "4", "5") writer.processAllAvailable() } + } finally { writer.stop() - assert(ex.getMessage.toLowerCase.contains("job aborted")) } + assert(ex.getMessage.toLowerCase.contains( + "key attribute type must be a string or binarytype")) } - test("write batch to kafka") { + test("streaming - write to non-existing topic") { + val input = MemoryStream[String] val topic = newTopic() - testUtils.createTopic(topic) - val df = spark - .sparkContext - .parallelize(Seq("1", "2", "3", "4", "5")) - .map(v => (topic, v)) - .toDF("topic", "value") - - df.write - .format("kafka") - .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("topic", topic) - .save() - } - - test("write batch with null topic field value, and no topic option") { - val df = spark - .sparkContext - .parallelize(Seq("1")) - .map(v => (null.asInstanceOf[String], v)) - .toDF("topic", "value") - val ex = intercept[SparkException] { - df.write - .format("kafka") - .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .save() + var writer: StreamingQuery = null + var ex: Exception = null + try { + ex = intercept[StreamingQueryException] { + writer = createKafkaWriter(input.toDF(), withTopic = Some(topic))() + input.addData("1", "2", "3", "4", "5") + writer.processAllAvailable() + } + } finally { + writer.stop() } - assert(ex.getMessage.toLowerCase.contains( - "null topic present in the data")) + assert(ex.getMessage.toLowerCase.contains("job aborted")) } - test("write batch unsupported save modes") { - val topic = newTopic() - testUtils.createTopic(topic) - val df = spark - .sparkContext - .parallelize(Seq("1")) - .map(v => (null.asInstanceOf[String], v)) - .toDF("topic", "value") - - // Test bad save mode Ignore - var ex = intercept[AnalysisException] { - df.write - .format("kafka") - .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .mode(SaveMode.Ignore) - .save() + test("streaming - exception on config serializer") { + val input = MemoryStream[String] + var writer: StreamingQuery = null + var ex: Exception = null + ex = intercept[IllegalArgumentException] { + writer = createKafkaWriter( + input.toDF(), + withOptions = Some(Map("kafka.key.deserializer" -> "foo")))() } assert(ex.getMessage.toLowerCase.contains( - "save mode ignore not allowed for kafka")) + "kafka option 'key.deserializer' is not supported")) - // Test bad save mode Overwrite - ex = intercept[AnalysisException] { - df.write - .format("kafka") - .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .mode(SaveMode.Overwrite) - .save() + ex = intercept[IllegalArgumentException] { + writer = createKafkaWriter( + input.toDF(), + withOptions = Some(Map("kafka.value.deserializer" -> "foo")))() } assert(ex.getMessage.toLowerCase.contains( - "save mode overwrite not allowed for kafka")) + "kafka option 'value.deserializer' is not supported")) } - test("write big data with small producer buffer") { + test("generic - write big data with small producer buffer") { + /* This test ensures that we understand the semantics of Kafka when + * is comes to blocking on a call to send when the send buffer is full. + * This test will configure the smallest possible producer buffer and + * indicate that we should block when it is full. Thus, no exception should + * be thrown in the case of a full buffer. + */ val topic = newTopic() testUtils.createTopic(topic, 1) val options = new java.util.HashMap[String, Object] options.put("bootstrap.servers", testUtils.brokerAddress) options.put("buffer.memory", "16384") // min buffer size + options.put("block.on.buffer.full", "true") val inputSchema = Seq(AttributeReference("value", BinaryType)()) val data = new Array[Byte](15000) // large value val writeTask = new KafkaWriteTask(options, inputSchema, Some(topic)) - writeTask.execute(new Iterator[InternalRow]() { - var count = 0 - override def hasNext: Boolean = count < 1000 - - override def next(): InternalRow = { - count += 1 - val fieldTypes: Array[DataType] = Array(BinaryType) - val converter = UnsafeProjection.create(fieldTypes) - - val row = new SpecificInternalRow(fieldTypes) - row.update(0, data) - converter.apply(row) - } - }) - writeTask.close() + try { + val fieldTypes: Array[DataType] = Array(BinaryType) + val converter = UnsafeProjection.create(fieldTypes) + val row = new SpecificInternalRow(fieldTypes) + row.update(0, data) + val iter = Seq.fill(1000)(converter.apply(row)).iterator + writeTask.execute(iter) + } finally { + writeTask.close() + } } } From 2dd3ffbe4de854f9c008b2706dad3809fa8a241b Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Fri, 3 Mar 2017 15:11:21 -0800 Subject: [PATCH 38/40] update --- .../main/scala/org/apache/spark/sql/kafka010/KafkaSink.scala | 4 ++-- .../scala/org/apache/spark/sql/kafka010/KafkaWriter.scala | 2 ++ 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSink.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSink.scala index e1a0713cdc43..08914d82fffd 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSink.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSink.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.execution.streaming.Sink private[kafka010] class KafkaSink( sqlContext: SQLContext, executorKafkaParams: ju.Map[String, Object], - defaultTopic: Option[String]) extends Sink with Logging { + topic: Option[String]) extends Sink with Logging { @volatile private var latestBatchId = -1L override def toString(): String = "KafkaSink" @@ -36,7 +36,7 @@ private[kafka010] class KafkaSink( logInfo(s"Skipping already committed batch $batchId") } else { KafkaWriter.write(sqlContext.sparkSession, - data.queryExecution, executorKafkaParams, defaultTopic) + data.queryExecution, executorKafkaParams, topic) latestBatchId = batchId } } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala index ee3f6c18abb6..a637d52c933a 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala @@ -41,6 +41,8 @@ private[kafka010] object KafkaWriter extends Logging { val KEY_ATTRIBUTE_NAME: String = "key" val VALUE_ATTRIBUTE_NAME: String = "value" + override def toString: String = "KafkaWriter" + def validateQuery( queryExecution: QueryExecution, kafkaParameters: ju.Map[String, Object], From b1d554ad31938e63cf1896f9fd7906b4163694db Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Mon, 6 Mar 2017 11:58:12 -0800 Subject: [PATCH 39/40] address comments from @zsxwing --- .../sql/kafka010/KafkaSourceProvider.scala | 35 +++++++++++-------- .../spark/sql/kafka010/KafkaWriteTask.scala | 12 +++---- .../spark/sql/kafka010/KafkaSinkSuite.scala | 14 +++++--- 3 files changed, 34 insertions(+), 27 deletions(-) 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 b371ee307081..febe3c217122 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 @@ -23,7 +23,8 @@ import java.util.UUID import scala.collection.JavaConverters._ import org.apache.kafka.clients.consumer.ConsumerConfig -import org.apache.kafka.common.serialization.{ByteArrayDeserializer, BytesSerializer} +import org.apache.kafka.clients.producer.ProducerConfig +import org.apache.kafka.common.serialization.{ByteArrayDeserializer, ByteArraySerializer} import org.apache.spark.internal.Logging import org.apache.spark.sql.{AnalysisException, DataFrame, SaveMode, SQLContext} @@ -180,43 +181,47 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister s"${SaveMode.ErrorIfExists} (default).") case _ => // good } - val defaultTopic = parameters.get(TOPIC_OPTION_KEY).map(_.trim.toLowerCase) + val topic = parameters.get(TOPIC_OPTION_KEY).map(_.trim) val specifiedKafkaParams = kafkaParamsForProducer(parameters) KafkaWriter.write(outerSQLContext.sparkSession, data.queryExecution, - new ju.HashMap[String, Object](specifiedKafkaParams.asJava), defaultTopic) + new ju.HashMap[String, Object](specifiedKafkaParams.asJava), topic) /* This method is suppose to return a relation that reads the data that was written. * We cannot support this for Kafka. Therefore, in order to make things consistent, * we return an empty base relation. */ new BaseRelation { - override def sqlContext: SQLContext = outerSQLContext - override def schema: StructType = KafkaOffsetReader.kafkaSchema + override def sqlContext: SQLContext = unsupportedException + override def schema: StructType = unsupportedException + override def needConversion: Boolean = unsupportedException + override def sizeInBytes: Long = unsupportedException + override def unhandledFilters(filters: Array[Filter]): Array[Filter] = unsupportedException + private def unsupportedException = + throw new UnsupportedOperationException("BaseRelation from Kafka write " + + "operation is not usable.") } } private def kafkaParamsForProducer(parameters: Map[String, String]): Map[String, String] = { val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase, v) } - if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG}")) { + if (caseInsensitiveParams.contains(s"kafka.${ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG}")) { throw new IllegalArgumentException( - s"Kafka option '${ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG}' is not supported as keys " - + "are deserialized as byte arrays with ByteArrayDeserializer. Use DataFrame operations " - + "to explicitly deserialize the keys.") + s"Kafka option '${ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG}' is not supported as keys " + + "are serialized with ByteArraySerializer.") } - if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG}")) + if (caseInsensitiveParams.contains(s"kafka.${ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG}")) { throw new IllegalArgumentException( - s"Kafka option '${ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG}' is not supported as " - + "value are deserialized as byte arrays with ByteArrayDeserializer. Use DataFrame " - + "operations to explicitly deserialize the values.") + s"Kafka option '${ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG}' is not supported as " + + "value are serialized with ByteArraySerializer.") } parameters .keySet .filter(_.toLowerCase.startsWith("kafka.")) .map { k => k.drop(6).toString -> parameters(k) } - .toMap + (ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG -> classOf[BytesSerializer].getName, - ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG -> classOf[BytesSerializer].getName) + .toMap + (ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG -> classOf[ByteArraySerializer].getName, + ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG -> classOf[ByteArraySerializer].getName) } private def kafkaParamsForDriver(specifiedKafkaParams: Map[String, String]) = diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala index 13ab77dc4bda..6e160cbe2db5 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala @@ -36,19 +36,15 @@ private[kafka010] class KafkaWriteTask( inputSchema: Seq[Attribute], topic: Option[String]) { // used to synchronize with Kafka callbacks - @volatile var failedWrite: Exception = null - val projection = createProjection - var producer: KafkaProducer[Array[Byte], Array[Byte]] = _ + @volatile private var failedWrite: Exception = null + private val projection = createProjection + private var producer: KafkaProducer[Array[Byte], Array[Byte]] = _ /** * Writes key value data out to topics. */ def execute(iterator: Iterator[InternalRow]): Unit = { - producer = { - producerConfiguration.put("key.serializer", classOf[ByteArraySerializer].getName) - producerConfiguration.put("value.serializer", classOf[ByteArraySerializer].getName) - new KafkaProducer[Array[Byte], Array[Byte]](producerConfiguration) - } + producer = new KafkaProducer[Array[Byte], Array[Byte]](producerConfiguration) while (iterator.hasNext && failedWrite == null) { val currentRow = iterator.next() val projectedRow = projection(currentRow) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala index daa7f6a4fd08..64f7057b096d 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.kafka010 import java.util.concurrent.atomic.AtomicInteger +import org.apache.kafka.clients.producer.ProducerConfig +import org.apache.kafka.common.serialization.ByteArraySerializer import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkException @@ -98,6 +100,8 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext { .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("topic", topic) .save() + checkAnswer(createKafkaReader(topic).selectExpr("CAST(value as STRING) value"), + Row("1") :: Row("2") :: Row("3") :: Row("4") :: Row("5") :: Nil) } test("batch - null topic field value, and no topic option") { @@ -361,18 +365,18 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext { ex = intercept[IllegalArgumentException] { writer = createKafkaWriter( input.toDF(), - withOptions = Some(Map("kafka.key.deserializer" -> "foo")))() + withOptions = Some(Map("kafka.key.serializer" -> "foo")))() } assert(ex.getMessage.toLowerCase.contains( - "kafka option 'key.deserializer' is not supported")) + "kafka option 'key.serializer' is not supported")) ex = intercept[IllegalArgumentException] { writer = createKafkaWriter( input.toDF(), - withOptions = Some(Map("kafka.value.deserializer" -> "foo")))() + withOptions = Some(Map("kafka.value.serializer" -> "foo")))() } assert(ex.getMessage.toLowerCase.contains( - "kafka option 'value.deserializer' is not supported")) + "kafka option 'value.serializer' is not supported")) } test("generic - write big data with small producer buffer") { @@ -388,6 +392,8 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext { options.put("bootstrap.servers", testUtils.brokerAddress) options.put("buffer.memory", "16384") // min buffer size options.put("block.on.buffer.full", "true") + options.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, classOf[ByteArraySerializer].getName) + options.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, classOf[ByteArraySerializer].getName) val inputSchema = Seq(AttributeReference("value", BinaryType)()) val data = new Array[Byte](15000) // large value val writeTask = new KafkaWriteTask(options, inputSchema, Some(topic)) From 107e51306e0f234cd074cb2eecb9e30b51703f41 Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Mon, 6 Mar 2017 14:38:03 -0800 Subject: [PATCH 40/40] update --- .../spark/sql/kafka010/KafkaSinkSuite.scala | 83 ++++++++++--------- 1 file changed, 42 insertions(+), 41 deletions(-) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala index 64f7057b096d..490535623cb3 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala @@ -53,44 +53,6 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext { } } - private val topicId = new AtomicInteger(0) - - private def newTopic(): String = s"topic-${topicId.getAndIncrement()}" - - private def createKafkaReader(topic: String): DataFrame = { - spark.read - .format("kafka") - .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("startingOffsets", "earliest") - .option("endingOffsets", "latest") - .option("subscribe", topic) - .load() - } - - private def createKafkaWriter( - input: DataFrame, - withTopic: Option[String] = None, - withOutputMode: Option[OutputMode] = None, - withOptions: Option[Map[String, String]] = None) - (withSelectExpr: String*): StreamingQuery = { - var stream: DataStreamWriter[Row] = null - withTempDir { checkpointDir => - var df = input.toDF() - if (withSelectExpr.length > 0) { - df = df.selectExpr(withSelectExpr: _*) - } - stream = df.writeStream - .format("kafka") - .option("checkpointLocation", checkpointDir.getCanonicalPath) - .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .queryName("kafkaStream") - withTopic.foreach(stream.option("topic", _)) - withOutputMode.foreach(stream.outputMode(_)) - withOptions.map(_.foreach(opt => stream.option(opt._1, opt._2))) - } - stream.start() - } - test("batch - write to kafka") { val topic = newTopic() testUtils.createTopic(topic) @@ -100,7 +62,8 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext { .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("topic", topic) .save() - checkAnswer(createKafkaReader(topic).selectExpr("CAST(value as STRING) value"), + checkAnswer( + createKafkaReader(topic).selectExpr("CAST(value as STRING) value"), Row("1") :: Row("2") :: Row("3") :: Row("4") :: Row("5") :: Nil) } @@ -365,7 +328,7 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext { ex = intercept[IllegalArgumentException] { writer = createKafkaWriter( input.toDF(), - withOptions = Some(Map("kafka.key.serializer" -> "foo")))() + withOptions = Map("kafka.key.serializer" -> "foo"))() } assert(ex.getMessage.toLowerCase.contains( "kafka option 'key.serializer' is not supported")) @@ -373,7 +336,7 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext { ex = intercept[IllegalArgumentException] { writer = createKafkaWriter( input.toDF(), - withOptions = Some(Map("kafka.value.serializer" -> "foo")))() + withOptions = Map("kafka.value.serializer" -> "foo"))() } assert(ex.getMessage.toLowerCase.contains( "kafka option 'value.serializer' is not supported")) @@ -408,4 +371,42 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext { writeTask.close() } } + + private val topicId = new AtomicInteger(0) + + private def newTopic(): String = s"topic-${topicId.getAndIncrement()}" + + private def createKafkaReader(topic: String): DataFrame = { + spark.read + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("startingOffsets", "earliest") + .option("endingOffsets", "latest") + .option("subscribe", topic) + .load() + } + + private def createKafkaWriter( + input: DataFrame, + withTopic: Option[String] = None, + withOutputMode: Option[OutputMode] = None, + withOptions: Map[String, String] = Map[String, String]()) + (withSelectExpr: String*): StreamingQuery = { + var stream: DataStreamWriter[Row] = null + withTempDir { checkpointDir => + var df = input.toDF() + if (withSelectExpr.length > 0) { + df = df.selectExpr(withSelectExpr: _*) + } + stream = df.writeStream + .format("kafka") + .option("checkpointLocation", checkpointDir.getCanonicalPath) + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .queryName("kafkaStream") + withTopic.foreach(stream.option("topic", _)) + withOutputMode.foreach(stream.outputMode(_)) + withOptions.foreach(opt => stream.option(opt._1, opt._2)) + } + stream.start() + } }