Skip to content

Commit a57afd4

Browse files
committed
[SPARK-29223][SQL][SS] New option to specify timestamp on all subscribing topic-partitions in Kafka source
### What changes were proposed in this pull request? This patch is a follow-up of SPARK-26848 (#23747). In SPARK-26848, we decided to open possibility to let end users set individual timestamp per partition. But in many cases, specifying timestamp represents the intention that we would want to go back to specific timestamp and reprocess records, which should be applied to all topics and partitions. This patch proposes to provide a way to set a global timestamp across topic-partitions which the source is subscribing to, so that end users can set all offsets by specific timestamp easily. To provide the way to config the timestamp easier, the new options only receive "a" timestamp for start/end timestamp. New options introduced in this PR: * startingTimestamp * endingTimestamp All two options receive timestamp as string. There're priorities for options regarding starting/ending offset as we will have three options for start offsets and another three options for end offsets. Priorities are following: * starting offsets: startingTimestamp -> startingOffsetsByTimestamp -> startingOffsets * ending offsets: startingTimestamp -> startingOffsetsByTimestamp -> startingOffsets ### Why are the changes needed? Existing option to specify timestamp as offset is quite verbose if there're a lot of partitions across topics. Suppose there're 100s of partitions in a topic, the json should contain 100s of times of the same timestamp. Also, the number of partitions can also change, which requires either: * fixing the code if the json is statically created * introducing the dependencies on Kafka client and deal with Kafka API on crafting json programmatically Both approaches are even not "acceptable" if we're dealing with ad-hoc query; anyone doesn't want to write the code more complicated than the query itself. Flink [provides the option](https://ci.apache.org/projects/flink/flink-docs-release-1.13/docs/connectors/datastream/kafka/#kafka-consumers-start-position-configuration) to specify a timestamp for all topic-partitions like this PR, and even doesn't provide the option to specify the timestamp per topic-partition. With this PR, end users are only required to provide a single timestamp value. No more complicated JSON format end users need to know about the structure. ### Does this PR introduce _any_ user-facing change? Yes, this PR introduces two new options, described in above section. Doc changes are following: ![스크린샷 2021-05-21 오후 12 01 02](https://user-images.githubusercontent.com/1317309/119076244-3034e680-ba2d-11eb-8323-0e227932d2e5.png) ![스크린샷 2021-05-21 오후 12 01 12](https://user-images.githubusercontent.com/1317309/119076255-35923100-ba2d-11eb-9d79-538a7f9ee738.png) ![스크린샷 2021-05-21 오후 12 01 24](https://user-images.githubusercontent.com/1317309/119076264-39be4e80-ba2d-11eb-8265-ac158f55c360.png) ![스크린샷 2021-05-21 오후 12 06 01](https://user-images.githubusercontent.com/1317309/119076271-3d51d580-ba2d-11eb-98ea-35fd72b1bbfc.png) ### How was this patch tested? New UTs covering new functionalities. Also manually tested via simple batch & streaming queries. Closes #32609 from HeartSaVioR/SPARK-29223-v2. Authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com> Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
1 parent 58d4da1 commit a57afd4

File tree

11 files changed

+394
-79
lines changed

11 files changed

+394
-79
lines changed

docs/structured-streaming-kafka-integration.md

Lines changed: 44 additions & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -362,24 +362,33 @@ The following configurations are optional:
362362

