diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md index 0411b3076b55..d1334f07f24b 100644 --- a/docs/structured-streaming-kafka-integration.md +++ b/docs/structured-streaming-kafka-integration.md @@ -362,24 +362,33 @@ The following configurations are optional:
| Option | value | default | query type | meaning | |
|---|---|---|---|---|---|
| startingTimestamp | +timestamp string e.g. "1000" | +none (next preference is startingOffsetsByTimestamp) |
+ streaming and batch | +The start point of timestamp when a query is started, a string specifying a starting timestamp for
+ all partitions in topics being subscribed. Please refer the details on timestamp offset options below. If Kafka doesn't return the matched offset,
+ the query will fail immediately to prevent unintended read from such partition. (This is a kind of limitation as of now, and will be addressed in near future.)
+
+ Note1: startingTimestamp takes precedence over startingOffsetsByTimestamp and startingOffsets.
+ Note2: For streaming queries, this only applies when a new query is started, and that resuming will
+ always pick up from where the query left off. Newly discovered partitions during a query will start at
+ earliest. |
+|
| startingOffsetsByTimestamp | json string """ {"topicA":{"0": 1000, "1": 1000}, "topicB": {"0": 2000, "1": 2000}} """ | -none (the value of startingOffsets will apply) |
+ none (next preference is startingOffsets) |
streaming and batch | The start point of timestamp when a query is started, a json string specifying a starting timestamp for
- each TopicPartition. The returned offset for each partition is the earliest offset whose timestamp is greater than or
- equal to the given timestamp in the corresponding partition. If the matched offset doesn't exist,
+ each TopicPartition. Please refer the details on timestamp offset options below. If Kafka doesn't return the matched offset,
the query will fail immediately to prevent unintended read from such partition. (This is a kind of limitation as of now, and will be addressed in near future.)
- Spark simply passes the timestamp information to KafkaConsumer.offsetsForTimes, and doesn't interpret or reason about the value.
- For more details on KafkaConsumer.offsetsForTimes, please refer javadoc for details.
- Also the meaning of timestamp here can be vary according to Kafka configuration (log.message.timestamp.type): please refer Kafka documentation for further details.
- Note: This option requires Kafka 0.10.1.0 or higher.
- Note2: startingOffsetsByTimestamp takes precedence over startingOffsets.
- Note3: For streaming queries, this only applies when a new query is started, and that resuming will
+ Note1: startingOffsetsByTimestamp takes precedence over startingOffsets.
+ Note2: For streaming queries, this only applies when a new query is started, and that resuming will
always pick up from where the query left off. Newly discovered partitions during a query will start at
earliest. |
| endingTimestamp | +timestamp string e.g. "1000" | +none (next preference is endingOffsetsByTimestamp) |
+ batch query | +The end point when a batch query is ended, a json string specifying an ending timestamp for
+ all partitions in topics being subscribed. Please refer the details on timestamp offset options below.
+ If Kafka doesn't return the matched offset, the offset will be set to latest.
+ Note: endingTimestamp takes precedence over endingOffsetsByTimestamp and endingOffsets.
+ |
+|
| endingOffsetsByTimestamp | json string """ {"topicA":{"0": 1000, "1": 1000}, "topicB": {"0": 2000, "1": 2000}} """ | -latest | +none (next preference is endingOffsets) |
batch query | The end point when a batch query is ended, a json string specifying an ending timestamp for each TopicPartition.
- The returned offset for each partition is the earliest offset whose timestamp is greater than or equal to
- the given timestamp in the corresponding partition. If the matched offset doesn't exist, the offset will
- be set to latest.
-
- Spark simply passes the timestamp information to KafkaConsumer.offsetsForTimes, and doesn't interpret or reason about the value.
- For more details on KafkaConsumer.offsetsForTimes, please refer javadoc for details.
- Also the meaning of timestamp here can be vary according to Kafka configuration (log.message.timestamp.type): please refer Kafka documentation for further details.
- Note: This option requires Kafka 0.10.1.0 or higher.
- Note2: endingOffsetsByTimestamp takes precedence over endingOffsets.
+ Please refer the details on timestamp offset options below. If Kafka doesn't return the matched offset,
+ the offset will be set to latest.
+ Note: endingOffsetsByTimestamp takes precedence over endingOffsets.
|
KafkaConsumer.offsetsForTimes, and doesn't interpret or reason about the value.
+For more details on KafkaConsumer.offsetsForTimes, please refer javadoc for details.
+Also, the meaning of timestamp here can be vary according to Kafka configuration (log.message.timestamp.type): please refer Kafka documentation for further details.
+
+Timestamp offset options require Kafka 0.10.1.0 or higher.
+
### Offset fetching
In Spark 3.0 and before Spark uses KafkaConsumer for offset fetching which could cause infinite wait in the driver.
diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala
index 0b549870a348..abdb322156f2 100644
--- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala
+++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala
@@ -72,6 +72,8 @@ class KafkaContinuousStream(
case SpecificOffsetRangeLimit(p) => offsetReader.fetchSpecificOffsets(p, reportDataLoss)
case SpecificTimestampRangeLimit(p) => offsetReader.fetchSpecificTimestampBasedOffsets(p,
failsOnNoMatchingOffset = true)
+ case GlobalTimestampRangeLimit(ts) => offsetReader.fetchGlobalTimestampBasedOffsets(
+ ts, failsOnNoMatchingOffset = true)
}
logInfo(s"Initial offsets: $offsets")
offsets
diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala
index 822184842deb..cdd8aa8d8acc 100644
--- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala
+++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala
@@ -166,6 +166,8 @@ private[kafka010] class KafkaMicroBatchStream(
kafkaOffsetReader.fetchSpecificOffsets(p, reportDataLoss)
case SpecificTimestampRangeLimit(p) =>
kafkaOffsetReader.fetchSpecificTimestampBasedOffsets(p, failsOnNoMatchingOffset = true)
+ case GlobalTimestampRangeLimit(ts) =>
+ kafkaOffsetReader.fetchGlobalTimestampBasedOffsets(ts, failsOnNoMatchingOffset = true)
}
metadataLog.add(0, offsets)
logInfo(s"Initial offsets: $offsets")
diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeLimit.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeLimit.scala
index d64b5d4f7e9e..5c9a01d6d1db 100644
--- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeLimit.scala
+++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeLimit.scala
@@ -49,6 +49,13 @@ private[kafka010] case class SpecificOffsetRangeLimit(
private[kafka010] case class SpecificTimestampRangeLimit(
topicTimestamps: Map[TopicPartition, Long]) extends KafkaOffsetRangeLimit
+/**
+ * Represents the desire to bind to earliest offset which timestamp for the offset is equal or
+ * greater than specific timestamp. This applies the timestamp to the all topics/partitions.
+ */
+private[kafka010] case class GlobalTimestampRangeLimit(
+ timestamp: Long) extends KafkaOffsetRangeLimit
+
private[kafka010] object KafkaOffsetRangeLimit {
/**
* Used to denote offset range limits that are resolved via Kafka
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 546970507a2e..764c6434627b 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
@@ -77,6 +77,20 @@ private[kafka010] trait KafkaOffsetReader {
partitionTimestamps: Map[TopicPartition, Long],
failsOnNoMatchingOffset: Boolean): KafkaSourceOffset
+ /**
+ * Resolves the specific offsets based on timestamp per all topic-partitions being subscribed.
+ * The returned offset for each partition is the earliest offset whose timestamp is greater
+ * than or equal to the given timestamp in the corresponding partition. If the matched offset
+ * doesn't exist, depending on `failsOnNoMatchingOffset` parameter, the offset will be set to
+ * latest or this method throws an error.
+ *
+ * @param timestamp the timestamp.
+ * @param failsOnNoMatchingOffset whether to fail the query when no matched offset can be found.
+ */
+ def fetchGlobalTimestampBasedOffsets(
+ timestamp: Long,
+ failsOnNoMatchingOffset: Boolean): KafkaSourceOffset
+
/**
* 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/KafkaOffsetReaderAdmin.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReaderAdmin.scala
index 6f4cb895f363..ceca011830ad 100644
--- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReaderAdmin.scala
+++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReaderAdmin.scala
@@ -138,6 +138,9 @@ private[kafka010] class KafkaOffsetReaderAdmin(
case SpecificTimestampRangeLimit(partitionTimestamps) =>
fetchSpecificTimestampBasedOffsets(partitionTimestamps,
failsOnNoMatchingOffset = isStartingOffsets).partitionToOffsets
+ case GlobalTimestampRangeLimit(timestamp) =>
+ fetchGlobalTimestampBasedOffsets(timestamp,
+ failsOnNoMatchingOffset = isStartingOffsets).partitionToOffsets
}
}
@@ -193,6 +196,37 @@ private[kafka010] class KafkaOffsetReaderAdmin(
fetchSpecificOffsets0(fnAssertParametersWithPartitions, fnRetrievePartitionOffsets)
}
+
+ override def fetchGlobalTimestampBasedOffsets(
+ timestamp: Long,
+ failsOnNoMatchingOffset: Boolean): KafkaSourceOffset = {
+ val fnAssertParametersWithPartitions: ju.Set[TopicPartition] => Unit = { partitions =>
+ logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $timestamp")
+ }
+
+ val fnRetrievePartitionOffsets: ju.Set[TopicPartition] => Map[TopicPartition, Long] = { tps =>
+ val listOffsetsParams = tps.asScala.map { tp =>
+ tp -> OffsetSpec.forTimestamp(timestamp)
+ }.toMap.asJava
+ admin.listOffsets(listOffsetsParams, listOffsetsOptions).all().get().asScala.map {
+ case (tp, offsetSpec) =>
+ if (failsOnNoMatchingOffset) {
+ assert(offsetSpec.offset() != OffsetFetchResponse.INVALID_OFFSET, "No offset " +
+ s"matched from request of topic-partition $tp and timestamp " +
+ s"$timestamp.")
+ }
+
+ if (offsetSpec.offset() == OffsetFetchResponse.INVALID_OFFSET) {
+ tp -> KafkaOffsetRangeLimit.LATEST
+ } else {
+ tp -> offsetSpec.offset()
+ }
+ }.toMap
+ }
+
+ fetchSpecificOffsets0(fnAssertParametersWithPartitions, fnRetrievePartitionOffsets)
+ }
+
private def fetchSpecificOffsets0(
fnAssertParametersWithPartitions: ju.Set[TopicPartition] => Unit,
fnRetrievePartitionOffsets: ju.Set[TopicPartition] => Map[TopicPartition, Long]
diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReaderConsumer.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReaderConsumer.scala
index ead819e4c27a..0733e6d76157 100644
--- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReaderConsumer.scala
+++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReaderConsumer.scala
@@ -160,6 +160,9 @@ private[kafka010] class KafkaOffsetReaderConsumer(
case SpecificTimestampRangeLimit(partitionTimestamps) =>
fetchSpecificTimestampBasedOffsets(partitionTimestamps,
failsOnNoMatchingOffset = isStartingOffsets).partitionToOffsets
+ case GlobalTimestampRangeLimit(timestamp) =>
+ fetchGlobalTimestampBasedOffsets(timestamp,
+ failsOnNoMatchingOffset = isStartingOffsets).partitionToOffsets
}
}
@@ -234,6 +237,39 @@ private[kafka010] class KafkaOffsetReaderConsumer(
fnAssertFetchedOffsets)
}
+ override def fetchGlobalTimestampBasedOffsets(
+ timestamp: Long,
+ failsOnNoMatchingOffset: Boolean): KafkaSourceOffset = {
+ val fnAssertParametersWithPartitions: ju.Set[TopicPartition] => Unit = { partitions =>
+ logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $timestamp")
+ }
+
+ val fnRetrievePartitionOffsets: ju.Set[TopicPartition] => Map[TopicPartition, Long] = { tps =>
+ val converted = tps.asScala.map(_ -> java.lang.Long.valueOf(timestamp)).toMap.asJava
+
+ val offsetForTime: ju.Map[TopicPartition, OffsetAndTimestamp] =
+ consumer.offsetsForTimes(converted)
+
+ offsetForTime.asScala.map { case (tp, offsetAndTimestamp) =>
+ if (failsOnNoMatchingOffset) {
+ assert(offsetAndTimestamp != null, "No offset matched from request of " +
+ s"topic-partition $tp and timestamp $timestamp.")
+ }
+
+ if (offsetAndTimestamp == null) {
+ tp -> KafkaOffsetRangeLimit.LATEST
+ } else {
+ tp -> offsetAndTimestamp.offset()
+ }
+ }.toMap
+ }
+
+ val fnAssertFetchedOffsets: Map[TopicPartition, Long] => Unit = { _ => }
+
+ fetchSpecificOffsets0(fnAssertParametersWithPartitions, fnRetrievePartitionOffsets,
+ fnAssertFetchedOffsets)
+ }
+
private def fetchSpecificOffsets0(
fnAssertParametersWithPartitions: ju.Set[TopicPartition] => Unit,
fnRetrievePartitionOffsets: ju.Set[TopicPartition] => Map[TopicPartition, Long],
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 1e17f9a7407a..a3b8c6fc31f9 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
@@ -104,6 +104,8 @@ private[kafka010] class KafkaSource(
case SpecificOffsetRangeLimit(p) => kafkaReader.fetchSpecificOffsets(p, reportDataLoss)
case SpecificTimestampRangeLimit(p) =>
kafkaReader.fetchSpecificTimestampBasedOffsets(p, failsOnNoMatchingOffset = true)
+ case GlobalTimestampRangeLimit(ts) =>
+ kafkaReader.fetchGlobalTimestampBasedOffsets(ts, failsOnNoMatchingOffset = true)
}
metadataLog.add(0, offsets)
logInfo(s"Initial offsets: $offsets")
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 0c891cd725ae..b6092e161d1e 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
@@ -90,8 +90,9 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
val specifiedKafkaParams = convertToSpecifiedParams(caseInsensitiveParameters)
val startingStreamOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit(
- caseInsensitiveParameters, STARTING_OFFSETS_BY_TIMESTAMP_OPTION_KEY,
- STARTING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit)
+ caseInsensitiveParameters, STARTING_TIMESTAMP_OPTION_KEY,
+ STARTING_OFFSETS_BY_TIMESTAMP_OPTION_KEY, STARTING_OFFSETS_OPTION_KEY,
+ LatestOffsetRangeLimit)
val kafkaOffsetReader = KafkaOffsetReader.build(
strategy(caseInsensitiveParameters),
@@ -128,13 +129,15 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
val specifiedKafkaParams = convertToSpecifiedParams(caseInsensitiveParameters)
val startingRelationOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit(
- caseInsensitiveParameters, STARTING_OFFSETS_BY_TIMESTAMP_OPTION_KEY,
- STARTING_OFFSETS_OPTION_KEY, EarliestOffsetRangeLimit)
+ caseInsensitiveParameters, STARTING_TIMESTAMP_OPTION_KEY,
+ STARTING_OFFSETS_BY_TIMESTAMP_OPTION_KEY, STARTING_OFFSETS_OPTION_KEY,
+ EarliestOffsetRangeLimit)
assert(startingRelationOffsets != LatestOffsetRangeLimit)
val endingRelationOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit(
- caseInsensitiveParameters, ENDING_OFFSETS_BY_TIMESTAMP_OPTION_KEY,
- ENDING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit)
+ caseInsensitiveParameters, ENDING_TIMESTAMP_OPTION_KEY,
+ ENDING_OFFSETS_BY_TIMESTAMP_OPTION_KEY, ENDING_OFFSETS_OPTION_KEY,
+ LatestOffsetRangeLimit)
assert(endingRelationOffsets != EarliestOffsetRangeLimit)
val includeHeaders = caseInsensitiveParameters.getOrElse(INCLUDE_HEADERS, "false").toBoolean
@@ -334,8 +337,8 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
private def validateBatchOptions(params: CaseInsensitiveMap[String]) = {
// Batch specific options
KafkaSourceProvider.getKafkaOffsetRangeLimit(
- params, STARTING_OFFSETS_BY_TIMESTAMP_OPTION_KEY, STARTING_OFFSETS_OPTION_KEY,
- EarliestOffsetRangeLimit) match {
+ params, STARTING_TIMESTAMP_OPTION_KEY, STARTING_OFFSETS_BY_TIMESTAMP_OPTION_KEY,
+ STARTING_OFFSETS_OPTION_KEY, EarliestOffsetRangeLimit) match {
case EarliestOffsetRangeLimit => // good to go
case LatestOffsetRangeLimit =>
throw new IllegalArgumentException("starting offset can't be latest " +
@@ -348,11 +351,12 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
case _ => // ignore
}
case _: SpecificTimestampRangeLimit => // good to go
+ case _: GlobalTimestampRangeLimit => // good to go
}
KafkaSourceProvider.getKafkaOffsetRangeLimit(
- params, ENDING_OFFSETS_BY_TIMESTAMP_OPTION_KEY, ENDING_OFFSETS_OPTION_KEY,
- LatestOffsetRangeLimit) match {
+ params, ENDING_TIMESTAMP_OPTION_KEY, ENDING_OFFSETS_BY_TIMESTAMP_OPTION_KEY,
+ ENDING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) match {
case EarliestOffsetRangeLimit =>
throw new IllegalArgumentException("ending offset can't be earliest " +
"for batch queries on Kafka")
@@ -365,6 +369,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
case _ => // ignore
}
case _: SpecificTimestampRangeLimit => // good to go
+ case _: GlobalTimestampRangeLimit => // good to go
}
validateGeneralOptions(params)
@@ -421,12 +426,14 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
val specifiedKafkaParams = convertToSpecifiedParams(caseInsensitiveOptions)
val startingRelationOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit(
- caseInsensitiveOptions, STARTING_OFFSETS_BY_TIMESTAMP_OPTION_KEY,
- STARTING_OFFSETS_OPTION_KEY, EarliestOffsetRangeLimit)
+ caseInsensitiveOptions, STARTING_TIMESTAMP_OPTION_KEY,
+ STARTING_OFFSETS_BY_TIMESTAMP_OPTION_KEY, STARTING_OFFSETS_OPTION_KEY,
+ EarliestOffsetRangeLimit)
val endingRelationOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit(
- caseInsensitiveOptions, ENDING_OFFSETS_BY_TIMESTAMP_OPTION_KEY,
- ENDING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit)
+ caseInsensitiveOptions, ENDING_TIMESTAMP_OPTION_KEY,
+ ENDING_OFFSETS_BY_TIMESTAMP_OPTION_KEY, ENDING_OFFSETS_OPTION_KEY,
+ LatestOffsetRangeLimit)
new KafkaBatch(
strategy(caseInsensitiveOptions),
@@ -449,8 +456,9 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
val specifiedKafkaParams = convertToSpecifiedParams(caseInsensitiveOptions)
val startingStreamOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit(
- caseInsensitiveOptions, STARTING_OFFSETS_BY_TIMESTAMP_OPTION_KEY,
- STARTING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit)
+ caseInsensitiveOptions, STARTING_TIMESTAMP_OPTION_KEY,
+ STARTING_OFFSETS_BY_TIMESTAMP_OPTION_KEY, STARTING_OFFSETS_OPTION_KEY,
+ LatestOffsetRangeLimit)
val kafkaOffsetReader = KafkaOffsetReader.build(
strategy(caseInsensitiveOptions),
@@ -478,8 +486,9 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
val specifiedKafkaParams = convertToSpecifiedParams(caseInsensitiveOptions)
val startingStreamOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit(
- caseInsensitiveOptions, STARTING_OFFSETS_BY_TIMESTAMP_OPTION_KEY,
- STARTING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit)
+ caseInsensitiveOptions, STARTING_TIMESTAMP_OPTION_KEY,
+ STARTING_OFFSETS_BY_TIMESTAMP_OPTION_KEY, STARTING_OFFSETS_OPTION_KEY,
+ LatestOffsetRangeLimit)
val kafkaOffsetReader = KafkaOffsetReader.build(
strategy(caseInsensitiveOptions),
@@ -521,6 +530,8 @@ private[kafka010] object KafkaSourceProvider extends Logging {
private[kafka010] val ENDING_OFFSETS_OPTION_KEY = "endingoffsets"
private[kafka010] val STARTING_OFFSETS_BY_TIMESTAMP_OPTION_KEY = "startingoffsetsbytimestamp"
private[kafka010] val ENDING_OFFSETS_BY_TIMESTAMP_OPTION_KEY = "endingoffsetsbytimestamp"
+ private[kafka010] val STARTING_TIMESTAMP_OPTION_KEY = "startingtimestamp"
+ private[kafka010] val ENDING_TIMESTAMP_OPTION_KEY = "endingtimestamp"
private val FAIL_ON_DATA_LOSS_OPTION_KEY = "failondataloss"
private[kafka010] val MIN_PARTITIONS_OPTION_KEY = "minpartitions"
private[kafka010] val MAX_OFFSET_PER_TRIGGER = "maxoffsetspertrigger"
@@ -564,20 +575,36 @@ private[kafka010] object KafkaSourceProvider extends Logging {
def getKafkaOffsetRangeLimit(
params: CaseInsensitiveMap[String],
+ globalOffsetTimestampOptionKey: String,
offsetByTimestampOptionKey: String,
offsetOptionKey: String,
defaultOffsets: KafkaOffsetRangeLimit): KafkaOffsetRangeLimit = {
- params.get(offsetByTimestampOptionKey).map(_.trim) match {
- case Some(json) => SpecificTimestampRangeLimit(JsonUtils.partitionTimestamps(json))
- case None =>
- params.get(offsetOptionKey).map(_.trim) match {
- case Some(offset) if offset.toLowerCase(Locale.ROOT) == "latest" =>
- LatestOffsetRangeLimit
- case Some(offset) if offset.toLowerCase(Locale.ROOT) == "earliest" =>
- EarliestOffsetRangeLimit
- case Some(json) => SpecificOffsetRangeLimit(JsonUtils.partitionOffsets(json))
- case None => defaultOffsets
- }
+ // The order below represents "preferences"
+
+ if (params.contains(globalOffsetTimestampOptionKey)) {
+ // 1. global timestamp
+ val tsStr = params(globalOffsetTimestampOptionKey).trim
+ try {
+ val ts = tsStr.toLong
+ GlobalTimestampRangeLimit(ts)
+ } catch {
+ case _: NumberFormatException =>
+ throw new IllegalArgumentException(s"Expected a single long value, got $tsStr")
+ }
+ } else if (params.contains(offsetByTimestampOptionKey)) {
+ // 2. timestamp per topic partition
+ val json = params(offsetByTimestampOptionKey).trim
+ SpecificTimestampRangeLimit(JsonUtils.partitionTimestamps(json))
+ } else {
+ // 3. latest/earliest/offset
+ params.get(offsetOptionKey).map(_.trim) match {
+ case Some(offset) if offset.toLowerCase(Locale.ROOT) == "latest" =>
+ LatestOffsetRangeLimit
+ case Some(offset) if offset.toLowerCase(Locale.ROOT) == "earliest" =>
+ EarliestOffsetRangeLimit
+ case Some(json) => SpecificOffsetRangeLimit(JsonUtils.partitionOffsets(json))
+ case None => defaultOffsets
+ }
}
}
diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
index 4826e7a25144..ad8939d920a6 100644
--- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
+++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
@@ -1470,6 +1470,16 @@ abstract class KafkaSourceSuiteBase extends KafkaSourceTest {
"failOnDataLoss" -> failOnDataLoss.toString)
}
+ test(s"assign from global timestamp per topic (failOnDataLoss: $failOnDataLoss)") {
+ val topic = newTopic()
+ testFromGlobalTimestamp(
+ topic,
+ failOnDataLoss = failOnDataLoss,
+ addPartitions = false,
+ "assign" -> assignString(topic, 0 to 4),
+ "failOnDataLoss" -> failOnDataLoss.toString)
+ }
+
test(s"subscribing topic by name from latest offsets (failOnDataLoss: $failOnDataLoss)") {
val topic = newTopic()
testFromLatestOffsets(
@@ -1499,6 +1509,13 @@ abstract class KafkaSourceSuiteBase extends KafkaSourceTest {
"subscribe" -> topic)
}
+ test(s"subscribing topic by name from global timestamp per topic" +
+ s" (failOnDataLoss: $failOnDataLoss)") {
+ val topic = newTopic()
+ testFromGlobalTimestamp(topic, failOnDataLoss = failOnDataLoss, addPartitions = true,
+ "subscribe" -> topic)
+ }
+
test(s"subscribing topic by pattern from latest offsets (failOnDataLoss: $failOnDataLoss)") {
val topicPrefix = newTopic()
val topic = topicPrefix + "-suffix"
@@ -1538,6 +1555,17 @@ abstract class KafkaSourceSuiteBase extends KafkaSourceTest {
addPartitions = true,
"subscribePattern" -> s"$topicPrefix-.*")
}
+
+ test(s"subscribing topic by pattern from global timestamp per topic " +
+ s"(failOnDataLoss: $failOnDataLoss)") {
+ val topicPrefix = newTopic()
+ val topic = topicPrefix + "-suffix"
+ testFromGlobalTimestamp(
+ topic,
+ failOnDataLoss = failOnDataLoss,
+ addPartitions = true,
+ "subscribePattern" -> s"$topicPrefix-.*")
+ }
}
test("bad source options") {
@@ -1608,7 +1636,7 @@ abstract class KafkaSourceSuiteBase extends KafkaSourceTest {
(STARTING_OFFSETS_OPTION_KEY, """{"topic-A":{"0":23}}""",
SpecificOffsetRangeLimit(Map(new TopicPartition("topic-A", 0) -> 23))))) {
val offset = getKafkaOffsetRangeLimit(
- CaseInsensitiveMap[String](Map(optionKey -> optionValue)), "dummy", optionKey,
+ CaseInsensitiveMap[String](Map(optionKey -> optionValue)), "dummy", "dummy", optionKey,
answer)
assert(offset === answer)
}
@@ -1617,7 +1645,7 @@ abstract class KafkaSourceSuiteBase extends KafkaSourceTest {
(STARTING_OFFSETS_OPTION_KEY, EarliestOffsetRangeLimit),
(ENDING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit))) {
val offset = getKafkaOffsetRangeLimit(
- CaseInsensitiveMap[String](Map.empty), "dummy", optionKey, answer)
+ CaseInsensitiveMap[String](Map.empty), "dummy", "dummy", optionKey, answer)
assert(offset === answer)
}
}
@@ -1687,27 +1715,11 @@ abstract class KafkaSourceSuiteBase extends KafkaSourceTest {
failOnDataLoss: Boolean,
addPartitions: Boolean,
options: (String, String)*): Unit = {
- def sendMessages(topic: String, msgs: Seq[String], part: Int, ts: Long): Unit = {
- val records = msgs.map { msg =>
- new RecordBuilder(topic, msg).partition(part).timestamp(ts).build()
- }
- testUtils.sendMessages(records)
- }
-
testUtils.createTopic(topic, partitions = 5)
val firstTimestamp = System.currentTimeMillis() - 5000
- sendMessages(topic, Array(-20).map(_.toString), 0, firstTimestamp)
- sendMessages(topic, Array(-10).map(_.toString), 1, firstTimestamp)
- sendMessages(topic, Array(0, 1).map(_.toString), 2, firstTimestamp)
- sendMessages(topic, Array(10, 11).map(_.toString), 3, firstTimestamp)
- sendMessages(topic, Array(20, 21, 22).map(_.toString), 4, firstTimestamp)
-
val secondTimestamp = firstTimestamp + 1000
- sendMessages(topic, Array(-21, -22).map(_.toString), 0, secondTimestamp)
- sendMessages(topic, Array(-11, -12).map(_.toString), 1, secondTimestamp)
- sendMessages(topic, Array(2).map(_.toString), 2, secondTimestamp)
- sendMessages(topic, Array(12).map(_.toString), 3, secondTimestamp)
+ setupTestMessagesForTestOnTimestampOffsets(topic, firstTimestamp, secondTimestamp)
// no data after second timestamp for partition 4
require(testUtils.getLatestOffsets(Set(topic)).size === 5)
@@ -1719,18 +1731,8 @@ abstract class KafkaSourceSuiteBase extends KafkaSourceTest {
) ++ Map(new TopicPartition(topic, 4) -> firstTimestamp)
val startingTimestamps = JsonUtils.partitionTimestamps(startPartitionTimestamps)
- val reader = spark
- .readStream
- .format("kafka")
- .option("startingOffsetsByTimestamp", startingTimestamps)
- .option("kafka.bootstrap.servers", testUtils.brokerAddress)
- .option("kafka.metadata.max.age.ms", "1")
- .option("failOnDataLoss", failOnDataLoss.toString)
- options.foreach { case (k, v) => reader.option(k, v) }
- 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 mapped = setupDataFrameForTestOnTimestampOffsets(startingTimestamps, failOnDataLoss,
+ options: _*)
testStream(mapped)(
makeSureGetOffsetCalled,
@@ -1758,6 +1760,123 @@ abstract class KafkaSourceSuiteBase extends KafkaSourceTest {
)
}
+ private def testFromGlobalTimestamp(
+ topic: String,
+ failOnDataLoss: Boolean,
+ addPartitions: Boolean,
+ options: (String, String)*): Unit = {
+ testUtils.createTopic(topic, partitions = 5)
+
+ val firstTimestamp = System.currentTimeMillis() - 5000
+ val secondTimestamp = firstTimestamp + 1000
+ setupTestMessagesForTestOnTimestampOffsets(topic, firstTimestamp, secondTimestamp)
+ // here we should add records in partition 4 which match with second timestamp
+ // as the query will break if there's no matching records
+ sendMessagesWithTimestamp(topic, Array(23, 24).map(_.toString), 4, secondTimestamp)
+
+ require(testUtils.getLatestOffsets(Set(topic)).size === 5)
+
+ // we intentionally starts from second timestamp for all partitions
+ // via setting global partition
+ val mapped = setupDataFrameForTestOnGlobalTimestamp(secondTimestamp, failOnDataLoss,
+ options: _*)
+ testStream(mapped)(
+ makeSureGetOffsetCalled,
+ Execute { q =>
+ // wait to reach the last offset in every partition
+ val partAndOffsets = (0 to 4).map(new TopicPartition(topic, _)).map { tp =>
+ if (tp.partition() < 4) {
+ tp -> 3L
+ } else {
+ tp -> 5L // we added 2 more records to partition 4
+ }
+ }.toMap
+ q.awaitOffset(0, KafkaSourceOffset(partAndOffsets), streamingTimeout.toMillis)
+ },
+ CheckAnswer(-21, -22, -11, -12, 2, 12, 23, 24),
+ StopStream,
+ StartStream(),
+ CheckAnswer(-21, -22, -11, -12, 2, 12, 23, 24), // Should get the data back on recovery
+ StopStream,
+ AddKafkaData(Set(topic), 30, 31, 32), // Add data when stream is stopped
+ StartStream(),
+ CheckAnswer(-21, -22, -11, -12, 2, 12, 23, 24, 30, 31, 32), // Should get the added data
+ AssertOnQuery("Add partitions") { query: StreamExecution =>
+ if (addPartitions) setTopicPartitions(topic, 10, query)
+ true
+ },
+ AddKafkaData(Set(topic), 40, 41, 42, 43, 44)(ensureDataInMultiplePartition = true),
+ CheckAnswer(-21, -22, -11, -12, 2, 12, 23, 24, 30, 31, 32, 40, 41, 42, 43, 44),
+ StopStream
+ )
+ }
+
+ private def sendMessagesWithTimestamp(
+ topic: String,
+ msgs: Seq[String],
+ part: Int,
+ ts: Long): Unit = {
+ val records = msgs.map { msg =>
+ new RecordBuilder(topic, msg).partition(part).timestamp(ts).build()
+ }
+ testUtils.sendMessages(records)
+ }
+
+ private def setupTestMessagesForTestOnTimestampOffsets(
+ topic: String,
+ firstTimestamp: Long,
+ secondTimestamp: Long): Unit = {
+ sendMessagesWithTimestamp(topic, Array(-20).map(_.toString), 0, firstTimestamp)
+ sendMessagesWithTimestamp(topic, Array(-10).map(_.toString), 1, firstTimestamp)
+ sendMessagesWithTimestamp(topic, Array(0, 1).map(_.toString), 2, firstTimestamp)
+ sendMessagesWithTimestamp(topic, Array(10, 11).map(_.toString), 3, firstTimestamp)
+ sendMessagesWithTimestamp(topic, Array(20, 21, 22).map(_.toString), 4, firstTimestamp)
+
+ sendMessagesWithTimestamp(topic, Array(-21, -22).map(_.toString), 0, secondTimestamp)
+ sendMessagesWithTimestamp(topic, Array(-11, -12).map(_.toString), 1, secondTimestamp)
+ sendMessagesWithTimestamp(topic, Array(2).map(_.toString), 2, secondTimestamp)
+ sendMessagesWithTimestamp(topic, Array(12).map(_.toString), 3, secondTimestamp)
+ // no data after second timestamp for partition 4
+ }
+
+ private def setupDataFrameForTestOnTimestampOffsets(
+ startingTimestamps: String,
+ failOnDataLoss: Boolean,
+ options: (String, String)*): Dataset[_] = {
+ val reader = spark
+ .readStream
+ .format("kafka")
+ .option("startingOffsetsByTimestamp", startingTimestamps)
+ .option("kafka.bootstrap.servers", testUtils.brokerAddress)
+ .option("kafka.metadata.max.age.ms", "1")
+ .option("failOnDataLoss", failOnDataLoss.toString)
+ options.foreach { case (k, v) => reader.option(k, v) }
+ val kafka = reader.load()
+ .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
+ .as[(String, String)]
+ val mapped: org.apache.spark.sql.Dataset[_] = kafka.map(kv => kv._2.toInt)
+ mapped
+ }
+
+ private def setupDataFrameForTestOnGlobalTimestamp(
+ startingTimestamp: Long,
+ failOnDataLoss: Boolean,
+ options: (String, String)*): Dataset[_] = {
+ val reader = spark
+ .readStream
+ .format("kafka")
+ .option("startingTimestamp", startingTimestamp)
+ .option("kafka.bootstrap.servers", testUtils.brokerAddress)
+ .option("kafka.metadata.max.age.ms", "1")
+ .option("failOnDataLoss", failOnDataLoss.toString)
+ options.foreach { case (k, v) => reader.option(k, v) }
+ val kafka = reader.load()
+ .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
+ .as[(String, String)]
+ val mapped: org.apache.spark.sql.Dataset[_] = kafka.map(kv => kv._2.toInt)
+ mapped
+ }
+
test("Kafka column types") {
val now = System.currentTimeMillis()
val topic = newTopic()
diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala
index 9cec37e708db..6e1ece3f436a 100644
--- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala
+++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala
@@ -263,6 +263,15 @@ abstract class KafkaRelationSuiteBase extends QueryTest with SharedSparkSession
}, topic, 0 to 19)
}
+ test("global timestamp provided for starting and ending") {
+ val (topic, timestamps) = prepareTimestampRelatedUnitTest
+
+ // timestamp both presented: starting "first" ending "finalized"
+ verifyTimestampRelatedQueryResult({ df =>
+ df.option("startingTimestamp", timestamps(1)).option("endingTimestamp", timestamps(2))
+ }, topic, 10 to 19)
+ }
+
test("no matched offset for timestamp - startingOffsets") {
val (topic, timestamps) = prepareTimestampRelatedUnitTest
@@ -284,6 +293,44 @@ abstract class KafkaRelationSuiteBase extends QueryTest with SharedSparkSession
TestUtils.assertExceptionMsg(e, "No offset matched from request")
}
+ test("preferences on offset related options") {
+ val (topic, timestamps) = prepareTimestampRelatedUnitTest
+
+ /*
+ The test will set both configs differently:
+
+ * global timestamp
+ starting only presented as "third", and ending not presented
+
+ * specific timestamp for partition
+ starting only presented as "second", and ending not presented
+
+ * offsets
+ starting only presented as "earliest", and ending not presented
+
+ The preference goes to global timestamp -> timestamp for partition -> offsets
+ */
+
+ val startTopicTimestamps = Map(
+ (0 to 2).map(new TopicPartition(topic, _) -> timestamps(1)): _*)
+ val startingTimestamps = JsonUtils.partitionTimestamps(startTopicTimestamps)
+
+ // all options are specified: global timestamp
+ verifyTimestampRelatedQueryResult({ df =>
+ df
+ .option("startingTimestamp", timestamps(2))
+ .option("startingOffsetsByTimestamp", startingTimestamps)
+ .option("startingOffsets", "earliest")
+ }, topic, 20 to 29)
+
+ // timestamp for partition and offsets are specified: timestamp for partition
+ verifyTimestampRelatedQueryResult({ df =>
+ df
+ .option("startingOffsetsByTimestamp", startingTimestamps)
+ .option("startingOffsets", "earliest")
+ }, topic, 10 to 29)
+ }
+
test("no matched offset for timestamp - endingOffsets") {
val (topic, timestamps) = prepareTimestampRelatedUnitTest