From 9f80665ec6deff8525b61096034af8dc0cc9a03c Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 10 Jun 2015 11:28:53 -0700 Subject: [PATCH 01/14] KAFKA-2253; fix deadlock between removeWatchersLock and watcher operations list lock; reviewed by Onur Karaman and Jiangjie Qin --- .../main/scala/kafka/server/DelayedOperation.scala | 45 ++++++++++++++-------- 1 file changed, 29 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/kafka/server/DelayedOperation.scala b/core/src/main/scala/kafka/server/DelayedOperation.scala index 123078d..0b53532 100644 --- a/core/src/main/scala/kafka/server/DelayedOperation.scala +++ b/core/src/main/scala/kafka/server/DelayedOperation.scala @@ -189,8 +189,7 @@ class DelayedOperationPurgatory[T <: DelayedOperation](purgatoryName: String, br // If the operation is already completed, stop adding it to the rest of the watcher list. if (operation.isCompleted()) return false - val watchers = watchersFor(key) - watchers.watch(operation) + watchForOperation(key, operation) if (!watchCreated) { watchCreated = true @@ -241,22 +240,34 @@ class DelayedOperationPurgatory[T <: DelayedOperation](purgatoryName: String, br def delayed() = timeoutTimer.size /* - * Return the watch list of the given key + * Return all the current watcher lists, + * note that the returned watchers may be removed from the list by other threads */ - private def watchersFor(key: Any) = inReadLock(removeWatchersLock) { watchersForKey.getAndMaybePut(key) } + private def allWatchers = inReadLock(removeWatchersLock) { watchersForKey.values } /* - * Return all the current watcher lists + * Return the watch list of the given key, note that we need to + * grab the removeWatchersLock to avoid the operation being added to a removed watcher list */ - private def allWatchers = inReadLock(removeWatchersLock) { watchersForKey.values } + private def watchForOperation(key: Any, operation: T) { + inReadLock(removeWatchersLock) { + val watcher = watchersForKey.getAndMaybePut(key) + watcher.watch(operation) + } + } /* * Remove the key from watcher lists if its list is empty */ - private def removeKeyIfEmpty(key: Any) = inWriteLock(removeWatchersLock) { - val watchers = watchersForKey.get(key) - if (watchers != null && watchers.watched == 0) { - watchersForKey.remove(key) + private def removeKeyIfEmpty(key: Any, watchers: Watchers) { + inWriteLock(removeWatchersLock) { + // if the current key is no longer correlated to the watchers to remove, skip + if (watchersForKey.get(key) != watchers) + return + + if (watchers != null && watchers.watched == 0) { + watchersForKey.remove(key) + } } } @@ -298,10 +309,11 @@ class DelayedOperationPurgatory[T <: DelayedOperation](purgatoryName: String, br iter.remove() } } - - if (operations.size == 0) - removeKeyIfEmpty(key) } + + if (operations.size == 0) + removeKeyIfEmpty(key, this) + completed } @@ -317,10 +329,11 @@ class DelayedOperationPurgatory[T <: DelayedOperation](purgatoryName: String, br purged += 1 } } - - if (operations.size == 0) - removeKeyIfEmpty(key) } + + if (operations.size == 0) + removeKeyIfEmpty(key, this) + purged } } -- 2.4.3 From ab76dbd1f7f0bfdc9841baee64cfa37de1d870fb Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Thu, 11 Jun 2015 15:27:51 -0700 Subject: [PATCH 02/14] kafka-2005; Generate html report for system tests; patched by Ashish Singh; reviewed by Jun Rao --- system_test/system_test_runner.py | 135 ++++++++++++++++++++++++++++++++++++-- 1 file changed, 131 insertions(+), 4 deletions(-) diff --git a/system_test/system_test_runner.py b/system_test/system_test_runner.py index 5078d44..d6251b2 100644 --- a/system_test/system_test_runner.py +++ b/system_test/system_test_runner.py @@ -52,6 +52,131 @@ logging.config.fileConfig('logging.conf') # message to facilitate debugging. d = {'name_of_class': '(system_test_runner)'} +class report: + systemTestEnv = None + reportString = "" + reportFileName = "system_test_report.html" + systemTestReport = None + header = """ + Kafka System Test Report + + + + + + """ + footer = """ """ + + def __init__(self, systemTestEnv): + self.totalSkipped = 0 + self.totalPassed = 0 + self.totalTests = 0 + self.totalFailed = 0 + self.systemTestEnv = systemTestEnv + self.systemTestReport = open(self.reportFileName, 'w') + + def __del__(self): + self.systemTestReport.close() + self.systemTestReport = None + + def writeHtmlPage(self, body): + html = """ + + + """ + html += self.header + html += body + html += self.footer + html += """ + + """ + self.systemTestReport.write(html) + + def wrapIn(self, tag, content): + html = "\n<" + tag + ">" + html += "\n " + content + html += "\n" + return html + + def genModal(self, className, caseName, systemTestResult): + key = "validation_status" + id = className + "_" + caseName + info = self.wrapIn("h4", "Validation Status") + for validatedItem in sorted(systemTestResult[key].iterkeys()): + testItemStatus = systemTestResult[key][validatedItem] + info += validatedItem + " : " + testItemStatus + return self.wrapIn("div class=\"modal fade\" id=\"" + id + "\" tabindex=\"-1\" role=\"dialog\" aria-labelledby=\"" + id + "Label\" aria-hidden=\"true\"", + self.wrapIn("div class=\"modal-dialog\"", + self.wrapIn("div class=\"modal-content\"", + self.wrapIn("div class=\"modal-header\"", + self.wrapIn("h4 class=\"modal-title\" id=\"" + id + "Label\"", + className + " - " + caseName)) + + self.wrapIn("div class=\"modal-body\"", + info) + + self.wrapIn("div class=\"modal-footer\"", + self.wrapIn("button type=\"button\" class=\"btn btn-default\" data-dismiss=\"modal\"", "Close"))))) + + def summarize(self): + testItemsTableHeader = self.wrapIn("thead", + self.wrapIn("tr", + self.wrapIn("th", "Test Class Name") + + self.wrapIn("th", "Test Case Name") + + self.wrapIn("th", "Validation Status"))) + testItemsTableBody = "" + modals = "" + + for systemTestResult in self.systemTestEnv.systemTestResultsList: + self.totalTests += 1 + if "_test_class_name" in systemTestResult: + testClassName = systemTestResult["_test_class_name"] + else: + testClassName = "" + + if "_test_case_name" in systemTestResult: + testCaseName = systemTestResult["_test_case_name"] + else: + testCaseName = "" + + if "validation_status" in systemTestResult: + testItemStatus = "SKIPPED" + for key in systemTestResult["validation_status"].iterkeys(): + testItemStatus = systemTestResult["validation_status"][key] + if "FAILED" == testItemStatus: + break; + if "FAILED" == testItemStatus: + self.totalFailed += 1 + validationStatus = self.wrapIn("div class=\"text-danger\" data-toggle=\"modal\" data-target=\"#" + testClassName + "_" + testCaseName + "\"", "FAILED") + modals += self.genModal(testClassName, testCaseName, systemTestResult) + elif "PASSED" == testItemStatus: + self.totalPassed += 1 + validationStatus = self.wrapIn("div class=\"text-success\"", "PASSED") + else: + self.totalSkipped += 1 + validationStatus = self.wrapIn("div class=\"text-warning\"", "SKIPPED") + else: + self.reportString += "|" + + testItemsTableBody += self.wrapIn("tr", + self.wrapIn("td", testClassName) + + self.wrapIn("td", testCaseName) + + self.wrapIn("td", validationStatus)) + + testItemsTableBody = self.wrapIn("tbody", testItemsTableBody) + testItemsTable = self.wrapIn("table class=\"table table-striped\"", testItemsTableHeader + testItemsTableBody) + + statsTblBody = self.wrapIn("tr class=\"active\"", self.wrapIn("td", "Total tests") + self.wrapIn("td", str(self.totalTests))) + statsTblBody += self.wrapIn("tr class=\"success\"", self.wrapIn("td", "Total tests passed") + self.wrapIn("td", str(self.totalPassed))) + statsTblBody += self.wrapIn("tr class=\"danger\"", self.wrapIn("td", "Total tests failed") + self.wrapIn("td", str(self.totalFailed))) + statsTblBody += self.wrapIn("tr class=\"warning\"", self.wrapIn("td", "Total tests skipped") + self.wrapIn("td", str(self.totalSkipped))) + testStatsTable = self.wrapIn("table class=\"table\"", statsTblBody) + + body = self.wrapIn("div class=\"container\"", + self.wrapIn("h2", "Kafka System Test Report") + + self.wrapIn("div class=\"row\"", self.wrapIn("div class=\"col-md-4\"", testStatsTable)) + + self.wrapIn("div class=\"row\"", self.wrapIn("div class=\"col-md-6\"", testItemsTable)) + + modals) + self.writeHtmlPage(self.wrapIn("body", body)) + def main(): nLogger = logging.getLogger('namedLogger') aLogger = logging.getLogger('anonymousLogger') @@ -98,7 +223,7 @@ def main(): nLogger.info("SKIPPING : checking remote machines", extra=d) print - # get all defined names within a module: + # get all defined names within a module: definedItemList = dir(SystemTestEnv) aLogger.debug("=================================================") aLogger.debug("SystemTestEnv keys:") @@ -129,7 +254,7 @@ def main(): and not moduleFileName.startswith("__"): # found a test module file - nLogger.info("found a test module file : " + moduleFileName, extra=d) + nLogger.info("found a test module file : " + moduleFileName, extra=d) testModuleClassName = system_test_utils.sys_call("grep ^class " + testModulePathName + "/" + \ moduleFileName + " | sed 's/^class //g' | sed 's/(.*):.*//g'") @@ -138,7 +263,7 @@ def main(): # collect the test suite class data testSuiteClassDict = {} testSuiteClassDict["suite"] = dirName - extLenToRemove = systemTestEnv.SYSTEM_TEST_MODULE_EXT.__len__() * -1 + extLenToRemove = systemTestEnv.SYSTEM_TEST_MODULE_EXT.__len__() * -1 testSuiteClassDict["module"] = moduleFileName[:extLenToRemove] testSuiteClassDict["class"] = testModuleClassName testSuiteClassDictList.append(testSuiteClassDict) @@ -149,7 +274,7 @@ def main(): # add testsuite directory to sys.path such that the module can be loaded sys.path.append(systemTestEnv.SYSTEM_TEST_BASE_DIR + "/" + suiteName) - + if not systemTestEnv.printTestDescriptionsOnly: aLogger.info("=================================================") aLogger.info("Running Test for : ") @@ -165,6 +290,8 @@ def main(): instance.runTest() print + report(systemTestEnv).summarize() + if not systemTestEnv.printTestDescriptionsOnly: totalFailureCount = 0 print -- 2.4.3 From 017c00caf44aaad3418cb99d3ef42c4d1b066ddd Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Fri, 12 Jun 2015 10:24:54 -0700 Subject: [PATCH 03/14] kafka-2266; Client Selector can drop idle connections without notifying NetworkClient; patched by Jason Gustafson; reviewed by Jun Rao --- .../java/org/apache/kafka/common/network/Selector.java | 2 ++ .../org/apache/kafka/common/network/SelectorTest.java | 17 ++++++++++++++++- 2 files changed, 18 insertions(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/common/network/Selector.java b/clients/src/main/java/org/apache/kafka/common/network/Selector.java index effb1e6..1da215b 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/Selector.java +++ b/clients/src/main/java/org/apache/kafka/common/network/Selector.java @@ -393,6 +393,8 @@ public class Selector implements Selectable { if (log.isTraceEnabled()) log.trace("About to close the idle connection from " + connectionId + " due to being idle for " + (currentTimeNanos - connectionLastActiveTime) / 1000 / 1000 + " millis"); + + disconnected.add(connectionId); close(connectionId); } } diff --git a/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java b/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java index d23b4b6..158f982 100644 --- a/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java @@ -26,6 +26,7 @@ import java.util.*; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.test.TestUtils; import org.junit.After; @@ -40,13 +41,15 @@ public class SelectorTest { private static final int BUFFER_SIZE = 4 * 1024; private EchoServer server; + private Time time; private Selectable selector; @Before public void setup() throws Exception { this.server = new EchoServer(); this.server.start(); - this.selector = new Selector(5000, new Metrics(), new MockTime() , "MetricGroup", new LinkedHashMap()); + this.time = new MockTime(); + this.selector = new Selector(5000, new Metrics(), time, "MetricGroup", new LinkedHashMap()); } @After @@ -244,6 +247,18 @@ public class SelectorTest { assertEquals("The response should be from the previously muted node", "1", selector.completedReceives().get(0).source()); } + + @Test + public void testCloseOldestConnection() throws Exception { + String id = "0"; + blockingConnect(id); + + time.sleep(6000); // The max idle time is 5000ms + selector.poll(0); + + assertTrue("The idle connection should have been closed", selector.disconnected().contains(id)); + } + private String blockingRequest(String node, String s) throws IOException { selector.send(createSend(node, s)); selector.poll(1000L); -- 2.4.3 From d31a2c2381bebc9c4b27e36fdf986183732e13eb Mon Sep 17 00:00:00 2001 From: Alexander Pakulov Date: Fri, 12 Jun 2015 14:16:03 -0700 Subject: [PATCH 04/14] kafka-2232; make MockProducer generic; patched by Alexander Pakulov; reviewed by Jun Rao --- .../kafka/clients/producer/MockProducer.java | 53 +++++++++++++--------- .../kafka/clients/producer/MockProducerTest.java | 31 +++++++++++-- .../java/org/apache/kafka/test/MockSerializer.java | 1 - 3 files changed, 58 insertions(+), 27 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java index e66491c..36e7ffa 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java @@ -30,6 +30,7 @@ import org.apache.kafka.clients.producer.internals.FutureRecordMetadata; import org.apache.kafka.clients.producer.internals.DefaultPartitioner; import org.apache.kafka.clients.producer.internals.ProduceRequestResult; import org.apache.kafka.common.*; +import org.apache.kafka.common.serialization.Serializer; /** @@ -38,14 +39,16 @@ import org.apache.kafka.common.*; * By default this mock will synchronously complete each send call successfully. However it can be configured to allow * the user to control the completion of the call and supply an optional error for the producer to throw. */ -public class MockProducer implements Producer { +public class MockProducer implements Producer { private final Cluster cluster; - private final Partitioner partitioner = new DefaultPartitioner(); - private final List> sent; + private final Partitioner partitioner; + private final List> sent; private final Deque completions; private boolean autoComplete; private Map offsets; + private final Serializer keySerializer; + private final Serializer valueSerializer; /** * Create a mock producer @@ -55,31 +58,37 @@ public class MockProducer implements Producer { * the user must call {@link #completeNext()} or {@link #errorNext(RuntimeException)} after * {@link #send(ProducerRecord) send()} to complete the call and unblock the @{link * java.util.concurrent.Future Future<RecordMetadata>} that is returned. + * @param partitioner The partition strategy + * @param keySerializer The serializer for key that implements {@link Serializer}. + * @param valueSerializer The serializer for value that implements {@link Serializer}. */ - public MockProducer(Cluster cluster, boolean autoComplete) { + public MockProducer(Cluster cluster, boolean autoComplete, Partitioner partitioner, Serializer keySerializer, Serializer valueSerializer) { this.cluster = cluster; this.autoComplete = autoComplete; + this.partitioner = partitioner; + this.keySerializer = keySerializer; + this.valueSerializer = valueSerializer; this.offsets = new HashMap(); - this.sent = new ArrayList>(); + this.sent = new ArrayList>(); this.completions = new ArrayDeque(); } /** - * Create a new mock producer with invented metadata the given autoComplete setting. + * Create a new mock producer with invented metadata the given autoComplete setting and key\value serializers * - * Equivalent to {@link #MockProducer(Cluster, boolean) new MockProducer(null, autoComplete)} + * Equivalent to {@link #MockProducer(Cluster, boolean, Partitioner, Serializer, Serializer)} new MockProducer(Cluster.empty(), autoComplete, new DefaultPartitioner(), keySerializer, valueSerializer)} */ - public MockProducer(boolean autoComplete) { - this(Cluster.empty(), autoComplete); + public MockProducer(boolean autoComplete, Serializer keySerializer, Serializer valueSerializer) { + this(Cluster.empty(), autoComplete, new DefaultPartitioner(), keySerializer, valueSerializer); } /** - * Create a new auto completing mock producer + * Create a new mock producer with invented metadata the given autoComplete setting, partitioner and key\value serializers * - * Equivalent to {@link #MockProducer(boolean) new MockProducer(true)} + * Equivalent to {@link #MockProducer(Cluster, boolean, Partitioner, Serializer, Serializer)} new MockProducer(Cluster.empty(), autoComplete, partitioner, keySerializer, valueSerializer)} */ - public MockProducer() { - this(true); + public MockProducer(boolean autoComplete, Partitioner partitioner, Serializer keySerializer, Serializer valueSerializer) { + this(Cluster.empty(), autoComplete, partitioner, keySerializer, valueSerializer); } /** @@ -88,7 +97,7 @@ public class MockProducer implements Producer { * @see #history() */ @Override - public synchronized Future send(ProducerRecord record) { + public synchronized Future send(ProducerRecord record) { return send(record, null); } @@ -98,7 +107,7 @@ public class MockProducer implements Producer { * @see #history() */ @Override - public synchronized Future send(ProducerRecord record, Callback callback) { + public synchronized Future send(ProducerRecord record, Callback callback) { int partition = 0; if (this.cluster.partitionsForTopic(record.topic()) != null) partition = partition(record, this.cluster); @@ -154,8 +163,8 @@ public class MockProducer implements Producer { /** * Get the list of sent records since the last call to {@link #clear()} */ - public synchronized List> history() { - return new ArrayList>(this.sent); + public synchronized List> history() { + return new ArrayList>(this.sent); } /** @@ -193,10 +202,11 @@ public class MockProducer implements Producer { /** * computes partition for given record. */ - private int partition(ProducerRecord record, Cluster cluster) { + private int partition(ProducerRecord record, Cluster cluster) { Integer partition = record.partition(); + String topic = record.topic(); if (partition != null) { - List partitions = cluster.partitionsForTopic(record.topic()); + List partitions = cluster.partitionsForTopic(topic); int numPartitions = partitions.size(); // they have given us a partition, use it if (partition < 0 || partition >= numPartitions) @@ -206,10 +216,11 @@ public class MockProducer implements Producer { + "]."); return partition; } - return this.partitioner.partition(record.topic(), null, record.key(), null, record.value(), cluster); + byte[] keyBytes = keySerializer.serialize(topic, record.key()); + byte[] valueBytes = valueSerializer.serialize(topic, record.value()); + return this.partitioner.partition(topic, record.key(), keyBytes, record.value(), valueBytes, cluster); } - private static class Completion { private final long offset; private final RecordMetadata metadata; diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java index 6372f1a..7a46c56 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java @@ -17,14 +17,22 @@ package org.apache.kafka.clients.producer; import static java.util.Arrays.asList; +import static java.util.Collections.singletonList; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import java.util.ArrayList; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; +import org.apache.kafka.clients.producer.internals.DefaultPartitioner; +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.test.MockSerializer; import org.junit.Test; public class MockProducerTest { @@ -34,23 +42,36 @@ public class MockProducerTest { @Test @SuppressWarnings("unchecked") public void testAutoCompleteMock() throws Exception { - MockProducer producer = new MockProducer(true); + MockProducer producer = new MockProducer(true, new MockSerializer(), new MockSerializer()); ProducerRecord record = new ProducerRecord(topic, "key".getBytes(), "value".getBytes()); Future metadata = producer.send(record); assertTrue("Send should be immediately complete", metadata.isDone()); assertFalse("Send should be successful", isError(metadata)); assertEquals("Offset should be 0", 0L, metadata.get().offset()); assertEquals(topic, metadata.get().topic()); - assertEquals("We should have the record in our history", asList(record), producer.history()); + assertEquals("We should have the record in our history", singletonList(record), producer.history()); + producer.clear(); + assertEquals("Clear should erase our history", 0, producer.history().size()); + } + + @Test + public void testPartitioner() throws Exception { + PartitionInfo partitionInfo0 = new PartitionInfo(topic, 0, null, null, null); + PartitionInfo partitionInfo1 = new PartitionInfo(topic, 1, null, null, null); + Cluster cluster = new Cluster(new ArrayList(0), asList(partitionInfo0, partitionInfo1)); + MockProducer producer = new MockProducer(cluster, true, new DefaultPartitioner(), new StringSerializer(), new StringSerializer()); + ProducerRecord record = new ProducerRecord(topic, "key", "value"); + Future metadata = producer.send(record); + assertEquals("Partition should be correct", 1, metadata.get().partition()); producer.clear(); assertEquals("Clear should erase our history", 0, producer.history().size()); } @Test public void testManualCompletion() throws Exception { - MockProducer producer = new MockProducer(false); - ProducerRecord record1 = new ProducerRecord("topic", "key1".getBytes(), "value1".getBytes()); - ProducerRecord record2 = new ProducerRecord("topic", "key2".getBytes(), "value2".getBytes()); + MockProducer producer = new MockProducer(false, new MockSerializer(), new MockSerializer()); + ProducerRecord record1 = new ProducerRecord(topic, "key1".getBytes(), "value1".getBytes()); + ProducerRecord record2 = new ProducerRecord(topic, "key2".getBytes(), "value2".getBytes()); Future md1 = producer.send(record1); assertFalse("Send shouldn't have completed", md1.isDone()); Future md2 = producer.send(record2); diff --git a/clients/src/test/java/org/apache/kafka/test/MockSerializer.java b/clients/src/test/java/org/apache/kafka/test/MockSerializer.java index e75d2e4..0348258 100644 --- a/clients/src/test/java/org/apache/kafka/test/MockSerializer.java +++ b/clients/src/test/java/org/apache/kafka/test/MockSerializer.java @@ -31,7 +31,6 @@ public class MockSerializer implements Serializer { @Override public void configure(Map configs, boolean isKey) { - } @Override -- 2.4.3 From 719f2bddd147d4583a15ee8398bb0edccbbcc3f4 Mon Sep 17 00:00:00 2001 From: Alexey Ozeritskiy Date: Fri, 12 Jun 2015 18:45:48 -0700 Subject: [PATCH 05/14] kafka-2164; ReplicaFetcherThread: suspicious log message on reset offset; patched by Alexey Ozeritski; reviewed by Jun Rao --- core/src/main/scala/kafka/server/ReplicaFetcherThread.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index b31b432..181cbc1 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -111,9 +111,9 @@ class ReplicaFetcherThread(name:String, * Roll out a new log at the follower with the start offset equal to the current leader's start offset and continue fetching. */ val leaderStartOffset = simpleConsumer.earliestOrLatestOffset(topicAndPartition, OffsetRequest.EarliestTime, brokerConfig.brokerId) - replicaMgr.logManager.truncateFullyAndStartAt(topicAndPartition, leaderStartOffset) warn("Replica %d for partition %s reset its fetch offset from %d to current leader %d's start offset %d" .format(brokerConfig.brokerId, topicAndPartition, replica.logEndOffset.messageOffset, sourceBroker.id, leaderStartOffset)) + replicaMgr.logManager.truncateFullyAndStartAt(topicAndPartition, leaderStartOffset) leaderStartOffset } } -- 2.4.3 From 20a31a29f7aa6ce6687a13aa0cf60b92c5ac4d1e Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Mon, 15 Jun 2015 17:43:56 -0700 Subject: [PATCH 06/14] kafka-2101; Metric metadata-age is reset on a failed update; patched by Tim Brooks; reviewed by Jun Rao --- .../src/main/java/org/apache/kafka/clients/Metadata.java | 11 +++++++---- .../apache/kafka/clients/producer/internals/Sender.java | 2 +- .../test/java/org/apache/kafka/clients/MetadataTest.java | 14 ++++++++++++++ 3 files changed, 22 insertions(+), 5 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/Metadata.java b/clients/src/main/java/org/apache/kafka/clients/Metadata.java index 07f1cdb..0387f26 100644 --- a/clients/src/main/java/org/apache/kafka/clients/Metadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/Metadata.java @@ -36,6 +36,7 @@ public final class Metadata { private final long metadataExpireMs; private int version; private long lastRefreshMs; + private long lastSuccessfulRefreshMs; private Cluster cluster; private boolean needUpdate; private final Set topics; @@ -57,6 +58,7 @@ public final class Metadata { this.refreshBackoffMs = refreshBackoffMs; this.metadataExpireMs = metadataExpireMs; this.lastRefreshMs = 0L; + this.lastSuccessfulRefreshMs = 0L; this.version = 0; this.cluster = Cluster.empty(); this.needUpdate = false; @@ -83,7 +85,7 @@ public final class Metadata { * is now */ public synchronized long timeToNextUpdate(long nowMs) { - long timeToExpire = needUpdate ? 0 : Math.max(this.lastRefreshMs + this.metadataExpireMs - nowMs, 0); + long timeToExpire = needUpdate ? 0 : Math.max(this.lastSuccessfulRefreshMs + this.metadataExpireMs - nowMs, 0); long timeToAllowUpdate = this.lastRefreshMs + this.refreshBackoffMs - nowMs; return Math.max(timeToExpire, timeToAllowUpdate); } @@ -146,6 +148,7 @@ public final class Metadata { public synchronized void update(Cluster cluster, long now) { this.needUpdate = false; this.lastRefreshMs = now; + this.lastSuccessfulRefreshMs = now; this.version += 1; this.cluster = cluster; notifyAll(); @@ -168,10 +171,10 @@ public final class Metadata { } /** - * The last time metadata was updated. + * The last time metadata was successfully updated. */ - public synchronized long lastUpdate() { - return this.lastRefreshMs; + public synchronized long lastSuccessfulUpdate() { + return this.lastSuccessfulRefreshMs; } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index 07e65d4..0baf16e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -410,7 +410,7 @@ public class Sender implements Runnable { m = new MetricName("metadata-age", metricGrpName, "The age in seconds of the current producer metadata being used.", metricTags); metrics.addMetric(m, new Measurable() { public double measure(MetricConfig config, long now) { - return (now - metadata.lastUpdate()) / 1000.0; + return (now - metadata.lastSuccessfulUpdate()) / 1000.0; } }); } diff --git a/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java b/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java index 928087d..249d6b8 100644 --- a/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java @@ -85,6 +85,20 @@ public class MetadataTest { } } + @Test + public void testFailedUpdate() { + long time = 100; + metadata.update(Cluster.empty(), time); + + assertEquals(100, metadata.timeToNextUpdate(1000)); + metadata.failedUpdate(1100); + + assertEquals(100, metadata.timeToNextUpdate(1100)); + assertEquals(100, metadata.lastSuccessfulUpdate()); + + } + + private Thread asyncFetch(final String topic) { Thread thread = new Thread() { public void run() { -- 2.4.3 From 54e54f08077c9d71a5121e640b55836e6f7f2c9b Mon Sep 17 00:00:00 2001 From: Andrii Biletskyi Date: Tue, 16 Jun 2015 14:46:48 -0700 Subject: [PATCH 07/14] kafka-2195; Add versionId to AbstractRequest.getErrorResponse and AbstractRequest.getRequest; patched by Andrii Biletskyi; reviewed by Jun Rao --- .../kafka/common/requests/AbstractRequest.java | 24 +++---- .../common/requests/ConsumerMetadataRequest.java | 14 ++++- .../apache/kafka/common/requests/FetchRequest.java | 18 ++++-- .../kafka/common/requests/HeartbeatRequest.java | 20 ++++-- .../kafka/common/requests/JoinGroupRequest.java | 26 ++++++-- .../kafka/common/requests/JoinGroupResponse.java | 4 -- .../kafka/common/requests/ListOffsetRequest.java | 14 ++++- .../kafka/common/requests/MetadataRequest.java | 21 ++++++- .../kafka/common/requests/MetadataResponse.java | 73 ++++++++++------------ .../kafka/common/requests/OffsetCommitRequest.java | 14 ++++- .../kafka/common/requests/OffsetFetchRequest.java | 20 ++++-- .../kafka/common/requests/ProduceRequest.java | 17 +++-- .../kafka/common/requests/RequestResponseTest.java | 23 ++++--- .../main/scala/kafka/network/RequestChannel.scala | 2 +- core/src/main/scala/kafka/server/KafkaApis.scala | 2 +- 15 files changed, 192 insertions(+), 100 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java index 5e5308e..5d3d528 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java @@ -28,33 +28,33 @@ public abstract class AbstractRequest extends AbstractRequestResponse { } /** - * Get an error response for a request + * Get an error response for a request for a given api version */ - public abstract AbstractRequestResponse getErrorResponse(Throwable e); + public abstract AbstractRequestResponse getErrorResponse(int versionId, Throwable e); /** * Factory method for getting a request object based on ApiKey ID and a buffer */ - public static AbstractRequest getRequest(int requestId, ByteBuffer buffer) { + public static AbstractRequest getRequest(int requestId, int versionId, ByteBuffer buffer) { switch (ApiKeys.forId(requestId)) { case PRODUCE: - return ProduceRequest.parse(buffer); + return ProduceRequest.parse(buffer, versionId); case FETCH: - return FetchRequest.parse(buffer); + return FetchRequest.parse(buffer, versionId); case LIST_OFFSETS: - return ListOffsetRequest.parse(buffer); + return ListOffsetRequest.parse(buffer, versionId); case METADATA: - return MetadataRequest.parse(buffer); + return MetadataRequest.parse(buffer, versionId); case OFFSET_COMMIT: - return OffsetCommitRequest.parse(buffer); + return OffsetCommitRequest.parse(buffer, versionId); case OFFSET_FETCH: - return OffsetFetchRequest.parse(buffer); + return OffsetFetchRequest.parse(buffer, versionId); case CONSUMER_METADATA: - return ConsumerMetadataRequest.parse(buffer); + return ConsumerMetadataRequest.parse(buffer, versionId); case JOIN_GROUP: - return JoinGroupRequest.parse(buffer); + return JoinGroupRequest.parse(buffer, versionId); case HEARTBEAT: - return HeartbeatRequest.parse(buffer); + return HeartbeatRequest.parse(buffer, versionId); default: return null; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java index 04b90bf..fef22d7 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java @@ -41,14 +41,24 @@ public class ConsumerMetadataRequest extends AbstractRequest { } @Override - public AbstractRequestResponse getErrorResponse(Throwable e) { - return new ConsumerMetadataResponse(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code(), Node.noNode()); + public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { + switch(versionId) { + case 0: + return new ConsumerMetadataResponse(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code(), Node.noNode()); + default: + throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", + versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.CONSUMER_METADATA.id))); + } } public String groupId() { return groupId; } + public static ConsumerMetadataRequest parse(ByteBuffer buffer, int versionId) { + return new ConsumerMetadataRequest(ProtoUtils.parseRequest(ApiKeys.CONSUMER_METADATA.id, versionId, buffer)); + } + public static ConsumerMetadataRequest parse(ByteBuffer buffer) { return new ConsumerMetadataRequest((Struct) CURRENT_SCHEMA.read(buffer)); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java index 8686d83..4f52c32 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java @@ -120,17 +120,23 @@ public class FetchRequest extends AbstractRequest { } @Override - public AbstractRequestResponse getErrorResponse(Throwable e) { + public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { Map responseData = new HashMap(); for (Map.Entry entry: fetchData.entrySet()) { FetchResponse.PartitionData partitionResponse = new FetchResponse.PartitionData(Errors.forException(e).code(), - FetchResponse.INVALID_HIGHWATERMARK, - FetchResponse.EMPTY_RECORD_SET); + FetchResponse.INVALID_HIGHWATERMARK, + FetchResponse.EMPTY_RECORD_SET); responseData.put(entry.getKey(), partitionResponse); } - return new FetchResponse(responseData); + switch(versionId) { + case 0: + return new FetchResponse(responseData); + default: + throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", + versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.FETCH.id))); + } } public int replicaId() { @@ -149,6 +155,10 @@ public class FetchRequest extends AbstractRequest { return fetchData; } + public static FetchRequest parse(ByteBuffer buffer, int versionId) { + return new FetchRequest(ProtoUtils.parseRequest(ApiKeys.FETCH.id, versionId, buffer)); + } + public static FetchRequest parse(ByteBuffer buffer) { return new FetchRequest((Struct) CURRENT_SCHEMA.read(buffer)); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java index 51d081f..d4d4a35 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java @@ -48,6 +48,17 @@ public class HeartbeatRequest extends AbstractRequest { consumerId = struct.getString(CONSUMER_ID_KEY_NAME); } + @Override + public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { + switch(versionId) { + case 0: + return new HeartbeatResponse(Errors.forException(e).code()); + default: + throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", + versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.HEARTBEAT.id))); + } + } + public String groupId() { return groupId; } @@ -60,12 +71,11 @@ public class HeartbeatRequest extends AbstractRequest { return consumerId; } - public static HeartbeatRequest parse(ByteBuffer buffer) { - return new HeartbeatRequest((Struct) CURRENT_SCHEMA.read(buffer)); + public static HeartbeatRequest parse(ByteBuffer buffer, int versionId) { + return new HeartbeatRequest(ProtoUtils.parseRequest(ApiKeys.HEARTBEAT.id, versionId, buffer)); } - @Override - public AbstractRequestResponse getErrorResponse(Throwable e) { - return new HeartbeatResponse(Errors.forException(e).code()); + public static HeartbeatRequest parse(ByteBuffer buffer) { + return new HeartbeatRequest((Struct) CURRENT_SCHEMA.read(buffer)); } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java index 6795682..1ffe076 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java @@ -12,6 +12,7 @@ */ package org.apache.kafka.common.requests; +import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; @@ -20,6 +21,7 @@ import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Collections; import java.util.List; public class JoinGroupRequest extends AbstractRequest { @@ -65,6 +67,21 @@ public class JoinGroupRequest extends AbstractRequest { strategy = struct.getString(STRATEGY_KEY_NAME); } + @Override + public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { + switch (versionId) { + case 0: + return new JoinGroupResponse( + Errors.forException(e).code(), + JoinGroupResponse.UNKNOWN_GENERATION_ID, + JoinGroupResponse.UNKNOWN_CONSUMER_ID, + Collections.emptyList()); + default: + throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", + versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.JOIN_GROUP.id))); + } + } + public String groupId() { return groupId; } @@ -85,12 +102,11 @@ public class JoinGroupRequest extends AbstractRequest { return strategy; } - public static JoinGroupRequest parse(ByteBuffer buffer) { - return new JoinGroupRequest((Struct) CURRENT_SCHEMA.read(buffer)); + public static JoinGroupRequest parse(ByteBuffer buffer, int versionId) { + return new JoinGroupRequest(ProtoUtils.parseRequest(ApiKeys.JOIN_GROUP.id, versionId, buffer)); } - @Override - public AbstractRequestResponse getErrorResponse(Throwable e) { - return new JoinGroupResponse(Errors.forException(e).code()); + public static JoinGroupRequest parse(ByteBuffer buffer) { + return new JoinGroupRequest((Struct) CURRENT_SCHEMA.read(buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java index 8d418cd..7bf544e 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java @@ -75,10 +75,6 @@ public class JoinGroupResponse extends AbstractRequestResponse { this.assignedPartitions = assignedPartitions; } - public JoinGroupResponse(short errorCode) { - this(errorCode, UNKNOWN_GENERATION_ID, UNKNOWN_CONSUMER_ID, Collections.emptyList()); - } - public JoinGroupResponse(Struct struct) { super(struct); assignedPartitions = new ArrayList(); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java index 19267ee..a1bb2b2 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java @@ -107,7 +107,7 @@ public class ListOffsetRequest extends AbstractRequest { } @Override - public AbstractRequestResponse getErrorResponse(Throwable e) { + public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { Map responseData = new HashMap(); for (Map.Entry entry: offsetData.entrySet()) { @@ -115,7 +115,13 @@ public class ListOffsetRequest extends AbstractRequest { responseData.put(entry.getKey(), partitionResponse); } - return new ListOffsetResponse(responseData); + switch(versionId) { + case 0: + return new ListOffsetResponse(responseData); + default: + throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", + versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.LIST_OFFSETS.id))); + } } public int replicaId() { @@ -126,6 +132,10 @@ public class ListOffsetRequest extends AbstractRequest { return offsetData; } + public static ListOffsetRequest parse(ByteBuffer buffer, int versionId) { + return new ListOffsetRequest(ProtoUtils.parseRequest(ApiKeys.LIST_OFFSETS.id, versionId, buffer)); + } + public static ListOffsetRequest parse(ByteBuffer buffer) { return new ListOffsetRequest((Struct) CURRENT_SCHEMA.read(buffer)); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java index 7e0ce15..f70e8da 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java @@ -18,6 +18,9 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; @@ -47,18 +50,30 @@ public class MetadataRequest extends AbstractRequest { } @Override - public AbstractRequestResponse getErrorResponse(Throwable e) { + public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { Map topicErrors = new HashMap(); - for (String topic: topics) { + for (String topic : topics) { topicErrors.put(topic, Errors.forException(e)); } - return new MetadataResponse(topicErrors); + + Cluster cluster = new Cluster(new ArrayList(), new ArrayList()); + switch (versionId) { + case 0: + return new MetadataResponse(cluster, topicErrors); + default: + throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", + versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.METADATA.id))); + } } public List topics() { return topics; } + public static MetadataRequest parse(ByteBuffer buffer, int versionId) { + return new MetadataRequest(ProtoUtils.parseRequest(ApiKeys.METADATA.id, versionId, buffer)); + } + public static MetadataRequest parse(ByteBuffer buffer) { return new MetadataRequest((Struct) CURRENT_SCHEMA.read(buffer)); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java index 44e2ce6..c8f2d08 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java @@ -28,7 +28,7 @@ import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; public class MetadataResponse extends AbstractRequestResponse { - + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.METADATA.id); private static final String BROKERS_KEY_NAME = "brokers"; private static final String TOPIC_METATDATA_KEY_NAME = "topic_metadata"; @@ -67,30 +67,15 @@ public class MetadataResponse extends AbstractRequestResponse { private final Cluster cluster; private final Map errors; - /* Constructor for error responses where most of the data, except error per topic, is irrelevant */ - public MetadataResponse(Map topicErrors) { - super(new Struct(CURRENT_SCHEMA)); - - struct.set(BROKERS_KEY_NAME, new ArrayList().toArray()); - List topicArray = new ArrayList(); - for (Map.Entry topicError : topicErrors.entrySet()) { - Struct topicData = struct.instance(TOPIC_METATDATA_KEY_NAME); - topicData.set(TOPIC_ERROR_CODE_KEY_NAME, topicError.getValue().code()); - topicData.set(TOPIC_KEY_NAME, topicError.getKey()); - topicData.set(PARTITION_METADATA_KEY_NAME, new ArrayList().toArray()); - topicArray.add(topicData); - } - struct.set(TOPIC_METATDATA_KEY_NAME, topicArray.toArray()); - - this.errors = topicErrors; - this.cluster = new Cluster(new ArrayList(), new ArrayList()); - } - - public MetadataResponse(Cluster cluster) { + /** + * Constructor for MetadataResponse where there are errors for some of the topics, + * error data take precedence over cluster information for particular topic + */ + public MetadataResponse(Cluster cluster, Map errors) { super(new Struct(CURRENT_SCHEMA)); List brokerArray = new ArrayList(); - for (Node node: cluster.nodes()) { + for (Node node : cluster.nodes()) { Struct broker = struct.instance(BROKERS_KEY_NAME); broker.set(NODE_ID_KEY_NAME, node.id()); broker.set(HOST_KEY_NAME, node.host()); @@ -100,27 +85,33 @@ public class MetadataResponse extends AbstractRequestResponse { struct.set(BROKERS_KEY_NAME, brokerArray.toArray()); List topicArray = new ArrayList(); - for (String topic: cluster.topics()) { + for (String topic : cluster.topics()) { Struct topicData = struct.instance(TOPIC_METATDATA_KEY_NAME); - topicData.set(TOPIC_ERROR_CODE_KEY_NAME, Errors.NONE.code()); + topicData.set(TOPIC_KEY_NAME, topic); - List partitionArray = new ArrayList(); - for (PartitionInfo fetchPartitionData : cluster.partitionsForTopic(topic)) { - Struct partitionData = topicData.instance(PARTITION_METADATA_KEY_NAME); - partitionData.set(PARTITION_ERROR_CODE_KEY_NAME, Errors.NONE.code()); - partitionData.set(PARTITION_KEY_NAME, fetchPartitionData.partition()); - partitionData.set(LEADER_KEY_NAME, fetchPartitionData.leader().id()); - ArrayList replicas = new ArrayList(); - for (Node node: fetchPartitionData.replicas()) - replicas.add(node.id()); - partitionData.set(REPLICAS_KEY_NAME, replicas.toArray()); - ArrayList isr = new ArrayList(); - for (Node node: fetchPartitionData.inSyncReplicas()) - isr.add(node.id()); - partitionData.set(ISR_KEY_NAME, isr.toArray()); - partitionArray.add(partitionData); + if (errors.containsKey(topic)) { + topicData.set(TOPIC_ERROR_CODE_KEY_NAME, errors.get(topic).code()); + } else { + topicData.set(TOPIC_ERROR_CODE_KEY_NAME, Errors.NONE.code()); + List partitionArray = new ArrayList(); + for (PartitionInfo fetchPartitionData : cluster.partitionsForTopic(topic)) { + Struct partitionData = topicData.instance(PARTITION_METADATA_KEY_NAME); + partitionData.set(PARTITION_ERROR_CODE_KEY_NAME, Errors.NONE.code()); + partitionData.set(PARTITION_KEY_NAME, fetchPartitionData.partition()); + partitionData.set(LEADER_KEY_NAME, fetchPartitionData.leader().id()); + ArrayList replicas = new ArrayList(); + for (Node node : fetchPartitionData.replicas()) + replicas.add(node.id()); + partitionData.set(REPLICAS_KEY_NAME, replicas.toArray()); + ArrayList isr = new ArrayList(); + for (Node node : fetchPartitionData.inSyncReplicas()) + isr.add(node.id()); + partitionData.set(ISR_KEY_NAME, isr.toArray()); + partitionArray.add(partitionData); + } + topicData.set(PARTITION_METADATA_KEY_NAME, partitionArray.toArray()); } - topicData.set(PARTITION_METADATA_KEY_NAME, partitionArray.toArray()); + topicArray.add(topicData); } struct.set(TOPIC_METATDATA_KEY_NAME, topicArray.toArray()); @@ -183,4 +174,4 @@ public class MetadataResponse extends AbstractRequestResponse { public static MetadataResponse parse(ByteBuffer buffer) { return new MetadataResponse((Struct) CURRENT_SCHEMA.read(buffer)); } -} +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java index 8bf6cbb..d6e6386 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java @@ -217,12 +217,22 @@ public class OffsetCommitRequest extends AbstractRequest { } @Override - public AbstractRequestResponse getErrorResponse(Throwable e) { + public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { Map responseData = new HashMap(); for (Map.Entry entry: offsetData.entrySet()) { responseData.put(entry.getKey(), Errors.forException(e).code()); } - return new OffsetCommitResponse(responseData); + + switch (versionId) { + // OffsetCommitResponseV0 == OffsetCommitResponseV1 == OffsetCommitResponseV2 + case 0: + case 1: + case 2: + return new OffsetCommitResponse(responseData); + default: + throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", + versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.OFFSET_COMMIT.id))); + } } public String groupId() { diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java index deec1fa..b5e8a0f 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java @@ -88,16 +88,24 @@ public class OffsetFetchRequest extends AbstractRequest { } @Override - public AbstractRequestResponse getErrorResponse(Throwable e) { + public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { Map responseData = new HashMap(); for (TopicPartition partition: partitions) { responseData.put(partition, new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET, - OffsetFetchResponse.NO_METADATA, - Errors.forException(e).code())); + OffsetFetchResponse.NO_METADATA, + Errors.forException(e).code())); } - return new OffsetFetchResponse(responseData); + switch (versionId) { + // OffsetFetchResponseV0 == OffsetFetchResponseV1 + case 0: + case 1: + return new OffsetFetchResponse(responseData); + default: + throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", + versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.OFFSET_FETCH.id))); + } } public String groupId() { @@ -108,6 +116,10 @@ public class OffsetFetchRequest extends AbstractRequest { return partitions; } + public static OffsetFetchRequest parse(ByteBuffer buffer, int versionId) { + return new OffsetFetchRequest(ProtoUtils.parseRequest(ApiKeys.OFFSET_FETCH.id, versionId, buffer)); + } + public static OffsetFetchRequest parse(ByteBuffer buffer) { return new OffsetFetchRequest((Struct) CURRENT_SCHEMA.read(buffer)); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java index fabeae3..715504b 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java @@ -90,19 +90,24 @@ public class ProduceRequest extends AbstractRequest { } @Override - public AbstractRequestResponse getErrorResponse(Throwable e) { - + public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { /* In case the producer doesn't actually want any response */ if (acks == 0) return null; Map responseMap = new HashMap(); - for (Map.Entry entry: partitionRecords.entrySet()) { + for (Map.Entry entry : partitionRecords.entrySet()) { responseMap.put(entry.getKey(), new ProduceResponse.PartitionResponse(Errors.forException(e).code(), ProduceResponse.INVALID_OFFSET)); } - return new ProduceResponse(responseMap); + switch (versionId) { + case 0: + return new ProduceResponse(responseMap); + default: + throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", + versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.PRODUCE.id))); + } } public short acks() { @@ -117,6 +122,10 @@ public class ProduceRequest extends AbstractRequest { return partitionRecords; } + public static ProduceRequest parse(ByteBuffer buffer, int versionId) { + return new ProduceRequest(ProtoUtils.parseRequest(ApiKeys.PRODUCE.id, versionId, buffer)); + } + public static ProduceRequest parse(ByteBuffer buffer) { return new ProduceRequest((Struct) CURRENT_SCHEMA.read(buffer)); } diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index e3cc196..8b2aca8 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -38,31 +38,31 @@ public class RequestResponseTest { createRequestHeader(), createResponseHeader(), createConsumerMetadataRequest(), - createConsumerMetadataRequest().getErrorResponse(new UnknownServerException()), + createConsumerMetadataRequest().getErrorResponse(0, new UnknownServerException()), createConsumerMetadataResponse(), createFetchRequest(), - createFetchRequest().getErrorResponse(new UnknownServerException()), + createFetchRequest().getErrorResponse(0, new UnknownServerException()), createFetchResponse(), createHeartBeatRequest(), - createHeartBeatRequest().getErrorResponse(new UnknownServerException()), + createHeartBeatRequest().getErrorResponse(0, new UnknownServerException()), createHeartBeatResponse(), createJoinGroupRequest(), - createJoinGroupRequest().getErrorResponse(new UnknownServerException()), + createJoinGroupRequest().getErrorResponse(0, new UnknownServerException()), createJoinGroupResponse(), createListOffsetRequest(), - createListOffsetRequest().getErrorResponse(new UnknownServerException()), + createListOffsetRequest().getErrorResponse(0, new UnknownServerException()), createListOffsetResponse(), createMetadataRequest(), - createMetadataRequest().getErrorResponse(new UnknownServerException()), + createMetadataRequest().getErrorResponse(0, new UnknownServerException()), createMetadataResponse(), createOffsetCommitRequest(), - createOffsetCommitRequest().getErrorResponse(new UnknownServerException()), + createOffsetCommitRequest().getErrorResponse(0, new UnknownServerException()), createOffsetCommitResponse(), createOffsetFetchRequest(), - createOffsetFetchRequest().getErrorResponse(new UnknownServerException()), + createOffsetFetchRequest().getErrorResponse(0, new UnknownServerException()), createOffsetFetchResponse(), createProduceRequest(), - createProduceRequest().getErrorResponse(new UnknownServerException()), + createProduceRequest().getErrorResponse(0, new UnknownServerException()), createProduceResponse()); for (AbstractRequestResponse req: requestResponseList) { @@ -145,7 +145,10 @@ public class RequestResponseTest { Node[] isr = new Node[1]; isr[0] = node; Cluster cluster = new Cluster(Arrays.asList(node), Arrays.asList(new PartitionInfo("topic1", 1, node, replicas, isr))); - return new MetadataResponse(cluster); + + Map errors = new HashMap(); + errors.put("topic2", Errors.LEADER_NOT_AVAILABLE); + return new MetadataResponse(cluster, errors); } private AbstractRequest createOffsetCommitRequest() { diff --git a/core/src/main/scala/kafka/network/RequestChannel.scala b/core/src/main/scala/kafka/network/RequestChannel.scala index 357d8b9..2074128 100644 --- a/core/src/main/scala/kafka/network/RequestChannel.scala +++ b/core/src/main/scala/kafka/network/RequestChannel.scala @@ -66,7 +66,7 @@ object RequestChannel extends Logging { null val body: AbstractRequest = if (requestObj == null) - AbstractRequest.getRequest(header.apiKey, buffer) + AbstractRequest.getRequest(header.apiKey, header.apiVersion, buffer) else null diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index d63bc18..c7debe4 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -74,7 +74,7 @@ class KafkaApis(val requestChannel: RequestChannel, if ( request.requestObj != null) request.requestObj.handleError(e, requestChannel, request) else { - val response = request.body.getErrorResponse(e) + val response = request.body.getErrorResponse(request.header.apiVersion, e) val respHeader = new ResponseHeader(request.header.correlationId) /* If request doesn't have a default error response, we just close the connection. -- 2.4.3 From 395716ebf0af2520a33ef352a62c10a71239bb4f Mon Sep 17 00:00:00 2001 From: Proneet Verma Date: Tue, 16 Jun 2015 15:03:40 -0700 Subject: [PATCH 08/14] kafka-2270; incorrect package name in unit tests; patched by Proneet Verma; reviewed by Jun Rao --- core/src/test/scala/unit/kafka/KafkaConfigTest.scala | 3 +-- core/src/test/scala/unit/kafka/common/ConfigTest.scala | 3 +-- core/src/test/scala/unit/kafka/common/TopicTest.scala | 3 +-- .../test/scala/unit/kafka/consumer/PartitionAssignorTest.scala | 10 ++++------ .../test/scala/unit/kafka/integration/MinIsrConfigTest.scala | 3 +-- core/src/test/scala/unit/kafka/log/LogConfigTest.scala | 3 +-- .../scala/unit/kafka/server/KafkaConfigConfigDefTest.scala | 3 +-- .../scala/unit/kafka/utils/ByteBoundedBlockingQueueTest.scala | 5 ++--- .../src/test/scala/unit/kafka/utils/CommandLineUtilsTest.scala | 5 ++--- core/src/test/scala/unit/kafka/zk/ZKPathTest.scala | 4 +--- 10 files changed, 15 insertions(+), 27 deletions(-) diff --git a/core/src/test/scala/unit/kafka/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/KafkaConfigTest.scala index 4cb92de..4764c89 100644 --- a/core/src/test/scala/unit/kafka/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/KafkaConfigTest.scala @@ -14,12 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package unit.kafka +package kafka import java.io.{FileOutputStream, File} import java.security.Permission -import kafka.Kafka import kafka.server.KafkaConfig import org.junit.{After, Before, Test} import junit.framework.Assert._ diff --git a/core/src/test/scala/unit/kafka/common/ConfigTest.scala b/core/src/test/scala/unit/kafka/common/ConfigTest.scala index 7bff96c..0aca938 100644 --- a/core/src/test/scala/unit/kafka/common/ConfigTest.scala +++ b/core/src/test/scala/unit/kafka/common/ConfigTest.scala @@ -15,12 +15,11 @@ * limitations under the License. */ -package unit.kafka.common +package kafka.common import junit.framework.Assert._ import collection.mutable.ArrayBuffer import org.junit.Test -import kafka.common.InvalidConfigException import kafka.producer.ProducerConfig import kafka.consumer.ConsumerConfig diff --git a/core/src/test/scala/unit/kafka/common/TopicTest.scala b/core/src/test/scala/unit/kafka/common/TopicTest.scala index 0fb2588..79532c8 100644 --- a/core/src/test/scala/unit/kafka/common/TopicTest.scala +++ b/core/src/test/scala/unit/kafka/common/TopicTest.scala @@ -15,11 +15,10 @@ * limitations under the License. */ -package unit.kafka.common +package kafka.common import junit.framework.Assert._ import collection.mutable.ArrayBuffer -import kafka.common.{Topic, InvalidTopicException} import org.junit.Test class TopicTest { diff --git a/core/src/test/scala/unit/kafka/consumer/PartitionAssignorTest.scala b/core/src/test/scala/unit/kafka/consumer/PartitionAssignorTest.scala index 1910fcb..adf0801 100644 --- a/core/src/test/scala/unit/kafka/consumer/PartitionAssignorTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/PartitionAssignorTest.scala @@ -15,20 +15,18 @@ * limitations under the License. */ -package unit.kafka.consumer +package kafka.consumer import org.scalatest.junit.JUnit3Suite import org.easymock.EasyMock import org.I0Itec.zkclient.ZkClient import org.apache.zookeeper.data.Stat -import kafka.consumer._ import kafka.utils.{TestUtils, Logging, ZkUtils, Json} import junit.framework.Assert._ import kafka.common.TopicAndPartition -import unit.kafka.consumer.PartitionAssignorTest.StaticSubscriptionInfo -import kafka.consumer.ConsumerThreadId -import unit.kafka.consumer.PartitionAssignorTest.Scenario -import unit.kafka.consumer.PartitionAssignorTest.WildcardSubscriptionInfo +import kafka.consumer.PartitionAssignorTest.StaticSubscriptionInfo +import kafka.consumer.PartitionAssignorTest.Scenario +import kafka.consumer.PartitionAssignorTest.WildcardSubscriptionInfo class PartitionAssignorTest extends JUnit3Suite with Logging { diff --git a/core/src/test/scala/unit/kafka/integration/MinIsrConfigTest.scala b/core/src/test/scala/unit/kafka/integration/MinIsrConfigTest.scala index 70a5b3a..a2c9713 100644 --- a/core/src/test/scala/unit/kafka/integration/MinIsrConfigTest.scala +++ b/core/src/test/scala/unit/kafka/integration/MinIsrConfigTest.scala @@ -15,11 +15,10 @@ * limitations under the License. */ -package unit.kafka.integration +package kafka.integration import java.util.Properties -import kafka.integration.KafkaServerTestHarness import kafka.server.KafkaConfig import kafka.utils.TestUtils import org.scalatest.junit.JUnit3Suite diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala index f3546ad..3fd5a53 100644 --- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala @@ -15,11 +15,10 @@ * limitations under the License. */ -package unit.kafka.log +package kafka.log import java.util.Properties -import kafka.log.{Defaults, LogConfig} import org.apache.kafka.common.config.ConfigException import org.junit.{Assert, Test} import org.scalatest.junit.JUnit3Suite diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala index ace6321..c487f36 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala @@ -14,13 +14,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package unit.kafka.server +package kafka.server import java.util.Properties import kafka.api.ApiVersion import kafka.message._ -import kafka.server.{Defaults, KafkaConfig} import org.apache.kafka.common.protocol.SecurityProtocol import org.junit.{Assert, Test} import org.scalatest.junit.JUnit3Suite diff --git a/core/src/test/scala/unit/kafka/utils/ByteBoundedBlockingQueueTest.scala b/core/src/test/scala/unit/kafka/utils/ByteBoundedBlockingQueueTest.scala index fe8d2ae..fd8cf7b 100644 --- a/core/src/test/scala/unit/kafka/utils/ByteBoundedBlockingQueueTest.scala +++ b/core/src/test/scala/unit/kafka/utils/ByteBoundedBlockingQueueTest.scala @@ -15,13 +15,12 @@ * limitations under the License. */ -package unit.kafka.utils +package kafka.utils import java.util.concurrent.TimeUnit import junit.framework.Assert._ -import org.junit.{Test} -import kafka.utils.ByteBoundedBlockingQueue +import org.junit.Test class ByteBoundedBlockingQueueTest { val sizeFunction = (a: String) => a.length diff --git a/core/src/test/scala/unit/kafka/utils/CommandLineUtilsTest.scala b/core/src/test/scala/unit/kafka/utils/CommandLineUtilsTest.scala index e832a01..6380b6e 100644 --- a/core/src/test/scala/unit/kafka/utils/CommandLineUtilsTest.scala +++ b/core/src/test/scala/unit/kafka/utils/CommandLineUtilsTest.scala @@ -15,11 +15,10 @@ * limitations under the License. */ -package unit.kafka.utils +package kafka.utils import junit.framework.Assert._ -import org.junit.{Test, After, Before} -import kafka.utils.CommandLineUtils; +import org.junit.Test class CommandLineUtilsTest { diff --git a/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala b/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala index 64c3ba2..d3e44c6 100644 --- a/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala +++ b/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala @@ -15,13 +15,11 @@ * limitations under the License. */ -package unit.kafka.zk +package kafka.zk import junit.framework.Assert import kafka.consumer.ConsumerConfig import kafka.utils.{ZkPath, TestUtils, ZkUtils} -import kafka.zk.ZooKeeperTestHarness -import org.I0Itec.zkclient.ZkClient import org.apache.kafka.common.config.ConfigException import org.scalatest.junit.JUnit3Suite -- 2.4.3 From 28ecea421794d0c9a1c4f95375ccd1a6dfd8f365 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Tue, 16 Jun 2015 15:25:16 -0700 Subject: [PATCH 09/14] kafka-2272; listeners endpoint parsing fails if the hostname has capital letter; patched by Sriharsha Chintalapani; reviewed by Jun Rao --- core/src/main/scala/kafka/cluster/EndPoint.scala | 2 +- core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala | 6 ++++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/cluster/EndPoint.scala b/core/src/main/scala/kafka/cluster/EndPoint.scala index e9008e6..76997b5 100644 --- a/core/src/main/scala/kafka/cluster/EndPoint.scala +++ b/core/src/main/scala/kafka/cluster/EndPoint.scala @@ -42,7 +42,7 @@ object EndPoint { * @return */ def createEndPoint(connectionString: String): EndPoint = { - val uriParseExp = """^(.*)://\[?([0-9a-z\-.:]*)\]?:(-?[0-9]+)""".r + val uriParseExp = """^(.*)://\[?([0-9a-zA-Z\-.:]*)\]?:(-?[0-9]+)""".r connectionString match { case uriParseExp(protocol, "", port) => new EndPoint(null, port.toInt, SecurityProtocol.valueOf(protocol)) case uriParseExp(protocol, host, port) => new EndPoint(host, port.toInt, SecurityProtocol.valueOf(protocol)) diff --git a/core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala b/core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala index bb2506c..abe511f 100644 --- a/core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala @@ -120,5 +120,11 @@ class BrokerEndPointTest extends JUnit3Suite with Logging { assert(endpoint.host == "::1") assert(endpoint.port == 9092) assert(endpoint.connectionString == "PLAINTEXT://[::1]:9092") + // test hostname + connectionString = "PLAINTEXT://MyHostname:9092" + endpoint = EndPoint.createEndPoint(connectionString) + assert(endpoint.host == "MyHostname") + assert(endpoint.port == 9092) + assert(endpoint.connectionString == "PLAINTEXT://MyHostname:9092") } } -- 2.4.3 From 7009f1d6fffe3866723d1d33a28a4572053eb4e5 Mon Sep 17 00:00:00 2001 From: Manikumar Reddy Date: Tue, 16 Jun 2015 15:30:52 -0700 Subject: [PATCH 10/14] kafka-2264; SESSION_TIMEOUT_MS_CONFIG in ConsumerConfig should be int; patched by Manikumar Reddy; reviewed by Jun Rao --- .../main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java | 2 +- .../main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java | 2 +- .../org/apache/kafka/clients/consumer/internals/Coordinator.java | 6 +++--- .../apache/kafka/clients/consumer/internals/CoordinatorTest.java | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index 1e90524..daff34d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -167,7 +167,7 @@ public class ConsumerConfig extends AbstractConfig { CommonClientConfigs.BOOSTRAP_SERVERS_DOC) .define(GROUP_ID_CONFIG, Type.STRING, "", Importance.HIGH, GROUP_ID_DOC) .define(SESSION_TIMEOUT_MS_CONFIG, - Type.LONG, + Type.INT, 30000, Importance.HIGH, SESSION_TIMEOUT_MS_DOC) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index d1d1ec1..951c34c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -484,7 +484,7 @@ public class KafkaConsumer implements Consumer { this.coordinator = new Coordinator(this.client, config.getString(ConsumerConfig.GROUP_ID_CONFIG), this.retryBackoffMs, - config.getLong(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG), + config.getInt(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG), config.getString(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG), this.metadata, this.subscriptions, diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java index c1496a0..41cb945 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java @@ -69,7 +69,7 @@ public final class Coordinator { private final String groupId; private final Metadata metadata; private final Heartbeat heartbeat; - private final long sessionTimeoutMs; + private final int sessionTimeoutMs; private final String assignmentStrategy; private final SubscriptionState subscriptions; private final CoordinatorMetrics sensors; @@ -84,7 +84,7 @@ public final class Coordinator { public Coordinator(KafkaClient client, String groupId, long retryBackoffMs, - long sessionTimeoutMs, + int sessionTimeoutMs, String assignmentStrategy, Metadata metadata, SubscriptionState subscriptions, @@ -123,7 +123,7 @@ public final class Coordinator { // repeat processing the response until succeed or fatal error do { JoinGroupRequest request = new JoinGroupRequest(groupId, - (int) this.sessionTimeoutMs, + this.sessionTimeoutMs, subscribedTopics, this.consumerId, this.assignmentStrategy); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java index b06c4a7..1454ab7 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java @@ -50,7 +50,7 @@ public class CoordinatorTest { private String groupId = "test-group"; private TopicPartition tp = new TopicPartition(topicName, 0); private long retryBackoffMs = 0L; - private long sessionTimeoutMs = 10L; + private int sessionTimeoutMs = 10; private String rebalanceStrategy = "not-matter"; private MockTime time = new MockTime(); private MockClient client = new MockClient(time); -- 2.4.3 From 478505632edc8f4b51e4ed561d1adf455256c3e4 Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Tue, 16 Jun 2015 15:37:58 -0700 Subject: [PATCH 11/14] kafka-2252; Socket connection closing is logged, but not corresponding opening of socket; patched by Gwen Shapira; reviewed by Jun Rao --- clients/src/main/java/org/apache/kafka/common/network/Selector.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/common/network/Selector.java b/clients/src/main/java/org/apache/kafka/common/network/Selector.java index 1da215b..4aee214 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/Selector.java +++ b/clients/src/main/java/org/apache/kafka/common/network/Selector.java @@ -267,6 +267,7 @@ public class Selector implements Selectable { key.interestOps(key.interestOps() & ~SelectionKey.OP_CONNECT | SelectionKey.OP_READ); this.connected.add(transmissions.id); this.sensors.connectionCreated.record(); + log.debug("Connection {} created", transmissions.id); } /* read from any connections that have readable data */ @@ -307,7 +308,7 @@ public class Selector implements Selectable { } catch (IOException e) { String desc = socketDescription(channel); if (e instanceof EOFException || e instanceof ConnectException) - log.info("Connection {} disconnected", desc); + log.debug("Connection {} disconnected", desc); else log.warn("Error in I/O with connection to {}", desc, e); close(transmissions.id); -- 2.4.3 From 9650e12df2502f85ea665ae46a982d6fd0d2a954 Mon Sep 17 00:00:00 2001 From: Manikumar Reddy Date: Tue, 16 Jun 2015 15:50:17 -0700 Subject: [PATCH 12/14] kafka-2262; LogSegmentSize validation should be consistent; patched by Manikumar Reddy; reviewed by Jun Rao --- core/src/main/scala/kafka/log/LogConfig.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala index a907da0..f64fd79 100755 --- a/core/src/main/scala/kafka/log/LogConfig.scala +++ b/core/src/main/scala/kafka/log/LogConfig.scala @@ -22,6 +22,7 @@ import org.apache.kafka.common.utils.Utils import scala.collection._ import org.apache.kafka.common.config.ConfigDef import kafka.message.BrokerCompressionCodec +import kafka.message.Message object Defaults { val SegmentSize = 1024 * 1024 @@ -162,7 +163,7 @@ object LogConfig { import java.util.Arrays.asList new ConfigDef() - .define(SegmentBytesProp, INT, Defaults.SegmentSize, atLeast(0), MEDIUM, SegmentSizeDoc) + .define(SegmentBytesProp, INT, Defaults.SegmentSize, atLeast(Message.MinHeaderSize), MEDIUM, SegmentSizeDoc) .define(SegmentMsProp, LONG, Defaults.SegmentMs, atLeast(0), MEDIUM, SegmentMsDoc) .define(SegmentJitterMsProp, LONG, Defaults.SegmentJitterMs, atLeast(0), MEDIUM, SegmentJitterMsDoc) .define(SegmentIndexBytesProp, INT, Defaults.MaxIndexSize, atLeast(0), MEDIUM, MaxIndexSizeDoc) -- 2.4.3 From ba86f0a25dd8675d3c840c33605494a2f637576f Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Tue, 16 Jun 2015 16:41:20 -0700 Subject: [PATCH 13/14] trivial fix for stylecheck error on Jenkins --- .../java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java | 2 +- .../src/main/java/org/apache/kafka/common/requests/FetchRequest.java | 2 +- .../main/java/org/apache/kafka/common/requests/HeartbeatRequest.java | 2 +- .../main/java/org/apache/kafka/common/requests/ListOffsetRequest.java | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java index fef22d7..5b3e04a 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java @@ -42,7 +42,7 @@ public class ConsumerMetadataRequest extends AbstractRequest { @Override public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { - switch(versionId) { + switch (versionId) { case 0: return new ConsumerMetadataResponse(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code(), Node.noNode()); default: diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java index 4f52c32..df073a0 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java @@ -130,7 +130,7 @@ public class FetchRequest extends AbstractRequest { responseData.put(entry.getKey(), partitionResponse); } - switch(versionId) { + switch (versionId) { case 0: return new FetchResponse(responseData); default: diff --git a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java index d4d4a35..89719f1 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java @@ -50,7 +50,7 @@ public class HeartbeatRequest extends AbstractRequest { @Override public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { - switch(versionId) { + switch (versionId) { case 0: return new HeartbeatResponse(Errors.forException(e).code()); default: diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java index a1bb2b2..6da4a0e 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java @@ -115,7 +115,7 @@ public class ListOffsetRequest extends AbstractRequest { responseData.put(entry.getKey(), partitionResponse); } - switch(versionId) { + switch (versionId) { case 0: return new ListOffsetResponse(responseData); default: -- 2.4.3 From 234b71440ed24a8c92f59fbf21fae63d79d81516 Mon Sep 17 00:00:00 2001 From: Alexander Pakulov Date: Thu, 18 Jun 2015 11:01:14 -0700 Subject: [PATCH 14/14] KAFKA-1782; Junit3 Misusage --- .../integration/kafka/api/ConsumerBounceTest.scala | 2 ++ .../scala/integration/kafka/api/ConsumerTest.scala | 5 ++- .../kafka/api/IntegrationTestHarness.scala | 5 ++- .../integration/kafka/api/ProducerBounceTest.scala | 4 ++- .../kafka/api/ProducerCompressionTest.scala | 3 +- .../kafka/api/ProducerFailureHandlingTest.scala | 13 ++++--- .../integration/kafka/api/ProducerSendTest.scala | 13 +++---- .../test/scala/unit/kafka/KafkaConfigTest.scala | 2 +- .../scala/unit/kafka/admin/AddPartitionsTest.scala | 8 +++-- .../test/scala/unit/kafka/admin/AdminTest.scala | 7 ++-- .../unit/kafka/admin/DeleteConsumerGroupTest.scala | 3 +- .../scala/unit/kafka/admin/DeleteTopicTest.scala | 5 ++- .../scala/unit/kafka/admin/TopicCommandTest.scala | 8 ++--- .../test/scala/unit/kafka/api/ApiUtilsTest.scala | 2 +- .../api/RequestResponseSerializationTest.scala | 2 +- .../unit/kafka/cluster/BrokerEndPointTest.scala | 3 +- .../test/scala/unit/kafka/common/ConfigTest.scala | 2 +- .../test/scala/unit/kafka/common/TopicTest.scala | 2 +- .../unit/kafka/consumer/ConsumerIteratorTest.scala | 9 +++-- .../kafka/consumer/PartitionAssignorTest.scala | 5 ++- .../unit/kafka/consumer/TopicFilterTest.scala | 2 +- .../consumer/ZookeeperConsumerConnectorTest.scala | 8 +++-- .../coordinator/ConsumerGroupMetadataTest.scala | 2 +- .../coordinator/CoordinatorMetadataTest.scala | 2 +- .../kafka/coordinator/PartitionAssignorTest.scala | 2 +- .../kafka/integration/AutoOffsetResetTest.scala | 9 ++--- .../scala/unit/kafka/integration/FetcherTest.scala | 11 +++--- .../kafka/integration/KafkaServerTestHarness.scala | 8 +++-- .../unit/kafka/integration/MinIsrConfigTest.scala | 3 +- .../unit/kafka/integration/PrimitiveApiTest.scala | 5 ++- .../integration/ProducerConsumerTestHarness.scala | 40 ++++++++++++---------- .../unit/kafka/integration/RollingBounceTest.scala | 8 +++-- .../unit/kafka/integration/TopicMetadataTest.scala | 8 +++-- .../integration/UncleanLeaderElectionTest.scala | 10 +++--- .../consumer/ZookeeperConsumerConnectorTest.scala | 6 ++-- .../javaapi/message/BaseMessageSetTestCases.scala | 2 +- .../javaapi/message/ByteBufferMessageSetTest.scala | 2 +- .../test/scala/unit/kafka/log/CleanerTest.scala | 2 +- .../scala/unit/kafka/log/FileMessageSetTest.scala | 2 +- .../unit/kafka/log/LogCleanerIntegrationTest.scala | 4 +-- .../test/scala/unit/kafka/log/LogConfigTest.scala | 4 +-- .../test/scala/unit/kafka/log/LogManagerTest.scala | 17 +++++---- .../test/scala/unit/kafka/log/LogSegmentTest.scala | 8 ++--- core/src/test/scala/unit/kafka/log/LogTest.scala | 2 +- .../scala/unit/kafka/log/OffsetIndexTest.scala | 2 +- .../test/scala/unit/kafka/log/OffsetMapTest.scala | 2 +- .../unit/kafka/log4j/KafkaLog4jAppenderTest.scala | 5 ++- .../kafka/message/BaseMessageSetTestCases.scala | 2 +- .../kafka/message/ByteBufferMessageSetTest.scala | 2 +- .../kafka/message/MessageCompressionTest.scala | 2 +- .../scala/unit/kafka/message/MessageTest.scala | 2 +- .../unit/kafka/message/MessageWriterTest.scala | 2 +- .../scala/unit/kafka/metrics/KafkaTimerTest.scala | 5 ++- .../scala/unit/kafka/metrics/MetricsTest.scala | 6 ++-- .../unit/kafka/network/SocketServerTest.scala | 10 +++--- .../unit/kafka/producer/AsyncProducerTest.scala | 15 ++------ .../scala/unit/kafka/producer/ProducerTest.scala | 27 +++++++-------- .../unit/kafka/producer/SyncProducerTest.scala | 5 ++- .../unit/kafka/server/AdvertiseBrokerTest.scala | 8 +++-- .../unit/kafka/server/DelayedOperationTest.scala | 15 ++++---- .../kafka/server/DynamicConfigChangeTest.scala | 5 ++- .../server/HighwatermarkPersistenceTest.scala | 3 +- .../unit/kafka/server/ISRExpirationTest.scala | 12 +++---- .../kafka/server/KafkaConfigConfigDefTest.scala | 4 +-- .../scala/unit/kafka/server/KafkaConfigTest.scala | 6 ++-- .../unit/kafka/server/LeaderElectionTest.scala | 8 +++-- .../scala/unit/kafka/server/LogOffsetTest.scala | 5 ++- .../scala/unit/kafka/server/LogRecoveryTest.scala | 6 ++-- .../scala/unit/kafka/server/OffsetCommitTest.scala | 5 ++- .../scala/unit/kafka/server/ReplicaFetchTest.scala | 6 ++-- .../unit/kafka/server/ReplicaManagerTest.scala | 5 ++- .../kafka/server/ServerGenerateBrokerIdTest.scala | 8 ++--- .../unit/kafka/server/ServerShutdownTest.scala | 14 ++++---- .../unit/kafka/server/ServerStartupTest.scala | 5 ++- .../scala/unit/kafka/server/SimpleFetchTest.scala | 16 ++++----- .../kafka/utils/ByteBoundedBlockingQueueTest.scala | 2 +- .../unit/kafka/utils/CommandLineUtilsTest.scala | 2 +- .../unit/kafka/utils/IteratorTemplateTest.scala | 2 +- .../src/test/scala/unit/kafka/utils/JsonTest.scala | 2 +- .../unit/kafka/utils/ReplicationUtilsTest.scala | 7 ++-- .../scala/unit/kafka/utils/SchedulerTest.scala | 2 +- .../test/scala/unit/kafka/utils/TestUtils.scala | 5 ++- .../unit/kafka/utils/timer/TimerTaskListTest.scala | 2 +- .../scala/unit/kafka/utils/timer/TimerTest.scala | 2 +- .../test/scala/unit/kafka/zk/ZKEphemeralTest.scala | 4 +-- core/src/test/scala/unit/kafka/zk/ZKPathTest.scala | 13 ++++--- .../scala/unit/kafka/zk/ZooKeeperTestHarness.scala | 12 +++---- 87 files changed, 266 insertions(+), 277 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala index f56096b..32be00c 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala @@ -23,6 +23,7 @@ import org.apache.kafka.common.TopicPartition import kafka.utils.{ShutdownableThread, TestUtils, Logging} import org.junit.Assert._ +import org.junit.Before import scala.collection.JavaConversions._ @@ -55,6 +56,7 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging { .map(KafkaConfig.fromProps(_, serverConfig)) } + @Before override def setUp() { super.setUp() diff --git a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala index 17b17b9..6d9acd5 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala @@ -29,6 +29,8 @@ import kafka.server.{KafkaConfig, OffsetManager} import java.util.ArrayList import org.junit.Assert._ +import org.junit.Before +import org.scalatest.Assertions._ import scala.collection.JavaConversions._ @@ -55,7 +57,8 @@ class ConsumerTest extends IntegrationTestHarness with Logging { 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") - + + @Before override def setUp() { super.setUp() diff --git a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala index 07b1ff4..316d6ba 100644 --- a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala +++ b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala @@ -25,6 +25,7 @@ import org.apache.kafka.clients.consumer.KafkaConsumer import org.apache.kafka.clients.producer.KafkaProducer import kafka.server.{OffsetManager, KafkaConfig} import kafka.integration.KafkaServerTestHarness +import org.junit.{After, Before} import scala.collection.mutable.Buffer /** @@ -48,6 +49,7 @@ trait IntegrationTestHarness extends KafkaServerTestHarness { cfgs.map(KafkaConfig.fromProps) } + @Before override def setUp() { super.setUp() producerConfig.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, this.bootstrapUrl) @@ -69,7 +71,8 @@ trait IntegrationTestHarness extends KafkaServerTestHarness { servers, servers(0).offsetManager.offsetsTopicConfig) } - + + @After override def tearDown() { producers.foreach(_.close()) consumers.foreach(_.close()) diff --git a/core/src/test/scala/integration/kafka/api/ProducerBounceTest.scala b/core/src/test/scala/integration/kafka/api/ProducerBounceTest.scala index ce70a0a..2dbb9dc 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerBounceTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerBounceTest.scala @@ -22,7 +22,7 @@ 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 +import org.junit.{After, Before, Test} class ProducerBounceTest extends KafkaServerTestHarness { private val producerBufferSize = 30000 @@ -62,6 +62,7 @@ class ProducerBounceTest extends KafkaServerTestHarness { private val topic1 = "topic-1" private val topic2 = "topic-2" + @Before override def setUp() { super.setUp() @@ -70,6 +71,7 @@ class ProducerBounceTest extends KafkaServerTestHarness { producer3 = TestUtils.createNewProducer(brokerList, acks = -1, blockOnBufferFull = false, bufferSize = producerBufferSize) } + @After override def tearDown() { if (producer1 != null) producer1.close if (producer2 != null) producer2.close diff --git a/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala b/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala index 83de81c..87db255 100755 --- a/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala @@ -19,7 +19,6 @@ package kafka.api.test import java.util.{Properties, Collection, ArrayList} -import org.scalatest.junit.JUnit3Suite import org.junit.runners.Parameterized import org.junit.runner.RunWith import org.junit.runners.Parameterized.Parameters @@ -36,7 +35,7 @@ import kafka.utils.{CoreUtils, TestUtils} @RunWith(value = classOf[Parameterized]) -class ProducerCompressionTest(compression: String) extends JUnit3Suite with ZooKeeperTestHarness { +class ProducerCompressionTest(compression: String) extends ZooKeeperTestHarness { private val brokerId = 0 private var server: KafkaServer = null diff --git a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala index ee94011..426ce11 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala @@ -17,8 +17,9 @@ package kafka.api -import org.junit.Test +import org.junit.{After, Before, Assert, Test} import org.junit.Assert._ +import org.scalatest.Assertions._ import java.util.{Properties, Random} import java.util.concurrent.{TimeoutException, TimeUnit, ExecutionException} @@ -61,6 +62,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { private val topic1 = "topic-1" private val topic2 = "topic-2" + @Before override def setUp() { super.setUp() @@ -69,6 +71,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { producer3 = TestUtils.createNewProducer(brokerList, acks = -1, blockOnBufferFull = false, bufferSize = producerBufferSize) } + @After override def tearDown() { if (producer1 != null) producer1.close if (producer2 != null) producer2.close @@ -257,11 +260,11 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { val record = new ProducerRecord[Array[Byte],Array[Byte]](topicName, null, "key".getBytes, "value".getBytes) try { producer3.send(record).get - fail("Expected exception when producing to topic with fewer brokers than min.insync.replicas") + Assert.fail("Expected exception when producing to topic with fewer brokers than min.insync.replicas") } catch { case e: ExecutionException => if (!e.getCause.isInstanceOf[NotEnoughReplicasException]) { - fail("Expected NotEnoughReplicasException when producing to topic with fewer brokers than min.insync.replicas") + Assert.fail("Expected NotEnoughReplicasException when producing to topic with fewer brokers than min.insync.replicas") } } } @@ -283,12 +286,12 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { servers.head.awaitShutdown() try { producer3.send(record).get - fail("Expected exception when producing to topic with fewer brokers than min.insync.replicas") + Assert.fail("Expected exception when producing to topic with fewer brokers than min.insync.replicas") } catch { case e: ExecutionException => if (!e.getCause.isInstanceOf[NotEnoughReplicasException] && !e.getCause.isInstanceOf[NotEnoughReplicasAfterAppendException]) { - fail("Expected NotEnoughReplicasException or NotEnoughReplicasAfterAppendException when producing to topic " + + Assert.fail("Expected NotEnoughReplicasException or NotEnoughReplicasAfterAppendException when producing to topic " + "with fewer brokers than min.insync.replicas, but saw " + e.getCause) } } diff --git a/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala index 9ce4bd5..398ae0d 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala @@ -29,12 +29,11 @@ import org.apache.kafka.clients.producer._ import org.apache.kafka.common.config.ConfigException import org.apache.kafka.common.errors.SerializationException import org.apache.kafka.common.serialization.ByteArraySerializer +import org.junit.{Before, After, Test} import org.junit.Assert._ -import org.junit.Test -import org.scalatest.junit.JUnit3Suite +import org.scalatest.Assertions - -class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness { +class ProducerSendTest extends KafkaServerTestHarness { val numServers = 2 val overridingProps = new Properties() @@ -49,6 +48,7 @@ class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness { private val topic = "topic" private val numRecords = 100 + @Before override def setUp() { super.setUp() @@ -57,6 +57,7 @@ class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness { consumer2 = new SimpleConsumer("localhost", servers(1).boundPort(), 100, 1024*1024, "") } + @After override def tearDown() { consumer1.close() consumer2.close() @@ -84,7 +85,7 @@ class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness { assertEquals(partition, metadata.partition()) offset += 1 } else { - fail("Send callback returns the following exception", exception) + Assertions.fail("Send callback returns the following exception", exception) } } } @@ -116,7 +117,7 @@ class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness { fail("Should not allow sending a record without topic") } catch { case iae: IllegalArgumentException => // this is ok - case e: Throwable => fail("Only expecting IllegalArgumentException", e) + case e: Throwable => Assertions.fail("Only expecting IllegalArgumentException", e) } // non-blocking send a list of records diff --git a/core/src/test/scala/unit/kafka/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/KafkaConfigTest.scala index 4764c89..1233104 100644 --- a/core/src/test/scala/unit/kafka/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/KafkaConfigTest.scala @@ -21,7 +21,7 @@ import java.security.Permission import kafka.server.KafkaConfig import org.junit.{After, Before, Test} -import junit.framework.Assert._ +import org.junit.Assert._ class KafkaTest { diff --git a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala index df5c6ba..42d655f 100755 --- a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala @@ -17,17 +17,17 @@ package kafka.admin +import org.junit.Assert._ import org.apache.kafka.common.protocol.SecurityProtocol -import org.scalatest.junit.JUnit3Suite import kafka.zk.ZooKeeperTestHarness import kafka.utils.TestUtils._ -import junit.framework.Assert._ import kafka.utils.{ZkUtils, CoreUtils, TestUtils} import kafka.cluster.Broker import kafka.client.ClientUtils import kafka.server.{KafkaConfig, KafkaServer} +import org.junit.{After, Before} -class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { +class AddPartitionsTest extends ZooKeeperTestHarness { var configs: Seq[KafkaConfig] = null var servers: Seq[KafkaServer] = Seq.empty[KafkaServer] var brokers: Seq[Broker] = Seq.empty[Broker] @@ -39,6 +39,7 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { val topic3 = "new-topic3" val topic4 = "new-topic4" + @Before override def setUp() { super.setUp() @@ -54,6 +55,7 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { createTopic(zkClient, topic4, partitionReplicaAssignment = Map(0->Seq(0,3)), servers = servers) } + @After override def tearDown() { servers.foreach(_.shutdown()) servers.foreach(server => CoreUtils.rm(server.config.logDirs)) diff --git a/core/src/test/scala/unit/kafka/admin/AdminTest.scala b/core/src/test/scala/unit/kafka/admin/AdminTest.scala index efb2f8e..73821ce 100755 --- a/core/src/test/scala/unit/kafka/admin/AdminTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AdminTest.scala @@ -16,9 +16,7 @@ */ package kafka.admin -import junit.framework.Assert._ import org.junit.Test -import org.scalatest.junit.JUnit3Suite import java.util.Properties import kafka.utils._ import kafka.log._ @@ -28,9 +26,10 @@ import kafka.common.{TopicExistsException, TopicAndPartition} import kafka.server.{KafkaServer, KafkaConfig} import java.io.File import TestUtils._ +import org.junit.Assert._ +import org.scalatest.Assertions._ - -class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { +class AdminTest extends ZooKeeperTestHarness with Logging { @Test def testReplicaAssignment() { diff --git a/core/src/test/scala/unit/kafka/admin/DeleteConsumerGroupTest.scala b/core/src/test/scala/unit/kafka/admin/DeleteConsumerGroupTest.scala index 1913ad6..d3abf08 100644 --- a/core/src/test/scala/unit/kafka/admin/DeleteConsumerGroupTest.scala +++ b/core/src/test/scala/unit/kafka/admin/DeleteConsumerGroupTest.scala @@ -16,7 +16,6 @@ */ package kafka.admin -import org.scalatest.junit.JUnit3Suite import kafka.utils._ import kafka.server.KafkaConfig import org.junit.Test @@ -25,7 +24,7 @@ import org.apache.kafka.clients.producer.{ProducerRecord, KafkaProducer} import kafka.integration.KafkaServerTestHarness -class DeleteConsumerGroupTest extends JUnit3Suite with KafkaServerTestHarness { +class DeleteConsumerGroupTest extends KafkaServerTestHarness { def generateConfigs() = TestUtils.createBrokerConfigs(3, zkConnect, false, true).map(KafkaConfig.fromProps) @Test diff --git a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala index fa8ce25..6279180 100644 --- a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala +++ b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala @@ -17,16 +17,15 @@ package kafka.admin import kafka.log.Log -import org.scalatest.junit.JUnit3Suite import kafka.zk.ZooKeeperTestHarness -import junit.framework.Assert._ import kafka.utils.{ZkUtils, TestUtils} import kafka.server.{KafkaServer, KafkaConfig} import org.junit.Test +import org.junit.Assert._ import java.util.Properties import kafka.common.TopicAndPartition -class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { +class DeleteTopicTest extends ZooKeeperTestHarness { @Test def testDeleteTopicWithAllAliveReplicas() { diff --git a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala index c7136f2..d1b4673 100644 --- a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala @@ -16,17 +16,17 @@ */ package kafka.admin -import junit.framework.Assert._ import org.junit.Test -import org.scalatest.junit.JUnit3Suite import kafka.utils.Logging import kafka.utils.TestUtils import kafka.zk.ZooKeeperTestHarness -import kafka.server.{OffsetManager, KafkaConfig} +import kafka.server.{OffsetManager} import kafka.admin.TopicCommand.TopicCommandOptions import kafka.utils.ZkUtils +import org.junit.Assert._ +import org.scalatest.Assertions._ -class TopicCommandTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { +class TopicCommandTest extends ZooKeeperTestHarness with Logging { @Test def testConfigPreservationAcrossPartitionAlteration() { diff --git a/core/src/test/scala/unit/kafka/api/ApiUtilsTest.scala b/core/src/test/scala/unit/kafka/api/ApiUtilsTest.scala index 2554425..fff3e7b 100644 --- a/core/src/test/scala/unit/kafka/api/ApiUtilsTest.scala +++ b/core/src/test/scala/unit/kafka/api/ApiUtilsTest.scala @@ -19,7 +19,7 @@ package kafka.api import org.junit._ import org.scalatest.junit.JUnitSuite -import junit.framework.Assert._ +import org.junit.Assert._ import scala.util.Random import java.nio.ByteBuffer import kafka.common.KafkaException diff --git a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala index 5717165..b4c2a22 100644 --- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala +++ b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala @@ -32,7 +32,7 @@ import java.nio.ByteBuffer import org.apache.kafka.common.protocol.SecurityProtocol import org.junit._ import org.scalatest.junit.JUnitSuite -import junit.framework.Assert._ +import org.junit.Assert._ object SerializationTestUtils { diff --git a/core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala b/core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala index abe511f..2d3a9c3 100644 --- a/core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala @@ -22,11 +22,10 @@ import java.nio.ByteBuffer import kafka.utils.Logging import org.apache.kafka.common.protocol.SecurityProtocol import org.junit.Test -import org.scalatest.junit.JUnit3Suite import scala.collection.mutable -class BrokerEndPointTest extends JUnit3Suite with Logging { +class BrokerEndPointTest extends Logging { @Test def testSerDe() = { diff --git a/core/src/test/scala/unit/kafka/common/ConfigTest.scala b/core/src/test/scala/unit/kafka/common/ConfigTest.scala index 0aca938..a42836c 100644 --- a/core/src/test/scala/unit/kafka/common/ConfigTest.scala +++ b/core/src/test/scala/unit/kafka/common/ConfigTest.scala @@ -17,7 +17,7 @@ package kafka.common -import junit.framework.Assert._ +import org.junit.Assert._ import collection.mutable.ArrayBuffer import org.junit.Test import kafka.producer.ProducerConfig diff --git a/core/src/test/scala/unit/kafka/common/TopicTest.scala b/core/src/test/scala/unit/kafka/common/TopicTest.scala index 79532c8..137dcc4 100644 --- a/core/src/test/scala/unit/kafka/common/TopicTest.scala +++ b/core/src/test/scala/unit/kafka/common/TopicTest.scala @@ -17,7 +17,7 @@ package kafka.common -import junit.framework.Assert._ +import org.junit.Assert._ import collection.mutable.ArrayBuffer import org.junit.Test diff --git a/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala b/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala index db5302f..ca63c80 100755 --- a/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala @@ -18,22 +18,20 @@ package kafka.consumer -import java.util.Properties import java.util.concurrent._ import java.util.concurrent.atomic._ import scala.collection._ -import junit.framework.Assert._ +import org.junit.Assert._ import kafka.message._ import kafka.server._ import kafka.utils.TestUtils._ import kafka.utils._ -import org.junit.Test +import org.junit.{Before, Test} import kafka.serializer._ -import org.scalatest.junit.JUnit3Suite import kafka.integration.KafkaServerTestHarness -class ConsumerIteratorTest extends JUnit3Suite with KafkaServerTestHarness { +class ConsumerIteratorTest extends KafkaServerTestHarness { val numNodes = 1 @@ -49,6 +47,7 @@ class ConsumerIteratorTest extends JUnit3Suite with KafkaServerTestHarness { def consumerConfig = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer0)) + @Before override def setUp() { super.setUp() topicInfos = configs.map(c => new PartitionTopicInfo(topic, diff --git a/core/src/test/scala/unit/kafka/consumer/PartitionAssignorTest.scala b/core/src/test/scala/unit/kafka/consumer/PartitionAssignorTest.scala index adf0801..c1071b8 100644 --- a/core/src/test/scala/unit/kafka/consumer/PartitionAssignorTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/PartitionAssignorTest.scala @@ -17,18 +17,17 @@ package kafka.consumer -import org.scalatest.junit.JUnit3Suite import org.easymock.EasyMock import org.I0Itec.zkclient.ZkClient import org.apache.zookeeper.data.Stat import kafka.utils.{TestUtils, Logging, ZkUtils, Json} -import junit.framework.Assert._ +import org.junit.Assert._ import kafka.common.TopicAndPartition import kafka.consumer.PartitionAssignorTest.StaticSubscriptionInfo import kafka.consumer.PartitionAssignorTest.Scenario import kafka.consumer.PartitionAssignorTest.WildcardSubscriptionInfo -class PartitionAssignorTest extends JUnit3Suite with Logging { +class PartitionAssignorTest extends Logging { def testRoundRobinPartitionAssignor() { val assignor = new RoundRobinAssignor diff --git a/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala b/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala index 4f124af..db0ebfa 100644 --- a/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala @@ -18,7 +18,7 @@ package kafka.consumer -import junit.framework.Assert._ +import org.junit.Assert._ import org.scalatest.junit.JUnitSuite import org.junit.Test import kafka.server.OffsetManager diff --git a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala b/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala index 359b0f5..c851e27 100644 --- a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala @@ -19,7 +19,7 @@ package kafka.consumer import java.util.{Collections, Properties} -import junit.framework.Assert._ +import org.junit.Assert._ import kafka.common.MessageStreamsExistException import kafka.integration.KafkaServerTestHarness import kafka.javaapi.consumer.ConsumerRebalanceListener @@ -30,11 +30,11 @@ import kafka.utils.TestUtils._ import kafka.utils._ import org.I0Itec.zkclient.ZkClient import org.apache.log4j.{Level, Logger} -import org.scalatest.junit.JUnit3Suite +import org.junit.{After, Before} import scala.collection._ -class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHarness with Logging { +class ZookeeperConsumerConnectorTest extends KafkaServerTestHarness with Logging { val RebalanceBackoffMs = 5000 var dirs : ZKGroupTopicDirs = null @@ -54,11 +54,13 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar val consumer3 = "consumer3" val nMessages = 2 + @Before override def setUp() { super.setUp() dirs = new ZKGroupTopicDirs(group, topic) } + @After override def tearDown() { super.tearDown() } diff --git a/core/src/test/scala/unit/kafka/coordinator/ConsumerGroupMetadataTest.scala b/core/src/test/scala/unit/kafka/coordinator/ConsumerGroupMetadataTest.scala index b69c993..5d812c2 100644 --- a/core/src/test/scala/unit/kafka/coordinator/ConsumerGroupMetadataTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/ConsumerGroupMetadataTest.scala @@ -17,7 +17,7 @@ package kafka.coordinator -import junit.framework.Assert._ +import org.junit.Assert._ import org.junit.{Before, Test} import org.scalatest.junit.JUnitSuite diff --git a/core/src/test/scala/unit/kafka/coordinator/CoordinatorMetadataTest.scala b/core/src/test/scala/unit/kafka/coordinator/CoordinatorMetadataTest.scala index 08854c5..8ffca13 100644 --- a/core/src/test/scala/unit/kafka/coordinator/CoordinatorMetadataTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/CoordinatorMetadataTest.scala @@ -20,7 +20,7 @@ package kafka.coordinator import kafka.server.KafkaConfig import kafka.utils.{ZkUtils, TestUtils} -import junit.framework.Assert._ +import org.junit.Assert._ import org.I0Itec.zkclient.{IZkDataListener, ZkClient} import org.apache.zookeeper.data.Stat import org.easymock.EasyMock diff --git a/core/src/test/scala/unit/kafka/coordinator/PartitionAssignorTest.scala b/core/src/test/scala/unit/kafka/coordinator/PartitionAssignorTest.scala index 887cee5..79c691f 100644 --- a/core/src/test/scala/unit/kafka/coordinator/PartitionAssignorTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/PartitionAssignorTest.scala @@ -19,7 +19,7 @@ package kafka.coordinator import kafka.common.TopicAndPartition -import junit.framework.Assert._ +import org.junit.Assert._ import org.junit.Test import org.scalatest.junit.JUnitSuite diff --git a/core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala b/core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala index 139dc9a..818673f 100644 --- a/core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala +++ b/core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala @@ -24,12 +24,11 @@ import kafka.utils.TestUtils import kafka.serializer._ import kafka.producer.{Producer, KeyedMessage} -import org.junit.Test +import org.junit.{After, Before, Test} import org.apache.log4j.{Level, Logger} -import org.scalatest.junit.JUnit3Suite -import junit.framework.Assert._ +import org.junit.Assert._ -class AutoOffsetResetTest extends JUnit3Suite with KafkaServerTestHarness with Logging { +class AutoOffsetResetTest extends KafkaServerTestHarness with Logging { def generateConfigs() = List(KafkaConfig.fromProps(TestUtils.createBrokerConfig(0, zkConnect))) @@ -42,12 +41,14 @@ class AutoOffsetResetTest extends JUnit3Suite with KafkaServerTestHarness with L val requestHandlerLogger = Logger.getLogger(classOf[kafka.server.KafkaRequestHandler]) + @Before override def setUp() { super.setUp() // temporarily set request handler logger to a higher level requestHandlerLogger.setLevel(Level.FATAL) } + @After override def tearDown() { // restore set request handler logger to a higher level requestHandlerLogger.setLevel(Level.ERROR) diff --git a/core/src/test/scala/unit/kafka/integration/FetcherTest.scala b/core/src/test/scala/unit/kafka/integration/FetcherTest.scala index facebd8..92af0a1 100644 --- a/core/src/test/scala/unit/kafka/integration/FetcherTest.scala +++ b/core/src/test/scala/unit/kafka/integration/FetcherTest.scala @@ -19,18 +19,17 @@ package kafka.integration import java.util.concurrent._ import java.util.concurrent.atomic._ +import org.junit.{After, Before} + import scala.collection._ -import junit.framework.Assert._ +import org.junit.Assert._ import kafka.cluster._ import kafka.server._ -import org.scalatest.junit.JUnit3Suite import kafka.consumer._ -import kafka.serializer._ -import kafka.producer.{KeyedMessage, Producer} import kafka.utils.TestUtils -class FetcherTest extends JUnit3Suite with KafkaServerTestHarness { +class FetcherTest extends KafkaServerTestHarness { val numNodes = 1 def generateConfigs() = TestUtils.createBrokerConfigs(numNodes, zkConnect).map(KafkaConfig.fromProps) @@ -40,6 +39,7 @@ class FetcherTest extends JUnit3Suite with KafkaServerTestHarness { var fetcher: ConsumerFetcherManager = null + @Before override def setUp() { super.setUp TestUtils.createTopic(zkClient, topic, partitionReplicaAssignment = Map(0 -> Seq(configs.head.brokerId)), servers = servers) @@ -59,6 +59,7 @@ class FetcherTest extends JUnit3Suite with KafkaServerTestHarness { fetcher.startConnections(topicInfos, cluster) } + @After override def tearDown() { fetcher.stopConnections() super.tearDown diff --git a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala index 87c6315..c7e79ca 100755 --- a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala +++ b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala @@ -19,17 +19,18 @@ package kafka.integration import java.util.Arrays +import org.junit.{Before, After} + import scala.collection.mutable.Buffer import kafka.server._ import kafka.utils.{CoreUtils, TestUtils} -import org.scalatest.junit.JUnit3Suite import kafka.zk.ZooKeeperTestHarness import kafka.common.KafkaException /** * A test harness that brings up some number of broker nodes */ -trait KafkaServerTestHarness extends JUnit3Suite with ZooKeeperTestHarness { +trait KafkaServerTestHarness extends ZooKeeperTestHarness { var instanceConfigs: Seq[KafkaConfig] = null var servers: Buffer[KafkaServer] = null var brokerList: String = null @@ -51,7 +52,7 @@ trait KafkaServerTestHarness extends JUnit3Suite with ZooKeeperTestHarness { def bootstrapUrl = servers.map(s => s.config.hostName + ":" + s.boundPort()).mkString(",") - + @Before override def setUp() { super.setUp if(configs.size <= 0) @@ -62,6 +63,7 @@ trait KafkaServerTestHarness extends JUnit3Suite with ZooKeeperTestHarness { Arrays.fill(alive, true) } + @After override def tearDown() { servers.foreach(_.shutdown()) servers.foreach(_.config.logDirs.foreach(CoreUtils.rm(_))) diff --git a/core/src/test/scala/unit/kafka/integration/MinIsrConfigTest.scala b/core/src/test/scala/unit/kafka/integration/MinIsrConfigTest.scala index a2c9713..3c1cade 100644 --- a/core/src/test/scala/unit/kafka/integration/MinIsrConfigTest.scala +++ b/core/src/test/scala/unit/kafka/integration/MinIsrConfigTest.scala @@ -21,9 +21,8 @@ import java.util.Properties import kafka.server.KafkaConfig import kafka.utils.TestUtils -import org.scalatest.junit.JUnit3Suite -class MinIsrConfigTest extends JUnit3Suite with KafkaServerTestHarness { +class MinIsrConfigTest extends KafkaServerTestHarness { val overridingProps = new Properties() overridingProps.put(KafkaConfig.MinInSyncReplicasProp, "5") diff --git a/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala b/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala index 6a758a7..e05d16b 100755 --- a/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala +++ b/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala @@ -18,13 +18,12 @@ package kafka.integration import java.nio.ByteBuffer -import junit.framework.Assert._ +import org.junit.Assert._ import kafka.api.{PartitionFetchInfo, FetchRequest, FetchRequestBuilder} import kafka.server.{KafkaRequestHandler, KafkaConfig} import kafka.producer.{KeyedMessage, Producer} import org.apache.log4j.{Level, Logger} import kafka.zk.ZooKeeperTestHarness -import org.scalatest.junit.JUnit3Suite import scala.collection._ import kafka.common.{TopicAndPartition, ErrorMapping, UnknownTopicOrPartitionException, OffsetOutOfRangeException} import kafka.utils.{StaticPartitioner, TestUtils, CoreUtils} @@ -34,7 +33,7 @@ import java.util.Properties /** * End to end tests of the primitive apis against a local server */ -class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with ZooKeeperTestHarness { +class PrimitiveApiTest extends ProducerConsumerTestHarness with ZooKeeperTestHarness { val requestHandlerLogger = Logger.getLogger(classOf[KafkaRequestHandler]) def generateConfigs() = List(KafkaConfig.fromProps(TestUtils.createBrokerConfig(0, zkConnect))) diff --git a/core/src/test/scala/unit/kafka/integration/ProducerConsumerTestHarness.scala b/core/src/test/scala/unit/kafka/integration/ProducerConsumerTestHarness.scala index 4614a92..cc5954d 100644 --- a/core/src/test/scala/unit/kafka/integration/ProducerConsumerTestHarness.scala +++ b/core/src/test/scala/unit/kafka/integration/ProducerConsumerTestHarness.scala @@ -5,8 +5,8 @@ * The ASF licenses this file to You under the Apache License, Version 2.0 * (the "License"); you may not use this file except in compliance with * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * + * 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, @@ -18,28 +18,30 @@ package kafka.integration import kafka.consumer.SimpleConsumer -import org.scalatest.junit.JUnit3Suite +import org.junit.{After, Before} import kafka.producer.Producer import kafka.utils.{StaticPartitioner, TestUtils} import kafka.serializer.StringEncoder -trait ProducerConsumerTestHarness extends JUnit3Suite with KafkaServerTestHarness { - val host = "localhost" - var producer: Producer[String, String] = null - var consumer: SimpleConsumer = null +trait ProducerConsumerTestHarness extends KafkaServerTestHarness { + val host = "localhost" + var producer: Producer[String, String] = null + var consumer: SimpleConsumer = null + @Before override def setUp() { - super.setUp - producer = TestUtils.createProducer[String, String](TestUtils.getBrokerListStrFromServers(servers), - encoder = classOf[StringEncoder].getName, - keyEncoder = classOf[StringEncoder].getName, - partitioner = classOf[StaticPartitioner].getName) - consumer = new SimpleConsumer(host, servers(0).boundPort(), 1000000, 64*1024, "") - } + super.setUp + producer = TestUtils.createProducer[String, String](TestUtils.getBrokerListStrFromServers(servers), + encoder = classOf[StringEncoder].getName, + keyEncoder = classOf[StringEncoder].getName, + partitioner = classOf[StaticPartitioner].getName) + consumer = new SimpleConsumer(host, servers(0).boundPort(), 1000000, 64 * 1024, "") + } - override def tearDown() { - producer.close() - consumer.close() - super.tearDown - } + @After + override def tearDown() { + producer.close() + consumer.close() + super.tearDown + } } diff --git a/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala b/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala index 12d0733..2fd10d8 100755 --- a/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala +++ b/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala @@ -17,18 +17,19 @@ package kafka.integration -import org.scalatest.junit.JUnit3Suite +import org.junit.{After, Before} import kafka.zk.ZooKeeperTestHarness import kafka.utils.TestUtils._ -import junit.framework.Assert._ +import org.junit.Assert._ import kafka.utils.{CoreUtils, TestUtils} import kafka.server.{KafkaConfig, KafkaServer} -class RollingBounceTest extends JUnit3Suite with ZooKeeperTestHarness { +class RollingBounceTest extends ZooKeeperTestHarness { val partitionId = 0 var servers: Seq[KafkaServer] = null + @Before override def setUp() { super.setUp() // controlled.shutdown.enable is true by default @@ -39,6 +40,7 @@ class RollingBounceTest extends JUnit3Suite with ZooKeeperTestHarness { servers = configs.map(c => TestUtils.createServer(KafkaConfig.fromProps(c))) } + @After override def tearDown() { servers.foreach(_.shutdown()) servers.foreach(server => CoreUtils.rm(server.config.logDirs)) diff --git a/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala b/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala index 995b059..4c339bd 100644 --- a/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala +++ b/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala @@ -18,11 +18,11 @@ package kafka.integration import org.apache.kafka.common.protocol.SecurityProtocol -import org.scalatest.junit.JUnit3Suite +import org.junit.{After, Before} import kafka.zk.ZooKeeperTestHarness import kafka.admin.AdminUtils import java.nio.ByteBuffer -import junit.framework.Assert._ +import org.junit.Assert._ import kafka.cluster.{BrokerEndPoint, Broker} import kafka.utils.TestUtils import kafka.utils.TestUtils._ @@ -31,10 +31,11 @@ import kafka.api.TopicMetadataRequest import kafka.common.ErrorMapping import kafka.client.ClientUtils -class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { +class TopicMetadataTest extends ZooKeeperTestHarness { private var server1: KafkaServer = null var brokerEndPoints: Seq[BrokerEndPoint] = null + @Before override def setUp() { super.setUp() val props = createBrokerConfigs(1, zkConnect) @@ -43,6 +44,7 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { brokerEndPoints = Seq(new Broker(server1.config.brokerId, server1.config.hostName, server1.boundPort()).getBrokerEndPoint(SecurityProtocol.PLAINTEXT)) } + @After override def tearDown() { server1.shutdown() super.tearDown() diff --git a/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala b/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala index e4bf2df..5a7c412 100755 --- a/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala +++ b/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala @@ -18,13 +18,11 @@ package kafka.integration import org.apache.kafka.common.config.ConfigException +import org.junit.{After, Before} -import scala.collection.mutable.MutableList import scala.util.Random import org.apache.log4j.{Level, Logger} -import org.scalatest.junit.JUnit3Suite import java.util.Properties -import junit.framework.Assert._ import kafka.admin.AdminUtils import kafka.common.FailedToSendMessageException import kafka.consumer.{Consumer, ConsumerConfig, ConsumerTimeoutException} @@ -34,8 +32,10 @@ import kafka.server.{KafkaConfig, KafkaServer} import kafka.utils.CoreUtils import kafka.utils.TestUtils._ import kafka.zk.ZooKeeperTestHarness +import org.junit.Assert._ +import org.scalatest.Assertions._ -class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { +class UncleanLeaderElectionTest extends ZooKeeperTestHarness { val brokerId1 = 0 val brokerId2 = 1 @@ -58,6 +58,7 @@ class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { val syncProducerLogger = Logger.getLogger(classOf[kafka.producer.SyncProducer]) val eventHandlerLogger = Logger.getLogger(classOf[kafka.producer.async.DefaultEventHandler[Object, Object]]) + @Before override def setUp() { super.setUp() @@ -77,6 +78,7 @@ class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { eventHandlerLogger.setLevel(Level.FATAL) } + @After override def tearDown() { servers.foreach(server => shutdownServer(server)) servers.foreach(server => CoreUtils.rm(server.config.logDirs)) diff --git a/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala b/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala index 74c761d..cf6b9a9 100644 --- a/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala +++ b/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala @@ -20,7 +20,6 @@ package kafka.javaapi.consumer import java.util.Properties import kafka.server._ -import kafka.message._ import kafka.serializer._ import kafka.integration.KafkaServerTestHarness import kafka.producer.KeyedMessage @@ -33,12 +32,11 @@ import kafka.common.MessageStreamsExistException import scala.collection.JavaConversions -import org.scalatest.junit.JUnit3Suite import org.apache.log4j.{Level, Logger} -import junit.framework.Assert._ +import org.junit.Assert._ -class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHarness with ZooKeeperTestHarness with Logging { +class ZookeeperConsumerConnectorTest extends KafkaServerTestHarness with ZooKeeperTestHarness with Logging { val numNodes = 2 val numParts = 2 val topic = "topic1" diff --git a/core/src/test/scala/unit/kafka/javaapi/message/BaseMessageSetTestCases.scala b/core/src/test/scala/unit/kafka/javaapi/message/BaseMessageSetTestCases.scala index 726399e..80f809e 100644 --- a/core/src/test/scala/unit/kafka/javaapi/message/BaseMessageSetTestCases.scala +++ b/core/src/test/scala/unit/kafka/javaapi/message/BaseMessageSetTestCases.scala @@ -17,7 +17,7 @@ package kafka.javaapi.message -import junit.framework.Assert._ +import org.junit.Assert._ import org.scalatest.junit.JUnitSuite import org.junit.Test import kafka.utils.TestUtils diff --git a/core/src/test/scala/unit/kafka/javaapi/message/ByteBufferMessageSetTest.scala b/core/src/test/scala/unit/kafka/javaapi/message/ByteBufferMessageSetTest.scala index 383fcef..fbdb000 100644 --- a/core/src/test/scala/unit/kafka/javaapi/message/ByteBufferMessageSetTest.scala +++ b/core/src/test/scala/unit/kafka/javaapi/message/ByteBufferMessageSetTest.scala @@ -17,7 +17,7 @@ package kafka.javaapi.message -import junit.framework.Assert._ +import org.junit.Assert._ import org.junit.Test import kafka.message.{DefaultCompressionCodec, CompressionCodec, NoCompressionCodec, Message} diff --git a/core/src/test/scala/unit/kafka/log/CleanerTest.scala b/core/src/test/scala/unit/kafka/log/CleanerTest.scala index 8b8249a..ae9f38a 100755 --- a/core/src/test/scala/unit/kafka/log/CleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/CleanerTest.scala @@ -17,7 +17,7 @@ package kafka.log -import junit.framework.Assert._ +import org.junit.Assert._ import org.scalatest.junit.JUnitSuite import org.junit.{After, Test} import java.nio._ diff --git a/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala b/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala index cec1cae..255a669 100644 --- a/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala +++ b/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala @@ -19,7 +19,7 @@ package kafka.log import java.nio._ import java.util.concurrent.atomic._ -import junit.framework.Assert._ +import org.junit.Assert._ import kafka.utils.TestUtils._ import kafka.message._ import org.junit.Test diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala index 471ddff..da80f6e 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala @@ -29,16 +29,14 @@ import org.junit._ import org.junit.runner.RunWith import org.junit.runners.Parameterized import org.junit.runners.Parameterized.Parameters -import org.scalatest.junit.JUnit3Suite import scala.collection._ - /** * This is an integration test that tests the fully integrated log cleaner */ @RunWith(value = classOf[Parameterized]) -class LogCleanerIntegrationTest(compressionCodec: String) extends JUnit3Suite { +class LogCleanerIntegrationTest(compressionCodec: String) { val time = new MockTime() val segmentSize = 100 diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala index 3fd5a53..0513df4 100644 --- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala @@ -21,9 +21,9 @@ import java.util.Properties import org.apache.kafka.common.config.ConfigException import org.junit.{Assert, Test} -import org.scalatest.junit.JUnit3Suite +import org.scalatest.Assertions._ -class LogConfigTest extends JUnit3Suite { +class LogConfigTest { @Test def testFromPropsDefaults() { diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index 01dfbc4..1e14f6e 100755 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -18,14 +18,13 @@ package kafka.log import java.io._ -import junit.framework.Assert._ -import org.junit.Test -import org.scalatest.junit.JUnit3Suite -import kafka.server.{BrokerState, OffsetCheckpoint} +import kafka.server.OffsetCheckpoint import kafka.common._ import kafka.utils._ +import org.junit.{After, Before, Test} +import org.junit.Assert._ -class LogManagerTest extends JUnit3Suite { +class LogManagerTest { val time: MockTime = new MockTime() val maxRollInterval = 100 @@ -36,20 +35,20 @@ class LogManagerTest extends JUnit3Suite { val name = "kafka" val veryLargeLogFlushInterval = 10000000L - override def setUp() { - super.setUp() + @Before + def setUp() { logDir = TestUtils.tempDir() logManager = createLogManager() logManager.startup logDir = logManager.logDirs(0) } - override def tearDown() { + @After + def tearDown() { if(logManager != null) logManager.shutdown() CoreUtils.rm(logDir) logManager.logDirs.foreach(CoreUtils.rm(_)) - super.tearDown() } /** diff --git a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala index 03fb351..25eb2a0 100644 --- a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala @@ -16,19 +16,15 @@ */ package kafka.log -import junit.framework.Assert._ +import org.junit.Assert._ import java.util.concurrent.atomic._ -import java.io.File -import java.io.RandomAccessFile -import java.util.Random import org.junit.{Test, After} -import org.scalatest.junit.JUnit3Suite import kafka.utils.TestUtils import kafka.message._ import kafka.utils.SystemTime import scala.collection._ -class LogSegmentTest extends JUnit3Suite { +class LogSegmentTest { val segments = mutable.ArrayBuffer[LogSegment]() diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index 8e095d6..e036ee8 100755 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -19,7 +19,7 @@ package kafka.log import java.io._ import java.util.concurrent.atomic._ -import junit.framework.Assert._ +import org.junit.Assert._ import org.scalatest.junit.JUnitSuite import org.junit.{After, Before, Test} import kafka.message._ diff --git a/core/src/test/scala/unit/kafka/log/OffsetIndexTest.scala b/core/src/test/scala/unit/kafka/log/OffsetIndexTest.scala index 9213a5d..dfd7b54 100644 --- a/core/src/test/scala/unit/kafka/log/OffsetIndexTest.scala +++ b/core/src/test/scala/unit/kafka/log/OffsetIndexTest.scala @@ -18,7 +18,7 @@ package kafka.log import java.io._ -import junit.framework.Assert._ +import org.junit.Assert._ import java.util.{Collections, Arrays} import org.junit._ import org.scalatest.junit.JUnitSuite diff --git a/core/src/test/scala/unit/kafka/log/OffsetMapTest.scala b/core/src/test/scala/unit/kafka/log/OffsetMapTest.scala index 12ce39e..f50daa4 100644 --- a/core/src/test/scala/unit/kafka/log/OffsetMapTest.scala +++ b/core/src/test/scala/unit/kafka/log/OffsetMapTest.scala @@ -20,7 +20,7 @@ package kafka.log import java.nio._ import org.junit._ import org.scalatest.junit.JUnitSuite -import junit.framework.Assert._ +import org.junit.Assert._ class OffsetMapTest extends JUnitSuite { diff --git a/core/src/test/scala/unit/kafka/log4j/KafkaLog4jAppenderTest.scala b/core/src/test/scala/unit/kafka/log4j/KafkaLog4jAppenderTest.scala index 41366a1..b4c13ed 100755 --- a/core/src/test/scala/unit/kafka/log4j/KafkaLog4jAppenderTest.scala +++ b/core/src/test/scala/unit/kafka/log4j/KafkaLog4jAppenderTest.scala @@ -31,11 +31,10 @@ import java.io.File import org.apache.log4j.spi.LoggingEvent import org.apache.log4j.{PropertyConfigurator, Logger} import org.junit.{After, Before, Test} -import org.scalatest.junit.JUnit3Suite -import junit.framework.Assert._ +import org.junit.Assert._ -class KafkaLog4jAppenderTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { +class KafkaLog4jAppenderTest extends ZooKeeperTestHarness with Logging { var logDirZk: File = null var config: KafkaConfig = null diff --git a/core/src/test/scala/unit/kafka/message/BaseMessageSetTestCases.scala b/core/src/test/scala/unit/kafka/message/BaseMessageSetTestCases.scala index dd8847f..208994b 100644 --- a/core/src/test/scala/unit/kafka/message/BaseMessageSetTestCases.scala +++ b/core/src/test/scala/unit/kafka/message/BaseMessageSetTestCases.scala @@ -18,7 +18,7 @@ package kafka.message import java.io.RandomAccessFile -import junit.framework.Assert._ +import org.junit.Assert._ import kafka.utils.TestUtils._ import kafka.log.FileMessageSet import org.scalatest.junit.JUnitSuite diff --git a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala b/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala index 07bc317..511060e 100644 --- a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala +++ b/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala @@ -19,7 +19,7 @@ package kafka.message import java.nio._ import java.util.concurrent.atomic.AtomicLong -import junit.framework.Assert._ +import org.junit.Assert._ import org.junit.Test import kafka.utils.TestUtils diff --git a/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala b/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala index 76987d4..f45bead 100644 --- a/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala +++ b/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala @@ -21,7 +21,7 @@ import java.io.ByteArrayOutputStream import scala.collection._ import org.scalatest.junit.JUnitSuite import org.junit._ -import junit.framework.Assert._ +import org.junit.Assert._ class MessageCompressionTest extends JUnitSuite { diff --git a/core/src/test/scala/unit/kafka/message/MessageTest.scala b/core/src/test/scala/unit/kafka/message/MessageTest.scala index 11c0f81..3c12d13 100755 --- a/core/src/test/scala/unit/kafka/message/MessageTest.scala +++ b/core/src/test/scala/unit/kafka/message/MessageTest.scala @@ -20,7 +20,7 @@ package kafka.message import java.nio._ import java.util.HashMap import scala.collection._ -import junit.framework.Assert._ +import org.junit.Assert._ import org.scalatest.junit.JUnitSuite import org.junit.{Before, Test} import kafka.utils.TestUtils diff --git a/core/src/test/scala/unit/kafka/message/MessageWriterTest.scala b/core/src/test/scala/unit/kafka/message/MessageWriterTest.scala index b08a343..3993fdb 100644 --- a/core/src/test/scala/unit/kafka/message/MessageWriterTest.scala +++ b/core/src/test/scala/unit/kafka/message/MessageWriterTest.scala @@ -20,7 +20,7 @@ package kafka.message import java.io.{InputStream, ByteArrayInputStream, ByteArrayOutputStream} import java.nio.ByteBuffer import java.util.Random -import junit.framework.Assert._ +import org.junit.Assert._ import org.junit.Test import org.scalatest.junit.JUnitSuite diff --git a/core/src/test/scala/unit/kafka/metrics/KafkaTimerTest.scala b/core/src/test/scala/unit/kafka/metrics/KafkaTimerTest.scala index 7df7405..3b3e4c3 100644 --- a/core/src/test/scala/unit/kafka/metrics/KafkaTimerTest.scala +++ b/core/src/test/scala/unit/kafka/metrics/KafkaTimerTest.scala @@ -18,12 +18,11 @@ package kafka.metrics */ import org.junit.Test -import org.scalatest.junit.JUnit3Suite import java.util.concurrent.TimeUnit -import junit.framework.Assert._ +import org.junit.Assert._ import com.yammer.metrics.core.{MetricsRegistry, Clock} -class KafkaTimerTest extends JUnit3Suite { +class KafkaTimerTest { @Test def testKafkaTimer() { diff --git a/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala index b42101b..5a268d1 100644 --- a/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala +++ b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala @@ -22,10 +22,9 @@ import java.util.Properties import com.yammer.metrics.Metrics import com.yammer.metrics.core.MetricPredicate import org.junit.Test -import junit.framework.Assert._ +import org.junit.Assert._ import kafka.integration.KafkaServerTestHarness import kafka.server._ -import kafka.message._ import kafka.serializer._ import kafka.utils._ import kafka.admin.AdminUtils @@ -33,9 +32,8 @@ import kafka.utils.TestUtils._ import scala.collection._ import scala.collection.JavaConversions._ import scala.util.matching.Regex -import org.scalatest.junit.JUnit3Suite -class MetricsTest extends JUnit3Suite with KafkaServerTestHarness with Logging { +class MetricsTest extends KafkaServerTestHarness with Logging { val numNodes = 2 val numParts = 2 val topic = "topic1" diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index 7dc2fad..b97a6eb 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -25,16 +25,14 @@ import org.apache.kafka.common.network.NetworkSend import org.apache.kafka.common.protocol.SecurityProtocol import org.apache.kafka.common.utils.SystemTime import org.junit._ -import org.scalatest.junit.JUnitSuite import java.util.Random -import junit.framework.Assert._ +import org.junit.Assert._ import kafka.producer.SyncProducerConfig import kafka.api.ProducerRequest import java.nio.ByteBuffer import kafka.common.TopicAndPartition import kafka.message.ByteBufferMessageSet -import java.nio.channels.SelectionKey -import kafka.utils.TestUtils +import org.scalatest.junit.JUnitSuite import scala.collection.Map class SocketServerTest extends JUnitSuite { @@ -84,11 +82,11 @@ class SocketServerTest extends JUnitSuite { new Socket("localhost", server.boundPort(protocol)) } - @After def cleanup() { server.shutdown() } + @Test def simpleRequest() { val plainSocket = connect(protocol = SecurityProtocol.PLAINTEXT) @@ -175,7 +173,7 @@ class SocketServerTest extends JUnitSuite { } @Test - def testMaxConnectionsPerIPOverrides(): Unit = { + def testMaxConnectionsPerIPOverrides() { val overrideNum = 6 val overrides: Map[String, Int] = Map("localhost" -> overrideNum) val overrideServer: SocketServer = new SocketServer(0, diff --git a/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala b/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala index be4bb87..b54f30e 100755 --- a/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala @@ -19,36 +19,27 @@ package kafka.producer import java.util.Properties import java.util.concurrent.LinkedBlockingQueue -import junit.framework.Assert._ +import org.junit.Assert._ import org.easymock.EasyMock import org.junit.Test import kafka.api._ -import kafka.cluster.{BrokerEndPoint, Broker} +import kafka.cluster.BrokerEndPoint import kafka.common._ import kafka.message._ import kafka.producer.async._ import kafka.serializer._ import kafka.server.KafkaConfig import kafka.utils.TestUtils._ -import org.scalatest.junit.JUnit3Suite import scala.collection.Map import scala.collection.mutable.ArrayBuffer import kafka.utils._ -class AsyncProducerTest extends JUnit3Suite { +class AsyncProducerTest { // One of the few cases we can just set a fixed port because the producer is mocked out here since this uses mocks 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() { - super.setUp() - } - - override def tearDown() { - super.tearDown() - } - @Test def testProducerQueueSize() { // a mock event handler that blocks diff --git a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala index 4d2536b..a7267b9 100755 --- a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala @@ -17,26 +17,23 @@ package kafka.producer -import org.scalatest.TestFailedException -import org.scalatest.junit.JUnit3Suite +import org.scalatest.{Assertions, TestFailedException} import kafka.consumer.SimpleConsumer import kafka.message.Message import kafka.server.{KafkaConfig, KafkaRequestHandler, KafkaServer} import kafka.zk.ZooKeeperTestHarness import org.apache.log4j.{Level, Logger} -import org.junit.Test +import org.junit.{After, Before, Test} import kafka.utils._ import java.util import kafka.admin.AdminUtils import util.Properties import kafka.api.FetchRequestBuilder -import org.junit.Assert.assertTrue -import org.junit.Assert.assertFalse -import org.junit.Assert.assertEquals import kafka.common.{ErrorMapping, FailedToSendMessageException} import kafka.serializer.StringEncoder +import org.junit.Assert._ -class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ +class ProducerTest extends ZooKeeperTestHarness with Logging{ private val brokerId1 = 0 private val brokerId2 = 1 private var server1: KafkaServer = null @@ -60,6 +57,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ consumer2 } + @Before override def setUp() { super.setUp() // set up 2 brokers with 4 partitions each @@ -81,6 +79,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ requestHandlerLogger.setLevel(Level.FATAL) } + @After override def tearDown() { // restore set request handler logger to a higher level requestHandlerLogger.setLevel(Level.ERROR) @@ -116,7 +115,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ fail("Test should fail because the broker list provided are not valid") } catch { case e: FailedToSendMessageException => // this is expected - case oe: Throwable => fail("fails with exception", oe) + case oe: Throwable => Assertions.fail("fails with exception", oe) } finally { producer1.close() } @@ -129,7 +128,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ try{ producer2.send(new KeyedMessage[String, String](topic, "test", "test1")) } catch { - case e: Throwable => fail("Should succeed sending the message", e) + case e: Throwable => Assertions.fail("Should succeed sending the message", e) } finally { producer2.close() } @@ -142,7 +141,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ try{ producer3.send(new KeyedMessage[String, String](topic, "test", "test1")) } catch { - case e: Throwable => fail("Should succeed sending the message", e) + case e: Throwable => Assertions.fail("Should succeed sending the message", e) } finally { producer3.close() } @@ -201,7 +200,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ } catch { case iae: IllegalArgumentException => // this is expected - case e: Throwable => fail("Not expected", e) + case e: Throwable => Assertions.fail("Not expected", e) } } @@ -261,7 +260,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ assertEquals(new Message(bytes = "test1".getBytes, key = "test".getBytes), messageSet1.next.message) assertFalse("Message set should have another message", messageSet1.hasNext) } catch { - case e: Exception => fail("Not expected", e) + case e: Exception => Assertions.fail("Not expected", e) } producer.close } @@ -295,7 +294,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ assertTrue("Message set should have 1 message", messageSet1.hasNext) assertEquals(new Message("test".getBytes), messageSet1.next.message) } catch { - case e: Throwable => case e: Exception => producer.close; fail("Not expected", e) + case e: Throwable => case e: Exception => producer.close; Assertions.fail("Not expected", e) } // stop IO threads and request handling, but leave networking operational @@ -309,7 +308,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ producer.send(new KeyedMessage[String, String](topic, "test", "test")) } catch { case e: FailedToSendMessageException => /* success */ - case e: Exception => fail("Not expected", e) + case e: Exception => Assertions.fail("Not expected", e) } finally { producer.close() } diff --git a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala index 8c3fb7a..90689f6 100644 --- a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala @@ -20,7 +20,7 @@ package kafka.producer import java.net.SocketTimeoutException import java.util.Properties -import junit.framework.Assert +import org.junit.Assert import kafka.admin.AdminUtils import kafka.api.ProducerResponseStatus import kafka.common.{ErrorMapping, TopicAndPartition} @@ -30,9 +30,8 @@ import kafka.server.KafkaConfig import kafka.utils._ import org.apache.kafka.common.protocol.SecurityProtocol import org.junit.Test -import org.scalatest.junit.JUnit3Suite -class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { +class SyncProducerTest extends KafkaServerTestHarness { private val messageBytes = new Array[Byte](2) // turning off controlled shutdown since testProducerCanTimeout() explicitly shuts down request handler pool. def generateConfigs() = List(KafkaConfig.fromProps(TestUtils.createBrokerConfigs(1, zkConnect, false).head)) diff --git a/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala b/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala index e899b02..066f506 100755 --- a/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala +++ b/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala @@ -17,18 +17,19 @@ package kafka.server -import junit.framework.Assert._ +import org.junit.Assert._ import kafka.utils.{TestUtils, CoreUtils, ZkUtils} import kafka.zk.ZooKeeperTestHarness import org.apache.kafka.common.protocol.SecurityProtocol -import org.scalatest.junit.JUnit3Suite +import org.junit.{After, Before} -class AdvertiseBrokerTest extends JUnit3Suite with ZooKeeperTestHarness { +class AdvertiseBrokerTest extends ZooKeeperTestHarness { var server : KafkaServer = null val brokerId = 0 val advertisedHostName = "routable-host" val advertisedPort = 1234 + @Before override def setUp() { super.setUp() @@ -39,6 +40,7 @@ class AdvertiseBrokerTest extends JUnit3Suite with ZooKeeperTestHarness { server = TestUtils.createServer(KafkaConfig.fromProps(props)) } + @After override def tearDown() { server.shutdown() CoreUtils.rm(server.config.logDirs) diff --git a/core/src/test/scala/unit/kafka/server/DelayedOperationTest.scala b/core/src/test/scala/unit/kafka/server/DelayedOperationTest.scala index f3ab3f4..df8d5b1 100644 --- a/core/src/test/scala/unit/kafka/server/DelayedOperationTest.scala +++ b/core/src/test/scala/unit/kafka/server/DelayedOperationTest.scala @@ -17,22 +17,21 @@ package kafka.server -import org.junit.Test -import org.scalatest.junit.JUnit3Suite -import junit.framework.Assert._ +import org.junit.{After, Before, Test} +import org.junit.Assert._ -class DelayedOperationTest extends JUnit3Suite { +class DelayedOperationTest { var purgatory: DelayedOperationPurgatory[MockDelayedOperation] = null - override def setUp() { - super.setUp() + @Before + def setUp() { purgatory = new DelayedOperationPurgatory[MockDelayedOperation](purgatoryName = "mock") } - override def tearDown() { + @After + def tearDown() { purgatory.shutdown() - super.tearDown() } @Test diff --git a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala index 7877f6c..405ccf8 100644 --- a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala @@ -16,16 +16,15 @@ */ package kafka.server -import junit.framework.Assert._ +import org.junit.Assert._ import org.junit.Test import kafka.integration.KafkaServerTestHarness import kafka.utils._ import kafka.common._ import kafka.log.LogConfig import kafka.admin.{AdminOperationException, AdminUtils} -import org.scalatest.junit.JUnit3Suite -class DynamicConfigChangeTest extends JUnit3Suite with KafkaServerTestHarness { +class DynamicConfigChangeTest extends KafkaServerTestHarness { def generateConfigs() = List(KafkaConfig.fromProps(TestUtils.createBrokerConfig(0, zkConnect))) @Test diff --git a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala index 60cd824..7f55a80 100755 --- a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala +++ b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala @@ -19,7 +19,6 @@ package kafka.server import kafka.log._ import java.io.File import org.I0Itec.zkclient.ZkClient -import org.scalatest.junit.JUnit3Suite import org.easymock.EasyMock import org.junit._ import org.junit.Assert._ @@ -28,7 +27,7 @@ import kafka.cluster.Replica import kafka.utils.{SystemTime, KafkaScheduler, TestUtils, MockTime, CoreUtils} import java.util.concurrent.atomic.AtomicBoolean -class HighwatermarkPersistenceTest extends JUnit3Suite { +class HighwatermarkPersistenceTest { val configs = TestUtils.createBrokerConfigs(2, TestUtils.MockZkConnect).map(KafkaConfig.fromProps) val topic = "foo" diff --git a/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala b/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala index 90529fa..25f0d41 100644 --- a/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala +++ b/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala @@ -18,7 +18,7 @@ package kafka.server import java.util.Properties -import org.scalatest.junit.JUnit3Suite +import org.junit.{Before, After} import collection.mutable.HashMap import collection.mutable.Map import kafka.cluster.{Partition, Replica} @@ -30,7 +30,7 @@ import java.util.concurrent.atomic.AtomicBoolean import kafka.message.MessageSet -class IsrExpirationTest extends JUnit3Suite { +class IsrExpirationTest { var topicPartitionIsr: Map[(String, Int), Seq[Int]] = new HashMap[(String, Int), Seq[Int]]() val replicaLagTimeMaxMs = 100L @@ -46,14 +46,14 @@ class IsrExpirationTest extends JUnit3Suite { var replicaManager: ReplicaManager = null - override def setUp() { - super.setUp() + @Before + def setUp() { replicaManager = new ReplicaManager(configs.head, time, null, null, null, new AtomicBoolean(false)) } - override def tearDown() { + @After + def tearDown() { replicaManager.shutdown(false) - super.tearDown() } /* diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala index c487f36..13e2bdd 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala @@ -22,12 +22,12 @@ import kafka.api.ApiVersion import kafka.message._ import org.apache.kafka.common.protocol.SecurityProtocol import org.junit.{Assert, Test} -import org.scalatest.junit.JUnit3Suite +import org.scalatest.Assertions._ import scala.collection.Map import scala.util.Random._ -class KafkaConfigConfigDefTest extends JUnit3Suite { +class KafkaConfigConfigDefTest { @Test def testFromPropsDefaults() { diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 2428dbd..192e814 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -19,15 +19,15 @@ package kafka.server import java.util.Properties -import junit.framework.Assert._ import kafka.api.{ApiVersion, KAFKA_082} import kafka.utils.{TestUtils, CoreUtils} import org.apache.kafka.common.config.ConfigException import org.apache.kafka.common.protocol.SecurityProtocol import org.junit.Test -import org.scalatest.junit.JUnit3Suite +import org.junit.Assert._ +import org.scalatest.Assertions._ -class KafkaConfigTest extends JUnit3Suite { +class KafkaConfigTest { @Test def testLogRetentionTimeHoursProvided() { diff --git a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala index f1977d8..f77f186 100755 --- a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala +++ b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala @@ -17,7 +17,7 @@ package kafka.server -import junit.framework.Assert._ +import org.junit.Assert._ import kafka.api._ import kafka.utils.{TestUtils, ZkUtils, CoreUtils} import kafka.cluster.Broker @@ -26,9 +26,9 @@ import kafka.controller.{ControllerChannelManager, ControllerContext, LeaderIsrA import kafka.utils.TestUtils._ import kafka.zk.ZooKeeperTestHarness import org.apache.kafka.common.protocol.SecurityProtocol -import org.scalatest.junit.JUnit3Suite +import org.junit.{After, Before} -class LeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { +class LeaderElectionTest extends ZooKeeperTestHarness { val brokerId1 = 0 val brokerId2 = 1 @@ -36,6 +36,7 @@ class LeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { var staleControllerEpochDetected = false + @Before override def setUp() { super.setUp() @@ -48,6 +49,7 @@ class LeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { servers ++= List(server1, server2) } + @After override def tearDown() { servers.foreach(_.shutdown()) servers.foreach(server => CoreUtils.rm(server.config.logDirs)) diff --git a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala index e57c1de..344001d 100755 --- a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala @@ -19,12 +19,11 @@ package kafka.server import java.io.File import kafka.utils._ -import junit.framework.Assert._ +import org.junit.Assert._ import java.util.{Random, Properties} import kafka.consumer.SimpleConsumer import kafka.message.{NoCompressionCodec, ByteBufferMessageSet, Message} import kafka.zk.ZooKeeperTestHarness -import org.scalatest.junit.JUnit3Suite import kafka.admin.AdminUtils import kafka.api.{PartitionOffsetRequestInfo, FetchRequestBuilder, OffsetRequest} import kafka.utils.TestUtils._ @@ -33,7 +32,7 @@ import org.junit.After import org.junit.Before import org.junit.Test -class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness { +class LogOffsetTest extends ZooKeeperTestHarness { val random = new Random() var logDir: File = null var topicLogDir: File = null diff --git a/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala b/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala index 7688f26..77e17e1 100755 --- a/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala @@ -27,10 +27,10 @@ import kafka.serializer.StringEncoder import java.io.File -import org.scalatest.junit.JUnit3Suite +import org.junit.{After, Before} import org.junit.Assert._ -class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { +class LogRecoveryTest extends ZooKeeperTestHarness { val replicaLagTimeMaxMs = 5000L val replicaLagMaxMessages = 10L @@ -69,6 +69,7 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { keyEncoder = classOf[IntEncoder].getName) } + @Before override def setUp() { super.setUp() @@ -86,6 +87,7 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { updateProducer() } + @After override def tearDown() { producer.close() for(server <- servers) { diff --git a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala index 528525b..5b0e4a5 100755 --- a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala @@ -25,7 +25,6 @@ import kafka.utils.TestUtils._ import kafka.zk.ZooKeeperTestHarness import org.junit.{After, Before, Test} -import org.scalatest.junit.JUnit3Suite import java.util.Properties import java.io.File @@ -33,9 +32,9 @@ import java.io.File import scala.util.Random import scala.collection._ -import junit.framework.Assert._ +import org.junit.Assert._ -class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { +class OffsetCommitTest extends ZooKeeperTestHarness { val random: Random = new Random() val group = "test-group" val retentionCheckInterval: Long = 100L diff --git a/core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala index a3a03db..dead087 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala @@ -17,7 +17,7 @@ package kafka.server -import org.scalatest.junit.JUnit3Suite +import org.junit.{After, Before} import kafka.zk.ZooKeeperTestHarness import kafka.utils.TestUtils._ import kafka.producer.KeyedMessage @@ -25,11 +25,12 @@ import kafka.serializer.StringEncoder import kafka.utils.{TestUtils} import kafka.common._ -class ReplicaFetchTest extends JUnit3Suite with ZooKeeperTestHarness { +class ReplicaFetchTest extends ZooKeeperTestHarness { var brokers: Seq[KafkaServer] = null val topic1 = "foo" val topic2 = "bar" + @Before override def setUp() { super.setUp() brokers = createBrokerConfigs(2, zkConnect, false) @@ -37,6 +38,7 @@ class ReplicaFetchTest extends JUnit3Suite with ZooKeeperTestHarness { .map(config => TestUtils.createServer(config)) } + @After override def tearDown() { brokers.foreach(_.shutdown()) super.tearDown() diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 00d5933..3770cb4 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -27,12 +27,11 @@ import java.io.File import org.apache.kafka.common.protocol.Errors import org.easymock.EasyMock import org.I0Itec.zkclient.ZkClient -import org.scalatest.junit.JUnit3Suite import org.junit.Test import scala.collection.Map -class ReplicaManagerTest extends JUnit3Suite { +class ReplicaManagerTest { val topic = "test-topic" @@ -84,7 +83,7 @@ class ReplicaManagerTest extends JUnit3Suite { rm.appendMessages(timeout = 0, requiredAcks = 3, internalTopicsAllowed = false, messagesPerPartition = produceRequest.data, responseCallback = callback) - rm.shutdown(false); + rm.shutdown(false) TestUtils.verifyNonDaemonThreadsStatus diff --git a/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala b/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala index 12269cd..1185a6f 100755 --- a/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala @@ -20,18 +20,18 @@ import java.util.Properties import kafka.zk.ZooKeeperTestHarness import kafka.utils.{TestUtils, CoreUtils} -import org.junit.Test -import org.scalatest.junit.JUnit3Suite -import junit.framework.Assert._ +import org.junit.{Before, Test} +import org.junit.Assert._ import java.io.File -class ServerGenerateBrokerIdTest extends JUnit3Suite with ZooKeeperTestHarness { +class ServerGenerateBrokerIdTest extends ZooKeeperTestHarness { var props1: Properties = null var config1: KafkaConfig = null var props2: Properties = null var config2: KafkaConfig = null val brokerMetaPropsFile = "meta.properties" + @Before override def setUp() { super.setUp() props1 = TestUtils.createBrokerConfig(-1, zkConnect) diff --git a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala index 95534e3..60086c5 100755 --- a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala @@ -16,7 +16,7 @@ */ package kafka.server -import kafka.zk.ZooKeeperTestHarness +import kafka.zk.{ZooKeeperTestHarnessBeforeAndAfter, ZooKeeperTestHarness} import kafka.consumer.SimpleConsumer import kafka.producer._ import kafka.utils.{IntEncoder, TestUtils, CoreUtils} @@ -27,19 +27,19 @@ import kafka.serializer.StringEncoder import java.io.File -import org.junit.Test -import org.scalatest.junit.JUnit3Suite -import junit.framework.Assert._ +import org.junit.{Before, Test} +import org.junit.Assert._ +import org.scalatest.BeforeAndAfter -class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness { +class ServerShutdownTest extends ZooKeeperTestHarnessBeforeAndAfter { var config: KafkaConfig = null val host = "localhost" val topic = "test" val sent1 = List("hello", "there") val sent2 = List("more", "messages") - override def setUp(): Unit = { - super.setUp() + override def beforeEach() { + super.beforeEach() val props = TestUtils.createBrokerConfig(0, zkConnect) config = KafkaConfig.fromProps(props) } diff --git a/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala b/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala index 60e10b3..7d986ad 100755 --- a/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala @@ -17,15 +17,14 @@ package kafka.server -import org.scalatest.junit.JUnit3Suite import kafka.utils.ZkUtils import kafka.utils.CoreUtils import kafka.utils.TestUtils import kafka.zk.ZooKeeperTestHarness -import junit.framework.Assert._ +import org.junit.Assert._ -class ServerStartupTest extends JUnit3Suite with ZooKeeperTestHarness { +class ServerStartupTest extends ZooKeeperTestHarness { def testBrokerCreatesZKChroot { val brokerId = 0 diff --git a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala index 09a0961..d950665 100644 --- a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala +++ b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala @@ -22,18 +22,17 @@ import kafka.cluster.Replica import kafka.common.TopicAndPartition import kafka.log.Log import kafka.message.{MessageSet, ByteBufferMessageSet, Message} +import org.junit.{After, Before} -import scala.Some import java.util.{Properties, Collections} import java.util.concurrent.atomic.AtomicBoolean import collection.JavaConversions._ import org.easymock.EasyMock import org.I0Itec.zkclient.ZkClient -import org.scalatest.junit.JUnit3Suite -import junit.framework.Assert._ +import org.junit.Assert._ -class SimpleFetchTest extends JUnit3Suite { +class SimpleFetchTest { val replicaLagTimeMaxMs = 100L val replicaFetchWaitMaxMs = 100 @@ -63,9 +62,8 @@ class SimpleFetchTest extends JUnit3Suite { var replicaManager: ReplicaManager = null - override def setUp() { - super.setUp() - + @Before + def setUp() { // create nice mock since we don't particularly care about zkclient calls val zkClient = EasyMock.createNiceMock(classOf[ZkClient]) EasyMock.replay(zkClient) @@ -117,9 +115,9 @@ class SimpleFetchTest extends JUnit3Suite { partition.inSyncReplicas = allReplicas.toSet } - override def tearDown() { + @After + def tearDown() { replicaManager.shutdown(false) - super.tearDown() } /** diff --git a/core/src/test/scala/unit/kafka/utils/ByteBoundedBlockingQueueTest.scala b/core/src/test/scala/unit/kafka/utils/ByteBoundedBlockingQueueTest.scala index fd8cf7b..4a070bd 100644 --- a/core/src/test/scala/unit/kafka/utils/ByteBoundedBlockingQueueTest.scala +++ b/core/src/test/scala/unit/kafka/utils/ByteBoundedBlockingQueueTest.scala @@ -19,7 +19,7 @@ package kafka.utils import java.util.concurrent.TimeUnit -import junit.framework.Assert._ +import org.junit.Assert._ import org.junit.Test class ByteBoundedBlockingQueueTest { diff --git a/core/src/test/scala/unit/kafka/utils/CommandLineUtilsTest.scala b/core/src/test/scala/unit/kafka/utils/CommandLineUtilsTest.scala index 6380b6e..068526e 100644 --- a/core/src/test/scala/unit/kafka/utils/CommandLineUtilsTest.scala +++ b/core/src/test/scala/unit/kafka/utils/CommandLineUtilsTest.scala @@ -17,7 +17,7 @@ package kafka.utils -import junit.framework.Assert._ +import org.junit.Assert._ import org.junit.Test class CommandLineUtilsTest { diff --git a/core/src/test/scala/unit/kafka/utils/IteratorTemplateTest.scala b/core/src/test/scala/unit/kafka/utils/IteratorTemplateTest.scala index fbd245c..56f5905 100644 --- a/core/src/test/scala/unit/kafka/utils/IteratorTemplateTest.scala +++ b/core/src/test/scala/unit/kafka/utils/IteratorTemplateTest.scala @@ -16,7 +16,7 @@ */ package kafka.utils -import junit.framework.Assert._ +import org.junit.Assert._ import org.scalatest.Assertions import org.junit.{Test, After, Before} diff --git a/core/src/test/scala/unit/kafka/utils/JsonTest.scala b/core/src/test/scala/unit/kafka/utils/JsonTest.scala index 93550e8..6c8ed97 100644 --- a/core/src/test/scala/unit/kafka/utils/JsonTest.scala +++ b/core/src/test/scala/unit/kafka/utils/JsonTest.scala @@ -16,7 +16,7 @@ */ package kafka.utils -import junit.framework.Assert._ +import org.junit.Assert._ import org.junit.{Test, After, Before} class JsonTest { diff --git a/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala b/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala index c96c0ff..1fda068 100644 --- a/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala +++ b/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala @@ -22,13 +22,12 @@ import kafka.server.{ReplicaFetcherManager, KafkaConfig} import kafka.api.LeaderAndIsr import kafka.zk.ZooKeeperTestHarness import kafka.common.TopicAndPartition -import org.scalatest.junit.JUnit3Suite import org.junit.Assert._ -import org.junit.Test +import org.junit.{Before, Test} import org.easymock.EasyMock -class ReplicationUtilsTest extends JUnit3Suite with ZooKeeperTestHarness { +class ReplicationUtilsTest extends ZooKeeperTestHarness { val topic = "my-topic-test" val partitionId = 0 val brokerId = 1 @@ -45,7 +44,7 @@ class ReplicationUtilsTest extends JUnit3Suite with ZooKeeperTestHarness { val topicDataLeaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(LeaderAndIsr(1,leaderEpoch,List(1,2),0), controllerEpoch) - + @Before override def setUp() { super.setUp() ZkUtils.createPersistentPath(zkClient,topicPath,topicData) diff --git a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala index cfea63b..7c131fc 100644 --- a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala +++ b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala @@ -16,7 +16,7 @@ */ package kafka.utils -import junit.framework.Assert._ +import org.junit.Assert._ import java.util.concurrent.atomic._ import org.junit.{Test, After, Before} import kafka.utils.TestUtils.retry diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 17e9fe4..a241473 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -43,8 +43,7 @@ import kafka.admin.AdminUtils import kafka.producer.ProducerConfig import kafka.log._ -import junit.framework.AssertionFailedError -import junit.framework.Assert._ +import org.junit.Assert._ import org.apache.kafka.clients.producer.KafkaProducer import scala.collection.Map @@ -594,7 +593,7 @@ object TestUtils extends Logging { block return } catch { - case e: AssertionFailedError => + case e: AssertionError => val ellapsed = System.currentTimeMillis - startTime if(ellapsed > maxWaitMs) { throw e diff --git a/core/src/test/scala/unit/kafka/utils/timer/TimerTaskListTest.scala b/core/src/test/scala/unit/kafka/utils/timer/TimerTaskListTest.scala index 052aecd..a018dde 100644 --- a/core/src/test/scala/unit/kafka/utils/timer/TimerTaskListTest.scala +++ b/core/src/test/scala/unit/kafka/utils/timer/TimerTaskListTest.scala @@ -16,7 +16,7 @@ */ package kafka.utils.timer -import junit.framework.Assert._ +import org.junit.Assert._ import java.util.concurrent.atomic._ import org.junit.{Test, After, Before} diff --git a/core/src/test/scala/unit/kafka/utils/timer/TimerTest.scala b/core/src/test/scala/unit/kafka/utils/timer/TimerTest.scala index 8507592..95de378 100644 --- a/core/src/test/scala/unit/kafka/utils/timer/TimerTest.scala +++ b/core/src/test/scala/unit/kafka/utils/timer/TimerTest.scala @@ -18,7 +18,7 @@ package kafka.utils.timer import java.util.concurrent.{CountDownLatch, ExecutorService, Executors, TimeUnit} -import junit.framework.Assert._ +import org.junit.Assert._ import java.util.concurrent.atomic._ import org.junit.{Test, After, Before} diff --git a/core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala b/core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala index 2be1619..247aa6e 100644 --- a/core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala +++ b/core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala @@ -18,13 +18,11 @@ package kafka.zk import kafka.consumer.ConsumerConfig -import org.I0Itec.zkclient.ZkClient import kafka.utils.ZkUtils import kafka.utils.TestUtils import org.junit.Assert -import org.scalatest.junit.JUnit3Suite -class ZKEphemeralTest extends JUnit3Suite with ZooKeeperTestHarness { +class ZKEphemeralTest extends ZooKeeperTestHarness { var zkSessionTimeoutMs = 1000 def testEphemeralNodeCleanup = { diff --git a/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala b/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala index d3e44c6..35c635a 100644 --- a/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala +++ b/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala @@ -17,13 +17,12 @@ package kafka.zk -import junit.framework.Assert import kafka.consumer.ConsumerConfig import kafka.utils.{ZkPath, TestUtils, ZkUtils} import org.apache.kafka.common.config.ConfigException -import org.scalatest.junit.JUnit3Suite +import org.junit.Assert._ -class ZKPathTest extends JUnit3Suite with ZooKeeperTestHarness { +class ZKPathTest extends ZooKeeperTestHarness { val path: String = "/some_dir" val zkSessionTimeoutMs = 1000 @@ -54,7 +53,7 @@ class ZKPathTest extends JUnit3Suite with ZooKeeperTestHarness { case exception: Throwable => fail("Failed to create persistent path") } - Assert.assertTrue("Failed to create persistent path", ZkUtils.pathExists(zkClient, path)) + assertTrue("Failed to create persistent path", ZkUtils.pathExists(zkClient, path)) } def testMakeSurePersistsPathExistsThrowsException { @@ -82,7 +81,7 @@ class ZKPathTest extends JUnit3Suite with ZooKeeperTestHarness { case exception: Throwable => fail("Failed to create persistent path") } - Assert.assertTrue("Failed to create persistent path", ZkUtils.pathExists(zkClient, path)) + assertTrue("Failed to create persistent path", ZkUtils.pathExists(zkClient, path)) } def testCreateEphemeralPathThrowsException { @@ -110,7 +109,7 @@ class ZKPathTest extends JUnit3Suite with ZooKeeperTestHarness { case exception: Throwable => fail("Failed to create ephemeral path") } - Assert.assertTrue("Failed to create ephemeral path", ZkUtils.pathExists(zkClient, path)) + assertTrue("Failed to create ephemeral path", ZkUtils.pathExists(zkClient, path)) } def testCreatePersistentSequentialThrowsException { @@ -140,6 +139,6 @@ class ZKPathTest extends JUnit3Suite with ZooKeeperTestHarness { case exception: Throwable => fail("Failed to create persistent path") } - Assert.assertTrue("Failed to create persistent path", ZkUtils.pathExists(zkClient, actualPath)) + assertTrue("Failed to create persistent path", ZkUtils.pathExists(zkClient, actualPath)) } } diff --git a/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala b/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala index 1f4d10d..ed5fe9b 100755 --- a/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala +++ b/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala @@ -17,11 +17,11 @@ package kafka.zk -import org.scalatest.junit.JUnit3Suite import org.I0Itec.zkclient.ZkClient import kafka.utils.{ZkUtils, CoreUtils} +import org.junit.{After, Before} -trait ZooKeeperTestHarness extends JUnit3Suite { +trait ZooKeeperTestHarness { var zkPort: Int = -1 var zookeeper: EmbeddedZookeeper = null var zkClient: ZkClient = null @@ -30,17 +30,17 @@ trait ZooKeeperTestHarness extends JUnit3Suite { def zkConnect: String = "127.0.0.1:" + zkPort - override def setUp() { - super.setUp + @Before + def setUp() { zookeeper = new EmbeddedZookeeper() zkPort = zookeeper.port zkClient = ZkUtils.createZkClient(zkConnect, zkSessionTimeout, zkConnectionTimeout) } - override def tearDown() { + @After + def tearDown() { CoreUtils.swallow(zkClient.close()) CoreUtils.swallow(zookeeper.shutdown()) - super.tearDown } } -- 2.4.3