Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Next Next commit
[SPARK-17812][SQL][KAFKA] implement specified offsets and assign
  • Loading branch information
koeninger committed Oct 14, 2016
commit 3120fd8ade24140777c29fc1487aa3f6e76152fb
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ import java.{util => ju}
import scala.collection.JavaConverters._
import scala.util.control.NonFatal

import org.apache.kafka.clients.consumer.{Consumer, KafkaConsumer}
import org.apache.kafka.clients.consumer.{Consumer, KafkaConsumer, OffsetOutOfRangeException}
import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener
import org.apache.kafka.common.TopicPartition

Expand Down Expand Up @@ -82,7 +82,7 @@ private[kafka010] case class KafkaSource(
executorKafkaParams: ju.Map[String, Object],
sourceOptions: Map[String, String],
metadataPath: String,
startFromEarliestOffset: Boolean,
startingOffsets: StartingOffsets,
failOnDataLoss: Boolean)
extends Source with Logging {

Expand Down Expand Up @@ -110,10 +110,10 @@ private[kafka010] case class KafkaSource(
private lazy val initialPartitionOffsets = {
val metadataLog = new HDFSMetadataLog[KafkaSourceOffset](sqlContext.sparkSession, metadataPath)
metadataLog.get(0).getOrElse {
val offsets = if (startFromEarliestOffset) {
KafkaSourceOffset(fetchEarliestOffsets())
} else {
KafkaSourceOffset(fetchLatestOffsets())
val offsets = startingOffsets match {
case EarliestOffsets => KafkaSourceOffset(fetchEarliestOffsets())
case LatestOffsets => KafkaSourceOffset(fetchLatestOffsets())
case SpecificOffsets(p) => KafkaSourceOffset(fetchSpecificStartingOffsets(p))
}
metadataLog.add(0, offsets)
logInfo(s"Initial offsets: $offsets")
Expand Down Expand Up @@ -231,6 +231,33 @@ private[kafka010] case class KafkaSource(

override def toString(): String = s"KafkaSource[$consumerStrategy]"

/**
* Set consumer position to specified offsets, making sure all assignments are set.
*/
private def fetchSpecificStartingOffsets(
partitionOffsets: Map[TopicPartition, Long]): Map[TopicPartition, Long] =
withRetriesWithoutInterrupt {
// Poll to get the latest assigned partitions
consumer.poll(0)
val partitions = consumer.assignment()
consumer.pause(partitions)
assert(partitions.asScala == partitionOffsets.keySet,
"If startingOffsets contains specific offsets, you must specify all TopicPartitions.\n" +
"Use -1 for latest, -2 for earliest, if you don't care.\n" +
s"Specified: ${partitionOffsets.keySet} Assigned: ${partitions.asScala}")
logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $partitionOffsets")

// These offsets may be out of range, but there isn't a good way of determining that here,
// even poll(0) afterwards may not throw immediately.
// The executor should throw if it is assigned out of range offsets.
partitionOffsets.foreach {
case (tp, -1) => consumer.seekToEnd(ju.Arrays.asList(tp))
case (tp, -2) => consumer.seekToBeginning(ju.Arrays.asList(tp))
case (tp, off) => consumer.seek(tp, off)
}
partitionOffsets
}

/**
* Fetch the earliest offsets of partitions.
*/
Expand Down Expand Up @@ -273,7 +300,7 @@ private[kafka010] case class KafkaSource(
consumer.poll(0)
val partitions = consumer.assignment()
consumer.pause(partitions)
logDebug(s"\tPartitioned assigned to consumer: $partitions")
logDebug(s"\tPartitions assigned to consumer: $partitions")

// Get the earliest offset of each partition
consumer.seekToBeginning(partitions)
Expand Down Expand Up @@ -317,6 +344,8 @@ private[kafka010] case class KafkaSource(
try {
result = Some(body)
} catch {
case x: OffsetOutOfRangeException =>
reportDataLoss(x.getMessage)
case NonFatal(e) =>
lastException = e
logWarning(s"Error in attempt $attempt getting Kafka offsets: ", e)
Expand Down Expand Up @@ -373,6 +402,17 @@ private[kafka010] object KafkaSource {
def createConsumer(): Consumer[Array[Byte], Array[Byte]]
}

case class AssignStrategy(partitions: Array[TopicPartition], kafkaParams: ju.Map[String, Object])
extends ConsumerStrategy {
override def createConsumer(): Consumer[Array[Byte], Array[Byte]] = {
val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams)
consumer.assign(ju.Arrays.asList(partitions: _*))
consumer
}

override def toString: String = s"Assign[${partitions.mkString(", ")}]"
}

case class SubscribeStrategy(topics: Seq[String], kafkaParams: ju.Map[String, Object])
extends ConsumerStrategy {
override def createConsumer(): Consumer[Array[Byte], Array[Byte]] = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -77,14 +77,12 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider
// id. Hence, we should generate a unique id for each query.
val uniqueGroupId = s"spark-kafka-source-${UUID.randomUUID}-${metadataPath.hashCode}"

val startFromEarliestOffset =
caseInsensitiveParams.get(STARTING_OFFSET_OPTION_KEY).map(_.trim.toLowerCase) match {
case Some("latest") => false
case Some("earliest") => true
case Some(pos) =>
// This should not happen since we have already checked the options.
throw new IllegalStateException(s"Invalid $STARTING_OFFSET_OPTION_KEY: $pos")
case None => false
val startingOffsets =
caseInsensitiveParams.get(STARTING_OFFSETS_OPTION_KEY).map(_.trim.toLowerCase) match {
case Some("latest") => LatestOffsets
case Some("earliest") => EarliestOffsets
case Some(json) => SpecificOffsets(JsonUtils.partitionOffsets(json))
case None => LatestOffsets
}

val kafkaParamsForStrategy =
Expand All @@ -95,9 +93,9 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider
// So that consumers in Kafka source do not mess with any existing group id
.set(ConsumerConfig.GROUP_ID_CONFIG, s"$uniqueGroupId-driver")

// Set to "latest" to avoid exceptions. However, KafkaSource will fetch the initial offsets
// by itself instead of counting on KafkaConsumer.
.set(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest")
// Set to "earliest" to avoid exceptions. However, KafkaSource will fetch the initial
// offsets by itself instead of counting on KafkaConsumer.
.set(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest")

// So that consumers in the driver does not commit offsets unnecessarily
.set(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false")
Expand Down Expand Up @@ -130,6 +128,10 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider
.build()

val strategy = caseInsensitiveParams.find(x => STRATEGY_OPTION_KEYS.contains(x._1)).get match {
case ("assign", value) =>
AssignStrategy(
JsonUtils.partitions(value),
kafkaParamsForStrategy)
case ("subscribe", value) =>
SubscribeStrategy(
value.split(",").map(_.trim()).filter(_.nonEmpty),
Expand All @@ -153,7 +155,7 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider
kafkaParamsForExecutors,
parameters,
metadataPath,
startFromEarliestOffset,
startingOffsets,
failOnDataLoss)
}

Expand Down Expand Up @@ -195,14 +197,6 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider
throw new IllegalArgumentException("Unknown option")
}

caseInsensitiveParams.get(STARTING_OFFSET_OPTION_KEY) match {
case Some(pos) if !STARTING_OFFSET_OPTION_VALUES.contains(pos.trim.toLowerCase) =>
throw new IllegalArgumentException(
s"Illegal value '$pos' for option '$STARTING_OFFSET_OPTION_KEY', " +
s"acceptable values are: ${STARTING_OFFSET_OPTION_VALUES.mkString(", ")}")
case _ =>
}

// Validate user-specified Kafka options

if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.GROUP_ID_CONFIG}")) {
Expand All @@ -215,11 +209,11 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider
throw new IllegalArgumentException(
s"""
|Kafka option '${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG}' is not supported.
|Instead set the source option '$STARTING_OFFSET_OPTION_KEY' to 'earliest' or 'latest' to
|specify where to start. Structured Streaming manages which offsets are consumed
|Instead set the source option '$STARTING_OFFSETS_OPTION_KEY' to 'earliest' or 'latest'
|to specify where to start. Structured Streaming manages which offsets are consumed
|internally, rather than relying on the kafkaConsumer to do it. This will ensure that no
|data is missed when when new topics/partitions are dynamically subscribed. Note that
|'$STARTING_OFFSET_OPTION_KEY' only applies when a new Streaming query is started, and
|'$STARTING_OFFSETS_OPTION_KEY' only applies when a new Streaming query is started, and
|that resuming will always pick up from where the query left off. See the docs for more
|details.
""".stripMargin)
Expand Down Expand Up @@ -282,8 +276,7 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider
}

private[kafka010] object KafkaSourceProvider {
private val STRATEGY_OPTION_KEYS = Set("subscribe", "subscribepattern")
private val STARTING_OFFSET_OPTION_KEY = "startingoffset"
private val STARTING_OFFSET_OPTION_VALUES = Set("earliest", "latest")
private val STRATEGY_OPTION_KEYS = Set("subscribe", "subscribepattern", "assign")
private val STARTING_OFFSETS_OPTION_KEY = "startingoffsets"
private val FAIL_ON_DATA_LOSS_OPTION_KEY = "failondataloss"
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,32 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.sql.kafka010

import org.apache.kafka.common.TopicPartition

/*
* Values that can be specified for config startingOffsets
*/
private[kafka010] sealed trait StartingOffsets

private[kafka010] case object EarliestOffsets extends StartingOffsets

private[kafka010] case object LatestOffsets extends StartingOffsets

private[kafka010] case class SpecificOffsets(
partitionOffsets: Map[TopicPartition, Long]) extends StartingOffsets
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,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 contion,
// we don't know which data should be fetched when `startingOffset` is latest.
// we don't know which data should be fetched when `startingOffsets` is latest.
q.processAllAvailable()
true
}
Expand Down Expand Up @@ -301,7 +301,7 @@ class KafkaSourceSuite extends KafkaSourceTest {
val reader = spark
.readStream
.format("kafka")
.option("startingOffset", s"latest")
.option("startingOffsets", s"latest")
.option("kafka.bootstrap.servers", testUtils.brokerAddress)
.option("kafka.metadata.max.age.ms", "1")
options.foreach { case (k, v) => reader.option(k, v) }
Expand Down Expand Up @@ -340,7 +340,7 @@ class KafkaSourceSuite extends KafkaSourceTest {
val reader = spark.readStream
reader
.format(classOf[KafkaSourceProvider].getCanonicalName.stripSuffix("$"))
.option("startingOffset", s"earliest")
.option("startingOffsets", s"earliest")
.option("kafka.bootstrap.servers", testUtils.brokerAddress)
.option("kafka.metadata.max.age.ms", "1")
options.foreach { case (k, v) => reader.option(k, v) }
Expand Down