Skip to content
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

Merged
merged 15 commits into from
Jan 12, 2016
12 changes: 12 additions & 0 deletions external/kafka/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -41,13 +41,25 @@
<version>${project.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-sql_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-core_${scala.binary.version}</artifactId>
<version>${project.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-sql_${scala.binary.version}</artifactId>
<version>${project.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.kafka</groupId>
<artifactId>kafka_${scala.binary.version}</artifactId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,17 +17,14 @@

package org.apache.spark.streaming.kafka

import java.util.Properties

import scala.collection.JavaConverters._
import scala.collection.mutable.ArrayBuffer
import scala.util.Random
import scala.util.control.NonFatal

import scala.util.Random
import scala.collection.mutable.ArrayBuffer
import scala.collection.JavaConverters._
import java.util.Properties
import kafka.api._
import kafka.common.{ErrorMapping, OffsetAndMetadata, OffsetMetadataAndError, TopicAndPartition}
import kafka.consumer.{ConsumerConfig, SimpleConsumer}

import org.apache.spark.SparkException

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ import java.util.Properties
import scala.collection.Map
import scala.reflect.{classTag, ClassTag}

import kafka.consumer.{Consumer, ConsumerConfig, ConsumerConnector, KafkaStream}
import kafka.consumer.{KafkaStream, Consumer, ConsumerConfig, ConsumerConnector}
import kafka.serializer.Decoder
import kafka.utils.VerifiableProperties

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,18 +20,18 @@ package org.apache.spark.streaming.kafka
import scala.collection.mutable.ArrayBuffer
import scala.reflect.{classTag, ClassTag}

import org.apache.spark.{Logging, Partition, SparkContext, SparkException, TaskContext}
import org.apache.spark.partial.{PartialResult, BoundedDouble}
import org.apache.spark.rdd.RDD
import org.apache.spark.util.NextIterator

import kafka.api.{FetchRequestBuilder, FetchResponse}
import kafka.common.{ErrorMapping, TopicAndPartition}
import kafka.consumer.SimpleConsumer
import kafka.message.{MessageAndMetadata, MessageAndOffset}
import kafka.serializer.Decoder
import kafka.utils.VerifiableProperties

import org.apache.spark.{Logging, Partition, SparkContext, SparkException, TaskContext}
import org.apache.spark.partial.{BoundedDouble, PartialResult}
import org.apache.spark.rdd.RDD
import org.apache.spark.util.NextIterator

/**
* A batch-oriented interface for consuming from Kafka.
* Starting and ending offsets are specified in advance,
Expand Down
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 {
Copy link
Owner

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.

Copy link
Owner

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 the Source?

Copy link
Collaborator Author

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.

/**
* 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)
}
Copy link
Owner

Choose a reason for hiding this comment

The 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.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

But MemoryStream has them defined in the class.

Copy link
Owner

Choose a reason for hiding this comment

The 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")) {
Copy link
Owner

Choose a reason for hiding this comment

The 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.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The 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(", ")}]"
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,8 +20,8 @@ package org.apache.spark.streaming.kafka
import java.io.File
import java.lang.{Integer => JInt}
import java.net.InetSocketAddress
import java.util.concurrent.{TimeUnit, TimeoutException}
import java.util.{Map => JMap, Properties}
import java.util.concurrent.TimeoutException

import scala.annotation.tailrec
import scala.collection.JavaConverters._
Expand All @@ -30,16 +30,16 @@ import scala.util.control.NonFatal

import kafka.admin.AdminUtils
import kafka.api.Request
import kafka.producer.{KeyedMessage, Producer, ProducerConfig}
import kafka.serializer.StringEncoder
import kafka.server.{KafkaConfig, KafkaServer}
import kafka.utils.{ZKStringSerializer, ZkUtils}
import org.I0Itec.zkclient.ZkClient
import org.apache.kafka.clients.producer._
import org.apache.kafka.common.serialization.StringSerializer
import org.apache.zookeeper.server.{NIOServerCnxnFactory, ZooKeeperServer}

import org.apache.spark.{Logging, SparkConf}
import org.apache.spark.streaming.Time
import org.apache.spark.util.Utils
import org.apache.spark.{Logging, SparkConf}

/**
* This is a helper class for Kafka test suites. This has the functionality to set up
Expand Down Expand Up @@ -170,11 +170,19 @@ private[kafka] class KafkaTestUtils extends Logging {
}

/** Send the array of messages to the Kafka broker */
def sendMessages(topic: String, messages: Array[String]): Unit = {
producer = new Producer[String, String](new ProducerConfig(producerConfiguration))
producer.send(messages.map { new KeyedMessage[String, String](topic, _ ) }: _*)
producer.close()
producer = null
def sendMessages(topic: String, messages: Array[String]): Seq[RecordMetadata] = {
producer = new KafkaProducer[String, String](producerConfiguration)
val offsets = try {
messages.map { m =>
producer.send(new ProducerRecord[String, String](topic, m)).get(10, TimeUnit.SECONDS)
}
} finally {
if (producer != null) {
producer.close()
producer = null
}
}
offsets
}

private def brokerConfiguration: Properties = {
Expand All @@ -191,10 +199,11 @@ private[kafka] class KafkaTestUtils extends Logging {

private def producerConfiguration: Properties = {
val props = new Properties()
props.put("metadata.broker.list", brokerAddress)
props.put("serializer.class", classOf[StringEncoder].getName)
props.put("bootstrap.servers", brokerAddress)
props.put("value.serializer", classOf[StringSerializer].getName)
props.put("key.serializer", classOf[StringSerializer].getName)
// wait for all in-sync replicas to ack sends
props.put("request.required.acks", "-1")
props.put("acks", "-1")
props
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,19 +27,19 @@ import scala.reflect.ClassTag
import com.google.common.base.Charsets.UTF_8
import kafka.common.TopicAndPartition
import kafka.message.MessageAndMetadata
import kafka.serializer.{Decoder, DefaultDecoder, StringDecoder}
import net.razorvine.pickle.{IObjectPickler, Opcodes, Pickler}
import kafka.serializer.{DefaultDecoder, Decoder, StringDecoder}
import net.razorvine.pickle.{Opcodes, Pickler, IObjectPickler}

import org.apache.spark.{SparkContext, SparkException}
import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext}
import org.apache.spark.api.java.function.{Function => JFunction}
import org.apache.spark.streaming.util.WriteAheadLogUtils
import org.apache.spark.{SparkContext, SparkException}
import org.apache.spark.api.java.{JavaSparkContext, JavaPairRDD, JavaRDD}
import org.apache.spark.api.python.SerDeUtil
import org.apache.spark.rdd.RDD
import org.apache.spark.storage.StorageLevel
import org.apache.spark.streaming.StreamingContext
import org.apache.spark.streaming.api.java._
import org.apache.spark.streaming.dstream.{DStream, InputDStream, ReceiverInputDStream}
import org.apache.spark.streaming.util.WriteAheadLogUtils

object KafkaUtils {
/**
Expand Down Expand Up @@ -173,7 +173,7 @@ object KafkaUtils {
}

/** get leaders for the given offset ranges, or throw an exception */
private def leadersForRanges(
private[spark] def leadersForRanges(
kc: KafkaCluster,
offsetRanges: Array[OffsetRange]): Map[TopicAndPartition, (String, Int)] = {
val topics = offsetRanges.map(o => TopicAndPartition(o.topic, o.partition)).toSet
Expand Down
Loading