363363
<table class="table">
364364
<tr><th>Option</th><th>value</th><th>default</th><th>query type</th><th>meaning</th></tr>
365+
<tr>
366+
<td>startingTimestamp</td>
367+
<td>timestamp string e.g. "1000"</td>
368+
<td>none (next preference is <code>startingOffsetsByTimestamp</code>)</td>
369+
<td>streaming and batch</td>
370+
<td>The start point of timestamp when a query is started, a string specifying a starting timestamp for
371+
all partitions in topics being subscribed. Please refer the details on timestamp offset options below. If Kafka doesn't return the matched offset,
372+
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.)<p/>
373+
<p/>
374+
Note1: <code>startingTimestamp</code> takes precedence over <code>startingOffsetsByTimestamp</code> and <code>startingOffsets</code>.<p/>
375+
Note2: For streaming queries, this only applies when a new query is started, and that resuming will
376+
always pick up from where the query left off. Newly discovered partitions during a query will start at
377+
earliest.</td>
378+
</tr>
365379
<tr>
366380
<td>startingOffsetsByTimestamp</td>
367381
<td>json string
368382
""" {"topicA":{"0": 1000, "1": 1000}, "topicB": {"0": 2000, "1": 2000}} """
369383
</td>
370-
<td>none (the value of <code>startingOffsets</code> will apply)</td>
384+
<td>none (next preference is <code>startingOffsets</code>)</td>
371385
<td>streaming and batch</td>
372386
<td>The start point of timestamp when a query is started, a json string specifying a starting timestamp for
373-
each TopicPartition. The returned offset for each partition is the earliest offset whose timestamp is greater than or
374-
equal to the given timestamp in the corresponding partition. If the matched offset doesn't exist,
387+
each TopicPartition. Please refer the details on timestamp offset options below. If Kafka doesn't return the matched offset,
375388
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.)<p/>
376389
<p/>
377-
Spark simply passes the timestamp information to <code>KafkaConsumer.offsetsForTimes</code>, and doesn't interpret or reason about the value. <p/>
378-
For more details on <code>KafkaConsumer.offsetsForTimes</code>, please refer <a href="https://kafka.apache.org/21/javadoc/org/apache/kafka/clients/consumer/KafkaConsumer.html#offsetsForTimes-java.util.Map-">javadoc</a> for details.<p/>
379-
Also the meaning of <code>timestamp</code> here can be vary according to Kafka configuration (<code>log.message.timestamp.type</code>): please refer <a href="https://kafka.apache.org/documentation/">Kafka documentation</a> for further details.<p/>
380-
Note: This option requires Kafka 0.10.1.0 or higher.<p/>
381-
Note2: <code>startingOffsetsByTimestamp</code> takes precedence over <code>startingOffsets</code>.<p/>
382-
Note3: For streaming queries, this only applies when a new query is started, and that resuming will
390+
Note1: <code>startingOffsetsByTimestamp</code> takes precedence over <code>startingOffsets</code>.<p/>
391+
Note2: For streaming queries, this only applies when a new query is started, and that resuming will
383392
always pick up from where the query left off. Newly discovered partitions during a query will start at
384393
earliest.</td>
385394
</tr>
@@ -398,23 +407,28 @@ The following configurations are optional:
398407
always pick up from where the query left off. Newly discovered partitions during a query will start at
399408
earliest.</td>
400409
</tr>
410+
<tr>
411+
<td>endingTimestamp</td>
412+
<td>timestamp string e.g. "1000"</td>
413+
<td>none (next preference is <code>endingOffsetsByTimestamp</code>)</td>
414+
<td>batch query</td>
415+
<td>The end point when a batch query is ended, a json string specifying an ending timestamp for
416+
all partitions in topics being subscribed. Please refer the details on timestamp offset options below.
417+
If Kafka doesn't return the matched offset, the offset will be set to latest.<p/>
418+
Note: <code>endingTimestamp</code> takes precedence over <code>endingOffsetsByTimestamp</code> and <code>endingOffsets</code>.<p/>
419+
</td>
420+
</tr>
401421
<tr>
402422
<td>endingOffsetsByTimestamp</td>
403423
<td>json string
404424
""" {"topicA":{"0": 1000, "1": 1000}, "topicB": {"0": 2000, "1": 2000}} """
405425
</td>
406-
<td>latest</td>
426+
<td>none (next preference is <code>endingOffsets</code>)</td>
407427
<td>batch query</td>
408428
<td>The end point when a batch query is ended, a json string specifying an ending timestamp for each TopicPartition.
409-
The returned offset for each partition is the earliest offset whose timestamp is greater than or equal to
410-
the given timestamp in the corresponding partition. If the matched offset doesn't exist, the offset will
411-
be set to latest.<p/>
412-
<p/>
413-
Spark simply passes the timestamp information to <code>KafkaConsumer.offsetsForTimes</code>, and doesn't interpret or reason about the value. <p/>
414-
For more details on <code>KafkaConsumer.offsetsForTimes</code>, please refer <a href="https://kafka.apache.org/21/javadoc/org/apache/kafka/clients/consumer/KafkaConsumer.html#offsetsForTimes-java.util.Map-">javadoc</a> for details.<p/>
415-
Also the meaning of <code>timestamp</code> here can be vary according to Kafka configuration (<code>log.message.timestamp.type</code>): please refer <a href="https://kafka.apache.org/documentation/">Kafka documentation</a> for further details.<p/>
416-
Note: This option requires Kafka 0.10.1.0 or higher.<p/>
417-
Note2: <code>endingOffsetsByTimestamp</code> takes precedence over <code>endingOffsets</code>.
429+
Please refer the details on timestamp offset options below. If Kafka doesn't return the matched offset,
430+
the offset will be set to latest.<p/>
431+
Note: <code>endingOffsetsByTimestamp</code> takes precedence over <code>endingOffsets</code>.
418432
</td>
419433
</tr>
420434
<tr>
@@ -512,6 +526,17 @@ The following configurations are optional:
512526
</tr>
513527
</table>
514528

529+
### Details on timestamp offset options
530+
531+
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.
532+
The behavior varies across options if Kafka doesn't return the matched offset - check the description of each option.
533+
534+
Spark simply passes the timestamp information to <code>KafkaConsumer.offsetsForTimes</code>, and doesn't interpret or reason about the value.
535+
For more details on <code>KafkaConsumer.offsetsForTimes</code>, please refer <a href="http://kafka.apache.org/0101/javadoc/org/apache/kafka/clients/consumer/KafkaConsumer.html#offsetsForTimes(java.util.Map)">javadoc</a> for details.
536+
Also, the meaning of <code>timestamp</code> here can be vary according to Kafka configuration (<code>log.message.timestamp.type</code>): please refer <a href="https://kafka.apache.org/documentation/">Kafka documentation</a> for further details.
537+
538+
Timestamp offset options require Kafka 0.10.1.0 or higher.
539+
515540
### Offset fetching
516541

517542
In Spark 3.0 and before Spark uses <code>KafkaConsumer</code> for offset fetching which could cause infinite wait in the driver.

external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -72,6 +72,8 @@ class KafkaContinuousStream(
7272
case SpecificOffsetRangeLimit(p) => offsetReader.fetchSpecificOffsets(p, reportDataLoss)
7373
case SpecificTimestampRangeLimit(p) => offsetReader.fetchSpecificTimestampBasedOffsets(p,
7474
failsOnNoMatchingOffset = true)
75+
case GlobalTimestampRangeLimit(ts) => offsetReader.fetchGlobalTimestampBasedOffsets(
76+
ts, failsOnNoMatchingOffset = true)
7577
}
7678
logInfo(s"Initial offsets: $offsets")
7779
offsets

external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -166,6 +166,8 @@ private[kafka010] class KafkaMicroBatchStream(
166166
kafkaOffsetReader.fetchSpecificOffsets(p, reportDataLoss)
167167
case SpecificTimestampRangeLimit(p) =>
168168
kafkaOffsetReader.fetchSpecificTimestampBasedOffsets(p, failsOnNoMatchingOffset = true)
169+
case GlobalTimestampRangeLimit(ts) =>
170+
kafkaOffsetReader.fetchGlobalTimestampBasedOffsets(ts, failsOnNoMatchingOffset = true)
169171
}
170172
metadataLog.add(0, offsets)
171173
logInfo(s"Initial offsets: $offsets")

external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeLimit.scala

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -49,6 +49,13 @@ private[kafka010] case class SpecificOffsetRangeLimit(
4949
private[kafka010] case class SpecificTimestampRangeLimit(
5050
topicTimestamps: Map[TopicPartition, Long]) extends KafkaOffsetRangeLimit
5151

52+
/**
53+
* Represents the desire to bind to earliest offset which timestamp for the offset is equal or
54+
* greater than specific timestamp. This applies the timestamp to the all topics/partitions.
55+
*/
56+
private[kafka010] case class GlobalTimestampRangeLimit(
57+
timestamp: Long) extends KafkaOffsetRangeLimit
58+
5259
private[kafka010] object KafkaOffsetRangeLimit {
5360
/**
5461
* Used to denote offset range limits that are resolved via Kafka

external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala

Lines changed: 14 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -77,6 +77,20 @@ private[kafka010] trait KafkaOffsetReader {
7777
partitionTimestamps: Map[TopicPartition, Long],
7878
failsOnNoMatchingOffset: Boolean): KafkaSourceOffset
7979

80+
/**
81+
* Resolves the specific offsets based on timestamp per all topic-partitions being subscribed.
82+
* The returned offset for each partition is the earliest offset whose timestamp is greater
83+
* than or equal to the given timestamp in the corresponding partition. If the matched offset
84+
* doesn't exist, depending on `failsOnNoMatchingOffset` parameter, the offset will be set to
85+
* latest or this method throws an error.
86+
*
87+
* @param timestamp the timestamp.
88+
* @param failsOnNoMatchingOffset whether to fail the query when no matched offset can be found.
89+
*/
90+
def fetchGlobalTimestampBasedOffsets(
91+
timestamp: Long,
92+
failsOnNoMatchingOffset: Boolean): KafkaSourceOffset
93+
8094
/**
8195
* Fetch the earliest offsets for the topic partitions that are indicated
8296
* in the [[ConsumerStrategy]].

external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReaderAdmin.scala

Lines changed: 34 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -138,6 +138,9 @@ private[kafka010] class KafkaOffsetReaderAdmin(
138138
case SpecificTimestampRangeLimit(partitionTimestamps) =>
139139
fetchSpecificTimestampBasedOffsets(partitionTimestamps,
140140
failsOnNoMatchingOffset = isStartingOffsets).partitionToOffsets
141+
case GlobalTimestampRangeLimit(timestamp) =>
142+
fetchGlobalTimestampBasedOffsets(timestamp,
143+
failsOnNoMatchingOffset = isStartingOffsets).partitionToOffsets
141144
}
142145
}
143146

@@ -193,6 +196,37 @@ private[kafka010] class KafkaOffsetReaderAdmin(
193196
fetchSpecificOffsets0(fnAssertParametersWithPartitions, fnRetrievePartitionOffsets)
194197
}
195198

199+
200+
override def fetchGlobalTimestampBasedOffsets(
201+
timestamp: Long,
202+
failsOnNoMatchingOffset: Boolean): KafkaSourceOffset = {
203+
val fnAssertParametersWithPartitions: ju.Set[TopicPartition] => Unit = { partitions =>
204+
logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $timestamp")
205+
}
206+
207+
val fnRetrievePartitionOffsets: ju.Set[TopicPartition] => Map[TopicPartition, Long] = { tps =>
208+
val listOffsetsParams = tps.asScala.map { tp =>
209+
tp -> OffsetSpec.forTimestamp(timestamp)
210+
}.toMap.asJava
211+
admin.listOffsets(listOffsetsParams, listOffsetsOptions).all().get().asScala.map {
212+
case (tp, offsetSpec) =>
213+
if (failsOnNoMatchingOffset) {
214+
assert(offsetSpec.offset() != OffsetFetchResponse.INVALID_OFFSET, "No offset " +
215+
s"matched from request of topic-partition $tp and timestamp " +
216+
s"$timestamp.")
217+
}
218+
219+
if (offsetSpec.offset() == OffsetFetchResponse.INVALID_OFFSET) {
220+
tp -> KafkaOffsetRangeLimit.LATEST
221+
} else {
222+
tp -> offsetSpec.offset()
223+
}
224+
}.toMap
225+
}
226+
227+
fetchSpecificOffsets0(fnAssertParametersWithPartitions, fnRetrievePartitionOffsets)
228+
}
229+
196230
private def fetchSpecificOffsets0(
197231
fnAssertParametersWithPartitions: ju.Set[TopicPartition] => Unit,
198232
fnRetrievePartitionOffsets: ju.Set[TopicPartition] => Map[TopicPartition, Long]

external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReaderConsumer.scala

Lines changed: 36 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -160,6 +160,9 @@ private[kafka010] class KafkaOffsetReaderConsumer(
160160
case SpecificTimestampRangeLimit(partitionTimestamps) =>
161161
fetchSpecificTimestampBasedOffsets(partitionTimestamps,
162162
failsOnNoMatchingOffset = isStartingOffsets).partitionToOffsets
163+
case GlobalTimestampRangeLimit(timestamp) =>
164+
fetchGlobalTimestampBasedOffsets(timestamp,
165+
failsOnNoMatchingOffset = isStartingOffsets).partitionToOffsets
163166
}
164167
}
165168

@@ -234,6 +237,39 @@ private[kafka010] class KafkaOffsetReaderConsumer(
234237
fnAssertFetchedOffsets)
235238
}
236239

240+
override def fetchGlobalTimestampBasedOffsets(
241+
timestamp: Long,
242+
failsOnNoMatchingOffset: Boolean): KafkaSourceOffset = {
243+
val fnAssertParametersWithPartitions: ju.Set[TopicPartition] => Unit = { partitions =>
244+
logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $timestamp")
245+
}
246+
247+
val fnRetrievePartitionOffsets: ju.Set[TopicPartition] => Map[TopicPartition, Long] = { tps =>
248+
val converted = tps.asScala.map(_ -> java.lang.Long.valueOf(timestamp)).toMap.asJava
249+
250+
val offsetForTime: ju.Map[TopicPartition, OffsetAndTimestamp] =
251+
consumer.offsetsForTimes(converted)
252+
253+
offsetForTime.asScala.map { case (tp, offsetAndTimestamp) =>
254+
if (failsOnNoMatchingOffset) {
255+
assert(offsetAndTimestamp != null, "No offset matched from request of " +
256+
s"topic-partition $tp and timestamp $timestamp.")
257+
}
258+
259+
if (offsetAndTimestamp == null) {
260+
tp -> KafkaOffsetRangeLimit.LATEST
261+
} else {
262+
tp -> offsetAndTimestamp.offset()
263+
}
264+
}.toMap
265+
}
266+
267+
val fnAssertFetchedOffsets: Map[TopicPartition, Long] => Unit = { _ => }
268+
269+
fetchSpecificOffsets0(fnAssertParametersWithPartitions, fnRetrievePartitionOffsets,
270+
fnAssertFetchedOffsets)
271+
}
272+
237273
private def fetchSpecificOffsets0(
238274
fnAssertParametersWithPartitions: ju.Set[TopicPartition] => Unit,
239275
fnRetrievePartitionOffsets: ju.Set[TopicPartition] => Map[TopicPartition, Long],

external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -104,6 +104,8 @@ private[kafka010] class KafkaSource(
104104
case SpecificOffsetRangeLimit(p) => kafkaReader.fetchSpecificOffsets(p, reportDataLoss)
105105
case SpecificTimestampRangeLimit(p) =>
106106
kafkaReader.fetchSpecificTimestampBasedOffsets(p, failsOnNoMatchingOffset = true)
107+
case GlobalTimestampRangeLimit(ts) =>
108+
kafkaReader.fetchGlobalTimestampBasedOffsets(ts, failsOnNoMatchingOffset = true)
107109
}
108110
metadataLog.add(0, offsets)
109111
logInfo(s"Initial offsets: $offsets")

0 commit comments

Comments
 (0)