From 63086902a78c41684c7e8e19fd995faa466f9d4d Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Thu, 21 Dec 2017 13:07:35 -0800 Subject: [PATCH 01/53] basic kafka --- .../main/scala/ContinuousKafkaReader.scala | 246 ++++++++++++++++++ .../sql/kafka010/CachedKafkaConsumer.scala | 4 + .../spark/sql/kafka010/KafkaSinkV2.scala | 141 ++++++++++ .../sql/kafka010/KafkaSourceOffset.scala | 9 +- .../sql/kafka010/KafkaSourceProvider.scala | 102 ++++++-- .../spark/sql/kafka010/KafkaWriter.scala | 5 +- .../sql/kafka010/ContinuousKafkaSuite.scala | 58 +++++ .../spark/sql/kafka010/KafkaSourceSuite.scala | 63 +++-- .../execution/streaming/StreamExecution.scala | 2 +- .../continuous/ContinuousExecution.scala | 14 +- .../spark/sql/streaming/StreamTest.scala | 31 ++- 11 files changed, 614 insertions(+), 61 deletions(-) create mode 100644 external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala create mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSinkV2.scala create mode 100644 external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/ContinuousKafkaSuite.scala diff --git a/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala b/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala new file mode 100644 index 0000000000000..db5fbb73b98d8 --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala @@ -0,0 +1,246 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.kafka010 + +import java.io._ +import java.nio.charset.StandardCharsets +import java.util.concurrent.atomic.AtomicBoolean + +import org.apache.commons.io.IOUtils +import org.apache.kafka.clients.consumer.ConsumerRecord +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.errors.WakeupException + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.{DataFrame, Row, SparkSession, SQLContext} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Literal, UnsafeProjection, UnsafeRow} +import org.apache.spark.sql.catalyst.expressions.codegen.{BufferHolder, UnsafeRowWriter} +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.execution.streaming.{HDFSMetadataLog, SerializedOffset} +import org.apache.spark.sql.kafka010.KafkaSource.{INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE, INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE, VERSION} +import org.apache.spark.sql.sources.v2.reader._ +import org.apache.spark.sql.types.{StructType} +import org.apache.spark.unsafe.types.UTF8String + +class ContinuousKafkaReader( + kafkaReader: KafkaOffsetReader, + executorKafkaParams: java.util.Map[String, Object], + sourceOptions: Map[String, String], + metadataPath: String, + initialOffsets: KafkaOffsetRangeLimit, + failOnDataLoss: Boolean) + extends ContinuousReader with SupportsScanUnsafeRow with Logging { + + override def mergeOffsets(offsets: Array[PartitionOffset]): Offset = { + val mergedMap = offsets.map { + case KafkaSourcePartitionOffset(p, o) => Map(p -> o) + }.reduce(_ ++ _) + KafkaSourceOffset(mergedMap) + } + + private lazy val session = SparkSession.getActiveSession.get + private lazy val sc = session.sparkContext + + private lazy val pollTimeoutMs = sourceOptions.getOrElse( + "kafkaConsumer.pollTimeoutMs", + sc.conf.getTimeAsMs("spark.network.timeout", "120s").toString + ).toLong + + private val maxOffsetsPerTrigger = + sourceOptions.get("maxOffsetsPerTrigger").map(_.toLong) + + /** + * Lazily initialize `initialPartitionOffsets` to make sure that `KafkaConsumer.poll` is only + * called in StreamExecutionThread. Otherwise, interrupting a thread while running + * `KafkaConsumer.poll` may hang forever (KAFKA-1894). + */ + private lazy val initialPartitionOffsets = { + val offsets = initialOffsets match { + case EarliestOffsetRangeLimit => KafkaSourceOffset(kafkaReader.fetchEarliestOffsets()) + case LatestOffsetRangeLimit => KafkaSourceOffset(kafkaReader.fetchLatestOffsets()) + case SpecificOffsetRangeLimit(p) => fetchAndVerify(p) + } + logInfo(s"Initial offsets: $offsets") + offsets.partitionToOffsets + } + + private def fetchAndVerify(specificOffsets: Map[TopicPartition, Long]) = { + val result = kafkaReader.fetchSpecificOffsets(specificOffsets) + specificOffsets.foreach { + 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)}") + } + case _ => + // no real way to check that beginning or end is reasonable + } + KafkaSourceOffset(result) + } + + // Initialized when creating read tasks. If this diverges from the partitions at the latest + // offsets, we need to reconfigure. + private var knownPartitions: Set[TopicPartition] = _ + + override def readSchema: StructType = KafkaOffsetReader.kafkaSchema + + private var offset: Offset = _ + override def setOffset(start: java.util.Optional[Offset]): Unit = { + offset = start.orElse { + val offsets = initialOffsets match { + case EarliestOffsetRangeLimit => KafkaSourceOffset(kafkaReader.fetchEarliestOffsets()) + case LatestOffsetRangeLimit => KafkaSourceOffset(kafkaReader.fetchLatestOffsets()) + case SpecificOffsetRangeLimit(p) => fetchAndVerify(p) + } + logInfo(s"Initial offsets: $offsets") + offsets + } + } + + override def getStartOffset(): Offset = offset + + override def deserializeOffset(json: String): Offset = { + KafkaSourceOffset(JsonUtils.partitionOffsets(json)) + } + + override def createUnsafeRowReadTasks(): java.util.List[ReadTask[UnsafeRow]] = { + import scala.collection.JavaConverters._ + + val oldStartOffsets = KafkaSourceOffset.getPartitionOffsets(offset) + + val newPartitions = + kafkaReader.fetchLatestOffsets().keySet.diff(oldStartOffsets.keySet) + val newPartitionOffsets = kafkaReader.fetchEarliestOffsets(newPartitions.toSeq) + val startOffsets = oldStartOffsets ++ newPartitionOffsets + + knownPartitions = startOffsets.keySet + + startOffsets.toSeq.map { + case (topicPartition, start) => + ContinuousKafkaReadTask( + topicPartition, start, executorKafkaParams, pollTimeoutMs, failOnDataLoss) + .asInstanceOf[ReadTask[UnsafeRow]] + }.asJava + } + + /** Stop this source and free any resources it has allocated. */ + def stop(): Unit = synchronized { + kafkaReader.close() + } + + override def commit(end: Offset): Unit = {} + + override def needsReconfiguration(): Boolean = { + knownPartitions != null && kafkaReader.fetchLatestOffsets().keySet != knownPartitions + } + + override def toString(): String = s"KafkaSource[$kafkaReader]" + + /** + * If `failOnDataLoss` is true, this method will throw an `IllegalStateException`. + * Otherwise, just log a warning. + */ + private def reportDataLoss(message: String): Unit = { + if (failOnDataLoss) { + throw new IllegalStateException(message + s". $INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE") + } else { + logWarning(message + s". $INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE") + } + } +} + +case class ContinuousKafkaReadTask( + topicPartition: TopicPartition, + start: Long, + kafkaParams: java.util.Map[String, Object], + pollTimeoutMs: Long, + failOnDataLoss: Boolean) + extends ReadTask[UnsafeRow] { + override def createDataReader(): ContinuousKafkaDataReader = { + new ContinuousKafkaDataReader(topicPartition, start, kafkaParams, pollTimeoutMs, failOnDataLoss) + } +} + +class ContinuousKafkaDataReader( + topicPartition: TopicPartition, + start: Long, + kafkaParams: java.util.Map[String, Object], + pollTimeoutMs: Long, + failOnDataLoss: Boolean) + extends ContinuousDataReader[UnsafeRow] { + private val topic = topicPartition.topic + private val kafkaPartition = topicPartition.partition + private val consumer = CachedKafkaConsumer.createUncached(topic, kafkaPartition, kafkaParams) + + private val closed = new AtomicBoolean(false) + + private var nextKafkaOffset = start match { + case s if s >= 0 => s + case KafkaOffsetRangeLimit.EARLIEST => consumer.getAvailableOffsetRange().earliest + case _ => throw new IllegalArgumentException(s"Invalid start Kafka offset $start.") + } + private var currentRecord: ConsumerRecord[Array[Byte], Array[Byte]] = _ + + override def next(): Boolean = { + try { + val r = consumer.get( + nextKafkaOffset, + untilOffset = Long.MaxValue, + pollTimeoutMs = Long.MaxValue, + failOnDataLoss) + nextKafkaOffset = r.offset + 1 + currentRecord = r + } catch { + case _: WakeupException if closed.get() => return false + } + true + } + + val sharedRow = new UnsafeRow(7) + val bufferHolder = new BufferHolder(sharedRow) + val rowWriter = new UnsafeRowWriter(bufferHolder, 7) + + override def get(): UnsafeRow = { + bufferHolder.reset() + + if (currentRecord.key == null) { + rowWriter.isNullAt(0) + } else { + rowWriter.write(0, currentRecord.key) + } + rowWriter.write(1, currentRecord.value) + rowWriter.write(2, UTF8String.fromString(currentRecord.topic)) + rowWriter.write(3, currentRecord.partition) + rowWriter.write(4, currentRecord.offset) + rowWriter.write(5, + DateTimeUtils.fromJavaTimestamp(new java.sql.Timestamp(currentRecord.timestamp))) + rowWriter.write(6, currentRecord.timestampType.id) + sharedRow.setTotalSize(bufferHolder.totalSize) + sharedRow + } + + override def getOffset(): KafkaSourcePartitionOffset = { + KafkaSourcePartitionOffset(topicPartition, nextKafkaOffset) + } + + override def close(): Unit = { + closed.set(true) + consumer.wakeup() + } +} 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 90ed7b1fba2f8..e251f0d7d804b 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 @@ -261,6 +261,10 @@ private[kafka010] case class CachedKafkaConsumer private( } } + def wakeup(): Unit = { + consumer.wakeup() + } + /** Create a new consumer and reset cached states */ private def resetConsumer(): Unit = { consumer.close() diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSinkV2.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSinkV2.scala new file mode 100644 index 0000000000000..b6188c0bf8275 --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSinkV2.scala @@ -0,0 +1,141 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.kafka010 + +import org.apache.kafka.clients.producer.{Callback, ProducerRecord, RecordMetadata} + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Literal, UnsafeProjection} +import org.apache.spark.sql.kafka010.KafkaSourceProvider.{kafkaParamsForProducer, TOPIC_OPTION_KEY} +import org.apache.spark.sql.sources.v2.{ContinuousWriteSupport, DataSourceV2, DataSourceV2Options} +import org.apache.spark.sql.sources.v2.writer._ +import org.apache.spark.sql.streaming.OutputMode +import org.apache.spark.sql.types.{BinaryType, StringType, StructType} + +class KafkaWriterV2(topic: Option[String], producerParams: Map[String, String], schema: StructType) + extends ContinuousWriter with SupportsWriteInternalRow { + + override def createInternalRowWriterFactory(): KafkaWriterFactory = + KafkaWriterFactory(topic, producerParams, schema) + + override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {} + override def abort(messages: Array[WriterCommitMessage]): Unit = {} +} + +case class KafkaWriterFactory( + topic: Option[String], producerParams: Map[String, String], schema: StructType) + extends DataWriterFactory[InternalRow] { + + override def createDataWriter(partitionId: Int, attemptNumber: Int): DataWriter[InternalRow] = { + new KafkaDataWriter(topic, producerParams, schema.toAttributes) + } +} + +case class KafkaWriterCommitMessage() extends WriterCommitMessage {} + +class KafkaDataWriter( + topic: Option[String], producerParams: Map[String, String], inputSchema: Seq[Attribute]) + extends DataWriter[InternalRow] { + import scala.collection.JavaConverters._ + + @volatile private var failedWrite: Exception = _ + private val projection = createProjection + private lazy val producer = CachedKafkaProducer.getOrCreate( + new java.util.HashMap[String, Object](producerParams.asJava)) + + private val callback = new Callback() { + override def onCompletion(recordMetadata: RecordMetadata, e: Exception): Unit = { + if (failedWrite == null && e != null) { + failedWrite = e + } + } + } + + def write(row: InternalRow): Unit = { + if (failedWrite != null) return + + val projectedRow = projection(row) + 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.") + } + val record = new ProducerRecord[Array[Byte], Array[Byte]](topic.toString, key, value) + producer.send(record, callback) + } + + def commit(): WriterCommitMessage = KafkaWriterCommitMessage() + def abort(): Unit = {} + + def close(): Unit = { + checkForErrors() + if (producer != null) { + producer.flush() + checkForErrors() + } + } + + 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} " + + "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("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(): Unit = { + if (failedWrite != null) { + throw failedWrite + } + } +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala index b5da415b3097e..5d7fecfe96fb7 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala @@ -19,7 +19,8 @@ package org.apache.spark.sql.kafka010 import org.apache.kafka.common.TopicPartition -import org.apache.spark.sql.execution.streaming.{Offset, SerializedOffset} +import org.apache.spark.sql.execution.streaming.{Offset => LegacyOffset, SerializedOffset} +import org.apache.spark.sql.sources.v2.reader.{Offset, PartitionOffset} /** * An [[Offset]] for the [[KafkaSource]]. This one tracks all partitions of subscribed topics and @@ -31,10 +32,14 @@ case class KafkaSourceOffset(partitionToOffsets: Map[TopicPartition, Long]) exte override val json = JsonUtils.partitionOffsets(partitionToOffsets) } +private[kafka010] +case class KafkaSourcePartitionOffset(topicPartition: TopicPartition, partitionOffset: Long) + extends PartitionOffset + /** Companion object of the [[KafkaSourceOffset]] */ private[kafka010] object KafkaSourceOffset { - def getPartitionOffsets(offset: Offset): Map[TopicPartition, Long] = { + def getPartitionOffsets(offset: LegacyOffset): Map[TopicPartition, Long] = { offset match { case o: KafkaSourceOffset => o.partitionToOffsets case so: SerializedOffset => KafkaSourceOffset(so).partitionToOffsets 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 3cb4d8cad12cc..acaf02eabbdb8 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 @@ -27,9 +27,11 @@ 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} -import org.apache.spark.sql.execution.streaming.{Sink, Source} +import org.apache.spark.sql.{AnalysisException, DataFrame, SaveMode, SparkSession, SQLContext} +import org.apache.spark.sql.execution.streaming.{Offset, Sink, Source} import org.apache.spark.sql.sources._ +import org.apache.spark.sql.sources.v2.{ContinuousReadSupport, ContinuousWriteSupport, DataSourceV2, DataSourceV2Options} +import org.apache.spark.sql.sources.v2.writer.ContinuousWriter import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.StructType @@ -43,6 +45,8 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister with StreamSinkProvider with RelationProvider with CreatableRelationProvider + with ContinuousWriteSupport + with ContinuousReadSupport with Logging { import KafkaSourceProvider._ @@ -101,6 +105,43 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister failOnDataLoss(caseInsensitiveParams)) } + override def createContinuousReader( + schema: java.util.Optional[StructType], + metadataPath: String, + options: DataSourceV2Options): ContinuousKafkaReader = { + val parameters = options.asMap().asScala.toMap + 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. + val uniqueGroupId = s"spark-kafka-source-${UUID.randomUUID}-${metadataPath.hashCode}" + + val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) } + val specifiedKafkaParams = + parameters + .keySet + .filter(_.toLowerCase(Locale.ROOT).startsWith("kafka.")) + .map { k => k.drop(6).toString -> parameters(k) } + .toMap + + val startingStreamOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit(caseInsensitiveParams, + STARTING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) + + val kafkaOffsetReader = new KafkaOffsetReader( + strategy(caseInsensitiveParams), + kafkaParamsForDriver(specifiedKafkaParams), + parameters, + driverGroupIdPrefix = s"$uniqueGroupId-driver") + + new ContinuousKafkaReader( + kafkaOffsetReader, + kafkaParamsForExecutors(specifiedKafkaParams, uniqueGroupId), + parameters, + metadataPath, + startingStreamOffsets, + failOnDataLoss(caseInsensitiveParams)) + } + /** * Returns a new base relation with the given parameters. * @@ -181,26 +222,22 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister } } - private def kafkaParamsForProducer(parameters: Map[String, String]): Map[String, String] = { - val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) } - if (caseInsensitiveParams.contains(s"kafka.${ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG}")) { - throw new IllegalArgumentException( - s"Kafka option '${ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG}' is not supported as keys " - + "are serialized with ByteArraySerializer.") - } + override def createContinuousWriter( + queryId: String, + schema: StructType, + mode: OutputMode, + options: DataSourceV2Options): java.util.Optional[ContinuousWriter] = { + import scala.collection.JavaConverters._ - if (caseInsensitiveParams.contains(s"kafka.${ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG}")) - { - throw new IllegalArgumentException( - s"Kafka option '${ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG}' is not supported as " - + "value are serialized with ByteArraySerializer.") - } - parameters - .keySet - .filter(_.toLowerCase(Locale.ROOT).startsWith("kafka.")) - .map { k => k.drop(6).toString -> parameters(k) } - .toMap + (ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG -> classOf[ByteArraySerializer].getName, - ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG -> classOf[ByteArraySerializer].getName) + val spark = SparkSession.getActiveSession.get + val topic = Option(options.get(TOPIC_OPTION_KEY).orElse(null)).map(_.trim) + // We convert the options argument from V2 -> Java map -> scala mutable -> scala immutable. + val producerParams = kafkaParamsForProducer(options.asMap.asScala.toMap) + + KafkaWriter.validateQuery( + schema.toAttributes, new java.util.HashMap[String, Object](producerParams.asJava), topic) + + java.util.Optional.of(new KafkaWriterV2(topic, producerParams, schema)) } private def strategy(caseInsensitiveParams: Map[String, String]) = @@ -450,4 +487,27 @@ private[kafka010] object KafkaSourceProvider extends Logging { def build(): ju.Map[String, Object] = map } + + private[kafka010] def kafkaParamsForProducer( + parameters: Map[String, String]): Map[String, String] = { + val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) } + if (caseInsensitiveParams.contains(s"kafka.${ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG}")) { + throw new IllegalArgumentException( + s"Kafka option '${ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG}' is not supported as keys " + + "are serialized with ByteArraySerializer.") + } + + if (caseInsensitiveParams.contains(s"kafka.${ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG}")) + { + throw new IllegalArgumentException( + s"Kafka option '${ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG}' is not supported as " + + "value are serialized with ByteArraySerializer.") + } + parameters + .keySet + .filter(_.toLowerCase(Locale.ROOT).startsWith("kafka.")) + .map { k => k.drop(6).toString -> parameters(k) } + .toMap + (ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG -> classOf[ByteArraySerializer].getName, + ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG -> classOf[ByteArraySerializer].getName) + } } 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 5e9ae35b3f008..15cd44812cb0c 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,10 +43,9 @@ private[kafka010] object KafkaWriter extends Logging { override def toString: String = "KafkaWriter" def validateQuery( - queryExecution: QueryExecution, + schema: Seq[Attribute], kafkaParameters: ju.Map[String, Object], topic: Option[String] = None): Unit = { - val schema = queryExecution.analyzed.output schema.find(_.name == TOPIC_ATTRIBUTE_NAME).getOrElse( if (topic.isEmpty) { throw new AnalysisException(s"topic option required when no " + @@ -84,7 +83,7 @@ private[kafka010] object KafkaWriter extends Logging { kafkaParameters: ju.Map[String, Object], topic: Option[String] = None): Unit = { val schema = queryExecution.analyzed.output - validateQuery(queryExecution, kafkaParameters, topic) + validateQuery(schema, kafkaParameters, topic) queryExecution.toRdd.foreachPartition { iter => val writeTask = new KafkaWriteTask(kafkaParameters, schema, topic) Utils.tryWithSafeFinally(block = writeTask.execute(iter))( diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/ContinuousKafkaSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/ContinuousKafkaSuite.scala new file mode 100644 index 0000000000000..04328a4c44c6a --- /dev/null +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/ContinuousKafkaSuite.scala @@ -0,0 +1,58 @@ +/* + * 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.sql.streaming.Trigger +import org.apache.spark.sql.test.TestSparkSession + +class ContinuousKafkaSuite extends KafkaSourceTest { + import testImplicits._ + + // We need more than the default local[2] to be able to schedule all partitions simultaneously. + override protected def createSparkSession = new TestSparkSession( + new SparkContext( + "local[10]", + "continuous-stream-test-sql-context", + sparkConf.set("spark.sql.testkey", "true"))) + + test("basic") { + val topic = newTopic() + testUtils.createTopic(topic, partitions = 5) + testUtils.sendMessages(topic, Array("0")) + require(testUtils.getLatestOffsets(Set(topic)).size === 5) + + val reader = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("subscribe", topic) + + val kafka = reader.load() + .selectExpr("CAST(value AS STRING)") + .as[String] + .map(_.toInt + 1) + + testStream(kafka, useV2Sink = true)( + StartStream(Trigger.Continuous(100)), + AddKafkaData(Set(topic), 1, 2, 3), + IncrementEpoch(), + CheckAnswer(2, 3, 4) + ) + } +} 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 2034b9be07f24..cfad77ab04f19 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 @@ -34,11 +34,14 @@ import org.scalatest.concurrent.PatienceConfiguration.Timeout import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkContext -import org.apache.spark.sql.ForeachWriter +import org.apache.spark.sql.{ForeachWriter, Row} +import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, WriteToDataSourceV2Exec} import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution +import org.apache.spark.sql.execution.streaming.sources.ContinuousMemoryWriter import org.apache.spark.sql.functions.{count, window} import org.apache.spark.sql.kafka010.KafkaSourceProvider._ -import org.apache.spark.sql.streaming.{ProcessingTime, StreamTest} +import org.apache.spark.sql.streaming.{ProcessingTime, StreamTest, Trigger} import org.apache.spark.sql.streaming.util.StreamManualClock import org.apache.spark.sql.test.{SharedSQLContext, TestSparkSession} import org.apache.spark.util.Utils @@ -82,7 +85,7 @@ abstract class KafkaSourceTest extends StreamTest with SharedSQLContext { message: String = "", topicAction: (String, Option[Int]) => Unit = (_, _) => {}) extends AddData { - override def addData(query: Option[StreamExecution]): (Source, Offset) = { + override def addData(query: Option[StreamExecution]): (BaseStreamingSource, Offset) = { if (query.get.isActive) { // Make sure no Spark job is running when deleting a topic query.get.processAllAvailable() @@ -104,8 +107,9 @@ abstract class KafkaSourceTest extends StreamTest with SharedSQLContext { "Cannot add data when there is no query for finding the active kafka source") val sources = query.get.logicalPlan.collect { - case StreamingExecutionRelation(source, _) if source.isInstanceOf[KafkaSource] => - source.asInstanceOf[KafkaSource] + case StreamingExecutionRelation(source: KafkaSource, _) => source + } ++ query.get.lastExecution.logical.collect { + case DataSourceV2Relation(_, reader: ContinuousKafkaReader) => reader } if (sources.isEmpty) { throw new Exception( @@ -137,6 +141,9 @@ abstract class KafkaSourceTest extends StreamTest with SharedSQLContext { override def toString: String = s"AddKafkaData(topics = $topics, data = $data, message = $message)" } + + private val topicId = new AtomicInteger(0) + protected def newTopic(): String = s"topic-${topicId.getAndIncrement()}" } @@ -144,8 +151,6 @@ class KafkaSourceSuite extends KafkaSourceTest { import testImplicits._ - private val topicId = new AtomicInteger(0) - testWithUninterruptibleThread( "deserialization of initial offset with Spark 2.1.0") { withTempDir { metadataPath => @@ -457,28 +462,48 @@ class KafkaSourceSuite extends KafkaSourceTest { ) } - test("starting offset is latest by default") { - val topic = newTopic() - testUtils.createTopic(topic, partitions = 5) - testUtils.sendMessages(topic, Array("0")) - require(testUtils.getLatestOffsets(Set(topic)).size === 5) + test("continuous with reconfigure") { + val topicPrefix = newTopic() + testUtils.createTopic(s"$topicPrefix-1", partitions = 3) + testUtils.sendMessages(s"$topicPrefix-1", Array("0")) + require(testUtils.getLatestOffsets(Set(s"$topicPrefix-1")).size === 3) val reader = spark .readStream .format("kafka") .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("subscribe", topic) + .option("subscribe", s"$topicPrefix-1") + .option("kafka.metadata.max.age.ms", "100") val kafka = reader.load() .selectExpr("CAST(value AS STRING)") .as[String] val mapped = kafka.map(_.toInt) + val query = mapped.writeStream + .format("memory") + .queryName("memory") + .trigger(Trigger.Continuous(1000)) + .start() + .asInstanceOf[ContinuousExecution] + query.awaitInitialization(streamingTimeout.toMillis) - testStream(mapped)( - makeSureGetOffsetCalled, - AddKafkaData(Set(topic), 1, 2, 3), - CheckAnswer(1, 2, 3) // should not have 0 - ) + Thread.sleep(1000) + + testUtils.addPartitions(s"$topicPrefix-1", 5) + + require(testUtils.getLatestOffsets(Set(s"$topicPrefix-1")).size === 5) + + print(testUtils.sendMessages(s"$topicPrefix-1", Seq(1, 2, 3).map{ _.toString }.toArray)) + Thread.sleep(5500) + + val sink = query.lastExecution.executedPlan.find(_.isInstanceOf[WriteToDataSourceV2Exec]).get + .asInstanceOf[WriteToDataSourceV2Exec].writer.asInstanceOf[ContinuousMemoryWriter] + .sink + try { + assert(sink.allData.sortBy(_.getInt(0)) == Seq(1, 2, 3).sorted.map(Row(_))) + } finally { + query.stop() + } } test("bad source options") { @@ -629,8 +654,6 @@ class KafkaSourceSuite extends KafkaSourceTest { } } - 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))) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 3e76bf7b7ca8f..736709706e23e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -418,7 +418,7 @@ abstract class StreamExecution( * Blocks the current thread until processing for data from the given `source` has reached at * least the given `Offset`. This method is intended for use primarily when writing tests. */ - private[sql] def awaitOffset(source: Source, newOffset: Offset): Unit = { + private[sql] def awaitOffset(source: BaseStreamingSource, newOffset: Offset): Unit = { assertAwaitThread() def notDone = { val localCommittedOffsets = committedOffsets diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala index 2843ab13bde2b..5f6ef2808ffc6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala @@ -278,12 +278,14 @@ class ContinuousExecution( // there's nothing meaningful to add to the offset log. } val globalOffset = reader.mergeOffsets(partitionOffsets.toArray) - synchronized { - if (queryExecutionThread.isAlive) { - offsetLog.add(epoch, OffsetSeq.fill(globalOffset)) - } else { - return - } + val oldOffset = synchronized { + offsetLog.add(epoch, OffsetSeq.fill(globalOffset)) + offsetLog.get(epoch - 1) + } + + // If offset hasn't changed since last epoch, there's been no new data. + if (oldOffset.contains(OffsetSeq.fill(globalOffset))) { + noNewData = true } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index fb9ebc81dd750..60e2c5505742a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -38,6 +38,7 @@ import org.apache.spark.sql.{Dataset, Encoder, QueryTest, Row} import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder, RowEncoder} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.continuous.{ContinuousExecution, EpochCoordinatorRef, IncrementAndGetEpoch} import org.apache.spark.sql.execution.streaming.sources.MemorySinkV2 @@ -105,7 +106,7 @@ trait StreamTest extends QueryTest with SharedSQLContext with TimeLimits with Be * the active query, and then return the source object the data was added, as well as the * offset of added data. */ - def addData(query: Option[StreamExecution]): (Source, Offset) + def addData(query: Option[StreamExecution]): (BaseStreamingSource, Offset) } /** A trait that can be extended when testing a source. */ @@ -403,12 +404,19 @@ trait StreamTest extends QueryTest with SharedSQLContext with TimeLimits with Be def fetchStreamAnswer(currentStream: StreamExecution, lastOnly: Boolean) = { verify(currentStream != null, "stream not running") // Get the map of source index to the current source objects - val indexToSource = currentStream - .logicalPlan - .collect { case StreamingExecutionRelation(s, _) => s } - .zipWithIndex - .map(_.swap) - .toMap + val indexToSource: Map[Int, BaseStreamingSource] = + currentStream + .logicalPlan + .collect { case StreamingExecutionRelation(s, _) => s } + .zipWithIndex + .map(_.swap) + .toMap ++ + currentStream.lastExecution + .logical + .collect { case DataSourceV2Relation(_, r: BaseStreamingSource) => r } + .zipWithIndex + .map(_.swap) + .toMap // Block until all data added has been processed for all the source awaiting.foreach { case (sourceIndex, offset) => @@ -599,7 +607,10 @@ trait StreamTest extends QueryTest with SharedSQLContext with TimeLimits with Be def findSourceIndex(plan: LogicalPlan): Option[Int] = { plan - .collect { case StreamingExecutionRelation(s, _) => s } + .collect { + case StreamingExecutionRelation(s, _) => s + case DataSourceV2Relation(_, r) => r + } .zipWithIndex .find(_._1 == source) .map(_._2) @@ -612,6 +623,10 @@ trait StreamTest extends QueryTest with SharedSQLContext with TimeLimits with Be findSourceIndex(query.logicalPlan) }.orElse { findSourceIndex(stream.logicalPlan) + }.orElse { + queryToUse.flatMap { q => + findSourceIndex(q.lastExecution.logical) + } }.getOrElse { throw new IllegalArgumentException( "Could find index of the source to which data was added") From 95ff1038b84de684a381be5a5c729476a24d3b3a Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Sun, 24 Dec 2017 13:40:20 -0800 Subject: [PATCH 02/53] move reader close to data reader thread in case reader isn't thread safe --- .../streaming/continuous/ContinuousDataSourceRDDIter.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousDataSourceRDDIter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousDataSourceRDDIter.scala index d79e4bd65f563..e700aa4f9aea7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousDataSourceRDDIter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousDataSourceRDDIter.scala @@ -77,7 +77,6 @@ class ContinuousDataSourceRDD( dataReaderThread.start() context.addTaskCompletionListener(_ => { - reader.close() dataReaderThread.interrupt() epochPollExecutor.shutdown() }) @@ -201,6 +200,8 @@ class DataReaderThread( failedFlag.set(true) // Don't rethrow the exception in this thread. It's not needed, and the default Spark // exception handler will kill the executor. + } finally { + reader.close() } } } From 599d001ec7ed5fa4ffe192e831f0a0727763b700 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Wed, 27 Dec 2017 12:43:16 -0800 Subject: [PATCH 03/53] test + small fixes --- .../main/scala/ContinuousKafkaReader.scala | 13 +- .../sql/kafka010/ContinuousKafkaSuite.scala | 157 +++++++++++++++++- .../execution/streaming/StreamExecution.scala | 11 +- .../continuous/ContinuousExecution.scala | 20 ++- .../spark/sql/streaming/StreamTest.scala | 22 +-- 5 files changed, 189 insertions(+), 34 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala b/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala index db5fbb73b98d8..4c682f8c78d49 100644 --- a/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala @@ -199,11 +199,14 @@ class ContinuousKafkaDataReader( override def next(): Boolean = { try { - val r = consumer.get( - nextKafkaOffset, - untilOffset = Long.MaxValue, - pollTimeoutMs = Long.MaxValue, - failOnDataLoss) + var r: ConsumerRecord[Array[Byte], Array[Byte]] = null + while (r == null) { + r = consumer.get( + nextKafkaOffset, + untilOffset = Long.MaxValue, + pollTimeoutMs = Long.MaxValue, + failOnDataLoss) + } nextKafkaOffset = r.offset + 1 currentRecord = r } catch { diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/ContinuousKafkaSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/ContinuousKafkaSuite.scala index 04328a4c44c6a..468edae35ab33 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/ContinuousKafkaSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/ContinuousKafkaSuite.scala @@ -17,7 +17,15 @@ package org.apache.spark.sql.kafka010 +import java.util.Properties + +import org.scalatest.time.SpanSugar._ +import scala.collection.mutable +import scala.util.Random + import org.apache.spark.SparkContext +import org.apache.spark.sql.ForeachWriter +import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution import org.apache.spark.sql.streaming.Trigger import org.apache.spark.sql.test.TestSparkSession @@ -51,8 +59,155 @@ class ContinuousKafkaSuite extends KafkaSourceTest { testStream(kafka, useV2Sink = true)( StartStream(Trigger.Continuous(100)), AddKafkaData(Set(topic), 1, 2, 3), - IncrementEpoch(), CheckAnswer(2, 3, 4) ) } + + test("add partitions") { + val topic = newTopic() + testUtils.createTopic(topic, partitions = 2) + testUtils.sendMessages(topic, Array("0")) + require(testUtils.getLatestOffsets(Set(topic)).size === 2) + + val reader = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("subscribe", topic) + .option("kafka.metadata.max.age.ms", "100") + + val kafka = reader.load() + .selectExpr("CAST(value AS STRING)") + .as[String] + .map(_.toInt + 1) + + testStream(kafka, useV2Sink = true)( + StartStream(Trigger.Continuous(1000)), + AddKafkaData(Set(topic), scala.Range(0, 5): _*), + CheckAnswer(1, 2, 3, 4, 5), + Execute(_ => testUtils.addPartitions(topic, 5)), + AddKafkaData(Set(topic), scala.Range(10, 20): _*), + CheckAnswer(1, 2, 3, 4, 5, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20) + ) + } +} + +class ContinuousKafkaStressSuite extends KafkaSourceTest { + import testImplicits._ + + // We need more than the default local[2] to be able to schedule all partitions simultaneously. + override protected def createSparkSession = new TestSparkSession( + new SparkContext( + "local[10]", + "continuous-stream-test-sql-context", + sparkConf.set("spark.sql.testkey", "true"))) + + override def beforeAll(): Unit = { + super.beforeAll() + testUtils = new KafkaTestUtils { + override def brokerConfiguration: Properties = { + val props = super.brokerConfiguration + // Try to make Kafka clean up messages as fast as possible. However, there is a hard-code + // 30 seconds delay (kafka.log.LogManager.InitialTaskDelayMs) so this test should run at + // least 30 seconds. + props.put("log.cleaner.backoff.ms", "100") + props.put("log.segment.bytes", "40") + props.put("log.retention.bytes", "40") + props.put("log.retention.check.interval.ms", "100") + props.put("delete.retention.ms", "10") + props.put("log.flush.scheduler.interval.ms", "10") + props + } + } + testUtils.setup() + } + + override def afterAll(): Unit = { + if (testUtils != null) { + testUtils.teardown() + testUtils = null + super.afterAll() + } + } + + test("stress test for failOnDataLoss=false") { + val reader = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + .option("subscribePattern", "failOnDataLoss.*") + .option("startingOffsets", "earliest") + .option("failOnDataLoss", "false") + .option("fetchOffset.retryIntervalMs", "3000") + val kafka = reader.load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + val query = kafka.map(kv => kv._2.toInt).writeStream.foreach(new ForeachWriter[Int] { + + override def open(partitionId: Long, version: Long): Boolean = { + true + } + + override def process(value: Int): Unit = { + // Slow down the processing speed so that messages may be aged out. + Thread.sleep(Random.nextInt(500)) + } + + override def close(errorOrNull: Throwable): Unit = { + } + }).start() + + val testTime = 1.minutes + val startTime = System.currentTimeMillis() + // Track the current existing topics + val topics = mutable.ArrayBuffer[String]() + // Track topics that have been deleted + val deletedTopics = mutable.Set[String]() + while (System.currentTimeMillis() - testTime.toMillis < startTime) { + Random.nextInt(10) match { + case 0 => // Create a new topic + val topic = newTopic() + topics += topic + // As pushing messages into Kafka updates Zookeeper asynchronously, there is a small + // chance that a topic will be recreated after deletion due to the asynchronous update. + // Hence, always overwrite to handle this race condition. + testUtils.createTopic(topic, partitions = 1, overwrite = true) + logInfo(s"Create topic $topic") + case 1 if topics.nonEmpty => // Delete an existing topic + val topic = topics.remove(Random.nextInt(topics.size)) + testUtils.deleteTopic(topic) + logInfo(s"Delete topic $topic") + deletedTopics += topic + case 2 if deletedTopics.nonEmpty => // Recreate a topic that was deleted. + val topic = deletedTopics.toSeq(Random.nextInt(deletedTopics.size)) + deletedTopics -= topic + topics += topic + // As pushing messages into Kafka updates Zookeeper asynchronously, there is a small + // chance that a topic will be recreated after deletion due to the asynchronous update. + // Hence, always overwrite to handle this race condition. + testUtils.createTopic(topic, partitions = 1, overwrite = true) + logInfo(s"Create topic $topic") + case 3 => + Thread.sleep(1000) + case _ => // Push random messages + for (topic <- topics) { + val size = Random.nextInt(10) + for (_ <- 0 until size) { + testUtils.sendMessages(topic, Array(Random.nextInt(10).toString)) + } + } + } + // `failOnDataLoss` is `false`, we should not fail the query + if (query.exception.nonEmpty) { + throw query.exception.get + } + } + + query.stop() + // `failOnDataLoss` is `false`, we should not fail the query + if (query.exception.nonEmpty) { + throw query.exception.get + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 736709706e23e..8035d97e16e3e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -418,11 +418,16 @@ abstract class StreamExecution( * Blocks the current thread until processing for data from the given `source` has reached at * least the given `Offset`. This method is intended for use primarily when writing tests. */ - private[sql] def awaitOffset(source: BaseStreamingSource, newOffset: Offset): Unit = { + private[sql] def awaitOffset(sourceIndex: Int, newOffset: Offset): Unit = { assertAwaitThread() def notDone = { val localCommittedOffsets = committedOffsets - !localCommittedOffsets.contains(source) || localCommittedOffsets(source) != newOffset + if (sources.length <= sourceIndex) { + false + } else { + val source = sources(sourceIndex) + !localCommittedOffsets.contains(source) || localCommittedOffsets(source) != newOffset + } } while (notDone) { @@ -436,7 +441,7 @@ abstract class StreamExecution( awaitProgressLock.unlock() } } - logDebug(s"Unblocked at $newOffset for $source") + logDebug(s"Unblocked at $newOffset for ${sources(sourceIndex)}") } /** A flag to indicate that a batch has completed with no new data available. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala index 5f6ef2808ffc6..7034ecfb820bd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.streaming.continuous import java.util.concurrent.TimeUnit +import java.util.function.UnaryOperator import scala.collection.JavaConverters._ import scala.collection.mutable.{ArrayBuffer, Map => MutableMap} @@ -78,15 +79,17 @@ class ContinuousExecution( } override protected def runActivatedStream(sparkSessionForStream: SparkSession): Unit = { - do { - try { - runContinuous(sparkSessionForStream) - } catch { - case _: InterruptedException if state.get().equals(RECONFIGURING) => - // swallow exception and run again - state.set(ACTIVE) + val stateUpdate = new UnaryOperator[State] { + override def apply(s: State) = s match { + // If we ended the query to reconfigure, reset the state to active. + case RECONFIGURING => ACTIVE + case _ => s } - } while (state.get() == ACTIVE) + } + + do { + runContinuous(sparkSessionForStream) + } while (state.updateAndGet(stateUpdate) == ACTIVE) } /** @@ -231,7 +234,6 @@ class ContinuousExecution( if (queryExecutionThread.isAlive) { sparkSession.sparkContext.cancelJobGroup(runId.toString) queryExecutionThread.interrupt() - // No need to join - this thread is about to end anyway. } false } else if (isActive) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index 60e2c5505742a..9f0783d170870 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -40,7 +40,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.execution.streaming._ -import org.apache.spark.sql.execution.streaming.continuous.{ContinuousExecution, EpochCoordinatorRef, IncrementAndGetEpoch} +import org.apache.spark.sql.execution.streaming.continuous.{ContinuousExecution, ContinuousTrigger, EpochCoordinatorRef, IncrementAndGetEpoch} import org.apache.spark.sql.execution.streaming.sources.MemorySinkV2 import org.apache.spark.sql.execution.streaming.state.StateStore import org.apache.spark.sql.streaming.StreamingQueryListener._ @@ -403,25 +403,11 @@ trait StreamTest extends QueryTest with SharedSQLContext with TimeLimits with Be def fetchStreamAnswer(currentStream: StreamExecution, lastOnly: Boolean) = { verify(currentStream != null, "stream not running") - // Get the map of source index to the current source objects - val indexToSource: Map[Int, BaseStreamingSource] = - currentStream - .logicalPlan - .collect { case StreamingExecutionRelation(s, _) => s } - .zipWithIndex - .map(_.swap) - .toMap ++ - currentStream.lastExecution - .logical - .collect { case DataSourceV2Relation(_, r: BaseStreamingSource) => r } - .zipWithIndex - .map(_.swap) - .toMap // Block until all data added has been processed for all the source awaiting.foreach { case (sourceIndex, offset) => failAfter(streamingTimeout) { - currentStream.awaitOffset(indexToSource(sourceIndex), offset) + currentStream.awaitOffset(sourceIndex, offset) } } @@ -480,6 +466,10 @@ trait StreamTest extends QueryTest with SharedSQLContext with TimeLimits with Be // after starting the query. try { currentStream.awaitInitialization(streamingTimeout.toMillis) + currentStream match { + case s: ContinuousExecution => s.awaitEpoch(0) + case _ => + } } catch { case _: StreamingQueryException => // Ignore the exception. `StopStream` or `ExpectFailure` will catch it as well. From 88b261d6b3d61b03ffb07bb8d1ff2312d9aa818e Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Wed, 27 Dec 2017 13:21:56 -0800 Subject: [PATCH 04/53] fixes lost in cherrypick --- .../main/scala/ContinuousKafkaReader.scala | 25 ++++++++----------- .../sql/kafka010/ContinuousKafkaSuite.scala | 23 ++++++++++++++++- .../continuous/ContinuousExecution.scala | 3 +-- 3 files changed, 33 insertions(+), 18 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala b/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala index 4c682f8c78d49..b09f2ce3e2dca 100644 --- a/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala @@ -198,20 +198,16 @@ class ContinuousKafkaDataReader( private var currentRecord: ConsumerRecord[Array[Byte], Array[Byte]] = _ override def next(): Boolean = { - try { - var r: ConsumerRecord[Array[Byte], Array[Byte]] = null - while (r == null) { - r = consumer.get( - nextKafkaOffset, - untilOffset = Long.MaxValue, - pollTimeoutMs = Long.MaxValue, - failOnDataLoss) - } - nextKafkaOffset = r.offset + 1 - currentRecord = r - } catch { - case _: WakeupException if closed.get() => return false + var r: ConsumerRecord[Array[Byte], Array[Byte]] = null + while (r == null) { + r = consumer.get( + nextKafkaOffset, + untilOffset = Long.MaxValue, + pollTimeoutMs = Long.MaxValue, + failOnDataLoss) } + nextKafkaOffset = r.offset + 1 + currentRecord = r true } @@ -243,7 +239,6 @@ class ContinuousKafkaDataReader( } override def close(): Unit = { - closed.set(true) - consumer.wakeup() + consumer.close() } } diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/ContinuousKafkaSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/ContinuousKafkaSuite.scala index 468edae35ab33..944734d441e5f 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/ContinuousKafkaSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/ContinuousKafkaSuite.scala @@ -24,7 +24,8 @@ import scala.collection.mutable import scala.util.Random import org.apache.spark.SparkContext -import org.apache.spark.sql.ForeachWriter +import org.apache.spark.sql.{ForeachWriter, Row} +import org.apache.spark.sql.execution.streaming.StreamingQueryWrapper import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution import org.apache.spark.sql.streaming.Trigger import org.apache.spark.sql.test.TestSparkSession @@ -90,6 +91,26 @@ class ContinuousKafkaSuite extends KafkaSourceTest { CheckAnswer(1, 2, 3, 4, 5, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20) ) } + + test("kafka sink") { + val query = spark.readStream + .format("rate") + .option("numPartitions", "6") + .option("rowsPerSecond", "10") + .load() + .select('value) + .writeStream + .format("memory") + .queryName("kafkaSink") + .trigger(Trigger.Continuous(100)) + .start() + + eventually(timeout(streamingTimeout)) { + val results = spark.read.table("kafkaSink").collect() + assert(Range(0, 20).map(Row(_)).toSet.subsetOf(results.toSet)) + } + query.stop() + } } class ContinuousKafkaStressSuite extends KafkaSourceTest { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala index 7034ecfb820bd..14fd52bf4f105 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala @@ -228,8 +228,7 @@ class ContinuousExecution( triggerExecutor.execute(() => { startTrigger() - if (reader.needsReconfiguration()) { - state.set(RECONFIGURING) + if (reader.needsReconfiguration() && state.compareAndSet(ACTIVE, RECONFIGURING)) { stopSources() if (queryExecutionThread.isAlive) { sparkSession.sparkContext.cancelJobGroup(runId.toString) From 1ff378b9e0e9afe95bbc9f576e250e1aa08a826b Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Wed, 27 Dec 2017 16:39:56 -0800 Subject: [PATCH 05/53] fix sink test to use sink --- .../sql/kafka010/ContinuousKafkaSuite.scala | 48 ++++++++++++------- .../apache/spark/sql/DataFrameReader.scala | 29 ++++++----- .../sql/streaming/DataStreamWriter.scala | 26 +++++++--- 3 files changed, 69 insertions(+), 34 deletions(-) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/ContinuousKafkaSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/ContinuousKafkaSuite.scala index 944734d441e5f..bbfae2dd0bbce 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/ContinuousKafkaSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/ContinuousKafkaSuite.scala @@ -93,23 +93,39 @@ class ContinuousKafkaSuite extends KafkaSourceTest { } test("kafka sink") { - val query = spark.readStream - .format("rate") - .option("numPartitions", "6") - .option("rowsPerSecond", "10") - .load() - .select('value) - .writeStream - .format("memory") - .queryName("kafkaSink") - .trigger(Trigger.Continuous(100)) - .start() - - eventually(timeout(streamingTimeout)) { - val results = spark.read.table("kafkaSink").collect() - assert(Range(0, 20).map(Row(_)).toSet.subsetOf(results.toSet)) + withTempDir { dir => + val topic = newTopic() + testUtils.createTopic(topic) + val query = spark.readStream + .format("rate") + .option("numPartitions", "6") + .option("rowsPerSecond", "10") + .load() + .select('value) + .selectExpr("CAST(value as STRING) value") + .writeStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("topic", topic) + .option("checkpointLocation", dir.getCanonicalPath) + .trigger(Trigger.Continuous(100)) + .start() + + eventually(timeout(streamingTimeout)) { + val results = spark.read + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("startingOffsets", "earliest") + .option("endingOffsets", "latest") + .option("subscribe", topic) + .load() + .selectExpr("CAST(value as STRING) value") + .selectExpr("CAST(value as INT) value") + .collect() + assert(Range(0, 20).map(Row(_)).toSet.subsetOf(results.toSet)) + } + query.stop() } - query.stop() } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index c43ee91294a27..280c043581a1c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -208,23 +208,30 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { } reader - case _ => - throw new AnalysisException(s"$cls does not support data reading.") + case _ => null // fall back to v1 } - Dataset.ofRows(sparkSession, DataSourceV2Relation(reader)) + if (reader == null) { + loadV1Source(paths: _*) + } else { + Dataset.ofRows(sparkSession, DataSourceV2Relation(reader)) + } } else { - // Code path for data source v1. - sparkSession.baseRelationToDataFrame( - DataSource.apply( - sparkSession, - paths = paths, - userSpecifiedSchema = userSpecifiedSchema, - className = source, - options = extraOptions.toMap).resolveRelation()) + loadV1Source(paths: _*) } } + private def loadV1Source(paths: String*) = { + // Code path for data source v1. + sparkSession.baseRelationToDataFrame( + DataSource.apply( + sparkSession, + paths = paths, + userSpecifiedSchema = userSpecifiedSchema, + className = source, + options = extraOptions.toMap).resolveRelation()) + } + /** * Construct a `DataFrame` representing the database table accessible via JDBC URL * url named table and connection properties. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala index db588ae282f38..58fc18de07f49 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.continuous.ContinuousTrigger import org.apache.spark.sql.execution.streaming.sources.{MemoryPlanV2, MemorySinkV2} +import org.apache.spark.sql.sources.v2.ContinuousWriteSupport /** * Interface used to write a streaming `Dataset` to external storage systems (e.g. file systems, @@ -279,18 +280,29 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { useTempCheckpointLocation = true, trigger = trigger) } else { - val dataSource = - DataSource( - df.sparkSession, - className = source, - options = extraOptions.toMap, - partitionColumns = normalizedParCols.getOrElse(Nil)) + val sink = trigger match { + case _: ContinuousTrigger => + val ds = DataSource.lookupDataSource(source, df.sparkSession.sessionState.conf) + ds.newInstance() match { + case w: ContinuousWriteSupport => w + case _ => throw new AnalysisException( + s"Data source $source does not support continuous writing") + } + case _ => + val ds = DataSource( + df.sparkSession, + className = source, + options = extraOptions.toMap, + partitionColumns = normalizedParCols.getOrElse(Nil)) + ds.createSink(outputMode) + } + df.sparkSession.sessionState.streamingQueryManager.startQuery( extraOptions.get("queryName"), extraOptions.get("checkpointLocation"), df, extraOptions.toMap, - dataSource.createSink(outputMode), + sink, outputMode, useTempCheckpointLocation = source == "console", recoverFromCheckpointLocation = true, From cd778ce42142f5e17e51f21358071b322d7d760d Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Wed, 27 Dec 2017 16:47:13 -0800 Subject: [PATCH 06/53] add SharedSQLContext to avoid multiple context error in jenkins --- .../apache/spark/sql/kafka010/ContinuousKafkaSuite.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/ContinuousKafkaSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/ContinuousKafkaSuite.scala index bbfae2dd0bbce..f27a0838eade4 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/ContinuousKafkaSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/ContinuousKafkaSuite.scala @@ -28,9 +28,9 @@ import org.apache.spark.sql.{ForeachWriter, Row} import org.apache.spark.sql.execution.streaming.StreamingQueryWrapper import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution import org.apache.spark.sql.streaming.Trigger -import org.apache.spark.sql.test.TestSparkSession +import org.apache.spark.sql.test.{SharedSQLContext, TestSparkSession} -class ContinuousKafkaSuite extends KafkaSourceTest { +class ContinuousKafkaSuite extends KafkaSourceTest with SharedSQLContext { import testImplicits._ // We need more than the default local[2] to be able to schedule all partitions simultaneously. @@ -129,7 +129,7 @@ class ContinuousKafkaSuite extends KafkaSourceTest { } } -class ContinuousKafkaStressSuite extends KafkaSourceTest { +class ContinuousKafkaStressSuite extends KafkaSourceTest with SharedSQLContext { import testImplicits._ // We need more than the default local[2] to be able to schedule all partitions simultaneously. From de98a8b6b5688eb6ddefeb028bbfe580220f214c Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Wed, 27 Dec 2017 19:57:01 -0800 Subject: [PATCH 07/53] await termination so SparkContext doesn't leak in jenkins --- .../org/apache/spark/sql/kafka010/ContinuousKafkaSuite.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/ContinuousKafkaSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/ContinuousKafkaSuite.scala index f27a0838eade4..1a4ddc58fba2d 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/ContinuousKafkaSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/ContinuousKafkaSuite.scala @@ -242,6 +242,7 @@ class ContinuousKafkaStressSuite extends KafkaSourceTest with SharedSQLContext { } query.stop() + query.awaitTermination() // `failOnDataLoss` is `false`, we should not fail the query if (query.exception.nonEmpty) { throw query.exception.get From db2dc93df7632cfc8e093279fed11b4eb7681a15 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Thu, 28 Dec 2017 11:39:56 -0800 Subject: [PATCH 08/53] fix after rebase --- .../main/scala/ContinuousKafkaReader.scala | 3 +- .../spark/sql/kafka010/KafkaSinkV2.scala | 5 +- .../sql/kafka010/KafkaSourceOffset.scala | 2 +- .../sql/kafka010/KafkaSourceProvider.scala | 7 +-- .../spark/sql/kafka010/KafkaSourceSuite.scala | 53 +++---------------- .../apache/spark/sql/DataFrameWriter.scala | 22 ++++---- .../sql/streaming/DataStreamWriter.scala | 2 +- 7 files changed, 30 insertions(+), 64 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala b/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala index b09f2ce3e2dca..92a9025a1e426 100644 --- a/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala @@ -34,7 +34,8 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.streaming.{HDFSMetadataLog, SerializedOffset} import org.apache.spark.sql.kafka010.KafkaSource.{INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE, INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE, VERSION} import org.apache.spark.sql.sources.v2.reader._ -import org.apache.spark.sql.types.{StructType} +import org.apache.spark.sql.sources.v2.streaming.reader.{ContinuousDataReader, ContinuousReader, Offset, PartitionOffset} +import org.apache.spark.sql.types.StructType import org.apache.spark.unsafe.types.UTF8String class ContinuousKafkaReader( diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSinkV2.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSinkV2.scala index b6188c0bf8275..fe0a1c1dfa828 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSinkV2.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSinkV2.scala @@ -24,12 +24,13 @@ import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Literal, UnsafeProjection} import org.apache.spark.sql.kafka010.KafkaSourceProvider.{kafkaParamsForProducer, TOPIC_OPTION_KEY} -import org.apache.spark.sql.sources.v2.{ContinuousWriteSupport, DataSourceV2, DataSourceV2Options} +import org.apache.spark.sql.sources.v2.streaming.writer.ContinuousWriter import org.apache.spark.sql.sources.v2.writer._ import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.{BinaryType, StringType, StructType} -class KafkaWriterV2(topic: Option[String], producerParams: Map[String, String], schema: StructType) +class ContinuousKafkaWriter( + topic: Option[String], producerParams: Map[String, String], schema: StructType) extends ContinuousWriter with SupportsWriteInternalRow { override def createInternalRowWriterFactory(): KafkaWriterFactory = diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala index 5d7fecfe96fb7..d157249c70a72 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.kafka010 import org.apache.kafka.common.TopicPartition import org.apache.spark.sql.execution.streaming.{Offset => LegacyOffset, SerializedOffset} -import org.apache.spark.sql.sources.v2.reader.{Offset, PartitionOffset} +import org.apache.spark.sql.sources.v2.streaming.reader.{Offset, PartitionOffset} /** * An [[Offset]] for the [[KafkaSource]]. This one tracks all partitions of subscribed topics and 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 acaf02eabbdb8..a51d5104edc87 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 @@ -30,8 +30,9 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.{AnalysisException, DataFrame, SaveMode, SparkSession, SQLContext} import org.apache.spark.sql.execution.streaming.{Offset, Sink, Source} import org.apache.spark.sql.sources._ -import org.apache.spark.sql.sources.v2.{ContinuousReadSupport, ContinuousWriteSupport, DataSourceV2, DataSourceV2Options} -import org.apache.spark.sql.sources.v2.writer.ContinuousWriter +import org.apache.spark.sql.sources.v2.{DataSourceV2, DataSourceV2Options} +import org.apache.spark.sql.sources.v2.streaming.{ContinuousReadSupport, ContinuousWriteSupport} +import org.apache.spark.sql.sources.v2.streaming.writer.ContinuousWriter import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.StructType @@ -237,7 +238,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister KafkaWriter.validateQuery( schema.toAttributes, new java.util.HashMap[String, Object](producerParams.asJava), topic) - java.util.Optional.of(new KafkaWriterV2(topic, producerParams, schema)) + java.util.Optional.of(new ContinuousKafkaWriter(topic, producerParams, schema)) } private def strategy(caseInsensitiveParams: Map[String, String]) = diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala index cfad77ab04f19..5512979dcc715 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 @@ -108,9 +108,12 @@ abstract class KafkaSourceTest extends StreamTest with SharedSQLContext { val sources = query.get.logicalPlan.collect { case StreamingExecutionRelation(source: KafkaSource, _) => source - } ++ query.get.lastExecution.logical.collect { - case DataSourceV2Relation(_, reader: ContinuousKafkaReader) => reader - } + } ++ (query.get.lastExecution match { + case null => Seq() + case e => e.logical.collect { + case DataSourceV2Relation(_, reader: ContinuousKafkaReader) => reader + } + }) if (sources.isEmpty) { throw new Exception( "Could not find Kafka source in the StreamExecution logical plan to add data to") @@ -462,50 +465,6 @@ class KafkaSourceSuite extends KafkaSourceTest { ) } - test("continuous with reconfigure") { - val topicPrefix = newTopic() - testUtils.createTopic(s"$topicPrefix-1", partitions = 3) - testUtils.sendMessages(s"$topicPrefix-1", Array("0")) - require(testUtils.getLatestOffsets(Set(s"$topicPrefix-1")).size === 3) - - val reader = spark - .readStream - .format("kafka") - .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("subscribe", s"$topicPrefix-1") - .option("kafka.metadata.max.age.ms", "100") - - val kafka = reader.load() - .selectExpr("CAST(value AS STRING)") - .as[String] - val mapped = kafka.map(_.toInt) - val query = mapped.writeStream - .format("memory") - .queryName("memory") - .trigger(Trigger.Continuous(1000)) - .start() - .asInstanceOf[ContinuousExecution] - query.awaitInitialization(streamingTimeout.toMillis) - - Thread.sleep(1000) - - testUtils.addPartitions(s"$topicPrefix-1", 5) - - require(testUtils.getLatestOffsets(Set(s"$topicPrefix-1")).size === 5) - - print(testUtils.sendMessages(s"$topicPrefix-1", Seq(1, 2, 3).map{ _.toString }.toArray)) - Thread.sleep(5500) - - val sink = query.lastExecution.executedPlan.find(_.isInstanceOf[WriteToDataSourceV2Exec]).get - .asInstanceOf[WriteToDataSourceV2Exec].writer.asInstanceOf[ContinuousMemoryWriter] - .sink - try { - assert(sink.allData.sortBy(_.getInt(0)) == Seq(1, 2, 3).sorted.map(Row(_))) - } finally { - query.stop() - } - } - test("bad source options") { def testBadOptions(options: (String, String)*)(expectedMsgs: String*): Unit = { val ex = intercept[IllegalArgumentException] { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 7ccda0ad36d13..8c63cab89aa11 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -255,17 +255,21 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { } } - case _ => throw new AnalysisException(s"$cls does not support data writing.") + case _ => saveToV1Source() } } else { - // Code path for data source v1. - runCommand(df.sparkSession, "save") { - DataSource( - sparkSession = df.sparkSession, - className = source, - partitionColumns = partitioningColumns.getOrElse(Nil), - options = extraOptions.toMap).planForWriting(mode, df.logicalPlan) - } + saveToV1Source() + } + } + + private def saveToV1Source(): Unit = { + // Code path for data source v1. + runCommand(df.sparkSession, "save") { + DataSource( + sparkSession = df.sparkSession, + className = source, + partitionColumns = partitioningColumns.getOrElse(Nil), + options = extraOptions.toMap).planForWriting(mode, df.logicalPlan) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala index 58fc18de07f49..b5b4a05ab4973 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.continuous.ContinuousTrigger import org.apache.spark.sql.execution.streaming.sources.{MemoryPlanV2, MemorySinkV2} -import org.apache.spark.sql.sources.v2.ContinuousWriteSupport +import org.apache.spark.sql.sources.v2.streaming.ContinuousWriteSupport /** * Interface used to write a streaming `Dataset` to external storage systems (e.g. file systems, From df194c69329248f25dd2069999817c286efcf625 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Fri, 29 Dec 2017 16:56:32 -0800 Subject: [PATCH 09/53] fix test framework race condition --- .../main/scala/ContinuousKafkaReader.scala | 3 ++- .../sql/kafka010/ContinuousKafkaSuite.scala | 19 ++++++++++++++++--- 2 files changed, 18 insertions(+), 4 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala b/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala index 92a9025a1e426..e3d4bd4f55fac 100644 --- a/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala @@ -97,7 +97,8 @@ class ContinuousKafkaReader( // Initialized when creating read tasks. If this diverges from the partitions at the latest // offsets, we need to reconfigure. - private var knownPartitions: Set[TopicPartition] = _ + // Exposed outside this object only for unit tests. + private[sql] var knownPartitions: Set[TopicPartition] = _ override def readSchema: StructType = KafkaOffsetReader.kafkaSchema diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/ContinuousKafkaSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/ContinuousKafkaSuite.scala index 1a4ddc58fba2d..4bf5e0c79c20c 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/ContinuousKafkaSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/ContinuousKafkaSuite.scala @@ -25,7 +25,8 @@ import scala.util.Random import org.apache.spark.SparkContext import org.apache.spark.sql.{ForeachWriter, Row} -import org.apache.spark.sql.execution.streaming.StreamingQueryWrapper +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import org.apache.spark.sql.execution.streaming.{ContinuousExecutionRelation, StreamingExecutionRelation, StreamingQueryWrapper} import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution import org.apache.spark.sql.streaming.Trigger import org.apache.spark.sql.test.{SharedSQLContext, TestSparkSession} @@ -75,7 +76,7 @@ class ContinuousKafkaSuite extends KafkaSourceTest with SharedSQLContext { .format("kafka") .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("subscribe", topic) - .option("kafka.metadata.max.age.ms", "100") + .option("kafka.metadata.max.age.ms", "1") val kafka = reader.load() .selectExpr("CAST(value AS STRING)") @@ -83,10 +84,22 @@ class ContinuousKafkaSuite extends KafkaSourceTest with SharedSQLContext { .map(_.toInt + 1) testStream(kafka, useV2Sink = true)( - StartStream(Trigger.Continuous(1000)), + StartStream(Trigger.Continuous(100)), AddKafkaData(Set(topic), scala.Range(0, 5): _*), CheckAnswer(1, 2, 3, 4, 5), Execute(_ => testUtils.addPartitions(topic, 5)), + Execute { q => + // We have to assert on this first to avoid a race condition in the test framework. + // AddKafkaData will not work if the query is in the middle of reconfiguring, because it + // assumes it can find the right reader in the plan. + eventually(timeout(streamingTimeout)) { + assert( + q.lastExecution.logical.collectFirst { + case DataSourceV2Relation(_, r: ContinuousKafkaReader) => r + }.exists(_.knownPartitions.size == 5), + "query never reconfigured to 5 partitions") + } + }, AddKafkaData(Set(topic), scala.Range(10, 20): _*), CheckAnswer(1, 2, 3, 4, 5, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20) ) From dae3a09e48565439ed8c22dda857a7747e518b3b Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Thu, 4 Jan 2018 13:40:28 -0800 Subject: [PATCH 10/53] fix failure semantics in continuous kafka writer --- .../org/apache/spark/sql/kafka010/KafkaSinkV2.scala | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSinkV2.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSinkV2.scala index fe0a1c1dfa828..b330ba6d611d1 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSinkV2.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSinkV2.scala @@ -70,8 +70,6 @@ class KafkaDataWriter( } def write(row: InternalRow): Unit = { - if (failedWrite != null) return - val projectedRow = projection(row) val topic = projectedRow.getUTF8String(0) val key = projectedRow.getBinary(1) @@ -85,7 +83,13 @@ class KafkaDataWriter( producer.send(record, callback) } - def commit(): WriterCommitMessage = KafkaWriterCommitMessage() + def commit(): WriterCommitMessage = { + // Send is asynchronous, but we can't commit until all rows are actually in Kafka. + // This requires flushing and then checking that no callbacks produced errors. + producer.flush() + checkForErrors() + KafkaWriterCommitMessage() + } def abort(): Unit = {} def close(): Unit = { From 25748188d88a22022cd0e66bf2b405105ac8b36e Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Thu, 4 Jan 2018 17:19:42 -0800 Subject: [PATCH 11/53] dedup fetchAndVerify --- .../main/scala/ContinuousKafkaReader.scala | 82 +++++++------------ .../sql/kafka010/KafkaOffsetReader.scala | 21 ++++- .../spark/sql/kafka010/KafkaSource.scala | 17 +--- .../sql/kafka010/KafkaSourceOffset.scala | 4 +- 4 files changed, 53 insertions(+), 71 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala b/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala index e3d4bd4f55fac..24d1963f6d551 100644 --- a/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala @@ -38,22 +38,28 @@ import org.apache.spark.sql.sources.v2.streaming.reader.{ContinuousDataReader, C import org.apache.spark.sql.types.StructType import org.apache.spark.unsafe.types.UTF8String +/** + * A [[ContinuousReader]] for data from kafka. + * + * @param offsetReader a reader used to get kafka offsets. Note that the actual data will be + * read by per-task consumers generated later. + * @param kafkaParams String params for per-task Kafka consumers. + * @param sourceOptions The [[org.apache.spark.sql.sources.v2.DataSourceV2Options]] params which + * are not Kafka consumer params. + * @param metadataPath Path to a directory this reader can use for writing metadata. + * @param initialOffsets The Kafka offsets to start reading data at. + * @param failOnDataLoss Flag indicating whether read task generation should fail if some offsets + * after the specified initial offsets can't be found. + */ class ContinuousKafkaReader( - kafkaReader: KafkaOffsetReader, - executorKafkaParams: java.util.Map[String, Object], + offsetReader: KafkaOffsetReader, + kafkaParams: java.util.Map[String, Object], sourceOptions: Map[String, String], metadataPath: String, initialOffsets: KafkaOffsetRangeLimit, failOnDataLoss: Boolean) extends ContinuousReader with SupportsScanUnsafeRow with Logging { - override def mergeOffsets(offsets: Array[PartitionOffset]): Offset = { - val mergedMap = offsets.map { - case KafkaSourcePartitionOffset(p, o) => Map(p -> o) - }.reduce(_ ++ _) - KafkaSourceOffset(mergedMap) - } - private lazy val session = SparkSession.getActiveSession.get private lazy val sc = session.sparkContext @@ -62,39 +68,6 @@ class ContinuousKafkaReader( sc.conf.getTimeAsMs("spark.network.timeout", "120s").toString ).toLong - private val maxOffsetsPerTrigger = - sourceOptions.get("maxOffsetsPerTrigger").map(_.toLong) - - /** - * Lazily initialize `initialPartitionOffsets` to make sure that `KafkaConsumer.poll` is only - * called in StreamExecutionThread. Otherwise, interrupting a thread while running - * `KafkaConsumer.poll` may hang forever (KAFKA-1894). - */ - private lazy val initialPartitionOffsets = { - val offsets = initialOffsets match { - case EarliestOffsetRangeLimit => KafkaSourceOffset(kafkaReader.fetchEarliestOffsets()) - case LatestOffsetRangeLimit => KafkaSourceOffset(kafkaReader.fetchLatestOffsets()) - case SpecificOffsetRangeLimit(p) => fetchAndVerify(p) - } - logInfo(s"Initial offsets: $offsets") - offsets.partitionToOffsets - } - - private def fetchAndVerify(specificOffsets: Map[TopicPartition, Long]) = { - val result = kafkaReader.fetchSpecificOffsets(specificOffsets) - specificOffsets.foreach { - 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)}") - } - case _ => - // no real way to check that beginning or end is reasonable - } - KafkaSourceOffset(result) - } - // Initialized when creating read tasks. If this diverges from the partitions at the latest // offsets, we need to reconfigure. // Exposed outside this object only for unit tests. @@ -106,9 +79,9 @@ class ContinuousKafkaReader( override def setOffset(start: java.util.Optional[Offset]): Unit = { offset = start.orElse { val offsets = initialOffsets match { - case EarliestOffsetRangeLimit => KafkaSourceOffset(kafkaReader.fetchEarliestOffsets()) - case LatestOffsetRangeLimit => KafkaSourceOffset(kafkaReader.fetchLatestOffsets()) - case SpecificOffsetRangeLimit(p) => fetchAndVerify(p) + case EarliestOffsetRangeLimit => KafkaSourceOffset(offsetReader.fetchEarliestOffsets()) + case LatestOffsetRangeLimit => KafkaSourceOffset(offsetReader.fetchLatestOffsets()) + case SpecificOffsetRangeLimit(p) => offsetReader.fetchSpecificOffsets(p, reportDataLoss) } logInfo(s"Initial offsets: $offsets") offsets @@ -127,8 +100,8 @@ class ContinuousKafkaReader( val oldStartOffsets = KafkaSourceOffset.getPartitionOffsets(offset) val newPartitions = - kafkaReader.fetchLatestOffsets().keySet.diff(oldStartOffsets.keySet) - val newPartitionOffsets = kafkaReader.fetchEarliestOffsets(newPartitions.toSeq) + offsetReader.fetchLatestOffsets().keySet.diff(oldStartOffsets.keySet) + val newPartitionOffsets = offsetReader.fetchEarliestOffsets(newPartitions.toSeq) val startOffsets = oldStartOffsets ++ newPartitionOffsets knownPartitions = startOffsets.keySet @@ -136,23 +109,30 @@ class ContinuousKafkaReader( startOffsets.toSeq.map { case (topicPartition, start) => ContinuousKafkaReadTask( - topicPartition, start, executorKafkaParams, pollTimeoutMs, failOnDataLoss) + topicPartition, start, kafkaParams, pollTimeoutMs, failOnDataLoss) .asInstanceOf[ReadTask[UnsafeRow]] }.asJava } /** Stop this source and free any resources it has allocated. */ def stop(): Unit = synchronized { - kafkaReader.close() + offsetReader.close() } override def commit(end: Offset): Unit = {} + override def mergeOffsets(offsets: Array[PartitionOffset]): Offset = { + val mergedMap = offsets.map { + case KafkaSourcePartitionOffset(p, o) => Map(p -> o) + }.reduce(_ ++ _) + KafkaSourceOffset(mergedMap) + } + override def needsReconfiguration(): Boolean = { - knownPartitions != null && kafkaReader.fetchLatestOffsets().keySet != knownPartitions + knownPartitions != null && offsetReader.fetchLatestOffsets().keySet != knownPartitions } - override def toString(): String = s"KafkaSource[$kafkaReader]" + override def toString(): String = s"KafkaSource[$offsetReader]" /** * If `failOnDataLoss` is true, this method will throw an `IllegalStateException`. 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 3e65949a6fd1b..7a42e12f2418d 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,10 +117,14 @@ private[kafka010] class KafkaOffsetReader( * 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. + * + * @param partitionOffsets the specific offsets to resolve + * @param reportDataLoss callback to either report or log data loss depending on setting */ def fetchSpecificOffsets( - partitionOffsets: Map[TopicPartition, Long]): Map[TopicPartition, Long] = - runUninterruptibly { + partitionOffsets: Map[TopicPartition, Long], + reportDataLoss: String => Unit): KafkaSourceOffset = { + val fetched = runUninterruptibly { withRetriesWithoutInterrupt { // Poll to get the latest assigned partitions consumer.poll(0) @@ -145,6 +149,19 @@ private[kafka010] class KafkaOffsetReader( } } + partitionOffsets.foreach { + case (tp, off) if off != KafkaOffsetRangeLimit.LATEST && + off != KafkaOffsetRangeLimit.EARLIEST => + if (fetched(tp) != off) { + reportDataLoss( + s"startingOffsets for $tp was $off but consumer reset to ${fetched(tp)}") + } + case _ => + // no real way to check that beginning or end is reasonable + } + KafkaSourceOffset(fetched) + } + /** * Fetch the earliest offsets for the topic partitions that are indicated * in the [[ConsumerStrategy]]. 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 e9cff04ba5f2e..27da76068a66f 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 @@ -130,7 +130,7 @@ private[kafka010] class KafkaSource( val offsets = startingOffsets match { case EarliestOffsetRangeLimit => KafkaSourceOffset(kafkaReader.fetchEarliestOffsets()) case LatestOffsetRangeLimit => KafkaSourceOffset(kafkaReader.fetchLatestOffsets()) - case SpecificOffsetRangeLimit(p) => fetchAndVerify(p) + case SpecificOffsetRangeLimit(p) => kafkaReader.fetchSpecificOffsets(p, reportDataLoss) } metadataLog.add(0, offsets) logInfo(s"Initial offsets: $offsets") @@ -138,21 +138,6 @@ private[kafka010] class KafkaSource( }.partitionToOffsets } - private def fetchAndVerify(specificOffsets: Map[TopicPartition, Long]) = { - val result = kafkaReader.fetchSpecificOffsets(specificOffsets) - specificOffsets.foreach { - 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)}") - } - 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 = KafkaOffsetReader.kafkaSchema diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala index d157249c70a72..59bd5af398098 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.kafka010 import org.apache.kafka.common.TopicPartition -import org.apache.spark.sql.execution.streaming.{Offset => LegacyOffset, SerializedOffset} +import org.apache.spark.sql.execution.streaming.{Offset => OffsetV1, SerializedOffset} import org.apache.spark.sql.sources.v2.streaming.reader.{Offset, PartitionOffset} /** @@ -39,7 +39,7 @@ case class KafkaSourcePartitionOffset(topicPartition: TopicPartition, partitionO /** Companion object of the [[KafkaSourceOffset]] */ private[kafka010] object KafkaSourceOffset { - def getPartitionOffsets(offset: LegacyOffset): Map[TopicPartition, Long] = { + def getPartitionOffsets(offset: OffsetV1): Map[TopicPartition, Long] = { offset match { case o: KafkaSourceOffset => o.partitionToOffsets case so: SerializedOffset => KafkaSourceOffset(so).partitionToOffsets From eac756bc6b7a1b80a9a1f67c8e1537143386e6d7 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Thu, 4 Jan 2018 17:27:37 -0800 Subject: [PATCH 12/53] document read task and remove unused poll timeout --- .../main/scala/ContinuousKafkaReader.scala | 32 +++++++++++-------- 1 file changed, 18 insertions(+), 14 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala b/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala index 24d1963f6d551..8c2c5e748cee9 100644 --- a/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala @@ -48,8 +48,9 @@ import org.apache.spark.unsafe.types.UTF8String * are not Kafka consumer params. * @param metadataPath Path to a directory this reader can use for writing metadata. * @param initialOffsets The Kafka offsets to start reading data at. - * @param failOnDataLoss Flag indicating whether read task generation should fail if some offsets - * after the specified initial offsets can't be found. + * @param failOnDataLoss Flag indicating whether reading should fail in data loss + * scenarios, where some offsets after the specified initial ones can't be + * properly read. */ class ContinuousKafkaReader( offsetReader: KafkaOffsetReader, @@ -63,11 +64,6 @@ class ContinuousKafkaReader( private lazy val session = SparkSession.getActiveSession.get private lazy val sc = session.sparkContext - private lazy val pollTimeoutMs = sourceOptions.getOrElse( - "kafkaConsumer.pollTimeoutMs", - sc.conf.getTimeAsMs("spark.network.timeout", "120s").toString - ).toLong - // Initialized when creating read tasks. If this diverges from the partitions at the latest // offsets, we need to reconfigure. // Exposed outside this object only for unit tests. @@ -97,19 +93,19 @@ class ContinuousKafkaReader( override def createUnsafeRowReadTasks(): java.util.List[ReadTask[UnsafeRow]] = { import scala.collection.JavaConverters._ - val oldStartOffsets = KafkaSourceOffset.getPartitionOffsets(offset) + val oldStartPartitionOffsets = KafkaSourceOffset.getPartitionOffsets(offset) val newPartitions = - offsetReader.fetchLatestOffsets().keySet.diff(oldStartOffsets.keySet) + offsetReader.fetchLatestOffsets().keySet.diff(oldStartPartitionOffsets.keySet) val newPartitionOffsets = offsetReader.fetchEarliestOffsets(newPartitions.toSeq) - val startOffsets = oldStartOffsets ++ newPartitionOffsets + val startOffsets = oldStartPartitionOffsets ++ newPartitionOffsets knownPartitions = startOffsets.keySet startOffsets.toSeq.map { case (topicPartition, start) => ContinuousKafkaReadTask( - topicPartition, start, kafkaParams, pollTimeoutMs, failOnDataLoss) + topicPartition, start, kafkaParams, failOnDataLoss) .asInstanceOf[ReadTask[UnsafeRow]] }.asJava } @@ -147,15 +143,24 @@ class ContinuousKafkaReader( } } +/** + * A read task for continuous Kafka processing. This will be serialized and transformed into a + * full reader on executors. + * + * @param topicPartition The (topic, partition) pair this task is responsible for. + * @param start The offset to start reading from within the partition. + * @param kafkaParams Kafka consumer params to use. + * @param failOnDataLoss Flag indicating whether data reader should fail if some offsets + * are skipped. + */ case class ContinuousKafkaReadTask( topicPartition: TopicPartition, start: Long, kafkaParams: java.util.Map[String, Object], - pollTimeoutMs: Long, failOnDataLoss: Boolean) extends ReadTask[UnsafeRow] { override def createDataReader(): ContinuousKafkaDataReader = { - new ContinuousKafkaDataReader(topicPartition, start, kafkaParams, pollTimeoutMs, failOnDataLoss) + new ContinuousKafkaDataReader(topicPartition, start, kafkaParams, failOnDataLoss) } } @@ -163,7 +168,6 @@ class ContinuousKafkaDataReader( topicPartition: TopicPartition, start: Long, kafkaParams: java.util.Map[String, Object], - pollTimeoutMs: Long, failOnDataLoss: Boolean) extends ContinuousDataReader[UnsafeRow] { private val topic = topicPartition.topic From 9e95f63ffb7d3608f77b822b4ea781969baf0c46 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Thu, 4 Jan 2018 17:28:07 -0800 Subject: [PATCH 13/53] rename from start to startOffset --- .../src/main/scala/ContinuousKafkaReader.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala b/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala index 8c2c5e748cee9..c02bd9a87677a 100644 --- a/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala @@ -166,7 +166,7 @@ case class ContinuousKafkaReadTask( class ContinuousKafkaDataReader( topicPartition: TopicPartition, - start: Long, + startOffset: Long, kafkaParams: java.util.Map[String, Object], failOnDataLoss: Boolean) extends ContinuousDataReader[UnsafeRow] { @@ -176,10 +176,10 @@ class ContinuousKafkaDataReader( private val closed = new AtomicBoolean(false) - private var nextKafkaOffset = start match { + private var nextKafkaOffset = startOffset match { case s if s >= 0 => s case KafkaOffsetRangeLimit.EARLIEST => consumer.getAvailableOffsetRange().earliest - case _ => throw new IllegalArgumentException(s"Invalid start Kafka offset $start.") + case _ => throw new IllegalArgumentException(s"Invalid start Kafka offset $startOffset.") } private var currentRecord: ConsumerRecord[Array[Byte], Array[Byte]] = _ From 973fc7ddece0101401095c53ceeccf4a836984b3 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Thu, 4 Jan 2018 17:29:46 -0800 Subject: [PATCH 14/53] document data reader --- .../main/scala/ContinuousKafkaReader.scala | 21 ++++++++++++------- 1 file changed, 14 insertions(+), 7 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala b/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala index c02bd9a87677a..59d97d91f7339 100644 --- a/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala @@ -148,28 +148,35 @@ class ContinuousKafkaReader( * full reader on executors. * * @param topicPartition The (topic, partition) pair this task is responsible for. - * @param start The offset to start reading from within the partition. + * @param startOffset The offset to start reading from within the partition. * @param kafkaParams Kafka consumer params to use. * @param failOnDataLoss Flag indicating whether data reader should fail if some offsets * are skipped. */ case class ContinuousKafkaReadTask( topicPartition: TopicPartition, - start: Long, + startOffset: Long, kafkaParams: java.util.Map[String, Object], - failOnDataLoss: Boolean) - extends ReadTask[UnsafeRow] { + failOnDataLoss: Boolean) extends ReadTask[UnsafeRow] { override def createDataReader(): ContinuousKafkaDataReader = { - new ContinuousKafkaDataReader(topicPartition, start, kafkaParams, failOnDataLoss) + new ContinuousKafkaDataReader(topicPartition, startOffset, kafkaParams, failOnDataLoss) } } +/** + * A per-task data reader for continuous Kafka processing. + * + * @param topicPartition The (topic, partition) pair this data reader is responsible for. + * @param startOffset The offset to start reading from within the partition. + * @param kafkaParams Kafka consumer params to use. + * @param failOnDataLoss Flag indicating whether data reader should fail if some offsets + * are skipped. + */ class ContinuousKafkaDataReader( topicPartition: TopicPartition, startOffset: Long, kafkaParams: java.util.Map[String, Object], - failOnDataLoss: Boolean) - extends ContinuousDataReader[UnsafeRow] { + failOnDataLoss: Boolean) extends ContinuousDataReader[UnsafeRow] { private val topic = topicPartition.topic private val kafkaPartition = topicPartition.partition private val consumer = CachedKafkaConsumer.createUncached(topic, kafkaPartition, kafkaParams) From 9998d91b56cd3b7211d9e0a4a3f0e0383b5fb8aa Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Thu, 4 Jan 2018 17:38:00 -0800 Subject: [PATCH 15/53] remove redundant resolution --- .../src/main/scala/ContinuousKafkaReader.scala | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala b/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala index 59d97d91f7339..100c502ce1d01 100644 --- a/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala @@ -183,11 +183,7 @@ class ContinuousKafkaDataReader( private val closed = new AtomicBoolean(false) - private var nextKafkaOffset = startOffset match { - case s if s >= 0 => s - case KafkaOffsetRangeLimit.EARLIEST => consumer.getAvailableOffsetRange().earliest - case _ => throw new IllegalArgumentException(s"Invalid start Kafka offset $startOffset.") - } + private var nextKafkaOffset = startOffset private var currentRecord: ConsumerRecord[Array[Byte], Array[Byte]] = _ override def next(): Boolean = { From a3adf1d12206188febaf826863aed6ab2ff090bc Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Thu, 4 Jan 2018 17:38:51 -0800 Subject: [PATCH 16/53] consolidate vals and remove unused flag --- .../src/main/scala/ContinuousKafkaReader.scala | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala b/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala index 100c502ce1d01..d2bfe4433aa69 100644 --- a/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala @@ -181,8 +181,10 @@ class ContinuousKafkaDataReader( private val kafkaPartition = topicPartition.partition private val consumer = CachedKafkaConsumer.createUncached(topic, kafkaPartition, kafkaParams) - private val closed = new AtomicBoolean(false) - + private val sharedRow = new UnsafeRow(7) + private val bufferHolder = new BufferHolder(sharedRow) + private val rowWriter = new UnsafeRowWriter(bufferHolder, 7) + private var nextKafkaOffset = startOffset private var currentRecord: ConsumerRecord[Array[Byte], Array[Byte]] = _ @@ -200,10 +202,6 @@ class ContinuousKafkaDataReader( true } - val sharedRow = new UnsafeRow(7) - val bufferHolder = new BufferHolder(sharedRow) - val rowWriter = new UnsafeRowWriter(bufferHolder, 7) - override def get(): UnsafeRow = { bufferHolder.reset() From 71f236b9c270afdb3e317b1f8221b8cad85dbca4 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Thu, 4 Jan 2018 17:43:02 -0800 Subject: [PATCH 17/53] explicit close --- .../kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala | 2 +- .../main/scala/org/apache/spark/sql/kafka010/KafkaSinkV2.scala | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala b/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala index d2bfe4433aa69..968786d700a3e 100644 --- a/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala @@ -184,7 +184,7 @@ class ContinuousKafkaDataReader( private val sharedRow = new UnsafeRow(7) private val bufferHolder = new BufferHolder(sharedRow) private val rowWriter = new UnsafeRowWriter(bufferHolder, 7) - + private var nextKafkaOffset = startOffset private var currentRecord: ConsumerRecord[Array[Byte], Array[Byte]] = _ diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSinkV2.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSinkV2.scala index b330ba6d611d1..b0751dea707cd 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSinkV2.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSinkV2.scala @@ -97,6 +97,7 @@ class KafkaDataWriter( if (producer != null) { producer.flush() checkForErrors() + CachedKafkaProducer.close(new java.util.HashMap[String, Object](producerParams.asJava)) } } From 953060495dc024b123f1d7e1823c763db0a350fd Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Thu, 4 Jan 2018 17:44:42 -0800 Subject: [PATCH 18/53] put back error check in write --- .../main/scala/org/apache/spark/sql/kafka010/KafkaSinkV2.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSinkV2.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSinkV2.scala index b0751dea707cd..8ed454d9b4035 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSinkV2.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSinkV2.scala @@ -70,6 +70,7 @@ class KafkaDataWriter( } def write(row: InternalRow): Unit = { + checkForErrors() val projectedRow = projection(row) val topic = projectedRow.getUTF8String(0) val key = projectedRow.getBinary(1) @@ -90,6 +91,7 @@ class KafkaDataWriter( checkForErrors() KafkaWriterCommitMessage() } + def abort(): Unit = {} def close(): Unit = { From 4dca800933b4145f3a8a7dcc0178fb7090eb396d Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Thu, 4 Jan 2018 17:45:38 -0800 Subject: [PATCH 19/53] name constructor param targetTopic --- .../scala/org/apache/spark/sql/kafka010/KafkaSinkV2.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSinkV2.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSinkV2.scala index 8ed454d9b4035..c0aa1f842902c 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSinkV2.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSinkV2.scala @@ -52,7 +52,7 @@ case class KafkaWriterFactory( case class KafkaWriterCommitMessage() extends WriterCommitMessage {} class KafkaDataWriter( - topic: Option[String], producerParams: Map[String, String], inputSchema: Seq[Attribute]) + targetTopic: Option[String], producerParams: Map[String, String], inputSchema: Seq[Attribute]) extends DataWriter[InternalRow] { import scala.collection.JavaConverters._ @@ -104,7 +104,7 @@ class KafkaDataWriter( } private def createProjection: UnsafeProjection = { - val topicExpression = topic.map(Literal(_)).orElse { + val topicExpression = targetTopic.map(Literal(_)).orElse { inputSchema.find(_.name == KafkaWriter.TOPIC_ATTRIBUTE_NAME) }.getOrElse { throw new IllegalStateException(s"topic option required when no " + From fec5a00720b94c92fa4efec0f8d0f2784e0a709a Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Thu, 4 Jan 2018 17:54:34 -0800 Subject: [PATCH 20/53] refactor class names to start with kafka --- .../sql/kafka010/KafkaContinuousReader.scala} | 25 +++++++------------ ...nkV2.scala => KafkaContinuousWriter.scala} | 12 ++++----- .../sql/kafka010/KafkaSourceProvider.scala | 6 ++--- .../sql/kafka010/ContinuousKafkaSuite.scala | 2 +- .../spark/sql/kafka010/KafkaSourceSuite.scala | 2 +- 5 files changed, 20 insertions(+), 27 deletions(-) rename external/kafka-0-10-sql/src/main/scala/{ContinuousKafkaReader.scala => org/apache/spark/sql/kafka010/KafkaContinuousReader.scala} (92%) rename external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/{KafkaSinkV2.scala => KafkaContinuousWriter.scala} (94%) diff --git a/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala similarity index 92% rename from external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala rename to external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala index 968786d700a3e..2ec5afce80da2 100644 --- a/external/kafka-0-10-sql/src/main/scala/ContinuousKafkaReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala @@ -17,22 +17,15 @@ package org.apache.spark.sql.kafka010 -import java.io._ -import java.nio.charset.StandardCharsets -import java.util.concurrent.atomic.AtomicBoolean - -import org.apache.commons.io.IOUtils import org.apache.kafka.clients.consumer.ConsumerRecord import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.errors.WakeupException import org.apache.spark.internal.Logging -import org.apache.spark.sql.{DataFrame, Row, SparkSession, SQLContext} -import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Literal, UnsafeProjection, UnsafeRow} +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.catalyst.expressions.codegen.{BufferHolder, UnsafeRowWriter} import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.execution.streaming.{HDFSMetadataLog, SerializedOffset} -import org.apache.spark.sql.kafka010.KafkaSource.{INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE, INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE, VERSION} +import org.apache.spark.sql.kafka010.KafkaSource.{INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE, INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE} import org.apache.spark.sql.sources.v2.reader._ import org.apache.spark.sql.sources.v2.streaming.reader.{ContinuousDataReader, ContinuousReader, Offset, PartitionOffset} import org.apache.spark.sql.types.StructType @@ -52,7 +45,7 @@ import org.apache.spark.unsafe.types.UTF8String * scenarios, where some offsets after the specified initial ones can't be * properly read. */ -class ContinuousKafkaReader( +class KafkaContinuousReader( offsetReader: KafkaOffsetReader, kafkaParams: java.util.Map[String, Object], sourceOptions: Map[String, String], @@ -104,7 +97,7 @@ class ContinuousKafkaReader( startOffsets.toSeq.map { case (topicPartition, start) => - ContinuousKafkaReadTask( + KafkaContinuousReadTask( topicPartition, start, kafkaParams, failOnDataLoss) .asInstanceOf[ReadTask[UnsafeRow]] }.asJava @@ -153,13 +146,13 @@ class ContinuousKafkaReader( * @param failOnDataLoss Flag indicating whether data reader should fail if some offsets * are skipped. */ -case class ContinuousKafkaReadTask( +case class KafkaContinuousReadTask( topicPartition: TopicPartition, startOffset: Long, kafkaParams: java.util.Map[String, Object], failOnDataLoss: Boolean) extends ReadTask[UnsafeRow] { - override def createDataReader(): ContinuousKafkaDataReader = { - new ContinuousKafkaDataReader(topicPartition, startOffset, kafkaParams, failOnDataLoss) + override def createDataReader(): KafkaContinuousDataReader = { + new KafkaContinuousDataReader(topicPartition, startOffset, kafkaParams, failOnDataLoss) } } @@ -172,7 +165,7 @@ case class ContinuousKafkaReadTask( * @param failOnDataLoss Flag indicating whether data reader should fail if some offsets * are skipped. */ -class ContinuousKafkaDataReader( +class KafkaContinuousDataReader( topicPartition: TopicPartition, startOffset: Long, kafkaParams: java.util.Map[String, Object], diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSinkV2.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousWriter.scala similarity index 94% rename from external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSinkV2.scala rename to external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousWriter.scala index c0aa1f842902c..520837b557306 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSinkV2.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousWriter.scala @@ -29,29 +29,29 @@ import org.apache.spark.sql.sources.v2.writer._ import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.{BinaryType, StringType, StructType} -class ContinuousKafkaWriter( +class KafkaContinuousWriter( topic: Option[String], producerParams: Map[String, String], schema: StructType) extends ContinuousWriter with SupportsWriteInternalRow { - override def createInternalRowWriterFactory(): KafkaWriterFactory = - KafkaWriterFactory(topic, producerParams, schema) + override def createInternalRowWriterFactory(): KafkaContinuousWriterFactory = + KafkaContinuousWriterFactory(topic, producerParams, schema) override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {} override def abort(messages: Array[WriterCommitMessage]): Unit = {} } -case class KafkaWriterFactory( +case class KafkaContinuousWriterFactory( topic: Option[String], producerParams: Map[String, String], schema: StructType) extends DataWriterFactory[InternalRow] { override def createDataWriter(partitionId: Int, attemptNumber: Int): DataWriter[InternalRow] = { - new KafkaDataWriter(topic, producerParams, schema.toAttributes) + new KafkaContinuousDataWriter(topic, producerParams, schema.toAttributes) } } case class KafkaWriterCommitMessage() extends WriterCommitMessage {} -class KafkaDataWriter( +class KafkaContinuousDataWriter( targetTopic: Option[String], producerParams: Map[String, String], inputSchema: Seq[Attribute]) extends DataWriter[InternalRow] { import scala.collection.JavaConverters._ 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 a51d5104edc87..00e633812bd5c 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 @@ -109,7 +109,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister override def createContinuousReader( schema: java.util.Optional[StructType], metadataPath: String, - options: DataSourceV2Options): ContinuousKafkaReader = { + options: DataSourceV2Options): KafkaContinuousReader = { val parameters = options.asMap().asScala.toMap validateStreamOptions(parameters) // Each running query should use its own group id. Otherwise, the query may be only assigned @@ -134,7 +134,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister parameters, driverGroupIdPrefix = s"$uniqueGroupId-driver") - new ContinuousKafkaReader( + new KafkaContinuousReader( kafkaOffsetReader, kafkaParamsForExecutors(specifiedKafkaParams, uniqueGroupId), parameters, @@ -238,7 +238,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister KafkaWriter.validateQuery( schema.toAttributes, new java.util.HashMap[String, Object](producerParams.asJava), topic) - java.util.Optional.of(new ContinuousKafkaWriter(topic, producerParams, schema)) + java.util.Optional.of(new KafkaContinuousWriter(topic, producerParams, schema)) } private def strategy(caseInsensitiveParams: Map[String, String]) = diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/ContinuousKafkaSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/ContinuousKafkaSuite.scala index 4bf5e0c79c20c..0689cb70087db 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/ContinuousKafkaSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/ContinuousKafkaSuite.scala @@ -95,7 +95,7 @@ class ContinuousKafkaSuite extends KafkaSourceTest with SharedSQLContext { eventually(timeout(streamingTimeout)) { assert( q.lastExecution.logical.collectFirst { - case DataSourceV2Relation(_, r: ContinuousKafkaReader) => r + case DataSourceV2Relation(_, r: KafkaContinuousReader) => r }.exists(_.knownPartitions.size == 5), "query never reconfigured to 5 partitions") } 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 5512979dcc715..f3d9bfa0660dc 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 @@ -111,7 +111,7 @@ abstract class KafkaSourceTest extends StreamTest with SharedSQLContext { } ++ (query.get.lastExecution match { case null => Seq() case e => e.logical.collect { - case DataSourceV2Relation(_, reader: ContinuousKafkaReader) => reader + case DataSourceV2Relation(_, reader: KafkaContinuousReader) => reader } }) if (sources.isEmpty) { From 65ecf8557dada337bcb297a10153bcfef6a228a4 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Thu, 4 Jan 2018 17:54:56 -0800 Subject: [PATCH 21/53] use case object --- .../org/apache/spark/sql/kafka010/KafkaContinuousWriter.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousWriter.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousWriter.scala index 520837b557306..23fb68591ceee 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousWriter.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousWriter.scala @@ -49,7 +49,7 @@ case class KafkaContinuousWriterFactory( } } -case class KafkaWriterCommitMessage() extends WriterCommitMessage {} +case object KafkaWriterCommitMessage extends WriterCommitMessage class KafkaContinuousDataWriter( targetTopic: Option[String], producerParams: Map[String, String], inputSchema: Seq[Attribute]) From 24f7e1f9ff973e0e158953d5a46e9dc4b127a3e2 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Thu, 4 Jan 2018 18:13:09 -0800 Subject: [PATCH 22/53] dedup kafka writing --- .../sql/kafka010/KafkaContinuousWriter.scala | 70 +----------------- .../spark/sql/kafka010/KafkaWriteTask.scala | 71 ++++++++++++------- 2 files changed, 47 insertions(+), 94 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousWriter.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousWriter.scala index 23fb68591ceee..f206805374ec9 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousWriter.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousWriter.scala @@ -53,35 +53,15 @@ case object KafkaWriterCommitMessage extends WriterCommitMessage class KafkaContinuousDataWriter( targetTopic: Option[String], producerParams: Map[String, String], inputSchema: Seq[Attribute]) - extends DataWriter[InternalRow] { + extends KafkaRowWriter(inputSchema, targetTopic) with DataWriter[InternalRow] { import scala.collection.JavaConverters._ - @volatile private var failedWrite: Exception = _ - private val projection = createProjection private lazy val producer = CachedKafkaProducer.getOrCreate( new java.util.HashMap[String, Object](producerParams.asJava)) - private val callback = new Callback() { - override def onCompletion(recordMetadata: RecordMetadata, e: Exception): Unit = { - if (failedWrite == null && e != null) { - failedWrite = e - } - } - } - def write(row: InternalRow): Unit = { checkForErrors() - val projectedRow = projection(row) - 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.") - } - val record = new ProducerRecord[Array[Byte], Array[Byte]](topic.toString, key, value) - producer.send(record, callback) + sendRow(row, producer) } def commit(): WriterCommitMessage = { @@ -89,7 +69,7 @@ class KafkaContinuousDataWriter( // This requires flushing and then checking that no callbacks produced errors. producer.flush() checkForErrors() - KafkaWriterCommitMessage() + KafkaWriterCommitMessage } def abort(): Unit = {} @@ -102,48 +82,4 @@ class KafkaContinuousDataWriter( CachedKafkaProducer.close(new java.util.HashMap[String, Object](producerParams.asJava)) } } - - private def createProjection: UnsafeProjection = { - val topicExpression = targetTopic.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} " + - "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("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(): Unit = { - if (failedWrite != null) { - throw failedWrite - } - } } 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 6fd333e2f43ba..cee69a4759d2a 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 @@ -33,10 +33,8 @@ import org.apache.spark.sql.types.{BinaryType, StringType} private[kafka010] class KafkaWriteTask( producerConfiguration: ju.Map[String, Object], inputSchema: Seq[Attribute], - topic: Option[String]) { + topic: Option[String]) extends KafkaRowWriter(inputSchema, topic) { // used to synchronize with Kafka callbacks - @volatile private var failedWrite: Exception = null - private val projection = createProjection private var producer: KafkaProducer[Array[Byte], Array[Byte]] = _ /** @@ -46,23 +44,7 @@ private[kafka010] class KafkaWriteTask( producer = CachedKafkaProducer.getOrCreate(producerConfiguration) while (iterator.hasNext && failedWrite == null) { val currentRow = iterator.next() - val projectedRow = projection(currentRow) - 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.") - } - 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) { - failedWrite = e - } - } - } - producer.send(record, callback) + sendRow(currentRow, producer) } } @@ -74,8 +56,49 @@ private[kafka010] class KafkaWriteTask( producer = null } } +} + +private[kafka010] class KafkaRowWriter( + inputSchema: Seq[Attribute], topic: Option[String]) { + + // used to synchronize with Kafka callbacks + @volatile protected var failedWrite: Exception = _ + protected val projection = createProjection + + private val callback = new Callback() { + override def onCompletion(recordMetadata: RecordMetadata, e: Exception): Unit = { + if (failedWrite == null && e != null) { + failedWrite = e + } + } + } - private def createProjection: UnsafeProjection = { + /** + * Send the specified row to the producer, with a callback that will save any exception + * to failedWrite. Note that send is asynchronous; subclasses must flush() their producer before + * assuming the row is in Kafka. + */ + protected def sendRow( + row: InternalRow, producer: KafkaProducer[Array[Byte], Array[Byte]]): Unit = { + val projectedRow = projection(row) + 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.") + } + val record = new ProducerRecord[Array[Byte], Array[Byte]](topic.toString, key, value) + producer.send(record, callback) + } + + protected def checkForErrors(): Unit = { + if (failedWrite != null) { + throw failedWrite + } + } + + private def createProjection = { val topicExpression = topic.map(Literal(_)).orElse { inputSchema.find(_.name == KafkaWriter.TOPIC_ATTRIBUTE_NAME) }.getOrElse { @@ -112,11 +135,5 @@ private[kafka010] class KafkaWriteTask( Seq(topicExpression, Cast(keyExpression, BinaryType), Cast(valueExpression, BinaryType)), inputSchema) } - - private def checkForErrors(): Unit = { - if (failedWrite != null) { - throw failedWrite - } - } } From 16628286fafe4aa8d84d7016a52bada7f0885bcd Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Thu, 4 Jan 2018 18:20:39 -0800 Subject: [PATCH 23/53] document writer classes --- .../sql/kafka010/KafkaContinuousWriter.scala | 32 +++++++++++++++++-- 1 file changed, 30 insertions(+), 2 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousWriter.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousWriter.scala index f206805374ec9..49ac9145e7f99 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousWriter.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousWriter.scala @@ -29,6 +29,19 @@ import org.apache.spark.sql.sources.v2.writer._ import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.{BinaryType, StringType, StructType} +/** + * Dummy commit message. The DataSourceV2 framework requires a commit message implementation but we + * don't need to really send one. + */ +case object KafkaWriterCommitMessage extends WriterCommitMessage + +/** + * A [[ContinuousWriter]] for Kafka writing. Responsible for generating the writer factory. + * @param topic The topic this writer is responsible for. If None, topic will be inferred from + * a `topic` field in the incoming data. + * @param producerParams Parameters for Kafka producers in each task. + * @param schema The schema of the input data. + */ class KafkaContinuousWriter( topic: Option[String], producerParams: Map[String, String], schema: StructType) extends ContinuousWriter with SupportsWriteInternalRow { @@ -40,6 +53,14 @@ class KafkaContinuousWriter( override def abort(messages: Array[WriterCommitMessage]): Unit = {} } +/** + * A [[DataWriterFactory]] for Kafka writing. Will be serialized and sent to executors to generate + * the per-task data writers. + * @param topic The topic that should be written to. If None, topic will be inferred from + * a `topic` field in the incoming data. + * @param producerParams Parameters for Kafka producers in each task. + * @param schema The schema of the input data. + */ case class KafkaContinuousWriterFactory( topic: Option[String], producerParams: Map[String, String], schema: StructType) extends DataWriterFactory[InternalRow] { @@ -49,8 +70,15 @@ case class KafkaContinuousWriterFactory( } } -case object KafkaWriterCommitMessage extends WriterCommitMessage - +/** + * A [[DataWriter]] for Kafka writing. One data writer will be created in each partition to + * process incoming rows. + * + * @param targetTopic The topic that this data writer is targeting. If None, topic will be inferred + * from a `topic` field in the incoming data. + * @param producerParams Parameters to use for the Kafka producer. + * @param inputSchema The attributes in the input data. + */ class KafkaContinuousDataWriter( targetTopic: Option[String], producerParams: Map[String, String], inputSchema: Seq[Attribute]) extends KafkaRowWriter(inputSchema, targetTopic) with DataWriter[InternalRow] { From c66850e6af6666458a2313aceaac68e1932df9e0 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Thu, 4 Jan 2018 18:21:44 -0800 Subject: [PATCH 24/53] import optional --- .../apache/spark/sql/kafka010/KafkaSourceProvider.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 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 00e633812bd5c..3914370a96595 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 @@ -18,7 +18,7 @@ package org.apache.spark.sql.kafka010 import java.{util => ju} -import java.util.{Locale, UUID} +import java.util.{Locale, Optional, UUID} import scala.collection.JavaConverters._ @@ -107,7 +107,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister } override def createContinuousReader( - schema: java.util.Optional[StructType], + schema: Optional[StructType], metadataPath: String, options: DataSourceV2Options): KafkaContinuousReader = { val parameters = options.asMap().asScala.toMap @@ -227,7 +227,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister queryId: String, schema: StructType, mode: OutputMode, - options: DataSourceV2Options): java.util.Optional[ContinuousWriter] = { + options: DataSourceV2Options): Optional[ContinuousWriter] = { import scala.collection.JavaConverters._ val spark = SparkSession.getActiveSession.get @@ -238,7 +238,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister KafkaWriter.validateQuery( schema.toAttributes, new java.util.HashMap[String, Object](producerParams.asJava), topic) - java.util.Optional.of(new KafkaContinuousWriter(topic, producerParams, schema)) + Optional.of(new KafkaContinuousWriter(topic, producerParams, schema)) } private def strategy(caseInsensitiveParams: Map[String, String]) = From 7871de069c7c491c307a7464ce076ff0d6323868 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Thu, 4 Jan 2018 18:22:24 -0800 Subject: [PATCH 25/53] put kafka first in test class names --- ...{ContinuousKafkaSuite.scala => KafkaContinuousSuite.scala} | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) rename external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/{ContinuousKafkaSuite.scala => KafkaContinuousSuite.scala} (98%) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/ContinuousKafkaSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala similarity index 98% rename from external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/ContinuousKafkaSuite.scala rename to external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala index 0689cb70087db..69794aa86f73b 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/ContinuousKafkaSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution import org.apache.spark.sql.streaming.Trigger import org.apache.spark.sql.test.{SharedSQLContext, TestSparkSession} -class ContinuousKafkaSuite extends KafkaSourceTest with SharedSQLContext { +class KafkaContinuousSuite extends KafkaSourceTest with SharedSQLContext { import testImplicits._ // We need more than the default local[2] to be able to schedule all partitions simultaneously. @@ -142,7 +142,7 @@ class ContinuousKafkaSuite extends KafkaSourceTest with SharedSQLContext { } } -class ContinuousKafkaStressSuite extends KafkaSourceTest with SharedSQLContext { +class KafkaContinuousStressSuite extends KafkaSourceTest with SharedSQLContext { import testImplicits._ // We need more than the default local[2] to be able to schedule all partitions simultaneously. From f9ad94e8aa753892f40ee8fc069969563347764c Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Thu, 4 Jan 2018 18:26:45 -0800 Subject: [PATCH 26/53] add comments about fallback --- .../src/main/scala/org/apache/spark/sql/DataFrameReader.scala | 3 +++ .../src/main/scala/org/apache/spark/sql/DataFrameWriter.scala | 3 +++ 2 files changed, 6 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 75fbe548b1dae..b714a46b5f786 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -191,6 +191,9 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { ds = ds.asInstanceOf[DataSourceV2], conf = sparkSession.sessionState.conf)).asJava) + // Streaming also uses the data source V2 API. So it may be that the data source implements + // v2, but has no v2 implementation for batch reads. In that case, we fall back to loading + // the dataframe as a v1 source. val reader = (ds, userSpecifiedSchema) match { case (ds: ReadSupportWithSchema, Some(schema)) => ds.createReader(schema, options) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index cf370c0a36ecf..97f12ff625c42 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -255,6 +255,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { } } + // Streaming also uses the data source V2 API. So it may be that the data source implements + // v2, but has no v2 implementation for batch writes. In that case, we fall back to saving + // as though it's a V1 source. case _ => saveToV1Source() } } else { From be6c378969920aeda6506d1c2cfb91a33dfe7027 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Fri, 5 Jan 2018 01:51:36 -0800 Subject: [PATCH 27/53] fix test suite --- .../sql/kafka010/KafkaContinuousSuite.scala | 27 ++++--------------- .../datasources/v2/WriteToDataSourceV2.scala | 8 +++--- 2 files changed, 10 insertions(+), 25 deletions(-) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala index 69794aa86f73b..94cbafc4f38c8 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala @@ -172,14 +172,6 @@ class KafkaContinuousStressSuite extends KafkaSourceTest with SharedSQLContext { testUtils.setup() } - override def afterAll(): Unit = { - if (testUtils != null) { - testUtils.teardown() - testUtils = null - super.afterAll() - } - } - test("stress test for failOnDataLoss=false") { val reader = spark .readStream @@ -193,20 +185,11 @@ class KafkaContinuousStressSuite extends KafkaSourceTest with SharedSQLContext { val kafka = reader.load() .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") .as[(String, String)] - val query = kafka.map(kv => kv._2.toInt).writeStream.foreach(new ForeachWriter[Int] { - - override def open(partitionId: Long, version: Long): Boolean = { - true - } - - override def process(value: Int): Unit = { - // Slow down the processing speed so that messages may be aged out. - Thread.sleep(Random.nextInt(500)) - } - - override def close(errorOrNull: Throwable): Unit = { - } - }).start() + val query = kafka.map(kv => kv._2.toInt).writeStream + .format("memory") + .queryName("stress") + .trigger(Trigger.Continuous(100)) + .start() val testTime = 1.minutes val startTime = System.currentTimeMillis() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2.scala index f0bdf84bb7a84..a4a857f2d4d9b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2.scala @@ -81,9 +81,11 @@ case class WriteToDataSourceV2Exec(writer: DataSourceV2Writer, query: SparkPlan) (index, message: WriterCommitMessage) => messages(index) = message ) - logInfo(s"Data source writer $writer is committing.") - writer.commit(messages) - logInfo(s"Data source writer $writer committed.") + if (!writer.isInstanceOf[ContinuousWriter]) { + logInfo(s"Data source writer $writer is committing.") + writer.commit(messages) + logInfo(s"Data source writer $writer committed.") + } } catch { case _: InterruptedException if writer.isInstanceOf[ContinuousWriter] => // Interruption is how continuous queries are ended, so accept and ignore the exception. From 1e5c7a99c5a75539500c572770c03ee4c7e6d4a0 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Fri, 5 Jan 2018 10:04:30 -0800 Subject: [PATCH 28/53] fix flakiness in temp checkpoint cleanup --- .../org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala index 94cbafc4f38c8..f9c668e3511fb 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala @@ -138,6 +138,7 @@ class KafkaContinuousSuite extends KafkaSourceTest with SharedSQLContext { assert(Range(0, 20).map(Row(_)).toSet.subsetOf(results.toSet)) } query.stop() + query.awaitTermination() } } } From d94461adc9828ecfdc33f10472d60c7257210cae Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Mon, 8 Jan 2018 15:26:17 -0800 Subject: [PATCH 29/53] partial test refactor --- .../sql/kafka010/KafkaContinuousReader.scala | 12 +- .../sql/kafka010/KafkaContinuousSuite.scala | 214 ++++--------- .../spark/sql/kafka010/KafkaSourceSuite.scala | 282 +++++++++--------- .../execution/streaming/StreamExecution.scala | 6 +- .../continuous/ContinuousExecution.scala | 29 +- .../spark/sql/streaming/StreamTest.scala | 11 +- 6 files changed, 246 insertions(+), 308 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala index 2ec5afce80da2..8dd299cc8645d 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala @@ -88,11 +88,17 @@ class KafkaContinuousReader( val oldStartPartitionOffsets = KafkaSourceOffset.getPartitionOffsets(offset) - val newPartitions = - offsetReader.fetchLatestOffsets().keySet.diff(oldStartPartitionOffsets.keySet) + val currentPartitionSet = offsetReader.fetchEarliestOffsets().keySet + val newPartitions = currentPartitionSet.diff(oldStartPartitionOffsets.keySet) val newPartitionOffsets = offsetReader.fetchEarliestOffsets(newPartitions.toSeq) - val startOffsets = oldStartPartitionOffsets ++ newPartitionOffsets + val deletedPartitions = oldStartPartitionOffsets.keySet.diff(currentPartitionSet) + if (deletedPartitions.nonEmpty) { + reportDataLoss(s"Some partitions were deleted: $deletedPartitions") + } + + val startOffsets = newPartitionOffsets ++ + oldStartPartitionOffsets.filterKeys(!deletedPartitions.contains(_)) knownPartitions = startOffsets.keySet startOffsets.toSeq.map { diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala index f9c668e3511fb..ef14768e51c91 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala @@ -26,13 +26,14 @@ import scala.util.Random import org.apache.spark.SparkContext import org.apache.spark.sql.{ForeachWriter, Row} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation -import org.apache.spark.sql.execution.streaming.{ContinuousExecutionRelation, StreamingExecutionRelation, StreamingQueryWrapper} +import org.apache.spark.sql.execution.streaming.StreamExecution import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution -import org.apache.spark.sql.streaming.Trigger +import org.apache.spark.sql.streaming.{StreamTest, Trigger} import org.apache.spark.sql.test.{SharedSQLContext, TestSparkSession} -class KafkaContinuousSuite extends KafkaSourceTest with SharedSQLContext { - import testImplicits._ +trait KafkaContinuousTest extends KafkaSourceTest { + override val defaultTrigger = Trigger.Continuous(100) + override val defaultUseV2Sink = true // We need more than the default local[2] to be able to schedule all partitions simultaneously. override protected def createSparkSession = new TestSparkSession( @@ -41,69 +42,34 @@ class KafkaContinuousSuite extends KafkaSourceTest with SharedSQLContext { "continuous-stream-test-sql-context", sparkConf.set("spark.sql.testkey", "true"))) - test("basic") { - val topic = newTopic() - testUtils.createTopic(topic, partitions = 5) - testUtils.sendMessages(topic, Array("0")) - require(testUtils.getLatestOffsets(Set(topic)).size === 5) - - val reader = spark - .readStream - .format("kafka") - .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("subscribe", topic) + override protected def setTopicPartitions( + topic: String, newCount: Int, query: StreamExecution) = { + testUtils.addPartitions(topic, newCount) + eventually(timeout(streamingTimeout)) { + assert( + query.lastExecution.logical.collectFirst { + case DataSourceV2Relation(_, r: KafkaContinuousReader) => r + }.exists(_.knownPartitions.size == newCount), + s"query never reconfigured to $newCount partitions") + } + } - val kafka = reader.load() - .selectExpr("CAST(value AS STRING)") - .as[String] - .map(_.toInt + 1) + test("ensure continuous stream is being used") { + val query = spark.readStream + .format("rate") + .option("numPartitions", "1") + .option("rowsPerSecond", "1") + .load() - testStream(kafka, useV2Sink = true)( - StartStream(Trigger.Continuous(100)), - AddKafkaData(Set(topic), 1, 2, 3), - CheckAnswer(2, 3, 4) + testStream(query)( + Execute(q => assert(q.isInstanceOf[ContinuousExecution])) ) } +} - test("add partitions") { - val topic = newTopic() - testUtils.createTopic(topic, partitions = 2) - testUtils.sendMessages(topic, Array("0")) - require(testUtils.getLatestOffsets(Set(topic)).size === 2) +class KafkaContinuousSourceSuite extends KafkaSourceSuite with KafkaContinuousTest { - val reader = spark - .readStream - .format("kafka") - .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("subscribe", topic) - .option("kafka.metadata.max.age.ms", "1") - - val kafka = reader.load() - .selectExpr("CAST(value AS STRING)") - .as[String] - .map(_.toInt + 1) - - testStream(kafka, useV2Sink = true)( - StartStream(Trigger.Continuous(100)), - AddKafkaData(Set(topic), scala.Range(0, 5): _*), - CheckAnswer(1, 2, 3, 4, 5), - Execute(_ => testUtils.addPartitions(topic, 5)), - Execute { q => - // We have to assert on this first to avoid a race condition in the test framework. - // AddKafkaData will not work if the query is in the middle of reconfiguring, because it - // assumes it can find the right reader in the plan. - eventually(timeout(streamingTimeout)) { - assert( - q.lastExecution.logical.collectFirst { - case DataSourceV2Relation(_, r: KafkaContinuousReader) => r - }.exists(_.knownPartitions.size == 5), - "query never reconfigured to 5 partitions") - } - }, - AddKafkaData(Set(topic), scala.Range(10, 20): _*), - CheckAnswer(1, 2, 3, 4, 5, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20) - ) - } + import testImplicits._ test("kafka sink") { withTempDir { dir => @@ -141,108 +107,52 @@ class KafkaContinuousSuite extends KafkaSourceTest with SharedSQLContext { query.awaitTermination() } } -} -class KafkaContinuousStressSuite extends KafkaSourceTest with SharedSQLContext { - import testImplicits._ - - // We need more than the default local[2] to be able to schedule all partitions simultaneously. - override protected def createSparkSession = new TestSparkSession( - new SparkContext( - "local[10]", - "continuous-stream-test-sql-context", - sparkConf.set("spark.sql.testkey", "true"))) - - override def beforeAll(): Unit = { - super.beforeAll() - testUtils = new KafkaTestUtils { - override def brokerConfiguration: Properties = { - val props = super.brokerConfiguration - // Try to make Kafka clean up messages as fast as possible. However, there is a hard-code - // 30 seconds delay (kafka.log.LogManager.InitialTaskDelayMs) so this test should run at - // least 30 seconds. - props.put("log.cleaner.backoff.ms", "100") - props.put("log.segment.bytes", "40") - props.put("log.retention.bytes", "40") - props.put("log.retention.check.interval.ms", "100") - props.put("delete.retention.ms", "10") - props.put("log.flush.scheduler.interval.ms", "10") - props - } - } - testUtils.setup() - } + test("subscribing topic by pattern with topic deletions") { + val topicPrefix = newTopic() + val topic = topicPrefix + "-seems" + val topic2 = topicPrefix + "-bad" + testUtils.createTopic(topic, partitions = 5) + testUtils.sendMessages(topic, Array("-1")) + require(testUtils.getLatestOffsets(Set(topic)).size === 5) - test("stress test for failOnDataLoss=false") { val reader = spark .readStream .format("kafka") .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.metadata.max.age.ms", "1") - .option("subscribePattern", "failOnDataLoss.*") - .option("startingOffsets", "earliest") + .option("subscribePattern", s"$topicPrefix-.*") .option("failOnDataLoss", "false") - .option("fetchOffset.retryIntervalMs", "3000") + val kafka = reader.load() .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") .as[(String, String)] - val query = kafka.map(kv => kv._2.toInt).writeStream - .format("memory") - .queryName("stress") - .trigger(Trigger.Continuous(100)) - .start() - - val testTime = 1.minutes - val startTime = System.currentTimeMillis() - // Track the current existing topics - val topics = mutable.ArrayBuffer[String]() - // Track topics that have been deleted - val deletedTopics = mutable.Set[String]() - while (System.currentTimeMillis() - testTime.toMillis < startTime) { - Random.nextInt(10) match { - case 0 => // Create a new topic - val topic = newTopic() - topics += topic - // As pushing messages into Kafka updates Zookeeper asynchronously, there is a small - // chance that a topic will be recreated after deletion due to the asynchronous update. - // Hence, always overwrite to handle this race condition. - testUtils.createTopic(topic, partitions = 1, overwrite = true) - logInfo(s"Create topic $topic") - case 1 if topics.nonEmpty => // Delete an existing topic - val topic = topics.remove(Random.nextInt(topics.size)) - testUtils.deleteTopic(topic) - logInfo(s"Delete topic $topic") - deletedTopics += topic - case 2 if deletedTopics.nonEmpty => // Recreate a topic that was deleted. - val topic = deletedTopics.toSeq(Random.nextInt(deletedTopics.size)) - deletedTopics -= topic - topics += topic - // As pushing messages into Kafka updates Zookeeper asynchronously, there is a small - // chance that a topic will be recreated after deletion due to the asynchronous update. - // Hence, always overwrite to handle this race condition. - testUtils.createTopic(topic, partitions = 1, overwrite = true) - logInfo(s"Create topic $topic") - case 3 => - Thread.sleep(1000) - case _ => // Push random messages - for (topic <- topics) { - val size = Random.nextInt(10) - for (_ <- 0 until size) { - testUtils.sendMessages(topic, Array(Random.nextInt(10).toString)) - } - } - } - // `failOnDataLoss` is `false`, we should not fail the query - if (query.exception.nonEmpty) { - throw query.exception.get - } - } + val mapped = kafka.map(kv => kv._2.toInt + 1) - query.stop() - query.awaitTermination() - // `failOnDataLoss` is `false`, we should not fail the query - if (query.exception.nonEmpty) { - throw query.exception.get - } + testStream(mapped)( + makeSureGetOffsetCalled, + AddKafkaData(Set(topic), 1, 2, 3), + CheckAnswer(2, 3, 4), + Execute { query => + testUtils.deleteTopic(topic) + testUtils.createTopic(topic2, partitions = 5) + eventually(timeout(streamingTimeout)) { + assert( + query.lastExecution.logical.collectFirst { + case DataSourceV2Relation(_, r: KafkaContinuousReader) => r + }.exists { r => + // Ensure the new topic is present and the old topic is gone. + r.knownPartitions.exists(_.topic == topic2) + }, + s"query never reconfigured to new topic $topic2") + } + }, + AddKafkaData(Set(topic2), 4, 5, 6), + CheckAnswer(2, 3, 4, 5, 6, 7) + ) } } + +/* class KafkaContinuousStressSuite extends KafkaSourceStressSuite with KafkaContinuousTest { + +} */ 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 f3d9bfa0660dc..44394ab5ad246 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 @@ -74,6 +74,10 @@ abstract class KafkaSourceTest extends StreamTest with SharedSQLContext { true } + protected def setTopicPartitions(topic: String, newCount: Int, query: StreamExecution) : Unit = { + testUtils.addPartitions(topic, newCount) + } + /** * Add data to Kafka. * @@ -100,8 +104,6 @@ abstract class KafkaSourceTest extends StreamTest with SharedSQLContext { topicAction(existingTopicPartitions._1, Some(existingTopicPartitions._2)) } - // Read all topics again in case some topics are delete. - val allTopics = testUtils.getAllTopicsAndPartitionSize().toMap.keys require( query.nonEmpty, "Cannot add data when there is no query for finding the active kafka source") @@ -149,6 +151,137 @@ abstract class KafkaSourceTest extends StreamTest with SharedSQLContext { protected def newTopic(): String = s"topic-${topicId.getAndIncrement()}" } +class KafkaMicroBatchSourceSuite extends KafkaSourceTest { + + import testImplicits._ + + 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 + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + .option("maxOffsetsPerTrigger", 10) + .option("subscribe", topic) + .option("startingOffsets", "earliest") + 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) + + val clock = new StreamManualClock + + val waitUntilBatchProcessed = AssertOnQuery { q => + eventually(Timeout(streamingTimeout)) { + if (!q.exception.isDefined) { + assert(clock.isStreamWaitingAt(clock.getTimeMillis())) + } + } + if (q.exception.isDefined) { + throw q.exception.get + } + true + } + + testStream(mapped)( + StartStream(ProcessingTime(100), clock), + waitUntilBatchProcessed, + // 1 from smallest, 1 from middle, 8 from biggest + CheckAnswer(1, 10, 100, 101, 102, 103, 104, 105, 106, 107), + AdvanceManualClock(100), + waitUntilBatchProcessed, + // smallest now empty, 1 more from middle, 9 more from biggest + CheckAnswer(1, 10, 100, 101, 102, 103, 104, 105, 106, 107, + 11, 108, 109, 110, 111, 112, 113, 114, 115, 116 + ), + StopStream, + StartStream(ProcessingTime(100), clock), + waitUntilBatchProcessed, + // smallest now empty, 1 more from middle, 9 more from biggest + CheckAnswer(1, 10, 100, 101, 102, 103, 104, 105, 106, 107, + 11, 108, 109, 110, 111, 112, 113, 114, 115, 116, + 12, 117, 118, 119, 120, 121, 122, 123, 124, 125 + ), + AdvanceManualClock(100), + waitUntilBatchProcessed, + // smallest now empty, 1 more from middle, 9 more from biggest + CheckAnswer(1, 10, 100, 101, 102, 103, 104, 105, 106, 107, + 11, 108, 109, 110, 111, 112, 113, 114, 115, 116, + 12, 117, 118, 119, 120, 121, 122, 123, 124, 125, + 13, 126, 127, 128, 129, 130, 131, 132, 133, 134 + ) + ) + } + + test("input row metrics") { + val topic = newTopic() + testUtils.createTopic(topic, partitions = 5) + testUtils.sendMessages(topic, Array("-1")) + require(testUtils.getLatestOffsets(Set(topic)).size === 5) + + val kafka = spark + .readStream + .format("kafka") + .option("subscribe", topic) + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + + val mapped = kafka.map(kv => kv._2.toInt + 1) + testStream(mapped)( + StartStream(trigger = ProcessingTime(1)), + makeSureGetOffsetCalled, + AddKafkaData(Set(topic), 1, 2, 3), + CheckAnswer(2, 3, 4), + AssertOnQuery { query => + val recordsRead = query.recentProgress.map(_.numInputRows).sum + recordsRead == 3 + } + ) + } + + test("subscribing topic by pattern with topic deletions") { + val topicPrefix = newTopic() + val topic = topicPrefix + "-seems" + val topic2 = topicPrefix + "-bad" + testUtils.createTopic(topic, partitions = 5) + testUtils.sendMessages(topic, Array("-1")) + require(testUtils.getLatestOffsets(Set(topic)).size === 5) + + val reader = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + .option("subscribePattern", s"$topicPrefix-.*") + .option("failOnDataLoss", "false") + + val kafka = reader.load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + val mapped = kafka.map(kv => kv._2.toInt + 1) + + testStream(mapped)( + makeSureGetOffsetCalled, + AddKafkaData(Set(topic), 1, 2, 3), + CheckAnswer(2, 3, 4), + Assert { + testUtils.deleteTopic(topic) + testUtils.createTopic(topic2, partitions = 5) + true + }, + AddKafkaData(Set(topic2), 4, 5, 6), + CheckAnswer(2, 3, 4, 5, 6, 7) + ) + } +} class KafkaSourceSuite extends KafkaSourceTest { @@ -247,7 +380,7 @@ class KafkaSourceSuite extends KafkaSourceTest { test("(de)serialization of initial offsets") { val topic = newTopic() - testUtils.createTopic(topic, partitions = 64) + testUtils.createTopic(topic, partitions = 5) val reader = spark .readStream @@ -262,70 +395,6 @@ class KafkaSourceSuite extends KafkaSourceTest { StopStream) } - 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 - .readStream - .format("kafka") - .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("kafka.metadata.max.age.ms", "1") - .option("maxOffsetsPerTrigger", 10) - .option("subscribe", topic) - .option("startingOffsets", "earliest") - 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) - - val clock = new StreamManualClock - - val waitUntilBatchProcessed = AssertOnQuery { q => - eventually(Timeout(streamingTimeout)) { - if (!q.exception.isDefined) { - assert(clock.isStreamWaitingAt(clock.getTimeMillis())) - } - } - if (q.exception.isDefined) { - throw q.exception.get - } - true - } - - testStream(mapped)( - StartStream(ProcessingTime(100), clock), - waitUntilBatchProcessed, - // 1 from smallest, 1 from middle, 8 from biggest - CheckAnswer(1, 10, 100, 101, 102, 103, 104, 105, 106, 107), - AdvanceManualClock(100), - waitUntilBatchProcessed, - // smallest now empty, 1 more from middle, 9 more from biggest - CheckAnswer(1, 10, 100, 101, 102, 103, 104, 105, 106, 107, - 11, 108, 109, 110, 111, 112, 113, 114, 115, 116 - ), - StopStream, - StartStream(ProcessingTime(100), clock), - waitUntilBatchProcessed, - // smallest now empty, 1 more from middle, 9 more from biggest - CheckAnswer(1, 10, 100, 101, 102, 103, 104, 105, 106, 107, - 11, 108, 109, 110, 111, 112, 113, 114, 115, 116, - 12, 117, 118, 119, 120, 121, 122, 123, 124, 125 - ), - AdvanceManualClock(100), - waitUntilBatchProcessed, - // smallest now empty, 1 more from middle, 9 more from biggest - CheckAnswer(1, 10, 100, 101, 102, 103, 104, 105, 106, 107, - 11, 108, 109, 110, 111, 112, 113, 114, 115, 116, - 12, 117, 118, 119, 120, 121, 122, 123, 124, 125, - 13, 126, 127, 128, 129, 130, 131, 132, 133, 134 - ) - ) - } - test("cannot stop Kafka stream") { val topic = newTopic() testUtils.createTopic(topic, partitions = 5) @@ -336,7 +405,7 @@ class KafkaSourceSuite extends KafkaSourceTest { .format("kafka") .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.metadata.max.age.ms", "1") - .option("subscribePattern", s"topic-.*") + .option("subscribePattern", s"$topic.*") val kafka = reader.load() .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") @@ -430,41 +499,6 @@ class KafkaSourceSuite extends KafkaSourceTest { } } - test("subscribing topic by pattern with topic deletions") { - val topicPrefix = newTopic() - val topic = topicPrefix + "-seems" - val topic2 = topicPrefix + "-bad" - testUtils.createTopic(topic, partitions = 5) - testUtils.sendMessages(topic, Array("-1")) - require(testUtils.getLatestOffsets(Set(topic)).size === 5) - - val reader = spark - .readStream - .format("kafka") - .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("kafka.metadata.max.age.ms", "1") - .option("subscribePattern", s"$topicPrefix-.*") - .option("failOnDataLoss", "false") - - val kafka = reader.load() - .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") - .as[(String, String)] - val mapped = kafka.map(kv => kv._2.toInt + 1) - - testStream(mapped)( - makeSureGetOffsetCalled, - AddKafkaData(Set(topic), 1, 2, 3), - CheckAnswer(2, 3, 4), - Assert { - testUtils.deleteTopic(topic) - testUtils.createTopic(topic2, partitions = 5) - true - }, - AddKafkaData(Set(topic2), 4, 5, 6), - CheckAnswer(2, 3, 4, 5, 6, 7) - ) - } - test("bad source options") { def testBadOptions(options: (String, String)*)(expectedMsgs: String*): Unit = { val ex = intercept[IllegalArgumentException] { @@ -524,34 +558,6 @@ class KafkaSourceSuite extends KafkaSourceTest { testUnsupportedConfig("kafka.auto.offset.reset", "latest") } - test("input row metrics") { - val topic = newTopic() - testUtils.createTopic(topic, partitions = 5) - testUtils.sendMessages(topic, Array("-1")) - require(testUtils.getLatestOffsets(Set(topic)).size === 5) - - val kafka = spark - .readStream - .format("kafka") - .option("subscribe", topic) - .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .load() - .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") - .as[(String, String)] - - val mapped = kafka.map(kv => kv._2.toInt + 1) - testStream(mapped)( - StartStream(trigger = ProcessingTime(1)), - makeSureGetOffsetCalled, - AddKafkaData(Set(topic), 1, 2, 3), - CheckAnswer(2, 3, 4), - AssertOnQuery { query => - val recordsRead = query.recentProgress.map(_.numInputRows).sum - recordsRead == 3 - } - ) - } - test("delete a topic when a Spark job is running") { KafkaSourceSuite.collectedData.clear() @@ -658,6 +664,10 @@ class KafkaSourceSuite extends KafkaSourceTest { testStream(mapped)( makeSureGetOffsetCalled, + Execute { q => + // wait to reach the last offset in every partition + q.awaitOffset(0, KafkaSourceOffset(partitionOffsets.mapValues(_ => 3L))) + }, CheckAnswer(-20, -21, -22, 0, 1, 2, 11, 12, 22), StopStream, StartStream(), @@ -778,13 +788,12 @@ class KafkaSourceSuite extends KafkaSourceTest { StopStream, AddKafkaData(Set(topic), 4, 5, 6), // Add data when stream is stopped StartStream(), + Execute { q => q.awaitOffset(0, KafkaSourceOffset(testUtils.getLatestOffsets(Set(topic)))) }, CheckAnswer(2, 3, 4, 5, 6, 7), // Should get the added data AddKafkaData(Set(topic), 7, 8), CheckAnswer(2, 3, 4, 5, 6, 7, 8, 9), AssertOnQuery("Add partitions") { query: StreamExecution => - if (addPartitions) { - testUtils.addPartitions(topic, 10) - } + if (addPartitions) setTopicPartitions(topic, 10, query) true }, AddKafkaData(Set(topic), 9, 10, 11, 12, 13, 14, 15, 16), @@ -823,11 +832,10 @@ class KafkaSourceSuite extends KafkaSourceTest { StopStream, AddKafkaData(Set(topic), 7, 8), StartStream(), + Execute { q => q.awaitOffset(0, KafkaSourceOffset(testUtils.getLatestOffsets(Set(topic)))) }, CheckAnswer(2, 3, 4, 5, 6, 7, 8, 9), AssertOnQuery("Add partitions") { query: StreamExecution => - if (addPartitions) { - testUtils.addPartitions(topic, 10) - } + if (addPartitions) setTopicPartitions(topic, 10, query) true }, AddKafkaData(Set(topic), 9, 10, 11, 12, 13, 14, 15, 16), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 8035d97e16e3e..e335a170099e2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -142,7 +142,8 @@ abstract class StreamExecution( override val id: UUID = UUID.fromString(streamMetadata.id) - override val runId: UUID = UUID.randomUUID + override def runId: UUID = currentRunId + protected var currentRunId = UUID.randomUUID /** * Pretty identified string of printing in logs. Format is @@ -422,7 +423,8 @@ abstract class StreamExecution( assertAwaitThread() def notDone = { val localCommittedOffsets = committedOffsets - if (sources.length <= sourceIndex) { + if (sources == null) { + // sources might not be initialized yet false } else { val source = sources(sourceIndex) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala index 14fd52bf4f105..dd7e8935b1ef8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.streaming.continuous +import java.util.UUID import java.util.concurrent.TimeUnit import java.util.function.UnaryOperator @@ -53,7 +54,7 @@ class ContinuousExecution( sparkSession, name, checkpointRoot, analyzedPlan, sink, trigger, triggerClock, outputMode, deleteCheckpointOnStop) { - @volatile protected var continuousSources: Seq[ContinuousReader] = Seq.empty + @volatile protected var continuousSources: Seq[ContinuousReader] = _ override protected def sources: Seq[BaseStreamingSource] = continuousSources override lazy val logicalPlan: LogicalPlan = { @@ -123,12 +124,16 @@ class ContinuousExecution( } committedOffsets = nextOffsets.toStreamProgress(sources) - // Forcibly align commit and offset logs by slicing off any spurious offset logs from - // a previous run. We can't allow commits to an epoch that a previous run reached but - // this run has not. - offsetLog.purgeAfter(latestEpochId) + // Get to an epoch ID that has definitely never been sent to a sink before. Since sink + // commit happens between offset log write and commit log write, this means an epoch ID + // which is not in the offset log. + val (latestOffsetEpoch, _) = offsetLog.getLatest().getOrElse { + throw new IllegalStateException( + s"Offset log had no latest element. This shouldn't be possible because nextOffsets is" + + s"an element.") + } + currentBatchId = latestOffsetEpoch + 1 - currentBatchId = latestEpochId + 1 logDebug(s"Resuming at epoch $currentBatchId with committed offsets $committedOffsets") nextOffsets case None => @@ -144,6 +149,7 @@ class ContinuousExecution( * @param sparkSessionForQuery Isolated [[SparkSession]] to run the continuous query with. */ private def runContinuous(sparkSessionForQuery: SparkSession): Unit = { + currentRunId = UUID.randomUUID // A list of attributes that will need to be updated. val replacements = new ArrayBuffer[(Attribute, Attribute)] // Translate from continuous relation to the underlying data source. @@ -274,10 +280,6 @@ class ContinuousExecution( epoch: Long, reader: ContinuousReader, partitionOffsets: Seq[PartitionOffset]): Unit = { assert(continuousSources.length == 1, "only one continuous source supported currently") - if (partitionOffsets.contains(null)) { - // If any offset is null, that means the corresponding partition hasn't seen any data yet, so - // there's nothing meaningful to add to the offset log. - } val globalOffset = reader.mergeOffsets(partitionOffsets.toArray) val oldOffset = synchronized { offsetLog.add(epoch, OffsetSeq.fill(globalOffset)) @@ -288,6 +290,13 @@ class ContinuousExecution( if (oldOffset.contains(OffsetSeq.fill(globalOffset))) { noNewData = true } + + awaitProgressLock.lock() + try { + awaitProgressLockCondition.signalAll() + } finally { + awaitProgressLock.unlock() + } } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index eeadeea8545e0..6b384ea2eac2e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -81,6 +81,9 @@ trait StreamTest extends QueryTest with SharedSQLContext with TimeLimits with Be StateStore.stop() // stop the state store maintenance thread and unload store providers } + protected val defaultTrigger = Trigger.ProcessingTime(0) + protected val defaultUseV2Sink = false + /** How long to wait for an active stream to catch up when checking a result. */ val streamingTimeout = 10.seconds @@ -190,7 +193,7 @@ trait StreamTest extends QueryTest with SharedSQLContext with TimeLimits with Be /** Starts the stream, resuming if data has already been processed. It must not be running. */ case class StartStream( - trigger: Trigger = Trigger.ProcessingTime(0), + trigger: Trigger = defaultTrigger, triggerClock: Clock = new SystemClock, additionalConfs: Map[String, String] = Map.empty, checkpointLocation: String = null) @@ -277,7 +280,7 @@ trait StreamTest extends QueryTest with SharedSQLContext with TimeLimits with Be def testStream( _stream: Dataset[_], outputMode: OutputMode = OutputMode.Append, - useV2Sink: Boolean = false)(actions: StreamAction*): Unit = synchronized { + useV2Sink: Boolean = defaultUseV2Sink)(actions: StreamAction*): Unit = synchronized { import org.apache.spark.sql.streaming.util.StreamManualClock // `synchronized` is added to prevent the user from calling multiple `testStream`s concurrently @@ -468,7 +471,7 @@ trait StreamTest extends QueryTest with SharedSQLContext with TimeLimits with Be try { currentStream.awaitInitialization(streamingTimeout.toMillis) currentStream match { - case s: ContinuousExecution => s.awaitEpoch(0) + case s: ContinuousExecution => s.processAllAvailable() case _ => } } catch { @@ -620,7 +623,7 @@ trait StreamTest extends QueryTest with SharedSQLContext with TimeLimits with Be } }.getOrElse { throw new IllegalArgumentException( - "Could find index of the source to which data was added") + "Could not find index of the source to which data was added") } // Store the expected offset of added data to wait for it later From 04cfae6c11fdfa6b9e4027f5f7d2aded843126d3 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Mon, 8 Jan 2018 16:10:14 -0800 Subject: [PATCH 30/53] stress test --- .../sql/kafka010/KafkaContinuousSuite.scala | 117 +++++++++++++++++- .../spark/sql/kafka010/KafkaSourceSuite.scala | 2 - 2 files changed, 115 insertions(+), 4 deletions(-) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala index ef14768e51c91..a1f65b4affc7d 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.kafka010 import java.util.Properties +import java.util.concurrent.atomic.AtomicInteger import org.scalatest.time.SpanSugar._ import scala.collection.mutable @@ -153,6 +154,118 @@ class KafkaContinuousSourceSuite extends KafkaSourceSuite with KafkaContinuousTe } } -/* class KafkaContinuousStressSuite extends KafkaSourceStressSuite with KafkaContinuousTest { +class KafkaContinuousSourceStressSuite + extends StreamTest with SharedSQLContext { -} */ + import testImplicits._ + + private var testUtils: KafkaTestUtils = _ + + private val topicId = new AtomicInteger(0) + + private def newTopic(): String = s"failOnDataLoss-${topicId.getAndIncrement()}" + + override def createSparkSession(): TestSparkSession = { + // Set maxRetries to 3 to handle NPE from `poll` when deleting a topic + new TestSparkSession(new SparkContext("local[2,3]", "test-sql-context", sparkConf)) + } + + override def beforeAll(): Unit = { + super.beforeAll() + testUtils = new KafkaTestUtils { + override def brokerConfiguration: Properties = { + val props = super.brokerConfiguration + // Try to make Kafka clean up messages as fast as possible. However, there is a hard-code + // 30 seconds delay (kafka.log.LogManager.InitialTaskDelayMs) so this test should run at + // least 30 seconds. + props.put("log.cleaner.backoff.ms", "100") + props.put("log.segment.bytes", "40") + props.put("log.retention.bytes", "40") + props.put("log.retention.check.interval.ms", "100") + props.put("delete.retention.ms", "10") + props.put("log.flush.scheduler.interval.ms", "10") + props + } + } + testUtils.setup() + } + + override def afterAll(): Unit = { + if (testUtils != null) { + testUtils.teardown() + testUtils = null + super.afterAll() + } + } + + test("stress test for failOnDataLoss=false") { + val reader = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + .option("subscribePattern", "failOnDataLoss.*") + .option("startingOffsets", "earliest") + .option("failOnDataLoss", "false") + .option("fetchOffset.retryIntervalMs", "3000") + val kafka = reader.load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + val query = kafka.map(kv => kv._2.toInt).writeStream + .format("memory") + .queryName("memory") + .start() + + val testTime = 1.minutes + val startTime = System.currentTimeMillis() + // Track the current existing topics + val topics = mutable.ArrayBuffer[String]() + // Track topics that have been deleted + val deletedTopics = mutable.Set[String]() + while (System.currentTimeMillis() - testTime.toMillis < startTime) { + Random.nextInt(10) match { + case 0 => // Create a new topic + val topic = newTopic() + topics += topic + // As pushing messages into Kafka updates Zookeeper asynchronously, there is a small + // chance that a topic will be recreated after deletion due to the asynchronous update. + // Hence, always overwrite to handle this race condition. + testUtils.createTopic(topic, partitions = 1, overwrite = true) + logInfo(s"Create topic $topic") + case 1 if topics.nonEmpty => // Delete an existing topic + val topic = topics.remove(Random.nextInt(topics.size)) + testUtils.deleteTopic(topic) + logInfo(s"Delete topic $topic") + deletedTopics += topic + case 2 if deletedTopics.nonEmpty => // Recreate a topic that was deleted. + val topic = deletedTopics.toSeq(Random.nextInt(deletedTopics.size)) + deletedTopics -= topic + topics += topic + // As pushing messages into Kafka updates Zookeeper asynchronously, there is a small + // chance that a topic will be recreated after deletion due to the asynchronous update. + // Hence, always overwrite to handle this race condition. + testUtils.createTopic(topic, partitions = 1, overwrite = true) + logInfo(s"Create topic $topic") + case 3 => + Thread.sleep(1000) + case _ => // Push random messages + for (topic <- topics) { + val size = Random.nextInt(10) + for (_ <- 0 until size) { + testUtils.sendMessages(topic, Array(Random.nextInt(10).toString)) + } + } + } + // `failOnDataLoss` is `false`, we should not fail the query + if (query.exception.nonEmpty) { + throw query.exception.get + } + } + + query.stop() + // `failOnDataLoss` is `false`, we should not fail the query + if (query.exception.nonEmpty) { + throw query.exception.get + } + } +} 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 44394ab5ad246..37d2282a62960 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 @@ -788,7 +788,6 @@ class KafkaSourceSuite extends KafkaSourceTest { StopStream, AddKafkaData(Set(topic), 4, 5, 6), // Add data when stream is stopped StartStream(), - Execute { q => q.awaitOffset(0, KafkaSourceOffset(testUtils.getLatestOffsets(Set(topic)))) }, CheckAnswer(2, 3, 4, 5, 6, 7), // Should get the added data AddKafkaData(Set(topic), 7, 8), CheckAnswer(2, 3, 4, 5, 6, 7, 8, 9), @@ -832,7 +831,6 @@ class KafkaSourceSuite extends KafkaSourceTest { StopStream, AddKafkaData(Set(topic), 7, 8), StartStream(), - Execute { q => q.awaitOffset(0, KafkaSourceOffset(testUtils.getLatestOffsets(Set(topic)))) }, CheckAnswer(2, 3, 4, 5, 6, 7, 8, 9), AssertOnQuery("Add partitions") { query: StreamExecution => if (addPartitions) setTopicPartitions(topic, 10, query) From 3e5b78716e97eae822db64b5fd1764f9387fb94e Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Mon, 8 Jan 2018 16:30:56 -0800 Subject: [PATCH 31/53] comment out deletion test for now --- .../org/apache/spark/sql/kafka010/KafkaContinuousSuite.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/KafkaContinuousSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala index a1f65b4affc7d..9341cb82d23d5 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala @@ -109,7 +109,7 @@ class KafkaContinuousSourceSuite extends KafkaSourceSuite with KafkaContinuousTe } } - test("subscribing topic by pattern with topic deletions") { + /* test("subscribing topic by pattern with topic deletions") { val topicPrefix = newTopic() val topic = topicPrefix + "-seems" val topic2 = topicPrefix + "-bad" @@ -151,7 +151,7 @@ class KafkaContinuousSourceSuite extends KafkaSourceSuite with KafkaContinuousTe AddKafkaData(Set(topic2), 4, 5, 6), CheckAnswer(2, 3, 4, 5, 6, 7) ) - } + } */ } class KafkaContinuousSourceStressSuite From 34c0a90ea7934a6a3918cc4907ddcc87cc8a1bea Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Mon, 8 Jan 2018 16:44:04 -0800 Subject: [PATCH 32/53] refactor stress test --- .../sql/kafka010/KafkaContinuousSuite.scala | 116 +----------------- .../spark/sql/kafka010/KafkaSourceSuite.scala | 34 ++--- 2 files changed, 24 insertions(+), 126 deletions(-) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala index 9341cb82d23d5..bf15e8b30aed7 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala @@ -25,7 +25,7 @@ import scala.collection.mutable import scala.util.Random import org.apache.spark.SparkContext -import org.apache.spark.sql.{ForeachWriter, Row} +import org.apache.spark.sql.{DataFrame, Dataset, ForeachWriter, Row} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.execution.streaming.StreamExecution import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution @@ -154,118 +154,12 @@ class KafkaContinuousSourceSuite extends KafkaSourceSuite with KafkaContinuousTe } */ } -class KafkaContinuousSourceStressSuite - extends StreamTest with SharedSQLContext { - - import testImplicits._ - - private var testUtils: KafkaTestUtils = _ - - private val topicId = new AtomicInteger(0) - - private def newTopic(): String = s"failOnDataLoss-${topicId.getAndIncrement()}" - - override def createSparkSession(): TestSparkSession = { - // Set maxRetries to 3 to handle NPE from `poll` when deleting a topic - new TestSparkSession(new SparkContext("local[2,3]", "test-sql-context", sparkConf)) - } - - override def beforeAll(): Unit = { - super.beforeAll() - testUtils = new KafkaTestUtils { - override def brokerConfiguration: Properties = { - val props = super.brokerConfiguration - // Try to make Kafka clean up messages as fast as possible. However, there is a hard-code - // 30 seconds delay (kafka.log.LogManager.InitialTaskDelayMs) so this test should run at - // least 30 seconds. - props.put("log.cleaner.backoff.ms", "100") - props.put("log.segment.bytes", "40") - props.put("log.retention.bytes", "40") - props.put("log.retention.check.interval.ms", "100") - props.put("delete.retention.ms", "10") - props.put("log.flush.scheduler.interval.ms", "10") - props - } - } - testUtils.setup() - } - - override def afterAll(): Unit = { - if (testUtils != null) { - testUtils.teardown() - testUtils = null - super.afterAll() - } - } - - test("stress test for failOnDataLoss=false") { - val reader = spark - .readStream - .format("kafka") - .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("kafka.metadata.max.age.ms", "1") - .option("subscribePattern", "failOnDataLoss.*") - .option("startingOffsets", "earliest") - .option("failOnDataLoss", "false") - .option("fetchOffset.retryIntervalMs", "3000") - val kafka = reader.load() - .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") - .as[(String, String)] - val query = kafka.map(kv => kv._2.toInt).writeStream +class KafkaContinuousSourceStressForDontFailOnDataLossSuite + extends KafkaSourceStressForDontFailOnDataLossSuite { + override protected def startStream(ds: Dataset[Int]) = { + ds.writeStream .format("memory") .queryName("memory") .start() - - val testTime = 1.minutes - val startTime = System.currentTimeMillis() - // Track the current existing topics - val topics = mutable.ArrayBuffer[String]() - // Track topics that have been deleted - val deletedTopics = mutable.Set[String]() - while (System.currentTimeMillis() - testTime.toMillis < startTime) { - Random.nextInt(10) match { - case 0 => // Create a new topic - val topic = newTopic() - topics += topic - // As pushing messages into Kafka updates Zookeeper asynchronously, there is a small - // chance that a topic will be recreated after deletion due to the asynchronous update. - // Hence, always overwrite to handle this race condition. - testUtils.createTopic(topic, partitions = 1, overwrite = true) - logInfo(s"Create topic $topic") - case 1 if topics.nonEmpty => // Delete an existing topic - val topic = topics.remove(Random.nextInt(topics.size)) - testUtils.deleteTopic(topic) - logInfo(s"Delete topic $topic") - deletedTopics += topic - case 2 if deletedTopics.nonEmpty => // Recreate a topic that was deleted. - val topic = deletedTopics.toSeq(Random.nextInt(deletedTopics.size)) - deletedTopics -= topic - topics += topic - // As pushing messages into Kafka updates Zookeeper asynchronously, there is a small - // chance that a topic will be recreated after deletion due to the asynchronous update. - // Hence, always overwrite to handle this race condition. - testUtils.createTopic(topic, partitions = 1, overwrite = true) - logInfo(s"Create topic $topic") - case 3 => - Thread.sleep(1000) - case _ => // Push random messages - for (topic <- topics) { - val size = Random.nextInt(10) - for (_ <- 0 until size) { - testUtils.sendMessages(topic, Array(Random.nextInt(10).toString)) - } - } - } - // `failOnDataLoss` is `false`, we should not fail the query - if (query.exception.nonEmpty) { - throw query.exception.get - } - } - - query.stop() - // `failOnDataLoss` is `false`, we should not fail the query - if (query.exception.nonEmpty) { - throw query.exception.get - } } } 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 37d2282a62960..b64c7ce02854a 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 @@ -34,7 +34,7 @@ import org.scalatest.concurrent.PatienceConfiguration.Timeout import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkContext -import org.apache.spark.sql.{ForeachWriter, Row} +import org.apache.spark.sql.{DataFrame, Dataset, ForeachWriter, Row} import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, WriteToDataSourceV2Exec} import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution @@ -965,20 +965,8 @@ class KafkaSourceStressForDontFailOnDataLossSuite extends StreamTest with Shared } } - test("stress test for failOnDataLoss=false") { - val reader = spark - .readStream - .format("kafka") - .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("kafka.metadata.max.age.ms", "1") - .option("subscribePattern", "failOnDataLoss.*") - .option("startingOffsets", "earliest") - .option("failOnDataLoss", "false") - .option("fetchOffset.retryIntervalMs", "3000") - val kafka = reader.load() - .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") - .as[(String, String)] - val query = kafka.map(kv => kv._2.toInt).writeStream.foreach(new ForeachWriter[Int] { + protected def startStream(ds: Dataset[Int]) = { + ds.writeStream.foreach(new ForeachWriter[Int] { override def open(partitionId: Long, version: Long): Boolean = { true @@ -992,6 +980,22 @@ class KafkaSourceStressForDontFailOnDataLossSuite extends StreamTest with Shared override def close(errorOrNull: Throwable): Unit = { } }).start() + } + + test("stress test for failOnDataLoss=false") { + val reader = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + .option("subscribePattern", "failOnDataLoss.*") + .option("startingOffsets", "earliest") + .option("failOnDataLoss", "false") + .option("fetchOffset.retryIntervalMs", "3000") + val kafka = reader.load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + val query = startStream(kafka.map(kv => kv._2.toInt)) val testTime = 1.minutes val startTime = System.currentTimeMillis() From 265a11819d9b74c28300167ca291632073558afe Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Mon, 8 Jan 2018 17:09:56 -0800 Subject: [PATCH 33/53] refactor main tests --- .../sql/kafka010/KafkaContinuousReader.scala | 2 +- .../sql/kafka010/KafkaContinuousSuite.scala | 6 +- .../spark/sql/kafka010/KafkaSourceSuite.scala | 95 ++++++++++--------- 3 files changed, 52 insertions(+), 51 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala index 8dd299cc8645d..d5969928cd16d 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala @@ -205,7 +205,7 @@ class KafkaContinuousDataReader( bufferHolder.reset() if (currentRecord.key == null) { - rowWriter.isNullAt(0) + rowWriter.setNullAt(0) } else { rowWriter.write(0, currentRecord.key) } diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala index bf15e8b30aed7..83e8843ac7887 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala @@ -68,11 +68,11 @@ trait KafkaContinuousTest extends KafkaSourceTest { } } -class KafkaContinuousSourceSuite extends KafkaSourceSuite with KafkaContinuousTest { +class KafkaContinuousSourceSuite extends KafkaSourceSuiteBase with KafkaContinuousTest { import testImplicits._ - test("kafka sink") { + /* test("kafka sink") { withTempDir { dir => val topic = newTopic() testUtils.createTopic(topic) @@ -107,7 +107,7 @@ class KafkaContinuousSourceSuite extends KafkaSourceSuite with KafkaContinuousTe query.stop() query.awaitTermination() } - } + } */ /* test("subscribing topic by pattern with topic deletions") { val topicPrefix = newTopic() 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 b64c7ce02854a..2b8284e37327d 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 @@ -151,7 +151,7 @@ abstract class KafkaSourceTest extends StreamTest with SharedSQLContext { protected def newTopic(): String = s"topic-${topicId.getAndIncrement()}" } -class KafkaMicroBatchSourceSuite extends KafkaSourceTest { +class KafkaMicroBatchSourceSuite extends KafkaSourceSuiteBase { import testImplicits._ @@ -281,11 +281,6 @@ class KafkaMicroBatchSourceSuite extends KafkaSourceTest { CheckAnswer(2, 3, 4, 5, 6, 7) ) } -} - -class KafkaSourceSuite extends KafkaSourceTest { - - import testImplicits._ testWithUninterruptibleThread( "deserialization of initial offset with Spark 2.1.0") { @@ -378,6 +373,52 @@ class KafkaSourceSuite extends KafkaSourceTest { } } + test("KafkaSource with watermark") { + val now = System.currentTimeMillis() + val topic = newTopic() + testUtils.createTopic(newTopic(), partitions = 1) + testUtils.sendMessages(topic, Array(1).map(_.toString)) + + val kafka = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + .option("startingOffsets", s"earliest") + .option("subscribe", topic) + .load() + + val windowedAggregation = kafka + .withWatermark("timestamp", "10 seconds") + .groupBy(window($"timestamp", "5 seconds") as 'window) + .agg(count("*") as 'count) + .select($"window".getField("start") as 'window, $"count") + + val query = windowedAggregation + .writeStream + .format("memory") + .outputMode("complete") + .queryName("kafkaWatermark") + .start() + query.processAllAvailable() + val rows = spark.table("kafkaWatermark").collect() + assert(rows.length === 1, s"Unexpected results: ${rows.toList}") + val row = rows(0) + // We cannot check the exact window start time as it depands on the time that messages were + // inserted by the producer. So here we just use a low bound to make sure the internal + // conversion works. + assert( + row.getAs[java.sql.Timestamp]("window").getTime >= now - 5 * 1000, + s"Unexpected results: $row") + assert(row.getAs[Int]("count") === 1, s"Unexpected results: $row") + query.stop() + } +} + +class KafkaSourceSuiteBase extends KafkaSourceTest { + + import testImplicits._ + test("(de)serialization of initial offsets") { val topic = newTopic() testUtils.createTopic(topic, partitions = 5) @@ -698,6 +739,7 @@ class KafkaSourceSuite extends KafkaSourceTest { .format("memory") .outputMode("append") .queryName("kafkaColumnTypes") + .trigger(defaultTrigger) .start() query.processAllAvailable() val rows = spark.table("kafkaColumnTypes").collect() @@ -715,47 +757,6 @@ class KafkaSourceSuite extends KafkaSourceTest { query.stop() } - test("KafkaSource with watermark") { - val now = System.currentTimeMillis() - val topic = newTopic() - testUtils.createTopic(newTopic(), partitions = 1) - testUtils.sendMessages(topic, Array(1).map(_.toString)) - - val kafka = spark - .readStream - .format("kafka") - .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("kafka.metadata.max.age.ms", "1") - .option("startingOffsets", s"earliest") - .option("subscribe", topic) - .load() - - val windowedAggregation = kafka - .withWatermark("timestamp", "10 seconds") - .groupBy(window($"timestamp", "5 seconds") as 'window) - .agg(count("*") as 'count) - .select($"window".getField("start") as 'window, $"count") - - val query = windowedAggregation - .writeStream - .format("memory") - .outputMode("complete") - .queryName("kafkaWatermark") - .start() - query.processAllAvailable() - val rows = spark.table("kafkaWatermark").collect() - assert(rows.length === 1, s"Unexpected results: ${rows.toList}") - val row = rows(0) - // We cannot check the exact window start time as it depands on the time that messages were - // inserted by the producer. So here we just use a low bound to make sure the internal - // conversion works. - assert( - row.getAs[java.sql.Timestamp]("window").getTime >= now - 5 * 1000, - s"Unexpected results: $row") - assert(row.getAs[Int]("count") === 1, s"Unexpected results: $row") - query.stop() - } - private def testFromLatestOffsets( topic: String, addPartitions: Boolean, From 518c36a009205395b3b352c4337e550bfa098a5a Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Mon, 8 Jan 2018 17:10:44 -0800 Subject: [PATCH 34/53] rm wakeup --- .../org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala | 4 ---- 1 file changed, 4 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 e251f0d7d804b..90ed7b1fba2f8 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 @@ -261,10 +261,6 @@ private[kafka010] case class CachedKafkaConsumer private( } } - def wakeup(): Unit = { - consumer.wakeup() - } - /** Create a new consumer and reset cached states */ private def resetConsumer(): Unit = { consumer.close() From e8cab9587088145bce1febc70fed12445ccb7014 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Mon, 8 Jan 2018 17:12:17 -0800 Subject: [PATCH 35/53] java.util => ju --- .../spark/sql/kafka010/KafkaContinuousReader.scala | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala index d5969928cd16d..928379544758c 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.kafka010 +import java.{util => ju} + import org.apache.kafka.clients.consumer.ConsumerRecord import org.apache.kafka.common.TopicPartition @@ -47,7 +49,7 @@ import org.apache.spark.unsafe.types.UTF8String */ class KafkaContinuousReader( offsetReader: KafkaOffsetReader, - kafkaParams: java.util.Map[String, Object], + kafkaParams: ju.Map[String, Object], sourceOptions: Map[String, String], metadataPath: String, initialOffsets: KafkaOffsetRangeLimit, @@ -65,7 +67,7 @@ class KafkaContinuousReader( override def readSchema: StructType = KafkaOffsetReader.kafkaSchema private var offset: Offset = _ - override def setOffset(start: java.util.Optional[Offset]): Unit = { + override def setOffset(start: ju.Optional[Offset]): Unit = { offset = start.orElse { val offsets = initialOffsets match { case EarliestOffsetRangeLimit => KafkaSourceOffset(offsetReader.fetchEarliestOffsets()) @@ -83,7 +85,7 @@ class KafkaContinuousReader( KafkaSourceOffset(JsonUtils.partitionOffsets(json)) } - override def createUnsafeRowReadTasks(): java.util.List[ReadTask[UnsafeRow]] = { + override def createUnsafeRowReadTasks(): ju.List[ReadTask[UnsafeRow]] = { import scala.collection.JavaConverters._ val oldStartPartitionOffsets = KafkaSourceOffset.getPartitionOffsets(offset) @@ -155,7 +157,7 @@ class KafkaContinuousReader( case class KafkaContinuousReadTask( topicPartition: TopicPartition, startOffset: Long, - kafkaParams: java.util.Map[String, Object], + kafkaParams: ju.Map[String, Object], failOnDataLoss: Boolean) extends ReadTask[UnsafeRow] { override def createDataReader(): KafkaContinuousDataReader = { new KafkaContinuousDataReader(topicPartition, startOffset, kafkaParams, failOnDataLoss) @@ -174,7 +176,7 @@ case class KafkaContinuousReadTask( class KafkaContinuousDataReader( topicPartition: TopicPartition, startOffset: Long, - kafkaParams: java.util.Map[String, Object], + kafkaParams: ju.Map[String, Object], failOnDataLoss: Boolean) extends ContinuousDataReader[UnsafeRow] { private val topic = topicPartition.topic private val kafkaPartition = topicPartition.partition From 9e2a16baf995939b64a4af90b22323e6a36f8d05 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Mon, 8 Jan 2018 17:12:42 -0800 Subject: [PATCH 36/53] fix indent --- .../scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 7a42e12f2418d..551641cfdbca8 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 @@ -157,7 +157,7 @@ private[kafka010] class KafkaOffsetReader( s"startingOffsets for $tp was $off but consumer reset to ${fetched(tp)}") } case _ => - // no real way to check that beginning or end is reasonable + // no real way to check that beginning or end is reasonable } KafkaSourceOffset(fetched) } From cc49d2f5820264a453ec068aa0a5121949fd07e7 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Mon, 8 Jan 2018 17:13:48 -0800 Subject: [PATCH 37/53] row writer abstract --- .../scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 cee69a4759d2a..baa60febf661d 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 @@ -58,7 +58,7 @@ private[kafka010] class KafkaWriteTask( } } -private[kafka010] class KafkaRowWriter( +private[kafka010] abstract class KafkaRowWriter( inputSchema: Seq[Attribute], topic: Option[String]) { // used to synchronize with Kafka callbacks From 2155acca291afc3e86ab31f56fdd2c95c236f7b6 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Mon, 8 Jan 2018 17:14:39 -0800 Subject: [PATCH 38/53] check err early --- .../org/apache/spark/sql/kafka010/KafkaContinuousWriter.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousWriter.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousWriter.scala index 49ac9145e7f99..ff7f2a765d727 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousWriter.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousWriter.scala @@ -95,6 +95,8 @@ class KafkaContinuousDataWriter( def commit(): WriterCommitMessage = { // Send is asynchronous, but we can't commit until all rows are actually in Kafka. // This requires flushing and then checking that no callbacks produced errors. + // We also check for errors before to fail as soon as possible - the check is cheap. + checkForErrors() producer.flush() checkForErrors() KafkaWriterCommitMessage From b18072fe61e77991a3cf086491bc85b48435a3f2 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Mon, 8 Jan 2018 21:01:32 -0800 Subject: [PATCH 39/53] sink tests --- .../sql/kafka010/KafkaContinuousWriter.scala | 4 + .../kafka010/KafkaContinuousSinkSuite.scala | 340 ++++++++++++++++++ .../sql/kafka010/KafkaContinuousSuite.scala | 38 -- 3 files changed, 344 insertions(+), 38 deletions(-) create mode 100644 external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSinkSuite.scala diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousWriter.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousWriter.scala index ff7f2a765d727..9843f469c5b25 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousWriter.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousWriter.scala @@ -18,12 +18,14 @@ package org.apache.spark.sql.kafka010 import org.apache.kafka.clients.producer.{Callback, ProducerRecord, RecordMetadata} +import scala.collection.JavaConverters._ import org.apache.spark.internal.Logging import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Literal, UnsafeProjection} import org.apache.spark.sql.kafka010.KafkaSourceProvider.{kafkaParamsForProducer, TOPIC_OPTION_KEY} +import org.apache.spark.sql.kafka010.KafkaWriter.validateQuery import org.apache.spark.sql.sources.v2.streaming.writer.ContinuousWriter import org.apache.spark.sql.sources.v2.writer._ import org.apache.spark.sql.streaming.OutputMode @@ -46,6 +48,8 @@ class KafkaContinuousWriter( topic: Option[String], producerParams: Map[String, String], schema: StructType) extends ContinuousWriter with SupportsWriteInternalRow { + validateQuery(schema.toAttributes, producerParams.toMap[String, Object].asJava, topic) + override def createInternalRowWriterFactory(): KafkaContinuousWriterFactory = KafkaContinuousWriterFactory(topic, producerParams, schema) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSinkSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSinkSuite.scala new file mode 100644 index 0000000000000..71e1deeba595c --- /dev/null +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSinkSuite.scala @@ -0,0 +1,340 @@ +/* + * 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.Locale +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 scala.collection.JavaConverters._ + +import org.apache.spark.sql.{AnalysisException, DataFrame, Row, SaveMode} +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, SpecificInternalRow, UnsafeProjection} +import org.apache.spark.sql.streaming._ +import org.apache.spark.sql.types.{BinaryType, DataType} +import org.apache.spark.util.Utils + +/** + * This is a temporary port of KafkaSinkSuite, since we do not yet have a V2 memory stream. + * Once we have one, this will be changed to a specialization of KafkaSinkSuite and we won't have + * to duplicate all the code. + */ +class KafkaContinuousSinkSuite extends KafkaContinuousTest { + import testImplicits._ + + override val streamingTimeout = 30.seconds + + override def beforeAll(): Unit = { + super.beforeAll() + testUtils = new KafkaTestUtils( + withBrokerProps = Map("auto.create.topics.enable" -> "false")) + testUtils.setup() + } + + override def afterAll(): Unit = { + if (testUtils != null) { + testUtils.teardown() + testUtils = null + super.afterAll() + } + } + + test("streaming - write to kafka with topic field") { + val inputTopic = newTopic() + testUtils.createTopic(inputTopic, partitions = 1) + + val input = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("subscribe", inputTopic) + .option("startingOffsets", "earliest") + .load() + + 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 { + testUtils.sendMessages(inputTopic, Array("1", "2", "3", "4", "5")) + failAfter(streamingTimeout) { + writer.processAllAvailable() + } + checkDatasetUnorderly(reader, 1, 2, 3, 4, 5) + testUtils.sendMessages(inputTopic, Array("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("streaming - write data with bad schema") { + val inputTopic = newTopic() + testUtils.createTopic(inputTopic, partitions = 1) + + val input = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("subscribe", inputTopic) + .option("startingOffsets", "earliest") + .load() + 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" + ) + testUtils.sendMessages(inputTopic, Array("1", "2", "3", "4", "5")) + writer.processAllAvailable() + } + } finally { + writer.stop() + } + assert(ex.getMessage + .toLowerCase(Locale.ROOT) + .contains("topic option required when no 'topic' attribute is present")) + + try { + /* No value field */ + ex = intercept[StreamingQueryException] { + writer = createKafkaWriter(input.toDF())( + withSelectExpr = s"'$topic' as topic", "value as key" + ) + testUtils.sendMessages(inputTopic, Array("1", "2", "3", "4", "5")) + writer.processAllAvailable() + } + } finally { + writer.stop() + } + assert(ex.getMessage.toLowerCase(Locale.ROOT).contains( + "required attribute 'value' not found")) + } + + test("streaming - write data with valid schema but wrong types") { + val inputTopic = newTopic() + testUtils.createTopic(inputTopic, partitions = 1) + + val input = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("subscribe", inputTopic) + .option("startingOffsets", "earliest") + .load() + .selectExpr("CAST(value as STRING) value") + val topic = newTopic() + testUtils.createTopic(topic) + + 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" + ) + testUtils.sendMessages(inputTopic, Array("1", "2", "3", "4", "5")) + writer.processAllAvailable() + } + } finally { + writer.stop() + } + assert(ex.getMessage.toLowerCase(Locale.ROOT).contains("topic type must be a string")) + + try { + /* value field wrong type */ + ex = intercept[StreamingQueryException] { + writer = createKafkaWriter(input.toDF())( + withSelectExpr = s"'$topic' as topic", "CAST(value as INT) as value" + ) + testUtils.sendMessages(inputTopic, Array("1", "2", "3", "4", "5")) + writer.processAllAvailable() + } + } finally { + writer.stop() + } + assert(ex.getMessage.toLowerCase(Locale.ROOT).contains( + "value attribute type must be a string or binarytype")) + + try { + ex = intercept[StreamingQueryException] { + /* key field wrong type */ + writer = createKafkaWriter(input.toDF())( + withSelectExpr = s"'$topic' as topic", "CAST(value as INT) as key", "value" + ) + testUtils.sendMessages(inputTopic, Array("1", "2", "3", "4", "5")) + writer.processAllAvailable() + } + } finally { + writer.stop() + } + assert(ex.getMessage.toLowerCase(Locale.ROOT).contains( + "key attribute type must be a string or binarytype")) + } + + test("streaming - write to non-existing topic") { + val inputTopic = newTopic() + testUtils.createTopic(inputTopic, partitions = 1) + + val input = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("subscribe", inputTopic) + .option("startingOffsets", "earliest") + .load() + val topic = newTopic() + + var writer: StreamingQuery = null + var ex: Exception = null + try { + ex = intercept[StreamingQueryException] { + writer = createKafkaWriter(input.toDF(), withTopic = Some(topic))() + testUtils.sendMessages(inputTopic, Array("1", "2", "3", "4", "5")) + eventually(timeout(streamingTimeout)) { + assert(writer.exception.isDefined) + } + throw writer.exception.get + } + } finally { + writer.stop() + } + assert(ex.getMessage.toLowerCase(Locale.ROOT).contains("job aborted")) + } + + test("streaming - exception on config serializer") { + val inputTopic = newTopic() + testUtils.createTopic(inputTopic, partitions = 1) + testUtils.sendMessages(inputTopic, Array("0")) + + val input = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("subscribe", inputTopic) + .load() + var writer: StreamingQuery = null + var ex: Exception = null + ex = intercept[StreamingQueryException] { + writer = createKafkaWriter( + input.toDF(), + withOptions = Map("kafka.key.serializer" -> "foo"))() + writer.processAllAvailable() + } + assert(ex.getMessage.toLowerCase(Locale.ROOT).contains( + "kafka option 'key.serializer' is not supported")) + + ex = intercept[StreamingQueryException] { + writer = createKafkaWriter( + input.toDF(), + withOptions = Map("kafka.value.serializer" -> "foo"))() + writer.processAllAvailable() + } + assert(ex.getMessage.toLowerCase(Locale.ROOT).contains( + "kafka option 'value.serializer' is not supported")) + } + + 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, String] + 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 KafkaContinuousDataWriter(Some(topic), options.asScala.toMap, inputSchema) + 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 + iter.foreach(writeTask.write(_)) + writeTask.commit() + } finally { + writeTask.close() + } + } + + 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 + val checkpointDir = Utils.createTempDir() + 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) + // We need to reduce blocking time to efficiently test non-existent partition behavior. + .option("kafka.max.block.ms", "1000") + .trigger(Trigger.Continuous(1000)) + .queryName("kafkaStream") + withTopic.foreach(stream.option("topic", _)) + withOutputMode.foreach(stream.outputMode(_)) + withOptions.foreach(opt => stream.option(opt._1, opt._2)) + stream.start() + } +} diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala index 83e8843ac7887..acf6e12fec247 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala @@ -71,44 +71,6 @@ trait KafkaContinuousTest extends KafkaSourceTest { class KafkaContinuousSourceSuite extends KafkaSourceSuiteBase with KafkaContinuousTest { import testImplicits._ - - /* test("kafka sink") { - withTempDir { dir => - val topic = newTopic() - testUtils.createTopic(topic) - val query = spark.readStream - .format("rate") - .option("numPartitions", "6") - .option("rowsPerSecond", "10") - .load() - .select('value) - .selectExpr("CAST(value as STRING) value") - .writeStream - .format("kafka") - .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("topic", topic) - .option("checkpointLocation", dir.getCanonicalPath) - .trigger(Trigger.Continuous(100)) - .start() - - eventually(timeout(streamingTimeout)) { - val results = spark.read - .format("kafka") - .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("startingOffsets", "earliest") - .option("endingOffsets", "latest") - .option("subscribe", topic) - .load() - .selectExpr("CAST(value as STRING) value") - .selectExpr("CAST(value as INT) value") - .collect() - assert(Range(0, 20).map(Row(_)).toSet.subsetOf(results.toSet)) - } - query.stop() - query.awaitTermination() - } - } */ - /* test("subscribing topic by pattern with topic deletions") { val topicPrefix = newTopic() val topic = topicPrefix + "-seems" From 22615669cc20cda77819786df4ff34aab925a958 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Mon, 8 Jan 2018 21:09:54 -0800 Subject: [PATCH 40/53] delete topic test --- .../spark/sql/kafka010/KafkaContinuousSuite.scala | 10 ++++++++-- .../apache/spark/sql/kafka010/KafkaSourceSuite.scala | 4 +++- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala index acf6e12fec247..8a81b3e20b8ff 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala @@ -70,8 +70,14 @@ trait KafkaContinuousTest extends KafkaSourceTest { class KafkaContinuousSourceSuite extends KafkaSourceSuiteBase with KafkaContinuousTest { +} + +class KafkaContinuousSourceTopicDeletionSuite extends KafkaContinuousTest { import testImplicits._ - /* test("subscribing topic by pattern with topic deletions") { + + override val brokerProps = Map("auto.create.topics.enable" -> "false") + + test("subscribing topic by pattern with topic deletions") { val topicPrefix = newTopic() val topic = topicPrefix + "-seems" val topic2 = topicPrefix + "-bad" @@ -113,7 +119,7 @@ class KafkaContinuousSourceSuite extends KafkaSourceSuiteBase with KafkaContinuo AddKafkaData(Set(topic2), 4, 5, 6), CheckAnswer(2, 3, 4, 5, 6, 7) ) - } */ + } } class KafkaContinuousSourceStressForDontFailOnDataLossSuite 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 2b8284e37327d..8d16f87291575 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 @@ -52,9 +52,11 @@ abstract class KafkaSourceTest extends StreamTest with SharedSQLContext { override val streamingTimeout = 30.seconds + protected val brokerProps = Map[String, Object]() + override def beforeAll(): Unit = { super.beforeAll() - testUtils = new KafkaTestUtils + testUtils = new KafkaTestUtils(brokerProps) testUtils.setup() } From 341fb20aa4d18f6964d27c87b48822588dfb1833 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Mon, 8 Jan 2018 21:31:55 -0800 Subject: [PATCH 41/53] v2 offset --- .../org/apache/spark/sql/kafka010/KafkaSourceOffset.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala index 59bd5af398098..c82154cfbad7f 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala @@ -19,15 +19,15 @@ package org.apache.spark.sql.kafka010 import org.apache.kafka.common.TopicPartition -import org.apache.spark.sql.execution.streaming.{Offset => OffsetV1, SerializedOffset} -import org.apache.spark.sql.sources.v2.streaming.reader.{Offset, PartitionOffset} +import org.apache.spark.sql.execution.streaming.{Offset, SerializedOffset} +import org.apache.spark.sql.sources.v2.streaming.reader.{Offset => OffsetV2, PartitionOffset} /** * An [[Offset]] for the [[KafkaSource]]. This one tracks all partitions of subscribed topics and * their offsets. */ private[kafka010] -case class KafkaSourceOffset(partitionToOffsets: Map[TopicPartition, Long]) extends Offset { +case class KafkaSourceOffset(partitionToOffsets: Map[TopicPartition, Long]) extends OffsetV2 { override val json = JsonUtils.partitionOffsets(partitionToOffsets) } @@ -39,7 +39,7 @@ case class KafkaSourcePartitionOffset(topicPartition: TopicPartition, partitionO /** Companion object of the [[KafkaSourceOffset]] */ private[kafka010] object KafkaSourceOffset { - def getPartitionOffsets(offset: OffsetV1): Map[TopicPartition, Long] = { + def getPartitionOffsets(offset: Offset): Map[TopicPartition, Long] = { offset match { case o: KafkaSourceOffset => o.partitionToOffsets case so: SerializedOffset => KafkaSourceOffset(so).partitionToOffsets From 2628bd4fd170b2d11dd77947312a57361b186bf7 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Tue, 9 Jan 2018 01:54:04 -0800 Subject: [PATCH 42/53] don't processAllAvailable for continuous execution in StreamTest - this makes ContinuousSuite hang --- .../org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala | 2 +- .../org/apache/spark/sql/kafka010/KafkaSourceSuite.scala | 5 ++++- .../scala/org/apache/spark/sql/streaming/StreamTest.scala | 4 +++- 3 files changed, 8 insertions(+), 3 deletions(-) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala index 8a81b3e20b8ff..8ce445a462f97 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala @@ -33,7 +33,7 @@ import org.apache.spark.sql.streaming.{StreamTest, Trigger} import org.apache.spark.sql.test.{SharedSQLContext, TestSparkSession} trait KafkaContinuousTest extends KafkaSourceTest { - override val defaultTrigger = Trigger.Continuous(100) + override val defaultTrigger = Trigger.Continuous(1000) override val defaultUseV2Sink = true // We need more than the default local[2] to be able to schedule all partitions simultaneously. 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 8d16f87291575..af91e0931c9c2 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 @@ -72,7 +72,10 @@ abstract class KafkaSourceTest extends StreamTest with SharedSQLContext { // Because KafkaSource's initialPartitionOffsets is set lazily, we need to make sure // its "getOffset" is called before pushing any data. Otherwise, because of the race contion, // we don't know which data should be fetched when `startingOffsets` is latest. - q.processAllAvailable() + q match { + case c: ContinuousExecution => c.awaitEpoch(0) + case m: MicroBatchExecution => m.processAllAvailable() + } true } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index 6b384ea2eac2e..0762895fdc620 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -471,7 +471,9 @@ trait StreamTest extends QueryTest with SharedSQLContext with TimeLimits with Be try { currentStream.awaitInitialization(streamingTimeout.toMillis) currentStream match { - case s: ContinuousExecution => s.processAllAvailable() + case s: ContinuousExecution => eventually("IncrementalExecution was not created") { + s.lastExecution.executedPlan // will fail if lastExecution is null + } case _ => } } catch { From 3bdc5e71f3617b33934b8fc8d48b6cd5615cfa01 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Tue, 9 Jan 2018 10:38:56 -0800 Subject: [PATCH 43/53] cleanup sink properly --- .../kafka010/KafkaContinuousSinkSuite.scala | 38 +++++++++++-------- .../spark/sql/kafka010/KafkaSourceSuite.scala | 4 +- 2 files changed, 25 insertions(+), 17 deletions(-) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSinkSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSinkSuite.scala index 71e1deeba595c..e5e534b534c99 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSinkSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSinkSuite.scala @@ -52,8 +52,8 @@ class KafkaContinuousSinkSuite extends KafkaContinuousTest { if (testUtils != null) { testUtils.teardown() testUtils = null - super.afterAll() } + super.afterAll() } test("streaming - write to kafka with topic field") { @@ -252,23 +252,31 @@ class KafkaContinuousSinkSuite extends KafkaContinuousTest { .load() var writer: StreamingQuery = null var ex: Exception = null - ex = intercept[StreamingQueryException] { - writer = createKafkaWriter( - input.toDF(), - withOptions = Map("kafka.key.serializer" -> "foo"))() - writer.processAllAvailable() + try { + ex = intercept[StreamingQueryException] { + writer = createKafkaWriter( + input.toDF(), + withOptions = Map("kafka.key.serializer" -> "foo"))() + writer.processAllAvailable() + } + assert(ex.getMessage.toLowerCase(Locale.ROOT).contains( + "kafka option 'key.serializer' is not supported")) + } finally { + writer.stop() } - assert(ex.getMessage.toLowerCase(Locale.ROOT).contains( - "kafka option 'key.serializer' is not supported")) - ex = intercept[StreamingQueryException] { - writer = createKafkaWriter( - input.toDF(), - withOptions = Map("kafka.value.serializer" -> "foo"))() - writer.processAllAvailable() + try { + ex = intercept[StreamingQueryException] { + writer = createKafkaWriter( + input.toDF(), + withOptions = Map("kafka.value.serializer" -> "foo"))() + writer.processAllAvailable() + } + assert(ex.getMessage.toLowerCase(Locale.ROOT).contains( + "kafka option 'value.serializer' is not supported")) + } finally { + writer.stop() } - assert(ex.getMessage.toLowerCase(Locale.ROOT).contains( - "kafka option 'value.serializer' is not supported")) } test("generic - write big data with small producer buffer") { 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 af91e0931c9c2..5991579e6d1b2 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 @@ -64,13 +64,13 @@ abstract class KafkaSourceTest extends StreamTest with SharedSQLContext { if (testUtils != null) { testUtils.teardown() testUtils = null - super.afterAll() } + super.afterAll() } protected def makeSureGetOffsetCalled = AssertOnQuery { q => // Because KafkaSource's initialPartitionOffsets is set lazily, we need to make sure - // its "getOffset" is called before pushing any data. Otherwise, because of the race contion, + // its "getOffset" is called before pushing any data. Otherwise, because of the race contOOion, // we don't know which data should be fetched when `startingOffsets` is latest. q match { case c: ContinuousExecution => c.awaitEpoch(0) From 2f1cc76ba5d27f2faff9730693d9ffa454b926ff Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Tue, 9 Jan 2018 11:07:06 -0800 Subject: [PATCH 44/53] cleanup right map --- .../sql/execution/streaming/continuous/EpochCoordinator.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/EpochCoordinator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/EpochCoordinator.scala index 98017c3ac6a33..9193203e2f98c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/EpochCoordinator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/EpochCoordinator.scala @@ -147,7 +147,7 @@ private[continuous] class EpochCoordinator( partitionCommits.remove(k) } for (k <- partitionOffsets.keys.filter { case (e, _) => e < epoch }) { - partitionCommits.remove(k) + partitionOffsets.remove(k) } } } From eafe670b298e39ec06e7abfa3d349c5633182960 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Tue, 9 Jan 2018 12:03:54 -0800 Subject: [PATCH 45/53] synchronously stop epoch coordinator --- .../continuous/ContinuousExecution.scala | 1 + .../streaming/continuous/EpochCoordinator.scala | 15 +++++++++++++++ 2 files changed, 16 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala index 960dfca875be8..5d68b752fa463 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala @@ -266,6 +266,7 @@ class ContinuousExecution( sparkSessionForQuery, lastExecution)(lastExecution.toRdd) } } finally { + epochEndpoint.askSync(StopEpochCoordinator) SparkEnv.get.rpcEnv.stop(epochEndpoint) epochUpdateThread.interrupt() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/EpochCoordinator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/EpochCoordinator.scala index 9193203e2f98c..0a231eb44c761 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/EpochCoordinator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/EpochCoordinator.scala @@ -39,6 +39,13 @@ private[continuous] sealed trait EpochCoordinatorMessage extends Serializable */ private[sql] case object IncrementAndGetEpoch extends EpochCoordinatorMessage +/** + * Synchronously stop the epoch coordinator. The RpcEndpoint stop() will clear out the message queue + * before terminating the endpoint, but we must be sure no more messages will be processed before we + * can restart the query. The framework unfortunately provides no handle to wait for the queue. + */ +private[sql] case object StopEpochCoordinator extends EpochCoordinatorMessage + // Init messages /** * Set the reader and writer partition counts. Tasks may not be started until the coordinator @@ -116,6 +123,8 @@ private[continuous] class EpochCoordinator( override val rpcEnv: RpcEnv) extends ThreadSafeRpcEndpoint with Logging { + private var stopped: Boolean = false + private var numReaderPartitions: Int = _ private var numWriterPartitions: Int = _ @@ -153,6 +162,7 @@ private[continuous] class EpochCoordinator( } override def receive: PartialFunction[Any, Unit] = { + case _ if stopped => throw new IllegalStateException(s"Coordinator $this stopped") case CommitPartitionEpoch(partitionId, epoch, message) => logDebug(s"Got commit from partition $partitionId at epoch $epoch: $message") if (!partitionCommits.isDefinedAt((epoch, partitionId))) { @@ -172,6 +182,7 @@ private[continuous] class EpochCoordinator( } override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case _ if stopped => throw new IllegalStateException(s"Coordinator $this stopped") case GetCurrentEpoch => val result = currentDriverEpoch logDebug(s"Epoch $result") @@ -188,5 +199,9 @@ private[continuous] class EpochCoordinator( case SetWriterPartitions(numPartitions) => numWriterPartitions = numPartitions context.reply(()) + + case StopEpochCoordinator => + stopped = true + context.reply(()) } } From f8251552398f980768b23059c1bbbd028cfee859 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Tue, 9 Jan 2018 12:42:34 -0800 Subject: [PATCH 46/53] fix semantics --- .../continuous/ContinuousExecution.scala | 2 +- .../continuous/EpochCoordinator.scala | 22 +++++++++++-------- 2 files changed, 14 insertions(+), 10 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala index 5d68b752fa463..667410ef9f1c6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala @@ -266,7 +266,7 @@ class ContinuousExecution( sparkSessionForQuery, lastExecution)(lastExecution.toRdd) } } finally { - epochEndpoint.askSync(StopEpochCoordinator) + epochEndpoint.askSync[Unit](StopContinuousExecutionWrites) SparkEnv.get.rpcEnv.stop(epochEndpoint) epochUpdateThread.interrupt() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/EpochCoordinator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/EpochCoordinator.scala index 0a231eb44c761..40dcbecade814 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/EpochCoordinator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/EpochCoordinator.scala @@ -40,11 +40,13 @@ private[continuous] sealed trait EpochCoordinatorMessage extends Serializable private[sql] case object IncrementAndGetEpoch extends EpochCoordinatorMessage /** - * Synchronously stop the epoch coordinator. The RpcEndpoint stop() will clear out the message queue - * before terminating the endpoint, but we must be sure no more messages will be processed before we - * can restart the query. The framework unfortunately provides no handle to wait for the queue. + * The RpcEndpoint stop() will wait to clear out the message queue before terminating the + * object. This can lead to a race condition where the query restarts at epoch n, a new + * EpochCoordinator starts at epoch n, and then the old epoch coordinator commits epoch n + 1. + * The framework doesn't provide a handle to wait on the message queue, so we use a synchronous + * message to stop any writes to the ContinuousExecution object. */ -private[sql] case object StopEpochCoordinator extends EpochCoordinatorMessage +private[sql] case object StopContinuousExecutionWrites extends EpochCoordinatorMessage // Init messages /** @@ -123,7 +125,7 @@ private[continuous] class EpochCoordinator( override val rpcEnv: RpcEnv) extends ThreadSafeRpcEndpoint with Logging { - private var stopped: Boolean = false + private var queryWritesStopped: Boolean = false private var numReaderPartitions: Int = _ private var numWriterPartitions: Int = _ @@ -162,7 +164,10 @@ private[continuous] class EpochCoordinator( } override def receive: PartialFunction[Any, Unit] = { - case _ if stopped => throw new IllegalStateException(s"Coordinator $this stopped") + // If we just drop these messages, we won't do any writes to the query. The lame duck tasks + // won't shed errors or anything. + case _ if queryWritesStopped => () + case CommitPartitionEpoch(partitionId, epoch, message) => logDebug(s"Got commit from partition $partitionId at epoch $epoch: $message") if (!partitionCommits.isDefinedAt((epoch, partitionId))) { @@ -182,7 +187,6 @@ private[continuous] class EpochCoordinator( } override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { - case _ if stopped => throw new IllegalStateException(s"Coordinator $this stopped") case GetCurrentEpoch => val result = currentDriverEpoch logDebug(s"Epoch $result") @@ -200,8 +204,8 @@ private[continuous] class EpochCoordinator( numWriterPartitions = numPartitions context.reply(()) - case StopEpochCoordinator => - stopped = true + case StopContinuousExecutionWrites => + queryWritesStopped = true context.reply(()) } } From 9101ea6ef5dfd77eb0dcf3aee622b2d7a145323f Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Tue, 9 Jan 2018 12:50:14 -0800 Subject: [PATCH 47/53] add comment --- .../org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala index 8ce445a462f97..5305ca095e4e3 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala @@ -43,6 +43,8 @@ trait KafkaContinuousTest extends KafkaSourceTest { "continuous-stream-test-sql-context", sparkConf.set("spark.sql.testkey", "true"))) + // In addition to setting the partitions in Kafka, we have to wait until the query has + // reconfigured to the new count so the test framework can hook in properly. override protected def setTopicPartitions( topic: String, newCount: Int, query: StreamExecution) = { testUtils.addPartitions(topic, newCount) From a3aaf27709289f4777769309986f1a1c157701a6 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Tue, 9 Jan 2018 15:36:56 -0800 Subject: [PATCH 48/53] move ser/deser test to microbatch --- .../spark/sql/kafka010/KafkaSourceSuite.scala | 34 +++++++++---------- 1 file changed, 17 insertions(+), 17 deletions(-) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala index 5991579e6d1b2..67e2c41b28a05 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 @@ -159,6 +159,23 @@ abstract class KafkaSourceTest extends StreamTest with SharedSQLContext { class KafkaMicroBatchSourceSuite extends KafkaSourceSuiteBase { import testImplicits._ + + test("(de)serialization of initial offsets") { + val topic = newTopic() + testUtils.createTopic(topic, partitions = 5) + + val reader = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("subscribe", topic) + + testStream(reader.load)( + makeSureGetOffsetCalled, + StopStream, + StartStream(), + StopStream) + } test("maxOffsetsPerTrigger") { val topic = newTopic() @@ -424,23 +441,6 @@ class KafkaSourceSuiteBase extends KafkaSourceTest { import testImplicits._ - test("(de)serialization of initial offsets") { - val topic = newTopic() - testUtils.createTopic(topic, partitions = 5) - - val reader = spark - .readStream - .format("kafka") - .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("subscribe", topic) - - testStream(reader.load)( - makeSureGetOffsetCalled, - StopStream, - StartStream(), - StopStream) - } - test("cannot stop Kafka stream") { val topic = newTopic() testUtils.createTopic(topic, partitions = 5) From 9158af23dfff674641143b892f0f1093814035a3 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Tue, 9 Jan 2018 15:43:02 -0800 Subject: [PATCH 49/53] rm whitespace --- .../scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala index 67e2c41b28a05..01d36216ea41e 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 @@ -159,7 +159,7 @@ abstract class KafkaSourceTest extends StreamTest with SharedSQLContext { class KafkaMicroBatchSourceSuite extends KafkaSourceSuiteBase { import testImplicits._ - + test("(de)serialization of initial offsets") { val topic = newTopic() testUtils.createTopic(topic, partitions = 5) From f434c09ae13686b61550c2c7f59a0d1901d17c29 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Tue, 9 Jan 2018 17:02:59 -0800 Subject: [PATCH 50/53] add sink tests --- .../kafka010/KafkaContinuousSinkSuite.scala | 126 ++++++++++++++++++ 1 file changed, 126 insertions(+) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSinkSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSinkSuite.scala index e5e534b534c99..dfc97b1c38bb5 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSinkSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSinkSuite.scala @@ -27,6 +27,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.sql.{AnalysisException, DataFrame, Row, SaveMode} 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.types.{BinaryType, DataType} import org.apache.spark.util.Utils @@ -99,6 +100,131 @@ class KafkaContinuousSinkSuite extends KafkaContinuousTest { } } + test("streaming - write w/o topic field, with topic option") { + val inputTopic = newTopic() + testUtils.createTopic(inputTopic, partitions = 1) + + val input = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("subscribe", inputTopic) + .option("startingOffsets", "earliest") + .load() + + val topic = newTopic() + testUtils.createTopic(topic) + + val writer = createKafkaWriter( + input.toDF(), + withTopic = Some(topic), + withOutputMode = Some(OutputMode.Append()))() + + 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 { + testUtils.sendMessages(inputTopic, Array("1", "2", "3", "4", "5")) + failAfter(streamingTimeout) { + writer.processAllAvailable() + } + checkDatasetUnorderly(reader, 1, 2, 3, 4, 5) + testUtils.sendMessages(inputTopic, Array("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("streaming - 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 + */ + val inputTopic = newTopic() + testUtils.createTopic(inputTopic, partitions = 1) + + val input = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("subscribe", inputTopic) + .option("startingOffsets", "earliest") + .load() + + val topic = newTopic() + testUtils.createTopic(topic) + + val writer = createKafkaWriter( + input.toDF(), + withTopic = Some(topic), + withOutputMode = Some(OutputMode.Append()))( + withSelectExpr = "'foo' as topic", "CAST(value 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)] + .map(_._2) + + try { + testUtils.sendMessages(inputTopic, Array("1", "2", "3", "4", "5")) + failAfter(streamingTimeout) { + writer.processAllAvailable() + } + checkDatasetUnorderly(reader, 1, 2, 3, 4, 5) + testUtils.sendMessages(inputTopic, Array("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("null topic attribute") { + val inputTopic = newTopic() + testUtils.createTopic(inputTopic, partitions = 1) + + val input = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("subscribe", inputTopic) + .option("startingOffsets", "earliest") + .load() + 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 = "CAST(null as STRING) as topic", "value" + ) + testUtils.sendMessages(inputTopic, Array("1", "2", "3", "4", "5")) + writer.processAllAvailable() + } + } finally { + writer.stop() + } + assert(ex.getCause.getCause.getMessage + .toLowerCase(Locale.ROOT) + .contains("null topic present in the data.")) + } + test("streaming - write data with bad schema") { val inputTopic = newTopic() testUtils.createTopic(inputTopic, partitions = 1) From cd1bf24a86a7aa483438dbc16ffe693e4dce58fb Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Tue, 9 Jan 2018 17:10:42 -0800 Subject: [PATCH 51/53] improve docs --- .../apache/spark/sql/kafka010/KafkaContinuousSuite.scala | 6 +++--- .../org/apache/spark/sql/kafka010/KafkaSourceSuite.scala | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala index 5305ca095e4e3..c99161f2b2ecc 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala @@ -32,6 +32,7 @@ import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution import org.apache.spark.sql.streaming.{StreamTest, Trigger} import org.apache.spark.sql.test.{SharedSQLContext, TestSparkSession} +// Trait to configure StreamTest for kafka continuous execution tests. trait KafkaContinuousTest extends KafkaSourceTest { override val defaultTrigger = Trigger.Continuous(1000) override val defaultUseV2Sink = true @@ -70,9 +71,8 @@ trait KafkaContinuousTest extends KafkaSourceTest { } } -class KafkaContinuousSourceSuite extends KafkaSourceSuiteBase with KafkaContinuousTest { - -} +// Run tests in KafkaSourceSuiteBase in continuous execution mode. +class KafkaContinuousSourceSuite extends KafkaSourceSuiteBase with KafkaContinuousTest class KafkaContinuousSourceTopicDeletionSuite extends KafkaContinuousTest { import testImplicits._ 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 01d36216ea41e..d66908f86ccc7 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 @@ -70,7 +70,7 @@ abstract class KafkaSourceTest extends StreamTest with SharedSQLContext { protected def makeSureGetOffsetCalled = AssertOnQuery { q => // Because KafkaSource's initialPartitionOffsets is set lazily, we need to make sure - // its "getOffset" is called before pushing any data. Otherwise, because of the race contOOion, + // its "getOffset" is called before pushing any data. Otherwise, because of the race condition, // we don't know which data should be fetched when `startingOffsets` is latest. q match { case c: ContinuousExecution => c.awaitEpoch(0) From 514021cb3fa22da15caab419ae471030c855aacf Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Tue, 9 Jan 2018 17:14:14 -0800 Subject: [PATCH 52/53] move files --- ...scala => KafkaContinuousSourceSuite.scala} | 39 --------------- .../sql/kafka010/KafkaContinuousTest.scala | 48 +++++++++++++++++++ 2 files changed, 48 insertions(+), 39 deletions(-) rename external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/{KafkaContinuousSuite.scala => KafkaContinuousSourceSuite.scala} (71%) create mode 100644 external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousTest.scala diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala similarity index 71% rename from external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala rename to external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala index c99161f2b2ecc..b3dade414f625 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala @@ -32,45 +32,6 @@ import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution import org.apache.spark.sql.streaming.{StreamTest, Trigger} import org.apache.spark.sql.test.{SharedSQLContext, TestSparkSession} -// Trait to configure StreamTest for kafka continuous execution tests. -trait KafkaContinuousTest extends KafkaSourceTest { - override val defaultTrigger = Trigger.Continuous(1000) - override val defaultUseV2Sink = true - - // We need more than the default local[2] to be able to schedule all partitions simultaneously. - override protected def createSparkSession = new TestSparkSession( - new SparkContext( - "local[10]", - "continuous-stream-test-sql-context", - sparkConf.set("spark.sql.testkey", "true"))) - - // In addition to setting the partitions in Kafka, we have to wait until the query has - // reconfigured to the new count so the test framework can hook in properly. - override protected def setTopicPartitions( - topic: String, newCount: Int, query: StreamExecution) = { - testUtils.addPartitions(topic, newCount) - eventually(timeout(streamingTimeout)) { - assert( - query.lastExecution.logical.collectFirst { - case DataSourceV2Relation(_, r: KafkaContinuousReader) => r - }.exists(_.knownPartitions.size == newCount), - s"query never reconfigured to $newCount partitions") - } - } - - test("ensure continuous stream is being used") { - val query = spark.readStream - .format("rate") - .option("numPartitions", "1") - .option("rowsPerSecond", "1") - .load() - - testStream(query)( - Execute(q => assert(q.isInstanceOf[ContinuousExecution])) - ) - } -} - // Run tests in KafkaSourceSuiteBase in continuous execution mode. class KafkaContinuousSourceSuite extends KafkaSourceSuiteBase with KafkaContinuousTest diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousTest.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousTest.scala new file mode 100644 index 0000000000000..0cdbb90072081 --- /dev/null +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousTest.scala @@ -0,0 +1,48 @@ +package org.apache.spark.sql.kafka010 + +import org.apache.spark.SparkContext + +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import org.apache.spark.sql.execution.streaming.StreamExecution +import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution +import org.apache.spark.sql.streaming.Trigger +import org.apache.spark.sql.test.TestSparkSession + +// Trait to configure StreamTest for kafka continuous execution tests. +trait KafkaContinuousTest extends KafkaSourceTest { + override val defaultTrigger = Trigger.Continuous(1000) + override val defaultUseV2Sink = true + + // We need more than the default local[2] to be able to schedule all partitions simultaneously. + override protected def createSparkSession = new TestSparkSession( + new SparkContext( + "local[10]", + "continuous-stream-test-sql-context", + sparkConf.set("spark.sql.testkey", "true"))) + + // In addition to setting the partitions in Kafka, we have to wait until the query has + // reconfigured to the new count so the test framework can hook in properly. + override protected def setTopicPartitions( + topic: String, newCount: Int, query: StreamExecution) = { + testUtils.addPartitions(topic, newCount) + eventually(timeout(streamingTimeout)) { + assert( + query.lastExecution.logical.collectFirst { + case DataSourceV2Relation(_, r: KafkaContinuousReader) => r + }.exists(_.knownPartitions.size == newCount), + s"query never reconfigured to $newCount partitions") + } + } + + test("ensure continuous stream is being used") { + val query = spark.readStream + .format("rate") + .option("numPartitions", "1") + .option("rowsPerSecond", "1") + .load() + + testStream(query)( + Execute(q => assert(q.isInstanceOf[ContinuousExecution])) + ) + } +} From f94b53e3ab7e37fdcb9f34cf7d1313a4905fa341 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Tue, 9 Jan 2018 17:22:13 -0800 Subject: [PATCH 53/53] add header --- .../sql/kafka010/KafkaContinuousTest.scala | 18 +++++++++++++++++- 1 file changed, 17 insertions(+), 1 deletion(-) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousTest.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousTest.scala index 0cdbb90072081..e713e6695d2bd 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousTest.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousTest.scala @@ -1,7 +1,23 @@ +/* + * 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.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.execution.streaming.StreamExecution import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution