/
KafkaSourceRDD.scala
211 lines (186 loc) · 8.17 KB
/
KafkaSourceRDD.scala
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
/*
* 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 => ju}
import scala.collection.mutable.ArrayBuffer
import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord}
import org.apache.kafka.common.TopicPartition
import org.apache.spark.{Partition, SparkContext, TaskContext}
import org.apache.spark.partial.{BoundedDouble, PartialResult}
import org.apache.spark.rdd.RDD
import org.apache.spark.storage.StorageLevel
import org.apache.spark.util.NextIterator
/** Offset range that one partition of the KafkaSourceRDD has to read */
private[kafka010] case class KafkaSourceRDDOffsetRange(
topicPartition: TopicPartition,
fromOffset: Long,
untilOffset: Long,
preferredLoc: Option[String]) {
def topic: String = topicPartition.topic
def partition: Int = topicPartition.partition
def size: Long = untilOffset - fromOffset
}
/** Partition of the KafkaSourceRDD */
private[kafka010] case class KafkaSourceRDDPartition(
index: Int, offsetRange: KafkaSourceRDDOffsetRange) extends Partition
/**
* An RDD that reads data from Kafka based on offset ranges across multiple partitions.
* Additionally, it allows preferred locations to be set for each topic + partition, so that
* the [[KafkaSource]] can ensure the same executor always reads the same topic + partition
* and cached KafkaConsuemrs (see [[CachedKafkaConsumer]] can be used read data efficiently.
*
* @param sc the [[SparkContext]]
* @param executorKafkaParams Kafka configuration for creating KafkaConsumer on the executors
* @param offsetRanges Offset ranges that define the Kafka data belonging to this RDD
*/
private[kafka010] class KafkaSourceRDD(
sc: SparkContext,
executorKafkaParams: ju.Map[String, Object],
offsetRanges: Seq[KafkaSourceRDDOffsetRange],
pollTimeoutMs: Long,
failOnDataLoss: Boolean,
reuseKafkaConsumer: Boolean)
extends RDD[ConsumerRecord[Array[Byte], Array[Byte]]](sc, Nil) {
override def persist(newLevel: StorageLevel): this.type = {
logError("Kafka ConsumerRecord is not serializable. " +
"Use .map to extract fields before calling .persist or .window")
super.persist(newLevel)
}
override def getPartitions: Array[Partition] = {
offsetRanges.zipWithIndex.map { case (o, i) => new KafkaSourceRDDPartition(i, o) }.toArray
}
override def count(): Long = offsetRanges.map(_.size).sum
override def countApprox(timeout: Long, confidence: Double): PartialResult[BoundedDouble] = {
val c = count
new PartialResult(new BoundedDouble(c, 1.0, c, c), true)
}
override def isEmpty(): Boolean = count == 0L
override def take(num: Int): Array[ConsumerRecord[Array[Byte], Array[Byte]]] = {
val nonEmptyPartitions =
this.partitions.map(_.asInstanceOf[KafkaSourceRDDPartition]).filter(_.offsetRange.size > 0)
if (num < 1 || nonEmptyPartitions.isEmpty) {
return new Array[ConsumerRecord[Array[Byte], Array[Byte]]](0)
}
// Determine in advance how many messages need to be taken from each partition
val parts = nonEmptyPartitions.foldLeft(Map[Int, Int]()) { (result, part) =>
val remain = num - result.values.sum
if (remain > 0) {
val taken = Math.min(remain, part.offsetRange.size)
result + (part.index -> taken.toInt)
} else {
result
}
}
val buf = new ArrayBuffer[ConsumerRecord[Array[Byte], Array[Byte]]]
val res = context.runJob(
this,
(tc: TaskContext, it: Iterator[ConsumerRecord[Array[Byte], Array[Byte]]]) =>
it.take(parts(tc.partitionId)).toArray, parts.keys.toArray
)
res.foreach(buf ++= _)
buf.toArray
}
override def getPreferredLocations(split: Partition): Seq[String] = {
val part = split.asInstanceOf[KafkaSourceRDDPartition]
part.offsetRange.preferredLoc.map(Seq(_)).getOrElse(Seq.empty)
}
override def compute(
thePart: Partition,
context: TaskContext): Iterator[ConsumerRecord[Array[Byte], Array[Byte]]] = {
val sourcePartition = thePart.asInstanceOf[KafkaSourceRDDPartition]
val topic = sourcePartition.offsetRange.topic
val kafkaPartition = sourcePartition.offsetRange.partition
val consumer =
if (!reuseKafkaConsumer) {
// If we can't reuse CachedKafkaConsumers, creating a new CachedKafkaConsumer. As here we
// uses `assign`, we don't need to worry about the "group.id" conflicts.
CachedKafkaConsumer.createUncached(topic, kafkaPartition, executorKafkaParams)
} else {
CachedKafkaConsumer.getOrCreate(topic, kafkaPartition, executorKafkaParams)
}
val range = resolveRange(consumer, sourcePartition.offsetRange)
assert(
range.fromOffset <= range.untilOffset,
s"Beginning offset ${range.fromOffset} is after the ending offset ${range.untilOffset} " +
s"for topic ${range.topic} partition ${range.partition}. " +
"You either provided an invalid fromOffset, or the Kafka topic has been damaged")
if (range.fromOffset == range.untilOffset) {
logInfo(s"Beginning offset ${range.fromOffset} is the same as ending offset " +
s"skipping ${range.topic} ${range.partition}")
Iterator.empty
} else {
val underlying = new NextIterator[ConsumerRecord[Array[Byte], Array[Byte]]]() {
var requestOffset = range.fromOffset
override def getNext(): ConsumerRecord[Array[Byte], Array[Byte]] = {
if (requestOffset >= range.untilOffset) {
// Processed all offsets in this partition.
finished = true
null
} else {
val r = consumer.get(requestOffset, range.untilOffset, pollTimeoutMs, failOnDataLoss)
if (r == null) {
// Losing some data. Skip the rest offsets in this partition.
finished = true
null
} else {
requestOffset = r.offset + 1
r
}
}
}
override protected def close(): Unit = {
if (!reuseKafkaConsumer) {
// Don't forget to close non-reuse KafkaConsumers. You may take down your cluster!
consumer.close()
} else {
// Indicate that we're no longer using this consumer
CachedKafkaConsumer.releaseKafkaConsumer(topic, kafkaPartition, executorKafkaParams)
}
}
}
// Release consumer, either by removing it or indicating we're no longer using it
context.addTaskCompletionListener { _ =>
underlying.closeIfNeeded()
}
underlying
}
}
private def resolveRange(consumer: CachedKafkaConsumer, range: KafkaSourceRDDOffsetRange) = {
if (range.fromOffset < 0 || range.untilOffset < 0) {
// Late bind the offset range
val availableOffsetRange = consumer.getAvailableOffsetRange()
val fromOffset = if (range.fromOffset < 0) {
assert(range.fromOffset == KafkaOffsetRangeLimit.EARLIEST,
s"earliest offset ${range.fromOffset} does not equal ${KafkaOffsetRangeLimit.EARLIEST}")
availableOffsetRange.earliest
} else {
range.fromOffset
}
val untilOffset = if (range.untilOffset < 0) {
assert(range.untilOffset == KafkaOffsetRangeLimit.LATEST,
s"latest offset ${range.untilOffset} does not equal ${KafkaOffsetRangeLimit.LATEST}")
availableOffsetRange.latest
} else {
range.untilOffset
}
KafkaSourceRDDOffsetRange(range.topicPartition,
fromOffset, untilOffset, range.preferredLoc)
} else {
range
}
}
}