Skip to content

Commit

Permalink
data source V2 read side API refactoring
Browse files Browse the repository at this point in the history
  • Loading branch information
cloud-fan committed Oct 18, 2018
1 parent c3eaee7 commit 9f63721
Show file tree
Hide file tree
Showing 70 changed files with 1,966 additions and 1,610 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -30,10 +30,9 @@ import org.apache.spark.sql.catalyst.expressions.UnsafeRow
import org.apache.spark.sql.kafka010.KafkaSourceProvider.{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.reader.streaming._
import org.apache.spark.sql.types.StructType

/**
* A [[ContinuousReadSupport]] for data from kafka.
* A [[ContinuousInputStream]] that reads 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.
Expand All @@ -46,17 +45,22 @@ import org.apache.spark.sql.types.StructType
* scenarios, where some offsets after the specified initial ones can't be
* properly read.
*/
class KafkaContinuousReadSupport(
class KafkaContinuousInputStream(
offsetReader: KafkaOffsetReader,
kafkaParams: ju.Map[String, Object],
sourceOptions: Map[String, String],
metadataPath: String,
initialOffsets: KafkaOffsetRangeLimit,
failOnDataLoss: Boolean)
extends ContinuousReadSupport with Logging {
extends ContinuousInputStream with Logging {

private val pollTimeoutMs = sourceOptions.getOrElse("kafkaConsumer.pollTimeoutMs", "512").toLong

// Initialized when creating read support. If this diverges from the partitions at the latest
// offsets, we need to reconfigure.
// Exposed outside this object only for unit tests.
@volatile private[sql] var knownPartitions: Set[TopicPartition] = _

override def initialOffset(): Offset = {
val offsets = initialOffsets match {
case EarliestOffsetRangeLimit => KafkaSourceOffset(offsetReader.fetchEarliestOffsets())
Expand All @@ -67,28 +71,29 @@ class KafkaContinuousReadSupport(
offsets
}

override def fullSchema(): StructType = KafkaOffsetReader.kafkaSchema

override def newScanConfigBuilder(start: Offset): ScanConfigBuilder = {
new KafkaContinuousScanConfigBuilder(fullSchema(), start, offsetReader, reportDataLoss)
}

override def deserializeOffset(json: String): Offset = {
KafkaSourceOffset(JsonUtils.partitionOffsets(json))
}

override def planInputPartitions(config: ScanConfig): Array[InputPartition] = {
val startOffsets = config.asInstanceOf[KafkaContinuousScanConfig].startOffsets
startOffsets.toSeq.map {
case (topicPartition, start) =>
KafkaContinuousInputPartition(
topicPartition, start, kafkaParams, pollTimeoutMs, failOnDataLoss)
}.toArray
}
override def createContinuousScan(start: Offset): ContinuousScan = {
val oldStartPartitionOffsets = KafkaSourceOffset.getPartitionOffsets(start)

override def createContinuousReaderFactory(
config: ScanConfig): ContinuousPartitionReaderFactory = {
KafkaContinuousReaderFactory
val currentPartitionSet = offsetReader.fetchEarliestOffsets().keySet
val newPartitions = currentPartitionSet.diff(oldStartPartitionOffsets.keySet)
val newPartitionOffsets = offsetReader.fetchEarliestOffsets(newPartitions.toSeq)

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

new KafkaContinuousScan(
offsetReader, kafkaParams, pollTimeoutMs, failOnDataLoss, startOffsets)
}

/** Stop this source and free any resources it has allocated. */
Expand All @@ -105,9 +110,8 @@ class KafkaContinuousReadSupport(
KafkaSourceOffset(mergedMap)
}

override def needsReconfiguration(config: ScanConfig): Boolean = {
val knownPartitions = config.asInstanceOf[KafkaContinuousScanConfig].knownPartitions
offsetReader.fetchLatestOffsets().keySet != knownPartitions
override def needsReconfiguration(): Boolean = {
knownPartitions != null && offsetReader.fetchLatestOffsets().keySet != knownPartitions
}

override def toString(): String = s"KafkaSource[$offsetReader]"
Expand All @@ -125,6 +129,25 @@ class KafkaContinuousReadSupport(
}
}

class KafkaContinuousScan(
offsetReader: KafkaOffsetReader,
kafkaParams: ju.Map[String, Object],
pollTimeoutMs: Long,
failOnDataLoss: Boolean,
startOffsets: Map[TopicPartition, Long]) extends ContinuousScan {

override def createContinuousReaderFactory(): ContinuousPartitionReaderFactory = {
KafkaContinuousReaderFactory
}

override def planInputPartitions(): Array[InputPartition] = {
startOffsets.toSeq.map { case (topicPartition, start) =>
KafkaContinuousInputPartition(
topicPartition, start, kafkaParams, pollTimeoutMs, failOnDataLoss)
}.toArray
}
}

/**
* An input partition for continuous Kafka processing. This will be serialized and transformed
* into a full reader on executors.
Expand All @@ -151,41 +174,6 @@ object KafkaContinuousReaderFactory extends ContinuousPartitionReaderFactory {
}
}

class KafkaContinuousScanConfigBuilder(
schema: StructType,
startOffset: Offset,
offsetReader: KafkaOffsetReader,
reportDataLoss: String => Unit)
extends ScanConfigBuilder {

override def build(): ScanConfig = {
val oldStartPartitionOffsets = KafkaSourceOffset.getPartitionOffsets(startOffset)

val currentPartitionSet = offsetReader.fetchEarliestOffsets().keySet
val newPartitions = currentPartitionSet.diff(oldStartPartitionOffsets.keySet)
val newPartitionOffsets = offsetReader.fetchEarliestOffsets(newPartitions.toSeq)

val deletedPartitions = oldStartPartitionOffsets.keySet.diff(currentPartitionSet)
if (deletedPartitions.nonEmpty) {
reportDataLoss(s"Some partitions were deleted: $deletedPartitions")
}

val startOffsets = newPartitionOffsets ++
oldStartPartitionOffsets.filterKeys(!deletedPartitions.contains(_))
KafkaContinuousScanConfig(schema, startOffsets)
}
}

case class KafkaContinuousScanConfig(
readSchema: StructType,
startOffsets: Map[TopicPartition, Long])
extends ScanConfig {

// Created when building the scan config builder. If this diverges from the partitions at the
// latest offsets, we need to reconfigure the kafka read support.
def knownPartitions: Set[TopicPartition] = startOffsets.keySet
}

/**
* A per-task data reader for continuous Kafka processing.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,17 +29,16 @@ import org.apache.spark.scheduler.ExecutorCacheTaskLocation
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.UnsafeRow
import org.apache.spark.sql.execution.streaming.{HDFSMetadataLog, SerializedOffset, SimpleStreamingScanConfig, SimpleStreamingScanConfigBuilder}
import org.apache.spark.sql.execution.streaming.sources.RateControlMicroBatchReadSupport
import org.apache.spark.sql.execution.streaming.{HDFSMetadataLog, SerializedOffset}
import org.apache.spark.sql.execution.streaming.sources.RateControlMicroBatchInputStream
import org.apache.spark.sql.kafka010.KafkaSourceProvider.{INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE, INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE}
import org.apache.spark.sql.sources.v2.DataSourceOptions
import org.apache.spark.sql.sources.v2.reader._
import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchReadSupport, Offset}
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchInputStream, MicroBatchScan, Offset}
import org.apache.spark.util.UninterruptibleThread

/**
* A [[MicroBatchReadSupport]] that reads data from Kafka.
* A [[MicroBatchInputStream]] that reads data from Kafka.
*
* The [[KafkaSourceOffset]] is the custom [[Offset]] defined for this source that contains
* a map of TopicPartition -> offset. Note that this offset is 1 + (available offset). For
Expand All @@ -54,13 +53,13 @@ import org.apache.spark.util.UninterruptibleThread
* To avoid this issue, you should make sure stopping the query before stopping the Kafka brokers
* and not use wrong broker addresses.
*/
private[kafka010] class KafkaMicroBatchReadSupport(
private[kafka010] class KafkaMicroBatchInputStream(
kafkaOffsetReader: KafkaOffsetReader,
executorKafkaParams: ju.Map[String, Object],
options: DataSourceOptions,
metadataPath: String,
startingOffsets: KafkaOffsetRangeLimit,
failOnDataLoss: Boolean) extends RateControlMicroBatchReadSupport with Logging {
failOnDataLoss: Boolean) extends RateControlMicroBatchInputStream with Logging {

private val pollTimeoutMs = options.getLong(
"kafkaConsumer.pollTimeoutMs",
Expand Down Expand Up @@ -93,65 +92,16 @@ private[kafka010] class KafkaMicroBatchReadSupport(
endPartitionOffsets
}

override def fullSchema(): StructType = KafkaOffsetReader.kafkaSchema

override def newScanConfigBuilder(start: Offset, end: Offset): ScanConfigBuilder = {
new SimpleStreamingScanConfigBuilder(fullSchema(), start, Some(end))
}

override def planInputPartitions(config: ScanConfig): Array[InputPartition] = {
val sc = config.asInstanceOf[SimpleStreamingScanConfig]
val startPartitionOffsets = sc.start.asInstanceOf[KafkaSourceOffset].partitionToOffsets
val endPartitionOffsets = sc.end.get.asInstanceOf[KafkaSourceOffset].partitionToOffsets

// Find the new partitions, and get their earliest offsets
val newPartitions = endPartitionOffsets.keySet.diff(startPartitionOffsets.keySet)
val newPartitionInitialOffsets = kafkaOffsetReader.fetchEarliestOffsets(newPartitions.toSeq)
if (newPartitionInitialOffsets.keySet != newPartitions) {
// We cannot get from offsets for some partitions. It means they got deleted.
val deletedPartitions = newPartitions.diff(newPartitionInitialOffsets.keySet)
reportDataLoss(
s"Cannot find earliest offsets of ${deletedPartitions}. Some data may have been missed")
}
logInfo(s"Partitions added: $newPartitionInitialOffsets")
newPartitionInitialOffsets.filter(_._2 != 0).foreach { case (p, o) =>
reportDataLoss(
s"Added partition $p starts from $o instead of 0. Some data may have been missed")
}

// Find deleted partitions, and report data loss if required
val deletedPartitions = startPartitionOffsets.keySet.diff(endPartitionOffsets.keySet)
if (deletedPartitions.nonEmpty) {
reportDataLoss(s"$deletedPartitions are gone. Some data may have been missed")
}

// Use the end partitions to calculate offset ranges to ignore partitions that have
// been deleted
val topicPartitions = endPartitionOffsets.keySet.filter { tp =>
// Ignore partitions that we don't know the from offsets.
newPartitionInitialOffsets.contains(tp) || startPartitionOffsets.contains(tp)
}.toSeq
logDebug("TopicPartitions: " + topicPartitions.mkString(", "))

// Calculate offset ranges
val offsetRanges = rangeCalculator.getRanges(
fromOffsets = startPartitionOffsets ++ newPartitionInitialOffsets,
untilOffsets = endPartitionOffsets,
executorLocations = getSortedExecutorList())

// Reuse Kafka consumers only when all the offset ranges have distinct TopicPartitions,
// that is, concurrent tasks will not read the same TopicPartitions.
val reuseKafkaConsumer = offsetRanges.map(_.topicPartition).toSet.size == offsetRanges.size

// Generate factories based on the offset ranges
offsetRanges.map { range =>
KafkaMicroBatchInputPartition(
range, executorKafkaParams, pollTimeoutMs, failOnDataLoss, reuseKafkaConsumer)
}.toArray
}

override def createReaderFactory(config: ScanConfig): PartitionReaderFactory = {
KafkaMicroBatchReaderFactory
override def createMicroBatchScan(start: Offset, end: Offset): MicroBatchScan = {
new KafkaMicroBatchScan(
kafkaOffsetReader,
rangeCalculator,
executorKafkaParams,
pollTimeoutMs,
failOnDataLoss,
reportDataLoss,
start.asInstanceOf[KafkaSourceOffset],
end.asInstanceOf[KafkaSourceOffset])
}

override def deserializeOffset(json: String): Offset = {
Expand Down Expand Up @@ -229,23 +179,6 @@ private[kafka010] class KafkaMicroBatchReadSupport(
}
}

private def getSortedExecutorList(): Array[String] = {

def compare(a: ExecutorCacheTaskLocation, b: ExecutorCacheTaskLocation): Boolean = {
if (a.host == b.host) {
a.executorId > b.executorId
} else {
a.host > b.host
}
}

val bm = SparkEnv.get.blockManager
bm.master.getPeers(bm.blockManagerId).toArray
.map(x => ExecutorCacheTaskLocation(x.host, x.executorId))
.sortWith(compare)
.map(_.toString)
}

/**
* If `failOnDataLoss` is true, this method will throw an `IllegalStateException`.
* Otherwise, just log a warning.
Expand Down Expand Up @@ -294,6 +227,88 @@ private[kafka010] class KafkaMicroBatchReadSupport(
}
}

private[kafka010] class KafkaMicroBatchScan(
kafkaOffsetReader: KafkaOffsetReader,
rangeCalculator: KafkaOffsetRangeCalculator,
executorKafkaParams: ju.Map[String, Object],
pollTimeoutMs: Long,
failOnDataLoss: Boolean,
reportDataLoss: String => Unit,
start: KafkaSourceOffset,
end: KafkaSourceOffset) extends MicroBatchScan with Logging {

override def createReaderFactory(): PartitionReaderFactory = {
KafkaMicroBatchReaderFactory
}

override def planInputPartitions(): Array[InputPartition] = {
val startPartitionOffsets = start.partitionToOffsets
val endPartitionOffsets = end.partitionToOffsets

// Find the new partitions, and get their earliest offsets
val newPartitions = endPartitionOffsets.keySet.diff(startPartitionOffsets.keySet)
val newPartitionInitialOffsets = kafkaOffsetReader.fetchEarliestOffsets(newPartitions.toSeq)
if (newPartitionInitialOffsets.keySet != newPartitions) {
// We cannot get from offsets for some partitions. It means they got deleted.
val deletedPartitions = newPartitions.diff(newPartitionInitialOffsets.keySet)
reportDataLoss(
s"Cannot find earliest offsets of ${deletedPartitions}. Some data may have been missed")
}
logInfo(s"Partitions added: $newPartitionInitialOffsets")
newPartitionInitialOffsets.filter(_._2 != 0).foreach { case (p, o) =>
reportDataLoss(
s"Added partition $p starts from $o instead of 0. Some data may have been missed")
}

// Find deleted partitions, and report data loss if required
val deletedPartitions = startPartitionOffsets.keySet.diff(endPartitionOffsets.keySet)
if (deletedPartitions.nonEmpty) {
reportDataLoss(s"$deletedPartitions are gone. Some data may have been missed")
}

// Use the end partitions to calculate offset ranges to ignore partitions that have
// been deleted
val topicPartitions = endPartitionOffsets.keySet.filter { tp =>
// Ignore partitions that we don't know the from offsets.
newPartitionInitialOffsets.contains(tp) || startPartitionOffsets.contains(tp)
}.toSeq
logDebug("TopicPartitions: " + topicPartitions.mkString(", "))

// Calculate offset ranges
val offsetRanges = rangeCalculator.getRanges(
fromOffsets = startPartitionOffsets ++ newPartitionInitialOffsets,
untilOffsets = endPartitionOffsets,
executorLocations = getSortedExecutorList())

// Reuse Kafka consumers only when all the offset ranges have distinct TopicPartitions,
// that is, concurrent tasks will not read the same TopicPartitions.
val reuseKafkaConsumer = offsetRanges.map(_.topicPartition).toSet.size == offsetRanges.size

// Generate factories based on the offset ranges
offsetRanges.map { range =>
KafkaMicroBatchInputPartition(
range, executorKafkaParams, pollTimeoutMs, failOnDataLoss, reuseKafkaConsumer)
}.toArray
}

private def getSortedExecutorList(): Array[String] = {

def compare(a: ExecutorCacheTaskLocation, b: ExecutorCacheTaskLocation): Boolean = {
if (a.host == b.host) {
a.executorId > b.executorId
} else {
a.host > b.host
}
}

val bm = SparkEnv.get.blockManager
bm.master.getPeers(bm.blockManagerId).toArray
.map(x => ExecutorCacheTaskLocation(x.host, x.executorId))
.sortWith(compare)
.map(_.toString)
}
}

/** A [[InputPartition]] for reading Kafka data in a micro-batch streaming query. */
private[kafka010] case class KafkaMicroBatchInputPartition(
offsetRange: KafkaOffsetRange,
Expand Down
Loading

0 comments on commit 9f63721

Please sign in to comment.