mirror of https://github.com/apache/kafka.git
				
				
				
			KAFKA-1501 Let the OS choose the port in unit tests to avoid collisions. Patch by Ewen CP, reviewed by Guozhang and me.
This commit is contained in:
		
							parent
							
								
									15b93a410a
								
							
						
					
					
						commit
						6adaffd8ea
					
				|  | @ -131,9 +131,12 @@ public class SelectorTest { | ||||||
|     @Test |     @Test | ||||||
|     public void testConnectionRefused() throws Exception { |     public void testConnectionRefused() throws Exception { | ||||||
|         int node = 0; |         int node = 0; | ||||||
|         selector.connect(node, new InetSocketAddress("localhost", TestUtils.choosePort()), BUFFER_SIZE, BUFFER_SIZE); |         ServerSocket nonListeningSocket = new ServerSocket(0); | ||||||
|  |         int nonListeningPort = nonListeningSocket.getLocalPort(); | ||||||
|  |         selector.connect(node, new InetSocketAddress("localhost", nonListeningPort), BUFFER_SIZE, BUFFER_SIZE); | ||||||
|         while (selector.disconnected().contains(node)) |         while (selector.disconnected().contains(node)) | ||||||
|             selector.poll(1000L); |             selector.poll(1000L); | ||||||
|  |         nonListeningSocket.close(); | ||||||
|     } |     } | ||||||
| 
 | 
 | ||||||
|     /** |     /** | ||||||
|  | @ -271,8 +274,8 @@ public class SelectorTest { | ||||||
|         private final List<Socket> sockets; |         private final List<Socket> sockets; | ||||||
| 
 | 
 | ||||||
|         public EchoServer() throws Exception { |         public EchoServer() throws Exception { | ||||||
|             this.port = TestUtils.choosePort(); |             this.serverSocket = new ServerSocket(0); | ||||||
|             this.serverSocket = new ServerSocket(port); |             this.port = this.serverSocket.getLocalPort(); | ||||||
|             this.threads = Collections.synchronizedList(new ArrayList<Thread>()); |             this.threads = Collections.synchronizedList(new ArrayList<Thread>()); | ||||||
|             this.sockets = Collections.synchronizedList(new ArrayList<Socket>()); |             this.sockets = Collections.synchronizedList(new ArrayList<Socket>()); | ||||||
|         } |         } | ||||||
|  |  | ||||||
|  | @ -19,8 +19,6 @@ package org.apache.kafka.test; | ||||||
| import static java.util.Arrays.asList; | import static java.util.Arrays.asList; | ||||||
| 
 | 
 | ||||||
| import java.io.File; | import java.io.File; | ||||||
| import java.io.IOException; |  | ||||||
| import java.net.ServerSocket; |  | ||||||
| import java.util.ArrayList; | import java.util.ArrayList; | ||||||
| import java.util.List; | import java.util.List; | ||||||
| import java.util.Random; | import java.util.Random; | ||||||
|  | @ -59,32 +57,6 @@ public class TestUtils { | ||||||
|         return new Cluster(asList(ns), parts); |         return new Cluster(asList(ns), parts); | ||||||
|     } |     } | ||||||
| 
 | 
 | ||||||
|     /** |  | ||||||
|      * Choose a number of random available ports |  | ||||||
|      */ |  | ||||||
|     public static int[] choosePorts(int count) { |  | ||||||
|         try { |  | ||||||
|             ServerSocket[] sockets = new ServerSocket[count]; |  | ||||||
|             int[] ports = new int[count]; |  | ||||||
|             for (int i = 0; i < count; i++) { |  | ||||||
|                 sockets[i] = new ServerSocket(0); |  | ||||||
|                 ports[i] = sockets[i].getLocalPort(); |  | ||||||
|             } |  | ||||||
|             for (int i = 0; i < count; i++) |  | ||||||
|                 sockets[i].close(); |  | ||||||
|             return ports; |  | ||||||
|         } catch (IOException e) { |  | ||||||
|             throw new RuntimeException(e); |  | ||||||
|         } |  | ||||||
|     } |  | ||||||
| 
 |  | ||||||
|     /** |  | ||||||
|      * Choose an available port |  | ||||||
|      */ |  | ||||||
|     public static int choosePort() { |  | ||||||
|         return choosePorts(1)[0]; |  | ||||||
|     } |  | ||||||
| 
 |  | ||||||
|     /** |     /** | ||||||
|      * Generate an array of random bytes |      * Generate an array of random bytes | ||||||
|      *  |      *  | ||||||
|  |  | ||||||
|  | @ -39,7 +39,7 @@ import com.yammer.metrics.core.{Gauge, Meter} | ||||||
|  */ |  */ | ||||||
| class SocketServer(val brokerId: Int, | class SocketServer(val brokerId: Int, | ||||||
|                    val host: String, |                    val host: String, | ||||||
|                    val port: Int, |                    private val port: Int, | ||||||
|                    val numProcessorThreads: Int, |                    val numProcessorThreads: Int, | ||||||
|                    val maxQueuedRequests: Int, |                    val maxQueuedRequests: Int, | ||||||
|                    val sendBufferSize: Int, |                    val sendBufferSize: Int, | ||||||
|  | @ -72,7 +72,7 @@ class SocketServer(val brokerId: Int, | ||||||
|                                     requestChannel, |                                     requestChannel, | ||||||
|                                     quotas, |                                     quotas, | ||||||
|                                     connectionsMaxIdleMs) |                                     connectionsMaxIdleMs) | ||||||
|       Utils.newThread("kafka-network-thread-%d-%d".format(port, i), processors(i), false).start() |       Utils.newThread("kafka-network-thread-%d-%d".format(brokerId, i), processors(i), false).start() | ||||||
|     } |     } | ||||||
| 
 | 
 | ||||||
