Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/4961#discussion_r27717233
  
    --- Diff: 
external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala
 ---
    @@ -0,0 +1,272 @@
    +/*
    + * 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.Random
    +import scala.util.control.NonFatal
    +
    +import kafka.admin.AdminUtils
    +import kafka.common.KafkaException
    +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
    +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
    +    var bindSuccess: Boolean = false
    +    while(!bindSuccess) {
    +      try {
    +        brokerConf = new KafkaConfig(brokerConfigure)
    +        server = new KafkaServer(brokerConf)
    +        server.startup()
    +        bindSuccess = true
    +      } catch {
    +        case e: KafkaException =>
    +          if (e.getMessage != null && e.getMessage.contains("Socket server 
failed to bind to")) {
    +            brokerPort += 1
    +          }
    +        case e: Exception => throw new Exception("Kafka server create 
failed", e)
    +      }
    +    }
    +
    +    Thread.sleep(2000)
    +    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(producerConfigure))
    +    producer.send(messages.map { new KeyedMessage[String, String](topic, _ 
) }: _*)
    +    producer.close()
    +    producer = null
    +  }
    +
    +  private def brokerConfigure: 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 producerConfigure: Properties = {
    +    val props = new Properties()
    +    props.put("metadata.broker.list", brokerAddress)
    +    props.put("serializer.class", classOf[StringEncoder].getName)
    +    props
    +  }
    +
    +  private def waitUntilMetadataIsPropagated(topic: String, partition: 
Int): Unit = {
    +    // A simplified version of scalatest eventually, rewrite here is 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)
    +    }
    +
    +    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 random = new Random()
    --- End diff --
    
    This variable looks unused?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to