Skip to content

Commit

Permalink
[SPARK-6211][Streaming] Add Python Kafka API unit test
Browse files Browse the repository at this point in the history
Refactor the Kafka unit test and add Python API support. CC tdas davies please help to review, thanks a lot.

Author: jerryshao <saisai.shao@intel.com>
Author: Saisai Shao <saisai.shao@intel.com>

Closes #4961 from jerryshao/SPARK-6211 and squashes the following commits:

ee4b919 [jerryshao] Fixed newly merged issue
82c756e [jerryshao] Address the comments
92912d1 [jerryshao] Address the commits
0708bb1 [jerryshao] Fix rebase issue
40b47a3 [Saisai Shao] Style fix
f889657 [Saisai Shao] Update the code according
8a2f3e2 [jerryshao] Address the issues
0f1b7ce [jerryshao] Still fix the bug
61a04f0 [jerryshao] Fix bugs and address the issues
64d9877 [jerryshao] Fix rebase bugs
8ad442f [jerryshao] Add kafka-assembly in run-tests
6020b00 [jerryshao] Add more debug info in Shell
8102d6e [jerryshao] Fix bug in Jenkins test
fde1213 [jerryshao] Code style changes
5536f95 [jerryshao] Refactor the Kafka unit test and add Python Kafka unittest support
  • Loading branch information