|     newGauge("ResponsesBeingSent", new Gauge[Int] { |     newGauge("ResponsesBeingSent", new Gauge[Int] { | ||||||
|  | @ -100,6 +100,12 @@ class SocketServer(val brokerId: Int, | ||||||
|       processor.shutdown() |       processor.shutdown() | ||||||
|     info("Shutdown completed") |     info("Shutdown completed") | ||||||
|   } |   } | ||||||
|  | 
 | ||||||
|  |   def boundPort(): Int = { | ||||||
|  |     if (acceptor == null) | ||||||
|  |       throw new KafkaException("Tried to check server's port before server was started") | ||||||
|  |     acceptor.serverChannel.socket().getLocalPort | ||||||
|  |   } | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| /** | /** | ||||||
|  | @ -197,7 +203,7 @@ private[kafka] abstract class AbstractServerThread(connectionQuotas: ConnectionQ | ||||||
|  * Thread that accepts and configures new connections. There is only need for one of these |  * Thread that accepts and configures new connections. There is only need for one of these | ||||||
|  */ |  */ | ||||||
| private[kafka] class Acceptor(val host: String,  | private[kafka] class Acceptor(val host: String,  | ||||||
|                               val port: Int,  |                               private val port: Int, | ||||||
|                               private val processors: Array[Processor], |                               private val processors: Array[Processor], | ||||||
|                               val sendBufferSize: Int,  |                               val sendBufferSize: Int,  | ||||||
|                               val recvBufferSize: Int, |                               val recvBufferSize: Int, | ||||||
|  |  | ||||||
|  | @ -157,7 +157,8 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg | ||||||
|         topicConfigManager.startup() |         topicConfigManager.startup() | ||||||
| 
 | 
 | ||||||
|         /* tell everyone we are alive */ |         /* tell everyone we are alive */ | ||||||
|         kafkaHealthcheck = new KafkaHealthcheck(config.brokerId, config.advertisedHostName, config.advertisedPort, config.zkSessionTimeoutMs, zkClient) |         val advertisedPort = if (config.advertisedPort != 0) config.advertisedPort else socketServer.boundPort() | ||||||
|  |         kafkaHealthcheck = new KafkaHealthcheck(config.brokerId, config.advertisedHostName, advertisedPort, config.zkSessionTimeoutMs, zkClient) | ||||||
|         kafkaHealthcheck.startup() |         kafkaHealthcheck.startup() | ||||||
| 
 | 
 | ||||||
|         /* register broker metrics */ |         /* register broker metrics */ | ||||||
|  | @ -357,6 +358,8 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg | ||||||
| 
 | 
 | ||||||
|   def getLogManager(): LogManager = logManager |   def getLogManager(): LogManager = logManager | ||||||
| 
 | 
 | ||||||
|  |   def boundPort(): Int = socketServer.boundPort() | ||||||
|  | 
 | ||||||
|   private def createLogManager(zkClient: ZkClient, brokerState: BrokerState): LogManager = { |   private def createLogManager(zkClient: ZkClient, brokerState: BrokerState): LogManager = { | ||||||
|     val defaultLogConfig = LogConfig(segmentSize = config.logSegmentBytes, |     val defaultLogConfig = LogConfig(segmentSize = config.logSegmentBytes, | ||||||
|                                      segmentMs = config.logRollTimeMillis, |                                      segmentMs = config.logRollTimeMillis, | ||||||
|  |  | ||||||
|  | @ -0,0 +1,154 @@ | ||||||
|  | /** | ||||||
|  |  * Copyright 2015 Confluent Inc. | ||||||
|  |  * | ||||||
|  |  * Licensed 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 kafka.api | ||||||
|  | 
 | ||||||
|  | import kafka.server.KafkaConfig | ||||||
|  | import org.apache.kafka.clients.producer.ProducerConfig | ||||||
|  | import org.apache.kafka.clients.producer.ProducerRecord | ||||||
|  | import org.apache.kafka.clients.consumer.ConsumerConfig | ||||||
|  | import org.apache.kafka.clients.consumer.CommitType | ||||||
|  | import org.apache.kafka.common.TopicPartition | ||||||
|  | 
 | ||||||
|  | import kafka.utils.{ShutdownableThread, TestUtils, Logging} | ||||||
|  | 
 | ||||||
|  | import org.junit.Assert._ | ||||||
|  | 
 | ||||||
|  | import scala.collection.JavaConversions._ | ||||||
|  | 
 | ||||||
|  | /** | ||||||
|  |  * Integration tests for the new consumer that cover basic usage as well as server failures | ||||||
|  |  */ | ||||||
|  | class ConsumerBounceTest extends IntegrationTestHarness with Logging { | ||||||
|  | 
 | ||||||
|  |   val producerCount = 1 | ||||||
|  |   val consumerCount = 2 | ||||||
|  |   val serverCount = 3 | ||||||
|  | 
 | ||||||
|  |   val topic = "topic" | ||||||
|  |   val part = 0 | ||||||
|  |   val tp = new TopicPartition(topic, part) | ||||||
|  | 
 | ||||||
|  |   // configure the servers and clients | ||||||
|  |   this.serverConfig.setProperty("controlled.shutdown.enable", "false") // speed up shutdown | ||||||
|  |   this.serverConfig.setProperty("offsets.topic.replication.factor", "3") // don't want to lose offset | ||||||
|  |   this.serverConfig.setProperty("offsets.topic.num.partitions", "1") | ||||||
|  |   this.producerConfig.setProperty(ProducerConfig.ACKS_CONFIG, "all") | ||||||
|  |   this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "my-test") | ||||||
|  |   this.consumerConfig.setProperty(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, 4096.toString) | ||||||
|  |   this.consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") | ||||||
|  | 
 | ||||||
|  |   override def generateConfigs() = { | ||||||
|  |     FixedPortTestUtils.createBrokerConfigs(serverCount, zkConnect,enableControlledShutdown = false) | ||||||
|  |       .map(KafkaConfig.fromProps(_, serverConfig)) | ||||||
|  |   } | ||||||
|  | 
 | ||||||
|  |   override def setUp() { | ||||||
|  |     super.setUp() | ||||||
|  | 
 | ||||||
|  |     // create the test topic with all the brokers as replicas | ||||||
|  |     TestUtils.createTopic(this.zkClient, topic, 1, serverCount, this.servers) | ||||||
|  |   } | ||||||
|  | 
 | ||||||
|  |   def testConsumptionWithBrokerFailures() = consumeWithBrokerFailures(5) | ||||||
|  | 
 | ||||||
|  |   /* | ||||||
|  |    * 1. Produce a bunch of messages | ||||||
|  |    * 2. Then consume the messages while killing and restarting brokers at random | ||||||
|  |    */ | ||||||
|  |   def consumeWithBrokerFailures(numIters: Int) { | ||||||
|  |     val numRecords = 1000 | ||||||
|  |     sendRecords(numRecords) | ||||||
|  |     this.producers.map(_.close) | ||||||
|  | 
 | ||||||
|  |     var consumed = 0 | ||||||
|  |     val consumer = this.consumers(0) | ||||||
|  |     consumer.subscribe(topic) | ||||||
|  | 
 | ||||||
|  |     val scheduler = new BounceBrokerScheduler(numIters) | ||||||
|  |     scheduler.start() | ||||||
|  | 
 | ||||||
|  |     while (scheduler.isRunning.get()) { | ||||||
|  |       for (record <- consumer.poll(100)) { | ||||||
|  |         assertEquals(consumed.toLong, record.offset()) | ||||||
|  |         consumed += 1 | ||||||
|  |       } | ||||||
|  |       consumer.commit(CommitType.SYNC) | ||||||
|  | 
 | ||||||
|  |       if (consumed == numRecords) { | ||||||
|  |         consumer.seekToBeginning() | ||||||
|  |         consumed = 0 | ||||||
|  |       } | ||||||
|  |     } | ||||||
|  |     scheduler.shutdown() | ||||||
|  |   } | ||||||
|  | 
 | ||||||
|  |   def testSeekAndCommitWithBrokerFailures() = seekAndCommitWithBrokerFailures(5) | ||||||
|  | 
 | ||||||
|  |   def seekAndCommitWithBrokerFailures(numIters: Int) { | ||||||
|  |     val numRecords = 1000 | ||||||
|  |     sendRecords(numRecords) | ||||||
|  |     this.producers.map(_.close) | ||||||
|  | 
 | ||||||
|  |     val consumer = this.consumers(0) | ||||||
|  |     consumer.subscribe(tp) | ||||||
|  |     consumer.seek(tp, 0) | ||||||
|  | 
 | ||||||
|  |     val scheduler = new BounceBrokerScheduler(numIters) | ||||||
|  |     scheduler.start() | ||||||
|  | 
 | ||||||
|  |     while(scheduler.isRunning.get()) { | ||||||
|  |       val coin = TestUtils.random.nextInt(3) | ||||||
|  |       if (coin == 0) { | ||||||
|  |         info("Seeking to end of log") | ||||||
|  |         consumer.seekToEnd() | ||||||
|  |         assertEquals(numRecords.toLong, consumer.position(tp)) | ||||||
|  |       } else if (coin == 1) { | ||||||
|  |         val pos = TestUtils.random.nextInt(numRecords).toLong | ||||||
|  |         info("Seeking to " + pos) | ||||||
|  |         consumer.seek(tp, pos) | ||||||
|  |         assertEquals(pos, consumer.position(tp)) | ||||||
|  |       } else if (coin == 2) { | ||||||
|  |         info("Committing offset.") | ||||||
|  |         consumer.commit(CommitType.SYNC) | ||||||
|  |         assertEquals(consumer.position(tp), consumer.committed(tp)) | ||||||
|  |       } | ||||||
|  |     } | ||||||
|  |   } | ||||||
|  | 
 | ||||||
|  |   private class BounceBrokerScheduler(val numIters: Int) extends ShutdownableThread("daemon-bounce-broker", false) | ||||||
|  |   { | ||||||
|  |     var iter: Int = 0 | ||||||
|  | 
 | ||||||
|  |     override def doWork(): Unit = { | ||||||
|  |       killRandomBroker() | ||||||
|  |       Thread.sleep(500) | ||||||
|  |       restartDeadBrokers() | ||||||
|  | 
 | ||||||
|  |       iter += 1 | ||||||
|  |       if (iter == numIters) | ||||||
|  |         initiateShutdown() | ||||||
|  |       else | ||||||
|  |         Thread.sleep(500) | ||||||
|  |     } | ||||||
|  |   } | ||||||
|  | 
 | ||||||
|  |   private def sendRecords(numRecords: Int) { | ||||||
|  |     val futures = (0 until numRecords).map { i => | ||||||
|  |       this.producers(0).send(new ProducerRecord(topic, part, i.toString.getBytes, i.toString.getBytes)) | ||||||
|  |     } | ||||||
|  |     futures.map(_.get) | ||||||
|  |   } | ||||||
|  | } | ||||||
|  | @ -146,72 +146,6 @@ class ConsumerTest extends IntegrationTestHarness with Logging { | ||||||
|     assertNull(this.consumers(0).partitionsFor("non-exist-topic")) |     assertNull(this.consumers(0).partitionsFor("non-exist-topic")) | ||||||
|   } |   } | ||||||
| 
 | 
 | ||||||
|   def testConsumptionWithBrokerFailures() = consumeWithBrokerFailures(5) |  | ||||||
| 
 |  | ||||||
|   /* |  | ||||||
|    * 1. Produce a bunch of messages |  | ||||||
|    * 2. Then consume the messages while killing and restarting brokers at random |  | ||||||
|    */ |  | ||||||
|   def consumeWithBrokerFailures(numIters: Int) { |  | ||||||
|     val numRecords = 1000 |  | ||||||
|     sendRecords(numRecords) |  | ||||||
|     this.producers.map(_.close) |  | ||||||
| 
 |  | ||||||
|     var consumed = 0 |  | ||||||
|     val consumer = this.consumers(0) |  | ||||||
|     consumer.subscribe(topic) |  | ||||||
| 
 |  | ||||||
|     val scheduler = new BounceBrokerScheduler(numIters) |  | ||||||
|     scheduler.start() |  | ||||||
| 
 |  | ||||||
|     while (scheduler.isRunning.get()) { |  | ||||||
|       for (record <- consumer.poll(100)) { |  | ||||||
|         assertEquals(consumed.toLong, record.offset()) |  | ||||||
|         consumed += 1 |  | ||||||
|       } |  | ||||||
|       consumer.commit(CommitType.SYNC) |  | ||||||
| 
 |  | ||||||
|       if (consumed == numRecords) { |  | ||||||
|         consumer.seekToBeginning() |  | ||||||
|         consumed = 0 |  | ||||||
|       } |  | ||||||
|     } |  | ||||||
|     scheduler.shutdown() |  | ||||||
|   } |  | ||||||
| 
 |  | ||||||
|   def testSeekAndCommitWithBrokerFailures() = seekAndCommitWithBrokerFailures(5) |  | ||||||
| 
 |  | ||||||
|   def seekAndCommitWithBrokerFailures(numIters: Int) { |  | ||||||
|     val numRecords = 1000 |  | ||||||
|     sendRecords(numRecords) |  | ||||||
|     this.producers.map(_.close) |  | ||||||
| 
 |  | ||||||
|     val consumer = this.consumers(0) |  | ||||||
|     consumer.subscribe(tp) |  | ||||||
|     consumer.seek(tp, 0) |  | ||||||
| 
 |  | ||||||
|     val scheduler = new BounceBrokerScheduler(numIters) |  | ||||||
|     scheduler.start() |  | ||||||
| 
 |  | ||||||
|     while(scheduler.isRunning.get()) { |  | ||||||
|       val coin = TestUtils.random.nextInt(3) |  | ||||||
|       if (coin == 0) { |  | ||||||
|         info("Seeking to end of log") |  | ||||||
|         consumer.seekToEnd() |  | ||||||
|         assertEquals(numRecords.toLong, consumer.position(tp)) |  | ||||||
|       } else if (coin == 1) { |  | ||||||
|         val pos = TestUtils.random.nextInt(numRecords).toLong |  | ||||||
|         info("Seeking to " + pos) |  | ||||||
|         consumer.seek(tp, pos) |  | ||||||
|         assertEquals(pos, consumer.position(tp)) |  | ||||||
|       } else if (coin == 2) { |  | ||||||
|         info("Committing offset.") |  | ||||||
|         consumer.commit(CommitType.SYNC) |  | ||||||
|         assertEquals(consumer.position(tp), consumer.committed(tp)) |  | ||||||
|       } |  | ||||||
|     } |  | ||||||
|   } |  | ||||||
|    |  | ||||||
|   def testPartitionReassignmentCallback() { |   def testPartitionReassignmentCallback() { | ||||||
|     val callback = new TestConsumerReassignmentCallback() |     val callback = new TestConsumerReassignmentCallback() | ||||||
|     this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "200"); // timeout quickly to avoid slow test |     this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "200"); // timeout quickly to avoid slow test | ||||||
|  | @ -255,23 +189,6 @@ class ConsumerTest extends IntegrationTestHarness with Logging { | ||||||
|     }  |     }  | ||||||
|   } |   } | ||||||
| 
 | 
 | ||||||
|   private class BounceBrokerScheduler(val numIters: Int) extends ShutdownableThread("daemon-bounce-broker", false) |  | ||||||
|   { |  | ||||||
|     var iter: Int = 0 |  | ||||||
| 
 |  | ||||||
|     override def doWork(): Unit = { |  | ||||||
|       killRandomBroker() |  | ||||||
|       Thread.sleep(500) |  | ||||||
|       restartDeadBrokers() |  | ||||||
| 
 |  | ||||||
|       iter += 1 |  | ||||||
|       if (iter == numIters) |  | ||||||
|         initiateShutdown() |  | ||||||
|       else |  | ||||||
|         Thread.sleep(500) |  | ||||||
|     } |  | ||||||
|   } |  | ||||||
| 
 |  | ||||||
|   private def sendRecords(numRecords: Int) { |   private def sendRecords(numRecords: Int) { | ||||||
|     val futures = (0 until numRecords).map { i => |     val futures = (0 until numRecords).map { i => | ||||||
|       this.producers(0).send(new ProducerRecord(topic, part, i.toString.getBytes, i.toString.getBytes)) |       this.producers(0).send(new ProducerRecord(topic, part, i.toString.getBytes, i.toString.getBytes)) | ||||||
|  |  | ||||||
|  | @ -0,0 +1,55 @@ | ||||||
|  | /** | ||||||
|  |  * Copyright 2015 Confluent Inc. | ||||||
|  |  * | ||||||
|  |  * Licensed 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 kafka.api | ||||||
|  | 
 | ||||||
|  | import java.io.IOException | ||||||
|  | import java.net.ServerSocket | ||||||
|  | import java.util.Properties | ||||||
|  | 
 | ||||||
|  | import kafka.utils.TestUtils | ||||||
|  | 
 | ||||||
|  | /** | ||||||
|  |  * DO NOT USE THESE UTILITIES UNLESS YOU ABSOLUTELY MUST | ||||||
|  |  * | ||||||
|  |  * These are utilities for selecting fixed (preselected), ephemeral ports to use with tests. This is not a reliable way | ||||||
|  |  * of testing on most machines because you can easily run into port conflicts. If you're using this class, you're almost | ||||||
|  |  * certainly doing something wrong unless you can prove that your test **cannot function** properly without it. | ||||||
|  |  */ | ||||||
|  | object FixedPortTestUtils { | ||||||
|  |   def choosePorts(count: Int): Seq[Int] = { | ||||||
|  |     try { | ||||||
|  |       val sockets = (0 until count).map(i => new ServerSocket(0)) | ||||||
|  |       val ports = sockets.map(_.getLocalPort()) | ||||||
|  |       sockets.foreach(_.close()) | ||||||
|  |       ports | ||||||
|  |     } catch { | ||||||
|  |       case e: IOException => { | ||||||
|  |         throw new RuntimeException(e) | ||||||
|  |       } | ||||||
|  |     } | ||||||
|  |   } | ||||||
|  | 
 | ||||||
|  |   def createBrokerConfigs(numConfigs: Int, | ||||||
|  |     zkConnect: String, | ||||||
|  |     enableControlledShutdown: Boolean = true, | ||||||
|  |     enableDeleteTopic: Boolean = false): Seq[Properties] = { | ||||||
|  |     val ports = FixedPortTestUtils.choosePorts(numConfigs) | ||||||
|  |     (0 until numConfigs) | ||||||
|  |       .map(node => TestUtils.createBrokerConfig(node, zkConnect, enableControlledShutdown, enableDeleteTopic, ports(node))) | ||||||
|  |   } | ||||||
|  | 
 | ||||||
|  | } | ||||||
|  | @ -38,15 +38,16 @@ trait IntegrationTestHarness extends KafkaServerTestHarness { | ||||||
|   lazy val producerConfig = new Properties |   lazy val producerConfig = new Properties | ||||||
|   lazy val consumerConfig = new Properties |   lazy val consumerConfig = new Properties | ||||||
|   lazy val serverConfig = new Properties |   lazy val serverConfig = new Properties | ||||||
|   override lazy val configs = { | 
 | ||||||
|     val cfgs = TestUtils.createBrokerConfigs(serverCount) |   var consumers = Buffer[KafkaConsumer[Array[Byte], Array[Byte]]]() | ||||||
|  |   var producers = Buffer[KafkaProducer[Array[Byte], Array[Byte]]]() | ||||||
|  | 
 | ||||||
|  |   override def generateConfigs() = { | ||||||
|  |     val cfgs = TestUtils.createBrokerConfigs(serverCount, zkConnect) | ||||||
|     cfgs.map(_.putAll(serverConfig)) |     cfgs.map(_.putAll(serverConfig)) | ||||||
|     cfgs.map(KafkaConfig.fromProps) |     cfgs.map(KafkaConfig.fromProps) | ||||||
|   } |   } | ||||||
|    | 
 | ||||||
|   var consumers = Buffer[KafkaConsumer[Array[Byte], Array[Byte]]]() |  | ||||||
|   var producers = Buffer[KafkaProducer[Array[Byte], Array[Byte]]]() |  | ||||||
|    |  | ||||||
|   override def setUp() { |   override def setUp() { | ||||||
|     super.setUp() |     super.setUp() | ||||||
|     producerConfig.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, this.bootstrapUrl) |     producerConfig.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, this.bootstrapUrl) | ||||||
|  |  | ||||||
|  | @ -0,0 +1,164 @@ | ||||||
|  | /** | ||||||
|  |  * Copyright 2015 Confluent Inc. | ||||||
|  |  * | ||||||
|  |  * Licensed 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 kafka.api | ||||||
|  | 
 | ||||||
|  | import java.util.Properties | ||||||
|  | 
 | ||||||
|  | import kafka.consumer.SimpleConsumer | ||||||
|  | import kafka.integration.KafkaServerTestHarness | ||||||
|  | import kafka.server.KafkaConfig | ||||||
|  | import kafka.utils.{ShutdownableThread, TestUtils} | ||||||
|  | import org.apache.kafka.clients.producer._ | ||||||
|  | import org.apache.kafka.clients.producer.internals.ErrorLoggingCallback | ||||||
|  | import org.junit.Assert._ | ||||||
|  | import org.junit.Test | ||||||
|  | 
 | ||||||
|  | class ProducerBounceTest extends KafkaServerTestHarness { | ||||||
|  |   private val producerBufferSize = 30000 | ||||||
|  |   private val serverMessageMaxBytes =  producerBufferSize/2 | ||||||
|  | 
 | ||||||
|  |   val numServers = 2 | ||||||
|  | 
 | ||||||
|  |   val overridingProps = new Properties() | ||||||
|  |   overridingProps.put(KafkaConfig.AutoCreateTopicsEnableProp, false.toString) | ||||||
|  |   overridingProps.put(KafkaConfig.MessageMaxBytesProp, serverMessageMaxBytes.toString) | ||||||
|  |   // Set a smaller value for the number of partitions for the offset commit topic (__consumer_offset topic) | ||||||
|  |   // so that the creation of that topic/partition(s) and subsequent leader assignment doesn't take relatively long | ||||||
|  |   overridingProps.put(KafkaConfig.OffsetsTopicPartitionsProp, 1.toString) | ||||||
|  | 
 | ||||||
|  |   // This is the one of the few tests we currently allow to preallocate ports, despite the fact that this can result in transient | ||||||
|  |   // failures due to ports getting reused. We can't use random ports because of bad behavior that can result from bouncing | ||||||
|  |   // brokers too quickly when they get new, random ports. If we're not careful, the client can end up in a situation | ||||||
|  |   // where metadata is not refreshed quickly enough, and by the time it's actually trying to, all the servers have | ||||||
|  |   // been bounced and have new addresses. None of the bootstrap nodes or current metadata can get them connected to a | ||||||
|  |   // running server. | ||||||
|  |   // | ||||||
|  |   // Since such quick rotation of servers is incredibly unrealistic, we allow this one test to preallocate ports, leaving | ||||||
|  |   // a small risk of hitting errors due to port conflicts. Hopefully this is infrequent enough to not cause problems. | ||||||
|  |   override def generateConfigs() = { | ||||||
|  |     FixedPortTestUtils.createBrokerConfigs(numServers, zkConnect,enableControlledShutdown = false) | ||||||
|  |       .map(KafkaConfig.fromProps(_, overridingProps)) | ||||||
|  |   } | ||||||
|  | 
 | ||||||
|  |   private var consumer1: SimpleConsumer = null | ||||||
|  |   private var consumer2: SimpleConsumer = null | ||||||
|  | 
 | ||||||
|  |   private var producer1: KafkaProducer[Array[Byte],Array[Byte]] = null | ||||||
|  |   private var producer2: KafkaProducer[Array[Byte],Array[Byte]] = null | ||||||
|  |   private var producer3: KafkaProducer[Array[Byte],Array[Byte]] = null | ||||||
|  |   private var producer4: KafkaProducer[Array[Byte],Array[Byte]] = null | ||||||
|  | 
 | ||||||
|  |   private val topic1 = "topic-1" | ||||||
|  |   private val topic2 = "topic-2" | ||||||
|  | 
 | ||||||
|  |   override def setUp() { | ||||||
|  |     super.setUp() | ||||||
|  | 
 | ||||||
|  |     producer1 = TestUtils.createNewProducer(brokerList, acks = 0, blockOnBufferFull = false, bufferSize = producerBufferSize) | ||||||
|  |     producer2 = TestUtils.createNewProducer(brokerList, acks = 1, blockOnBufferFull = false, bufferSize = producerBufferSize) | ||||||
|  |     producer3 = TestUtils.createNewProducer(brokerList, acks = -1, blockOnBufferFull = false, bufferSize = producerBufferSize) | ||||||
|  |   } | ||||||
|  | 
 | ||||||
|  |   override def tearDown() { | ||||||
|  |     if (producer1 != null) producer1.close | ||||||
|  |     if (producer2 != null) producer2.close | ||||||
|  |     if (producer3 != null) producer3.close | ||||||
|  |     if (producer4 != null) producer4.close | ||||||
|  | 
 | ||||||
|  |     super.tearDown() | ||||||
|  |   } | ||||||
|  | 
 | ||||||
|  |   /** | ||||||
|  |    * With replication, producer should able able to find new leader after it detects broker failure | ||||||
|  |    */ | ||||||
|  |   @Test | ||||||
|  |   def testBrokerFailure() { | ||||||
|  |     val numPartitions = 3 | ||||||
|  |     val leaders = TestUtils.createTopic(zkClient, topic1, numPartitions, numServers, servers) | ||||||
|  |     assertTrue("Leader of all partitions of the topic should exist", leaders.values.forall(leader => leader.isDefined)) | ||||||
|  | 
 | ||||||
|  |     val scheduler = new ProducerScheduler() | ||||||
|  |     scheduler.start | ||||||
|  | 
 | ||||||
|  |     // rolling bounce brokers | ||||||
|  |     for (i <- 0 until numServers) { | ||||||
|  |       for (server <- servers) { | ||||||
|  |         server.shutdown() | ||||||
|  |         server.awaitShutdown() | ||||||
|  |         server.startup() | ||||||
|  |         Thread.sleep(2000) | ||||||
|  |       } | ||||||
|  | 
 | ||||||
|  |       // Make sure the producer do not see any exception | ||||||
|  |       // in returned metadata due to broker failures | ||||||
|  |       assertTrue(scheduler.failed == false) | ||||||
|  | 
 | ||||||
|  |       // Make sure the leader still exists after bouncing brokers | ||||||
|  |       (0 until numPartitions).foreach(partition => TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic1, partition)) | ||||||
|  |     } | ||||||
|  | 
 | ||||||
|  |     scheduler.shutdown | ||||||
|  | 
 | ||||||
|  |     // Make sure the producer do not see any exception | ||||||
|  |     // when draining the left messages on shutdown | ||||||
|  |     assertTrue(scheduler.failed == false) | ||||||
|  | 
 | ||||||
|  |     // double check that the leader info has been propagated after consecutive bounces | ||||||
|  |     val newLeaders = (0 until numPartitions).map(i => TestUtils.waitUntilMetadataIsPropagated(servers, topic1, i)) | ||||||
|  |     val fetchResponses = newLeaders.zipWithIndex.map { case (leader, partition) => | ||||||
|  |       // Consumers must be instantiated after all the restarts since they use random ports each time they start up | ||||||
|  |       val consumer = new SimpleConsumer("localhost", servers(leader).boundPort(), 100, 1024 * 1024, "") | ||||||
|  |       val response = consumer.fetch(new FetchRequestBuilder().addFetch(topic1, partition, 0, Int.MaxValue).build()).messageSet(topic1, partition) | ||||||
|  |       consumer.close | ||||||
|  |       response | ||||||
|  |     } | ||||||
|  |     val messages = fetchResponses.flatMap(r => r.iterator.toList.map(_.message)) | ||||||
|  |     val uniqueMessages = messages.toSet | ||||||
|  |     val uniqueMessageSize = uniqueMessages.size | ||||||
|  | 
 | ||||||
|  |     assertEquals("Should have fetched " + scheduler.sent + " unique messages", scheduler.sent, uniqueMessageSize) | ||||||
|  |   } | ||||||
|  | 
 | ||||||
|  |   private class ProducerScheduler extends ShutdownableThread("daemon-producer", false) | ||||||
|  |   { | ||||||
|  |     val numRecords = 1000 | ||||||
|  |     var sent = 0 | ||||||
|  |     var failed = false | ||||||
|  | 
 | ||||||
|  |     val producer = TestUtils.createNewProducer(brokerList, bufferSize = producerBufferSize, retries = 10) | ||||||
|  | 
 | ||||||
|  |     override def doWork(): Unit = { | ||||||
|  |       val responses = | ||||||
|  |         for (i <- sent+1 to sent+numRecords) | ||||||
|  |         yield producer.send(new ProducerRecord[Array[Byte],Array[Byte]](topic1, null, null, i.toString.getBytes), | ||||||
|  |                             new ErrorLoggingCallback(topic1, null, null, true)) | ||||||
|  |       val futures = responses.toList | ||||||
|  | 
 | ||||||
|  |       try { | ||||||
|  |         futures.map(_.get) | ||||||
|  |         sent += numRecords | ||||||
|  |       } catch { | ||||||
|  |         case e : Exception => failed = true | ||||||
|  |       } | ||||||
|  |     } | ||||||
|  | 
 | ||||||
|  |     override def shutdown(){ | ||||||
|  |       super.shutdown() | ||||||
|  |       producer.close | ||||||
|  |     } | ||||||
|  |   } | ||||||
|  | } | ||||||
|  | @ -34,24 +34,22 @@ import kafka.message.Message | ||||||
| import kafka.zk.ZooKeeperTestHarness | import kafka.zk.ZooKeeperTestHarness | ||||||
| import kafka.utils.{Utils, TestUtils} | import kafka.utils.{Utils, TestUtils} | ||||||
| 
 | 
 | ||||||
| import scala.Array |  | ||||||
| 
 |  | ||||||
| 
 | 
 | ||||||
| @RunWith(value = classOf[Parameterized]) | @RunWith(value = classOf[Parameterized]) | ||||||
| class ProducerCompressionTest(compression: String) extends JUnit3Suite with ZooKeeperTestHarness { | class ProducerCompressionTest(compression: String) extends JUnit3Suite with ZooKeeperTestHarness { | ||||||
|   private val brokerId = 0 |   private val brokerId = 0 | ||||||
|   private val port = TestUtils.choosePort |  | ||||||
|   private var server: KafkaServer = null |   private var server: KafkaServer = null | ||||||
| 
 | 
 | ||||||
|   private val props = TestUtils.createBrokerConfig(brokerId, port) |  | ||||||
|   private val config = KafkaConfig.fromProps(props) |  | ||||||
| 
 |  | ||||||
|   private val topic = "topic" |   private val topic = "topic" | ||||||
|   private val numRecords = 2000 |   private val numRecords = 2000 | ||||||
| 
 | 
 | ||||||
|   @Before |   @Before | ||||||
|   override def setUp() { |   override def setUp() { | ||||||
|     super.setUp() |     super.setUp() | ||||||
|  | 
 | ||||||
|  |     val props = TestUtils.createBrokerConfig(brokerId, zkConnect) | ||||||
|  |     val config = KafkaConfig.fromProps(props) | ||||||
|  | 
 | ||||||
|     server = TestUtils.createServer(config) |     server = TestUtils.createServer(config) | ||||||
|   } |   } | ||||||
| 
 | 
 | ||||||
|  | @ -71,14 +69,14 @@ class ProducerCompressionTest(compression: String) extends JUnit3Suite with ZooK | ||||||
|   def testCompression() { |   def testCompression() { | ||||||
| 
 | 
 | ||||||
|     val props = new Properties() |     val props = new Properties() | ||||||
|     props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, TestUtils.getBrokerListStrFromConfigs(Seq(config))) |     props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, TestUtils.getBrokerListStrFromServers(Seq(server))) | ||||||
|     props.put(ProducerConfig.COMPRESSION_TYPE_CONFIG, compression) |     props.put(ProducerConfig.COMPRESSION_TYPE_CONFIG, compression) | ||||||
|     props.put(ProducerConfig.BATCH_SIZE_CONFIG, "66000") |     props.put(ProducerConfig.BATCH_SIZE_CONFIG, "66000") | ||||||
|     props.put(ProducerConfig.LINGER_MS_CONFIG, "200") |     props.put(ProducerConfig.LINGER_MS_CONFIG, "200") | ||||||
|     props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") |     props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") | ||||||
|     props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") |     props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") | ||||||
|     var producer = new KafkaProducer[Array[Byte],Array[Byte]](props) |     var producer = new KafkaProducer[Array[Byte],Array[Byte]](props) | ||||||
|     val consumer = new SimpleConsumer("localhost", port, 100, 1024*1024, "") |     val consumer = new SimpleConsumer("localhost", server.boundPort(), 100, 1024*1024, "") | ||||||
| 
 | 
 | ||||||
|     try { |     try { | ||||||
|       // create topic |       // create topic | ||||||
|  |  | ||||||
|  | @ -20,7 +20,6 @@ package kafka.api | ||||||
| import org.junit.Test | import org.junit.Test | ||||||
| import org.junit.Assert._ | import org.junit.Assert._ | ||||||
| 
 | 
 | ||||||
| import java.lang.Integer |  | ||||||
| import java.util.{Properties, Random} | import java.util.{Properties, Random} | ||||||
| import java.util.concurrent.{TimeoutException, TimeUnit, ExecutionException} | import java.util.concurrent.{TimeoutException, TimeUnit, ExecutionException} | ||||||
| 
 | 
 | ||||||
|  | @ -28,7 +27,7 @@ import kafka.common.Topic | ||||||
| import kafka.consumer.SimpleConsumer | import kafka.consumer.SimpleConsumer | ||||||
| import kafka.integration.KafkaServerTestHarness | import kafka.integration.KafkaServerTestHarness | ||||||
| import kafka.server.KafkaConfig | import kafka.server.KafkaConfig | ||||||
| import kafka.utils.{TestZKUtils, ShutdownableThread, TestUtils} | import kafka.utils.{ShutdownableThread, TestUtils} | ||||||
| 
 | 
 | ||||||
| import org.apache.kafka.common.KafkaException | import org.apache.kafka.common.KafkaException | ||||||
| import org.apache.kafka.common.errors.{InvalidTopicException, NotEnoughReplicasException, NotEnoughReplicasAfterAppendException} | import org.apache.kafka.common.errors.{InvalidTopicException, NotEnoughReplicasException, NotEnoughReplicasAfterAppendException} | ||||||
|  | @ -42,16 +41,14 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { | ||||||
|   val numServers = 2 |   val numServers = 2 | ||||||
| 
 | 
 | ||||||
|   val overridingProps = new Properties() |   val overridingProps = new Properties() | ||||||
|   overridingProps.put(KafkaConfig.ZkConnectProp, TestZKUtils.zookeeperConnect) |  | ||||||
|   overridingProps.put(KafkaConfig.AutoCreateTopicsEnableProp, false.toString) |   overridingProps.put(KafkaConfig.AutoCreateTopicsEnableProp, false.toString) | ||||||
|   overridingProps.put(KafkaConfig.MessageMaxBytesProp, serverMessageMaxBytes.toString) |   overridingProps.put(KafkaConfig.MessageMaxBytesProp, serverMessageMaxBytes.toString) | ||||||
|   // Set a smaller value for the number of partitions for the offset commit topic (__consumer_offset topic) |   // Set a smaller value for the number of partitions for the offset commit topic (__consumer_offset topic) | ||||||
|   // so that the creation of that topic/partition(s) and subsequent leader assignment doesn't take relatively long |   // so that the creation of that topic/partition(s) and subsequent leader assignment doesn't take relatively long | ||||||
|   overridingProps.put(KafkaConfig.OffsetsTopicPartitionsProp, 1.toString) |   overridingProps.put(KafkaConfig.OffsetsTopicPartitionsProp, 1.toString) | ||||||
| 
 | 
 | ||||||
|   val configs = |   def generateConfigs() = | ||||||
|     for (props <- TestUtils.createBrokerConfigs(numServers, false)) |     TestUtils.createBrokerConfigs(numServers, zkConnect, false).map(KafkaConfig.fromProps(_, overridingProps)) | ||||||
|     yield KafkaConfig.fromProps(props, overridingProps) |  | ||||||
| 
 | 
 | ||||||
|   private var consumer1: SimpleConsumer = null |   private var consumer1: SimpleConsumer = null | ||||||
|   private var consumer2: SimpleConsumer = null |   private var consumer2: SimpleConsumer = null | ||||||
|  | @ -67,19 +64,12 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { | ||||||
|   override def setUp() { |   override def setUp() { | ||||||
|     super.setUp() |     super.setUp() | ||||||
| 
 | 
 | ||||||
|     // TODO: we need to migrate to new consumers when 0.9 is final |  | ||||||
|     consumer1 = new SimpleConsumer("localhost", configs(0).port, 100, 1024*1024, "") |  | ||||||
|     consumer2 = new SimpleConsumer("localhost", configs(1).port, 100, 1024*1024, "") |  | ||||||
| 
 |  | ||||||
|     producer1 = TestUtils.createNewProducer(brokerList, acks = 0, blockOnBufferFull = false, bufferSize = producerBufferSize) |     producer1 = TestUtils.createNewProducer(brokerList, acks = 0, blockOnBufferFull = false, bufferSize = producerBufferSize) | ||||||
|     producer2 = TestUtils.createNewProducer(brokerList, acks = 1, blockOnBufferFull = false, bufferSize = producerBufferSize) |     producer2 = TestUtils.createNewProducer(brokerList, acks = 1, blockOnBufferFull = false, bufferSize = producerBufferSize) | ||||||
|     producer3 = TestUtils.createNewProducer(brokerList, acks = -1, blockOnBufferFull = false, bufferSize = producerBufferSize) |     producer3 = TestUtils.createNewProducer(brokerList, acks = -1, blockOnBufferFull = false, bufferSize = producerBufferSize) | ||||||
|   } |   } | ||||||
| 
 | 
 | ||||||
|   override def tearDown() { |   override def tearDown() { | ||||||
|     consumer1.close |  | ||||||
|     consumer2.close |  | ||||||
| 
 |  | ||||||
|     if (producer1 != null) producer1.close |     if (producer1 != null) producer1.close | ||||||
|     if (producer2 != null) producer2.close |     if (producer2 != null) producer2.close | ||||||
|     if (producer3 != null) producer3.close |     if (producer3 != null) producer3.close | ||||||
|  | @ -94,7 +84,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { | ||||||
|   @Test |   @Test | ||||||
|   def testTooLargeRecordWithAckZero() { |   def testTooLargeRecordWithAckZero() { | ||||||
|     // create topic |     // create topic | ||||||
|     TestUtils.createTopic(zkClient, topic1, 1, 2, servers) |     TestUtils.createTopic(zkClient, topic1, 1, numServers, servers) | ||||||
| 
 | 
 | ||||||
|     // send a too-large record |     // send a too-large record | ||||||
|     val record = new ProducerRecord[Array[Byte],Array[Byte]](topic1, null, "key".getBytes, new Array[Byte](serverMessageMaxBytes + 1)) |     val record = new ProducerRecord[Array[Byte],Array[Byte]](topic1, null, "key".getBytes, new Array[Byte](serverMessageMaxBytes + 1)) | ||||||
|  | @ -107,7 +97,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { | ||||||
|   @Test |   @Test | ||||||
|   def testTooLargeRecordWithAckOne() { |   def testTooLargeRecordWithAckOne() { | ||||||
|     // create topic |     // create topic | ||||||
|     TestUtils.createTopic(zkClient, topic1, 1, 2, servers) |     TestUtils.createTopic(zkClient, topic1, 1, numServers, servers) | ||||||
| 
 | 
 | ||||||
|     // send a too-large record |     // send a too-large record | ||||||
|     val record = new ProducerRecord[Array[Byte],Array[Byte]](topic1, null, "key".getBytes, new Array[Byte](serverMessageMaxBytes + 1)) |     val record = new ProducerRecord[Array[Byte],Array[Byte]](topic1, null, "key".getBytes, new Array[Byte](serverMessageMaxBytes + 1)) | ||||||
|  | @ -141,7 +131,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { | ||||||
|   @Test |   @Test | ||||||
|   def testWrongBrokerList() { |   def testWrongBrokerList() { | ||||||
|     // create topic |     // create topic | ||||||
|     TestUtils.createTopic(zkClient, topic1, 1, 2, servers) |     TestUtils.createTopic(zkClient, topic1, 1, numServers, servers) | ||||||
| 
 | 
 | ||||||
|     // producer with incorrect broker list |     // producer with incorrect broker list | ||||||
|     producer4 = TestUtils.createNewProducer("localhost:8686,localhost:4242", acks = 1, blockOnBufferFull = false, bufferSize = producerBufferSize) |     producer4 = TestUtils.createNewProducer("localhost:8686,localhost:4242", acks = 1, blockOnBufferFull = false, bufferSize = producerBufferSize) | ||||||
|  | @ -161,7 +151,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { | ||||||
|   @Test |   @Test | ||||||
|   def testNoResponse() { |   def testNoResponse() { | ||||||
|     // create topic |     // create topic | ||||||
|     TestUtils.createTopic(zkClient, topic1, 1, 2, servers) |     TestUtils.createTopic(zkClient, topic1, 1, numServers, servers) | ||||||
| 
 | 
 | ||||||
|     // first send a message to make sure the metadata is refreshed |     // first send a message to make sure the metadata is refreshed | ||||||
|     val record1 = new ProducerRecord[Array[Byte],Array[Byte]](topic1, null, "key".getBytes, "value".getBytes) |     val record1 = new ProducerRecord[Array[Byte],Array[Byte]](topic1, null, "key".getBytes, "value".getBytes) | ||||||
|  | @ -202,7 +192,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { | ||||||
|   @Test |   @Test | ||||||
|   def testInvalidPartition() { |   def testInvalidPartition() { | ||||||
|     // create topic |     // create topic | ||||||
|     TestUtils.createTopic(zkClient, topic1, 1, 2, servers) |     TestUtils.createTopic(zkClient, topic1, 1, numServers, servers) | ||||||
| 
 | 
 | ||||||
|     // create a record with incorrect partition id, send should fail |     // create a record with incorrect partition id, send should fail | ||||||
|     val record = new ProducerRecord[Array[Byte],Array[Byte]](topic1, new Integer(1), "key".getBytes, "value".getBytes) |     val record = new ProducerRecord[Array[Byte],Array[Byte]](topic1, new Integer(1), "key".getBytes, "value".getBytes) | ||||||
|  | @ -223,7 +213,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { | ||||||
|   @Test |   @Test | ||||||
|   def testSendAfterClosed() { |   def testSendAfterClosed() { | ||||||
|     // create topic |     // create topic | ||||||
|     TestUtils.createTopic(zkClient, topic1, 1, 2, servers) |     TestUtils.createTopic(zkClient, topic1, 1, numServers, servers) | ||||||
| 
 | 
 | ||||||
|     val record = new ProducerRecord[Array[Byte],Array[Byte]](topic1, null, "key".getBytes, "value".getBytes) |     val record = new ProducerRecord[Array[Byte],Array[Byte]](topic1, null, "key".getBytes, "value".getBytes) | ||||||
| 
 | 
 | ||||||
|  | @ -248,59 +238,6 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { | ||||||
|     // re-close producer is fine |     // re-close producer is fine | ||||||
|   } |   } | ||||||
| 
 | 
 | ||||||
|   /** |  | ||||||
|    * With replication, producer should able able to find new leader after it detects broker failure |  | ||||||
|    */ |  | ||||||
|   @Test |  | ||||||
|   def testBrokerFailure() { |  | ||||||
|     // create topic |  | ||||||
|     val leaders = TestUtils.createTopic(zkClient, topic1, 1, 2, servers) |  | ||||||
|     val partition = 0 |  | ||||||
|     assertTrue("Leader of partition 0 of the topic should exist", leaders(partition).isDefined) |  | ||||||
| 
 |  | ||||||
|     val scheduler = new ProducerScheduler() |  | ||||||
|     scheduler.start |  | ||||||
| 
 |  | ||||||
|     // rolling bounce brokers |  | ||||||
|     for (i <- 0 until 2) { |  | ||||||
|       for (server <- servers) { |  | ||||||
|         server.shutdown() |  | ||||||
|         server.awaitShutdown() |  | ||||||
|         server.startup() |  | ||||||
| 
 |  | ||||||
|         Thread.sleep(2000) |  | ||||||
|       } |  | ||||||
| 
 |  | ||||||
|       // Make sure the producer do not see any exception |  | ||||||
|       // in returned metadata due to broker failures |  | ||||||
|       assertTrue(scheduler.failed == false) |  | ||||||
| 
 |  | ||||||
|       // Make sure the leader still exists after bouncing brokers |  | ||||||
|       TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic1, partition) |  | ||||||
|     } |  | ||||||
| 
 |  | ||||||
|     scheduler.shutdown |  | ||||||
| 
 |  | ||||||
|     // Make sure the producer do not see any exception |  | ||||||
|     // when draining the left messages on shutdown |  | ||||||
|     assertTrue(scheduler.failed == false) |  | ||||||
| 
 |  | ||||||
|     // double check that the leader info has been propagated after consecutive bounces |  | ||||||
|     val leader = TestUtils.waitUntilMetadataIsPropagated(servers, topic1, partition) |  | ||||||
| 
 |  | ||||||
|     val fetchResponse = if(leader == configs(0).brokerId) { |  | ||||||
|       consumer1.fetch(new FetchRequestBuilder().addFetch(topic1, partition, 0, Int.MaxValue).build()).messageSet(topic1, partition) |  | ||||||
|     } else { |  | ||||||
|       consumer2.fetch(new FetchRequestBuilder().addFetch(topic1, partition, 0, Int.MaxValue).build()).messageSet(topic1, partition) |  | ||||||
|     } |  | ||||||
| 
 |  | ||||||
|     val messages = fetchResponse.iterator.toList.map(_.message) |  | ||||||
|     val uniqueMessages = messages.toSet |  | ||||||
|     val uniqueMessageSize = uniqueMessages.size |  | ||||||
| 
 |  | ||||||
|     assertEquals("Should have fetched " + scheduler.sent + " unique messages", scheduler.sent, uniqueMessageSize) |  | ||||||
|   } |  | ||||||
| 
 |  | ||||||
|   @Test |   @Test | ||||||
|   def testCannotSendToInternalTopic() { |   def testCannotSendToInternalTopic() { | ||||||
|     val thrown = intercept[ExecutionException] { |     val thrown = intercept[ExecutionException] { | ||||||
|  | @ -313,9 +250,9 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { | ||||||
|   def testNotEnoughReplicas() { |   def testNotEnoughReplicas() { | ||||||
|     val topicName = "minisrtest" |     val topicName = "minisrtest" | ||||||
|     val topicProps = new Properties() |     val topicProps = new Properties() | ||||||
|     topicProps.put("min.insync.replicas","3") |     topicProps.put("min.insync.replicas",(numServers+1).toString) | ||||||
| 
 | 
 | ||||||
|     TestUtils.createTopic(zkClient, topicName, 1, 2, servers,topicProps) |     TestUtils.createTopic(zkClient, topicName, 1, numServers, servers, topicProps) | ||||||
| 
 | 
 | ||||||
|     val record = new ProducerRecord[Array[Byte],Array[Byte]](topicName, null, "key".getBytes, "value".getBytes) |     val record = new ProducerRecord[Array[Byte],Array[Byte]](topicName, null, "key".getBytes, "value".getBytes) | ||||||
|     try { |     try { | ||||||
|  | @ -333,9 +270,9 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { | ||||||
|   def testNotEnoughReplicasAfterBrokerShutdown() { |   def testNotEnoughReplicasAfterBrokerShutdown() { | ||||||
|     val topicName = "minisrtest2" |     val topicName = "minisrtest2" | ||||||
|     val topicProps = new Properties() |     val topicProps = new Properties() | ||||||
|     topicProps.put("min.insync.replicas","2") |     topicProps.put("min.insync.replicas",numServers.toString) | ||||||
| 
 | 
 | ||||||
|     TestUtils.createTopic(zkClient, topicName, 1, 2, servers,topicProps) |     TestUtils.createTopic(zkClient, topicName, 1, numServers, servers,topicProps) | ||||||
| 
 | 
 | ||||||
|     val record = new ProducerRecord[Array[Byte],Array[Byte]](topicName, null, "key".getBytes, "value".getBytes) |     val record = new ProducerRecord[Array[Byte],Array[Byte]](topicName, null, "key".getBytes, "value".getBytes) | ||||||
|     // this should work with all brokers up and running |     // this should work with all brokers up and running | ||||||
|  |  | ||||||
|  | @ -25,7 +25,7 @@ import org.junit.Test | ||||||
| import org.junit.Assert._ | import org.junit.Assert._ | ||||||
| 
 | 
 | ||||||
| import kafka.server.KafkaConfig | import kafka.server.KafkaConfig | ||||||
| import kafka.utils.{TestZKUtils, TestUtils} | import kafka.utils.TestUtils | ||||||
| import kafka.consumer.SimpleConsumer | import kafka.consumer.SimpleConsumer | ||||||
| import kafka.message.Message | import kafka.message.Message | ||||||
| import kafka.integration.KafkaServerTestHarness | import kafka.integration.KafkaServerTestHarness | ||||||
|  | @ -39,12 +39,10 @@ class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness { | ||||||
|   val numServers = 2 |   val numServers = 2 | ||||||
| 
 | 
 | ||||||
|   val overridingProps = new Properties() |   val overridingProps = new Properties() | ||||||
|   overridingProps.put(KafkaConfig.ZkConnectProp, TestZKUtils.zookeeperConnect) |  | ||||||
|   overridingProps.put(KafkaConfig.NumPartitionsProp, 4.toString) |   overridingProps.put(KafkaConfig.NumPartitionsProp, 4.toString) | ||||||
| 
 | 
 | ||||||
|   val configs = |   def generateConfigs() = | ||||||
|     for (props <- TestUtils.createBrokerConfigs(numServers, false)) |     TestUtils.createBrokerConfigs(numServers, zkConnect, false).map(KafkaConfig.fromProps(_, overridingProps)) | ||||||
|     yield KafkaConfig.fromProps(props, overridingProps) |  | ||||||
| 
 | 
 | ||||||
|   private var consumer1: SimpleConsumer = null |   private var consumer1: SimpleConsumer = null | ||||||
|   private var consumer2: SimpleConsumer = null |   private var consumer2: SimpleConsumer = null | ||||||
|  | @ -56,8 +54,8 @@ class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness { | ||||||
|     super.setUp() |     super.setUp() | ||||||
| 
 | 
 | ||||||
|     // TODO: we need to migrate to new consumers when 0.9 is final |     // TODO: we need to migrate to new consumers when 0.9 is final | ||||||
|     consumer1 = new SimpleConsumer("localhost", configs(0).port, 100, 1024*1024, "") |     consumer1 = new SimpleConsumer("localhost", servers(0).boundPort(), 100, 1024*1024, "") | ||||||
|     consumer2 = new SimpleConsumer("localhost", configs(1).port, 100, 1024*1024, "") |     consumer2 = new SimpleConsumer("localhost", servers(1).boundPort(), 100, 1024*1024, "") | ||||||
|   } |   } | ||||||
| 
 | 
 | ||||||
|   override def tearDown() { |   override def tearDown() { | ||||||
|  |  | ||||||
|  | @ -27,21 +27,7 @@ import kafka.client.ClientUtils | ||||||
| import kafka.server.{KafkaConfig, KafkaServer} | import kafka.server.{KafkaConfig, KafkaServer} | ||||||
| 
 | 
 | ||||||
| class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { | class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { | ||||||
|   val brokerId1 = 0 |   var configs: Seq[KafkaConfig] = null | ||||||
|   val brokerId2 = 1 |  | ||||||
|   val brokerId3 = 2 |  | ||||||
|   val brokerId4 = 3 |  | ||||||
| 
 |  | ||||||
|   val port1 = TestUtils.choosePort() |  | ||||||
|   val port2 = TestUtils.choosePort() |  | ||||||
|   val port3 = TestUtils.choosePort() |  | ||||||
|   val port4 = TestUtils.choosePort() |  | ||||||
| 
 |  | ||||||
|   val configProps1 = TestUtils.createBrokerConfig(brokerId1, port1, false) |  | ||||||
|   val configProps2 = TestUtils.createBrokerConfig(brokerId2, port2, false) |  | ||||||
|   val configProps3 = TestUtils.createBrokerConfig(brokerId3, port3, false) |  | ||||||
|   val configProps4 = TestUtils.createBrokerConfig(brokerId4, port4, false) |  | ||||||
| 
 |  | ||||||
|   var servers: Seq[KafkaServer] = Seq.empty[KafkaServer] |   var servers: Seq[KafkaServer] = Seq.empty[KafkaServer] | ||||||
|   var brokers: Seq[Broker] = Seq.empty[Broker] |   var brokers: Seq[Broker] = Seq.empty[Broker] | ||||||
| 
 | 
 | ||||||
|  | @ -54,14 +40,11 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { | ||||||
| 
 | 
 | ||||||
|   override def setUp() { |   override def setUp() { | ||||||
|     super.setUp() |     super.setUp() | ||||||
|     // start all the servers |  | ||||||
|     val server1 = TestUtils.createServer(KafkaConfig.fromProps(configProps1)) |  | ||||||
|     val server2 = TestUtils.createServer(KafkaConfig.fromProps(configProps2)) |  | ||||||
|     val server3 = TestUtils.createServer(KafkaConfig.fromProps(configProps3)) |  | ||||||
|     val server4 = TestUtils.createServer(KafkaConfig.fromProps(configProps4)) |  | ||||||
| 
 | 
 | ||||||
|     servers ++= List(server1, server2, server3, server4) |     configs = (0 until 4).map(i => KafkaConfig.fromProps(TestUtils.createBrokerConfig(i, zkConnect, enableControlledShutdown = false))) | ||||||
|     brokers = servers.map(s => new Broker(s.config.brokerId, s.config.hostName, s.config.port)) |     // start all the servers | ||||||
|  |     servers = configs.map(c => TestUtils.createServer(c)) | ||||||
|  |     brokers = servers.map(s => new Broker(s.config.brokerId, s.config.hostName, s.boundPort)) | ||||||
| 
 | 
 | ||||||
|     // create topics first |     // create topics first | ||||||
|     createTopic(zkClient, topic1, partitionReplicaAssignment = Map(0->Seq(0,1)), servers = servers) |     createTopic(zkClient, topic1, partitionReplicaAssignment = Map(0->Seq(0,1)), servers = servers) | ||||||
|  |  | ||||||
|  | @ -145,7 +145,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { | ||||||
|     val expectedReplicaAssignment = Map(0  -> List(0, 1, 2)) |     val expectedReplicaAssignment = Map(0  -> List(0, 1, 2)) | ||||||
|     val topic = "test" |     val topic = "test" | ||||||
|     // create brokers |     // create brokers | ||||||
|     val servers = TestUtils.createBrokerConfigs(4, false).map(b => TestUtils.createServer(KafkaConfig.fromProps(b))) |     val servers = TestUtils.createBrokerConfigs(4, zkConnect, false).map(b => TestUtils.createServer(KafkaConfig.fromProps(b))) | ||||||
|     // create the topic |     // create the topic | ||||||
|     AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) |     AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) | ||||||
|     // reassign partition 0 |     // reassign partition 0 | ||||||
|  | @ -176,7 +176,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { | ||||||
|     val expectedReplicaAssignment = Map(0  -> List(0, 1, 2)) |     val expectedReplicaAssignment = Map(0  -> List(0, 1, 2)) | ||||||
|     val topic = "test" |     val topic = "test" | ||||||
|     // create brokers |     // create brokers | ||||||
|     val servers = TestUtils.createBrokerConfigs(4, false).map(b => TestUtils.createServer(KafkaConfig.fromProps(b))) |     val servers = TestUtils.createBrokerConfigs(4, zkConnect, false).map(b => TestUtils.createServer(KafkaConfig.fromProps(b))) | ||||||
|     // create the topic |     // create the topic | ||||||
|     AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) |     AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) | ||||||
|     // reassign partition 0 |     // reassign partition 0 | ||||||
|  | @ -207,7 +207,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { | ||||||
|     val expectedReplicaAssignment = Map(0  -> List(0, 1)) |     val expectedReplicaAssignment = Map(0  -> List(0, 1)) | ||||||
|     val topic = "test" |     val topic = "test" | ||||||
|     // create brokers |     // create brokers | ||||||
|     val servers = TestUtils.createBrokerConfigs(4, false).map(b => TestUtils.createServer(KafkaConfig.fromProps(b))) |     val servers = TestUtils.createBrokerConfigs(4, zkConnect, false).map(b => TestUtils.createServer(KafkaConfig.fromProps(b))) | ||||||
|     // create the topic |     // create the topic | ||||||
|     AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) |     AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) | ||||||
|     // reassign partition 0 |     // reassign partition 0 | ||||||
|  | @ -236,7 +236,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { | ||||||
|   def testReassigningNonExistingPartition() { |   def testReassigningNonExistingPartition() { | ||||||
|     val topic = "test" |     val topic = "test" | ||||||
|     // create brokers |     // create brokers | ||||||
|     val servers = TestUtils.createBrokerConfigs(4, false).map(b => TestUtils.createServer(KafkaConfig.fromProps(b))) |     val servers = TestUtils.createBrokerConfigs(4, zkConnect, false).map(b => TestUtils.createServer(KafkaConfig.fromProps(b))) | ||||||
|     // reassign partition 0 |     // reassign partition 0 | ||||||
|     val newReplicas = Seq(2, 3) |     val newReplicas = Seq(2, 3) | ||||||
|     val partitionToBeReassigned = 0 |     val partitionToBeReassigned = 0 | ||||||
|  | @ -262,7 +262,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { | ||||||
|     val reassignPartitionsCommand = new ReassignPartitionsCommand(zkClient, Map(topicAndPartition -> newReplicas)) |     val reassignPartitionsCommand = new ReassignPartitionsCommand(zkClient, Map(topicAndPartition -> newReplicas)) | ||||||
|     reassignPartitionsCommand.reassignPartitions |     reassignPartitionsCommand.reassignPartitions | ||||||
|     // create brokers |     // create brokers | ||||||
|     val servers = TestUtils.createBrokerConfigs(2, false).map(b => TestUtils.createServer(KafkaConfig.fromProps(b))) |     val servers = TestUtils.createBrokerConfigs(2, zkConnect, false).map(b => TestUtils.createServer(KafkaConfig.fromProps(b))) | ||||||
| 
 | 
 | ||||||
|     // wait until reassignment completes |     // wait until reassignment completes | ||||||
|     TestUtils.waitUntilTrue(() => !checkIfReassignPartitionPathExists(zkClient), |     TestUtils.waitUntilTrue(() => !checkIfReassignPartitionPathExists(zkClient), | ||||||
|  | @ -298,7 +298,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { | ||||||
|     val partition = 1 |     val partition = 1 | ||||||
|     val preferredReplica = 0 |     val preferredReplica = 0 | ||||||
|     // create brokers |     // create brokers | ||||||
|     val serverConfigs = TestUtils.createBrokerConfigs(3, false).map(KafkaConfig.fromProps) |     val serverConfigs = TestUtils.createBrokerConfigs(3, zkConnect, false).map(KafkaConfig.fromProps) | ||||||
|     // create the topic |     // create the topic | ||||||
|     AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) |     AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) | ||||||
|     val servers = serverConfigs.reverse.map(s => TestUtils.createServer(s)) |     val servers = serverConfigs.reverse.map(s => TestUtils.createServer(s)) | ||||||
|  | @ -318,7 +318,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { | ||||||
|     val topic = "test" |     val topic = "test" | ||||||
|     val partition = 1 |     val partition = 1 | ||||||
|     // create brokers |     // create brokers | ||||||
|     val serverConfigs = TestUtils.createBrokerConfigs(3, false).map(KafkaConfig.fromProps) |     val serverConfigs = TestUtils.createBrokerConfigs(3, zkConnect, false).map(KafkaConfig.fromProps) | ||||||
|     val servers = serverConfigs.reverse.map(s => TestUtils.createServer(s)) |     val servers = serverConfigs.reverse.map(s => TestUtils.createServer(s)) | ||||||
|     // create the topic |     // create the topic | ||||||
|     TestUtils.createTopic(zkClient, topic, partitionReplicaAssignment = expectedReplicaAssignment, servers = servers) |     TestUtils.createTopic(zkClient, topic, partitionReplicaAssignment = expectedReplicaAssignment, servers = servers) | ||||||
|  | @ -365,7 +365,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { | ||||||
|   def testTopicConfigChange() { |   def testTopicConfigChange() { | ||||||
|     val partitions = 3 |     val partitions = 3 | ||||||
|     val topic = "my-topic" |     val topic = "my-topic" | ||||||
|     val server = TestUtils.createServer(KafkaConfig.fromProps(TestUtils.createBrokerConfig(0))) |     val server = TestUtils.createServer(KafkaConfig.fromProps(TestUtils.createBrokerConfig(0, zkConnect))) | ||||||
| 
 | 
 | ||||||
|     def makeConfig(messageSize: Int, retentionMs: Long) = { |     def makeConfig(messageSize: Int, retentionMs: Long) = { | ||||||
|       var props = new Properties() |       var props = new Properties() | ||||||
|  |  | ||||||
|  | @ -17,7 +17,7 @@ | ||||||
| package kafka.admin | package kafka.admin | ||||||
| 
 | 
 | ||||||
| import org.scalatest.junit.JUnit3Suite | import org.scalatest.junit.JUnit3Suite | ||||||
| import kafka.utils.{ZKGroupDirs, ZKGroupTopicDirs, ZkUtils, TestUtils} | import kafka.utils._ | ||||||
| import kafka.server.KafkaConfig | import kafka.server.KafkaConfig | ||||||
| import org.junit.Test | import org.junit.Test | ||||||
| import kafka.consumer._ | import kafka.consumer._ | ||||||
|  | @ -26,7 +26,7 @@ import kafka.integration.KafkaServerTestHarness | ||||||
| 
 | 
 | ||||||
| 
 | 
 | ||||||
| class DeleteConsumerGroupTest extends JUnit3Suite with KafkaServerTestHarness { | class DeleteConsumerGroupTest extends JUnit3Suite with KafkaServerTestHarness { | ||||||
|   val configs = TestUtils.createBrokerConfigs(3, false, true).map(KafkaConfig.fromProps) |   def generateConfigs() = TestUtils.createBrokerConfigs(3, zkConnect, false, true).map(KafkaConfig.fromProps) | ||||||
| 
 | 
 | ||||||
|   @Test |   @Test | ||||||
|   def testGroupWideDeleteInZK() { |   def testGroupWideDeleteInZK() { | ||||||
|  |  | ||||||
|  | @ -96,7 +96,7 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { | ||||||
|     val expectedReplicaAssignment = Map(0 -> List(0, 1, 2)) |     val expectedReplicaAssignment = Map(0 -> List(0, 1, 2)) | ||||||
|     val topic = "test" |     val topic = "test" | ||||||
|     val topicAndPartition = TopicAndPartition(topic, 0) |     val topicAndPartition = TopicAndPartition(topic, 0) | ||||||
|     val brokerConfigs = TestUtils.createBrokerConfigs(4, false) |     val brokerConfigs = TestUtils.createBrokerConfigs(4, zkConnect, false) | ||||||
|     brokerConfigs.foreach(p => p.setProperty("delete.topic.enable", "true")) |     brokerConfigs.foreach(p => p.setProperty("delete.topic.enable", "true")) | ||||||
|     // create brokers |     // create brokers | ||||||
|     val allServers = brokerConfigs.map(b => TestUtils.createServer(KafkaConfig.fromProps(b))) |     val allServers = brokerConfigs.map(b => TestUtils.createServer(KafkaConfig.fromProps(b))) | ||||||
|  | @ -224,7 +224,7 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { | ||||||
|     val topicAndPartition = TopicAndPartition(topicName, 0) |     val topicAndPartition = TopicAndPartition(topicName, 0) | ||||||
|     val topic = topicAndPartition.topic |     val topic = topicAndPartition.topic | ||||||
| 
 | 
 | ||||||
|     val brokerConfigs = TestUtils.createBrokerConfigs(3, false) |     val brokerConfigs = TestUtils.createBrokerConfigs(3, zkConnect, false) | ||||||
|     brokerConfigs(0).setProperty("delete.topic.enable", "true") |     brokerConfigs(0).setProperty("delete.topic.enable", "true") | ||||||
|     brokerConfigs(0).setProperty("log.cleaner.enable","true") |     brokerConfigs(0).setProperty("log.cleaner.enable","true") | ||||||
|     brokerConfigs(0).setProperty("log.cleanup.policy","compact") |     brokerConfigs(0).setProperty("log.cleanup.policy","compact") | ||||||
|  | @ -253,7 +253,7 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { | ||||||
| 
 | 
 | ||||||
|   private def createTestTopicAndCluster(topic: String): Seq[KafkaServer] = { |   private def createTestTopicAndCluster(topic: String): Seq[KafkaServer] = { | ||||||
| 
 | 
 | ||||||
|     val brokerConfigs = TestUtils.createBrokerConfigs(3, false) |     val brokerConfigs = TestUtils.createBrokerConfigs(3, zkConnect, false) | ||||||
|     brokerConfigs.foreach(p => p.setProperty("delete.topic.enable", "true") |     brokerConfigs.foreach(p => p.setProperty("delete.topic.enable", "true") | ||||||
|     ) |     ) | ||||||
|     createTestTopicAndCluster(topic,brokerConfigs) |     createTestTopicAndCluster(topic,brokerConfigs) | ||||||
|  |  | ||||||
|  | @ -30,7 +30,6 @@ import kafka.utils.TestUtils._ | ||||||
| import kafka.utils._ | import kafka.utils._ | ||||||
| import org.junit.Test | import org.junit.Test | ||||||
| import kafka.serializer._ | import kafka.serializer._ | ||||||
| import kafka.cluster.{Broker, Cluster} |  | ||||||
| import org.scalatest.junit.JUnit3Suite | import org.scalatest.junit.JUnit3Suite | ||||||
| import kafka.integration.KafkaServerTestHarness | import kafka.integration.KafkaServerTestHarness | ||||||
| 
 | 
 | ||||||
|  | @ -38,31 +37,27 @@ class ConsumerIteratorTest extends JUnit3Suite with KafkaServerTestHarness { | ||||||
| 
 | 
 | ||||||
|   val numNodes = 1 |   val numNodes = 1 | ||||||
| 
 | 
 | ||||||
|   val overridingProps = new Properties() |   def generateConfigs() = TestUtils.createBrokerConfigs(numNodes, zkConnect).map(KafkaConfig.fromProps) | ||||||
|   overridingProps.put(KafkaConfig.ZkConnectProp, TestZKUtils.zookeeperConnect) |  | ||||||
| 
 |  | ||||||
|   val configs = |  | ||||||
|     for(props <- TestUtils.createBrokerConfigs(numNodes)) |  | ||||||
|     yield KafkaConfig.fromProps(props, overridingProps) |  | ||||||
| 
 | 
 | ||||||
|   val messages = new mutable.HashMap[Int, Seq[Message]] |   val messages = new mutable.HashMap[Int, Seq[Message]] | ||||||
|   val topic = "topic" |   val topic = "topic" | ||||||
|   val group = "group1" |   val group = "group1" | ||||||
|   val consumer0 = "consumer0" |   val consumer0 = "consumer0" | ||||||
|   val consumedOffset = 5 |   val consumedOffset = 5 | ||||||
|   val cluster = new Cluster(configs.map(c => new Broker(c.brokerId, "localhost", c.port))) |  | ||||||
|   val queue = new LinkedBlockingQueue[FetchedDataChunk] |   val queue = new LinkedBlockingQueue[FetchedDataChunk] | ||||||
|   val topicInfos = configs.map(c => new PartitionTopicInfo(topic, |   var topicInfos: Seq[PartitionTopicInfo] = null | ||||||
|                                                            0, | 
 | ||||||
|                                                            queue, |   def consumerConfig = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer0)) | ||||||
|                                                            new AtomicLong(consumedOffset), |  | ||||||
|                                                            new AtomicLong(0), |  | ||||||
|                                                            new AtomicInteger(0), |  | ||||||
|                                                            "")) |  | ||||||
|   val consumerConfig = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer0)) |  | ||||||
| 
 | 
 | ||||||
|   override def setUp() { |   override def setUp() { | ||||||
|     super.setUp |     super.setUp() | ||||||
|  |     topicInfos = configs.map(c => new PartitionTopicInfo(topic, | ||||||
|  |       0, | ||||||
|  |       queue, | ||||||
|  |       new AtomicLong(consumedOffset), | ||||||
|  |       new AtomicLong(0), | ||||||
|  |       new AtomicInteger(0), | ||||||
|  |       "")) | ||||||
|     createTopic(zkClient, topic, partitionReplicaAssignment = Map(0 -> Seq(configs.head.brokerId)), servers = servers) |     createTopic(zkClient, topic, partitionReplicaAssignment = Map(0 -> Seq(configs.head.brokerId)), servers = servers) | ||||||
|   } |   } | ||||||
| 
 | 
 | ||||||
|  |  | ||||||
|  | @ -38,17 +38,14 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar | ||||||
| 
 | 
 | ||||||
|   val RebalanceBackoffMs = 5000 |   val RebalanceBackoffMs = 5000 | ||||||
|   var dirs : ZKGroupTopicDirs = null |   var dirs : ZKGroupTopicDirs = null | ||||||
|   val zookeeperConnect = TestZKUtils.zookeeperConnect |  | ||||||
|   val numNodes = 2 |   val numNodes = 2 | ||||||
|   val numParts = 2 |   val numParts = 2 | ||||||
|   val topic = "topic1" |   val topic = "topic1" | ||||||
|   val overridingProps = new Properties() |   val overridingProps = new Properties() | ||||||
|   overridingProps.put(KafkaConfig.ZkConnectProp, zookeeperConnect) |  | ||||||
|   overridingProps.put(KafkaConfig.NumPartitionsProp, numParts.toString) |   overridingProps.put(KafkaConfig.NumPartitionsProp, numParts.toString) | ||||||
| 
 | 
 | ||||||
|   val configs = |   override def generateConfigs() = TestUtils.createBrokerConfigs(numNodes, zkConnect) | ||||||
|     for (props <- TestUtils.createBrokerConfigs(numNodes)) |     .map(KafkaConfig.fromProps(_, overridingProps)) | ||||||
|     yield KafkaConfig.fromProps(props, overridingProps) |  | ||||||
| 
 | 
 | ||||||
|   val group = "group1" |   val group = "group1" | ||||||
|   val consumer0 = "consumer0" |   val consumer0 = "consumer0" | ||||||
|  | @ -93,8 +90,8 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar | ||||||
|     zkConsumerConnector0.shutdown |     zkConsumerConnector0.shutdown | ||||||
| 
 | 
 | ||||||
|     // send some messages to each broker |     // send some messages to each broker | ||||||
|     val sentMessages1 = sendMessagesToPartition(configs, topic, 0, nMessages) ++ |     val sentMessages1 = sendMessagesToPartition(servers, topic, 0, nMessages) ++ | ||||||
|                         sendMessagesToPartition(configs, topic, 1, nMessages) |                         sendMessagesToPartition(servers, topic, 1, nMessages) | ||||||
| 
 | 
 | ||||||
|     // wait to make sure the topic and partition have a leader for the successful case |     // wait to make sure the topic and partition have a leader for the successful case | ||||||
|     waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) |     waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) | ||||||
|  | @ -127,8 +124,8 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar | ||||||
|     val zkConsumerConnector2 = new ZookeeperConsumerConnector(consumerConfig2, true) |     val zkConsumerConnector2 = new ZookeeperConsumerConnector(consumerConfig2, true) | ||||||
|     val topicMessageStreams2 = zkConsumerConnector2.createMessageStreams(Map(topic -> 1), new StringDecoder(), new StringDecoder()) |     val topicMessageStreams2 = zkConsumerConnector2.createMessageStreams(Map(topic -> 1), new StringDecoder(), new StringDecoder()) | ||||||
|     // send some messages to each broker |     // send some messages to each broker | ||||||
|     val sentMessages2 = sendMessagesToPartition(configs, topic, 0, nMessages) ++ |     val sentMessages2 = sendMessagesToPartition(servers, topic, 0, nMessages) ++ | ||||||
|                         sendMessagesToPartition(configs, topic, 1, nMessages) |                         sendMessagesToPartition(servers, topic, 1, nMessages) | ||||||
| 
 | 
 | ||||||
|     waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) |     waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) | ||||||
|     waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1) |     waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1) | ||||||
|  | @ -148,8 +145,8 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar | ||||||
|     val zkConsumerConnector3 = new ZookeeperConsumerConnector(consumerConfig3, true) |     val zkConsumerConnector3 = new ZookeeperConsumerConnector(consumerConfig3, true) | ||||||
|     val topicMessageStreams3 = zkConsumerConnector3.createMessageStreams(new mutable.HashMap[String, Int]()) |     val topicMessageStreams3 = zkConsumerConnector3.createMessageStreams(new mutable.HashMap[String, Int]()) | ||||||
|     // send some messages to each broker |     // send some messages to each broker | ||||||
|     val sentMessages3 = sendMessagesToPartition(configs, topic, 0, nMessages) ++ |     val sentMessages3 = sendMessagesToPartition(servers, topic, 0, nMessages) ++ | ||||||
|                         sendMessagesToPartition(configs, topic, 1, nMessages) |                         sendMessagesToPartition(servers, topic, 1, nMessages) | ||||||
| 
 | 
 | ||||||
|     waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) |     waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) | ||||||
|     waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1) |     waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1) | ||||||
|  | @ -182,8 +179,8 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar | ||||||
|     requestHandlerLogger.setLevel(Level.FATAL) |     requestHandlerLogger.setLevel(Level.FATAL) | ||||||
| 
 | 
 | ||||||
|     // send some messages to each broker |     // send some messages to each broker | ||||||
|     val sentMessages1 = sendMessagesToPartition(configs, topic, 0, nMessages, GZIPCompressionCodec) ++ |     val sentMessages1 = sendMessagesToPartition(servers, topic, 0, nMessages, GZIPCompressionCodec) ++ | ||||||
|                         sendMessagesToPartition(configs, topic, 1, nMessages, GZIPCompressionCodec) |                         sendMessagesToPartition(servers, topic, 1, nMessages, GZIPCompressionCodec) | ||||||
| 
 | 
 | ||||||
|     waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) |     waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) | ||||||
|     waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1) |     waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1) | ||||||
|  | @ -215,8 +212,8 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar | ||||||
|     val zkConsumerConnector2 = new ZookeeperConsumerConnector(consumerConfig2, true) |     val zkConsumerConnector2 = new ZookeeperConsumerConnector(consumerConfig2, true) | ||||||
|     val topicMessageStreams2 = zkConsumerConnector2.createMessageStreams(Map(topic -> 1), new StringDecoder(), new StringDecoder()) |     val topicMessageStreams2 = zkConsumerConnector2.createMessageStreams(Map(topic -> 1), new StringDecoder(), new StringDecoder()) | ||||||
|     // send some messages to each broker |     // send some messages to each broker | ||||||
|     val sentMessages2 = sendMessagesToPartition(configs, topic, 0, nMessages, GZIPCompressionCodec) ++ |     val sentMessages2 = sendMessagesToPartition(servers, topic, 0, nMessages, GZIPCompressionCodec) ++ | ||||||
|                         sendMessagesToPartition(configs, topic, 1, nMessages, GZIPCompressionCodec) |                         sendMessagesToPartition(servers, topic, 1, nMessages, GZIPCompressionCodec) | ||||||
| 
 | 
 | ||||||
|     waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) |     waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) | ||||||
|     waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1) |     waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1) | ||||||
|  | @ -236,8 +233,8 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar | ||||||
|     val zkConsumerConnector3 = new ZookeeperConsumerConnector(consumerConfig3, true) |     val zkConsumerConnector3 = new ZookeeperConsumerConnector(consumerConfig3, true) | ||||||
|     val topicMessageStreams3 = zkConsumerConnector3.createMessageStreams(new mutable.HashMap[String, Int](), new StringDecoder(), new StringDecoder()) |     val topicMessageStreams3 = zkConsumerConnector3.createMessageStreams(new mutable.HashMap[String, Int](), new StringDecoder(), new StringDecoder()) | ||||||
|     // send some messages to each broker |     // send some messages to each broker | ||||||
|     val sentMessages3 = sendMessagesToPartition(configs, topic, 0, nMessages, GZIPCompressionCodec) ++ |     val sentMessages3 = sendMessagesToPartition(servers, topic, 0, nMessages, GZIPCompressionCodec) ++ | ||||||
|                         sendMessagesToPartition(configs, topic, 1, nMessages, GZIPCompressionCodec) |                         sendMessagesToPartition(servers, topic, 1, nMessages, GZIPCompressionCodec) | ||||||
| 
 | 
 | ||||||
|     waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) |     waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) | ||||||
|     waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1) |     waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1) | ||||||
|  | @ -258,8 +255,8 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar | ||||||
| 
 | 
 | ||||||
|   def testCompressionSetConsumption() { |   def testCompressionSetConsumption() { | ||||||
|     // send some messages to each broker |     // send some messages to each broker | ||||||
|     val sentMessages = sendMessagesToPartition(configs, topic, 0, 200, DefaultCompressionCodec) ++ |     val sentMessages = sendMessagesToPartition(servers, topic, 0, 200, DefaultCompressionCodec) ++ | ||||||
|                        sendMessagesToPartition(configs, topic, 1, 200, DefaultCompressionCodec) |                        sendMessagesToPartition(servers, topic, 1, 200, DefaultCompressionCodec) | ||||||
| 
 | 
 | ||||||
|     TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0) |     TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0) | ||||||
|     TestUtils.waitUntilMetadataIsPropagated(servers, topic, 1) |     TestUtils.waitUntilMetadataIsPropagated(servers, topic, 1) | ||||||
|  | @ -284,8 +281,8 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar | ||||||
|     requestHandlerLogger.setLevel(Level.FATAL) |     requestHandlerLogger.setLevel(Level.FATAL) | ||||||
| 
 | 
 | ||||||
|     // send some messages to each broker |     // send some messages to each broker | ||||||
|     val sentMessages = sendMessagesToPartition(configs, topic, 0, nMessages, NoCompressionCodec) ++ |     val sentMessages = sendMessagesToPartition(servers, topic, 0, nMessages, NoCompressionCodec) ++ | ||||||
|                        sendMessagesToPartition(configs, topic, 1, nMessages, NoCompressionCodec) |                        sendMessagesToPartition(servers, topic, 1, nMessages, NoCompressionCodec) | ||||||
| 
 | 
 | ||||||
|     TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0) |     TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0) | ||||||
|     TestUtils.waitUntilMetadataIsPropagated(servers, topic, 1) |     TestUtils.waitUntilMetadataIsPropagated(servers, topic, 1) | ||||||
|  | @ -319,13 +316,13 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar | ||||||
|   } |   } | ||||||
| 
 | 
 | ||||||
|   def testLeaderSelectionForPartition() { |   def testLeaderSelectionForPartition() { | ||||||
|     val zkClient = new ZkClient(zookeeperConnect, 6000, 30000, ZKStringSerializer) |     val zkClient = new ZkClient(zkConnect, 6000, 30000, ZKStringSerializer) | ||||||
| 
 | 
 | ||||||
|     // create topic topic1 with 1 partition on broker 0 |     // create topic topic1 with 1 partition on broker 0 | ||||||
|     createTopic(zkClient, topic, numPartitions = 1, replicationFactor = 1, servers = servers) |     createTopic(zkClient, topic, numPartitions = 1, replicationFactor = 1, servers = servers) | ||||||
| 
 | 
 | ||||||
|     // send some messages to each broker |     // send some messages to each broker | ||||||
|     val sentMessages1 = sendMessages(configs, topic, "producer1", nMessages, "batch1", NoCompressionCodec, 1) |     val sentMessages1 = sendMessages(servers, topic, "producer1", nMessages, "batch1", NoCompressionCodec, 1) | ||||||
| 
 | 
 | ||||||
|     // create a consumer |     // create a consumer | ||||||
|     val consumerConfig1 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer1)) |     val consumerConfig1 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer1)) | ||||||
|  | @ -351,8 +348,8 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar | ||||||
| 
 | 
 | ||||||
|   def testConsumerRebalanceListener() { |   def testConsumerRebalanceListener() { | ||||||
|     // Send messages to create topic |     // Send messages to create topic | ||||||
|     sendMessagesToPartition(configs, topic, 0, nMessages) |     sendMessagesToPartition(servers, topic, 0, nMessages) | ||||||
|     sendMessagesToPartition(configs, topic, 1, nMessages) |     sendMessagesToPartition(servers, topic, 1, nMessages) | ||||||
| 
 | 
 | ||||||
|     val consumerConfig1 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer1)) |     val consumerConfig1 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer1)) | ||||||
|     val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, true) |     val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, true) | ||||||
|  |  | ||||||
|  | @ -31,7 +31,7 @@ import junit.framework.Assert._ | ||||||
| 
 | 
 | ||||||
| class AutoOffsetResetTest extends JUnit3Suite with KafkaServerTestHarness with Logging { | class AutoOffsetResetTest extends JUnit3Suite with KafkaServerTestHarness with Logging { | ||||||
| 
 | 
 | ||||||
|   val configs = List(KafkaConfig.fromProps(TestUtils.createBrokerConfig(0))) |   def generateConfigs() = List(KafkaConfig.fromProps(TestUtils.createBrokerConfig(0, zkConnect))) | ||||||
| 
 | 
 | ||||||
|   val topic = "test_topic" |   val topic = "test_topic" | ||||||
|   val group = "default_group" |   val group = "default_group" | ||||||
|  | @ -78,7 +78,7 @@ class AutoOffsetResetTest extends JUnit3Suite with KafkaServerTestHarness with L | ||||||
|     TestUtils.createTopic(zkClient, topic, 1, 1, servers) |     TestUtils.createTopic(zkClient, topic, 1, 1, servers) | ||||||
| 
 | 
 | ||||||
|     val producer: Producer[String, Array[Byte]] = TestUtils.createProducer( |     val producer: Producer[String, Array[Byte]] = TestUtils.createProducer( | ||||||
|       TestUtils.getBrokerListStrFromConfigs(configs), |       TestUtils.getBrokerListStrFromServers(servers), | ||||||
|       keyEncoder = classOf[StringEncoder].getName) |       keyEncoder = classOf[StringEncoder].getName) | ||||||
| 
 | 
 | ||||||
|     for(i <- 0 until numMessages) |     for(i <- 0 until numMessages) | ||||||
|  |  | ||||||
|  | @ -32,23 +32,13 @@ import kafka.utils.TestUtils._ | ||||||
| import kafka.utils.TestUtils | import kafka.utils.TestUtils | ||||||
| 
 | 
 | ||||||
| class FetcherTest extends JUnit3Suite with KafkaServerTestHarness { | class FetcherTest extends JUnit3Suite with KafkaServerTestHarness { | ||||||
| 
 |  | ||||||
|   val numNodes = 1 |   val numNodes = 1 | ||||||
|   val configs = |   def generateConfigs() = TestUtils.createBrokerConfigs(numNodes, zkConnect).map(KafkaConfig.fromProps) | ||||||
|     for(props <- TestUtils.createBrokerConfigs(numNodes)) | 
 | ||||||
|     yield KafkaConfig.fromProps(props) |  | ||||||
|   val messages = new mutable.HashMap[Int, Seq[Array[Byte]]] |   val messages = new mutable.HashMap[Int, Seq[Array[Byte]]] | ||||||
|   val topic = "topic" |   val topic = "topic" | ||||||
|   val cluster = new Cluster(configs.map(c => new Broker(c.brokerId, "localhost", c.port))) | 
 | ||||||
|   val shutdown = ZookeeperConsumerConnector.shutdownCommand |  | ||||||
|   val queue = new LinkedBlockingQueue[FetchedDataChunk] |   val queue = new LinkedBlockingQueue[FetchedDataChunk] | ||||||
|   val topicInfos = configs.map(c => new PartitionTopicInfo(topic, |  | ||||||
|                                                            0, |  | ||||||
|                                                            queue, |  | ||||||
|                                                            new AtomicLong(0), |  | ||||||
|                                                            new AtomicLong(0), |  | ||||||
|                                                            new AtomicInteger(0), |  | ||||||
|                                                            "")) |  | ||||||
| 
 | 
 | ||||||
|   var fetcher: ConsumerFetcherManager = null |   var fetcher: ConsumerFetcherManager = null | ||||||
| 
 | 
 | ||||||
|  | @ -56,8 +46,18 @@ class FetcherTest extends JUnit3Suite with KafkaServerTestHarness { | ||||||
|     super.setUp |     super.setUp | ||||||
|     createTopic(zkClient, topic, partitionReplicaAssignment = Map(0 -> Seq(configs.head.brokerId)), servers = servers) |     createTopic(zkClient, topic, partitionReplicaAssignment = Map(0 -> Seq(configs.head.brokerId)), servers = servers) | ||||||
| 
 | 
 | ||||||
|  |     val cluster = new Cluster(servers.map(s => new Broker(s.config.brokerId, "localhost", s.boundPort()))) | ||||||
|  | 
 | ||||||
|     fetcher = new ConsumerFetcherManager("consumer1", new ConsumerConfig(TestUtils.createConsumerProperties("", "", "")), zkClient) |     fetcher = new ConsumerFetcherManager("consumer1", new ConsumerConfig(TestUtils.createConsumerProperties("", "", "")), zkClient) | ||||||
|     fetcher.stopConnections() |     fetcher.stopConnections() | ||||||
|  |     val topicInfos = configs.map(c => | ||||||
|  |       new PartitionTopicInfo(topic, | ||||||
|  |         0, | ||||||
|  |         queue, | ||||||
|  |         new AtomicLong(0), | ||||||
|  |         new AtomicLong(0), | ||||||
|  |         new AtomicInteger(0), | ||||||
|  |         "")) | ||||||
|     fetcher.startConnections(topicInfos, cluster) |     fetcher.startConnections(topicInfos, cluster) | ||||||
|   } |   } | ||||||
| 
 | 
 | ||||||
|  | @ -83,7 +83,7 @@ class FetcherTest extends JUnit3Suite with KafkaServerTestHarness { | ||||||
|     var count = 0 |     var count = 0 | ||||||
|     for(conf <- configs) { |     for(conf <- configs) { | ||||||
|       val producer: Producer[String, Array[Byte]] = TestUtils.createProducer( |       val producer: Producer[String, Array[Byte]] = TestUtils.createProducer( | ||||||
|         TestUtils.getBrokerListStrFromConfigs(configs), |         TestUtils.getBrokerListStrFromServers(servers), | ||||||
|         keyEncoder = classOf[StringEncoder].getName) |         keyEncoder = classOf[StringEncoder].getName) | ||||||
|       val ms = 0.until(messagesPerNode).map(x => (conf.brokerId * 5 + x).toString.getBytes).toArray |       val ms = 0.until(messagesPerNode).map(x => (conf.brokerId * 5 + x).toString.getBytes).toArray | ||||||
|       messages += conf.brokerId -> ms |       messages += conf.brokerId -> ms | ||||||
|  |  | ||||||
|  | @ -24,28 +24,38 @@ import kafka.utils.{Utils, TestUtils} | ||||||
| import org.scalatest.junit.JUnit3Suite | import org.scalatest.junit.JUnit3Suite | ||||||
| import kafka.zk.ZooKeeperTestHarness | import kafka.zk.ZooKeeperTestHarness | ||||||
| import kafka.common.KafkaException | import kafka.common.KafkaException | ||||||
| import kafka.utils.TestUtils |  | ||||||
| 
 | 
 | ||||||
| /** | /** | ||||||
|  * A test harness that brings up some number of broker nodes |  * A test harness that brings up some number of broker nodes | ||||||
|  */ |  */ | ||||||
| trait KafkaServerTestHarness extends JUnit3Suite with ZooKeeperTestHarness { | trait KafkaServerTestHarness extends JUnit3Suite with ZooKeeperTestHarness { | ||||||
| 
 |   var instanceConfigs: Seq[KafkaConfig] = null | ||||||
|   val configs: List[KafkaConfig] |  | ||||||
|   var servers: Buffer[KafkaServer] = null |   var servers: Buffer[KafkaServer] = null | ||||||
|   var brokerList: String = null |   var brokerList: String = null | ||||||
|   var alive: Array[Boolean] = null |   var alive: Array[Boolean] = null | ||||||
|    | 
 | ||||||
|  |   /** | ||||||
|  |    * Implementations must override this method to return a set of KafkaConfigs. This method will be invoked for every | ||||||
|  |    * test and should not reuse previous configurations unless they select their ports randomly when servers are started. | ||||||
|  |    */ | ||||||
|  |   def generateConfigs(): Seq[KafkaConfig] | ||||||
|  | 
 | ||||||
|  |   def configs: Seq[KafkaConfig] = { | ||||||
|  |     if (instanceConfigs == null) | ||||||
|  |       instanceConfigs = generateConfigs() | ||||||
|  |     instanceConfigs | ||||||
|  |   } | ||||||
|  | 
 | ||||||
|   def serverForId(id: Int) = servers.find(s => s.config.brokerId == id) |   def serverForId(id: Int) = servers.find(s => s.config.brokerId == id) | ||||||
|    |    | ||||||
|   def bootstrapUrl = configs.map(c => c.hostName + ":" + c.port).mkString(",") |   def bootstrapUrl = servers.map(s => s.config.hostName + ":" + s.boundPort()).mkString(",") | ||||||
|    |    | ||||||
|   override def setUp() { |   override def setUp() { | ||||||
|     super.setUp |     super.setUp | ||||||
|     if(configs.size <= 0) |     if(configs.size <= 0) | ||||||
|       throw new KafkaException("Must suply at least one server config.") |       throw new KafkaException("Must supply at least one server config.") | ||||||
|     brokerList = TestUtils.getBrokerListStrFromConfigs(configs) |  | ||||||
|     servers = configs.map(TestUtils.createServer(_)).toBuffer |     servers = configs.map(TestUtils.createServer(_)).toBuffer | ||||||
|  |     brokerList = TestUtils.getBrokerListStrFromServers(servers) | ||||||
|     alive = new Array[Boolean](servers.length) |     alive = new Array[Boolean](servers.length) | ||||||
|     Arrays.fill(alive, true) |     Arrays.fill(alive, true) | ||||||
|   } |   } | ||||||
|  |  | ||||||
|  | @ -23,16 +23,13 @@ import kafka.api.{PartitionFetchInfo, FetchRequest, FetchRequestBuilder} | ||||||
| import kafka.server.{KafkaRequestHandler, KafkaConfig} | import kafka.server.{KafkaRequestHandler, KafkaConfig} | ||||||
| import kafka.producer.{KeyedMessage, Producer} | import kafka.producer.{KeyedMessage, Producer} | ||||||
| import org.apache.log4j.{Level, Logger} | import org.apache.log4j.{Level, Logger} | ||||||
| import org.I0Itec.zkclient.ZkClient |  | ||||||
| import kafka.zk.ZooKeeperTestHarness | import kafka.zk.ZooKeeperTestHarness | ||||||
| import org.scalatest.junit.JUnit3Suite | import org.scalatest.junit.JUnit3Suite | ||||||
| import scala.collection._ | import scala.collection._ | ||||||
| import kafka.admin.AdminUtils |  | ||||||
| import kafka.common.{TopicAndPartition, ErrorMapping, UnknownTopicOrPartitionException, OffsetOutOfRangeException} | import kafka.common.{TopicAndPartition, ErrorMapping, UnknownTopicOrPartitionException, OffsetOutOfRangeException} | ||||||
| import kafka.utils.{StaticPartitioner, TestUtils, Utils} | import kafka.utils.{StaticPartitioner, TestUtils, Utils} | ||||||
| import kafka.serializer.StringEncoder | import kafka.serializer.StringEncoder | ||||||
| import java.util.Properties | import java.util.Properties | ||||||
| import TestUtils._ |  | ||||||
| 
 | 
 | ||||||
| /** | /** | ||||||
|  * End to end tests of the primitive apis against a local server |  * End to end tests of the primitive apis against a local server | ||||||
|  | @ -40,10 +37,7 @@ import TestUtils._ | ||||||
| class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with ZooKeeperTestHarness { | class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with ZooKeeperTestHarness { | ||||||
|   val requestHandlerLogger = Logger.getLogger(classOf[KafkaRequestHandler]) |   val requestHandlerLogger = Logger.getLogger(classOf[KafkaRequestHandler]) | ||||||
| 
 | 
 | ||||||
|   val port = TestUtils.choosePort() |   def generateConfigs() = List(KafkaConfig.fromProps(TestUtils.createBrokerConfig(0, zkConnect))) | ||||||
|   val props = TestUtils.createBrokerConfig(0, port) |  | ||||||
|   val config = KafkaConfig.fromProps(props) |  | ||||||
|   val configs = List(config) |  | ||||||
| 
 | 
 | ||||||
|   def testFetchRequestCanProperlySerialize() { |   def testFetchRequestCanProperlySerialize() { | ||||||
|     val request = new FetchRequestBuilder() |     val request = new FetchRequestBuilder() | ||||||
|  | @ -97,7 +91,7 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with | ||||||
|     props.put("compression.codec", "gzip") |     props.put("compression.codec", "gzip") | ||||||
| 
 | 
 | ||||||
|     val stringProducer1 = TestUtils.createProducer[String, String]( |     val stringProducer1 = TestUtils.createProducer[String, String]( | ||||||
|       TestUtils.getBrokerListStrFromConfigs(configs), |       TestUtils.getBrokerListStrFromServers(servers), | ||||||
|       encoder = classOf[StringEncoder].getName, |       encoder = classOf[StringEncoder].getName, | ||||||
|       keyEncoder = classOf[StringEncoder].getName, |       keyEncoder = classOf[StringEncoder].getName, | ||||||
|       partitioner = classOf[StaticPartitioner].getName, |       partitioner = classOf[StaticPartitioner].getName, | ||||||
|  | @ -222,7 +216,7 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with | ||||||
|     props.put("request.required.acks", "0") |     props.put("request.required.acks", "0") | ||||||
|     val pipelinedProducer: Producer[String, String] = |     val pipelinedProducer: Producer[String, String] = | ||||||
|       TestUtils.createProducer[String, String]( |       TestUtils.createProducer[String, String]( | ||||||
|         TestUtils.getBrokerListStrFromConfigs(configs), |         TestUtils.getBrokerListStrFromServers(servers), | ||||||
|         encoder = classOf[StringEncoder].getName, |         encoder = classOf[StringEncoder].getName, | ||||||
|         keyEncoder = classOf[StringEncoder].getName, |         keyEncoder = classOf[StringEncoder].getName, | ||||||
|         partitioner = classOf[StaticPartitioner].getName, |         partitioner = classOf[StaticPartitioner].getName, | ||||||
|  |  | ||||||
|  | @ -24,18 +24,17 @@ import kafka.utils.{StaticPartitioner, TestUtils} | ||||||
| import kafka.serializer.StringEncoder | import kafka.serializer.StringEncoder | ||||||
| 
 | 
 | ||||||
| trait ProducerConsumerTestHarness extends JUnit3Suite with KafkaServerTestHarness { | trait ProducerConsumerTestHarness extends JUnit3Suite with KafkaServerTestHarness { | ||||||
|     val port: Int |  | ||||||
|     val host = "localhost" |     val host = "localhost" | ||||||
|     var producer: Producer[String, String] = null |     var producer: Producer[String, String] = null | ||||||
|     var consumer: SimpleConsumer = null |     var consumer: SimpleConsumer = null | ||||||
| 
 | 
 | ||||||
|   override def setUp() { |   override def setUp() { | ||||||
|       super.setUp |       super.setUp | ||||||
|       producer = TestUtils.createProducer[String, String](TestUtils.getBrokerListStrFromConfigs(configs), |       producer = TestUtils.createProducer[String, String](TestUtils.getBrokerListStrFromServers(servers), | ||||||
|         encoder = classOf[StringEncoder].getName, |         encoder = classOf[StringEncoder].getName, | ||||||
|         keyEncoder = classOf[StringEncoder].getName, |         keyEncoder = classOf[StringEncoder].getName, | ||||||
|         partitioner = classOf[StaticPartitioner].getName) |         partitioner = classOf[StaticPartitioner].getName) | ||||||
|       consumer = new SimpleConsumer(host, port, 1000000, 64*1024, "") |       consumer = new SimpleConsumer(host, servers(0).boundPort(), 1000000, 64*1024, "") | ||||||
|     } |     } | ||||||
| 
 | 
 | ||||||
|    override def tearDown() { |    override def tearDown() { | ||||||
|  |  | ||||||
|  | @ -25,35 +25,18 @@ import kafka.utils.{Utils, TestUtils} | ||||||
| import kafka.server.{KafkaConfig, KafkaServer} | import kafka.server.{KafkaConfig, KafkaServer} | ||||||
| 
 | 
 | ||||||
| class RollingBounceTest extends JUnit3Suite with ZooKeeperTestHarness { | class RollingBounceTest extends JUnit3Suite with ZooKeeperTestHarness { | ||||||
|   val brokerId1 = 0 |  | ||||||
|   val brokerId2 = 1 |  | ||||||
|   val brokerId3 = 2 |  | ||||||
|   val brokerId4 = 3 |  | ||||||
| 
 |  | ||||||
|   val port1 = TestUtils.choosePort() |  | ||||||
|   val port2 = TestUtils.choosePort() |  | ||||||
|   val port3 = TestUtils.choosePort() |  | ||||||
|   val port4 = TestUtils.choosePort() |  | ||||||
| 
 |  | ||||||
|   // controlled.shutdown.enable is true by default |  | ||||||
|   val configProps1 = TestUtils.createBrokerConfig(brokerId1, port1) |  | ||||||
|   val configProps2 = TestUtils.createBrokerConfig(brokerId2, port2) |  | ||||||
|   val configProps3 = TestUtils.createBrokerConfig(brokerId3, port3) |  | ||||||
|   val configProps4 = TestUtils.createBrokerConfig(brokerId4, port4) |  | ||||||
| 
 |  | ||||||
|   var servers: Seq[KafkaServer] = Seq.empty[KafkaServer] |  | ||||||
| 
 | 
 | ||||||
|   val partitionId = 0 |   val partitionId = 0 | ||||||
|  |   var servers: Seq[KafkaServer] = null | ||||||
| 
 | 
 | ||||||
|   override def setUp() { |   override def setUp() { | ||||||
|     super.setUp() |     super.setUp() | ||||||
|  |     // controlled.shutdown.enable is true by default | ||||||
|  |     val configs = (0 until 4).map(i => TestUtils.createBrokerConfig(i, zkConnect)) | ||||||
|  |     configs(3).put("controlled.shutdown.retry.backoff.ms", "100") | ||||||
|  |   | ||||||
|     // start all the servers |     // start all the servers | ||||||
|     val server1 = TestUtils.createServer(KafkaConfig.fromProps(configProps1)) |     servers = configs.map(c => TestUtils.createServer(KafkaConfig.fromProps(c))) | ||||||
|     val server2 = TestUtils.createServer(KafkaConfig.fromProps(configProps2)) |  | ||||||
|     val server3 = TestUtils.createServer(KafkaConfig.fromProps(configProps3)) |  | ||||||
|     val server4 = TestUtils.createServer(KafkaConfig.fromProps(configProps4)) |  | ||||||
| 
 |  | ||||||
|     servers ++= List(server1, server2, server3, server4) |  | ||||||
|   } |   } | ||||||
| 
 | 
 | ||||||
|   override def tearDown() { |   override def tearDown() { | ||||||
|  |  | ||||||
|  | @ -31,14 +31,15 @@ import kafka.common.ErrorMapping | ||||||
| import kafka.client.ClientUtils | import kafka.client.ClientUtils | ||||||
| 
 | 
 | ||||||
| class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { | class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { | ||||||
|   val props = createBrokerConfigs(1) |  | ||||||
|   val configs = props.map(p => KafkaConfig.fromProps(p)) |  | ||||||
|   private var server1: KafkaServer = null |   private var server1: KafkaServer = null | ||||||
|   val brokers = configs.map(c => new Broker(c.brokerId,c.hostName,c.port)) |   var brokers: Seq[Broker] = null | ||||||
| 
 | 
 | ||||||
|   override def setUp() { |   override def setUp() { | ||||||
|     super.setUp() |     super.setUp() | ||||||
|  |     val props = createBrokerConfigs(1, zkConnect) | ||||||
|  |     val configs = props.map(KafkaConfig.fromProps) | ||||||
|     server1 = TestUtils.createServer(configs.head) |     server1 = TestUtils.createServer(configs.head) | ||||||
|  |     brokers = Seq(new Broker(server1.config.brokerId, server1.config.hostName, server1.boundPort())) | ||||||
|   } |   } | ||||||
| 
 | 
 | ||||||
|   override def tearDown() { |   override def tearDown() { | ||||||
|  |  | ||||||
|  | @ -29,7 +29,7 @@ import kafka.admin.AdminUtils | ||||||
| import kafka.common.FailedToSendMessageException | import kafka.common.FailedToSendMessageException | ||||||
| import kafka.consumer.{Consumer, ConsumerConfig, ConsumerTimeoutException} | import kafka.consumer.{Consumer, ConsumerConfig, ConsumerTimeoutException} | ||||||
| import kafka.producer.{KeyedMessage, Producer} | import kafka.producer.{KeyedMessage, Producer} | ||||||
| import kafka.serializer.{DefaultEncoder, StringEncoder} | import kafka.serializer.StringEncoder | ||||||
| import kafka.server.{KafkaConfig, KafkaServer} | import kafka.server.{KafkaConfig, KafkaServer} | ||||||
| import kafka.utils.Utils | import kafka.utils.Utils | ||||||
| import kafka.utils.TestUtils._ | import kafka.utils.TestUtils._ | ||||||
|  | @ -39,20 +39,12 @@ class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { | ||||||
|   val brokerId1 = 0 |   val brokerId1 = 0 | ||||||
|   val brokerId2 = 1 |   val brokerId2 = 1 | ||||||
| 
 | 
 | ||||||
|   val port1 = choosePort() |  | ||||||
|   val port2 = choosePort() |  | ||||||
| 
 |  | ||||||
|   // controlled shutdown is needed for these tests, but we can trim the retry count and backoff interval to |   // controlled shutdown is needed for these tests, but we can trim the retry count and backoff interval to | ||||||
|   // reduce test execution time |   // reduce test execution time | ||||||
|   val enableControlledShutdown = true |   val enableControlledShutdown = true | ||||||
|   val configProps1 = createBrokerConfig(brokerId1, port1) |  | ||||||
|   val configProps2 = createBrokerConfig(brokerId2, port2) |  | ||||||
| 
 | 
 | ||||||
|   for (configProps <- List(configProps1, configProps2)) { |   var configProps1: Properties = null | ||||||
|     configProps.put("controlled.shutdown.enable", String.valueOf(enableControlledShutdown)) |   var configProps2: Properties = null | ||||||
|     configProps.put("controlled.shutdown.max.retries", String.valueOf(1)) |  | ||||||
|     configProps.put("controlled.shutdown.retry.backoff.ms", String.valueOf(1000)) |  | ||||||
|   } |  | ||||||
| 
 | 
 | ||||||
|   var configs: Seq[KafkaConfig] = Seq.empty[KafkaConfig] |   var configs: Seq[KafkaConfig] = Seq.empty[KafkaConfig] | ||||||
|   var servers: Seq[KafkaServer] = Seq.empty[KafkaServer] |   var servers: Seq[KafkaServer] = Seq.empty[KafkaServer] | ||||||
|  | @ -69,6 +61,15 @@ class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { | ||||||
|   override def setUp() { |   override def setUp() { | ||||||
|     super.setUp() |     super.setUp() | ||||||
| 
 | 
 | ||||||
|  |     configProps1 = createBrokerConfig(brokerId1, zkConnect) | ||||||
|  |     configProps2 = createBrokerConfig(brokerId2, zkConnect) | ||||||
|  | 
 | ||||||
|  |     for (configProps <- List(configProps1, configProps2)) { | ||||||
|  |       configProps.put("controlled.shutdown.enable", String.valueOf(enableControlledShutdown)) | ||||||
|  |       configProps.put("controlled.shutdown.max.retries", String.valueOf(1)) | ||||||
|  |       configProps.put("controlled.shutdown.retry.backoff.ms", String.valueOf(1000)) | ||||||
|  |     } | ||||||
|  | 
 | ||||||
|     // temporarily set loggers to a higher level so that tests run quietly |     // temporarily set loggers to a higher level so that tests run quietly | ||||||
|     kafkaApisLogger.setLevel(Level.FATAL) |     kafkaApisLogger.setLevel(Level.FATAL) | ||||||
|     networkProcessorLogger.setLevel(Level.FATAL) |     networkProcessorLogger.setLevel(Level.FATAL) | ||||||
|  | @ -254,7 +255,7 @@ class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { | ||||||
| 
 | 
 | ||||||
|   private def produceMessage(topic: String, message: String) = { |   private def produceMessage(topic: String, message: String) = { | ||||||
|     val producer: Producer[String, Array[Byte]] = createProducer( |     val producer: Producer[String, Array[Byte]] = createProducer( | ||||||
|       getBrokerListStrFromConfigs(configs), |       getBrokerListStrFromServers(servers), | ||||||
|       keyEncoder = classOf[StringEncoder].getName) |       keyEncoder = classOf[StringEncoder].getName) | ||||||
|     producer.send(new KeyedMessage[String, Array[Byte]](topic, topic, message.getBytes)) |     producer.send(new KeyedMessage[String, Array[Byte]](topic, topic, message.getBytes)) | ||||||
|     producer.close() |     producer.close() | ||||||
|  |  | ||||||
|  | @ -39,19 +39,14 @@ import junit.framework.Assert._ | ||||||
| 
 | 
 | ||||||
| 
 | 
 | ||||||
| class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHarness with ZooKeeperTestHarness with Logging { | class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHarness with ZooKeeperTestHarness with Logging { | ||||||
| 
 |  | ||||||
|   val zookeeperConnect = zkConnect |  | ||||||
|   val numNodes = 2 |   val numNodes = 2 | ||||||
|   val numParts = 2 |   val numParts = 2 | ||||||
|   val topic = "topic1" |   val topic = "topic1" | ||||||
| 
 | 
 | ||||||
|   val overridingProps = new Properties() |   val overridingProps = new Properties() | ||||||
|   overridingProps.put(KafkaConfig.ZkConnectProp, zookeeperConnect) |  | ||||||
|   overridingProps.put(KafkaConfig.NumPartitionsProp, numParts.toString) |   overridingProps.put(KafkaConfig.NumPartitionsProp, numParts.toString) | ||||||
| 
 | 
 | ||||||
|   val configs = |   def generateConfigs() = TestUtils.createBrokerConfigs(numNodes, zkConnect).map(KafkaConfig.fromProps(_, overridingProps)) | ||||||
|     for (props <- TestUtils.createBrokerConfigs(numNodes)) |  | ||||||
|     yield KafkaConfig.fromProps(props, overridingProps) |  | ||||||
| 
 | 
 | ||||||
|   val group = "group1" |   val group = "group1" | ||||||
|   val consumer1 = "consumer1" |   val consumer1 = "consumer1" | ||||||
|  | @ -68,7 +63,7 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar | ||||||
|     val sentMessages1 = sendMessages(nMessages, "batch1") |     val sentMessages1 = sendMessages(nMessages, "batch1") | ||||||
| 
 | 
 | ||||||
|     // create a consumer |     // create a consumer | ||||||
|     val consumerConfig1 = new ConsumerConfig(TestUtils.createConsumerProperties(zookeeperConnect, group, consumer1)) |     val consumerConfig1 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer1)) | ||||||
|     val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, true) |     val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, true) | ||||||
|     val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(toJavaMap(Map(topic -> numNodes*numParts/2)), new StringDecoder(), new StringDecoder()) |     val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(toJavaMap(Map(topic -> numNodes*numParts/2)), new StringDecoder(), new StringDecoder()) | ||||||
| 
 | 
 | ||||||
|  | @ -93,7 +88,7 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar | ||||||
|                    compressed: CompressionCodec): List[String] = { |                    compressed: CompressionCodec): List[String] = { | ||||||
|     var messages: List[String] = Nil |     var messages: List[String] = Nil | ||||||
|     val producer: kafka.producer.Producer[Int, String] = |     val producer: kafka.producer.Producer[Int, String] = | ||||||
|       TestUtils.createProducer(TestUtils.getBrokerListStrFromConfigs(configs), |       TestUtils.createProducer(TestUtils.getBrokerListStrFromServers(servers), | ||||||
|         encoder = classOf[StringEncoder].getName, |         encoder = classOf[StringEncoder].getName, | ||||||
|         keyEncoder = classOf[IntEncoder].getName) |         keyEncoder = classOf[IntEncoder].getName) | ||||||
|     val javaProducer: Producer[Int, String] = new kafka.javaapi.producer.Producer(producer) |     val javaProducer: Producer[Int, String] = new kafka.javaapi.producer.Producer(producer) | ||||||
|  |  | ||||||
|  | @ -37,7 +37,7 @@ class LogTest extends JUnitSuite { | ||||||
|   @Before |   @Before | ||||||
|   def setUp() { |   def setUp() { | ||||||
|     logDir = TestUtils.tempDir() |     logDir = TestUtils.tempDir() | ||||||
|     val props = TestUtils.createBrokerConfig(0, -1) |     val props = TestUtils.createBrokerConfig(0, "127.0.0.1:1", port = -1) | ||||||
|     config = KafkaConfig.fromProps(props) |     config = KafkaConfig.fromProps(props) | ||||||
|   } |   } | ||||||
| 
 | 
 | ||||||
|  |  | ||||||
|  | @ -47,19 +47,16 @@ class KafkaLog4jAppenderTest extends JUnit3Suite with ZooKeeperTestHarness with | ||||||
| 
 | 
 | ||||||
|   private val brokerZk = 0 |   private val brokerZk = 0 | ||||||
| 
 | 
 | ||||||
|   private val ports = TestUtils.choosePorts(2) |  | ||||||
|   private val portZk = ports(0) |  | ||||||
| 
 |  | ||||||
|   @Before |   @Before | ||||||
|   override def setUp() { |   override def setUp() { | ||||||
|     super.setUp() |     super.setUp() | ||||||
| 
 | 
 | ||||||
|     val propsZk = TestUtils.createBrokerConfig(brokerZk, portZk) |     val propsZk = TestUtils.createBrokerConfig(brokerZk, zkConnect) | ||||||
|     val logDirZkPath = propsZk.getProperty("log.dir") |     val logDirZkPath = propsZk.getProperty("log.dir") | ||||||
|     logDirZk = new File(logDirZkPath) |     logDirZk = new File(logDirZkPath) | ||||||
|     config = KafkaConfig.fromProps(propsZk) |     config = KafkaConfig.fromProps(propsZk) | ||||||
|     server = TestUtils.createServer(config) |     server = TestUtils.createServer(config) | ||||||
|     simpleConsumerZk = new SimpleConsumer("localhost", portZk, 1000000, 64 * 1024, "") |     simpleConsumerZk = new SimpleConsumer("localhost", server.boundPort(), 1000000, 64 * 1024, "") | ||||||
|   } |   } | ||||||
| 
 | 
 | ||||||
|   @After |   @After | ||||||
|  | @ -94,7 +91,7 @@ class KafkaLog4jAppenderTest extends JUnit3Suite with ZooKeeperTestHarness with | ||||||
|     props.put("log4j.appender.KAFKA", "kafka.producer.KafkaLog4jAppender") |     props.put("log4j.appender.KAFKA", "kafka.producer.KafkaLog4jAppender") | ||||||
|     props.put("log4j.appender.KAFKA.layout", "org.apache.log4j.PatternLayout") |     props.put("log4j.appender.KAFKA.layout", "org.apache.log4j.PatternLayout") | ||||||
|     props.put("log4j.appender.KAFKA.layout.ConversionPattern", "%-5p: %c - %m%n") |     props.put("log4j.appender.KAFKA.layout.ConversionPattern", "%-5p: %c - %m%n") | ||||||
|     props.put("log4j.appender.KAFKA.brokerList", TestUtils.getBrokerListStrFromConfigs(Seq(config))) |     props.put("log4j.appender.KAFKA.brokerList", TestUtils.getBrokerListStrFromServers(Seq(server))) | ||||||
|     props.put("log4j.logger.kafka.log4j", "INFO, KAFKA") |     props.put("log4j.logger.kafka.log4j", "INFO, KAFKA") | ||||||
| 
 | 
 | ||||||
|     try { |     try { | ||||||
|  | @ -129,7 +126,7 @@ class KafkaLog4jAppenderTest extends JUnit3Suite with ZooKeeperTestHarness with | ||||||
|     props.put("log4j.appender.KAFKA", "kafka.producer.KafkaLog4jAppender") |     props.put("log4j.appender.KAFKA", "kafka.producer.KafkaLog4jAppender") | ||||||
|     props.put("log4j.appender.KAFKA.layout", "org.apache.log4j.PatternLayout") |     props.put("log4j.appender.KAFKA.layout", "org.apache.log4j.PatternLayout") | ||||||
|     props.put("log4j.appender.KAFKA.layout.ConversionPattern", "%-5p: %c - %m%n") |     props.put("log4j.appender.KAFKA.layout.ConversionPattern", "%-5p: %c - %m%n") | ||||||
|     props.put("log4j.appender.KAFKA.BrokerList", TestUtils.getBrokerListStrFromConfigs(Seq(config))) |     props.put("log4j.appender.KAFKA.BrokerList", TestUtils.getBrokerListStrFromServers(Seq(server))) | ||||||
|     props.put("log4j.appender.KAFKA.Topic", "test-topic") |     props.put("log4j.appender.KAFKA.Topic", "test-topic") | ||||||
|     props.put("log4j.appender.KAFKA.RequiredNumAcks", "1") |     props.put("log4j.appender.KAFKA.RequiredNumAcks", "1") | ||||||
|     props.put("log4j.appender.KAFKA.SyncSend", "true") |     props.put("log4j.appender.KAFKA.SyncSend", "true") | ||||||
|  |  | ||||||
|  | @ -36,18 +36,15 @@ import scala.util.matching.Regex | ||||||
| import org.scalatest.junit.JUnit3Suite | import org.scalatest.junit.JUnit3Suite | ||||||
| 
 | 
 | ||||||
| class MetricsTest extends JUnit3Suite with KafkaServerTestHarness with Logging { | class MetricsTest extends JUnit3Suite with KafkaServerTestHarness with Logging { | ||||||
|   val zookeeperConnect = TestZKUtils.zookeeperConnect |  | ||||||
|   val numNodes = 2 |   val numNodes = 2 | ||||||
|   val numParts = 2 |   val numParts = 2 | ||||||
|   val topic = "topic1" |   val topic = "topic1" | ||||||
| 
 | 
 | ||||||
|   val overridingProps = new Properties() |   val overridingProps = new Properties() | ||||||
|   overridingProps.put(KafkaConfig.ZkConnectProp, zookeeperConnect) |  | ||||||
|   overridingProps.put(KafkaConfig.NumPartitionsProp, numParts.toString) |   overridingProps.put(KafkaConfig.NumPartitionsProp, numParts.toString) | ||||||
| 
 | 
 | ||||||
|   val configs = |   def generateConfigs() = | ||||||
|     for (props <- TestUtils.createBrokerConfigs(numNodes, enableDeleteTopic = true)) |     TestUtils.createBrokerConfigs(numNodes, zkConnect, enableDeleteTopic=true).map(KafkaConfig.fromProps(_, overridingProps)) | ||||||
|     yield KafkaConfig.fromProps(props, overridingProps) |  | ||||||
| 
 | 
 | ||||||
|   val nMessages = 2 |   val nMessages = 2 | ||||||
| 
 | 
 | ||||||
|  | @ -80,7 +77,7 @@ class MetricsTest extends JUnit3Suite with KafkaServerTestHarness with Logging { | ||||||
|   } |   } | ||||||
| 
 | 
 | ||||||
|   def createAndShutdownStep(group: String, consumerId: String, producerId: String): Unit = { |   def createAndShutdownStep(group: String, consumerId: String, producerId: String): Unit = { | ||||||
|     val sentMessages1 = sendMessages(configs, topic, producerId, nMessages, "batch1", NoCompressionCodec, 1) |     val sentMessages1 = sendMessages(servers, topic, producerId, nMessages, "batch1", NoCompressionCodec, 1) | ||||||
|     // create a consumer |     // create a consumer | ||||||
|     val consumerConfig1 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumerId)) |     val consumerConfig1 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumerId)) | ||||||
|     val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, true) |     val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, true) | ||||||
|  |  | ||||||
|  | @ -36,7 +36,7 @@ class SocketServerTest extends JUnitSuite { | ||||||
| 
 | 
 | ||||||
|   val server: SocketServer = new SocketServer(0, |   val server: SocketServer = new SocketServer(0, | ||||||
|                                               host = null, |                                               host = null, | ||||||
|                                               port = kafka.utils.TestUtils.choosePort, |                                               port = 0, | ||||||
|                                               numProcessorThreads = 1, |                                               numProcessorThreads = 1, | ||||||
|                                               maxQueuedRequests = 50, |                                               maxQueuedRequests = 50, | ||||||
|                                               sendBufferSize = 300000, |                                               sendBufferSize = 300000, | ||||||
|  | @ -73,7 +73,7 @@ class SocketServerTest extends JUnitSuite { | ||||||
|     channel.sendResponse(new RequestChannel.Response(request.processor, request, send)) |     channel.sendResponse(new RequestChannel.Response(request.processor, request, send)) | ||||||
|   } |   } | ||||||
| 
 | 
 | ||||||
|   def connect(s:SocketServer = server) = new Socket("localhost", s.port) |   def connect(s:SocketServer = server) = new Socket("localhost", s.boundPort) | ||||||
| 
 | 
 | ||||||
|   @After |   @After | ||||||
|   def cleanup() { |   def cleanup() { | ||||||
|  | @ -162,7 +162,7 @@ class SocketServerTest extends JUnitSuite { | ||||||
|     val overrides: Map[String, Int] = Map("localhost" -> overrideNum) |     val overrides: Map[String, Int] = Map("localhost" -> overrideNum) | ||||||
|     val overrideServer: SocketServer = new SocketServer(0, |     val overrideServer: SocketServer = new SocketServer(0, | ||||||
|                                                 host = null, |                                                 host = null, | ||||||
|                                                 port = kafka.utils.TestUtils.choosePort, |                                                 port = 0, | ||||||
|                                                 numProcessorThreads = 1, |                                                 numProcessorThreads = 1, | ||||||
|                                                 maxQueuedRequests = 50, |                                                 maxQueuedRequests = 50, | ||||||
|                                                 sendBufferSize = 300000, |                                                 sendBufferSize = 300000, | ||||||
|  |  | ||||||
|  | @ -36,8 +36,10 @@ import scala.collection.mutable.ArrayBuffer | ||||||
| import kafka.utils._ | import kafka.utils._ | ||||||
| 
 | 
 | ||||||
| class AsyncProducerTest extends JUnit3Suite { | class AsyncProducerTest extends JUnit3Suite { | ||||||
|   val props = createBrokerConfigs(1) |   // One of the few cases we can just set a fixed port because the producer is mocked out here since this uses mocks | ||||||
|   val configs = props.map(p => KafkaConfig.fromProps(p)) |   val props = Seq(createBrokerConfig(1, "127.0.0.1:1", port=65534)) | ||||||
|  |   val configs = props.map(KafkaConfig.fromProps) | ||||||
|  |   val brokerList = configs.map(c => org.apache.kafka.common.utils.Utils.formatAddress(c.hostName, c.port)).mkString(",") | ||||||
| 
 | 
 | ||||||
|   override def setUp() { |   override def setUp() { | ||||||
|     super.setUp() |     super.setUp() | ||||||
|  | @ -61,7 +63,7 @@ class AsyncProducerTest extends JUnit3Suite { | ||||||
| 
 | 
 | ||||||
|     val props = new Properties() |     val props = new Properties() | ||||||
|     props.put("serializer.class", "kafka.serializer.StringEncoder") |     props.put("serializer.class", "kafka.serializer.StringEncoder") | ||||||
|     props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) |     props.put("metadata.broker.list", brokerList) | ||||||
|     props.put("producer.type", "async") |     props.put("producer.type", "async") | ||||||
|     props.put("queue.buffering.max.messages", "10") |     props.put("queue.buffering.max.messages", "10") | ||||||
|     props.put("batch.num.messages", "1") |     props.put("batch.num.messages", "1") | ||||||
|  | @ -86,7 +88,7 @@ class AsyncProducerTest extends JUnit3Suite { | ||||||
|   def testProduceAfterClosed() { |   def testProduceAfterClosed() { | ||||||
|     val produceData = getProduceData(10) |     val produceData = getProduceData(10) | ||||||
|     val producer = createProducer[String, String]( |     val producer = createProducer[String, String]( | ||||||
|       getBrokerListStrFromConfigs(configs), |       brokerList, | ||||||
|       encoder = classOf[StringEncoder].getName) |       encoder = classOf[StringEncoder].getName) | ||||||
| 
 | 
 | ||||||
|     producer.close |     producer.close | ||||||
|  | @ -162,7 +164,7 @@ class AsyncProducerTest extends JUnit3Suite { | ||||||
|     producerDataList.append(new KeyedMessage[Int,Message]("topic2", key = 4, message = new Message("msg5".getBytes))) |     producerDataList.append(new KeyedMessage[Int,Message]("topic2", key = 4, message = new Message("msg5".getBytes))) | ||||||
| 
 | 
 | ||||||
|     val props = new Properties() |     val props = new Properties() | ||||||
|     props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) |     props.put("metadata.broker.list", brokerList) | ||||||
|     val broker1 = new Broker(0, "localhost", 9092) |     val broker1 = new Broker(0, "localhost", 9092) | ||||||
|     val broker2 = new Broker(1, "localhost", 9093) |     val broker2 = new Broker(1, "localhost", 9093) | ||||||
| 
 | 
 | ||||||
|  | @ -212,7 +214,7 @@ class AsyncProducerTest extends JUnit3Suite { | ||||||
|   def testSerializeEvents() { |   def testSerializeEvents() { | ||||||
|     val produceData = TestUtils.getMsgStrings(5).map(m => new KeyedMessage[String,String]("topic1",m)) |     val produceData = TestUtils.getMsgStrings(5).map(m => new KeyedMessage[String,String]("topic1",m)) | ||||||
|     val props = new Properties() |     val props = new Properties() | ||||||
|     props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) |     props.put("metadata.broker.list", brokerList) | ||||||
|     val config = new ProducerConfig(props) |     val config = new ProducerConfig(props) | ||||||
|     // form expected partitions metadata |     // form expected partitions metadata | ||||||
|     val topic1Metadata = getTopicMetadata("topic1", 0, 0, "localhost", 9092) |     val topic1Metadata = getTopicMetadata("topic1", 0, 0, "localhost", 9092) | ||||||
|  | @ -244,7 +246,7 @@ class AsyncProducerTest extends JUnit3Suite { | ||||||
|     val producerDataList = new ArrayBuffer[KeyedMessage[String,Message]] |     val producerDataList = new ArrayBuffer[KeyedMessage[String,Message]] | ||||||
|     producerDataList.append(new KeyedMessage[String,Message]("topic1", "key1", new Message("msg1".getBytes))) |     producerDataList.append(new KeyedMessage[String,Message]("topic1", "key1", new Message("msg1".getBytes))) | ||||||
|     val props = new Properties() |     val props = new Properties() | ||||||
|     props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) |     props.put("metadata.broker.list", brokerList) | ||||||
|     val config = new ProducerConfig(props) |     val config = new ProducerConfig(props) | ||||||
| 
 | 
 | ||||||
|     // form expected partitions metadata |     // form expected partitions metadata | ||||||
|  | @ -274,7 +276,7 @@ class AsyncProducerTest extends JUnit3Suite { | ||||||
|   @Test |   @Test | ||||||
|   def testNoBroker() { |   def testNoBroker() { | ||||||
|     val props = new Properties() |     val props = new Properties() | ||||||
|     props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) |     props.put("metadata.broker.list", brokerList) | ||||||
| 
 | 
 | ||||||
|     val config = new ProducerConfig(props) |     val config = new ProducerConfig(props) | ||||||
|     // create topic metadata with 0 partitions |     // create topic metadata with 0 partitions | ||||||
|  | @ -308,7 +310,7 @@ class AsyncProducerTest extends JUnit3Suite { | ||||||
|     // no need to retry since the send will always fail |     // no need to retry since the send will always fail | ||||||
|     props.put("message.send.max.retries", "0") |     props.put("message.send.max.retries", "0") | ||||||
|     val producer= createProducer[String, String]( |     val producer= createProducer[String, String]( | ||||||
|       brokerList = getBrokerListStrFromConfigs(configs), |       brokerList = brokerList, | ||||||
|       encoder = classOf[DefaultEncoder].getName, |       encoder = classOf[DefaultEncoder].getName, | ||||||
|       keyEncoder = classOf[DefaultEncoder].getName, |       keyEncoder = classOf[DefaultEncoder].getName, | ||||||
|       producerProps = props) |       producerProps = props) | ||||||
|  | @ -326,7 +328,7 @@ class AsyncProducerTest extends JUnit3Suite { | ||||||
|   @Test |   @Test | ||||||
|   def testRandomPartitioner() { |   def testRandomPartitioner() { | ||||||
|     val props = new Properties() |     val props = new Properties() | ||||||
|     props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) |     props.put("metadata.broker.list", brokerList) | ||||||
|     val config = new ProducerConfig(props) |     val config = new ProducerConfig(props) | ||||||
| 
 | 
 | ||||||
|     // create topic metadata with 0 partitions |     // create topic metadata with 0 partitions | ||||||
|  | @ -364,7 +366,7 @@ class AsyncProducerTest extends JUnit3Suite { | ||||||
|   @Test |   @Test | ||||||
|   def testFailedSendRetryLogic() { |   def testFailedSendRetryLogic() { | ||||||
|     val props = new Properties() |     val props = new Properties() | ||||||
|     props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) |     props.put("metadata.broker.list", brokerList) | ||||||
|     props.put("request.required.acks", "1") |     props.put("request.required.acks", "1") | ||||||
|     props.put("serializer.class", classOf[StringEncoder].getName.toString) |     props.put("serializer.class", classOf[StringEncoder].getName.toString) | ||||||
|     props.put("key.serializer.class", classOf[NullEncoder[Int]].getName.toString) |     props.put("key.serializer.class", classOf[NullEncoder[Int]].getName.toString) | ||||||
|  |  | ||||||
|  | @ -17,7 +17,6 @@ | ||||||
| 
 | 
 | ||||||
| package kafka.producer | package kafka.producer | ||||||
| 
 | 
 | ||||||
| import org.apache.kafka.common.config.ConfigException |  | ||||||
| import org.scalatest.TestFailedException | import org.scalatest.TestFailedException | ||||||
| import org.scalatest.junit.JUnit3Suite | import org.scalatest.junit.JUnit3Suite | ||||||
| import kafka.consumer.SimpleConsumer | import kafka.consumer.SimpleConsumer | ||||||
|  | @ -40,8 +39,6 @@ import kafka.serializer.StringEncoder | ||||||
| class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ | class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ | ||||||
|   private val brokerId1 = 0 |   private val brokerId1 = 0 | ||||||
|   private val brokerId2 = 1 |   private val brokerId2 = 1 | ||||||
|   private val ports = TestUtils.choosePorts(2) |  | ||||||
|   private val (port1, port2) = (ports(0), ports(1)) |  | ||||||
|   private var server1: KafkaServer = null |   private var server1: KafkaServer = null | ||||||
|   private var server2: KafkaServer = null |   private var server2: KafkaServer = null | ||||||
|   private var consumer1: SimpleConsumer = null |   private var consumer1: SimpleConsumer = null | ||||||
|  | @ -49,26 +46,36 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ | ||||||
|   private val requestHandlerLogger = Logger.getLogger(classOf[KafkaRequestHandler]) |   private val requestHandlerLogger = Logger.getLogger(classOf[KafkaRequestHandler]) | ||||||
|   private var servers = List.empty[KafkaServer] |   private var servers = List.empty[KafkaServer] | ||||||
| 
 | 
 | ||||||
|   private val props1 = TestUtils.createBrokerConfig(brokerId1, port1, false) |   // Creation of consumers is deferred until they are actually needed. This allows us to kill brokers that use random | ||||||
|   props1.put("num.partitions", "4") |   // ports and then get a consumer instance that will be pointed at the correct port | ||||||
|   private val config1 = KafkaConfig.fromProps(props1) |   def getConsumer1() = { | ||||||
|   private val props2 = TestUtils.createBrokerConfig(brokerId2, port2, false) |     if (consumer1 == null) | ||||||
|   props2.put("num.partitions", "4") |       consumer1 = new SimpleConsumer("localhost", server1.boundPort(), 1000000, 64*1024, "") | ||||||
|   private val config2 = KafkaConfig.fromProps(props2) |     consumer1 | ||||||
|  |   } | ||||||
|  | 
 | ||||||
|  |   def getConsumer2() = { | ||||||
|  |     if (consumer2 == null) | ||||||
|  |       consumer2 = new SimpleConsumer("localhost", server2.boundPort(), 100, 64*1024, "") | ||||||
|  |     consumer2 | ||||||
|  |   } | ||||||
| 
 | 
 | ||||||
|   override def setUp() { |   override def setUp() { | ||||||
|     super.setUp() |     super.setUp() | ||||||
|     // set up 2 brokers with 4 partitions each |     // set up 2 brokers with 4 partitions each | ||||||
|  |     val props1 = TestUtils.createBrokerConfig(brokerId1, zkConnect, false) | ||||||
|  |     props1.put("num.partitions", "4") | ||||||
|  |     val config1 = KafkaConfig.fromProps(props1) | ||||||
|  |     val props2 = TestUtils.createBrokerConfig(brokerId2, zkConnect, false) | ||||||
|  |     props2.put("num.partitions", "4") | ||||||
|  |     val config2 = KafkaConfig.fromProps(props2) | ||||||
|     server1 = TestUtils.createServer(config1) |     server1 = TestUtils.createServer(config1) | ||||||
|     server2 = TestUtils.createServer(config2) |     server2 = TestUtils.createServer(config2) | ||||||
|     servers = List(server1,server2) |     servers = List(server1,server2) | ||||||
| 
 | 
 | ||||||
|     val props = new Properties() |     val props = new Properties() | ||||||
|     props.put("host", "localhost") |     props.put("host", "localhost") | ||||||
|     props.put("port", port1.toString) |     props.put("port", server1.boundPort().toString) | ||||||
| 
 |  | ||||||
|     consumer1 = new SimpleConsumer("localhost", port1, 1000000, 64*1024, "") |  | ||||||
|     consumer2 = new SimpleConsumer("localhost", port2, 100, 64*1024, "") |  | ||||||
| 
 | 
 | ||||||
|     // temporarily set request handler logger to a higher level |     // temporarily set request handler logger to a higher level | ||||||
|     requestHandlerLogger.setLevel(Level.FATAL) |     requestHandlerLogger.setLevel(Level.FATAL) | ||||||
|  | @ -115,7 +122,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ | ||||||
|     } |     } | ||||||
| 
 | 
 | ||||||
|     val producer2 = TestUtils.createProducer[String, String]( |     val producer2 = TestUtils.createProducer[String, String]( | ||||||
|       brokerList = "localhost:80," + TestUtils.getBrokerListStrFromConfigs(Seq(config1)), |       brokerList = "localhost:80," + TestUtils.getBrokerListStrFromServers(Seq(server1)), | ||||||
|       encoder = classOf[StringEncoder].getName, |       encoder = classOf[StringEncoder].getName, | ||||||
|       keyEncoder = classOf[StringEncoder].getName) |       keyEncoder = classOf[StringEncoder].getName) | ||||||
| 
 | 
 | ||||||
|  | @ -128,7 +135,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ | ||||||
|     } |     } | ||||||
| 
 | 
 | ||||||
|     val producer3 =  TestUtils.createProducer[String, String]( |     val producer3 =  TestUtils.createProducer[String, String]( | ||||||
|       brokerList = TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2)), |       brokerList = TestUtils.getBrokerListStrFromServers(Seq(server1, server2)), | ||||||
|       encoder = classOf[StringEncoder].getName, |       encoder = classOf[StringEncoder].getName, | ||||||
|       keyEncoder = classOf[StringEncoder].getName) |       keyEncoder = classOf[StringEncoder].getName) | ||||||
| 
 | 
 | ||||||
|  | @ -151,7 +158,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ | ||||||
|     TestUtils.createTopic(zkClient, topic, numPartitions = 1, replicationFactor = 2, servers = servers) |     TestUtils.createTopic(zkClient, topic, numPartitions = 1, replicationFactor = 2, servers = servers) | ||||||
| 
 | 
 | ||||||
|     val producer1 = TestUtils.createProducer[String, String]( |     val producer1 = TestUtils.createProducer[String, String]( | ||||||
|       brokerList = TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2)), |       brokerList = TestUtils.getBrokerListStrFromServers(Seq(server1, server2)), | ||||||
|       encoder = classOf[StringEncoder].getName, |       encoder = classOf[StringEncoder].getName, | ||||||
|       keyEncoder = classOf[StringEncoder].getName, |       keyEncoder = classOf[StringEncoder].getName, | ||||||
|       partitioner = classOf[StaticPartitioner].getName, |       partitioner = classOf[StaticPartitioner].getName, | ||||||
|  | @ -166,10 +173,10 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ | ||||||
|     val leader = leaderOpt.get |     val leader = leaderOpt.get | ||||||
| 
 | 
 | ||||||
|     val messageSet = if(leader == server1.config.brokerId) { |     val messageSet = if(leader == server1.config.brokerId) { | ||||||
|       val response1 = consumer1.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build()) |       val response1 = getConsumer1().fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build()) | ||||||
|       response1.messageSet("new-topic", 0).iterator.toBuffer |       response1.messageSet("new-topic", 0).iterator.toBuffer | ||||||
|     }else { |     }else { | ||||||
|       val response2 = consumer2.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build()) |       val response2 = getConsumer2().fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build()) | ||||||
|       response2.messageSet("new-topic", 0).iterator.toBuffer |       response2.messageSet("new-topic", 0).iterator.toBuffer | ||||||
|     } |     } | ||||||
|     assertEquals("Should have fetched 2 messages", 2, messageSet.size) |     assertEquals("Should have fetched 2 messages", 2, messageSet.size) | ||||||
|  | @ -184,7 +191,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ | ||||||
| 
 | 
 | ||||||
|     try { |     try { | ||||||
|       val producer2 = TestUtils.createProducer[String, String]( |       val producer2 = TestUtils.createProducer[String, String]( | ||||||
|         brokerList = TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2)), |         brokerList = TestUtils.getBrokerListStrFromServers(Seq(server1, server2)), | ||||||
|         encoder = classOf[StringEncoder].getName, |         encoder = classOf[StringEncoder].getName, | ||||||
|         keyEncoder = classOf[StringEncoder].getName, |         keyEncoder = classOf[StringEncoder].getName, | ||||||
|         partitioner = classOf[StaticPartitioner].getName, |         partitioner = classOf[StaticPartitioner].getName, | ||||||
|  | @ -214,7 +221,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ | ||||||
|                           servers = servers) |                           servers = servers) | ||||||
| 
 | 
 | ||||||
|     val producer = TestUtils.createProducer[String, String]( |     val producer = TestUtils.createProducer[String, String]( | ||||||
|       brokerList = TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2)), |       brokerList = TestUtils.getBrokerListStrFromServers(Seq(server1, server2)), | ||||||
|       encoder = classOf[StringEncoder].getName, |       encoder = classOf[StringEncoder].getName, | ||||||
|       keyEncoder = classOf[StringEncoder].getName, |       keyEncoder = classOf[StringEncoder].getName, | ||||||
|       partitioner = classOf[StaticPartitioner].getName, |       partitioner = classOf[StaticPartitioner].getName, | ||||||
|  | @ -248,7 +255,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ | ||||||
| 
 | 
 | ||||||
|     try { |     try { | ||||||
|       // cross check if broker 1 got the messages |       // cross check if broker 1 got the messages | ||||||
|       val response1 = consumer1.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build()) |       val response1 = getConsumer1().fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build()) | ||||||
|       val messageSet1 = response1.messageSet(topic, 0).iterator |       val messageSet1 = response1.messageSet(topic, 0).iterator | ||||||
|       assertTrue("Message set should have 1 message", messageSet1.hasNext) |       assertTrue("Message set should have 1 message", messageSet1.hasNext) | ||||||
|       assertEquals(new Message(bytes = "test1".getBytes, key = "test".getBytes), messageSet1.next.message) |       assertEquals(new Message(bytes = "test1".getBytes, key = "test".getBytes), messageSet1.next.message) | ||||||
|  | @ -268,7 +275,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ | ||||||
|     props.put("message.send.max.retries", "0") |     props.put("message.send.max.retries", "0") | ||||||
|     props.put("client.id","ProducerTest-testAsyncSendCanCorrectlyFailWithTimeout") |     props.put("client.id","ProducerTest-testAsyncSendCanCorrectlyFailWithTimeout") | ||||||
|     val producer = TestUtils.createProducer[String, String]( |     val producer = TestUtils.createProducer[String, String]( | ||||||
|       brokerList = TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2)), |       brokerList = TestUtils.getBrokerListStrFromServers(Seq(server1, server2)), | ||||||
|       encoder = classOf[StringEncoder].getName, |       encoder = classOf[StringEncoder].getName, | ||||||
|       keyEncoder = classOf[StringEncoder].getName, |       keyEncoder = classOf[StringEncoder].getName, | ||||||
|       partitioner = classOf[StaticPartitioner].getName, |       partitioner = classOf[StaticPartitioner].getName, | ||||||
|  | @ -283,7 +290,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ | ||||||
|       // this message should be assigned to partition 0 whose leader is on broker 0 |       // this message should be assigned to partition 0 whose leader is on broker 0 | ||||||
|       producer.send(new KeyedMessage[String, String](topic, "test", "test")) |       producer.send(new KeyedMessage[String, String](topic, "test", "test")) | ||||||
|       // cross check if brokers got the messages |       // cross check if brokers got the messages | ||||||
|       val response1 = consumer1.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build()) |       val response1 = getConsumer1().fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build()) | ||||||
|       val messageSet1 = response1.messageSet("new-topic", 0).iterator |       val messageSet1 = response1.messageSet("new-topic", 0).iterator | ||||||
|       assertTrue("Message set should have 1 message", messageSet1.hasNext) |       assertTrue("Message set should have 1 message", messageSet1.hasNext) | ||||||
|       assertEquals(new Message("test".getBytes), messageSet1.next.message) |       assertEquals(new Message("test".getBytes), messageSet1.next.message) | ||||||
|  | @ -315,7 +322,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ | ||||||
|   @Test |   @Test | ||||||
|   def testSendNullMessage() { |   def testSendNullMessage() { | ||||||
|     val producer = TestUtils.createProducer[String, String]( |     val producer = TestUtils.createProducer[String, String]( | ||||||
|       brokerList = TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2)), |       brokerList = TestUtils.getBrokerListStrFromServers(Seq(server1, server2)), | ||||||
|       encoder = classOf[StringEncoder].getName, |       encoder = classOf[StringEncoder].getName, | ||||||
|       keyEncoder = classOf[StringEncoder].getName, |       keyEncoder = classOf[StringEncoder].getName, | ||||||
|       partitioner = classOf[StaticPartitioner].getName) |       partitioner = classOf[StaticPartitioner].getName) | ||||||
|  |  | ||||||
|  | @ -33,13 +33,12 @@ import kafka.common.{TopicAndPartition, ErrorMapping} | ||||||
| class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { | class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { | ||||||
|   private val messageBytes =  new Array[Byte](2) |   private val messageBytes =  new Array[Byte](2) | ||||||
|   // turning off controlled shutdown since testProducerCanTimeout() explicitly shuts down request handler pool. |   // turning off controlled shutdown since testProducerCanTimeout() explicitly shuts down request handler pool. | ||||||
|   val configs = List(KafkaConfig.fromProps(TestUtils.createBrokerConfigs(1, false).head)) |   def generateConfigs() = List(KafkaConfig.fromProps(TestUtils.createBrokerConfigs(1, zkConnect, false).head)) | ||||||
|   val zookeeperConnect = TestZKUtils.zookeeperConnect |  | ||||||
| 
 | 
 | ||||||
|   @Test |   @Test | ||||||
|   def testReachableServer() { |   def testReachableServer() { | ||||||
|     val server = servers.head |     val server = servers.head | ||||||
|     val props = TestUtils.getSyncProducerConfig(server.socketServer.port) |     val props = TestUtils.getSyncProducerConfig(server.socketServer.boundPort) | ||||||
| 
 | 
 | ||||||
|     val producer = new SyncProducer(new SyncProducerConfig(props)) |     val producer = new SyncProducer(new SyncProducerConfig(props)) | ||||||
|     val firstStart = SystemTime.milliseconds |     val firstStart = SystemTime.milliseconds | ||||||
|  | @ -74,7 +73,7 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { | ||||||
|   @Test |   @Test | ||||||
|   def testEmptyProduceRequest() { |   def testEmptyProduceRequest() { | ||||||
|     val server = servers.head |     val server = servers.head | ||||||
|     val props = TestUtils.getSyncProducerConfig(server.socketServer.port) |     val props = TestUtils.getSyncProducerConfig(server.socketServer.boundPort) | ||||||
| 
 | 
 | ||||||
|     val correlationId = 0 |     val correlationId = 0 | ||||||
|     val clientId = SyncProducerConfig.DefaultClientId |     val clientId = SyncProducerConfig.DefaultClientId | ||||||
|  | @ -91,7 +90,7 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { | ||||||
|   @Test |   @Test | ||||||
|   def testMessageSizeTooLarge() { |   def testMessageSizeTooLarge() { | ||||||
|     val server = servers.head |     val server = servers.head | ||||||
|     val props = TestUtils.getSyncProducerConfig(server.socketServer.port) |     val props = TestUtils.getSyncProducerConfig(server.socketServer.boundPort) | ||||||
| 
 | 
 | ||||||
|     val producer = new SyncProducer(new SyncProducerConfig(props)) |     val producer = new SyncProducer(new SyncProducerConfig(props)) | ||||||
|     TestUtils.createTopic(zkClient, "test", numPartitions = 1, replicationFactor = 1, servers = servers) |     TestUtils.createTopic(zkClient, "test", numPartitions = 1, replicationFactor = 1, servers = servers) | ||||||
|  | @ -119,7 +118,7 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { | ||||||
|   def testMessageSizeTooLargeWithAckZero() { |   def testMessageSizeTooLargeWithAckZero() { | ||||||
|     val server = servers.head |     val server = servers.head | ||||||
| 
 | 
 | ||||||
|     val props = TestUtils.getSyncProducerConfig(server.socketServer.port) |     val props = TestUtils.getSyncProducerConfig(server.socketServer.boundPort) | ||||||
|     props.put("request.required.acks", "0") |     props.put("request.required.acks", "0") | ||||||
| 
 | 
 | ||||||
|     val producer = new SyncProducer(new SyncProducerConfig(props)) |     val producer = new SyncProducer(new SyncProducerConfig(props)) | ||||||
|  | @ -145,7 +144,7 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { | ||||||
|   @Test |   @Test | ||||||
|   def testProduceCorrectlyReceivesResponse() { |   def testProduceCorrectlyReceivesResponse() { | ||||||
|     val server = servers.head |     val server = servers.head | ||||||
|     val props = TestUtils.getSyncProducerConfig(server.socketServer.port) |     val props = TestUtils.getSyncProducerConfig(server.socketServer.boundPort) | ||||||
| 
 | 
 | ||||||
|     val producer = new SyncProducer(new SyncProducerConfig(props)) |     val producer = new SyncProducer(new SyncProducerConfig(props)) | ||||||
|     val messages = new ByteBufferMessageSet(NoCompressionCodec, new Message(messageBytes)) |     val messages = new ByteBufferMessageSet(NoCompressionCodec, new Message(messageBytes)) | ||||||
|  | @ -191,7 +190,7 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { | ||||||
|     val timeoutMs = 500 |     val timeoutMs = 500 | ||||||
| 
 | 
 | ||||||
|     val server = servers.head |     val server = servers.head | ||||||
|     val props = TestUtils.getSyncProducerConfig(server.socketServer.port) |     val props = TestUtils.getSyncProducerConfig(server.socketServer.boundPort) | ||||||
|     val producer = new SyncProducer(new SyncProducerConfig(props)) |     val producer = new SyncProducer(new SyncProducerConfig(props)) | ||||||
| 
 | 
 | ||||||
|     val messages = new ByteBufferMessageSet(NoCompressionCodec, new Message(messageBytes)) |     val messages = new ByteBufferMessageSet(NoCompressionCodec, new Message(messageBytes)) | ||||||
|  | @ -217,7 +216,7 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { | ||||||
|   @Test |   @Test | ||||||
|   def testProduceRequestWithNoResponse() { |   def testProduceRequestWithNoResponse() { | ||||||
|     val server = servers.head |     val server = servers.head | ||||||
|     val props = TestUtils.getSyncProducerConfig(server.socketServer.port) |     val props = TestUtils.getSyncProducerConfig(server.socketServer.boundPort) | ||||||
|     val correlationId = 0 |     val correlationId = 0 | ||||||
|     val clientId = SyncProducerConfig.DefaultClientId |     val clientId = SyncProducerConfig.DefaultClientId | ||||||
|     val ackTimeoutMs = SyncProducerConfig.DefaultAckTimeoutMs |     val ackTimeoutMs = SyncProducerConfig.DefaultAckTimeoutMs | ||||||
|  | @ -233,7 +232,7 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { | ||||||
|     val topicName = "minisrtest" |     val topicName = "minisrtest" | ||||||
|     val server = servers.head |     val server = servers.head | ||||||
| 
 | 
 | ||||||
|     val props = TestUtils.getSyncProducerConfig(server.socketServer.port) |     val props = TestUtils.getSyncProducerConfig(server.socketServer.boundPort) | ||||||
|     props.put("request.required.acks", "-1") |     props.put("request.required.acks", "-1") | ||||||
| 
 | 
 | ||||||
|     val producer = new SyncProducer(new SyncProducerConfig(props)) |     val producer = new SyncProducer(new SyncProducerConfig(props)) | ||||||
|  |  | ||||||
|  | @ -30,7 +30,7 @@ class AdvertiseBrokerTest extends JUnit3Suite with ZooKeeperTestHarness { | ||||||
| 
 | 
 | ||||||
|   override def setUp() { |   override def setUp() { | ||||||
|     super.setUp() |     super.setUp() | ||||||
|     val props = TestUtils.createBrokerConfig(brokerId, TestUtils.choosePort()) |     val props = TestUtils.createBrokerConfig(brokerId, zkConnect) | ||||||
|     props.put("advertised.host.name", advertisedHostName) |     props.put("advertised.host.name", advertisedHostName) | ||||||
|     props.put("advertised.port", advertisedPort.toString) |     props.put("advertised.port", advertisedPort.toString) | ||||||
|      |      | ||||||
|  |  | ||||||
|  | @ -26,8 +26,7 @@ import kafka.admin.{AdminOperationException, AdminUtils} | ||||||
| import org.scalatest.junit.JUnit3Suite | import org.scalatest.junit.JUnit3Suite | ||||||
| 
 | 
 | ||||||
| class DynamicConfigChangeTest extends JUnit3Suite with KafkaServerTestHarness { | class DynamicConfigChangeTest extends JUnit3Suite with KafkaServerTestHarness { | ||||||
|    |   def generateConfigs() = List(KafkaConfig.fromProps(TestUtils.createBrokerConfig(0, zkConnect))) | ||||||
|   override val configs = List(KafkaConfig.fromProps(TestUtils.createBrokerConfig(0, TestUtils.choosePort))) |  | ||||||
| 
 | 
 | ||||||
|   @Test |   @Test | ||||||
|   def testConfigChange() { |   def testConfigChange() { | ||||||
|  |  | ||||||
|  | @ -30,7 +30,7 @@ import java.util.concurrent.atomic.AtomicBoolean | ||||||
| 
 | 
 | ||||||
| class HighwatermarkPersistenceTest extends JUnit3Suite { | class HighwatermarkPersistenceTest extends JUnit3Suite { | ||||||
| 
 | 
 | ||||||
|   val configs = TestUtils.createBrokerConfigs(2).map(KafkaConfig.fromProps) |   val configs = TestUtils.createBrokerConfigs(2, TestUtils.MockZkConnect).map(KafkaConfig.fromProps) | ||||||
|   val topic = "foo" |   val topic = "foo" | ||||||
|   val logManagers = configs map { config => |   val logManagers = configs map { config => | ||||||
|     TestUtils.createLogManager( |     TestUtils.createLogManager( | ||||||
|  |  | ||||||
|  | @ -39,7 +39,7 @@ class IsrExpirationTest extends JUnit3Suite { | ||||||
|   val overridingProps = new Properties() |   val overridingProps = new Properties() | ||||||
|   overridingProps.put(KafkaConfig.ReplicaLagTimeMaxMsProp, replicaLagTimeMaxMs.toString) |   overridingProps.put(KafkaConfig.ReplicaLagTimeMaxMsProp, replicaLagTimeMaxMs.toString) | ||||||
|   overridingProps.put(KafkaConfig.ReplicaFetchWaitMaxMsProp, replicaFetchWaitMaxMs.toString) |   overridingProps.put(KafkaConfig.ReplicaFetchWaitMaxMsProp, replicaFetchWaitMaxMs.toString) | ||||||
|   val configs = TestUtils.createBrokerConfigs(2).map(KafkaConfig.fromProps(_, overridingProps)) |   val configs = TestUtils.createBrokerConfigs(2, TestUtils.MockZkConnect).map(KafkaConfig.fromProps(_, overridingProps)) | ||||||
|   val topic = "foo" |   val topic = "foo" | ||||||
| 
 | 
 | ||||||
|   val time = new MockTime |   val time = new MockTime | ||||||
|  |  | ||||||
|  | @ -29,7 +29,7 @@ class KafkaConfigTest extends JUnit3Suite { | ||||||
| 
 | 
 | ||||||
|   @Test |   @Test | ||||||
|   def testLogRetentionTimeHoursProvided() { |   def testLogRetentionTimeHoursProvided() { | ||||||
|     val props = TestUtils.createBrokerConfig(0, 8181) |     val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) | ||||||
|     props.put("log.retention.hours", "1") |     props.put("log.retention.hours", "1") | ||||||
| 
 | 
 | ||||||
|     val cfg = KafkaConfig.fromProps(props) |     val cfg = KafkaConfig.fromProps(props) | ||||||
|  | @ -39,7 +39,7 @@ class KafkaConfigTest extends JUnit3Suite { | ||||||
|    |    | ||||||
|   @Test |   @Test | ||||||
|   def testLogRetentionTimeMinutesProvided() { |   def testLogRetentionTimeMinutesProvided() { | ||||||
|     val props = TestUtils.createBrokerConfig(0, 8181) |     val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) | ||||||
|     props.put("log.retention.minutes", "30") |     props.put("log.retention.minutes", "30") | ||||||
| 
 | 
 | ||||||
|     val cfg = KafkaConfig.fromProps(props) |     val cfg = KafkaConfig.fromProps(props) | ||||||
|  | @ -49,7 +49,7 @@ class KafkaConfigTest extends JUnit3Suite { | ||||||
|    |    | ||||||
|   @Test |   @Test | ||||||
|   def testLogRetentionTimeMsProvided() { |   def testLogRetentionTimeMsProvided() { | ||||||
|     val props = TestUtils.createBrokerConfig(0, 8181) |     val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) | ||||||
|     props.put("log.retention.ms", "1800000") |     props.put("log.retention.ms", "1800000") | ||||||
| 
 | 
 | ||||||
|     val cfg = KafkaConfig.fromProps(props) |     val cfg = KafkaConfig.fromProps(props) | ||||||
|  | @ -59,7 +59,7 @@ class KafkaConfigTest extends JUnit3Suite { | ||||||
|    |    | ||||||
|   @Test |   @Test | ||||||
|   def testLogRetentionTimeNoConfigProvided() { |   def testLogRetentionTimeNoConfigProvided() { | ||||||
|     val props = TestUtils.createBrokerConfig(0, 8181) |     val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) | ||||||
| 
 | 
 | ||||||
|     val cfg = KafkaConfig.fromProps(props) |     val cfg = KafkaConfig.fromProps(props) | ||||||
|     assertEquals(24 * 7 * 60L * 60L * 1000L, cfg.logRetentionTimeMillis) |     assertEquals(24 * 7 * 60L * 60L * 1000L, cfg.logRetentionTimeMillis) | ||||||
|  | @ -68,7 +68,7 @@ class KafkaConfigTest extends JUnit3Suite { | ||||||
|    |    | ||||||
|   @Test |   @Test | ||||||
|   def testLogRetentionTimeBothMinutesAndHoursProvided() { |   def testLogRetentionTimeBothMinutesAndHoursProvided() { | ||||||
|     val props = TestUtils.createBrokerConfig(0, 8181) |     val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) | ||||||
|     props.put("log.retention.minutes", "30") |     props.put("log.retention.minutes", "30") | ||||||
|     props.put("log.retention.hours", "1") |     props.put("log.retention.hours", "1") | ||||||
| 
 | 
 | ||||||
|  | @ -79,7 +79,7 @@ class KafkaConfigTest extends JUnit3Suite { | ||||||
|    |    | ||||||
|   @Test |   @Test | ||||||
|   def testLogRetentionTimeBothMinutesAndMsProvided() { |   def testLogRetentionTimeBothMinutesAndMsProvided() { | ||||||
|     val props = TestUtils.createBrokerConfig(0, 8181) |     val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) | ||||||
|     props.put("log.retention.ms", "1800000") |     props.put("log.retention.ms", "1800000") | ||||||
|     props.put("log.retention.minutes", "10") |     props.put("log.retention.minutes", "10") | ||||||
| 
 | 
 | ||||||
|  | @ -93,7 +93,7 @@ class KafkaConfigTest extends JUnit3Suite { | ||||||
|     val port = 9999 |     val port = 9999 | ||||||
|     val hostName = "fake-host" |     val hostName = "fake-host" | ||||||
|      |      | ||||||
|     val props = TestUtils.createBrokerConfig(0, port) |     val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = port) | ||||||
|     props.put("host.name", hostName) |     props.put("host.name", hostName) | ||||||
|      |      | ||||||
|     val serverConfig = KafkaConfig.fromProps(props) |     val serverConfig = KafkaConfig.fromProps(props) | ||||||
|  | @ -108,7 +108,7 @@ class KafkaConfigTest extends JUnit3Suite { | ||||||
|     val advertisedHostName = "routable-host" |     val advertisedHostName = "routable-host" | ||||||
|     val advertisedPort = 1234 |     val advertisedPort = 1234 | ||||||
|      |      | ||||||
|     val props = TestUtils.createBrokerConfig(0, port) |     val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = port) | ||||||
|     props.put("advertised.host.name", advertisedHostName) |     props.put("advertised.host.name", advertisedHostName) | ||||||
|     props.put("advertised.port", advertisedPort.toString) |     props.put("advertised.port", advertisedPort.toString) | ||||||
|      |      | ||||||
|  | @ -120,7 +120,7 @@ class KafkaConfigTest extends JUnit3Suite { | ||||||
| 
 | 
 | ||||||
|   @Test |   @Test | ||||||
|   def testUncleanLeaderElectionDefault() { |   def testUncleanLeaderElectionDefault() { | ||||||
|     val props = TestUtils.createBrokerConfig(0, 8181) |     val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) | ||||||
|     val serverConfig = KafkaConfig.fromProps(props) |     val serverConfig = KafkaConfig.fromProps(props) | ||||||
| 
 | 
 | ||||||
|     assertEquals(serverConfig.uncleanLeaderElectionEnable, true) |     assertEquals(serverConfig.uncleanLeaderElectionEnable, true) | ||||||
|  | @ -128,7 +128,7 @@ class KafkaConfigTest extends JUnit3Suite { | ||||||
| 
 | 
 | ||||||
|   @Test |   @Test | ||||||
|   def testUncleanElectionDisabled() { |   def testUncleanElectionDisabled() { | ||||||
|     val props = TestUtils.createBrokerConfig(0, 8181) |     val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) | ||||||
|     props.put("unclean.leader.election.enable", String.valueOf(false)) |     props.put("unclean.leader.election.enable", String.valueOf(false)) | ||||||
|     val serverConfig = KafkaConfig.fromProps(props) |     val serverConfig = KafkaConfig.fromProps(props) | ||||||
| 
 | 
 | ||||||
|  | @ -137,7 +137,7 @@ class KafkaConfigTest extends JUnit3Suite { | ||||||
| 
 | 
 | ||||||
|   @Test |   @Test | ||||||
|   def testUncleanElectionEnabled() { |   def testUncleanElectionEnabled() { | ||||||
|     val props = TestUtils.createBrokerConfig(0, 8181) |     val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) | ||||||
|     props.put("unclean.leader.election.enable", String.valueOf(true)) |     props.put("unclean.leader.election.enable", String.valueOf(true)) | ||||||
|     val serverConfig = KafkaConfig.fromProps(props) |     val serverConfig = KafkaConfig.fromProps(props) | ||||||
| 
 | 
 | ||||||
|  | @ -146,7 +146,7 @@ class KafkaConfigTest extends JUnit3Suite { | ||||||
| 
 | 
 | ||||||
|   @Test |   @Test | ||||||
|   def testUncleanElectionInvalid() { |   def testUncleanElectionInvalid() { | ||||||
|     val props = TestUtils.createBrokerConfig(0, 8181) |     val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) | ||||||
|     props.put("unclean.leader.election.enable", "invalid") |     props.put("unclean.leader.election.enable", "invalid") | ||||||
| 
 | 
 | ||||||
|     intercept[ConfigException] { |     intercept[ConfigException] { | ||||||
|  | @ -156,7 +156,7 @@ class KafkaConfigTest extends JUnit3Suite { | ||||||
|    |    | ||||||
|   @Test |   @Test | ||||||
|   def testLogRollTimeMsProvided() { |   def testLogRollTimeMsProvided() { | ||||||
|     val props = TestUtils.createBrokerConfig(0, 8181) |     val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) | ||||||
|     props.put("log.roll.ms", "1800000") |     props.put("log.roll.ms", "1800000") | ||||||
| 
 | 
 | ||||||
|     val cfg = KafkaConfig.fromProps(props) |     val cfg = KafkaConfig.fromProps(props) | ||||||
|  | @ -166,7 +166,7 @@ class KafkaConfigTest extends JUnit3Suite { | ||||||
|    |    | ||||||
|   @Test |   @Test | ||||||
|   def testLogRollTimeBothMsAndHoursProvided() { |   def testLogRollTimeBothMsAndHoursProvided() { | ||||||
|     val props = TestUtils.createBrokerConfig(0, 8181) |     val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) | ||||||
|     props.put("log.roll.ms", "1800000") |     props.put("log.roll.ms", "1800000") | ||||||
|     props.put("log.roll.hours", "1") |     props.put("log.roll.hours", "1") | ||||||
| 
 | 
 | ||||||
|  | @ -177,7 +177,7 @@ class KafkaConfigTest extends JUnit3Suite { | ||||||
|      |      | ||||||
|   @Test |   @Test | ||||||
|   def testLogRollTimeNoConfigProvided() { |   def testLogRollTimeNoConfigProvided() { | ||||||
|     val props = TestUtils.createBrokerConfig(0, 8181) |     val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) | ||||||
| 
 | 
 | ||||||
|     val cfg = KafkaConfig.fromProps(props) |     val cfg = KafkaConfig.fromProps(props) | ||||||
|     assertEquals(24 * 7 * 60L * 60L * 1000L, cfg.logRollTimeMillis																									) |     assertEquals(24 * 7 * 60L * 60L * 1000L, cfg.logRollTimeMillis																									) | ||||||
|  | @ -186,7 +186,7 @@ class KafkaConfigTest extends JUnit3Suite { | ||||||
| 
 | 
 | ||||||
|   @Test |   @Test | ||||||
|   def testDefaultCompressionType() { |   def testDefaultCompressionType() { | ||||||
|     val props = TestUtils.createBrokerConfig(0, 8181) |     val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) | ||||||
|     val serverConfig = KafkaConfig.fromProps(props) |     val serverConfig = KafkaConfig.fromProps(props) | ||||||
| 
 | 
 | ||||||
|     assertEquals(serverConfig.compressionType, "producer") |     assertEquals(serverConfig.compressionType, "producer") | ||||||
|  | @ -194,7 +194,7 @@ class KafkaConfigTest extends JUnit3Suite { | ||||||
| 
 | 
 | ||||||
|   @Test |   @Test | ||||||
|   def testValidCompressionType() { |   def testValidCompressionType() { | ||||||
|     val props = TestUtils.createBrokerConfig(0, 8181) |     val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) | ||||||
|     props.put("compression.type", "gzip") |     props.put("compression.type", "gzip") | ||||||
|     val serverConfig = KafkaConfig.fromProps(props) |     val serverConfig = KafkaConfig.fromProps(props) | ||||||
| 
 | 
 | ||||||
|  | @ -203,7 +203,7 @@ class KafkaConfigTest extends JUnit3Suite { | ||||||
| 
 | 
 | ||||||
|   @Test |   @Test | ||||||
|   def testInvalidCompressionType() { |   def testInvalidCompressionType() { | ||||||
|     val props = TestUtils.createBrokerConfig(0, 8181) |     val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) | ||||||
|     props.put("compression.type", "abc") |     props.put("compression.type", "abc") | ||||||
|     intercept[IllegalArgumentException] { |     intercept[IllegalArgumentException] { | ||||||
|       KafkaConfig.fromProps(props) |       KafkaConfig.fromProps(props) | ||||||
|  |  | ||||||
|  | @ -31,17 +31,16 @@ class LeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { | ||||||
|   val brokerId1 = 0 |   val brokerId1 = 0 | ||||||
|   val brokerId2 = 1 |   val brokerId2 = 1 | ||||||
| 
 | 
 | ||||||
|   val port1 = TestUtils.choosePort() |  | ||||||
|   val port2 = TestUtils.choosePort() |  | ||||||
| 
 |  | ||||||
|   val configProps1 = TestUtils.createBrokerConfig(brokerId1, port1, false) |  | ||||||
|   val configProps2 = TestUtils.createBrokerConfig(brokerId2, port2, false) |  | ||||||
|   var servers: Seq[KafkaServer] = Seq.empty[KafkaServer] |   var servers: Seq[KafkaServer] = Seq.empty[KafkaServer] | ||||||
| 
 | 
 | ||||||
|   var staleControllerEpochDetected = false |   var staleControllerEpochDetected = false | ||||||
| 
 | 
 | ||||||
|   override def setUp() { |   override def setUp() { | ||||||
|     super.setUp() |     super.setUp() | ||||||
|  | 
 | ||||||
|  |     val configProps1 = TestUtils.createBrokerConfig(brokerId1, zkConnect, enableControlledShutdown = false) | ||||||
|  |     val configProps2 = TestUtils.createBrokerConfig(brokerId2, zkConnect, enableControlledShutdown = false) | ||||||
|  | 
 | ||||||
|     // start both servers |     // start both servers | ||||||
|     val server1 = TestUtils.createServer(KafkaConfig.fromProps(configProps1)) |     val server1 = TestUtils.createServer(KafkaConfig.fromProps(configProps1)) | ||||||
|     val server2 = TestUtils.createServer(KafkaConfig.fromProps(configProps2)) |     val server2 = TestUtils.createServer(KafkaConfig.fromProps(configProps2)) | ||||||
|  | @ -117,8 +116,8 @@ class LeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { | ||||||
| 
 | 
 | ||||||
|     // start another controller |     // start another controller | ||||||
|     val controllerId = 2 |     val controllerId = 2 | ||||||
|     val controllerConfig = KafkaConfig.fromProps(TestUtils.createBrokerConfig(controllerId, TestUtils.choosePort())) |     val controllerConfig = KafkaConfig.fromProps(TestUtils.createBrokerConfig(controllerId, zkConnect)) | ||||||
|     val brokers = servers.map(s => new Broker(s.config.brokerId, "localhost", s.config.port)) |     val brokers = servers.map(s => new Broker(s.config.brokerId, "localhost", s.boundPort())) | ||||||
|     val controllerContext = new ControllerContext(zkClient, 6000) |     val controllerContext = new ControllerContext(zkClient, 6000) | ||||||
|     controllerContext.liveBrokers = brokers.toSet |     controllerContext.liveBrokers = brokers.toSet | ||||||
|     val controllerChannelManager = new ControllerChannelManager(controllerContext, controllerConfig) |     val controllerChannelManager = new ControllerChannelManager(controllerContext, controllerConfig) | ||||||
|  |  | ||||||
|  | @ -39,19 +39,18 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness { | ||||||
|   var topicLogDir: File = null |   var topicLogDir: File = null | ||||||
|   var server: KafkaServer = null |   var server: KafkaServer = null | ||||||
|   var logSize: Int = 100 |   var logSize: Int = 100 | ||||||
|   val brokerPort: Int = 9099 |  | ||||||
|   var simpleConsumer: SimpleConsumer = null |   var simpleConsumer: SimpleConsumer = null | ||||||
|   var time: Time = new MockTime() |   var time: Time = new MockTime() | ||||||
| 
 | 
 | ||||||
|   @Before |   @Before | ||||||
|   override def setUp() { |   override def setUp() { | ||||||
|     super.setUp() |     super.setUp() | ||||||
|     val config: Properties = createBrokerConfig(1, brokerPort) |     val config: Properties = createBrokerConfig(1) | ||||||
|     val logDirPath = config.getProperty("log.dir") |     val logDirPath = config.getProperty("log.dir") | ||||||
|     logDir = new File(logDirPath) |     logDir = new File(logDirPath) | ||||||
|     time = new MockTime() |     time = new MockTime() | ||||||
|     server = TestUtils.createServer(KafkaConfig.fromProps(config), time) |     server = TestUtils.createServer(KafkaConfig.fromProps(config), time) | ||||||
|     simpleConsumer = new SimpleConsumer("localhost", brokerPort, 1000000, 64*1024, "") |     simpleConsumer = new SimpleConsumer("localhost", server.boundPort(), 1000000, 64*1024, "") | ||||||
|   } |   } | ||||||
| 
 | 
 | ||||||
|   @After |   @After | ||||||
|  | @ -194,10 +193,10 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness { | ||||||
|     assertEquals(Seq(0L), consumerOffsets) |     assertEquals(Seq(0L), consumerOffsets) | ||||||
|   } |   } | ||||||
| 
 | 
 | ||||||
|   private def createBrokerConfig(nodeId: Int, port: Int): Properties = { |   private def createBrokerConfig(nodeId: Int): Properties = { | ||||||
|     val props = new Properties |     val props = new Properties | ||||||
|     props.put("broker.id", nodeId.toString) |     props.put("broker.id", nodeId.toString) | ||||||
|     props.put("port", port.toString) |     props.put("port", TestUtils.RandomPort.toString()) | ||||||
|     props.put("log.dir", getLogDir.getAbsolutePath) |     props.put("log.dir", getLogDir.getAbsolutePath) | ||||||
|     props.put("log.flush.interval.messages", "1") |     props.put("log.flush.interval.messages", "1") | ||||||
|     props.put("enable.zookeeper", "false") |     props.put("enable.zookeeper", "false") | ||||||
|  |  | ||||||
|  | @ -19,8 +19,7 @@ package kafka.server | ||||||
| import java.util.Properties | import java.util.Properties | ||||||
| 
 | 
 | ||||||
| import kafka.utils.TestUtils._ | import kafka.utils.TestUtils._ | ||||||
| import kafka.utils.IntEncoder | import kafka.utils.{IntEncoder, Utils, TestUtils} | ||||||
| import kafka.utils.{Utils, TestUtils} |  | ||||||
| import kafka.zk.ZooKeeperTestHarness | import kafka.zk.ZooKeeperTestHarness | ||||||
| import kafka.common._ | import kafka.common._ | ||||||
| import kafka.producer.{KeyedMessage, Producer} | import kafka.producer.{KeyedMessage, Producer} | ||||||
|  | @ -43,38 +42,48 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { | ||||||
|   overridingProps.put(KafkaConfig.ReplicaFetchWaitMaxMsProp, replicaFetchWaitMaxMs.toString) |   overridingProps.put(KafkaConfig.ReplicaFetchWaitMaxMsProp, replicaFetchWaitMaxMs.toString) | ||||||
|   overridingProps.put(KafkaConfig.ReplicaFetchMinBytesProp, replicaFetchMinBytes.toString) |   overridingProps.put(KafkaConfig.ReplicaFetchMinBytesProp, replicaFetchMinBytes.toString) | ||||||
| 
 | 
 | ||||||
|   val configs = TestUtils.createBrokerConfigs(2, false).map(KafkaConfig.fromProps(_, overridingProps)) |   var configs: Seq[KafkaConfig] = null | ||||||
|   val topic = "new-topic" |   val topic = "new-topic" | ||||||
|   val partitionId = 0 |   val partitionId = 0 | ||||||
| 
 | 
 | ||||||
|   var server1: KafkaServer = null |   var server1: KafkaServer = null | ||||||
|   var server2: KafkaServer = null |   var server2: KafkaServer = null | ||||||
| 
 | 
 | ||||||
|   val configProps1 = configs.head |   def configProps1 = configs.head | ||||||
|   val configProps2 = configs.last |   def configProps2 = configs.last | ||||||
| 
 | 
 | ||||||
|   val message = "hello" |   val message = "hello" | ||||||
| 
 | 
 | ||||||
|   var producer: Producer[Int, String] = null |   var producer: Producer[Int, String] = null | ||||||
|   var hwFile1: OffsetCheckpoint = new OffsetCheckpoint(new File(configProps1.logDirs(0), ReplicaManager.HighWatermarkFilename)) |   def hwFile1: OffsetCheckpoint = new OffsetCheckpoint(new File(configProps1.logDirs(0), ReplicaManager.HighWatermarkFilename)) | ||||||
|   var hwFile2: OffsetCheckpoint = new OffsetCheckpoint(new File(configProps2.logDirs(0), ReplicaManager.HighWatermarkFilename)) |   def hwFile2: OffsetCheckpoint = new OffsetCheckpoint(new File(configProps2.logDirs(0), ReplicaManager.HighWatermarkFilename)) | ||||||
|   var servers: Seq[KafkaServer] = Seq.empty[KafkaServer] |   var servers: Seq[KafkaServer] = Seq.empty[KafkaServer] | ||||||
| 
 | 
 | ||||||
|  |   // Some tests restart the brokers then produce more data. But since test brokers use random ports, we need | ||||||
|  |   // to use a new producer that knows the new ports | ||||||
|  |   def updateProducer() = { | ||||||
|  |     if (producer != null) | ||||||
|  |       producer.close() | ||||||
|  |     producer = TestUtils.createProducer[Int, String](TestUtils.getBrokerListStrFromServers(servers), | ||||||
|  |       encoder = classOf[StringEncoder].getName, | ||||||
|  |       keyEncoder = classOf[IntEncoder].getName) | ||||||
|  |   } | ||||||
|  | 
 | ||||||
|   override def setUp() { |   override def setUp() { | ||||||
|     super.setUp() |     super.setUp() | ||||||
| 
 | 
 | ||||||
|  |     configs = TestUtils.createBrokerConfigs(2, zkConnect, false).map(KafkaConfig.fromProps(_, overridingProps)) | ||||||
|  | 
 | ||||||
|     // start both servers |     // start both servers | ||||||
|     server1 = TestUtils.createServer(configProps1) |     server1 = TestUtils.createServer(configProps1) | ||||||
|     server2 = TestUtils.createServer(configProps2) |     server2 = TestUtils.createServer(configProps2) | ||||||
|     servers ++= List(server1, server2) |     servers = List(server1, server2) | ||||||
| 
 | 
 | ||||||
|     // create topic with 1 partition, 2 replicas, one on each broker |     // create topic with 1 partition, 2 replicas, one on each broker | ||||||
|     createTopic(zkClient, topic, partitionReplicaAssignment = Map(0->Seq(0,1)), servers = servers) |     createTopic(zkClient, topic, partitionReplicaAssignment = Map(0->Seq(0,1)), servers = servers) | ||||||
| 
 | 
 | ||||||
|     // create the producer |     // create the producer | ||||||
|     producer = TestUtils.createProducer[Int, String](TestUtils.getBrokerListStrFromConfigs(configs), |     updateProducer() | ||||||
|       encoder = classOf[StringEncoder].getName, |  | ||||||
|       keyEncoder = classOf[IntEncoder].getName) |  | ||||||
|   } |   } | ||||||
| 
 | 
 | ||||||
|   override def tearDown() { |   override def tearDown() { | ||||||
|  | @ -121,6 +130,8 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { | ||||||
| 
 | 
 | ||||||
|     // bring the preferred replica back |     // bring the preferred replica back | ||||||
|     server1.startup() |     server1.startup() | ||||||
|  |     // Update producer with new server settings | ||||||
|  |     updateProducer() | ||||||
| 
 | 
 | ||||||
|     leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId) |     leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId) | ||||||
|     assertTrue("Leader must remain on broker 1, in case of zookeeper session expiration it can move to broker 0", |     assertTrue("Leader must remain on broker 1, in case of zookeeper session expiration it can move to broker 0", | ||||||
|  | @ -132,6 +143,7 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { | ||||||
|     assertEquals(hw, hwFile2.read.getOrElse(TopicAndPartition(topic, 0), 0L)) |     assertEquals(hw, hwFile2.read.getOrElse(TopicAndPartition(topic, 0), 0L)) | ||||||
| 
 | 
 | ||||||
|     server2.startup() |     server2.startup() | ||||||
|  |     updateProducer() | ||||||
|     leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, oldLeaderOpt = leader) |     leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, oldLeaderOpt = leader) | ||||||
|     assertTrue("Leader must remain on broker 0, in case of zookeeper session expiration it can move to broker 1", |     assertTrue("Leader must remain on broker 0, in case of zookeeper session expiration it can move to broker 1", | ||||||
|       leader.isDefined && (leader.get == 0 || leader.get == 1)) |       leader.isDefined && (leader.get == 0 || leader.get == 1)) | ||||||
|  | @ -181,6 +193,7 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { | ||||||
|     assertEquals(hw, hwFile2.read.getOrElse(TopicAndPartition(topic, 0), 0L)) |     assertEquals(hw, hwFile2.read.getOrElse(TopicAndPartition(topic, 0), 0L)) | ||||||
| 
 | 
 | ||||||
|     server2.startup() |     server2.startup() | ||||||
|  |     updateProducer() | ||||||
|     // check if leader moves to the other server |     // check if leader moves to the other server | ||||||
|     leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, oldLeaderOpt = leader) |     leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, oldLeaderOpt = leader) | ||||||
|     assertEquals("Leader must move to broker 1", 1, leader.getOrElse(-1)) |     assertEquals("Leader must move to broker 1", 1, leader.getOrElse(-1)) | ||||||
|  | @ -189,6 +202,7 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { | ||||||
| 
 | 
 | ||||||
|     // bring the preferred replica back |     // bring the preferred replica back | ||||||
|     server1.startup() |     server1.startup() | ||||||
|  |     updateProducer() | ||||||
| 
 | 
 | ||||||
|     assertEquals(hw, hwFile1.read.getOrElse(TopicAndPartition(topic, 0), 0L)) |     assertEquals(hw, hwFile1.read.getOrElse(TopicAndPartition(topic, 0), 0L)) | ||||||
|     assertEquals(hw, hwFile2.read.getOrElse(TopicAndPartition(topic, 0), 0L)) |     assertEquals(hw, hwFile2.read.getOrElse(TopicAndPartition(topic, 0), 0L)) | ||||||
|  |  | ||||||
|  | @ -37,7 +37,6 @@ import junit.framework.Assert._ | ||||||
| 
 | 
 | ||||||
| class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { | class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { | ||||||
|   val random: Random = new Random() |   val random: Random = new Random() | ||||||
|   val brokerPort: Int = 9099 |  | ||||||
|   val group = "test-group" |   val group = "test-group" | ||||||
|   val retentionCheckInterval: Long = 100L |   val retentionCheckInterval: Long = 100L | ||||||
|   var logDir: File = null |   var logDir: File = null | ||||||
|  | @ -50,14 +49,14 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { | ||||||
|   @Before |   @Before | ||||||
|   override def setUp() { |   override def setUp() { | ||||||
|     super.setUp() |     super.setUp() | ||||||
|     val config: Properties = createBrokerConfig(1, brokerPort) |     val config: Properties = createBrokerConfig(1, zkConnect) | ||||||
|     config.setProperty(KafkaConfig.OffsetsTopicReplicationFactorProp, "1") |     config.setProperty(KafkaConfig.OffsetsTopicReplicationFactorProp, "1") | ||||||
|     config.setProperty(KafkaConfig.OffsetsRetentionCheckIntervalMsProp, retentionCheckInterval.toString) |     config.setProperty(KafkaConfig.OffsetsRetentionCheckIntervalMsProp, retentionCheckInterval.toString) | ||||||
|     val logDirPath = config.getProperty("log.dir") |     val logDirPath = config.getProperty("log.dir") | ||||||
|     logDir = new File(logDirPath) |     logDir = new File(logDirPath) | ||||||
|     time = new MockTime() |     time = new MockTime() | ||||||
|     server = TestUtils.createServer(KafkaConfig.fromProps(config), time) |     server = TestUtils.createServer(KafkaConfig.fromProps(config), time) | ||||||
|     simpleConsumer = new SimpleConsumer("localhost", brokerPort, 1000000, 64*1024, "test-client") |     simpleConsumer = new SimpleConsumer("localhost", server.boundPort(), 1000000, 64*1024, "test-client") | ||||||
|     val consumerMetadataRequest = ConsumerMetadataRequest(group) |     val consumerMetadataRequest = ConsumerMetadataRequest(group) | ||||||
|     Stream.continually { |     Stream.continually { | ||||||
|       val consumerMetadataResponse = simpleConsumer.send(consumerMetadataRequest) |       val consumerMetadataResponse = simpleConsumer.send(consumerMetadataRequest) | ||||||
|  |  | ||||||
|  | @ -22,20 +22,19 @@ import kafka.zk.ZooKeeperTestHarness | ||||||
| import kafka.utils.TestUtils._ | import kafka.utils.TestUtils._ | ||||||
| import kafka.producer.KeyedMessage | import kafka.producer.KeyedMessage | ||||||
| import kafka.serializer.StringEncoder | import kafka.serializer.StringEncoder | ||||||
| import kafka.utils.TestUtils | import kafka.utils.{TestUtils} | ||||||
| import junit.framework.Assert._ |  | ||||||
| import kafka.common._ | import kafka.common._ | ||||||
| 
 | 
 | ||||||
| class ReplicaFetchTest extends JUnit3Suite with ZooKeeperTestHarness  { | class ReplicaFetchTest extends JUnit3Suite with ZooKeeperTestHarness  { | ||||||
|   val props = createBrokerConfigs(2,false) |  | ||||||
|   val configs = props.map(p => KafkaConfig.fromProps(p)) |  | ||||||
|   var brokers: Seq[KafkaServer] = null |   var brokers: Seq[KafkaServer] = null | ||||||
|   val topic1 = "foo" |   val topic1 = "foo" | ||||||
|   val topic2 = "bar" |   val topic2 = "bar" | ||||||
| 
 | 
 | ||||||
|   override def setUp() { |   override def setUp() { | ||||||
|     super.setUp() |     super.setUp() | ||||||
|     brokers = configs.map(config => TestUtils.createServer(config)) |     brokers = createBrokerConfigs(2, zkConnect, false) | ||||||
|  |       .map(KafkaConfig.fromProps) | ||||||
|  |       .map(config => TestUtils.createServer(config)) | ||||||
|   } |   } | ||||||
| 
 | 
 | ||||||
|   override def tearDown() { |   override def tearDown() { | ||||||
|  | @ -54,7 +53,7 @@ class ReplicaFetchTest extends JUnit3Suite with ZooKeeperTestHarness  { | ||||||
|     } |     } | ||||||
| 
 | 
 | ||||||
|     // send test messages to leader |     // send test messages to leader | ||||||
|     val producer = TestUtils.createProducer[String, String](TestUtils.getBrokerListStrFromConfigs(configs), |     val producer = TestUtils.createProducer[String, String](TestUtils.getBrokerListStrFromServers(brokers), | ||||||
|                                                             encoder = classOf[StringEncoder].getName, |                                                             encoder = classOf[StringEncoder].getName, | ||||||
|                                                             keyEncoder = classOf[StringEncoder].getName) |                                                             keyEncoder = classOf[StringEncoder].getName) | ||||||
|     val messages = testMessageList1.map(m => new KeyedMessage(topic1, m, m)) ++ testMessageList2.map(m => new KeyedMessage(topic2, m, m)) |     val messages = testMessageList1.map(m => new KeyedMessage(topic1, m, m)) ++ testMessageList2.map(m => new KeyedMessage(topic2, m, m)) | ||||||
|  |  | ||||||
|  | @ -38,7 +38,7 @@ class ReplicaManagerTest extends JUnit3Suite { | ||||||
| 
 | 
 | ||||||
|   @Test |   @Test | ||||||
|   def testHighWaterMarkDirectoryMapping() { |   def testHighWaterMarkDirectoryMapping() { | ||||||
|     val props = TestUtils.createBrokerConfig(1) |     val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect) | ||||||
|     val config = KafkaConfig.fromProps(props) |     val config = KafkaConfig.fromProps(props) | ||||||
|     val zkClient = EasyMock.createMock(classOf[ZkClient]) |     val zkClient = EasyMock.createMock(classOf[ZkClient]) | ||||||
|     val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray) |     val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray) | ||||||
|  | @ -54,7 +54,7 @@ class ReplicaManagerTest extends JUnit3Suite { | ||||||
| 
 | 
 | ||||||
|   @Test |   @Test | ||||||
|   def testHighwaterMarkRelativeDirectoryMapping() { |   def testHighwaterMarkRelativeDirectoryMapping() { | ||||||
|     val props = TestUtils.createBrokerConfig(1) |     val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect) | ||||||
|     props.put("log.dir", TestUtils.tempRelativeDir("data").getAbsolutePath) |     props.put("log.dir", TestUtils.tempRelativeDir("data").getAbsolutePath) | ||||||
|     val config = KafkaConfig.fromProps(props) |     val config = KafkaConfig.fromProps(props) | ||||||
|     val zkClient = EasyMock.createMock(classOf[ZkClient]) |     val zkClient = EasyMock.createMock(classOf[ZkClient]) | ||||||
|  | @ -71,7 +71,7 @@ class ReplicaManagerTest extends JUnit3Suite { | ||||||
| 
 | 
 | ||||||
|   @Test |   @Test | ||||||
|   def testIllegalRequiredAcks() { |   def testIllegalRequiredAcks() { | ||||||
|     val props = TestUtils.createBrokerConfig(1) |     val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect) | ||||||
|     val config = KafkaConfig.fromProps(props) |     val config = KafkaConfig.fromProps(props) | ||||||
|     val zkClient = EasyMock.createMock(classOf[ZkClient]) |     val zkClient = EasyMock.createMock(classOf[ZkClient]) | ||||||
|     val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray) |     val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray) | ||||||
|  |  | ||||||
|  | @ -16,6 +16,8 @@ | ||||||
|   */ |   */ | ||||||
| package kafka.server | package kafka.server | ||||||
| 
 | 
 | ||||||
|  | import java.util.Properties | ||||||
|  | 
 | ||||||
| import kafka.zk.ZooKeeperTestHarness | import kafka.zk.ZooKeeperTestHarness | ||||||
| import kafka.utils.{TestUtils, Utils} | import kafka.utils.{TestUtils, Utils} | ||||||
| import org.junit.Test | import org.junit.Test | ||||||
|  | @ -24,12 +26,19 @@ import junit.framework.Assert._ | ||||||
| import java.io.File | import java.io.File | ||||||
| 
 | 
 | ||||||
| class ServerGenerateBrokerIdTest extends JUnit3Suite with ZooKeeperTestHarness { | class ServerGenerateBrokerIdTest extends JUnit3Suite with ZooKeeperTestHarness { | ||||||
|   var props1 = TestUtils.createBrokerConfig(-1, TestUtils.choosePort) |   var props1: Properties = null | ||||||
|   var config1 = KafkaConfig.fromProps(props1) |   var config1: KafkaConfig = null | ||||||
|   var props2 = TestUtils.createBrokerConfig(0, TestUtils.choosePort) |   var props2: Properties = null | ||||||
|   var config2 = KafkaConfig.fromProps(props2) |   var config2: KafkaConfig = null | ||||||
|   val brokerMetaPropsFile = "meta.properties" |   val brokerMetaPropsFile = "meta.properties" | ||||||
| 
 | 
 | ||||||
|  |   override def setUp() { | ||||||
|  |     super.setUp() | ||||||
|  |     props1 = TestUtils.createBrokerConfig(-1, zkConnect) | ||||||
|  |     config1 = KafkaConfig.fromProps(props1) | ||||||
|  |     props2 = TestUtils.createBrokerConfig(0, zkConnect) | ||||||
|  |     config2 = KafkaConfig.fromProps(props2) | ||||||
|  |   } | ||||||
| 
 | 
 | ||||||
|   @Test |   @Test | ||||||
|   def testAutoGenerateBrokerId() { |   def testAutoGenerateBrokerId() { | ||||||
|  | @ -51,7 +60,7 @@ class ServerGenerateBrokerIdTest extends JUnit3Suite with ZooKeeperTestHarness { | ||||||
|     // start the server with broker.id as part of config |     // start the server with broker.id as part of config | ||||||
|     val server1 = new KafkaServer(config1) |     val server1 = new KafkaServer(config1) | ||||||
|     val server2 = new KafkaServer(config2) |     val server2 = new KafkaServer(config2) | ||||||
|     val props3 = TestUtils.createBrokerConfig(-1, TestUtils.choosePort) |     val props3 = TestUtils.createBrokerConfig(-1, zkConnect) | ||||||
|     val config3 = KafkaConfig.fromProps(props3) |     val config3 = KafkaConfig.fromProps(props3) | ||||||
|     val server3 = new KafkaServer(config3) |     val server3 = new KafkaServer(config3) | ||||||
|     server1.startup() |     server1.startup() | ||||||
|  |  | ||||||
|  | @ -32,20 +32,23 @@ import org.scalatest.junit.JUnit3Suite | ||||||
| import junit.framework.Assert._ | import junit.framework.Assert._ | ||||||
| 
 | 
 | ||||||
| class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness { | class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness { | ||||||
|   val port = TestUtils.choosePort |   var config: KafkaConfig = null | ||||||
|   val props = TestUtils.createBrokerConfig(0, port) |  | ||||||
|   val config = KafkaConfig.fromProps(props) |  | ||||||
| 
 |  | ||||||
|   val host = "localhost" |   val host = "localhost" | ||||||
|   val topic = "test" |   val topic = "test" | ||||||
|   val sent1 = List("hello", "there") |   val sent1 = List("hello", "there") | ||||||
|   val sent2 = List("more", "messages") |   val sent2 = List("more", "messages") | ||||||
| 
 | 
 | ||||||
|  |   override def setUp(): Unit = { | ||||||
|  |     super.setUp() | ||||||
|  |     val props = TestUtils.createBrokerConfig(0, zkConnect) | ||||||
|  |     config = KafkaConfig.fromProps(props) | ||||||
|  |   } | ||||||
|  | 
 | ||||||
|   @Test |   @Test | ||||||
|   def testCleanShutdown() { |   def testCleanShutdown() { | ||||||
|     var server = new KafkaServer(config) |     var server = new KafkaServer(config) | ||||||
|     server.startup() |     server.startup() | ||||||
|     var producer = TestUtils.createProducer[Int, String](TestUtils.getBrokerListStrFromConfigs(Seq(config)), |     var producer = TestUtils.createProducer[Int, String](TestUtils.getBrokerListStrFromServers(Seq(server)), | ||||||
|       encoder = classOf[StringEncoder].getName, |       encoder = classOf[StringEncoder].getName, | ||||||
|       keyEncoder = classOf[IntEncoder].getName) |       keyEncoder = classOf[IntEncoder].getName) | ||||||
| 
 | 
 | ||||||
|  | @ -71,10 +74,10 @@ class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness { | ||||||
|     // wait for the broker to receive the update metadata request after startup |     // wait for the broker to receive the update metadata request after startup | ||||||
|     TestUtils.waitUntilMetadataIsPropagated(Seq(server), topic, 0) |     TestUtils.waitUntilMetadataIsPropagated(Seq(server), topic, 0) | ||||||
| 
 | 
 | ||||||
|     producer = TestUtils.createProducer[Int, String](TestUtils.getBrokerListStrFromConfigs(Seq(config)), |     producer = TestUtils.createProducer[Int, String](TestUtils.getBrokerListStrFromServers(Seq(server)), | ||||||
|       encoder = classOf[StringEncoder].getName, |       encoder = classOf[StringEncoder].getName, | ||||||
|       keyEncoder = classOf[IntEncoder].getName) |       keyEncoder = classOf[IntEncoder].getName) | ||||||
|     val consumer = new SimpleConsumer(host, port, 1000000, 64*1024, "") |     val consumer = new SimpleConsumer(host, server.boundPort(), 1000000, 64*1024, "") | ||||||
| 
 | 
 | ||||||
|     var fetchedMessage: ByteBufferMessageSet = null |     var fetchedMessage: ByteBufferMessageSet = null | ||||||
|     while(fetchedMessage == null || fetchedMessage.validBytes == 0) { |     while(fetchedMessage == null || fetchedMessage.validBytes == 0) { | ||||||
|  | @ -103,7 +106,7 @@ class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness { | ||||||
| 
 | 
 | ||||||
|   @Test |   @Test | ||||||
|   def testCleanShutdownWithDeleteTopicEnabled() { |   def testCleanShutdownWithDeleteTopicEnabled() { | ||||||
|     val newProps = TestUtils.createBrokerConfig(0, port) |     val newProps = TestUtils.createBrokerConfig(0, zkConnect) | ||||||
|     newProps.setProperty("delete.topic.enable", "true") |     newProps.setProperty("delete.topic.enable", "true") | ||||||
|     val newConfig = KafkaConfig.fromProps(newProps) |     val newConfig = KafkaConfig.fromProps(newProps) | ||||||
|     val server = new KafkaServer(newConfig) |     val server = new KafkaServer(newConfig) | ||||||
|  | @ -116,7 +119,7 @@ class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness { | ||||||
| 
 | 
 | ||||||
|   @Test |   @Test | ||||||
|   def testCleanShutdownAfterFailedStartup() { |   def testCleanShutdownAfterFailedStartup() { | ||||||
|     val newProps = TestUtils.createBrokerConfig(0, port) |     val newProps = TestUtils.createBrokerConfig(0, zkConnect) | ||||||
|     newProps.setProperty("zookeeper.connect", "fakehostthatwontresolve:65535") |     newProps.setProperty("zookeeper.connect", "fakehostthatwontresolve:65535") | ||||||
|     val newConfig = KafkaConfig.fromProps(newProps) |     val newConfig = KafkaConfig.fromProps(newProps) | ||||||
|     val server = new KafkaServer(newConfig) |     val server = new KafkaServer(newConfig) | ||||||
|  |  | ||||||
|  | @ -30,7 +30,7 @@ class ServerStartupTest extends JUnit3Suite with ZooKeeperTestHarness { | ||||||
|   def testBrokerCreatesZKChroot { |   def testBrokerCreatesZKChroot { | ||||||
|     val brokerId = 0 |     val brokerId = 0 | ||||||
|     val zookeeperChroot = "/kafka-chroot-for-unittest" |     val zookeeperChroot = "/kafka-chroot-for-unittest" | ||||||
|     val props = TestUtils.createBrokerConfig(brokerId, TestUtils.choosePort()) |     val props = TestUtils.createBrokerConfig(brokerId, zkConnect) | ||||||
|     val zooKeeperConnect = props.get("zookeeper.connect") |     val zooKeeperConnect = props.get("zookeeper.connect") | ||||||
|     props.put("zookeeper.connect", zooKeeperConnect + zookeeperChroot) |     props.put("zookeeper.connect", zooKeeperConnect + zookeeperChroot) | ||||||
|     val server = TestUtils.createServer(KafkaConfig.fromProps(props)) |     val server = TestUtils.createServer(KafkaConfig.fromProps(props)) | ||||||
|  | @ -47,11 +47,11 @@ class ServerStartupTest extends JUnit3Suite with ZooKeeperTestHarness { | ||||||
|     // This shouldn't affect the existing broker registration. |     // This shouldn't affect the existing broker registration. | ||||||
| 
 | 
 | ||||||
|     val brokerId = 0 |     val brokerId = 0 | ||||||
|     val props1 = TestUtils.createBrokerConfig(brokerId) |     val props1 = TestUtils.createBrokerConfig(brokerId, zkConnect) | ||||||
|     val server1 = TestUtils.createServer(KafkaConfig.fromProps(props1)) |     val server1 = TestUtils.createServer(KafkaConfig.fromProps(props1)) | ||||||
|     val brokerRegistration = ZkUtils.readData(zkClient, ZkUtils.BrokerIdsPath + "/" + brokerId)._1 |     val brokerRegistration = ZkUtils.readData(zkClient, ZkUtils.BrokerIdsPath + "/" + brokerId)._1 | ||||||
| 
 | 
 | ||||||
|     val props2 = TestUtils.createBrokerConfig(brokerId) |     val props2 = TestUtils.createBrokerConfig(brokerId, zkConnect) | ||||||
|     try { |     try { | ||||||
|       TestUtils.createServer(KafkaConfig.fromProps(props2)) |       TestUtils.createServer(KafkaConfig.fromProps(props2)) | ||||||
|       fail("Registering a broker with a conflicting id should fail") |       fail("Registering a broker with a conflicting id should fail") | ||||||
|  |  | ||||||
|  | @ -43,7 +43,7 @@ class SimpleFetchTest extends JUnit3Suite { | ||||||
|   overridingProps.put(KafkaConfig.ReplicaLagTimeMaxMsProp, replicaLagTimeMaxMs.toString) |   overridingProps.put(KafkaConfig.ReplicaLagTimeMaxMsProp, replicaLagTimeMaxMs.toString) | ||||||
|   overridingProps.put(KafkaConfig.ReplicaFetchWaitMaxMsProp, replicaFetchWaitMaxMs.toString) |   overridingProps.put(KafkaConfig.ReplicaFetchWaitMaxMsProp, replicaFetchWaitMaxMs.toString) | ||||||
| 
 | 
 | ||||||
|   val configs = TestUtils.createBrokerConfigs(2).map(KafkaConfig.fromProps(_, overridingProps)) |   val configs = TestUtils.createBrokerConfigs(2, TestUtils.MockZkConnect).map(KafkaConfig.fromProps(_, overridingProps)) | ||||||
| 
 | 
 | ||||||
|   // set the replica manager with the partition |   // set the replica manager with the partition | ||||||
|   val time = new MockTime |   val time = new MockTime | ||||||
|  |  | ||||||
|  | @ -53,7 +53,7 @@ class ReplicationUtilsTest extends JUnit3Suite with ZooKeeperTestHarness { | ||||||
| 
 | 
 | ||||||
|   @Test |   @Test | ||||||
|   def testUpdateLeaderAndIsr() { |   def testUpdateLeaderAndIsr() { | ||||||
|     val configs = TestUtils.createBrokerConfigs(1).map(KafkaConfig.fromProps) |     val configs = TestUtils.createBrokerConfigs(1, zkConnect).map(KafkaConfig.fromProps) | ||||||
|     val log = EasyMock.createMock(classOf[kafka.log.Log]) |     val log = EasyMock.createMock(classOf[kafka.log.Log]) | ||||||
|     EasyMock.expect(log.logEndOffset).andReturn(20).anyTimes() |     EasyMock.expect(log.logEndOffset).andReturn(20).anyTimes() | ||||||
|     EasyMock.expect(log) |     EasyMock.expect(log) | ||||||
|  |  | ||||||
|  | @ -65,23 +65,13 @@ object TestUtils extends Logging { | ||||||
|   val seededRandom = new Random(192348092834L) |   val seededRandom = new Random(192348092834L) | ||||||
|   val random = new Random() |   val random = new Random() | ||||||
| 
 | 
 | ||||||
|   /** |   /* 0 gives a random port; you can then retrieve the assigned port from the Socket object. */ | ||||||
|    * Choose a number of random available ports |   val RandomPort = 0 | ||||||
|    */ |  | ||||||
|   def choosePorts(count: Int): List[Int] = { |  | ||||||
|     val sockets = |  | ||||||
|       for(i <- 0 until count) |  | ||||||
|       yield new ServerSocket(0) |  | ||||||
|     val socketList = sockets.toList |  | ||||||
|     val ports = socketList.map(_.getLocalPort) |  | ||||||
|     socketList.map(_.close) |  | ||||||
|     ports |  | ||||||
|   } |  | ||||||
| 
 | 
 | ||||||
|   /** |   /** Port to use for unit tests that mock/don't require a real ZK server. */ | ||||||
|    * Choose an available port |   val MockZkPort = 1 | ||||||
|    */ |   /** Zookeeper connection string to use for unit tests that mock/don't require a real ZK server. */ | ||||||
|   def choosePort(): Int = choosePorts(1).head |   val MockZkConnect = "127.0.0.1:" + MockZkPort | ||||||
| 
 | 
 | ||||||
|   /** |   /** | ||||||
|    * Create a temporary directory |    * Create a temporary directory | ||||||
|  | @ -141,28 +131,29 @@ object TestUtils extends Logging { | ||||||
|    * Create a test config for the given node id |    * Create a test config for the given node id | ||||||
|    */ |    */ | ||||||
|   def createBrokerConfigs(numConfigs: Int, |   def createBrokerConfigs(numConfigs: Int, | ||||||
|  |     zkConnect: String, | ||||||
|     enableControlledShutdown: Boolean = true, |     enableControlledShutdown: Boolean = true, | ||||||
|     enableDeleteTopic: Boolean = false): List[Properties] = { |     enableDeleteTopic: Boolean = false): Seq[Properties] = { | ||||||
|     for((port, node) <- choosePorts(numConfigs).zipWithIndex) |     (0 until numConfigs).map(node => createBrokerConfig(node, zkConnect, enableControlledShutdown, enableDeleteTopic)) | ||||||
|     yield createBrokerConfig(node, port, enableControlledShutdown, enableDeleteTopic) |  | ||||||
|   } |   } | ||||||
| 
 | 
 | ||||||
|   def getBrokerListStrFromConfigs(configs: Seq[KafkaConfig]): String = { |   def getBrokerListStrFromServers(servers: Seq[KafkaServer]): String = { | ||||||
|     configs.map(c => formatAddress(c.hostName, c.port)).mkString(",") |     servers.map(s => formatAddress(s.config.hostName, s.boundPort())).mkString(",") | ||||||
|   } |   } | ||||||
| 
 | 
 | ||||||
|   /** |   /** | ||||||
|    * Create a test config for the given node id |    * Create a test config for the given node id | ||||||
|    */ |    */ | ||||||
|   def createBrokerConfig(nodeId: Int, port: Int = choosePort(), |   def createBrokerConfig(nodeId: Int, zkConnect: String, | ||||||
|     enableControlledShutdown: Boolean = true, |     enableControlledShutdown: Boolean = true, | ||||||
|     enableDeleteTopic: Boolean = false): Properties = { |     enableDeleteTopic: Boolean = false, | ||||||
|  |     port: Int = RandomPort): Properties = { | ||||||
|     val props = new Properties |     val props = new Properties | ||||||
|     if (nodeId >= 0) props.put("broker.id", nodeId.toString) |     if (nodeId >= 0) props.put("broker.id", nodeId.toString) | ||||||
|     props.put("host.name", "localhost") |     props.put("host.name", "localhost") | ||||||
|     props.put("port", port.toString) |     props.put("port", port.toString) | ||||||
|     props.put("log.dir", TestUtils.tempDir().getAbsolutePath) |     props.put("log.dir", TestUtils.tempDir().getAbsolutePath) | ||||||
|     props.put("zookeeper.connect", TestZKUtils.zookeeperConnect) |     props.put("zookeeper.connect", zkConnect) | ||||||
|     props.put("replica.socket.timeout.ms", "1500") |     props.put("replica.socket.timeout.ms", "1500") | ||||||
|     props.put("controller.socket.timeout.ms", "1500") |     props.put("controller.socket.timeout.ms", "1500") | ||||||
|     props.put("controlled.shutdown.enable", enableControlledShutdown.toString) |     props.put("controlled.shutdown.enable", enableControlledShutdown.toString) | ||||||
|  | @ -756,7 +747,7 @@ object TestUtils extends Logging { | ||||||
|                    brokerState = new BrokerState()) |                    brokerState = new BrokerState()) | ||||||
|   } |   } | ||||||
| 
 | 
 | ||||||
|   def sendMessagesToPartition(configs: Seq[KafkaConfig], |   def sendMessagesToPartition(servers: Seq[KafkaServer], | ||||||
|                               topic: String, |                               topic: String, | ||||||
|                               partition: Int, |                               partition: Int, | ||||||
|                               numMessages: Int, |                               numMessages: Int, | ||||||
|  | @ -765,7 +756,7 @@ object TestUtils extends Logging { | ||||||
|     val props = new Properties() |     val props = new Properties() | ||||||
|     props.put("compression.codec", compression.codec.toString) |     props.put("compression.codec", compression.codec.toString) | ||||||
|     val producer: Producer[Int, String] = |     val producer: Producer[Int, String] = | ||||||
|       createProducer(TestUtils.getBrokerListStrFromConfigs(configs), |       createProducer(TestUtils.getBrokerListStrFromServers(servers), | ||||||
|         encoder = classOf[StringEncoder].getName, |         encoder = classOf[StringEncoder].getName, | ||||||
|         keyEncoder = classOf[IntEncoder].getName, |         keyEncoder = classOf[IntEncoder].getName, | ||||||
|         partitioner = classOf[FixedValuePartitioner].getName, |         partitioner = classOf[FixedValuePartitioner].getName, | ||||||
|  | @ -778,7 +769,7 @@ object TestUtils extends Logging { | ||||||
|     ms.toList |     ms.toList | ||||||
|   } |   } | ||||||
| 
 | 
 | ||||||
|   def sendMessages(configs: Seq[KafkaConfig], |   def sendMessages(servers: Seq[KafkaServer], | ||||||
|                    topic: String, |                    topic: String, | ||||||
|                    producerId: String, |                    producerId: String, | ||||||
|                    messagesPerNode: Int, |                    messagesPerNode: Int, | ||||||
|  | @ -790,7 +781,7 @@ object TestUtils extends Logging { | ||||||
|     props.put("compression.codec", compression.codec.toString) |     props.put("compression.codec", compression.codec.toString) | ||||||
|     props.put("client.id", producerId) |     props.put("client.id", producerId) | ||||||
|     val   producer: Producer[Int, String] = |     val   producer: Producer[Int, String] = | ||||||
|       createProducer(brokerList = TestUtils.getBrokerListStrFromConfigs(configs), |       createProducer(brokerList = TestUtils.getBrokerListStrFromServers(servers), | ||||||
|         encoder = classOf[StringEncoder].getName, |         encoder = classOf[StringEncoder].getName, | ||||||
|         keyEncoder = classOf[IntEncoder].getName, |         keyEncoder = classOf[IntEncoder].getName, | ||||||
|         partitioner = classOf[FixedValuePartitioner].getName, |         partitioner = classOf[FixedValuePartitioner].getName, | ||||||
|  | @ -848,10 +839,6 @@ object TestUtils extends Logging { | ||||||
| 
 | 
 | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| object TestZKUtils { |  | ||||||
|   val zookeeperConnect = "127.0.0.1:" + TestUtils.choosePort() |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| class IntEncoder(props: VerifiableProperties = null) extends Encoder[Int] { | class IntEncoder(props: VerifiableProperties = null) extends Encoder[Int] { | ||||||
|   override def toBytes(n: Int) = n.toString.getBytes |   override def toBytes(n: Int) = n.toString.getBytes | ||||||
| } | } | ||||||
|  |  | ||||||
|  | @ -24,14 +24,16 @@ import java.net.InetSocketAddress | ||||||
| import kafka.utils.Utils | import kafka.utils.Utils | ||||||
| import org.apache.kafka.common.utils.Utils.getPort | import org.apache.kafka.common.utils.Utils.getPort | ||||||
| 
 | 
 | ||||||
| class EmbeddedZookeeper(val connectString: String) { | class EmbeddedZookeeper() { | ||||||
|   val snapshotDir = TestUtils.tempDir() |   val snapshotDir = TestUtils.tempDir() | ||||||
|   val logDir = TestUtils.tempDir() |   val logDir = TestUtils.tempDir() | ||||||
|   val tickTime = 500 |   val tickTime = 500 | ||||||
|   val zookeeper = new ZooKeeperServer(snapshotDir, logDir, tickTime) |   val zookeeper = new ZooKeeperServer(snapshotDir, logDir, tickTime) | ||||||
|   val factory = new NIOServerCnxnFactory() |   val factory = new NIOServerCnxnFactory() | ||||||
|   factory.configure(new InetSocketAddress("127.0.0.1", getPort(connectString)), 0) |   private val addr = new InetSocketAddress("127.0.0.1", TestUtils.RandomPort) | ||||||
|  |   factory.configure(addr, 0) | ||||||
|   factory.startup(zookeeper) |   factory.startup(zookeeper) | ||||||
|  |   val port = zookeeper.getClientPort() | ||||||
| 
 | 
 | ||||||
|   def shutdown() { |   def shutdown() { | ||||||
|     Utils.swallow(zookeeper.shutdown()) |     Utils.swallow(zookeeper.shutdown()) | ||||||
|  |  | ||||||
|  | @ -29,7 +29,7 @@ class ZKPathTest extends JUnit3Suite with ZooKeeperTestHarness { | ||||||
| 
 | 
 | ||||||
|   val path: String = "/some_dir" |   val path: String = "/some_dir" | ||||||
|   val zkSessionTimeoutMs = 1000 |   val zkSessionTimeoutMs = 1000 | ||||||
|   val zkConnectWithInvalidRoot: String = zkConnect + "/ghost" |   def zkConnectWithInvalidRoot: String = zkConnect + "/ghost" | ||||||
| 
 | 
 | ||||||
|   def testCreatePersistentPathThrowsException { |   def testCreatePersistentPathThrowsException { | ||||||
|     val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnectWithInvalidRoot, |     val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnectWithInvalidRoot, | ||||||
|  |  | ||||||
|  | @ -19,19 +19,22 @@ package kafka.zk | ||||||
| 
 | 
 | ||||||
| import org.scalatest.junit.JUnit3Suite | import org.scalatest.junit.JUnit3Suite | ||||||
| import org.I0Itec.zkclient.ZkClient | import org.I0Itec.zkclient.ZkClient | ||||||
| import kafka.utils.{ZKStringSerializer, TestZKUtils, Utils} | import kafka.utils.{ZKStringSerializer, Utils} | ||||||
| 
 | 
 | ||||||
| trait ZooKeeperTestHarness extends JUnit3Suite { | trait ZooKeeperTestHarness extends JUnit3Suite { | ||||||
|   val zkConnect: String = TestZKUtils.zookeeperConnect |   var zkPort: Int = -1 | ||||||
|   var zookeeper: EmbeddedZookeeper = null |   var zookeeper: EmbeddedZookeeper = null | ||||||
|   var zkClient: ZkClient = null |   var zkClient: ZkClient = null | ||||||
|   val zkConnectionTimeout = 6000 |   val zkConnectionTimeout = 6000 | ||||||
|   val zkSessionTimeout = 6000 |   val zkSessionTimeout = 6000 | ||||||
| 
 | 
 | ||||||
|  |   def zkConnect: String = "127.0.0.1:" + zkPort | ||||||
|  | 
 | ||||||
|   override def setUp() { |   override def setUp() { | ||||||
|     super.setUp |     super.setUp | ||||||
|     zookeeper = new EmbeddedZookeeper(zkConnect) |     zookeeper = new EmbeddedZookeeper() | ||||||
|     zkClient = new ZkClient(zookeeper.connectString, zkSessionTimeout, zkConnectionTimeout, ZKStringSerializer) |     zkPort = zookeeper.port | ||||||
|  |     zkClient = new ZkClient(zkConnect, zkSessionTimeout, zkConnectionTimeout, ZKStringSerializer) | ||||||
|   } |   } | ||||||
| 
 | 
 | ||||||
|   override def tearDown() { |   override def tearDown() { | ||||||
|  |  | ||||||
		Loading…
	
		Reference in New Issue