-
Notifications
You must be signed in to change notification settings - Fork 1
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Kafka Source #24
Kafka Source #24
Changes from 1 commit
3d61f82
89b1ef0
5fa1a20
426cd7a
4079217
2990039
c4e2849
3cbc73f
2767a06
97c4aa5
35dea1b
b6a081b
d747866
81cda63
bcfd6a9
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,169 @@ | ||
/* | ||
* 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.streaming.kafka | ||
|
||
import kafka.common.TopicAndPartition | ||
import kafka.serializer._ | ||
|
||
import org.apache.spark.rdd.RDD | ||
import org.apache.spark.sql.{DataFrame, Dataset, SQLContext} | ||
import org.apache.spark.sql.catalyst.InternalRow | ||
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder | ||
import org.apache.spark.sql.execution.streaming.{Batch, StreamingRelation, Offset, Source} | ||
import org.apache.spark.sql.types.StructType | ||
|
||
|
||
|
||
|
||
|
||
private[kafka] | ||
case class KafkaSourceOffset(offsets: Map[TopicAndPartition, Long]) extends Offset { | ||
/** | ||
* Returns a negative integer, zero, or a positive integer as this object is less than, equal to, | ||
* or greater than the specified object. | ||
*/ | ||
override def compareTo(other: Offset): Int = other match { | ||
case KafkaSourceOffset(otherOffsets) => | ||
val allTopicAndPartitions = (this.offsets.keySet ++ otherOffsets.keySet).toSeq | ||
|
||
val comparisons = allTopicAndPartitions.map { tp => | ||
(this.offsets.get(tp), otherOffsets.get(tp)) match { | ||
|
||
case (Some(a), Some(b)) => | ||
if (a < b) { | ||
-1 | ||
} else if (a > b) { | ||
1 | ||
} else { | ||
0 | ||
} | ||
case (None, None) => 0 | ||
case (None, _) => -1 | ||
case (_, None) => 1 | ||
} | ||
} | ||
val signs = comparisons.distinct | ||
if (signs.size != 1) { | ||
throw new IllegalArgumentException( | ||
s"Invalid comparison between to sets of Kafka offets: $this <=> $other") | ||
} | ||
signs.head | ||
case _ => | ||
throw new IllegalArgumentException(s"Cannot compare $this <=> $other") | ||
} | ||
|
||
override def toString(): String = offsets.toSeq.mkString("[", ", ", "]") | ||
} | ||
|
||
private[kafka] object KafkaSourceOffset { | ||
def fromOffset(offset: Offset): KafkaSourceOffset = { | ||
offset match { | ||
case o: KafkaSourceOffset => o | ||
case _ => | ||
throw new IllegalArgumentException( | ||
s"Invalid conversion from offset of ${offset.getClass} to $getClass") | ||
} | ||
} | ||
} | ||
|
||
|
||
private[kafka] case class KafkaSource( | ||
topics: Set[String], params: Map[String, String])(implicit sqlContext: SQLContext) extends Source { | ||
|
||
type OffsetMap = Map[TopicAndPartition, Long] | ||
implicit private val encoder = ExpressionEncoder.tuple( | ||
ExpressionEncoder[Array[Byte]](), ExpressionEncoder[Array[Byte]]()) | ||
|
||
@transient private val logicalPlan = StreamingRelation(this) | ||
@transient private val kc = new KafkaCluster(params) | ||
@transient private val topicAndPartitions = KafkaCluster.checkErrors(kc.getPartitions(topics)) | ||
@transient private lazy val initialOffsets = getInitialOffsets() | ||
|
||
override def schema: StructType = encoder.schema | ||
|
||
/** | ||
* Returns the next batch of data that is available after `start`, if any is available. | ||
*/ | ||
override def getNextBatch(start: Option[Offset]): Option[Batch] = { | ||
val latestOffset = getLatestOffsets() | ||
val offsetRanges = getOffsetRanges( | ||
start.map(KafkaSourceOffset.fromOffset(_).offsets), latestOffset) | ||
|
||
val kafkaParams = params | ||
val encodingFunc = encoder.toRow _ | ||
val sparkContext = sqlContext.sparkContext | ||
|
||
println("Creating DF with offset ranges: " + offsetRanges) | ||
if (offsetRanges.nonEmpty) { | ||
val rdd = KafkaUtils.createRDD[Array[Byte], Array[Byte], DefaultDecoder, DefaultDecoder]( | ||
sparkContext, kafkaParams, offsetRanges.toArray) | ||
Some(new Batch(KafkaSourceOffset(latestOffset), sqlContext.createDataset(rdd).toDF)) | ||
} else { | ||
None | ||
} | ||
} | ||
|
||
def toDS(): Dataset[(Array[Byte], Array[Byte])] = { | ||
toDF.as[(Array[Byte], Array[Byte])] | ||
} | ||
|
||
def toDF(): DataFrame = { | ||
new DataFrame(sqlContext, logicalPlan) | ||
} | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I would consider defining these in the tests since they are really only for testing. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. But MemoryStream has them defined in the class. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Okay, we should consider moving these once we have a real provider for this Source. |
||
|
||
private def getOffsetRanges(start: Option[OffsetMap], end: OffsetMap): Seq[OffsetRange] = { | ||
// Get the offsets from which to start reading. If its none, find initial offsets to start from. | ||
val fromOffsets = start.getOrElse { initialOffsets } | ||
|
||
// Get the latest offsets | ||
val untilOffsets = getLatestOffsets() | ||
|
||
// Get all the partitions referenced in both sets of offsets | ||
val allTopicAndPartitions = (fromOffsets.keySet ++ untilOffsets.keySet).toSeq | ||
|
||
// For each partition, figure out the non-empty ranges of offsets | ||
allTopicAndPartitions.flatMap { tp => | ||
(fromOffsets.get(tp), untilOffsets.get(tp)) match { | ||
|
||
// Data was read till fromOffset and needs to be read till untilOffset | ||
case (Some(fromOffset), Some(untilOffset)) => | ||
if (untilOffset > fromOffset) { | ||
Some(OffsetRange(tp, fromOffset, untilOffset)) | ||
} else None | ||
|
||
case _ => | ||
None | ||
} | ||
} | ||
} | ||
|
||
private def getLatestOffsets(): OffsetMap = { | ||
val partitionLeaders = KafkaCluster.checkErrors(kc.findLeaders(topicAndPartitions)) | ||
val leadersAndOffsets = KafkaCluster.checkErrors(kc.getLatestLeaderOffsets(topicAndPartitions)) | ||
println("Getting offsets " + leadersAndOffsets) | ||
leadersAndOffsets.map { x => (x._1, x._2.offset) } | ||
} | ||
|
||
private def getInitialOffsets(): OffsetMap = { | ||
if (params.get("auto.offset.reset").map(_.toLowerCase) == Some("smallest")) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. typically we do all extraction / validation of the options in the provider and then the arguments to the source are typed (instead of passing the map into the class itself). The nice thing about this is it centralizes the location for all configuration. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I can fix that while building the provider. |
||
KafkaCluster.checkErrors(kc.getEarliestLeaderOffsets(topicAndPartitions)).mapValues(_.offset) | ||
} else Map.empty | ||
} | ||
|
||
override def toString(): String = s"KafkaSource[${topics.mkString(", ")}]" | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This probably does not need to be a case class give you are implementing equality and the toString already.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is there a reason this isn't just
KafkaOffset
? Will there be one for anything other than theSource
?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The package has other classes like OffsetRange etc that are used for Spark Streaming stuff. I thought its better differentiate this. Also, since Kafka's record id is also called offset, KafkaOffset sounds like the offset for a single topic+partition. So I think its less ambiguous to distinguish it by naming it
KafkaSourceOffset
.