jerryshao authored and tdas committed Apr 10, 2015
1 parent e236081 commit 3290d2d
Show file tree
Hide file tree
Showing 12 changed files with 502 additions and 311 deletions.
2 changes: 1 addition & 1 deletion dev/run-tests
Original file line number Diff line number Diff line change
Expand Up @@ -173,7 +173,7 @@ CURRENT_BLOCK=$BLOCK_BUILD
build/mvn $HIVE_BUILD_ARGS clean package -DskipTests
else
echo -e "q\n" \
| build/sbt $HIVE_BUILD_ARGS package assembly/assembly \
| build/sbt $HIVE_BUILD_ARGS package assembly/assembly streaming-kafka-assembly/assembly \
| grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including"
fi
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,261 @@
/*
* 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 java.io.File
import java.lang.{Integer => JInt}
import java.net.InetSocketAddress
import java.util.{Map => JMap}
import java.util.Properties
import java.util.concurrent.TimeoutException

import scala.annotation.tailrec
import scala.language.postfixOps
import scala.util.control.NonFatal

import kafka.admin.AdminUtils
import kafka.producer.{KeyedMessage, Producer, ProducerConfig}
import kafka.serializer.StringEncoder
import kafka.server.{KafkaConfig, KafkaServer}
import kafka.utils.ZKStringSerializer
import org.apache.zookeeper.server.{NIOServerCnxnFactory, ZooKeeperServer}
import org.I0Itec.zkclient.ZkClient

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

/**
* This is a helper class for Kafka test suites. This has the functionality to set up
* and tear down local Kafka servers, and to push data using Kafka producers.
*
* The reason to put Kafka test utility class in src is to test Python related Kafka APIs.
*/
private class KafkaTestUtils extends Logging {

// Zookeeper related configurations
private val zkHost = "localhost"
private var zkPort: Int = 0
private val zkConnectionTimeout = 6000
private val zkSessionTimeout = 6000

private var zookeeper: EmbeddedZookeeper = _

private var zkClient: ZkClient = _

// Kafka broker related configurations
private val brokerHost = "localhost"
private var brokerPort = 9092
private var brokerConf: KafkaConfig = _

// Kafka broker server
private var server: KafkaServer = _

// Kafka producer
private var producer: Producer[String, String] = _

// Flag to test whether the system is correctly started
private var zkReady = false
private var brokerReady = false

def zkAddress: String = {
assert(zkReady, "Zookeeper not setup yet or already torn down, cannot get zookeeper address")
s"$zkHost:$zkPort"
}

def brokerAddress: String = {
assert(brokerReady, "Kafka not setup yet or already torn down, cannot get broker address")
s"$brokerHost:$brokerPort"
}

def zookeeperClient: ZkClient = {
assert(zkReady, "Zookeeper not setup yet or already torn down, cannot get zookeeper client")
Option(zkClient).getOrElse(
throw new IllegalStateException("Zookeeper client is not yet initialized"))
}

// Set up the Embedded Zookeeper server and get the proper Zookeeper port
private def setupEmbeddedZookeeper(): Unit = {
// Zookeeper server startup
zookeeper = new EmbeddedZookeeper(s"$zkHost:$zkPort")
// Get the actual zookeeper binding port
zkPort = zookeeper.actualPort
zkClient = new ZkClient(s"$zkHost:$zkPort", zkSessionTimeout, zkConnectionTimeout,
ZKStringSerializer)
zkReady = true
}

// Set up the Embedded Kafka server
private def setupEmbeddedKafkaServer(): Unit = {
assert(zkReady, "Zookeeper should be set up beforehand")

// Kafka broker startup
Utils.startServiceOnPort(brokerPort, port => {
brokerPort = port
brokerConf = new KafkaConfig(brokerConfiguration)
server = new KafkaServer(brokerConf)
server.startup()
(server, port)
}, new SparkConf(), "KafkaBroker")

brokerReady = true
}

/** setup the whole embedded servers, including Zookeeper and Kafka brokers */
def setup(): Unit = {
setupEmbeddedZookeeper()
setupEmbeddedKafkaServer()
}

/** Teardown the whole servers, including Kafka broker and Zookeeper */
def teardown(): Unit = {
brokerReady = false
zkReady = false

if (producer != null) {
producer.close()
producer = null
}

if (server != null) {
server.shutdown()
server = null
}

brokerConf.logDirs.foreach { f => Utils.deleteRecursively(new File(f)) }

if (zkClient != null) {
zkClient.close()
zkClient = null
}

if (zookeeper != null) {
zookeeper.shutdown()
zookeeper = null
}
}

/** Create a Kafka topic and wait until it propagated to the whole cluster */
def createTopic(topic: String): Unit = {
AdminUtils.createTopic(zkClient, topic, 1, 1)
// wait until metadata is propagated
waitUntilMetadataIsPropagated(topic, 0)
}

/** Java-friendly function for sending messages to the Kafka broker */
def sendMessages(topic: String, messageToFreq: JMap[String, JInt]): Unit = {
import scala.collection.JavaConversions._
sendMessages(topic, Map(messageToFreq.mapValues(_.intValue()).toSeq: _*))
}

/** Send the messages to the Kafka broker */
def sendMessages(topic: String, messageToFreq: Map[String, Int]): Unit = {
val messages = messageToFreq.flatMap { case (s, freq) => Seq.fill(freq)(s) }.toArray
sendMessages(topic, messages)
}

/** 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
}

private def brokerConfiguration: Properties = {
val props = new Properties()
props.put("broker.id", "0")
props.put("host.name", "localhost")
props.put("port", brokerPort.toString)
props.put("log.dir", Utils.createTempDir().getAbsolutePath)
props.put("zookeeper.connect", zkAddress)
props.put("log.flush.interval.messages", "1")
props.put("replica.socket.timeout.ms", "1500")
props
}

private def producerConfiguration: Properties = {
val props = new Properties()
props.put("metadata.broker.list", brokerAddress)
props.put("serializer.class", classOf[StringEncoder].getName)
props
}

// A simplified version of scalatest eventually, rewritten here to avoid adding extra test
// dependency
def eventually[T](timeout: Time, interval: Time)(func: => T): T = {
def makeAttempt(): Either[Throwable, T] = {
try {
Right(func)
} catch {
case e if NonFatal(e) => Left(e)
}
}

val startTime = System.currentTimeMillis()
@tailrec
def tryAgain(attempt: Int): T = {
makeAttempt() match {
case Right(result) => result
case Left(e) =>
val duration = System.currentTimeMillis() - startTime
if (duration < timeout.milliseconds) {
Thread.sleep(interval.milliseconds)
} else {
throw new TimeoutException(e.getMessage)
}

tryAgain(attempt + 1)
}
}

tryAgain(1)
}

private def waitUntilMetadataIsPropagated(topic: String, partition: Int): Unit = {
eventually(Time(10000), Time(100)) {
assert(
server.apis.metadataCache.containsTopicAndPartition(topic, partition),
s"Partition [$topic, $partition] metadata not propagated after timeout"
)
}
}

private class EmbeddedZookeeper(val zkConnect: String) {
val snapshotDir = Utils.createTempDir()
val logDir = Utils.createTempDir()

val zookeeper = new ZooKeeperServer(snapshotDir, logDir, 500)
val (ip, port) = {
val splits = zkConnect.split(":")
(splits(0), splits(1).toInt)
}
val factory = new NIOServerCnxnFactory()
factory.configure(new InetSocketAddress(ip, port), 16)
factory.startup(zookeeper)

val actualPort = factory.getLocalPort

def shutdown() {
factory.shutdown()
Utils.deleteRecursively(snapshotDir)
Utils.deleteRecursively(logDir)
}
}
}

Original file line number Diff line number Diff line change
Expand Up @@ -41,24 +41,28 @@

public class JavaDirectKafkaStreamSuite implements Serializable {
private transient JavaStreamingContext ssc = null;
private transient KafkaStreamSuiteBase suiteBase = null;
private transient KafkaTestUtils kafkaTestUtils = null;

@Before
public void setUp() {
suiteBase = new KafkaStreamSuiteBase() { };
suiteBase.setupKafka();
System.clearProperty("spark.driver.port");
SparkConf sparkConf = new SparkConf()
.setMaster("local[4]").setAppName(this.getClass().getSimpleName());
ssc = new JavaStreamingContext(sparkConf, Durations.milliseconds(200));
kafkaTestUtils = new KafkaTestUtils();
kafkaTestUtils.setup();
SparkConf sparkConf = new SparkConf()
.setMaster("local[4]").setAppName(this.getClass().getSimpleName());
ssc = new JavaStreamingContext(sparkConf, Durations.milliseconds(200));
}

@After
public void tearDown() {
if (ssc != null) {
ssc.stop();
ssc = null;
System.clearProperty("spark.driver.port");
suiteBase.tearDownKafka();
}

if (kafkaTestUtils != null) {
kafkaTestUtils.teardown();
kafkaTestUtils = null;
}
}

@Test
Expand All @@ -74,7 +78,7 @@ public void testKafkaStream() throws InterruptedException {
sent.addAll(Arrays.asList(topic2data));

HashMap<String, String> kafkaParams = new HashMap<String, String>();
kafkaParams.put("metadata.broker.list", suiteBase.brokerAddress());
kafkaParams.put("metadata.broker.list", kafkaTestUtils.brokerAddress());
kafkaParams.put("auto.offset.reset", "smallest");

JavaDStream<String> stream1 = KafkaUtils.createDirectStream(
Expand Down Expand Up @@ -147,8 +151,8 @@ private HashMap<TopicAndPartition, Long> topicOffsetToMap(String topic, Long off

private String[] createTopicAndSendData(String topic) {
String[] data = { topic + "-1", topic + "-2", topic + "-3"};
suiteBase.createTopic(topic);
suiteBase.sendMessages(topic, data);
kafkaTestUtils.createTopic(topic);
kafkaTestUtils.sendMessages(topic, data);
return data;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -37,24 +37,28 @@

public class JavaKafkaRDDSuite implements Serializable {
private transient JavaSparkContext sc = null;
private transient KafkaStreamSuiteBase suiteBase = null;
private transient KafkaTestUtils kafkaTestUtils = null;

@Before
public void setUp() {
suiteBase = new KafkaStreamSuiteBase() { };
suiteBase.setupKafka();
System.clearProperty("spark.driver.port");
kafkaTestUtils = new KafkaTestUtils();
kafkaTestUtils.setup();
SparkConf sparkConf = new SparkConf()
.setMaster("local[4]").setAppName(this.getClass().getSimpleName());
sc = new JavaSparkContext(sparkConf);
}

@After
public void tearDown() {
sc.stop();
sc = null;
System.clearProperty("spark.driver.port");
suiteBase.tearDownKafka();
if (sc != null) {
sc.stop();
sc = null;
}

if (kafkaTestUtils != null) {
kafkaTestUtils.teardown();
kafkaTestUtils = null;
}
}

@Test
Expand All @@ -66,7 +70,7 @@ public void testKafkaRDD() throws InterruptedException {
String[] topic2data = createTopicAndSendData(topic2);

HashMap<String, String> kafkaParams = new HashMap<String, String>();
kafkaParams.put("metadata.broker.list", suiteBase.brokerAddress());
kafkaParams.put("metadata.broker.list", kafkaTestUtils.brokerAddress());

OffsetRange[] offsetRanges = {
OffsetRange.create(topic1, 0, 0, 1),
Expand All @@ -75,7 +79,7 @@ public void testKafkaRDD() throws InterruptedException {

HashMap<TopicAndPartition, Broker> emptyLeaders = new HashMap<TopicAndPartition, Broker>();
HashMap<TopicAndPartition, Broker> leaders = new HashMap<TopicAndPartition, Broker>();
String[] hostAndPort = suiteBase.brokerAddress().split(":");
String[] hostAndPort = kafkaTestUtils.brokerAddress().split(":");
Broker broker = Broker.create(hostAndPort[0], Integer.parseInt(hostAndPort[1]));
leaders.put(new TopicAndPartition(topic1, 0), broker);
leaders.put(new TopicAndPartition(topic2, 0), broker);
Expand Down Expand Up @@ -144,8 +148,8 @@ public String call(MessageAndMetadata<String, String> msgAndMd) throws Exception

private String[] createTopicAndSendData(String topic) {
String[] data = { topic + "-1", topic + "-2", topic + "-3"};
suiteBase.createTopic(topic);
suiteBase.sendMessages(topic, data);
kafkaTestUtils.createTopic(topic);
kafkaTestUtils.sendMessages(topic, data);
return data;
}
}
Loading

0 comments on commit 3290d2d

Please sign in to comment.