From 9f80665ec6deff8525b61096034af8dc0cc9a03c Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 10 Jun 2015 11:28:53 -0700 Subject: [PATCH 001/120] 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 } } -- 1.7.12.4 From ab76dbd1f7f0bfdc9841baee64cfa37de1d870fb Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Thu, 11 Jun 2015 15:27:51 -0700 Subject: [PATCH 002/120] 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 -- 1.7.12.4 From 017c00caf44aaad3418cb99d3ef42c4d1b066ddd Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Fri, 12 Jun 2015 10:24:54 -0700 Subject: [PATCH 003/120] 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); -- 1.7.12.4 From d31a2c2381bebc9c4b27e36fdf986183732e13eb Mon Sep 17 00:00:00 2001 From: Alexander Pakulov Date: Fri, 12 Jun 2015 14:16:03 -0700 Subject: [PATCH 004/120] 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 -- 1.7.12.4 From 719f2bddd147d4583a15ee8398bb0edccbbcc3f4 Mon Sep 17 00:00:00 2001 From: Alexey Ozeritskiy Date: Fri, 12 Jun 2015 18:45:48 -0700 Subject: [PATCH 005/120] 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 } } -- 1.7.12.4 From 20a31a29f7aa6ce6687a13aa0cf60b92c5ac4d1e Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Mon, 15 Jun 2015 17:43:56 -0700 Subject: [PATCH 006/120] 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() { -- 1.7.12.4 From 54e54f08077c9d71a5121e640b55836e6f7f2c9b Mon Sep 17 00:00:00 2001 From: Andrii Biletskyi Date: Tue, 16 Jun 2015 14:46:48 -0700 Subject: [PATCH 007/120] 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. -- 1.7.12.4 From 395716ebf0af2520a33ef352a62c10a71239bb4f Mon Sep 17 00:00:00 2001 From: Proneet Verma Date: Tue, 16 Jun 2015 15:03:40 -0700 Subject: [PATCH 008/120] 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 -- 1.7.12.4 From 28ecea421794d0c9a1c4f95375ccd1a6dfd8f365 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Tue, 16 Jun 2015 15:25:16 -0700 Subject: [PATCH 009/120] 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") } } -- 1.7.12.4 From 7009f1d6fffe3866723d1d33a28a4572053eb4e5 Mon Sep 17 00:00:00 2001 From: Manikumar Reddy Date: Tue, 16 Jun 2015 15:30:52 -0700 Subject: [PATCH 010/120] 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); -- 1.7.12.4 From 478505632edc8f4b51e4ed561d1adf455256c3e4 Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Tue, 16 Jun 2015 15:37:58 -0700 Subject: [PATCH 011/120] 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); -- 1.7.12.4 From 9650e12df2502f85ea665ae46a982d6fd0d2a954 Mon Sep 17 00:00:00 2001 From: Manikumar Reddy Date: Tue, 16 Jun 2015 15:50:17 -0700 Subject: [PATCH 012/120] 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) -- 1.7.12.4 From ba86f0a25dd8675d3c840c33605494a2f637576f Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Tue, 16 Jun 2015 16:41:20 -0700 Subject: [PATCH 013/120] 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: -- 1.7.12.4 From 5c9040745466945a04ea0315de583ccdab0614ac Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Thu, 18 Jun 2015 14:07:33 -0700 Subject: [PATCH 014/120] kafka-2249; KafkaConfig does not preserve original Properties; patched by Gwen Shapira; reviewed by Jun Rao --- .../apache/kafka/common/config/AbstractConfig.java | 12 +- core/src/main/scala/kafka/cluster/Partition.scala | 2 +- .../scala/kafka/controller/KafkaController.scala | 4 +- .../kafka/controller/PartitionLeaderSelector.scala | 2 +- core/src/main/scala/kafka/log/LogConfig.scala | 156 ++---- core/src/main/scala/kafka/log/LogManager.scala | 2 +- core/src/main/scala/kafka/server/KafkaApis.scala | 4 +- core/src/main/scala/kafka/server/KafkaConfig.scala | 573 ++++++--------------- core/src/main/scala/kafka/server/KafkaServer.scala | 55 +- .../scala/kafka/server/ReplicaFetcherThread.scala | 4 +- .../scala/kafka/server/TopicConfigManager.scala | 5 +- core/src/main/scala/kafka/utils/CoreUtils.scala | 26 - .../src/test/scala/other/kafka/StressTestLog.scala | 10 +- .../scala/other/kafka/TestLinearWriteSpeed.scala | 7 +- .../unit/kafka/log/BrokerCompressionTest.scala | 7 +- .../test/scala/unit/kafka/log/CleanerTest.scala | 55 +- .../unit/kafka/log/LogCleanerIntegrationTest.scala | 8 +- .../test/scala/unit/kafka/log/LogConfigTest.scala | 19 +- .../test/scala/unit/kafka/log/LogManagerTest.scala | 17 +- core/src/test/scala/unit/kafka/log/LogTest.scala | 121 +++-- .../kafka/server/DynamicConfigChangeTest.scala | 17 +- .../kafka/server/KafkaConfigConfigDefTest.scala | 20 +- 22 files changed, 444 insertions(+), 682 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java index c4fa058..bae528d 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java @@ -57,15 +57,19 @@ public class AbstractConfig { return values.get(key); } - public int getInt(String key) { + public Short getShort(String key) { + return (Short) get(key); + } + + public Integer getInt(String key) { return (Integer) get(key); } - public long getLong(String key) { + public Long getLong(String key) { return (Long) get(key); } - public double getDouble(String key) { + public Double getDouble(String key) { return (Double) get(key); } @@ -92,7 +96,7 @@ public class AbstractConfig { return keys; } - public Map originals() { + public Map originals() { Map copy = new HashMap(); copy.putAll(originals); return copy; diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 730a232..6cb6477 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -86,7 +86,7 @@ class Partition(val topic: String, case Some(replica) => replica case None => if (isReplicaLocal(replicaId)) { - val config = LogConfig.fromProps(logManager.defaultConfig.toProps, AdminUtils.fetchTopicConfig(zkClient, topic)) + val config = LogConfig.fromProps(logManager.defaultConfig.originals, AdminUtils.fetchTopicConfig(zkClient, topic)) val log = logManager.createLog(TopicAndPartition(topic, partitionId), config) val checkpoint = replicaManager.highWatermarkCheckpoints(log.dir.getParentFile.getAbsolutePath) val offsetMap = checkpoint.read diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 69bba24..3635057 100755 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -325,7 +325,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt info("starting the partition rebalance scheduler") autoRebalanceScheduler.startup() autoRebalanceScheduler.schedule("partition-rebalance-thread", checkAndTriggerPartitionRebalance, - 5, config.leaderImbalanceCheckIntervalSeconds, TimeUnit.SECONDS) + 5, config.leaderImbalanceCheckIntervalSeconds.toLong, TimeUnit.SECONDS) } deleteTopicManager.start() } @@ -1013,7 +1013,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt // if the replica to be removed from the ISR is the last surviving member of the ISR and unclean leader election // is disallowed for the corresponding topic, then we must preserve the ISR membership so that the replica can // eventually be restored as the leader. - if (newIsr.isEmpty && !LogConfig.fromProps(config.toProps, AdminUtils.fetchTopicConfig(zkClient, + if (newIsr.isEmpty && !LogConfig.fromProps(config.originals, AdminUtils.fetchTopicConfig(zkClient, topicAndPartition.topic)).uncleanLeaderElectionEnable) { info("Retaining last ISR %d of partition %s since unclean leader election is disabled".format(replicaId, topicAndPartition)) newIsr = leaderAndIsr.isr diff --git a/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala b/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala index 3b15ab4..bb6b5c8 100644 --- a/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala +++ b/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala @@ -61,7 +61,7 @@ class OfflinePartitionLeaderSelector(controllerContext: ControllerContext, confi case true => // Prior to electing an unclean (i.e. non-ISR) leader, ensure that doing so is not disallowed by the configuration // for unclean leader election. - if (!LogConfig.fromProps(config.toProps, AdminUtils.fetchTopicConfig(controllerContext.zkClient, + if (!LogConfig.fromProps(config.originals, AdminUtils.fetchTopicConfig(controllerContext.zkClient, topicAndPartition.topic)).uncleanLeaderElectionEnable) { throw new NoReplicaOnlineException(("No broker in ISR for partition " + "%s is alive. Live brokers are: [%s],".format(topicAndPartition, controllerContext.liveBrokerIds)) + diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala index f64fd79..e9af221 100755 --- a/core/src/main/scala/kafka/log/LogConfig.scala +++ b/core/src/main/scala/kafka/log/LogConfig.scala @@ -18,92 +18,52 @@ package kafka.log import java.util.Properties +import kafka.server.KafkaConfig import org.apache.kafka.common.utils.Utils import scala.collection._ -import org.apache.kafka.common.config.ConfigDef +import org.apache.kafka.common.config.{AbstractConfig, ConfigDef} import kafka.message.BrokerCompressionCodec import kafka.message.Message object Defaults { - val SegmentSize = 1024 * 1024 - val SegmentMs = Long.MaxValue - val SegmentJitterMs = 0L - val FlushInterval = Long.MaxValue - val FlushMs = Long.MaxValue - val RetentionSize = Long.MaxValue - val RetentionMs = Long.MaxValue - val MaxMessageSize = Int.MaxValue - val MaxIndexSize = 1024 * 1024 - val IndexInterval = 4096 - val FileDeleteDelayMs = 60 * 1000L - val DeleteRetentionMs = 24 * 60 * 60 * 1000L - val MinCleanableDirtyRatio = 0.5 - val Compact = false - val UncleanLeaderElectionEnable = true - val MinInSyncReplicas = 1 - val CompressionType = "producer" + val SegmentSize = kafka.server.Defaults.LogSegmentBytes + val SegmentMs = kafka.server.Defaults.LogRollHours * 60 * 60 * 1000L + val SegmentJitterMs = kafka.server.Defaults.LogRollJitterHours * 60 * 60 * 1000L + val FlushInterval = kafka.server.Defaults.LogFlushIntervalMessages + val FlushMs = kafka.server.Defaults.LogFlushSchedulerIntervalMs + val RetentionSize = kafka.server.Defaults.LogRetentionBytes + val RetentionMs = kafka.server.Defaults.LogRetentionHours * 60 * 60 * 1000L + val MaxMessageSize = kafka.server.Defaults.MessageMaxBytes + val MaxIndexSize = kafka.server.Defaults.LogIndexSizeMaxBytes + val IndexInterval = kafka.server.Defaults.LogIndexIntervalBytes + val FileDeleteDelayMs = kafka.server.Defaults.LogDeleteDelayMs + val DeleteRetentionMs = kafka.server.Defaults.LogCleanerDeleteRetentionMs + val MinCleanableDirtyRatio = kafka.server.Defaults.LogCleanerMinCleanRatio + val Compact = kafka.server.Defaults.LogCleanupPolicy + val UncleanLeaderElectionEnable = kafka.server.Defaults.UncleanLeaderElectionEnable + val MinInSyncReplicas = kafka.server.Defaults.MinInSyncReplicas + val CompressionType = kafka.server.Defaults.CompressionType } -/** - * Configuration settings for a log - * @param segmentSize The hard maximum for the size of a segment file in the log - * @param segmentMs The soft maximum on the amount of time before a new log segment is rolled - * @param segmentJitterMs The maximum random jitter subtracted from segmentMs to avoid thundering herds of segment rolling - * @param flushInterval The number of messages that can be written to the log before a flush is forced - * @param flushMs The amount of time the log can have dirty data before a flush is forced - * @param retentionSize The approximate total number of bytes this log can use - * @param retentionMs The approximate maximum age of the last segment that is retained - * @param maxIndexSize The maximum size of an index file - * @param indexInterval The approximate number of bytes between index entries - * @param fileDeleteDelayMs The time to wait before deleting a file from the filesystem - * @param deleteRetentionMs The time to retain delete markers in the log. Only applicable for logs that are being compacted. - * @param minCleanableRatio The ratio of bytes that are available for cleaning to the bytes already cleaned - * @param compact Should old segments in this log be deleted or deduplicated? - * @param uncleanLeaderElectionEnable Indicates whether unclean leader election is enabled - * @param minInSyncReplicas If number of insync replicas drops below this number, we stop accepting writes with -1 (or all) required acks - * @param compressionType compressionType for a given topic - * - */ -case class LogConfig(segmentSize: Int = Defaults.SegmentSize, - segmentMs: Long = Defaults.SegmentMs, - segmentJitterMs: Long = Defaults.SegmentJitterMs, - flushInterval: Long = Defaults.FlushInterval, - flushMs: Long = Defaults.FlushMs, - retentionSize: Long = Defaults.RetentionSize, - retentionMs: Long = Defaults.RetentionMs, - maxMessageSize: Int = Defaults.MaxMessageSize, - maxIndexSize: Int = Defaults.MaxIndexSize, - indexInterval: Int = Defaults.IndexInterval, - fileDeleteDelayMs: Long = Defaults.FileDeleteDelayMs, - deleteRetentionMs: Long = Defaults.DeleteRetentionMs, - minCleanableRatio: Double = Defaults.MinCleanableDirtyRatio, - compact: Boolean = Defaults.Compact, - uncleanLeaderElectionEnable: Boolean = Defaults.UncleanLeaderElectionEnable, - minInSyncReplicas: Int = Defaults.MinInSyncReplicas, - compressionType: String = Defaults.CompressionType) { - - def toProps: Properties = { - val props = new Properties() - import LogConfig._ - props.put(SegmentBytesProp, segmentSize.toString) - props.put(SegmentMsProp, segmentMs.toString) - props.put(SegmentJitterMsProp, segmentJitterMs.toString) - props.put(SegmentIndexBytesProp, maxIndexSize.toString) - props.put(FlushMessagesProp, flushInterval.toString) - props.put(FlushMsProp, flushMs.toString) - props.put(RetentionBytesProp, retentionSize.toString) - props.put(RetentionMsProp, retentionMs.toString) - props.put(MaxMessageBytesProp, maxMessageSize.toString) - props.put(IndexIntervalBytesProp, indexInterval.toString) - props.put(DeleteRetentionMsProp, deleteRetentionMs.toString) - props.put(FileDeleteDelayMsProp, fileDeleteDelayMs.toString) - props.put(MinCleanableDirtyRatioProp, minCleanableRatio.toString) - props.put(CleanupPolicyProp, if(compact) "compact" else "delete") - props.put(UncleanLeaderElectionEnableProp, uncleanLeaderElectionEnable.toString) - props.put(MinInSyncReplicasProp, minInSyncReplicas.toString) - props.put(CompressionTypeProp, compressionType) - props - } +case class LogConfig(props: java.util.Map[_, _]) extends AbstractConfig(LogConfig.configDef, props) { + + val segmentSize = getInt(LogConfig.SegmentBytesProp) + val segmentMs = getLong(LogConfig.SegmentMsProp) + val segmentJitterMs = getLong(LogConfig.SegmentJitterMsProp) + val maxIndexSize = getInt(LogConfig.SegmentIndexBytesProp) + val flushInterval = getLong(LogConfig.FlushMessagesProp) + val flushMs = getLong(LogConfig.FlushMsProp) + val retentionSize = getLong(LogConfig.RetentionBytesProp) + val retentionMs = getLong(LogConfig.RetentionMsProp) + val maxMessageSize = getInt(LogConfig.MaxMessageBytesProp) + val indexInterval = getInt(LogConfig.IndexIntervalBytesProp) + val fileDeleteDelayMs = getLong(LogConfig.FileDeleteDelayMsProp) + val deleteRetentionMs = getLong(LogConfig.DeleteRetentionMsProp) + val minCleanableRatio = getDouble(LogConfig.MinCleanableDirtyRatioProp) + val compact = getString(LogConfig.CleanupPolicyProp).toLowerCase != LogConfig.Delete + val uncleanLeaderElectionEnable = getBoolean(LogConfig.UncleanLeaderElectionEnableProp) + val minInSyncReplicas = getInt(LogConfig.MinInSyncReplicasProp) + val compressionType = getString(LogConfig.CompressionTypeProp).toLowerCase def randomSegmentJitter: Long = if (segmentJitterMs == 0) 0 else Utils.abs(scala.util.Random.nextInt()) % math.min(segmentJitterMs, segmentMs) @@ -111,6 +71,10 @@ case class LogConfig(segmentSize: Int = Defaults.SegmentSize, object LogConfig { + def main(args: Array[String]) { + System.out.println(configDef.toHtmlTable) + } + val Delete = "delete" val Compact = "compact" @@ -179,7 +143,7 @@ object LogConfig { .define(FileDeleteDelayMsProp, LONG, Defaults.FileDeleteDelayMs, atLeast(0), MEDIUM, FileDeleteDelayMsDoc) .define(MinCleanableDirtyRatioProp, DOUBLE, Defaults.MinCleanableDirtyRatio, between(0, 1), MEDIUM, MinCleanableRatioDoc) - .define(CleanupPolicyProp, STRING, if (Defaults.Compact) Compact else Delete, in(Compact, Delete), MEDIUM, + .define(CleanupPolicyProp, STRING, Defaults.Compact, in(Compact, Delete), MEDIUM, CompactDoc) .define(UncleanLeaderElectionEnableProp, BOOLEAN, Defaults.UncleanLeaderElectionEnable, MEDIUM, UncleanLeaderElectionEnableDoc) @@ -187,6 +151,8 @@ object LogConfig { .define(CompressionTypeProp, STRING, Defaults.CompressionType, in(BrokerCompressionCodec.brokerCompressionOptions:_*), MEDIUM, CompressionTypeDoc) } + def apply(): LogConfig = LogConfig(new Properties()) + def configNames() = { import JavaConversions._ configDef.names().toList.sorted @@ -194,37 +160,13 @@ object LogConfig { /** - * Parse the given properties instance into a LogConfig object - */ - def fromProps(props: Properties): LogConfig = { - import kafka.utils.CoreUtils.evaluateDefaults - val parsed = configDef.parse(evaluateDefaults(props)) - new LogConfig(segmentSize = parsed.get(SegmentBytesProp).asInstanceOf[Int], - segmentMs = parsed.get(SegmentMsProp).asInstanceOf[Long], - segmentJitterMs = parsed.get(SegmentJitterMsProp).asInstanceOf[Long], - maxIndexSize = parsed.get(SegmentIndexBytesProp).asInstanceOf[Int], - flushInterval = parsed.get(FlushMessagesProp).asInstanceOf[Long], - flushMs = parsed.get(FlushMsProp).asInstanceOf[Long], - retentionSize = parsed.get(RetentionBytesProp).asInstanceOf[Long], - retentionMs = parsed.get(RetentionMsProp).asInstanceOf[Long], - maxMessageSize = parsed.get(MaxMessageBytesProp).asInstanceOf[Int], - indexInterval = parsed.get(IndexIntervalBytesProp).asInstanceOf[Int], - fileDeleteDelayMs = parsed.get(FileDeleteDelayMsProp).asInstanceOf[Long], - deleteRetentionMs = parsed.get(DeleteRetentionMsProp).asInstanceOf[Long], - minCleanableRatio = parsed.get(MinCleanableDirtyRatioProp).asInstanceOf[Double], - compact = parsed.get(CleanupPolicyProp).asInstanceOf[String].toLowerCase != Delete, - uncleanLeaderElectionEnable = parsed.get(UncleanLeaderElectionEnableProp).asInstanceOf[Boolean], - minInSyncReplicas = parsed.get(MinInSyncReplicasProp).asInstanceOf[Int], - compressionType = parsed.get(CompressionTypeProp).asInstanceOf[String].toLowerCase()) - } - - /** * Create a log config instance using the given properties and defaults */ - def fromProps(defaults: Properties, overrides: Properties): LogConfig = { - val props = new Properties(defaults) + def fromProps(defaults: java.util.Map[_ <: Object, _ <: Object], overrides: Properties): LogConfig = { + val props = new Properties() + props.putAll(defaults) props.putAll(overrides) - fromProps(props) + LogConfig(props) } /** diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index e781eba..538fc83 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -356,7 +356,7 @@ class LogManager(val logDirs: Array[File], .format(topicAndPartition.topic, topicAndPartition.partition, dataDir.getAbsolutePath, - {import JavaConversions._; config.toProps.mkString(", ")})) + {import JavaConversions._; config.originals.mkString(", ")})) log } } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index c7debe4..ad6f058 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -428,9 +428,9 @@ class KafkaApis(val requestChannel: RequestChannel, val aliveBrokers = metadataCache.getAliveBrokers val offsetsTopicReplicationFactor = if (aliveBrokers.length > 0) - Math.min(config.offsetsTopicReplicationFactor, aliveBrokers.length) + Math.min(config.offsetsTopicReplicationFactor.toInt, aliveBrokers.length) else - config.offsetsTopicReplicationFactor + config.offsetsTopicReplicationFactor.toInt AdminUtils.createTopic(zkClient, topic, config.offsetsTopicPartitions, offsetsTopicReplicationFactor, offsetManager.offsetsTopicConfig) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 2d75186..e0b2480 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -26,7 +26,7 @@ import kafka.consumer.ConsumerConfig import kafka.message.{BrokerCompressionCodec, CompressionCodec, Message, MessageSet} import kafka.utils.CoreUtils import org.apache.kafka.clients.CommonClientConfigs -import org.apache.kafka.common.config.ConfigDef +import org.apache.kafka.common.config.{ConfigException, AbstractConfig, ConfigDef} import org.apache.kafka.common.metrics.MetricsReporter import org.apache.kafka.common.protocol.SecurityProtocol import scala.collection.{mutable, immutable, JavaConversions, Map} @@ -141,6 +141,10 @@ object Defaults { object KafkaConfig { + def main(args: Array[String]) { + System.out.println(configDef.toHtmlTable) + } + /** ********* Zookeeper Configuration ***********/ val ZkConnectProp = "zookeeper.connect" val ZkSessionTimeoutMsProp = "zookeeper.session.timeout.ms" @@ -482,14 +486,14 @@ object KafkaConfig { .define(ProducerPurgatoryPurgeIntervalRequestsProp, INT, Defaults.ProducerPurgatoryPurgeIntervalRequests, MEDIUM, ProducerPurgatoryPurgeIntervalRequestsDoc) .define(AutoLeaderRebalanceEnableProp, BOOLEAN, Defaults.AutoLeaderRebalanceEnable, HIGH, AutoLeaderRebalanceEnableDoc) .define(LeaderImbalancePerBrokerPercentageProp, INT, Defaults.LeaderImbalancePerBrokerPercentage, HIGH, LeaderImbalancePerBrokerPercentageDoc) - .define(LeaderImbalanceCheckIntervalSecondsProp, INT, Defaults.LeaderImbalanceCheckIntervalSeconds, HIGH, LeaderImbalanceCheckIntervalSecondsDoc) + .define(LeaderImbalanceCheckIntervalSecondsProp, LONG, Defaults.LeaderImbalanceCheckIntervalSeconds, HIGH, LeaderImbalanceCheckIntervalSecondsDoc) .define(UncleanLeaderElectionEnableProp, BOOLEAN, Defaults.UncleanLeaderElectionEnable, HIGH, UncleanLeaderElectionEnableDoc) .define(InterBrokerSecurityProtocolProp, STRING, Defaults.InterBrokerSecurityProtocol, MEDIUM, InterBrokerSecurityProtocolDoc) .define(InterBrokerProtocolVersionProp, STRING, Defaults.InterBrokerProtocolVersion, MEDIUM, InterBrokerProtocolVersionDoc) /** ********* Controlled shutdown configuration ***********/ .define(ControlledShutdownMaxRetriesProp, INT, Defaults.ControlledShutdownMaxRetries, MEDIUM, ControlledShutdownMaxRetriesDoc) - .define(ControlledShutdownRetryBackoffMsProp, INT, Defaults.ControlledShutdownRetryBackoffMs, MEDIUM, ControlledShutdownRetryBackoffMsDoc) + .define(ControlledShutdownRetryBackoffMsProp, LONG, Defaults.ControlledShutdownRetryBackoffMs, MEDIUM, ControlledShutdownRetryBackoffMsDoc) .define(ControlledShutdownEnableProp, BOOLEAN, Defaults.ControlledShutdownEnable, MEDIUM, ControlledShutdownEnableDoc) /** ********* Consumer coordinator configuration ***********/ @@ -520,139 +524,6 @@ object KafkaConfig { } /** - * Parse the given properties instance into a KafkaConfig object - */ - def fromProps(props: Properties): KafkaConfig = { - import kafka.utils.CoreUtils.evaluateDefaults - val parsed = configDef.parse(evaluateDefaults(props)) - new KafkaConfig( - /** ********* Zookeeper Configuration ***********/ - zkConnect = parsed.get(ZkConnectProp).asInstanceOf[String], - zkSessionTimeoutMs = parsed.get(ZkSessionTimeoutMsProp).asInstanceOf[Int], - _zkConnectionTimeoutMs = Option(parsed.get(ZkConnectionTimeoutMsProp)).map(_.asInstanceOf[Int]), - zkSyncTimeMs = parsed.get(ZkSyncTimeMsProp).asInstanceOf[Int], - - /** ********* General Configuration ***********/ - maxReservedBrokerId = parsed.get(MaxReservedBrokerIdProp).asInstanceOf[Int], - brokerId = parsed.get(BrokerIdProp).asInstanceOf[Int], - messageMaxBytes = parsed.get(MessageMaxBytesProp).asInstanceOf[Int], - numNetworkThreads = parsed.get(NumNetworkThreadsProp).asInstanceOf[Int], - numIoThreads = parsed.get(NumIoThreadsProp).asInstanceOf[Int], - backgroundThreads = parsed.get(BackgroundThreadsProp).asInstanceOf[Int], - queuedMaxRequests = parsed.get(QueuedMaxRequestsProp).asInstanceOf[Int], - - /** ********* Socket Server Configuration ***********/ - port = parsed.get(PortProp).asInstanceOf[Int], - hostName = parsed.get(HostNameProp).asInstanceOf[String], - _listeners = Option(parsed.get(ListenersProp)).map(_.asInstanceOf[String]), - _advertisedHostName = Option(parsed.get(AdvertisedHostNameProp)).map(_.asInstanceOf[String]), - _advertisedPort = Option(parsed.get(AdvertisedPortProp)).map(_.asInstanceOf[Int]), - _advertisedListeners = Option(parsed.get(AdvertisedListenersProp)).map(_.asInstanceOf[String]), - socketSendBufferBytes = parsed.get(SocketSendBufferBytesProp).asInstanceOf[Int], - socketReceiveBufferBytes = parsed.get(SocketReceiveBufferBytesProp).asInstanceOf[Int], - socketRequestMaxBytes = parsed.get(SocketRequestMaxBytesProp).asInstanceOf[Int], - maxConnectionsPerIp = parsed.get(MaxConnectionsPerIpProp).asInstanceOf[Int], - _maxConnectionsPerIpOverrides = parsed.get(MaxConnectionsPerIpOverridesProp).asInstanceOf[String], - connectionsMaxIdleMs = parsed.get(ConnectionsMaxIdleMsProp).asInstanceOf[Long], - - /** ********* Log Configuration ***********/ - numPartitions = parsed.get(NumPartitionsProp).asInstanceOf[Int], - _logDir = parsed.get(LogDirProp).asInstanceOf[String], - _logDirs = Option(parsed.get(LogDirsProp)).map(_.asInstanceOf[String]), - - logSegmentBytes = parsed.get(LogSegmentBytesProp).asInstanceOf[Int], - logRollTimeHours = parsed.get(LogRollTimeHoursProp).asInstanceOf[Int], - _logRollTimeMillis = Option(parsed.get(LogRollTimeMillisProp)).map(_.asInstanceOf[Long]), - - logRollTimeJitterHours = parsed.get(LogRollTimeJitterHoursProp).asInstanceOf[Int], - _logRollTimeJitterMillis = Option(parsed.get(LogRollTimeJitterMillisProp)).map(_.asInstanceOf[Long]), - - logRetentionTimeHours = parsed.get(LogRetentionTimeHoursProp).asInstanceOf[Int], - _logRetentionTimeMins = Option(parsed.get(LogRetentionTimeMinutesProp)).map(_.asInstanceOf[Int]), - _logRetentionTimeMillis = Option(parsed.get(LogRetentionTimeMillisProp)).map(_.asInstanceOf[Long]), - - logRetentionBytes = parsed.get(LogRetentionBytesProp).asInstanceOf[Long], - logCleanupIntervalMs = parsed.get(LogCleanupIntervalMsProp).asInstanceOf[Long], - logCleanupPolicy = parsed.get(LogCleanupPolicyProp).asInstanceOf[String], - logCleanerThreads = parsed.get(LogCleanerThreadsProp).asInstanceOf[Int], - logCleanerIoMaxBytesPerSecond = parsed.get(LogCleanerIoMaxBytesPerSecondProp).asInstanceOf[Double], - logCleanerDedupeBufferSize = parsed.get(LogCleanerDedupeBufferSizeProp).asInstanceOf[Long], - logCleanerIoBufferSize = parsed.get(LogCleanerIoBufferSizeProp).asInstanceOf[Int], - logCleanerDedupeBufferLoadFactor = parsed.get(LogCleanerDedupeBufferLoadFactorProp).asInstanceOf[Double], - logCleanerBackoffMs = parsed.get(LogCleanerBackoffMsProp).asInstanceOf[Long], - logCleanerMinCleanRatio = parsed.get(LogCleanerMinCleanRatioProp).asInstanceOf[Double], - logCleanerEnable = parsed.get(LogCleanerEnableProp).asInstanceOf[Boolean], - logCleanerDeleteRetentionMs = parsed.get(LogCleanerDeleteRetentionMsProp).asInstanceOf[Long], - logIndexSizeMaxBytes = parsed.get(LogIndexSizeMaxBytesProp).asInstanceOf[Int], - logIndexIntervalBytes = parsed.get(LogIndexIntervalBytesProp).asInstanceOf[Int], - logFlushIntervalMessages = parsed.get(LogFlushIntervalMessagesProp).asInstanceOf[Long], - logDeleteDelayMs = parsed.get(LogDeleteDelayMsProp).asInstanceOf[Long], - logFlushSchedulerIntervalMs = parsed.get(LogFlushSchedulerIntervalMsProp).asInstanceOf[Long], - _logFlushIntervalMs = Option(parsed.get(LogFlushIntervalMsProp)).map(_.asInstanceOf[Long]), - logFlushOffsetCheckpointIntervalMs = parsed.get(LogFlushOffsetCheckpointIntervalMsProp).asInstanceOf[Int], - numRecoveryThreadsPerDataDir = parsed.get(NumRecoveryThreadsPerDataDirProp).asInstanceOf[Int], - autoCreateTopicsEnable = parsed.get(AutoCreateTopicsEnableProp).asInstanceOf[Boolean], - minInSyncReplicas = parsed.get(MinInSyncReplicasProp).asInstanceOf[Int], - - /** ********* Replication configuration ***********/ - controllerSocketTimeoutMs = parsed.get(ControllerSocketTimeoutMsProp).asInstanceOf[Int], - defaultReplicationFactor = parsed.get(DefaultReplicationFactorProp).asInstanceOf[Int], - replicaLagTimeMaxMs = parsed.get(ReplicaLagTimeMaxMsProp).asInstanceOf[Long], - replicaSocketTimeoutMs = parsed.get(ReplicaSocketTimeoutMsProp).asInstanceOf[Int], - replicaSocketReceiveBufferBytes = parsed.get(ReplicaSocketReceiveBufferBytesProp).asInstanceOf[Int], - replicaFetchMaxBytes = parsed.get(ReplicaFetchMaxBytesProp).asInstanceOf[Int], - replicaFetchWaitMaxMs = parsed.get(ReplicaFetchWaitMaxMsProp).asInstanceOf[Int], - replicaFetchMinBytes = parsed.get(ReplicaFetchMinBytesProp).asInstanceOf[Int], - replicaFetchBackoffMs = parsed.get(ReplicaFetchBackoffMsProp).asInstanceOf[Int], - numReplicaFetchers = parsed.get(NumReplicaFetchersProp).asInstanceOf[Int], - replicaHighWatermarkCheckpointIntervalMs = parsed.get(ReplicaHighWatermarkCheckpointIntervalMsProp).asInstanceOf[Long], - fetchPurgatoryPurgeIntervalRequests = parsed.get(FetchPurgatoryPurgeIntervalRequestsProp).asInstanceOf[Int], - producerPurgatoryPurgeIntervalRequests = parsed.get(ProducerPurgatoryPurgeIntervalRequestsProp).asInstanceOf[Int], - autoLeaderRebalanceEnable = parsed.get(AutoLeaderRebalanceEnableProp).asInstanceOf[Boolean], - leaderImbalancePerBrokerPercentage = parsed.get(LeaderImbalancePerBrokerPercentageProp).asInstanceOf[Int], - leaderImbalanceCheckIntervalSeconds = parsed.get(LeaderImbalanceCheckIntervalSecondsProp).asInstanceOf[Int], - uncleanLeaderElectionEnable = parsed.get(UncleanLeaderElectionEnableProp).asInstanceOf[Boolean], - interBrokerSecurityProtocol = SecurityProtocol.valueOf(parsed.get(InterBrokerSecurityProtocolProp).asInstanceOf[String]), - interBrokerProtocolVersion = ApiVersion(parsed.get(InterBrokerProtocolVersionProp).asInstanceOf[String]), - - /** ********* Controlled shutdown configuration ***********/ - controlledShutdownMaxRetries = parsed.get(ControlledShutdownMaxRetriesProp).asInstanceOf[Int], - controlledShutdownRetryBackoffMs = parsed.get(ControlledShutdownRetryBackoffMsProp).asInstanceOf[Int], - controlledShutdownEnable = parsed.get(ControlledShutdownEnableProp).asInstanceOf[Boolean], - - /** ********* Consumer coordinator configuration ***********/ - consumerMinSessionTimeoutMs = parsed.get(ConsumerMinSessionTimeoutMsProp).asInstanceOf[Int], - consumerMaxSessionTimeoutMs = parsed.get(ConsumerMaxSessionTimeoutMsProp).asInstanceOf[Int], - - /** ********* Offset management configuration ***********/ - offsetMetadataMaxSize = parsed.get(OffsetMetadataMaxSizeProp).asInstanceOf[Int], - offsetsLoadBufferSize = parsed.get(OffsetsLoadBufferSizeProp).asInstanceOf[Int], - offsetsTopicReplicationFactor = parsed.get(OffsetsTopicReplicationFactorProp).asInstanceOf[Short], - offsetsTopicPartitions = parsed.get(OffsetsTopicPartitionsProp).asInstanceOf[Int], - offsetsTopicSegmentBytes = parsed.get(OffsetsTopicSegmentBytesProp).asInstanceOf[Int], - offsetsTopicCompressionCodec = Option(parsed.get(OffsetsTopicCompressionCodecProp)).map(_.asInstanceOf[Int]).map(value => CompressionCodec.getCompressionCodec(value)).orNull, - offsetsRetentionMinutes = parsed.get(OffsetsRetentionMinutesProp).asInstanceOf[Int], - offsetsRetentionCheckIntervalMs = parsed.get(OffsetsRetentionCheckIntervalMsProp).asInstanceOf[Long], - offsetCommitTimeoutMs = parsed.get(OffsetCommitTimeoutMsProp).asInstanceOf[Int], - offsetCommitRequiredAcks = parsed.get(OffsetCommitRequiredAcksProp).asInstanceOf[Short], - deleteTopicEnable = parsed.get(DeleteTopicEnableProp).asInstanceOf[Boolean], - compressionType = parsed.get(CompressionTypeProp).asInstanceOf[String], - metricNumSamples = parsed.get(MetricNumSamplesProp).asInstanceOf[Int], - metricSampleWindowMs = parsed.get(MetricSampleWindowMsProp).asInstanceOf[Long], - _metricReporterClasses = parsed.get(MetricReporterClassesProp).asInstanceOf[java.util.List[String]] - ) - } - - /** - * Create a log config instance using the given properties and defaults - */ - def fromProps(defaults: Properties, overrides: Properties): KafkaConfig = { - val props = new Properties(defaults) - props.putAll(overrides) - fromProps(props) - } - - /** * Check that property names are valid */ def validateNames(props: Properties) { @@ -662,171 +533,149 @@ object KafkaConfig { require(names.contains(name), "Unknown configuration \"%s\".".format(name)) } - /** - * Check that the given properties contain only valid kafka config names and that all values can be parsed and are valid - */ - def validate(props: Properties) { - validateNames(props) - configDef.parse(props) + def fromProps(props: Properties): KafkaConfig = { + KafkaConfig(props) + } - // to bootstrap KafkaConfig.validateValues() - KafkaConfig.fromProps(props) + def fromProps(defaults: Properties, overrides: Properties): KafkaConfig = { + val props = new Properties() + props.putAll(defaults) + props.putAll(overrides) + fromProps(props) } + } -class KafkaConfig (/** ********* Zookeeper Configuration ***********/ - val zkConnect: String, - val zkSessionTimeoutMs: Int = Defaults.ZkSessionTimeoutMs, - private val _zkConnectionTimeoutMs: Option[Int] = None, - val zkSyncTimeMs: Int = Defaults.ZkSyncTimeMs, - - /** ********* General Configuration ***********/ - val maxReservedBrokerId: Int = Defaults.MaxReservedBrokerId, - var brokerId: Int = Defaults.BrokerId, - val messageMaxBytes: Int = Defaults.MessageMaxBytes, - val numNetworkThreads: Int = Defaults.NumNetworkThreads, - val numIoThreads: Int = Defaults.NumIoThreads, - val backgroundThreads: Int = Defaults.BackgroundThreads, - val queuedMaxRequests: Int = Defaults.QueuedMaxRequests, - - /** ********* Socket Server Configuration ***********/ - val port: Int = Defaults.Port, - val hostName: String = Defaults.HostName, - private val _listeners: Option[String] = None, - private val _advertisedHostName: Option[String] = None, - private val _advertisedPort: Option[Int] = None, - private val _advertisedListeners: Option[String] = None, - val socketSendBufferBytes: Int = Defaults.SocketSendBufferBytes, - val socketReceiveBufferBytes: Int = Defaults.SocketReceiveBufferBytes, - val socketRequestMaxBytes: Int = Defaults.SocketRequestMaxBytes, - val maxConnectionsPerIp: Int = Defaults.MaxConnectionsPerIp, - private val _maxConnectionsPerIpOverrides: String = Defaults.MaxConnectionsPerIpOverrides, - val connectionsMaxIdleMs: Long = Defaults.ConnectionsMaxIdleMs, - - /** ********* Log Configuration ***********/ - val numPartitions: Int = Defaults.NumPartitions, - private val _logDir: String = Defaults.LogDir, - private val _logDirs: Option[String] = None, - - val logSegmentBytes: Int = Defaults.LogSegmentBytes, - - val logRollTimeHours: Int = Defaults.LogRollHours, - private val _logRollTimeMillis: Option[Long] = None, - - val logRollTimeJitterHours: Int = Defaults.LogRollJitterHours, - private val _logRollTimeJitterMillis: Option[Long] = None, - - val logRetentionTimeHours: Int = Defaults.LogRetentionHours, - private val _logRetentionTimeMins: Option[Int] = None, - private val _logRetentionTimeMillis: Option[Long] = None, - - val logRetentionBytes: Long = Defaults.LogRetentionBytes, - val logCleanupIntervalMs: Long = Defaults.LogCleanupIntervalMs, - val logCleanupPolicy: String = Defaults.LogCleanupPolicy, - val logCleanerThreads: Int = Defaults.LogCleanerThreads, - val logCleanerIoMaxBytesPerSecond: Double = Defaults.LogCleanerIoMaxBytesPerSecond, - val logCleanerDedupeBufferSize: Long = Defaults.LogCleanerDedupeBufferSize, - val logCleanerIoBufferSize: Int = Defaults.LogCleanerIoBufferSize, - val logCleanerDedupeBufferLoadFactor: Double = Defaults.LogCleanerDedupeBufferLoadFactor, - val logCleanerBackoffMs: Long = Defaults.LogCleanerBackoffMs, - val logCleanerMinCleanRatio: Double = Defaults.LogCleanerMinCleanRatio, - val logCleanerEnable: Boolean = Defaults.LogCleanerEnable, - val logCleanerDeleteRetentionMs: Long = Defaults.LogCleanerDeleteRetentionMs, - val logIndexSizeMaxBytes: Int = Defaults.LogIndexSizeMaxBytes, - val logIndexIntervalBytes: Int = Defaults.LogIndexIntervalBytes, - val logFlushIntervalMessages: Long = Defaults.LogFlushIntervalMessages, - val logDeleteDelayMs: Long = Defaults.LogDeleteDelayMs, - val logFlushSchedulerIntervalMs: Long = Defaults.LogFlushSchedulerIntervalMs, - private val _logFlushIntervalMs: Option[Long] = None, - val logFlushOffsetCheckpointIntervalMs: Int = Defaults.LogFlushOffsetCheckpointIntervalMs, - val numRecoveryThreadsPerDataDir: Int = Defaults.NumRecoveryThreadsPerDataDir, - val autoCreateTopicsEnable: Boolean = Defaults.AutoCreateTopicsEnable, - - val minInSyncReplicas: Int = Defaults.MinInSyncReplicas, - - /** ********* Replication configuration ***********/ - val controllerSocketTimeoutMs: Int = Defaults.ControllerSocketTimeoutMs, - val defaultReplicationFactor: Int = Defaults.DefaultReplicationFactor, - val replicaLagTimeMaxMs: Long = Defaults.ReplicaLagTimeMaxMs, - val replicaSocketTimeoutMs: Int = Defaults.ReplicaSocketTimeoutMs, - val replicaSocketReceiveBufferBytes: Int = Defaults.ReplicaSocketReceiveBufferBytes, - val replicaFetchMaxBytes: Int = Defaults.ReplicaFetchMaxBytes, - val replicaFetchWaitMaxMs: Int = Defaults.ReplicaFetchWaitMaxMs, - val replicaFetchMinBytes: Int = Defaults.ReplicaFetchMinBytes, - val replicaFetchBackoffMs: Int = Defaults.ReplicaFetchBackoffMs, - val numReplicaFetchers: Int = Defaults.NumReplicaFetchers, - val replicaHighWatermarkCheckpointIntervalMs: Long = Defaults.ReplicaHighWatermarkCheckpointIntervalMs, - val fetchPurgatoryPurgeIntervalRequests: Int = Defaults.FetchPurgatoryPurgeIntervalRequests, - val producerPurgatoryPurgeIntervalRequests: Int = Defaults.ProducerPurgatoryPurgeIntervalRequests, - val autoLeaderRebalanceEnable: Boolean = Defaults.AutoLeaderRebalanceEnable, - val leaderImbalancePerBrokerPercentage: Int = Defaults.LeaderImbalancePerBrokerPercentage, - val leaderImbalanceCheckIntervalSeconds: Int = Defaults.LeaderImbalanceCheckIntervalSeconds, - val uncleanLeaderElectionEnable: Boolean = Defaults.UncleanLeaderElectionEnable, - val interBrokerSecurityProtocol: SecurityProtocol = SecurityProtocol.valueOf(Defaults.InterBrokerSecurityProtocol), - val interBrokerProtocolVersion: ApiVersion = ApiVersion(Defaults.InterBrokerProtocolVersion), - - /** ********* Controlled shutdown configuration ***********/ - val controlledShutdownMaxRetries: Int = Defaults.ControlledShutdownMaxRetries, - val controlledShutdownRetryBackoffMs: Int = Defaults.ControlledShutdownRetryBackoffMs, - val controlledShutdownEnable: Boolean = Defaults.ControlledShutdownEnable, - - /** ********* Consumer coordinator configuration ***********/ - val consumerMinSessionTimeoutMs: Int = Defaults.ConsumerMinSessionTimeoutMs, - val consumerMaxSessionTimeoutMs: Int = Defaults.ConsumerMaxSessionTimeoutMs, - - /** ********* Offset management configuration ***********/ - val offsetMetadataMaxSize: Int = Defaults.OffsetMetadataMaxSize, - val offsetsLoadBufferSize: Int = Defaults.OffsetsLoadBufferSize, - val offsetsTopicReplicationFactor: Short = Defaults.OffsetsTopicReplicationFactor, - val offsetsTopicPartitions: Int = Defaults.OffsetsTopicPartitions, - val offsetsTopicSegmentBytes: Int = Defaults.OffsetsTopicSegmentBytes, - val offsetsTopicCompressionCodec: CompressionCodec = CompressionCodec.getCompressionCodec(Defaults.OffsetsTopicCompressionCodec), - val offsetsRetentionMinutes: Int = Defaults.OffsetsRetentionMinutes, - val offsetsRetentionCheckIntervalMs: Long = Defaults.OffsetsRetentionCheckIntervalMs, - val offsetCommitTimeoutMs: Int = Defaults.OffsetCommitTimeoutMs, - val offsetCommitRequiredAcks: Short = Defaults.OffsetCommitRequiredAcks, - - val deleteTopicEnable: Boolean = Defaults.DeleteTopicEnable, - val compressionType: String = Defaults.CompressionType, - - val metricSampleWindowMs: Long = Defaults.MetricSampleWindowMs, - val metricNumSamples: Int = Defaults.MetricNumSamples, - private val _metricReporterClasses: java.util.List[String] = util.Arrays.asList(Defaults.MetricReporterClasses) - ) { - - val zkConnectionTimeoutMs: Int = _zkConnectionTimeoutMs.getOrElse(zkSessionTimeoutMs) - - val listeners = getListeners() - val advertisedHostName: String = _advertisedHostName.getOrElse(hostName) - val advertisedPort: Int = _advertisedPort.getOrElse(port) - val advertisedListeners = getAdvertisedListeners() - val logDirs = CoreUtils.parseCsvList(_logDirs.getOrElse(_logDir)) - - val logRollTimeMillis = _logRollTimeMillis.getOrElse(60 * 60 * 1000L * logRollTimeHours) - val logRollTimeJitterMillis = _logRollTimeJitterMillis.getOrElse(60 * 60 * 1000L * logRollTimeJitterHours) - val logRetentionTimeMillis = getLogRetentionTimeMillis +case class KafkaConfig (props: java.util.Map[_, _]) extends AbstractConfig(KafkaConfig.configDef, props) { + + /** ********* Zookeeper Configuration ***********/ + val zkConnect: String = getString(KafkaConfig.ZkConnectProp) + val zkSessionTimeoutMs: Int = getInt(KafkaConfig.ZkSessionTimeoutMsProp) + val zkConnectionTimeoutMs: java.lang.Integer = + Option(getInt(KafkaConfig.ZkConnectionTimeoutMsProp)).getOrElse(getInt(KafkaConfig.ZkSessionTimeoutMsProp)) + val zkSyncTimeMs: Int = getInt(KafkaConfig.ZkSyncTimeMsProp) - val logFlushIntervalMs = _logFlushIntervalMs.getOrElse(logFlushSchedulerIntervalMs) + /** ********* General Configuration ***********/ + val maxReservedBrokerId: Int = getInt(KafkaConfig.MaxReservedBrokerIdProp) + var brokerId: Int = getInt(KafkaConfig.BrokerIdProp) + val numNetworkThreads = getInt(KafkaConfig.NumNetworkThreadsProp) + val backgroundThreads = getInt(KafkaConfig.BackgroundThreadsProp) + val queuedMaxRequests = getInt(KafkaConfig.QueuedMaxRequestsProp) + val numIoThreads = getInt(KafkaConfig.NumIoThreadsProp) + val messageMaxBytes = getInt(KafkaConfig.MessageMaxBytesProp) + /** ********* Socket Server Configuration ***********/ + val hostName = getString(KafkaConfig.HostNameProp) + val port = getInt(KafkaConfig.PortProp) + val advertisedHostName = Option(getString(KafkaConfig.AdvertisedHostNameProp)).getOrElse(hostName) + val advertisedPort: java.lang.Integer = Option(getInt(KafkaConfig.AdvertisedPortProp)).getOrElse(port) + + val socketSendBufferBytes = getInt(KafkaConfig.SocketSendBufferBytesProp) + val socketReceiveBufferBytes = getInt(KafkaConfig.SocketReceiveBufferBytesProp) + val socketRequestMaxBytes = getInt(KafkaConfig.SocketRequestMaxBytesProp) + val maxConnectionsPerIp = getInt(KafkaConfig.MaxConnectionsPerIpProp) val maxConnectionsPerIpOverrides: Map[String, Int] = - getMap(KafkaConfig.MaxConnectionsPerIpOverridesProp, _maxConnectionsPerIpOverrides).map { case (k, v) => (k, v.toInt)} + getMap(KafkaConfig.MaxConnectionsPerIpOverridesProp, getString(KafkaConfig.MaxConnectionsPerIpOverridesProp)).map { case (k, v) => (k, v.toInt)} + val connectionsMaxIdleMs = getLong(KafkaConfig.ConnectionsMaxIdleMsProp) + - val metricReporterClasses: java.util.List[MetricsReporter] = getMetricClasses(_metricReporterClasses) + /** ********* Log Configuration ***********/ + val autoCreateTopicsEnable = getBoolean(KafkaConfig.AutoCreateTopicsEnableProp) + val numPartitions = getInt(KafkaConfig.NumPartitionsProp) + val logDirs = CoreUtils.parseCsvList( Option(getString(KafkaConfig.LogDirsProp)).getOrElse(getString(KafkaConfig.LogDirProp))) + val logSegmentBytes = getInt(KafkaConfig.LogSegmentBytesProp) + val logFlushIntervalMessages = getLong(KafkaConfig.LogFlushIntervalMessagesProp) + val logCleanerThreads = getInt(KafkaConfig.LogCleanerThreadsProp) + val numRecoveryThreadsPerDataDir = getInt(KafkaConfig.NumRecoveryThreadsPerDataDirProp) + val logFlushSchedulerIntervalMs = getLong(KafkaConfig.LogFlushSchedulerIntervalMsProp) + val logFlushOffsetCheckpointIntervalMs = getInt(KafkaConfig.LogFlushOffsetCheckpointIntervalMsProp).toLong + val logCleanupIntervalMs = getLong(KafkaConfig.LogCleanupIntervalMsProp) + val logCleanupPolicy = getString(KafkaConfig.LogCleanupPolicyProp) + val offsetsRetentionMinutes = getInt(KafkaConfig.OffsetsRetentionMinutesProp) + val offsetsRetentionCheckIntervalMs = getLong(KafkaConfig.OffsetsRetentionCheckIntervalMsProp) + val logRetentionBytes = getLong(KafkaConfig.LogRetentionBytesProp) + val logCleanerDedupeBufferSize = getLong(KafkaConfig.LogCleanerDedupeBufferSizeProp) + val logCleanerDedupeBufferLoadFactor = getDouble(KafkaConfig.LogCleanerDedupeBufferLoadFactorProp) + val logCleanerIoBufferSize = getInt(KafkaConfig.LogCleanerIoBufferSizeProp) + val logCleanerIoMaxBytesPerSecond = getDouble(KafkaConfig.LogCleanerIoMaxBytesPerSecondProp) + val logCleanerDeleteRetentionMs = getLong(KafkaConfig.LogCleanerDeleteRetentionMsProp) + val logCleanerBackoffMs = getLong(KafkaConfig.LogCleanerBackoffMsProp) + val logCleanerMinCleanRatio = getDouble(KafkaConfig.LogCleanerMinCleanRatioProp) + val logCleanerEnable = getBoolean(KafkaConfig.LogCleanerEnableProp) + val logIndexSizeMaxBytes = getInt(KafkaConfig.LogIndexSizeMaxBytesProp) + val logIndexIntervalBytes = getInt(KafkaConfig.LogIndexIntervalBytesProp) + val logDeleteDelayMs = getLong(KafkaConfig.LogDeleteDelayMsProp) + val logRollTimeMillis: java.lang.Long = Option(getLong(KafkaConfig.LogRollTimeMillisProp)).getOrElse(60 * 60 * 1000L * getInt(KafkaConfig.LogRollTimeHoursProp)) + val logRollTimeJitterMillis: java.lang.Long = Option(getLong(KafkaConfig.LogRollTimeJitterMillisProp)).getOrElse(60 * 60 * 1000L * getInt(KafkaConfig.LogRollTimeJitterHoursProp)) + val logFlushIntervalMs: java.lang.Long = Option(getLong(KafkaConfig.LogFlushIntervalMsProp)).getOrElse(getLong(KafkaConfig.LogFlushSchedulerIntervalMsProp)) + val minInSyncReplicas = getInt(KafkaConfig.MinInSyncReplicasProp) + + /** ********* Replication configuration ***********/ + val controllerSocketTimeoutMs: Int = getInt(KafkaConfig.ControllerSocketTimeoutMsProp) + val defaultReplicationFactor: Int = getInt(KafkaConfig.DefaultReplicationFactorProp) + val replicaLagTimeMaxMs = getLong(KafkaConfig.ReplicaLagTimeMaxMsProp) + val replicaSocketTimeoutMs = getInt(KafkaConfig.ReplicaSocketTimeoutMsProp) + val replicaSocketReceiveBufferBytes = getInt(KafkaConfig.ReplicaSocketReceiveBufferBytesProp) + val replicaFetchMaxBytes = getInt(KafkaConfig.ReplicaFetchMaxBytesProp) + val replicaFetchWaitMaxMs = getInt(KafkaConfig.ReplicaFetchWaitMaxMsProp) + val replicaFetchMinBytes = getInt(KafkaConfig.ReplicaFetchMinBytesProp) + val replicaFetchBackoffMs = getInt(KafkaConfig.ReplicaFetchBackoffMsProp) + val numReplicaFetchers = getInt(KafkaConfig.NumReplicaFetchersProp) + val replicaHighWatermarkCheckpointIntervalMs = getLong(KafkaConfig.ReplicaHighWatermarkCheckpointIntervalMsProp) + val fetchPurgatoryPurgeIntervalRequests = getInt(KafkaConfig.FetchPurgatoryPurgeIntervalRequestsProp) + val producerPurgatoryPurgeIntervalRequests = getInt(KafkaConfig.ProducerPurgatoryPurgeIntervalRequestsProp) + val autoLeaderRebalanceEnable = getBoolean(KafkaConfig.AutoLeaderRebalanceEnableProp) + val leaderImbalancePerBrokerPercentage = getInt(KafkaConfig.LeaderImbalancePerBrokerPercentageProp) + val leaderImbalanceCheckIntervalSeconds = getLong(KafkaConfig.LeaderImbalanceCheckIntervalSecondsProp) + val uncleanLeaderElectionEnable = getBoolean(KafkaConfig.UncleanLeaderElectionEnableProp) + val interBrokerSecurityProtocol = SecurityProtocol.valueOf(getString(KafkaConfig.InterBrokerSecurityProtocolProp)) + val interBrokerProtocolVersion = ApiVersion(getString(KafkaConfig.InterBrokerProtocolVersionProp)) + + /** ********* Controlled shutdown configuration ***********/ + val controlledShutdownMaxRetries = getInt(KafkaConfig.ControlledShutdownMaxRetriesProp) + val controlledShutdownRetryBackoffMs = getLong(KafkaConfig.ControlledShutdownRetryBackoffMsProp) + val controlledShutdownEnable = getBoolean(KafkaConfig.ControlledShutdownEnableProp) + + /** ********* Consumer coordinator configuration ***********/ + val consumerMinSessionTimeoutMs = getInt(KafkaConfig.ConsumerMinSessionTimeoutMsProp) + val consumerMaxSessionTimeoutMs = getInt(KafkaConfig.ConsumerMaxSessionTimeoutMsProp) + + /** ********* Offset management configuration ***********/ + val offsetMetadataMaxSize = getInt(KafkaConfig.OffsetMetadataMaxSizeProp) + val offsetsLoadBufferSize = getInt(KafkaConfig.OffsetsLoadBufferSizeProp) + val offsetsTopicReplicationFactor = getShort(KafkaConfig.OffsetsTopicReplicationFactorProp) + val offsetsTopicPartitions = getInt(KafkaConfig.OffsetsTopicPartitionsProp) + val offsetCommitTimeoutMs = getInt(KafkaConfig.OffsetCommitTimeoutMsProp) + val offsetCommitRequiredAcks = getShort(KafkaConfig.OffsetCommitRequiredAcksProp) + val offsetsTopicSegmentBytes = getInt(KafkaConfig.OffsetsTopicSegmentBytesProp) + val offsetsTopicCompressionCodec = Option(getInt(KafkaConfig.OffsetsTopicCompressionCodecProp)).map(value => CompressionCodec.getCompressionCodec(value)).orNull + + /** ********* Metric Configuration **************/ + val metricNumSamples = getInt(KafkaConfig.MetricNumSamplesProp) + val metricSampleWindowMs = getLong(KafkaConfig.MetricSampleWindowMsProp) + val metricReporterClasses: java.util.List[MetricsReporter] = getConfiguredInstances(KafkaConfig.MetricReporterClassesProp, classOf[MetricsReporter]) + + val deleteTopicEnable = getBoolean(KafkaConfig.DeleteTopicEnableProp) + val compressionType = getString(KafkaConfig.CompressionTypeProp) + + + val listeners = getListeners + val advertisedListeners = getAdvertisedListeners + val logRetentionTimeMillis = getLogRetentionTimeMillis private def getLogRetentionTimeMillis: Long = { val millisInMinute = 60L * 1000L val millisInHour = 60L * millisInMinute - val millis = { - _logRetentionTimeMillis.getOrElse( - _logRetentionTimeMins match { - case Some(mins) => millisInMinute * mins - case None => millisInHour * logRetentionTimeHours - } - ) - } - if (millis < 0) return -1 - millis + val millis: java.lang.Long = + Option(getLong(KafkaConfig.LogRetentionTimeMillisProp)).getOrElse( + Option(getInt(KafkaConfig.LogRetentionTimeMinutesProp)) match { + case Some(mins) => millisInMinute * mins + case None => getInt(KafkaConfig.LogRetentionTimeHoursProp) * millisInHour + }) + + if (millis < 0) return -1 + millis } private def getMap(propName: String, propValue: String): Map[String, String] = { @@ -855,9 +704,9 @@ class KafkaConfig (/** ********* Zookeeper Configuration ***********/ // If the user did not define listeners but did define host or port, let's use them in backward compatible way // If none of those are defined, we default to PLAINTEXT://:9092 private def getListeners(): immutable.Map[SecurityProtocol, EndPoint] = { - if (_listeners.isDefined) { - validateUniquePortAndProtocol(_listeners.get) - CoreUtils.listenerListToEndPoints(_listeners.get) + if (getString(KafkaConfig.ListenersProp) != null) { + validateUniquePortAndProtocol(getString(KafkaConfig.ListenersProp)) + CoreUtils.listenerListToEndPoints(getString(KafkaConfig.ListenersProp)) } else { CoreUtils.listenerListToEndPoints("PLAINTEXT://" + hostName + ":" + port) } @@ -867,11 +716,12 @@ class KafkaConfig (/** ********* Zookeeper Configuration ***********/ // If he didn't but did define advertised host or port, we'll use those and fill in the missing value from regular host / port or defaults // If none of these are defined, we'll use the listeners private def getAdvertisedListeners(): immutable.Map[SecurityProtocol, EndPoint] = { - if (_advertisedListeners.isDefined) { - validateUniquePortAndProtocol(_advertisedListeners.get) - CoreUtils.listenerListToEndPoints(_advertisedListeners.get) - } else if (_advertisedHostName.isDefined || _advertisedPort.isDefined ) { - CoreUtils.listenerListToEndPoints("PLAINTEXT://" + advertisedHostName + ":" + advertisedPort) + if (getString(KafkaConfig.AdvertisedListenersProp) != null) { + validateUniquePortAndProtocol(getString(KafkaConfig.AdvertisedListenersProp)) + CoreUtils.listenerListToEndPoints(getString(KafkaConfig.AdvertisedListenersProp)) + } else if (getString(KafkaConfig.AdvertisedHostNameProp) != null || getInt(KafkaConfig.AdvertisedPortProp) != null) { + CoreUtils.listenerListToEndPoints("PLAINTEXT://" + + getString(KafkaConfig.AdvertisedHostNameProp) + ":" + getInt(KafkaConfig.AdvertisedPortProp)) } else { getListeners() } @@ -886,7 +736,7 @@ class KafkaConfig (/** ********* Zookeeper Configuration ***********/ val reporterName = iterator.next() if (!reporterName.isEmpty) { val reporter: MetricsReporter = CoreUtils.createObject[MetricsReporter](reporterName) - reporter.configure(toProps.asInstanceOf[java.util.Map[String, _]]) + reporter.configure(originals) reporterList.add(reporter) } } @@ -895,19 +745,13 @@ class KafkaConfig (/** ********* Zookeeper Configuration ***********/ } - - validateValues() private def validateValues() { require(brokerId >= -1 && brokerId <= maxReservedBrokerId, "broker.id must be equal or greater than -1 and not greater than reserved.broker.max.id") require(logRollTimeMillis >= 1, "log.roll.ms must be equal or greater than 1") require(logRollTimeJitterMillis >= 0, "log.roll.jitter.ms must be equal or greater than 0") - - require(_logRetentionTimeMins.forall(_ >= 1)|| _logRetentionTimeMins.forall(_ .equals(-1)), "log.retention.minutes must be unlimited (-1) or, equal or greater than 1") - require(logRetentionTimeHours >= 1 || logRetentionTimeHours == -1, "log.retention.hours must be unlimited (-1) or, equal or greater than 1") require(logRetentionTimeMillis >= 1 || logRetentionTimeMillis == -1, "log.retention.ms must be unlimited (-1) or, equal or greater than 1") - require(logDirs.size > 0) require(logCleanerDedupeBufferSize / logCleanerThreads > 1024 * 1024, "log.cleaner.dedupe.buffer.size must be at least 1MB per cleaner thread.") require(replicaFetchWaitMaxMs <= replicaSocketTimeoutMs, "replica.socket.timeout.ms should always be at least replica.fetch.wait.max.ms" + @@ -920,127 +764,4 @@ class KafkaConfig (/** ********* Zookeeper Configuration ***********/ require(BrokerCompressionCodec.isValid(compressionType), "compression.type : " + compressionType + " is not valid." + " Valid options are " + BrokerCompressionCodec.brokerCompressionOptions.mkString(",")) } - - def toProps: Properties = { - val props = new Properties() - import kafka.server.KafkaConfig._ - /** ********* Zookeeper Configuration ***********/ - props.put(ZkConnectProp, zkConnect) - props.put(ZkSessionTimeoutMsProp, zkSessionTimeoutMs.toString) - _zkConnectionTimeoutMs.foreach(value => props.put(ZkConnectionTimeoutMsProp, value.toString)) - props.put(ZkSyncTimeMsProp, zkSyncTimeMs.toString) - - /** ********* General Configuration ***********/ - props.put(MaxReservedBrokerIdProp, maxReservedBrokerId.toString) - props.put(BrokerIdProp, brokerId.toString) - props.put(MessageMaxBytesProp, messageMaxBytes.toString) - props.put(NumNetworkThreadsProp, numNetworkThreads.toString) - props.put(NumIoThreadsProp, numIoThreads.toString) - props.put(BackgroundThreadsProp, backgroundThreads.toString) - props.put(QueuedMaxRequestsProp, queuedMaxRequests.toString) - - /** ********* Socket Server Configuration ***********/ - props.put(PortProp, port.toString) - props.put(HostNameProp, hostName) - _listeners.foreach(props.put(ListenersProp, _)) - _advertisedHostName.foreach(props.put(AdvertisedHostNameProp, _)) - _advertisedPort.foreach(value => props.put(AdvertisedPortProp, value.toString)) - _advertisedListeners.foreach(props.put(AdvertisedListenersProp, _)) - props.put(SocketSendBufferBytesProp, socketSendBufferBytes.toString) - props.put(SocketReceiveBufferBytesProp, socketReceiveBufferBytes.toString) - props.put(SocketRequestMaxBytesProp, socketRequestMaxBytes.toString) - props.put(MaxConnectionsPerIpProp, maxConnectionsPerIp.toString) - props.put(MaxConnectionsPerIpOverridesProp, _maxConnectionsPerIpOverrides) - props.put(ConnectionsMaxIdleMsProp, connectionsMaxIdleMs.toString) - - /** ********* Log Configuration ***********/ - props.put(NumPartitionsProp, numPartitions.toString) - props.put(LogDirProp, _logDir) - _logDirs.foreach(value => props.put(LogDirsProp, value)) - props.put(LogSegmentBytesProp, logSegmentBytes.toString) - - props.put(LogRollTimeHoursProp, logRollTimeHours.toString) - _logRollTimeMillis.foreach(v => props.put(LogRollTimeMillisProp, v.toString)) - - props.put(LogRollTimeJitterHoursProp, logRollTimeJitterHours.toString) - _logRollTimeJitterMillis.foreach(v => props.put(LogRollTimeJitterMillisProp, v.toString)) - - - props.put(LogRetentionTimeHoursProp, logRetentionTimeHours.toString) - _logRetentionTimeMins.foreach(v => props.put(LogRetentionTimeMinutesProp, v.toString)) - _logRetentionTimeMillis.foreach(v => props.put(LogRetentionTimeMillisProp, v.toString)) - - props.put(LogRetentionBytesProp, logRetentionBytes.toString) - props.put(LogCleanupIntervalMsProp, logCleanupIntervalMs.toString) - props.put(LogCleanupPolicyProp, logCleanupPolicy) - props.put(LogCleanerThreadsProp, logCleanerThreads.toString) - props.put(LogCleanerIoMaxBytesPerSecondProp, logCleanerIoMaxBytesPerSecond.toString) - props.put(LogCleanerDedupeBufferSizeProp, logCleanerDedupeBufferSize.toString) - props.put(LogCleanerIoBufferSizeProp, logCleanerIoBufferSize.toString) - props.put(LogCleanerDedupeBufferLoadFactorProp, logCleanerDedupeBufferLoadFactor.toString) - props.put(LogCleanerBackoffMsProp, logCleanerBackoffMs.toString) - props.put(LogCleanerMinCleanRatioProp, logCleanerMinCleanRatio.toString) - props.put(LogCleanerEnableProp, logCleanerEnable.toString) - props.put(LogCleanerDeleteRetentionMsProp, logCleanerDeleteRetentionMs.toString) - props.put(LogIndexSizeMaxBytesProp, logIndexSizeMaxBytes.toString) - props.put(LogIndexIntervalBytesProp, logIndexIntervalBytes.toString) - props.put(LogFlushIntervalMessagesProp, logFlushIntervalMessages.toString) - props.put(LogDeleteDelayMsProp, logDeleteDelayMs.toString) - props.put(LogFlushSchedulerIntervalMsProp, logFlushSchedulerIntervalMs.toString) - _logFlushIntervalMs.foreach(v => props.put(LogFlushIntervalMsProp, v.toString)) - props.put(LogFlushOffsetCheckpointIntervalMsProp, logFlushOffsetCheckpointIntervalMs.toString) - props.put(NumRecoveryThreadsPerDataDirProp, numRecoveryThreadsPerDataDir.toString) - props.put(AutoCreateTopicsEnableProp, autoCreateTopicsEnable.toString) - props.put(MinInSyncReplicasProp, minInSyncReplicas.toString) - - /** ********* Replication configuration ***********/ - props.put(ControllerSocketTimeoutMsProp, controllerSocketTimeoutMs.toString) - props.put(DefaultReplicationFactorProp, defaultReplicationFactor.toString) - props.put(ReplicaLagTimeMaxMsProp, replicaLagTimeMaxMs.toString) - props.put(ReplicaSocketTimeoutMsProp, replicaSocketTimeoutMs.toString) - props.put(ReplicaSocketReceiveBufferBytesProp, replicaSocketReceiveBufferBytes.toString) - props.put(ReplicaFetchMaxBytesProp, replicaFetchMaxBytes.toString) - props.put(ReplicaFetchWaitMaxMsProp, replicaFetchWaitMaxMs.toString) - props.put(ReplicaFetchMinBytesProp, replicaFetchMinBytes.toString) - props.put(ReplicaFetchBackoffMsProp, replicaFetchBackoffMs.toString) - props.put(NumReplicaFetchersProp, numReplicaFetchers.toString) - props.put(ReplicaHighWatermarkCheckpointIntervalMsProp, replicaHighWatermarkCheckpointIntervalMs.toString) - props.put(FetchPurgatoryPurgeIntervalRequestsProp, fetchPurgatoryPurgeIntervalRequests.toString) - props.put(ProducerPurgatoryPurgeIntervalRequestsProp, producerPurgatoryPurgeIntervalRequests.toString) - props.put(AutoLeaderRebalanceEnableProp, autoLeaderRebalanceEnable.toString) - props.put(LeaderImbalancePerBrokerPercentageProp, leaderImbalancePerBrokerPercentage.toString) - props.put(LeaderImbalanceCheckIntervalSecondsProp, leaderImbalanceCheckIntervalSeconds.toString) - props.put(UncleanLeaderElectionEnableProp, uncleanLeaderElectionEnable.toString) - props.put(InterBrokerSecurityProtocolProp, interBrokerSecurityProtocol.toString) - props.put(InterBrokerProtocolVersionProp, interBrokerProtocolVersion.toString) - - - /** ********* Controlled shutdown configuration ***********/ - props.put(ControlledShutdownMaxRetriesProp, controlledShutdownMaxRetries.toString) - props.put(ControlledShutdownRetryBackoffMsProp, controlledShutdownRetryBackoffMs.toString) - props.put(ControlledShutdownEnableProp, controlledShutdownEnable.toString) - - /** ********* Consumer coordinator configuration ***********/ - props.put(ConsumerMinSessionTimeoutMsProp, consumerMinSessionTimeoutMs.toString) - props.put(ConsumerMaxSessionTimeoutMsProp, consumerMaxSessionTimeoutMs.toString) - - /** ********* Offset management configuration ***********/ - props.put(OffsetMetadataMaxSizeProp, offsetMetadataMaxSize.toString) - props.put(OffsetsLoadBufferSizeProp, offsetsLoadBufferSize.toString) - props.put(OffsetsTopicReplicationFactorProp, offsetsTopicReplicationFactor.toString) - props.put(OffsetsTopicPartitionsProp, offsetsTopicPartitions.toString) - props.put(OffsetsTopicSegmentBytesProp, offsetsTopicSegmentBytes.toString) - props.put(OffsetsTopicCompressionCodecProp, offsetsTopicCompressionCodec.codec.toString) - props.put(OffsetsRetentionMinutesProp, offsetsRetentionMinutes.toString) - props.put(OffsetsRetentionCheckIntervalMsProp, offsetsRetentionCheckIntervalMs.toString) - props.put(OffsetCommitTimeoutMsProp, offsetCommitTimeoutMs.toString) - props.put(OffsetCommitRequiredAcksProp, offsetCommitRequiredAcks.toString) - props.put(DeleteTopicEnableProp, deleteTopicEnable.toString) - props.put(CompressionTypeProp, compressionType.toString) - props.put(MetricNumSamplesProp, metricNumSamples.toString) - props.put(MetricSampleWindowMsProp, metricSampleWindowMs.toString) - props.put(MetricReporterClassesProp, JavaConversions.collectionAsScalaIterable(_metricReporterClasses).mkString(",")) - - props - } } diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index b320ce9..9de2a6f 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -17,6 +17,9 @@ package kafka.server +import java.util +import java.util.Properties + import kafka.admin._ import kafka.log.LogConfig import kafka.log.CleanerConfig @@ -388,23 +391,9 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg def boundPort(): Int = socketServer.boundPort() private def createLogManager(zkClient: ZkClient, brokerState: BrokerState): LogManager = { - val defaultLogConfig = LogConfig(segmentSize = config.logSegmentBytes, - segmentMs = config.logRollTimeMillis, - segmentJitterMs = config.logRollTimeJitterMillis, - flushInterval = config.logFlushIntervalMessages, - flushMs = config.logFlushIntervalMs.toLong, - retentionSize = config.logRetentionBytes, - retentionMs = config.logRetentionTimeMillis, - maxMessageSize = config.messageMaxBytes, - maxIndexSize = config.logIndexSizeMaxBytes, - indexInterval = config.logIndexIntervalBytes, - deleteRetentionMs = config.logCleanerDeleteRetentionMs, - fileDeleteDelayMs = config.logDeleteDelayMs, - minCleanableRatio = config.logCleanerMinCleanRatio, - compact = config.logCleanupPolicy.trim.toLowerCase == "compact", - minInSyncReplicas = config.minInSyncReplicas, - compressionType = config.compressionType) - val defaultProps = defaultLogConfig.toProps + val defaultProps = copyKafkaConfigToLog(config.originals) + val defaultLogConfig = LogConfig(defaultProps) + val configs = AdminUtils.fetchAllTopicConfigs(zkClient).mapValues(LogConfig.fromProps(defaultProps, _)) // read the log configurations from zookeeper val cleanerConfig = CleanerConfig(numThreads = config.logCleanerThreads, @@ -428,6 +417,38 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg time = time) } + // Copy the subset of properties that are relevant to Logs + // I'm listing out individual properties here since the names are slightly different in each Config class... + private def copyKafkaConfigToLog(serverProps: java.util.Map[String, Object]): java.util.Map[String, Object] = { + + val logProps = new util.HashMap[String, Object]() + val entryset = serverProps.entrySet.iterator + while (entryset.hasNext) { + val entry = entryset.next + entry.getKey match { + case KafkaConfig.LogSegmentBytesProp => logProps.put(LogConfig.SegmentBytesProp, entry.getValue) + case KafkaConfig.LogRollTimeMillisProp => logProps.put(LogConfig.SegmentMsProp, entry.getValue) + case KafkaConfig.LogRollTimeJitterMillisProp => logProps.put(LogConfig.SegmentJitterMsProp, entry.getValue) + case KafkaConfig.LogIndexSizeMaxBytesProp => logProps.put(LogConfig.SegmentIndexBytesProp, entry.getValue) + case KafkaConfig.LogFlushIntervalMessagesProp => logProps.put(LogConfig.FlushMessagesProp, entry.getValue) + case KafkaConfig.LogFlushIntervalMsProp => logProps.put(LogConfig.FlushMsProp, entry.getValue) + case KafkaConfig.LogRetentionBytesProp => logProps.put(LogConfig.RetentionBytesProp, entry.getValue) + case KafkaConfig.LogRetentionTimeMillisProp => logProps.put(LogConfig.RetentionMsProp, entry.getValue) + case KafkaConfig.MessageMaxBytesProp => logProps.put(LogConfig.MaxMessageBytesProp, entry.getValue) + case KafkaConfig.LogIndexIntervalBytesProp => logProps.put(LogConfig.IndexIntervalBytesProp, entry.getValue) + case KafkaConfig.LogCleanerDeleteRetentionMsProp => logProps.put(LogConfig.DeleteRetentionMsProp, entry.getValue) + case KafkaConfig.LogDeleteDelayMsProp => logProps.put(LogConfig.FileDeleteDelayMsProp, entry.getValue) + case KafkaConfig.LogCleanerMinCleanRatioProp => logProps.put(LogConfig.MinCleanableDirtyRatioProp, entry.getValue) + case KafkaConfig.LogCleanupPolicyProp => logProps.put(LogConfig.CleanupPolicyProp, entry.getValue) + case KafkaConfig.MinInSyncReplicasProp => logProps.put(LogConfig.MinInSyncReplicasProp, entry.getValue) + case KafkaConfig.CompressionTypeProp => logProps.put(LogConfig.CompressionTypeProp, entry.getValue) + case KafkaConfig.UncleanLeaderElectionEnableProp => logProps.put(LogConfig.UncleanLeaderElectionEnableProp, entry.getValue) + case _ => // we just leave those out + } + } + logProps + } + private def createOffsetManager(): OffsetManager = { val offsetManagerConfig = OffsetManagerConfig( maxMetadataSize = config.offsetMetadataMaxSize, diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index 181cbc1..c89d00b 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -90,7 +90,7 @@ class ReplicaFetcherThread(name:String, // Prior to truncating the follower's log, ensure that doing so is not disallowed by the configuration for unclean leader election. // This situation could only happen if the unclean election configuration for a topic changes while a replica is down. Otherwise, // we should never encounter this situation since a non-ISR leader cannot be elected if disallowed by the broker configuration. - if (!LogConfig.fromProps(brokerConfig.toProps, AdminUtils.fetchTopicConfig(replicaMgr.zkClient, + if (!LogConfig.fromProps(brokerConfig.originals, AdminUtils.fetchTopicConfig(replicaMgr.zkClient, topicAndPartition.topic)).uncleanLeaderElectionEnable) { // Log a fatal error and shutdown the broker to ensure that data loss does not unexpectedly occur. fatal("Halting because log truncation is not allowed for topic %s,".format(topicAndPartition.topic) + @@ -120,6 +120,6 @@ class ReplicaFetcherThread(name:String, // any logic for partitions whose leader has changed def handlePartitionsWithErrors(partitions: Iterable[TopicAndPartition]) { - delayPartitions(partitions, brokerConfig.replicaFetchBackoffMs) + delayPartitions(partitions, brokerConfig.replicaFetchBackoffMs.toLong) } } diff --git a/core/src/main/scala/kafka/server/TopicConfigManager.scala b/core/src/main/scala/kafka/server/TopicConfigManager.scala index b675a7e..01b1b0a 100644 --- a/core/src/main/scala/kafka/server/TopicConfigManager.scala +++ b/core/src/main/scala/kafka/server/TopicConfigManager.scala @@ -101,9 +101,10 @@ class TopicConfigManager(private val zkClient: ZkClient, val topic = json.substring(1, json.length - 1) // hacky way to dequote if (logsByTopic.contains(topic)) { /* combine the default properties with the overrides in zk to create the new LogConfig */ - val props = new Properties(logManager.defaultConfig.toProps) + val props = new Properties() + props.putAll(logManager.defaultConfig.originals) props.putAll(AdminUtils.fetchTopicConfig(zkClient, topic)) - val logConfig = LogConfig.fromProps(props) + val logConfig = LogConfig(props) for (log <- logsByTopic(topic)) log.config = logConfig info("Processed topic config change %d for topic %s, setting new config to %s.".format(changeId, topic, props)) diff --git a/core/src/main/scala/kafka/utils/CoreUtils.scala b/core/src/main/scala/kafka/utils/CoreUtils.scala index d0a8fa7..f5d704c 100755 --- a/core/src/main/scala/kafka/utils/CoreUtils.scala +++ b/core/src/main/scala/kafka/utils/CoreUtils.scala @@ -254,32 +254,6 @@ object CoreUtils extends Logging { } /** - * Turn {@linkplain java.util.Properties} with default values into a {@linkplain java.util.Map}. Following example - * illustrates difference from the cast - *
-   * val defaults = new Properties()
-   * defaults.put("foo", "bar")
-   * val props = new Properties(defaults)
-   *
-   * props.getProperty("foo") // "bar"
-   * props.get("foo") // null
-   * evaluateDefaults(props).get("foo") // "bar"
-   * 
- * - * @param props properties to evaluate - * @return new java.util.Map instance - */ - def evaluateDefaults(props: Properties): java.util.Map[String, String] = { - import java.util._ - import JavaConversions.asScalaSet - val evaluated = new HashMap[String, String]() - for (name <- props.stringPropertyNames()) { - evaluated.put(name, props.getProperty(name)) - } - evaluated - } - - /** * Read a big-endian integer from a byte array */ def readInt(bytes: Array[Byte], offset: Int): Int = { diff --git a/core/src/test/scala/other/kafka/StressTestLog.scala b/core/src/test/scala/other/kafka/StressTestLog.scala index c0e248d..225d77b 100755 --- a/core/src/test/scala/other/kafka/StressTestLog.scala +++ b/core/src/test/scala/other/kafka/StressTestLog.scala @@ -17,6 +17,7 @@ package kafka +import java.util.Properties import java.util.concurrent.atomic._ import kafka.common._ import kafka.message._ @@ -33,10 +34,13 @@ object StressTestLog { def main(args: Array[String]) { val dir = TestUtils.tempDir() val time = new MockTime + val logProprties = new Properties() + logProprties.put(LogConfig.SegmentBytesProp, 64*1024*1024: java.lang.Integer) + logProprties.put(LogConfig.MaxMessageBytesProp, Int.MaxValue: java.lang.Integer) + logProprties.put(LogConfig.SegmentIndexBytesProp, 1024*1024: java.lang.Integer) + val log = new Log(dir = dir, - config = LogConfig(segmentSize = 64*1024*1024, - maxMessageSize = Int.MaxValue, - maxIndexSize = 1024*1024), + config = LogConfig(logProprties), recoveryPoint = 0L, scheduler = time.scheduler, time = time) diff --git a/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala b/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala index 3034c4f..236d857 100755 --- a/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala +++ b/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala @@ -20,7 +20,7 @@ package kafka import java.io._ import java.nio._ import java.nio.channels._ -import java.util.Random +import java.util.{Properties, Random} import kafka.log._ import kafka.utils._ import kafka.message._ @@ -110,7 +110,10 @@ object TestLinearWriteSpeed { writables(i) = new ChannelWritable(new File(dir, "kafka-test-" + i + ".dat"), buffer) } else if(options.has(logOpt)) { val segmentSize = rand.nextInt(512)*1024*1024 + 64*1024*1024 // vary size to avoid herd effect - writables(i) = new LogWritable(new File(dir, "kafka-test-" + i), new LogConfig(segmentSize=segmentSize, flushInterval = flushInterval), scheduler, messageSet) + val logProperties = new Properties() + logProperties.put(LogConfig.SegmentBytesProp, segmentSize: java.lang.Integer) + logProperties.put(LogConfig.FlushMessagesProp, flushInterval: java.lang.Long) + writables(i) = new LogWritable(new File(dir, "kafka-test-" + i), new LogConfig(logProperties), scheduler, messageSet) } else { System.err.println("Must specify what to write to with one of --log, --channel, or --mmap") System.exit(1) diff --git a/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala b/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala index 375555f..6180b87 100755 --- a/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala +++ b/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala @@ -26,7 +26,7 @@ import org.junit.Assert._ import org.junit.runner.RunWith import org.junit.runners.Parameterized import org.junit.runners.Parameterized.Parameters -import java.util.{ Collection, ArrayList } +import java.util.{Properties, Collection, ArrayList} import kafka.server.KafkaConfig import org.apache.kafka.common.record.CompressionType import scala.collection.JavaConversions._ @@ -54,9 +54,10 @@ class BrokerCompressionTest(messageCompression: String, brokerCompression: Strin @Test def testBrokerSideCompression() { val messageCompressionCode = CompressionCodec.getCompressionCodec(messageCompression) - + val logProps = new Properties() + logProps.put(LogConfig.CompressionTypeProp,brokerCompression) /*configure broker-side compression */ - val log = new Log(logDir, logConfig.copy(compressionType = brokerCompression), recoveryPoint = 0L, time.scheduler, time = time) + val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) /* append two messages */ log.append(new ByteBufferMessageSet(messageCompressionCode, new Message("hello".getBytes), new Message("there".getBytes))) diff --git a/core/src/test/scala/unit/kafka/log/CleanerTest.scala b/core/src/test/scala/unit/kafka/log/CleanerTest.scala index 8b8249a..0e2a6a1 100755 --- a/core/src/test/scala/unit/kafka/log/CleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/CleanerTest.scala @@ -17,6 +17,8 @@ package kafka.log +import java.util.Properties + import junit.framework.Assert._ import org.scalatest.junit.JUnitSuite import org.junit.{After, Test} @@ -35,7 +37,11 @@ import org.apache.kafka.common.utils.Utils class CleanerTest extends JUnitSuite { val dir = TestUtils.tempDir() - val logConfig = LogConfig(segmentSize=1024, maxIndexSize=1024, compact=true) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) + logProps.put(LogConfig.SegmentIndexBytesProp, 1024: java.lang.Integer) + logProps.put(LogConfig.CleanupPolicyProp, LogConfig.Compact) + val logConfig = LogConfig(logProps) val time = new MockTime() val throttler = new Throttler(desiredRatePerSec = Double.MaxValue, checkIntervalMs = Long.MaxValue, time = time) @@ -50,8 +56,11 @@ class CleanerTest extends JUnitSuite { @Test def testCleanSegments() { val cleaner = makeCleaner(Int.MaxValue) - val log = makeLog(config = logConfig.copy(segmentSize = 1024)) - + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) + + val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) + // append messages to the log until we have four segments while(log.numberOfSegments < 4) log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt)) @@ -72,7 +81,10 @@ class CleanerTest extends JUnitSuite { @Test def testCleaningWithDeletes() { val cleaner = makeCleaner(Int.MaxValue) - val log = makeLog(config = logConfig.copy(segmentSize = 1024)) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) + + val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) // append messages with the keys 0 through N while(log.numberOfSegments < 2) @@ -98,7 +110,11 @@ class CleanerTest extends JUnitSuite { val cleaner = makeCleaner(Int.MaxValue) // create a log with compaction turned off so we can append unkeyed messages - val log = makeLog(config = logConfig.copy(segmentSize = 1024, compact = false)) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) + logProps.put(LogConfig.CleanupPolicyProp, LogConfig.Delete) + + val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) // append unkeyed messages while(log.numberOfSegments < 2) @@ -114,7 +130,9 @@ class CleanerTest extends JUnitSuite { val expectedSizeAfterCleaning = log.size - sizeWithUnkeyedMessages // turn on compaction and compact the log - val compactedLog = makeLog(config = logConfig.copy(segmentSize = 1024)) + logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) + + val compactedLog = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) cleaner.clean(LogToClean(TopicAndPartition("test", 0), log, 0)) assertEquals("Log should only contain keyed messages after cleaning.", 0, unkeyedMessageCountInLog(log)) @@ -139,7 +157,10 @@ class CleanerTest extends JUnitSuite { @Test def testCleanSegmentsWithAbort() { val cleaner = makeCleaner(Int.MaxValue, abortCheckDone) - val log = makeLog(config = logConfig.copy(segmentSize = 1024)) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) + + val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) // append messages to the log until we have four segments while(log.numberOfSegments < 4) @@ -159,7 +180,11 @@ class CleanerTest extends JUnitSuite { @Test def testSegmentGrouping() { val cleaner = makeCleaner(Int.MaxValue) - val log = makeLog(config = logConfig.copy(segmentSize = 300, indexInterval = 1)) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 300: java.lang.Integer) + logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer) + + val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) // append some messages to the log var i = 0 @@ -208,7 +233,12 @@ class CleanerTest extends JUnitSuite { @Test def testSegmentGroupingWithSparseOffsets() { val cleaner = makeCleaner(Int.MaxValue) - val log = makeLog(config = logConfig.copy(segmentSize = 1024, indexInterval = 1)) + + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 300: java.lang.Integer) + logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer) + + val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) // fill up first segment while (log.numberOfSegments == 1) @@ -288,7 +318,12 @@ class CleanerTest extends JUnitSuite { @Test def testRecoveryAfterCrash() { val cleaner = makeCleaner(Int.MaxValue) - val config = logConfig.copy(segmentSize = 300, indexInterval = 1, fileDeleteDelayMs = 10) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 300: java.lang.Integer) + logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer) + logProps.put(LogConfig.FileDeleteDelayMsProp, 10: java.lang.Integer) + + val config = LogConfig.fromProps(logConfig.originals, logProps) def recoverAndCheck(config: LogConfig, expectedKeys : Iterable[Int]) : Log = { // Recover log file and check that after recovery, keys are as expected diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala index 471ddff..381e9aa 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala @@ -18,6 +18,7 @@ package kafka.log import java.io.File +import java.util.Properties import kafka.common.TopicAndPartition import kafka.message._ @@ -127,8 +128,13 @@ class LogCleanerIntegrationTest(compressionCodec: String) extends JUnit3Suite { for(i <- 0 until parts) { val dir = new File(logDir, "log-" + i) dir.mkdirs() + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, segmentSize: java.lang.Integer) + logProps.put(LogConfig.SegmentIndexBytesProp, 100*1024: java.lang.Integer) + logProps.put(LogConfig.FileDeleteDelayMsProp, deleteDelay: java.lang.Integer) + logProps.put(LogConfig.CleanupPolicyProp, LogConfig.Compact) val log = new Log(dir = dir, - LogConfig(segmentSize = segmentSize, maxIndexSize = 100*1024, fileDeleteDelayMs = deleteDelay, compact = true), + LogConfig(logProps), recoveryPoint = 0L, scheduler = time.scheduler, time = time) diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala index 3fd5a53..c31f884 100644 --- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala @@ -26,22 +26,9 @@ import org.scalatest.junit.JUnit3Suite class LogConfigTest extends JUnit3Suite { @Test - def testFromPropsDefaults() { - val defaults = new Properties() - defaults.put(LogConfig.SegmentBytesProp, "4242") - val props = new Properties(defaults) - - val config = LogConfig.fromProps(props) - - Assert.assertEquals(4242, config.segmentSize) - Assert.assertEquals("LogConfig defaults should be retained", Defaults.MaxMessageSize, config.maxMessageSize) - Assert.assertEquals("producer", config.compressionType) - } - - @Test def testFromPropsEmpty() { val p = new Properties() - val config = LogConfig.fromProps(p) + val config = LogConfig(p) Assert.assertEquals(LogConfig(), config) } @@ -62,7 +49,7 @@ class LogConfigTest extends JUnit3Suite { } }) - val actual = LogConfig.fromProps(expected).toProps + val actual = LogConfig(expected).originals Assert.assertEquals(expected, actual) } @@ -86,7 +73,7 @@ class LogConfigTest extends JUnit3Suite { val props = new Properties props.setProperty(name, value.toString) intercept[ConfigException] { - LogConfig.fromProps(props) + LogConfig(props) } }) } diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index 01dfbc4..a13f2be 100755 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -18,6 +18,7 @@ package kafka.log import java.io._ +import java.util.Properties import junit.framework.Assert._ import org.junit.Test import org.scalatest.junit.JUnit3Suite @@ -30,7 +31,11 @@ class LogManagerTest extends JUnit3Suite { val time: MockTime = new MockTime() val maxRollInterval = 100 val maxLogAgeMs = 10*60*60*1000 - val logConfig = LogConfig(segmentSize = 1024, maxIndexSize = 4096, retentionMs = maxLogAgeMs) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) + logProps.put(LogConfig.SegmentIndexBytesProp, 4096: java.lang.Integer) + logProps.put(LogConfig.RetentionMsProp, maxLogAgeMs: java.lang.Integer) + val logConfig = LogConfig(logProps) var logDir: File = null var logManager: LogManager = null val name = "kafka" @@ -113,8 +118,11 @@ class LogManagerTest extends JUnit3Suite { def testCleanupSegmentsToMaintainSize() { val setSize = TestUtils.singleMessageSet("test".getBytes()).sizeInBytes logManager.shutdown() + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 10 * setSize: java.lang.Integer) + logProps.put(LogConfig.RetentionBytesProp, 5L * 10L * setSize + 10L: java.lang.Long) + val config = LogConfig.fromProps(logConfig.originals, logProps) - val config = logConfig.copy(segmentSize = 10 * setSize, retentionSize = 5L * 10L * setSize + 10L) logManager = createLogManager() logManager.startup @@ -154,7 +162,10 @@ class LogManagerTest extends JUnit3Suite { @Test def testTimeBasedFlush() { logManager.shutdown() - val config = logConfig.copy(flushMs = 1000) + val logProps = new Properties() + logProps.put(LogConfig.FlushMsProp, 1000: java.lang.Integer) + val config = LogConfig.fromProps(logConfig.originals, logProps) + logManager = createLogManager() logManager.startup val log = logManager.createLog(TopicAndPartition(name, 0), config) diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index 8e095d6..a8e57c2 100755 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -18,6 +18,7 @@ package kafka.log import java.io._ +import java.util.Properties import java.util.concurrent.atomic._ import junit.framework.Assert._ import org.scalatest.junit.JUnitSuite @@ -61,9 +62,12 @@ class LogTest extends JUnitSuite { def testTimeBasedLogRoll() { val set = TestUtils.singleMessageSet("test".getBytes()) + val logProps = new Properties() + logProps.put(LogConfig.SegmentMsProp, 1 * 60 * 60: java.lang.Long) + // create a log val log = new Log(logDir, - logConfig.copy(segmentMs = 1 * 60 * 60L), + LogConfig(logProps), recoveryPoint = 0L, scheduler = time.scheduler, time = time) @@ -96,9 +100,12 @@ class LogTest extends JUnitSuite { val set = TestUtils.singleMessageSet("test".getBytes()) val maxJitter = 20 * 60L + val logProps = new Properties() + logProps.put(LogConfig.SegmentMsProp, 1 * 60 * 60: java.lang.Long) + logProps.put(LogConfig.SegmentJitterMsProp, maxJitter: java.lang.Long) // create a log val log = new Log(logDir, - logConfig.copy(segmentMs = 1 * 60 * 60L, segmentJitterMs = maxJitter), + LogConfig(logProps), recoveryPoint = 0L, scheduler = time.scheduler, time = time) @@ -123,8 +130,10 @@ class LogTest extends JUnitSuite { val msgPerSeg = 10 val segmentSize = msgPerSeg * (setSize - 1) // each segment will be 10 messages + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, segmentSize: java.lang.Integer) // create a log - val log = new Log(logDir, logConfig.copy(segmentSize = segmentSize), recoveryPoint = 0L, time.scheduler, time = time) + val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments) // segments expire in size @@ -149,7 +158,9 @@ class LogTest extends JUnitSuite { */ @Test def testAppendAndReadWithSequentialOffsets() { - val log = new Log(logDir, logConfig.copy(segmentSize = 71), recoveryPoint = 0L, time.scheduler, time = time) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 71: java.lang.Integer) + val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) val messages = (0 until 100 by 2).map(id => new Message(id.toString.getBytes)).toArray for(i <- 0 until messages.length) @@ -168,7 +179,9 @@ class LogTest extends JUnitSuite { */ @Test def testAppendAndReadWithNonSequentialOffsets() { - val log = new Log(logDir, logConfig.copy(segmentSize = 71), recoveryPoint = 0L, time.scheduler, time = time) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 71: java.lang.Integer) + val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) val messageIds = ((0 until 50) ++ (50 until 200 by 7)).toArray val messages = messageIds.map(id => new Message(id.toString.getBytes)) @@ -191,7 +204,9 @@ class LogTest extends JUnitSuite { */ @Test def testReadAtLogGap() { - val log = new Log(logDir, logConfig.copy(segmentSize = 300), recoveryPoint = 0L, time.scheduler, time = time) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 300: java.lang.Integer) + val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) // keep appending until we have two segments with only a single message in the second segment while(log.numberOfSegments == 1) @@ -211,7 +226,9 @@ class LogTest extends JUnitSuite { @Test def testReadOutOfRange() { createEmptyLogs(logDir, 1024) - val log = new Log(logDir, logConfig.copy(segmentSize = 1024), recoveryPoint = 0L, time.scheduler, time = time) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) + val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) assertEquals("Reading just beyond end of log should produce 0 byte read.", 0, log.read(1024, 1000).messageSet.sizeInBytes) try { log.read(0, 1024) @@ -234,7 +251,9 @@ class LogTest extends JUnitSuite { @Test def testLogRolls() { /* create a multipart log with 100 messages */ - val log = new Log(logDir, logConfig.copy(segmentSize = 100), recoveryPoint = 0L, time.scheduler, time = time) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 100: java.lang.Integer) + val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) val numMessages = 100 val messageSets = (0 until numMessages).map(i => TestUtils.singleMessageSet(i.toString.getBytes)) messageSets.foreach(log.append(_)) @@ -263,7 +282,9 @@ class LogTest extends JUnitSuite { @Test def testCompressedMessages() { /* this log should roll after every messageset */ - val log = new Log(logDir, logConfig.copy(segmentSize = 100), recoveryPoint = 0L, time.scheduler, time = time) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 100: java.lang.Integer) + val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) /* append 2 compressed message sets, each with two messages giving offsets 0, 1, 2, 3 */ log.append(new ByteBufferMessageSet(DefaultCompressionCodec, new Message("hello".getBytes), new Message("there".getBytes))) @@ -286,7 +307,9 @@ class LogTest extends JUnitSuite { for(messagesToAppend <- List(0, 1, 25)) { logDir.mkdirs() // first test a log segment starting at 0 - val log = new Log(logDir, logConfig.copy(segmentSize = 100), recoveryPoint = 0L, time.scheduler, time = time) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 100: java.lang.Integer) + val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) for(i <- 0 until messagesToAppend) log.append(TestUtils.singleMessageSet(i.toString.getBytes)) @@ -318,7 +341,9 @@ class LogTest extends JUnitSuite { val messageSet = new ByteBufferMessageSet(NoCompressionCodec, new Message ("You".getBytes), new Message("bethe".getBytes)) // append messages to log val configSegmentSize = messageSet.sizeInBytes - 1 - val log = new Log(logDir, logConfig.copy(segmentSize = configSegmentSize), recoveryPoint = 0L, time.scheduler, time = time) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, configSegmentSize: java.lang.Integer) + val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) try { log.append(messageSet) @@ -342,7 +367,10 @@ class LogTest extends JUnitSuite { val messageSetWithKeyedMessage = new ByteBufferMessageSet(NoCompressionCodec, keyedMessage) val messageSetWithKeyedMessages = new ByteBufferMessageSet(NoCompressionCodec, keyedMessage, anotherKeyedMessage) - val log = new Log(logDir, logConfig.copy(compact = true), recoveryPoint = 0L, time.scheduler, time) + val logProps = new Properties() + logProps.put(LogConfig.CleanupPolicyProp, LogConfig.Compact) + + val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time) try { log.append(messageSetWithUnkeyedMessage) @@ -380,7 +408,9 @@ class LogTest extends JUnitSuite { // append messages to log val maxMessageSize = second.sizeInBytes - 1 - val log = new Log(logDir, logConfig.copy(maxMessageSize = maxMessageSize), recoveryPoint = 0L, time.scheduler, time = time) + val logProps = new Properties() + logProps.put(LogConfig.MaxMessageBytesProp, maxMessageSize: java.lang.Integer) + val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) // should be able to append the small message log.append(first) @@ -401,7 +431,11 @@ class LogTest extends JUnitSuite { val messageSize = 100 val segmentSize = 7 * messageSize val indexInterval = 3 * messageSize - val config = logConfig.copy(segmentSize = segmentSize, indexInterval = indexInterval, maxIndexSize = 4096) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, segmentSize: java.lang.Integer) + logProps.put(LogConfig.IndexIntervalBytesProp, indexInterval: java.lang.Integer) + logProps.put(LogConfig.SegmentIndexBytesProp, 4096: java.lang.Integer) + val config = LogConfig(logProps) var log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time) for(i <- 0 until numMessages) log.append(TestUtils.singleMessageSet(TestUtils.randomBytes(messageSize))) @@ -432,7 +466,11 @@ class LogTest extends JUnitSuite { def testIndexRebuild() { // publish the messages and close the log val numMessages = 200 - val config = logConfig.copy(segmentSize = 200, indexInterval = 1) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 200: java.lang.Integer) + logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer) + + val config = LogConfig(logProps) var log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time) for(i <- 0 until numMessages) log.append(TestUtils.singleMessageSet(TestUtils.randomBytes(10))) @@ -460,8 +498,11 @@ class LogTest extends JUnitSuite { val msgPerSeg = 10 val segmentSize = msgPerSeg * setSize // each segment will be 10 messages + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, segmentSize: java.lang.Integer) + // create a log - val log = new Log(logDir, logConfig.copy(segmentSize = segmentSize), recoveryPoint = 0L, scheduler = time.scheduler, time = time) + val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, scheduler = time.scheduler, time = time) assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments) for (i<- 1 to msgPerSeg) @@ -513,7 +554,9 @@ class LogTest extends JUnitSuite { val setSize = set.sizeInBytes val msgPerSeg = 10 val segmentSize = msgPerSeg * setSize // each segment will be 10 messages - val config = logConfig.copy(segmentSize = segmentSize) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, segmentSize: java.lang.Integer) + val config = LogConfig(logProps) val log = new Log(logDir, config, recoveryPoint = 0L, scheduler = time.scheduler, time = time) assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments) for (i<- 1 to msgPerSeg) @@ -540,10 +583,12 @@ class LogTest extends JUnitSuite { val bogusIndex2 = Log.indexFilename(logDir, 5) val set = TestUtils.singleMessageSet("test".getBytes()) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer) + logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer) + logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer) val log = new Log(logDir, - logConfig.copy(segmentSize = set.sizeInBytes * 5, - maxIndexSize = 1000, - indexInterval = 1), + LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time) @@ -564,9 +609,11 @@ class LogTest extends JUnitSuite { @Test def testReopenThenTruncate() { val set = TestUtils.singleMessageSet("test".getBytes()) - val config = logConfig.copy(segmentSize = set.sizeInBytes * 5, - maxIndexSize = 1000, - indexInterval = 10000) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer) + logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer) + logProps.put(LogConfig.IndexIntervalBytesProp, 10000: java.lang.Integer) + val config = LogConfig(logProps) // create a log var log = new Log(logDir, @@ -596,10 +643,13 @@ class LogTest extends JUnitSuite { def testAsyncDelete() { val set = TestUtils.singleMessageSet("test".getBytes()) val asyncDeleteMs = 1000 - val config = logConfig.copy(segmentSize = set.sizeInBytes * 5, - fileDeleteDelayMs = asyncDeleteMs, - maxIndexSize = 1000, - indexInterval = 10000) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer) + logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer) + logProps.put(LogConfig.IndexIntervalBytesProp, 10000: java.lang.Integer) + logProps.put(LogConfig.FileDeleteDelayMsProp, asyncDeleteMs: java.lang.Integer) + val config = LogConfig(logProps) + val log = new Log(logDir, config, recoveryPoint = 0L, @@ -634,7 +684,10 @@ class LogTest extends JUnitSuite { @Test def testOpenDeletesObsoleteFiles() { val set = TestUtils.singleMessageSet("test".getBytes()) - val config = logConfig.copy(segmentSize = set.sizeInBytes * 5, maxIndexSize = 1000) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer) + logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer) + val config = LogConfig(logProps) var log = new Log(logDir, config, recoveryPoint = 0L, @@ -672,7 +725,11 @@ class LogTest extends JUnitSuite { @Test def testCorruptLog() { // append some messages to create some segments - val config = logConfig.copy(indexInterval = 1, maxMessageSize = 64*1024, segmentSize = 1000) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 1000: java.lang.Integer) + logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer) + logProps.put(LogConfig.MaxMessageBytesProp, 64*1024: java.lang.Integer) + val config = LogConfig(logProps) val set = TestUtils.singleMessageSet("test".getBytes()) val recoveryPoint = 50L for(iteration <- 0 until 50) { @@ -704,7 +761,11 @@ class LogTest extends JUnitSuite { @Test def testCleanShutdownFile() { // append some messages to create some segments - val config = logConfig.copy(indexInterval = 1, maxMessageSize = 64*1024, segmentSize = 1000) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 1000: java.lang.Integer) + logProps.put(LogConfig.MaxMessageBytesProp, 64*1024: java.lang.Integer) + logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer) + val config = LogConfig(logProps) val set = TestUtils.singleMessageSet("test".getBytes()) val parentLogDir = logDir.getParentFile assertTrue("Data directory %s must exist", parentLogDir.isDirectory) diff --git a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala index 7877f6c..8a871cf 100644 --- a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala @@ -16,6 +16,8 @@ */ package kafka.server +import java.util.Properties + import junit.framework.Assert._ import org.junit.Test import kafka.integration.KafkaServerTestHarness @@ -30,16 +32,19 @@ class DynamicConfigChangeTest extends JUnit3Suite with KafkaServerTestHarness { @Test def testConfigChange() { - val oldVal = 100000 - val newVal = 200000 + val oldVal: java.lang.Long = 100000 + val newVal: java.lang.Long = 200000 val tp = TopicAndPartition("test", 0) - AdminUtils.createTopic(zkClient, tp.topic, 1, 1, LogConfig(flushInterval = oldVal).toProps) + val logProps = new Properties() + logProps.put(LogConfig.FlushMessagesProp, oldVal.toString) + AdminUtils.createTopic(zkClient, tp.topic, 1, 1, logProps) TestUtils.retry(10000) { val logOpt = this.servers(0).logManager.getLog(tp) assertTrue(logOpt.isDefined) assertEquals(oldVal, logOpt.get.config.flushInterval) } - AdminUtils.changeTopicConfig(zkClient, tp.topic, LogConfig(flushInterval = newVal).toProps) + logProps.put(LogConfig.FlushMessagesProp, newVal.toString) + AdminUtils.changeTopicConfig(zkClient, tp.topic, logProps) TestUtils.retry(10000) { assertEquals(newVal, this.servers(0).logManager.getLog(tp).get.config.flushInterval) } @@ -49,7 +54,9 @@ class DynamicConfigChangeTest extends JUnit3Suite with KafkaServerTestHarness { def testConfigChangeOnNonExistingTopic() { val topic = TestUtils.tempTopic try { - AdminUtils.changeTopicConfig(zkClient, topic, LogConfig(flushInterval = 10000).toProps) + val logProps = new Properties() + logProps.put(LogConfig.FlushMessagesProp, 10000: java.lang.Integer) + AdminUtils.changeTopicConfig(zkClient, topic, logProps) fail("Should fail with AdminOperationException for topic doesn't exist") } catch { case e: AdminOperationException => // expected diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala index c487f36..8268852 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala @@ -30,29 +30,13 @@ import scala.util.Random._ class KafkaConfigConfigDefTest extends JUnit3Suite { @Test - def testFromPropsDefaults() { - val defaults = new Properties() - defaults.put(KafkaConfig.ZkConnectProp, "127.0.0.1:2181") - - // some ordinary setting - defaults.put(KafkaConfig.AdvertisedPortProp, "1818") - - val props = new Properties(defaults) - - val config = KafkaConfig.fromProps(props) - - Assert.assertEquals(1818, config.advertisedPort) - Assert.assertEquals("KafkaConfig defaults should be retained", Defaults.ConnectionsMaxIdleMs, config.connectionsMaxIdleMs) - } - - @Test def testFromPropsEmpty() { // only required val p = new Properties() p.put(KafkaConfig.ZkConnectProp, "127.0.0.1:2181") val actualConfig = KafkaConfig.fromProps(p) - val expectedConfig = new KafkaConfig(zkConnect = "127.0.0.1:2181") + val expectedConfig = new KafkaConfig(p) Assert.assertEquals(expectedConfig.zkConnect, actualConfig.zkConnect) Assert.assertEquals(expectedConfig.zkSessionTimeoutMs, actualConfig.zkSessionTimeoutMs) @@ -252,7 +236,7 @@ class KafkaConfigConfigDefTest extends JUnit3Suite { } }) - val actual = KafkaConfig.fromProps(expected).toProps + val actual = KafkaConfig.fromProps(expected).originals Assert.assertEquals(expected, actual) } -- 1.7.12.4 From d9c0ad6855b4871694ddf17c9d4546b021302ee8 Mon Sep 17 00:00:00 2001 From: Manikumar Reddy Date: Thu, 18 Jun 2015 15:59:11 -0700 Subject: [PATCH 015/120] kafka-2265; creating a topic with large number of partitions takes a long time; patched by Manikumar Reddy; reviewed by Jun Rao --- .../main/scala/kafka/controller/PartitionStateMachine.scala | 13 +------------ 1 file changed, 1 insertion(+), 12 deletions(-) diff --git a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala index 92fd92d..b4e7c88 100755 --- a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala +++ b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala @@ -188,7 +188,6 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { case NewPartition => // pre: partition did not exist before this assertValidPreviousStates(topicAndPartition, List(NonExistentPartition), NewPartition) - assignReplicasToPartitions(topic, partition) partitionState.put(topicAndPartition, NewPartition) val assignedReplicas = controllerContext.partitionReplicaAssignment(topicAndPartition).mkString(",") stateChangeLogger.trace("Controller %d epoch %d changed partition %s state from %s to %s with assigned replicas %s" @@ -266,17 +265,6 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { } /** - * Invoked on the NonExistentPartition->NewPartition state transition to update the controller's cache with the - * partition's replica assignment. - * @param topic The topic of the partition whose replica assignment is to be cached - * @param partition The partition whose replica assignment is to be cached - */ - private def assignReplicasToPartitions(topic: String, partition: Int) { - val assignedReplicas = ZkUtils.getReplicasForPartition(controllerContext.zkClient, topic, partition) - controllerContext.partitionReplicaAssignment += TopicAndPartition(topic, partition) -> assignedReplicas - } - - /** * Invoked on the NewPartition->OnlinePartition state change. When a partition is in the New state, it does not have * a leader and isr path in zookeeper. Once the partition moves to the OnlinePartition state, it's leader and isr * path gets initialized and it never goes back to the NewPartition state. From here, it can only go to the @@ -526,6 +514,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { else { if (partitionsToBeAdded.size > 0) { info("New partitions to be added %s".format(partitionsToBeAdded)) + controllerContext.partitionReplicaAssignment.++=(partitionsToBeAdded) controller.onNewPartitionCreation(partitionsToBeAdded.keySet.toSet) } } -- 1.7.12.4 From 5c2ca30f229c7f39fca65aed6bd45c382aacda77 Mon Sep 17 00:00:00 2001 From: Manikumar Reddy Date: Thu, 18 Jun 2015 16:37:25 -0700 Subject: [PATCH 016/120] kafka-2234; Partition reassignment of a nonexistent topic prevents future reassignments; patched by Manikumar Reddy; reviewed by Jun Rao --- .../main/scala/kafka/admin/ReassignPartitionsCommand.scala | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala b/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala index 912b718..ea34589 100755 --- a/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala +++ b/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala @@ -208,9 +208,14 @@ class ReassignPartitionsCommand(zkClient: ZkClient, partitions: collection.Map[T def reassignPartitions(): Boolean = { try { val validPartitions = partitions.filter(p => validatePartition(zkClient, p._1.topic, p._1.partition)) - val jsonReassignmentData = ZkUtils.getPartitionReassignmentZkData(validPartitions) - ZkUtils.createPersistentPath(zkClient, ZkUtils.ReassignPartitionsPath, jsonReassignmentData) - true + if(validPartitions.isEmpty) { + false + } + else { + val jsonReassignmentData = ZkUtils.getPartitionReassignmentZkData(validPartitions) + ZkUtils.createPersistentPath(zkClient, ZkUtils.ReassignPartitionsPath, jsonReassignmentData) + true + } } catch { case ze: ZkNodeExistsException => val partitionsBeingReassigned = ZkUtils.getPartitionsBeingReassigned(zkClient) -- 1.7.12.4 From 1c93bb16a91788d3489d4d784c13f1f0fddb6fb7 Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Thu, 18 Jun 2015 17:00:34 -0700 Subject: [PATCH 017/120] trivial change to fix unit test failure introduced in kafka-2234 --- core/src/test/scala/unit/kafka/admin/AdminTest.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/unit/kafka/admin/AdminTest.scala b/core/src/test/scala/unit/kafka/admin/AdminTest.scala index efb2f8e..252ac81 100755 --- a/core/src/test/scala/unit/kafka/admin/AdminTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AdminTest.scala @@ -242,7 +242,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { val partitionToBeReassigned = 0 val topicAndPartition = TopicAndPartition(topic, partitionToBeReassigned) val reassignPartitionsCommand = new ReassignPartitionsCommand(zkClient, Map(topicAndPartition -> newReplicas)) - assertTrue("Partition reassignment failed for test, 0", reassignPartitionsCommand.reassignPartitions()) + assertFalse("Partition reassignment failed for test, 0", reassignPartitionsCommand.reassignPartitions()) val reassignedPartitions = ZkUtils.getPartitionsBeingReassigned(zkClient) assertFalse("Partition should not be reassigned", reassignedPartitions.contains(topicAndPartition)) servers.foreach(_.shutdown()) -- 1.7.12.4 From 19c98cb8ed69d3d6da787dc04e1e88add5f6b9d7 Mon Sep 17 00:00:00 2001 From: Manikumar Reddy Date: Thu, 18 Jun 2015 18:50:52 -0700 Subject: [PATCH 018/120] kafka-1758; corrupt recovery file prevents startup; patched by Manikumar Reddy; reviewed by Neha Narkhede and Jun Rao --- core/src/main/scala/kafka/log/LogManager.scala | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index 538fc83..69386c1 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -126,7 +126,15 @@ class LogManager(val logDirs: Array[File], brokerState.newState(RecoveringFromUncleanShutdown) } - val recoveryPoints = this.recoveryPointCheckpoints(dir).read + var recoveryPoints = Map[TopicAndPartition, Long]() + try { + recoveryPoints = this.recoveryPointCheckpoints(dir).read + } catch { + case e: Exception => { + warn("Error occured while reading recovery-point-offset-checkpoint file of directory " + dir, e) + warn("Resetting the recovery checkpoint to 0") + } + } val jobsForDir = for { dirContent <- Option(dir.listFiles).toList -- 1.7.12.4 From ca758252c5a524fe6135a585282dd4bf747afef2 Mon Sep 17 00:00:00 2001 From: Honghai Chen Date: Fri, 19 Jun 2015 07:52:37 -0700 Subject: [PATCH 019/120] kafka-1646; Improve consumer read performance for Windows; patched by Honghai Chen; reviewed by Jay Kreps and Jun Rao --- core/src/main/scala/kafka/log/FileMessageSet.scala | 58 ++++++++++++++++++++-- core/src/main/scala/kafka/log/Log.scala | 30 +++++++++-- core/src/main/scala/kafka/log/LogCleaner.scala | 2 +- core/src/main/scala/kafka/log/LogConfig.scala | 6 +++ core/src/main/scala/kafka/log/LogSegment.scala | 4 +- core/src/main/scala/kafka/server/KafkaConfig.scala | 5 ++ core/src/main/scala/kafka/server/KafkaServer.scala | 1 + core/src/main/scala/kafka/utils/CoreUtils.scala | 10 ---- .../scala/unit/kafka/log/FileMessageSetTest.scala | 55 +++++++++++++++++++- .../test/scala/unit/kafka/log/LogConfigTest.scala | 1 + .../test/scala/unit/kafka/log/LogSegmentTest.scala | 54 +++++++++++++++++++- .../kafka/server/KafkaConfigConfigDefTest.scala | 1 + 12 files changed, 204 insertions(+), 23 deletions(-) diff --git a/core/src/main/scala/kafka/log/FileMessageSet.scala b/core/src/main/scala/kafka/log/FileMessageSet.scala index 2522604..39361fe 100755 --- a/core/src/main/scala/kafka/log/FileMessageSet.scala +++ b/core/src/main/scala/kafka/log/FileMessageSet.scala @@ -54,7 +54,7 @@ class FileMessageSet private[kafka](@volatile var file: File, /* if this is not a slice, update the file pointer to the end of the file */ if (!isSlice) /* set the file position to the last byte in the file */ - channel.position(channel.size) + channel.position(math.min(channel.size().toInt, end)) /** * Create a file message set with no slicing. @@ -66,12 +66,25 @@ class FileMessageSet private[kafka](@volatile var file: File, * Create a file message set with no slicing */ def this(file: File) = - this(file, CoreUtils.openChannel(file, mutable = true)) + this(file, FileMessageSet.openChannel(file, mutable = true)) + + /** + * Create a file message set with no slicing, and with initFileSize and preallocate. + * For windows NTFS and some old LINUX file system, set preallocate to true and initFileSize + * with one value (for example 512 * 1024 *1024 ) can improve the kafka produce performance. + * If it's new file and preallocate is true, end will be set to 0. Otherwise set to Int.MaxValue. + */ + def this(file: File, fileAlreadyExists: Boolean, initFileSize: Int, preallocate: Boolean) = + this(file, + channel = FileMessageSet.openChannel(file, mutable = true, fileAlreadyExists, initFileSize, preallocate), + start = 0, + end = ( if ( !fileAlreadyExists && preallocate ) 0 else Int.MaxValue), + isSlice = false) /** * Create a file message set with mutable option */ - def this(file: File, mutable: Boolean) = this(file, CoreUtils.openChannel(file, mutable)) + def this(file: File, mutable: Boolean) = this(file, FileMessageSet.openChannel(file, mutable)) /** * Create a slice view of the file message set that begins and ends at the given byte offsets @@ -223,10 +236,18 @@ class FileMessageSet private[kafka](@volatile var file: File, */ def close() { flush() + trim() channel.close() } /** + * Trim file when close or roll to next file + */ + def trim() { + truncateTo(sizeInBytes()) + } + + /** * Delete this message set from the filesystem * @return True iff this message set was deleted. */ @@ -272,6 +293,37 @@ class FileMessageSet private[kafka](@volatile var file: File, } } + +object FileMessageSet +{ + /** + * Open a channel for the given file + * For windows NTFS and some old LINUX file system, set preallocate to true and initFileSize + * with one value (for example 512 * 1025 *1024 ) can improve the kafka produce performance. + * @param file File path + * @param mutable mutable + * @param fileAlreadyExists File already exists or not + * @param initFileSize The size used for pre allocate file, for example 512 * 1025 *1024 + * @param preallocate Pre allocate file or not, gotten from configuration. + */ + def openChannel(file: File, mutable: Boolean, fileAlreadyExists: Boolean = false, initFileSize: Int = 0, preallocate: Boolean = false): FileChannel = { + if (mutable) { + if (fileAlreadyExists) + new RandomAccessFile(file, "rw").getChannel() + else { + if (preallocate) { + val randomAccessFile = new RandomAccessFile(file, "rw") + randomAccessFile.setLength(initFileSize) + randomAccessFile.getChannel() + } + else + new RandomAccessFile(file, "rw").getChannel() + } + } + else + new FileInputStream(file).getChannel() + } +} object LogFlushStats extends KafkaMetricsGroup { val logFlushTimer = new KafkaTimer(newTimer("LogFlushRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS)) diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 84e7b8f..6b9274d 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -78,6 +78,13 @@ class Log(val dir: File, /* last time it was flushed */ private val lastflushedTime = new AtomicLong(time.milliseconds) + def initFileSize() : Int = { + if (config.preallocate) + config.segmentSize + else + 0 + } + /* the actual segments of the log */ private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment] loadSegments() @@ -168,7 +175,8 @@ class Log(val dir: File, indexIntervalBytes = config.indexInterval, maxIndexSize = config.maxIndexSize, rollJitterMs = config.randomSegmentJitter, - time = time) + time = time, + fileAlreadyExists = true) if(!hasIndex) { error("Could not find index file corresponding to log file %s, rebuilding index...".format(segment.log.file.getAbsolutePath)) segment.recover(config.maxMessageSize) @@ -205,7 +213,10 @@ class Log(val dir: File, indexIntervalBytes = config.indexInterval, maxIndexSize = config.maxIndexSize, rollJitterMs = config.randomSegmentJitter, - time = time)) + time = time, + fileAlreadyExists = false, + initFileSize = this.initFileSize(), + preallocate = config.preallocate)) } else { recoverLog() // reset the index size of the currently active log segment to allow more entries @@ -586,14 +597,20 @@ class Log(val dir: File, segments.lastEntry() match { case null => - case entry => entry.getValue.index.trimToValidSize() + case entry => { + entry.getValue.index.trimToValidSize() + entry.getValue.log.trim() + } } val segment = new LogSegment(dir, startOffset = newOffset, indexIntervalBytes = config.indexInterval, maxIndexSize = config.maxIndexSize, rollJitterMs = config.randomSegmentJitter, - time = time) + time = time, + fileAlreadyExists = false, + initFileSize = initFileSize, + preallocate = config.preallocate) val prev = addSegment(segment) if(prev != null) throw new KafkaException("Trying to roll a new log segment for topic partition %s with start offset %d while it already exists.".format(name, newOffset)) @@ -687,7 +704,10 @@ class Log(val dir: File, indexIntervalBytes = config.indexInterval, maxIndexSize = config.maxIndexSize, rollJitterMs = config.randomSegmentJitter, - time = time)) + time = time, + fileAlreadyExists = false, + initFileSize = initFileSize, + preallocate = config.preallocate)) updateLogEndOffset(newOffset) this.recoveryPoint = math.min(newOffset, this.recoveryPoint) } diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index c9ade72..d07a391 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -344,7 +344,7 @@ private[log] class Cleaner(val id: Int, logFile.delete() val indexFile = new File(segments.head.index.file.getPath + Log.CleanedFileSuffix) indexFile.delete() - val messages = new FileMessageSet(logFile) + val messages = new FileMessageSet(logFile, fileAlreadyExists = false, initFileSize = log.initFileSize(), preallocate = log.config.preallocate) val index = new OffsetIndex(indexFile, segments.head.baseOffset, segments.head.index.maxIndexSize) val cleaned = new LogSegment(messages, index, segments.head.baseOffset, segments.head.indexIntervalBytes, log.config.randomSegmentJitter, time) diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala index e9af221..fc41132 100755 --- a/core/src/main/scala/kafka/log/LogConfig.scala +++ b/core/src/main/scala/kafka/log/LogConfig.scala @@ -43,6 +43,7 @@ object Defaults { val UncleanLeaderElectionEnable = kafka.server.Defaults.UncleanLeaderElectionEnable val MinInSyncReplicas = kafka.server.Defaults.MinInSyncReplicas val CompressionType = kafka.server.Defaults.CompressionType + val PreAllocateEnable = kafka.server.Defaults.LogPreAllocateEnable } case class LogConfig(props: java.util.Map[_, _]) extends AbstractConfig(LogConfig.configDef, props) { @@ -64,6 +65,7 @@ case class LogConfig(props: java.util.Map[_, _]) extends AbstractConfig(LogConfi val uncleanLeaderElectionEnable = getBoolean(LogConfig.UncleanLeaderElectionEnableProp) val minInSyncReplicas = getInt(LogConfig.MinInSyncReplicasProp) val compressionType = getString(LogConfig.CompressionTypeProp).toLowerCase + val preallocate = getBoolean(LogConfig.PreAllocateEnableProp) def randomSegmentJitter: Long = if (segmentJitterMs == 0) 0 else Utils.abs(scala.util.Random.nextInt()) % math.min(segmentJitterMs, segmentMs) @@ -95,6 +97,7 @@ object LogConfig { val UncleanLeaderElectionEnableProp = "unclean.leader.election.enable" val MinInSyncReplicasProp = "min.insync.replicas" val CompressionTypeProp = "compression.type" + val PreAllocateEnableProp = "preallocate" val SegmentSizeDoc = "The hard maximum for the size of a segment file in the log" val SegmentMsDoc = "The soft maximum on the amount of time before a new log segment is rolled" @@ -118,6 +121,7 @@ object LogConfig { val CompressionTypeDoc = "Specify the final compression type for a given topic. This configuration accepts the " + "standard compression codecs ('gzip', 'snappy', lz4). It additionally accepts 'uncompressed' which is equivalent to " + "no compression; and 'producer' which means retain the original compression codec set by the producer." + val PreAllocateEnableDoc ="Should pre allocate file when create new segment?" private val configDef = { import ConfigDef.Range._ @@ -149,6 +153,8 @@ object LogConfig { MEDIUM, UncleanLeaderElectionEnableDoc) .define(MinInSyncReplicasProp, INT, Defaults.MinInSyncReplicas, atLeast(1), MEDIUM, MinInSyncReplicasDoc) .define(CompressionTypeProp, STRING, Defaults.CompressionType, in(BrokerCompressionCodec.brokerCompressionOptions:_*), MEDIUM, CompressionTypeDoc) + .define(PreAllocateEnableProp, BOOLEAN, Defaults.PreAllocateEnable, + MEDIUM, PreAllocateEnableDoc) } def apply(): LogConfig = LogConfig(new Properties()) diff --git a/core/src/main/scala/kafka/log/LogSegment.scala b/core/src/main/scala/kafka/log/LogSegment.scala index ed03953..1377e8f 100755 --- a/core/src/main/scala/kafka/log/LogSegment.scala +++ b/core/src/main/scala/kafka/log/LogSegment.scala @@ -52,8 +52,8 @@ class LogSegment(val log: FileMessageSet, /* the number of bytes since we last added an entry in the offset index */ private var bytesSinceLastIndexEntry = 0 - def this(dir: File, startOffset: Long, indexIntervalBytes: Int, maxIndexSize: Int, rollJitterMs: Long, time: Time) = - this(new FileMessageSet(file = Log.logFilename(dir, startOffset)), + def this(dir: File, startOffset: Long, indexIntervalBytes: Int, maxIndexSize: Int, rollJitterMs: Long, time: Time, fileAlreadyExists: Boolean = false, initFileSize: Int = 0, preallocate: Boolean = false) = + this(new FileMessageSet(file = Log.logFilename(dir, startOffset), fileAlreadyExists = fileAlreadyExists, initFileSize = initFileSize, preallocate = preallocate), new OffsetIndex(file = Log.indexFilename(dir, startOffset), baseOffset = startOffset, maxIndexSize = maxIndexSize), startOffset, indexIntervalBytes, diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index e0b2480..c1f0cca 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -83,6 +83,7 @@ object Defaults { val LogDeleteDelayMs = 60000 val LogFlushSchedulerIntervalMs = Long.MaxValue val LogFlushOffsetCheckpointIntervalMs = 60000 + val LogPreAllocateEnable = false val NumRecoveryThreadsPerDataDir = 1 val AutoCreateTopicsEnable = true val MinInSyncReplicas = 1 @@ -206,6 +207,7 @@ object KafkaConfig { val LogFlushSchedulerIntervalMsProp = "log.flush.scheduler.interval.ms" val LogFlushIntervalMsProp = "log.flush.interval.ms" val LogFlushOffsetCheckpointIntervalMsProp = "log.flush.offset.checkpoint.interval.ms" + val LogPreAllocateProp = "log.preallocate" val NumRecoveryThreadsPerDataDirProp = "num.recovery.threads.per.data.dir" val AutoCreateTopicsEnableProp = "auto.create.topics.enable" val MinInSyncReplicasProp = "min.insync.replicas" @@ -332,6 +334,7 @@ object KafkaConfig { val LogFlushSchedulerIntervalMsDoc = "The frequency in ms that the log flusher checks whether any log needs to be flushed to disk" val LogFlushIntervalMsDoc = "The maximum time in ms that a message in any topic is kept in memory before flushed to disk" val LogFlushOffsetCheckpointIntervalMsDoc = "The frequency with which we update the persistent record of the last flush which acts as the log recovery point" + val LogPreAllocateEnableDoc = "Should pre allocate file when create new segment? If you are using Kafka on Windows, you probably need to set it to true." val NumRecoveryThreadsPerDataDirDoc = "The number of threads per data directory to be used for log recovery at startup and flushing at shutdown" val AutoCreateTopicsEnableDoc = "Enable auto creation of topic on the server" val MinInSyncReplicasDoc = "define the minimum number of replicas in ISR needed to satisfy a produce request with required.acks=-1 (or all)" @@ -466,6 +469,7 @@ object KafkaConfig { .define(LogFlushSchedulerIntervalMsProp, LONG, Defaults.LogFlushSchedulerIntervalMs, HIGH, LogFlushSchedulerIntervalMsDoc) .define(LogFlushIntervalMsProp, LONG, HIGH, LogFlushIntervalMsDoc, false) .define(LogFlushOffsetCheckpointIntervalMsProp, INT, Defaults.LogFlushOffsetCheckpointIntervalMs, atLeast(0), HIGH, LogFlushOffsetCheckpointIntervalMsDoc) + .define(LogPreAllocateProp, BOOLEAN, Defaults.LogPreAllocateEnable, MEDIUM, LogPreAllocateEnableDoc) .define(NumRecoveryThreadsPerDataDirProp, INT, Defaults.NumRecoveryThreadsPerDataDir, atLeast(1), HIGH, NumRecoveryThreadsPerDataDirDoc) .define(AutoCreateTopicsEnableProp, BOOLEAN, Defaults.AutoCreateTopicsEnable, HIGH, AutoCreateTopicsEnableDoc) .define(MinInSyncReplicasProp, INT, Defaults.MinInSyncReplicas, atLeast(1), HIGH, MinInSyncReplicasDoc) @@ -609,6 +613,7 @@ case class KafkaConfig (props: java.util.Map[_, _]) extends AbstractConfig(Kafka val logRollTimeJitterMillis: java.lang.Long = Option(getLong(KafkaConfig.LogRollTimeJitterMillisProp)).getOrElse(60 * 60 * 1000L * getInt(KafkaConfig.LogRollTimeJitterHoursProp)) val logFlushIntervalMs: java.lang.Long = Option(getLong(KafkaConfig.LogFlushIntervalMsProp)).getOrElse(getLong(KafkaConfig.LogFlushSchedulerIntervalMsProp)) val minInSyncReplicas = getInt(KafkaConfig.MinInSyncReplicasProp) + val logPreAllocateEnable: Boolean = getBoolean(KafkaConfig.LogPreAllocateProp) /** ********* Replication configuration ***********/ val controllerSocketTimeoutMs: Int = getInt(KafkaConfig.ControllerSocketTimeoutMsProp) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 9de2a6f..52dc728 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -443,6 +443,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg case KafkaConfig.MinInSyncReplicasProp => logProps.put(LogConfig.MinInSyncReplicasProp, entry.getValue) case KafkaConfig.CompressionTypeProp => logProps.put(LogConfig.CompressionTypeProp, entry.getValue) case KafkaConfig.UncleanLeaderElectionEnableProp => logProps.put(LogConfig.UncleanLeaderElectionEnableProp, entry.getValue) + case KafkaConfig.LogPreAllocateProp => logProps.put(LogConfig.PreAllocateEnableProp, entry.getValue) case _ => // we just leave those out } } diff --git a/core/src/main/scala/kafka/utils/CoreUtils.scala b/core/src/main/scala/kafka/utils/CoreUtils.scala index f5d704c..168a18d 100755 --- a/core/src/main/scala/kafka/utils/CoreUtils.scala +++ b/core/src/main/scala/kafka/utils/CoreUtils.scala @@ -70,16 +70,6 @@ object CoreUtils extends Logging { Utils.daemonThread(name, runnable(fun)) /** - * Open a channel for the given file - */ - def openChannel(file: File, mutable: Boolean): FileChannel = { - if(mutable) - new RandomAccessFile(file, "rw").getChannel() - else - new FileInputStream(file).getChannel() - } - - /** * Do the given action and log any exceptions thrown without rethrowing them * @param log The log method to use for logging. E.g. logger.warn * @param action The action to execute diff --git a/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala b/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala index cec1cae..02cf668 100644 --- a/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala +++ b/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala @@ -17,6 +17,7 @@ package kafka.log +import java.io._ import java.nio._ import java.util.concurrent.atomic._ import junit.framework.Assert._ @@ -146,5 +147,57 @@ class FileMessageSetTest extends BaseMessageSetTestCases { assertEquals(List(message), messageSet.toList) assertEquals(MessageSet.entrySize(message.message), messageSet.sizeInBytes) } - + + /** + * Test the new FileMessageSet with pre allocate as true + */ + @Test + def testPreallocateTrue() { + val temp = tempFile() + val set = new FileMessageSet(temp, false, 512 *1024 *1024, true) + val position = set.channel.position + val size = set.sizeInBytes() + assertEquals(0, position) + assertEquals(0, size) + assertEquals(512 *1024 *1024, temp.length) + } + + /** + * Test the new FileMessageSet with pre allocate as false + */ + @Test + def testPreallocateFalse() { + val temp = tempFile() + val set = new FileMessageSet(temp, false, 512 *1024 *1024, false) + val position = set.channel.position + val size = set.sizeInBytes() + assertEquals(0, position) + assertEquals(0, size) + assertEquals(0, temp.length) + } + + /** + * Test the new FileMessageSet with pre allocate as true and file has been clearly shut down, the file will be truncate to end of valid data. + */ + @Test + def testPreallocateClearShutdown() { + val temp = tempFile() + val set = new FileMessageSet(temp, false, 512 *1024 *1024, true) + set.append(new ByteBufferMessageSet(NoCompressionCodec, messages: _*)) + val oldposition = set.channel.position + val oldsize = set.sizeInBytes() + assertEquals(messageSet.sizeInBytes, oldposition) + assertEquals(messageSet.sizeInBytes, oldsize) + set.close() + + val tempReopen = new File(temp.getAbsolutePath()) + val setReopen = new FileMessageSet(tempReopen, true, 512 *1024 *1024, true) + val position = setReopen.channel.position + val size = setReopen.sizeInBytes() + + assertEquals(oldposition, position) + assertEquals(oldposition, size) + assertEquals(oldposition, tempReopen.length) + } + } diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala index c31f884..19dcb47 100644 --- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala @@ -45,6 +45,7 @@ class LogConfigTest extends JUnit3Suite { case LogConfig.MinInSyncReplicasProp => expected.setProperty(name, (nextInt(Int.MaxValue - 1) + 1).toString) case LogConfig.RetentionBytesProp => expected.setProperty(name, nextInt().toString) case LogConfig.RetentionMsProp => expected.setProperty(name, nextLong().toString) + case LogConfig.PreAllocateEnableProp => expected.setProperty(name, randFrom("true", "false")) case positiveIntProperty => expected.setProperty(name, nextInt(Int.MaxValue).toString) } }) diff --git a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala index 03fb351..abcd1f0 100644 --- a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala @@ -226,5 +226,57 @@ class LogSegmentTest extends JUnit3Suite { seg.delete() } } - + + /* create a segment with pre allocate */ + def createSegment(offset: Long, fileAlreadyExists: Boolean = false, initFileSize: Int = 0, preallocate: Boolean = false): LogSegment = { + val tempDir = TestUtils.tempDir() + val seg = new LogSegment(tempDir, offset, 10, 1000, 0, SystemTime, fileAlreadyExists = fileAlreadyExists, initFileSize = initFileSize, preallocate = preallocate) + segments += seg + seg + } + + /* create a segment with pre allocate, put message to it and verify */ + @Test + def testCreateWithInitFileSizeAppendMessage() { + val seg = createSegment(40, false, 512*1024*1024, true) + val ms = messages(50, "hello", "there") + seg.append(50, ms) + val ms2 = messages(60, "alpha", "beta") + seg.append(60, ms2) + val read = seg.read(startOffset = 55, maxSize = 200, maxOffset = None) + assertEquals(ms2.toList, read.messageSet.toList) + } + + /* create a segment with pre allocate and clearly shut down*/ + @Test + def testCreateWithInitFileSizeClearShutdown() { + val tempDir = TestUtils.tempDir() + val seg = new LogSegment(tempDir, 40, 10, 1000, 0, SystemTime, false, 512*1024*1024, true) + + val ms = messages(50, "hello", "there") + seg.append(50, ms) + val ms2 = messages(60, "alpha", "beta") + seg.append(60, ms2) + val read = seg.read(startOffset = 55, maxSize = 200, maxOffset = None) + assertEquals(ms2.toList, read.messageSet.toList) + val oldSize = seg.log.sizeInBytes() + val oldPosition = seg.log.channel.position + val oldFileSize = seg.log.file.length + assertEquals(512*1024*1024, oldFileSize) + seg.close() + //After close, file should be trimed + assertEquals(oldSize, seg.log.file.length) + + val segReopen = new LogSegment(tempDir, 40, 10, 1000, 0, SystemTime, true, 512*1024*1024, true) + segments += segReopen + + val readAgain = segReopen.read(startOffset = 55, maxSize = 200, maxOffset = None) + assertEquals(ms2.toList, readAgain.messageSet.toList) + val size = segReopen.log.sizeInBytes() + val position = segReopen.log.channel.position + val fileSize = segReopen.log.file.length + assertEquals(oldPosition, position) + assertEquals(oldSize, size) + assertEquals(size, fileSize) + } } \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala index 8268852..98a5b04 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala @@ -192,6 +192,7 @@ class KafkaConfigConfigDefTest extends JUnit3Suite { case KafkaConfig.MinInSyncReplicasProp => expected.setProperty(name, atLeastOneIntProp) case KafkaConfig.AutoLeaderRebalanceEnableProp => expected.setProperty(name, randFrom("true", "false")) case KafkaConfig.UncleanLeaderElectionEnableProp => expected.setProperty(name, randFrom("true", "false")) + case KafkaConfig.LogPreAllocateProp => expected.setProperty(name, randFrom("true", "false")) case KafkaConfig.InterBrokerSecurityProtocolProp => expected.setProperty(name, SecurityProtocol.PLAINTEXT.toString) case KafkaConfig.InterBrokerProtocolVersionProp => expected.setProperty(name, ApiVersion.latestVersion.toString) -- 1.7.12.4 From 16ecf9806b286d9510103a4426bf0901d7dc8778 Mon Sep 17 00:00:00 2001 From: Manikumar Reddy Date: Fri, 19 Jun 2015 09:34:22 -0700 Subject: [PATCH 020/120] kafka-2012; Broker should automatically handle corrupt index files; patched by Manikumar Reddy; reviewed by Jun Rao --- core/src/main/scala/kafka/log/Log.scala | 18 +++++++++---- core/src/test/scala/unit/kafka/log/LogTest.scala | 33 ++++++++++++++++++++++++ 2 files changed, 46 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 6b9274d..e5e8007 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -169,7 +169,7 @@ class Log(val dir: File, } else if(filename.endsWith(LogFileSuffix)) { // if its a log file, load the corresponding log segment val start = filename.substring(0, filename.length - LogFileSuffix.length).toLong - val hasIndex = Log.indexFilename(dir, start).exists + val indexFile = Log.indexFilename(dir, start) val segment = new LogSegment(dir = dir, startOffset = start, indexIntervalBytes = config.indexInterval, @@ -177,7 +177,18 @@ class Log(val dir: File, rollJitterMs = config.randomSegmentJitter, time = time, fileAlreadyExists = true) - if(!hasIndex) { + + if(indexFile.exists()) { + try { + segment.index.sanityCheck() + } catch { + case e: java.lang.IllegalArgumentException => + warn("Found an corrupted index file, %s, deleting and rebuilding index...".format(indexFile.getAbsolutePath)) + indexFile.delete() + segment.recover(config.maxMessageSize) + } + } + else { error("Could not find index file corresponding to log file %s, rebuilding index...".format(segment.log.file.getAbsolutePath)) segment.recover(config.maxMessageSize) } @@ -223,9 +234,6 @@ class Log(val dir: File, activeSegment.index.resize(config.maxIndexSize) } - // sanity check the index file of every segment to ensure we don't proceed with a corrupt segment - for (s <- logSegments) - s.index.sanityCheck() } private def updateLogEndOffset(messageOffset: Long) { diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index a8e57c2..9e26190 100755 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -489,6 +489,39 @@ class LogTest extends JUnitSuite { } /** + * Test that if we have corrupted an index segment it is rebuilt when the log is re-opened + */ + @Test + def testCorruptIndexRebuild() { + // publish the messages and close the log + val numMessages = 200 + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 200: java.lang.Integer) + logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer) + + val config = LogConfig(logProps) + var log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time) + for(i <- 0 until numMessages) + log.append(TestUtils.singleMessageSet(TestUtils.randomBytes(10))) + val indexFiles = log.logSegments.map(_.index.file) + log.close() + + // corrupt all the index files + for( file <- indexFiles) { + val bw = new BufferedWriter(new FileWriter(file)) + bw.write(" ") + bw.close() + } + + // reopen the log + log = new Log(logDir, config, recoveryPoint = 200L, time.scheduler, time) + assertEquals("Should have %d messages when log is reopened".format(numMessages), numMessages, log.logEndOffset) + for(i <- 0 until numMessages) + assertEquals(i, log.read(i, 100, None).messageSet.head.offset) + log.close() + } + + /** * Test the Log truncate operations */ @Test -- 1.7.12.4 From cf28f893963c363bca43747e2f37fad3bb67d033 Mon Sep 17 00:00:00 2001 From: Chris Black Date: Mon, 22 Jun 2015 08:59:05 -0700 Subject: [PATCH 021/120] kafka-2290; OffsetIndex should open RandomAccessFile consistently; patched by Chris Black; reviewed by Jun Rao --- core/src/main/scala/kafka/log/OffsetIndex.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/log/OffsetIndex.scala b/core/src/main/scala/kafka/log/OffsetIndex.scala index a1082ae..332d5e2 100755 --- a/core/src/main/scala/kafka/log/OffsetIndex.scala +++ b/core/src/main/scala/kafka/log/OffsetIndex.scala @@ -274,7 +274,7 @@ class OffsetIndex(@volatile var file: File, val baseOffset: Long, val maxIndexSi */ def resize(newSize: Int) { inLock(lock) { - val raf = new RandomAccessFile(file, "rws") + val raf = new RandomAccessFile(file, "rw") val roundedNewSize = roundToExactMultiple(newSize, 8) val position = this.mmap.position -- 1.7.12.4 From dc54055d05742a4a7729a1fe1073c18e3d95cbb2 Mon Sep 17 00:00:00 2001 From: Ivan Simoneko Date: Mon, 22 Jun 2015 09:19:45 -0700 Subject: [PATCH 022/120] kafka-2235; LogCleaner offset map overflow; patched by Ivan Simoneko; reviewed by Jun Rao --- core/src/main/scala/kafka/log/LogCleaner.scala | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index d07a391..b36ea0d 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -559,11 +559,17 @@ private[log] class Cleaner(val id: Int, // but we may be able to fit more (if there is lots of duplication in the dirty section of the log) var offset = dirty.head.baseOffset require(offset == start, "Last clean offset is %d but segment base offset is %d for log %s.".format(start, offset, log.name)) - val minStopOffset = (start + map.slots * this.dupBufferLoadFactor).toLong - for (segment <- dirty) { + val maxDesiredMapSize = (map.slots * this.dupBufferLoadFactor).toInt + var full = false + for (segment <- dirty if !full) { checkDone(log.topicAndPartition) - if(segment.baseOffset <= minStopOffset || map.utilization < this.dupBufferLoadFactor) + val segmentSize = segment.nextOffset() - segment.baseOffset + + require(segmentSize <= maxDesiredMapSize, "%d messages in segment %s/%s but offset map can fit only %d. You can increase log.cleaner.dedupe.buffer.size or decrease log.cleaner.threads".format(segmentSize, log.name, segment.log.file.getName, maxDesiredMapSize)) + if (map.size + segmentSize <= maxDesiredMapSize) offset = buildOffsetMapForSegment(log.topicAndPartition, segment, map) + else + full = true } info("Offset map for log %s complete.".format(log.name)) offset -- 1.7.12.4 From 1eac3ceaf94b3e7583c7b6de2cfe13539ab06dd6 Mon Sep 17 00:00:00 2001 From: Onur Karaman Date: Mon, 22 Jun 2015 10:14:14 -0700 Subject: [PATCH 023/120] KAFKA-2245; Add response tests for consumer coordinator; reviewed by Joel Koshy --- .../kafka/coordinator/ConsumerCoordinator.scala | 4 +- .../kafka/coordinator/CoordinatorMetadata.scala | 4 +- .../ConsumerCoordinatorResponseTest.scala | 293 +++++++++++++++++++++ 3 files changed, 297 insertions(+), 4 deletions(-) create mode 100644 core/src/test/scala/unit/kafka/coordinator/ConsumerCoordinatorResponseTest.scala diff --git a/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala b/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala index 51e89c8..a385adb 100644 --- a/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala @@ -46,8 +46,8 @@ class ConsumerCoordinator(val config: KafkaConfig, private var coordinatorMetadata: CoordinatorMetadata = null /** - * NOTE: If a group lock and coordinatorLock are simultaneously needed, - * be sure to acquire the group lock before coordinatorLock to prevent deadlock + * NOTE: If a group lock and metadataLock are simultaneously needed, + * be sure to acquire the group lock before metadataLock to prevent deadlock */ /** diff --git a/core/src/main/scala/kafka/coordinator/CoordinatorMetadata.scala b/core/src/main/scala/kafka/coordinator/CoordinatorMetadata.scala index c39e6de..0cd5605 100644 --- a/core/src/main/scala/kafka/coordinator/CoordinatorMetadata.scala +++ b/core/src/main/scala/kafka/coordinator/CoordinatorMetadata.scala @@ -37,8 +37,8 @@ private[coordinator] class CoordinatorMetadata(config: KafkaConfig, maybePrepareRebalance: ConsumerGroupMetadata => Unit) { /** - * NOTE: If a group lock and coordinatorLock are simultaneously needed, - * be sure to acquire the group lock before coordinatorLock to prevent deadlock + * NOTE: If a group lock and metadataLock are simultaneously needed, + * be sure to acquire the group lock before metadataLock to prevent deadlock */ private val metadataLock = new ReentrantReadWriteLock() diff --git a/core/src/test/scala/unit/kafka/coordinator/ConsumerCoordinatorResponseTest.scala b/core/src/test/scala/unit/kafka/coordinator/ConsumerCoordinatorResponseTest.scala new file mode 100644 index 0000000..a44fbd6 --- /dev/null +++ b/core/src/test/scala/unit/kafka/coordinator/ConsumerCoordinatorResponseTest.scala @@ -0,0 +1,293 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.coordinator + + +import java.util.concurrent.TimeUnit + +import junit.framework.Assert._ +import kafka.common.TopicAndPartition +import kafka.server.{KafkaConfig, OffsetManager} +import kafka.utils.TestUtils +import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.requests.JoinGroupRequest +import org.easymock.EasyMock +import org.junit.{After, Before, Test} +import org.scalatest.junit.JUnitSuite + +import scala.concurrent.duration.Duration +import scala.concurrent.{Await, Future, Promise} + +/** + * Test ConsumerCoordinator responses + */ +class ConsumerCoordinatorResponseTest extends JUnitSuite { + type JoinGroupCallbackParams = (Set[TopicAndPartition], String, Int, Short) + type JoinGroupCallback = (Set[TopicAndPartition], String, Int, Short) => Unit + type HeartbeatCallbackParams = Short + type HeartbeatCallback = Short => Unit + + val ConsumerMinSessionTimeout = 10 + val ConsumerMaxSessionTimeout = 30 + val DefaultSessionTimeout = 20 + var offsetManager: OffsetManager = null + var consumerCoordinator: ConsumerCoordinator = null + + @Before + def setUp() { + val props = TestUtils.createBrokerConfig(nodeId = 0, zkConnect = "") + props.setProperty(KafkaConfig.ConsumerMinSessionTimeoutMsProp, ConsumerMinSessionTimeout.toString) + props.setProperty(KafkaConfig.ConsumerMaxSessionTimeoutMsProp, ConsumerMaxSessionTimeout.toString) + offsetManager = EasyMock.createStrictMock(classOf[OffsetManager]) + consumerCoordinator = new ConsumerCoordinator(KafkaConfig.fromProps(props), null, offsetManager) + consumerCoordinator.startup() + } + + @After + def tearDown() { + consumerCoordinator.shutdown() + } + + @Test + def testJoinGroupWrongCoordinator() { + val groupId = "groupId" + val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID + val partitionAssignmentStrategy = "range" + + val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = false) + val joinGroupErrorCode = joinGroupResult._4 + assertEquals(Errors.NOT_COORDINATOR_FOR_CONSUMER.code, joinGroupErrorCode) + } + + @Test + def testJoinGroupUnknownPartitionAssignmentStrategy() { + val groupId = "groupId" + val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID + val partitionAssignmentStrategy = "foo" + + val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true) + val joinGroupErrorCode = joinGroupResult._4 + assertEquals(Errors.UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY.code, joinGroupErrorCode) + } + + @Test + def testJoinGroupSessionTimeoutTooSmall() { + val groupId = "groupId" + val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID + val partitionAssignmentStrategy = "range" + + val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, ConsumerMinSessionTimeout - 1, isCoordinatorForGroup = true) + val joinGroupErrorCode = joinGroupResult._4 + assertEquals(Errors.INVALID_SESSION_TIMEOUT.code, joinGroupErrorCode) + } + + @Test + def testJoinGroupSessionTimeoutTooLarge() { + val groupId = "groupId" + val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID + val partitionAssignmentStrategy = "range" + + val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, ConsumerMaxSessionTimeout + 1, isCoordinatorForGroup = true) + val joinGroupErrorCode = joinGroupResult._4 + assertEquals(Errors.INVALID_SESSION_TIMEOUT.code, joinGroupErrorCode) + } + + @Test + def testJoinGroupUnknownConsumerNewGroup() { + val groupId = "groupId" + val consumerId = "consumerId" + val partitionAssignmentStrategy = "range" + + val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true) + val joinGroupErrorCode = joinGroupResult._4 + assertEquals(Errors.UNKNOWN_CONSUMER_ID.code, joinGroupErrorCode) + } + + @Test + def testValidJoinGroup() { + val groupId = "groupId" + val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID + val partitionAssignmentStrategy = "range" + + val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true) + val joinGroupErrorCode = joinGroupResult._4 + assertEquals(Errors.NONE.code, joinGroupErrorCode) + } + + @Test + def testJoinGroupInconsistentPartitionAssignmentStrategy() { + val groupId = "groupId" + val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID + val otherConsumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID + val partitionAssignmentStrategy = "range" + val otherPartitionAssignmentStrategy = "roundrobin" + + val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true) + val joinGroupErrorCode = joinGroupResult._4 + assertEquals(Errors.NONE.code, joinGroupErrorCode) + + EasyMock.reset(offsetManager) + val otherJoinGroupResult = joinGroup(groupId, otherConsumerId, otherPartitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true) + val otherJoinGroupErrorCode = otherJoinGroupResult._4 + assertEquals(Errors.INCONSISTENT_PARTITION_ASSIGNMENT_STRATEGY.code, otherJoinGroupErrorCode) + } + + @Test + def testJoinGroupUnknownConsumerExistingGroup() { + val groupId = "groupId" + val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID + val otherConsumerId = "consumerId" + val partitionAssignmentStrategy = "range" + + val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true) + val joinGroupErrorCode = joinGroupResult._4 + assertEquals(Errors.NONE.code, joinGroupErrorCode) + + EasyMock.reset(offsetManager) + val otherJoinGroupResult = joinGroup(groupId, otherConsumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true) + val otherJoinGroupErrorCode = otherJoinGroupResult._4 + assertEquals(Errors.UNKNOWN_CONSUMER_ID.code, otherJoinGroupErrorCode) + } + + @Test + def testHeartbeatWrongCoordinator() { + val groupId = "groupId" + val consumerId = "consumerId" + + val heartbeatResult = heartbeat(groupId, consumerId, -1, isCoordinatorForGroup = false) + assertEquals(Errors.NOT_COORDINATOR_FOR_CONSUMER.code, heartbeatResult) + } + + @Test + def testHeartbeatUnknownGroup() { + val groupId = "groupId" + val consumerId = "consumerId" + + val heartbeatResult = heartbeat(groupId, consumerId, -1, isCoordinatorForGroup = true) + assertEquals(Errors.UNKNOWN_CONSUMER_ID.code, heartbeatResult) + } + + @Test + def testHeartbeatUnknownConsumerExistingGroup() { + val groupId = "groupId" + val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID + val otherConsumerId = "consumerId" + val partitionAssignmentStrategy = "range" + + val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true) + val joinGroupErrorCode = joinGroupResult._4 + assertEquals(Errors.NONE.code, joinGroupErrorCode) + + EasyMock.reset(offsetManager) + val heartbeatResult = heartbeat(groupId, otherConsumerId, 1, isCoordinatorForGroup = true) + assertEquals(Errors.UNKNOWN_CONSUMER_ID.code, heartbeatResult) + } + + @Test + def testHeartbeatIllegalGeneration() { + val groupId = "groupId" + val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID + val partitionAssignmentStrategy = "range" + + val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true) + val assignedConsumerId = joinGroupResult._2 + val joinGroupErrorCode = joinGroupResult._4 + assertEquals(Errors.NONE.code, joinGroupErrorCode) + + EasyMock.reset(offsetManager) + val heartbeatResult = heartbeat(groupId, assignedConsumerId, 2, isCoordinatorForGroup = true) + assertEquals(Errors.ILLEGAL_GENERATION.code, heartbeatResult) + } + + @Test + def testValidHeartbeat() { + val groupId = "groupId" + val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID + val partitionAssignmentStrategy = "range" + + val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true) + val assignedConsumerId = joinGroupResult._2 + val joinGroupErrorCode = joinGroupResult._4 + assertEquals(Errors.NONE.code, joinGroupErrorCode) + + EasyMock.reset(offsetManager) + val heartbeatResult = heartbeat(groupId, assignedConsumerId, 1, isCoordinatorForGroup = true) + assertEquals(Errors.NONE.code, heartbeatResult) + } + + @Test + def testGenerationIdIncrementsOnRebalance() { + val groupId = "groupId" + val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID + val otherConsumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID + val partitionAssignmentStrategy = "range" + + val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true) + val initialGenerationId = joinGroupResult._3 + val joinGroupErrorCode = joinGroupResult._4 + assertEquals(1, initialGenerationId) + assertEquals(Errors.NONE.code, joinGroupErrorCode) + + EasyMock.reset(offsetManager) + val otherJoinGroupResult = joinGroup(groupId, otherConsumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true) + val nextGenerationId = otherJoinGroupResult._3 + val otherJoinGroupErrorCode = otherJoinGroupResult._4 + assertEquals(2, nextGenerationId) + assertEquals(Errors.NONE.code, otherJoinGroupErrorCode) + } + + private def setupJoinGroupCallback: (Future[JoinGroupCallbackParams], JoinGroupCallback) = { + val responsePromise = Promise[JoinGroupCallbackParams] + val responseFuture = responsePromise.future + val responseCallback: JoinGroupCallback = (partitions, consumerId, generationId, errorCode) => + responsePromise.success((partitions, consumerId, generationId, errorCode)) + (responseFuture, responseCallback) + } + + private def setupHeartbeatCallback: (Future[HeartbeatCallbackParams], HeartbeatCallback) = { + val responsePromise = Promise[HeartbeatCallbackParams] + val responseFuture = responsePromise.future + val responseCallback: HeartbeatCallback = errorCode => responsePromise.success(errorCode) + (responseFuture, responseCallback) + } + + private def joinGroup(groupId: String, + consumerId: String, + partitionAssignmentStrategy: String, + sessionTimeout: Int, + isCoordinatorForGroup: Boolean): JoinGroupCallbackParams = { + val (responseFuture, responseCallback) = setupJoinGroupCallback + EasyMock.expect(offsetManager.partitionFor(groupId)).andReturn(1) + EasyMock.expect(offsetManager.leaderIsLocal(1)).andReturn(isCoordinatorForGroup) + EasyMock.replay(offsetManager) + consumerCoordinator.handleJoinGroup(groupId, consumerId, Set.empty, sessionTimeout, partitionAssignmentStrategy, responseCallback) + Await.result(responseFuture, Duration(40, TimeUnit.MILLISECONDS)) + } + + private def heartbeat(groupId: String, + consumerId: String, + generationId: Int, + isCoordinatorForGroup: Boolean): HeartbeatCallbackParams = { + val (responseFuture, responseCallback) = setupHeartbeatCallback + EasyMock.expect(offsetManager.partitionFor(groupId)).andReturn(1) + EasyMock.expect(offsetManager.leaderIsLocal(1)).andReturn(isCoordinatorForGroup) + EasyMock.replay(offsetManager) + consumerCoordinator.handleHeartbeat(groupId, consumerId, generationId, responseCallback) + Await.result(responseFuture, Duration(40, TimeUnit.MILLISECONDS)) + } +} -- 1.7.12.4 From 2270a7537f7734ec02b2391fae280624520a664d Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Mon, 22 Jun 2015 15:28:35 -0700 Subject: [PATCH 024/120] KAFKA-2293; Fix incorrect format specification in Partition.scala; reviewed by Joel Koshy --- core/src/main/scala/kafka/cluster/Partition.scala | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 6cb6477..0990938 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -249,13 +249,12 @@ class Partition(val topic: String, TopicAndPartition(topic, partitionId))) case None => throw new NotAssignedReplicaException(("Leader %d failed to record follower %d's position %d since the replica" + - " is not recognized to be one of the assigned replicas %s for partition [%s,%d]") + " is not recognized to be one of the assigned replicas %s for partition %s.") .format(localBrokerId, replicaId, - logReadResult.info.fetchOffsetMetadata, + logReadResult.info.fetchOffsetMetadata.messageOffset, assignedReplicas().map(_.brokerId).mkString(","), - topic, - partitionId)) + TopicAndPartition(topic, partitionId))) } } -- 1.7.12.4 From b6d326b0893e60b350608260fd1bd2542337cb5a Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Tue, 23 Jun 2015 00:07:19 -0400 Subject: [PATCH 025/120] kafka-2168; New consumer poll() can block other calls like position(), commit(), and close() indefinitely; patched by Jason Gustafson; reviewed by Jay Kreps, Ewen Cheslack-Postava, Guozhang Wang and Jun Rao --- .../apache/kafka/clients/consumer/Consumer.java | 5 + .../kafka/clients/consumer/ConsumerRecords.java | 7 + .../clients/consumer/ConsumerWakeupException.java | 20 + .../kafka/clients/consumer/KafkaConsumer.java | 715 ++++++++++++++++----- .../kafka/clients/consumer/MockConsumer.java | 9 +- .../clients/consumer/OffsetResetStrategy.java | 17 + .../clients/consumer/internals/Coordinator.java | 447 ++++++------- .../kafka/clients/consumer/internals/Fetcher.java | 159 ++--- .../clients/consumer/internals/Heartbeat.java | 10 + .../clients/consumer/internals/RequestFuture.java | 209 ++++++ .../consumer/internals/SubscriptionState.java | 41 +- .../java/org/apache/kafka/common/utils/Utils.java | 15 + .../kafka/clients/consumer/MockConsumerTest.java | 2 +- .../consumer/internals/CoordinatorTest.java | 148 ++++- .../clients/consumer/internals/FetcherTest.java | 32 +- .../clients/consumer/internals/HeartbeatTest.java | 9 + .../consumer/internals/SubscriptionStateTest.java | 19 +- .../org/apache/kafka/common/utils/UtilsTest.java | 8 + 18 files changed, 1330 insertions(+), 542 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerWakeupException.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/OffsetResetStrategy.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFuture.java diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java index 8f587bc..fd98740 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java @@ -108,4 +108,9 @@ public interface Consumer extends Closeable { */ public void close(); + /** + * @see KafkaConsumer#wakeup() + */ + public void wakeup(); + } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java index 1ca75f8..eb75d2e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java @@ -27,6 +27,8 @@ import java.util.Map; * {@link Consumer#poll(long)} operation. */ public class ConsumerRecords implements Iterable> { + public static final ConsumerRecords EMPTY = + new ConsumerRecords(Collections.EMPTY_MAP); private final Map>> records; @@ -103,4 +105,9 @@ public class ConsumerRecords implements Iterable> { } } + @SuppressWarnings("unchecked") + public static ConsumerRecords empty() { + return (ConsumerRecords) EMPTY; + } + } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerWakeupException.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerWakeupException.java new file mode 100644 index 0000000..35f1ec9 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerWakeupException.java @@ -0,0 +1,20 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package org.apache.kafka.clients.consumer; + +import org.apache.kafka.common.KafkaException; + +public class ConsumerWakeupException extends KafkaException { + private static final long serialVersionUID = 1L; + +} 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 951c34c..9be8fbc 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 @@ -12,44 +12,48 @@ */ package org.apache.kafka.clients.consumer; -import java.net.InetSocketAddress; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.Set; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; - import org.apache.kafka.clients.ClientUtils; import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.NetworkClient; import org.apache.kafka.clients.consumer.internals.Coordinator; import org.apache.kafka.clients.consumer.internals.Fetcher; +import org.apache.kafka.clients.consumer.internals.RequestFuture; import org.apache.kafka.clients.consumer.internals.SubscriptionState; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Metric; +import org.apache.kafka.common.MetricName; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.metrics.JmxReporter; import org.apache.kafka.common.metrics.MetricConfig; -import org.apache.kafka.common.MetricName; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.MetricsReporter; -import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.network.Selector; +import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.utils.SystemTime; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.net.InetSocketAddress; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.ConcurrentModificationException; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; + +import static org.apache.kafka.common.utils.Utils.min; + /** * A Kafka client that consumes records from a Kafka cluster. *

@@ -298,10 +302,54 @@ import org.slf4j.LoggerFactory; * *

Multithreaded Processing

* - * The Kafka consumer is threadsafe but coarsely synchronized. All network I/O happens in the thread of the application - * making the call. We have intentionally avoided implementing a particular threading model for processing. + * The Kafka consumer is NOT thread-safe. All network I/O happens in the thread of the application + * making the call. It is the responsibility of the user to ensure that multi-threaded access + * is properly synchronized. Un-synchronized access will result in {@link ConcurrentModificationException}. + * + *

+ * The only exception to this rule is {@link #wakeup()}, which can safely be used from an external thread to + * interrupt an active operation. In this case, a {@link ConsumerWakeupException} will be thrown from the thread + * blocking on the operation. This can be used to shutdown the consumer from another thread. The following + * snippet shows the typical pattern: + * + *

+ * public class KafkaConsumerRunner implements Runnable {
+ *     private final AtomicBoolean closed = new AtomicBoolean(false);
+ *     private final KafkaConsumer consumer;
+ *
+ *     public void run() {
+ *         try {
+ *             consumer.subscribe("topic");
+ *             while (!closed.get()) {
+ *                 ConsumerRecords records = consumer.poll(10000);
+ *                 // Handle new records
+ *             }
+ *         } catch (ConsumerWakeupException e) {
+ *             // Ignore exception if closing
+ *             if (!closed.get()) throw e;
+ *         } finally {
+ *             consumer.close();
+ *         }
+ *     }
+ *
+ *     public void shutdown() {
+ *         closed.set(true);
+ *         consumer.wakeup();
+ *     }
+ * }
+ * 
+ * + * Then in a separate thread, the consumer can be shutdown by setting the closed flag and waking up the consumer. + * + *
+ *     closed.set(true);
+ *     consumer.wakeup();
+ * 
+ * *

- * This leaves several options for implementing multi-threaded processing of records. + * We have intentionally avoided implementing a particular threading model for processing. This leaves several + * options for implementing multi-threaded processing of records. + * * *

1. One Consumer Per Thread

* @@ -363,6 +411,17 @@ public class KafkaConsumer implements Consumer { private final ConsumerRebalanceCallback rebalanceCallback; private long lastCommitAttemptMs; private boolean closed = false; + private final AtomicBoolean wakeup = new AtomicBoolean(false); + + // currentThread holds the threadId of the current thread accessing KafkaConsumer + // and is used to prevent multi-threaded access + private final AtomicReference currentThread = new AtomicReference(); + // refcount is used to allow reentrant access by the thread who has acquired currentThread + private int refcount = 0; // reference count for reentrant access + + // TODO: This timeout controls how long we should wait before retrying a request. We should be able + // to leverage the work of KAFKA-2120 to get this value from configuration. + private long requestTimeoutMs = 5000L; /** * A consumer is instantiated by providing a set of key-value pairs as configuration. Valid configuration strings @@ -480,13 +539,12 @@ public class KafkaConsumer implements Consumer { config.getLong(ConsumerConfig.RECONNECT_BACKOFF_MS_CONFIG), config.getInt(ConsumerConfig.SEND_BUFFER_CONFIG), config.getInt(ConsumerConfig.RECEIVE_BUFFER_CONFIG)); - this.subscriptions = new SubscriptionState(); + OffsetResetStrategy offsetResetStrategy = OffsetResetStrategy.valueOf(config.getString(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG).toUpperCase()); + this.subscriptions = new SubscriptionState(offsetResetStrategy); this.coordinator = new Coordinator(this.client, config.getString(ConsumerConfig.GROUP_ID_CONFIG), - this.retryBackoffMs, config.getInt(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG), config.getString(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG), - this.metadata, this.subscriptions, metrics, metricGrpPrefix, @@ -508,12 +566,10 @@ public class KafkaConsumer implements Consumer { this.valueDeserializer = valueDeserializer; } this.fetcher = new Fetcher(this.client, - this.retryBackoffMs, config.getInt(ConsumerConfig.FETCH_MIN_BYTES_CONFIG), config.getInt(ConsumerConfig.FETCH_MAX_WAIT_MS_CONFIG), config.getInt(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG), config.getBoolean(ConsumerConfig.CHECK_CRCS_CONFIG), - config.getString(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG).toUpperCase(), this.keyDeserializer, this.valueDeserializer, this.metadata, @@ -542,8 +598,13 @@ public class KafkaConsumer implements Consumer { * then this will give the set of topics currently assigned to the consumer (which may be none if the assignment * hasn't happened yet, or the partitions are in the process of getting reassigned). */ - public synchronized Set subscriptions() { - return Collections.unmodifiableSet(this.subscriptions.assignedPartitions()); + public Set subscriptions() { + acquire(); + try { + return Collections.unmodifiableSet(this.subscriptions.assignedPartitions()); + } finally { + release(); + } } /** @@ -561,12 +622,16 @@ public class KafkaConsumer implements Consumer { * @param topics A variable list of topics that the consumer wants to subscribe to */ @Override - public synchronized void subscribe(String... topics) { - ensureNotClosed(); - log.debug("Subscribed to topic(s): {}", Utils.join(topics, ", ")); - for (String topic : topics) - this.subscriptions.subscribe(topic); - metadata.addTopics(topics); + public void subscribe(String... topics) { + acquire(); + try { + log.debug("Subscribed to topic(s): {}", Utils.join(topics, ", ")); + for (String topic : topics) + this.subscriptions.subscribe(topic); + metadata.addTopics(topics); + } finally { + release(); + } } /** @@ -574,16 +639,20 @@ public class KafkaConsumer implements Consumer { * functionality. As such, there will be no rebalance operation triggered when group membership or cluster and topic * metadata change. *

- * + * * @param partitions Partitions to incrementally subscribe to */ @Override - public synchronized void subscribe(TopicPartition... partitions) { - ensureNotClosed(); - log.debug("Subscribed to partitions(s): {}", Utils.join(partitions, ", ")); - for (TopicPartition tp : partitions) { - this.subscriptions.subscribe(tp); - metadata.addTopics(tp.topic()); + public void subscribe(TopicPartition... partitions) { + acquire(); + try { + log.debug("Subscribed to partitions(s): {}", Utils.join(partitions, ", ")); + for (TopicPartition tp : partitions) { + this.subscriptions.subscribe(tp); + metadata.addTopics(tp.topic()); + } + } finally { + release(); } } @@ -593,12 +662,16 @@ public class KafkaConsumer implements Consumer { * * @param topics Topics to unsubscribe from */ - public synchronized void unsubscribe(String... topics) { - ensureNotClosed(); - log.debug("Unsubscribed from topic(s): {}", Utils.join(topics, ", ")); - // throw an exception if the topic was never subscribed to - for (String topic : topics) - this.subscriptions.unsubscribe(topic); + public void unsubscribe(String... topics) { + acquire(); + try { + log.debug("Unsubscribed from topic(s): {}", Utils.join(topics, ", ")); + // throw an exception if the topic was never subscribed to + for (String topic : topics) + this.subscriptions.unsubscribe(topic); + } finally { + release(); + } } /** @@ -607,12 +680,16 @@ public class KafkaConsumer implements Consumer { * * @param partitions Partitions to unsubscribe from */ - public synchronized void unsubscribe(TopicPartition... partitions) { - ensureNotClosed(); - log.debug("Unsubscribed from partitions(s): {}", Utils.join(partitions, ", ")); - // throw an exception if the partition was never subscribed to - for (TopicPartition partition : partitions) - this.subscriptions.unsubscribe(partition); + public void unsubscribe(TopicPartition... partitions) { + acquire(); + try { + log.debug("Unsubscribed from partitions(s): {}", Utils.join(partitions, ", ")); + // throw an exception if the partition was never subscribed to + for (TopicPartition partition : partitions) + this.subscriptions.unsubscribe(partition); + } finally { + release(); + } } /** @@ -624,17 +701,65 @@ public class KafkaConsumer implements Consumer { * rebalance, to consume data from that offset sequentially on every poll. If not, it will use the last checkpointed * offset using {@link #commit(Map, CommitType) commit(offsets, sync)} for the subscribed list of partitions. * - * @param timeout The time, in milliseconds, spent waiting in poll if data is not available. If 0, waits - * indefinitely. Must not be negative + * @param timeout The time, in milliseconds, spent waiting in poll if data is not available. If 0, returns + * immediately with any records available now. Must not be negative. * @return map of topic to records since the last fetch for the subscribed list of topics and partitions * * @throws NoOffsetForPartitionException If there is no stored offset for a subscribed partition and no automatic * offset reset policy has been configured. */ @Override - public synchronized ConsumerRecords poll(long timeout) { - ensureNotClosed(); - long now = time.milliseconds(); + public ConsumerRecords poll(long timeout) { + acquire(); + try { + if (timeout < 0) + throw new IllegalArgumentException("Timeout must not be negative"); + + // Poll for new data until the timeout expires + long remaining = timeout; + while (remaining >= 0) { + long start = time.milliseconds(); + long pollTimeout = min(remaining, timeToNextCommit(start), coordinator.timeToNextHeartbeat(start)); + + Map>> records = pollOnce(pollTimeout, start); + long end = time.milliseconds(); + + if (!records.isEmpty()) { + // If data is available, then return it, but first send off the + // next round of fetches to enable pipelining while the user is + // handling the fetched records. + fetcher.initFetches(metadata.fetch(), end); + pollClient(0, end); + return new ConsumerRecords(records); + } + + remaining -= end - start; + + // Nothing was available, so we should backoff before retrying + if (remaining > 0) { + Utils.sleep(min(remaining, retryBackoffMs)); + remaining -= time.milliseconds() - end; + } + } + + return ConsumerRecords.empty(); + } finally { + release(); + } + } + + + /** + * Do one round of polling. In addition to checking for new data, this does any needed + * heart-beating, auto-commits, and offset updates. + * @param timeout The maximum time to block in the underlying poll + * @param now Current time in millis + * @return The fetched records (may be empty) + */ + private Map>> pollOnce(long timeout, long now) { + Cluster cluster = this.metadata.fetch(); + + // TODO: Sub-requests should take into account the poll timeout (KAFKA-1894) if (subscriptions.partitionsAutoAssigned()) { if (subscriptions.partitionAssignmentNeeded()) { @@ -649,26 +774,18 @@ public class KafkaConsumer implements Consumer { // fetch positions if we have partitions we're subscribed to that we // don't know the offset for if (!subscriptions.hasAllFetchPositions()) - updateFetchPositions(this.subscriptions.missingFetchPositions(), now); + updateFetchPositions(this.subscriptions.missingFetchPositions()); // maybe autocommit position if (shouldAutoCommit(now)) commit(CommitType.ASYNC); - /* - * initiate any needed fetches, then block for the timeout the user specified - */ - Cluster cluster = this.metadata.fetch(); + // Init any new fetches (won't resend pending fetches) fetcher.initFetches(cluster, now); - client.poll(timeout, now); - /* - * initiate a fetch request for any nodes that we just got a response from without blocking - */ - fetcher.initFetches(cluster, now); - client.poll(0, now); + pollClient(timeout, now); - return new ConsumerRecords(fetcher.fetchedRecords()); + return fetcher.fetchedRecords(); } /** @@ -686,18 +803,20 @@ public class KafkaConsumer implements Consumer { * @param commitType Control whether the commit is blocking */ @Override - public synchronized void commit(final Map offsets, CommitType commitType) { - ensureNotClosed(); - log.debug("Committing offsets ({}): {} ", commitType.toString().toLowerCase(), offsets); + public void commit(final Map offsets, CommitType commitType) { + acquire(); + try { + log.debug("Committing offsets ({}): {} ", commitType.toString().toLowerCase(), offsets); - long now = time.milliseconds(); - this.lastCommitAttemptMs = now; + this.lastCommitAttemptMs = time.milliseconds(); - // commit the offsets with the coordinator - boolean syncCommit = commitType.equals(CommitType.SYNC); - if (!syncCommit) - this.subscriptions.needRefreshCommits(); - coordinator.commitOffsets(offsets, syncCommit, now); + // commit the offsets with the coordinator + if (commitType == CommitType.ASYNC) + this.subscriptions.needRefreshCommits(); + commitOffsets(offsets, commitType); + } finally { + release(); + } } /** @@ -710,9 +829,13 @@ public class KafkaConsumer implements Consumer { * @param commitType Whether or not the commit should block until it is acknowledged. */ @Override - public synchronized void commit(CommitType commitType) { - ensureNotClosed(); - commit(this.subscriptions.allConsumed(), commitType); + public void commit(CommitType commitType) { + acquire(); + try { + commit(this.subscriptions.allConsumed(), commitType); + } finally { + release(); + } } /** @@ -721,35 +844,43 @@ public class KafkaConsumer implements Consumer { * you may lose data if this API is arbitrarily used in the middle of consumption, to reset the fetch offsets */ @Override - public synchronized void seek(TopicPartition partition, long offset) { - ensureNotClosed(); - log.debug("Seeking to offset {} for partition {}", offset, partition); - this.subscriptions.seek(partition, offset); + public void seek(TopicPartition partition, long offset) { + acquire(); + try { + log.debug("Seeking to offset {} for partition {}", offset, partition); + this.subscriptions.seek(partition, offset); + } finally { + release(); + } } /** * Seek to the first offset for each of the given partitions */ - public synchronized void seekToBeginning(TopicPartition... partitions) { - ensureNotClosed(); - Collection parts = partitions.length == 0 ? this.subscriptions.assignedPartitions() - : Arrays.asList(partitions); - for (TopicPartition tp : parts) { - // TODO: list offset call could be optimized by grouping by node - seek(tp, fetcher.offsetBefore(tp, EARLIEST_OFFSET_TIMESTAMP)); + public void seekToBeginning(TopicPartition... partitions) { + acquire(); + try { + Collection parts = partitions.length == 0 ? this.subscriptions.assignedPartitions() + : Arrays.asList(partitions); + for (TopicPartition tp : parts) + subscriptions.needOffsetReset(tp, OffsetResetStrategy.EARLIEST); + } finally { + release(); } } /** * Seek to the last offset for each of the given partitions */ - public synchronized void seekToEnd(TopicPartition... partitions) { - ensureNotClosed(); - Collection parts = partitions.length == 0 ? this.subscriptions.assignedPartitions() - : Arrays.asList(partitions); - for (TopicPartition tp : parts) { - // TODO: list offset call could be optimized by grouping by node - seek(tp, fetcher.offsetBefore(tp, LATEST_OFFSET_TIMESTAMP)); + public void seekToEnd(TopicPartition... partitions) { + acquire(); + try { + Collection parts = partitions.length == 0 ? this.subscriptions.assignedPartitions() + : Arrays.asList(partitions); + for (TopicPartition tp : parts) + subscriptions.needOffsetReset(tp, OffsetResetStrategy.LATEST); + } finally { + release(); } } @@ -761,16 +892,20 @@ public class KafkaConsumer implements Consumer { * @throws NoOffsetForPartitionException If a position hasn't been set for a given partition, and no reset policy is * available. */ - public synchronized long position(TopicPartition partition) { - ensureNotClosed(); - if (!this.subscriptions.assignedPartitions().contains(partition)) - throw new IllegalArgumentException("You can only check the position for partitions assigned to this consumer."); - Long offset = this.subscriptions.consumed(partition); - if (offset == null) { - updateFetchPositions(Collections.singleton(partition), time.milliseconds()); - return this.subscriptions.consumed(partition); - } else { - return offset; + public long position(TopicPartition partition) { + acquire(); + try { + if (!this.subscriptions.assignedPartitions().contains(partition)) + throw new IllegalArgumentException("You can only check the position for partitions assigned to this consumer."); + Long offset = this.subscriptions.consumed(partition); + if (offset == null) { + updateFetchPositions(Collections.singleton(partition)); + return this.subscriptions.consumed(partition); + } else { + return offset; + } + } finally { + release(); } } @@ -787,22 +922,26 @@ public class KafkaConsumer implements Consumer { * partition. */ @Override - public synchronized long committed(TopicPartition partition) { - ensureNotClosed(); - Set partitionsToFetch; - if (subscriptions.assignedPartitions().contains(partition)) { + public long committed(TopicPartition partition) { + acquire(); + try { + Set partitionsToFetch; + if (subscriptions.assignedPartitions().contains(partition)) { + Long committed = this.subscriptions.committed(partition); + if (committed != null) + return committed; + partitionsToFetch = subscriptions.assignedPartitions(); + } else { + partitionsToFetch = Collections.singleton(partition); + } + refreshCommittedOffsets(partitionsToFetch); Long committed = this.subscriptions.committed(partition); - if (committed != null) - return committed; - partitionsToFetch = subscriptions.assignedPartitions(); - } else { - partitionsToFetch = Collections.singleton(partition); + if (committed == null) + throw new NoOffsetForPartitionException("No offset has been committed for partition " + partition); + return committed; + } finally { + release(); } - refreshCommittedOffsets(partitionsToFetch, time.milliseconds()); - Long committed = this.subscriptions.committed(partition); - if (committed == null) - throw new NoOffsetForPartitionException("No offset has been committed for partition " + partition); - return committed; } /** @@ -822,19 +961,41 @@ public class KafkaConsumer implements Consumer { */ @Override public List partitionsFor(String topic) { - Cluster cluster = this.metadata.fetch(); - List parts = cluster.partitionsForTopic(topic); - if (parts == null) { - metadata.add(topic); - awaitMetadataUpdate(); - parts = metadata.fetch().partitionsForTopic(topic); + acquire(); + try { + Cluster cluster = this.metadata.fetch(); + List parts = cluster.partitionsForTopic(topic); + if (parts == null) { + metadata.add(topic); + awaitMetadataUpdate(); + parts = metadata.fetch().partitionsForTopic(topic); + } + return parts; + } finally { + release(); } - return parts; } @Override - public synchronized void close() { - close(false); + public void close() { + if (closed) return; + + acquire(); + try { + close(false); + } finally { + release(); + } + } + + /** + * Wakeup the consumer. This method is thread-safe and is useful in particular to abort a long poll. + * The thread which is blocking in an operation will throw {@link ConsumerWakeupException}. + */ + @Override + public void wakeup() { + this.wakeup.set(true); + this.client.wakeup(); } private void close(boolean swallowException) { @@ -856,6 +1017,15 @@ public class KafkaConsumer implements Consumer { return this.autoCommit && this.lastCommitAttemptMs <= now - this.autoCommitIntervalMs; } + private long timeToNextCommit(long now) { + if (!this.autoCommit) + return Long.MAX_VALUE; + long timeSinceLastCommit = now - this.lastCommitAttemptMs; + if (timeSinceLastCommit > this.autoCommitIntervalMs) + return 0; + return this.autoCommitIntervalMs - timeSinceLastCommit; + } + /** * Request a metadata update and wait until it has occurred */ @@ -863,7 +1033,7 @@ public class KafkaConsumer implements Consumer { int version = this.metadata.requestUpdate(); do { long now = time.milliseconds(); - this.client.poll(this.retryBackoffMs, now); + this.pollClient(this.retryBackoffMs, now); } while (this.metadata.version() == version); } @@ -881,8 +1051,7 @@ public class KafkaConsumer implements Consumer { } // get new assigned partitions from the coordinator - this.subscriptions.changePartitionAssignment(coordinator.assignPartitions( - new ArrayList(this.subscriptions.subscribedTopics()), now)); + assignPartitions(); // execute the user's callback after rebalance log.debug("Setting newly assigned partitions {}", this.subscriptions.assignedPartitions()); @@ -899,25 +1068,73 @@ public class KafkaConsumer implements Consumer { * or reset it using the offset reset policy the user has configured. * * @param partitions The partitions that needs updating fetch positions - * @param now The current time * @throws org.apache.kafka.clients.consumer.NoOffsetForPartitionException If no offset is stored for a given partition and no offset reset policy is * defined */ - private void updateFetchPositions(Set partitions, long now) { + private void updateFetchPositions(Set partitions) { // first refresh the committed positions in case they are not up-to-date - refreshCommittedOffsets(partitions, now); + refreshCommittedOffsets(partitions); // reset the fetch position to the committed position for (TopicPartition tp : partitions) { - if (subscriptions.fetched(tp) == null) { - if (subscriptions.committed(tp) == null) { - // if the committed position is unknown reset the position - fetcher.resetOffset(tp); - } else { - log.debug("Resetting offset for partition {} to the committed offset {}", - tp, subscriptions.committed(tp)); - subscriptions.seek(tp, subscriptions.committed(tp)); - } + // Skip if we already have a fetch position + if (subscriptions.fetched(tp) != null) + continue; + + // TODO: If there are several offsets to reset, we could submit offset requests in parallel + if (subscriptions.isOffsetResetNeeded(tp)) { + resetOffset(tp); + } else if (subscriptions.committed(tp) == null) { + // There's no committed position, so we need to reset with the default strategy + subscriptions.needOffsetReset(tp); + resetOffset(tp); + } else { + log.debug("Resetting offset for partition {} to the committed offset {}", + tp, subscriptions.committed(tp)); + subscriptions.seek(tp, subscriptions.committed(tp)); + } + } + } + + /** + * Reset offsets for the given partition using the offset reset strategy. + * + * @param partition The given partition that needs reset offset + * @throws org.apache.kafka.clients.consumer.NoOffsetForPartitionException If no offset reset strategy is defined + */ + private void resetOffset(TopicPartition partition) { + OffsetResetStrategy strategy = subscriptions.resetStrategy(partition); + final long timestamp; + if (strategy == OffsetResetStrategy.EARLIEST) + timestamp = EARLIEST_OFFSET_TIMESTAMP; + else if (strategy == OffsetResetStrategy.LATEST) + timestamp = LATEST_OFFSET_TIMESTAMP; + else + throw new NoOffsetForPartitionException("No offset is set and no reset policy is defined"); + + log.debug("Resetting offset for partition {} to {} offset.", partition, strategy.name().toLowerCase()); + long offset = listOffset(partition, timestamp); + this.subscriptions.seek(partition, offset); + } + + /** + * Fetch a single offset before the given timestamp for the partition. + * + * @param partition The partition that needs fetching offset. + * @param timestamp The timestamp for fetching offset. + * @return The offset of the message that is published before the given timestamp + */ + private long listOffset(TopicPartition partition, long timestamp) { + while (true) { + RequestFuture future = fetcher.listOffset(partition, timestamp); + + if (!future.isDone()) + pollFuture(future, requestTimeoutMs); + + if (future.isDone()) { + if (future.succeeded()) + return future.value(); + handleRequestFailure(future); } } } @@ -925,13 +1142,13 @@ public class KafkaConsumer implements Consumer { /** * Refresh the committed offsets for given set of partitions and update the cache */ - private void refreshCommittedOffsets(Set partitions, long now) { + private void refreshCommittedOffsets(Set partitions) { // we only need to fetch latest committed offset from coordinator if there // is some commit process in progress, otherwise our current // committed cache is up-to-date if (subscriptions.refreshCommitsNeeded()) { // contact coordinator to fetch committed offsets - Map offsets = coordinator.fetchOffsets(partitions, now); + Map offsets = fetchCommittedOffsets(partitions); // update the position with the offsets for (Map.Entry entry : offsets.entrySet()) { @@ -941,6 +1158,183 @@ public class KafkaConsumer implements Consumer { } } + /** + * Block until we have received a partition assignment from the coordinator. + */ + private void assignPartitions() { + // Ensure that there are no pending requests to the coordinator. This is important + // in particular to avoid resending a pending JoinGroup request. + awaitCoordinatorInFlightRequests(); + + while (subscriptions.partitionAssignmentNeeded()) { + RequestFuture future = coordinator.assignPartitions(time.milliseconds()); + + // Block indefinitely for the join group request (which can take as long as a session timeout) + if (!future.isDone()) + pollFuture(future); + + if (future.failed()) + handleRequestFailure(future); + } + } + + /** + * Block until the coordinator for this group is known. + */ + private void ensureCoordinatorKnown() { + while (coordinator.coordinatorUnknown()) { + RequestFuture future = coordinator.discoverConsumerCoordinator(); + + if (!future.isDone()) + pollFuture(future, requestTimeoutMs); + + if (future.failed()) + handleRequestFailure(future); + } + } + + /** + * Block until any pending requests to the coordinator have been handled. + */ + public void awaitCoordinatorInFlightRequests() { + while (coordinator.hasInFlightRequests()) { + long now = time.milliseconds(); + pollClient(-1, now); + } + } + + /** + * Lookup the committed offsets for a set of partitions. This will block until the coordinator has + * responded to the offset fetch request. + * @param partitions List of partitions to get offsets for + * @return Map from partition to its respective offset + */ + private Map fetchCommittedOffsets(Set partitions) { + while (true) { + long now = time.milliseconds(); + RequestFuture> future = coordinator.fetchOffsets(partitions, now); + + if (!future.isDone()) + pollFuture(future, requestTimeoutMs); + + if (future.isDone()) { + if (future.succeeded()) + return future.value(); + handleRequestFailure(future); + } + } + } + + /** + * Commit offsets. This call blocks (regardless of commitType) until the coordinator + * can receive the commit request. Once the request has been made, however, only the + * synchronous commits will wait for a successful response from the coordinator. + * @param offsets Offsets to commit. + * @param commitType Commit policy + */ + private void commitOffsets(Map offsets, CommitType commitType) { + if (commitType == CommitType.ASYNC) { + commitOffsetsAsync(offsets); + } else { + commitOffsetsSync(offsets); + } + } + + private void commitOffsetsAsync(Map offsets) { + while (true) { + long now = time.milliseconds(); + RequestFuture future = coordinator.commitOffsets(offsets, now); + + if (!future.isDone() || future.succeeded()) + return; + + handleRequestFailure(future); + } + } + + private void commitOffsetsSync(Map offsets) { + while (true) { + long now = time.milliseconds(); + RequestFuture future = coordinator.commitOffsets(offsets, now); + + if (!future.isDone()) + pollFuture(future, requestTimeoutMs); + + if (future.isDone()) { + if (future.succeeded()) + return; + else + handleRequestFailure(future); + } + } + } + + private void handleRequestFailure(RequestFuture future) { + if (future.hasException()) + throw future.exception(); + + switch (future.retryAction()) { + case BACKOFF: + Utils.sleep(retryBackoffMs); + break; + case POLL: + pollClient(retryBackoffMs, time.milliseconds()); + break; + case FIND_COORDINATOR: + ensureCoordinatorKnown(); + break; + case REFRESH_METADATA: + awaitMetadataUpdate(); + break; + case NOOP: + // Do nothing (retry now) + } + } + + /** + * Poll until a result is ready or timeout expires + * @param future The future to poll for + * @param timeout The time in milliseconds to wait for the result + */ + private void pollFuture(RequestFuture future, long timeout) { + // TODO: Update this code for KAFKA-2120, which adds request timeout to NetworkClient + // In particular, we must ensure that "timed out" requests will not have their callbacks + // invoked at a later time. + long remaining = timeout; + while (!future.isDone() && remaining >= 0) { + long start = time.milliseconds(); + pollClient(remaining, start); + if (future.isDone()) return; + remaining -= time.milliseconds() - start; + } + } + + /** + * Poll indefinitely until the result is ready. + * @param future The future to poll for. + */ + private void pollFuture(RequestFuture future) { + while (!future.isDone()) { + long now = time.milliseconds(); + pollClient(-1, now); + } + } + + /** + * Poll for IO. + * @param timeout The maximum time to wait for IO to become available + * @param now The current time in milliseconds + * @throws ConsumerWakeupException if {@link #wakeup()} is invoked while the poll is active + */ + private void pollClient(long timeout, long now) { + this.client.poll(timeout, now); + + if (wakeup.get()) { + wakeup.set(false); + throw new ConsumerWakeupException(); + } + } + /* * Check that the consumer hasn't been closed. */ @@ -948,4 +1342,27 @@ public class KafkaConsumer implements Consumer { if (this.closed) throw new IllegalStateException("This consumer has already been closed."); } + + /** + * Acquire the light lock protecting this consumer from multi-threaded access. Instead of blocking + * when the lock is not available, however, we just throw an exception (since multi-threaded usage is not + * supported). + * @throws IllegalStateException if the consumer has been closed + * @throws ConcurrentModificationException if another thread already has the lock + */ + private void acquire() { + ensureNotClosed(); + Long threadId = Thread.currentThread().getId(); + if (!threadId.equals(currentThread.get()) && !currentThread.compareAndSet(null, threadId)) + throw new ConcurrentModificationException("KafkaConsumer is not safe for multi-threaded access"); + refcount++; + } + + /** + * Release the light lock protecting the consumer from multi-threaded access. + */ + private void release() { + if (--refcount == 0) + currentThread.set(null); + } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java index f50da82..46e26a6 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java @@ -40,8 +40,8 @@ public class MockConsumer implements Consumer { private Map>> records; private boolean closed; - public MockConsumer() { - this.subscriptions = new SubscriptionState(); + public MockConsumer(OffsetResetStrategy offsetResetStrategy) { + this.subscriptions = new SubscriptionState(offsetResetStrategy); this.partitions = new HashMap>(); this.records = new HashMap>>(); this.closed = false; @@ -175,6 +175,11 @@ public class MockConsumer implements Consumer { this.closed = true; } + @Override + public void wakeup() { + + } + private void ensureNotClosed() { if (this.closed) throw new IllegalStateException("This consumer has already been closed."); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetResetStrategy.java b/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetResetStrategy.java new file mode 100644 index 0000000..542da7f --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetResetStrategy.java @@ -0,0 +1,17 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package org.apache.kafka.clients.consumer; + +public enum OffsetResetStrategy { + LATEST, EARLIEST, NONE +} 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 41cb945..6c26667 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 @@ -15,7 +15,6 @@ package org.apache.kafka.clients.consumer.internals; import org.apache.kafka.clients.ClientRequest; import org.apache.kafka.clients.ClientResponse; import org.apache.kafka.clients.KafkaClient; -import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.RequestCompletionHandler; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.MetricName; @@ -57,7 +56,7 @@ import java.util.Set; import java.util.concurrent.TimeUnit; /** - * This class manage the coordination process with the consumer coordinator. + * This class manages the coordination process with the consumer coordinator. */ public final class Coordinator { @@ -67,13 +66,11 @@ public final class Coordinator { private final Time time; private final String groupId; - private final Metadata metadata; private final Heartbeat heartbeat; private final int sessionTimeoutMs; private final String assignmentStrategy; private final SubscriptionState subscriptions; private final CoordinatorMetrics sensors; - private final long retryBackoffMs; private Node consumerCoordinator; private String consumerId; private int generation; @@ -83,10 +80,8 @@ public final class Coordinator { */ public Coordinator(KafkaClient client, String groupId, - long retryBackoffMs, int sessionTimeoutMs, String assignmentStrategy, - Metadata metadata, SubscriptionState subscriptions, Metrics metrics, String metricGrpPrefix, @@ -98,10 +93,8 @@ public final class Coordinator { this.generation = -1; this.consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID; this.groupId = groupId; - this.metadata = metadata; this.consumerCoordinator = null; this.subscriptions = subscriptions; - this.retryBackoffMs = retryBackoffMs; this.sessionTimeoutMs = sessionTimeoutMs; this.assignmentStrategy = assignmentStrategy; this.heartbeat = new Heartbeat(this.sessionTimeoutMs, time.milliseconds()); @@ -109,84 +102,110 @@ public final class Coordinator { } /** - * Assign partitions for the subscribed topics. - * - * @param subscribedTopics The subscribed topics list - * @param now The current time - * @return The assigned partition info + * Send a request to get a new partition assignment. This is a non-blocking call which sends + * a JoinGroup request to the coordinator (if it is available). The returned future must + * be polled to see if the request completed successfully. + * @param now The current time in milliseconds + * @return A request future whose completion indicates the result of the JoinGroup request. */ - public List assignPartitions(List subscribedTopics, long now) { + public RequestFuture assignPartitions(final long now) { + final RequestFuture future = newCoordinatorRequestFuture(now); + if (future.isDone()) return future; // send a join group request to the coordinator + List subscribedTopics = new ArrayList(subscriptions.subscribedTopics()); log.debug("(Re-)joining group {} with subscribed topics {}", groupId, subscribedTopics); - // repeat processing the response until succeed or fatal error - do { - JoinGroupRequest request = new JoinGroupRequest(groupId, + JoinGroupRequest request = new JoinGroupRequest(groupId, this.sessionTimeoutMs, subscribedTopics, this.consumerId, this.assignmentStrategy); - ClientResponse resp = this.blockingCoordinatorRequest(ApiKeys.JOIN_GROUP, request.toStruct(), null, now); - JoinGroupResponse response = new JoinGroupResponse(resp.responseBody()); - short errorCode = response.errorCode(); + // create the request for the coordinator + log.debug("Issuing request ({}: {}) to coordinator {}", ApiKeys.JOIN_GROUP, request, this.consumerCoordinator.id()); + + RequestCompletionHandler completionHandler = new RequestCompletionHandler() { + @Override + public void onComplete(ClientResponse resp) { + handleJoinResponse(resp, future); + } + }; + + sendCoordinator(ApiKeys.JOIN_GROUP, request.toStruct(), completionHandler, now); + return future; + } + + private void handleJoinResponse(ClientResponse response, RequestFuture future) { + if (response.wasDisconnected()) { + handleCoordinatorDisconnect(response); + future.retryWithNewCoordinator(); + } else { + // process the response + JoinGroupResponse joinResponse = new JoinGroupResponse(response.responseBody()); + short errorCode = joinResponse.errorCode(); if (errorCode == Errors.NONE.code()) { - this.consumerId = response.consumerId(); - this.generation = response.generationId(); + Coordinator.this.consumerId = joinResponse.consumerId(); + Coordinator.this.generation = joinResponse.generationId(); // set the flag to refresh last committed offsets - this.subscriptions.needRefreshCommits(); + subscriptions.needRefreshCommits(); log.debug("Joined group: {}", response); // record re-assignment time - this.sensors.partitionReassignments.record(time.milliseconds() - now); + this.sensors.partitionReassignments.record(response.requestLatencyMs()); - // return assigned partitions - return response.assignedPartitions(); + // update partition assignment + subscriptions.changePartitionAssignment(joinResponse.assignedPartitions()); + future.complete(null); } else if (errorCode == Errors.UNKNOWN_CONSUMER_ID.code()) { // reset the consumer id and retry immediately - this.consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID; + Coordinator.this.consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID; log.info("Attempt to join group {} failed due to unknown consumer id, resetting and retrying.", - groupId); + groupId); + + future.retryNow(); } else if (errorCode == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code() || errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { // re-discover the coordinator and retry with backoff coordinatorDead(); - Utils.sleep(this.retryBackoffMs); - log.info("Attempt to join group {} failed due to obsolete coordinator information, retrying.", - groupId); + groupId); + future.retryWithNewCoordinator(); } else if (errorCode == Errors.UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY.code() || errorCode == Errors.INCONSISTENT_PARTITION_ASSIGNMENT_STRATEGY.code() || errorCode == Errors.INVALID_SESSION_TIMEOUT.code()) { // log the error and re-throw the exception + KafkaException e = Errors.forCode(errorCode).exception(); log.error("Attempt to join group {} failed due to: {}", - groupId, Errors.forCode(errorCode).exception().getMessage()); - Errors.forCode(errorCode).maybeThrow(); + groupId, e.getMessage()); + future.raise(e); } else { // unexpected error, throw the exception - throw new KafkaException("Unexpected error in join group response: " - + Errors.forCode(response.errorCode()).exception().getMessage()); + future.raise(new KafkaException("Unexpected error in join group response: " + + Errors.forCode(joinResponse.errorCode()).exception().getMessage())); } - } while (true); + } } /** - * Commit offsets for the specified list of topics and partitions. - * - * A non-blocking commit will attempt to commit offsets asychronously. No error will be thrown if the commit fails. - * A blocking commit will wait for a response acknowledging the commit. In the event of an error it will retry until - * the commit succeeds. + * Commit offsets for the specified list of topics and partitions. This is a non-blocking call + * which returns a request future that can be polled in the case of a synchronous commit or ignored in the + * asynchronous case. * * @param offsets The list of offsets per partition that should be committed. - * @param blocking Control whether the commit is blocking * @param now The current time + * @return A request future whose value indicates whether the commit was successful or not */ - public void commitOffsets(final Map offsets, boolean blocking, long now) { - if (!offsets.isEmpty()) { + public RequestFuture commitOffsets(final Map offsets, long now) { + final RequestFuture future = newCoordinatorRequestFuture(now); + if (future.isDone()) return future; + + if (offsets.isEmpty()) { + future.complete(null); + } else { // create the offset commit request Map offsetData; offsetData = new HashMap(offsets.size()); @@ -198,52 +217,63 @@ public final class Coordinator { OffsetCommitRequest.DEFAULT_RETENTION_TIME, offsetData); - // send request and possibly wait for response if it is blocking - RequestCompletionHandler handler = new CommitOffsetCompletionHandler(offsets); + RequestCompletionHandler handler = new CommitOffsetCompletionHandler(offsets, future); + sendCoordinator(ApiKeys.OFFSET_COMMIT, req.toStruct(), handler, now); + } - if (blocking) { - boolean done; - do { - ClientResponse response = blockingCoordinatorRequest(ApiKeys.OFFSET_COMMIT, req.toStruct(), handler, now); + return future; + } - // check for errors - done = true; - OffsetCommitResponse commitResponse = new OffsetCommitResponse(response.responseBody()); - for (short errorCode : commitResponse.responseData().values()) { - if (errorCode != Errors.NONE.code()) - done = false; - } - if (!done) { - log.debug("Error in offset commit, backing off for {} ms before retrying again.", - this.retryBackoffMs); - Utils.sleep(this.retryBackoffMs); - } - } while (!done); - } else { - this.client.send(initiateCoordinatorRequest(ApiKeys.OFFSET_COMMIT, req.toStruct(), handler, now)); - } + private RequestFuture newCoordinatorRequestFuture(long now) { + if (coordinatorUnknown()) + return RequestFuture.newCoordinatorNeeded(); + + if (client.ready(this.consumerCoordinator, now)) + // We have an open connection and we're ready to send + return new RequestFuture(); + + if (this.client.connectionFailed(this.consumerCoordinator)) { + coordinatorDead(); + return RequestFuture.newCoordinatorNeeded(); } + + // The connection has been initiated, so we need to poll to finish it + return RequestFuture.pollNeeded(); } /** - * Fetch the committed offsets of the given set of partitions. + * Fetch the committed offsets for a set of partitions. This is a non-blocking call. The + * returned future can be polled to get the actual offsets returned from the broker. * - * @param partitions The list of partitions which need to ask for committed offsets - * @param now The current time - * @return The fetched offset values + * @param partitions The set of partitions to get offsets for. + * @param now The current time in milliseconds + * @return A request future containing the committed offsets. */ - public Map fetchOffsets(Set partitions, long now) { - log.debug("Fetching committed offsets for partitions: " + Utils.join(partitions, ", ")); - - while (true) { - // construct the request - OffsetFetchRequest request = new OffsetFetchRequest(this.groupId, new ArrayList(partitions)); + public RequestFuture> fetchOffsets(Set partitions, long now) { + final RequestFuture> future = newCoordinatorRequestFuture(now); + if (future.isDone()) return future; - // send the request and block on waiting for response - ClientResponse resp = this.blockingCoordinatorRequest(ApiKeys.OFFSET_FETCH, request.toStruct(), null, now); + log.debug("Fetching committed offsets for partitions: " + Utils.join(partitions, ", ")); + // construct the request + OffsetFetchRequest request = new OffsetFetchRequest(this.groupId, new ArrayList(partitions)); + + // send the request with a callback + RequestCompletionHandler completionHandler = new RequestCompletionHandler() { + @Override + public void onComplete(ClientResponse resp) { + handleOffsetResponse(resp, future); + } + }; + sendCoordinator(ApiKeys.OFFSET_FETCH, request.toStruct(), completionHandler, now); + return future; + } + private void handleOffsetResponse(ClientResponse resp, RequestFuture> future) { + if (resp.wasDisconnected()) { + handleCoordinatorDisconnect(resp); + future.retryWithNewCoordinator(); + } else { // parse the response to get the offsets - boolean offsetsReady = true; OffsetFetchResponse response = new OffsetFetchResponse(resp.responseBody()); Map offsets = new HashMap(response.responseData().size()); for (Map.Entry entry : response.responseData().entrySet()) { @@ -251,23 +281,21 @@ public final class Coordinator { OffsetFetchResponse.PartitionData data = entry.getValue(); if (data.hasError()) { log.debug("Error fetching offset for topic-partition {}: {}", tp, Errors.forCode(data.errorCode) - .exception() - .getMessage()); + .exception() + .getMessage()); if (data.errorCode == Errors.OFFSET_LOAD_IN_PROGRESS.code()) { // just retry - offsetsReady = false; - Utils.sleep(this.retryBackoffMs); + future.retryAfterBackoff(); } else if (data.errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { // re-discover the coordinator and retry coordinatorDead(); - offsetsReady = false; - Utils.sleep(this.retryBackoffMs); + future.retryWithNewCoordinator(); } else if (data.errorCode == Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) { // just ignore this partition log.debug("Unknown topic or partition for " + tp); } else { - throw new KafkaException("Unexpected error in fetch offset response: " - + Errors.forCode(data.errorCode).exception().getMessage()); + future.raise(new KafkaException("Unexpected error in fetch offset response: " + + Errors.forCode(data.errorCode).exception().getMessage())); } } else if (data.offset >= 0) { // record the position with the offset (-1 indicates no committed offset to fetch) @@ -277,8 +305,8 @@ public final class Coordinator { } } - if (offsetsReady) - return offsets; + if (!future.isDone()) + future.complete(offsets); } } @@ -288,124 +316,105 @@ public final class Coordinator { * @param now The current time */ public void maybeHeartbeat(long now) { - if (heartbeat.shouldHeartbeat(now)) { + if (heartbeat.shouldHeartbeat(now) && coordinatorReady(now)) { HeartbeatRequest req = new HeartbeatRequest(this.groupId, this.generation, this.consumerId); - this.client.send(initiateCoordinatorRequest(ApiKeys.HEARTBEAT, req.toStruct(), new HeartbeatCompletionHandler(), now)); + sendCoordinator(ApiKeys.HEARTBEAT, req.toStruct(), new HeartbeatCompletionHandler(), now); this.heartbeat.sentHeartbeat(now); } } - public boolean coordinatorUnknown() { - return this.consumerCoordinator == null; - } - /** - * Repeatedly attempt to send a request to the coordinator until a response is received (retry if we are - * disconnected). Note that this means any requests sent this way must be idempotent. - * - * @return The response + * Get the time until the next heartbeat is needed. + * @param now The current time + * @return The duration in milliseconds before the next heartbeat will be needed. */ - private ClientResponse blockingCoordinatorRequest(ApiKeys api, - Struct request, - RequestCompletionHandler handler, - long now) { - while (true) { - ClientRequest coordinatorRequest = initiateCoordinatorRequest(api, request, handler, now); - ClientResponse coordinatorResponse = sendAndReceive(coordinatorRequest, now); - if (coordinatorResponse.wasDisconnected()) { - handleCoordinatorDisconnect(coordinatorResponse); - Utils.sleep(this.retryBackoffMs); - } else { - return coordinatorResponse; - } - } + public long timeToNextHeartbeat(long now) { + return heartbeat.timeToNextHeartbeat(now); } /** - * Ensure the consumer coordinator is known and we have a ready connection to it. + * Check whether the coordinator has any in-flight requests. + * @return true if the coordinator has pending requests. */ - private void ensureCoordinatorReady() { - while (true) { - if (this.consumerCoordinator == null) - discoverCoordinator(); - - while (true) { - boolean ready = this.client.ready(this.consumerCoordinator, time.milliseconds()); - if (ready) { - return; - } else { - log.debug("No connection to coordinator, attempting to connect."); - this.client.poll(this.retryBackoffMs, time.milliseconds()); + public boolean hasInFlightRequests() { + return !coordinatorUnknown() && client.inFlightRequestCount(consumerCoordinator.idString()) > 0; + } - // if the coordinator connection has failed, we need to - // break the inner loop to re-discover the coordinator - if (this.client.connectionFailed(this.consumerCoordinator)) { - log.debug("Coordinator connection failed. Attempting to re-discover."); - coordinatorDead(); - break; - } - } - } - } + public boolean coordinatorUnknown() { + return this.consumerCoordinator == null; } - /** - * Mark the current coordinator as dead. - */ - private void coordinatorDead() { - if (this.consumerCoordinator != null) { - log.info("Marking the coordinator {} dead.", this.consumerCoordinator.id()); - this.consumerCoordinator = null; - } + private boolean coordinatorReady(long now) { + return !coordinatorUnknown() && this.client.ready(this.consumerCoordinator, now); } /** - * Keep discovering the consumer coordinator until it is found. + * Discover the current coordinator for the consumer group. Sends a ConsumerMetadata request to + * one of the brokers. The returned future should be polled to get the result of the request. + * @return A request future which indicates the completion of the metadata request */ - private void discoverCoordinator() { - while (this.consumerCoordinator == null) { - log.debug("No coordinator known, attempting to discover one."); - Node coordinator = fetchConsumerCoordinator(); - - if (coordinator == null) { - log.debug("No coordinator found, backing off."); - Utils.sleep(this.retryBackoffMs); + public RequestFuture discoverConsumerCoordinator() { + // initiate the consumer metadata request + // find a node to ask about the coordinator + long now = time.milliseconds(); + Node node = this.client.leastLoadedNode(now); + + if (node == null) { + return RequestFuture.metadataRefreshNeeded(); + } else if (!this.client.ready(node, now)) { + if (this.client.connectionFailed(node)) { + return RequestFuture.metadataRefreshNeeded(); } else { - log.debug("Found coordinator: " + coordinator); - this.consumerCoordinator = coordinator; + return RequestFuture.pollNeeded(); } + } else { + final RequestFuture future = new RequestFuture(); + + // create a consumer metadata request + log.debug("Issuing consumer metadata request to broker {}", node.id()); + ConsumerMetadataRequest metadataRequest = new ConsumerMetadataRequest(this.groupId); + RequestCompletionHandler completionHandler = new RequestCompletionHandler() { + @Override + public void onComplete(ClientResponse resp) { + handleConsumerMetadataResponse(resp, future); + } + }; + send(node, ApiKeys.CONSUMER_METADATA, metadataRequest.toStruct(), completionHandler, now); + return future; } } - /** - * Get the current consumer coordinator information via consumer metadata request. - * - * @return the consumer coordinator node - */ - private Node fetchConsumerCoordinator() { - - // initiate the consumer metadata request - ClientRequest request = initiateConsumerMetadataRequest(); - - // send the request and wait for its response - ClientResponse response = sendAndReceive(request, request.createdTime()); + private void handleConsumerMetadataResponse(ClientResponse resp, RequestFuture future) { + log.debug("Consumer metadata response {}", resp); // parse the response to get the coordinator info if it is not disconnected, // otherwise we need to request metadata update - if (!response.wasDisconnected()) { - ConsumerMetadataResponse consumerMetadataResponse = new ConsumerMetadataResponse(response.responseBody()); + if (resp.wasDisconnected()) { + future.retryAfterMetadataRefresh(); + } else { + ConsumerMetadataResponse consumerMetadataResponse = new ConsumerMetadataResponse(resp.responseBody()); // use MAX_VALUE - node.id as the coordinator id to mimic separate connections // for the coordinator in the underlying network client layer // TODO: this needs to be better handled in KAFKA-1935 - if (consumerMetadataResponse.errorCode() == Errors.NONE.code()) - return new Node(Integer.MAX_VALUE - consumerMetadataResponse.node().id(), - consumerMetadataResponse.node().host(), - consumerMetadataResponse.node().port()); - } else { - this.metadata.requestUpdate(); + if (consumerMetadataResponse.errorCode() == Errors.NONE.code()) { + this.consumerCoordinator = new Node(Integer.MAX_VALUE - consumerMetadataResponse.node().id(), + consumerMetadataResponse.node().host(), + consumerMetadataResponse.node().port()); + future.complete(null); + } else { + future.retryAfterBackoff(); + } } + } - return null; + /** + * Mark the current coordinator as dead. + */ + private void coordinatorDead() { + if (this.consumerCoordinator != null) { + log.info("Marking the coordinator {} dead.", this.consumerCoordinator.id()); + this.consumerCoordinator = null; + } } /** @@ -414,79 +423,23 @@ public final class Coordinator { private void handleCoordinatorDisconnect(ClientResponse response) { int correlation = response.request().request().header().correlationId(); log.debug("Cancelled request {} with correlation id {} due to coordinator {} being disconnected", - response.request(), - correlation, - response.request().request().destination()); + response.request(), + correlation, + response.request().request().destination()); // mark the coordinator as dead coordinatorDead(); } - /** - * Initiate a consumer metadata request to the least loaded node. - * - * @return The created request - */ - private ClientRequest initiateConsumerMetadataRequest() { - // find a node to ask about the coordinator - Node node = this.client.leastLoadedNode(time.milliseconds()); - while (node == null || !this.client.ready(node, time.milliseconds())) { - long now = time.milliseconds(); - this.client.poll(this.retryBackoffMs, now); - node = this.client.leastLoadedNode(now); - - // if there is no ready node, backoff before retry - if (node == null) - Utils.sleep(this.retryBackoffMs); - } - - // create a consumer metadata request - log.debug("Issuing consumer metadata request to broker {}", node.id()); - - ConsumerMetadataRequest request = new ConsumerMetadataRequest(this.groupId); - RequestSend send = new RequestSend(node.idString(), - this.client.nextRequestHeader(ApiKeys.CONSUMER_METADATA), - request.toStruct()); - long now = time.milliseconds(); - return new ClientRequest(now, true, send, null); + private void sendCoordinator(ApiKeys api, Struct request, RequestCompletionHandler handler, long now) { + send(this.consumerCoordinator, api, request, handler, now); } - /** - * Initiate a request to the coordinator. - */ - private ClientRequest initiateCoordinatorRequest(ApiKeys api, Struct request, RequestCompletionHandler handler, long now) { - - // first make sure the coordinator is known and ready - ensureCoordinatorReady(); - - // create the request for the coordinator - log.debug("Issuing request ({}: {}) to coordinator {}", api, request, this.consumerCoordinator.id()); - + private void send(Node node, ApiKeys api, Struct request, RequestCompletionHandler handler, long now) { RequestHeader header = this.client.nextRequestHeader(api); - RequestSend send = new RequestSend(this.consumerCoordinator.idString(), header, request); - return new ClientRequest(now, true, send, handler); - } - - /** - * Attempt to send a request and receive its response. - * - * @return The response - */ - private ClientResponse sendAndReceive(ClientRequest clientRequest, long now) { - - // send the request - this.client.send(clientRequest); - - // drain all responses from the destination node - List responses = this.client.completeAll(clientRequest.request().destination(), now); - if (responses.isEmpty()) { - throw new IllegalStateException("This should not happen."); - } else { - // other requests should be handled by the callback, and - // we only care about the response of the last request - return responses.get(responses.size() - 1); - } + RequestSend send = new RequestSend(node.idString(), header, request); + this.client.send(new ClientRequest(now, true, send, handler)); } private class HeartbeatCompletionHandler implements RequestCompletionHandler { @@ -521,18 +474,21 @@ public final class Coordinator { private class CommitOffsetCompletionHandler implements RequestCompletionHandler { private final Map offsets; + private final RequestFuture future; - public CommitOffsetCompletionHandler(Map offsets) { + public CommitOffsetCompletionHandler(Map offsets, RequestFuture future) { this.offsets = offsets; + this.future = future; } @Override public void onComplete(ClientResponse resp) { if (resp.wasDisconnected()) { handleCoordinatorDisconnect(resp); + future.retryWithNewCoordinator(); } else { - OffsetCommitResponse response = new OffsetCommitResponse(resp.responseBody()); - for (Map.Entry entry : response.responseData().entrySet()) { + OffsetCommitResponse commitResponse = new OffsetCommitResponse(resp.responseBody()); + for (Map.Entry entry : commitResponse.responseData().entrySet()) { TopicPartition tp = entry.getKey(); short errorCode = entry.getValue(); long offset = this.offsets.get(tp); @@ -542,14 +498,19 @@ public final class Coordinator { } else if (errorCode == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code() || errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { coordinatorDead(); + future.retryWithNewCoordinator(); } else { // do not need to throw the exception but just log the error + future.retryAfterBackoff(); log.error("Error committing partition {} at offset {}: {}", tp, offset, Errors.forCode(errorCode).exception().getMessage()); } } + + if (!future.isDone()) + future.complete(null); } sensors.commitLatency.record(resp.requestLatencyMs()); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index 56281ee..695eaf6 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -19,7 +19,6 @@ import org.apache.kafka.clients.KafkaClient; import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.RequestCompletionHandler; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.NoOffsetForPartitionException; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.Node; @@ -61,9 +60,6 @@ import java.util.Map; public class Fetcher { private static final Logger log = LoggerFactory.getLogger(Fetcher.class); - private static final long EARLIEST_OFFSET_TIMESTAMP = -2L; - private static final long LATEST_OFFSET_TIMESTAMP = -1L; - private final KafkaClient client; @@ -72,23 +68,19 @@ public class Fetcher { private final int maxWaitMs; private final int fetchSize; private final boolean checkCrcs; - private final long retryBackoffMs; private final Metadata metadata; private final FetchManagerMetrics sensors; private final SubscriptionState subscriptions; private final List> records; - private final AutoOffsetResetStrategy offsetResetStrategy; private final Deserializer keyDeserializer; private final Deserializer valueDeserializer; public Fetcher(KafkaClient client, - long retryBackoffMs, int minBytes, int maxWaitMs, int fetchSize, boolean checkCrcs, - String offsetReset, Deserializer keyDeserializer, Deserializer valueDeserializer, Metadata metadata, @@ -102,17 +94,16 @@ public class Fetcher { this.client = client; this.metadata = metadata; this.subscriptions = subscriptions; - this.retryBackoffMs = retryBackoffMs; this.minBytes = minBytes; this.maxWaitMs = maxWaitMs; this.fetchSize = fetchSize; this.checkCrcs = checkCrcs; - this.offsetResetStrategy = AutoOffsetResetStrategy.valueOf(offsetReset); this.keyDeserializer = keyDeserializer; this.valueDeserializer = valueDeserializer; this.records = new LinkedList>(); + this.sensors = new FetchManagerMetrics(metrics, metricGrpPrefix, metricTags); } @@ -166,84 +157,76 @@ public class Fetcher { } /** - * Reset offsets for the given partition using the offset reset strategy. - * - * @param partition The given partition that needs reset offset - * @throws org.apache.kafka.clients.consumer.NoOffsetForPartitionException If no offset reset strategy is defined - */ - public void resetOffset(TopicPartition partition) { - long timestamp; - if (this.offsetResetStrategy == AutoOffsetResetStrategy.EARLIEST) - timestamp = EARLIEST_OFFSET_TIMESTAMP; - else if (this.offsetResetStrategy == AutoOffsetResetStrategy.LATEST) - timestamp = LATEST_OFFSET_TIMESTAMP; - else - throw new NoOffsetForPartitionException("No offset is set and no reset policy is defined"); - - log.debug("Resetting offset for partition {} to {} offset.", partition, this.offsetResetStrategy.name() - .toLowerCase()); - this.subscriptions.seek(partition, offsetBefore(partition, timestamp)); - } - - /** * Fetch a single offset before the given timestamp for the partition. * * @param topicPartition The partition that needs fetching offset. * @param timestamp The timestamp for fetching offset. - * @return The offset of the message that is published before the given timestamp + * @return A response which can be polled to obtain the corresponding offset. */ - public long offsetBefore(TopicPartition topicPartition, long timestamp) { - log.debug("Fetching offsets for partition {}.", topicPartition); + public RequestFuture listOffset(final TopicPartition topicPartition, long timestamp) { Map partitions = new HashMap(1); partitions.put(topicPartition, new ListOffsetRequest.PartitionData(timestamp, 1)); - while (true) { - long now = time.milliseconds(); - PartitionInfo info = metadata.fetch().partition(topicPartition); - if (info == null) { - metadata.add(topicPartition.topic()); - log.debug("Partition {} is unknown for fetching offset, wait for metadata refresh", topicPartition); - awaitMetadataUpdate(); - } else if (info.leader() == null) { - log.debug("Leader for partition {} unavailable for fetching offset, wait for metadata refresh", topicPartition); - awaitMetadataUpdate(); - } else if (this.client.ready(info.leader(), now)) { - Node node = info.leader(); - ListOffsetRequest request = new ListOffsetRequest(-1, partitions); - RequestSend send = new RequestSend(node.idString(), + long now = time.milliseconds(); + PartitionInfo info = metadata.fetch().partition(topicPartition); + if (info == null) { + metadata.add(topicPartition.topic()); + log.debug("Partition {} is unknown for fetching offset, wait for metadata refresh", topicPartition); + return RequestFuture.metadataRefreshNeeded(); + } else if (info.leader() == null) { + log.debug("Leader for partition {} unavailable for fetching offset, wait for metadata refresh", topicPartition); + return RequestFuture.metadataRefreshNeeded(); + } else if (this.client.ready(info.leader(), now)) { + final RequestFuture future = new RequestFuture(); + Node node = info.leader(); + ListOffsetRequest request = new ListOffsetRequest(-1, partitions); + RequestSend send = new RequestSend(node.idString(), this.client.nextRequestHeader(ApiKeys.LIST_OFFSETS), request.toStruct()); - ClientRequest clientRequest = new ClientRequest(now, true, send, null); - this.client.send(clientRequest); - List responses = this.client.completeAll(node.idString(), now); - if (responses.isEmpty()) - throw new IllegalStateException("This should not happen."); - ClientResponse response = responses.get(responses.size() - 1); - if (response.wasDisconnected()) { - awaitMetadataUpdate(); - } else { - ListOffsetResponse lor = new ListOffsetResponse(response.responseBody()); - short errorCode = lor.responseData().get(topicPartition).errorCode; - if (errorCode == Errors.NONE.code()) { - List offsets = lor.responseData().get(topicPartition).offsets; - if (offsets.size() != 1) - throw new IllegalStateException("This should not happen."); - long offset = offsets.get(0); - log.debug("Fetched offset {} for partition {}", offset, topicPartition); - return offset; - } else if (errorCode == Errors.NOT_LEADER_FOR_PARTITION.code() - || errorCode == Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) { - log.warn("Attempt to fetch offsets for partition {} failed due to obsolete leadership information, retrying.", - topicPartition); - awaitMetadataUpdate(); - } else { - log.error("Attempt to fetch offsets for partition {} failed due to: {}", - topicPartition, Errors.forCode(errorCode).exception().getMessage()); - awaitMetadataUpdate(); - } + RequestCompletionHandler completionHandler = new RequestCompletionHandler() { + @Override + public void onComplete(ClientResponse resp) { + handleListOffsetResponse(topicPartition, resp, future); } + }; + ClientRequest clientRequest = new ClientRequest(now, true, send, completionHandler); + this.client.send(clientRequest); + return future; + } else { + // We initiated a connect to the leader, but we need to poll to finish it. + return RequestFuture.pollNeeded(); + } + } + + /** + * Callback for the response of the list offset call above. + * @param topicPartition The partition that was fetched + * @param clientResponse The response from the server. + */ + private void handleListOffsetResponse(TopicPartition topicPartition, + ClientResponse clientResponse, + RequestFuture future) { + if (clientResponse.wasDisconnected()) { + future.retryAfterMetadataRefresh(); + } else { + ListOffsetResponse lor = new ListOffsetResponse(clientResponse.responseBody()); + short errorCode = lor.responseData().get(topicPartition).errorCode; + if (errorCode == Errors.NONE.code()) { + List offsets = lor.responseData().get(topicPartition).offsets; + if (offsets.size() != 1) + throw new IllegalStateException("This should not happen."); + long offset = offsets.get(0); + log.debug("Fetched offset {} for partition {}", offset, topicPartition); + + future.complete(offset); + } else if (errorCode == Errors.NOT_LEADER_FOR_PARTITION.code() + || errorCode == Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) { + log.warn("Attempt to fetch offsets for partition {} failed due to obsolete leadership information, retrying.", + topicPartition); + future.retryAfterMetadataRefresh(); } else { - log.debug("Leader for partition {} is not ready, retry fetching offsets", topicPartition); - client.poll(this.retryBackoffMs, now); + log.error("Attempt to fetch offsets for partition {} failed due to: {}", + topicPartition, Errors.forCode(errorCode).exception().getMessage()); + future.retryAfterMetadataRefresh(); } } } @@ -257,8 +240,10 @@ public class Fetcher { Map> fetchable = new HashMap>(); for (TopicPartition partition : subscriptions.assignedPartitions()) { Node node = cluster.leaderFor(partition); - // if there is a leader and no in-flight requests, issue a new fetch - if (node != null && this.client.inFlightRequestCount(node.idString()) == 0) { + if (node == null) { + metadata.requestUpdate(); + } else if (this.client.inFlightRequestCount(node.idString()) == 0) { + // if there is a leader and no in-flight requests, issue a new fetch Map fetch = fetchable.get(node.id()); if (fetch == null) { fetch = new HashMap(); @@ -327,7 +312,7 @@ public class Fetcher { } else if (partition.errorCode == Errors.OFFSET_OUT_OF_RANGE.code()) { // TODO: this could be optimized by grouping all out-of-range partitions log.info("Fetch offset {} is out of range, resetting offset", subscriptions.fetched(tp)); - resetOffset(tp); + subscriptions.needOffsetReset(tp); } else if (partition.errorCode == Errors.UNKNOWN.code()) { log.warn("Unknown error fetching data for topic-partition {}", tp); } else { @@ -356,17 +341,6 @@ public class Fetcher { return new ConsumerRecord(partition.topic(), partition.partition(), offset, key, value); } - /* - * Request a metadata update and wait until it has occurred - */ - private void awaitMetadataUpdate() { - int version = this.metadata.requestUpdate(); - do { - long now = time.milliseconds(); - this.client.poll(this.retryBackoffMs, now); - } while (this.metadata.version() == version); - } - private static class PartitionRecords { public long fetchOffset; public TopicPartition partition; @@ -379,9 +353,6 @@ public class Fetcher { } } - private static enum AutoOffsetResetStrategy { - LATEST, EARLIEST, NONE - } private class FetchManagerMetrics { public final Metrics metrics; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java index e7cfaaa..51eae19 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java @@ -42,4 +42,14 @@ public final class Heartbeat { public long lastHeartbeatSend() { return this.lastHeartbeatSend; } + + public long timeToNextHeartbeat(long now) { + long timeSinceLastHeartbeat = now - lastHeartbeatSend; + + long hbInterval = timeout / HEARTBEATS_PER_SESSION_INTERVAL; + if (timeSinceLastHeartbeat > hbInterval) + return 0; + else + return hbInterval - timeSinceLastHeartbeat; + } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFuture.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFuture.java new file mode 100644 index 0000000..13fc9af --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFuture.java @@ -0,0 +1,209 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals; + +/** + * Result of an asynchronous request through {@link org.apache.kafka.clients.KafkaClient}. To get the + * result of the request, you must use poll using {@link org.apache.kafka.clients.KafkaClient#poll(long, long)} + * until {@link #isDone()} returns true. Typical usage might look like this: + * + *

+ *     RequestFuture future = sendRequest();
+ *     while (!future.isDone()) {
+ *         client.poll(timeout, now);
+ *     }
+ *
+ *     switch (future.outcome()) {
+ *     case SUCCESS:
+ *         // handle request success
+ *         break;
+ *     case NEED_RETRY:
+ *         // retry after taking possible retry action
+ *         break;
+ *     case EXCEPTION:
+ *         // handle exception
+  *     }
+ * 
+ * + * When {@link #isDone()} returns true, there are three possible outcomes (obtained through {@link #outcome()}): + * + *
    + *
  1. {@link org.apache.kafka.clients.consumer.internals.RequestFuture.Outcome#SUCCESS}: If the request was + * successful, then you can use {@link #value()} to obtain the result.
  2. + *
  3. {@link org.apache.kafka.clients.consumer.internals.RequestFuture.Outcome#EXCEPTION}: If an unhandled exception + * was encountered, you can use {@link #exception()} to get it.
  4. + *
  5. {@link org.apache.kafka.clients.consumer.internals.RequestFuture.Outcome#NEED_RETRY}: The request may + * not have been successful, but the failure may be ephemeral and the caller just needs to try the request again. + * In this case, use {@link #retryAction()} to determine what action should be taken (if any) before + * retrying.
  6. + *
+ * + * @param Return type of the result (Can be Void if there is no response) + */ +public class RequestFuture { + public static final RequestFuture NEED_NEW_COORDINATOR = newRetryFuture(RetryAction.FIND_COORDINATOR); + public static final RequestFuture NEED_POLL = newRetryFuture(RetryAction.POLL); + public static final RequestFuture NEED_METADATA_REFRESH = newRetryFuture(RetryAction.REFRESH_METADATA); + + public enum RetryAction { + NOOP, // Retry immediately. + POLL, // Retry after calling poll (e.g. to finish a connection) + BACKOFF, // Retry after a delay + FIND_COORDINATOR, // Find a new coordinator before retrying + REFRESH_METADATA // Refresh metadata before retrying + } + + public enum Outcome { + SUCCESS, + NEED_RETRY, + EXCEPTION + } + + private Outcome outcome; + private RetryAction retryAction; + private T value; + private RuntimeException exception; + + /** + * Check whether the response is ready to be handled + * @return true if the response is ready, false otherwise + */ + public boolean isDone() { + return outcome != null; + } + + /** + * Get the value corresponding to this request (if it has one, as indicated by {@link #outcome()}). + * @return the value if it exists or null + */ + public T value() { + return value; + } + + /** + * Check if the request succeeded; + * @return true if a value is available, false otherwise + */ + public boolean succeeded() { + return outcome == Outcome.SUCCESS; + } + + /** + * Check if the request completed failed. + * @return true if the request failed (whether or not it can be retried) + */ + public boolean failed() { + return outcome != Outcome.SUCCESS; + } + + /** + * Return the error from this response (assuming {@link #succeeded()} has returned false. If the + * response is not ready or if there is no retryAction, null is returned. + * @return the error if it exists or null + */ + public RetryAction retryAction() { + return retryAction; + } + + /** + * Get the exception from a failed result. You should check that there is an exception + * with {@link #hasException()} before using this method. + * @return The exception if it exists or null + */ + public RuntimeException exception() { + return exception; + } + + /** + * Check whether there was an exception. + * @return true if this request failed with an exception + */ + public boolean hasException() { + return outcome == Outcome.EXCEPTION; + } + + /** + * Check the outcome of the future if it is ready. + * @return the outcome or null if the future is not finished + */ + public Outcome outcome() { + return outcome; + } + + /** + * The request failed, but should be retried using the provided retry action. + * @param retryAction The action that should be taken by the caller before retrying the request + */ + public void retry(RetryAction retryAction) { + this.outcome = Outcome.NEED_RETRY; + this.retryAction = retryAction; + } + + public void retryNow() { + retry(RetryAction.NOOP); + } + + public void retryAfterBackoff() { + retry(RetryAction.BACKOFF); + } + + public void retryWithNewCoordinator() { + retry(RetryAction.FIND_COORDINATOR); + } + + public void retryAfterMetadataRefresh() { + retry(RetryAction.REFRESH_METADATA); + } + + /** + * Complete the request successfully. After this call, {@link #succeeded()} will return true + * and the value can be obtained through {@link #value()}. + * @param value corresponding value (or null if there is none) + */ + public void complete(T value) { + this.outcome = Outcome.SUCCESS; + this.value = value; + } + + /** + * Raise an exception. The request will be marked as failed, and the caller can either + * handle the exception or throw it. + * @param e The exception that + */ + public void raise(RuntimeException e) { + this.outcome = Outcome.EXCEPTION; + this.exception = e; + } + + private static RequestFuture newRetryFuture(RetryAction retryAction) { + RequestFuture result = new RequestFuture(); + result.retry(retryAction); + return result; + } + + @SuppressWarnings("unchecked") + public static RequestFuture pollNeeded() { + return (RequestFuture) NEED_POLL; + } + + @SuppressWarnings("unchecked") + public static RequestFuture metadataRefreshNeeded() { + return (RequestFuture) NEED_METADATA_REFRESH; + } + + @SuppressWarnings("unchecked") + public static RequestFuture newCoordinatorNeeded() { + return (RequestFuture) NEED_NEW_COORDINATOR; + } + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java index cee7541..6837453 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java @@ -12,14 +12,15 @@ */ package org.apache.kafka.clients.consumer.internals; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.common.TopicPartition; + import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; -import org.apache.kafka.common.TopicPartition; - /** * A class for tracking the topics, partitions, and offsets for the consumer */ @@ -49,7 +50,14 @@ public class SubscriptionState { /* do we need to request the latest committed offsets from the coordinator? */ private boolean needsFetchCommittedOffsets; - public SubscriptionState() { + /* Partitions that need to be reset before fetching */ + private Map resetPartitions; + + /* Default offset reset strategy */ + private OffsetResetStrategy offsetResetStrategy; + + public SubscriptionState(OffsetResetStrategy offsetResetStrategy) { + this.offsetResetStrategy = offsetResetStrategy; this.subscribedTopics = new HashSet(); this.subscribedPartitions = new HashSet(); this.assignedPartitions = new HashSet(); @@ -58,6 +66,7 @@ public class SubscriptionState { this.committed = new HashMap(); this.needsPartitionAssignment = false; this.needsFetchCommittedOffsets = true; // initialize to true for the consumers to fetch offset upon starting up + this.resetPartitions = new HashMap(); } public void subscribe(String topic) { @@ -102,12 +111,14 @@ public class SubscriptionState { this.committed.remove(tp); this.fetched.remove(tp); this.consumed.remove(tp); + this.resetPartitions.remove(tp); } public void clearAssignment() { this.assignedPartitions.clear(); this.committed.clear(); this.fetched.clear(); + this.consumed.clear(); this.needsPartitionAssignment = !subscribedTopics().isEmpty(); } @@ -145,6 +156,7 @@ public class SubscriptionState { public void seek(TopicPartition tp, long offset) { fetched(tp, offset); consumed(tp, offset); + resetPartitions.remove(tp); } public Set assignedPartitions() { @@ -169,6 +181,28 @@ public class SubscriptionState { return this.consumed; } + public void needOffsetReset(TopicPartition partition, OffsetResetStrategy offsetResetStrategy) { + this.resetPartitions.put(partition, offsetResetStrategy); + this.fetched.remove(partition); + this.consumed.remove(partition); + } + + public void needOffsetReset(TopicPartition partition) { + needOffsetReset(partition, offsetResetStrategy); + } + + public boolean isOffsetResetNeeded(TopicPartition partition) { + return resetPartitions.containsKey(partition); + } + + public boolean isOffsetResetNeeded() { + return !resetPartitions.isEmpty(); + } + + public OffsetResetStrategy resetStrategy(TopicPartition partition) { + return resetPartitions.get(partition); + } + public boolean hasAllFetchPositions() { return this.fetched.size() >= this.assignedPartitions.size(); } @@ -192,4 +226,5 @@ public class SubscriptionState { this.needsPartitionAssignment = false; } + } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java index f73eedb..af9993c 100755 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -182,6 +182,21 @@ public class Utils { } /** + * Get the minimum of some long values. + * @param first Used to ensure at least one value + * @param rest The rest of longs to compare + * @return The minimum of all passed argument. + */ + public static long min(long first, long ... rest) { + long min = first; + for (int i = 0; i < rest.length; i++) { + if (rest[i] < min) + min = rest[i]; + } + return min; + } + + /** * Get the length for UTF8-encoding a string without encoding it first * * @param s The string to calculate the length for diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java index 677edd3..26b6b40 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java @@ -25,7 +25,7 @@ import org.junit.Test; public class MockConsumerTest { - private MockConsumer consumer = new MockConsumer(); + private MockConsumer consumer = new MockConsumer(OffsetResetStrategy.EARLIEST); @Test public void testSimpleMock() { 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 1454ab7..613b192 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 @@ -17,10 +17,11 @@ package org.apache.kafka.clients.consumer.internals; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.MockClient; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; @@ -49,24 +50,20 @@ public class CoordinatorTest { private String topicName = "test"; private String groupId = "test-group"; private TopicPartition tp = new TopicPartition(topicName, 0); - private long retryBackoffMs = 0L; private int sessionTimeoutMs = 10; private String rebalanceStrategy = "not-matter"; private MockTime time = new MockTime(); private MockClient client = new MockClient(time); - private Metadata metadata = new Metadata(0, Long.MAX_VALUE); private Cluster cluster = TestUtils.singletonCluster(topicName, 1); private Node node = cluster.nodes().get(0); - private SubscriptionState subscriptions = new SubscriptionState(); + private SubscriptionState subscriptions = new SubscriptionState(OffsetResetStrategy.EARLIEST); private Metrics metrics = new Metrics(time); private Map metricTags = new LinkedHashMap(); private Coordinator coordinator = new Coordinator(client, groupId, - retryBackoffMs, sessionTimeoutMs, rebalanceStrategy, - metadata, subscriptions, metrics, "consumer" + groupId, @@ -75,13 +72,14 @@ public class CoordinatorTest { @Before public void setup() { - metadata.update(cluster, time.milliseconds()); client.setNode(node); } @Test public void testNormalHeartbeat() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); // normal heartbeat time.sleep(sessionTimeoutMs); @@ -94,6 +92,8 @@ public class CoordinatorTest { @Test public void testCoordinatorNotAvailable() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); // consumer_coordinator_not_available will mark coordinator as unknown time.sleep(sessionTimeoutMs); @@ -108,6 +108,8 @@ public class CoordinatorTest { @Test public void testNotCoordinator() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); // not_coordinator will mark coordinator as unknown time.sleep(sessionTimeoutMs); @@ -122,6 +124,8 @@ public class CoordinatorTest { @Test public void testIllegalGeneration() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); // illegal_generation will cause re-partition subscriptions.subscribe(topicName); @@ -139,6 +143,8 @@ public class CoordinatorTest { @Test public void testCoordinatorDisconnect() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); // coordinator disconnect will mark coordinator as unknown time.sleep(sessionTimeoutMs); @@ -152,39 +158,67 @@ public class CoordinatorTest { @Test public void testNormalJoinGroup() { + subscriptions.subscribe(topicName); + subscriptions.needReassignment(); + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); // normal join group client.prepareResponse(joinGroupResponse(1, "consumer", Collections.singletonList(tp), Errors.NONE.code())); - assertEquals(Collections.singletonList(tp), - coordinator.assignPartitions(Collections.singletonList(topicName), time.milliseconds())); - assertEquals(0, client.inFlightRequestCount()); + coordinator.assignPartitions(time.milliseconds()); + client.poll(0, time.milliseconds()); + + assertFalse(subscriptions.partitionAssignmentNeeded()); + assertEquals(Collections.singleton(tp), subscriptions.assignedPartitions()); } @Test public void testReJoinGroup() { + subscriptions.subscribe(topicName); + subscriptions.needReassignment(); + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); + assertTrue(subscriptions.partitionAssignmentNeeded()); // diconnected from original coordinator will cause re-discover and join again client.prepareResponse(joinGroupResponse(1, "consumer", Collections.singletonList(tp), Errors.NONE.code()), true); + coordinator.assignPartitions(time.milliseconds()); + client.poll(0, time.milliseconds()); + assertTrue(subscriptions.partitionAssignmentNeeded()); + + // rediscover the coordinator client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); + + // try assigning partitions again client.prepareResponse(joinGroupResponse(1, "consumer", Collections.singletonList(tp), Errors.NONE.code())); - assertEquals(Collections.singletonList(tp), - coordinator.assignPartitions(Collections.singletonList(topicName), time.milliseconds())); - assertEquals(0, client.inFlightRequestCount()); + coordinator.assignPartitions(time.milliseconds()); + client.poll(0, time.milliseconds()); + assertFalse(subscriptions.partitionAssignmentNeeded()); + assertEquals(Collections.singleton(tp), subscriptions.assignedPartitions()); } @Test public void testCommitOffsetNormal() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); - // sync commit + // With success flag client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); - coordinator.commitOffsets(Collections.singletonMap(tp, 100L), true, time.milliseconds()); + RequestFuture result = coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds()); + assertEquals(1, client.poll(0, time.milliseconds()).size()); + assertTrue(result.isDone()); + assertTrue(result.succeeded()); - // async commit - coordinator.commitOffsets(Collections.singletonMap(tp, 100L), false, time.milliseconds()); + // Without success flag + coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds()); client.respond(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); assertEquals(1, client.poll(0, time.milliseconds()).size()); } @@ -192,34 +226,55 @@ public class CoordinatorTest { @Test public void testCommitOffsetError() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); // async commit with coordinator not available client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code()))); - coordinator.commitOffsets(Collections.singletonMap(tp, 100L), false, time.milliseconds()); + coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds()); assertEquals(1, client.poll(0, time.milliseconds()).size()); assertTrue(coordinator.coordinatorUnknown()); // resume client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); // async commit with not coordinator client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NOT_COORDINATOR_FOR_CONSUMER.code()))); - coordinator.commitOffsets(Collections.singletonMap(tp, 100L), false, time.milliseconds()); + coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds()); assertEquals(1, client.poll(0, time.milliseconds()).size()); assertTrue(coordinator.coordinatorUnknown()); // resume client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); // sync commit with not_coordinator client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NOT_COORDINATOR_FOR_CONSUMER.code()))); client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); - coordinator.commitOffsets(Collections.singletonMap(tp, 100L), true, time.milliseconds()); + RequestFuture result = coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds()); + assertEquals(1, client.poll(0, time.milliseconds()).size()); + assertTrue(result.isDone()); + assertEquals(RequestFuture.RetryAction.FIND_COORDINATOR, result.retryAction()); // sync commit with coordinator disconnected client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())), true); - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); - coordinator.commitOffsets(Collections.singletonMap(tp, 100L), true, time.milliseconds()); + result = coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds()); + + assertEquals(0, client.poll(0, time.milliseconds()).size()); + assertTrue(result.isDone()); + assertEquals(RequestFuture.RetryAction.FIND_COORDINATOR, result.retryAction()); + + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); + + result = coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds()); + assertEquals(1, client.poll(0, time.milliseconds()).size()); + assertTrue(result.isDone()); + assertTrue(result.succeeded()); } @@ -227,33 +282,70 @@ public class CoordinatorTest { public void testFetchOffset() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); // normal fetch client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", 100L)); - assertEquals(100L, (long) coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()).get(tp)); + RequestFuture> result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); + client.poll(0, time.milliseconds()); + assertTrue(result.isDone()); + assertEquals(100L, (long) result.value().get(tp)); // fetch with loading in progress client.prepareResponse(offsetFetchResponse(tp, Errors.OFFSET_LOAD_IN_PROGRESS.code(), "", 100L)); client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", 100L)); - assertEquals(100L, (long) coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()).get(tp)); + + result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); + client.poll(0, time.milliseconds()); + assertTrue(result.isDone()); + assertTrue(result.failed()); + assertEquals(RequestFuture.RetryAction.BACKOFF, result.retryAction()); + + result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); + client.poll(0, time.milliseconds()); + assertTrue(result.isDone()); + assertEquals(100L, (long) result.value().get(tp)); // fetch with not coordinator client.prepareResponse(offsetFetchResponse(tp, Errors.NOT_COORDINATOR_FOR_CONSUMER.code(), "", 100L)); client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", 100L)); - assertEquals(100L, (long) coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()).get(tp)); + + result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); + client.poll(0, time.milliseconds()); + assertTrue(result.isDone()); + assertTrue(result.failed()); + assertEquals(RequestFuture.RetryAction.FIND_COORDINATOR, result.retryAction()); + + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); + + result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); + client.poll(0, time.milliseconds()); + assertTrue(result.isDone()); + assertEquals(100L, (long) result.value().get(tp)); // fetch with no fetchable offsets client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", -1L)); - assertEquals(0, coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()).size()); + result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); + client.poll(0, time.milliseconds()); + assertTrue(result.isDone()); + assertTrue(result.value().isEmpty()); // fetch with offset topic unknown client.prepareResponse(offsetFetchResponse(tp, Errors.UNKNOWN_TOPIC_OR_PARTITION.code(), "", 100L)); - assertEquals(0, coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()).size()); + result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); + client.poll(0, time.milliseconds()); + assertTrue(result.isDone()); + assertTrue(result.value().isEmpty()); // fetch with offset -1 client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", -1L)); - assertEquals(0, coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()).size()); + result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); + client.poll(0, time.milliseconds()); + assertTrue(result.isDone()); + assertTrue(result.value().isEmpty()); } private Struct consumerMetadataResponse(Node node, short error) { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java index 4195410..405efdc 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -16,11 +16,10 @@ */ package org.apache.kafka.clients.consumer.internals; -import static org.junit.Assert.assertEquals; - import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.MockClient; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; @@ -30,10 +29,11 @@ import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.requests.FetchResponse; -import org.apache.kafka.common.requests.ListOffsetResponse; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.test.TestUtils; +import org.junit.Before; +import org.junit.Test; import java.nio.ByteBuffer; import java.util.Collections; @@ -41,37 +41,33 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import org.junit.Before; -import org.junit.Test; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; public class FetcherTest { private String topicName = "test"; private String groupId = "test-group"; private TopicPartition tp = new TopicPartition(topicName, 0); - private long retryBackoffMs = 0L; private int minBytes = 1; private int maxWaitMs = 0; private int fetchSize = 1000; - private String offsetReset = "EARLIEST"; private MockTime time = new MockTime(); private MockClient client = new MockClient(time); private Metadata metadata = new Metadata(0, Long.MAX_VALUE); private Cluster cluster = TestUtils.singletonCluster(topicName, 1); private Node node = cluster.nodes().get(0); - private SubscriptionState subscriptions = new SubscriptionState(); + private SubscriptionState subscriptions = new SubscriptionState(OffsetResetStrategy.EARLIEST); private Metrics metrics = new Metrics(time); private Map metricTags = new LinkedHashMap(); private MemoryRecords records = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), CompressionType.NONE); private Fetcher fetcher = new Fetcher(client, - retryBackoffMs, minBytes, maxWaitMs, fetchSize, true, // check crc - offsetReset, new ByteArrayDeserializer(), new ByteArrayDeserializer(), metadata, @@ -140,11 +136,11 @@ public class FetcherTest { subscriptions.fetched(tp, 5); fetcher.initFetches(cluster, time.milliseconds()); client.respond(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L)); - client.prepareResponse(listOffsetResponse(Collections.singletonList(0L), Errors.NONE.code())); client.poll(0, time.milliseconds()); + assertTrue(subscriptions.isOffsetResetNeeded(tp)); assertEquals(0, fetcher.fetchedRecords().size()); - assertEquals(0L, (long) subscriptions.fetched(tp)); - assertEquals(0L, (long) subscriptions.consumed(tp)); + assertEquals(null, subscriptions.fetched(tp)); + assertEquals(null, subscriptions.consumed(tp)); } @Test @@ -157,11 +153,11 @@ public class FetcherTest { // fetch with out of range fetcher.initFetches(cluster, time.milliseconds()); client.respond(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L)); - client.prepareResponse(listOffsetResponse(Collections.singletonList(0L), Errors.NONE.code())); client.poll(0, time.milliseconds()); + assertTrue(subscriptions.isOffsetResetNeeded(tp)); assertEquals(0, fetcher.fetchedRecords().size()); - assertEquals(0L, (long) subscriptions.fetched(tp)); - assertEquals(0L, (long) subscriptions.consumed(tp)); + assertEquals(null, subscriptions.fetched(tp)); + assertEquals(null, subscriptions.consumed(tp)); } private Struct fetchResponse(ByteBuffer buffer, short error, long hw) { @@ -169,9 +165,5 @@ public class FetcherTest { return response.toStruct(); } - private Struct listOffsetResponse(List offsets, short error) { - ListOffsetResponse response = new ListOffsetResponse(Collections.singletonMap(tp, new ListOffsetResponse.PartitionData(error, offsets))); - return response.toStruct(); - } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatTest.java index ecc78ce..ee1ede0 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatTest.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.utils.MockTime; import org.junit.Test; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -42,4 +43,12 @@ public class HeartbeatTest { time.sleep(timeout / (2 * Heartbeat.HEARTBEATS_PER_SESSION_INTERVAL)); assertFalse(heartbeat.shouldHeartbeat(time.milliseconds())); } + + @Test + public void testTimeToNextHeartbeat() { + heartbeat.sentHeartbeat(0); + assertEquals(100, heartbeat.timeToNextHeartbeat(0)); + assertEquals(0, heartbeat.timeToNextHeartbeat(100)); + assertEquals(0, heartbeat.timeToNextHeartbeat(200)); + } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java index e000cf8..319751c 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java @@ -22,12 +22,13 @@ import static java.util.Arrays.asList; import java.util.Collections; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; import org.apache.kafka.common.TopicPartition; import org.junit.Test; public class SubscriptionStateTest { - private final SubscriptionState state = new SubscriptionState(); + private final SubscriptionState state = new SubscriptionState(OffsetResetStrategy.EARLIEST); private final TopicPartition tp0 = new TopicPartition("test", 0); private final TopicPartition tp1 = new TopicPartition("test", 1); @@ -43,7 +44,21 @@ public class SubscriptionStateTest { assertTrue(state.assignedPartitions().isEmpty()); assertAllPositions(tp0, null); } - + + @Test + public void partitionReset() { + state.subscribe(tp0); + state.seek(tp0, 5); + assertEquals(5L, (long) state.fetched(tp0)); + assertEquals(5L, (long) state.consumed(tp0)); + state.needOffsetReset(tp0); + assertTrue(state.isOffsetResetNeeded()); + assertTrue(state.isOffsetResetNeeded(tp0)); + assertEquals(null, state.fetched(tp0)); + assertEquals(null, state.consumed(tp0)); + } + + @Test public void topicSubscription() { state.subscribe("test"); assertEquals(1, state.subscribedTopics().size()); diff --git a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java index 2ebe3c2..e7951d8 100755 --- a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java @@ -100,4 +100,12 @@ public class UtilsTest { buffer = ByteBuffer.wrap(myvar).asReadOnlyBuffer(); this.subTest(buffer); } + + @Test + public void testMin() { + assertEquals(1, Utils.min(1)); + assertEquals(1, Utils.min(1, 2, 3)); + assertEquals(1, Utils.min(2, 1, 3)); + assertEquals(1, Utils.min(2, 3, 1)); + } } \ No newline at end of file -- 1.7.12.4 From 6d4991e312548bc81e0f6b790a370b6165a92df9 Mon Sep 17 00:00:00 2001 From: Jeff Maxwell Date: Tue, 23 Jun 2015 10:52:21 -0700 Subject: [PATCH 026/120] KAFKA-2294; javadoc compile error due to illegal

, build failing (jdk 8); patched by Jeff Maxwell; reviewed by Jakob Homan --- .../src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 5a37580..5671a3f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -522,7 +522,8 @@ public class KafkaProducer implements Producer { * If close() is called from {@link Callback}, a warning message will be logged and close(0, TimeUnit.MILLISECONDS) * will be called instead. We do this because the sender thread would otherwise try to join itself and * block forever. - *

+ *

+ * * @throws InterruptException If the thread is interrupted while blocked */ @Override -- 1.7.12.4 From 9ff5b27bc572850863f58a3767da3c72aa2a9831 Mon Sep 17 00:00:00 2001 From: Tao Xiao Date: Mon, 29 Jun 2015 18:47:47 -0700 Subject: [PATCH 027/120] KAFKA-2281: avoid unnecessary value copying if logAsString is false; reviewed by Guozhang Wang --- .../clients/producer/internals/ErrorLoggingCallback.java | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ErrorLoggingCallback.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ErrorLoggingCallback.java index 678d1c6..747e29f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ErrorLoggingCallback.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ErrorLoggingCallback.java @@ -23,12 +23,18 @@ public class ErrorLoggingCallback implements Callback { private String topic; private byte[] key; private byte[] value; + private int valueLength; private boolean logAsString; public ErrorLoggingCallback(String topic, byte[] key, byte[] value, boolean logAsString) { this.topic = topic; this.key = key; - this.value = value; + + if (logAsString) { + this.value = value; + } + + this.valueLength = value == null ? -1 : value.length; this.logAsString = logAsString; } @@ -36,10 +42,10 @@ public class ErrorLoggingCallback implements Callback { if (e != null) { String keyString = (key == null) ? "null" : logAsString ? new String(key) : key.length + " bytes"; - String valueString = (value == null) ? "null" : - logAsString ? new String(value) : value.length + " bytes"; + String valueString = (valueLength == -1) ? "null" : + logAsString ? new String(value) : valueLength + " bytes"; log.error("Error when sending message to topic {} with key: {}, value: {} with error: {}", - topic, keyString, valueString, e.getMessage()); + topic, keyString, valueString, e.getMessage()); } } } -- 1.7.12.4 From 14e0ce0a47fb7f6ae6dab085b2ea9d5a1f644433 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Wed, 1 Jul 2015 15:28:11 -0700 Subject: [PATCH 028/120] KAFKA-2168: minor follow-up patch; reviewed by Guozhang Wang --- .../apache/kafka/clients/consumer/KafkaConsumer.java | 17 ++++++++++------- .../kafka/clients/consumer/internals/Coordinator.java | 10 +++++----- .../integration/kafka/api/ConsumerBounceTest.scala | 14 ++++++-------- 3 files changed, 21 insertions(+), 20 deletions(-) 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 9be8fbc..1f0e515 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 @@ -42,6 +42,7 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.ConcurrentModificationException; +import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -332,6 +333,7 @@ import static org.apache.kafka.common.utils.Utils.min; * } * } * + * // Shutdown hook which can be called from a separate thread * public void shutdown() { * closed.set(true); * consumer.wakeup(); @@ -417,7 +419,7 @@ public class KafkaConsumer implements Consumer { // and is used to prevent multi-threaded access private final AtomicReference currentThread = new AtomicReference(); // refcount is used to allow reentrant access by the thread who has acquired currentThread - private int refcount = 0; // reference count for reentrant access + private final AtomicInteger refcount = new AtomicInteger(0); // TODO: This timeout controls how long we should wait before retrying a request. We should be able // to leverage the work of KAFKA-2120 to get this value from configuration. @@ -795,7 +797,7 @@ public class KafkaConsumer implements Consumer { * rebalance and also on startup. As such, if you need to store offsets in anything other than Kafka, this API * should not be used. *

- * A non-blocking commit will attempt to commit offsets asychronously. No error will be thrown if the commit fails. + * A non-blocking commit will attempt to commit offsets asynchronously. No error will be thrown if the commit fails. * A blocking commit will wait for a response acknowledging the commit. In the event of an error it will retry until * the commit succeeds. * @@ -832,7 +834,9 @@ public class KafkaConsumer implements Consumer { public void commit(CommitType commitType) { acquire(); try { - commit(this.subscriptions.allConsumed(), commitType); + // Need defensive copy to ensure offsets are not removed before completion (e.g. in rebalance) + Map allConsumed = new HashMap(this.subscriptions.allConsumed()); + commit(allConsumed, commitType); } finally { release(); } @@ -978,10 +982,9 @@ public class KafkaConsumer implements Consumer { @Override public void close() { - if (closed) return; - acquire(); try { + if (closed) return; close(false); } finally { release(); @@ -1355,14 +1358,14 @@ public class KafkaConsumer implements Consumer { Long threadId = Thread.currentThread().getId(); if (!threadId.equals(currentThread.get()) && !currentThread.compareAndSet(null, threadId)) throw new ConcurrentModificationException("KafkaConsumer is not safe for multi-threaded access"); - refcount++; + refcount.incrementAndGet(); } /** * Release the light lock protecting the consumer from multi-threaded access. */ private void release() { - if (--refcount == 0) + if (refcount.decrementAndGet() == 0) currentThread.set(null); } } 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 6c26667..68b4cb1 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 @@ -217,7 +217,7 @@ public final class Coordinator { OffsetCommitRequest.DEFAULT_RETENTION_TIME, offsetData); - RequestCompletionHandler handler = new CommitOffsetCompletionHandler(offsets, future); + RequestCompletionHandler handler = new OffsetCommitCompletionHandler(offsets, future); sendCoordinator(ApiKeys.OFFSET_COMMIT, req.toStruct(), handler, now); } @@ -261,14 +261,14 @@ public final class Coordinator { RequestCompletionHandler completionHandler = new RequestCompletionHandler() { @Override public void onComplete(ClientResponse resp) { - handleOffsetResponse(resp, future); + handleOffsetFetchResponse(resp, future); } }; sendCoordinator(ApiKeys.OFFSET_FETCH, request.toStruct(), completionHandler, now); return future; } - private void handleOffsetResponse(ClientResponse resp, RequestFuture> future) { + private void handleOffsetFetchResponse(ClientResponse resp, RequestFuture> future) { if (resp.wasDisconnected()) { handleCoordinatorDisconnect(resp); future.retryWithNewCoordinator(); @@ -471,12 +471,12 @@ public final class Coordinator { } } - private class CommitOffsetCompletionHandler implements RequestCompletionHandler { + private class OffsetCommitCompletionHandler implements RequestCompletionHandler { private final Map offsets; private final RequestFuture future; - public CommitOffsetCompletionHandler(Map offsets, RequestFuture future) { + public OffsetCommitCompletionHandler(Map offsets, RequestFuture future) { this.offsets = offsets; this.future = future; } diff --git a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala index f56096b..b0750fa 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala @@ -14,14 +14,10 @@ package kafka.api import kafka.server.KafkaConfig -import org.apache.kafka.clients.producer.ProducerConfig -import org.apache.kafka.clients.producer.ProducerRecord -import org.apache.kafka.clients.consumer.ConsumerConfig -import org.apache.kafka.clients.consumer.CommitType +import kafka.utils.{Logging, ShutdownableThread, TestUtils} +import org.apache.kafka.clients.consumer._ +import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord} import org.apache.kafka.common.TopicPartition - -import kafka.utils.{ShutdownableThread, TestUtils, Logging} - import org.junit.Assert._ import scala.collection.JavaConversions._ @@ -85,9 +81,11 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging { assertEquals(consumed.toLong, record.offset()) consumed += 1 } + consumer.commit(CommitType.SYNC) + assertEquals(consumer.position(tp), consumer.committed(tp)) - if (consumed == numRecords) { + if (consumer.position(tp) == numRecords) { consumer.seekToBeginning() consumed = 0 } -- 1.7.12.4 From 3f8480ccfb011eb43da774737597c597f703e11b Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 2 Jul 2015 11:41:51 -0700 Subject: [PATCH 029/120] KAFKA-1740: merge offset manager into consumer coordinator; reviewed by Onur Karaman and Jason Gustafson --- .../clients/consumer/internals/Coordinator.java | 27 +++- .../org/apache/kafka/common/protocol/Errors.java | 6 +- .../common/requests/OffsetCommitResponse.java | 8 +- .../kafka/common/requests/OffsetFetchRequest.java | 3 - .../kafka/common/requests/OffsetFetchResponse.java | 5 +- .../consumer/internals/CoordinatorTest.java | 7 - core/src/main/scala/kafka/admin/TopicCommand.scala | 4 +- core/src/main/scala/kafka/cluster/Partition.scala | 16 +- .../kafka/common/OffsetMetadataAndError.scala | 14 +- core/src/main/scala/kafka/common/Topic.scala | 4 +- .../kafka/coordinator/ConsumerCoordinator.scala | 170 +++++++++++++++++++-- .../kafka/coordinator/CoordinatorMetadata.scala | 4 +- core/src/main/scala/kafka/server/KafkaApis.scala | 69 +++++---- core/src/main/scala/kafka/server/KafkaServer.scala | 26 +--- .../main/scala/kafka/server/OffsetManager.scala | 52 +++---- .../main/scala/kafka/server/ReplicaManager.scala | 95 +++++++----- .../scala/integration/kafka/api/ConsumerTest.scala | 7 +- .../kafka/api/IntegrationTestHarness.scala | 9 +- .../scala/unit/kafka/admin/TopicCommandTest.scala | 8 +- .../unit/kafka/consumer/TopicFilterTest.scala | 9 +- .../ConsumerCoordinatorResponseTest.scala | 9 +- .../coordinator/CoordinatorMetadataTest.scala | 2 +- .../scala/unit/kafka/server/OffsetCommitTest.scala | 10 +- 23 files changed, 357 insertions(+), 207 deletions(-) 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 68b4cb1..c1c8172 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 @@ -290,9 +290,10 @@ public final class Coordinator { // re-discover the coordinator and retry coordinatorDead(); future.retryWithNewCoordinator(); - } else if (data.errorCode == Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) { - // just ignore this partition - log.debug("Unknown topic or partition for " + tp); + } else if (data.errorCode == Errors.UNKNOWN_CONSUMER_ID.code() + || data.errorCode == Errors.ILLEGAL_GENERATION.code()) { + // need to re-join group + subscriptions.needReassignment(); } else { future.raise(new KafkaException("Unexpected error in fetch offset response: " + Errors.forCode(data.errorCode).exception().getMessage())); @@ -499,13 +500,23 @@ public final class Coordinator { || errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { coordinatorDead(); future.retryWithNewCoordinator(); - } else { + } else if (errorCode == Errors.OFFSET_METADATA_TOO_LARGE.code() + || errorCode == Errors.INVALID_COMMIT_OFFSET_SIZE.code()) { // do not need to throw the exception but just log the error - future.retryAfterBackoff(); log.error("Error committing partition {} at offset {}: {}", - tp, - offset, - Errors.forCode(errorCode).exception().getMessage()); + tp, + offset, + Errors.forCode(errorCode).exception().getMessage()); + } else if (errorCode == Errors.UNKNOWN_CONSUMER_ID.code() + || errorCode == Errors.ILLEGAL_GENERATION.code()) { + // need to re-join group + subscriptions.needReassignment(); + } else { + // re-throw the exception as these should not happen + log.error("Error committing partition {} at offset {}: {}", + tp, + offset, + Errors.forCode(errorCode).exception().getMessage()); } } diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java index 5b898c8..4c0ecc3 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java @@ -77,7 +77,11 @@ public enum Errors { UNKNOWN_CONSUMER_ID(25, new ApiException("The coordinator is not aware of this consumer.")), INVALID_SESSION_TIMEOUT(26, - new ApiException("The session timeout is not within an acceptable range.")); + new ApiException("The session timeout is not within an acceptable range.")), + COMMITTING_PARTITIONS_NOT_ASSIGNED(27, + new ApiException("Some of the committing partitions are not assigned the committer")), + INVALID_COMMIT_OFFSET_SIZE(28, + new ApiException("The committing offset data size is not valid")); private static Map, Errors> classToError = new HashMap, Errors>(); private static Map codeToError = new HashMap(); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java index 70844d6..a163333 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java @@ -41,7 +41,13 @@ public class OffsetCommitResponse extends AbstractRequestResponse { /** * Possible error code: * - * TODO + * OFFSET_METADATA_TOO_LARGE (12) + * CONSUMER_COORDINATOR_NOT_AVAILABLE (15) + * NOT_COORDINATOR_FOR_CONSUMER (16) + * ILLEGAL_GENERATION (22) + * UNKNOWN_CONSUMER_ID (25) + * COMMITTING_PARTITIONS_NOT_ASSIGNED (27) + * INVALID_COMMIT_OFFSET_SIZE (28) */ private final Map responseData; 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 b5e8a0f..6ee7597 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 @@ -42,9 +42,6 @@ public class OffsetFetchRequest extends AbstractRequest { // partition level field names private static final String PARTITION_KEY_NAME = "partition"; - public static final int DEFAULT_GENERATION_ID = -1; - public static final String DEFAULT_CONSUMER_ID = ""; - private final String groupId; private final List partitions; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java index 512a0ef..3dc8521 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java @@ -47,10 +47,11 @@ public class OffsetFetchResponse extends AbstractRequestResponse { /** * Possible error code: * - * UNKNOWN_TOPIC_OR_PARTITION (3) + * UNKNOWN_TOPIC_OR_PARTITION (3) <- only for request v0 * OFFSET_LOAD_IN_PROGRESS (14) * NOT_COORDINATOR_FOR_CONSUMER (16) - * NO_OFFSETS_FETCHABLE (23) + * ILLEGAL_GENERATION (22) + * UNKNOWN_CONSUMER_ID (25) */ private final Map responseData; 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 613b192..d085fe5 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 @@ -333,13 +333,6 @@ public class CoordinatorTest { assertTrue(result.isDone()); assertTrue(result.value().isEmpty()); - // fetch with offset topic unknown - client.prepareResponse(offsetFetchResponse(tp, Errors.UNKNOWN_TOPIC_OR_PARTITION.code(), "", 100L)); - result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); - client.poll(0, time.milliseconds()); - assertTrue(result.isDone()); - assertTrue(result.value().isEmpty()); - // fetch with offset -1 client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", -1L)); result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala index dacbdd0..a2ecb96 100755 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -27,8 +27,8 @@ import scala.collection._ import scala.collection.JavaConversions._ import kafka.log.LogConfig import kafka.consumer.Whitelist -import kafka.server.OffsetManager import org.apache.kafka.common.utils.Utils +import kafka.coordinator.ConsumerCoordinator object TopicCommand { @@ -111,7 +111,7 @@ object TopicCommand { println("Updated config for topic \"%s\".".format(topic)) } if(opts.options.has(opts.partitionsOpt)) { - if (topic == OffsetManager.OffsetsTopicName) { + if (topic == ConsumerCoordinator.OffsetsTopicName) { throw new IllegalArgumentException("The number of partitions for the offsets topic cannot be changed.") } println("WARNING: If partitions are increased for a topic that has a key, the partition " + diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 0990938..2649090 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -22,7 +22,7 @@ import kafka.utils.CoreUtils.{inReadLock,inWriteLock} import kafka.admin.AdminUtils import kafka.api.{PartitionStateInfo, LeaderAndIsr} import kafka.log.LogConfig -import kafka.server.{TopicPartitionOperationKey, LogOffsetMetadata, OffsetManager, LogReadResult, ReplicaManager} +import kafka.server.{TopicPartitionOperationKey, LogOffsetMetadata, LogReadResult, ReplicaManager} import kafka.metrics.KafkaMetricsGroup import kafka.controller.KafkaController import kafka.message.ByteBufferMessageSet @@ -160,8 +160,7 @@ class Partition(val topic: String, * and setting the new leader and ISR */ def makeLeader(controllerId: Int, - partitionStateInfo: PartitionStateInfo, correlationId: Int, - offsetManager: OffsetManager): Boolean = { + partitionStateInfo: PartitionStateInfo, correlationId: Int): Boolean = { inWriteLock(leaderIsrUpdateLock) { val allReplicas = partitionStateInfo.allReplicas val leaderIsrAndControllerEpoch = partitionStateInfo.leaderIsrAndControllerEpoch @@ -186,8 +185,6 @@ class Partition(val topic: String, if (r.brokerId != localBrokerId) r.updateLogReadResult(LogReadResult.UnknownLogReadResult)) // we may need to increment high watermark since ISR could be down to 1 maybeIncrementLeaderHW(newLeaderReplica) - if (topic == OffsetManager.OffsetsTopicName) - offsetManager.loadOffsetsFromLog(partitionId) true } } @@ -198,7 +195,7 @@ class Partition(val topic: String, */ def makeFollower(controllerId: Int, partitionStateInfo: PartitionStateInfo, - correlationId: Int, offsetManager: OffsetManager): Boolean = { + correlationId: Int): Boolean = { inWriteLock(leaderIsrUpdateLock) { val allReplicas = partitionStateInfo.allReplicas val leaderIsrAndControllerEpoch = partitionStateInfo.leaderIsrAndControllerEpoch @@ -215,13 +212,6 @@ class Partition(val topic: String, leaderEpoch = leaderAndIsr.leaderEpoch zkVersion = leaderAndIsr.zkVersion - leaderReplicaIdOpt.foreach { leaderReplica => - if (topic == OffsetManager.OffsetsTopicName && - /* if we are making a leader->follower transition */ - leaderReplica == localBrokerId) - offsetManager.removeOffsetsFromCacheForPartition(partitionId) - } - if (leaderReplicaIdOpt.isDefined && leaderReplicaIdOpt.get == newLeaderBrokerId) { false } diff --git a/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala b/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala index 6b4242c..deb48b1 100644 --- a/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala +++ b/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala @@ -17,6 +17,8 @@ package kafka.common +import org.apache.kafka.common.protocol.Errors + case class OffsetMetadata(offset: Long, metadata: String = OffsetMetadata.NoMetadata) { override def toString = "OffsetMetadata[%d,%s]" .format(offset, @@ -51,7 +53,7 @@ object OffsetAndMetadata { def apply(offset: Long) = new OffsetAndMetadata(OffsetMetadata(offset, OffsetMetadata.NoMetadata)) } -case class OffsetMetadataAndError(offsetMetadata: OffsetMetadata, error: Short = ErrorMapping.NoError) { +case class OffsetMetadataAndError(offsetMetadata: OffsetMetadata, error: Short = Errors.NONE.code) { def offset = offsetMetadata.offset def metadata = offsetMetadata.metadata @@ -60,10 +62,12 @@ case class OffsetMetadataAndError(offsetMetadata: OffsetMetadata, error: Short = } object OffsetMetadataAndError { - val NoOffset = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, ErrorMapping.NoError) - val OffsetsLoading = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, ErrorMapping.OffsetsLoadInProgressCode) - val UnknownTopicOrPartition = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, ErrorMapping.UnknownTopicOrPartitionCode) - val NotOffsetManagerForGroup = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, ErrorMapping.NotCoordinatorForConsumerCode) + val NoOffset = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, Errors.NONE.code) + val OffsetsLoading = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, Errors.OFFSET_LOAD_IN_PROGRESS.code) + val UnknownConsumer = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, Errors.UNKNOWN_CONSUMER_ID.code) + val NotCoordinatorForGroup = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, Errors.NOT_COORDINATOR_FOR_CONSUMER.code) + val UnknownTopicOrPartition = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, Errors.UNKNOWN_TOPIC_OR_PARTITION.code) + val IllegalGroupGenerationId = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, Errors.ILLEGAL_GENERATION.code) def apply(offset: Long) = new OffsetMetadataAndError(OffsetMetadata(offset, OffsetMetadata.NoMetadata), ErrorMapping.NoError) diff --git a/core/src/main/scala/kafka/common/Topic.scala b/core/src/main/scala/kafka/common/Topic.scala index ad75978..32595d6 100644 --- a/core/src/main/scala/kafka/common/Topic.scala +++ b/core/src/main/scala/kafka/common/Topic.scala @@ -18,7 +18,7 @@ package kafka.common import util.matching.Regex -import kafka.server.OffsetManager +import kafka.coordinator.ConsumerCoordinator object Topic { @@ -26,7 +26,7 @@ object Topic { private val maxNameLength = 255 private val rgx = new Regex(legalChars + "+") - val InternalTopics = Set(OffsetManager.OffsetsTopicName) + val InternalTopics = Set(ConsumerCoordinator.OffsetsTopicName) def validate(topic: String) { if (topic.length <= 0) diff --git a/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala b/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala index a385adb..476973b 100644 --- a/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala @@ -16,7 +16,9 @@ */ package kafka.coordinator -import kafka.common.TopicAndPartition +import kafka.common.{OffsetMetadataAndError, OffsetAndMetadata, TopicAndPartition} +import kafka.message.UncompressedCodec +import kafka.log.LogConfig import kafka.server._ import kafka.utils._ import org.apache.kafka.common.protocol.Errors @@ -24,7 +26,11 @@ import org.apache.kafka.common.requests.JoinGroupRequest import org.I0Itec.zkclient.ZkClient import java.util.concurrent.atomic.AtomicBoolean +import java.util.Properties +import scala.collection.{Map, Seq, immutable} +case class GroupManagerConfig(consumerMinSessionTimeoutMs: Int, + consumerMaxSessionTimeoutMs: Int) /** * ConsumerCoordinator handles consumer group and consumer offset management. @@ -33,11 +39,13 @@ import java.util.concurrent.atomic.AtomicBoolean * consumer groups. Consumer groups are assigned to coordinators based on their * group names. */ -class ConsumerCoordinator(val config: KafkaConfig, - val zkClient: ZkClient, - val offsetManager: OffsetManager) extends Logging { +class ConsumerCoordinator(val brokerId: Int, + val groupConfig: GroupManagerConfig, + val offsetConfig: OffsetManagerConfig, + private val offsetManager: OffsetManager, + zkClient: ZkClient) extends Logging { - this.logIdent = "[ConsumerCoordinator " + config.brokerId + "]: " + this.logIdent = "[ConsumerCoordinator " + brokerId + "]: " private val isActive = new AtomicBoolean(false) @@ -45,6 +53,22 @@ class ConsumerCoordinator(val config: KafkaConfig, private var rebalancePurgatory: DelayedOperationPurgatory[DelayedRebalance] = null private var coordinatorMetadata: CoordinatorMetadata = null + def this(brokerId: Int, + groupConfig: GroupManagerConfig, + offsetConfig: OffsetManagerConfig, + replicaManager: ReplicaManager, + zkClient: ZkClient, + scheduler: KafkaScheduler) = this(brokerId, groupConfig, offsetConfig, + new OffsetManager(offsetConfig, replicaManager, zkClient, scheduler), zkClient) + + def offsetsTopicConfigs: Properties = { + val props = new Properties + props.put(LogConfig.CleanupPolicyProp, LogConfig.Compact) + props.put(LogConfig.SegmentBytesProp, offsetConfig.offsetsTopicSegmentBytes.toString) + props.put(LogConfig.CompressionTypeProp, UncompressedCodec.name) + props + } + /** * NOTE: If a group lock and metadataLock are simultaneously needed, * be sure to acquire the group lock before metadataLock to prevent deadlock @@ -55,9 +79,9 @@ class ConsumerCoordinator(val config: KafkaConfig, */ def startup() { info("Starting up.") - heartbeatPurgatory = new DelayedOperationPurgatory[DelayedHeartbeat]("Heartbeat", config.brokerId) - rebalancePurgatory = new DelayedOperationPurgatory[DelayedRebalance]("Rebalance", config.brokerId) - coordinatorMetadata = new CoordinatorMetadata(config, zkClient, maybePrepareRebalance) + heartbeatPurgatory = new DelayedOperationPurgatory[DelayedHeartbeat]("Heartbeat", brokerId) + rebalancePurgatory = new DelayedOperationPurgatory[DelayedRebalance]("Rebalance", brokerId) + coordinatorMetadata = new CoordinatorMetadata(brokerId, zkClient, maybePrepareRebalance) isActive.set(true) info("Startup complete.") } @@ -69,6 +93,7 @@ class ConsumerCoordinator(val config: KafkaConfig, def shutdown() { info("Shutting down.") isActive.set(false) + offsetManager.shutdown() coordinatorMetadata.shutdown() heartbeatPurgatory.shutdown() rebalancePurgatory.shutdown() @@ -87,7 +112,8 @@ class ConsumerCoordinator(val config: KafkaConfig, responseCallback(Set.empty, consumerId, 0, Errors.NOT_COORDINATOR_FOR_CONSUMER.code) } else if (!PartitionAssignor.strategies.contains(partitionAssignmentStrategy)) { responseCallback(Set.empty, consumerId, 0, Errors.UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY.code) - } else if (sessionTimeoutMs < config.consumerMinSessionTimeoutMs || sessionTimeoutMs > config.consumerMaxSessionTimeoutMs) { + } else if (sessionTimeoutMs < groupConfig.consumerMinSessionTimeoutMs || + sessionTimeoutMs > groupConfig.consumerMaxSessionTimeoutMs) { responseCallback(Set.empty, consumerId, 0, Errors.INVALID_SESSION_TIMEOUT.code) } else { // only try to create the group if the group is not unknown AND @@ -196,6 +222,75 @@ class ConsumerCoordinator(val config: KafkaConfig, } } + def handleCommitOffsets(groupId: String, + consumerId: String, + generationId: Int, + offsetMetadata: immutable.Map[TopicAndPartition, OffsetAndMetadata], + responseCallback: immutable.Map[TopicAndPartition, Short] => Unit) { + if (!isActive.get) { + responseCallback(offsetMetadata.mapValues(_ => Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code)) + } else if (!isCoordinatorForGroup(groupId)) { + responseCallback(offsetMetadata.mapValues(_ => Errors.NOT_COORDINATOR_FOR_CONSUMER.code)) + } else { + val group = coordinatorMetadata.getGroup(groupId) + if (group == null) { + // if the group does not exist, it means this group is not relying + // on Kafka for partition management, and hence never send join-group + // request to the coordinator before; in this case blindly commit the offsets + offsetManager.storeOffsets(groupId, consumerId, generationId, offsetMetadata, responseCallback) + } else { + group synchronized { + if (group.is(Dead)) { + responseCallback(offsetMetadata.mapValues(_ => Errors.UNKNOWN_CONSUMER_ID.code)) + } else if (!group.has(consumerId)) { + responseCallback(offsetMetadata.mapValues(_ => Errors.UNKNOWN_CONSUMER_ID.code)) + } else if (generationId != group.generationId) { + responseCallback(offsetMetadata.mapValues(_ => Errors.ILLEGAL_GENERATION.code)) + } else if (!offsetMetadata.keySet.subsetOf(group.get(consumerId).assignedTopicPartitions)) { + responseCallback(offsetMetadata.mapValues(_ => Errors.COMMITTING_PARTITIONS_NOT_ASSIGNED.code)) + } else { + offsetManager.storeOffsets(groupId, consumerId, generationId, offsetMetadata, responseCallback) + } + } + } + } + } + + def handleFetchOffsets(groupId: String, + partitions: Seq[TopicAndPartition]): Map[TopicAndPartition, OffsetMetadataAndError] = { + if (!isActive.get) { + partitions.map {case topicAndPartition => (topicAndPartition, OffsetMetadataAndError.NotCoordinatorForGroup)}.toMap + } else if (!isCoordinatorForGroup(groupId)) { + partitions.map {case topicAndPartition => (topicAndPartition, OffsetMetadataAndError.NotCoordinatorForGroup)}.toMap + } else { + val group = coordinatorMetadata.getGroup(groupId) + if (group == null) { + // if the group does not exist, it means this group is not relying + // on Kafka for partition management, and hence never send join-group + // request to the coordinator before; in this case blindly fetch the offsets + offsetManager.getOffsets(groupId, partitions) + } else { + group synchronized { + if (group.is(Dead)) { + partitions.map {case topicAndPartition => (topicAndPartition, OffsetMetadataAndError.UnknownConsumer)}.toMap + } else { + offsetManager.getOffsets(groupId, partitions) + } + } + } + } + } + + def handleGroupImmigration(offsetTopicPartitionId: Int) = { + // TODO we may need to add more logic in KAFKA-2017 + offsetManager.loadOffsetsFromLog(offsetTopicPartitionId) + } + + def handleGroupEmigration(offsetTopicPartitionId: Int) = { + // TODO we may need to add more logic in KAFKA-2017 + offsetManager.removeOffsetsFromCacheForPartition(offsetTopicPartitionId) + } + /** * Complete existing DelayedHeartbeats for the given consumer and schedule the next one */ @@ -246,8 +341,7 @@ class ConsumerCoordinator(val config: KafkaConfig, private def prepareRebalance(group: ConsumerGroupMetadata) { group.transitionTo(PreparingRebalance) - group.generationId += 1 - info("Preparing to rebalance group %s generation %s".format(group.groupId, group.generationId)) + info("Preparing to rebalance group %s with old generation %s".format(group.groupId, group.generationId)) val rebalanceTimeout = group.rebalanceTimeout val delayedRebalance = new DelayedRebalance(this, group, rebalanceTimeout) @@ -259,7 +353,9 @@ class ConsumerCoordinator(val config: KafkaConfig, assert(group.notYetRejoinedConsumers == List.empty[ConsumerMetadata]) group.transitionTo(Rebalancing) - info("Rebalancing group %s generation %s".format(group.groupId, group.generationId)) + group.generationId += 1 + + info("Rebalancing group %s with new generation %s".format(group.groupId, group.generationId)) val assignedPartitionsPerConsumer = reassignPartitions(group) trace("Rebalance for group %s generation %s has assigned partitions: %s" @@ -275,8 +371,6 @@ class ConsumerCoordinator(val config: KafkaConfig, maybePrepareRebalance(group) } - private def isCoordinatorForGroup(groupId: String) = offsetManager.leaderIsLocal(offsetManager.partitionFor(groupId)) - private def reassignPartitions(group: ConsumerGroupMetadata) = { val assignor = PartitionAssignor.createInstance(group.partitionAssignmentStrategy) val topicsPerConsumer = group.topicsPerConsumer @@ -345,8 +439,54 @@ class ConsumerCoordinator(val config: KafkaConfig, } } - def onCompleteHeartbeat() {} + def onCompleteHeartbeat() { + // TODO: add metrics for complete heartbeats + } + + def partitionFor(group: String): Int = offsetManager.partitionFor(group) private def shouldKeepConsumerAlive(consumer: ConsumerMetadata, heartbeatDeadline: Long) = consumer.awaitingRebalanceCallback != null || consumer.latestHeartbeat + consumer.sessionTimeoutMs > heartbeatDeadline + + private def isCoordinatorForGroup(groupId: String) = offsetManager.leaderIsLocal(offsetManager.partitionFor(groupId)) +} + +object ConsumerCoordinator { + + val OffsetsTopicName = "__consumer_offsets" + + def create(config: KafkaConfig, + zkClient: ZkClient, + replicaManager: ReplicaManager, + kafkaScheduler: KafkaScheduler): ConsumerCoordinator = { + val offsetConfig = OffsetManagerConfig(maxMetadataSize = config.offsetMetadataMaxSize, + loadBufferSize = config.offsetsLoadBufferSize, + offsetsRetentionMs = config.offsetsRetentionMinutes * 60 * 1000L, + offsetsRetentionCheckIntervalMs = config.offsetsRetentionCheckIntervalMs, + offsetsTopicNumPartitions = config.offsetsTopicPartitions, + offsetsTopicReplicationFactor = config.offsetsTopicReplicationFactor, + offsetCommitTimeoutMs = config.offsetCommitTimeoutMs, + offsetCommitRequiredAcks = config.offsetCommitRequiredAcks) + val groupConfig = GroupManagerConfig(consumerMinSessionTimeoutMs = config.consumerMinSessionTimeoutMs, + consumerMaxSessionTimeoutMs = config.consumerMaxSessionTimeoutMs) + + new ConsumerCoordinator(config.brokerId, groupConfig, offsetConfig, replicaManager, zkClient, kafkaScheduler) + } + + def create(config: KafkaConfig, + zkClient: ZkClient, + offsetManager: OffsetManager): ConsumerCoordinator = { + val offsetConfig = OffsetManagerConfig(maxMetadataSize = config.offsetMetadataMaxSize, + loadBufferSize = config.offsetsLoadBufferSize, + offsetsRetentionMs = config.offsetsRetentionMinutes * 60 * 1000L, + offsetsRetentionCheckIntervalMs = config.offsetsRetentionCheckIntervalMs, + offsetsTopicNumPartitions = config.offsetsTopicPartitions, + offsetsTopicReplicationFactor = config.offsetsTopicReplicationFactor, + offsetCommitTimeoutMs = config.offsetCommitTimeoutMs, + offsetCommitRequiredAcks = config.offsetCommitRequiredAcks) + val groupConfig = GroupManagerConfig(consumerMinSessionTimeoutMs = config.consumerMinSessionTimeoutMs, + consumerMaxSessionTimeoutMs = config.consumerMaxSessionTimeoutMs) + + new ConsumerCoordinator(config.brokerId, groupConfig, offsetConfig, offsetManager, zkClient) + } } diff --git a/core/src/main/scala/kafka/coordinator/CoordinatorMetadata.scala b/core/src/main/scala/kafka/coordinator/CoordinatorMetadata.scala index 0cd5605..2920320 100644 --- a/core/src/main/scala/kafka/coordinator/CoordinatorMetadata.scala +++ b/core/src/main/scala/kafka/coordinator/CoordinatorMetadata.scala @@ -32,7 +32,7 @@ import scala.collection.mutable * It delegates all group logic to the callers. */ @threadsafe -private[coordinator] class CoordinatorMetadata(config: KafkaConfig, +private[coordinator] class CoordinatorMetadata(brokerId: Int, zkClient: ZkClient, maybePrepareRebalance: ConsumerGroupMetadata => Unit) { @@ -179,7 +179,7 @@ private[coordinator] class CoordinatorMetadata(config: KafkaConfig, * Zookeeper listener to handle topic partition changes */ class TopicPartitionChangeListener extends IZkDataListener with Logging { - this.logIdent = "[TopicPartitionChangeListener on Coordinator " + config.brokerId + "]: " + this.logIdent = "[TopicPartitionChangeListener on Coordinator " + brokerId + "]: " override def handleDataChange(dataPath: String, data: Object) { info("Handling data change for path: %s data: %s".format(dataPath, data)) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index ad6f058..18f5b5b 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -37,7 +37,6 @@ import org.I0Itec.zkclient.ZkClient */ class KafkaApis(val requestChannel: RequestChannel, val replicaManager: ReplicaManager, - val offsetManager: OffsetManager, val coordinator: ConsumerCoordinator, val controller: KafkaController, val zkClient: ZkClient, @@ -95,8 +94,23 @@ class KafkaApis(val requestChannel: RequestChannel, // stop serving data to clients for the topic being deleted val leaderAndIsrRequest = request.requestObj.asInstanceOf[LeaderAndIsrRequest] try { - val (response, error) = replicaManager.becomeLeaderOrFollower(leaderAndIsrRequest, offsetManager) - val leaderAndIsrResponse = new LeaderAndIsrResponse(leaderAndIsrRequest.correlationId, response, error) + // call replica manager to handle updating partitions to become leader or follower + val result = replicaManager.becomeLeaderOrFollower(leaderAndIsrRequest) + val leaderAndIsrResponse = new LeaderAndIsrResponse(leaderAndIsrRequest.correlationId, result.responseMap, result.errorCode) + // for each new leader or follower, call coordinator to handle + // consumer group migration + result.updatedLeaders.foreach { case partition => + if (partition.topic == ConsumerCoordinator.OffsetsTopicName) + coordinator.handleGroupImmigration(partition.partitionId) + } + result.updatedFollowers.foreach { case partition => + partition.leaderReplicaIdOpt.foreach { leaderReplica => + if (partition.topic == ConsumerCoordinator.OffsetsTopicName && + leaderReplica == brokerId) + coordinator.handleGroupEmigration(partition.partitionId) + } + } + requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, leaderAndIsrResponse))) } catch { case e: KafkaStorageException => @@ -142,6 +156,12 @@ class KafkaApis(val requestChannel: RequestChannel, def handleOffsetCommitRequest(request: RequestChannel.Request) { val offsetCommitRequest = request.requestObj.asInstanceOf[OffsetCommitRequest] + // filter non-exist topics + val invalidRequestsInfo = offsetCommitRequest.requestInfo.filter { case (topicAndPartition, offsetMetadata) => + !metadataCache.contains(topicAndPartition.topic) + } + val filteredRequestInfo = (offsetCommitRequest.requestInfo -- invalidRequestsInfo.keys) + // the callback for sending an offset commit response def sendResponseCallback(commitStatus: immutable.Map[TopicAndPartition, Short]) { commitStatus.foreach { case (topicAndPartition, errorCode) => @@ -154,14 +174,14 @@ class KafkaApis(val requestChannel: RequestChannel, topicAndPartition, ErrorMapping.exceptionNameFor(errorCode))) } } - - val response = OffsetCommitResponse(commitStatus, offsetCommitRequest.correlationId) + val combinedCommitStatus = commitStatus ++ invalidRequestsInfo.map(_._1 -> ErrorMapping.UnknownTopicOrPartitionCode) + val response = OffsetCommitResponse(combinedCommitStatus, offsetCommitRequest.correlationId) requestChannel.sendResponse(new RequestChannel.Response(request, new RequestOrResponseSend(request.connectionId, response))) } if (offsetCommitRequest.versionId == 0) { // for version 0 always store offsets to ZK - val responseInfo = offsetCommitRequest.requestInfo.map { + val responseInfo = filteredRequestInfo.map { case (topicAndPartition, metaAndError) => { val topicDirs = new ZKGroupTopicDirs(offsetCommitRequest.groupId, topicAndPartition.topic) try { @@ -189,7 +209,7 @@ class KafkaApis(val requestChannel: RequestChannel, val offsetRetention = if (offsetCommitRequest.versionId <= 1 || offsetCommitRequest.retentionMs == org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_RETENTION_TIME) { - offsetManager.config.offsetsRetentionMs + coordinator.offsetConfig.offsetsRetentionMs } else { offsetCommitRequest.retentionMs } @@ -203,7 +223,7 @@ class KafkaApis(val requestChannel: RequestChannel, val currentTimestamp = SystemTime.milliseconds val defaultExpireTimestamp = offsetRetention + currentTimestamp - val offsetData = offsetCommitRequest.requestInfo.mapValues(offsetAndMetadata => + val offsetData = filteredRequestInfo.mapValues(offsetAndMetadata => offsetAndMetadata.copy( commitTimestamp = currentTimestamp, expireTimestamp = { @@ -215,8 +235,8 @@ class KafkaApis(val requestChannel: RequestChannel, ) ) - // call offset manager to store offsets - offsetManager.storeOffsets( + // call coordinator to handle commit offset + coordinator.handleCommitOffsets( offsetCommitRequest.groupId, offsetCommitRequest.consumerId, offsetCommitRequest.groupGenerationId, @@ -422,9 +442,9 @@ class KafkaApis(val requestChannel: RequestChannel, if (topics.size > 0 && topicResponses.size != topics.size) { val nonExistentTopics = topics -- topicResponses.map(_.topic).toSet val responsesForNonExistentTopics = nonExistentTopics.map { topic => - if (topic == OffsetManager.OffsetsTopicName || config.autoCreateTopicsEnable) { + if (topic == ConsumerCoordinator.OffsetsTopicName || config.autoCreateTopicsEnable) { try { - if (topic == OffsetManager.OffsetsTopicName) { + if (topic == ConsumerCoordinator.OffsetsTopicName) { val aliveBrokers = metadataCache.getAliveBrokers val offsetsTopicReplicationFactor = if (aliveBrokers.length > 0) @@ -433,7 +453,7 @@ class KafkaApis(val requestChannel: RequestChannel, config.offsetsTopicReplicationFactor.toInt AdminUtils.createTopic(zkClient, topic, config.offsetsTopicPartitions, offsetsTopicReplicationFactor, - offsetManager.offsetsTopicConfig) + coordinator.offsetsTopicConfigs) info("Auto creation of topic %s with %d partitions and replication factor %d is successful!" .format(topic, config.offsetsTopicPartitions, offsetsTopicReplicationFactor)) } @@ -496,26 +516,19 @@ class KafkaApis(val requestChannel: RequestChannel, OffsetFetchResponse(collection.immutable.Map(responseInfo: _*), offsetFetchRequest.correlationId) } else { - // version 1 reads offsets from Kafka - val (unknownTopicPartitions, knownTopicPartitions) = offsetFetchRequest.requestInfo.partition(topicAndPartition => - metadataCache.getPartitionInfo(topicAndPartition.topic, topicAndPartition.partition).isEmpty - ) - val unknownStatus = unknownTopicPartitions.map(topicAndPartition => (topicAndPartition, OffsetMetadataAndError.UnknownTopicOrPartition)).toMap - val knownStatus = - if (knownTopicPartitions.size > 0) - offsetManager.getOffsets(offsetFetchRequest.groupId, knownTopicPartitions).toMap - else - Map.empty[TopicAndPartition, OffsetMetadataAndError] - val status = unknownStatus ++ knownStatus + // version 1 reads offsets from Kafka; + val offsets = coordinator.handleFetchOffsets(offsetFetchRequest.groupId, offsetFetchRequest.requestInfo).toMap - OffsetFetchResponse(status, offsetFetchRequest.correlationId) + // Note that we do not need to filter the partitions in the + // metadata cache as the topic partitions will be filtered + // in coordinator's offset manager through the offset cache + OffsetFetchResponse(offsets, offsetFetchRequest.correlationId) } trace("Sending offset fetch response %s for correlation id %d to client %s." .format(response, offsetFetchRequest.correlationId, offsetFetchRequest.clientId)) requestChannel.sendResponse(new RequestChannel.Response(request, new RequestOrResponseSend(request.connectionId, response))) - } /* @@ -524,10 +537,10 @@ class KafkaApis(val requestChannel: RequestChannel, def handleConsumerMetadataRequest(request: RequestChannel.Request) { val consumerMetadataRequest = request.requestObj.asInstanceOf[ConsumerMetadataRequest] - val partition = offsetManager.partitionFor(consumerMetadataRequest.group) + val partition = coordinator.partitionFor(consumerMetadataRequest.group) // get metadata (and create the topic if necessary) - val offsetsTopicMetadata = getTopicMetadata(Set(OffsetManager.OffsetsTopicName), request.securityProtocol).head + val offsetsTopicMetadata = getTopicMetadata(Set(ConsumerCoordinator.OffsetsTopicName), request.securityProtocol).head val errorResponse = ConsumerMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode, consumerMetadataRequest.correlationId) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 52dc728..18917bc 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -41,7 +41,7 @@ import kafka.common.{ErrorMapping, InconsistentBrokerIdException, GenerateBroker import kafka.network.{BlockingChannel, SocketServer} import kafka.metrics.KafkaMetricsGroup import com.yammer.metrics.core.Gauge -import kafka.coordinator.ConsumerCoordinator +import kafka.coordinator.{GroupManagerConfig, ConsumerCoordinator} /** * Represents the lifecycle of a single Kafka broker. Handles all functionality required @@ -75,8 +75,6 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg var logManager: LogManager = null - var offsetManager: OffsetManager = null - var replicaManager: ReplicaManager = null var topicConfigManager: TopicConfigManager = null @@ -157,19 +155,16 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg replicaManager = new ReplicaManager(config, time, zkClient, kafkaScheduler, logManager, isShuttingDown) replicaManager.startup() - /* start offset manager */ - offsetManager = createOffsetManager() - /* start kafka controller */ kafkaController = new KafkaController(config, zkClient, brokerState) kafkaController.startup() /* start kafka coordinator */ - consumerCoordinator = new ConsumerCoordinator(config, zkClient, offsetManager) + consumerCoordinator = ConsumerCoordinator.create(config, zkClient, replicaManager, kafkaScheduler) consumerCoordinator.startup() /* start processing requests */ - apis = new KafkaApis(socketServer.requestChannel, replicaManager, offsetManager, consumerCoordinator, + apis = new KafkaApis(socketServer.requestChannel, replicaManager, consumerCoordinator, kafkaController, zkClient, config.brokerId, config, metadataCache) requestHandlerPool = new KafkaRequestHandlerPool(config.brokerId, socketServer.requestChannel, apis, config.numIoThreads) brokerState.newState(RunningAsBroker) @@ -349,8 +344,6 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg CoreUtils.swallow(socketServer.shutdown()) if(requestHandlerPool != null) CoreUtils.swallow(requestHandlerPool.shutdown()) - if(offsetManager != null) - offsetManager.shutdown() CoreUtils.swallow(kafkaScheduler.shutdown()) if(apis != null) CoreUtils.swallow(apis.close()) @@ -450,19 +443,6 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg logProps } - private def createOffsetManager(): OffsetManager = { - val offsetManagerConfig = OffsetManagerConfig( - maxMetadataSize = config.offsetMetadataMaxSize, - loadBufferSize = config.offsetsLoadBufferSize, - offsetsRetentionMs = config.offsetsRetentionMinutes * 60 * 1000L, - offsetsRetentionCheckIntervalMs = config.offsetsRetentionCheckIntervalMs, - offsetsTopicNumPartitions = config.offsetsTopicPartitions, - offsetsTopicReplicationFactor = config.offsetsTopicReplicationFactor, - offsetCommitTimeoutMs = config.offsetCommitTimeoutMs, - offsetCommitRequiredAcks = config.offsetCommitRequiredAcks) - new OffsetManager(offsetManagerConfig, replicaManager, zkClient, kafkaScheduler, metadataCache) - } - /** * Generates new brokerId or reads from meta.properties based on following conditions *

    diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala index 5cca85c..47b6ce9 100755 --- a/core/src/main/scala/kafka/server/OffsetManager.scala +++ b/core/src/main/scala/kafka/server/OffsetManager.scala @@ -17,6 +17,7 @@ package kafka.server +import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.protocol.types.{Struct, Schema, Field} import org.apache.kafka.common.protocol.types.Type.STRING import org.apache.kafka.common.protocol.types.Type.INT32 @@ -25,19 +26,19 @@ import org.apache.kafka.common.utils.Utils import kafka.utils._ import kafka.common._ -import kafka.log.{FileMessageSet, LogConfig} +import kafka.log.FileMessageSet import kafka.message._ import kafka.metrics.KafkaMetricsGroup import kafka.common.TopicAndPartition import kafka.tools.MessageFormatter import kafka.api.ProducerResponseStatus +import kafka.coordinator.ConsumerCoordinator import scala.Some import scala.collection._ import java.io.PrintStream import java.util.concurrent.atomic.AtomicBoolean import java.nio.ByteBuffer -import java.util.Properties import java.util.concurrent.TimeUnit import com.yammer.metrics.core.Gauge @@ -87,8 +88,7 @@ object OffsetManagerConfig { class OffsetManager(val config: OffsetManagerConfig, replicaManager: ReplicaManager, zkClient: ZkClient, - scheduler: Scheduler, - metadataCache: MetadataCache) extends Logging with KafkaMetricsGroup { + scheduler: Scheduler) extends Logging with KafkaMetricsGroup { /* offsets and metadata cache */ private val offsetsCache = new Pool[GroupTopicPartition, OffsetAndMetadata] @@ -143,9 +143,9 @@ class OffsetManager(val config: OffsetManagerConfig, // Append the tombstone messages to the offset partitions. It is okay if the replicas don't receive these (say, // if we crash or leaders move) since the new leaders will get rid of expired offsets during their own purge cycles. tombstonesForPartition.flatMap { case (offsetsPartition, tombstones) => - val partitionOpt = replicaManager.getPartition(OffsetManager.OffsetsTopicName, offsetsPartition) + val partitionOpt = replicaManager.getPartition(ConsumerCoordinator.OffsetsTopicName, offsetsPartition) partitionOpt.map { partition => - val appendPartition = TopicAndPartition(OffsetManager.OffsetsTopicName, offsetsPartition) + val appendPartition = TopicAndPartition(ConsumerCoordinator.OffsetsTopicName, offsetsPartition) val messages = tombstones.map(_._2).toSeq trace("Marked %d offsets in %s for deletion.".format(messages.size, appendPartition)) @@ -170,14 +170,6 @@ class OffsetManager(val config: OffsetManagerConfig, } - def offsetsTopicConfig: Properties = { - val props = new Properties - props.put(LogConfig.SegmentBytesProp, config.offsetsTopicSegmentBytes.toString) - props.put(LogConfig.CleanupPolicyProp, "compact") - props.put(LogConfig.CompressionTypeProp, "uncompressed") - props - } - def partitionFor(group: String): Int = Utils.abs(group.hashCode) % config.offsetsTopicNumPartitions /** @@ -214,22 +206,14 @@ class OffsetManager(val config: OffsetManagerConfig, /** * Store offsets by appending it to the replicated log and then inserting to cache */ - // TODO: generation id and consumer id is needed by coordinator to do consumer checking in the future def storeOffsets(groupId: String, consumerId: String, generationId: Int, offsetMetadata: immutable.Map[TopicAndPartition, OffsetAndMetadata], responseCallback: immutable.Map[TopicAndPartition, Short] => Unit) { - // check if there are any non-existent topics - val nonExistentTopics = offsetMetadata.filter { case (topicAndPartition, offsetMetadata) => - !metadataCache.contains(topicAndPartition.topic) - } - - // first filter out partitions with offset metadata size exceeding limit or - // if its a non existing topic - // TODO: in the future we may want to only support atomic commit and hence fail the whole commit + // first filter out partitions with offset metadata size exceeding limit val filteredOffsetMetadata = offsetMetadata.filter { case (topicAndPartition, offsetAndMetadata) => - validateOffsetMetadataLength(offsetAndMetadata.metadata) || nonExistentTopics.contains(topicAndPartition) + validateOffsetMetadataLength(offsetAndMetadata.metadata) } // construct the message set to append @@ -240,7 +224,7 @@ class OffsetManager(val config: OffsetManagerConfig, ) }.toSeq - val offsetTopicPartition = TopicAndPartition(OffsetManager.OffsetsTopicName, partitionFor(groupId)) + val offsetTopicPartition = TopicAndPartition(ConsumerCoordinator.OffsetsTopicName, partitionFor(groupId)) val offsetsAndMetadataMessageSet = Map(offsetTopicPartition -> new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, messages:_*)) @@ -271,6 +255,10 @@ class OffsetManager(val config: OffsetManagerConfig, ErrorMapping.ConsumerCoordinatorNotAvailableCode else if (status.error == ErrorMapping.NotLeaderForPartitionCode) ErrorMapping.NotCoordinatorForConsumerCode + else if (status.error == ErrorMapping.MessageSizeTooLargeCode + || status.error == ErrorMapping.MessageSetSizeTooLargeCode + || status.error == ErrorMapping.InvalidFetchSizeCode) + Errors.INVALID_COMMIT_OFFSET_SIZE.code else status.error } @@ -278,9 +266,7 @@ class OffsetManager(val config: OffsetManagerConfig, // compute the final error codes for the commit response val commitStatus = offsetMetadata.map { case (topicAndPartition, offsetAndMetadata) => - if (nonExistentTopics.contains(topicAndPartition)) - (topicAndPartition, ErrorMapping.UnknownTopicOrPartitionCode) - else if (validateOffsetMetadataLength(offsetAndMetadata.metadata)) + if (validateOffsetMetadataLength(offsetAndMetadata.metadata)) (topicAndPartition, responseCode) else (topicAndPartition, ErrorMapping.OffsetMetadataTooLargeCode) @@ -338,7 +324,7 @@ class OffsetManager(val config: OffsetManagerConfig, debug("Could not fetch offsets for group %s (not offset coordinator).".format(group)) topicPartitions.map { topicAndPartition => val groupTopicPartition = GroupTopicPartition(group, topicAndPartition) - (groupTopicPartition.topicPartition, OffsetMetadataAndError.NotOffsetManagerForGroup) + (groupTopicPartition.topicPartition, OffsetMetadataAndError.NotCoordinatorForGroup) }.toMap } } @@ -349,7 +335,7 @@ class OffsetManager(val config: OffsetManagerConfig, */ def loadOffsetsFromLog(offsetsPartition: Int) { - val topicPartition = TopicAndPartition(OffsetManager.OffsetsTopicName, offsetsPartition) + val topicPartition = TopicAndPartition(ConsumerCoordinator.OffsetsTopicName, offsetsPartition) loadingPartitions synchronized { if (loadingPartitions.contains(offsetsPartition)) { @@ -421,7 +407,7 @@ class OffsetManager(val config: OffsetManagerConfig, } private def getHighWatermark(partitionId: Int): Long = { - val partitionOpt = replicaManager.getPartition(OffsetManager.OffsetsTopicName, partitionId) + val partitionOpt = replicaManager.getPartition(ConsumerCoordinator.OffsetsTopicName, partitionId) val hw = partitionOpt.map { partition => partition.leaderReplicaIfLocal().map(_.highWatermark.messageOffset).getOrElse(-1L) @@ -449,7 +435,7 @@ class OffsetManager(val config: OffsetManagerConfig, } if (numRemoved > 0) info("Removed %d cached offsets for %s on follower transition." - .format(numRemoved, TopicAndPartition(OffsetManager.OffsetsTopicName, offsetsPartition))) + .format(numRemoved, TopicAndPartition(ConsumerCoordinator.OffsetsTopicName, offsetsPartition))) } @@ -461,8 +447,6 @@ class OffsetManager(val config: OffsetManagerConfig, object OffsetManager { - val OffsetsTopicName = "__consumer_offsets" - private case class KeyAndValueSchemas(keySchema: Schema, valueSchema: Schema) private val CURRENT_OFFSET_SCHEMA_VERSION = 1.toShort diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 59c9bc3..795220e 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -23,19 +23,19 @@ import kafka.cluster.{BrokerEndPoint, Partition, Replica} import kafka.log.{LogAppendInfo, LogManager} import kafka.metrics.KafkaMetricsGroup import kafka.controller.KafkaController -import kafka.common.TopicAndPartition import kafka.message.{ByteBufferMessageSet, MessageSet} +import kafka.api.ProducerResponseStatus +import kafka.common.TopicAndPartition +import kafka.api.PartitionFetchInfo + +import org.apache.kafka.common.protocol.Errors import java.util.concurrent.atomic.AtomicBoolean import java.io.{IOException, File} import java.util.concurrent.TimeUnit -import org.apache.kafka.common.protocol.Errors -import scala.Predef._ +import scala.Some import scala.collection._ -import scala.collection.mutable.HashMap -import scala.collection.Map -import scala.collection.Set import org.I0Itec.zkclient.ZkClient import com.yammer.metrics.core.Gauge @@ -84,6 +84,17 @@ object LogReadResult { false) } +case class BecomeLeaderOrFollowerResult(responseMap: collection.Map[(String, Int), Short], + updatedLeaders: Set[Partition], + updatedFollowers: Set[Partition], + errorCode: Short) { + + override def toString = { + "updated leaders: [%s], updated followers: [%s], update results: [%s], global error: [%d]" + .format(updatedLeaders, updatedFollowers, responseMap, errorCode) + } +} + object ReplicaManager { val HighWatermarkFilename = "replication-offset-checkpoint" } @@ -393,10 +404,10 @@ class ReplicaManager(val config: KafkaConfig, (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(utpe))) case nle: NotLeaderForPartitionException => (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(nle))) - case mtl: MessageSizeTooLargeException => - (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(mtl))) - case mstl: MessageSetSizeTooLargeException => - (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(mstl))) + case mtle: MessageSizeTooLargeException => + (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(mtle))) + case mstle: MessageSetSizeTooLargeException => + (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(mstle))) case imse : InvalidMessageSizeException => (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(imse))) case t: Throwable => @@ -416,7 +427,7 @@ class ReplicaManager(val config: KafkaConfig, def fetchMessages(timeout: Long, replicaId: Int, fetchMinBytes: Int, - fetchInfo: Map[TopicAndPartition, PartitionFetchInfo], + fetchInfo: immutable.Map[TopicAndPartition, PartitionFetchInfo], responseCallback: Map[TopicAndPartition, FetchResponsePartitionData] => Unit) { val isFromFollower = replicaId >= 0 @@ -544,30 +555,29 @@ class ReplicaManager(val config: KafkaConfig, } } - def becomeLeaderOrFollower(leaderAndISRRequest: LeaderAndIsrRequest, - offsetManager: OffsetManager): (collection.Map[(String, Int), Short], Short) = { + def becomeLeaderOrFollower(leaderAndISRRequest: LeaderAndIsrRequest): BecomeLeaderOrFollowerResult = { leaderAndISRRequest.partitionStateInfos.foreach { case ((topic, partition), stateInfo) => stateChangeLogger.trace("Broker %d received LeaderAndIsr request %s correlation id %d from controller %d epoch %d for partition [%s,%d]" .format(localBrokerId, stateInfo, leaderAndISRRequest.correlationId, leaderAndISRRequest.controllerId, leaderAndISRRequest.controllerEpoch, topic, partition)) } replicaStateChangeLock synchronized { - val responseMap = new collection.mutable.HashMap[(String, Int), Short] - if(leaderAndISRRequest.controllerEpoch < controllerEpoch) { + val responseMap = new mutable.HashMap[(String, Int), Short] + if (leaderAndISRRequest.controllerEpoch < controllerEpoch) { leaderAndISRRequest.partitionStateInfos.foreach { case ((topic, partition), stateInfo) => stateChangeLogger.warn(("Broker %d ignoring LeaderAndIsr request from controller %d with correlation id %d since " + "its controller epoch %d is old. Latest known controller epoch is %d").format(localBrokerId, leaderAndISRRequest.controllerId, leaderAndISRRequest.correlationId, leaderAndISRRequest.controllerEpoch, controllerEpoch)) } - (responseMap, ErrorMapping.StaleControllerEpochCode) + BecomeLeaderOrFollowerResult(responseMap, Set.empty[Partition], Set.empty[Partition], ErrorMapping.StaleControllerEpochCode) } else { val controllerId = leaderAndISRRequest.controllerId val correlationId = leaderAndISRRequest.correlationId controllerEpoch = leaderAndISRRequest.controllerEpoch // First check partition's leader epoch - val partitionState = new HashMap[Partition, PartitionStateInfo]() - leaderAndISRRequest.partitionStateInfos.foreach{ case ((topic, partitionId), partitionStateInfo) => + val partitionState = new mutable.HashMap[Partition, PartitionStateInfo]() + leaderAndISRRequest.partitionStateInfos.foreach { case ((topic, partitionId), partitionStateInfo) => val partition = getOrCreatePartition(topic, partitionId) val partitionLeaderEpoch = partition.getLeaderEpoch() // If the leader epoch is valid record the epoch of the controller that made the leadership decision. @@ -591,14 +601,19 @@ class ReplicaManager(val config: KafkaConfig, } } - val partitionsTobeLeader = partitionState - .filter{ case (partition, partitionStateInfo) => partitionStateInfo.leaderIsrAndControllerEpoch.leaderAndIsr.leader == config.brokerId} + val partitionsTobeLeader = partitionState.filter { case (partition, partitionStateInfo) => + partitionStateInfo.leaderIsrAndControllerEpoch.leaderAndIsr.leader == config.brokerId + } val partitionsToBeFollower = (partitionState -- partitionsTobeLeader.keys) - if (!partitionsTobeLeader.isEmpty) - makeLeaders(controllerId, controllerEpoch, partitionsTobeLeader, leaderAndISRRequest.correlationId, responseMap, offsetManager) - if (!partitionsToBeFollower.isEmpty) - makeFollowers(controllerId, controllerEpoch, partitionsToBeFollower, leaderAndISRRequest.leaders, leaderAndISRRequest.correlationId, responseMap, offsetManager) + val partitionsBecomeLeader = if (!partitionsTobeLeader.isEmpty) + makeLeaders(controllerId, controllerEpoch, partitionsTobeLeader, leaderAndISRRequest.correlationId, responseMap) + else + Set.empty[Partition] + val partitionsBecomeFollower = if (!partitionsToBeFollower.isEmpty) + makeFollowers(controllerId, controllerEpoch, partitionsToBeFollower, leaderAndISRRequest.leaders, leaderAndISRRequest.correlationId, responseMap) + else + Set.empty[Partition] // we initialize highwatermark thread after the first leaderisrrequest. This ensures that all the partitions // have been completely populated before starting the checkpointing there by avoiding weird race conditions @@ -607,7 +622,7 @@ class ReplicaManager(val config: KafkaConfig, hwThreadInitialized = true } replicaFetcherManager.shutdownIdleFetcherThreads() - (responseMap, ErrorMapping.NoError) + BecomeLeaderOrFollowerResult(responseMap, partitionsBecomeLeader, partitionsBecomeFollower, ErrorMapping.NoError) } } } @@ -623,10 +638,11 @@ class ReplicaManager(val config: KafkaConfig, * the error message will be set on each partition since we do not know which partition caused it * TODO: the above may need to be fixed later */ - private def makeLeaders(controllerId: Int, epoch: Int, + private def makeLeaders(controllerId: Int, + epoch: Int, partitionState: Map[Partition, PartitionStateInfo], - correlationId: Int, responseMap: mutable.Map[(String, Int), Short], - offsetManager: OffsetManager) = { + correlationId: Int, + responseMap: mutable.Map[(String, Int), Short]): Set[Partition] = { partitionState.foreach(state => stateChangeLogger.trace(("Broker %d handling LeaderAndIsr request correlationId %d from controller %d epoch %d " + "starting the become-leader transition for partition %s") @@ -645,7 +661,7 @@ class ReplicaManager(val config: KafkaConfig, } // Update the partition information to be the leader partitionState.foreach{ case (partition, partitionStateInfo) => - partition.makeLeader(controllerId, partitionStateInfo, correlationId, offsetManager)} + partition.makeLeader(controllerId, partitionStateInfo, correlationId)} } catch { case e: Throwable => @@ -664,6 +680,8 @@ class ReplicaManager(val config: KafkaConfig, "for the become-leader transition for partition %s") .format(localBrokerId, correlationId, controllerId, epoch, TopicAndPartition(state._1.topic, state._1.partitionId))) } + + partitionState.keySet } /* @@ -682,9 +700,12 @@ class ReplicaManager(val config: KafkaConfig, * If an unexpected error is thrown in this function, it will be propagated to KafkaApis where * the error message will be set on each partition since we do not know which partition caused it */ - private def makeFollowers(controllerId: Int, epoch: Int, partitionState: Map[Partition, PartitionStateInfo], - leaders: Set[BrokerEndPoint], correlationId: Int, responseMap: mutable.Map[(String, Int), Short], - offsetManager: OffsetManager) { + private def makeFollowers(controllerId: Int, + epoch: Int, + partitionState: Map[Partition, PartitionStateInfo], + leaders: Set[BrokerEndPoint], + correlationId: Int, + responseMap: mutable.Map[(String, Int), Short]) : Set[Partition] = { partitionState.foreach { state => stateChangeLogger.trace(("Broker %d handling LeaderAndIsr request correlationId %d from controller %d epoch %d " + "starting the become-follower transition for partition %s") @@ -694,18 +715,18 @@ class ReplicaManager(val config: KafkaConfig, for (partition <- partitionState.keys) responseMap.put((partition.topic, partition.partitionId), ErrorMapping.NoError) - try { + val partitionsToMakeFollower: mutable.Set[Partition] = mutable.Set() - var partitionsToMakeFollower: Set[Partition] = Set() + try { - // TODO: Delete leaders from LeaderAndIsrRequest in 0.8.1 + // TODO: Delete leaders from LeaderAndIsrRequest partitionState.foreach{ case (partition, partitionStateInfo) => val leaderIsrAndControllerEpoch = partitionStateInfo.leaderIsrAndControllerEpoch val newLeaderBrokerId = leaderIsrAndControllerEpoch.leaderAndIsr.leader leaders.find(_.id == newLeaderBrokerId) match { // Only change partition state when the leader is available case Some(leaderBroker) => - if (partition.makeFollower(controllerId, partitionStateInfo, correlationId, offsetManager)) + if (partition.makeFollower(controllerId, partitionStateInfo, correlationId)) partitionsToMakeFollower += partition else stateChangeLogger.info(("Broker %d skipped the become-follower state change after marking its partition as follower with correlation id %d from " + @@ -775,6 +796,8 @@ class ReplicaManager(val config: KafkaConfig, "for the become-follower transition for partition %s") .format(localBrokerId, correlationId, controllerId, epoch, TopicAndPartition(state._1.topic, state._1.partitionId))) } + + partitionsToMakeFollower } private def maybeShrinkIsr(): Unit = { diff --git a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala index 17b17b9..92ffb91 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala @@ -25,12 +25,13 @@ import org.apache.kafka.common.TopicPartition import org.apache.kafka.clients.consumer.NoOffsetForPartitionException import kafka.utils.{TestUtils, Logging} -import kafka.server.{KafkaConfig, OffsetManager} +import kafka.server.KafkaConfig import java.util.ArrayList import org.junit.Assert._ import scala.collection.JavaConversions._ +import kafka.coordinator.ConsumerCoordinator /** @@ -158,9 +159,9 @@ class ConsumerTest extends IntegrationTestHarness with Logging { consumer0.poll(50) // get metadata for the topic - var parts = consumer0.partitionsFor(OffsetManager.OffsetsTopicName) + var parts = consumer0.partitionsFor(ConsumerCoordinator.OffsetsTopicName) while(parts == null) - parts = consumer0.partitionsFor(OffsetManager.OffsetsTopicName) + parts = consumer0.partitionsFor(ConsumerCoordinator.OffsetsTopicName) assertEquals(1, parts.size) assertNotNull(parts(0).leader()) diff --git a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala index 07b1ff4..afcc349 100644 --- a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala +++ b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala @@ -26,6 +26,7 @@ import org.apache.kafka.clients.producer.KafkaProducer import kafka.server.{OffsetManager, KafkaConfig} import kafka.integration.KafkaServerTestHarness import scala.collection.mutable.Buffer +import kafka.coordinator.ConsumerCoordinator /** * A helper class for writing integration tests that involve producers, consumers, and servers @@ -63,11 +64,11 @@ trait IntegrationTestHarness extends KafkaServerTestHarness { consumers += new KafkaConsumer(consumerConfig) // create the consumer offset topic - TestUtils.createTopic(zkClient, OffsetManager.OffsetsTopicName, - serverConfig.getProperty("offsets.topic.num.partitions").toInt, - serverConfig.getProperty("offsets.topic.replication.factor").toInt, + TestUtils.createTopic(zkClient, ConsumerCoordinator.OffsetsTopicName, + serverConfig.getProperty(KafkaConfig.OffsetsTopicPartitionsProp).toInt, + serverConfig.getProperty(KafkaConfig.OffsetsTopicReplicationFactorProp).toInt, servers, - servers(0).offsetManager.offsetsTopicConfig) + servers(0).consumerCoordinator.offsetsTopicConfigs) } override def tearDown() { diff --git a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala index c7136f2..dcd6988 100644 --- a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala @@ -22,9 +22,9 @@ import org.scalatest.junit.JUnit3Suite import kafka.utils.Logging import kafka.utils.TestUtils import kafka.zk.ZooKeeperTestHarness -import kafka.server.{OffsetManager, KafkaConfig} import kafka.admin.TopicCommand.TopicCommandOptions import kafka.utils.ZkUtils +import kafka.coordinator.ConsumerCoordinator class TopicCommandTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { @@ -87,12 +87,12 @@ class TopicCommandTest extends JUnit3Suite with ZooKeeperTestHarness with Loggin // create the offset topic val createOffsetTopicOpts = new TopicCommandOptions(Array("--partitions", numPartitionsOriginal.toString, "--replication-factor", "1", - "--topic", OffsetManager.OffsetsTopicName)) + "--topic", ConsumerCoordinator.OffsetsTopicName)) TopicCommand.createTopic(zkClient, createOffsetTopicOpts) // try to delete the OffsetManager.OffsetsTopicName and make sure it doesn't - val deleteOffsetTopicOpts = new TopicCommandOptions(Array("--topic", OffsetManager.OffsetsTopicName)) - val deleteOffsetTopicPath = ZkUtils.getDeleteTopicPath(OffsetManager.OffsetsTopicName) + val deleteOffsetTopicOpts = new TopicCommandOptions(Array("--topic", ConsumerCoordinator.OffsetsTopicName)) + val deleteOffsetTopicPath = ZkUtils.getDeleteTopicPath(ConsumerCoordinator.OffsetsTopicName) assertFalse("Delete path for topic shouldn't exist before deletion.", zkClient.exists(deleteOffsetTopicPath)) intercept[AdminOperationException] { TopicCommand.deleteTopic(zkClient, deleteOffsetTopicOpts) diff --git a/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala b/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala index 4f124af..4b326d0 100644 --- a/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala @@ -22,6 +22,7 @@ import junit.framework.Assert._ import org.scalatest.junit.JUnitSuite import org.junit.Test import kafka.server.OffsetManager +import kafka.coordinator.ConsumerCoordinator class TopicFilterTest extends JUnitSuite { @@ -37,8 +38,8 @@ class TopicFilterTest extends JUnitSuite { val topicFilter2 = new Whitelist(".+") assertTrue(topicFilter2.isTopicAllowed("alltopics", excludeInternalTopics = true)) - assertFalse(topicFilter2.isTopicAllowed(OffsetManager.OffsetsTopicName, excludeInternalTopics = true)) - assertTrue(topicFilter2.isTopicAllowed(OffsetManager.OffsetsTopicName, excludeInternalTopics = false)) + assertFalse(topicFilter2.isTopicAllowed(ConsumerCoordinator.OffsetsTopicName, excludeInternalTopics = true)) + assertTrue(topicFilter2.isTopicAllowed(ConsumerCoordinator.OffsetsTopicName, excludeInternalTopics = false)) val topicFilter3 = new Whitelist("white_listed-topic.+") assertTrue(topicFilter3.isTopicAllowed("white_listed-topic1", excludeInternalTopics = true)) @@ -57,8 +58,8 @@ class TopicFilterTest extends JUnitSuite { assertFalse(topicFilter1.isTopicAllowed("black1", excludeInternalTopics = true)) assertFalse(topicFilter1.isTopicAllowed("black1", excludeInternalTopics = false)) - assertFalse(topicFilter1.isTopicAllowed(OffsetManager.OffsetsTopicName, excludeInternalTopics = true)) - assertTrue(topicFilter1.isTopicAllowed(OffsetManager.OffsetsTopicName, excludeInternalTopics = false)) + assertFalse(topicFilter1.isTopicAllowed(ConsumerCoordinator.OffsetsTopicName, excludeInternalTopics = true)) + assertTrue(topicFilter1.isTopicAllowed(ConsumerCoordinator.OffsetsTopicName, excludeInternalTopics = false)) } @Test diff --git a/core/src/test/scala/unit/kafka/coordinator/ConsumerCoordinatorResponseTest.scala b/core/src/test/scala/unit/kafka/coordinator/ConsumerCoordinatorResponseTest.scala index a44fbd6..3cd726d 100644 --- a/core/src/test/scala/unit/kafka/coordinator/ConsumerCoordinatorResponseTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/ConsumerCoordinatorResponseTest.scala @@ -22,8 +22,8 @@ import java.util.concurrent.TimeUnit import junit.framework.Assert._ import kafka.common.TopicAndPartition -import kafka.server.{KafkaConfig, OffsetManager} -import kafka.utils.TestUtils +import kafka.server.{OffsetManager, ReplicaManager, KafkaConfig} +import kafka.utils.{KafkaScheduler, TestUtils} import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.JoinGroupRequest import org.easymock.EasyMock @@ -45,8 +45,8 @@ class ConsumerCoordinatorResponseTest extends JUnitSuite { val ConsumerMinSessionTimeout = 10 val ConsumerMaxSessionTimeout = 30 val DefaultSessionTimeout = 20 - var offsetManager: OffsetManager = null var consumerCoordinator: ConsumerCoordinator = null + var offsetManager : OffsetManager = null @Before def setUp() { @@ -54,12 +54,13 @@ class ConsumerCoordinatorResponseTest extends JUnitSuite { props.setProperty(KafkaConfig.ConsumerMinSessionTimeoutMsProp, ConsumerMinSessionTimeout.toString) props.setProperty(KafkaConfig.ConsumerMaxSessionTimeoutMsProp, ConsumerMaxSessionTimeout.toString) offsetManager = EasyMock.createStrictMock(classOf[OffsetManager]) - consumerCoordinator = new ConsumerCoordinator(KafkaConfig.fromProps(props), null, offsetManager) + consumerCoordinator = ConsumerCoordinator.create(KafkaConfig.fromProps(props), null, offsetManager) consumerCoordinator.startup() } @After def tearDown() { + EasyMock.reset(offsetManager) consumerCoordinator.shutdown() } diff --git a/core/src/test/scala/unit/kafka/coordinator/CoordinatorMetadataTest.scala b/core/src/test/scala/unit/kafka/coordinator/CoordinatorMetadataTest.scala index 08854c5..2cbf6e2 100644 --- a/core/src/test/scala/unit/kafka/coordinator/CoordinatorMetadataTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/CoordinatorMetadataTest.scala @@ -40,7 +40,7 @@ class CoordinatorMetadataTest extends JUnitSuite { def setUp() { val props = TestUtils.createBrokerConfig(nodeId = 0, zkConnect = "") zkClient = EasyMock.createStrictMock(classOf[ZkClient]) - coordinatorMetadata = new CoordinatorMetadata(KafkaConfig.fromProps(props), zkClient, null) + coordinatorMetadata = new CoordinatorMetadata(KafkaConfig.fromProps(props).brokerId, zkClient, null) } @Test diff --git a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala index 528525b..39a6852 100755 --- a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala @@ -120,7 +120,7 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { val fetchRequest2 = OffsetFetchRequest(group, Seq(unknownTopicAndPartition)) val fetchResponse2 = simpleConsumer.fetchOffsets(fetchRequest2) - assertEquals(OffsetMetadataAndError.UnknownTopicOrPartition, fetchResponse2.requestInfo.get(unknownTopicAndPartition).get) + assertEquals(OffsetMetadataAndError.NoOffset, fetchResponse2.requestInfo.get(unknownTopicAndPartition).get) assertEquals(1, fetchResponse2.requestInfo.size) } @@ -166,14 +166,14 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 1)).get.error) assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 0)).get.error) - assertEquals(ErrorMapping.UnknownTopicOrPartitionCode, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get.error) - assertEquals(OffsetMetadataAndError.UnknownTopicOrPartition, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get) + assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get.error) + assertEquals(OffsetMetadataAndError.NoOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get) assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get.error) assertEquals(OffsetMetadataAndError.NoOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get) - assertEquals(ErrorMapping.UnknownTopicOrPartitionCode, fetchResponse.requestInfo.get(TopicAndPartition(topic5, 0)).get.error) - assertEquals(OffsetMetadataAndError.UnknownTopicOrPartition, fetchResponse.requestInfo.get(TopicAndPartition(topic5, 0)).get) + assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic5, 0)).get.error) + assertEquals(OffsetMetadataAndError.NoOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic5, 0)).get) assertEquals("metadata one", fetchResponse.requestInfo.get(TopicAndPartition(topic1, 0)).get.metadata) assertEquals("metadata two", fetchResponse.requestInfo.get(TopicAndPartition(topic2, 0)).get.metadata) -- 1.7.12.4 From fd612a2d50f1ee13009395f082357403c4277164 Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Mon, 6 Jul 2015 15:47:40 -0700 Subject: [PATCH 030/120] kafka-2248; Use Apache Rat to enforce copyright headers; patched by Ewen Cheslack-Postava; reviewed by Gwen Shapira, Joel Joshy and Jun Rao --- .rat-excludes | 26 --- build.gradle | 27 ++- .../test/scala/other/kafka/TestOffsetManager.scala | 17 ++ gradle/buildscript.gradle | 19 +- gradle/license.gradle | 9 - gradle/rat.gradle | 115 ++++++++++++ gradle/resources/rat-output-to-html.xsl | 206 +++++++++++++++++++++ kafka-patch-review.py | 17 ++ scala.gradle | 15 ++ topics.json | 4 - 10 files changed, 409 insertions(+), 46 deletions(-) delete mode 100644 .rat-excludes delete mode 100644 gradle/license.gradle create mode 100644 gradle/rat.gradle create mode 100644 gradle/resources/rat-output-to-html.xsl delete mode 100644 topics.json diff --git a/.rat-excludes b/.rat-excludes deleted file mode 100644 index 01d6298..0000000 --- a/.rat-excludes +++ /dev/null @@ -1,26 +0,0 @@ -.rat-excludes -rat.out -sbt -sbt.boot.lock -README* -.gitignore -.git -.svn -build.properties -target -src_managed -update.log -clients/target -core/target -contrib/target -project/plugins/target -project/build/target -*.iml -*.csproj -TODO -Makefile* -*.html -*.xml -*expected.out -*.kafka - diff --git a/build.gradle b/build.gradle index 30d1cf2..727d7c5 100644 --- a/build.gradle +++ b/build.gradle @@ -13,11 +13,18 @@ // See the License for the specific language governing permissions and // limitations under the License. +import org.ajoberstar.grgit.Grgit + buildscript { repositories { mavenCentral() } apply from: file('gradle/buildscript.gradle'), to: buildscript + + dependencies { + // For Apache Rat plugin to ignore non-Git files, need ancient version for Java 6 compatibility + classpath group: 'org.ajoberstar', name: 'grgit', version: '0.2.3' + } } def slf4jlog4j='org.slf4j:slf4j-log4j12:1.7.6' @@ -41,8 +48,24 @@ ext { } apply from: file('wrapper.gradle') -apply from: file('gradle/license.gradle') apply from: file('scala.gradle') +apply from: file('gradle/rat.gradle') + +rat { + // Exclude everything under the directory that git should be ignoring via .gitignore or that isn't checked in. These + // restrict us only to files that are checked in or are staged. + def repo = Grgit.open(project.file('.')) + excludes = new ArrayList(repo.clean(ignore: false, directories: true, dryRun: true)) + // And some of the files that we have checked in should also be excluded from this check + excludes.addAll([ + '**/.git/**', + 'gradlew', + 'gradlew.bat', + '**/README.md', + '.reviewboardrc', + 'system_test/**', + ]) +} subprojects { apply plugin: 'java' @@ -52,8 +75,6 @@ subprojects { sourceCompatibility = 1.6 - licenseTest.onlyIf { isVerificationRequired(project) } - uploadArchives { repositories { signing { diff --git a/core/src/test/scala/other/kafka/TestOffsetManager.scala b/core/src/test/scala/other/kafka/TestOffsetManager.scala index 8047da4..e0e46c8 100644 --- a/core/src/test/scala/other/kafka/TestOffsetManager.scala +++ b/core/src/test/scala/other/kafka/TestOffsetManager.scala @@ -1,3 +1,20 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package other.kafka import org.I0Itec.zkclient.ZkClient diff --git a/gradle/buildscript.gradle b/gradle/buildscript.gradle index 5e45c06..047632b 100644 --- a/gradle/buildscript.gradle +++ b/gradle/buildscript.gradle @@ -1,3 +1,18 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + repositories { repositories { // For license plugin. @@ -6,7 +21,3 @@ repositories { } } } - -dependencies { - classpath 'nl.javadude.gradle.plugins:license-gradle-plugin:0.10.0' -} diff --git a/gradle/license.gradle b/gradle/license.gradle deleted file mode 100644 index b4b62eb..0000000 --- a/gradle/license.gradle +++ /dev/null @@ -1,9 +0,0 @@ -subprojects { - apply plugin: 'license' - - license { - header rootProject.file('HEADER') - // Skip Twitter bootstrap JS and CSS. - skipExistingHeaders = true - } -} diff --git a/gradle/rat.gradle b/gradle/rat.gradle new file mode 100644 index 0000000..d62b372 --- /dev/null +++ b/gradle/rat.gradle @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +import org.gradle.api.Plugin +import org.gradle.api.Project +import org.gradle.api.Task +import org.gradle.api.internal.project.IsolatedAntBuilder + +apply plugin: RatPlugin + +class RatTask extends DefaultTask { + @Input + List excludes + + def reportPath = 'build/rat' + def stylesheet = 'gradle/resources/rat-output-to-html.xsl' + def xmlReport = reportPath + '/rat-report.xml' + def htmlReport = reportPath + '/rat-report.html' + + def generateXmlReport(File reportDir) { + def antBuilder = services.get(IsolatedAntBuilder) + def ratClasspath = project.configurations.rat + antBuilder.withClasspath(ratClasspath).execute { + ant.taskdef(resource: 'org/apache/rat/anttasks/antlib.xml') + ant.report(format: 'xml', reportFile: xmlReport) { + fileset(dir: ".") { + patternset { + excludes.each { + exclude(name: it) + } + } + } + } + } + } + + def printUnknownFiles() { + def ratXml = new XmlParser().parse(xmlReport) + def unknownLicenses = 0 + ratXml.resource.each { resource -> + if (resource.'license-approval'.@name[0] == "false") { + println('Unknown license: ' + resource.@name) + unknownLicenses++ + } + } + if (unknownLicenses > 0) { + throw new GradleException("Found " + unknownLicenses + " files with " + + "unknown licenses.") + } + } + + def generateHtmlReport() { + def antBuilder = services.get(IsolatedAntBuilder) + def ratClasspath = project.configurations.rat + antBuilder.withClasspath(ratClasspath).execute { + ant.xslt( + in: xmlReport, + style: stylesheet, + out: htmlReport, + classpath: ratClasspath) + } + println('Rat report: ' + htmlReport) + } + + @TaskAction + def rat() { + File reportDir = new File(reportPath) + if (!reportDir.exists()) { + reportDir.mkdirs() + } + generateXmlReport(reportDir) + printUnknownFiles() + generateHtmlReport() + } +} + +class RatPlugin implements Plugin { + void apply(Project project) { + configureDependencies(project) + project.plugins.apply(JavaPlugin); + Task ratTask = project.task("rat", + type: RatTask, + group: 'Build', + description: 'Runs Apache Rat checks.') + project.tasks[JavaPlugin.TEST_TASK_NAME].dependsOn ratTask + } + + void configureDependencies(final Project project) { + project.configurations { + rat + } + project.repositories { + mavenCentral() + } + project.dependencies { + rat 'org.apache.rat:apache-rat-tasks:0.11' + } + } +} diff --git a/gradle/resources/rat-output-to-html.xsl b/gradle/resources/rat-output-to-html.xsl new file mode 100644 index 0000000..97ea7a1 --- /dev/null +++ b/gradle/resources/rat-output-to-html.xsl @@ -0,0 +1,206 @@ + + + + + + + + + + + + + + + + + + + + + + +

    Rat Report

    +

    This HTML version (yes, it is!) is generated from the RAT xml reports using Saxon9B. All the outputs required are displayed below, similar to the .txt version. + This is obviously a work in progress; and a prettier, easier to read and manage version will be available soon

    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + +
    +Table 1: A snapshot summary of this rat report. +
    Notes: Binaries: Archives: Standards:
    Apache Licensed: Generated Documents:
    Note: JavaDocs are generated and so license header is optionalNote: Generated files do not require license headers
    Unknown Licenses - or files without a license. Unknown Licenses - or files without a license.
    +
    +
    +

    Unapproved Licenses:

    + + + +
    + + +
    +
    + +

    Archives:

    + + + + +
    +
    +
    + +

    + Files with Apache License headers will be marked AL
    + Binary files (which do not require AL headers) will be marked B
    + Compressed archives will be marked A
    + Notices, licenses etc will be marked N
    +

    + + + + ! + + + + N + A + B + + !!!!! + + +
    + + +
    +
    + +

    Printing headers for files without AL header...

    + + + +

    + +
    +
    +
    + + + +
    + + +
    +

    Resource:

    + +
    +
    + + + +

    First few lines of non-compliant file

    +

    + +

    +
    +

    Other Info:

    +
    + + + Header Type: +
    +
    + + + License Family: +
    +
    + + + License Approval: +
    +
    + + + Type: +
    +
    + + + +
    diff --git a/kafka-patch-review.py b/kafka-patch-review.py index b5a2e95..94873c3 100644 --- a/kafka-patch-review.py +++ b/kafka-patch-review.py @@ -1,4 +1,21 @@ #!/usr/bin/env python +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. import argparse import sys diff --git a/scala.gradle b/scala.gradle index cabb59c..5eb2a65 100644 --- a/scala.gradle +++ b/scala.gradle @@ -1,3 +1,18 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + if (!hasProperty('scalaVersion')) { ext.scalaVersion = '2.10.5' } diff --git a/topics.json b/topics.json deleted file mode 100644 index ff011ed..0000000 --- a/topics.json +++ /dev/null @@ -1,4 +0,0 @@ -{"topics": - [{"topic": "foo"}], - "version":1 - } -- 1.7.12.4 From 2d96da05a0af7847aca5edc6d003a18be7f5216a Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Mon, 6 Jul 2015 16:36:20 -0700 Subject: [PATCH 031/120] kafka-2132; Move Log4J appender to a separate module; patched by Ashish Singh; reviewed by Gwen Shapira, Aditya Auradkar and Jun Rao --- build.gradle | 60 ++++++-- checkstyle/import-control.xml | 9 +- .../scala/kafka/producer/KafkaLog4jAppender.scala | 97 ------------ .../unit/kafka/log4j/KafkaLog4jAppenderTest.scala | 143 ------------------ .../kafka/log4jappender/KafkaLog4jAppender.java | 167 +++++++++++++++++++++ .../log4jappender/KafkaLog4jAppenderTest.java | 98 ++++++++++++ .../log4jappender/MockKafkaLog4jAppender.java | 47 ++++++ settings.gradle | 2 +- 8 files changed, 370 insertions(+), 253 deletions(-) delete mode 100644 core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala delete mode 100755 core/src/test/scala/unit/kafka/log4j/KafkaLog4jAppenderTest.scala create mode 100644 log4j-appender/src/main/java/org/apache/kafka/log4jappender/KafkaLog4jAppender.java create mode 100644 log4j-appender/src/test/java/org/apache/kafka/log4jappender/KafkaLog4jAppenderTest.java create mode 100644 log4j-appender/src/test/java/org/apache/kafka/log4jappender/MockKafkaLog4jAppender.java diff --git a/build.gradle b/build.gradle index 727d7c5..ab86987 100644 --- a/build.gradle +++ b/build.gradle @@ -132,7 +132,7 @@ subprojects { archives srcJar archives javadocJar } - + plugins.withType(ScalaPlugin) { //source jar should also contain scala source: srcJar.from sourceSets.main.scala @@ -202,20 +202,20 @@ for ( sv in ['2_9_1', '2_9_2', '2_10_5', '2_11_6'] ) { } } -tasks.create(name: "jarAll", dependsOn: ['jar_core_2_9_1', 'jar_core_2_9_2', 'jar_core_2_10_5', 'jar_core_2_11_6', 'clients:jar', 'examples:jar', 'contrib:hadoop-consumer:jar', 'contrib:hadoop-producer:jar']) { +tasks.create(name: "jarAll", dependsOn: ['jar_core_2_9_1', 'jar_core_2_9_2', 'jar_core_2_10_5', 'jar_core_2_11_6', 'clients:jar', 'examples:jar', 'contrib:hadoop-consumer:jar', 'contrib:hadoop-producer:jar', 'log4j-appender:jar']) { } -tasks.create(name: "srcJarAll", dependsOn: ['srcJar_2_9_1', 'srcJar_2_9_2', 'srcJar_2_10_5', 'srcJar_2_11_6', 'clients:srcJar', 'examples:srcJar', 'contrib:hadoop-consumer:srcJar', 'contrib:hadoop-producer:srcJar']) { } +tasks.create(name: "srcJarAll", dependsOn: ['srcJar_2_9_1', 'srcJar_2_9_2', 'srcJar_2_10_5', 'srcJar_2_11_6', 'clients:srcJar', 'examples:srcJar', 'contrib:hadoop-consumer:srcJar', 'contrib:hadoop-producer:srcJar', 'log4j-appender:srcJar']) { } -tasks.create(name: "docsJarAll", dependsOn: ['docsJar_2_9_1', 'docsJar_2_9_2', 'docsJar_2_10_5', 'docsJar_2_11_6', 'clients:docsJar', 'examples:docsJar', 'contrib:hadoop-consumer:docsJar', 'contrib:hadoop-producer:docsJar']) { } +tasks.create(name: "docsJarAll", dependsOn: ['docsJar_2_9_1', 'docsJar_2_9_2', 'docsJar_2_10_5', 'docsJar_2_11_6', 'clients:docsJar', 'examples:docsJar', 'contrib:hadoop-consumer:docsJar', 'contrib:hadoop-producer:docsJar', 'log4j-appender:docsJar']) { } -tasks.create(name: "testAll", dependsOn: ['test_core_2_9_1', 'test_core_2_9_2', 'test_core_2_10_5', 'test_core_2_11_6', 'clients:test']) { +tasks.create(name: "testAll", dependsOn: ['test_core_2_9_1', 'test_core_2_9_2', 'test_core_2_10_5', 'test_core_2_11_6', 'clients:test', 'log4j-appender:test']) { } tasks.create(name: "releaseTarGzAll", dependsOn: ['releaseTarGz_2_9_1', 'releaseTarGz_2_9_2', 'releaseTarGz_2_10_5', 'releaseTarGz_2_11_6']) { } -tasks.create(name: "uploadArchivesAll", dependsOn: ['uploadCoreArchives_2_9_1', 'uploadCoreArchives_2_9_2', 'uploadCoreArchives_2_10_5', 'uploadCoreArchives_2_11_6', 'clients:uploadArchives', 'examples:uploadArchives', 'contrib:hadoop-consumer:uploadArchives', 'contrib:hadoop-producer:uploadArchives']) { +tasks.create(name: "uploadArchivesAll", dependsOn: ['uploadCoreArchives_2_9_1', 'uploadCoreArchives_2_9_2', 'uploadCoreArchives_2_10_5', 'uploadCoreArchives_2_11_6', 'clients:uploadArchives', 'examples:uploadArchives', 'contrib:hadoop-consumer:uploadArchives', 'contrib:hadoop-producer:uploadArchives', 'log4j-appender:uploadArchives']) { } project(':core') { @@ -228,6 +228,7 @@ project(':core') { dependencies { compile project(':clients') + compile project(':log4j-appender') compile "org.scala-lang:scala-library:$scalaVersion" compile 'org.apache.zookeeper:zookeeper:3.4.6' compile 'com.101tec:zkclient:0.5' @@ -237,7 +238,6 @@ project(':core') { testCompile 'junit:junit:4.6' testCompile 'org.easymock:easymock:3.0' testCompile 'org.objenesis:objenesis:1.2' - testCompile project(':clients') if (scalaVersion.startsWith('2.10')) { testCompile 'org.scalatest:scalatest_2.10:1.9.1' } else if (scalaVersion.startsWith('2.11')) { @@ -273,9 +273,9 @@ project(':core') { into "$buildDir/dependant-libs-${scalaVersion}" } - tasks.create(name: "releaseTarGz", dependsOn: configurations.archives.artifacts, type: Tar) { + tasks.create(name: "releaseTarGz", dependsOn: configurations.archives.artifacts, type: Tar) { into "kafka_${baseScalaVersion}-${version}" - compression = Compression.GZIP + compression = Compression.GZIP from(project.file("../bin")) { into "bin/" } from(project.file("../config")) { into "config/" } from '../LICENSE' @@ -378,7 +378,7 @@ project(':clients') { compile 'org.xerial.snappy:snappy-java:1.1.1.7' compile 'net.jpountz.lz4:lz4:1.2.0' - testCompile 'com.novocode:junit-interface:0.9' + testCompile 'junit:junit:4.6' testRuntime "$slf4jlog4j" } @@ -405,7 +405,45 @@ project(':clients') { artifacts { archives testJar } - + + configurations { + archives.extendsFrom (testCompile) + } + + checkstyle { + configFile = new File(rootDir, "checkstyle/checkstyle.xml") + } + test.dependsOn('checkstyleMain', 'checkstyleTest') +} + +project(':log4j-appender') { + apply plugin: 'checkstyle' + archivesBaseName = "kafka-log4j-appender" + + dependencies { + compile project(':clients') + compile "$slf4jlog4j" + + testCompile 'junit:junit:4.6' + testCompile project(path: ':clients', configuration: 'archives') + } + + task testJar(type: Jar) { + classifier = 'test' + from sourceSets.test.output + } + + test { + testLogging { + events "passed", "skipped", "failed" + exceptionFormat = 'full' + } + } + + javadoc { + include "**/org/apache/kafka/log4jappender/*" + } + checkstyle { configFile = new File(rootDir, "checkstyle/checkstyle.xml") } diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index f2e6cec..19e0659 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -95,8 +95,15 @@ + + + + + + + - + diff --git a/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala b/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala deleted file mode 100644 index 5d36a01..0000000 --- a/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala +++ /dev/null @@ -1,97 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. -*/ - -package kafka.producer - -import async.MissingConfigException -import org.apache.log4j.spi.LoggingEvent -import org.apache.log4j.AppenderSkeleton -import org.apache.log4j.helpers.LogLog -import kafka.utils.Logging -import java.util.{Properties, Date} -import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} - -class KafkaLog4jAppender extends AppenderSkeleton with Logging { - var topic: String = null - var brokerList: String = null - var compressionType: String = null - var retries: Int = 0 - var requiredNumAcks: Int = Int.MaxValue - var syncSend: Boolean = false - - private var producer: KafkaProducer[Array[Byte],Array[Byte]] = null - - def getTopic: String = topic - def setTopic(topic: String) { this.topic = topic } - - def getBrokerList: String = brokerList - def setBrokerList(brokerList: String) { this.brokerList = brokerList } - - def getCompressionType: String = compressionType - def setCompressionType(compressionType: String) { this.compressionType = compressionType } - - def getRequiredNumAcks: Int = requiredNumAcks - def setRequiredNumAcks(requiredNumAcks: Int) { this.requiredNumAcks = requiredNumAcks } - - def getSyncSend: Boolean = syncSend - def setSyncSend(syncSend: Boolean) { this.syncSend = syncSend } - - def getRetries: Int = retries - def setRetries(retries: Int) { this.retries = retries } - - override def activateOptions() { - // check for config parameter validity - val props = new Properties() - if(brokerList != null) - props.put(org.apache.kafka.clients.producer.ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) - if(props.isEmpty) - throw new MissingConfigException("The bootstrap servers property should be specified") - if(topic == null) - throw new MissingConfigException("topic must be specified by the Kafka log4j appender") - if(compressionType != null) props.put(org.apache.kafka.clients.producer.ProducerConfig.COMPRESSION_TYPE_CONFIG, compressionType) - if(requiredNumAcks != Int.MaxValue) props.put(org.apache.kafka.clients.producer.ProducerConfig.ACKS_CONFIG, requiredNumAcks.toString) - if(retries > 0) props.put(org.apache.kafka.clients.producer.ProducerConfig.RETRIES_CONFIG, retries.toString) - props.put(org.apache.kafka.clients.producer.ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") - props.put(org.apache.kafka.clients.producer.ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") - producer = new KafkaProducer[Array[Byte],Array[Byte]](props) - LogLog.debug("Kafka producer connected to " + brokerList) - LogLog.debug("Logging for topic: " + topic) - } - - override def append(event: LoggingEvent) { - val message = subAppend(event) - LogLog.debug("[" + new Date(event.getTimeStamp).toString + "]" + message) - val response = producer.send(new ProducerRecord[Array[Byte],Array[Byte]](topic, message.getBytes())) - if (syncSend) response.get - } - - def subAppend(event: LoggingEvent): String = { - if(this.layout == null) - event.getRenderedMessage - else - this.layout.format(event) - } - - override def close() { - if(!this.closed) { - this.closed = true - producer.close() - } - } - - override def requiresLayout: Boolean = true -} diff --git a/core/src/test/scala/unit/kafka/log4j/KafkaLog4jAppenderTest.scala b/core/src/test/scala/unit/kafka/log4j/KafkaLog4jAppenderTest.scala deleted file mode 100755 index 41366a1..0000000 --- a/core/src/test/scala/unit/kafka/log4j/KafkaLog4jAppenderTest.scala +++ /dev/null @@ -1,143 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.log4j - -import kafka.consumer.SimpleConsumer -import kafka.server.{KafkaConfig, KafkaServer} -import kafka.utils.{TestUtils, CoreUtils, Logging} -import kafka.api.FetchRequestBuilder -import kafka.producer.async.MissingConfigException -import kafka.serializer.Encoder -import kafka.zk.ZooKeeperTestHarness - -import java.util.Properties -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._ - -class KafkaLog4jAppenderTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { - - var logDirZk: File = null - var config: KafkaConfig = null - var server: KafkaServer = null - - var simpleConsumerZk: SimpleConsumer = null - - val tLogger = Logger.getLogger(getClass()) - - private val brokerZk = 0 - - @Before - override def setUp() { - super.setUp() - - val propsZk = TestUtils.createBrokerConfig(brokerZk, zkConnect) - val logDirZkPath = propsZk.getProperty("log.dir") - logDirZk = new File(logDirZkPath) - config = KafkaConfig.fromProps(propsZk) - server = TestUtils.createServer(config) - simpleConsumerZk = new SimpleConsumer("localhost", server.boundPort(), 1000000, 64 * 1024, "") - } - - @After - override def tearDown() { - simpleConsumerZk.close - server.shutdown - CoreUtils.rm(logDirZk) - super.tearDown() - } - - @Test - def testKafkaLog4jConfigs() { - // host missing - var props = new Properties() - props.put("log4j.rootLogger", "INFO") - props.put("log4j.appender.KAFKA", "kafka.producer.KafkaLog4jAppender") - props.put("log4j.appender.KAFKA.layout", "org.apache.log4j.PatternLayout") - props.put("log4j.appender.KAFKA.layout.ConversionPattern", "%-5p: %c - %m%n") - props.put("log4j.appender.KAFKA.Topic", "test-topic") - props.put("log4j.logger.kafka.log4j", "INFO, KAFKA") - - try { - PropertyConfigurator.configure(props) - fail("Missing properties exception was expected !") - } catch { - case e: MissingConfigException => - } - - // topic missing - props = new Properties() - props.put("log4j.rootLogger", "INFO") - props.put("log4j.appender.KAFKA", "kafka.producer.KafkaLog4jAppender") - props.put("log4j.appender.KAFKA.layout", "org.apache.log4j.PatternLayout") - props.put("log4j.appender.KAFKA.layout.ConversionPattern", "%-5p: %c - %m%n") - props.put("log4j.appender.KAFKA.brokerList", TestUtils.getBrokerListStrFromServers(Seq(server))) - props.put("log4j.logger.kafka.log4j", "INFO, KAFKA") - - try { - PropertyConfigurator.configure(props) - fail("Missing properties exception was expected !") - } catch { - case e: MissingConfigException => - } - } - - @Test - def testLog4jAppends() { - PropertyConfigurator.configure(getLog4jConfig) - - for(i <- 1 to 5) - info("test") - - val response = simpleConsumerZk.fetch(new FetchRequestBuilder().addFetch("test-topic", 0, 0L, 1024*1024).build()) - val fetchMessage = response.messageSet("test-topic", 0) - - var count = 0 - for(message <- fetchMessage) { - count = count + 1 - } - - assertEquals(5, count) - } - - private def getLog4jConfig: Properties = { - val props = new Properties() - props.put("log4j.rootLogger", "INFO") - props.put("log4j.appender.KAFKA", "kafka.producer.KafkaLog4jAppender") - props.put("log4j.appender.KAFKA.layout", "org.apache.log4j.PatternLayout") - props.put("log4j.appender.KAFKA.layout.ConversionPattern", "%-5p: %c - %m%n") - props.put("log4j.appender.KAFKA.BrokerList", TestUtils.getBrokerListStrFromServers(Seq(server))) - props.put("log4j.appender.KAFKA.Topic", "test-topic") - props.put("log4j.appender.KAFKA.RequiredNumAcks", "1") - props.put("log4j.appender.KAFKA.SyncSend", "true") - props.put("log4j.logger.kafka.log4j", "INFO, KAFKA") - props - } -} - -class AppenderStringEncoder(encoding: String = "UTF-8") extends Encoder[LoggingEvent] { - def toBytes(event: LoggingEvent): Array[Byte] = { - event.getMessage.toString.getBytes(encoding) - } -} - diff --git a/log4j-appender/src/main/java/org/apache/kafka/log4jappender/KafkaLog4jAppender.java b/log4j-appender/src/main/java/org/apache/kafka/log4jappender/KafkaLog4jAppender.java new file mode 100644 index 0000000..628ff53 --- /dev/null +++ b/log4j-appender/src/main/java/org/apache/kafka/log4jappender/KafkaLog4jAppender.java @@ -0,0 +1,167 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.log4jappender; + +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.config.ConfigException; +import org.apache.log4j.AppenderSkeleton; +import org.apache.log4j.helpers.LogLog; +import org.apache.log4j.spi.LoggingEvent; + +import java.util.Date; +import java.util.Properties; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; + +/** + * A log4j appender that produces log messages to Kafka + */ +public class KafkaLog4jAppender extends AppenderSkeleton { + + private static final String BOOTSTRAP_SERVERS_CONFIG = "bootstrap.servers"; + private static final String COMPRESSION_TYPE_CONFIG = "compression.type"; + private static final String ACKS_CONFIG = "acks"; + private static final String RETRIES_CONFIG = "retries"; + private static final String KEY_SERIALIZER_CLASS_CONFIG = "key.serializer"; + private static final String VALUE_SERIALIZER_CLASS_CONFIG = "value.serializer"; + + private String brokerList = null; + private String topic = null; + private String compressionType = null; + + private int retries = 0; + private int requiredNumAcks = Integer.MAX_VALUE; + private boolean syncSend = false; + private Producer producer = null; + + public Producer getProducer() { + return producer; + } + + public String getBrokerList() { + return brokerList; + } + + public void setBrokerList(String brokerList) { + this.brokerList = brokerList; + } + + public int getRequiredNumAcks() { + return requiredNumAcks; + } + + public void setRequiredNumAcks(int requiredNumAcks) { + this.requiredNumAcks = requiredNumAcks; + } + + public int getRetries() { + return retries; + } + + public void setRetries(int retries) { + this.retries = retries; + } + + public String getCompressionType() { + return compressionType; + } + + public void setCompressionType(String compressionType) { + this.compressionType = compressionType; + } + + public String getTopic() { + return topic; + } + + public void setTopic(String topic) { + this.topic = topic; + } + + public boolean getSyncSend() { + return syncSend; + } + + public void setSyncSend(boolean syncSend) { + this.syncSend = syncSend; + } + + @Override + public void activateOptions() { + // check for config parameter validity + Properties props = new Properties(); + if (brokerList != null) + props.put(BOOTSTRAP_SERVERS_CONFIG, brokerList); + if (props.isEmpty()) + throw new ConfigException("The bootstrap servers property should be specified"); + if (topic == null) + throw new ConfigException("Topic must be specified by the Kafka log4j appender"); + if (compressionType != null) + props.put(COMPRESSION_TYPE_CONFIG, compressionType); + if (requiredNumAcks != Integer.MAX_VALUE) + props.put(ACKS_CONFIG, requiredNumAcks); + if (retries > 0) + props.put(RETRIES_CONFIG, retries); + + props.put(KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); + props.put(VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); + this.producer = getKafkaProducer(props); + LogLog.debug("Kafka producer connected to " + brokerList); + LogLog.debug("Logging for topic: " + topic); + } + + protected Producer getKafkaProducer(Properties props) { + return new KafkaProducer(props); + } + + @Override + protected void append(LoggingEvent event) { + String message = subAppend(event); + LogLog.debug("[" + new Date(event.getTimeStamp()) + "]" + message); + Future response = producer.send(new ProducerRecord(topic, message.getBytes())); + if (syncSend) { + try { + response.get(); + } catch (InterruptedException ex) { + throw new RuntimeException(ex); + } catch (ExecutionException ex) { + throw new RuntimeException(ex); + } + } + } + + private String subAppend(LoggingEvent event) { + return (this.layout == null) ? event.getRenderedMessage() : this.layout.format(event); + } + + @Override + public void close() { + if (!this.closed) { + this.closed = true; + producer.close(); + } + } + + @Override + public boolean requiresLayout() { + return true; + } +} diff --git a/log4j-appender/src/test/java/org/apache/kafka/log4jappender/KafkaLog4jAppenderTest.java b/log4j-appender/src/test/java/org/apache/kafka/log4jappender/KafkaLog4jAppenderTest.java new file mode 100644 index 0000000..71bdd94 --- /dev/null +++ b/log4j-appender/src/test/java/org/apache/kafka/log4jappender/KafkaLog4jAppenderTest.java @@ -0,0 +1,98 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.log4jappender; + +import org.apache.kafka.common.config.ConfigException; +import org.apache.log4j.Logger; +import org.apache.log4j.PropertyConfigurator; +import org.junit.Assert; +import org.junit.Test; + +import java.io.UnsupportedEncodingException; +import java.util.Properties; + +public class KafkaLog4jAppenderTest { + + Logger logger = Logger.getLogger(KafkaLog4jAppenderTest.class); + + @Test + public void testKafkaLog4jConfigs() { + // host missing + Properties props = new Properties(); + props.put("log4j.rootLogger", "INFO"); + props.put("log4j.appender.KAFKA", "org.apache.kafka.log4jappender.KafkaLog4jAppender"); + props.put("log4j.appender.KAFKA.layout", "org.apache.log4j.PatternLayout"); + props.put("log4j.appender.KAFKA.layout.ConversionPattern", "%-5p: %c - %m%n"); + props.put("log4j.appender.KAFKA.Topic", "test-topic"); + props.put("log4j.logger.kafka.log4j", "INFO, KAFKA"); + + try { + PropertyConfigurator.configure(props); + Assert.fail("Missing properties exception was expected !"); + } catch (ConfigException ex) { + // It's OK! + } + + // topic missing + props = new Properties(); + props.put("log4j.rootLogger", "INFO"); + props.put("log4j.appender.KAFKA", "org.apache.kafka.log4jappender.KafkaLog4jAppender"); + props.put("log4j.appender.KAFKA.layout", "org.apache.log4j.PatternLayout"); + props.put("log4j.appender.KAFKA.layout.ConversionPattern", "%-5p: %c - %m%n"); + props.put("log4j.appender.KAFKA.brokerList", "127.0.0.1:9093"); + props.put("log4j.logger.kafka.log4j", "INFO, KAFKA"); + + try { + PropertyConfigurator.configure(props); + Assert.fail("Missing properties exception was expected !"); + } catch (ConfigException ex) { + // It's OK! + } + } + + + @Test + public void testLog4jAppends() throws UnsupportedEncodingException { + PropertyConfigurator.configure(getLog4jConfig()); + + for (int i = 1; i <= 5; ++i) { + logger.error(getMessage(i)); + } + + Assert.assertEquals( + 5, ((MockKafkaLog4jAppender) (logger.getRootLogger().getAppender("KAFKA"))).getHistory().size()); + } + + private byte[] getMessage(int i) throws UnsupportedEncodingException { + return ("test_" + i).getBytes("UTF-8"); + } + + private Properties getLog4jConfig() { + Properties props = new Properties(); + props.put("log4j.rootLogger", "INFO, KAFKA"); + props.put("log4j.appender.KAFKA", "org.apache.kafka.log4jappender.MockKafkaLog4jAppender"); + props.put("log4j.appender.KAFKA.layout", "org.apache.log4j.PatternLayout"); + props.put("log4j.appender.KAFKA.layout.ConversionPattern", "%-5p: %c - %m%n"); + props.put("log4j.appender.KAFKA.BrokerList", "127.0.0.1:9093"); + props.put("log4j.appender.KAFKA.Topic", "test-topic"); + props.put("log4j.appender.KAFKA.RequiredNumAcks", "1"); + props.put("log4j.appender.KAFKA.SyncSend", "false"); + props.put("log4j.logger.kafka.log4j", "INFO, KAFKA"); + return props; + } +} + diff --git a/log4j-appender/src/test/java/org/apache/kafka/log4jappender/MockKafkaLog4jAppender.java b/log4j-appender/src/test/java/org/apache/kafka/log4jappender/MockKafkaLog4jAppender.java new file mode 100644 index 0000000..c35f26a --- /dev/null +++ b/log4j-appender/src/test/java/org/apache/kafka/log4jappender/MockKafkaLog4jAppender.java @@ -0,0 +1,47 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.log4jappender; + +import org.apache.kafka.clients.producer.MockProducer; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.test.MockSerializer; +import org.apache.log4j.spi.LoggingEvent; + +import java.util.Properties; + +public class MockKafkaLog4jAppender extends KafkaLog4jAppender { + private MockProducer mockProducer = + new MockProducer(false, new MockSerializer(), new MockSerializer()); + + @Override + protected Producer getKafkaProducer(Properties props) { + return mockProducer; + } + + @Override + protected void append(LoggingEvent event) { + if (super.getProducer() == null) { + activateOptions(); + } + super.append(event); + } + + protected java.util.List> getHistory() { + return mockProducer.history(); + } +} diff --git a/settings.gradle b/settings.gradle index 83f764e..3b6a952 100644 --- a/settings.gradle +++ b/settings.gradle @@ -14,4 +14,4 @@ // limitations under the License. apply from: file('scala.gradle') -include 'core', 'contrib:hadoop-consumer', 'contrib:hadoop-producer', 'examples', 'clients' +include 'core', 'contrib:hadoop-consumer', 'contrib:hadoop-producer', 'examples', 'clients', 'log4j-appender' -- 1.7.12.4 From 271b18d119fdc37952c36c573ba185aa672e3f96 Mon Sep 17 00:00:00 2001 From: Kostya Golikov Date: Tue, 7 Jul 2015 09:12:57 -0700 Subject: [PATCH 032/120] KAFKA-2314: proper MirrorMaker's message handler help message; reviewed by Guozhang Wang --- core/src/main/scala/kafka/tools/MirrorMaker.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/tools/MirrorMaker.scala b/core/src/main/scala/kafka/tools/MirrorMaker.scala index 459aaec..797b4bb 100755 --- a/core/src/main/scala/kafka/tools/MirrorMaker.scala +++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala @@ -131,9 +131,9 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { .ofType(classOf[String]) val messageHandlerOpt = parser.accepts("message.handler", - "The consumer rebalance listener to use for mirror maker consumer.") + "Message handler which will process every record in-between consumer and producer.") .withRequiredArg() - .describedAs("A custom rebalance listener of type MirrorMakerMessageHandler") + .describedAs("A custom message handler of type MirrorMakerMessageHandler") .ofType(classOf[String]) val messageHandlerArgsOpt = parser.accepts("message.handler.args", -- 1.7.12.4 From f77dc386c099da5ff0bac4d2a12b04f7f17f07d3 Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Tue, 7 Jul 2015 09:45:26 -0700 Subject: [PATCH 033/120] kafka-1367; Broker topic metadata not kept in sync with ZooKeeper; patched by Ashish Singh; reviewed by Jun Rao --- .../scala/kafka/common/TopicAndPartition.scala | 6 +- .../scala/kafka/controller/KafkaController.scala | 69 ++++++++++++++++-- .../main/scala/kafka/utils/ReplicationUtils.scala | 16 ++++- core/src/main/scala/kafka/utils/ZkUtils.scala | 1 + .../unit/kafka/integration/TopicMetadataTest.scala | 84 +++++++++++++++++++--- .../unit/kafka/utils/ReplicationUtilsTest.scala | 2 + 6 files changed, 158 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/kafka/common/TopicAndPartition.scala b/core/src/main/scala/kafka/common/TopicAndPartition.scala index df3db91..13a3f28 100644 --- a/core/src/main/scala/kafka/common/TopicAndPartition.scala +++ b/core/src/main/scala/kafka/common/TopicAndPartition.scala @@ -1,6 +1,7 @@ package kafka.common import kafka.cluster.{Replica, Partition} +import kafka.utils.Json /** * Licensed to the Apache Software Foundation (ASF) under one or more @@ -24,6 +25,8 @@ import kafka.cluster.{Replica, Partition} */ case class TopicAndPartition(topic: String, partition: Int) { + private val version: Long = 1L + def this(tuple: (String, Int)) = this(tuple._1, tuple._2) def this(partition: Partition) = this(partition.topic, partition.partitionId) @@ -33,5 +36,6 @@ case class TopicAndPartition(topic: String, partition: Int) { def asTuple = (topic, partition) override def toString = "[%s,%d]".format(topic, partition) -} + def toJson = Json.encode(Map("version" -> version, "topic" -> topic, "partition" -> partition)) +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 3635057..09630d0 100755 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -16,8 +16,9 @@ */ package kafka.controller -import collection._ -import collection.Set +import java.util + +import scala.collection._ import com.yammer.metrics.core.Gauge import java.util.concurrent.TimeUnit import kafka.admin.AdminUtils @@ -31,7 +32,7 @@ import kafka.utils.ZkUtils._ import kafka.utils._ import kafka.utils.CoreUtils._ import org.apache.zookeeper.Watcher.Event.KeeperState -import org.I0Itec.zkclient.{IZkDataListener, IZkStateListener, ZkClient} +import org.I0Itec.zkclient.{IZkChildListener, IZkDataListener, IZkStateListener, ZkClient} import org.I0Itec.zkclient.exception.{ZkNodeExistsException, ZkNoNodeException} import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.locks.ReentrantLock @@ -169,6 +170,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt private val partitionReassignedListener = new PartitionsReassignedListener(this) private val preferredReplicaElectionListener = new PreferredReplicaElectionListener(this) + private val isrChangeNotificationListener = new IsrChangeNotificationListener(this) newGauge( "ActiveControllerCount", @@ -307,6 +309,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt incrementControllerEpoch(zkClient) // before reading source of truth from zookeeper, register the listeners to get broker/topic callbacks registerReassignedPartitionsListener() + registerIsrChangeNotificationListener() registerPreferredReplicaElectionListener() partitionStateMachine.registerListeners() replicaStateMachine.registerListeners() @@ -792,8 +795,8 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt controllerContext.controllerChannelManager.startup() } - private def updateLeaderAndIsrCache() { - val leaderAndIsrInfo = ZkUtils.getPartitionLeaderAndIsrForTopics(zkClient, controllerContext.partitionReplicaAssignment.keySet) + def updateLeaderAndIsrCache(topicAndPartitions: Set[TopicAndPartition] = controllerContext.partitionReplicaAssignment.keySet) { + val leaderAndIsrInfo = ZkUtils.getPartitionLeaderAndIsrForTopics(zkClient, topicAndPartitions) for((topicPartition, leaderIsrAndControllerEpoch) <- leaderAndIsrInfo) controllerContext.partitionLeadershipInfo.put(topicPartition, leaderIsrAndControllerEpoch) } @@ -892,6 +895,12 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt zkClient.subscribeDataChanges(ZkUtils.ReassignPartitionsPath, partitionReassignedListener) } + private def registerIsrChangeNotificationListener() = { + debug("Registering IsrChangeNotificationListener") + ZkUtils.makeSurePersistentPathExists(zkClient, ZkUtils.IsrChangeNotificationPath) + zkClient.subscribeChildChanges(ZkUtils.IsrChangeNotificationPath, isrChangeNotificationListener) + } + private def deregisterReassignedPartitionsListener() = { zkClient.unsubscribeDataChanges(ZkUtils.ReassignPartitionsPath, partitionReassignedListener) } @@ -1281,6 +1290,56 @@ class ReassignedPartitionsIsrChangeListener(controller: KafkaController, topic: } /** + * Called when leader intimates of isr change + * @param controller + */ +class IsrChangeNotificationListener(controller: KafkaController) extends IZkChildListener with Logging { + var topicAndPartitionSet: Set[TopicAndPartition] = Set() + + override def handleChildChange(parentPath: String, currentChildren: util.List[String]): Unit = { + import scala.collection.JavaConverters._ + + inLock(controller.controllerContext.controllerLock) { + debug("[IsrChangeNotificationListener] Fired!!!") + val childrenAsScala: mutable.Buffer[String] = currentChildren.asScala + val topicAndPartitions: immutable.Set[TopicAndPartition] = childrenAsScala.map(x => getTopicAndPartition(x)).flatten.toSet + controller.updateLeaderAndIsrCache(topicAndPartitions) + processUpdateNotifications(topicAndPartitions) + + // delete processed children + childrenAsScala.map(x => ZkUtils.deletePath(controller.controllerContext.zkClient, ZkUtils.TopicConfigChangesPath + "/" + x)) + } + } + + private def processUpdateNotifications(topicAndPartitions: immutable.Set[TopicAndPartition]) { + val liveBrokers: Seq[Int] = controller.controllerContext.liveOrShuttingDownBrokerIds.toSeq + controller.sendUpdateMetadataRequest(liveBrokers, topicAndPartitions) + debug("Sending MetadataRequest to Brokers:" + liveBrokers + " for TopicAndPartitions:" + topicAndPartitions) + } + + private def getTopicAndPartition(child: String): Option[TopicAndPartition] = { + val changeZnode: String = ZkUtils.IsrChangeNotificationPath + "/" + child + val (jsonOpt, stat) = ZkUtils.readDataMaybeNull(controller.controllerContext.zkClient, changeZnode) + if (jsonOpt.isDefined) { + val json = Json.parseFull(jsonOpt.get) + + json match { + case Some(m) => + val topicAndPartition = m.asInstanceOf[Map[String, Any]] + val topic = topicAndPartition("topic").asInstanceOf[String] + val partition = topicAndPartition("partition").asInstanceOf[Int] + Some(TopicAndPartition(topic, partition)) + case None => + error("Invalid topic and partition JSON: " + json + " in ZK: " + changeZnode) + None + } + } else { + None + } + } +} + +/** * Starts the preferred replica leader election for the list of partitions specified under * /admin/preferred_replica_election - */ diff --git a/core/src/main/scala/kafka/utils/ReplicationUtils.scala b/core/src/main/scala/kafka/utils/ReplicationUtils.scala index 6068733..783ba10 100644 --- a/core/src/main/scala/kafka/utils/ReplicationUtils.scala +++ b/core/src/main/scala/kafka/utils/ReplicationUtils.scala @@ -18,22 +18,32 @@ package kafka.utils import kafka.api.LeaderAndIsr +import kafka.common.TopicAndPartition import kafka.controller.LeaderIsrAndControllerEpoch -import org.apache.zookeeper.data.Stat import org.I0Itec.zkclient.ZkClient +import org.apache.zookeeper.data.Stat -import scala.Some import scala.collection._ object ReplicationUtils extends Logging { + val IsrChangeNotificationPrefix = "isr_change_" + def updateLeaderAndIsr(zkClient: ZkClient, topic: String, partitionId: Int, newLeaderAndIsr: LeaderAndIsr, controllerEpoch: Int, zkVersion: Int): (Boolean,Int) = { debug("Updated ISR for partition [%s,%d] to %s".format(topic, partitionId, newLeaderAndIsr.isr.mkString(","))) val path = ZkUtils.getTopicPartitionLeaderAndIsrPath(topic, partitionId) val newLeaderData = ZkUtils.leaderAndIsrZkData(newLeaderAndIsr, controllerEpoch) // use the epoch of the controller that made the leadership decision, instead of the current controller epoch - ZkUtils.conditionalUpdatePersistentPath(zkClient, path, newLeaderData, zkVersion, Some(checkLeaderAndIsrZkData)) + val updatePersistentPath: (Boolean, Int) = ZkUtils.conditionalUpdatePersistentPath(zkClient, path, newLeaderData, zkVersion, Some(checkLeaderAndIsrZkData)) + if (updatePersistentPath._1) { + val topicAndPartition: TopicAndPartition = TopicAndPartition(topic, partitionId) + val isrChangeNotificationPath: String = ZkUtils.createSequentialPersistentPath( + zkClient, ZkUtils.IsrChangeNotificationPath + "/" + IsrChangeNotificationPrefix, + topicAndPartition.toJson) + debug("Added " + isrChangeNotificationPath + " for " + topicAndPartition) + } + updatePersistentPath } def checkLeaderAndIsrZkData(zkClient: ZkClient, path: String, expectedLeaderAndIsrInfo: String): (Boolean,Int) = { diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index 78475e3..166814c 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -47,6 +47,7 @@ object ZkUtils extends Logging { val DeleteTopicsPath = "/admin/delete_topics" val PreferredReplicaLeaderElectionPath = "/admin/preferred_replica_election" val BrokerSequenceIdPath = "/brokers/seqid" + val IsrChangeNotificationPath = "/isr_change_notification" def getTopicPath(topic: String): String = { BrokerTopicsPath + "/" + topic diff --git a/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala b/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala index 995b059..a95ee5e 100644 --- a/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala +++ b/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala @@ -17,28 +17,32 @@ package kafka.integration -import org.apache.kafka.common.protocol.SecurityProtocol -import org.scalatest.junit.JUnit3Suite -import kafka.zk.ZooKeeperTestHarness -import kafka.admin.AdminUtils import java.nio.ByteBuffer + import junit.framework.Assert._ -import kafka.cluster.{BrokerEndPoint, Broker} +import kafka.admin.AdminUtils +import kafka.api.{TopicMetadataResponse, TopicMetadataRequest} +import kafka.client.ClientUtils +import kafka.cluster.{Broker, BrokerEndPoint} +import kafka.common.ErrorMapping +import kafka.server.{NotRunning, KafkaConfig, KafkaServer} import kafka.utils.TestUtils import kafka.utils.TestUtils._ -import kafka.server.{KafkaServer, KafkaConfig} -import kafka.api.TopicMetadataRequest -import kafka.common.ErrorMapping -import kafka.client.ClientUtils +import kafka.zk.ZooKeeperTestHarness +import org.apache.kafka.common.protocol.SecurityProtocol +import org.scalatest.junit.JUnit3Suite class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { private var server1: KafkaServer = null var brokerEndPoints: Seq[BrokerEndPoint] = null + var adHocConfigs: Seq[KafkaConfig] = null + val numConfigs: Int = 2 override def setUp() { super.setUp() - val props = createBrokerConfigs(1, zkConnect) - val configs = props.map(KafkaConfig.fromProps) + val props = createBrokerConfigs(numConfigs, zkConnect) + val configs: Seq[KafkaConfig] = props.map(KafkaConfig.fromProps) + adHocConfigs = configs.takeRight(configs.size - 1) // Started and stopped by individual test cases server1 = TestUtils.createServer(configs.head) brokerEndPoints = Seq(new Broker(server1.config.brokerId, server1.config.hostName, server1.boundPort()).getBrokerEndPoint(SecurityProtocol.PLAINTEXT)) } @@ -130,4 +134,62 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { assertEquals(1, partitionMetadata.head.replicas.size) assertTrue(partitionMetadata.head.leader.isDefined) } + + private def checkIsr(servers: Seq[KafkaServer]): Unit = { + val activeBrokers: Seq[KafkaServer] = servers.filter(x => x.brokerState.currentState != NotRunning.state) + val expectedIsr: Seq[BrokerEndPoint] = activeBrokers.map( + x => new BrokerEndPoint(x.config.brokerId, + if (x.config.hostName.nonEmpty) x.config.hostName else "localhost", + x.boundPort()) + ) + + // Assert that topic metadata at new brokers is updated correctly + activeBrokers.foreach(x => { + var metadata: TopicMetadataResponse = new TopicMetadataResponse(Seq(), Seq(), -1) + waitUntilTrue(() => { + metadata = ClientUtils.fetchTopicMetadata( + Set.empty, + Seq(new BrokerEndPoint( + x.config.brokerId, + if (x.config.hostName.nonEmpty) x.config.hostName else "localhost", + x.boundPort())), + "TopicMetadataTest-testBasicTopicMetadata", + 2000, 0) + metadata.topicsMetadata.nonEmpty && + metadata.topicsMetadata.head.partitionsMetadata.nonEmpty && + expectedIsr == metadata.topicsMetadata.head.partitionsMetadata.head.isr + }, + "Topic metadata is not correctly updated for broker " + x + ".\n" + + "Expected ISR: " + expectedIsr + "\n" + + "Actual ISR : " + (if (metadata.topicsMetadata.nonEmpty && + metadata.topicsMetadata.head.partitionsMetadata.nonEmpty) + metadata.topicsMetadata.head.partitionsMetadata.head.isr + else + "")) + }) + } + + + def testIsrAfterBrokerShutDownAndJoinsBack { + // start adHoc brokers + val adHocServers = adHocConfigs.map(p => createServer(p)) + val allServers: Seq[KafkaServer] = Seq(server1) ++ adHocServers + + // create topic + val topic: String = "test" + AdminUtils.createTopic(zkClient, topic, 1, numConfigs) + + // shutdown a broker + adHocServers.last.shutdown() + adHocServers.last.awaitShutdown() + + // startup a broker + adHocServers.last.startup() + + // check metadata is still correct and updated at all brokers + checkIsr(allServers) + + // shutdown adHoc brokers + adHocServers.map(p => p.shutdown()) + } } diff --git a/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala b/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala index c96c0ff..b9de8d6 100644 --- a/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala +++ b/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala @@ -70,6 +70,8 @@ class ReplicationUtilsTest extends JUnit3Suite with ZooKeeperTestHarness { EasyMock.expect(replicaManager.zkClient).andReturn(zkClient) EasyMock.replay(replicaManager) + ZkUtils.makeSurePersistentPathExists(zkClient, ZkUtils.IsrChangeNotificationPath) + val replicas = List(0,1) // regular update -- 1.7.12.4 From ad485e148d7ac104abe173687ba27dccff8e4d39 Mon Sep 17 00:00:00 2001 From: Joe Stein Date: Tue, 7 Jul 2015 10:09:11 -0700 Subject: [PATCH 034/120] KAFKA-2304 Supported enabling JMX in Kafka Vagrantfile patch by Stevo Slavic reviewed by Ewen Cheslack-Postava --- Vagrantfile | 7 +++++-- vagrant/broker.sh | 5 +++++ vagrant/zk.sh | 7 ++++++- 3 files changed, 16 insertions(+), 3 deletions(-) diff --git a/Vagrantfile b/Vagrantfile index 55c67dd..1d7cc01 100644 --- a/Vagrantfile +++ b/Vagrantfile @@ -22,6 +22,7 @@ VAGRANTFILE_API_VERSION = "2" # General config enable_dns = false +enable_jmx = false num_zookeepers = 1 num_brokers = 3 num_workers = 0 # Generic workers that get the code, but don't start any services @@ -135,7 +136,8 @@ Vagrant.configure(VAGRANTFILE_API_VERSION) do |config| ip_address = "192.168.50." + (10 + i).to_s assign_local_ip(zookeeper, ip_address) zookeeper.vm.provision "shell", path: "vagrant/base.sh" - zookeeper.vm.provision "shell", path: "vagrant/zk.sh", :args => [i.to_s, num_zookeepers] + zk_jmx_port = enable_jmx ? (8000 + i).to_s : "" + zookeeper.vm.provision "shell", path: "vagrant/zk.sh", :args => [i.to_s, num_zookeepers, zk_jmx_port] end } @@ -151,7 +153,8 @@ Vagrant.configure(VAGRANTFILE_API_VERSION) do |config| # used to support clients running on the host. zookeeper_connect = zookeepers.map{ |zk_addr| zk_addr + ":2181"}.join(",") broker.vm.provision "shell", path: "vagrant/base.sh" - broker.vm.provision "shell", path: "vagrant/broker.sh", :args => [i.to_s, enable_dns ? name : ip_address, zookeeper_connect] + kafka_jmx_port = enable_jmx ? (9000 + i).to_s : "" + broker.vm.provision "shell", path: "vagrant/broker.sh", :args => [i.to_s, enable_dns ? name : ip_address, zookeeper_connect, kafka_jmx_port] end } diff --git a/vagrant/broker.sh b/vagrant/broker.sh index 63f2d4f..bc040c9 100644 --- a/vagrant/broker.sh +++ b/vagrant/broker.sh @@ -22,6 +22,7 @@ set -e BROKER_ID=$1 PUBLIC_ADDRESS=$2 PUBLIC_ZOOKEEPER_ADDRESSES=$3 +JMX_PORT=$4 cd /opt/kafka @@ -35,4 +36,8 @@ echo "Killing server" bin/kafka-server-stop.sh || true sleep 5 # Because kafka-server-stop.sh doesn't actually wait echo "Starting server" +if [[ -n $JMX_PORT ]]; then + export JMX_PORT=$JMX_PORT + export KAFKA_JMX_OPTS="-Djava.rmi.server.hostname=$PUBLIC_ADDRESS -Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.ssl=false " +fi bin/kafka-server-start.sh /opt/kafka/config/server-$BROKER_ID.properties 1>> /tmp/broker.log 2>> /tmp/broker.log & diff --git a/vagrant/zk.sh b/vagrant/zk.sh index 15517f8..6fc4b7c 100644 --- a/vagrant/zk.sh +++ b/vagrant/zk.sh @@ -21,6 +21,7 @@ set -e ZKID=$1 NUM_ZK=$2 +JMX_PORT=$3 cd /opt/kafka @@ -37,6 +38,10 @@ echo "$ZKID" > /tmp/zookeeper/myid echo "Killing ZooKeeper" bin/zookeeper-server-stop.sh || true -sleep 5 # Because kafka-server-stop.sh doesn't actually wait +sleep 5 # Because zookeeper-server-stop.sh doesn't actually wait echo "Starting ZooKeeper" +if [[ -n $JMX_PORT ]]; then + export JMX_PORT=$JMX_PORT + export KAFKA_JMX_OPTS="-Djava.rmi.server.hostname=zk$ZKID -Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.ssl=false " +fi bin/zookeeper-server-start.sh config/zookeeper-$ZKID.properties 1>> /tmp/zk.log 2>> /tmp/zk.log & -- 1.7.12.4 From a99f70feb23db9ac4274cad9e8cbbf9934d3d075 Mon Sep 17 00:00:00 2001 From: Dong Lin Date: Tue, 7 Jul 2015 12:42:49 -0700 Subject: [PATCH 035/120] KAFKA-2306: add another metric for buffer exhausted; reviewed by Guozhang Wang --- .../java/org/apache/kafka/clients/producer/KafkaProducer.java | 4 ++++ .../kafka/clients/producer/internals/RecordAccumulator.java | 10 ++++++++-- 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 5671a3f..03b8dd2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -403,6 +403,10 @@ public class KafkaProducer implements Producer { } catch (InterruptedException e) { this.errors.record(); throw new InterruptException(e); + } catch (BufferExhaustedException e) { + this.errors.record(); + this.metrics.sensor("buffer-exhausted-records").record(); + throw e; } catch (KafkaException e) { this.errors.record(); throw e; diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java index 87dbd64..a152bd7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java @@ -21,6 +21,8 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.metrics.Measurable; import org.apache.kafka.common.metrics.MetricConfig; import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.metrics.stats.Rate; import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.Record; @@ -112,7 +114,6 @@ public final class RecordAccumulator { } private void registerMetrics(Metrics metrics, String metricGrpName, Map metricTags) { - MetricName metricName = new MetricName("waiting-threads", metricGrpName, "The number of user threads blocked waiting for buffer memory to enqueue their records", metricTags); Measurable waitingThreads = new Measurable() { public double measure(MetricConfig config, long now) { @@ -120,7 +121,7 @@ public final class RecordAccumulator { } }; metrics.addMetric(metricName, waitingThreads); - + metricName = new MetricName("buffer-total-bytes", metricGrpName, "The maximum amount of buffer memory the client can use (whether or not it is currently used).", metricTags); Measurable totalBytes = new Measurable() { public double measure(MetricConfig config, long now) { @@ -128,6 +129,7 @@ public final class RecordAccumulator { } }; metrics.addMetric(metricName, totalBytes); + metricName = new MetricName("buffer-available-bytes", metricGrpName, "The total amount of buffer memory that is not being used (either unallocated or in the free list).", metricTags); Measurable availableBytes = new Measurable() { public double measure(MetricConfig config, long now) { @@ -135,6 +137,10 @@ public final class RecordAccumulator { } }; metrics.addMetric(metricName, availableBytes); + + Sensor bufferExhaustedRecordSensor = metrics.sensor("buffer-exhausted-records"); + metricName = new MetricName("buffer-exhausted-rate", metricGrpName, "The average per-second number of record sends that are dropped due to buffer exhaustion", metricTags); + bufferExhaustedRecordSensor.add(metricName, new Rate()); } /** -- 1.7.12.4 From 826276de1eaabfebf2a414c69ebcdff17429016c Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Tue, 7 Jul 2015 13:26:00 -0700 Subject: [PATCH 036/120] KAFKA-2317: follow-up of KAFKA1367; reviewed by Guozhang Wang --- core/src/main/scala/kafka/controller/KafkaController.scala | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 09630d0..20f1499 100755 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -342,6 +342,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt */ def onControllerResignation() { // de-register listeners + deregisterIsrChangeNotificationListener() deregisterReassignedPartitionsListener() deregisterPreferredReplicaElectionListener() @@ -891,16 +892,21 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt } } - private def registerReassignedPartitionsListener() = { - zkClient.subscribeDataChanges(ZkUtils.ReassignPartitionsPath, partitionReassignedListener) - } - private def registerIsrChangeNotificationListener() = { debug("Registering IsrChangeNotificationListener") ZkUtils.makeSurePersistentPathExists(zkClient, ZkUtils.IsrChangeNotificationPath) zkClient.subscribeChildChanges(ZkUtils.IsrChangeNotificationPath, isrChangeNotificationListener) } + private def deregisterIsrChangeNotificationListener() = { + debug("De-registering IsrChangeNotificationListener") + zkClient.unsubscribeChildChanges(ZkUtils.IsrChangeNotificationPath, isrChangeNotificationListener) + } + + private def registerReassignedPartitionsListener() = { + zkClient.subscribeDataChanges(ZkUtils.ReassignPartitionsPath, partitionReassignedListener) + } + private def deregisterReassignedPartitionsListener() = { zkClient.unsubscribeDataChanges(ZkUtils.ReassignPartitionsPath, partitionReassignedListener) } -- 1.7.12.4 From f13dd8024d5bc1c11587a3b539556ea01e2c84ca Mon Sep 17 00:00:00 2001 From: Onur Karaman Date: Tue, 7 Jul 2015 13:36:55 -0700 Subject: [PATCH 037/120] KAFKA-2313: javadoc fix for KafkaConsumer deserialization; reviewed by Guozhang Wang --- .../org/apache/kafka/clients/consumer/KafkaConsumer.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) 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 1f0e515..7aa0760 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 @@ -131,8 +131,8 @@ import static org.apache.kafka.common.utils.Utils.min; * props.put("enable.auto.commit", "true"); * props.put("auto.commit.interval.ms", "1000"); * props.put("session.timeout.ms", "30000"); - * props.put("key.serializer", "org.apache.kafka.common.serializers.StringSerializer"); - * props.put("value.serializer", "org.apache.kafka.common.serializers.StringSerializer"); + * props.put("key.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); + * props.put("value.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); * KafkaConsumer<String, String> consumer = new KafkaConsumer<String, String>(props); * consumer.subscribe("foo", "bar"); * while (true) { @@ -159,8 +159,8 @@ import static org.apache.kafka.common.utils.Utils.min; * to it. If it stops heartbeating for a period of time longer than session.timeout.ms then it will be * considered dead and it's partitions will be assigned to another process. *

    - * The serializers settings specify how to turn the objects the user provides into bytes. By specifying the string - * serializers we are saying that our record's key and value will just be simple strings. + * The deserializer settings specify how to turn bytes into objects. For example, by specifying string deserializers, we + * are saying that our record's key and value will just be simple strings. * *

    Controlling When Messages Are Considered Consumed

    * @@ -183,8 +183,8 @@ import static org.apache.kafka.common.utils.Utils.min; * props.put("enable.auto.commit", "false"); * props.put("auto.commit.interval.ms", "1000"); * props.put("session.timeout.ms", "30000"); - * props.put("key.serializer", "org.apache.kafka.common.serializers.StringSerializer"); - * props.put("value.serializer", "org.apache.kafka.common.serializers.StringSerializer"); + * props.put("key.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); + * props.put("value.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); * KafkaConsumer<String, String> consumer = new KafkaConsumer<String, String>(props); * consumer.subscribe("foo", "bar"); * int commitInterval = 200; -- 1.7.12.4 From 23d72bccbeeeb0774b86c47e05cdcd1575a5d38a Mon Sep 17 00:00:00 2001 From: Dong Lin Date: Tue, 7 Jul 2015 18:09:00 -0700 Subject: [PATCH 038/120] KAFKA-2298; Client Selector can drop connections on InvalidReceiveException without notifying NetworkClient; reviewed by Jason Gustafson and Joel Koshy --- clients/src/main/java/org/apache/kafka/common/network/Selector.java | 1 + 1 file changed, 1 insertion(+) 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 4aee214..aaf60c9 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 @@ -279,6 +279,7 @@ public class Selector implements Selectable { } catch (InvalidReceiveException e) { log.error("Invalid data received from " + transmissions.id + " closing connection", e); close(transmissions.id); + this.disconnected.add(transmissions.id); throw e; } if (transmissions.receive.complete()) { -- 1.7.12.4 From 4204f4a06bf23160ceec4aa54331db62681bff82 Mon Sep 17 00:00:00 2001 From: Joel Koshy Date: Tue, 7 Jul 2015 21:48:42 -0700 Subject: [PATCH 039/120] Trivial commit - explicitly exclude build/rat-report.xml from rat check --- build.gradle | 1 + 1 file changed, 1 insertion(+) diff --git a/build.gradle b/build.gradle index ab86987..2b3c009 100644 --- a/build.gradle +++ b/build.gradle @@ -59,6 +59,7 @@ rat { // And some of the files that we have checked in should also be excluded from this check excludes.addAll([ '**/.git/**', + 'build/rat/rat-report.xml', 'gradlew', 'gradlew.bat', '**/README.md', -- 1.7.12.4 From 67b6b9a45ba6f096645f4f344dfdcf1df1e50dd6 Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Wed, 8 Jul 2015 09:57:16 -0700 Subject: [PATCH 040/120] KAFKA-2308: make MemoryRecords idempotent; reviewed by Guozhang Wang --- .../main/java/org/apache/kafka/common/record/MemoryRecords.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java index b2db240..5f1b45c 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java @@ -115,9 +115,11 @@ public class MemoryRecords implements Records { * Close this batch for no more appends */ public void close() { - compressor.close(); - writable = false; - buffer = compressor.buffer(); + if (writable) { + compressor.close(); + writable = false; + buffer = compressor.buffer(); + } } /** Write the records in this set to the given channel */ -- 1.7.12.4 From 7df39e0394a6fd2f26e8f12768ebf7fecd56e3da Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Wed, 8 Jul 2015 10:22:42 -0700 Subject: [PATCH 041/120] KAFKA-2316: Drop java 1.6 support; patched by Sriharsha Chintalapani reviewed by Ismael Juma and Gwen Shapira --- build.gradle | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/build.gradle b/build.gradle index 2b3c009..535c35d 100644 --- a/build.gradle +++ b/build.gradle @@ -4,9 +4,9 @@ // The ASF licenses this file to You under the Apache License, Version 2.0 // (the "License"); you may not use this file except in compliance with // the License. You may obtain a copy of the License at -// +// // http://www.apache.org/licenses/LICENSE-2.0 -// +// // Unless required by applicable law or agreed to in writing, software // distributed under the License is distributed on an "AS IS" BASIS, // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. @@ -74,7 +74,7 @@ subprojects { apply plugin: 'maven' apply plugin: 'signing' - sourceCompatibility = 1.6 + sourceCompatibility = 1.7 uploadArchives { repositories { -- 1.7.12.4 From ee88dbb67f19b787e12ccef37982c9459b78c7b6 Mon Sep 17 00:00:00 2001 From: Geoff Anderson Date: Thu, 9 Jul 2015 14:58:01 -0700 Subject: [PATCH 042/120] KAFKA-2327; broker doesn't start if config defines advertised.host but not advertised.port Added unit tests as well. These fail without the fix, but pass with the fix. Author: Geoff Anderson Closes #73 from granders/KAFKA-2327 and squashes the following commits: 52a2085 [Geoff Anderson] Cleaned up unecessary toString calls 23b3340 [Geoff Anderson] Fixes KAFKA-2327 --- core/src/main/scala/kafka/server/KafkaConfig.scala | 3 +- .../scala/unit/kafka/server/KafkaConfigTest.scala | 36 +- kafka-merge-pr.py | 442 +++++++++++++++++++++ 3 files changed, 477 insertions(+), 4 deletions(-) create mode 100644 kafka-merge-pr.py diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index c1f0cca..dbe170f 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -725,8 +725,7 @@ case class KafkaConfig (props: java.util.Map[_, _]) extends AbstractConfig(Kafka validateUniquePortAndProtocol(getString(KafkaConfig.AdvertisedListenersProp)) CoreUtils.listenerListToEndPoints(getString(KafkaConfig.AdvertisedListenersProp)) } else if (getString(KafkaConfig.AdvertisedHostNameProp) != null || getInt(KafkaConfig.AdvertisedPortProp) != null) { - CoreUtils.listenerListToEndPoints("PLAINTEXT://" + - getString(KafkaConfig.AdvertisedHostNameProp) + ":" + getInt(KafkaConfig.AdvertisedPortProp)) + CoreUtils.listenerListToEndPoints("PLAINTEXT://" + advertisedHostName + ":" + advertisedPort) } else { getListeners() } diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 2428dbd..d354452 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -155,13 +155,12 @@ class KafkaConfigTest extends JUnit3Suite { @Test def testAdvertiseConfigured() { - val port = "9999" val advertisedHostName = "routable-host" val advertisedPort = "1234" val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect) props.put(KafkaConfig.AdvertisedHostNameProp, advertisedHostName) - props.put(KafkaConfig.AdvertisedPortProp, advertisedPort.toString) + props.put(KafkaConfig.AdvertisedPortProp, advertisedPort) val serverConfig = KafkaConfig.fromProps(props) val endpoints = serverConfig.advertisedListeners @@ -170,8 +169,41 @@ class KafkaConfigTest extends JUnit3Suite { assertEquals(endpoint.host, advertisedHostName) assertEquals(endpoint.port, advertisedPort.toInt) } + + @Test + def testAdvertisePortDefault() { + val advertisedHostName = "routable-host" + val port = "9999" + + val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect) + props.put(KafkaConfig.AdvertisedHostNameProp, advertisedHostName) + props.put(KafkaConfig.PortProp, port) + + val serverConfig = KafkaConfig.fromProps(props) + val endpoints = serverConfig.advertisedListeners + val endpoint = endpoints.get(SecurityProtocol.PLAINTEXT).get + + assertEquals(endpoint.host, advertisedHostName) + assertEquals(endpoint.port, port.toInt) + } + + @Test + def testAdvertiseHostNameDefault() { + val hostName = "routable-host" + val advertisedPort = "9999" + + val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect) + props.put(KafkaConfig.HostNameProp, hostName) + props.put(KafkaConfig.AdvertisedPortProp, advertisedPort) + val serverConfig = KafkaConfig.fromProps(props) + val endpoints = serverConfig.advertisedListeners + val endpoint = endpoints.get(SecurityProtocol.PLAINTEXT).get + assertEquals(endpoint.host, hostName) + assertEquals(endpoint.port, advertisedPort.toInt) + } + @Test def testDuplicateListeners() { val props = new Properties() diff --git a/kafka-merge-pr.py b/kafka-merge-pr.py new file mode 100644 index 0000000..4cc82e0 --- /dev/null +++ b/kafka-merge-pr.py @@ -0,0 +1,442 @@ +#!/usr/bin/env python + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Utility for creating well-formed pull request merges and pushing them to Apache. This script is a modified version +# of the one created by the Spark project (https://github.com/apache/spark/blob/master/dev/merge_spark_pr.py). +# +# Usage: ./kafka-merge-pr.py (see config env vars below) +# +# This utility assumes you already have local a kafka git folder and that you +# have added remotes corresponding to both: +# (i) the github apache kafka mirror and +# (ii) the apache kafka git repo. + +import json +import os +import re +import subprocess +import sys +import urllib2 + +try: + import jira.client + JIRA_IMPORTED = True +except ImportError: + JIRA_IMPORTED = False + +PROJECT_NAME = "kafka" + +CAPITALIZED_PROJECT_NAME = "kafka".upper() + +# Location of the local git repository +REPO_HOME = os.environ.get("%s_HOME" % CAPITALIZED_PROJECT_NAME, os.getcwd()) +# Remote name which points to the GitHub site +PR_REMOTE_NAME = os.environ.get("PR_REMOTE_NAME", "apache-github") +# Remote name which points to Apache git +PUSH_REMOTE_NAME = os.environ.get("PUSH_REMOTE_NAME", "apache") +# ASF JIRA username +JIRA_USERNAME = os.environ.get("JIRA_USERNAME", "") +# ASF JIRA password +JIRA_PASSWORD = os.environ.get("JIRA_PASSWORD", "") + +GITHUB_USER = os.environ.get("GITHUB_USER", "apache") +GITHUB_BASE = "https://github.com/%s/%s/pull" % (GITHUB_USER, PROJECT_NAME) +GITHUB_API_BASE = "https://api.github.com/repos/%s/%s" % (GITHUB_USER, PROJECT_NAME) +JIRA_BASE = "https://issues.apache.org/jira/browse" +JIRA_API_BASE = "https://issues.apache.org/jira" +# Prefix added to temporary branches +TEMP_BRANCH_PREFIX = "PR_TOOL" +# TODO Introduce a convention as this is too brittle +RELEASE_BRANCH_PREFIX = "0." + +DEV_BRANCH_NAME="trunk" + +def get_json(url): + try: + return json.load(urllib2.urlopen(url)) + except urllib2.HTTPError as e: + print "Unable to fetch URL, exiting: %s" % url + sys.exit(-1) + + +def fail(msg): + print msg + clean_up() + sys.exit(-1) + + +def run_cmd(cmd): + print cmd + if isinstance(cmd, list): + return subprocess.check_output(cmd) + else: + return subprocess.check_output(cmd.split(" ")) + + +def continue_maybe(prompt): + result = raw_input("\n%s (y/n): " % prompt) + if result.lower() != "y": + fail("Okay, exiting") + +def clean_up(): + print "Restoring head pointer to %s" % original_head + run_cmd("git checkout %s" % original_head) + + branches = run_cmd("git branch").replace(" ", "").split("\n") + + for branch in filter(lambda x: x.startswith(TEMP_BRANCH_PREFIX), branches): + print "Deleting local branch %s" % branch + run_cmd("git branch -D %s" % branch) + + +# merge the requested PR and return the merge hash +def merge_pr(pr_num, target_ref, title, body, pr_repo_desc): + pr_branch_name = "%s_MERGE_PR_%s" % (TEMP_BRANCH_PREFIX, pr_num) + target_branch_name = "%s_MERGE_PR_%s_%s" % (TEMP_BRANCH_PREFIX, pr_num, target_ref.upper()) + run_cmd("git fetch %s pull/%s/head:%s" % (PR_REMOTE_NAME, pr_num, pr_branch_name)) + run_cmd("git fetch %s %s:%s" % (PUSH_REMOTE_NAME, target_ref, target_branch_name)) + run_cmd("git checkout %s" % target_branch_name) + + had_conflicts = False + try: + run_cmd(['git', 'merge', pr_branch_name, '--squash']) + except Exception as e: + msg = "Error merging: %s\nWould you like to manually fix-up this merge?" % e + continue_maybe(msg) + msg = "Okay, please fix any conflicts and 'git add' conflicting files... Finished?" + continue_maybe(msg) + had_conflicts = True + + commit_authors = run_cmd(['git', 'log', 'HEAD..%s' % pr_branch_name, + '--pretty=format:%an <%ae>']).split("\n") + distinct_authors = sorted(set(commit_authors), + key=lambda x: commit_authors.count(x), reverse=True) + primary_author = distinct_authors[0] + commits = run_cmd(['git', 'log', 'HEAD..%s' % pr_branch_name, + '--pretty=format:%h [%an] %s']).split("\n\n") + + merge_message_flags = [] + + merge_message_flags += ["-m", title] + if body is not None: + # We remove @ symbols from the body to avoid triggering e-mails + # to people every time someone creates a public fork of the project. + merge_message_flags += ["-m", body.replace("@", "")] + + authors = "\n".join(["Author: %s" % a for a in distinct_authors]) + + merge_message_flags += ["-m", authors] + + if had_conflicts: + committer_name = run_cmd("git config --get user.name").strip() + committer_email = run_cmd("git config --get user.email").strip() + message = "This patch had conflicts when merged, resolved by\nCommitter: %s <%s>" % ( + committer_name, committer_email) + merge_message_flags += ["-m", message] + + # The string "Closes #%s" string is required for GitHub to correctly close the PR + merge_message_flags += [ + "-m", + "Closes #%s from %s and squashes the following commits:" % (pr_num, pr_repo_desc)] + for c in commits: + merge_message_flags += ["-m", c] + + run_cmd(['git', 'commit', '--author="%s"' % primary_author] + merge_message_flags) + + continue_maybe("Merge complete (local ref %s). Push to %s?" % ( + target_branch_name, PUSH_REMOTE_NAME)) + + try: + run_cmd('git push %s %s:%s' % (PUSH_REMOTE_NAME, target_branch_name, target_ref)) + except Exception as e: + clean_up() + fail("Exception while pushing: %s" % e) + + merge_hash = run_cmd("git rev-parse %s" % target_branch_name)[:8] + clean_up() + print("Pull request #%s merged!" % pr_num) + print("Merge hash: %s" % merge_hash) + return merge_hash + + +def cherry_pick(pr_num, merge_hash, default_branch): + pick_ref = raw_input("Enter a branch name [%s]: " % default_branch) + if pick_ref == "": + pick_ref = default_branch + + pick_branch_name = "%s_PICK_PR_%s_%s" % (TEMP_BRANCH_PREFIX, pr_num, pick_ref.upper()) + + run_cmd("git fetch %s %s:%s" % (PUSH_REMOTE_NAME, pick_ref, pick_branch_name)) + run_cmd("git checkout %s" % pick_branch_name) + + try: + run_cmd("git cherry-pick -sx %s" % merge_hash) + except Exception as e: + msg = "Error cherry-picking: %s\nWould you like to manually fix-up this merge?" % e + continue_maybe(msg) + msg = "Okay, please fix any conflicts and finish the cherry-pick. Finished?" + continue_maybe(msg) + + continue_maybe("Pick complete (local ref %s). Push to %s?" % ( + pick_branch_name, PUSH_REMOTE_NAME)) + + try: + run_cmd('git push %s %s:%s' % (PUSH_REMOTE_NAME, pick_branch_name, pick_ref)) + except Exception as e: + clean_up() + fail("Exception while pushing: %s" % e) + + pick_hash = run_cmd("git rev-parse %s" % pick_branch_name)[:8] + clean_up() + + print("Pull request #%s picked into %s!" % (pr_num, pick_ref)) + print("Pick hash: %s" % pick_hash) + return pick_ref + + +def fix_version_from_branch(branch, versions): + # Note: Assumes this is a sorted (newest->oldest) list of un-released versions + if branch == DEV_BRANCH_NAME: + return versions[0] + else: + return filter(lambda x: x.name.startswith(branch), versions)[-1] + + +def resolve_jira_issue(merge_branches, comment, default_jira_id=""): + asf_jira = jira.client.JIRA({'server': JIRA_API_BASE}, + basic_auth=(JIRA_USERNAME, JIRA_PASSWORD)) + + jira_id = raw_input("Enter a JIRA id [%s]: " % default_jira_id) + if jira_id == "": + jira_id = default_jira_id + + try: + issue = asf_jira.issue(jira_id) + except Exception as e: + fail("ASF JIRA could not find %s\n%s" % (jira_id, e)) + + cur_status = issue.fields.status.name + cur_summary = issue.fields.summary + cur_assignee = issue.fields.assignee + if cur_assignee is None: + cur_assignee = "NOT ASSIGNED!!!" + else: + cur_assignee = cur_assignee.displayName + + if cur_status == "Resolved" or cur_status == "Closed": + fail("JIRA issue %s already has status '%s'" % (jira_id, cur_status)) + print ("=== JIRA %s ===" % jira_id) + print ("summary\t\t%s\nassignee\t%s\nstatus\t\t%s\nurl\t\t%s/%s\n" % ( + cur_summary, cur_assignee, cur_status, JIRA_BASE, jira_id)) + + versions = asf_jira.project_versions(CAPITALIZED_PROJECT_NAME) + versions = sorted(versions, key=lambda x: x.name, reverse=True) + versions = filter(lambda x: x.raw['released'] is False, versions) + # Consider only x.y.z versions + versions = filter(lambda x: re.match('\d+\.\d+\.\d+', x.name), versions) + + default_fix_versions = map(lambda x: fix_version_from_branch(x, versions).name, merge_branches) + for v in default_fix_versions: + # Handles the case where we have forked a release branch but not yet made the release. + # In this case, if the PR is committed to the master branch and the release branch, we + # only consider the release branch to be the fix version. E.g. it is not valid to have + # both 1.1.0 and 1.0.0 as fix versions. + (major, minor, patch) = v.split(".") + if patch == "0": + previous = "%s.%s.%s" % (major, int(minor) - 1, 0) + if previous in default_fix_versions: + default_fix_versions = filter(lambda x: x != v, default_fix_versions) + default_fix_versions = ",".join(default_fix_versions) + + fix_versions = raw_input("Enter comma-separated fix version(s) [%s]: " % default_fix_versions) + if fix_versions == "": + fix_versions = default_fix_versions + fix_versions = fix_versions.replace(" ", "").split(",") + + def get_version_json(version_str): + return filter(lambda v: v.name == version_str, versions)[0].raw + + jira_fix_versions = map(lambda v: get_version_json(v), fix_versions) + + resolve = filter(lambda a: a['name'] == "Resolve Issue", asf_jira.transitions(jira_id))[0] + asf_jira.transition_issue(jira_id, resolve["id"], fixVersions=jira_fix_versions, comment=comment) + + print "Successfully resolved %s with fixVersions=%s!" % (jira_id, fix_versions) + + +def resolve_jira_issues(title, merge_branches, comment): + jira_ids = re.findall("%s-[0-9]{4,5}" % CAPITALIZED_PROJECT_NAME, title) + + if len(jira_ids) == 0: + resolve_jira_issue(merge_branches, comment) + for jira_id in jira_ids: + resolve_jira_issue(merge_branches, comment, jira_id) + + +def standardize_jira_ref(text): + """ + Standardize the jira reference commit message prefix to "PROJECT_NAME-XXX; Issue" + + >>> standardize_jira_ref("%s-5954; Top by key" % CAPITALIZED_PROJECT_NAME) + 'KAFKA-5954; Top by key' + >>> standardize_jira_ref("%s-5821; ParquetRelation2 CTAS should check if delete is successful" % PROJECT_NAME) + 'KAFKA-5821; ParquetRelation2 CTAS should check if delete is successful' + >>> standardize_jira_ref("%s-4123 [WIP] Show new dependencies added in pull requests" % PROJECT_NAME) + 'KAFKA-4123; [WIP] Show new dependencies added in pull requests' + >>> standardize_jira_ref("%s 5954: Top by key" % PROJECT_NAME) + 'KAFKA-5954; Top by key' + >>> standardize_jira_ref("%s-979 a LRU scheduler for load balancing in TaskSchedulerImpl" % PROJECT_NAME) + 'KAFKA-979; a LRU scheduler for load balancing in TaskSchedulerImpl' + >>> standardize_jira_ref("%s-1094 Support MiMa for reporting binary compatibility across versions." % CAPITALIZED_PROJECT_NAME) + 'KAFKA-1094; Support MiMa for reporting binary compatibility across versions.' + >>> standardize_jira_ref("[WIP] %s-1146; Vagrant support" % CAPITALIZED_PROJECT_NAME) + 'KAFKA-1146; [WIP] Vagrant support' + >>> standardize_jira_ref("%s-1032. If Yarn app fails before registering, app master stays aroun..." % PROJECT_NAME) + 'KAFKA-1032; If Yarn app fails before registering, app master stays aroun...' + >>> standardize_jira_ref("%s-6250 %s-6146 %s-5911: Types are now reserved words in DDL parser." % (PROJECT_NAME, PROJECT_NAME, CAPITALIZED_PROJECT_NAME)) + 'KAFKA-6250 KAFKA-6146 KAFKA-5911; Types are now reserved words in DDL parser.' + >>> standardize_jira_ref("Additional information for users building from source code") + 'Additional information for users building from source code' + """ + jira_refs = [] + components = [] + + # Extract JIRA ref(s): + pattern = re.compile(r'(%s[-\s]*[0-9]{3,6})+' % CAPITALIZED_PROJECT_NAME, re.IGNORECASE) + for ref in pattern.findall(text): + # Add brackets, replace spaces with a dash, & convert to uppercase + jira_refs.append(re.sub(r'\s+', '-', ref.upper())) + text = text.replace(ref, '') + + # Extract project name component(s): + # Look for alphanumeric chars, spaces, dashes, periods, and/or commas + pattern = re.compile(r'(\[[\w\s,-\.]+\])', re.IGNORECASE) + for component in pattern.findall(text): + components.append(component.upper()) + text = text.replace(component, '') + + # Cleanup any remaining symbols: + pattern = re.compile(r'^\W+(.*)', re.IGNORECASE) + if (pattern.search(text) is not None): + text = pattern.search(text).groups()[0] + + # Assemble full text (JIRA ref(s), module(s), remaining text) + jira_prefix = ' '.join(jira_refs).strip() + if jira_prefix: + jira_prefix = jira_prefix + "; " + clean_text = jira_prefix + ' '.join(components).strip() + " " + text.strip() + + # Replace multiple spaces with a single space, e.g. if no jira refs and/or components were included + clean_text = re.sub(r'\s+', ' ', clean_text.strip()) + + return clean_text + +def main(): + global original_head + + original_head = run_cmd("git rev-parse HEAD")[:8] + + branches = get_json("%s/branches" % GITHUB_API_BASE) + branch_names = filter(lambda x: x.startswith(RELEASE_BRANCH_PREFIX), [x['name'] for x in branches]) + # Assumes branch names can be sorted lexicographically + latest_branch = sorted(branch_names, reverse=True)[0] + + pr_num = raw_input("Which pull request would you like to merge? (e.g. 34): ") + pr = get_json("%s/pulls/%s" % (GITHUB_API_BASE, pr_num)) + pr_events = get_json("%s/issues/%s/events" % (GITHUB_API_BASE, pr_num)) + + url = pr["url"] + + # Decide whether to use the modified title or not + modified_title = standardize_jira_ref(pr["title"]) + if modified_title != pr["title"]: + print "I've re-written the title as follows to match the standard format:" + print "Original: %s" % pr["title"] + print "Modified: %s" % modified_title + result = raw_input("Would you like to use the modified title? (y/n): ") + if result.lower() == "y": + title = modified_title + print "Using modified title:" + else: + title = pr["title"] + print "Using original title:" + print title + else: + title = pr["title"] + + body = pr["body"] + target_ref = pr["base"]["ref"] + user_login = pr["user"]["login"] + base_ref = pr["head"]["ref"] + pr_repo_desc = "%s/%s" % (user_login, base_ref) + + # Merged pull requests don't appear as merged in the GitHub API; + # Instead, they're closed by asfgit. + merge_commits = \ + [e for e in pr_events if e["actor"]["login"] == "asfgit" and e["event"] == "closed"] + + if merge_commits: + merge_hash = merge_commits[0]["commit_id"] + message = get_json("%s/commits/%s" % (GITHUB_API_BASE, merge_hash))["commit"]["message"] + + print "Pull request %s has already been merged, assuming you want to backport" % pr_num + commit_is_downloaded = run_cmd(['git', 'rev-parse', '--quiet', '--verify', + "%s^{commit}" % merge_hash]).strip() != "" + if not commit_is_downloaded: + fail("Couldn't find any merge commit for #%s, you may need to update HEAD." % pr_num) + + print "Found commit %s:\n%s" % (merge_hash, message) + cherry_pick(pr_num, merge_hash, latest_branch) + sys.exit(0) + + if not bool(pr["mergeable"]): + msg = "Pull request %s is not mergeable in its current form.\n" % pr_num + \ + "Continue? (experts only!)" + continue_maybe(msg) + + print ("\n=== Pull Request #%s ===" % pr_num) + print ("title\t%s\nsource\t%s\ntarget\t%s\nurl\t%s" % ( + title, pr_repo_desc, target_ref, url)) + continue_maybe("Proceed with merging pull request #%s?" % pr_num) + + merged_refs = [target_ref] + + merge_hash = merge_pr(pr_num, target_ref, title, body, pr_repo_desc) + + pick_prompt = "Would you like to pick %s into another branch?" % merge_hash + while raw_input("\n%s (y/n): " % pick_prompt).lower() == "y": + merged_refs = merged_refs + [cherry_pick(pr_num, merge_hash, latest_branch)] + + if JIRA_IMPORTED: + if JIRA_USERNAME and JIRA_PASSWORD: + continue_maybe("Would you like to update an associated JIRA?") + jira_comment = "Issue resolved by pull request %s\n[%s/%s]" % (pr_num, GITHUB_BASE, pr_num) + resolve_jira_issues(title, merged_refs, jira_comment) + else: + print "JIRA_USERNAME and JIRA_PASSWORD not set" + print "Exiting without trying to close the associated JIRA." + else: + print "Could not find jira-python library. Run 'sudo pip install jira-python' to install." + print "Exiting without trying to close the associated JIRA." + +if __name__ == "__main__": + import doctest + doctest.testmod() + + main() -- 1.7.12.4 From 9ca61d17915f09b8010fa1da5ad0285b076a96e1 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Thu, 9 Jul 2015 18:15:31 -0700 Subject: [PATCH 043/120] KAFKA-1740 follow-up: add state checking in handling heartbeat request; reviewed by Onur Karaman, Ewen Cheslack-Postavam and Guozhang Wang --- .../kafka/coordinator/ConsumerCoordinator.scala | 2 +- .../ConsumerCoordinatorResponseTest.scala | 45 +++++++++++++++++++--- 2 files changed, 40 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala b/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala index 476973b..6c2df4c 100644 --- a/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala @@ -210,7 +210,7 @@ class ConsumerCoordinator(val brokerId: Int, responseCallback(Errors.UNKNOWN_CONSUMER_ID.code) } else if (!group.has(consumerId)) { responseCallback(Errors.UNKNOWN_CONSUMER_ID.code) - } else if (generationId != group.generationId) { + } else if (generationId != group.generationId || !group.is(Stable)) { responseCallback(Errors.ILLEGAL_GENERATION.code) } else { val consumer = group.get(consumerId) diff --git a/core/src/test/scala/unit/kafka/coordinator/ConsumerCoordinatorResponseTest.scala b/core/src/test/scala/unit/kafka/coordinator/ConsumerCoordinatorResponseTest.scala index 3cd726d..87a5330 100644 --- a/core/src/test/scala/unit/kafka/coordinator/ConsumerCoordinatorResponseTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/ConsumerCoordinatorResponseTest.scala @@ -43,7 +43,7 @@ class ConsumerCoordinatorResponseTest extends JUnitSuite { type HeartbeatCallback = Short => Unit val ConsumerMinSessionTimeout = 10 - val ConsumerMaxSessionTimeout = 30 + val ConsumerMaxSessionTimeout = 100 val DefaultSessionTimeout = 20 var consumerCoordinator: ConsumerCoordinator = null var offsetManager : OffsetManager = null @@ -232,6 +232,30 @@ class ConsumerCoordinatorResponseTest extends JUnitSuite { } @Test + def testHeartbeatDuringRebalanceCausesIllegalGeneration() { + val groupId = "groupId" + val partitionAssignmentStrategy = "range" + + // First start up a group (with a slightly larger timeout to give us time to heartbeat when the rebalance starts) + val joinGroupResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_CONSUMER_ID, partitionAssignmentStrategy, + 100, isCoordinatorForGroup = true) + val assignedConsumerId = joinGroupResult._2 + val initialGenerationId = joinGroupResult._3 + val joinGroupErrorCode = joinGroupResult._4 + assertEquals(Errors.NONE.code, joinGroupErrorCode) + + // Then join with a new consumer to trigger a rebalance + EasyMock.reset(offsetManager) + sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_CONSUMER_ID, partitionAssignmentStrategy, + DefaultSessionTimeout, isCoordinatorForGroup = true) + + // We should be in the middle of a rebalance, so the heartbeat should return illegal generation + EasyMock.reset(offsetManager) + val heartbeatResult = heartbeat(groupId, assignedConsumerId, initialGenerationId, isCoordinatorForGroup = true) + assertEquals(Errors.ILLEGAL_GENERATION.code, heartbeatResult) + } + + @Test def testGenerationIdIncrementsOnRebalance() { val groupId = "groupId" val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID @@ -267,16 +291,25 @@ class ConsumerCoordinatorResponseTest extends JUnitSuite { (responseFuture, responseCallback) } - private def joinGroup(groupId: String, - consumerId: String, - partitionAssignmentStrategy: String, - sessionTimeout: Int, - isCoordinatorForGroup: Boolean): JoinGroupCallbackParams = { + private def sendJoinGroup(groupId: String, + consumerId: String, + partitionAssignmentStrategy: String, + sessionTimeout: Int, + isCoordinatorForGroup: Boolean): Future[JoinGroupCallbackParams] = { val (responseFuture, responseCallback) = setupJoinGroupCallback EasyMock.expect(offsetManager.partitionFor(groupId)).andReturn(1) EasyMock.expect(offsetManager.leaderIsLocal(1)).andReturn(isCoordinatorForGroup) EasyMock.replay(offsetManager) consumerCoordinator.handleJoinGroup(groupId, consumerId, Set.empty, sessionTimeout, partitionAssignmentStrategy, responseCallback) + responseFuture + } + + private def joinGroup(groupId: String, + consumerId: String, + partitionAssignmentStrategy: String, + sessionTimeout: Int, + isCoordinatorForGroup: Boolean): JoinGroupCallbackParams = { + val responseFuture = sendJoinGroup(groupId, consumerId, partitionAssignmentStrategy, sessionTimeout, isCoordinatorForGroup) Await.result(responseFuture, Duration(40, TimeUnit.MILLISECONDS)) } -- 1.7.12.4 From 4aba4bc1dfb74cc378ea58d7b0a9f16c96cfe757 Mon Sep 17 00:00:00 2001 From: Joel Koshy Date: Thu, 9 Jul 2015 18:28:38 -0700 Subject: [PATCH 044/120] Trivial commit - explicitly exclude build/** from rat check --- build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index 535c35d..d86f1a8 100644 --- a/build.gradle +++ b/build.gradle @@ -59,7 +59,7 @@ rat { // And some of the files that we have checked in should also be excluded from this check excludes.addAll([ '**/.git/**', - 'build/rat/rat-report.xml', + 'build/**', 'gradlew', 'gradlew.bat', '**/README.md', -- 1.7.12.4 From 69b451e28944deb162f7427105c3090f41c8797f Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Mon, 13 Jul 2015 13:13:02 -0700 Subject: [PATCH 045/120] KAFKA-2312: use atomic long for thread id reference; reviewed by Ewen Cheslack-Postava, Jason Gustafson, Ismael Juma and Guozhang Wang --- .../java/org/apache/kafka/clients/consumer/KafkaConsumer.java | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) 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 7aa0760..b4e8f7f 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 @@ -51,6 +51,7 @@ import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import static org.apache.kafka.common.utils.Utils.min; @@ -395,6 +396,7 @@ public class KafkaConsumer implements Consumer { private static final Logger log = LoggerFactory.getLogger(KafkaConsumer.class); private static final long EARLIEST_OFFSET_TIMESTAMP = -2L; private static final long LATEST_OFFSET_TIMESTAMP = -1L; + private static final long NO_CURRENT_THREAD = -1L; private static final AtomicInteger CONSUMER_CLIENT_ID_SEQUENCE = new AtomicInteger(1); private final Coordinator coordinator; @@ -417,7 +419,7 @@ public class KafkaConsumer implements Consumer { // currentThread holds the threadId of the current thread accessing KafkaConsumer // and is used to prevent multi-threaded access - private final AtomicReference currentThread = new AtomicReference(); + private final AtomicLong currentThread = new AtomicLong(NO_CURRENT_THREAD); // refcount is used to allow reentrant access by the thread who has acquired currentThread private final AtomicInteger refcount = new AtomicInteger(0); @@ -1355,8 +1357,8 @@ public class KafkaConsumer implements Consumer { */ private void acquire() { ensureNotClosed(); - Long threadId = Thread.currentThread().getId(); - if (!threadId.equals(currentThread.get()) && !currentThread.compareAndSet(null, threadId)) + long threadId = Thread.currentThread().getId(); + if (threadId != currentThread.get() && !currentThread.compareAndSet(NO_CURRENT_THREAD, threadId)) throw new ConcurrentModificationException("KafkaConsumer is not safe for multi-threaded access"); refcount.incrementAndGet(); } @@ -1366,6 +1368,6 @@ public class KafkaConsumer implements Consumer { */ private void release() { if (refcount.decrementAndGet() == 0) - currentThread.set(null); + currentThread.set(NO_CURRENT_THREAD); } } -- 1.7.12.4 From bdbb9672f5e035fd00801037e2affe64811ec6ab Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Mon, 13 Jul 2015 17:16:34 -0700 Subject: [PATCH 046/120] kafka-972; MetadataRequest returns stale list of brokers; patched by Ashish Singh; reviewed by Jun Rao --- .../scala/kafka/controller/KafkaController.scala | 21 +++++-- .../unit/kafka/integration/TopicMetadataTest.scala | 66 ++++++++++++++++++++-- 2 files changed, 77 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 20f1499..b4fc755 100755 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -387,8 +387,9 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt /** * This callback is invoked by the replica state machine's broker change listener, with the list of newly started * brokers as input. It does the following - - * 1. Triggers the OnlinePartition state change for all new/offline partitions - * 2. It checks whether there are reassigned replicas assigned to any newly started brokers. If + * 1. Sends update metadata request to all live and shutting down brokers + * 2. Triggers the OnlinePartition state change for all new/offline partitions + * 3. It checks whether there are reassigned replicas assigned to any newly started brokers. If * so, it performs the reassignment logic for each topic/partition. * * Note that we don't need to refresh the leader/isr cache for all topic/partitions at this point for two reasons: @@ -400,10 +401,11 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt def onBrokerStartup(newBrokers: Seq[Int]) { info("New broker startup callback for %s".format(newBrokers.mkString(","))) val newBrokersSet = newBrokers.toSet - // send update metadata request for all partitions to the newly restarted brokers. In cases of controlled shutdown - // leaders will not be elected when a new broker comes up. So at least in the common controlled shutdown case, the - // metadata will reach the new brokers faster - sendUpdateMetadataRequest(newBrokers) + // send update metadata request to all live and shutting down brokers. Old brokers will get to know of the new + // broker via this update. + // In cases of controlled shutdown leaders will not be elected when a new broker comes up. So at least in the + // common controlled shutdown case, the metadata will reach the new brokers faster + sendUpdateMetadataRequest(controllerContext.liveOrShuttingDownBrokerIds.toSeq) // the very first thing to do when a new broker comes up is send it the entire list of partitions that it is // supposed to host. Based on that the broker starts the high watermark threads for the input list of partitions val allReplicasOnNewBrokers = controllerContext.replicasOnBrokers(newBrokersSet) @@ -433,6 +435,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt * 1. Mark partitions with dead leaders as offline * 2. Triggers the OnlinePartition state change for all new/offline partitions * 3. Invokes the OfflineReplica state change on the input list of newly started brokers + * 4. If no partitions are effected then send UpdateMetadataRequest to live or shutting down brokers * * Note that we don't need to refresh the leader/isr cache for all topic/partitions at this point. This is because * the partition state machine will refresh our cache for us when performing leader election for all new/offline @@ -464,6 +467,12 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt // since topic deletion cannot be retried until at least one replica is in TopicDeletionStarted state deleteTopicManager.failReplicaDeletion(replicasForTopicsToBeDeleted) } + + // If broker failure did not require leader re-election, inform brokers of failed broker + // Note that during leader re-election, brokers update their metadata + if (partitionsWithoutLeader.isEmpty) { + sendUpdateMetadataRequest(controllerContext.liveOrShuttingDownBrokerIds.toSeq) + } } /** diff --git a/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala b/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala index a95ee5e..5b6c9d6 100644 --- a/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala +++ b/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala @@ -25,7 +25,7 @@ import kafka.api.{TopicMetadataResponse, TopicMetadataRequest} import kafka.client.ClientUtils import kafka.cluster.{Broker, BrokerEndPoint} import kafka.common.ErrorMapping -import kafka.server.{NotRunning, KafkaConfig, KafkaServer} +import kafka.server.{KafkaConfig, KafkaServer, NotRunning} import kafka.utils.TestUtils import kafka.utils.TestUtils._ import kafka.zk.ZooKeeperTestHarness @@ -36,7 +36,7 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { private var server1: KafkaServer = null var brokerEndPoints: Seq[BrokerEndPoint] = null var adHocConfigs: Seq[KafkaConfig] = null - val numConfigs: Int = 2 + val numConfigs: Int = 4 override def setUp() { super.setUp() @@ -171,13 +171,15 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { def testIsrAfterBrokerShutDownAndJoinsBack { + val numBrokers = 2 //just 2 brokers are enough for the test + // start adHoc brokers - val adHocServers = adHocConfigs.map(p => createServer(p)) + val adHocServers = adHocConfigs.take(numBrokers - 1).map(p => createServer(p)) val allServers: Seq[KafkaServer] = Seq(server1) ++ adHocServers // create topic val topic: String = "test" - AdminUtils.createTopic(zkClient, topic, 1, numConfigs) + AdminUtils.createTopic(zkClient, topic, 1, numBrokers) // shutdown a broker adHocServers.last.shutdown() @@ -192,4 +194,60 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { // shutdown adHoc brokers adHocServers.map(p => p.shutdown()) } + + private def checkMetadata(servers: Seq[KafkaServer], expectedBrokersCount: Int): Unit = { + var topicMetadata: TopicMetadataResponse = new TopicMetadataResponse(Seq(), Seq(), -1) + + // Get topic metadata from old broker + // Wait for metadata to get updated by checking metadata from a new broker + waitUntilTrue(() => { + topicMetadata = ClientUtils.fetchTopicMetadata( + Set.empty, brokerEndPoints, "TopicMetadataTest-testBasicTopicMetadata", 2000, 0) + topicMetadata.brokers.size == expectedBrokersCount}, + "Alive brokers list is not correctly propagated by coordinator to brokers" + ) + + // Assert that topic metadata at new brokers is updated correctly + servers.filter(x => x.brokerState.currentState != NotRunning.state).foreach(x => + waitUntilTrue(() => + topicMetadata == ClientUtils.fetchTopicMetadata( + Set.empty, + Seq(new Broker(x.config.brokerId, + x.config.hostName, + x.boundPort()).getBrokerEndPoint(SecurityProtocol.PLAINTEXT)), + "TopicMetadataTest-testBasicTopicMetadata", + 2000, 0), "Topic metadata is not correctly updated")) + } + + + def testAliveBrokerListWithNoTopics { + checkMetadata(Seq(server1), 1) + } + + def testAliveBrokersListWithNoTopicsAfterNewBrokerStartup { + var adHocServers = adHocConfigs.takeRight(adHocConfigs.size - 1).map(p => createServer(p)) + + checkMetadata(adHocServers, numConfigs - 1) + + // Add a broker + adHocServers = adHocServers ++ Seq(createServer(adHocConfigs.head)) + + checkMetadata(adHocServers, numConfigs) + adHocServers.map(p => p.shutdown()) + } + + + def testAliveBrokersListWithNoTopicsAfterABrokerShutdown { + val adHocServers = adHocConfigs.map(p => createServer(p)) + + checkMetadata(adHocServers, numConfigs) + + // Shutdown a broker + adHocServers.last.shutdown() + adHocServers.last.awaitShutdown() + + checkMetadata(adHocServers, numConfigs - 1) + + adHocServers.map(p => p.shutdown()) + } } -- 1.7.12.4 From a7e0ac3659c2b499124a866bc0b16b6b1b412376 Mon Sep 17 00:00:00 2001 From: Manikumar Reddy Date: Mon, 13 Jul 2015 22:08:33 -0700 Subject: [PATCH 047/120] KAFKA-2198: kafka-topics.sh exits with 0 status on failures; patched by Manikumar Reddy reviewed by Gwen Shapira --- core/src/main/scala/kafka/admin/TopicCommand.scala | 17 +++++++++++------ 1 file changed, 11 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala index a2ecb96..a90aa87 100755 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -31,7 +31,7 @@ import org.apache.kafka.common.utils.Utils import kafka.coordinator.ConsumerCoordinator -object TopicCommand { +object TopicCommand extends Logging { def main(args: Array[String]): Unit = { @@ -48,7 +48,7 @@ object TopicCommand { opts.checkArgs() val zkClient = ZkUtils.createZkClient(opts.options.valueOf(opts.zkConnectOpt), 30000, 30000) - + var exitCode = 0 try { if(opts.options.has(opts.createOpt)) createTopic(zkClient, opts) @@ -62,11 +62,14 @@ object TopicCommand { deleteTopic(zkClient, opts) } catch { case e: Throwable => - println("Error while executing topic command " + e.getMessage) - println(Utils.stackTrace(e)) + println("Error while executing topic command : " + e.getMessage) + error(Utils.stackTrace(e)) + exitCode = 1 } finally { zkClient.close() + System.exit(exitCode) } + } private def getTopics(zkClient: ZkClient, opts: TopicCommandOptions): Seq[String] = { @@ -97,7 +100,8 @@ object TopicCommand { def alterTopic(zkClient: ZkClient, opts: TopicCommandOptions) { val topics = getTopics(zkClient, opts) if (topics.length == 0) { - println("Topic %s does not exist".format(opts.options.valueOf(opts.topicOpt))) + throw new IllegalArgumentException("Topic %s does not exist on ZK path %s".format(opts.options.valueOf(opts.topicOpt), + opts.options.valueOf(opts.zkConnectOpt))) } topics.foreach { topic => val configs = AdminUtils.fetchTopicConfig(zkClient, topic) @@ -138,7 +142,8 @@ object TopicCommand { def deleteTopic(zkClient: ZkClient, opts: TopicCommandOptions) { val topics = getTopics(zkClient, opts) if (topics.length == 0) { - println("Topic %s does not exist".format(opts.options.valueOf(opts.topicOpt))) + throw new IllegalArgumentException("Topic %s does not exist on ZK path %s".format(opts.options.valueOf(opts.topicOpt), + opts.options.valueOf(opts.zkConnectOpt))) } topics.foreach { topic => try { -- 1.7.12.4 From 99c0686be2141a0fffe1c55e279370a87ef8c1ea Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Wed, 15 Jul 2015 12:38:45 -0700 Subject: [PATCH 048/120] KAFKA-2123: add callback in commit api and use a delayed queue for async requests; reviewed by Ewen Cheslack-Postava and Guozhang Wang --- .../apache/kafka/clients/consumer/Consumer.java | 10 + .../clients/consumer/ConsumerCommitCallback.java | 33 ++ .../consumer/ConsumerRebalanceCallback.java | 6 +- .../kafka/clients/consumer/ConsumerRecords.java | 4 + .../kafka/clients/consumer/KafkaConsumer.java | 512 ++++------------ .../kafka/clients/consumer/MockConsumer.java | 19 +- .../consumer/internals/ConsumerNetworkClient.java | 296 +++++++++ .../clients/consumer/internals/Coordinator.java | 660 +++++++++++++-------- .../clients/consumer/internals/DelayedTask.java | 24 + .../consumer/internals/DelayedTaskQueue.java | 96 +++ .../kafka/clients/consumer/internals/Fetcher.java | 182 ++++-- .../clients/consumer/internals/Heartbeat.java | 27 +- .../internals/NoAvailableBrokersException.java | 23 + .../clients/consumer/internals/RequestFuture.java | 219 ++++--- .../consumer/internals/RequestFutureAdapter.java | 28 + .../consumer/internals/RequestFutureListener.java | 23 + .../consumer/internals/SendFailedException.java | 27 + .../consumer/internals/StaleMetadataException.java | 22 + .../consumer/internals/SubscriptionState.java | 5 +- .../ConsumerCoordinatorNotAvailableException.java | 40 ++ .../kafka/common/errors/DisconnectException.java | 39 ++ .../common/errors/IllegalGenerationException.java | 33 ++ .../errors/NotCoordinatorForConsumerException.java | 40 ++ .../errors/OffsetLoadInProgressException.java | 40 ++ .../common/errors/UnknownConsumerIdException.java | 33 ++ .../org/apache/kafka/common/protocol/Errors.java | 10 +- .../internals/ConsumerNetworkClientTest.java | 125 ++++ .../consumer/internals/CoordinatorTest.java | 479 ++++++++++----- .../consumer/internals/DelayedTaskQueueTest.java | 89 +++ .../clients/consumer/internals/FetcherTest.java | 37 +- .../clients/consumer/internals/HeartbeatTest.java | 15 + .../consumer/internals/RequestFutureTest.java | 57 ++ .../scala/integration/kafka/api/ConsumerTest.scala | 81 ++- 33 files changed, 2350 insertions(+), 984 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerCommitCallback.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedTask.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedTaskQueue.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoAvailableBrokersException.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFutureAdapter.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFutureListener.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/SendFailedException.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/StaleMetadataException.java create mode 100644 clients/src/main/java/org/apache/kafka/common/errors/ConsumerCoordinatorNotAvailableException.java create mode 100644 clients/src/main/java/org/apache/kafka/common/errors/DisconnectException.java create mode 100644 clients/src/main/java/org/apache/kafka/common/errors/IllegalGenerationException.java create mode 100644 clients/src/main/java/org/apache/kafka/common/errors/NotCoordinatorForConsumerException.java create mode 100644 clients/src/main/java/org/apache/kafka/common/errors/OffsetLoadInProgressException.java create mode 100644 clients/src/main/java/org/apache/kafka/common/errors/UnknownConsumerIdException.java create mode 100644 clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest.java create mode 100644 clients/src/test/java/org/apache/kafka/clients/consumer/internals/DelayedTaskQueueTest.java create mode 100644 clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestFutureTest.java diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java index fd98740..252b759 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java @@ -64,11 +64,21 @@ public interface Consumer extends Closeable { public void commit(CommitType commitType); /** + * @see KafkaConsumer#commit(CommitType, ConsumerCommitCallback) + */ + public void commit(CommitType commitType, ConsumerCommitCallback callback); + + /** * @see KafkaConsumer#commit(Map, CommitType) */ public void commit(Map offsets, CommitType commitType); /** + * @see KafkaConsumer#commit(Map, CommitType, ConsumerCommitCallback) + */ + public void commit(Map offsets, CommitType commitType, ConsumerCommitCallback callback); + + /** * @see KafkaConsumer#seek(TopicPartition, long) */ public void seek(TopicPartition partition, long offset); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerCommitCallback.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerCommitCallback.java new file mode 100644 index 0000000..f084385 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerCommitCallback.java @@ -0,0 +1,33 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package org.apache.kafka.clients.consumer; + +import org.apache.kafka.common.TopicPartition; + +import java.util.Map; + +/** + * A callback interface that the user can implement to trigger custom actions when a commit request completes. The callback + * may be executed in any thread calling {@link Consumer#poll(long) poll()}. + */ +public interface ConsumerCommitCallback { + + /** + * A callback method the user can implement to provide asynchronous handling of commit request completion. + * This method will be called when the commit request sent to the server has been acknowledged. + * + * @param offsets A map of the offsets that this callback applies to + * @param exception The exception thrown during processing of the request, or null if the commit completed successfully + */ + void onComplete(Map offsets, Exception exception); +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.java index 74dfdba..ff3f50f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.java @@ -73,7 +73,8 @@ public interface ConsumerRebalanceCallback { * It is guaranteed that all the processes in a consumer group will execute their * {@link #onPartitionsRevoked(Consumer, Collection)} callback before any instance executes its * {@link #onPartitionsAssigned(Consumer, Collection)} callback. - * + * + * @param consumer Reference to the consumer for convenience * @param partitions The list of partitions that are now assigned to the consumer (may include partitions previously * assigned to the consumer) */ @@ -86,7 +87,8 @@ public interface ConsumerRebalanceCallback { * custom offset store to prevent duplicate data *

    * For examples on usage of this API, see Usage Examples section of {@link KafkaConsumer KafkaConsumer} - * + * + * @param consumer Reference to the consumer for convenience * @param partitions The list of partitions that were assigned to the consumer on the last rebalance */ public void onPartitionsRevoked(Consumer consumer, Collection partitions); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java index eb75d2e..16a8357 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java @@ -105,6 +105,10 @@ public class ConsumerRecords implements Iterable> { } } + public boolean isEmpty() { + return records.isEmpty(); + } + @SuppressWarnings("unchecked") public static ConsumerRecords empty() { return (ConsumerRecords) EMPTY; 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 b4e8f7f..9f64255 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 @@ -15,9 +15,10 @@ package org.apache.kafka.clients.consumer; import org.apache.kafka.clients.ClientUtils; import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.NetworkClient; +import org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient; import org.apache.kafka.clients.consumer.internals.Coordinator; +import org.apache.kafka.clients.consumer.internals.DelayedTask; import org.apache.kafka.clients.consumer.internals.Fetcher; -import org.apache.kafka.clients.consumer.internals.RequestFuture; import org.apache.kafka.clients.consumer.internals.SubscriptionState; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.KafkaException; @@ -49,7 +50,6 @@ import java.util.Map; import java.util.Properties; import java.util.Set; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; @@ -394,8 +394,6 @@ import static org.apache.kafka.common.utils.Utils.min; public class KafkaConsumer implements Consumer { private static final Logger log = LoggerFactory.getLogger(KafkaConsumer.class); - private static final long EARLIEST_OFFSET_TIMESTAMP = -2L; - private static final long LATEST_OFFSET_TIMESTAMP = -1L; private static final long NO_CURRENT_THREAD = -1L; private static final AtomicInteger CONSUMER_CLIENT_ID_SEQUENCE = new AtomicInteger(1); @@ -405,17 +403,14 @@ public class KafkaConsumer implements Consumer { private final Fetcher fetcher; private final Time time; - private final NetworkClient client; + private final ConsumerNetworkClient client; private final Metrics metrics; private final SubscriptionState subscriptions; private final Metadata metadata; private final long retryBackoffMs; private final boolean autoCommit; private final long autoCommitIntervalMs; - private final ConsumerRebalanceCallback rebalanceCallback; - private long lastCommitAttemptMs; private boolean closed = false; - private final AtomicBoolean wakeup = new AtomicBoolean(false); // currentThread holds the threadId of the current thread accessing KafkaConsumer // and is used to prevent multi-threaded access @@ -507,14 +502,11 @@ public class KafkaConsumer implements Consumer { try { log.debug("Starting the Kafka consumer"); if (callback == null) - this.rebalanceCallback = config.getConfiguredInstance(ConsumerConfig.CONSUMER_REBALANCE_CALLBACK_CLASS_CONFIG, + callback = config.getConfiguredInstance(ConsumerConfig.CONSUMER_REBALANCE_CALLBACK_CLASS_CONFIG, ConsumerRebalanceCallback.class); - else - this.rebalanceCallback = callback; this.time = new SystemTime(); this.autoCommit = config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG); this.autoCommitIntervalMs = config.getLong(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG); - this.lastCommitAttemptMs = time.milliseconds(); MetricConfig metricConfig = new MetricConfig().samples(config.getInt(ConsumerConfig.METRICS_NUM_SAMPLES_CONFIG)) .timeWindow(config.getLong(ConsumerConfig.METRICS_SAMPLE_WINDOW_MS_CONFIG), @@ -535,7 +527,7 @@ public class KafkaConsumer implements Consumer { String metricGrpPrefix = "consumer"; Map metricsTags = new LinkedHashMap(); metricsTags.put("client-id", clientId); - this.client = new NetworkClient( + NetworkClient netClient = new NetworkClient( new Selector(config.getLong(ConsumerConfig.CONNECTIONS_MAX_IDLE_MS_CONFIG), metrics, time, metricGrpPrefix, metricsTags), this.metadata, clientId, @@ -543,6 +535,7 @@ public class KafkaConsumer implements Consumer { config.getLong(ConsumerConfig.RECONNECT_BACKOFF_MS_CONFIG), config.getInt(ConsumerConfig.SEND_BUFFER_CONFIG), config.getInt(ConsumerConfig.RECEIVE_BUFFER_CONFIG)); + this.client = new ConsumerNetworkClient(netClient, metadata, time, retryBackoffMs); OffsetResetStrategy offsetResetStrategy = OffsetResetStrategy.valueOf(config.getString(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG).toUpperCase()); this.subscriptions = new SubscriptionState(offsetResetStrategy); this.coordinator = new Coordinator(this.client, @@ -553,8 +546,10 @@ public class KafkaConsumer implements Consumer { metrics, metricGrpPrefix, metricsTags, - this.time); - + this.time, + requestTimeoutMs, + retryBackoffMs, + wrapRebalanceCallback(callback)); if (keyDeserializer == null) { this.keyDeserializer = config.getConfiguredInstance(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, Deserializer.class); @@ -581,10 +576,14 @@ public class KafkaConsumer implements Consumer { metrics, metricGrpPrefix, metricsTags, - this.time); + this.time, + this.retryBackoffMs); config.logUnused(); + if (autoCommit) + scheduleAutoCommitTask(autoCommitIntervalMs); + log.debug("Kafka consumer created"); } catch (Throwable t) { // call close methods if internal objects are already constructed @@ -719,27 +718,25 @@ public class KafkaConsumer implements Consumer { if (timeout < 0) throw new IllegalArgumentException("Timeout must not be negative"); - // Poll for new data until the timeout expires + // poll for new data until the timeout expires long remaining = timeout; while (remaining >= 0) { long start = time.milliseconds(); - long pollTimeout = min(remaining, timeToNextCommit(start), coordinator.timeToNextHeartbeat(start)); - - Map>> records = pollOnce(pollTimeout, start); + Map>> records = pollOnce(remaining); long end = time.milliseconds(); if (!records.isEmpty()) { - // If data is available, then return it, but first send off the + // if data is available, then return it, but first send off the // next round of fetches to enable pipelining while the user is // handling the fetched records. - fetcher.initFetches(metadata.fetch(), end); - pollClient(0, end); + fetcher.initFetches(metadata.fetch()); + client.poll(0); return new ConsumerRecords(records); } remaining -= end - start; - // Nothing was available, so we should backoff before retrying + // nothing was available, so we should backoff before retrying if (remaining > 0) { Utils.sleep(min(remaining, retryBackoffMs)); remaining -= time.milliseconds() - end; @@ -752,46 +749,42 @@ public class KafkaConsumer implements Consumer { } } - /** * Do one round of polling. In addition to checking for new data, this does any needed * heart-beating, auto-commits, and offset updates. * @param timeout The maximum time to block in the underlying poll - * @param now Current time in millis * @return The fetched records (may be empty) */ - private Map>> pollOnce(long timeout, long now) { - Cluster cluster = this.metadata.fetch(); - + private Map>> pollOnce(long timeout) { // TODO: Sub-requests should take into account the poll timeout (KAFKA-1894) + coordinator.ensureCoordinatorKnown(); - if (subscriptions.partitionsAutoAssigned()) { - if (subscriptions.partitionAssignmentNeeded()) { - // rebalance to get partition assignment - reassignPartitions(now); - } else { - // try to heartbeat with the coordinator if needed - coordinator.maybeHeartbeat(now); - } - } + // ensure we have partitions assigned if we expect to + if (subscriptions.partitionsAutoAssigned()) + coordinator.ensurePartitionAssignment(); // fetch positions if we have partitions we're subscribed to that we // don't know the offset for if (!subscriptions.hasAllFetchPositions()) updateFetchPositions(this.subscriptions.missingFetchPositions()); - // maybe autocommit position - if (shouldAutoCommit(now)) - commit(CommitType.ASYNC); - - // Init any new fetches (won't resend pending fetches) - fetcher.initFetches(cluster, now); - - pollClient(timeout, now); - + // init any new fetches (won't resend pending fetches) + Cluster cluster = this.metadata.fetch(); + fetcher.initFetches(cluster); + client.poll(timeout); return fetcher.fetchedRecords(); } + private void scheduleAutoCommitTask(final long interval) { + DelayedTask task = new DelayedTask() { + public void run(long now) { + commit(CommitType.ASYNC); + client.schedule(this, now + interval); + } + }; + client.schedule(task, time.milliseconds() + interval); + } + /** * Commits the specified offsets for the specified list of topics and partitions to Kafka. *

    @@ -799,25 +792,42 @@ public class KafkaConsumer implements Consumer { * rebalance and also on startup. As such, if you need to store offsets in anything other than Kafka, this API * should not be used. *

    - * A non-blocking commit will attempt to commit offsets asynchronously. No error will be thrown if the commit fails. - * A blocking commit will wait for a response acknowledging the commit. In the event of an error it will retry until - * the commit succeeds. - * + * Asynchronous commits (i.e. {@link CommitType#ASYNC} will not block. Any errors encountered during an asynchronous + * commit are silently discarded. If you need to determine the result of an asynchronous commit, you should use + * {@link #commit(Map, CommitType, ConsumerCommitCallback)}. Synchronous commits (i.e. {@link CommitType#SYNC}) + * block until either the commit succeeds or an unrecoverable error is encountered (in which case it is thrown + * to the caller). + * * @param offsets The list of offsets per partition that should be committed to Kafka. * @param commitType Control whether the commit is blocking */ @Override public void commit(final Map offsets, CommitType commitType) { + commit(offsets, commitType, null); + } + + /** + * Commits the specified offsets for the specified list of topics and partitions to Kafka. + *

    + * This commits offsets to Kafka. The offsets committed using this API will be used on the first fetch after every + * rebalance and also on startup. As such, if you need to store offsets in anything other than Kafka, this API + * should not be used. + *

    + * Asynchronous commits (i.e. {@link CommitType#ASYNC} will not block. Any errors encountered during an asynchronous + * commit are either passed to the callback (if provided) or silently discarded. Synchronous commits (i.e. + * {@link CommitType#SYNC}) block until either the commit succeeds or an unrecoverable error is encountered. In + * this case, the error is either passed to the callback (if provided) or thrown to the caller. + * + * @param offsets The list of offsets per partition that should be committed to Kafka. + * @param commitType Control whether the commit is blocking + * @param callback Callback to invoke when the commit completes + */ + @Override + public void commit(final Map offsets, CommitType commitType, ConsumerCommitCallback callback) { acquire(); try { log.debug("Committing offsets ({}): {} ", commitType.toString().toLowerCase(), offsets); - - this.lastCommitAttemptMs = time.milliseconds(); - - // commit the offsets with the coordinator - if (commitType == CommitType.ASYNC) - this.subscriptions.needRefreshCommits(); - commitOffsets(offsets, commitType); + coordinator.commitOffsets(offsets, commitType, callback); } finally { release(); } @@ -829,22 +839,48 @@ public class KafkaConsumer implements Consumer { * This commits offsets only to Kafka. The offsets committed using this API will be used on the first fetch after * every rebalance and also on startup. As such, if you need to store offsets in anything other than Kafka, this API * should not be used. - * + *

    + * Asynchronous commits (i.e. {@link CommitType#ASYNC} will not block. Any errors encountered during an asynchronous + * commit are either passed to the callback (if provided) or silently discarded. Synchronous commits (i.e. + * {@link CommitType#SYNC}) block until either the commit succeeds or an unrecoverable error is encountered. In + * this case, the error is either passed to the callback (if provided) or thrown to the caller. + * * @param commitType Whether or not the commit should block until it is acknowledged. + * @param callback Callback to invoke when the commit completes */ @Override - public void commit(CommitType commitType) { + public void commit(CommitType commitType, ConsumerCommitCallback callback) { acquire(); try { - // Need defensive copy to ensure offsets are not removed before completion (e.g. in rebalance) + // need defensive copy to ensure offsets are not removed before completion (e.g. in rebalance) Map allConsumed = new HashMap(this.subscriptions.allConsumed()); - commit(allConsumed, commitType); + commit(allConsumed, commitType, callback); } finally { release(); } } /** + * Commits offsets returned on the last {@link #poll(long) poll()} for the subscribed list of topics and partitions. + *

    + * This commits offsets only to Kafka. The offsets committed using this API will be used on the first fetch after + * every rebalance and also on startup. As such, if you need to store offsets in anything other than Kafka, this API + * should not be used. + *

    + * Asynchronous commits (i.e. {@link CommitType#ASYNC} will not block. Any errors encountered during an asynchronous + * commit are silently discarded. If you need to determine the result of an asynchronous commit, you should use + * {@link #commit(CommitType, ConsumerCommitCallback)}. Synchronous commits (i.e. {@link CommitType#SYNC}) + * block until either the commit succeeds or an unrecoverable error is encountered (in which case it is thrown + * to the caller). + * + * @param commitType Whether or not the commit should block until it is acknowledged. + */ + @Override + public void commit(CommitType commitType) { + commit(commitType, null); + } + + /** * Overrides the fetch offsets that the consumer will use on the next {@link #poll(long) poll(timeout)}. If this API * is invoked for the same partition more than once, the latest offset will be used on the next poll(). Note that * you may lose data if this API is arbitrarily used in the middle of consumption, to reset the fetch offsets @@ -868,8 +904,10 @@ public class KafkaConsumer implements Consumer { try { Collection parts = partitions.length == 0 ? this.subscriptions.assignedPartitions() : Arrays.asList(partitions); - for (TopicPartition tp : parts) + for (TopicPartition tp : parts) { + log.debug("Seeking to beginning of partition {}", tp); subscriptions.needOffsetReset(tp, OffsetResetStrategy.EARLIEST); + } } finally { release(); } @@ -883,8 +921,10 @@ public class KafkaConsumer implements Consumer { try { Collection parts = partitions.length == 0 ? this.subscriptions.assignedPartitions() : Arrays.asList(partitions); - for (TopicPartition tp : parts) + for (TopicPartition tp : parts) { + log.debug("Seeking to end of partition {}", tp); subscriptions.needOffsetReset(tp, OffsetResetStrategy.LATEST); + } } finally { release(); } @@ -931,19 +971,21 @@ public class KafkaConsumer implements Consumer { public long committed(TopicPartition partition) { acquire(); try { - Set partitionsToFetch; + Long committed; if (subscriptions.assignedPartitions().contains(partition)) { - Long committed = this.subscriptions.committed(partition); - if (committed != null) - return committed; - partitionsToFetch = subscriptions.assignedPartitions(); + committed = this.subscriptions.committed(partition); + if (committed == null) { + coordinator.refreshCommittedOffsetsIfNeeded(); + committed = this.subscriptions.committed(partition); + } } else { - partitionsToFetch = Collections.singleton(partition); + Map offsets = coordinator.fetchCommittedOffsets(Collections.singleton(partition)); + committed = offsets.get(partition); } - refreshCommittedOffsets(partitionsToFetch); - Long committed = this.subscriptions.committed(partition); + if (committed == null) throw new NoOffsetForPartitionException("No offset has been committed for partition " + partition); + return committed; } finally { release(); @@ -973,7 +1015,7 @@ public class KafkaConsumer implements Consumer { List parts = cluster.partitionsForTopic(topic); if (parts == null) { metadata.add(topic); - awaitMetadataUpdate(); + client.awaitMetadataUpdate(); parts = metadata.fetch().partitionsForTopic(topic); } return parts; @@ -999,7 +1041,6 @@ public class KafkaConsumer implements Consumer { */ @Override public void wakeup() { - this.wakeup.set(true); this.client.wakeup(); } @@ -1017,55 +1058,18 @@ public class KafkaConsumer implements Consumer { } } + private Coordinator.RebalanceCallback wrapRebalanceCallback(final ConsumerRebalanceCallback callback) { + return new Coordinator.RebalanceCallback() { + @Override + public void onPartitionsAssigned(Collection partitions) { + callback.onPartitionsAssigned(KafkaConsumer.this, partitions); + } - private boolean shouldAutoCommit(long now) { - return this.autoCommit && this.lastCommitAttemptMs <= now - this.autoCommitIntervalMs; - } - - private long timeToNextCommit(long now) { - if (!this.autoCommit) - return Long.MAX_VALUE; - long timeSinceLastCommit = now - this.lastCommitAttemptMs; - if (timeSinceLastCommit > this.autoCommitIntervalMs) - return 0; - return this.autoCommitIntervalMs - timeSinceLastCommit; - } - - /** - * Request a metadata update and wait until it has occurred - */ - private void awaitMetadataUpdate() { - int version = this.metadata.requestUpdate(); - do { - long now = time.milliseconds(); - this.pollClient(this.retryBackoffMs, now); - } while (this.metadata.version() == version); - } - - /** - * Get partition assignment - */ - private void reassignPartitions(long now) { - // execute the user's callback before rebalance - log.debug("Revoking previously assigned partitions {}", this.subscriptions.assignedPartitions()); - try { - this.rebalanceCallback.onPartitionsRevoked(this, this.subscriptions.assignedPartitions()); - } catch (Exception e) { - log.error("User provided callback " + this.rebalanceCallback.getClass().getName() - + " failed on partition revocation: ", e); - } - - // get new assigned partitions from the coordinator - assignPartitions(); - - // execute the user's callback after rebalance - log.debug("Setting newly assigned partitions {}", this.subscriptions.assignedPartitions()); - try { - this.rebalanceCallback.onPartitionsAssigned(this, this.subscriptions.assignedPartitions()); - } catch (Exception e) { - log.error("User provided callback " + this.rebalanceCallback.getClass().getName() - + " failed on partition assignment: ", e); - } + @Override + public void onPartitionsRevoked(Collection partitions) { + callback.onPartitionsRevoked(KafkaConsumer.this, partitions); + } + }; } /** @@ -1077,267 +1081,11 @@ public class KafkaConsumer implements Consumer { * defined */ private void updateFetchPositions(Set partitions) { - // first refresh the committed positions in case they are not up-to-date - refreshCommittedOffsets(partitions); - - // reset the fetch position to the committed position - for (TopicPartition tp : partitions) { - // Skip if we already have a fetch position - if (subscriptions.fetched(tp) != null) - continue; - - // TODO: If there are several offsets to reset, we could submit offset requests in parallel - if (subscriptions.isOffsetResetNeeded(tp)) { - resetOffset(tp); - } else if (subscriptions.committed(tp) == null) { - // There's no committed position, so we need to reset with the default strategy - subscriptions.needOffsetReset(tp); - resetOffset(tp); - } else { - log.debug("Resetting offset for partition {} to the committed offset {}", - tp, subscriptions.committed(tp)); - subscriptions.seek(tp, subscriptions.committed(tp)); - } - } - } + // refresh commits for all assigned partitions + coordinator.refreshCommittedOffsetsIfNeeded(); - /** - * Reset offsets for the given partition using the offset reset strategy. - * - * @param partition The given partition that needs reset offset - * @throws org.apache.kafka.clients.consumer.NoOffsetForPartitionException If no offset reset strategy is defined - */ - private void resetOffset(TopicPartition partition) { - OffsetResetStrategy strategy = subscriptions.resetStrategy(partition); - final long timestamp; - if (strategy == OffsetResetStrategy.EARLIEST) - timestamp = EARLIEST_OFFSET_TIMESTAMP; - else if (strategy == OffsetResetStrategy.LATEST) - timestamp = LATEST_OFFSET_TIMESTAMP; - else - throw new NoOffsetForPartitionException("No offset is set and no reset policy is defined"); - - log.debug("Resetting offset for partition {} to {} offset.", partition, strategy.name().toLowerCase()); - long offset = listOffset(partition, timestamp); - this.subscriptions.seek(partition, offset); - } - - /** - * Fetch a single offset before the given timestamp for the partition. - * - * @param partition The partition that needs fetching offset. - * @param timestamp The timestamp for fetching offset. - * @return The offset of the message that is published before the given timestamp - */ - private long listOffset(TopicPartition partition, long timestamp) { - while (true) { - RequestFuture future = fetcher.listOffset(partition, timestamp); - - if (!future.isDone()) - pollFuture(future, requestTimeoutMs); - - if (future.isDone()) { - if (future.succeeded()) - return future.value(); - handleRequestFailure(future); - } - } - } - - /** - * Refresh the committed offsets for given set of partitions and update the cache - */ - private void refreshCommittedOffsets(Set partitions) { - // we only need to fetch latest committed offset from coordinator if there - // is some commit process in progress, otherwise our current - // committed cache is up-to-date - if (subscriptions.refreshCommitsNeeded()) { - // contact coordinator to fetch committed offsets - Map offsets = fetchCommittedOffsets(partitions); - - // update the position with the offsets - for (Map.Entry entry : offsets.entrySet()) { - TopicPartition tp = entry.getKey(); - this.subscriptions.committed(tp, entry.getValue()); - } - } - } - - /** - * Block until we have received a partition assignment from the coordinator. - */ - private void assignPartitions() { - // Ensure that there are no pending requests to the coordinator. This is important - // in particular to avoid resending a pending JoinGroup request. - awaitCoordinatorInFlightRequests(); - - while (subscriptions.partitionAssignmentNeeded()) { - RequestFuture future = coordinator.assignPartitions(time.milliseconds()); - - // Block indefinitely for the join group request (which can take as long as a session timeout) - if (!future.isDone()) - pollFuture(future); - - if (future.failed()) - handleRequestFailure(future); - } - } - - /** - * Block until the coordinator for this group is known. - */ - private void ensureCoordinatorKnown() { - while (coordinator.coordinatorUnknown()) { - RequestFuture future = coordinator.discoverConsumerCoordinator(); - - if (!future.isDone()) - pollFuture(future, requestTimeoutMs); - - if (future.failed()) - handleRequestFailure(future); - } - } - - /** - * Block until any pending requests to the coordinator have been handled. - */ - public void awaitCoordinatorInFlightRequests() { - while (coordinator.hasInFlightRequests()) { - long now = time.milliseconds(); - pollClient(-1, now); - } - } - - /** - * Lookup the committed offsets for a set of partitions. This will block until the coordinator has - * responded to the offset fetch request. - * @param partitions List of partitions to get offsets for - * @return Map from partition to its respective offset - */ - private Map fetchCommittedOffsets(Set partitions) { - while (true) { - long now = time.milliseconds(); - RequestFuture> future = coordinator.fetchOffsets(partitions, now); - - if (!future.isDone()) - pollFuture(future, requestTimeoutMs); - - if (future.isDone()) { - if (future.succeeded()) - return future.value(); - handleRequestFailure(future); - } - } - } - - /** - * Commit offsets. This call blocks (regardless of commitType) until the coordinator - * can receive the commit request. Once the request has been made, however, only the - * synchronous commits will wait for a successful response from the coordinator. - * @param offsets Offsets to commit. - * @param commitType Commit policy - */ - private void commitOffsets(Map offsets, CommitType commitType) { - if (commitType == CommitType.ASYNC) { - commitOffsetsAsync(offsets); - } else { - commitOffsetsSync(offsets); - } - } - - private void commitOffsetsAsync(Map offsets) { - while (true) { - long now = time.milliseconds(); - RequestFuture future = coordinator.commitOffsets(offsets, now); - - if (!future.isDone() || future.succeeded()) - return; - - handleRequestFailure(future); - } - } - - private void commitOffsetsSync(Map offsets) { - while (true) { - long now = time.milliseconds(); - RequestFuture future = coordinator.commitOffsets(offsets, now); - - if (!future.isDone()) - pollFuture(future, requestTimeoutMs); - - if (future.isDone()) { - if (future.succeeded()) - return; - else - handleRequestFailure(future); - } - } - } - - private void handleRequestFailure(RequestFuture future) { - if (future.hasException()) - throw future.exception(); - - switch (future.retryAction()) { - case BACKOFF: - Utils.sleep(retryBackoffMs); - break; - case POLL: - pollClient(retryBackoffMs, time.milliseconds()); - break; - case FIND_COORDINATOR: - ensureCoordinatorKnown(); - break; - case REFRESH_METADATA: - awaitMetadataUpdate(); - break; - case NOOP: - // Do nothing (retry now) - } - } - - /** - * Poll until a result is ready or timeout expires - * @param future The future to poll for - * @param timeout The time in milliseconds to wait for the result - */ - private void pollFuture(RequestFuture future, long timeout) { - // TODO: Update this code for KAFKA-2120, which adds request timeout to NetworkClient - // In particular, we must ensure that "timed out" requests will not have their callbacks - // invoked at a later time. - long remaining = timeout; - while (!future.isDone() && remaining >= 0) { - long start = time.milliseconds(); - pollClient(remaining, start); - if (future.isDone()) return; - remaining -= time.milliseconds() - start; - } - } - - /** - * Poll indefinitely until the result is ready. - * @param future The future to poll for. - */ - private void pollFuture(RequestFuture future) { - while (!future.isDone()) { - long now = time.milliseconds(); - pollClient(-1, now); - } - } - - /** - * Poll for IO. - * @param timeout The maximum time to wait for IO to become available - * @param now The current time in milliseconds - * @throws ConsumerWakeupException if {@link #wakeup()} is invoked while the poll is active - */ - private void pollClient(long timeout, long now) { - this.client.poll(timeout, now); - - if (wakeup.get()) { - wakeup.set(false); - throw new ConsumerWakeupException(); - } + // then do any offset lookups in case some positions are not known + fetcher.updateFetchPositions(partitions); } /* diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java index 46e26a6..c14eed1 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java @@ -106,16 +106,29 @@ public class MockConsumer implements Consumer { } @Override - public synchronized void commit(Map offsets, CommitType commitType) { + public synchronized void commit(Map offsets, CommitType commitType, ConsumerCommitCallback callback) { ensureNotClosed(); for (Entry entry : offsets.entrySet()) subscriptions.committed(entry.getKey(), entry.getValue()); + if (callback != null) { + callback.onComplete(offsets, null); + } } @Override - public synchronized void commit(CommitType commitType) { + public synchronized void commit(Map offsets, CommitType commitType) { + commit(offsets, commitType, null); + } + + @Override + public synchronized void commit(CommitType commitType, ConsumerCommitCallback callback) { ensureNotClosed(); - commit(this.subscriptions.allConsumed(), commitType); + commit(this.subscriptions.allConsumed(), commitType, callback); + } + + @Override + public synchronized void commit(CommitType commitType) { + commit(commitType, null); } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java new file mode 100644 index 0000000..9517d9d --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java @@ -0,0 +1,296 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals; + +import org.apache.kafka.clients.ClientRequest; +import org.apache.kafka.clients.ClientResponse; +import org.apache.kafka.clients.KafkaClient; +import org.apache.kafka.clients.Metadata; +import org.apache.kafka.clients.RequestCompletionHandler; +import org.apache.kafka.clients.consumer.ConsumerWakeupException; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.requests.AbstractRequest; +import org.apache.kafka.common.requests.RequestHeader; +import org.apache.kafka.common.requests.RequestSend; +import org.apache.kafka.common.utils.Time; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * Higher level consumer access to the network layer with basic support for futures and + * task scheduling. NOT thread-safe! + * + * TODO: The current implementation is simplistic in that it provides a facility for queueing requests + * prior to delivery, but it makes no effort to retry requests which cannot be sent at the time + * {@link #poll(long)} is called. This makes the behavior of the queue predictable and easy to + * understand, but there are opportunities to provide timeout or retry capabilities in the future. + * How we do this may depend on KAFKA-2120, so for now, we retain the simplistic behavior. + */ +public class ConsumerNetworkClient implements Closeable { + private final KafkaClient client; + private final AtomicBoolean wakeup = new AtomicBoolean(false); + private final DelayedTaskQueue delayedTasks = new DelayedTaskQueue(); + private final Map> unsent = new HashMap>(); + private final Metadata metadata; + private final Time time; + private final long retryBackoffMs; + + public ConsumerNetworkClient(KafkaClient client, + Metadata metadata, + Time time, + long retryBackoffMs) { + this.client = client; + this.metadata = metadata; + this.time = time; + this.retryBackoffMs = retryBackoffMs; + } + + /** + * Schedule a new task to be executed at the given time. This is "best-effort" scheduling and + * should only be used for coarse synchronization. + * @param task The task to be scheduled + * @param at The time it should run + */ + public void schedule(DelayedTask task, long at) { + delayedTasks.add(task, at); + } + + /** + * Unschedule a task. This will remove all instances of the task from the task queue. + * This is a no-op if the task is not scheduled. + * @param task The task to be unscheduled. + */ + public void unschedule(DelayedTask task) { + delayedTasks.remove(task); + } + + /** + * Send a new request. Note that the request is not actually transmitted on the + * network until one of the {@link #poll(long)} variants is invoked. At this + * point the request will either be transmitted successfully or will fail. + * Use the returned future to obtain the result of the send. + * @param node The destination of the request + * @param api The Kafka API call + * @param request The request payload + * @return A future which indicates the result of the send. + */ + public RequestFuture send(Node node, + ApiKeys api, + AbstractRequest request) { + long now = time.milliseconds(); + RequestFutureCompletionHandler future = new RequestFutureCompletionHandler(); + RequestHeader header = client.nextRequestHeader(api); + RequestSend send = new RequestSend(node.idString(), header, request.toStruct()); + put(node, new ClientRequest(now, true, send, future)); + return future; + } + + private void put(Node node, ClientRequest request) { + List nodeUnsent = unsent.get(node); + if (nodeUnsent == null) { + nodeUnsent = new ArrayList(); + unsent.put(node, nodeUnsent); + } + nodeUnsent.add(request); + } + + public Node leastLoadedNode() { + return client.leastLoadedNode(time.milliseconds()); + } + + /** + * Block until the metadata has been refreshed. + */ + public void awaitMetadataUpdate() { + int version = this.metadata.requestUpdate(); + do { + poll(Long.MAX_VALUE); + } while (this.metadata.version() == version); + } + + /** + * Wakeup an active poll. This will cause the polling thread to throw an exception either + * on the current poll if one is active, or the next poll. + */ + public void wakeup() { + this.wakeup.set(true); + this.client.wakeup(); + } + + /** + * Block indefinitely until the given request future has finished. + * @param future The request future to await. + * @throws ConsumerWakeupException if {@link #wakeup()} is called from another thread + */ + public void poll(RequestFuture future) { + while (!future.isDone()) + poll(Long.MAX_VALUE); + } + + /** + * Block until the provided request future request has finished or the timeout has expired. + * @param future The request future to wait for + * @param timeout The maximum duration (in ms) to wait for the request + * @return true if the future is done, false otherwise + * @throws ConsumerWakeupException if {@link #wakeup()} is called from another thread + */ + public boolean poll(RequestFuture future, long timeout) { + long now = time.milliseconds(); + long deadline = now + timeout; + while (!future.isDone() && now < deadline) { + poll(deadline - now, now); + now = time.milliseconds(); + } + return future.isDone(); + } + + /** + * Poll for any network IO. All send requests will either be transmitted on the network + * or failed when this call completes. + * @param timeout The maximum time to wait for an IO event. + * @throws ConsumerWakeupException if {@link #wakeup()} is called from another thread + */ + public void poll(long timeout) { + poll(timeout, time.milliseconds()); + } + + private void poll(long timeout, long now) { + // send all the requests we can send now + pollUnsentRequests(now); + + // ensure we don't poll any longer than the deadline for + // the next scheduled task + timeout = Math.min(timeout, delayedTasks.nextTimeout(now)); + clientPoll(timeout, now); + + // execute scheduled tasks + now = time.milliseconds(); + delayedTasks.poll(now); + + // try again to send requests since buffer space may have been + // cleared or a connect finished in the poll + pollUnsentRequests(now); + + // fail all requests that couldn't be sent + clearUnsentRequests(now); + + } + + /** + * Block until all pending requests from the given node have finished. + * @param node The node to await requests from + */ + public void awaitPendingRequests(Node node) { + while (pendingRequestCount(node) > 0) + poll(retryBackoffMs); + } + + /** + * Get the count of pending requests to the given node. This includes both request that + * have been transmitted (i.e. in-flight requests) and those which are awaiting transmission. + * @param node The node in question + * @return The number of pending requests + */ + public int pendingRequestCount(Node node) { + List pending = unsent.get(node); + int unsentCount = pending == null ? 0 : pending.size(); + return unsentCount + client.inFlightRequestCount(node.idString()); + } + + /** + * Get the total count of pending requests from all nodes. This includes both requests that + * have been transmitted (i.e. in-flight requests) and those which are awaiting transmission. + * @return The total count of pending requests + */ + public int pendingRequestCount() { + int total = 0; + for (List requests: unsent.values()) + total += requests.size(); + return total + client.inFlightRequestCount(); + } + + private void pollUnsentRequests(long now) { + while (trySend(now)) + clientPoll(0, now); + } + + private void clearUnsentRequests(long now) { + // clear all unsent requests and fail their corresponding futures + for (Map.Entry> requestEntry: unsent.entrySet()) { + Iterator iterator = requestEntry.getValue().iterator(); + while (iterator.hasNext()) { + ClientRequest request = iterator.next(); + RequestFutureCompletionHandler handler = + (RequestFutureCompletionHandler) request.callback(); + handler.raise(SendFailedException.INSTANCE); + iterator.remove(); + } + } + unsent.clear(); + } + + private boolean trySend(long now) { + // send any requests that can be sent now + boolean requestsSent = false; + for (Map.Entry> requestEntry: unsent.entrySet()) { + Node node = requestEntry.getKey(); + Iterator iterator = requestEntry.getValue().iterator(); + while (iterator.hasNext()) { + ClientRequest request = iterator.next(); + if (client.ready(node, now)) { + client.send(request); + iterator.remove(); + requestsSent = true; + } else if (client.connectionFailed(node)) { + RequestFutureCompletionHandler handler = + (RequestFutureCompletionHandler) request.callback(); + handler.onComplete(new ClientResponse(request, now, true, null)); + iterator.remove(); + } + } + } + return requestsSent; + } + + private void clientPoll(long timeout, long now) { + client.poll(timeout, now); + if (wakeup.get()) { + clearUnsentRequests(now); + wakeup.set(false); + throw new ConsumerWakeupException(); + } + } + + @Override + public void close() throws IOException { + client.close(); + } + + public static class RequestFutureCompletionHandler + extends RequestFuture + implements RequestCompletionHandler { + + @Override + public void onComplete(ClientResponse response) { + complete(response); + } + } +} 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 c1c8172..8e3cd09 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 @@ -12,14 +12,14 @@ */ package org.apache.kafka.clients.consumer.internals; -import org.apache.kafka.clients.ClientRequest; import org.apache.kafka.clients.ClientResponse; -import org.apache.kafka.clients.KafkaClient; -import org.apache.kafka.clients.RequestCompletionHandler; +import org.apache.kafka.clients.consumer.CommitType; +import org.apache.kafka.clients.consumer.ConsumerCommitCallback; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.DisconnectException; import org.apache.kafka.common.metrics.Measurable; import org.apache.kafka.common.metrics.MetricConfig; import org.apache.kafka.common.metrics.Metrics; @@ -30,7 +30,6 @@ import org.apache.kafka.common.metrics.stats.Max; import org.apache.kafka.common.metrics.stats.Rate; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.requests.ConsumerMetadataRequest; import org.apache.kafka.common.requests.ConsumerMetadataResponse; import org.apache.kafka.common.requests.HeartbeatRequest; @@ -41,15 +40,15 @@ import org.apache.kafka.common.requests.OffsetCommitRequest; import org.apache.kafka.common.requests.OffsetCommitResponse; import org.apache.kafka.common.requests.OffsetFetchRequest; import org.apache.kafka.common.requests.OffsetFetchResponse; -import org.apache.kafka.common.requests.RequestHeader; -import org.apache.kafka.common.requests.RequestSend; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.ArrayList; +import java.util.Collection; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -62,23 +61,27 @@ public final class Coordinator { private static final Logger log = LoggerFactory.getLogger(Coordinator.class); - private final KafkaClient client; - + private final ConsumerNetworkClient client; private final Time time; private final String groupId; private final Heartbeat heartbeat; + private final HeartbeatTask heartbeatTask; private final int sessionTimeoutMs; private final String assignmentStrategy; private final SubscriptionState subscriptions; private final CoordinatorMetrics sensors; + private final long requestTimeoutMs; + private final long retryBackoffMs; + private final RebalanceCallback rebalanceCallback; private Node consumerCoordinator; private String consumerId; private int generation; + /** * Initialize the coordination manager. */ - public Coordinator(KafkaClient client, + public Coordinator(ConsumerNetworkClient client, String groupId, int sessionTimeoutMs, String assignmentStrategy, @@ -86,10 +89,13 @@ public final class Coordinator { Metrics metrics, String metricGrpPrefix, Map metricTags, - Time time) { + Time time, + long requestTimeoutMs, + long retryBackoffMs, + RebalanceCallback rebalanceCallback) { - this.time = time; this.client = client; + this.time = time; this.generation = -1; this.consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID; this.groupId = groupId; @@ -98,19 +104,190 @@ public final class Coordinator { this.sessionTimeoutMs = sessionTimeoutMs; this.assignmentStrategy = assignmentStrategy; this.heartbeat = new Heartbeat(this.sessionTimeoutMs, time.milliseconds()); + this.heartbeatTask = new HeartbeatTask(); this.sensors = new CoordinatorMetrics(metrics, metricGrpPrefix, metricTags); + this.requestTimeoutMs = requestTimeoutMs; + this.retryBackoffMs = retryBackoffMs; + this.rebalanceCallback = rebalanceCallback; + } + + /** + * Refresh the committed offsets for provided partitions. + */ + public void refreshCommittedOffsetsIfNeeded() { + if (subscriptions.refreshCommitsNeeded()) { + Map offsets = fetchCommittedOffsets(subscriptions.assignedPartitions()); + for (Map.Entry entry : offsets.entrySet()) { + TopicPartition tp = entry.getKey(); + this.subscriptions.committed(tp, entry.getValue()); + } + this.subscriptions.commitsRefreshed(); + } + } + + /** + * Fetch the current committed offsets from the coordinator for a set of partitions. + * @param partitions The partitions to fetch offsets for + * @return A map from partition to the committed offset + */ + public Map fetchCommittedOffsets(Set partitions) { + while (true) { + ensureCoordinatorKnown(); + ensurePartitionAssignment(); + + // contact coordinator to fetch committed offsets + RequestFuture> future = sendOffsetFetchRequest(partitions); + client.poll(future); + + if (future.succeeded()) + return future.value(); + + if (!future.isRetriable()) + throw future.exception(); + + Utils.sleep(retryBackoffMs); + } + } + + /** + * Ensure that we have a valid partition assignment from the coordinator. + */ + public void ensurePartitionAssignment() { + if (!subscriptions.partitionAssignmentNeeded()) + return; + + // execute the user's callback before rebalance + log.debug("Revoking previously assigned partitions {}", this.subscriptions.assignedPartitions()); + try { + Set revoked = new HashSet(subscriptions.assignedPartitions()); + rebalanceCallback.onPartitionsRevoked(revoked); + } catch (Exception e) { + log.error("User provided callback " + this.rebalanceCallback.getClass().getName() + + " failed on partition revocation: ", e); + } + + reassignPartitions(); + + // execute the user's callback after rebalance + log.debug("Setting newly assigned partitions {}", this.subscriptions.assignedPartitions()); + try { + Set assigned = new HashSet(subscriptions.assignedPartitions()); + rebalanceCallback.onPartitionsAssigned(assigned); + } catch (Exception e) { + log.error("User provided callback " + this.rebalanceCallback.getClass().getName() + + " failed on partition assignment: ", e); + } + } + + private void reassignPartitions() { + while (subscriptions.partitionAssignmentNeeded()) { + ensureCoordinatorKnown(); + + // ensure that there are no pending requests to the coordinator. This is important + // in particular to avoid resending a pending JoinGroup request. + if (client.pendingRequestCount(this.consumerCoordinator) > 0) { + client.awaitPendingRequests(this.consumerCoordinator); + continue; + } + + RequestFuture future = sendJoinGroupRequest(); + client.poll(future); + + if (future.failed()) { + if (!future.isRetriable()) + throw future.exception(); + Utils.sleep(retryBackoffMs); + } + } + } + + /** + * Block until the coordinator for this group is known. + */ + public void ensureCoordinatorKnown() { + while (coordinatorUnknown()) { + RequestFuture future = sendConsumerMetadataRequest(); + client.poll(future, requestTimeoutMs); + + if (future.failed()) + client.awaitMetadataUpdate(); + } + } + + /** + * Commit offsets. This call blocks (regardless of commitType) until the coordinator + * can receive the commit request. Once the request has been made, however, only the + * synchronous commits will wait for a successful response from the coordinator. + * @param offsets Offsets to commit. + * @param commitType Commit policy + * @param callback Callback to be executed when the commit request finishes + */ + public void commitOffsets(Map offsets, CommitType commitType, ConsumerCommitCallback callback) { + if (commitType == CommitType.ASYNC) + commitOffsetsAsync(offsets, callback); + else + commitOffsetsSync(offsets, callback); + } + + private class HeartbeatTask implements DelayedTask { + + public void reset() { + // start or restart the heartbeat task to be executed at the next chance + long now = time.milliseconds(); + heartbeat.resetSessionTimeout(now); + client.unschedule(this); + client.schedule(this, now); + } + + @Override + public void run(final long now) { + if (!subscriptions.partitionsAutoAssigned() || + subscriptions.partitionAssignmentNeeded() || + coordinatorUnknown()) + // no need to send if we're not using auto-assignment or if we are + // awaiting a rebalance + return; + + if (heartbeat.sessionTimeoutExpired(now)) { + // we haven't received a successful heartbeat in one session interval + // so mark the coordinator dead + coordinatorDead(); + return; + } + + if (!heartbeat.shouldHeartbeat(now)) { + // we don't need to heartbeat now, so reschedule for when we do + client.schedule(this, now + heartbeat.timeToNextHeartbeat(now)); + } else { + heartbeat.sentHeartbeat(now); + RequestFuture future = sendHeartbeatRequest(); + future.addListener(new RequestFutureListener() { + @Override + public void onSuccess(Void value) { + long now = time.milliseconds(); + heartbeat.receiveHeartbeat(now); + long nextHeartbeatTime = now + heartbeat.timeToNextHeartbeat(now); + client.schedule(HeartbeatTask.this, nextHeartbeatTime); + } + + @Override + public void onFailure(RuntimeException e) { + client.schedule(HeartbeatTask.this, retryBackoffMs); + } + }); + } + } } /** * Send a request to get a new partition assignment. This is a non-blocking call which sends * a JoinGroup request to the coordinator (if it is available). The returned future must * be polled to see if the request completed successfully. - * @param now The current time in milliseconds * @return A request future whose completion indicates the result of the JoinGroup request. */ - public RequestFuture assignPartitions(final long now) { - final RequestFuture future = newCoordinatorRequestFuture(now); - if (future.isDone()) return future; + private RequestFuture sendJoinGroupRequest() { + if (coordinatorUnknown()) + return RequestFuture.coordinatorNotAvailable(); // send a join group request to the coordinator List subscribedTopics = new ArrayList(subscriptions.subscribedTopics()); @@ -124,25 +301,20 @@ public final class Coordinator { // create the request for the coordinator log.debug("Issuing request ({}: {}) to coordinator {}", ApiKeys.JOIN_GROUP, request, this.consumerCoordinator.id()); + return client.send(consumerCoordinator, ApiKeys.JOIN_GROUP, request) + .compose(new JoinGroupResponseHandler()); + } - RequestCompletionHandler completionHandler = new RequestCompletionHandler() { - @Override - public void onComplete(ClientResponse resp) { - handleJoinResponse(resp, future); - } - }; + private class JoinGroupResponseHandler extends CoordinatorResponseHandler { - sendCoordinator(ApiKeys.JOIN_GROUP, request.toStruct(), completionHandler, now); - return future; - } + @Override + public JoinGroupResponse parse(ClientResponse response) { + return new JoinGroupResponse(response.responseBody()); + } - private void handleJoinResponse(ClientResponse response, RequestFuture future) { - if (response.wasDisconnected()) { - handleCoordinatorDisconnect(response); - future.retryWithNewCoordinator(); - } else { + @Override + public void handle(JoinGroupResponse joinResponse, RequestFuture future) { // process the response - JoinGroupResponse joinResponse = new JoinGroupResponse(response.responseBody()); short errorCode = joinResponse.errorCode(); if (errorCode == Errors.NONE.code()) { @@ -152,36 +324,36 @@ public final class Coordinator { // set the flag to refresh last committed offsets subscriptions.needRefreshCommits(); - log.debug("Joined group: {}", response); + log.debug("Joined group: {}", joinResponse.toStruct()); // record re-assignment time - this.sensors.partitionReassignments.record(response.requestLatencyMs()); + sensors.partitionReassignments.record(response.requestLatencyMs()); // update partition assignment subscriptions.changePartitionAssignment(joinResponse.assignedPartitions()); + heartbeatTask.reset(); future.complete(null); } else if (errorCode == Errors.UNKNOWN_CONSUMER_ID.code()) { // reset the consumer id and retry immediately Coordinator.this.consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID; log.info("Attempt to join group {} failed due to unknown consumer id, resetting and retrying.", groupId); - - future.retryNow(); + future.raise(Errors.UNKNOWN_CONSUMER_ID); } else if (errorCode == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code() || errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { // re-discover the coordinator and retry with backoff coordinatorDead(); log.info("Attempt to join group {} failed due to obsolete coordinator information, retrying.", groupId); - future.retryWithNewCoordinator(); + future.raise(Errors.forCode(errorCode)); } else if (errorCode == Errors.UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY.code() || errorCode == Errors.INCONSISTENT_PARTITION_ASSIGNMENT_STRATEGY.code() || errorCode == Errors.INVALID_SESSION_TIMEOUT.code()) { // log the error and re-throw the exception - KafkaException e = Errors.forCode(errorCode).exception(); + Errors error = Errors.forCode(errorCode); log.error("Attempt to join group {} failed due to: {}", - groupId, e.getMessage()); - future.raise(e); + groupId, error.exception().getMessage()); + future.raise(error); } else { // unexpected error, throw the exception future.raise(new KafkaException("Unexpected error in join group response: " @@ -190,55 +362,134 @@ public final class Coordinator { } } + private void commitOffsetsAsync(final Map offsets, final ConsumerCommitCallback callback) { + this.subscriptions.needRefreshCommits(); + RequestFuture future = sendOffsetCommitRequest(offsets); + if (callback != null) { + future.addListener(new RequestFutureListener() { + @Override + public void onSuccess(Void value) { + callback.onComplete(offsets, null); + } + + @Override + public void onFailure(RuntimeException e) { + callback.onComplete(offsets, e); + } + }); + } + } + + private void commitOffsetsSync(Map offsets, ConsumerCommitCallback callback) { + while (true) { + ensureCoordinatorKnown(); + ensurePartitionAssignment(); + + RequestFuture future = sendOffsetCommitRequest(offsets); + client.poll(future); + + if (future.succeeded()) { + if (callback != null) + callback.onComplete(offsets, null); + return; + } + + if (!future.isRetriable()) { + if (callback == null) + throw future.exception(); + else + callback.onComplete(offsets, future.exception()); + return; + } + + Utils.sleep(retryBackoffMs); + } + } + /** * Commit offsets for the specified list of topics and partitions. This is a non-blocking call * which returns a request future that can be polled in the case of a synchronous commit or ignored in the * asynchronous case. * * @param offsets The list of offsets per partition that should be committed. - * @param now The current time * @return A request future whose value indicates whether the commit was successful or not */ - public RequestFuture commitOffsets(final Map offsets, long now) { - final RequestFuture future = newCoordinatorRequestFuture(now); - if (future.isDone()) return future; + private RequestFuture sendOffsetCommitRequest(final Map offsets) { + if (coordinatorUnknown()) + return RequestFuture.coordinatorNotAvailable(); - if (offsets.isEmpty()) { - future.complete(null); - } else { - // create the offset commit request - Map offsetData; - offsetData = new HashMap(offsets.size()); - for (Map.Entry entry : offsets.entrySet()) - offsetData.put(entry.getKey(), new OffsetCommitRequest.PartitionData(entry.getValue(), "")); - OffsetCommitRequest req = new OffsetCommitRequest(this.groupId, + if (offsets.isEmpty()) + return RequestFuture.voidSuccess(); + + // create the offset commit request + Map offsetData; + offsetData = new HashMap(offsets.size()); + for (Map.Entry entry : offsets.entrySet()) + offsetData.put(entry.getKey(), new OffsetCommitRequest.PartitionData(entry.getValue(), "")); + OffsetCommitRequest req = new OffsetCommitRequest(this.groupId, this.generation, this.consumerId, OffsetCommitRequest.DEFAULT_RETENTION_TIME, offsetData); - RequestCompletionHandler handler = new OffsetCommitCompletionHandler(offsets, future); - sendCoordinator(ApiKeys.OFFSET_COMMIT, req.toStruct(), handler, now); - } - - return future; + return client.send(consumerCoordinator, ApiKeys.OFFSET_COMMIT, req) + .compose(new OffsetCommitResponseHandler(offsets)); } - private RequestFuture newCoordinatorRequestFuture(long now) { - if (coordinatorUnknown()) - return RequestFuture.newCoordinatorNeeded(); - if (client.ready(this.consumerCoordinator, now)) - // We have an open connection and we're ready to send - return new RequestFuture(); + private class OffsetCommitResponseHandler extends CoordinatorResponseHandler { + + private final Map offsets; + + public OffsetCommitResponseHandler(Map offsets) { + this.offsets = offsets; + } - if (this.client.connectionFailed(this.consumerCoordinator)) { - coordinatorDead(); - return RequestFuture.newCoordinatorNeeded(); + @Override + public OffsetCommitResponse parse(ClientResponse response) { + return new OffsetCommitResponse(response.responseBody()); } - // The connection has been initiated, so we need to poll to finish it - return RequestFuture.pollNeeded(); + @Override + public void handle(OffsetCommitResponse commitResponse, RequestFuture future) { + sensors.commitLatency.record(response.requestLatencyMs()); + for (Map.Entry entry : commitResponse.responseData().entrySet()) { + TopicPartition tp = entry.getKey(); + long offset = this.offsets.get(tp); + short errorCode = entry.getValue(); + if (errorCode == Errors.NONE.code()) { + log.debug("Committed offset {} for partition {}", offset, tp); + subscriptions.committed(tp, offset); + } else if (errorCode == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code() + || errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { + coordinatorDead(); + future.raise(Errors.forCode(errorCode)); + return; + } else if (errorCode == Errors.OFFSET_METADATA_TOO_LARGE.code() + || errorCode == Errors.INVALID_COMMIT_OFFSET_SIZE.code()) { + // do not need to throw the exception but just log the error + log.error("Error committing partition {} at offset {}: {}", + tp, + offset, + Errors.forCode(errorCode).exception().getMessage()); + } else if (errorCode == Errors.UNKNOWN_CONSUMER_ID.code() + || errorCode == Errors.ILLEGAL_GENERATION.code()) { + // need to re-join group + subscriptions.needReassignment(); + future.raise(Errors.forCode(errorCode)); + return; + } else { + // do not need to throw the exception but just log the error + future.raise(Errors.forCode(errorCode)); + log.error("Error committing partition {} at offset {}: {}", + tp, + offset, + Errors.forCode(errorCode).exception().getMessage()); + } + } + + future.complete(null); + } } /** @@ -246,35 +497,30 @@ public final class Coordinator { * returned future can be polled to get the actual offsets returned from the broker. * * @param partitions The set of partitions to get offsets for. - * @param now The current time in milliseconds * @return A request future containing the committed offsets. */ - public RequestFuture> fetchOffsets(Set partitions, long now) { - final RequestFuture> future = newCoordinatorRequestFuture(now); - if (future.isDone()) return future; + private RequestFuture> sendOffsetFetchRequest(Set partitions) { + if (coordinatorUnknown()) + return RequestFuture.coordinatorNotAvailable(); - log.debug("Fetching committed offsets for partitions: " + Utils.join(partitions, ", ")); + log.debug("Fetching committed offsets for partitions: {}", Utils.join(partitions, ", ")); // construct the request OffsetFetchRequest request = new OffsetFetchRequest(this.groupId, new ArrayList(partitions)); // send the request with a callback - RequestCompletionHandler completionHandler = new RequestCompletionHandler() { - @Override - public void onComplete(ClientResponse resp) { - handleOffsetFetchResponse(resp, future); - } - }; - sendCoordinator(ApiKeys.OFFSET_FETCH, request.toStruct(), completionHandler, now); - return future; + return client.send(consumerCoordinator, ApiKeys.OFFSET_FETCH, request) + .compose(new OffsetFetchResponseHandler()); } - private void handleOffsetFetchResponse(ClientResponse resp, RequestFuture> future) { - if (resp.wasDisconnected()) { - handleCoordinatorDisconnect(resp); - future.retryWithNewCoordinator(); - } else { - // parse the response to get the offsets - OffsetFetchResponse response = new OffsetFetchResponse(resp.responseBody()); + private class OffsetFetchResponseHandler extends CoordinatorResponseHandler> { + + @Override + public OffsetFetchResponse parse(ClientResponse response) { + return new OffsetFetchResponse(response.responseBody()); + } + + @Override + public void handle(OffsetFetchResponse response, RequestFuture> future) { Map offsets = new HashMap(response.responseData().size()); for (Map.Entry entry : response.responseData().entrySet()) { TopicPartition tp = entry.getKey(); @@ -285,19 +531,21 @@ public final class Coordinator { .getMessage()); if (data.errorCode == Errors.OFFSET_LOAD_IN_PROGRESS.code()) { // just retry - future.retryAfterBackoff(); + future.raise(Errors.OFFSET_LOAD_IN_PROGRESS); } else if (data.errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { // re-discover the coordinator and retry coordinatorDead(); - future.retryWithNewCoordinator(); + future.raise(Errors.NOT_COORDINATOR_FOR_CONSUMER); } else if (data.errorCode == Errors.UNKNOWN_CONSUMER_ID.code() || data.errorCode == Errors.ILLEGAL_GENERATION.code()) { // need to re-join group subscriptions.needReassignment(); + future.raise(Errors.forCode(data.errorCode)); } else { future.raise(new KafkaException("Unexpected error in fetch offset response: " + Errors.forCode(data.errorCode).exception().getMessage())); } + return; } else if (data.offset >= 0) { // record the position with the offset (-1 indicates no committed offset to fetch) offsets.put(tp, data.offset); @@ -306,82 +554,47 @@ public final class Coordinator { } } - if (!future.isDone()) - future.complete(offsets); - } - } - - /** - * Attempt to heartbeat the consumer coordinator if necessary, and check if the coordinator is still alive. - * - * @param now The current time - */ - public void maybeHeartbeat(long now) { - if (heartbeat.shouldHeartbeat(now) && coordinatorReady(now)) { - HeartbeatRequest req = new HeartbeatRequest(this.groupId, this.generation, this.consumerId); - sendCoordinator(ApiKeys.HEARTBEAT, req.toStruct(), new HeartbeatCompletionHandler(), now); - this.heartbeat.sentHeartbeat(now); + future.complete(offsets); } } /** - * Get the time until the next heartbeat is needed. - * @param now The current time - * @return The duration in milliseconds before the next heartbeat will be needed. + * Send a heartbeat request now (visible only for testing). */ - public long timeToNextHeartbeat(long now) { - return heartbeat.timeToNextHeartbeat(now); - } - - /** - * Check whether the coordinator has any in-flight requests. - * @return true if the coordinator has pending requests. - */ - public boolean hasInFlightRequests() { - return !coordinatorUnknown() && client.inFlightRequestCount(consumerCoordinator.idString()) > 0; + public RequestFuture sendHeartbeatRequest() { + HeartbeatRequest req = new HeartbeatRequest(this.groupId, this.generation, this.consumerId); + return client.send(consumerCoordinator, ApiKeys.HEARTBEAT, req) + .compose(new HeartbeatCompletionHandler()); } public boolean coordinatorUnknown() { return this.consumerCoordinator == null; } - private boolean coordinatorReady(long now) { - return !coordinatorUnknown() && this.client.ready(this.consumerCoordinator, now); - } - /** * Discover the current coordinator for the consumer group. Sends a ConsumerMetadata request to * one of the brokers. The returned future should be polled to get the result of the request. * @return A request future which indicates the completion of the metadata request */ - public RequestFuture discoverConsumerCoordinator() { + private RequestFuture sendConsumerMetadataRequest() { // initiate the consumer metadata request // find a node to ask about the coordinator - long now = time.milliseconds(); - Node node = this.client.leastLoadedNode(now); - + Node node = this.client.leastLoadedNode(); if (node == null) { - return RequestFuture.metadataRefreshNeeded(); - } else if (!this.client.ready(node, now)) { - if (this.client.connectionFailed(node)) { - return RequestFuture.metadataRefreshNeeded(); - } else { - return RequestFuture.pollNeeded(); - } + // TODO: If there are no brokers left, perhaps we should use the bootstrap set + // from configuration? + return RequestFuture.noBrokersAvailable(); } else { - final RequestFuture future = new RequestFuture(); - // create a consumer metadata request log.debug("Issuing consumer metadata request to broker {}", node.id()); ConsumerMetadataRequest metadataRequest = new ConsumerMetadataRequest(this.groupId); - RequestCompletionHandler completionHandler = new RequestCompletionHandler() { - @Override - public void onComplete(ClientResponse resp) { - handleConsumerMetadataResponse(resp, future); - } - }; - send(node, ApiKeys.CONSUMER_METADATA, metadataRequest.toStruct(), completionHandler, now); - return future; + return client.send(node, ApiKeys.CONSUMER_METADATA, metadataRequest) + .compose(new RequestFutureAdapter() { + @Override + public void onSuccess(ClientResponse response, RequestFuture future) { + handleConsumerMetadataResponse(response, future); + } + }); } } @@ -391,7 +604,10 @@ public final class Coordinator { // parse the response to get the coordinator info if it is not disconnected, // otherwise we need to request metadata update if (resp.wasDisconnected()) { - future.retryAfterMetadataRefresh(); + future.raise(new DisconnectException()); + } else if (!coordinatorUnknown()) { + // We already found the coordinator, so ignore the request + future.complete(null); } else { ConsumerMetadataResponse consumerMetadataResponse = new ConsumerMetadataResponse(resp.responseBody()); // use MAX_VALUE - node.id as the coordinator id to mimic separate connections @@ -401,9 +617,10 @@ public final class Coordinator { this.consumerCoordinator = new Node(Integer.MAX_VALUE - consumerMetadataResponse.node().id(), consumerMetadataResponse.node().host(), consumerMetadataResponse.node().port()); + heartbeatTask.reset(); future.complete(null); } else { - future.retryAfterBackoff(); + future.raise(Errors.forCode(consumerMetadataResponse.errorCode())); } } } @@ -418,115 +635,84 @@ public final class Coordinator { } } - /** - * Handle the case when the request gets cancelled due to coordinator disconnection. - */ - private void handleCoordinatorDisconnect(ClientResponse response) { - int correlation = response.request().request().header().correlationId(); - log.debug("Cancelled request {} with correlation id {} due to coordinator {} being disconnected", - response.request(), - correlation, - response.request().request().destination()); - - // mark the coordinator as dead - coordinatorDead(); - } - - - private void sendCoordinator(ApiKeys api, Struct request, RequestCompletionHandler handler, long now) { - send(this.consumerCoordinator, api, request, handler, now); - } - - private void send(Node node, ApiKeys api, Struct request, RequestCompletionHandler handler, long now) { - RequestHeader header = this.client.nextRequestHeader(api); - RequestSend send = new RequestSend(node.idString(), header, request); - this.client.send(new ClientRequest(now, true, send, handler)); - } + private class HeartbeatCompletionHandler extends CoordinatorResponseHandler { + @Override + public HeartbeatResponse parse(ClientResponse response) { + return new HeartbeatResponse(response.responseBody()); + } - private class HeartbeatCompletionHandler implements RequestCompletionHandler { @Override - public void onComplete(ClientResponse resp) { - if (resp.wasDisconnected()) { - handleCoordinatorDisconnect(resp); + public void handle(HeartbeatResponse heartbeatResponse, RequestFuture future) { + sensors.heartbeatLatency.record(response.requestLatencyMs()); + short error = heartbeatResponse.errorCode(); + if (error == Errors.NONE.code()) { + log.debug("Received successful heartbeat response."); + future.complete(null); + } else if (error == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code() + || error == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { + log.info("Attempt to heart beat failed since coordinator is either not started or not valid, marking it as dead."); + coordinatorDead(); + future.raise(Errors.forCode(error)); + } else if (error == Errors.ILLEGAL_GENERATION.code()) { + log.info("Attempt to heart beat failed since generation id is not legal, try to re-join group."); + subscriptions.needReassignment(); + future.raise(Errors.ILLEGAL_GENERATION); + } else if (error == Errors.UNKNOWN_CONSUMER_ID.code()) { + log.info("Attempt to heart beat failed since consumer id is not valid, reset it and try to re-join group."); + consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID; + subscriptions.needReassignment(); + future.raise(Errors.UNKNOWN_CONSUMER_ID); } else { - HeartbeatResponse response = new HeartbeatResponse(resp.responseBody()); - if (response.errorCode() == Errors.NONE.code()) { - log.debug("Received successful heartbeat response."); - } else if (response.errorCode() == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code() - || response.errorCode() == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { - log.info("Attempt to heart beat failed since coordinator is either not started or not valid, marking it as dead."); - coordinatorDead(); - } else if (response.errorCode() == Errors.ILLEGAL_GENERATION.code()) { - log.info("Attempt to heart beat failed since generation id is not legal, try to re-join group."); - subscriptions.needReassignment(); - } else if (response.errorCode() == Errors.UNKNOWN_CONSUMER_ID.code()) { - log.info("Attempt to heart beat failed since consumer id is not valid, reset it and try to re-join group."); - consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID; - subscriptions.needReassignment(); - } else { - throw new KafkaException("Unexpected error in heartbeat response: " - + Errors.forCode(response.errorCode()).exception().getMessage()); - } + future.raise(new KafkaException("Unexpected error in heartbeat response: " + + Errors.forCode(error).exception().getMessage())); } - sensors.heartbeatLatency.record(resp.requestLatencyMs()); } } - private class OffsetCommitCompletionHandler implements RequestCompletionHandler { + private abstract class CoordinatorResponseHandler + extends RequestFutureAdapter { + protected ClientResponse response; - private final Map offsets; - private final RequestFuture future; + public abstract R parse(ClientResponse response); - public OffsetCommitCompletionHandler(Map offsets, RequestFuture future) { - this.offsets = offsets; - this.future = future; - } + public abstract void handle(R response, RequestFuture future); @Override - public void onComplete(ClientResponse resp) { - if (resp.wasDisconnected()) { - handleCoordinatorDisconnect(resp); - future.retryWithNewCoordinator(); - } else { - OffsetCommitResponse commitResponse = new OffsetCommitResponse(resp.responseBody()); - for (Map.Entry entry : commitResponse.responseData().entrySet()) { - TopicPartition tp = entry.getKey(); - short errorCode = entry.getValue(); - long offset = this.offsets.get(tp); - if (errorCode == Errors.NONE.code()) { - log.debug("Committed offset {} for partition {}", offset, tp); - subscriptions.committed(tp, offset); - } else if (errorCode == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code() - || errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { - coordinatorDead(); - future.retryWithNewCoordinator(); - } else if (errorCode == Errors.OFFSET_METADATA_TOO_LARGE.code() - || errorCode == Errors.INVALID_COMMIT_OFFSET_SIZE.code()) { - // do not need to throw the exception but just log the error - log.error("Error committing partition {} at offset {}: {}", - tp, - offset, - Errors.forCode(errorCode).exception().getMessage()); - } else if (errorCode == Errors.UNKNOWN_CONSUMER_ID.code() - || errorCode == Errors.ILLEGAL_GENERATION.code()) { - // need to re-join group - subscriptions.needReassignment(); - } else { - // re-throw the exception as these should not happen - log.error("Error committing partition {} at offset {}: {}", - tp, - offset, - Errors.forCode(errorCode).exception().getMessage()); - } - } + public void onSuccess(ClientResponse clientResponse, RequestFuture future) { + this.response = clientResponse; - if (!future.isDone()) - future.complete(null); + if (clientResponse.wasDisconnected()) { + int correlation = response.request().request().header().correlationId(); + log.debug("Cancelled request {} with correlation id {} due to coordinator {} being disconnected", + response.request(), + correlation, + response.request().request().destination()); + + // mark the coordinator as dead + coordinatorDead(); + future.raise(new DisconnectException()); + return; + } + + R response = parse(clientResponse); + handle(response, future); + } + + @Override + public void onFailure(RuntimeException e, RequestFuture future) { + if (e instanceof DisconnectException) { + log.debug("Coordinator request failed", e); + coordinatorDead(); } - sensors.commitLatency.record(resp.requestLatencyMs()); + future.raise(e); } } + public interface RebalanceCallback { + void onPartitionsAssigned(Collection partitions); + void onPartitionsRevoked(Collection partitions); + } + private class CoordinatorMetrics { public final Metrics metrics; public final String metricGrpName; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedTask.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedTask.java new file mode 100644 index 0000000..61663f8 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedTask.java @@ -0,0 +1,24 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package org.apache.kafka.clients.consumer.internals; + + +public interface DelayedTask { + + /** + * Execute the task. + * @param now current time in milliseconds + */ + void run(long now); +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedTaskQueue.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedTaskQueue.java new file mode 100644 index 0000000..61cab20 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedTaskQueue.java @@ -0,0 +1,96 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package org.apache.kafka.clients.consumer.internals; + +import java.util.Iterator; +import java.util.PriorityQueue; + +/** + * Tracks a set of tasks to be executed after a delay. + */ +public class DelayedTaskQueue { + + private PriorityQueue tasks; + + public DelayedTaskQueue() { + tasks = new PriorityQueue(); + } + + /** + * Schedule a task for execution in the future. + * + * @param task the task to execute + * @param at the time at which to + */ + public void add(DelayedTask task, long at) { + tasks.add(new Entry(task, at)); + } + + /** + * Remove a task from the queue if it is present + * @param task the task to be removed + * @returns true if a task was removed as a result of this call + */ + public boolean remove(DelayedTask task) { + boolean wasRemoved = false; + Iterator iterator = tasks.iterator(); + while (iterator.hasNext()) { + Entry entry = iterator.next(); + if (entry.task.equals(task)) { + iterator.remove(); + wasRemoved = true; + } + } + return wasRemoved; + } + + /** + * Get amount of time in milliseconds until the next event. Returns Long.MAX_VALUE if no tasks are scheduled. + * + * @return the remaining time in milliseconds + */ + public long nextTimeout(long now) { + if (tasks.isEmpty()) + return Long.MAX_VALUE; + else + return Math.max(tasks.peek().timeout - now, 0); + } + + /** + * Run any ready tasks. + * + * @param now the current time + */ + public void poll(long now) { + while (!tasks.isEmpty() && tasks.peek().timeout <= now) { + Entry entry = tasks.poll(); + entry.task.run(now); + } + } + + private static class Entry implements Comparable { + DelayedTask task; + long timeout; + + public Entry(DelayedTask task, long timeout) { + this.task = task; + this.timeout = timeout; + } + + @Override + public int compareTo(Entry entry) { + return Long.compare(timeout, entry.timeout); + } + } +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index 695eaf6..d595c1c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -13,17 +13,18 @@ package org.apache.kafka.clients.consumer.internals; -import org.apache.kafka.clients.ClientRequest; import org.apache.kafka.clients.ClientResponse; -import org.apache.kafka.clients.KafkaClient; import org.apache.kafka.clients.Metadata; -import org.apache.kafka.clients.RequestCompletionHandler; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.NoOffsetForPartitionException; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.DisconnectException; +import org.apache.kafka.common.errors.InvalidMetadataException; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.metrics.stats.Avg; @@ -38,7 +39,6 @@ import org.apache.kafka.common.requests.FetchRequest; import org.apache.kafka.common.requests.FetchResponse; import org.apache.kafka.common.requests.ListOffsetRequest; import org.apache.kafka.common.requests.ListOffsetResponse; -import org.apache.kafka.common.requests.RequestSend; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; @@ -52,21 +52,24 @@ import java.util.HashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.Set; /** * This class manage the fetching process with the brokers. */ public class Fetcher { + private static final long EARLIEST_OFFSET_TIMESTAMP = -2L; + private static final long LATEST_OFFSET_TIMESTAMP = -1L; private static final Logger log = LoggerFactory.getLogger(Fetcher.class); - private final KafkaClient client; - + private final ConsumerNetworkClient client; private final Time time; private final int minBytes; private final int maxWaitMs; private final int fetchSize; + private final long retryBackoffMs; private final boolean checkCrcs; private final Metadata metadata; private final FetchManagerMetrics sensors; @@ -75,8 +78,7 @@ public class Fetcher { private final Deserializer keyDeserializer; private final Deserializer valueDeserializer; - - public Fetcher(KafkaClient client, + public Fetcher(ConsumerNetworkClient client, int minBytes, int maxWaitMs, int fetchSize, @@ -88,7 +90,8 @@ public class Fetcher { Metrics metrics, String metricGrpPrefix, Map metricTags, - Time time) { + Time time, + long retryBackoffMs) { this.time = time; this.client = client; @@ -105,25 +108,105 @@ public class Fetcher { this.records = new LinkedList>(); this.sensors = new FetchManagerMetrics(metrics, metricGrpPrefix, metricTags); + this.retryBackoffMs = retryBackoffMs; } /** * Set-up a fetch request for any node that we have assigned partitions for which doesn't have one. * * @param cluster The current cluster metadata - * @param now The current time */ - public void initFetches(Cluster cluster, long now) { - for (ClientRequest request : createFetchRequests(cluster)) { - Node node = cluster.nodeById(Integer.parseInt(request.request().destination())); - if (client.ready(node, now)) { - log.trace("Initiating fetch to node {}: {}", node.id(), request); - client.send(request); + public void initFetches(Cluster cluster) { + for (Map.Entry fetchEntry: createFetchRequests(cluster).entrySet()) { + final FetchRequest fetch = fetchEntry.getValue(); + client.send(fetchEntry.getKey(), ApiKeys.FETCH, fetch) + .addListener(new RequestFutureListener() { + @Override + public void onSuccess(ClientResponse response) { + handleFetchResponse(response, fetch); + } + + @Override + public void onFailure(RuntimeException e) { + log.debug("Fetch failed", e); + } + }); + } + } + + /** + * Update the fetch positions for the provided partitions. + * @param partitions + */ + public void updateFetchPositions(Set partitions) { + // reset the fetch position to the committed position + for (TopicPartition tp : partitions) { + // skip if we already have a fetch position + if (subscriptions.fetched(tp) != null) + continue; + + // TODO: If there are several offsets to reset, we could submit offset requests in parallel + if (subscriptions.isOffsetResetNeeded(tp)) { + resetOffset(tp); + } else if (subscriptions.committed(tp) == null) { + // there's no committed position, so we need to reset with the default strategy + subscriptions.needOffsetReset(tp); + resetOffset(tp); + } else { + log.debug("Resetting offset for partition {} to the committed offset {}", + tp, subscriptions.committed(tp)); + subscriptions.seek(tp, subscriptions.committed(tp)); } } } /** + * Reset offsets for the given partition using the offset reset strategy. + * + * @param partition The given partition that needs reset offset + * @throws org.apache.kafka.clients.consumer.NoOffsetForPartitionException If no offset reset strategy is defined + */ + private void resetOffset(TopicPartition partition) { + OffsetResetStrategy strategy = subscriptions.resetStrategy(partition); + final long timestamp; + if (strategy == OffsetResetStrategy.EARLIEST) + timestamp = EARLIEST_OFFSET_TIMESTAMP; + else if (strategy == OffsetResetStrategy.LATEST) + timestamp = LATEST_OFFSET_TIMESTAMP; + else + throw new NoOffsetForPartitionException("No offset is set and no reset policy is defined"); + + log.debug("Resetting offset for partition {} to {} offset.", partition, strategy.name().toLowerCase()); + long offset = listOffset(partition, timestamp); + this.subscriptions.seek(partition, offset); + } + + /** + * Fetch a single offset before the given timestamp for the partition. + * + * @param partition The partition that needs fetching offset. + * @param timestamp The timestamp for fetching offset. + * @return The offset of the message that is published before the given timestamp + */ + private long listOffset(TopicPartition partition, long timestamp) { + while (true) { + RequestFuture future = sendListOffsetRequest(partition, timestamp); + client.poll(future); + + if (future.succeeded()) + return future.value(); + + if (!future.isRetriable()) + throw future.exception(); + + if (future.exception() instanceof InvalidMetadataException) + client.awaitMetadataUpdate(); + else + Utils.sleep(retryBackoffMs); + } + } + + /** * Return the fetched records, empty the record buffer and update the consumed position. * * @return The fetched records per partition @@ -163,37 +246,27 @@ public class Fetcher { * @param timestamp The timestamp for fetching offset. * @return A response which can be polled to obtain the corresponding offset. */ - public RequestFuture listOffset(final TopicPartition topicPartition, long timestamp) { + private RequestFuture sendListOffsetRequest(final TopicPartition topicPartition, long timestamp) { Map partitions = new HashMap(1); partitions.put(topicPartition, new ListOffsetRequest.PartitionData(timestamp, 1)); - long now = time.milliseconds(); PartitionInfo info = metadata.fetch().partition(topicPartition); if (info == null) { metadata.add(topicPartition.topic()); log.debug("Partition {} is unknown for fetching offset, wait for metadata refresh", topicPartition); - return RequestFuture.metadataRefreshNeeded(); + return RequestFuture.staleMetadata(); } else if (info.leader() == null) { log.debug("Leader for partition {} unavailable for fetching offset, wait for metadata refresh", topicPartition); - return RequestFuture.metadataRefreshNeeded(); - } else if (this.client.ready(info.leader(), now)) { - final RequestFuture future = new RequestFuture(); + return RequestFuture.leaderNotAvailable(); + } else { Node node = info.leader(); ListOffsetRequest request = new ListOffsetRequest(-1, partitions); - RequestSend send = new RequestSend(node.idString(), - this.client.nextRequestHeader(ApiKeys.LIST_OFFSETS), - request.toStruct()); - RequestCompletionHandler completionHandler = new RequestCompletionHandler() { - @Override - public void onComplete(ClientResponse resp) { - handleListOffsetResponse(topicPartition, resp, future); - } - }; - ClientRequest clientRequest = new ClientRequest(now, true, send, completionHandler); - this.client.send(clientRequest); - return future; - } else { - // We initiated a connect to the leader, but we need to poll to finish it. - return RequestFuture.pollNeeded(); + return client.send(node, ApiKeys.LIST_OFFSETS, request) + .compose(new RequestFutureAdapter() { + @Override + public void onSuccess(ClientResponse response, RequestFuture future) { + handleListOffsetResponse(topicPartition, response, future); + } + }); } } @@ -206,7 +279,7 @@ public class Fetcher { ClientResponse clientResponse, RequestFuture future) { if (clientResponse.wasDisconnected()) { - future.retryAfterMetadataRefresh(); + future.raise(new DisconnectException()); } else { ListOffsetResponse lor = new ListOffsetResponse(clientResponse.responseBody()); short errorCode = lor.responseData().get(topicPartition).errorCode; @@ -222,11 +295,11 @@ public class Fetcher { || errorCode == Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) { log.warn("Attempt to fetch offsets for partition {} failed due to obsolete leadership information, retrying.", topicPartition); - future.retryAfterMetadataRefresh(); + future.raise(Errors.forCode(errorCode)); } else { log.error("Attempt to fetch offsets for partition {} failed due to: {}", topicPartition, Errors.forCode(errorCode).exception().getMessage()); - future.retryAfterMetadataRefresh(); + future.raise(new StaleMetadataException()); } } } @@ -235,37 +308,31 @@ public class Fetcher { * Create fetch requests for all nodes for which we have assigned partitions * that have no existing requests in flight. */ - private List createFetchRequests(Cluster cluster) { + private Map createFetchRequests(Cluster cluster) { // create the fetch info - Map> fetchable = new HashMap>(); + Map> fetchable = new HashMap>(); for (TopicPartition partition : subscriptions.assignedPartitions()) { Node node = cluster.leaderFor(partition); if (node == null) { metadata.requestUpdate(); - } else if (this.client.inFlightRequestCount(node.idString()) == 0) { + } else if (this.client.pendingRequestCount(node) == 0) { // if there is a leader and no in-flight requests, issue a new fetch - Map fetch = fetchable.get(node.id()); + Map fetch = fetchable.get(node); if (fetch == null) { fetch = new HashMap(); - fetchable.put(node.id(), fetch); + fetchable.put(node, fetch); } long offset = this.subscriptions.fetched(partition); fetch.put(partition, new FetchRequest.PartitionData(offset, this.fetchSize)); } } - // create the requests - List requests = new ArrayList(fetchable.size()); - for (Map.Entry> entry : fetchable.entrySet()) { - int nodeId = entry.getKey(); - final FetchRequest fetch = new FetchRequest(this.maxWaitMs, this.minBytes, entry.getValue()); - RequestSend send = new RequestSend(Integer.toString(nodeId), this.client.nextRequestHeader(ApiKeys.FETCH), fetch.toStruct()); - RequestCompletionHandler handler = new RequestCompletionHandler() { - public void onComplete(ClientResponse response) { - handleFetchResponse(response, fetch); - } - }; - requests.add(new ClientRequest(time.milliseconds(), true, send, handler)); + // create the fetches + Map requests = new HashMap(); + for (Map.Entry> entry : fetchable.entrySet()) { + Node node = entry.getKey(); + FetchRequest fetch = new FetchRequest(this.maxWaitMs, this.minBytes, entry.getValue()); + requests.put(node, fetch); } return requests; } @@ -353,7 +420,6 @@ public class Fetcher { } } - private class FetchManagerMetrics { public final Metrics metrics; public final String metricGrpName; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java index 51eae19..6da8936 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java @@ -13,7 +13,7 @@ package org.apache.kafka.clients.consumer.internals; /** - * A helper class for managing the heartbeat to the co-ordinator + * A helper class for managing the heartbeat to the coordinator */ public final class Heartbeat { @@ -25,18 +25,24 @@ public final class Heartbeat { private final long timeout; private long lastHeartbeatSend; + private long lastHeartbeatReceive; + private long lastSessionReset; public Heartbeat(long timeout, long now) { this.timeout = timeout; - this.lastHeartbeatSend = now; + this.lastSessionReset = now; } public void sentHeartbeat(long now) { this.lastHeartbeatSend = now; } + public void receiveHeartbeat(long now) { + this.lastHeartbeatReceive = now; + } + public boolean shouldHeartbeat(long now) { - return now - lastHeartbeatSend > (1.0 / HEARTBEATS_PER_SESSION_INTERVAL) * this.timeout; + return timeToNextHeartbeat(now) == 0; } public long lastHeartbeatSend() { @@ -44,7 +50,7 @@ public final class Heartbeat { } public long timeToNextHeartbeat(long now) { - long timeSinceLastHeartbeat = now - lastHeartbeatSend; + long timeSinceLastHeartbeat = now - Math.max(lastHeartbeatSend, lastSessionReset); long hbInterval = timeout / HEARTBEATS_PER_SESSION_INTERVAL; if (timeSinceLastHeartbeat > hbInterval) @@ -52,4 +58,17 @@ public final class Heartbeat { else return hbInterval - timeSinceLastHeartbeat; } + + public boolean sessionTimeoutExpired(long now) { + return now - Math.max(lastSessionReset, lastHeartbeatReceive) > timeout; + } + + public long interval() { + return timeout / HEARTBEATS_PER_SESSION_INTERVAL; + } + + public void resetSessionTimeout(long now) { + this.lastSessionReset = now; + } + } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoAvailableBrokersException.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoAvailableBrokersException.java new file mode 100644 index 0000000..0ec6017 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoAvailableBrokersException.java @@ -0,0 +1,23 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals; + +import org.apache.kafka.common.errors.InvalidMetadataException; + +/** + * No brokers were available to complete a request. + */ +public class NoAvailableBrokersException extends InvalidMetadataException { + private static final long serialVersionUID = 1L; + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFuture.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFuture.java index 13fc9af..5f00251 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFuture.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFuture.java @@ -12,78 +12,49 @@ */ package org.apache.kafka.clients.consumer.internals; +import org.apache.kafka.common.errors.RetriableException; +import org.apache.kafka.common.protocol.Errors; + +import java.util.ArrayList; +import java.util.List; + /** - * Result of an asynchronous request through {@link org.apache.kafka.clients.KafkaClient}. To get the - * result of the request, you must use poll using {@link org.apache.kafka.clients.KafkaClient#poll(long, long)} - * until {@link #isDone()} returns true. Typical usage might look like this: + * Result of an asynchronous request from {@link ConsumerNetworkClient}. Use {@link ConsumerNetworkClient#poll(long)} + * (and variants) to finish a request future. Use {@link #isDone()} to check if the future is complete, and + * {@link #succeeded()} to check if the request completed successfully. Typical usage might look like this: * *

    - *     RequestFuture future = sendRequest();
    - *     while (!future.isDone()) {
    - *         client.poll(timeout, now);
    - *     }
    + *     RequestFuture future = client.send(api, request);
    + *     client.poll(future);
      *
    - *     switch (future.outcome()) {
    - *     case SUCCESS:
    - *         // handle request success
    - *         break;
    - *     case NEED_RETRY:
    - *         // retry after taking possible retry action
    - *         break;
    - *     case EXCEPTION:
    - *         // handle exception
    -  *     }
    + *     if (future.succeeded()) {
    + *         ClientResponse response = future.value();
    + *         // Handle response
    + *     } else {
    + *         throw future.exception();
    + *     }
      * 
    * - * When {@link #isDone()} returns true, there are three possible outcomes (obtained through {@link #outcome()}): - * - *
      - *
    1. {@link org.apache.kafka.clients.consumer.internals.RequestFuture.Outcome#SUCCESS}: If the request was - * successful, then you can use {@link #value()} to obtain the result.
    2. - *
    3. {@link org.apache.kafka.clients.consumer.internals.RequestFuture.Outcome#EXCEPTION}: If an unhandled exception - * was encountered, you can use {@link #exception()} to get it.
    4. - *
    5. {@link org.apache.kafka.clients.consumer.internals.RequestFuture.Outcome#NEED_RETRY}: The request may - * not have been successful, but the failure may be ephemeral and the caller just needs to try the request again. - * In this case, use {@link #retryAction()} to determine what action should be taken (if any) before - * retrying.
    6. - *
    - * * @param Return type of the result (Can be Void if there is no response) */ public class RequestFuture { - public static final RequestFuture NEED_NEW_COORDINATOR = newRetryFuture(RetryAction.FIND_COORDINATOR); - public static final RequestFuture NEED_POLL = newRetryFuture(RetryAction.POLL); - public static final RequestFuture NEED_METADATA_REFRESH = newRetryFuture(RetryAction.REFRESH_METADATA); - - public enum RetryAction { - NOOP, // Retry immediately. - POLL, // Retry after calling poll (e.g. to finish a connection) - BACKOFF, // Retry after a delay - FIND_COORDINATOR, // Find a new coordinator before retrying - REFRESH_METADATA // Refresh metadata before retrying - } - - public enum Outcome { - SUCCESS, - NEED_RETRY, - EXCEPTION - } - private Outcome outcome; - private RetryAction retryAction; + private boolean isDone = false; private T value; private RuntimeException exception; + private List> listeners = new ArrayList>(); + /** * Check whether the response is ready to be handled * @return true if the response is ready, false otherwise */ public boolean isDone() { - return outcome != null; + return isDone; } /** - * Get the value corresponding to this request (if it has one, as indicated by {@link #outcome()}). + * Get the value corresponding to this request (only available if the request succeeded) * @return the value if it exists or null */ public T value() { @@ -92,32 +63,31 @@ public class RequestFuture { /** * Check if the request succeeded; - * @return true if a value is available, false otherwise + * @return true if the request completed and was successful */ public boolean succeeded() { - return outcome == Outcome.SUCCESS; + return isDone && exception == null; } /** - * Check if the request completed failed. - * @return true if the request failed (whether or not it can be retried) + * Check if the request failed. + * @return true if the request completed with a failure */ public boolean failed() { - return outcome != Outcome.SUCCESS; + return isDone && exception != null; } /** - * Return the error from this response (assuming {@link #succeeded()} has returned false. If the - * response is not ready or if there is no retryAction, null is returned. - * @return the error if it exists or null + * Check if the request is retriable (convenience method for checking if + * the exception is an instance of {@link RetriableException}. + * @return true if it is retriable, false otherwise */ - public RetryAction retryAction() { - return retryAction; + public boolean isRetriable() { + return exception instanceof RetriableException; } /** - * Get the exception from a failed result. You should check that there is an exception - * with {@link #hasException()} before using this method. + * Get the exception from a failed result (only available if the request failed) * @return The exception if it exists or null */ public RuntimeException exception() { @@ -125,85 +95,108 @@ public class RequestFuture { } /** - * Check whether there was an exception. - * @return true if this request failed with an exception + * Complete the request successfully. After this call, {@link #succeeded()} will return true + * and the value can be obtained through {@link #value()}. + * @param value corresponding value (or null if there is none) */ - public boolean hasException() { - return outcome == Outcome.EXCEPTION; + public void complete(T value) { + this.value = value; + this.isDone = true; + fireSuccess(); } /** - * Check the outcome of the future if it is ready. - * @return the outcome or null if the future is not finished + * Raise an exception. The request will be marked as failed, and the caller can either + * handle the exception or throw it. + * @param e corresponding exception to be passed to caller */ - public Outcome outcome() { - return outcome; + public void raise(RuntimeException e) { + this.exception = e; + this.isDone = true; + fireFailure(); } /** - * The request failed, but should be retried using the provided retry action. - * @param retryAction The action that should be taken by the caller before retrying the request + * Raise an error. The request will be marked as failed. + * @param error corresponding error to be passed to caller */ - public void retry(RetryAction retryAction) { - this.outcome = Outcome.NEED_RETRY; - this.retryAction = retryAction; - } - - public void retryNow() { - retry(RetryAction.NOOP); - } - - public void retryAfterBackoff() { - retry(RetryAction.BACKOFF); + public void raise(Errors error) { + raise(error.exception()); } - public void retryWithNewCoordinator() { - retry(RetryAction.FIND_COORDINATOR); + private void fireSuccess() { + for (RequestFutureListener listener: listeners) + listener.onSuccess(value); } - public void retryAfterMetadataRefresh() { - retry(RetryAction.REFRESH_METADATA); + private void fireFailure() { + for (RequestFutureListener listener: listeners) + listener.onFailure(exception); } /** - * Complete the request successfully. After this call, {@link #succeeded()} will return true - * and the value can be obtained through {@link #value()}. - * @param value corresponding value (or null if there is none) + * Add a listener which will be notified when the future completes + * @param listener */ - public void complete(T value) { - this.outcome = Outcome.SUCCESS; - this.value = value; + public void addListener(RequestFutureListener listener) { + if (isDone) { + if (exception != null) + listener.onFailure(exception); + else + listener.onSuccess(value); + } else { + this.listeners.add(listener); + } } /** - * Raise an exception. The request will be marked as failed, and the caller can either - * handle the exception or throw it. - * @param e The exception that + * Convert from a request future of one type to another type + * @param adapter The adapter which does the conversion + * @param The type of the future adapted to + * @return The new future */ - public void raise(RuntimeException e) { - this.outcome = Outcome.EXCEPTION; - this.exception = e; + public RequestFuture compose(final RequestFutureAdapter adapter) { + final RequestFuture adapted = new RequestFuture(); + addListener(new RequestFutureListener() { + @Override + public void onSuccess(T value) { + adapter.onSuccess(value, adapted); + } + + @Override + public void onFailure(RuntimeException e) { + adapter.onFailure(e, adapted); + } + }); + return adapted; + } + + public static RequestFuture failure(RuntimeException e) { + RequestFuture future = new RequestFuture(); + future.raise(e); + return future; + } + + public static RequestFuture voidSuccess() { + RequestFuture future = new RequestFuture(); + future.complete(null); + return future; } - private static RequestFuture newRetryFuture(RetryAction retryAction) { - RequestFuture result = new RequestFuture(); - result.retry(retryAction); - return result; + public static RequestFuture coordinatorNotAvailable() { + return failure(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.exception()); } - @SuppressWarnings("unchecked") - public static RequestFuture pollNeeded() { - return (RequestFuture) NEED_POLL; + public static RequestFuture leaderNotAvailable() { + return failure(Errors.LEADER_NOT_AVAILABLE.exception()); } - @SuppressWarnings("unchecked") - public static RequestFuture metadataRefreshNeeded() { - return (RequestFuture) NEED_METADATA_REFRESH; + public static RequestFuture noBrokersAvailable() { + return failure(new NoAvailableBrokersException()); } - @SuppressWarnings("unchecked") - public static RequestFuture newCoordinatorNeeded() { - return (RequestFuture) NEED_NEW_COORDINATOR; + public static RequestFuture staleMetadata() { + return failure(new StaleMetadataException()); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFutureAdapter.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFutureAdapter.java new file mode 100644 index 0000000..cc5322f --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFutureAdapter.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals; + +/** + * Adapt from a request future of one type to another. + * + * @param Type to adapt from + * @param Type to adapt to + */ +public abstract class RequestFutureAdapter { + + public abstract void onSuccess(F value, RequestFuture future); + + public void onFailure(RuntimeException e, RequestFuture future) { + future.raise(e); + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFutureListener.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFutureListener.java new file mode 100644 index 0000000..b39261b --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFutureListener.java @@ -0,0 +1,23 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals; + +/** + * Listener interface to hook into RequestFuture completion. + */ +public interface RequestFutureListener { + + void onSuccess(T value); + + void onFailure(RuntimeException e); +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SendFailedException.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SendFailedException.java new file mode 100644 index 0000000..3312a2c --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SendFailedException.java @@ -0,0 +1,27 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals; + +import org.apache.kafka.common.errors.RetriableException; + +/** + * Exception used in {@link ConsumerNetworkClient} to indicate the failure + * to transmit a request to the networking layer. This could be either because + * the client is still connecting to the given host or its send buffer is full. + */ +public class SendFailedException extends RetriableException { + public static final SendFailedException INSTANCE = new SendFailedException(); + + private static final long serialVersionUID = 1L; + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StaleMetadataException.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StaleMetadataException.java new file mode 100644 index 0000000..09114cb --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StaleMetadataException.java @@ -0,0 +1,22 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals; + +import org.apache.kafka.common.errors.InvalidMetadataException; + +/** + * Thrown when metadata is old and needs to be refreshed. + */ +public class StaleMetadataException extends InvalidMetadataException { + private static final long serialVersionUID = 1L; +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java index 6837453..4d9a425 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java @@ -138,7 +138,6 @@ public class SubscriptionState { public void committed(TopicPartition tp, long offset) { this.committed.put(tp, offset); - this.needsFetchCommittedOffsets = false; } public Long committed(TopicPartition tp) { @@ -152,6 +151,10 @@ public class SubscriptionState { public boolean refreshCommitsNeeded() { return this.needsFetchCommittedOffsets; } + + public void commitsRefreshed() { + this.needsFetchCommittedOffsets = false; + } public void seek(TopicPartition tp, long offset) { fetched(tp, offset); diff --git a/clients/src/main/java/org/apache/kafka/common/errors/ConsumerCoordinatorNotAvailableException.java b/clients/src/main/java/org/apache/kafka/common/errors/ConsumerCoordinatorNotAvailableException.java new file mode 100644 index 0000000..ba9ce82 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/ConsumerCoordinatorNotAvailableException.java @@ -0,0 +1,40 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package org.apache.kafka.common.errors; + +/** + * The broker returns this error code for consumer metadata requests or offset commit requests if the offsets topic has + * not yet been created. + */ +public class ConsumerCoordinatorNotAvailableException extends RetriableException { + + private static final long serialVersionUID = 1L; + + public ConsumerCoordinatorNotAvailableException() { + super(); + } + + public ConsumerCoordinatorNotAvailableException(String message) { + super(message); + } + + public ConsumerCoordinatorNotAvailableException(String message, Throwable cause) { + super(message, cause); + } + + public ConsumerCoordinatorNotAvailableException(Throwable cause) { + super(cause); + } + +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/errors/DisconnectException.java b/clients/src/main/java/org/apache/kafka/common/errors/DisconnectException.java new file mode 100644 index 0000000..18d61a2 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/DisconnectException.java @@ -0,0 +1,39 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package org.apache.kafka.common.errors; + + +/** + * Server disconnected before a request could be completed. + */ +public class DisconnectException extends RetriableException { + + private static final long serialVersionUID = 1L; + + public DisconnectException() { + super(); + } + + public DisconnectException(String message, Throwable cause) { + super(message, cause); + } + + public DisconnectException(String message) { + super(message); + } + + public DisconnectException(Throwable cause) { + super(cause); + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/errors/IllegalGenerationException.java b/clients/src/main/java/org/apache/kafka/common/errors/IllegalGenerationException.java new file mode 100644 index 0000000..d20b74a --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/IllegalGenerationException.java @@ -0,0 +1,33 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package org.apache.kafka.common.errors; + +public class IllegalGenerationException extends RetriableException { + private static final long serialVersionUID = 1L; + + public IllegalGenerationException() { + super(); + } + + public IllegalGenerationException(String message, Throwable cause) { + super(message, cause); + } + + public IllegalGenerationException(String message) { + super(message); + } + + public IllegalGenerationException(Throwable cause) { + super(cause); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/errors/NotCoordinatorForConsumerException.java b/clients/src/main/java/org/apache/kafka/common/errors/NotCoordinatorForConsumerException.java new file mode 100644 index 0000000..b6c83b4 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/NotCoordinatorForConsumerException.java @@ -0,0 +1,40 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package org.apache.kafka.common.errors; + +/** + * The broker returns this error code if it receives an offset fetch or commit request for a consumer group that it is + * not a coordinator for. + */ +public class NotCoordinatorForConsumerException extends RetriableException { + + private static final long serialVersionUID = 1L; + + public NotCoordinatorForConsumerException() { + super(); + } + + public NotCoordinatorForConsumerException(String message) { + super(message); + } + + public NotCoordinatorForConsumerException(String message, Throwable cause) { + super(message, cause); + } + + public NotCoordinatorForConsumerException(Throwable cause) { + super(cause); + } + +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/errors/OffsetLoadInProgressException.java b/clients/src/main/java/org/apache/kafka/common/errors/OffsetLoadInProgressException.java new file mode 100644 index 0000000..016506e --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/OffsetLoadInProgressException.java @@ -0,0 +1,40 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package org.apache.kafka.common.errors; + +/** + * The broker returns this error code for an offset fetch request if it is still loading offsets (after a leader change + * for that offsets topic partition). + */ +public class OffsetLoadInProgressException extends RetriableException { + + private static final long serialVersionUID = 1L; + + public OffsetLoadInProgressException() { + super(); + } + + public OffsetLoadInProgressException(String message) { + super(message); + } + + public OffsetLoadInProgressException(String message, Throwable cause) { + super(message, cause); + } + + public OffsetLoadInProgressException(Throwable cause) { + super(cause); + } + +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/errors/UnknownConsumerIdException.java b/clients/src/main/java/org/apache/kafka/common/errors/UnknownConsumerIdException.java new file mode 100644 index 0000000..9bcbd11 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/UnknownConsumerIdException.java @@ -0,0 +1,33 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package org.apache.kafka.common.errors; + +public class UnknownConsumerIdException extends RetriableException { + private static final long serialVersionUID = 1L; + + public UnknownConsumerIdException() { + super(); + } + + public UnknownConsumerIdException(String message, Throwable cause) { + super(message, cause); + } + + public UnknownConsumerIdException(String message) { + super(message); + } + + public UnknownConsumerIdException(Throwable cause) { + super(cause); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java index 4c0ecc3..d6c41c1 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java @@ -53,11 +53,11 @@ public enum Errors { NETWORK_EXCEPTION(13, new NetworkException("The server disconnected before a response was received.")), OFFSET_LOAD_IN_PROGRESS(14, - new ApiException("The coordinator is loading offsets and can't process requests.")), + new OffsetLoadInProgressException("The coordinator is loading offsets and can't process requests.")), CONSUMER_COORDINATOR_NOT_AVAILABLE(15, - new ApiException("The coordinator is not available.")), + new ConsumerCoordinatorNotAvailableException("The coordinator is not available.")), NOT_COORDINATOR_FOR_CONSUMER(16, - new ApiException("This is not the correct co-ordinator for this consumer.")), + new NotCoordinatorForConsumerException("This is not the correct coordinator for this consumer.")), INVALID_TOPIC_EXCEPTION(17, new InvalidTopicException("The request attempted to perform an operation on an invalid topic.")), RECORD_LIST_TOO_LARGE(18, @@ -69,13 +69,13 @@ public enum Errors { INVALID_REQUIRED_ACKS(21, new InvalidRequiredAcksException("Produce request specified an invalid value for required acks.")), ILLEGAL_GENERATION(22, - new ApiException("Specified consumer generation id is not valid.")), + new IllegalGenerationException("Specified consumer generation id is not valid.")), INCONSISTENT_PARTITION_ASSIGNMENT_STRATEGY(23, new ApiException("The request partition assignment strategy does not match that of the group.")), UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY(24, new ApiException("The request partition assignment strategy is unknown to the broker.")), UNKNOWN_CONSUMER_ID(25, - new ApiException("The coordinator is not aware of this consumer.")), + new UnknownConsumerIdException("The coordinator is not aware of this consumer.")), INVALID_SESSION_TIMEOUT(26, new ApiException("The session timeout is not within an acceptable range.")), COMMITTING_PARTITIONS_NOT_ASSIGNED(27, diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest.java new file mode 100644 index 0000000..9de1cee --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest.java @@ -0,0 +1,125 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals; + +import org.apache.kafka.clients.ClientResponse; +import org.apache.kafka.clients.Metadata; +import org.apache.kafka.clients.MockClient; +import org.apache.kafka.clients.consumer.ConsumerWakeupException; +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.types.Struct; +import org.apache.kafka.common.requests.HeartbeatRequest; +import org.apache.kafka.common.requests.HeartbeatResponse; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.test.TestUtils; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +public class ConsumerNetworkClientTest { + + private String topicName = "test"; + private MockTime time = new MockTime(); + private MockClient client = new MockClient(time); + private Cluster cluster = TestUtils.singletonCluster(topicName, 1); + private Node node = cluster.nodes().get(0); + private Metadata metadata = new Metadata(0, Long.MAX_VALUE); + private ConsumerNetworkClient consumerClient = new ConsumerNetworkClient(client, metadata, time, 100); + + @Test + public void send() { + client.prepareResponse(heartbeatResponse(Errors.NONE.code())); + RequestFuture future = consumerClient.send(node, ApiKeys.METADATA, heartbeatRequest()); + assertEquals(1, consumerClient.pendingRequestCount()); + assertEquals(1, consumerClient.pendingRequestCount(node)); + assertFalse(future.isDone()); + + consumerClient.poll(future); + assertTrue(future.isDone()); + assertTrue(future.succeeded()); + + ClientResponse clientResponse = future.value(); + HeartbeatResponse response = new HeartbeatResponse(clientResponse.responseBody()); + assertEquals(Errors.NONE.code(), response.errorCode()); + } + + @Test + public void multiSend() { + client.prepareResponse(heartbeatResponse(Errors.NONE.code())); + client.prepareResponse(heartbeatResponse(Errors.NONE.code())); + RequestFuture future1 = consumerClient.send(node, ApiKeys.METADATA, heartbeatRequest()); + RequestFuture future2 = consumerClient.send(node, ApiKeys.METADATA, heartbeatRequest()); + assertEquals(2, consumerClient.pendingRequestCount()); + assertEquals(2, consumerClient.pendingRequestCount(node)); + + consumerClient.awaitPendingRequests(node); + assertTrue(future1.succeeded()); + assertTrue(future2.succeeded()); + } + + @Test + public void schedule() { + TestDelayedTask task = new TestDelayedTask(); + consumerClient.schedule(task, time.milliseconds()); + consumerClient.poll(0); + assertEquals(1, task.executions); + + consumerClient.schedule(task, time.milliseconds() + 100); + consumerClient.poll(0); + assertEquals(1, task.executions); + + time.sleep(100); + consumerClient.poll(0); + assertEquals(2, task.executions); + } + + @Test + public void wakeup() { + RequestFuture future = consumerClient.send(node, ApiKeys.METADATA, heartbeatRequest()); + consumerClient.wakeup(); + try { + consumerClient.poll(0); + fail(); + } catch (ConsumerWakeupException e) { + } + + client.respond(heartbeatResponse(Errors.NONE.code())); + consumerClient.poll(future); + assertTrue(future.isDone()); + } + + + private HeartbeatRequest heartbeatRequest() { + return new HeartbeatRequest("group", 1, "consumerId"); + } + + private Struct heartbeatResponse(short error) { + HeartbeatResponse response = new HeartbeatResponse(error); + return response.toStruct(); + } + + private static class TestDelayedTask implements DelayedTask { + int executions = 0; + @Override + public void run(long now) { + executions++; + } + } + +} 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 d085fe5..ca832be 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 @@ -18,13 +18,19 @@ package org.apache.kafka.clients.consumer.internals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; +import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.MockClient; +import org.apache.kafka.clients.consumer.CommitType; +import org.apache.kafka.clients.consumer.ConsumerCommitCallback; import org.apache.kafka.clients.consumer.OffsetResetStrategy; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.ApiException; +import org.apache.kafka.common.errors.DisconnectException; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.types.Struct; @@ -36,10 +42,12 @@ import org.apache.kafka.common.requests.OffsetFetchResponse; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.test.TestUtils; +import java.util.Collection; import java.util.Collections; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; import org.junit.Before; import org.junit.Test; @@ -51,108 +59,173 @@ public class CoordinatorTest { private String groupId = "test-group"; private TopicPartition tp = new TopicPartition(topicName, 0); private int sessionTimeoutMs = 10; + private long retryBackoffMs = 100; + private long requestTimeoutMs = 5000; private String rebalanceStrategy = "not-matter"; - private MockTime time = new MockTime(); - private MockClient client = new MockClient(time); + private MockTime time; + private MockClient client; private Cluster cluster = TestUtils.singletonCluster(topicName, 1); private Node node = cluster.nodes().get(0); - private SubscriptionState subscriptions = new SubscriptionState(OffsetResetStrategy.EARLIEST); - private Metrics metrics = new Metrics(time); + private SubscriptionState subscriptions; + private Metadata metadata; + private Metrics metrics; private Map metricTags = new LinkedHashMap(); - - private Coordinator coordinator = new Coordinator(client, - groupId, - sessionTimeoutMs, - rebalanceStrategy, - subscriptions, - metrics, - "consumer" + groupId, - metricTags, - time); + private ConsumerNetworkClient consumerClient; + private MockRebalanceCallback rebalanceCallback; + private Coordinator coordinator; @Before public void setup() { + this.time = new MockTime(); + this.client = new MockClient(time); + this.subscriptions = new SubscriptionState(OffsetResetStrategy.EARLIEST); + this.metadata = new Metadata(0, Long.MAX_VALUE); + this.consumerClient = new ConsumerNetworkClient(client, metadata, time, 100); + this.metrics = new Metrics(time); + this.rebalanceCallback = new MockRebalanceCallback(); + client.setNode(node); + + this.coordinator = new Coordinator(consumerClient, + groupId, + sessionTimeoutMs, + rebalanceStrategy, + subscriptions, + metrics, + "consumer" + groupId, + metricTags, + time, + requestTimeoutMs, + retryBackoffMs, + rebalanceCallback); } @Test public void testNormalHeartbeat() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); - coordinator.discoverConsumerCoordinator(); - client.poll(0, time.milliseconds()); + coordinator.ensureCoordinatorKnown(); // normal heartbeat time.sleep(sessionTimeoutMs); - coordinator.maybeHeartbeat(time.milliseconds()); // should send out the heartbeat - assertEquals(1, client.inFlightRequestCount()); - client.respond(heartbeatResponse(Errors.NONE.code())); - assertEquals(1, client.poll(0, time.milliseconds()).size()); + RequestFuture future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat + assertEquals(1, consumerClient.pendingRequestCount()); + assertFalse(future.isDone()); + + client.prepareResponse(heartbeatResponse(Errors.NONE.code())); + consumerClient.poll(0); + + assertTrue(future.isDone()); + assertTrue(future.succeeded()); } @Test public void testCoordinatorNotAvailable() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); - coordinator.discoverConsumerCoordinator(); - client.poll(0, time.milliseconds()); + coordinator.ensureCoordinatorKnown(); // consumer_coordinator_not_available will mark coordinator as unknown time.sleep(sessionTimeoutMs); - coordinator.maybeHeartbeat(time.milliseconds()); // should send out the heartbeat - assertEquals(1, client.inFlightRequestCount()); - client.respond(heartbeatResponse(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code())); + RequestFuture future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat + assertEquals(1, consumerClient.pendingRequestCount()); + assertFalse(future.isDone()); + + client.prepareResponse(heartbeatResponse(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code())); time.sleep(sessionTimeoutMs); - assertEquals(1, client.poll(0, time.milliseconds()).size()); + consumerClient.poll(0); + + assertTrue(future.isDone()); + assertTrue(future.failed()); + assertEquals(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.exception(), future.exception()); assertTrue(coordinator.coordinatorUnknown()); } @Test public void testNotCoordinator() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); - coordinator.discoverConsumerCoordinator(); - client.poll(0, time.milliseconds()); + coordinator.ensureCoordinatorKnown(); // not_coordinator will mark coordinator as unknown time.sleep(sessionTimeoutMs); - coordinator.maybeHeartbeat(time.milliseconds()); // should send out the heartbeat - assertEquals(1, client.inFlightRequestCount()); - client.respond(heartbeatResponse(Errors.NOT_COORDINATOR_FOR_CONSUMER.code())); + RequestFuture future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat + assertEquals(1, consumerClient.pendingRequestCount()); + assertFalse(future.isDone()); + + client.prepareResponse(heartbeatResponse(Errors.NOT_COORDINATOR_FOR_CONSUMER.code())); time.sleep(sessionTimeoutMs); - assertEquals(1, client.poll(0, time.milliseconds()).size()); + consumerClient.poll(0); + + assertTrue(future.isDone()); + assertTrue(future.failed()); + assertEquals(Errors.NOT_COORDINATOR_FOR_CONSUMER.exception(), future.exception()); assertTrue(coordinator.coordinatorUnknown()); } @Test public void testIllegalGeneration() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); - coordinator.discoverConsumerCoordinator(); - client.poll(0, time.milliseconds()); + coordinator.ensureCoordinatorKnown(); // illegal_generation will cause re-partition subscriptions.subscribe(topicName); subscriptions.changePartitionAssignment(Collections.singletonList(tp)); time.sleep(sessionTimeoutMs); - coordinator.maybeHeartbeat(time.milliseconds()); // should send out the heartbeat - assertEquals(1, client.inFlightRequestCount()); - client.respond(heartbeatResponse(Errors.ILLEGAL_GENERATION.code())); + RequestFuture future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat + assertEquals(1, consumerClient.pendingRequestCount()); + assertFalse(future.isDone()); + + client.prepareResponse(heartbeatResponse(Errors.ILLEGAL_GENERATION.code())); time.sleep(sessionTimeoutMs); - assertEquals(1, client.poll(0, time.milliseconds()).size()); + consumerClient.poll(0); + + assertTrue(future.isDone()); + assertTrue(future.failed()); + assertEquals(Errors.ILLEGAL_GENERATION.exception(), future.exception()); + assertTrue(subscriptions.partitionAssignmentNeeded()); + } + + @Test + public void testUnknownConsumerId() { + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + // illegal_generation will cause re-partition + subscriptions.subscribe(topicName); + subscriptions.changePartitionAssignment(Collections.singletonList(tp)); + + time.sleep(sessionTimeoutMs); + RequestFuture future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat + assertEquals(1, consumerClient.pendingRequestCount()); + assertFalse(future.isDone()); + + client.prepareResponse(heartbeatResponse(Errors.UNKNOWN_CONSUMER_ID.code())); + time.sleep(sessionTimeoutMs); + consumerClient.poll(0); + + assertTrue(future.isDone()); + assertTrue(future.failed()); + assertEquals(Errors.UNKNOWN_CONSUMER_ID.exception(), future.exception()); assertTrue(subscriptions.partitionAssignmentNeeded()); } @Test public void testCoordinatorDisconnect() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); - coordinator.discoverConsumerCoordinator(); - client.poll(0, time.milliseconds()); + coordinator.ensureCoordinatorKnown(); // coordinator disconnect will mark coordinator as unknown time.sleep(sessionTimeoutMs); - coordinator.maybeHeartbeat(time.milliseconds()); // should send out the heartbeat - assertEquals(1, client.inFlightRequestCount()); - client.respond(heartbeatResponse(Errors.NONE.code()), true); // return disconnected + RequestFuture future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat + assertEquals(1, consumerClient.pendingRequestCount()); + assertFalse(future.isDone()); + + client.prepareResponse(heartbeatResponse(Errors.NONE.code()), true); // return disconnected time.sleep(sessionTimeoutMs); - assertEquals(1, client.poll(0, time.milliseconds()).size()); + consumerClient.poll(0); + + assertTrue(future.isDone()); + assertTrue(future.failed()); + assertTrue(future.exception() instanceof DisconnectException); assertTrue(coordinator.coordinatorUnknown()); } @@ -162,16 +235,18 @@ public class CoordinatorTest { subscriptions.needReassignment(); client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); - coordinator.discoverConsumerCoordinator(); - client.poll(0, time.milliseconds()); + coordinator.ensureCoordinatorKnown(); // normal join group client.prepareResponse(joinGroupResponse(1, "consumer", Collections.singletonList(tp), Errors.NONE.code())); - coordinator.assignPartitions(time.milliseconds()); - client.poll(0, time.milliseconds()); + coordinator.ensurePartitionAssignment(); assertFalse(subscriptions.partitionAssignmentNeeded()); assertEquals(Collections.singleton(tp), subscriptions.assignedPartitions()); + assertEquals(1, rebalanceCallback.revokedCount); + assertEquals(Collections.emptySet(), rebalanceCallback.revoked); + assertEquals(1, rebalanceCallback.assignedCount); + assertEquals(Collections.singleton(tp), rebalanceCallback.assigned); } @Test @@ -180,165 +255,228 @@ public class CoordinatorTest { subscriptions.needReassignment(); client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); - coordinator.discoverConsumerCoordinator(); - client.poll(0, time.milliseconds()); - assertTrue(subscriptions.partitionAssignmentNeeded()); + coordinator.ensureCoordinatorKnown(); - // diconnected from original coordinator will cause re-discover and join again + // disconnected from original coordinator will cause re-discover and join again client.prepareResponse(joinGroupResponse(1, "consumer", Collections.singletonList(tp), Errors.NONE.code()), true); - coordinator.assignPartitions(time.milliseconds()); - client.poll(0, time.milliseconds()); - assertTrue(subscriptions.partitionAssignmentNeeded()); - - // rediscover the coordinator client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); - coordinator.discoverConsumerCoordinator(); - client.poll(0, time.milliseconds()); - - // try assigning partitions again client.prepareResponse(joinGroupResponse(1, "consumer", Collections.singletonList(tp), Errors.NONE.code())); - coordinator.assignPartitions(time.milliseconds()); - client.poll(0, time.milliseconds()); + coordinator.ensurePartitionAssignment(); assertFalse(subscriptions.partitionAssignmentNeeded()); assertEquals(Collections.singleton(tp), subscriptions.assignedPartitions()); + assertEquals(1, rebalanceCallback.revokedCount); + assertEquals(Collections.emptySet(), rebalanceCallback.revoked); + assertEquals(1, rebalanceCallback.assignedCount); + assertEquals(Collections.singleton(tp), rebalanceCallback.assigned); } + @Test(expected = ApiException.class) + public void testUnknownPartitionAssignmentStrategy() { + subscriptions.subscribe(topicName); + subscriptions.needReassignment(); + + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + // coordinator doesn't like our assignment strategy + client.prepareResponse(joinGroupResponse(0, "consumer", Collections.emptyList(), Errors.UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY.code())); + coordinator.ensurePartitionAssignment(); + } + + @Test(expected = ApiException.class) + public void testInvalidSessionTimeout() { + subscriptions.subscribe(topicName); + subscriptions.needReassignment(); + + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + // coordinator doesn't like our assignment strategy + client.prepareResponse(joinGroupResponse(0, "consumer", Collections.emptyList(), Errors.INVALID_SESSION_TIMEOUT.code())); + coordinator.ensurePartitionAssignment(); + } @Test public void testCommitOffsetNormal() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); - coordinator.discoverConsumerCoordinator(); - client.poll(0, time.milliseconds()); + coordinator.ensureCoordinatorKnown(); - // With success flag client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); - RequestFuture result = coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds()); - assertEquals(1, client.poll(0, time.milliseconds()).size()); - assertTrue(result.isDone()); - assertTrue(result.succeeded()); - // Without success flag - coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds()); - client.respond(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); - assertEquals(1, client.poll(0, time.milliseconds()).size()); + AtomicBoolean success = new AtomicBoolean(false); + coordinator.commitOffsets(Collections.singletonMap(tp, 100L), CommitType.ASYNC, callback(success)); + consumerClient.poll(0); + assertTrue(success.get()); } @Test - public void testCommitOffsetError() { + public void testCommitOffsetAsyncCoordinatorNotAvailable() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); - coordinator.discoverConsumerCoordinator(); - client.poll(0, time.milliseconds()); + coordinator.ensureCoordinatorKnown(); // async commit with coordinator not available + MockCommitCallback cb = new MockCommitCallback(); client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code()))); - coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds()); - assertEquals(1, client.poll(0, time.milliseconds()).size()); + coordinator.commitOffsets(Collections.singletonMap(tp, 100L), CommitType.ASYNC, cb); + consumerClient.poll(0); + assertTrue(coordinator.coordinatorUnknown()); - // resume + assertEquals(1, cb.invoked); + assertEquals(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.exception(), cb.exception); + } + + @Test + public void testCommitOffsetAsyncNotCoordinator() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); - coordinator.discoverConsumerCoordinator(); - client.poll(0, time.milliseconds()); + coordinator.ensureCoordinatorKnown(); // async commit with not coordinator + MockCommitCallback cb = new MockCommitCallback(); client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NOT_COORDINATOR_FOR_CONSUMER.code()))); - coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds()); - assertEquals(1, client.poll(0, time.milliseconds()).size()); + coordinator.commitOffsets(Collections.singletonMap(tp, 100L), CommitType.ASYNC, cb); + consumerClient.poll(0); + assertTrue(coordinator.coordinatorUnknown()); - // resume + assertEquals(1, cb.invoked); + assertEquals(Errors.NOT_COORDINATOR_FOR_CONSUMER.exception(), cb.exception); + } + + @Test + public void testCommitOffsetAsyncDisconnected() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); - coordinator.discoverConsumerCoordinator(); - client.poll(0, time.milliseconds()); + coordinator.ensureCoordinatorKnown(); - // sync commit with not_coordinator + // async commit with coordinator disconnected + MockCommitCallback cb = new MockCommitCallback(); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())), true); + coordinator.commitOffsets(Collections.singletonMap(tp, 100L), CommitType.ASYNC, cb); + consumerClient.poll(0); + + assertTrue(coordinator.coordinatorUnknown()); + assertEquals(1, cb.invoked); + assertTrue(cb.exception instanceof DisconnectException); + } + + @Test + public void testCommitOffsetSyncNotCoordinator() { + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + // sync commit with coordinator disconnected (should connect, get metadata, and then submit the commit request) + MockCommitCallback cb = new MockCommitCallback(); client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NOT_COORDINATOR_FOR_CONSUMER.code()))); client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); - RequestFuture result = coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds()); - assertEquals(1, client.poll(0, time.milliseconds()).size()); - assertTrue(result.isDone()); - assertEquals(RequestFuture.RetryAction.FIND_COORDINATOR, result.retryAction()); + coordinator.commitOffsets(Collections.singletonMap(tp, 100L), CommitType.SYNC, cb); + assertEquals(1, cb.invoked); + assertNull(cb.exception); + } - // sync commit with coordinator disconnected - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())), true); - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); - result = coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds()); + @Test + public void testCommitOffsetSyncCoordinatorNotAvailable() { + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); - assertEquals(0, client.poll(0, time.milliseconds()).size()); - assertTrue(result.isDone()); - assertEquals(RequestFuture.RetryAction.FIND_COORDINATOR, result.retryAction()); + // sync commit with coordinator disconnected (should connect, get metadata, and then submit the commit request) + MockCommitCallback cb = new MockCommitCallback(); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code()))); + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); + coordinator.commitOffsets(Collections.singletonMap(tp, 100L), CommitType.SYNC, cb); + assertEquals(1, cb.invoked); + assertNull(cb.exception); + } + @Test + public void testCommitOffsetSyncCoordinatorDisconnected() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); - coordinator.discoverConsumerCoordinator(); - client.poll(0, time.milliseconds()); + coordinator.ensureCoordinatorKnown(); - result = coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds()); - assertEquals(1, client.poll(0, time.milliseconds()).size()); - assertTrue(result.isDone()); - assertTrue(result.succeeded()); + // sync commit with coordinator disconnected (should connect, get metadata, and then submit the commit request) + MockCommitCallback cb = new MockCommitCallback(); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())), true); + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); + coordinator.commitOffsets(Collections.singletonMap(tp, 100L), CommitType.SYNC, cb); + assertEquals(1, cb.invoked); + assertNull(cb.exception); } + @Test(expected = ApiException.class) + public void testCommitOffsetSyncThrowsNonRetriableException() { + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + // sync commit with invalid partitions should throw if we have no callback + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.COMMITTING_PARTITIONS_NOT_ASSIGNED.code())), false); + coordinator.commitOffsets(Collections.singletonMap(tp, 100L), CommitType.SYNC, null); + } @Test - public void testFetchOffset() { + public void testCommitOffsetSyncCallbackHandlesNonRetriableException() { + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + // sync commit with invalid partitions should throw if we have no callback + MockCommitCallback cb = new MockCommitCallback(); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.COMMITTING_PARTITIONS_NOT_ASSIGNED.code())), false); + coordinator.commitOffsets(Collections.singletonMap(tp, 100L), CommitType.SYNC, cb); + assertTrue(cb.exception instanceof ApiException); + } + @Test + public void testRefreshOffset() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); - coordinator.discoverConsumerCoordinator(); - client.poll(0, time.milliseconds()); + coordinator.ensureCoordinatorKnown(); - // normal fetch + subscriptions.subscribe(tp); + subscriptions.needRefreshCommits(); client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", 100L)); - RequestFuture> result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); - client.poll(0, time.milliseconds()); - assertTrue(result.isDone()); - assertEquals(100L, (long) result.value().get(tp)); + coordinator.refreshCommittedOffsetsIfNeeded(); + assertFalse(subscriptions.refreshCommitsNeeded()); + assertEquals(100L, (long) subscriptions.committed(tp)); + } - // fetch with loading in progress + @Test + public void testRefreshOffsetLoadInProgress() { + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + subscriptions.subscribe(tp); + subscriptions.needRefreshCommits(); client.prepareResponse(offsetFetchResponse(tp, Errors.OFFSET_LOAD_IN_PROGRESS.code(), "", 100L)); client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", 100L)); + coordinator.refreshCommittedOffsetsIfNeeded(); + assertFalse(subscriptions.refreshCommitsNeeded()); + assertEquals(100L, (long) subscriptions.committed(tp)); + } - result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); - client.poll(0, time.milliseconds()); - assertTrue(result.isDone()); - assertTrue(result.failed()); - assertEquals(RequestFuture.RetryAction.BACKOFF, result.retryAction()); - - result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); - client.poll(0, time.milliseconds()); - assertTrue(result.isDone()); - assertEquals(100L, (long) result.value().get(tp)); + @Test + public void testRefreshOffsetNotCoordinatorForConsumer() { + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); - // fetch with not coordinator + subscriptions.subscribe(tp); + subscriptions.needRefreshCommits(); client.prepareResponse(offsetFetchResponse(tp, Errors.NOT_COORDINATOR_FOR_CONSUMER.code(), "", 100L)); client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", 100L)); + coordinator.refreshCommittedOffsetsIfNeeded(); + assertFalse(subscriptions.refreshCommitsNeeded()); + assertEquals(100L, (long) subscriptions.committed(tp)); + } - result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); - client.poll(0, time.milliseconds()); - assertTrue(result.isDone()); - assertTrue(result.failed()); - assertEquals(RequestFuture.RetryAction.FIND_COORDINATOR, result.retryAction()); - - coordinator.discoverConsumerCoordinator(); - client.poll(0, time.milliseconds()); - - result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); - client.poll(0, time.milliseconds()); - assertTrue(result.isDone()); - assertEquals(100L, (long) result.value().get(tp)); - - // fetch with no fetchable offsets - client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", -1L)); - result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); - client.poll(0, time.milliseconds()); - assertTrue(result.isDone()); - assertTrue(result.value().isEmpty()); + @Test + public void testRefreshOffsetWithNoFetchableOffsets() { + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); - // fetch with offset -1 + subscriptions.subscribe(tp); + subscriptions.needRefreshCommits(); client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", -1L)); - result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); - client.poll(0, time.milliseconds()); - assertTrue(result.isDone()); - assertTrue(result.value().isEmpty()); + coordinator.refreshCommittedOffsetsIfNeeded(); + assertFalse(subscriptions.refreshCommitsNeeded()); + assertEquals(null, subscriptions.committed(tp)); } private Struct consumerMetadataResponse(Node node, short error) { @@ -366,4 +504,45 @@ public class CoordinatorTest { OffsetFetchResponse response = new OffsetFetchResponse(Collections.singletonMap(tp, data)); return response.toStruct(); } + + private ConsumerCommitCallback callback(final AtomicBoolean success) { + return new ConsumerCommitCallback() { + @Override + public void onComplete(Map offsets, Exception exception) { + if (exception == null) + success.set(true); + } + }; + } + + private static class MockCommitCallback implements ConsumerCommitCallback { + public int invoked = 0; + public Exception exception = null; + + @Override + public void onComplete(Map offsets, Exception exception) { + invoked++; + this.exception = exception; + } + } + + private static class MockRebalanceCallback implements Coordinator.RebalanceCallback { + public Collection revoked; + public Collection assigned; + public int revokedCount = 0; + public int assignedCount = 0; + + + @Override + public void onPartitionsAssigned(Collection partitions) { + this.assigned = partitions; + assignedCount++; + } + + @Override + public void onPartitionsRevoked(Collection partitions) { + this.revoked = partitions; + revokedCount++; + } + } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/DelayedTaskQueueTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/DelayedTaskQueueTest.java new file mode 100644 index 0000000..db87b66 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/DelayedTaskQueueTest.java @@ -0,0 +1,89 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals; + +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; + +import static org.junit.Assert.assertEquals; + +public class DelayedTaskQueueTest { + private DelayedTaskQueue scheduler = new DelayedTaskQueue(); + private ArrayList executed = new ArrayList(); + + @Test + public void testScheduling() { + // Empty scheduler + assertEquals(Long.MAX_VALUE, scheduler.nextTimeout(0)); + scheduler.poll(0); + assertEquals(Collections.emptyList(), executed); + + TestTask task1 = new TestTask(); + TestTask task2 = new TestTask(); + TestTask task3 = new TestTask(); + scheduler.add(task1, 20); + assertEquals(20, scheduler.nextTimeout(0)); + scheduler.add(task2, 10); + assertEquals(10, scheduler.nextTimeout(0)); + scheduler.add(task3, 30); + assertEquals(10, scheduler.nextTimeout(0)); + + scheduler.poll(5); + assertEquals(Collections.emptyList(), executed); + assertEquals(5, scheduler.nextTimeout(5)); + + scheduler.poll(10); + assertEquals(Arrays.asList(task2), executed); + assertEquals(10, scheduler.nextTimeout(10)); + + scheduler.poll(20); + assertEquals(Arrays.asList(task2, task1), executed); + assertEquals(20, scheduler.nextTimeout(10)); + + scheduler.poll(30); + assertEquals(Arrays.asList(task2, task1, task3), executed); + assertEquals(Long.MAX_VALUE, scheduler.nextTimeout(30)); + } + + @Test + public void testRemove() { + TestTask task1 = new TestTask(); + TestTask task2 = new TestTask(); + TestTask task3 = new TestTask(); + scheduler.add(task1, 20); + scheduler.add(task2, 10); + scheduler.add(task3, 30); + scheduler.add(task1, 40); + assertEquals(10, scheduler.nextTimeout(0)); + + scheduler.remove(task2); + assertEquals(20, scheduler.nextTimeout(0)); + + scheduler.remove(task1); + assertEquals(30, scheduler.nextTimeout(0)); + + scheduler.remove(task3); + assertEquals(Long.MAX_VALUE, scheduler.nextTimeout(0)); + } + + private class TestTask implements DelayedTask { + @Override + public void run(long now) { + executed.add(this); + } + } + +} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java index 405efdc..7a4e586 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -52,6 +52,7 @@ public class FetcherTest { private int minBytes = 1; private int maxWaitMs = 0; private int fetchSize = 1000; + private long retryBackoffMs = 100; private MockTime time = new MockTime(); private MockClient client = new MockClient(time); private Metadata metadata = new Metadata(0, Long.MAX_VALUE); @@ -60,10 +61,11 @@ public class FetcherTest { private SubscriptionState subscriptions = new SubscriptionState(OffsetResetStrategy.EARLIEST); private Metrics metrics = new Metrics(time); private Map metricTags = new LinkedHashMap(); + private ConsumerNetworkClient consumerClient = new ConsumerNetworkClient(client, metadata, time, 100); private MemoryRecords records = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), CompressionType.NONE); - private Fetcher fetcher = new Fetcher(client, + private Fetcher fetcher = new Fetcher(consumerClient, minBytes, maxWaitMs, fetchSize, @@ -75,7 +77,8 @@ public class FetcherTest { metrics, "consumer" + groupId, metricTags, - time); + time, + retryBackoffMs); @Before public void setup() throws Exception { @@ -97,9 +100,9 @@ public class FetcherTest { subscriptions.consumed(tp, 0); // normal fetch - fetcher.initFetches(cluster, time.milliseconds()); - client.respond(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L)); - client.poll(0, time.milliseconds()); + fetcher.initFetches(cluster); + client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L)); + consumerClient.poll(0); records = fetcher.fetchedRecords().get(tp); assertEquals(3, records.size()); assertEquals(4L, (long) subscriptions.fetched(tp)); // this is the next fetching position @@ -119,24 +122,24 @@ public class FetcherTest { subscriptions.consumed(tp, 0); // fetch with not leader - fetcher.initFetches(cluster, time.milliseconds()); - client.respond(fetchResponse(this.records.buffer(), Errors.NOT_LEADER_FOR_PARTITION.code(), 100L)); - client.poll(0, time.milliseconds()); + fetcher.initFetches(cluster); + client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NOT_LEADER_FOR_PARTITION.code(), 100L)); + consumerClient.poll(0); assertEquals(0, fetcher.fetchedRecords().size()); assertEquals(0L, metadata.timeToNextUpdate(time.milliseconds())); // fetch with unknown topic partition - fetcher.initFetches(cluster, time.milliseconds()); - client.respond(fetchResponse(this.records.buffer(), Errors.UNKNOWN_TOPIC_OR_PARTITION.code(), 100L)); - client.poll(0, time.milliseconds()); + fetcher.initFetches(cluster); + client.prepareResponse(fetchResponse(this.records.buffer(), Errors.UNKNOWN_TOPIC_OR_PARTITION.code(), 100L)); + consumerClient.poll(0); assertEquals(0, fetcher.fetchedRecords().size()); assertEquals(0L, metadata.timeToNextUpdate(time.milliseconds())); // fetch with out of range subscriptions.fetched(tp, 5); - fetcher.initFetches(cluster, time.milliseconds()); - client.respond(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L)); - client.poll(0, time.milliseconds()); + fetcher.initFetches(cluster); + client.prepareResponse(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L)); + consumerClient.poll(0); assertTrue(subscriptions.isOffsetResetNeeded(tp)); assertEquals(0, fetcher.fetchedRecords().size()); assertEquals(null, subscriptions.fetched(tp)); @@ -151,9 +154,9 @@ public class FetcherTest { subscriptions.consumed(tp, 5); // fetch with out of range - fetcher.initFetches(cluster, time.milliseconds()); - client.respond(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L)); - client.poll(0, time.milliseconds()); + fetcher.initFetches(cluster); + client.prepareResponse(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L)); + consumerClient.poll(0); assertTrue(subscriptions.isOffsetResetNeeded(tp)); assertEquals(0, fetcher.fetchedRecords().size()); assertEquals(null, subscriptions.fetched(tp)); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatTest.java index ee1ede0..b587e14 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatTest.java @@ -51,4 +51,19 @@ public class HeartbeatTest { assertEquals(0, heartbeat.timeToNextHeartbeat(100)); assertEquals(0, heartbeat.timeToNextHeartbeat(200)); } + + @Test + public void testSessionTimeoutExpired() { + heartbeat.sentHeartbeat(time.milliseconds()); + time.sleep(305); + assertTrue(heartbeat.sessionTimeoutExpired(time.milliseconds())); + } + + @Test + public void testResetSession() { + heartbeat.sentHeartbeat(time.milliseconds()); + time.sleep(305); + heartbeat.resetSessionTimeout(time.milliseconds()); + assertFalse(heartbeat.sessionTimeoutExpired(time.milliseconds())); + } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestFutureTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestFutureTest.java new file mode 100644 index 0000000..7372754 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestFutureTest.java @@ -0,0 +1,57 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals; + +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class RequestFutureTest { + + @Test + public void testComposeSuccessCase() { + RequestFuture future = new RequestFuture(); + RequestFuture composed = future.compose(new RequestFutureAdapter() { + @Override + public void onSuccess(String value, RequestFuture future) { + future.complete(value.length()); + } + }); + + future.complete("hello"); + + assertTrue(composed.isDone()); + assertTrue(composed.succeeded()); + assertEquals(5, (int) composed.value()); + } + + @Test + public void testComposeFailureCase() { + RequestFuture future = new RequestFuture(); + RequestFuture composed = future.compose(new RequestFutureAdapter() { + @Override + public void onSuccess(String value, RequestFuture future) { + future.complete(value.length()); + } + }); + + RuntimeException e = new RuntimeException(); + future.raise(e); + + assertTrue(composed.isDone()); + assertTrue(composed.failed()); + assertEquals(e, composed.exception()); + } + +} diff --git a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala index 92ffb91..3eb5f95 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala @@ -12,17 +12,13 @@ */ package kafka.api +import java.{lang, util} + import org.apache.kafka.clients.producer.ProducerConfig import org.apache.kafka.clients.producer.ProducerRecord -import org.apache.kafka.clients.consumer.Consumer -import org.apache.kafka.clients.consumer.KafkaConsumer -import org.apache.kafka.clients.consumer.ConsumerRebalanceCallback -import org.apache.kafka.clients.consumer.ConsumerRecord -import org.apache.kafka.clients.consumer.ConsumerConfig -import org.apache.kafka.clients.consumer.CommitType +import org.apache.kafka.clients.consumer._ import org.apache.kafka.common.serialization.ByteArrayDeserializer import org.apache.kafka.common.TopicPartition -import org.apache.kafka.clients.consumer.NoOffsetForPartitionException import kafka.utils.{TestUtils, Logging} import kafka.server.KafkaConfig @@ -46,6 +42,8 @@ class ConsumerTest extends IntegrationTestHarness with Logging { val topic = "topic" val part = 0 val tp = new TopicPartition(topic, part) + val part2 = 1 + val tp2 = new TopicPartition(topic, part2) // configure the servers and clients this.serverConfig.setProperty(KafkaConfig.ControlledShutdownEnableProp, "false") // speed up shutdown @@ -56,12 +54,13 @@ 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") - + this.consumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") + override def setUp() { super.setUp() // create the test topic with all the brokers as replicas - TestUtils.createTopic(this.zkClient, topic, 1, serverCount, this.servers) + TestUtils.createTopic(this.zkClient, topic, 2, serverCount, this.servers) } def testSimpleConsumption() { @@ -74,6 +73,45 @@ class ConsumerTest extends IntegrationTestHarness with Logging { this.consumers(0).seek(tp, 0) consumeRecords(this.consumers(0), numRecords = numRecords, startingOffset = 0) + + // check async commit callbacks + val commitCallback = new CountConsumerCommitCallback() + this.consumers(0).commit(CommitType.ASYNC, commitCallback) + + // shouldn't make progress until poll is invoked + Thread.sleep(10) + assertEquals(0, commitCallback.count) + awaitCommitCallback(this.consumers(0), commitCallback) + } + + def testCommitSpecifiedOffsets() { + sendRecords(5, tp) + sendRecords(7, tp2) + + this.consumers(0).subscribe(tp) + this.consumers(0).subscribe(tp2) + + // Need to poll to join the group + this.consumers(0).poll(50) + val pos1 = this.consumers(0).position(tp) + val pos2 = this.consumers(0).position(tp2) + this.consumers(0).commit(Map[TopicPartition,java.lang.Long]((tp, 3L)), CommitType.SYNC) + assertEquals(3, this.consumers(0).committed(tp)) + intercept[NoOffsetForPartitionException] { + this.consumers(0).committed(tp2) + } + // positions should not change + assertEquals(pos1, this.consumers(0).position(tp)) + assertEquals(pos2, this.consumers(0).position(tp2)) + this.consumers(0).commit(Map[TopicPartition,java.lang.Long]((tp2, 5L)), CommitType.SYNC) + assertEquals(3, this.consumers(0).committed(tp)) + assertEquals(5, this.consumers(0).committed(tp2)) + + // Using async should pick up the committed changes after commit completes + val commitCallback = new CountConsumerCommitCallback() + this.consumers(0).commit(Map[TopicPartition,java.lang.Long]((tp2, 7L)), CommitType.ASYNC, commitCallback) + awaitCommitCallback(this.consumers(0), commitCallback) + assertEquals(7, this.consumers(0).committed(tp2)) } def testAutoOffsetReset() { @@ -150,7 +188,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging { def testPartitionReassignmentCallback() { val callback = new TestConsumerReassignmentCallback() - this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "200"); // timeout quickly to avoid slow test + this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "100"); // timeout quickly to avoid slow test val consumer0 = new KafkaConsumer(this.consumerConfig, callback, new ByteArrayDeserializer(), new ByteArrayDeserializer()) consumer0.subscribe(topic) @@ -172,6 +210,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging { // this should cause another callback execution while(callback.callsToAssigned < 2) consumer0.poll(50) + assertEquals(2, callback.callsToAssigned) assertEquals(2, callback.callsToRevoked) @@ -191,9 +230,13 @@ class ConsumerTest extends IntegrationTestHarness with Logging { } } - private def sendRecords(numRecords: Int) { + private def sendRecords(numRecords: Int): Unit = { + sendRecords(numRecords, tp) + } + + private def sendRecords(numRecords: Int, tp: TopicPartition) { val futures = (0 until numRecords).map { i => - this.producers(0).send(new ProducerRecord(topic, part, i.toString.getBytes, i.toString.getBytes)) + this.producers(0).send(new ProducerRecord(tp.topic(), tp.partition(), i.toString.getBytes, i.toString.getBytes)) } futures.map(_.get) } @@ -218,4 +261,18 @@ class ConsumerTest extends IntegrationTestHarness with Logging { } } + private def awaitCommitCallback(consumer: Consumer[Array[Byte], Array[Byte]], commitCallback: CountConsumerCommitCallback): Unit = { + val startCount = commitCallback.count + val started = System.currentTimeMillis() + while (commitCallback.count == startCount && System.currentTimeMillis() - started < 10000) + this.consumers(0).poll(10000) + assertEquals(startCount + 1, commitCallback.count) + } + + private class CountConsumerCommitCallback extends ConsumerCommitCallback { + var count = 0 + + override def onComplete(offsets: util.Map[TopicPartition, lang.Long], exception: Exception): Unit = count += 1 + } + } \ No newline at end of file -- 1.7.12.4 From 4fcb7acafe4c98cd2930e6d348d456a3eb7f7444 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Wed, 15 Jul 2015 15:46:19 -0700 Subject: [PATCH 049/120] KAFKA-2335; fix comment about thread safety Author: Jason Gustafson Closes #78 from hachikuji/KAFKA-2335 and squashes the following commits: c697998 [Jason Gustafson] KAFKA-2335; fix comment about consumer thread safety --- .../java/org/apache/kafka/clients/consumer/KafkaConsumer.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) 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 9f64255..bea3d73 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 @@ -66,9 +66,8 @@ import static org.apache.kafka.common.utils.Utils.min; * The consumer maintains TCP connections to the necessary brokers to fetch data for the topics it subscribes to. * Failure to close the consumer after use will leak these connections. *

    - * The consumer is thread safe but generally will be used only from within a single thread. The consumer client has no - * threads of it's own, all work is done in the caller's thread when calls are made on the various methods exposed. - * + * The consumer is not thread-safe. See Multi-threaded Processing for more details. + * *

    Offsets and Consumer Position

    * Kafka maintains a numerical offset for each record in a partition. This offset acts as a kind of unique identifier of * a record within that partition, and also denotes the position of the consumer in the partition. That is, a consumer @@ -302,7 +301,8 @@ import static org.apache.kafka.common.utils.Utils.min; * methods for seeking to the earliest and latest offset the server maintains are also available ( * {@link #seekToBeginning(TopicPartition...)} and {@link #seekToEnd(TopicPartition...)} respectively). * - *

    Multithreaded Processing

    + * + *

    Multi-threaded Processing

    * * The Kafka consumer is NOT thread-safe. All network I/O happens in the thread of the application * making the call. It is the responsibility of the user to ensure that multi-threaded access -- 1.7.12.4 From 83d17e5b2403a4916d51ab69825ea657f8c802a7 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Wed, 15 Jul 2015 19:00:37 -0700 Subject: [PATCH 050/120] [MINOR] fix new consumer heartbeat reschedule bug This commit fixes a minor issue introduced in the patch for KAFKA-2123. The schedule method requires the time the task should be executed, not a delay. Author: Jason Gustafson Closes #79 from hachikuji/KAFKA-2123-fix and squashes the following commits: 6eb7ec6 [Jason Gustafson] [Minor] fix new consumer heartbeat reschedule bug --- .../java/org/apache/kafka/clients/consumer/internals/Coordinator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 8e3cd09..6026b23 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 @@ -272,7 +272,7 @@ public final class Coordinator { @Override public void onFailure(RuntimeException e) { - client.schedule(HeartbeatTask.this, retryBackoffMs); + client.schedule(HeartbeatTask.this, time.milliseconds() + retryBackoffMs); } }); } -- 1.7.12.4 From fa03a7c6c48550f01176402139201af75d3836a2 Mon Sep 17 00:00:00 2001 From: Parth Brahmbhatt Date: Thu, 16 Jul 2015 12:59:27 -0700 Subject: [PATCH 051/120] KAFKA-2032: validate consumer's partition-assignment config; reviewed by Jason Rosenberg, Sriharsha Chintalapani and Guozhang Wang --- .../java/org/apache/kafka/clients/consumer/ConsumerConfig.java | 3 ++- core/src/main/scala/kafka/consumer/ConsumerConfig.scala | 10 ++++++++++ 2 files changed, 12 insertions(+), 1 deletion(-) 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 daff34d..70377ae 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 @@ -173,7 +173,8 @@ public class ConsumerConfig extends AbstractConfig { SESSION_TIMEOUT_MS_DOC) .define(PARTITION_ASSIGNMENT_STRATEGY_CONFIG, Type.STRING, - "blah", + "range", + in("range", "roundrobin"), Importance.MEDIUM, PARTITION_ASSIGNMENT_STRATEGY_DOC) .define(METADATA_MAX_AGE_CONFIG, diff --git a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala index 0199317..97a56ce 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala @@ -58,6 +58,7 @@ object ConsumerConfig extends Config { validateGroupId(config.groupId) validateAutoOffsetReset(config.autoOffsetReset) validateOffsetsStorage(config.offsetsStorage) + validatePartitionAssignmentStrategy(config.partitionAssignmentStrategy) } def validateClientId(clientId: String) { @@ -85,6 +86,15 @@ object ConsumerConfig extends Config { "Valid values are 'zookeeper' and 'kafka'") } } + + def validatePartitionAssignmentStrategy(strategy: String) { + strategy match { + case "range" => + case "roundrobin" => + case _ => throw new InvalidConfigException("Wrong value " + strategy + " of partition.assignment.strategy in consumer config; " + + "Valid values are 'range' and 'roundrobin'") + } + } } class ConsumerConfig private (val props: VerifiableProperties) extends ZKConfig(props) { -- 1.7.12.4 From 458ebeb04bffb3237a46c12e05e270d9b26d4d40 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 16 Jul 2015 16:21:50 -0700 Subject: [PATCH 052/120] kafka-2323; Simplify ScalaTest dependency versions; patched by Ismael Juma; reviewed by Jun Rao --- build.gradle | 20 +++++++++----------- 1 file changed, 9 insertions(+), 11 deletions(-) diff --git a/build.gradle b/build.gradle index d86f1a8..9059bc4 100644 --- a/build.gradle +++ b/build.gradle @@ -225,8 +225,6 @@ project(':core') { apply plugin: 'scala' archivesBaseName = "kafka_${baseScalaVersion}" - def (major, minor, trivial) = scalaVersion.tokenize('.') - dependencies { compile project(':clients') compile project(':log4j-appender') @@ -235,19 +233,19 @@ project(':core') { compile 'com.101tec:zkclient:0.5' compile 'com.yammer.metrics:metrics-core:2.2.0' compile 'net.sf.jopt-simple:jopt-simple:3.2' + if (scalaVersion.startsWith('2.11')) { + compile 'org.scala-lang.modules:scala-xml_2.11:1.0.3' + compile 'org.scala-lang.modules:scala-parser-combinators_2.11:1.0.3' + } testCompile 'junit:junit:4.6' testCompile 'org.easymock:easymock:3.0' testCompile 'org.objenesis:objenesis:1.2' - if (scalaVersion.startsWith('2.10')) { - testCompile 'org.scalatest:scalatest_2.10:1.9.1' - } else if (scalaVersion.startsWith('2.11')) { - compile 'org.scala-lang.modules:scala-xml_2.11:1.0.3' - compile 'org.scala-lang.modules:scala-parser-combinators_2.11:1.0.3' - testCompile "org.scalatest:scalatest_2.11:2.2.0" - } else { - testCompile "org.scalatest:scalatest_$scalaVersion:1.8" - } + if (scalaVersion.startsWith('2.9')) + testCompile "org.scalatest:scalatest_$scalaVersion:1.9.1" + else + testCompile "org.scalatest:scalatest_$baseScalaVersion:2.2.5" + testRuntime "$slf4jlog4j" zinc 'com.typesafe.zinc:zinc:0.3.7' -- 1.7.12.4 From 8fa24a6171781bbdb2b814dcdc37520a92511f69 Mon Sep 17 00:00:00 2001 From: Dong Lin Date: Thu, 16 Jul 2015 16:38:41 -0700 Subject: [PATCH 053/120] kafka-2241; AbstractFetcherThread.shutdown() should not block on ReadableByteChannel.read(buffer); patched by Dong Lin; reviewed by Jun Rao --- core/src/main/scala/kafka/consumer/SimpleConsumer.scala | 15 ++++++++++++++- .../main/scala/kafka/server/AbstractFetcherThread.scala | 4 +++- 2 files changed, 17 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/consumer/SimpleConsumer.scala b/core/src/main/scala/kafka/consumer/SimpleConsumer.scala index c16f7ed..7ebc040 100644 --- a/core/src/main/scala/kafka/consumer/SimpleConsumer.scala +++ b/core/src/main/scala/kafka/consumer/SimpleConsumer.scala @@ -18,7 +18,7 @@ package kafka.consumer -import java.nio.channels.ClosedByInterruptException +import java.nio.channels.{AsynchronousCloseException, ClosedByInterruptException} import kafka.api._ import kafka.network._ @@ -59,6 +59,16 @@ class SimpleConsumer(val host: String, connect() } + /** + * Unblock thread by closing channel and triggering AsynchronousCloseException if a read operation is in progress. + * + * This handles a bug found in Java 1.7 and below, where interrupting a thread can not correctly unblock + * the thread from waiting on ReadableByteChannel.read(). + */ + def disconnectToHandleJavaIOBug() = { + disconnect() + } + def close() { lock synchronized { disconnect() @@ -76,6 +86,9 @@ class SimpleConsumer(val host: String, } catch { case e : ClosedByInterruptException => throw e + // Should not observe this exception when running Kafka with Java 1.8 + case e: AsynchronousCloseException => + throw e case e : Throwable => info("Reconnect due to socket error: %s".format(e.toString)) // retry once diff --git a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala index 83fc474..f843061 100755 --- a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala @@ -67,7 +67,9 @@ abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroke def handlePartitionsWithErrors(partitions: Iterable[TopicAndPartition]) override def shutdown(){ - initiateShutdown() + val justShutdown = initiateShutdown() + if (justShutdown && isInterruptible) + simpleConsumer.disconnectToHandleJavaIOBug() inLock(partitionMapLock) { partitionMapCond.signalAll() } -- 1.7.12.4 From 1e0ed7a592cd80cfca727b27ba01f6faa511531f Mon Sep 17 00:00:00 2001 From: Alexey Ozeritskiy Date: Thu, 16 Jul 2015 17:13:12 -0700 Subject: [PATCH 054/120] kafka-2174; Wrong TopicMetadata deserialization; patched by Alexey Ozeritskiy; reviewed by Jun Rao --- core/src/main/scala/kafka/api/TopicMetadata.scala | 2 +- .../test/scala/unit/kafka/admin/AddPartitionsTest.scala | 17 +++++++++-------- 2 files changed, 10 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/kafka/api/TopicMetadata.scala b/core/src/main/scala/kafka/api/TopicMetadata.scala index bd866bc..7b56b31 100644 --- a/core/src/main/scala/kafka/api/TopicMetadata.scala +++ b/core/src/main/scala/kafka/api/TopicMetadata.scala @@ -34,7 +34,7 @@ object TopicMetadata { val partitionsMetadata: Array[PartitionMetadata] = new Array[PartitionMetadata](numPartitions) for(i <- 0 until numPartitions) { val partitionMetadata = PartitionMetadata.readFrom(buffer, brokers) - partitionsMetadata(partitionMetadata.partitionId) = partitionMetadata + partitionsMetadata(i) = partitionMetadata } new TopicMetadata(topic, partitionsMetadata, errorCode) } diff --git a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala index df5c6ba..8b14bcf 100755 --- a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala @@ -96,7 +96,7 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { val metadata = ClientUtils.fetchTopicMetadata(Set(topic1), brokers.map(_.getBrokerEndPoint(SecurityProtocol.PLAINTEXT)), "AddPartitionsTest-testIncrementPartitions", 2000,0).topicsMetadata val metaDataForTopic1 = metadata.filter(p => p.topic.equals(topic1)) - val partitionDataForTopic1 = metaDataForTopic1.head.partitionsMetadata + val partitionDataForTopic1 = metaDataForTopic1.head.partitionsMetadata.sortBy(_.partitionId) assertEquals(partitionDataForTopic1.size, 3) assertEquals(partitionDataForTopic1(1).partitionId, 1) assertEquals(partitionDataForTopic1(2).partitionId, 2) @@ -121,7 +121,7 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { val metadata = ClientUtils.fetchTopicMetadata(Set(topic2), brokers.map(_.getBrokerEndPoint(SecurityProtocol.PLAINTEXT)), "AddPartitionsTest-testManualAssignmentOfReplicas", 2000,0).topicsMetadata val metaDataForTopic2 = metadata.filter(p => p.topic.equals(topic2)) - val partitionDataForTopic2 = metaDataForTopic2.head.partitionsMetadata + val partitionDataForTopic2 = metaDataForTopic2.head.partitionsMetadata.sortBy(_.partitionId) assertEquals(partitionDataForTopic2.size, 3) assertEquals(partitionDataForTopic2(1).partitionId, 1) assertEquals(partitionDataForTopic2(2).partitionId, 2) @@ -146,12 +146,13 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { 2000,0).topicsMetadata val metaDataForTopic3 = metadata.filter(p => p.topic.equals(topic3)).head - val partition1DataForTopic3 = metaDataForTopic3.partitionsMetadata(1) - val partition2DataForTopic3 = metaDataForTopic3.partitionsMetadata(2) - val partition3DataForTopic3 = metaDataForTopic3.partitionsMetadata(3) - val partition4DataForTopic3 = metaDataForTopic3.partitionsMetadata(4) - val partition5DataForTopic3 = metaDataForTopic3.partitionsMetadata(5) - val partition6DataForTopic3 = metaDataForTopic3.partitionsMetadata(6) + val partitionsMetadataForTopic3 = metaDataForTopic3.partitionsMetadata.sortBy(_.partitionId) + val partition1DataForTopic3 = partitionsMetadataForTopic3(1) + val partition2DataForTopic3 = partitionsMetadataForTopic3(2) + val partition3DataForTopic3 = partitionsMetadataForTopic3(3) + val partition4DataForTopic3 = partitionsMetadataForTopic3(4) + val partition5DataForTopic3 = partitionsMetadataForTopic3(5) + val partition6DataForTopic3 = partitionsMetadataForTopic3(6) assertEquals(partition1DataForTopic3.replicas.size, 4) assertEquals(partition1DataForTopic3.replicas(0).id, 3) -- 1.7.12.4 From 5033b9613e9e9f28c8eb60a734920d90d155e6f9 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Thu, 16 Jul 2015 17:29:25 -0700 Subject: [PATCH 055/120] kafka-2271; transient unit test failure in KafkaConfigConfigDefTest.testFromPropsToProps; patched by Jason Gustafson; reviewed by Jun Rao --- .../test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala index 98a5b04..04a02e0 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala @@ -134,7 +134,7 @@ class KafkaConfigConfigDefTest extends JUnit3Suite { Assert.assertEquals(expectedConfig.compressionType, actualConfig.compressionType) } - private def atLeastXIntProp(x: Int): String = (nextInt(Int.MaxValue - 1) + x).toString + private def atLeastXIntProp(x: Int): String = (nextInt(Int.MaxValue - x) + x).toString private def atLeastOneIntProp: String = atLeastXIntProp(1) @@ -157,9 +157,9 @@ class KafkaConfigConfigDefTest extends JUnit3Suite { case KafkaConfig.QueuedMaxRequestsProp => expected.setProperty(name, atLeastOneIntProp) case KafkaConfig.PortProp => expected.setProperty(name, "1234") - case KafkaConfig.HostNameProp => expected.setProperty(name, nextString(10)) + case KafkaConfig.HostNameProp => expected.setProperty(name, "hostname") case KafkaConfig.ListenersProp => expected.setProperty(name, "PLAINTEXT://:9092") - case KafkaConfig.AdvertisedHostNameProp => expected.setProperty(name, nextString(10)) + case KafkaConfig.AdvertisedHostNameProp => expected.setProperty(name, "advertised.hostname") case KafkaConfig.AdvertisedPortProp => expected.setProperty(name, "4321") case KafkaConfig.AdvertisedListenersProp => expected.setProperty(name, "PLAINTEXT://:2909") case KafkaConfig.SocketRequestMaxBytesProp => expected.setProperty(name, atLeastOneIntProp) @@ -225,7 +225,7 @@ class KafkaConfigConfigDefTest extends JUnit3Suite { case KafkaConfig.ReplicaLagTimeMaxMsProp => expected.setProperty(name, atLeastXIntProp(321)) //require(offsetCommitRequiredAcks >= -1 && offsetCommitRequiredAcks <= offsetsTopicReplicationFactor) case KafkaConfig.OffsetCommitRequiredAcksProp => expected.setProperty(name, "-1") - case KafkaConfig.OffsetsTopicReplicationFactorProp => expected.setProperty(name, inRangeIntProp(-1, Short.MaxValue)) + case KafkaConfig.OffsetsTopicReplicationFactorProp => expected.setProperty(name, inRangeIntProp(1, Short.MaxValue)) //BrokerCompressionCodec.isValid(compressionType) case KafkaConfig.CompressionTypeProp => expected.setProperty(name, randFrom(BrokerCompressionCodec.brokerCompressionOptions)) -- 1.7.12.4 From 31f621716ebace5a05d5cbf65d7d5bf2a902ea7b Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Thu, 16 Jul 2015 17:57:12 -0700 Subject: [PATCH 056/120] kafka-2103; kafka.producer.AsyncProducerTest failure; patched by Ewen Cheslack-Postava; reviewed by Jun Rao --- core/src/test/scala/unit/kafka/utils/TestUtils.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 17e9fe4..eb169d8 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -158,6 +158,7 @@ object TestUtils extends Logging { props.put("controlled.shutdown.enable", enableControlledShutdown.toString) props.put("delete.topic.enable", enableDeleteTopic.toString) props.put("controlled.shutdown.retry.backoff.ms", "100") + props.put("port", port.toString) props } -- 1.7.12.4 From 15cba9f00dc606dd49e428f4ac8ccae0c0b8b37d Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 16 Jul 2015 21:46:46 -0700 Subject: [PATCH 057/120] KAFKA-2324; Update to Scala 2.11.7 Author: Ismael Juma Closes #82 from ijuma/kafka-2324 and squashes the following commits: d71bf5c [Ismael Juma] KAFKA-2324; Update to Scala 2.11.7 --- README.md | 2 +- build.gradle | 18 +++++++++--------- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/README.md b/README.md index a9a5d1e..75877c0 100644 --- a/README.md +++ b/README.md @@ -54,7 +54,7 @@ The release file can be found inside ./core/build/distributions/. ### Cleaning the build ### ./gradlew clean -### Running a task on a particular version of Scala (either 2.9.1, 2.9.2, 2.10.5 or 2.11.6) ### +### Running a task on a particular version of Scala (either 2.9.1, 2.9.2, 2.10.5 or 2.11.7) ### #### (If building a jar with a version other than 2.10, need to set SCALA_BINARY_VERSION variable or change it in bin/kafka-run-class.sh to run quick start.) #### ./gradlew -PscalaVersion=2.9.1 jar ./gradlew -PscalaVersion=2.9.1 test diff --git a/build.gradle b/build.gradle index 9059bc4..fb90843 100644 --- a/build.gradle +++ b/build.gradle @@ -163,7 +163,7 @@ subprojects { } } -for ( sv in ['2_9_1', '2_9_2', '2_10_5', '2_11_6'] ) { +for ( sv in ['2_9_1', '2_9_2', '2_10_5', '2_11_7'] ) { String svInDot = sv.replaceAll( "_", ".") tasks.create(name: "jar_core_${sv}", type: GradleBuild) { @@ -203,20 +203,20 @@ for ( sv in ['2_9_1', '2_9_2', '2_10_5', '2_11_6'] ) { } } -tasks.create(name: "jarAll", dependsOn: ['jar_core_2_9_1', 'jar_core_2_9_2', 'jar_core_2_10_5', 'jar_core_2_11_6', 'clients:jar', 'examples:jar', 'contrib:hadoop-consumer:jar', 'contrib:hadoop-producer:jar', 'log4j-appender:jar']) { +tasks.create(name: "jarAll", dependsOn: ['jar_core_2_9_1', 'jar_core_2_9_2', 'jar_core_2_10_5', 'jar_core_2_11_7', 'clients:jar', 'examples:jar', 'contrib:hadoop-consumer:jar', 'contrib:hadoop-producer:jar', 'log4j-appender:jar']) { } -tasks.create(name: "srcJarAll", dependsOn: ['srcJar_2_9_1', 'srcJar_2_9_2', 'srcJar_2_10_5', 'srcJar_2_11_6', 'clients:srcJar', 'examples:srcJar', 'contrib:hadoop-consumer:srcJar', 'contrib:hadoop-producer:srcJar', 'log4j-appender:srcJar']) { } +tasks.create(name: "srcJarAll", dependsOn: ['srcJar_2_9_1', 'srcJar_2_9_2', 'srcJar_2_10_5', 'srcJar_2_11_7', 'clients:srcJar', 'examples:srcJar', 'contrib:hadoop-consumer:srcJar', 'contrib:hadoop-producer:srcJar', 'log4j-appender:srcJar']) { } -tasks.create(name: "docsJarAll", dependsOn: ['docsJar_2_9_1', 'docsJar_2_9_2', 'docsJar_2_10_5', 'docsJar_2_11_6', 'clients:docsJar', 'examples:docsJar', 'contrib:hadoop-consumer:docsJar', 'contrib:hadoop-producer:docsJar', 'log4j-appender:docsJar']) { } +tasks.create(name: "docsJarAll", dependsOn: ['docsJar_2_9_1', 'docsJar_2_9_2', 'docsJar_2_10_5', 'docsJar_2_11_7', 'clients:docsJar', 'examples:docsJar', 'contrib:hadoop-consumer:docsJar', 'contrib:hadoop-producer:docsJar', 'log4j-appender:docsJar']) { } -tasks.create(name: "testAll", dependsOn: ['test_core_2_9_1', 'test_core_2_9_2', 'test_core_2_10_5', 'test_core_2_11_6', 'clients:test', 'log4j-appender:test']) { +tasks.create(name: "testAll", dependsOn: ['test_core_2_9_1', 'test_core_2_9_2', 'test_core_2_10_5', 'test_core_2_11_7', 'clients:test', 'log4j-appender:test']) { } -tasks.create(name: "releaseTarGzAll", dependsOn: ['releaseTarGz_2_9_1', 'releaseTarGz_2_9_2', 'releaseTarGz_2_10_5', 'releaseTarGz_2_11_6']) { +tasks.create(name: "releaseTarGzAll", dependsOn: ['releaseTarGz_2_9_1', 'releaseTarGz_2_9_2', 'releaseTarGz_2_10_5', 'releaseTarGz_2_11_7']) { } -tasks.create(name: "uploadArchivesAll", dependsOn: ['uploadCoreArchives_2_9_1', 'uploadCoreArchives_2_9_2', 'uploadCoreArchives_2_10_5', 'uploadCoreArchives_2_11_6', 'clients:uploadArchives', 'examples:uploadArchives', 'contrib:hadoop-consumer:uploadArchives', 'contrib:hadoop-producer:uploadArchives', 'log4j-appender:uploadArchives']) { +tasks.create(name: "uploadArchivesAll", dependsOn: ['uploadCoreArchives_2_9_1', 'uploadCoreArchives_2_9_2', 'uploadCoreArchives_2_10_5', 'uploadCoreArchives_2_11_7', 'clients:uploadArchives', 'examples:uploadArchives', 'contrib:hadoop-consumer:uploadArchives', 'contrib:hadoop-producer:uploadArchives', 'log4j-appender:uploadArchives']) { } project(':core') { @@ -234,8 +234,8 @@ project(':core') { compile 'com.yammer.metrics:metrics-core:2.2.0' compile 'net.sf.jopt-simple:jopt-simple:3.2' if (scalaVersion.startsWith('2.11')) { - compile 'org.scala-lang.modules:scala-xml_2.11:1.0.3' - compile 'org.scala-lang.modules:scala-parser-combinators_2.11:1.0.3' + compile 'org.scala-lang.modules:scala-xml_2.11:1.0.4' + compile 'org.scala-lang.modules:scala-parser-combinators_2.11:1.0.4' } testCompile 'junit:junit:4.6' -- 1.7.12.4 From a5b11886df8c7aad0548efd2c7c3dbc579232f03 Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Fri, 17 Jul 2015 10:32:43 -0700 Subject: [PATCH 058/120] KAFKA-2345; Attempt to delete a topic already marked for deletion throws ZkNodeExistsException; patched by Ashish Singh; reviewed by Sriharsha Chintalapani and Ismael Juma --- core/src/main/scala/kafka/admin/AdminUtils.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala index f06edf4..2b4e028 100644 --- a/core/src/main/scala/kafka/admin/AdminUtils.scala +++ b/core/src/main/scala/kafka/admin/AdminUtils.scala @@ -163,7 +163,13 @@ object AdminUtils extends Logging { } def deleteTopic(zkClient: ZkClient, topic: String) { - ZkUtils.createPersistentPath(zkClient, ZkUtils.getDeleteTopicPath(topic)) + try { + ZkUtils.createPersistentPath(zkClient, ZkUtils.getDeleteTopicPath(topic)) + } catch { + case e1: ZkNodeExistsException => throw new TopicAlreadyMarkedForDeletionException( + "topic %s is already marked for deletion".format(topic)) + case e2: Throwable => throw new AdminOperationException(e2.toString) + } } def isConsumerGroupActive(zkClient: ZkClient, group: String) = { -- 1.7.12.4 From 84636272422b6379d57d4c5ef68b156edc1c67f8 Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Fri, 17 Jul 2015 10:49:40 -0700 Subject: [PATCH 059/120] Adding a file missed while committing KAFKA-2345 --- .../TopicAlreadyMarkedForDeletionException.scala | 21 +++++++++++++++++++++ 1 file changed, 21 insertions(+) create mode 100644 core/src/main/scala/kafka/common/TopicAlreadyMarkedForDeletionException.scala diff --git a/core/src/main/scala/kafka/common/TopicAlreadyMarkedForDeletionException.scala b/core/src/main/scala/kafka/common/TopicAlreadyMarkedForDeletionException.scala new file mode 100644 index 0000000..c83cea9 --- /dev/null +++ b/core/src/main/scala/kafka/common/TopicAlreadyMarkedForDeletionException.scala @@ -0,0 +1,21 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.common + +class TopicAlreadyMarkedForDeletionException(message: String) extends RuntimeException(message) { +} \ No newline at end of file -- 1.7.12.4 From 1d2bd6284b06b579c901e6be8919a8a27dbe11ee Mon Sep 17 00:00:00 2001 From: Grant Henke Date: Mon, 20 Jul 2015 16:15:42 -0700 Subject: [PATCH 060/120] KAFKA-2337; Verify that metric names will not collide when creating new topics; patched by Grant Henke; reviewed by Edward Ribeiro, Ashish Singh and Gwen Shapira --- core/src/main/scala/kafka/admin/AdminUtils.scala | 17 +++++++-- core/src/main/scala/kafka/admin/TopicCommand.scala | 4 ++- core/src/main/scala/kafka/common/Topic.scala | 22 ++++++++++++ .../test/scala/unit/kafka/admin/AdminTest.scala | 16 ++++++++- .../test/scala/unit/kafka/common/TopicTest.scala | 41 ++++++++++++++++++++++ 5 files changed, 95 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala index 2b4e028..4cc2376 100644 --- a/core/src/main/scala/kafka/admin/AdminUtils.scala +++ b/core/src/main/scala/kafka/admin/AdminUtils.scala @@ -250,8 +250,19 @@ object AdminUtils extends Logging { require(partitionReplicaAssignment.values.map(_.size).toSet.size == 1, "All partitions should have the same number of replicas.") val topicPath = ZkUtils.getTopicPath(topic) - if(!update && zkClient.exists(topicPath)) - throw new TopicExistsException("Topic \"%s\" already exists.".format(topic)) + + if (!update) { + if (zkClient.exists(topicPath)) + throw new TopicExistsException("Topic \"%s\" already exists.".format(topic)) + else if (Topic.hasCollisionChars(topic)) { + val allTopics = ZkUtils.getAllTopics(zkClient) + val collidingTopics = allTopics.filter(t => Topic.hasCollision(topic, t)) + if (collidingTopics.nonEmpty) { + throw new InvalidTopicException("Topic \"%s\" collides with existing topics: %s".format(topic, collidingTopics.mkString(", "))) + } + } + } + partitionReplicaAssignment.values.foreach(reps => require(reps.size == reps.toSet.size, "Duplicate replica assignment found: " + partitionReplicaAssignment)) // write out the config if there is any, this isn't transactional with the partition assignments @@ -260,7 +271,7 @@ object AdminUtils extends Logging { // create the partition assignment writeTopicPartitionAssignment(zkClient, topic, partitionReplicaAssignment, update) } - + private def writeTopicPartitionAssignment(zkClient: ZkClient, topic: String, replicaAssignment: Map[Int, Seq[Int]], update: Boolean) { try { val zkPath = ZkUtils.getTopicPath(topic) diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala index a90aa87..4e28bf1 100755 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -85,9 +85,11 @@ object TopicCommand extends Logging { def createTopic(zkClient: ZkClient, opts: TopicCommandOptions) { val topic = opts.options.valueOf(opts.topicOpt) val configs = parseTopicConfigsToBeAdded(opts) + if (Topic.hasCollisionChars(topic)) + println("WARNING: Due to limitations in metric names, topics with a period ('.') or underscore ('_') could collide. To avoid issues it is best to use either, but not both.") if (opts.options.has(opts.replicaAssignmentOpt)) { val assignment = parseReplicaAssignment(opts.options.valueOf(opts.replicaAssignmentOpt)) - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, assignment, configs) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, assignment, configs, update = false) } else { CommandLineUtils.checkRequiredArgs(opts.parser, opts.options, opts.partitionsOpt, opts.replicationFactorOpt) val partitions = opts.options.valueOf(opts.partitionsOpt).intValue diff --git a/core/src/main/scala/kafka/common/Topic.scala b/core/src/main/scala/kafka/common/Topic.scala index 32595d6..db75d4b 100644 --- a/core/src/main/scala/kafka/common/Topic.scala +++ b/core/src/main/scala/kafka/common/Topic.scala @@ -43,4 +43,26 @@ object Topic { case None => throw new InvalidTopicException("topic name " + topic + " is illegal, contains a character other than ASCII alphanumerics, '.', '_' and '-'") } } + + /** + * Due to limitations in metric names, topics with a period ('.') or underscore ('_') could collide. + * + * @param topic The topic to check for colliding character + * @return true if the topic has collision characters + */ + def hasCollisionChars(topic: String): Boolean = { + topic.contains("_") || topic.contains(".") + } + + /** + * Returns true if the topicNames collide due to a period ('.') or underscore ('_') in the same position. + * + * @param topicA A topic to check for collision + * @param topicB A topic to check for collision + * @return true if the topics collide + */ + def hasCollision(topicA: String, topicB: String): Boolean = { + topicA.replace('.', '_') == topicB.replace('.', '_') + } + } diff --git a/core/src/test/scala/unit/kafka/admin/AdminTest.scala b/core/src/test/scala/unit/kafka/admin/AdminTest.scala index 252ac81..93f200e 100755 --- a/core/src/test/scala/unit/kafka/admin/AdminTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AdminTest.scala @@ -24,7 +24,7 @@ import kafka.utils._ import kafka.log._ import kafka.zk.ZooKeeperTestHarness import kafka.utils.{Logging, ZkUtils, TestUtils} -import kafka.common.{TopicExistsException, TopicAndPartition} +import kafka.common.{InvalidTopicException, TopicExistsException, TopicAndPartition} import kafka.server.{KafkaServer, KafkaConfig} import java.io.File import TestUtils._ @@ -134,6 +134,20 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { } } + @Test + def testTopicCreationWithCollision() { + val topic = "test.topic" + val collidingTopic = "test_topic" + TestUtils.createBrokersInZk(zkClient, List(0, 1, 2, 3, 4)) + // create the topic + AdminUtils.createTopic(zkClient, topic, 3, 1) + + intercept[InvalidTopicException] { + // shouldn't be able to create a topic that collides + AdminUtils.createTopic(zkClient, collidingTopic, 3, 1) + } + } + private def getBrokersWithPartitionDir(servers: Iterable[KafkaServer], topic: String, partitionId: Int): Set[Int] = { servers.filter(server => new File(server.config.logDirs.head, topic + "-" + partitionId).exists) .map(_.config.brokerId) diff --git a/core/src/test/scala/unit/kafka/common/TopicTest.scala b/core/src/test/scala/unit/kafka/common/TopicTest.scala index 79532c8..17525fe 100644 --- a/core/src/test/scala/unit/kafka/common/TopicTest.scala +++ b/core/src/test/scala/unit/kafka/common/TopicTest.scala @@ -57,4 +57,45 @@ class TopicTest { } } } + + @Test + def testTopicHasCollisionChars() = { + val falseTopics = List("start", "end", "middle", "many") + val trueTopics = List( + ".start", "end.", "mid.dle", ".ma.ny.", + "_start", "end_", "mid_dle", "_ma_ny." + ) + + falseTopics.foreach( t => + assertFalse(Topic.hasCollisionChars(t)) + ) + + trueTopics.foreach( t => + assertTrue(Topic.hasCollisionChars(t)) + ) + } + + @Test + def testTopicHasCollision() = { + val periodFirstMiddleLastNone = List(".topic", "to.pic", "topic.", "topic") + val underscoreFirstMiddleLastNone = List("_topic", "to_pic", "topic_", "topic") + + // Self + periodFirstMiddleLastNone.foreach { t => + assertTrue(Topic.hasCollision(t, t)) + } + underscoreFirstMiddleLastNone.foreach { t => + assertTrue(Topic.hasCollision(t, t)) + } + + // Same Position + periodFirstMiddleLastNone.zip(underscoreFirstMiddleLastNone).foreach { case (t1, t2) => + assertTrue(Topic.hasCollision(t1, t2)) + } + + // Different Position + periodFirstMiddleLastNone.zip(underscoreFirstMiddleLastNone.reverse).foreach { case (t1, t2) => + assertFalse(Topic.hasCollision(t1, t2)) + } + } } -- 1.7.12.4 From 2040890462a22614ccb836e43837f8739855b84f Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Mon, 20 Jul 2015 16:40:54 -0700 Subject: [PATCH 061/120] KAFKA-2328; merge-kafka-pr.py script should not leave user in a detached branch The right command to get the branch name is `git rev-parse --abbrev-ref HEAD` instead of `git rev-parse HEAD`. The latter gives the commit hash causing a detached branch when we checkout to it. Seems like a bug we inherited from the Spark script. Author: Ismael Juma Closes #84 from ijuma/kafka-2328-merge-script-no-detached-branch and squashes the following commits: ae201dd [Ismael Juma] KAFKA-2328; merge-kafka-pr.py script should not leave user in a detached branch --- kafka-merge-pr.py | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/kafka-merge-pr.py b/kafka-merge-pr.py index 4cc82e0..c6ef3df 100644 --- a/kafka-merge-pr.py +++ b/kafka-merge-pr.py @@ -95,8 +95,9 @@ def continue_maybe(prompt): fail("Okay, exiting") def clean_up(): - print "Restoring head pointer to %s" % original_head - run_cmd("git checkout %s" % original_head) + if original_head != get_current_branch(): + print "Restoring head pointer to %s" % original_head + run_cmd("git checkout %s" % original_head) branches = run_cmd("git branch").replace(" ", "").split("\n") @@ -104,6 +105,8 @@ def clean_up(): print "Deleting local branch %s" % branch run_cmd("git branch -D %s" % branch) +def get_current_branch(): + return run_cmd("git rev-parse --abbrev-ref HEAD").replace("\n", "") # merge the requested PR and return the merge hash def merge_pr(pr_num, target_ref, title, body, pr_repo_desc): @@ -350,7 +353,7 @@ def standardize_jira_ref(text): def main(): global original_head - original_head = run_cmd("git rev-parse HEAD")[:8] + original_head = get_current_branch() branches = get_json("%s/branches" % GITHUB_API_BASE) branch_names = filter(lambda x: x.startswith(RELEASE_BRANCH_PREFIX), [x['name'] for x in branches]) -- 1.7.12.4 From fd3b4cc41e9249ec6848cde8137691d32b2e79e5 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Wed, 22 Jul 2015 13:00:03 -0700 Subject: [PATCH 062/120] KAFKA-2342; KafkaConsumer rebalance with in-flight fetch can cause invalid position Author: Jason Gustafson Closes #88 from hachikuji/KAFKA-2342 and squashes the following commits: cabb017 [Jason Gustafson] KAFKA-2342; KafkaConsumer rebalance with in-flight fetch can cause invalid position --- .../kafka/clients/consumer/internals/Fetcher.java | 16 +++++++++++++++- .../clients/consumer/internals/FetcherTest.java | 21 +++++++++++++++++++-- .../integration/kafka/api/ConsumerBounceTest.scala | 2 +- 3 files changed, 35 insertions(+), 4 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index d595c1c..d2a0e2b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -219,7 +219,7 @@ public class Fetcher { for (PartitionRecords part : this.records) { Long consumed = subscriptions.consumed(part.partition); if (this.subscriptions.assignedPartitions().contains(part.partition) - && (consumed == null || part.fetchOffset == consumed)) { + && consumed != null && part.fetchOffset == consumed) { List> records = drained.get(part.partition); if (records == null) { records = part.records; @@ -364,6 +364,20 @@ public class Fetcher { parsed.add(parseRecord(tp, logEntry)); bytes += logEntry.size(); } + + // we are interested in this fetch only if the beginning offset matches the + // current consumed position + Long consumed = subscriptions.consumed(tp); + if (consumed == null) { + continue; + } else if (consumed != fetchOffset) { + // the fetched position has gotten out of sync with the consumed position + // (which might happen when a rebalance occurs with a fetch in-flight), + // so we need to reset the fetch position so the next fetch is right + subscriptions.fetched(tp, consumed); + continue; + } + if (parsed.size() > 0) { ConsumerRecord record = parsed.get(parsed.size() - 1); this.subscriptions.fetched(tp, record.offset() + 1); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java index 7a4e586..4002679 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -36,6 +36,7 @@ import org.junit.Before; import org.junit.Test; import java.nio.ByteBuffer; +import java.util.Arrays; import java.util.Collections; import java.util.LinkedHashMap; import java.util.List; @@ -115,8 +116,25 @@ public class FetcherTest { } @Test + public void testFetchDuringRebalance() { + subscriptions.subscribe(topicName); + subscriptions.changePartitionAssignment(Arrays.asList(tp)); + subscriptions.fetched(tp, 0); + subscriptions.consumed(tp, 0); + + fetcher.initFetches(cluster); + + // Now the rebalance happens and fetch positions are cleared + subscriptions.changePartitionAssignment(Arrays.asList(tp)); + client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L)); + consumerClient.poll(0); + + // The active fetch should be ignored since its position is no longer valid + assertTrue(fetcher.fetchedRecords().isEmpty()); + } + + @Test public void testFetchFailed() { - List> records; subscriptions.subscribe(tp); subscriptions.fetched(tp, 0); subscriptions.consumed(tp, 0); @@ -148,7 +166,6 @@ public class FetcherTest { @Test public void testFetchOutOfRange() { - List> records; subscriptions.subscribe(tp); subscriptions.fetched(tp, 5); subscriptions.consumed(tp, 5); diff --git a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala index b0750fa..d8eee52 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala @@ -58,7 +58,7 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging { TestUtils.createTopic(this.zkClient, topic, 1, serverCount, this.servers) } - def testConsumptionWithBrokerFailures() = consumeWithBrokerFailures(20) + def testConsumptionWithBrokerFailures() = consumeWithBrokerFailures(10) /* * 1. Produce a bunch of messages -- 1.7.12.4 From 18adec7ed5ad71eb73171aff0e0c82d627cfa5d5 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 23 Jul 2015 08:46:47 -0700 Subject: [PATCH 063/120] KAFKA-2344; kafka-merge-pr improvements The first 4 commits are adapted from changes that have been done to the Spark version and the last one is the feature that gwenshap asked for. Author: Ismael Juma Reviewers: Gwen Shapira Closes #90 from ijuma/kafka-2344-merge-pr-improvements and squashes the following commits: 900c371 [Ismael Juma] Allow reviewers to be entered during merge ac06347 [Ismael Juma] Allow primary author to be overridden during merge b309829 [Ismael Juma] Set JIRA resolution to "Fixed" instead of relying on default transition 0c69a64 [Ismael Juma] Check return value of doctest.testmod() 061cdce [Ismael Juma] Fix instructions on how to install the `jira-python` library --- kafka-merge-pr.py | 24 ++++++++++++++++++++---- 1 file changed, 20 insertions(+), 4 deletions(-) diff --git a/kafka-merge-pr.py b/kafka-merge-pr.py index c6ef3df..876f530 100644 --- a/kafka-merge-pr.py +++ b/kafka-merge-pr.py @@ -130,7 +130,15 @@ def merge_pr(pr_num, target_ref, title, body, pr_repo_desc): '--pretty=format:%an <%ae>']).split("\n") distinct_authors = sorted(set(commit_authors), key=lambda x: commit_authors.count(x), reverse=True) - primary_author = distinct_authors[0] + primary_author = raw_input( + "Enter primary author in the format of \"name \" [%s]: " % + distinct_authors[0]) + if primary_author == "": + primary_author = distinct_authors[0] + + reviewers = raw_input( + "Enter reviewers in the format of \"name1 , name2 \": ").strip() + commits = run_cmd(['git', 'log', 'HEAD..%s' % pr_branch_name, '--pretty=format:%h [%an] %s']).split("\n\n") @@ -146,6 +154,9 @@ def merge_pr(pr_num, target_ref, title, body, pr_repo_desc): merge_message_flags += ["-m", authors] + if (reviewers != ""): + merge_message_flags += ["-m", "Reviewers: %s" % reviewers] + if had_conflicts: committer_name = run_cmd("git config --get user.name").strip() committer_email = run_cmd("git config --get user.email").strip() @@ -278,7 +289,10 @@ def resolve_jira_issue(merge_branches, comment, default_jira_id=""): jira_fix_versions = map(lambda v: get_version_json(v), fix_versions) resolve = filter(lambda a: a['name'] == "Resolve Issue", asf_jira.transitions(jira_id))[0] - asf_jira.transition_issue(jira_id, resolve["id"], fixVersions=jira_fix_versions, comment=comment) + resolution = filter(lambda r: r.raw['name'] == "Fixed", asf_jira.resolutions())[0] + asf_jira.transition_issue( + jira_id, resolve["id"], fixVersions = jira_fix_versions, + comment = comment, resolution = {'id': resolution.raw['id']}) print "Successfully resolved %s with fixVersions=%s!" % (jira_id, fix_versions) @@ -435,11 +449,13 @@ def main(): print "JIRA_USERNAME and JIRA_PASSWORD not set" print "Exiting without trying to close the associated JIRA." else: - print "Could not find jira-python library. Run 'sudo pip install jira-python' to install." + print "Could not find jira-python library. Run 'sudo pip install jira' to install." print "Exiting without trying to close the associated JIRA." if __name__ == "__main__": import doctest - doctest.testmod() + (failure_count, test_count) = doctest.testmod() + if (failure_count): + exit(-1) main() -- 1.7.12.4 From 66c8647d85f801893b9f882a7ad49242cef683fd Mon Sep 17 00:00:00 2001 From: Edward Ribeiro Date: Thu, 23 Jul 2015 09:17:19 -0700 Subject: [PATCH 064/120] KAFKA-2355; Add an unit test to validate the deletion of a partition marked as deleted; patched by Edward Ribeiro, reviewed by Ashish Singh, Ismael Juma and Grant Henke --- .../scala/unit/kafka/admin/DeleteTopicTest.scala | 23 +++++++++++++++++++++- 1 file changed, 22 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala index fa8ce25..c38df80 100644 --- a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala +++ b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala @@ -24,7 +24,7 @@ import kafka.utils.{ZkUtils, TestUtils} import kafka.server.{KafkaServer, KafkaConfig} import org.junit.Test import java.util.Properties -import kafka.common.TopicAndPartition +import kafka.common.{TopicAlreadyMarkedForDeletionException, TopicAndPartition} class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { @@ -249,6 +249,27 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { servers.foreach(_.shutdown()) } + @Test + def testDeleteTopicAlreadyMarkedAsDeleted() { + val topicAndPartition = TopicAndPartition("test", 0) + val topic = topicAndPartition.topic + val servers = createTestTopicAndCluster(topic) + + try { + // start topic deletion + AdminUtils.deleteTopic(zkClient, topic) + // try to delete topic marked as deleted + AdminUtils.deleteTopic(zkClient, topic) + fail("Expected TopicAlreadyMarkedForDeletionException") + } + catch { + case e: TopicAlreadyMarkedForDeletionException => // expected exception + } + + TestUtils.verifyTopicDeletion(zkClient, topic, 1, servers) + servers.foreach(_.shutdown()) + } + private def createTestTopicAndCluster(topic: String): Seq[KafkaServer] = { val brokerConfigs = TestUtils.createBrokerConfigs(3, zkConnect, false) -- 1.7.12.4 From 520a8135f4820ee92e63427c5dbb2d5356e6a473 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 23 Jul 2015 17:19:16 -0700 Subject: [PATCH 065/120] KAFKA-2353; SocketServer catch exception and close connection properly; reviewed by Gwen Shapira, Ismael Juma and Guozhang Wang --- .../main/scala/kafka/network/SocketServer.scala | 104 ++++++++++++--------- 1 file changed, 60 insertions(+), 44 deletions(-) diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala index 91319fa..dbe784b 100644 --- a/core/src/main/scala/kafka/network/SocketServer.scala +++ b/core/src/main/scala/kafka/network/SocketServer.scala @@ -37,6 +37,7 @@ import org.apache.kafka.common.protocol.types.SchemaException import org.apache.kafka.common.utils.{SystemTime, Time, Utils} import scala.collection._ +import scala.util.control.{NonFatal, ControlThrowable} /** * An NIO socket server. The threading model is @@ -357,49 +358,57 @@ private[kafka] class Processor(val id: Int, override def run() { startupComplete() while(isRunning) { - // setup any new connections that have been queued up - configureNewConnections() - // register any new responses for writing - processNewResponses() - try { - selector.poll(300) - } catch { - case e @ (_: IllegalStateException | _: IOException) => { - error("Closing processor %s due to illegal state or IO exception".format(id)) - swallow(closeAll()) - shutdownComplete() - throw e - } - case e: InvalidReceiveException => - // Log warning and continue since Selector already closed the connection - warn("Connection was closed due to invalid receive. Processor will continue handling other connections") - } - collection.JavaConversions.collectionAsScalaIterable(selector.completedReceives).foreach( receive => { + // setup any new connections that have been queued up + configureNewConnections() + // register any new responses for writing + processNewResponses() + try { - val req = RequestChannel.Request(processor = id, connectionId = receive.source, buffer = receive.payload, startTimeMs = time.milliseconds, securityProtocol = SecurityProtocol.PLAINTEXT) - requestChannel.sendRequest(req) + selector.poll(300) } catch { - case e @ (_: InvalidRequestException | _: SchemaException) => { - // note that even though we got an exception, we can assume that receive.source is valid. Issues with constructing a valid receive object were handled earlier - error("Closing socket for " + receive.source + " because of error", e) - selector.close(receive.source) + case e @ (_: IllegalStateException | _: IOException) => { + error("Closing processor %s due to illegal state or IO exception".format(id)) + swallow(closeAll()) + shutdownComplete() + throw e } + case e: InvalidReceiveException => + // Log warning and continue since Selector already closed the connection + warn("Connection was closed due to invalid receive. Processor will continue handling other connections") } - selector.mute(receive.source) - }) - - collection.JavaConversions.iterableAsScalaIterable(selector.completedSends()).foreach( send => { - val resp = inflightResponses.remove(send.destination()).get - resp.request.updateRequestMetrics() - selector.unmute(send.destination()) - }) + collection.JavaConversions.collectionAsScalaIterable(selector.completedReceives).foreach(receive => { + try { + val req = RequestChannel.Request(processor = id, connectionId = receive.source, buffer = receive.payload, startTimeMs = time.milliseconds, securityProtocol = SecurityProtocol.PLAINTEXT) + requestChannel.sendRequest(req) + } catch { + case e @ (_: InvalidRequestException | _: SchemaException) => { + // note that even though we got an exception, we can assume that receive.source is valid. Issues with constructing a valid receive object were handled earlier + error("Closing socket for " + receive.source + " because of error", e) + selector.close(receive.source) + } + } + selector.mute(receive.source) + }) + + collection.JavaConversions.iterableAsScalaIterable(selector.completedSends()).foreach(send => { + val resp = inflightResponses.remove(send.destination()).get + resp.request.updateRequestMetrics() + selector.unmute(send.destination()) + }) + } catch { + // We catch all the throwables here to prevent the processor thread from exiting. We do this because + // letting a processor exit might cause bigger impact on the broker. Usually the exceptions thrown would + // be either associated with a specific socket channel or a bad request. We just ignore the bad socket channel + // or request. This behavior might need to be reviewed if we see an exception that need the entire broker to stop. + case e : ControlThrowable => throw e + case e : Throwable => + error("Processor got uncaught exception.", e) + } } - - debug("Closing selector - processor " + id) - closeAll() + swallowError(closeAll()) shutdownComplete() } @@ -426,8 +435,6 @@ private[kafka] class Processor(val id: Int, selector.close(curr.request.connectionId) } } - - } finally { curr = requestChannel.receiveResponse(id) } @@ -448,13 +455,22 @@ private[kafka] class Processor(val id: Int, private def configureNewConnections() { while(!newConnections.isEmpty) { val channel = newConnections.poll() - debug("Processor " + id + " listening to new connection from " + channel.socket.getRemoteSocketAddress) - val localHost = channel.socket().getLocalAddress.getHostAddress - val localPort = channel.socket().getLocalPort - val remoteHost = channel.socket().getInetAddress.getHostAddress - val remotePort = channel.socket().getPort - val connectionId = localHost + ":" + localPort + "-" + remoteHost + ":" + remotePort - selector.register(connectionId, channel) + try { + debug("Processor " + id + " listening to new connection from " + channel.socket.getRemoteSocketAddress) + val localHost = channel.socket().getLocalAddress.getHostAddress + val localPort = channel.socket().getLocalPort + val remoteHost = channel.socket().getInetAddress.getHostAddress + val remotePort = channel.socket().getPort + val connectionId = localHost + ":" + localPort + "-" + remoteHost + ":" + remotePort + selector.register(connectionId, channel) + } catch { + // We explicitly catch all non fatal exceptions and close the socket to avoid socket leak. The other + // throwables will be caught in processor and logged as uncaught exception. + case NonFatal(e) => + // need to close the channel here to avoid socket leak. + close(channel) + error("Processor " + id + " closed connection from " + channel.getRemoteAddress, e) + } } } -- 1.7.12.4 From fd3a3cd59539a741b6d3c830870dae1d5a5f93b5 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Fri, 24 Jul 2015 09:19:59 -0700 Subject: [PATCH 066/120] KAFKA-2348; Drop support for Scala 2.9 `testAll` passed locally. Author: Ismael Juma Reviewers: Harsha, Ewen, Guozhang Closes #87 from ijuma/kafka-2348-drop-support-for-scala-2.9 and squashes the following commits: cf9796a [Ismael Juma] KAFKA-2348; Drop support for Scala 2.9 --- README.md | 8 ++++---- build.gradle | 19 ++++++++----------- 2 files changed, 12 insertions(+), 15 deletions(-) diff --git a/README.md b/README.md index 75877c0..a4a3f78 100644 --- a/README.md +++ b/README.md @@ -54,11 +54,11 @@ The release file can be found inside ./core/build/distributions/. ### Cleaning the build ### ./gradlew clean -### Running a task on a particular version of Scala (either 2.9.1, 2.9.2, 2.10.5 or 2.11.7) ### +### Running a task on a particular version of Scala (either 2.10.5 or 2.11.7) ### #### (If building a jar with a version other than 2.10, need to set SCALA_BINARY_VERSION variable or change it in bin/kafka-run-class.sh to run quick start.) #### - ./gradlew -PscalaVersion=2.9.1 jar - ./gradlew -PscalaVersion=2.9.1 test - ./gradlew -PscalaVersion=2.9.1 releaseTarGz + ./gradlew -PscalaVersion=2.11.7 jar + ./gradlew -PscalaVersion=2.11.7 test + ./gradlew -PscalaVersion=2.11.7 releaseTarGz ### Running a task for a specific project ### This is for 'core', 'contrib:hadoop-consumer', 'contrib:hadoop-producer', 'examples' and 'clients' diff --git a/build.gradle b/build.gradle index fb90843..0abec26 100644 --- a/build.gradle +++ b/build.gradle @@ -163,7 +163,7 @@ subprojects { } } -for ( sv in ['2_9_1', '2_9_2', '2_10_5', '2_11_7'] ) { +for ( sv in ['2_10_5', '2_11_7'] ) { String svInDot = sv.replaceAll( "_", ".") tasks.create(name: "jar_core_${sv}", type: GradleBuild) { @@ -203,20 +203,20 @@ for ( sv in ['2_9_1', '2_9_2', '2_10_5', '2_11_7'] ) { } } -tasks.create(name: "jarAll", dependsOn: ['jar_core_2_9_1', 'jar_core_2_9_2', 'jar_core_2_10_5', 'jar_core_2_11_7', 'clients:jar', 'examples:jar', 'contrib:hadoop-consumer:jar', 'contrib:hadoop-producer:jar', 'log4j-appender:jar']) { +tasks.create(name: "jarAll", dependsOn: ['jar_core_2_10_5', 'jar_core_2_11_7', 'clients:jar', 'examples:jar', 'contrib:hadoop-consumer:jar', 'contrib:hadoop-producer:jar', 'log4j-appender:jar']) { } -tasks.create(name: "srcJarAll", dependsOn: ['srcJar_2_9_1', 'srcJar_2_9_2', 'srcJar_2_10_5', 'srcJar_2_11_7', 'clients:srcJar', 'examples:srcJar', 'contrib:hadoop-consumer:srcJar', 'contrib:hadoop-producer:srcJar', 'log4j-appender:srcJar']) { } +tasks.create(name: "srcJarAll", dependsOn: ['srcJar_2_10_5', 'srcJar_2_11_7', 'clients:srcJar', 'examples:srcJar', 'contrib:hadoop-consumer:srcJar', 'contrib:hadoop-producer:srcJar', 'log4j-appender:srcJar']) { } -tasks.create(name: "docsJarAll", dependsOn: ['docsJar_2_9_1', 'docsJar_2_9_2', 'docsJar_2_10_5', 'docsJar_2_11_7', 'clients:docsJar', 'examples:docsJar', 'contrib:hadoop-consumer:docsJar', 'contrib:hadoop-producer:docsJar', 'log4j-appender:docsJar']) { } +tasks.create(name: "docsJarAll", dependsOn: ['docsJar_2_10_5', 'docsJar_2_11_7', 'clients:docsJar', 'examples:docsJar', 'contrib:hadoop-consumer:docsJar', 'contrib:hadoop-producer:docsJar', 'log4j-appender:docsJar']) { } -tasks.create(name: "testAll", dependsOn: ['test_core_2_9_1', 'test_core_2_9_2', 'test_core_2_10_5', 'test_core_2_11_7', 'clients:test', 'log4j-appender:test']) { +tasks.create(name: "testAll", dependsOn: ['test_core_2_10_5', 'test_core_2_11_7', 'clients:test', 'log4j-appender:test']) { } -tasks.create(name: "releaseTarGzAll", dependsOn: ['releaseTarGz_2_9_1', 'releaseTarGz_2_9_2', 'releaseTarGz_2_10_5', 'releaseTarGz_2_11_7']) { +tasks.create(name: "releaseTarGzAll", dependsOn: ['releaseTarGz_2_10_5', 'releaseTarGz_2_11_7']) { } -tasks.create(name: "uploadArchivesAll", dependsOn: ['uploadCoreArchives_2_9_1', 'uploadCoreArchives_2_9_2', 'uploadCoreArchives_2_10_5', 'uploadCoreArchives_2_11_7', 'clients:uploadArchives', 'examples:uploadArchives', 'contrib:hadoop-consumer:uploadArchives', 'contrib:hadoop-producer:uploadArchives', 'log4j-appender:uploadArchives']) { +tasks.create(name: "uploadArchivesAll", dependsOn: ['uploadCoreArchives_2_10_5', 'uploadCoreArchives_2_11_7', 'clients:uploadArchives', 'examples:uploadArchives', 'contrib:hadoop-consumer:uploadArchives', 'contrib:hadoop-producer:uploadArchives', 'log4j-appender:uploadArchives']) { } project(':core') { @@ -241,10 +241,7 @@ project(':core') { testCompile 'junit:junit:4.6' testCompile 'org.easymock:easymock:3.0' testCompile 'org.objenesis:objenesis:1.2' - if (scalaVersion.startsWith('2.9')) - testCompile "org.scalatest:scalatest_$scalaVersion:1.9.1" - else - testCompile "org.scalatest:scalatest_$baseScalaVersion:2.2.5" + testCompile "org.scalatest:scalatest_$baseScalaVersion:2.2.5" testRuntime "$slf4jlog4j" -- 1.7.12.4 From 4389a11f68f4c0e028441acd0ef30951296c38ce Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Mon, 27 Jul 2015 10:54:23 -0700 Subject: [PATCH 067/120] KAFKA-2321; Introduce CONTRIBUTING.md Author: Ismael Juma Reviewers: Guozhang Closes #97 from ijuma/kafka-2321 and squashes the following commits: 4834464 [Ismael Juma] KAFKA-2321; Introduce CONTRIBUTING.md --- CONTRIBUTING.md | 11 +++++++++++ build.gradle | 1 + 2 files changed, 12 insertions(+) create mode 100644 CONTRIBUTING.md diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md new file mode 100644 index 0000000..e9a4d25 --- /dev/null +++ b/CONTRIBUTING.md @@ -0,0 +1,11 @@ +## Contributing to Kafka + +*Before opening a pull request*, review the [Contributing](http://kafka.apache.org/contributing.html) and [Contributing Code Changes](https://cwiki.apache.org/confluence/display/KAFKA/Contributing+Code+Changes) pages. + +It lists steps that are required before creating a PR. + +When you contribute code, you affirm that the contribution is your original work and that you +license the work to the project under the project's open source license. Whether or not you +state this explicitly, by submitting any copyrighted material via pull request, email, or +other means you agree to license the material under the project's open source license and +warrant that you have the legal authority to do so. diff --git a/build.gradle b/build.gradle index 0abec26..9b6eb51 100644 --- a/build.gradle +++ b/build.gradle @@ -60,6 +60,7 @@ rat { excludes.addAll([ '**/.git/**', 'build/**', + 'CONTRIBUTING.md', 'gradlew', 'gradlew.bat', '**/README.md', -- 1.7.12.4 From 57386de64530edcb54e9dae4566fb6d4a8b4700e Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Tue, 28 Jul 2015 14:09:10 -0700 Subject: [PATCH 068/120] KAFKA-2301: Warn ConsumerOffsetChecker as deprecated; reviewed by Ewen Cheslack-Postava and Guozhang Wang --- core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala b/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala index 3d52f62..c39fbfe 100644 --- a/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala +++ b/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala @@ -107,6 +107,8 @@ object ConsumerOffsetChecker extends Logging { } def main(args: Array[String]) { + warn("WARNING: ConsumerOffsetChecker is deprecated and will be dropped in releases following 0.9.0. Use ConsumerGroupCommand instead.") + val parser = new OptionParser() val zkConnectOpt = parser.accepts("zookeeper", "ZooKeeper connect string."). -- 1.7.12.4 From 269c2407d4e6e38b3b6be00566c480121b5dc51a Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Tue, 28 Jul 2015 14:23:44 -0700 Subject: [PATCH 069/120] KAFKA-2381: Fix concurrent modification on assigned partition while looping over it; reviewed by Jason Gustafson, Aditya Auradkar, Ewen Cheslack-Postava, Ismael Juma and Guozhang Wang --- .../consumer/internals/SubscriptionState.java | 4 +++- .../consumer/internals/SubscriptionStateTest.java | 21 +++++++++++++++++++++ .../scala/integration/kafka/api/ConsumerTest.scala | 19 +++++++++++++++++++ 3 files changed, 43 insertions(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java index 4d9a425..8a2cb12 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java @@ -15,6 +15,7 @@ package org.apache.kafka.clients.consumer.internals; import org.apache.kafka.clients.consumer.OffsetResetStrategy; import org.apache.kafka.common.TopicPartition; +import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -83,7 +84,8 @@ public class SubscriptionState { throw new IllegalStateException("Topic " + topic + " was never subscribed to."); this.subscribedTopics.remove(topic); this.needsPartitionAssignment = true; - for (TopicPartition tp: assignedPartitions()) + final List existingAssignedPartitions = new ArrayList<>(assignedPartitions()); + for (TopicPartition tp: existingAssignedPartitions) if (topic.equals(tp.topic())) clearPartition(tp); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java index 319751c..c47f3fb 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java @@ -73,6 +73,27 @@ public class SubscriptionStateTest { assertAllPositions(tp0, null); assertEquals(Collections.singleton(tp1), state.assignedPartitions()); } + + @Test + public void topicUnsubscription() { + final String topic = "test"; + state.subscribe(topic); + assertEquals(1, state.subscribedTopics().size()); + assertTrue(state.assignedPartitions().isEmpty()); + assertTrue(state.partitionsAutoAssigned()); + state.changePartitionAssignment(asList(tp0)); + state.committed(tp0, 1); + state.fetched(tp0, 1); + state.consumed(tp0, 1); + assertAllPositions(tp0, 1L); + state.changePartitionAssignment(asList(tp1)); + assertAllPositions(tp0, null); + assertEquals(Collections.singleton(tp1), state.assignedPartitions()); + + state.unsubscribe(topic); + assertEquals(0, state.subscribedTopics().size()); + assertTrue(state.assignedPartitions().isEmpty()); + } @Test(expected = IllegalArgumentException.class) public void cantChangeFetchPositionForNonAssignedPartition() { diff --git a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala index 3eb5f95..cca6e94 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala @@ -217,6 +217,25 @@ class ConsumerTest extends IntegrationTestHarness with Logging { consumer0.close() } + def testUnsubscribeTopic() { + val callback = new TestConsumerReassignmentCallback() + this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "100"); // timeout quickly to avoid slow test + val consumer0 = new KafkaConsumer(this.consumerConfig, callback, new ByteArrayDeserializer(), new ByteArrayDeserializer()) + + try { + consumer0.subscribe(topic) + + // the initial subscription should cause a callback execution + while (callback.callsToAssigned == 0) + consumer0.poll(50) + + consumer0.unsubscribe(topic) + assertEquals(0, consumer0.subscriptions.size()) + } finally { + consumer0.close() + } + } + private class TestConsumerReassignmentCallback extends ConsumerRebalanceCallback { var callsToAssigned = 0 var callsToRevoked = 0 -- 1.7.12.4 From 3df46bf4ce9c134cc7b532be3d01f920127be706 Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Tue, 28 Jul 2015 14:27:35 -0700 Subject: [PATCH 070/120] KAFKA-2347: Add setConsumerRebalanceListener method to ZookeeperConsumerConnector java api; reviewed by Jiangjie Qin, Ismael Juma, Grant Henke and Guozhang Wang --- core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java b/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java index ca74ca8..444cd1d 100644 --- a/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java +++ b/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java @@ -76,6 +76,12 @@ public interface ConsumerConnector { public void commitOffsets(Map offsetsToCommit, boolean retryOnFailure); /** + * Wire in a consumer rebalance listener to be executed when consumer rebalance occurs. + * @param listener The consumer rebalance listener to wire in + */ + public void setConsumerRebalanceListener(ConsumerRebalanceListener listener); + + /** * Shut down the connector */ public void shutdown(); -- 1.7.12.4 From 594b963930c2054199ed54203415d1cb7917df27 Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Tue, 28 Jul 2015 15:49:22 -0700 Subject: [PATCH 071/120] KAFKA-2275: Add ListTopics() API to the Java consumer; reviewed by Jason Gustafson, Edward Ribeiro and Guozhang Wang --- .../org/apache/kafka/clients/ClientRequest.java | 29 ++++++++++++-- .../org/apache/kafka/clients/NetworkClient.java | 4 +- .../apache/kafka/clients/consumer/Consumer.java | 5 +++ .../kafka/clients/consumer/KafkaConsumer.java | 16 ++++++++ .../kafka/clients/consumer/MockConsumer.java | 6 +++ .../kafka/clients/consumer/internals/Fetcher.java | 44 ++++++++++++++++++++++ .../clients/consumer/internals/FetcherTest.java | 13 +++++++ .../scala/integration/kafka/api/ConsumerTest.scala | 18 +++++++++ 8 files changed, 130 insertions(+), 5 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java b/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java index ed4c0d9..dc8f0f1 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java @@ -23,6 +23,7 @@ public final class ClientRequest { private final boolean expectResponse; private final RequestSend request; private final RequestCompletionHandler callback; + private final boolean isInitiatedByNetworkClient; /** * @param createdMs The unix timestamp in milliseconds for the time at which this request was created. @@ -30,17 +31,35 @@ public final class ClientRequest { * @param request The request * @param callback A callback to execute when the response has been received (or null if no callback is necessary) */ - public ClientRequest(long createdMs, boolean expectResponse, RequestSend request, RequestCompletionHandler callback) { + public ClientRequest(long createdMs, boolean expectResponse, RequestSend request, + RequestCompletionHandler callback) { + this(createdMs, expectResponse, request, callback, false); + } + + /** + * @param createdMs The unix timestamp in milliseconds for the time at which this request was created. + * @param expectResponse Should we expect a response message or is this request complete once it is sent? + * @param request The request + * @param callback A callback to execute when the response has been received (or null if no callback is necessary) + * @param isInitiatedByNetworkClient Is request initiated by network client, if yes, its + * response will be consumed by network client + */ + public ClientRequest(long createdMs, boolean expectResponse, RequestSend request, + RequestCompletionHandler callback, boolean isInitiatedByNetworkClient) { this.createdMs = createdMs; this.callback = callback; this.request = request; this.expectResponse = expectResponse; + this.isInitiatedByNetworkClient = isInitiatedByNetworkClient; } @Override public String toString() { - return "ClientRequest(expectResponse=" + expectResponse + ", callback=" + callback + ", request=" + request - + ")"; + return "ClientRequest(expectResponse=" + expectResponse + + ", callback=" + callback + + ", request=" + request + + (isInitiatedByNetworkClient ? ", isInitiatedByNetworkClient" : "") + + ")"; } public boolean expectResponse() { @@ -63,4 +82,8 @@ public final class ClientRequest { return createdMs; } + public boolean isInitiatedByNetworkClient() { + return isInitiatedByNetworkClient; + } + } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index 48fe796..0e51d7b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -378,7 +378,7 @@ public class NetworkClient implements KafkaClient { short apiKey = req.request().header().apiKey(); Struct body = (Struct) ProtoUtils.currentResponseSchema(apiKey).read(receive.payload()); correlate(req.request().header(), header); - if (apiKey == ApiKeys.METADATA.id) { + if (apiKey == ApiKeys.METADATA.id && req.isInitiatedByNetworkClient()) { handleMetadataResponse(req.request().header(), body, now); } else { // need to add body/header to response here @@ -454,7 +454,7 @@ public class NetworkClient implements KafkaClient { private ClientRequest metadataRequest(long now, String node, Set topics) { MetadataRequest metadata = new MetadataRequest(new ArrayList(topics)); RequestSend send = new RequestSend(node, nextRequestHeader(ApiKeys.METADATA), metadata.toStruct()); - return new ClientRequest(now, true, send, null); + return new ClientRequest(now, true, send, null, true); } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java index 252b759..23e410b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java @@ -114,6 +114,11 @@ public interface Consumer extends Closeable { public List partitionsFor(String topic); /** + * @see KafkaConsumer#listTopics() + */ + public Map> listTopics(); + + /** * @see KafkaConsumer#close() */ public void close(); 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 bea3d73..923ff99 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 @@ -1024,6 +1024,22 @@ public class KafkaConsumer implements Consumer { } } + /** + * Get metadata about partitions for all topics. This method will issue a remote call to the + * server. + * + * @return The map of topics and its partitions + */ + @Override + public Map> listTopics() { + acquire(); + try { + return fetcher.getAllTopics(requestTimeoutMs); + } finally { + release(); + } + } + @Override public void close() { acquire(); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java index c14eed1..5b22fa0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java @@ -177,6 +177,12 @@ public class MockConsumer implements Consumer { return parts; } + @Override + public Map> listTopics() { + ensureNotClosed(); + return partitions; + } + public synchronized void updatePartitions(String topic, List partitions) { ensureNotClosed(); this.partitions.put(topic, partitions); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index d2a0e2b..9f71451 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -39,6 +39,8 @@ import org.apache.kafka.common.requests.FetchRequest; import org.apache.kafka.common.requests.FetchResponse; import org.apache.kafka.common.requests.ListOffsetRequest; import org.apache.kafka.common.requests.ListOffsetResponse; +import org.apache.kafka.common.requests.MetadataRequest; +import org.apache.kafka.common.requests.MetadataResponse; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; @@ -160,6 +162,48 @@ public class Fetcher { } } + + + /** + * Get metadata for all topics present in Kafka cluster + * + * @param timeout time for which getting all topics is attempted + * @return The map of topics and its partitions + */ + public Map> getAllTopics(long timeout) { + final HashMap> topicsPartitionInfos = new HashMap<>(); + long startTime = time.milliseconds(); + + while (time.milliseconds() - startTime < timeout) { + final Node node = client.leastLoadedNode(); + if (node != null) { + MetadataRequest metadataRequest = new MetadataRequest(Collections.emptyList()); + final RequestFuture requestFuture = + client.send(node, ApiKeys.METADATA, metadataRequest); + + client.poll(requestFuture); + + if (requestFuture.succeeded()) { + MetadataResponse response = + new MetadataResponse(requestFuture.value().responseBody()); + + for (String topic : response.cluster().topics()) + topicsPartitionInfos.put( + topic, response.cluster().availablePartitionsForTopic(topic)); + + return topicsPartitionInfos; + } + + if (!requestFuture.isRetriable()) + throw requestFuture.exception(); + } + + Utils.sleep(retryBackoffMs); + } + + return topicsPartitionInfos; + } + /** * Reset offsets for the given partition using the offset reset strategy. * diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java index 4002679..06e2990 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -22,6 +22,7 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.OffsetResetStrategy; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; +import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.protocol.Errors; @@ -29,6 +30,7 @@ import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.requests.FetchResponse; +import org.apache.kafka.common.requests.MetadataResponse; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.test.TestUtils; @@ -180,6 +182,17 @@ public class FetcherTest { assertEquals(null, subscriptions.consumed(tp)); } + @Test + public void testGetAllTopics() throws InterruptedException { + // sending response before request, as getAllTopics is a blocking call + client.prepareResponse( + new MetadataResponse(cluster, Collections.emptyMap()).toStruct()); + + Map> allTopics = fetcher.getAllTopics(5000L); + + assertEquals(cluster.topics().size(), allTopics.size()); + } + private Struct fetchResponse(ByteBuffer buffer, short error, long hw) { FetchResponse response = new FetchResponse(Collections.singletonMap(tp, new FetchResponse.PartitionData(error, hw, buffer))); return response.toStruct(); diff --git a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala index cca6e94..0c2755f 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala @@ -186,6 +186,24 @@ class ConsumerTest extends IntegrationTestHarness with Logging { assertNull(this.consumers(0).partitionsFor("non-exist-topic")) } + def testListTopics() { + val numParts = 2 + val topic1: String = "part-test-topic-1" + val topic2: String = "part-test-topic-2" + val topic3: String = "part-test-topic-3" + TestUtils.createTopic(this.zkClient, topic1, numParts, 1, this.servers) + TestUtils.createTopic(this.zkClient, topic2, numParts, 1, this.servers) + TestUtils.createTopic(this.zkClient, topic3, numParts, 1, this.servers) + + val topics = this.consumers.head.listTopics() + assertNotNull(topics) + assertEquals(5, topics.size()) + assertEquals(5, topics.keySet().size()) + assertEquals(2, topics.get(topic1).length) + assertEquals(2, topics.get(topic2).length) + assertEquals(2, topics.get(topic3).length) + } + def testPartitionReassignmentCallback() { val callback = new TestConsumerReassignmentCallback() this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "100"); // timeout quickly to avoid slow test -- 1.7.12.4 From f4101ab3fcf7ec65f6541b157f1894ffdc8d861d Mon Sep 17 00:00:00 2001 From: Rajini Sivaram Date: Tue, 28 Jul 2015 16:31:33 -0700 Subject: [PATCH 072/120] KAFKA-2089: Fix transient MetadataTest failure; reviewed by Jiangjie Qin and Guozhang Wang --- .../java/org/apache/kafka/clients/MetadataTest.java | 18 +++++++++++++----- 1 file changed, 13 insertions(+), 5 deletions(-) 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 249d6b8..5fe8821 100644 --- a/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java @@ -12,7 +12,7 @@ */ package org.apache.kafka.clients; -import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.errors.TimeoutException; @@ -27,11 +27,11 @@ public class MetadataTest { private long refreshBackoffMs = 100; private long metadataExpireMs = 1000; private Metadata metadata = new Metadata(refreshBackoffMs, metadataExpireMs); - private AtomicBoolean backgroundError = new AtomicBoolean(false); + private AtomicReference backgroundError = new AtomicReference(); @After public void tearDown() { - assertFalse(backgroundError.get()); + assertNull("Exception in background thread : " + backgroundError.get(), backgroundError.get()); } @Test @@ -48,7 +48,15 @@ public class MetadataTest { Thread t2 = asyncFetch(topic); assertTrue("Awaiting update", t1.isAlive()); assertTrue("Awaiting update", t2.isAlive()); - metadata.update(TestUtils.singletonCluster(topic, 1), time); + // Perform metadata update when an update is requested on the async fetch thread + // This simulates the metadata update sequence in KafkaProducer + while (t1.isAlive() || t2.isAlive()) { + if (metadata.timeToNextUpdate(time) == 0) { + metadata.update(TestUtils.singletonCluster(topic, 1), time); + time += refreshBackoffMs; + } + Thread.sleep(1); + } t1.join(); t2.join(); assertFalse("No update needed.", metadata.timeToNextUpdate(time) == 0); @@ -106,7 +114,7 @@ public class MetadataTest { try { metadata.awaitUpdate(metadata.requestUpdate(), refreshBackoffMs); } catch (Exception e) { - backgroundError.set(true); + backgroundError.set(e.toString()); } } } -- 1.7.12.4 From e43c9aff92c57da6abb0c1d0af3431a550110a89 Mon Sep 17 00:00:00 2001 From: Geoff Anderson Date: Tue, 28 Jul 2015 17:22:14 -0700 Subject: [PATCH 073/120] KAFKA-2276; KIP-25 initial patch Initial patch for KIP-25 Note that to install ducktape, do *not* use pip to install ducktape. Instead: ``` $ git clone gitgithub.com:confluentinc/ducktape.git $ cd ducktape $ python setup.py install ``` Author: Geoff Anderson Author: Geoff Author: Liquan Pei Reviewers: Ewen, Gwen, Jun, Guozhang Closes #70 from granders/KAFKA-2276 and squashes the following commits: a62fb6c [Geoff Anderson] fixed checkstyle errors a70f0f8 [Geoff Anderson] Merged in upstream trunk. 8b62019 [Geoff Anderson] Merged in upstream trunk. 47b7b64 [Geoff Anderson] Created separate tools jar so that the clients package does not pull in dependencies on the Jackson JSON tools or argparse4j. a9e6a14 [Geoff Anderson] Merged in upstream changes d18db7b [Geoff Anderson] fixed :rat errors (needed to add licenses) 321fdf8 [Geoff Anderson] Ignore tests/ and vagrant/ directories when running rat build task 795fc75 [Geoff Anderson] Merged in changes from upstream trunk. 1d93f06 [Geoff Anderson] Updated provisioning to use java 7 in light of KAFKA-2316 2ea4e29 [Geoff Anderson] Tweaked README, changed default log collection behavior on VerifiableProducer 0eb6fdc [Geoff Anderson] Merged in system-tests 69dd7be [Geoff Anderson] Merged in trunk 4034dd6 [Geoff Anderson] Merged in upstream trunk ede6450 [Geoff] Merge pull request #4 from confluentinc/move_muckrake 7751545 [Geoff Anderson] Corrected license headers e6d532f [Geoff Anderson] java 7 -> java 6 8c61e2d [Geoff Anderson] Reverted jdk back to 6 f14c507 [Geoff Anderson] Removed mode = "test" from Vagrantfile and Vagrantfile.local examples. Updated testing README to clarify aws setup. 98b7253 [Geoff Anderson] Updated consumer tests to pre-populate kafka logs e6a41f1 [Geoff Anderson] removed stray println b15b24f [Geoff Anderson] leftover KafkaBenchmark in super call 0f75187 [Geoff Anderson] Rmoved stray allow_fail. kafka_benchmark_test -> benchmark_test f469f84 [Geoff Anderson] Tweaked readme, added example Vagrantfile.local 3d73857 [Geoff Anderson] Merged downstream changes 42dcdb1 [Geoff Anderson] Tweaked behavior of stop_node, clean_node to generally fail fast 7f7c3e0 [Geoff Anderson] Updated setup.py for kafkatest c60125c [Geoff Anderson] TestEndToEndLatency -> EndToEndLatency 4f476fe [Geoff Anderson] Moved aws scripts to vagrant directory 5af88fc [Geoff Anderson] Updated README to include aws quickstart e5edf03 [Geoff Anderson] Updated example aws Vagrantfile.local 96533c3 [Geoff] Update aws-access-keys-commands 25a413d [Geoff] Update aws-example-Vagrantfile.local 884b20e [Geoff Anderson] Moved a bunch of files to kafkatest directory fc7c81c [Geoff Anderson] added setup.py 632be12 [Geoff] Merge pull request #3 from confluentinc/verbose-client 51a94fd [Geoff Anderson] Use argparse4j instead of joptsimple. ThroughputThrottler now has more intuitive behavior when targetThroughput is 0. a80a428 [Geoff Anderson] Added shell program for VerifiableProducer. d586fb0 [Geoff Anderson] Updated comments to reflect that throttler is not message-specific 6842ed1 [Geoff Anderson] left out a file from last commit 1228eef [Geoff Anderson] Renamed throttler 9100417 [Geoff Anderson] Updated command-line options for VerifiableProducer. Extracted throughput logic to make it reusable. 0a5de8e [Geoff Anderson] Fixed checkstyle errors. Changed name to VerifiableProducer. Added synchronization for thread safety on println statements. 475423b [Geoff Anderson] Convert class to string before adding to json object. bc009f2 [Geoff Anderson] Got rid of VerboseProducer in core (moved to clients) c0526fe [Geoff Anderson] Updates per review comments. 8b4b1f2 [Geoff Anderson] Minor updates to VerboseProducer 2777712 [Geoff Anderson] Added some metadata to producer output. da94b8c [Geoff Anderson] Added number of messages option. 07cd1c6 [Geoff Anderson] Added simple producer which prints status of produced messages to stdout. a278988 [Geoff Anderson] fixed typos f1914c3 [Liquan Pei] Merge pull request #2 from confluentinc/system_tests 81e4156 [Liquan Pei] Bootstrap Kafka system tests --- .gitignore | 5 + Vagrantfile | 39 ++- bin/kafka-run-class.sh | 10 + bin/kafka-verifiable-producer.sh | 20 ++ build.gradle | 60 +++- checkstyle/import-control.xml | 2 + .../kafka/clients/tools/ProducerPerformance.java | 219 --------------- .../main/scala/kafka/tools/EndToEndLatency.scala | 92 ++++++ .../scala/kafka/tools/ProducerPerformance.scala | 4 +- .../scala/kafka/tools/TestEndToEndLatency.scala | 91 ------ settings.gradle | 3 +- tests/.gitignore | 11 + tests/README.md | 150 ++++++++++ tests/kafkatest/__init__.py | 16 ++ tests/kafkatest/services/__init__.py | 15 + tests/kafkatest/services/console_consumer.py | 146 ++++++++++ tests/kafkatest/services/kafka.py | 227 +++++++++++++++ tests/kafkatest/services/performance.py | 163 +++++++++++ .../services/templates/console_consumer.properties | 19 ++ .../kafkatest/services/templates/kafka.properties | 121 ++++++++ .../services/templates/zookeeper.properties | 25 ++ tests/kafkatest/services/verifiable_producer.py | 107 +++++++ tests/kafkatest/services/zookeeper.py | 64 +++++ tests/kafkatest/tests/__init__.py | 15 + tests/kafkatest/tests/benchmark_test.py | 274 ++++++++++++++++++ tests/kafkatest/tests/kafka_test.py | 45 +++ tests/kafkatest/tests/replication_test.py | 165 +++++++++++ tests/setup.py | 27 ++ .../kafka/clients/tools/ProducerPerformance.java | 201 ++++++++++++++ .../kafka/clients/tools/ThroughputThrottler.java | 118 ++++++++ .../kafka/clients/tools/VerifiableProducer.java | 307 +++++++++++++++++++++ vagrant/aws/aws-access-keys-commands | 29 ++ vagrant/aws/aws-example-Vagrantfile.local | 28 ++ vagrant/aws/aws-init.sh | 73 +++++ vagrant/base.sh | 9 + vagrant/system-test-Vagrantfile.local | 22 ++ 36 files changed, 2603 insertions(+), 319 deletions(-) create mode 100755 bin/kafka-verifiable-producer.sh delete mode 100644 clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java create mode 100755 core/src/main/scala/kafka/tools/EndToEndLatency.scala delete mode 100755 core/src/test/scala/kafka/tools/TestEndToEndLatency.scala create mode 100644 tests/.gitignore create mode 100644 tests/README.md create mode 100644 tests/kafkatest/__init__.py create mode 100644 tests/kafkatest/services/__init__.py create mode 100644 tests/kafkatest/services/console_consumer.py create mode 100644 tests/kafkatest/services/kafka.py create mode 100644 tests/kafkatest/services/performance.py create mode 100644 tests/kafkatest/services/templates/console_consumer.properties create mode 100644 tests/kafkatest/services/templates/kafka.properties create mode 100644 tests/kafkatest/services/templates/zookeeper.properties create mode 100644 tests/kafkatest/services/verifiable_producer.py create mode 100644 tests/kafkatest/services/zookeeper.py create mode 100644 tests/kafkatest/tests/__init__.py create mode 100644 tests/kafkatest/tests/benchmark_test.py create mode 100644 tests/kafkatest/tests/kafka_test.py create mode 100644 tests/kafkatest/tests/replication_test.py create mode 100644 tests/setup.py create mode 100644 tools/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java create mode 100644 tools/src/main/java/org/apache/kafka/clients/tools/ThroughputThrottler.java create mode 100644 tools/src/main/java/org/apache/kafka/clients/tools/VerifiableProducer.java create mode 100644 vagrant/aws/aws-access-keys-commands create mode 100644 vagrant/aws/aws-example-Vagrantfile.local create mode 100755 vagrant/aws/aws-init.sh create mode 100644 vagrant/system-test-Vagrantfile.local diff --git a/.gitignore b/.gitignore index 1f3ba7d..4aae6e7 100644 --- a/.gitignore +++ b/.gitignore @@ -29,3 +29,8 @@ config/server-* config/zookeeper-* core/data/* gradle/wrapper/* + +results +tests/results +.ducktape +tests/.ducktape diff --git a/Vagrantfile b/Vagrantfile index 1d7cc01..ee8b352 100644 --- a/Vagrantfile +++ b/Vagrantfile @@ -31,6 +31,7 @@ ram_megabytes = 1280 # EC2 ec2_access_key = ENV['AWS_ACCESS_KEY'] ec2_secret_key = ENV['AWS_SECRET_KEY'] +ec2_session_token = ENV['AWS_SESSION_TOKEN'] ec2_keypair_name = nil ec2_keypair_file = nil @@ -50,6 +51,24 @@ if File.exists?(local_config_file) then eval(File.read(local_config_file), binding, "Vagrantfile.local") end +# This is a horrible hack to work around bad interactions between +# vagrant-hostmanager and vagrant-aws/vagrant's implementation. Hostmanager +# wants to update the /etc/hosts entries, but tries to do so even on nodes that +# aren't up (e.g. even when all nodes are stopped and you run vagrant +# destroy). Because of the way the underlying code in vagrant works, it still +# tries to communicate with the node and has to wait for a very long +# timeout. This modifies the update to check for hosts that are not created or +# stopped, skipping the update in that case since it's impossible to update +# nodes in that state. +Object.const_get("VagrantPlugins").const_get("HostManager").const_get("HostsFile").class_eval do + alias_method :old_update_guest, :update_guest + def update_guest(machine) + state_id = machine.state.id + return if state_id == :not_created || state_id == :stopped + old_update_guest(machine) + end +end + # TODO(ksweeney): RAM requirements are not empirical and can probably be significantly lowered. Vagrant.configure(VAGRANTFILE_API_VERSION) do |config| config.hostmanager.enabled = true @@ -85,13 +104,31 @@ Vagrant.configure(VAGRANTFILE_API_VERSION) do |config| override.vm.box = "dummy" override.vm.box_url = "https://github.com/mitchellh/vagrant-aws/raw/master/dummy.box" - override.hostmanager.ignore_private_ip = true + cached_addresses = {} + # Use a custom resolver that SSH's into the machine and finds the IP address + # directly. This lets us get at the private IP address directly, avoiding + # some issues with using the default IP resolver, which uses the public IP + # address. + override.hostmanager.ip_resolver = proc do |vm, resolving_vm| + if !cached_addresses.has_key?(vm.name) + state_id = vm.state.id + if state_id != :not_created && state_id != :stopped && vm.communicate.ready? + vm.communicate.execute("/sbin/ifconfig eth0 | grep 'inet addr' | tail -n 1 | egrep -o '[0-9\.]+' | head -n 1 2>&1") do |type, contents| + cached_addresses[vm.name] = contents.split("\n").first[/(\d+\.\d+\.\d+\.\d+)/, 1] + end + else + cached_addresses[vm.name] = nil + end + end + cached_addresses[vm.name] + end override.ssh.username = ec2_user override.ssh.private_key_path = ec2_keypair_file aws.access_key_id = ec2_access_key aws.secret_access_key = ec2_secret_key + aws.session_token = ec2_session_token aws.keypair_name = ec2_keypair_name aws.region = ec2_region diff --git a/bin/kafka-run-class.sh b/bin/kafka-run-class.sh index 8c3fa28..ebe7409 100755 --- a/bin/kafka-run-class.sh +++ b/bin/kafka-run-class.sh @@ -65,6 +65,16 @@ do CLASSPATH=$CLASSPATH:$file done +for file in $base_dir/tools/build/libs/kafka-tools*.jar; +do + CLASSPATH=$CLASSPATH:$file +done + +for file in $base_dir/tools/build/dependant-libs-${SCALA_VERSION}*/*.jar; +do + CLASSPATH=$CLASSPATH:$file +done + # classpath addition for release for file in $base_dir/libs/*.jar; do diff --git a/bin/kafka-verifiable-producer.sh b/bin/kafka-verifiable-producer.sh new file mode 100755 index 0000000..d0aa6c5 --- /dev/null +++ b/bin/kafka-verifiable-producer.sh @@ -0,0 +1,20 @@ +#!/bin/bash +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +if [ "x$KAFKA_HEAP_OPTS" = "x" ]; then + export KAFKA_HEAP_OPTS="-Xmx512M" +fi +exec $(dirname $0)/kafka-run-class.sh org.apache.kafka.clients.tools.VerifiableProducer $@ diff --git a/build.gradle b/build.gradle index 9b6eb51..1b67e62 100644 --- a/build.gradle +++ b/build.gradle @@ -204,20 +204,20 @@ for ( sv in ['2_10_5', '2_11_7'] ) { } } -tasks.create(name: "jarAll", dependsOn: ['jar_core_2_10_5', 'jar_core_2_11_7', 'clients:jar', 'examples:jar', 'contrib:hadoop-consumer:jar', 'contrib:hadoop-producer:jar', 'log4j-appender:jar']) { +tasks.create(name: "jarAll", dependsOn: ['jar_core_2_10_5', 'jar_core_2_11_7', 'clients:jar', 'examples:jar', 'contrib:hadoop-consumer:jar', 'contrib:hadoop-producer:jar', 'log4j-appender:jar', 'tools:jar']) { } -tasks.create(name: "srcJarAll", dependsOn: ['srcJar_2_10_5', 'srcJar_2_11_7', 'clients:srcJar', 'examples:srcJar', 'contrib:hadoop-consumer:srcJar', 'contrib:hadoop-producer:srcJar', 'log4j-appender:srcJar']) { } +tasks.create(name: "srcJarAll", dependsOn: ['srcJar_2_10_5', 'srcJar_2_11_7', 'clients:srcJar', 'examples:srcJar', 'contrib:hadoop-consumer:srcJar', 'contrib:hadoop-producer:srcJar', 'log4j-appender:srcJar', 'tools:srcJar']) { } -tasks.create(name: "docsJarAll", dependsOn: ['docsJar_2_10_5', 'docsJar_2_11_7', 'clients:docsJar', 'examples:docsJar', 'contrib:hadoop-consumer:docsJar', 'contrib:hadoop-producer:docsJar', 'log4j-appender:docsJar']) { } +tasks.create(name: "docsJarAll", dependsOn: ['docsJar_2_10_5', 'docsJar_2_11_7', 'clients:docsJar', 'examples:docsJar', 'contrib:hadoop-consumer:docsJar', 'contrib:hadoop-producer:docsJar', 'log4j-appender:docsJar', 'tools:docsJar']) { } -tasks.create(name: "testAll", dependsOn: ['test_core_2_10_5', 'test_core_2_11_7', 'clients:test', 'log4j-appender:test']) { +tasks.create(name: "testAll", dependsOn: ['test_core_2_10_5', 'test_core_2_11_7', 'clients:test', 'log4j-appender:test', 'tools:test']) { } tasks.create(name: "releaseTarGzAll", dependsOn: ['releaseTarGz_2_10_5', 'releaseTarGz_2_11_7']) { } -tasks.create(name: "uploadArchivesAll", dependsOn: ['uploadCoreArchives_2_10_5', 'uploadCoreArchives_2_11_7', 'clients:uploadArchives', 'examples:uploadArchives', 'contrib:hadoop-consumer:uploadArchives', 'contrib:hadoop-producer:uploadArchives', 'log4j-appender:uploadArchives']) { +tasks.create(name: "uploadArchivesAll", dependsOn: ['uploadCoreArchives_2_10_5', 'uploadCoreArchives_2_11_7', 'clients:uploadArchives', 'examples:uploadArchives', 'contrib:hadoop-consumer:uploadArchives', 'contrib:hadoop-producer:uploadArchives', 'log4j-appender:uploadArchives', 'tools:uploadArchives']) { } project(':core') { @@ -413,6 +413,56 @@ project(':clients') { test.dependsOn('checkstyleMain', 'checkstyleTest') } +project(':tools') { + apply plugin: 'checkstyle' + archivesBaseName = "kafka-tools" + + dependencies { + compile project(':clients') + compile 'net.sourceforge.argparse4j:argparse4j:0.5.0' + compile 'com.fasterxml.jackson.core:jackson-databind:2.5.4' + compile "$slf4jlog4j" + + testCompile 'junit:junit:4.6' + testCompile project(path: ':clients', configuration: 'archives') + } + + task testJar(type: Jar) { + classifier = 'test' + from sourceSets.test.output + } + + test { + testLogging { + events "passed", "skipped", "failed" + exceptionFormat = 'full' + } + } + + javadoc { + include "**/org/apache/kafka/tools/*" + } + + tasks.create(name: "copyDependantLibs", type: Copy) { + from (configurations.testRuntime) { + include('slf4j-log4j12*') + } + from (configurations.runtime) { + exclude('kafka-clients*') + } + into "$buildDir/dependant-libs-${scalaVersion}" + } + + jar { + dependsOn 'copyDependantLibs' + } + + checkstyle { + configFile = new File(rootDir, "checkstyle/checkstyle.xml") + } + test.dependsOn('checkstyleMain', 'checkstyleTest') +} + project(':log4j-appender') { apply plugin: 'checkstyle' archivesBaseName = "kafka-log4j-appender" diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 19e0659..18be1bb 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -92,6 +92,8 @@ + + diff --git a/clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java b/clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java deleted file mode 100644 index 13f4d59..0000000 --- a/clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java +++ /dev/null @@ -1,219 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE - * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file - * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the - * License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on - * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the - * specific language governing permissions and limitations under the License. - */ -package org.apache.kafka.clients.tools; - -import java.util.Arrays; -import java.util.Properties; - -import org.apache.kafka.clients.producer.*; - -public class ProducerPerformance { - - private static final long NS_PER_MS = 1000000L; - private static final long NS_PER_SEC = 1000 * NS_PER_MS; - private static final long MIN_SLEEP_NS = 2 * NS_PER_MS; - - public static void main(String[] args) throws Exception { - if (args.length < 4) { - System.err.println("USAGE: java " + ProducerPerformance.class.getName() + - " topic_name num_records record_size target_records_sec [prop_name=prop_value]*"); - System.exit(1); - } - - /* parse args */ - String topicName = args[0]; - long numRecords = Long.parseLong(args[1]); - int recordSize = Integer.parseInt(args[2]); - int throughput = Integer.parseInt(args[3]); - - Properties props = new Properties(); - for (int i = 4; i < args.length; i++) { - String[] pieces = args[i].split("="); - if (pieces.length != 2) - throw new IllegalArgumentException("Invalid property: " + args[i]); - props.put(pieces[0], pieces[1]); - } - props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); - props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); - KafkaProducer producer = new KafkaProducer(props); - - /* setup perf test */ - byte[] payload = new byte[recordSize]; - Arrays.fill(payload, (byte) 1); - ProducerRecord record = new ProducerRecord(topicName, payload); - long sleepTime = NS_PER_SEC / throughput; - long sleepDeficitNs = 0; - Stats stats = new Stats(numRecords, 5000); - long start = System.currentTimeMillis(); - for (int i = 0; i < numRecords; i++) { - long sendStart = System.currentTimeMillis(); - Callback cb = stats.nextCompletion(sendStart, payload.length, stats); - producer.send(record, cb); - - /* - * Maybe sleep a little to control throughput. Sleep time can be a bit inaccurate for times < 1 ms so - * instead of sleeping each time instead wait until a minimum sleep time accumulates (the "sleep deficit") - * and then make up the whole deficit in one longer sleep. - */ - if (throughput > 0) { - float elapsed = (sendStart - start) / 1000.f; - if (elapsed > 0 && i / elapsed > throughput) { - sleepDeficitNs += sleepTime; - if (sleepDeficitNs >= MIN_SLEEP_NS) { - long sleepMs = sleepDeficitNs / 1000000; - long sleepNs = sleepDeficitNs - sleepMs * 1000000; - Thread.sleep(sleepMs, (int) sleepNs); - sleepDeficitNs = 0; - } - } - } - } - - /* print final results */ - producer.close(); - stats.printTotal(); - } - - private static class Stats { - private long start; - private long windowStart; - private int[] latencies; - private int sampling; - private int iteration; - private int index; - private long count; - private long bytes; - private int maxLatency; - private long totalLatency; - private long windowCount; - private int windowMaxLatency; - private long windowTotalLatency; - private long windowBytes; - private long reportingInterval; - - public Stats(long numRecords, int reportingInterval) { - this.start = System.currentTimeMillis(); - this.windowStart = System.currentTimeMillis(); - this.index = 0; - this.iteration = 0; - this.sampling = (int) (numRecords / Math.min(numRecords, 500000)); - this.latencies = new int[(int) (numRecords / this.sampling) + 1]; - this.index = 0; - this.maxLatency = 0; - this.totalLatency = 0; - this.windowCount = 0; - this.windowMaxLatency = 0; - this.windowTotalLatency = 0; - this.windowBytes = 0; - this.totalLatency = 0; - this.reportingInterval = reportingInterval; - } - - public void record(int iter, int latency, int bytes, long time) { - this.count++; - this.bytes += bytes; - this.totalLatency += latency; - this.maxLatency = Math.max(this.maxLatency, latency); - this.windowCount++; - this.windowBytes += bytes; - this.windowTotalLatency += latency; - this.windowMaxLatency = Math.max(windowMaxLatency, latency); - if (iter % this.sampling == 0) { - this.latencies[index] = latency; - this.index++; - } - /* maybe report the recent perf */ - if (time - windowStart >= reportingInterval) { - printWindow(); - newWindow(); - } - } - - public Callback nextCompletion(long start, int bytes, Stats stats) { - Callback cb = new PerfCallback(this.iteration, start, bytes, stats); - this.iteration++; - return cb; - } - - public void printWindow() { - long ellapsed = System.currentTimeMillis() - windowStart; - double recsPerSec = 1000.0 * windowCount / (double) ellapsed; - double mbPerSec = 1000.0 * this.windowBytes / (double) ellapsed / (1024.0 * 1024.0); - System.out.printf("%d records sent, %.1f records/sec (%.2f MB/sec), %.1f ms avg latency, %.1f max latency.\n", - windowCount, - recsPerSec, - mbPerSec, - windowTotalLatency / (double) windowCount, - (double) windowMaxLatency); - } - - public void newWindow() { - this.windowStart = System.currentTimeMillis(); - this.windowCount = 0; - this.windowMaxLatency = 0; - this.windowTotalLatency = 0; - this.windowBytes = 0; - } - - public void printTotal() { - long ellapsed = System.currentTimeMillis() - start; - double recsPerSec = 1000.0 * count / (double) ellapsed; - double mbPerSec = 1000.0 * this.bytes / (double) ellapsed / (1024.0 * 1024.0); - int[] percs = percentiles(this.latencies, index, 0.5, 0.95, 0.99, 0.999); - System.out.printf("%d records sent, %f records/sec (%.2f MB/sec), %.2f ms avg latency, %.2f ms max latency, %d ms 50th, %d ms 95th, %d ms 99th, %d ms 99.9th.\n", - count, - recsPerSec, - mbPerSec, - totalLatency / (double) count, - (double) maxLatency, - percs[0], - percs[1], - percs[2], - percs[3]); - } - - private static int[] percentiles(int[] latencies, int count, double... percentiles) { - int size = Math.min(count, latencies.length); - Arrays.sort(latencies, 0, size); - int[] values = new int[percentiles.length]; - for (int i = 0; i < percentiles.length; i++) { - int index = (int) (percentiles[i] * size); - values[i] = latencies[index]; - } - return values; - } - } - - private static final class PerfCallback implements Callback { - private final long start; - private final int iteration; - private final int bytes; - private final Stats stats; - - public PerfCallback(int iter, long start, int bytes, Stats stats) { - this.start = start; - this.stats = stats; - this.iteration = iter; - this.bytes = bytes; - } - - public void onCompletion(RecordMetadata metadata, Exception exception) { - long now = System.currentTimeMillis(); - int latency = (int) (now - start); - this.stats.record(iteration, latency, bytes, now); - if (exception != null) - exception.printStackTrace(); - } - } - -} diff --git a/core/src/main/scala/kafka/tools/EndToEndLatency.scala b/core/src/main/scala/kafka/tools/EndToEndLatency.scala new file mode 100755 index 0000000..7bb69b7 --- /dev/null +++ b/core/src/main/scala/kafka/tools/EndToEndLatency.scala @@ -0,0 +1,92 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.tools + +import java.util.{Arrays, Properties} + +import kafka.consumer._ +import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} + +import scala.Option.option2Iterable + +object EndToEndLatency { + def main(args: Array[String]) { + if (args.length != 6) { + System.err.println("USAGE: java " + getClass().getName + " broker_list zookeeper_connect topic num_messages consumer_fetch_max_wait producer_acks") + System.exit(1) + } + + val brokerList = args(0) + val zkConnect = args(1) + val topic = args(2) + val numMessages = args(3).toInt + val consumerFetchMaxWait = args(4).toInt + val producerAcks = args(5).toInt + + val consumerProps = new Properties() + consumerProps.put("group.id", topic) + consumerProps.put("auto.commit.enable", "false") + consumerProps.put("auto.offset.reset", "largest") + consumerProps.put("zookeeper.connect", zkConnect) + consumerProps.put("fetch.wait.max.ms", consumerFetchMaxWait.toString) + consumerProps.put("socket.timeout.ms", 1201000.toString) + + val config = new ConsumerConfig(consumerProps) + val connector = Consumer.create(config) + val stream = connector.createMessageStreams(Map(topic -> 1)).get(topic).head.head + val iter = stream.iterator + + val producerProps = new Properties() + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) + producerProps.put(ProducerConfig.LINGER_MS_CONFIG, "0") + producerProps.put(ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG, "true") + producerProps.put(ProducerConfig.ACKS_CONFIG, producerAcks.toString) + producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") + producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") + val producer = new KafkaProducer[Array[Byte],Array[Byte]](producerProps) + + // make sure the consumer fetcher has started before sending data since otherwise + // the consumption from the tail will skip the first message and hence be blocked + Thread.sleep(5000) + + val message = "hello there beautiful".getBytes + var totalTime = 0.0 + val latencies = new Array[Long](numMessages) + for (i <- 0 until numMessages) { + val begin = System.nanoTime + producer.send(new ProducerRecord[Array[Byte],Array[Byte]](topic, message)) + val received = iter.next + val elapsed = System.nanoTime - begin + // poor man's progress bar + if (i % 1000 == 0) + println(i + "\t" + elapsed / 1000.0 / 1000.0) + totalTime += elapsed + latencies(i) = (elapsed / 1000 / 1000) + } + println("Avg latency: %.4f ms\n".format(totalTime / numMessages / 1000.0 / 1000.0)) + Arrays.sort(latencies) + val p50 = latencies((latencies.length * 0.5).toInt) + val p99 = latencies((latencies.length * 0.99).toInt) + val p999 = latencies((latencies.length * 0.999).toInt) + println("Percentiles: 50th = %d, 99th = %d, 99.9th = %d".format(p50, p99, p999)) + producer.close() + connector.commitOffsets(true) + connector.shutdown() + System.exit(0) + } +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/tools/ProducerPerformance.scala b/core/src/main/scala/kafka/tools/ProducerPerformance.scala index 0ebfa59..0335cc6 100644 --- a/core/src/main/scala/kafka/tools/ProducerPerformance.scala +++ b/core/src/main/scala/kafka/tools/ProducerPerformance.scala @@ -115,9 +115,9 @@ object ProducerPerformance extends Logging { .defaultsTo(0) val csvMetricsReporterEnabledOpt = parser.accepts("csv-reporter-enabled", "If set, the CSV metrics reporter will be enabled") val metricsDirectoryOpt = parser.accepts("metrics-dir", "If csv-reporter-enable is set, and this parameter is" + - "set, the csv metrics will be outputed here") + "set, the csv metrics will be output here") .withRequiredArg - .describedAs("metrics dictory") + .describedAs("metrics directory") .ofType(classOf[java.lang.String]) val useNewProducerOpt = parser.accepts("new-producer", "Use the new producer implementation.") diff --git a/core/src/test/scala/kafka/tools/TestEndToEndLatency.scala b/core/src/test/scala/kafka/tools/TestEndToEndLatency.scala deleted file mode 100755 index 99b77a1..0000000 --- a/core/src/test/scala/kafka/tools/TestEndToEndLatency.scala +++ /dev/null @@ -1,91 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.tools - -import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord, KafkaProducer} -import kafka.consumer._ -import java.util.Properties -import java.util.Arrays -import scala.Option.option2Iterable - -object TestEndToEndLatency { - def main(args: Array[String]) { - if (args.length != 6) { - System.err.println("USAGE: java " + getClass().getName + " broker_list zookeeper_connect topic num_messages consumer_fetch_max_wait producer_acks") - System.exit(1) - } - - val brokerList = args(0) - val zkConnect = args(1) - val topic = args(2) - val numMessages = args(3).toInt - val consumerFetchMaxWait = args(4).toInt - val producerAcks = args(5).toInt - - val consumerProps = new Properties() - consumerProps.put("group.id", topic) - consumerProps.put("auto.commit.enable", "false") - consumerProps.put("auto.offset.reset", "largest") - consumerProps.put("zookeeper.connect", zkConnect) - consumerProps.put("fetch.wait.max.ms", consumerFetchMaxWait.toString) - consumerProps.put("socket.timeout.ms", 1201000.toString) - - val config = new ConsumerConfig(consumerProps) - val connector = Consumer.create(config) - val stream = connector.createMessageStreams(Map(topic -> 1)).get(topic).head.head - val iter = stream.iterator - - val producerProps = new Properties() - producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) - producerProps.put(ProducerConfig.LINGER_MS_CONFIG, "0") - producerProps.put(ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG, "true") - producerProps.put(ProducerConfig.ACKS_CONFIG, producerAcks.toString) - producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") - producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") - val producer = new KafkaProducer[Array[Byte],Array[Byte]](producerProps) - - // make sure the consumer fetcher has started before sending data since otherwise - // the consumption from the tail will skip the first message and hence be blocked - Thread.sleep(5000) - - val message = "hello there beautiful".getBytes - var totalTime = 0.0 - val latencies = new Array[Long](numMessages) - for (i <- 0 until numMessages) { - val begin = System.nanoTime - producer.send(new ProducerRecord[Array[Byte],Array[Byte]](topic, message)) - val received = iter.next - val elapsed = System.nanoTime - begin - // poor man's progress bar - if (i % 1000 == 0) - println(i + "\t" + elapsed / 1000.0 / 1000.0) - totalTime += elapsed - latencies(i) = (elapsed / 1000 / 1000) - } - println("Avg latency: %.4f ms\n".format(totalTime / numMessages / 1000.0 / 1000.0)) - Arrays.sort(latencies) - val p50 = latencies((latencies.length * 0.5).toInt) - val p99 = latencies((latencies.length * 0.99).toInt) - val p999 = latencies((latencies.length * 0.999).toInt) - println("Percentiles: 50th = %d, 99th = %d, 99.9th = %d".format(p50, p99, p999)) - producer.close() - connector.commitOffsets(true) - connector.shutdown() - System.exit(0) - } -} \ No newline at end of file diff --git a/settings.gradle b/settings.gradle index 3b6a952..1944917 100644 --- a/settings.gradle +++ b/settings.gradle @@ -14,4 +14,5 @@ // limitations under the License. apply from: file('scala.gradle') -include 'core', 'contrib:hadoop-consumer', 'contrib:hadoop-producer', 'examples', 'clients', 'log4j-appender' +include 'core', 'contrib:hadoop-consumer', 'contrib:hadoop-producer', 'examples', 'clients', 'tools', 'log4j-appender' + diff --git a/tests/.gitignore b/tests/.gitignore new file mode 100644 index 0000000..b218b83 --- /dev/null +++ b/tests/.gitignore @@ -0,0 +1,11 @@ +Vagrantfile.local + +.idea/ + +*.pyc +*.ipynb + +.DS_Store + +.ducktape +results/ diff --git a/tests/README.md b/tests/README.md new file mode 100644 index 0000000..ffbc0d5 --- /dev/null +++ b/tests/README.md @@ -0,0 +1,150 @@ +System Integration & Performance Testing +======================================== + +This directory contains Kafka system integration and performance tests. +[ducktape](https://github.com/confluentinc/ducktape) is used to run the tests. +(ducktape is a distributed testing framework which provides test runner, +result reporter and utilities to pull up and tear down services.) + +Local Quickstart +---------------- +This quickstart will help you run the Kafka system tests on your local machine. + +* Install Virtual Box from [https://www.virtualbox.org/](https://www.virtualbox.org/) (run `$ vboxmanage --version` to check if it's installed). +* Install Vagrant >= 1.6.4 from [http://www.vagrantup.com/](http://www.vagrantup.com/) (run `vagrant --version` to check if it's installed). +* Install Vagrant Plugins: + + # Required + $ vagrant plugin install vagrant-hostmanager vagrant-cachier + +* Build a specific branch of Kafka + + $ cd kafka + $ git checkout $BRANCH + $ gradle + $ ./gradlew jar + +* Setup a testing cluster with Vagrant. Configure your Vagrant setup by creating the file + `Vagrantfile.local` in the directory of your Kafka checkout. For testing purposes, + `num_brokers` and `num_kafka` should be 0, and `num_workers` should be set high enough + to run all of you tests. An example resides in kafka/vagrant/system-test-Vagrantfile.local + + # Example Vagrantfile.local for use on local machine + # Vagrantfile.local should reside in the base Kafka directory + num_zookeepers = 0 + num_kafka = 0 + num_workers = 9 + +* Bring up the cluster (note that the initial provisioning process can be slow since it involves +installing dependencies and updates on every vm.): + + $ vagrant up + +* Install ducktape: + + $ pip install ducktape + +* Run the system tests using ducktape: + + $ cd tests + $ ducktape kafkatest/tests + +* If you make changes to your Kafka checkout, you'll need to rebuild and resync to your Vagrant cluster: + + $ cd kafka + $ ./gradlew jar + $ vagrant rsync # Re-syncs build output to cluster + +EC2 Quickstart +-------------- +This quickstart will help you run the Kafka system tests on EC2. In this setup, all logic is run +on EC2 and none on your local machine. + +There are a lot of steps here, but the basic goals are to create one distinguished EC2 instance that +will be our "test driver", and to set up the security groups and iam role so that the test driver +can create, destroy, and run ssh commands on any number of "workers". + +As a convention, we'll use "kafkatest" in most names, but you can use whatever name you want. + +Preparation +----------- +In these steps, we will create an IAM role which has permission to create and destroy EC2 instances, +set up a keypair used for ssh access to the test driver and worker machines, and create a security group to allow the test driver and workers to all communicate via TCP. + +* [Create an IAM role](http://docs.aws.amazon.com/IAM/latest/UserGuide/Using_SettingUpUser.html#Using_CreateUser_console). We'll give this role the ability to launch or kill additional EC2 machines. + - Create role "kafkatest-master" + - Role type: Amazon EC2 + - Attach policy: AmazonEC2FullAccess (this will allow our test-driver to create and destroy EC2 instances) + +* If you haven't already, [set up a keypair to use for SSH access](http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/ec2-key-pairs.html). For the purpose +of this quickstart, let's say the keypair name is kafkatest, and you've saved the private key in kafktest.pem + +* Next, create a security group called "kafkatest". + - After creating the group, inbound rules: allow SSH on port 22 from anywhere; also, allow access on all ports (0-65535) from other machines in the kafkatest group. + +Create the Test Driver +---------------------- +* Launch a new test driver machine + - OS: Ubuntu server is recommended + - Instance type: t2.medium is easily enough since this machine is just a driver + - Instance details: Most defaults are fine. + - IAM role -> kafkatest-master + - Tagging the instance with a useful name is recommended. + - Security group -> 'kafkatest' + + +* Once the machine is started, upload the SSH key to your test driver: + + $ scp -i /path/to/kafkatest.pem \ + /path/to/kafkatest.pem ubuntu@public.hostname.amazonaws.com:kafkatest.pem + +* Grab the public hostname/IP (available for example by navigating to your EC2 dashboard and viewing running instances) of your test driver and SSH into it: + + $ ssh -i /path/to/kafkatest.pem ubuntu@public.hostname.amazonaws.com + +Set Up the Test Driver +---------------------- +The following steps assume you have ssh'd into +the test driver machine. + +* Start by making sure you're up to date, and install git and ducktape: + + $ sudo apt-get update && sudo apt-get -y upgrade && sudo apt-get install -y git + $ pip install ducktape + +* Get Kafka: + + $ git clone https://git-wip-us.apache.org/repos/asf/kafka.git kafka + +* Install some dependencies: + + $ cd kafka + $ kafka/vagrant/aws/aws-init.sh + $ . ~/.bashrc + +* An example Vagrantfile.local has been created by aws-init.sh which looks something like: + + # Vagrantfile.local + ec2_instance_type = "..." # Pick something appropriate for your + # test. Note that the default m3.medium has + # a small disk. + num_zookeepers = 0 + num_kafka = 0 + num_workers = 9 + ec2_keypair_name = 'kafkatest' + ec2_keypair_file = '/home/ubuntu/kafkatest.pem' + ec2_security_groups = ['kafkatest'] + ec2_region = 'us-west-2' + ec2_ami = "ami-29ebb519" + +* Start up the instances (note we have found bringing up machines in parallel can cause errors on aws): + + $ vagrant up --provider=aws --no-provision --no-parallel && vagrant provision + +* Now you should be able to run tests: + + $ cd kafka/tests + $ ducktape kafkatest/tests + +* To halt your workers without destroying persistent state, run `vagrant halt`. Run `vagrant destroy -f` to destroy all traces of your workers. + diff --git a/tests/kafkatest/__init__.py b/tests/kafkatest/__init__.py new file mode 100644 index 0000000..28d269b --- /dev/null +++ b/tests/kafkatest/__init__.py @@ -0,0 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# see kafka.server.KafkaConfig for additional details and defaults + diff --git a/tests/kafkatest/services/__init__.py b/tests/kafkatest/services/__init__.py new file mode 100644 index 0000000..ebc9bb3 --- /dev/null +++ b/tests/kafkatest/services/__init__.py @@ -0,0 +1,15 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# see kafka.server.KafkaConfig for additional details and defaults diff --git a/tests/kafkatest/services/console_consumer.py b/tests/kafkatest/services/console_consumer.py new file mode 100644 index 0000000..33ef4ea --- /dev/null +++ b/tests/kafkatest/services/console_consumer.py @@ -0,0 +1,146 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from ducktape.services.background_thread import BackgroundThreadService + + +def is_int(msg): + """Default method used to check whether text pulled from console consumer is a message. + + return int or None + """ + try: + return int(msg) + except: + return None + + +""" +0.8.2.1 ConsoleConsumer options + +The console consumer is a tool that reads data from Kafka and outputs it to standard output. +Option Description +------ ----------- +--blacklist Blacklist of topics to exclude from + consumption. +--consumer.config Consumer config properties file. +--csv-reporter-enabled If set, the CSV metrics reporter will + be enabled +--delete-consumer-offsets If specified, the consumer path in + zookeeper is deleted when starting up +--formatter The name of a class to use for + formatting kafka messages for + display. (default: kafka.tools. + DefaultMessageFormatter) +--from-beginning If the consumer does not already have + an established offset to consume + from, start with the earliest + message present in the log rather + than the latest message. +--max-messages The maximum number of messages to + consume before exiting. If not set, + consumption is continual. +--metrics-dir If csv-reporter-enable is set, and + this parameter isset, the csv + metrics will be outputed here +--property +--skip-message-on-error If there is an error when processing a + message, skip it instead of halt. +--topic The topic id to consume on. +--whitelist Whitelist of topics to include for + consumption. +--zookeeper REQUIRED: The connection string for + the zookeeper connection in the form + host:port. Multiple URLS can be + given to allow fail-over. +""" + + +class ConsoleConsumer(BackgroundThreadService): + logs = { + "consumer_log": { + "path": "/mnt/consumer.log", + "collect_default": True} + } + + def __init__(self, context, num_nodes, kafka, topic, message_validator=is_int, from_beginning=True, consumer_timeout_ms=None): + """ + Args: + context: standard context + num_nodes: number of nodes to use (this should be 1) + kafka: kafka service + topic: consume from this topic + message_validator: function which returns message or None + from_beginning: consume from beginning if True, else from the end + consumer_timeout_ms: corresponds to consumer.timeout.ms. consumer process ends if time between + successively consumed messages exceeds this timeout. Setting this and + waiting for the consumer to stop is a pretty good way to consume all messages + in a topic. + """ + super(ConsoleConsumer, self).__init__(context, num_nodes) + self.kafka = kafka + self.args = { + 'topic': topic, + } + + self.consumer_timeout_ms = consumer_timeout_ms + + self.from_beginning = from_beginning + self.message_validator = message_validator + self.messages_consumed = {idx: [] for idx in range(1, num_nodes + 1)} + + @property + def start_cmd(self): + args = self.args.copy() + args.update({'zk_connect': self.kafka.zk.connect_setting()}) + cmd = "/opt/kafka/bin/kafka-console-consumer.sh --topic %(topic)s --zookeeper %(zk_connect)s" \ + " --consumer.config /mnt/console_consumer.properties" % args + + if self.from_beginning: + cmd += " --from-beginning" + + cmd += " 2>> /mnt/consumer.log | tee -a /mnt/consumer.log &" + return cmd + + def _worker(self, idx, node): + # form config file + if self.consumer_timeout_ms is not None: + prop_file = self.render('console_consumer.properties', consumer_timeout_ms=self.consumer_timeout_ms) + else: + prop_file = self.render('console_consumer.properties') + + self.logger.info("console_consumer.properties:") + self.logger.info(prop_file) + node.account.create_file("/mnt/console_consumer.properties", prop_file) + + # Run and capture output + cmd = self.start_cmd + self.logger.debug("Console consumer %d command: %s", idx, cmd) + for line in node.account.ssh_capture(cmd): + msg = line.strip() + msg = self.message_validator(msg) + if msg is not None: + self.logger.debug("consumed a message: " + str(msg)) + self.messages_consumed[idx].append(msg) + + def start_node(self, node): + super(ConsoleConsumer, self).start_node(node) + + def stop_node(self, node): + node.account.kill_process("java", allow_fail=False) + + def clean_node(self, node): + node.account.ssh("rm -rf /mnt/console_consumer.properties /mnt/consumer.log", allow_fail=False) + diff --git a/tests/kafkatest/services/kafka.py b/tests/kafkatest/services/kafka.py new file mode 100644 index 0000000..34ec5ef --- /dev/null +++ b/tests/kafkatest/services/kafka.py @@ -0,0 +1,227 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from ducktape.services.service import Service + +import json +import re +import signal +import time + + +class KafkaService(Service): + + logs = { + "kafka_log": { + "path": "/mnt/kafka.log", + "collect_default": True}, + "kafka_data": { + "path": "/mnt/kafka-logs", + "collect_default": False} + } + + def __init__(self, context, num_nodes, zk, topics=None): + """ + :type context + :type zk: ZookeeperService + :type topics: dict + """ + super(KafkaService, self).__init__(context, num_nodes) + self.zk = zk + self.topics = topics + + def start(self): + super(KafkaService, self).start() + + # Create topics if necessary + if self.topics is not None: + for topic, topic_cfg in self.topics.items(): + if topic_cfg is None: + topic_cfg = {} + + topic_cfg["topic"] = topic + self.create_topic(topic_cfg) + + def start_node(self, node): + props_file = self.render('kafka.properties', node=node, broker_id=self.idx(node)) + self.logger.info("kafka.properties:") + self.logger.info(props_file) + node.account.create_file("/mnt/kafka.properties", props_file) + + cmd = "/opt/kafka/bin/kafka-server-start.sh /mnt/kafka.properties 1>> /mnt/kafka.log 2>> /mnt/kafka.log & echo $! > /mnt/kafka.pid" + self.logger.debug("Attempting to start KafkaService on %s with command: %s" % (str(node.account), cmd)) + node.account.ssh(cmd) + time.sleep(5) + if len(self.pids(node)) == 0: + raise Exception("No process ids recorded on node %s" % str(node)) + + def pids(self, node): + """Return process ids associated with running processes on the given node.""" + try: + return [pid for pid in node.account.ssh_capture("cat /mnt/kafka.pid", callback=int)] + except: + return [] + + def signal_node(self, node, sig=signal.SIGTERM): + pids = self.pids(node) + for pid in pids: + node.account.signal(pid, sig) + + def signal_leader(self, topic, partition=0, sig=signal.SIGTERM): + leader = self.leader(topic, partition) + self.signal_node(leader, sig) + + def stop_node(self, node, clean_shutdown=True): + pids = self.pids(node) + sig = signal.SIGTERM if clean_shutdown else signal.SIGKILL + + for pid in pids: + node.account.signal(pid, sig, allow_fail=False) + + node.account.ssh("rm -f /mnt/kafka.pid", allow_fail=False) + + def clean_node(self, node): + node.account.ssh("rm -rf /mnt/kafka-logs /mnt/kafka.properties /mnt/kafka.log /mnt/kafka.pid", allow_fail=False) + + def create_topic(self, topic_cfg): + node = self.nodes[0] # any node is fine here + self.logger.info("Creating topic %s with settings %s", topic_cfg["topic"], topic_cfg) + + cmd = "/opt/kafka/bin/kafka-topics.sh --zookeeper %(zk_connect)s --create "\ + "--topic %(topic)s --partitions %(partitions)d --replication-factor %(replication)d" % { + 'zk_connect': self.zk.connect_setting(), + 'topic': topic_cfg.get("topic"), + 'partitions': topic_cfg.get('partitions', 1), + 'replication': topic_cfg.get('replication-factor', 1) + } + + if "configs" in topic_cfg.keys() and topic_cfg["configs"] is not None: + for config_name, config_value in topic_cfg["configs"].items(): + cmd += " --config %s=%s" % (config_name, str(config_value)) + + self.logger.info("Running topic creation command...\n%s" % cmd) + node.account.ssh(cmd) + + time.sleep(1) + self.logger.info("Checking to see if topic was properly created...\n%s" % cmd) + for line in self.describe_topic(topic_cfg["topic"]).split("\n"): + self.logger.info(line) + + def describe_topic(self, topic): + node = self.nodes[0] + cmd = "/opt/kafka/bin/kafka-topics.sh --zookeeper %s --topic %s --describe" % \ + (self.zk.connect_setting(), topic) + output = "" + for line in node.account.ssh_capture(cmd): + output += line + return output + + def verify_reassign_partitions(self, reassignment): + """Run the reassign partitions admin tool in "verify" mode + """ + node = self.nodes[0] + json_file = "/tmp/" + str(time.time()) + "_reassign.json" + + # reassignment to json + json_str = json.dumps(reassignment) + json_str = json.dumps(json_str) + + # create command + cmd = "echo %s > %s && " % (json_str, json_file) + cmd += "/opt/kafka/bin/kafka-reassign-partitions.sh "\ + "--zookeeper %(zk_connect)s "\ + "--reassignment-json-file %(reassignment_file)s "\ + "--verify" % {'zk_connect': self.zk.connect_setting(), + 'reassignment_file': json_file} + cmd += " && sleep 1 && rm -f %s" % json_file + + # send command + self.logger.info("Verifying parition reassignment...") + self.logger.debug(cmd) + output = "" + for line in node.account.ssh_capture(cmd): + output += line + + self.logger.debug(output) + + if re.match(".*is in progress.*", output) is not None: + return False + + return True + + def execute_reassign_partitions(self, reassignment): + """Run the reassign partitions admin tool in "verify" mode + """ + node = self.nodes[0] + json_file = "/tmp/" + str(time.time()) + "_reassign.json" + + # reassignment to json + json_str = json.dumps(reassignment) + json_str = json.dumps(json_str) + + # create command + cmd = "echo %s > %s && " % (json_str, json_file) + cmd += "/opt/kafka/bin/kafka-reassign-partitions.sh "\ + "--zookeeper %(zk_connect)s "\ + "--reassignment-json-file %(reassignment_file)s "\ + "--execute" % {'zk_connect': self.zk.connect_setting(), + 'reassignment_file': json_file} + cmd += " && sleep 1 && rm -f %s" % json_file + + # send command + self.logger.info("Executing parition reassignment...") + self.logger.debug(cmd) + output = "" + for line in node.account.ssh_capture(cmd): + output += line + + self.logger.debug("Verify partition reassignment:") + self.logger.debug(output) + + def restart_node(self, node, wait_sec=0, clean_shutdown=True): + """Restart the given node, waiting wait_sec in between stopping and starting up again.""" + self.stop_node(node, clean_shutdown) + time.sleep(wait_sec) + self.start_node(node) + + def leader(self, topic, partition=0): + """ Get the leader replica for the given topic and partition. + """ + cmd = "/opt/kafka/bin/kafka-run-class.sh kafka.tools.ZooKeeperMainWrapper -server %s " \ + % self.zk.connect_setting() + cmd += "get /brokers/topics/%s/partitions/%d/state" % (topic, partition) + self.logger.debug(cmd) + + node = self.nodes[0] + self.logger.debug("Querying zookeeper to find leader replica for topic %s: \n%s" % (cmd, topic)) + partition_state = None + for line in node.account.ssh_capture(cmd): + match = re.match("^({.+})$", line) + if match is not None: + partition_state = match.groups()[0] + break + + if partition_state is None: + raise Exception("Error finding partition state for topic %s and partition %d." % (topic, partition)) + + partition_state = json.loads(partition_state) + self.logger.info(partition_state) + + leader_idx = int(partition_state["leader"]) + self.logger.info("Leader for topic %s and partition %d is now: %d" % (topic, partition, leader_idx)) + return self.get_node(leader_idx) + + def bootstrap_servers(self): + return ','.join([node.account.hostname + ":9092" for node in self.nodes]) \ No newline at end of file diff --git a/tests/kafkatest/services/performance.py b/tests/kafkatest/services/performance.py new file mode 100644 index 0000000..65c1a4d --- /dev/null +++ b/tests/kafkatest/services/performance.py @@ -0,0 +1,163 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from ducktape.services.background_thread import BackgroundThreadService + + +class PerformanceService(BackgroundThreadService): + def __init__(self, context, num_nodes): + super(PerformanceService, self).__init__(context, num_nodes) + self.results = [None] * self.num_nodes + self.stats = [[] for x in range(self.num_nodes)] + + +class ProducerPerformanceService(PerformanceService): + def __init__(self, context, num_nodes, kafka, topic, num_records, record_size, throughput, settings={}, intermediate_stats=False): + super(ProducerPerformanceService, self).__init__(context, num_nodes) + self.kafka = kafka + self.args = { + 'topic': topic, + 'num_records': num_records, + 'record_size': record_size, + 'throughput': throughput + } + self.settings = settings + self.intermediate_stats = intermediate_stats + + def _worker(self, idx, node): + args = self.args.copy() + args.update({'bootstrap_servers': self.kafka.bootstrap_servers()}) + cmd = "/opt/kafka/bin/kafka-run-class.sh org.apache.kafka.clients.tools.ProducerPerformance "\ + "%(topic)s %(num_records)d %(record_size)d %(throughput)d bootstrap.servers=%(bootstrap_servers)s" % args + + for key,value in self.settings.items(): + cmd += " %s=%s" % (str(key), str(value)) + self.logger.debug("Producer performance %d command: %s", idx, cmd) + + def parse_stats(line): + parts = line.split(',') + return { + 'records': int(parts[0].split()[0]), + 'records_per_sec': float(parts[1].split()[0]), + 'mbps': float(parts[1].split('(')[1].split()[0]), + 'latency_avg_ms': float(parts[2].split()[0]), + 'latency_max_ms': float(parts[3].split()[0]), + 'latency_50th_ms': float(parts[4].split()[0]), + 'latency_95th_ms': float(parts[5].split()[0]), + 'latency_99th_ms': float(parts[6].split()[0]), + 'latency_999th_ms': float(parts[7].split()[0]), + } + last = None + for line in node.account.ssh_capture(cmd): + self.logger.debug("Producer performance %d: %s", idx, line.strip()) + if self.intermediate_stats: + try: + self.stats[idx-1].append(parse_stats(line)) + except: + # Sometimes there are extraneous log messages + pass + last = line + try: + self.results[idx-1] = parse_stats(last) + except: + self.logger.error("Bad last line: %s", last) + + +class ConsumerPerformanceService(PerformanceService): + def __init__(self, context, num_nodes, kafka, topic, num_records, throughput, threads=1, settings={}): + super(ConsumerPerformanceService, self).__init__(context, num_nodes) + self.kafka = kafka + self.args = { + 'topic': topic, + 'num_records': num_records, + 'throughput': throughput, + 'threads': threads, + } + self.settings = settings + + def _worker(self, idx, node): + args = self.args.copy() + args.update({'zk_connect': self.kafka.zk.connect_setting()}) + cmd = "/opt/kafka/bin/kafka-consumer-perf-test.sh "\ + "--topic %(topic)s --messages %(num_records)d --zookeeper %(zk_connect)s" % args + for key,value in self.settings.items(): + cmd += " %s=%s" % (str(key), str(value)) + self.logger.debug("Consumer performance %d command: %s", idx, cmd) + last = None + for line in node.account.ssh_capture(cmd): + self.logger.debug("Consumer performance %d: %s", idx, line.strip()) + last = line + # Parse and save the last line's information + parts = last.split(',') + + self.results[idx-1] = { + 'total_mb': float(parts[2]), + 'mbps': float(parts[3]), + 'records_per_sec': float(parts[5]), + } + + +class EndToEndLatencyService(PerformanceService): + def __init__(self, context, num_nodes, kafka, topic, num_records, consumer_fetch_max_wait=100, acks=1): + super(EndToEndLatencyService, self).__init__(context, num_nodes) + self.kafka = kafka + self.args = { + 'topic': topic, + 'num_records': num_records, + 'consumer_fetch_max_wait': consumer_fetch_max_wait, + 'acks': acks + } + + def _worker(self, idx, node): + args = self.args.copy() + args.update({ + 'zk_connect': self.kafka.zk.connect_setting(), + 'bootstrap_servers': self.kafka.bootstrap_servers(), + }) + cmd = "/opt/kafka/bin/kafka-run-class.sh kafka.tools.EndToEndLatency "\ + "%(bootstrap_servers)s %(zk_connect)s %(topic)s %(num_records)d "\ + "%(consumer_fetch_max_wait)d %(acks)d" % args + self.logger.debug("End-to-end latency %d command: %s", idx, cmd) + results = {} + for line in node.account.ssh_capture(cmd): + self.logger.debug("End-to-end latency %d: %s", idx, line.strip()) + if line.startswith("Avg latency:"): + results['latency_avg_ms'] = float(line.split()[2]) + if line.startswith("Percentiles"): + results['latency_50th_ms'] = float(line.split()[3][:-1]) + results['latency_99th_ms'] = float(line.split()[6][:-1]) + results['latency_999th_ms'] = float(line.split()[9]) + self.results[idx-1] = results + + +def parse_performance_output(summary): + parts = summary.split(',') + results = { + 'records': int(parts[0].split()[0]), + 'records_per_sec': float(parts[1].split()[0]), + 'mbps': float(parts[1].split('(')[1].split()[0]), + 'latency_avg_ms': float(parts[2].split()[0]), + 'latency_max_ms': float(parts[3].split()[0]), + 'latency_50th_ms': float(parts[4].split()[0]), + 'latency_95th_ms': float(parts[5].split()[0]), + 'latency_99th_ms': float(parts[6].split()[0]), + 'latency_999th_ms': float(parts[7].split()[0]), + } + # To provide compatibility with ConsumerPerformanceService + results['total_mb'] = results['mbps'] * (results['records'] / results['records_per_sec']) + results['rate_mbps'] = results['mbps'] + results['rate_mps'] = results['records_per_sec'] + + return results diff --git a/tests/kafkatest/services/templates/console_consumer.properties b/tests/kafkatest/services/templates/console_consumer.properties new file mode 100644 index 0000000..63782fc --- /dev/null +++ b/tests/kafkatest/services/templates/console_consumer.properties @@ -0,0 +1,19 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# see kafka.server.KafkaConfig for additional details and defaults + +{% if consumer_timeout_ms is defined %} +consumer.timeout.ms={{ consumer_timeout_ms }} +{% endif %} \ No newline at end of file diff --git a/tests/kafkatest/services/templates/kafka.properties b/tests/kafkatest/services/templates/kafka.properties new file mode 100644 index 0000000..db1077a --- /dev/null +++ b/tests/kafkatest/services/templates/kafka.properties @@ -0,0 +1,121 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# see kafka.server.KafkaConfig for additional details and defaults + +############################# Server Basics ############################# + +# The id of the broker. This must be set to a unique integer for each broker. +broker.id={{ broker_id }} + +############################# Socket Server Settings ############################# + +# The port the socket server listens on +port=9092 + +# Hostname the broker will bind to. If not set, the server will bind to all interfaces +#host.name=localhost + +# Hostname the broker will advertise to producers and consumers. If not set, it uses the +# value for "host.name" if configured. Otherwise, it will use the value returned from +# java.net.InetAddress.getCanonicalHostName(). +advertised.host.name={{ node.account.hostname }} + +# The port to publish to ZooKeeper for clients to use. If this is not set, +# it will publish the same port that the broker binds to. +#advertised.port= + +# The number of threads handling network requests +num.network.threads=3 + +# The number of threads doing disk I/O +num.io.threads=8 + +# The send buffer (SO_SNDBUF) used by the socket server +socket.send.buffer.bytes=102400 + +# The receive buffer (SO_RCVBUF) used by the socket server +socket.receive.buffer.bytes=65536 + +# The maximum size of a request that the socket server will accept (protection against OOM) +socket.request.max.bytes=104857600 + + +############################# Log Basics ############################# + +# A comma seperated list of directories under which to store log files +log.dirs=/mnt/kafka-logs + +# The default number of log partitions per topic. More partitions allow greater +# parallelism for consumption, but this will also result in more files across +# the brokers. +num.partitions=1 + +# The number of threads per data directory to be used for log recovery at startup and flushing at shutdown. +# This value is recommended to be increased for installations with data dirs located in RAID array. +num.recovery.threads.per.data.dir=1 + +############################# Log Flush Policy ############################# + +# Messages are immediately written to the filesystem but by default we only fsync() to sync +# the OS cache lazily. The following configurations control the flush of data to disk. +# There are a few important trade-offs here: +# 1. Durability: Unflushed data may be lost if you are not using replication. +# 2. Latency: Very large flush intervals may lead to latency spikes when the flush does occur as there will be a lot of data to flush. +# 3. Throughput: The flush is generally the most expensive operation, and a small flush interval may lead to exceessive seeks. +# The settings below allow one to configure the flush policy to flush data after a period of time or +# every N messages (or both). This can be done globally and overridden on a per-topic basis. + +# The number of messages to accept before forcing a flush of data to disk +#log.flush.interval.messages=10000 + +# The maximum amount of time a message can sit in a log before we force a flush +#log.flush.interval.ms=1000 + +############################# Log Retention Policy ############################# + +# The following configurations control the disposal of log segments. The policy can +# be set to delete segments after a period of time, or after a given size has accumulated. +# A segment will be deleted whenever *either* of these criteria are met. Deletion always happens +# from the end of the log. + +# The minimum age of a log file to be eligible for deletion +log.retention.hours=168 + +# A size-based retention policy for logs. Segments are pruned from the log as long as the remaining +# segments don't drop below log.retention.bytes. +#log.retention.bytes=1073741824 + +# The maximum size of a log segment file. When this size is reached a new log segment will be created. +log.segment.bytes=1073741824 + +# The interval at which log segments are checked to see if they can be deleted according +# to the retention policies +log.retention.check.interval.ms=300000 + +# By default the log cleaner is disabled and the log retention policy will default to just delete segments after their retention expires. +# If log.cleaner.enable=true is set the cleaner will be enabled and individual logs can then be marked for log compaction. +log.cleaner.enable=false + +############################# Zookeeper ############################# + +# Zookeeper connection string (see zookeeper docs for details). +# This is a comma separated host:port pairs, each corresponding to a zk +# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002". +# You can also append an optional chroot string to the urls to specify the +# root directory for all kafka znodes. +zookeeper.connect={{ zk.connect_setting() }} + +# Timeout in ms for connecting to zookeeper +zookeeper.connection.timeout.ms=2000 diff --git a/tests/kafkatest/services/templates/zookeeper.properties b/tests/kafkatest/services/templates/zookeeper.properties new file mode 100644 index 0000000..e66c53f --- /dev/null +++ b/tests/kafkatest/services/templates/zookeeper.properties @@ -0,0 +1,25 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# see kafka.server.KafkaConfig for additional details and defaults + +dataDir=/mnt/zookeeper +clientPort=2181 +maxClientCnxns=0 +initLimit=5 +syncLimit=2 +quorumListenOnAllIPs=true +{% for node in nodes %} +server.{{ loop.index }}={{ node.account.hostname }}:2888:3888 +{% endfor %} \ No newline at end of file diff --git a/tests/kafkatest/services/verifiable_producer.py b/tests/kafkatest/services/verifiable_producer.py new file mode 100644 index 0000000..cca8227 --- /dev/null +++ b/tests/kafkatest/services/verifiable_producer.py @@ -0,0 +1,107 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from ducktape.services.background_thread import BackgroundThreadService + +import json + + +class VerifiableProducer(BackgroundThreadService): + + logs = { + "producer_log": { + "path": "/mnt/producer.log", + "collect_default": False} + } + + def __init__(self, context, num_nodes, kafka, topic, max_messages=-1, throughput=100000): + super(VerifiableProducer, self).__init__(context, num_nodes) + + self.kafka = kafka + self.topic = topic + self.max_messages = max_messages + self.throughput = throughput + + self.acked_values = [] + self.not_acked_values = [] + + def _worker(self, idx, node): + cmd = self.start_cmd + self.logger.debug("VerifiableProducer %d command: %s" % (idx, cmd)) + + for line in node.account.ssh_capture(cmd): + line = line.strip() + + data = self.try_parse_json(line) + if data is not None: + + with self.lock: + if data["name"] == "producer_send_error": + data["node"] = idx + self.not_acked_values.append(int(data["value"])) + + elif data["name"] == "producer_send_success": + self.acked_values.append(int(data["value"])) + + @property + def start_cmd(self): + cmd = "/opt/kafka/bin/kafka-verifiable-producer.sh" \ + " --topic %s --broker-list %s" % (self.topic, self.kafka.bootstrap_servers()) + if self.max_messages > 0: + cmd += " --max-messages %s" % str(self.max_messages) + if self.throughput > 0: + cmd += " --throughput %s" % str(self.throughput) + + cmd += " 2>> /mnt/producer.log | tee -a /mnt/producer.log &" + return cmd + + @property + def acked(self): + with self.lock: + return self.acked_values + + @property + def not_acked(self): + with self.lock: + return self.not_acked_values + + @property + def num_acked(self): + with self.lock: + return len(self.acked_values) + + @property + def num_not_acked(self): + with self.lock: + return len(self.not_acked_values) + + def stop_node(self, node): + node.account.kill_process("VerifiableProducer", allow_fail=False) + # block until the corresponding thread exits + if len(self.worker_threads) >= self.idx(node): + # Need to guard this because stop is preemptively called before the worker threads are added and started + self.worker_threads[self.idx(node) - 1].join() + + def clean_node(self, node): + node.account.ssh("rm -rf /mnt/producer.log", allow_fail=False) + + def try_parse_json(self, string): + """Try to parse a string as json. Return None if not parseable.""" + try: + record = json.loads(string) + return record + except ValueError: + self.logger.debug("Could not parse as json: %s" % str(string)) + return None diff --git a/tests/kafkatest/services/zookeeper.py b/tests/kafkatest/services/zookeeper.py new file mode 100644 index 0000000..56f4606 --- /dev/null +++ b/tests/kafkatest/services/zookeeper.py @@ -0,0 +1,64 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + + +from ducktape.services.service import Service + +import time + + +class ZookeeperService(Service): + + logs = { + "zk_log": { + "path": "/mnt/zk.log", + "collect_default": True} + } + + def __init__(self, context, num_nodes): + """ + :type context + """ + super(ZookeeperService, self).__init__(context, num_nodes) + + def start_node(self, node): + idx = self.idx(node) + self.logger.info("Starting ZK node %d on %s", idx, node.account.hostname) + + node.account.ssh("mkdir -p /mnt/zookeeper") + node.account.ssh("echo %d > /mnt/zookeeper/myid" % idx) + + config_file = self.render('zookeeper.properties') + self.logger.info("zookeeper.properties:") + self.logger.info(config_file) + node.account.create_file("/mnt/zookeeper.properties", config_file) + + node.account.ssh( + "/opt/kafka/bin/zookeeper-server-start.sh /mnt/zookeeper.properties 1>> %(path)s 2>> %(path)s &" + % self.logs["zk_log"]) + + time.sleep(5) # give it some time to start + + def stop_node(self, node): + idx = self.idx(node) + self.logger.info("Stopping %s node %d on %s" % (type(self).__name__, idx, node.account.hostname)) + node.account.kill_process("zookeeper", allow_fail=False) + + def clean_node(self, node): + self.logger.info("Cleaning ZK node %d on %s", self.idx(node), node.account.hostname) + node.account.ssh("rm -rf /mnt/zookeeper /mnt/zookeeper.properties /mnt/zk.log", allow_fail=False) + + def connect_setting(self): + return ','.join([node.account.hostname + ':2181' for node in self.nodes]) diff --git a/tests/kafkatest/tests/__init__.py b/tests/kafkatest/tests/__init__.py new file mode 100644 index 0000000..ebc9bb3 --- /dev/null +++ b/tests/kafkatest/tests/__init__.py @@ -0,0 +1,15 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# see kafka.server.KafkaConfig for additional details and defaults diff --git a/tests/kafkatest/tests/benchmark_test.py b/tests/kafkatest/tests/benchmark_test.py new file mode 100644 index 0000000..b01f27b --- /dev/null +++ b/tests/kafkatest/tests/benchmark_test.py @@ -0,0 +1,274 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from ducktape.services.service import Service + +from kafkatest.tests.kafka_test import KafkaTest +from kafkatest.services.performance import ProducerPerformanceService, ConsumerPerformanceService, EndToEndLatencyService + + +class Benchmark(KafkaTest): + '''A benchmark of Kafka producer/consumer performance. This replicates the test + run here: + https://engineering.linkedin.com/kafka/benchmarking-apache-kafka-2-million-writes-second-three-cheap-machines + ''' + def __init__(self, test_context): + super(Benchmark, self).__init__(test_context, num_zk=1, num_brokers=3, topics={ + 'test-rep-one' : { 'partitions': 6, 'replication-factor': 1 }, + 'test-rep-three' : { 'partitions': 6, 'replication-factor': 3 } + }) + + if True: + # Works on both aws and local + self.msgs = 1000000 + self.msgs_default = 1000000 + else: + # Can use locally on Vagrant VMs, but may use too much memory for aws + self.msgs = 50000000 + self.msgs_default = 50000000 + + self.msgs_large = 10000000 + self.msg_size_default = 100 + self.batch_size = 8*1024 + self.buffer_memory = 64*1024*1024 + self.msg_sizes = [10, 100, 1000, 10000, 100000] + self.target_data_size = 128*1024*1024 + self.target_data_size_gb = self.target_data_size/float(1024*1024*1024) + + def test_single_producer_no_replication(self): + self.logger.info("BENCHMARK: Single producer, no replication") + self.perf = ProducerPerformanceService( + self.test_context, 1, self.kafka, + topic="test-rep-one", num_records=self.msgs_default, record_size=self.msg_size_default, throughput=-1, + settings={'acks':1, 'batch.size':self.batch_size, 'buffer.memory':self.buffer_memory} + ) + self.perf.run() + data = compute_throughput(self.perf) + self.logger.info("Single producer, no replication: %s", str(data)) + return data + + def test_single_producer_replication(self): + self.logger.info("BENCHMARK: Single producer, async 3x replication") + self.perf = ProducerPerformanceService( + self.test_context, 1, self.kafka, + topic="test-rep-three", num_records=self.msgs_default, record_size=self.msg_size_default, throughput=-1, + settings={'acks':1, 'batch.size':self.batch_size, 'buffer.memory':self.buffer_memory} + ) + self.perf.run() + data = compute_throughput(self.perf) + self.logger.info("Single producer, async 3x replication: %s" % str(data)) + return data + + def test_single_producer_sync(self): + self.logger.info("BENCHMARK: Single producer, sync 3x replication") + self.perf = ProducerPerformanceService( + self.test_context, 1, self.kafka, + topic="test-rep-three", num_records=self.msgs_default, record_size=self.msg_size_default, throughput=-1, + settings={'acks':-1, 'batch.size':self.batch_size, 'buffer.memory':self.buffer_memory} + ) + self.perf.run() + + data = compute_throughput(self.perf) + self.logger.info("Single producer, sync 3x replication: %s" % data) + return data + + def test_three_producers_async(self): + self.logger.info("BENCHMARK: Three producers, async 3x replication") + self.perf = ProducerPerformanceService( + self.test_context, 3, self.kafka, + topic="test-rep-three", num_records=self.msgs_default, record_size=self.msg_size_default, throughput=-1, + settings={'acks':1, 'batch.size':self.batch_size, 'buffer.memory':self.buffer_memory} + ) + self.perf.run() + + data = compute_throughput(self.perf) + self.logger.info("Three producers, async 3x replication: %s" % data) + return data + + def test_multiple_message_size(self): + # TODO this would be a great place to use parametrization + self.perfs = {} + for msg_size in self.msg_sizes: + self.logger.info("BENCHMARK: Message size %d (%f GB total, single producer, async 3x replication)", msg_size, self.target_data_size_gb) + # Always generate the same total amount of data + nrecords = int(self.target_data_size / msg_size) + self.perfs["perf-" + str(msg_size)] = ProducerPerformanceService( + self.test_context, 1, self.kafka, + topic="test-rep-three", num_records=nrecords, record_size=msg_size, throughput=-1, + settings={'acks': 1, 'batch.size': self.batch_size, 'buffer.memory': self.buffer_memory} + ) + + self.msg_size_perf = {} + for msg_size in self.msg_sizes: + perf = self.perfs["perf-" + str(msg_size)] + perf.run() + self.msg_size_perf[msg_size] = perf + + summary = ["Message size:"] + data = {} + for msg_size in self.msg_sizes: + datum = compute_throughput(self.msg_size_perf[msg_size]) + summary.append(" %d: %s" % (msg_size, datum)) + data[msg_size] = datum + self.logger.info("\n".join(summary)) + return data + + def test_long_term_throughput(self): + self.logger.info("BENCHMARK: Long production") + self.perf = ProducerPerformanceService( + self.test_context, 1, self.kafka, + topic="test-rep-three", num_records=self.msgs_large, record_size=self.msg_size_default, throughput=-1, + settings={'acks':1, 'batch.size':self.batch_size, 'buffer.memory':self.buffer_memory}, + intermediate_stats=True + ) + self.perf.run() + + summary = ["Throughput over long run, data > memory:"] + data = {} + # FIXME we should be generating a graph too + # Try to break it into 5 blocks, but fall back to a smaller number if + # there aren't even 5 elements + block_size = max(len(self.perf.stats[0]) / 5, 1) + nblocks = len(self.perf.stats[0]) / block_size + for i in range(nblocks): + subset = self.perf.stats[0][i*block_size:min((i+1)*block_size, len(self.perf.stats[0]))] + if len(subset) == 0: + summary.append(" Time block %d: (empty)" % i) + data[i] = None + else: + records_per_sec = sum([stat['records_per_sec'] for stat in subset])/float(len(subset)) + mb_per_sec = sum([stat['mbps'] for stat in subset])/float(len(subset)) + + summary.append(" Time block %d: %f rec/sec (%f MB/s)" % (i, records_per_sec, mb_per_sec)) + data[i] = throughput(records_per_sec, mb_per_sec) + + self.logger.info("\n".join(summary)) + return data + + def test_end_to_end_latency(self): + self.logger.info("BENCHMARK: End to end latency") + self.perf = EndToEndLatencyService( + self.test_context, 1, self.kafka, + topic="test-rep-three", num_records=10000 + ) + self.perf.run() + + data = latency(self.perf.results[0]['latency_50th_ms'], self.perf.results[0]['latency_99th_ms'], self.perf.results[0]['latency_999th_ms']) + self.logger.info("End-to-end latency: %s" % str(data)) + return data + + def test_producer_and_consumer(self): + self.logger.info("BENCHMARK: Producer + Consumer") + self.producer = ProducerPerformanceService( + self.test_context, 1, self.kafka, + topic="test-rep-three", num_records=self.msgs_default, record_size=self.msg_size_default, throughput=-1, + settings={'acks':1, 'batch.size':self.batch_size, 'buffer.memory':self.buffer_memory} + ) + + self.consumer = ConsumerPerformanceService( + self.test_context, 1, self.kafka, + topic="test-rep-three", num_records=self.msgs_default, throughput=-1, threads=1 + ) + + Service.run_parallel(self.producer, self.consumer) + + data = { + "producer": compute_throughput(self.producer), + "consumer": compute_throughput(self.consumer) + } + summary = [ + "Producer + consumer:", + str(data)] + self.logger.info("\n".join(summary)) + return data + + def test_single_consumer(self): + topic = "test-rep-three" + + self.producer = ProducerPerformanceService( + self.test_context, 1, self.kafka, + topic=topic, num_records=self.msgs_default, record_size=self.msg_size_default, throughput=-1, + settings={'acks':1, 'batch.size':self.batch_size, 'buffer.memory':self.buffer_memory} + ) + self.producer.run() + + # All consumer tests use the messages from the first benchmark, so + # they'll get messages of the default message size + self.logger.info("BENCHMARK: Single consumer") + self.perf = ConsumerPerformanceService( + self.test_context, 1, self.kafka, + topic=topic, num_records=self.msgs_default, throughput=-1, threads=1 + ) + self.perf.run() + + data = compute_throughput(self.perf) + self.logger.info("Single consumer: %s" % data) + return data + + def test_three_consumers(self): + topic = "test-rep-three" + + self.producer = ProducerPerformanceService( + self.test_context, 1, self.kafka, + topic=topic, num_records=self.msgs_default, record_size=self.msg_size_default, throughput=-1, + settings={'acks':1, 'batch.size':self.batch_size, 'buffer.memory':self.buffer_memory} + ) + self.producer.run() + + self.logger.info("BENCHMARK: Three consumers") + self.perf = ConsumerPerformanceService( + self.test_context, 3, self.kafka, + topic="test-rep-three", num_records=self.msgs_default, throughput=-1, threads=1 + ) + self.perf.run() + + data = compute_throughput(self.perf) + self.logger.info("Three consumers: %s", data) + return data + + +def throughput(records_per_sec, mb_per_sec): + """Helper method to ensure uniform representation of throughput data""" + return { + "records_per_sec": records_per_sec, + "mb_per_sec": mb_per_sec + } + + +def latency(latency_50th_ms, latency_99th_ms, latency_999th_ms): + """Helper method to ensure uniform representation of latency data""" + return { + "latency_50th_ms": latency_50th_ms, + "latency_99th_ms": latency_99th_ms, + "latency_999th_ms": latency_999th_ms + } + + +def compute_throughput(perf): + """Helper method for computing throughput after running a performance service.""" + aggregate_rate = sum([r['records_per_sec'] for r in perf.results]) + aggregate_mbps = sum([r['mbps'] for r in perf.results]) + + return throughput(aggregate_rate, aggregate_mbps) + + + + + + + + + + diff --git a/tests/kafkatest/tests/kafka_test.py b/tests/kafkatest/tests/kafka_test.py new file mode 100644 index 0000000..7118721 --- /dev/null +++ b/tests/kafkatest/tests/kafka_test.py @@ -0,0 +1,45 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from ducktape.tests.test import Test + + +from kafkatest.services.zookeeper import ZookeeperService +from kafkatest.services.kafka import KafkaService + + +class KafkaTest(Test): + """ + Helper class that manages setting up a Kafka cluster. Use this if the + default settings for Kafka are sufficient for your test; any customization + needs to be done manually. Your run() method should call tearDown and + setUp. The Zookeeper and Kafka services are available as the fields + KafkaTest.zk and KafkaTest.kafka. + """ + def __init__(self, test_context, num_zk, num_brokers, topics=None): + super(KafkaTest, self).__init__(test_context) + self.num_zk = num_zk + self.num_brokers = num_brokers + self.topics = topics + + self.zk = ZookeeperService(test_context, self.num_zk) + + self.kafka = KafkaService( + test_context, self.num_brokers, + self.zk, topics=self.topics) + + def setUp(self): + self.zk.start() + self.kafka.start() \ No newline at end of file diff --git a/tests/kafkatest/tests/replication_test.py b/tests/kafkatest/tests/replication_test.py new file mode 100644 index 0000000..fed1ea1 --- /dev/null +++ b/tests/kafkatest/tests/replication_test.py @@ -0,0 +1,165 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from ducktape.tests.test import Test +from ducktape.utils.util import wait_until + +from kafkatest.services.zookeeper import ZookeeperService +from kafkatest.services.kafka import KafkaService +from kafkatest.services.verifiable_producer import VerifiableProducer +from kafkatest.services.console_consumer import ConsoleConsumer + +import signal +import time + + +class ReplicationTest(Test): + """Replication tests. + These tests verify that replication provides simple durability guarantees by checking that data acked by + brokers is still available for consumption in the face of various failure scenarios.""" + + def __init__(self, test_context): + """:type test_context: ducktape.tests.test.TestContext""" + super(ReplicationTest, self).__init__(test_context=test_context) + + self.topic = "test_topic" + self.zk = ZookeeperService(test_context, num_nodes=1) + self.kafka = KafkaService(test_context, num_nodes=3, zk=self.zk, topics={self.topic: { + "partitions": 3, + "replication-factor": 3, + "min.insync.replicas": 2} + }) + self.producer_throughput = 10000 + self.num_producers = 1 + self.num_consumers = 1 + + def setUp(self): + self.zk.start() + self.kafka.start() + + def min_cluster_size(self): + """Override this since we're adding services outside of the constructor""" + return super(ReplicationTest, self).min_cluster_size() + self.num_producers + self.num_consumers + + def run_with_failure(self, failure): + """This is the top-level test template. + + The steps are: + Produce messages in the background while driving some failure condition + When done driving failures, immediately stop producing + Consume all messages + Validate that messages acked by brokers were consumed + + Note that consuming is a bit tricky, at least with console consumer. The goal is to consume all messages + (foreach partition) in the topic. In this case, waiting for the last message may cause the consumer to stop + too soon since console consumer is consuming multiple partitions from a single thread and therefore we lose + ordering guarantees. + + Waiting on a count of consumed messages can be unreliable: if we stop consuming when num_consumed == num_acked, + we might exit early if some messages are duplicated (though not an issue here since producer retries==0) + + Therefore rely here on the consumer.timeout.ms setting which times out on the interval between successively + consumed messages. Since we run the producer to completion before running the consumer, this is a reliable + indicator that nothing is left to consume. + + """ + self.producer = VerifiableProducer(self.test_context, self.num_producers, self.kafka, self.topic, throughput=self.producer_throughput) + self.consumer = ConsoleConsumer(self.test_context, self.num_consumers, self.kafka, self.topic, consumer_timeout_ms=3000) + + # Produce in a background thread while driving broker failures + self.producer.start() + if not wait_until(lambda: self.producer.num_acked > 5, timeout_sec=5): + raise RuntimeError("Producer failed to start in a reasonable amount of time.") + failure() + self.producer.stop() + + self.acked = self.producer.acked + self.not_acked = self.producer.not_acked + self.logger.info("num not acked: %d" % self.producer.num_not_acked) + self.logger.info("num acked: %d" % self.producer.num_acked) + + # Consume all messages + self.consumer.start() + self.consumer.wait() + self.consumed = self.consumer.messages_consumed[1] + self.logger.info("num consumed: %d" % len(self.consumed)) + + # Check produced vs consumed + success, msg = self.validate() + + if not success: + self.mark_for_collect(self.producer) + + assert success, msg + + def clean_shutdown(self): + """Discover leader node for our topic and shut it down cleanly.""" + self.kafka.signal_leader(self.topic, partition=0, sig=signal.SIGTERM) + + def hard_shutdown(self): + """Discover leader node for our topic and shut it down with a hard kill.""" + self.kafka.signal_leader(self.topic, partition=0, sig=signal.SIGKILL) + + def clean_bounce(self): + """Chase the leader of one partition and restart it cleanly.""" + for i in range(5): + prev_leader_node = self.kafka.leader(topic=self.topic, partition=0) + self.kafka.restart_node(prev_leader_node, wait_sec=5, clean_shutdown=True) + + def hard_bounce(self): + """Chase the leader and restart it cleanly.""" + for i in range(5): + prev_leader_node = self.kafka.leader(topic=self.topic, partition=0) + self.kafka.restart_node(prev_leader_node, wait_sec=5, clean_shutdown=False) + + # Wait long enough for previous leader to probably be awake again + time.sleep(6) + + def validate(self): + """Check that produced messages were consumed.""" + + success = True + msg = "" + + if len(set(self.consumed)) != len(self.consumed): + # There are duplicates. This is ok, so report it but don't fail the test + msg += "There are duplicate messages in the log\n" + + if not set(self.consumed).issuperset(set(self.acked)): + # Every acked message must appear in the logs. I.e. consumed messages must be superset of acked messages. + acked_minus_consumed = set(self.producer.acked) - set(self.consumed) + success = False + msg += "At least one acked message did not appear in the consumed messages. acked_minus_consumed: " + str(acked_minus_consumed) + + if not success: + # Collect all the data logs if there was a failure + self.mark_for_collect(self.kafka) + + return success, msg + + def test_clean_shutdown(self): + self.run_with_failure(self.clean_shutdown) + + def test_hard_shutdown(self): + self.run_with_failure(self.hard_shutdown) + + def test_clean_bounce(self): + self.run_with_failure(self.clean_bounce) + + def test_hard_bounce(self): + self.run_with_failure(self.hard_bounce) + + + diff --git a/tests/setup.py b/tests/setup.py new file mode 100644 index 0000000..5ce4bb7 --- /dev/null +++ b/tests/setup.py @@ -0,0 +1,27 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# see kafka.server.KafkaConfig for additional details and defaults + +from setuptools import find_packages, setup + +setup(name="kafkatest", + version="0.8.3-SNAPSHOT", + description="Apache Kafka System Tests", + author="Apache Kafka", + platforms=["any"], + license="apache2.0", + packages=find_packages(), + requires=["ducktape(>=0.2.0)"] + ) diff --git a/tools/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java b/tools/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java new file mode 100644 index 0000000..fd31c1a --- /dev/null +++ b/tools/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java @@ -0,0 +1,201 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package org.apache.kafka.clients.tools; + +import java.util.Arrays; +import java.util.Properties; + +import org.apache.kafka.clients.producer.*; + +public class ProducerPerformance { + + public static void main(String[] args) throws Exception { + if (args.length < 4) { + System.err.println("USAGE: java " + ProducerPerformance.class.getName() + + " topic_name num_records record_size target_records_sec [prop_name=prop_value]*"); + System.exit(1); + } + + /* parse args */ + String topicName = args[0]; + long numRecords = Long.parseLong(args[1]); + int recordSize = Integer.parseInt(args[2]); + int throughput = Integer.parseInt(args[3]); + + Properties props = new Properties(); + for (int i = 4; i < args.length; i++) { + String[] pieces = args[i].split("="); + if (pieces.length != 2) + throw new IllegalArgumentException("Invalid property: " + args[i]); + props.put(pieces[0], pieces[1]); + } + props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); + props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); + KafkaProducer producer = new KafkaProducer(props); + + /* setup perf test */ + byte[] payload = new byte[recordSize]; + Arrays.fill(payload, (byte) 1); + ProducerRecord record = new ProducerRecord(topicName, payload); + Stats stats = new Stats(numRecords, 5000); + long startMs = System.currentTimeMillis(); + + ThroughputThrottler throttler = new ThroughputThrottler(throughput, startMs); + for (int i = 0; i < numRecords; i++) { + long sendStartMs = System.currentTimeMillis(); + Callback cb = stats.nextCompletion(sendStartMs, payload.length, stats); + producer.send(record, cb); + + if (throttler.shouldThrottle(i, sendStartMs)) { + throttler.throttle(); + } + } + + /* print final results */ + producer.close(); + stats.printTotal(); + } + + private static class Stats { + private long start; + private long windowStart; + private int[] latencies; + private int sampling; + private int iteration; + private int index; + private long count; + private long bytes; + private int maxLatency; + private long totalLatency; + private long windowCount; + private int windowMaxLatency; + private long windowTotalLatency; + private long windowBytes; + private long reportingInterval; + + public Stats(long numRecords, int reportingInterval) { + this.start = System.currentTimeMillis(); + this.windowStart = System.currentTimeMillis(); + this.index = 0; + this.iteration = 0; + this.sampling = (int) (numRecords / Math.min(numRecords, 500000)); + this.latencies = new int[(int) (numRecords / this.sampling) + 1]; + this.index = 0; + this.maxLatency = 0; + this.totalLatency = 0; + this.windowCount = 0; + this.windowMaxLatency = 0; + this.windowTotalLatency = 0; + this.windowBytes = 0; + this.totalLatency = 0; + this.reportingInterval = reportingInterval; + } + + public void record(int iter, int latency, int bytes, long time) { + this.count++; + this.bytes += bytes; + this.totalLatency += latency; + this.maxLatency = Math.max(this.maxLatency, latency); + this.windowCount++; + this.windowBytes += bytes; + this.windowTotalLatency += latency; + this.windowMaxLatency = Math.max(windowMaxLatency, latency); + if (iter % this.sampling == 0) { + this.latencies[index] = latency; + this.index++; + } + /* maybe report the recent perf */ + if (time - windowStart >= reportingInterval) { + printWindow(); + newWindow(); + } + } + + public Callback nextCompletion(long start, int bytes, Stats stats) { + Callback cb = new PerfCallback(this.iteration, start, bytes, stats); + this.iteration++; + return cb; + } + + public void printWindow() { + long ellapsed = System.currentTimeMillis() - windowStart; + double recsPerSec = 1000.0 * windowCount / (double) ellapsed; + double mbPerSec = 1000.0 * this.windowBytes / (double) ellapsed / (1024.0 * 1024.0); + System.out.printf("%d records sent, %.1f records/sec (%.2f MB/sec), %.1f ms avg latency, %.1f max latency.\n", + windowCount, + recsPerSec, + mbPerSec, + windowTotalLatency / (double) windowCount, + (double) windowMaxLatency); + } + + public void newWindow() { + this.windowStart = System.currentTimeMillis(); + this.windowCount = 0; + this.windowMaxLatency = 0; + this.windowTotalLatency = 0; + this.windowBytes = 0; + } + + public void printTotal() { + long ellapsed = System.currentTimeMillis() - start; + double recsPerSec = 1000.0 * count / (double) ellapsed; + double mbPerSec = 1000.0 * this.bytes / (double) ellapsed / (1024.0 * 1024.0); + int[] percs = percentiles(this.latencies, index, 0.5, 0.95, 0.99, 0.999); + System.out.printf("%d records sent, %f records/sec (%.2f MB/sec), %.2f ms avg latency, %.2f ms max latency, %d ms 50th, %d ms 95th, %d ms 99th, %d ms 99.9th.\n", + count, + recsPerSec, + mbPerSec, + totalLatency / (double) count, + (double) maxLatency, + percs[0], + percs[1], + percs[2], + percs[3]); + } + + private static int[] percentiles(int[] latencies, int count, double... percentiles) { + int size = Math.min(count, latencies.length); + Arrays.sort(latencies, 0, size); + int[] values = new int[percentiles.length]; + for (int i = 0; i < percentiles.length; i++) { + int index = (int) (percentiles[i] * size); + values[i] = latencies[index]; + } + return values; + } + } + + private static final class PerfCallback implements Callback { + private final long start; + private final int iteration; + private final int bytes; + private final Stats stats; + + public PerfCallback(int iter, long start, int bytes, Stats stats) { + this.start = start; + this.stats = stats; + this.iteration = iter; + this.bytes = bytes; + } + + public void onCompletion(RecordMetadata metadata, Exception exception) { + long now = System.currentTimeMillis(); + int latency = (int) (now - start); + this.stats.record(iteration, latency, bytes, now); + if (exception != null) + exception.printStackTrace(); + } + } + +} diff --git a/tools/src/main/java/org/apache/kafka/clients/tools/ThroughputThrottler.java b/tools/src/main/java/org/apache/kafka/clients/tools/ThroughputThrottler.java new file mode 100644 index 0000000..06c443f --- /dev/null +++ b/tools/src/main/java/org/apache/kafka/clients/tools/ThroughputThrottler.java @@ -0,0 +1,118 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.clients.tools; + + +/** + * This class helps producers throttle throughput. + * + * If targetThroughput >= 0, the resulting average throughput will be approximately + * min(targetThroughput, maximumPossibleThroughput). If targetThroughput < 0, + * no throttling will occur. + * + * To use, do this between successive send attempts: + *
    + *     {@code     
    + *      if (throttler.shouldThrottle(...)) {
    + *          throttler.throttle();
    + *      } 
    + *     } 
    + * 
    + * + * Note that this can be used to throttle message throughput or data throughput. + */ +public class ThroughputThrottler { + + private static final long NS_PER_MS = 1000000L; + private static final long NS_PER_SEC = 1000 * NS_PER_MS; + private static final long MIN_SLEEP_NS = 2 * NS_PER_MS; + + long sleepTimeNs; + long sleepDeficitNs = 0; + long targetThroughput = -1; + long startMs; + + /** + * @param targetThroughput Can be messages/sec or bytes/sec + * @param startMs When the very first message is sent + */ + public ThroughputThrottler(long targetThroughput, long startMs) { + this.startMs = startMs; + this.targetThroughput = targetThroughput; + this.sleepTimeNs = targetThroughput > 0 ? + NS_PER_SEC / targetThroughput : + Long.MAX_VALUE; + } + + /** + * @param amountSoFar bytes produced so far if you want to throttle data throughput, or + * messages produced so far if you want to throttle message throughput. + * @param sendStartMs timestamp of the most recently sent message + * @return + */ + public boolean shouldThrottle(long amountSoFar, long sendStartMs) { + if (this.targetThroughput < 0) { + // No throttling in this case + return false; + } + + float elapsedMs = (sendStartMs - startMs) / 1000.f; + return elapsedMs > 0 && (amountSoFar / elapsedMs) > this.targetThroughput; + } + + /** + * Occasionally blocks for small amounts of time to achieve targetThroughput. + * + * Note that if targetThroughput is 0, this will block extremely aggressively. + */ + public void throttle() { + if (targetThroughput == 0) { + try { + Thread.sleep(Long.MAX_VALUE); + } catch (InterruptedException e) { + // do nothing + } + return; + } + + // throttle throughput by sleeping, on average, + // (1 / this.throughput) seconds between "things sent" + sleepDeficitNs += sleepTimeNs; + + // If enough sleep deficit has accumulated, sleep a little + if (sleepDeficitNs >= MIN_SLEEP_NS) { + long sleepMs = sleepDeficitNs / 1000000; + long sleepNs = sleepDeficitNs - sleepMs * 1000000; + + long sleepStartNs = System.nanoTime(); + try { + Thread.sleep(sleepMs, (int) sleepNs); + sleepDeficitNs = 0; + } catch (InterruptedException e) { + // If sleep is cut short, reduce deficit by the amount of + // time we actually spent sleeping + long sleepElapsedNs = System.nanoTime() - sleepStartNs; + if (sleepElapsedNs <= sleepDeficitNs) { + sleepDeficitNs -= sleepElapsedNs; + } + } + } + } +} + + \ No newline at end of file diff --git a/tools/src/main/java/org/apache/kafka/clients/tools/VerifiableProducer.java b/tools/src/main/java/org/apache/kafka/clients/tools/VerifiableProducer.java new file mode 100644 index 0000000..b04876f --- /dev/null +++ b/tools/src/main/java/org/apache/kafka/clients/tools/VerifiableProducer.java @@ -0,0 +1,307 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.clients.tools; + +import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +import static net.sourceforge.argparse4j.impl.Arguments.store; + +import net.sourceforge.argparse4j.ArgumentParsers; +import net.sourceforge.argparse4j.inf.ArgumentParser; +import net.sourceforge.argparse4j.inf.ArgumentParserException; +import net.sourceforge.argparse4j.inf.Namespace; + +/** + * Primarily intended for use with system testing, this producer prints metadata + * in the form of JSON to stdout on each "send" request. For example, this helps + * with end-to-end correctness tests by making externally visible which messages have been + * acked and which have not. + * + * When used as a command-line tool, it produces increasing integers. It will produce a + * fixed number of messages unless the default max-messages -1 is used, in which case + * it produces indefinitely. + * + * If logging is left enabled, log output on stdout can be easily ignored by checking + * whether a given line is valid JSON. + */ +public class VerifiableProducer { + + String topic; + private Producer producer; + // If maxMessages < 0, produce until the process is killed externally + private long maxMessages = -1; + + // Number of messages for which acks were received + private long numAcked = 0; + + // Number of send attempts + private long numSent = 0; + + // Throttle message throughput if this is set >= 0 + private long throughput; + + // Hook to trigger producing thread to stop sending messages + private boolean stopProducing = false; + + public VerifiableProducer( + Properties producerProps, String topic, int throughput, int maxMessages) { + + this.topic = topic; + this.throughput = throughput; + this.maxMessages = maxMessages; + this.producer = new KafkaProducer(producerProps); + } + + /** Get the command-line argument parser. */ + private static ArgumentParser argParser() { + ArgumentParser parser = ArgumentParsers + .newArgumentParser("verifiable-producer") + .defaultHelp(true) + .description("This tool produces increasing integers to the specified topic and prints JSON metadata to stdout on each \"send\" request, making externally visible which messages have been acked and which have not."); + + parser.addArgument("--topic") + .action(store()) + .required(true) + .type(String.class) + .metavar("TOPIC") + .help("Produce messages to this topic."); + + parser.addArgument("--broker-list") + .action(store()) + .required(true) + .type(String.class) + .metavar("HOST1:PORT1[,HOST2:PORT2[...]]") + .dest("brokerList") + .help("Comma-separated list of Kafka brokers in the form HOST1:PORT1,HOST2:PORT2,..."); + + parser.addArgument("--max-messages") + .action(store()) + .required(false) + .setDefault(-1) + .type(Integer.class) + .metavar("MAX-MESSAGES") + .dest("maxMessages") + .help("Produce this many messages. If -1, produce messages until the process is killed externally."); + + parser.addArgument("--throughput") + .action(store()) + .required(false) + .setDefault(-1) + .type(Integer.class) + .metavar("THROUGHPUT") + .help("If set >= 0, throttle maximum message throughput to *approximately* THROUGHPUT messages/sec."); + + parser.addArgument("--acks") + .action(store()) + .required(false) + .setDefault(-1) + .type(Integer.class) + .choices(0, 1, -1) + .metavar("ACKS") + .help("Acks required on each produced message. See Kafka docs on request.required.acks for details."); + + return parser; + } + + /** Construct a VerifiableProducer object from command-line arguments. */ + public static VerifiableProducer createFromArgs(String[] args) { + ArgumentParser parser = argParser(); + VerifiableProducer producer = null; + + try { + Namespace res; + res = parser.parseArgs(args); + + int maxMessages = res.getInt("maxMessages"); + String topic = res.getString("topic"); + int throughput = res.getInt("throughput"); + + Properties producerProps = new Properties(); + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, res.getString("brokerList")); + producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, + "org.apache.kafka.common.serialization.StringSerializer"); + producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, + "org.apache.kafka.common.serialization.StringSerializer"); + producerProps.put(ProducerConfig.ACKS_CONFIG, Integer.toString(res.getInt("acks"))); + // No producer retries + producerProps.put("retries", "0"); + + producer = new VerifiableProducer(producerProps, topic, throughput, maxMessages); + } catch (ArgumentParserException e) { + if (args.length == 0) { + parser.printHelp(); + System.exit(0); + } else { + parser.handleError(e); + System.exit(1); + } + } + + return producer; + } + + /** Produce a message with given key and value. */ + public void send(String key, String value) { + ProducerRecord record = new ProducerRecord(topic, key, value); + numSent++; + try { + producer.send(record, new PrintInfoCallback(key, value)); + } catch (Exception e) { + + synchronized (System.out) { + System.out.println(errorString(e, key, value, System.currentTimeMillis())); + } + } + } + + /** Close the producer to flush any remaining messages. */ + public void close() { + producer.close(); + } + + /** + * Return JSON string encapsulating basic information about the exception, as well + * as the key and value which triggered the exception. + */ + String errorString(Exception e, String key, String value, Long nowMs) { + assert e != null : "Expected non-null exception."; + + Map errorData = new HashMap<>(); + errorData.put("class", this.getClass().toString()); + errorData.put("name", "producer_send_error"); + + errorData.put("time_ms", nowMs); + errorData.put("exception", e.getClass().toString()); + errorData.put("message", e.getMessage()); + errorData.put("topic", this.topic); + errorData.put("key", key); + errorData.put("value", value); + + return toJsonString(errorData); + } + + String successString(RecordMetadata recordMetadata, String key, String value, Long nowMs) { + assert recordMetadata != null : "Expected non-null recordMetadata object."; + + Map successData = new HashMap<>(); + successData.put("class", this.getClass().toString()); + successData.put("name", "producer_send_success"); + + successData.put("time_ms", nowMs); + successData.put("topic", this.topic); + successData.put("partition", recordMetadata.partition()); + successData.put("offset", recordMetadata.offset()); + successData.put("key", key); + successData.put("value", value); + + return toJsonString(successData); + } + + private String toJsonString(Map data) { + String json; + try { + ObjectMapper mapper = new ObjectMapper(); + json = mapper.writeValueAsString(data); + } catch (JsonProcessingException e) { + json = "Bad data can't be written as json: " + e.getMessage(); + } + return json; + } + + /** Callback which prints errors to stdout when the producer fails to send. */ + private class PrintInfoCallback implements Callback { + + private String key; + private String value; + + PrintInfoCallback(String key, String value) { + this.key = key; + this.value = value; + } + + public void onCompletion(RecordMetadata recordMetadata, Exception e) { + synchronized (System.out) { + if (e == null) { + VerifiableProducer.this.numAcked++; + System.out.println(successString(recordMetadata, this.key, this.value, System.currentTimeMillis())); + } else { + System.out.println(errorString(e, this.key, this.value, System.currentTimeMillis())); + } + } + } + } + + public static void main(String[] args) throws IOException { + + final VerifiableProducer producer = createFromArgs(args); + final long startMs = System.currentTimeMillis(); + boolean infinite = producer.maxMessages < 0; + + Runtime.getRuntime().addShutdownHook(new Thread() { + @Override + public void run() { + // Trigger main thread to stop producing messages + producer.stopProducing = true; + + // Flush any remaining messages + producer.close(); + + // Print a summary + long stopMs = System.currentTimeMillis(); + double avgThroughput = 1000 * ((producer.numAcked) / (double) (stopMs - startMs)); + + Map data = new HashMap<>(); + data.put("class", producer.getClass().toString()); + data.put("name", "tool_data"); + data.put("sent", producer.numSent); + data.put("acked", producer.numAcked); + data.put("target_throughput", producer.throughput); + data.put("avg_throughput", avgThroughput); + + System.out.println(producer.toJsonString(data)); + } + }); + + ThroughputThrottler throttler = new ThroughputThrottler(producer.throughput, startMs); + for (int i = 0; i < producer.maxMessages || infinite; i++) { + if (producer.stopProducing) { + break; + } + long sendStartMs = System.currentTimeMillis(); + producer.send(null, String.format("%d", i)); + + if (throttler.shouldThrottle(i, sendStartMs)) { + throttler.throttle(); + } + } + } + +} diff --git a/vagrant/aws/aws-access-keys-commands b/vagrant/aws/aws-access-keys-commands new file mode 100644 index 0000000..9c923f8 --- /dev/null +++ b/vagrant/aws/aws-access-keys-commands @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# see kafka.server.KafkaConfig for additional details and defaults + +if [ -z "$AWS_IAM" ];then + echo "Warning: AWS_IAM is not set" +fi + +export AWS_ACCESS_KEY=`curl -s http://169.254.169.254/latest/meta-data/iam/security-credentials/$AWS_IAM | grep AccessKeyId | awk -F\" '{ print $4 }'` +export AWS_SECRET_KEY=`curl -s http://169.254.169.254/latest/meta-data/iam/security-credentials/$AWS_IAM | grep SecretAccessKey | awk -F\" '{ print $4 }'` +export AWS_SESSION_TOKEN=`curl -s http://169.254.169.254/latest/meta-data/iam/security-credentials/$AWS_IAM | grep Token | awk -F\" '{ print $4 }'` + +if [ -z "$AWS_ACCESS_KEY" ]; then + echo "Failed to populate environment variables AWS_ACCESS_KEY, AWS_SECRET_KEY, and AWS_SESSION_TOKEN." + echo "AWS_IAM is currently $AWS_IAM. Double-check that this is correct. If not set, add this command to your .bashrc file:" + echo "export AWS_IAM= # put this into your ~/.bashrc" +fi diff --git a/vagrant/aws/aws-example-Vagrantfile.local b/vagrant/aws/aws-example-Vagrantfile.local new file mode 100644 index 0000000..c3b075b --- /dev/null +++ b/vagrant/aws/aws-example-Vagrantfile.local @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# see kafka.server.KafkaConfig for additional details and defaults + +# Use this template Vagrantfile.local for running system tests on aws +# To use it, move it to the base kafka directory and rename +# it to Vagrantfile.local, and adjust variables as needed. +ec3_instance_type = "m3.medium" +num_zookeepers = 0 +num_brokers = 0 +num_workers = 9 +ec2_keypair_name = kafkatest +ec2_keypair_file = ../kafkatest.pem +ec2_security_groups = ['kafkatest'] +ec2_region = 'us-west-2' +ec2_ami = "ami-29ebb519" diff --git a/vagrant/aws/aws-init.sh b/vagrant/aws/aws-init.sh new file mode 100755 index 0000000..6151928 --- /dev/null +++ b/vagrant/aws/aws-init.sh @@ -0,0 +1,73 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# see kafka.server.KafkaConfig for additional details and defaults + +#!/bin/bash + +# This script can be used to set up a driver machine on aws from which you will run tests +# or bring up your mini Kafka cluster. + +# Install dependencies +sudo apt-get install -y maven openjdk-6-jdk build-essential \ + ruby-dev zlib1g-dev realpath python-setuptools + +base_dir=`dirname $0`/../.. + +if [ -z `which vagrant` ]; then + echo "Installing vagrant..." + wget https://dl.bintray.com/mitchellh/vagrant/vagrant_1.7.2_x86_64.deb + sudo dpkg -i vagrant_1.7.2_x86_64.deb + rm -f vagrant_1.7.2_x86_64.deb +fi + +# Install necessary vagrant plugins +# Note: Do NOT install vagrant-cachier since it doesn't work on AWS and only +# adds log noise +vagrant_plugins="vagrant-aws vagrant-hostmanager" +existing=`vagrant plugin list` +for plugin in $vagrant_plugins; do + echo $existing | grep $plugin > /dev/null + if [ $? != 0 ]; then + vagrant plugin install $plugin + fi +done + +# Create Vagrantfile.local as a convenience +if [ ! -e "$base_dir/Vagrantfile.local" ]; then + cp $base_dir/aws/aws-example-Vagrantfile.local $base_dir/Vagrantfile.local +fi + +gradle="gradle-2.2.1" +if [ -z `which gradle` ] && [ ! -d $base_dir/$gradle ]; then + if [ ! -e $gradle-bin.zip ]; then + wget https://services.gradle.org/distributions/$gradle-bin.zip + fi + unzip $gradle-bin.zip + rm -rf $gradle-bin.zip + mv $gradle $base_dir/$gradle +fi + +# Ensure aws access keys are in the environment when we use a EC2 driver machine +LOCAL_HOSTNAME=$(hostname -d) +if [[ ${LOCAL_HOSTNAME} =~ .*\.compute\.internal ]]; then + grep "AWS ACCESS KEYS" ~/.bashrc > /dev/null + if [ $? != 0 ]; then + echo "# --- AWS ACCESS KEYS ---" >> ~/.bashrc + echo ". `realpath $base_dir/aws/aws-access-keys-commands`" >> ~/.bashrc + echo "# -----------------------" >> ~/.bashrc + source ~/.bashrc + fi +fi + diff --git a/vagrant/base.sh b/vagrant/base.sh index 6f28dfe..133f10a 100644 --- a/vagrant/base.sh +++ b/vagrant/base.sh @@ -41,3 +41,12 @@ chmod a+rw /opt if [ ! -e /opt/kafka ]; then ln -s /vagrant /opt/kafka fi + +# For EC2 nodes, we want to use /mnt, which should have the local disk. On local +# VMs, we can just create it if it doesn't exist and use it like we'd use +# /tmp. Eventually, we'd like to also support more directories, e.g. when EC2 +# instances have multiple local disks. +if [ ! -e /mnt ]; then + mkdir /mnt +fi +chmod a+rwx /mnt diff --git a/vagrant/system-test-Vagrantfile.local b/vagrant/system-test-Vagrantfile.local new file mode 100644 index 0000000..7f280a4 --- /dev/null +++ b/vagrant/system-test-Vagrantfile.local @@ -0,0 +1,22 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# see kafka.server.KafkaConfig for additional details and defaults + +# Use this example Vagrantfile.local for running system tests +# To use it, move it to the base kafka directory and rename +# it to Vagrantfile.local +num_zookeepers = 0 +num_brokers = 0 +num_workers = 9 -- 1.7.12.4 From b7bd2978dc3947297fefc06ff9b22949d5bd1b50 Mon Sep 17 00:00:00 2001 From: David Jacot Date: Wed, 29 Jul 2015 10:34:42 -0700 Subject: [PATCH 074/120] KAFKA-2100; Client Error doesn't preserve or display original server error code when it is an unknown code; Reviewed by Gwen, Guozhang and Ewen --- .../main/java/org/apache/kafka/common/protocol/Errors.java | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java index d6c41c1..e17e390 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java @@ -20,6 +20,8 @@ import java.util.HashMap; import java.util.Map; import org.apache.kafka.common.errors.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * This class contains all the client-server errors--those errors that must be sent from the server to the client. These @@ -83,6 +85,8 @@ public enum Errors { INVALID_COMMIT_OFFSET_SIZE(28, new ApiException("The committing offset data size is not valid")); + private static final Logger log = LoggerFactory.getLogger(Errors.class); + private static Map, Errors> classToError = new HashMap, Errors>(); private static Map codeToError = new HashMap(); @@ -130,11 +134,16 @@ public enum Errors { */ public static Errors forCode(short code) { Errors error = codeToError.get(code); - return error == null ? UNKNOWN : error; + if (error != null) { + return error; + } else { + log.warn("Unexpected error code: {}.", code); + return UNKNOWN; + } } /** - * Return the error instance associated with this exception (or UKNOWN if there is none) + * Return the error instance associated with this exception (or UNKNOWN if there is none) */ public static Errors forException(Throwable t) { Errors error = classToError.get(t.getClass()); -- 1.7.12.4 From 1162cc1dd30e896e8c7a03f960b7d7bcbf883624 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Stevo=20Slavi=C4=87?= Date: Wed, 29 Jul 2015 18:23:54 -0700 Subject: [PATCH 075/120] MINOR: Fixed ConsumerRecord constructor javadoc MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Refactoring of ConsumerRecord made in https://github.com/apache/kafka/commit/0699ff2ce60abb466cab5315977a224f1a70a4da#diff-fafe8d3a3942f3c6394927881a9389b2 left ConsumerRecord constructor javadoc inconsistent with implementation. This patch fixes ConsumerRecord constructor javadoc to be inline with implementation. Author: Stevo Slavić Reviewers: Ismael, Guozhang Closes #85 from sslavic/patch-3 and squashes the following commits: c289c4f [Stevo Slavić] MINOR: Fixed ConsumerRecord constructor javadoc --- .../main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java index 49d9527..d4668c2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java @@ -24,11 +24,12 @@ public final class ConsumerRecord { private final V value; /** - * Create a record with no key - * + * Creates a record to be received from a specified topic and partition + * * @param topic The topic this record is received from * @param partition The partition of the topic this record is received from * @param offset The offset of this record in the corresponding Kafka partition + * @param key The key of the record, if one exists (null is allowed) * @param value The record contents */ public ConsumerRecord(String topic, int partition, long offset, K key, V value) { -- 1.7.12.4 From be82a2afc9e38adc0109dc694834ca5947128877 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Thu, 30 Jul 2015 14:23:43 -0700 Subject: [PATCH 076/120] KAFKA-2350; KafkaConsumer pause/resume API Author: Jason Gustafson Reviewers: Ismael, Ashish, Guozhang Closes #100 from hachikuji/KAFKA-2350 and squashes the following commits: 250e823 [Jason Gustafson] KAFKA-2350; KafkaConsumer pause/resume API --- .../apache/kafka/clients/consumer/Consumer.java | 10 + .../kafka/clients/consumer/KafkaConsumer.java | 48 ++++- .../kafka/clients/consumer/MockConsumer.java | 39 +++- .../clients/consumer/internals/Coordinator.java | 8 +- .../kafka/clients/consumer/internals/Fetcher.java | 45 ++-- .../consumer/internals/SubscriptionState.java | 238 +++++++++++++++------ .../kafka/clients/consumer/MockConsumerTest.java | 2 +- .../clients/consumer/internals/FetcherTest.java | 36 +++- .../consumer/internals/SubscriptionStateTest.java | 58 +++-- .../scala/integration/kafka/api/ConsumerTest.scala | 32 ++- 10 files changed, 386 insertions(+), 130 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java index 23e410b..158e1ea 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java @@ -119,6 +119,16 @@ public interface Consumer extends Closeable { public Map> listTopics(); /** + * @see KafkaConsumer#pause(TopicPartition...) + */ + public void pause(TopicPartition... partitions); + + /** + * @see KafkaConsumer#resume(TopicPartition...) + */ + public void resume(TopicPartition... partitions); + + /** * @see KafkaConsumer#close() */ public void close(); 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 923ff99..7851644 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 @@ -43,7 +43,6 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.ConcurrentModificationException; -import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -852,9 +851,7 @@ public class KafkaConsumer implements Consumer { public void commit(CommitType commitType, ConsumerCommitCallback callback) { acquire(); try { - // need defensive copy to ensure offsets are not removed before completion (e.g. in rebalance) - Map allConsumed = new HashMap(this.subscriptions.allConsumed()); - commit(allConsumed, commitType, callback); + commit(subscriptions.allConsumed(), commitType, callback); } finally { release(); } @@ -941,7 +938,7 @@ public class KafkaConsumer implements Consumer { public long position(TopicPartition partition) { acquire(); try { - if (!this.subscriptions.assignedPartitions().contains(partition)) + if (!this.subscriptions.isAssigned(partition)) throw new IllegalArgumentException("You can only check the position for partitions assigned to this consumer."); Long offset = this.subscriptions.consumed(partition); if (offset == null) { @@ -972,7 +969,7 @@ public class KafkaConsumer implements Consumer { acquire(); try { Long committed; - if (subscriptions.assignedPartitions().contains(partition)) { + if (subscriptions.isAssigned(partition)) { committed = this.subscriptions.committed(partition); if (committed == null) { coordinator.refreshCommittedOffsetsIfNeeded(); @@ -1040,6 +1037,45 @@ public class KafkaConsumer implements Consumer { } } + /** + * Suspend fetching from the requested partitions. Future calls to {@link #poll(long)} will not return + * any records from these partitions until they have been resumed using {@link #resume(TopicPartition...)}. + * Note that this method does not affect partition subscription. In particular, it does not cause a group + * rebalance when automatic assignment is used. + * @param partitions The partitions which should be paused + */ + @Override + public void pause(TopicPartition... partitions) { + acquire(); + try { + for (TopicPartition partition: partitions) { + log.debug("Pausing partition {}", partition); + subscriptions.pause(partition); + } + } finally { + release(); + } + } + + /** + * Resume any partitions which have been paused with {@link #pause(TopicPartition...)}. New calls to + * {@link #poll(long)} will return records from these partitions if there are any to be fetched. + * If the partitions were not previously paused, this method is a no-op. + * @param partitions The partitions which should be resumed + */ + @Override + public void resume(TopicPartition... partitions) { + acquire(); + try { + for (TopicPartition partition: partitions) { + log.debug("Resuming partition {}", partition); + subscriptions.resume(partition); + } + } finally { + release(); + } + } + @Override public void close() { acquire(); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java index 5b22fa0..b07e760 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java @@ -12,6 +12,12 @@ */ package org.apache.kafka.clients.consumer; +import org.apache.kafka.clients.consumer.internals.SubscriptionState; +import org.apache.kafka.common.Metric; +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; + import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -20,12 +26,6 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Set; -import org.apache.kafka.clients.consumer.internals.SubscriptionState; -import org.apache.kafka.common.Metric; -import org.apache.kafka.common.PartitionInfo; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.MetricName; - /** * A mock of the {@link Consumer} interface you can use for testing code that uses Kafka. This class is not * threadsafe @@ -83,9 +83,11 @@ public class MockConsumer implements Consumer { ensureNotClosed(); // update the consumed offset for (Map.Entry>> entry : this.records.entrySet()) { - List> recs = entry.getValue(); - if (!recs.isEmpty()) - this.subscriptions.consumed(entry.getKey(), recs.get(recs.size() - 1).offset()); + if (!subscriptions.isPaused(entry.getKey())) { + List> recs = entry.getValue(); + if (!recs.isEmpty()) + this.subscriptions.consumed(entry.getKey(), recs.get(recs.size() - 1).offset()); + } } ConsumerRecords copy = new ConsumerRecords(this.records); @@ -96,7 +98,12 @@ public class MockConsumer implements Consumer { public synchronized void addRecord(ConsumerRecord record) { ensureNotClosed(); TopicPartition tp = new TopicPartition(record.topic(), record.partition()); - this.subscriptions.assignedPartitions().add(tp); + ArrayList currentAssigned = new ArrayList<>(this.subscriptions.assignedPartitions()); + if (!currentAssigned.contains(tp)) { + currentAssigned.add(tp); + this.subscriptions.changePartitionAssignment(currentAssigned); + } + subscriptions.seek(tp, record.offset()); List> recs = this.records.get(tp); if (recs == null) { recs = new ArrayList>(); @@ -189,6 +196,18 @@ public class MockConsumer implements Consumer { } @Override + public void pause(TopicPartition... partitions) { + for (TopicPartition partition : partitions) + subscriptions.pause(partition); + } + + @Override + public void resume(TopicPartition... partitions) { + for (TopicPartition partition : partitions) + subscriptions.resume(partition); + } + + @Override public synchronized void close() { ensureNotClosed(); this.closed = true; 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 6026b23..cd5cdc3 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 @@ -119,7 +119,9 @@ public final class Coordinator { Map offsets = fetchCommittedOffsets(subscriptions.assignedPartitions()); for (Map.Entry entry : offsets.entrySet()) { TopicPartition tp = entry.getKey(); - this.subscriptions.committed(tp, entry.getValue()); + // verify assignment is still active + if (subscriptions.isAssigned(tp)) + this.subscriptions.committed(tp, entry.getValue()); } this.subscriptions.commitsRefreshed(); } @@ -459,7 +461,9 @@ public final class Coordinator { short errorCode = entry.getValue(); if (errorCode == Errors.NONE.code()) { log.debug("Committed offset {} for partition {}", offset, tp); - subscriptions.committed(tp, offset); + if (subscriptions.isAssigned(tp)) + // update the local cache only if the partition is still assigned + subscriptions.committed(tp, offset); } else if (errorCode == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code() || errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { coordinatorDead(); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index 9f71451..956197b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -143,8 +143,7 @@ public class Fetcher { public void updateFetchPositions(Set partitions) { // reset the fetch position to the committed position for (TopicPartition tp : partitions) { - // skip if we already have a fetch position - if (subscriptions.fetched(tp) != null) + if (!subscriptions.isAssigned(tp) || subscriptions.isFetchable(tp)) continue; // TODO: If there are several offsets to reset, we could submit offset requests in parallel @@ -222,7 +221,10 @@ public class Fetcher { log.debug("Resetting offset for partition {} to {} offset.", partition, strategy.name().toLowerCase()); long offset = listOffset(partition, timestamp); - this.subscriptions.seek(partition, offset); + + // we might lose the assignment while fetching the offset, so check it is still active + if (subscriptions.isAssigned(partition)) + this.subscriptions.seek(partition, offset); } /** @@ -259,11 +261,15 @@ public class Fetcher { if (this.subscriptions.partitionAssignmentNeeded()) { return Collections.emptyMap(); } else { - Map>> drained = new HashMap>>(); + Map>> drained = new HashMap<>(); for (PartitionRecords part : this.records) { + if (!subscriptions.isFetchable(part.partition)) { + log.debug("Ignoring fetched records for {} since it is no longer fetchable", part.partition); + continue; + } + Long consumed = subscriptions.consumed(part.partition); - if (this.subscriptions.assignedPartitions().contains(part.partition) - && consumed != null && part.fetchOffset == consumed) { + if (consumed != null && part.fetchOffset == consumed) { List> records = drained.get(part.partition); if (records == null) { records = part.records; @@ -354,8 +360,8 @@ public class Fetcher { */ private Map createFetchRequests(Cluster cluster) { // create the fetch info - Map> fetchable = new HashMap>(); - for (TopicPartition partition : subscriptions.assignedPartitions()) { + Map> fetchable = new HashMap<>(); + for (TopicPartition partition : subscriptions.fetchablePartitions()) { Node node = cluster.leaderFor(partition); if (node == null) { metadata.requestUpdate(); @@ -363,16 +369,17 @@ public class Fetcher { // if there is a leader and no in-flight requests, issue a new fetch Map fetch = fetchable.get(node); if (fetch == null) { - fetch = new HashMap(); + fetch = new HashMap<>(); fetchable.put(node, fetch); } + long offset = this.subscriptions.fetched(partition); fetch.put(partition, new FetchRequest.PartitionData(offset, this.fetchSize)); } } // create the fetches - Map requests = new HashMap(); + Map requests = new HashMap<>(); for (Map.Entry> entry : fetchable.entrySet()) { Node node = entry.getKey(); FetchRequest fetch = new FetchRequest(this.maxWaitMs, this.minBytes, entry.getValue()); @@ -399,15 +406,7 @@ public class Fetcher { if (!subscriptions.assignedPartitions().contains(tp)) { log.debug("Ignoring fetched data for partition {} which is no longer assigned.", tp); } else if (partition.errorCode == Errors.NONE.code()) { - int bytes = 0; - ByteBuffer buffer = partition.recordSet; - MemoryRecords records = MemoryRecords.readableRecords(buffer); long fetchOffset = request.fetchData().get(tp).offset; - List> parsed = new ArrayList>(); - for (LogEntry logEntry : records) { - parsed.add(parseRecord(tp, logEntry)); - bytes += logEntry.size(); - } // we are interested in this fetch only if the beginning offset matches the // current consumed position @@ -422,7 +421,15 @@ public class Fetcher { continue; } - if (parsed.size() > 0) { + int bytes = 0; + ByteBuffer buffer = partition.recordSet; + MemoryRecords records = MemoryRecords.readableRecords(buffer); + List> parsed = new ArrayList>(); + for (LogEntry logEntry : records) { + parsed.add(parseRecord(tp, logEntry)); + bytes += logEntry.size(); + } + if (!parsed.isEmpty()) { ConsumerRecord record = parsed.get(parsed.size() - 1); this.subscriptions.fetched(tp, record.offset() + 1); this.records.add(new PartitionRecords(fetchOffset, tp, parsed)); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java index 8a2cb12..6788ee6 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java @@ -23,7 +23,25 @@ import java.util.Map; import java.util.Set; /** - * A class for tracking the topics, partitions, and offsets for the consumer + * A class for tracking the topics, partitions, and offsets for the consumer. A partition + * is "assigned" either directly with {@link #subscribe(TopicPartition)} (manual assignment) + * or with {@link #changePartitionAssignment(List)} (automatic assignment). + * + * Once assigned, the partition is not considered "fetchable" until its initial position has + * been set with {@link #seek(TopicPartition, long)}. Fetchable partitions track a fetch + * position which is used to set the offset of the next fetch, and a consumed position + * which is the last offset that has been returned to the user. You can suspend fetching + * from a partition through {@link #pause(TopicPartition)} without affecting the fetched/consumed + * offsets. The partition will remain unfetchable until the {@link #resume(TopicPartition)} is + * used. You can also query the pause state independently with {@link #isPaused(TopicPartition)}. + * + * Note that pause state as well as fetch/consumed positions are not preserved when partition + * assignment is changed either with {@link #unsubscribe(TopicPartition)} or + * {@link #changePartitionAssignment(List)}. + * + * This class also maintains a cache of the latest commit position for each of the assigned + * partitions. This is updated through {@link #committed(TopicPartition, long)} and can be used + * to set the initial fetch position (e.g. {@link Fetcher#resetOffset(TopicPartition)}. */ public class SubscriptionState { @@ -34,16 +52,7 @@ public class SubscriptionState { private final Set subscribedPartitions; /* the list of partitions currently assigned */ - private final Set assignedPartitions; - - /* the offset exposed to the user */ - private final Map consumed; - - /* the current point we have fetched up to */ - private final Map fetched; - - /* the last committed offset for each partition */ - private final Map committed; + private final Map assignedPartitions; /* do we need to request a partition assignment from the coordinator? */ private boolean needsPartitionAssignment; @@ -51,28 +60,21 @@ public class SubscriptionState { /* do we need to request the latest committed offsets from the coordinator? */ private boolean needsFetchCommittedOffsets; - /* Partitions that need to be reset before fetching */ - private Map resetPartitions; - /* Default offset reset strategy */ - private OffsetResetStrategy offsetResetStrategy; - - public SubscriptionState(OffsetResetStrategy offsetResetStrategy) { - this.offsetResetStrategy = offsetResetStrategy; - this.subscribedTopics = new HashSet(); - this.subscribedPartitions = new HashSet(); - this.assignedPartitions = new HashSet(); - this.consumed = new HashMap(); - this.fetched = new HashMap(); - this.committed = new HashMap(); + private final OffsetResetStrategy defaultResetStrategy; + + public SubscriptionState(OffsetResetStrategy defaultResetStrategy) { + this.defaultResetStrategy = defaultResetStrategy; + this.subscribedTopics = new HashSet<>(); + this.subscribedPartitions = new HashSet<>(); + this.assignedPartitions = new HashMap<>(); this.needsPartitionAssignment = false; this.needsFetchCommittedOffsets = true; // initialize to true for the consumers to fetch offset upon starting up - this.resetPartitions = new HashMap(); } public void subscribe(String topic) { - if (this.subscribedPartitions.size() > 0) - throw new IllegalStateException("Subcription to topics and partitions are mutually exclusive"); + if (!this.subscribedPartitions.isEmpty()) + throw new IllegalStateException("Subscription to topics and partitions are mutually exclusive"); if (!this.subscribedTopics.contains(topic)) { this.subscribedTopics.add(topic); this.needsPartitionAssignment = true; @@ -95,10 +97,10 @@ public class SubscriptionState { } public void subscribe(TopicPartition tp) { - if (this.subscribedTopics.size() > 0) - throw new IllegalStateException("Subcription to topics and partitions are mutually exclusive"); + if (!this.subscribedTopics.isEmpty()) + throw new IllegalStateException("Subscription to topics and partitions are mutually exclusive"); this.subscribedPartitions.add(tp); - this.assignedPartitions.add(tp); + addAssignedPartition(tp); } public void unsubscribe(TopicPartition partition) { @@ -110,17 +112,10 @@ public class SubscriptionState { private void clearPartition(TopicPartition tp) { this.assignedPartitions.remove(tp); - this.committed.remove(tp); - this.fetched.remove(tp); - this.consumed.remove(tp); - this.resetPartitions.remove(tp); } public void clearAssignment() { this.assignedPartitions.clear(); - this.committed.clear(); - this.fetched.clear(); - this.consumed.clear(); this.needsPartitionAssignment = !subscribedTopics().isEmpty(); } @@ -129,21 +124,26 @@ public class SubscriptionState { } public Long fetched(TopicPartition tp) { - return this.fetched.get(tp); + return assignedState(tp).fetched; } public void fetched(TopicPartition tp, long offset) { - if (!this.assignedPartitions.contains(tp)) - throw new IllegalArgumentException("Can't change the fetch position for a partition you are not currently subscribed to."); - this.fetched.put(tp, offset); + assignedState(tp).fetched(offset); + } + + private TopicPartitionState assignedState(TopicPartition tp) { + TopicPartitionState state = this.assignedPartitions.get(tp); + if (state == null) + throw new IllegalStateException("No current assignment for partition " + tp); + return state; } public void committed(TopicPartition tp, long offset) { - this.committed.put(tp, offset); + assignedState(tp).committed(offset); } public Long committed(TopicPartition tp) { - return this.committed.get(tp); + return assignedState(tp).committed; } public void needRefreshCommits() { @@ -157,15 +157,22 @@ public class SubscriptionState { public void commitsRefreshed() { this.needsFetchCommittedOffsets = false; } - + public void seek(TopicPartition tp, long offset) { - fetched(tp, offset); - consumed(tp, offset); - resetPartitions.remove(tp); + assignedState(tp).seek(offset); } public Set assignedPartitions() { - return this.assignedPartitions; + return this.assignedPartitions.keySet(); + } + + public Set fetchablePartitions() { + Set fetchable = new HashSet<>(); + for (Map.Entry entry : assignedPartitions.entrySet()) { + if (entry.getValue().isFetchable()) + fetchable.add(entry.getKey()); + } + return fetchable; } public boolean partitionsAutoAssigned() { @@ -173,49 +180,52 @@ public class SubscriptionState { } public void consumed(TopicPartition tp, long offset) { - if (!this.assignedPartitions.contains(tp)) - throw new IllegalArgumentException("Can't change the consumed position for a partition you are not currently subscribed to."); - this.consumed.put(tp, offset); + assignedState(tp).consumed(offset); } - public Long consumed(TopicPartition partition) { - return this.consumed.get(partition); + public Long consumed(TopicPartition tp) { + return assignedState(tp).consumed; } public Map allConsumed() { - return this.consumed; + Map allConsumed = new HashMap<>(); + for (Map.Entry entry : assignedPartitions.entrySet()) { + TopicPartitionState state = entry.getValue(); + if (state.hasValidPosition) + allConsumed.put(entry.getKey(), state.consumed); + } + return allConsumed; } public void needOffsetReset(TopicPartition partition, OffsetResetStrategy offsetResetStrategy) { - this.resetPartitions.put(partition, offsetResetStrategy); - this.fetched.remove(partition); - this.consumed.remove(partition); + assignedState(partition).awaitReset(offsetResetStrategy); } public void needOffsetReset(TopicPartition partition) { - needOffsetReset(partition, offsetResetStrategy); + needOffsetReset(partition, defaultResetStrategy); } public boolean isOffsetResetNeeded(TopicPartition partition) { - return resetPartitions.containsKey(partition); - } - - public boolean isOffsetResetNeeded() { - return !resetPartitions.isEmpty(); + return assignedState(partition).awaitingReset; } public OffsetResetStrategy resetStrategy(TopicPartition partition) { - return resetPartitions.get(partition); + return assignedState(partition).resetStrategy; } public boolean hasAllFetchPositions() { - return this.fetched.size() >= this.assignedPartitions.size(); + for (TopicPartitionState state : assignedPartitions.values()) + if (!state.hasValidPosition) + return false; + return true; } public Set missingFetchPositions() { - Set copy = new HashSet(this.assignedPartitions); - copy.removeAll(this.fetched.keySet()); - return copy; + Set missing = new HashSet<>(this.assignedPartitions.keySet()); + for (Map.Entry entry : assignedPartitions.entrySet()) + if (!entry.getValue().hasValidPosition) + missing.add(entry.getKey()); + return missing; } public boolean partitionAssignmentNeeded() { @@ -227,9 +237,99 @@ public class SubscriptionState { if (!this.subscribedTopics.contains(tp.topic())) throw new IllegalArgumentException("Assigned partition " + tp + " for non-subscribed topic."); this.clearAssignment(); - this.assignedPartitions.addAll(assignments); + for (TopicPartition tp: assignments) + addAssignedPartition(tp); this.needsPartitionAssignment = false; } + public boolean isAssigned(TopicPartition tp) { + return assignedPartitions.containsKey(tp); + } + + public boolean isPaused(TopicPartition tp) { + return isAssigned(tp) && assignedState(tp).paused; + } + + public boolean isFetchable(TopicPartition tp) { + return isAssigned(tp) && assignedState(tp).isFetchable(); + } + + public void pause(TopicPartition tp) { + assignedState(tp).pause(); + } + + public void resume(TopicPartition tp) { + assignedState(tp).resume(); + } + + private void addAssignedPartition(TopicPartition tp) { + this.assignedPartitions.put(tp, new TopicPartitionState()); + } + + private static class TopicPartitionState { + private Long consumed; // offset exposed to the user + private Long fetched; // current fetch position + private Long committed; // last committed position + + private boolean hasValidPosition; // whether we have valid consumed and fetched positions + private boolean paused; // whether this partition has been paused by the user + private boolean awaitingReset; // whether we are awaiting reset + private OffsetResetStrategy resetStrategy; // the reset strategy if awaitingReset is set + + public TopicPartitionState() { + this.paused = false; + this.consumed = null; + this.fetched = null; + this.committed = null; + this.awaitingReset = false; + this.hasValidPosition = false; + this.resetStrategy = null; + } + + private void awaitReset(OffsetResetStrategy strategy) { + this.awaitingReset = true; + this.resetStrategy = strategy; + this.consumed = null; + this.fetched = null; + this.hasValidPosition = false; + } + + private void seek(long offset) { + this.consumed = offset; + this.fetched = offset; + this.awaitingReset = false; + this.resetStrategy = null; + this.hasValidPosition = true; + } + + private void fetched(long offset) { + if (!hasValidPosition) + throw new IllegalStateException("Cannot update fetch position without valid consumed/fetched positions"); + this.fetched = offset; + } + + private void consumed(long offset) { + if (!hasValidPosition) + throw new IllegalStateException("Cannot update consumed position without valid consumed/fetched positions"); + this.consumed = offset; + } + + private void committed(Long offset) { + this.committed = offset; + } + + private void pause() { + this.paused = true; + } + + private void resume() { + this.paused = false; + } + + private boolean isFetchable() { + return !paused && hasValidPosition; + } + + } } \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java index 26b6b40..d4da642 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java @@ -29,7 +29,7 @@ public class MockConsumerTest { @Test public void testSimpleMock() { - consumer.subscribe("topic"); + consumer.subscribe("test"); assertEquals(0, consumer.poll(1000).count()); ConsumerRecord rec1 = new ConsumerRecord("test", 0, 0, "key1", "value1"); ConsumerRecord rec2 = new ConsumerRecord("test", 0, 1, "key2", "value2"); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java index 06e2990..56850bb 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -45,6 +45,7 @@ import java.util.List; import java.util.Map; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; public class FetcherTest { @@ -99,8 +100,7 @@ public class FetcherTest { public void testFetchNormal() { List> records; subscriptions.subscribe(tp); - subscriptions.fetched(tp, 0); - subscriptions.consumed(tp, 0); + subscriptions.seek(tp, 0); // normal fetch fetcher.initFetches(cluster); @@ -121,8 +121,7 @@ public class FetcherTest { public void testFetchDuringRebalance() { subscriptions.subscribe(topicName); subscriptions.changePartitionAssignment(Arrays.asList(tp)); - subscriptions.fetched(tp, 0); - subscriptions.consumed(tp, 0); + subscriptions.seek(tp, 0); fetcher.initFetches(cluster); @@ -136,10 +135,32 @@ public class FetcherTest { } @Test + public void testInFlightFetchOnPausedPartition() { + subscriptions.subscribe(tp); + subscriptions.seek(tp, 0); + + fetcher.initFetches(cluster); + subscriptions.pause(tp); + + client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L)); + consumerClient.poll(0); + assertNull(fetcher.fetchedRecords().get(tp)); + } + + @Test + public void testFetchOnPausedPartition() { + subscriptions.subscribe(tp); + subscriptions.seek(tp, 0); + + subscriptions.pause(tp); + fetcher.initFetches(cluster); + assertTrue(client.requests().isEmpty()); + } + + @Test public void testFetchFailed() { subscriptions.subscribe(tp); - subscriptions.fetched(tp, 0); - subscriptions.consumed(tp, 0); + subscriptions.seek(tp, 0); // fetch with not leader fetcher.initFetches(cluster); @@ -169,8 +190,7 @@ public class FetcherTest { @Test public void testFetchOutOfRange() { subscriptions.subscribe(tp); - subscriptions.fetched(tp, 5); - subscriptions.consumed(tp, 5); + subscriptions.seek(tp, 5); // fetch with out of range fetcher.initFetches(cluster); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java index c47f3fb..1ba6f7a 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.clients.consumer.internals; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static java.util.Arrays.asList; @@ -37,12 +38,13 @@ public class SubscriptionStateTest { state.subscribe(tp0); assertEquals(Collections.singleton(tp0), state.assignedPartitions()); state.committed(tp0, 1); - state.fetched(tp0, 1); - state.consumed(tp0, 1); + state.seek(tp0, 1); + assertTrue(state.isFetchable(tp0)); assertAllPositions(tp0, 1L); state.unsubscribe(tp0); assertTrue(state.assignedPartitions().isEmpty()); - assertAllPositions(tp0, null); + assertFalse(state.isAssigned(tp0)); + assertFalse(state.isFetchable(tp0)); } @Test @@ -52,10 +54,15 @@ public class SubscriptionStateTest { assertEquals(5L, (long) state.fetched(tp0)); assertEquals(5L, (long) state.consumed(tp0)); state.needOffsetReset(tp0); - assertTrue(state.isOffsetResetNeeded()); + assertFalse(state.isFetchable(tp0)); assertTrue(state.isOffsetResetNeeded(tp0)); assertEquals(null, state.fetched(tp0)); assertEquals(null, state.consumed(tp0)); + + // seek should clear the reset and make the partition fetchable + state.seek(tp0, 0); + assertTrue(state.isFetchable(tp0)); + assertFalse(state.isOffsetResetNeeded(tp0)); } @Test @@ -65,16 +72,28 @@ public class SubscriptionStateTest { assertTrue(state.assignedPartitions().isEmpty()); assertTrue(state.partitionsAutoAssigned()); state.changePartitionAssignment(asList(tp0)); + state.seek(tp0, 1); state.committed(tp0, 1); - state.fetched(tp0, 1); - state.consumed(tp0, 1); assertAllPositions(tp0, 1L); state.changePartitionAssignment(asList(tp1)); - assertAllPositions(tp0, null); + assertTrue(state.isAssigned(tp1)); + assertFalse(state.isAssigned(tp0)); + assertFalse(state.isFetchable(tp1)); assertEquals(Collections.singleton(tp1), state.assignedPartitions()); } @Test + public void partitionPause() { + state.subscribe(tp0); + state.seek(tp0, 100); + assertTrue(state.isFetchable(tp0)); + state.pause(tp0); + assertFalse(state.isFetchable(tp0)); + state.resume(tp0); + assertTrue(state.isFetchable(tp0)); + } + + @Test public void topicUnsubscription() { final String topic = "test"; state.subscribe(topic); @@ -83,24 +102,37 @@ public class SubscriptionStateTest { assertTrue(state.partitionsAutoAssigned()); state.changePartitionAssignment(asList(tp0)); state.committed(tp0, 1); - state.fetched(tp0, 1); - state.consumed(tp0, 1); + state.seek(tp0, 1); assertAllPositions(tp0, 1L); state.changePartitionAssignment(asList(tp1)); - assertAllPositions(tp0, null); + assertFalse(state.isAssigned(tp0)); assertEquals(Collections.singleton(tp1), state.assignedPartitions()); state.unsubscribe(topic); assertEquals(0, state.subscribedTopics().size()); assertTrue(state.assignedPartitions().isEmpty()); } - - @Test(expected = IllegalArgumentException.class) + + @Test(expected = IllegalStateException.class) + public void invalidConsumedPositionUpdate() { + state.subscribe("test"); + state.changePartitionAssignment(asList(tp0)); + state.consumed(tp0, 0); + } + + @Test(expected = IllegalStateException.class) + public void invalidFetchPositionUpdate() { + state.subscribe("test"); + state.changePartitionAssignment(asList(tp0)); + state.fetched(tp0, 0); + } + + @Test(expected = IllegalStateException.class) public void cantChangeFetchPositionForNonAssignedPartition() { state.fetched(tp0, 1); } - @Test(expected = IllegalArgumentException.class) + @Test(expected = IllegalStateException.class) public void cantChangeConsumedPositionForNonAssignedPartition() { state.consumed(tp0, 1); } diff --git a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala index 0c2755f..4ea49f2 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala @@ -18,7 +18,7 @@ import org.apache.kafka.clients.producer.ProducerConfig import org.apache.kafka.clients.producer.ProducerRecord import org.apache.kafka.clients.consumer._ import org.apache.kafka.common.serialization.ByteArrayDeserializer -import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.{PartitionInfo, TopicPartition} import kafka.utils.{TestUtils, Logging} import kafka.server.KafkaConfig @@ -254,6 +254,34 @@ class ConsumerTest extends IntegrationTestHarness with Logging { } } + def testPartitionPauseAndResume() { + sendRecords(5) + this.consumers(0).subscribe(tp) + consumeRecords(this.consumers(0), 5, 0) + this.consumers(0).pause(tp) + sendRecords(5) + assertTrue(this.consumers(0).poll(0).isEmpty) + this.consumers(0).resume(tp) + consumeRecords(this.consumers(0), 5, 5) + } + + def testPauseStateNotPreservedByRebalance() { + this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "100"); // timeout quickly to avoid slow test + val consumer0 = new KafkaConsumer(this.consumerConfig, null, new ByteArrayDeserializer(), new ByteArrayDeserializer()) + + sendRecords(5) + consumer0.subscribe(topic) + consumeRecords(consumer0, 5, 0) + consumer0.pause(tp) + + // subscribe to a new topic to trigger a rebalance + consumer0.subscribe("topic2") + + // after rebalance, our position should be reset and our pause state lost, + // so we should be able to consume from the beginning + consumeRecords(consumer0, 0, 5) + } + private class TestConsumerReassignmentCallback extends ConsumerRebalanceCallback { var callsToAssigned = 0 var callsToRevoked = 0 @@ -264,7 +292,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging { def onPartitionsRevoked(consumer: Consumer[_,_], partitions: java.util.Collection[TopicPartition]) { info("onPartitionsRevoked called.") callsToRevoked += 1 - } + } } private def sendRecords(numRecords: Int): Unit = { -- 1.7.12.4 From 23a36eb54958485d1e84c64cc739ca268e96b181 Mon Sep 17 00:00:00 2001 From: Manikumar Reddy Date: Thu, 30 Jul 2015 14:29:21 -0700 Subject: [PATCH 077/120] KAFKA-2026: fix logging of unsued options always showing null; reviewed by Ewen Cheslack-Postava and Jiangjie Qin --- .../src/main/java/org/apache/kafka/common/config/AbstractConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java index bae528d..ec3ae15 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java @@ -122,7 +122,7 @@ public class AbstractConfig { */ public void logUnused() { for (String key : unused()) - log.warn("The configuration {} = {} was supplied but isn't a known config.", key, this.values.get(key)); + log.warn("The configuration {} = {} was supplied but isn't a known config.", key, this.originals.get(key)); } /** -- 1.7.12.4 From b152c0604cdce8882028b62601916b5d2b82d6a5 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Mon, 3 Aug 2015 13:08:57 -0700 Subject: [PATCH 078/120] KAFKA-2384: Override commit message title in kafka-merge-pr.py Author: Ismael Juma Reviewers: Guozhang Closes #105 from ijuma/kafka-2384-override-commit-message-title and squashes the following commits: e042242 [Ismael Juma] Support overriding of commit message title in kafka-merge-pr.py --- kafka-merge-pr.py | 26 ++++++++++++++------------ 1 file changed, 14 insertions(+), 12 deletions(-) diff --git a/kafka-merge-pr.py b/kafka-merge-pr.py index 876f530..576d315 100644 --- a/kafka-merge-pr.py +++ b/kafka-merge-pr.py @@ -380,22 +380,24 @@ def main(): url = pr["url"] + pr_title = pr["title"] + commit_title = raw_input("Commit title [%s]: " % pr_title) + if commit_title == "": + commit_title = pr_title + # Decide whether to use the modified title or not - modified_title = standardize_jira_ref(pr["title"]) - if modified_title != pr["title"]: + modified_title = standardize_jira_ref(commit_title) + if modified_title != commit_title: print "I've re-written the title as follows to match the standard format:" - print "Original: %s" % pr["title"] + print "Original: %s" % commit_title print "Modified: %s" % modified_title result = raw_input("Would you like to use the modified title? (y/n): ") if result.lower() == "y": - title = modified_title + commit_title = modified_title print "Using modified title:" else: - title = pr["title"] print "Using original title:" - print title - else: - title = pr["title"] + print commit_title body = pr["body"] target_ref = pr["base"]["ref"] @@ -428,13 +430,13 @@ def main(): continue_maybe(msg) print ("\n=== Pull Request #%s ===" % pr_num) - print ("title\t%s\nsource\t%s\ntarget\t%s\nurl\t%s" % ( - title, pr_repo_desc, target_ref, url)) + print ("PR title\t%s\nCommit title\t%s\nSource\t\t%s\nTarget\t\t%s\nURL\t\t%s" % ( + pr_title, commit_title, pr_repo_desc, target_ref, url)) continue_maybe("Proceed with merging pull request #%s?" % pr_num) merged_refs = [target_ref] - merge_hash = merge_pr(pr_num, target_ref, title, body, pr_repo_desc) + merge_hash = merge_pr(pr_num, target_ref, commit_title, body, pr_repo_desc) pick_prompt = "Would you like to pick %s into another branch?" % merge_hash while raw_input("\n%s (y/n): " % pick_prompt).lower() == "y": @@ -444,7 +446,7 @@ def main(): if JIRA_USERNAME and JIRA_PASSWORD: continue_maybe("Would you like to update an associated JIRA?") jira_comment = "Issue resolved by pull request %s\n[%s/%s]" % (pr_num, GITHUB_BASE, pr_num) - resolve_jira_issues(title, merged_refs, jira_comment) + resolve_jira_issues(commit_title, merged_refs, jira_comment) else: print "JIRA_USERNAME and JIRA_PASSWORD not set" print "Exiting without trying to close the associated JIRA." -- 1.7.12.4 From cd3dc7a5c9f3a68ad73d2ae7c975f9882f00036e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Stevo=20Slavi=C4=87?= Date: Mon, 3 Aug 2015 14:12:00 -0700 Subject: [PATCH 079/120] MINOR: Added to .gitignore Kafka server logs directory MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit When running Kafka server from sources, logs directory gets created in root of repository, and kafka server logs end up there. Currently that directory is not ignored by git. This change adds root logs directory to .gitignore so that Kafka server logs are ignored and do not get tracked by git. Author: Stevo Slavić Reviewers: Ismael Juma Closes #94 from sslavic/patch-7 and squashes the following commits: c7b62a7 [Stevo Slavić] MINOR: Added to .gitignore Kafka server logs --- .gitignore | 1 + 1 file changed, 1 insertion(+) diff --git a/.gitignore b/.gitignore index 4aae6e7..dbc0507 100644 --- a/.gitignore +++ b/.gitignore @@ -24,6 +24,7 @@ kafka.ipr kafka.iws .vagrant Vagrantfile.local +/logs config/server-* config/zookeeper-* -- 1.7.12.4 From 3c09630844f6e70793f53a9d4f0ef562fe9d91d3 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Mon, 3 Aug 2015 15:42:33 -0700 Subject: [PATCH 080/120] KAFKA-2386; increase timeouts for transient test failure in ConsumerCoordinatorResponseTests There are two race conditions in the test case "testGenerationIdIncrementsOnRebalance." First, a delay before the second join group request can timeout the initial group and cause the generationId to unexpectedly reset. Second, a delay in the join group request handling will timeout the request itself and cause the test to fail. This commit doesn't address these race conditions, but increases the timeouts to make them more unlikely. If the problem reoccurs, then we'll probably need a better solution. Author: Jason Gustafson Reviewers: Gwen Shapira Closes #107 from hachikuji/KAFKA-2386 and squashes the following commits: a53460a [Jason Gustafson] KAFKA-2386; increase timeouts for transient test failure in ConsumerCoordinatorResponseTest --- .../unit/kafka/coordinator/ConsumerCoordinatorResponseTest.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/core/src/test/scala/unit/kafka/coordinator/ConsumerCoordinatorResponseTest.scala b/core/src/test/scala/unit/kafka/coordinator/ConsumerCoordinatorResponseTest.scala index 87a5330..058daef 100644 --- a/core/src/test/scala/unit/kafka/coordinator/ConsumerCoordinatorResponseTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/ConsumerCoordinatorResponseTest.scala @@ -43,8 +43,8 @@ class ConsumerCoordinatorResponseTest extends JUnitSuite { type HeartbeatCallback = Short => Unit val ConsumerMinSessionTimeout = 10 - val ConsumerMaxSessionTimeout = 100 - val DefaultSessionTimeout = 20 + val ConsumerMaxSessionTimeout = 200 + val DefaultSessionTimeout = 100 var consumerCoordinator: ConsumerCoordinator = null var offsetManager : OffsetManager = null @@ -238,7 +238,7 @@ class ConsumerCoordinatorResponseTest extends JUnitSuite { // First start up a group (with a slightly larger timeout to give us time to heartbeat when the rebalance starts) val joinGroupResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_CONSUMER_ID, partitionAssignmentStrategy, - 100, isCoordinatorForGroup = true) + DefaultSessionTimeout, isCoordinatorForGroup = true) val assignedConsumerId = joinGroupResult._2 val initialGenerationId = joinGroupResult._3 val joinGroupErrorCode = joinGroupResult._4 @@ -310,7 +310,8 @@ class ConsumerCoordinatorResponseTest extends JUnitSuite { sessionTimeout: Int, isCoordinatorForGroup: Boolean): JoinGroupCallbackParams = { val responseFuture = sendJoinGroup(groupId, consumerId, partitionAssignmentStrategy, sessionTimeout, isCoordinatorForGroup) - Await.result(responseFuture, Duration(40, TimeUnit.MILLISECONDS)) + // should only have to wait as long as session timeout, but allow some extra time in case of an unexpected delay + Await.result(responseFuture, Duration(sessionTimeout+100, TimeUnit.MILLISECONDS)) } private def heartbeat(groupId: String, -- 1.7.12.4 From 4b400afceadcbe74e3ecbfeb7c3066fd436da192 Mon Sep 17 00:00:00 2001 From: lvfangmin Date: Tue, 4 Aug 2015 11:29:10 -0700 Subject: [PATCH 081/120] KAFKA-2055; Fix transient ConsumerBounceTest.testSeekAndCommitWithBrokerFailure failure. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit …kerFailures failure; Author: lvfangmin Reviewers: Guozhang Closes #98 from lvfangmin/KAFKA-2055 and squashes the following commits: 057a1f1 [lvfangmin] KAFKA-2055; Fix transient ConsumerBounceTest.testSeekAndCommitWithBrokerFailures failure; --- core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala index d8eee52..93f9468 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala @@ -104,6 +104,11 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging { consumer.subscribe(tp) consumer.seek(tp, 0) + // wait until all the followers have synced the last HW with leader + TestUtils.waitUntilTrue(() => servers.forall(server => + server.replicaManager.getReplica(tp.topic(), tp.partition()).get.highWatermark.messageOffset == numRecords + ), "Failed to update high watermark for followers after timeout") + val scheduler = new BounceBrokerScheduler(numIters) scheduler.start() -- 1.7.12.4 From 1a0179f21a3937065fea2b745de67967bcafcb68 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Tue, 4 Aug 2015 11:33:03 -0700 Subject: [PATCH 082/120] HOTFIX; Encode/decode to utf-8 for commit title IO in kafka-merge-pr.py after KAFKA-2384 This fix should be fine for Linux and OS X. Not sure about Windows though. This is a very specific fix for new functionality added in KAFKA-2384. There are other places where a similar error could occur, but are less likely. The script doesn't really support Unicode input at the moment. Author: Ismael Juma Reviewers: Guozhang Closes #109 from ijuma/kafka-2384-hotfix and squashes the following commits: 0ab8958 [Ismael Juma] Encode/decode to utf-8 for commit title IO in kafka-merge-pr.py --- kafka-merge-pr.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka-merge-pr.py b/kafka-merge-pr.py index 576d315..5e8cbf5 100644 --- a/kafka-merge-pr.py +++ b/kafka-merge-pr.py @@ -381,7 +381,7 @@ def main(): url = pr["url"] pr_title = pr["title"] - commit_title = raw_input("Commit title [%s]: " % pr_title) + commit_title = raw_input("Commit title [%s]: " % pr_title.encode("utf-8")).decode("utf-8") if commit_title == "": commit_title = pr_title -- 1.7.12.4 From a56a79055dfba4687f476b0a4d20aeec1c4ebff7 Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Tue, 4 Aug 2015 15:11:27 -0700 Subject: [PATCH 083/120] kafka-2205; Generalize TopicConfigManager to handle multiple entity configs; patched by Aditya Auradkar; reviewed Jun Rao --- bin/kafka-configs.sh | 17 ++ core/src/main/scala/kafka/admin/AdminUtils.scala | 77 +++++---- .../src/main/scala/kafka/admin/ConfigCommand.scala | 174 ++++++++++++++++++++ core/src/main/scala/kafka/admin/TopicCommand.scala | 39 +---- core/src/main/scala/kafka/cluster/Partition.scala | 5 +- .../scala/kafka/controller/KafkaController.scala | 7 +- .../kafka/controller/PartitionLeaderSelector.scala | 6 +- .../kafka/controller/TopicDeletionManager.scala | 5 +- .../main/scala/kafka/server/ConfigHandler.scala | 69 ++++++++ .../scala/kafka/server/DynamicConfigManager.scala | 183 +++++++++++++++++++++ core/src/main/scala/kafka/server/KafkaServer.scala | 15 +- .../scala/kafka/server/ReplicaFetcherThread.scala | 4 +- .../scala/kafka/server/TopicConfigManager.scala | 152 ----------------- core/src/main/scala/kafka/utils/ZkUtils.scala | 33 +++- .../test/scala/unit/kafka/admin/AdminTest.scala | 8 +- .../scala/unit/kafka/admin/ConfigCommandTest.scala | 73 ++++++++ .../scala/unit/kafka/admin/TopicCommandTest.scala | 13 +- .../kafka/server/DynamicConfigChangeTest.scala | 83 +++++++++- 18 files changed, 718 insertions(+), 245 deletions(-) create mode 100755 bin/kafka-configs.sh create mode 100644 core/src/main/scala/kafka/admin/ConfigCommand.scala create mode 100644 core/src/main/scala/kafka/server/ConfigHandler.scala create mode 100644 core/src/main/scala/kafka/server/DynamicConfigManager.scala delete mode 100644 core/src/main/scala/kafka/server/TopicConfigManager.scala create mode 100644 core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala diff --git a/bin/kafka-configs.sh b/bin/kafka-configs.sh new file mode 100755 index 0000000..417eaf5 --- /dev/null +++ b/bin/kafka-configs.sh @@ -0,0 +1,17 @@ +#!/bin/bash +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +exec $(dirname $0)/kafka-run-class.sh kafka.admin.ConfigCommand $@ diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala index 4cc2376..9966660 100644 --- a/core/src/main/scala/kafka/admin/AdminUtils.scala +++ b/core/src/main/scala/kafka/admin/AdminUtils.scala @@ -21,6 +21,7 @@ import kafka.common._ import kafka.cluster.{BrokerEndPoint, Broker} import kafka.log.LogConfig +import kafka.server.ConfigType import kafka.utils._ import kafka.api.{TopicMetadata, PartitionMetadata} @@ -40,10 +41,8 @@ import org.I0Itec.zkclient.exception.ZkNodeExistsException object AdminUtils extends Logging { val rand = new Random - val AdminClientId = "__admin_client" - - val TopicConfigChangeZnodePrefix = "config_change_" + val EntityConfigChangeZnodePrefix = "config_change_" /** * There are 2 goals of replica assignment: @@ -103,14 +102,12 @@ object AdminUtils extends Logging { * @param numPartitions Number of partitions to be set * @param replicaAssignmentStr Manual replica assignment * @param checkBrokerAvailable Ignore checking if assigned replica broker is available. Only used for testing - * @param config Pre-existing properties that should be preserved */ def addPartitions(zkClient: ZkClient, topic: String, numPartitions: Int = 1, replicaAssignmentStr: String = "", - checkBrokerAvailable: Boolean = true, - config: Properties = new Properties) { + checkBrokerAvailable: Boolean = true) { val existingPartitionsReplicaList = ZkUtils.getReplicaAssignmentForTopics(zkClient, List(topic)) if (existingPartitionsReplicaList.size == 0) throw new AdminOperationException("The topic %s does not exist".format(topic)) @@ -137,7 +134,7 @@ object AdminUtils extends Logging { val partitionReplicaList = existingPartitionsReplicaList.map(p => p._1.partition -> p._2) // add the new list partitionReplicaList ++= newPartitionReplicaList - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, partitionReplicaList, config, true) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, partitionReplicaList, update = true) } def getManualReplicaAssignment(replicaAssignmentList: String, availableBrokerList: Set[Int], startPartitionId: Int, checkBrokerAvailable: Boolean = true): Map[Int, List[Int]] = { @@ -238,7 +235,7 @@ object AdminUtils extends Logging { val replicaAssignment = AdminUtils.assignReplicasToBrokers(brokerList, partitions, replicationFactor) AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, replicaAssignment, topicConfig) } - + def createOrUpdateTopicPartitionAssignmentPathInZK(zkClient: ZkClient, topic: String, partitionReplicaAssignment: Map[Int, Seq[Int]], @@ -246,7 +243,6 @@ object AdminUtils extends Logging { update: Boolean = false) { // validate arguments Topic.validate(topic) - LogConfig.validate(config) require(partitionReplicaAssignment.values.map(_.size).toSet.size == 1, "All partitions should have the same number of replicas.") val topicPath = ZkUtils.getTopicPath(topic) @@ -264,10 +260,14 @@ object AdminUtils extends Logging { } partitionReplicaAssignment.values.foreach(reps => require(reps.size == reps.toSet.size, "Duplicate replica assignment found: " + partitionReplicaAssignment)) - - // write out the config if there is any, this isn't transactional with the partition assignments - writeTopicConfig(zkClient, topic, config) - + + // Configs only matter if a topic is being created. Changing configs via AlterTopic is not supported + if (!update) { + // write out the config if there is any, this isn't transactional with the partition assignments + LogConfig.validate(config) + writeEntityConfig(zkClient, ConfigType.Topic, topic, config) + } + // create the partition assignment writeTopicPartitionAssignment(zkClient, topic, partitionReplicaAssignment, update) } @@ -290,7 +290,19 @@ object AdminUtils extends Logging { case e2: Throwable => throw new AdminOperationException(e2.toString) } } - + + /** + * Update the config for a client and create a change notification so the change will propagate to other brokers + * @param zkClient: The ZkClient handle used to write the new config to zookeeper + * @param clientId: The clientId for which configs are being changed + * @param configs: The final set of configs that will be applied to the topic. If any new configs need to be added or + * existing configs need to be deleted, it should be done prior to invoking this API + * + */ + def changeClientIdConfig(zkClient: ZkClient, clientId: String, configs: Properties) { + changeEntityConfig(zkClient, ConfigType.Client, clientId, configs) + } + /** * Update the config for an existing topic and create a change notification so the change will propagate to other brokers * @param zkClient: The ZkClient handle used to write the new config to zookeeper @@ -302,34 +314,42 @@ object AdminUtils extends Logging { def changeTopicConfig(zkClient: ZkClient, topic: String, configs: Properties) { if(!topicExists(zkClient, topic)) throw new AdminOperationException("Topic \"%s\" does not exist.".format(topic)) - // remove the topic overrides LogConfig.validate(configs) + changeEntityConfig(zkClient, ConfigType.Topic, topic, configs) + } + private def changeEntityConfig(zkClient: ZkClient, entityType: String, entityName: String, configs: Properties) { // write the new config--may not exist if there were previously no overrides - writeTopicConfig(zkClient, topic, configs) - + writeEntityConfig(zkClient, entityType, entityName, configs) + // create the change notification - zkClient.createPersistentSequential(ZkUtils.TopicConfigChangesPath + "/" + TopicConfigChangeZnodePrefix, Json.encode(topic)) + val seqNode = ZkUtils.EntityConfigChangesPath + "/" + EntityConfigChangeZnodePrefix + val content = Json.encode(getConfigChangeZnodeData(entityType, entityName)) + zkClient.createPersistentSequential(seqNode, content) } - + + def getConfigChangeZnodeData(entityType: String, entityName: String) : Map[String, Any] = { + Map("version" -> 1, "entity_type" -> entityType, "entity_name" -> entityName) + } + /** * Write out the topic config to zk, if there is any */ - private def writeTopicConfig(zkClient: ZkClient, topic: String, config: Properties) { + private def writeEntityConfig(zkClient: ZkClient, entityType: String, entityName: String, config: Properties) { val configMap: mutable.Map[String, String] = { import JavaConversions._ config } val map = Map("version" -> 1, "config" -> configMap) - ZkUtils.updatePersistentPath(zkClient, ZkUtils.getTopicConfigPath(topic), Json.encode(map)) + ZkUtils.updatePersistentPath(zkClient, ZkUtils.getEntityConfigPath(entityType, entityName), Json.encode(map)) } /** - * Read the topic config (if any) from zk + * Read the entity (topic or client) config (if any) from zk */ - def fetchTopicConfig(zkClient: ZkClient, topic: String): Properties = { - val str: String = zkClient.readData(ZkUtils.getTopicConfigPath(topic), true) + def fetchEntityConfig(zkClient: ZkClient, entityType: String, entity: String): Properties = { + val str: String = zkClient.readData(ZkUtils.getEntityConfigPath(entityType, entity), true) val props = new Properties() if(str != null) { Json.parseFull(str) match { @@ -343,19 +363,20 @@ object AdminUtils extends Logging { configTup match { case (k: String, v: String) => props.setProperty(k, v) - case _ => throw new IllegalArgumentException("Invalid topic config: " + str) + case _ => throw new IllegalArgumentException("Invalid " + entityType + " config: " + str) } - case _ => throw new IllegalArgumentException("Invalid topic config: " + str) + case _ => throw new IllegalArgumentException("Invalid " + entityType + " config: " + str) } - case o => throw new IllegalArgumentException("Unexpected value in config: " + str) + case o => throw new IllegalArgumentException("Unexpected value in config:(%s), entity_type: (%s), entity: (%s)" + .format(str, entityType, entity)) } } props } def fetchAllTopicConfigs(zkClient: ZkClient): Map[String, Properties] = - ZkUtils.getAllTopics(zkClient).map(topic => (topic, fetchTopicConfig(zkClient, topic))).toMap + ZkUtils.getAllTopics(zkClient).map(topic => (topic, fetchEntityConfig(zkClient, ConfigType.Topic, topic))).toMap def fetchTopicMetadataFromZk(topic: String, zkClient: ZkClient): TopicMetadata = fetchTopicMetadataFromZk(topic, zkClient, new mutable.HashMap[Int, Broker]) diff --git a/core/src/main/scala/kafka/admin/ConfigCommand.scala b/core/src/main/scala/kafka/admin/ConfigCommand.scala new file mode 100644 index 0000000..2759476 --- /dev/null +++ b/core/src/main/scala/kafka/admin/ConfigCommand.scala @@ -0,0 +1,174 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.admin + +import joptsimple._ +import java.util.Properties +import kafka.log.LogConfig +import kafka.server.ConfigType +import kafka.utils.{ZkUtils, CommandLineUtils} +import org.I0Itec.zkclient.ZkClient +import scala.collection._ +import scala.collection.JavaConversions._ +import org.apache.kafka.common.utils.Utils + + +/** + * This script can be used to change configs for topics/clients dynamically + */ +object ConfigCommand { + + def main(args: Array[String]): Unit = { + + val opts = new ConfigCommandOptions(args) + + if(args.length == 0) + CommandLineUtils.printUsageAndDie(opts.parser, "Add/Remove entity (topic/client) configs") + + opts.checkArgs() + + val zkClient = ZkUtils.createZkClient(opts.options.valueOf(opts.zkConnectOpt), 30000, 30000) + + try { + if (opts.options.has(opts.alterOpt)) + alterConfig(zkClient, opts) + else if (opts.options.has(opts.describeOpt)) + describeConfig(zkClient, opts) + } catch { + case e: Throwable => + println("Error while executing topic command " + e.getMessage) + println(Utils.stackTrace(e)) + } finally { + zkClient.close() + } + } + + private def alterConfig(zkClient: ZkClient, opts: ConfigCommandOptions) { + val configsToBeAdded = parseConfigsToBeAdded(opts) + val configsToBeDeleted = parseConfigsToBeDeleted(opts) + val entityType = opts.options.valueOf(opts.entityType) + val entityName = opts.options.valueOf(opts.entityName) + + // compile the final set of configs + val configs = AdminUtils.fetchEntityConfig(zkClient, entityType, entityName) + configs.putAll(configsToBeAdded) + configsToBeDeleted.foreach(config => configs.remove(config)) + + if (entityType.equals(ConfigType.Topic)) { + AdminUtils.changeTopicConfig(zkClient, entityName, configs) + println("Updated config for topic: \"%s\".".format(entityName)) + } else { + AdminUtils.changeClientIdConfig(zkClient, entityName, configs) + println("Updated config for clientId: \"%s\".".format(entityName)) + } + } + + private def describeConfig(zkClient: ZkClient, opts: ConfigCommandOptions) { + val entityType = opts.options.valueOf(opts.entityType) + val entityNames: Seq[String] = + if (opts.options.has(opts.entityName)) + Seq(opts.options.valueOf(opts.entityName)) + else + ZkUtils.getAllEntitiesWithConfig(zkClient, entityType) + + for (entityName <- entityNames) { + val configs = AdminUtils.fetchEntityConfig(zkClient, entityType, entityName) + println("Configs for %s:%s are %s" + .format(entityType, entityName, configs.map(kv => kv._1 + "=" + kv._2).mkString(","))) + } + } + + private[admin] def parseConfigsToBeAdded(opts: ConfigCommandOptions): Properties = { + val configsToBeAdded = opts.options.valuesOf(opts.addedConfig).map(_.split("""\s*=\s*""")) + require(configsToBeAdded.forall(config => config.length == 2), + "Invalid entity config: all configs to be added must be in the format \"key=val\".") + val props = new Properties + configsToBeAdded.foreach(pair => props.setProperty(pair(0).trim, pair(1).trim)) + props + } + + private[admin] def parseConfigsToBeDeleted(opts: ConfigCommandOptions): Seq[String] = { + if (opts.options.has(opts.deletedConfig)) { + val configsToBeDeleted = opts.options.valuesOf(opts.deletedConfig).map(_.trim()) + val propsToBeDeleted = new Properties + configsToBeDeleted.foreach(propsToBeDeleted.setProperty(_, "")) + configsToBeDeleted + } + else + Seq.empty + } + + class ConfigCommandOptions(args: Array[String]) { + val parser = new OptionParser + val zkConnectOpt = parser.accepts("zookeeper", "REQUIRED: The connection string for the zookeeper connection in the form host:port. " + + "Multiple URLS can be given to allow fail-over.") + .withRequiredArg + .describedAs("urls") + .ofType(classOf[String]) + val alterOpt = parser.accepts("alter", "Alter the configuration for the entity.") + val describeOpt = parser.accepts("describe", "List configs for the given entity.") + val entityType = parser.accepts("entity-type", "Type of entity (topic/client)") + .withRequiredArg + .ofType(classOf[String]) + val entityName = parser.accepts("entity-name", "Name of entity (topic name/client id)") + .withRequiredArg + .ofType(classOf[String]) + + val nl = System.getProperty("line.separator") + val addedConfig = parser.accepts("added-config", "Key Value pairs configs to add 'k1=v1,k2=v2'. The following is a list of valid configurations: " + + "For entity_type '" + ConfigType.Topic + "': " + nl + LogConfig.configNames.map("\t" + _).mkString(nl) + nl + + "For entity_type '" + ConfigType.Client + "' currently no configs are processed by the brokers") + .withRequiredArg + .ofType(classOf[String]) + .withValuesSeparatedBy(',') + val deletedConfig = parser.accepts("deleted-config", "config keys to remove 'k1,k2'") + .withRequiredArg + .ofType(classOf[String]) + .withValuesSeparatedBy(',') + val helpOpt = parser.accepts("help", "Print usage information.") + val options = parser.parse(args : _*) + + val allOpts: Set[OptionSpec[_]] = Set(alterOpt, describeOpt, entityType, entityName, addedConfig, deletedConfig, helpOpt) + + def checkArgs() { + // should have exactly one action + val actions = Seq(alterOpt, describeOpt).count(options.has _) + if(actions != 1) + CommandLineUtils.printUsageAndDie(parser, "Command must include exactly one action: --describe, --alter") + + // check required args + CommandLineUtils.checkRequiredArgs(parser, options, zkConnectOpt, entityType) + CommandLineUtils.checkInvalidArgs(parser, options, alterOpt, Set(describeOpt)) + CommandLineUtils.checkInvalidArgs(parser, options, describeOpt, Set(alterOpt, addedConfig, deletedConfig)) + if(options.has(alterOpt)) { + if(! options.has(entityName)) + throw new IllegalArgumentException("--entity-name must be specified with --alter") + + val isAddedPresent: Boolean = options.has(addedConfig) + val isDeletedPresent: Boolean = options.has(deletedConfig) + if(! isAddedPresent && ! isDeletedPresent) + throw new IllegalArgumentException("At least one of --added-config or --deleted-config must be specified with --alter") + } + val entityTypeVal = options.valueOf(entityType) + if(! entityTypeVal.equals(ConfigType.Topic) && ! entityTypeVal.equals(ConfigType.Client)) { + throw new IllegalArgumentException("--entity-type must be '%s' or '%s'".format(ConfigType.Topic, ConfigType.Client)) + } + } + } + +} diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala index 4e28bf1..f1405a5 100755 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -20,6 +20,7 @@ package kafka.admin import joptsimple._ import java.util.Properties import kafka.common.{Topic, AdminCommandFailedException} +import kafka.utils.CommandLineUtils import kafka.utils._ import org.I0Itec.zkclient.ZkClient import org.I0Itec.zkclient.exception.ZkNodeExistsException @@ -27,6 +28,7 @@ import scala.collection._ import scala.collection.JavaConversions._ import kafka.log.LogConfig import kafka.consumer.Whitelist +import kafka.server.{ConfigType, OffsetManager} import org.apache.kafka.common.utils.Utils import kafka.coordinator.ConsumerCoordinator @@ -106,16 +108,6 @@ object TopicCommand extends Logging { opts.options.valueOf(opts.zkConnectOpt))) } topics.foreach { topic => - val configs = AdminUtils.fetchTopicConfig(zkClient, topic) - if(opts.options.has(opts.configOpt) || opts.options.has(opts.deleteConfigOpt)) { - val configsToBeAdded = parseTopicConfigsToBeAdded(opts) - val configsToBeDeleted = parseTopicConfigsToBeDeleted(opts) - // compile the final set of configs - configs.putAll(configsToBeAdded) - configsToBeDeleted.foreach(config => configs.remove(config)) - AdminUtils.changeTopicConfig(zkClient, topic, configs) - println("Updated config for topic \"%s\".".format(topic)) - } if(opts.options.has(opts.partitionsOpt)) { if (topic == ConsumerCoordinator.OffsetsTopicName) { throw new IllegalArgumentException("The number of partitions for the offsets topic cannot be changed.") @@ -124,7 +116,7 @@ object TopicCommand extends Logging { "logic or ordering of the messages will be affected") val nPartitions = opts.options.valueOf(opts.partitionsOpt).intValue val replicaAssignmentStr = opts.options.valueOf(opts.replicaAssignmentOpt) - AdminUtils.addPartitions(zkClient, topic, nPartitions, replicaAssignmentStr, config = configs) + AdminUtils.addPartitions(zkClient, topic, nPartitions, replicaAssignmentStr) println("Adding partitions succeeded!") } } @@ -180,7 +172,7 @@ object TopicCommand extends Logging { val describePartitions: Boolean = !reportOverriddenConfigs val sortedPartitions = topicPartitionAssignment.toList.sortWith((m1, m2) => m1._1 < m2._1) if (describeConfigs) { - val configs = AdminUtils.fetchTopicConfig(zkClient, topic) + val configs = AdminUtils.fetchEntityConfig(zkClient, ConfigType.Topic, topic) if (!reportOverriddenConfigs || configs.size() != 0) { val numPartitions = topicPartitionAssignment.size val replicationFactor = topicPartitionAssignment.head._2.size @@ -219,18 +211,6 @@ object TopicCommand extends Logging { props } - def parseTopicConfigsToBeDeleted(opts: TopicCommandOptions): Seq[String] = { - if (opts.options.has(opts.deleteConfigOpt)) { - val configsToBeDeleted = opts.options.valuesOf(opts.deleteConfigOpt).map(_.trim()) - val propsToBeDeleted = new Properties - configsToBeDeleted.foreach(propsToBeDeleted.setProperty(_, "")) - LogConfig.validateNames(propsToBeDeleted) - configsToBeDeleted - } - else - Seq.empty - } - def parseReplicaAssignment(replicaAssignmentList: String): Map[Int, List[Int]] = { val partitionList = replicaAssignmentList.split(",") val ret = new mutable.HashMap[Int, List[Int]]() @@ -256,7 +236,7 @@ object TopicCommand extends Logging { val listOpt = parser.accepts("list", "List all available topics.") val createOpt = parser.accepts("create", "Create a new topic.") val deleteOpt = parser.accepts("delete", "Delete a topic") - val alterOpt = parser.accepts("alter", "Alter the configuration for the topic.") + val alterOpt = parser.accepts("alter", "Alter the number of partitions and/or replica assignment for a topic") val describeOpt = parser.accepts("describe", "List details for the given topics.") val helpOpt = parser.accepts("help", "Print usage information.") val topicOpt = parser.accepts("topic", "The topic to be create, alter or describe. Can also accept a regular " + @@ -265,16 +245,12 @@ object TopicCommand extends Logging { .describedAs("topic") .ofType(classOf[String]) val nl = System.getProperty("line.separator") - val configOpt = parser.accepts("config", "A topic configuration override for the topic being created or altered." + + val configOpt = parser.accepts("config", "A configuration override for the topic being created." + "The following is a list of valid configurations: " + nl + LogConfig.configNames.map("\t" + _).mkString(nl) + nl + "See the Kafka documentation for full details on the topic configs.") .withRequiredArg .describedAs("name=value") .ofType(classOf[String]) - val deleteConfigOpt = parser.accepts("delete-config", "A topic configuration override to be removed for an existing topic (see the list of configurations under the --config option).") - .withRequiredArg - .describedAs("name") - .ofType(classOf[String]) val partitionsOpt = parser.accepts("partitions", "The number of partitions for the topic being created or " + "altered (WARNING: If partitions are increased for a topic that has a key, the partition logic or ordering of the messages will be affected") .withRequiredArg @@ -308,10 +284,11 @@ object TopicCommand extends Logging { // check invalid args CommandLineUtils.checkInvalidArgs(parser, options, configOpt, allTopicLevelOpts -- Set(alterOpt, createOpt)) - CommandLineUtils.checkInvalidArgs(parser, options, deleteConfigOpt, allTopicLevelOpts -- Set(alterOpt)) CommandLineUtils.checkInvalidArgs(parser, options, partitionsOpt, allTopicLevelOpts -- Set(alterOpt, createOpt)) CommandLineUtils.checkInvalidArgs(parser, options, replicationFactorOpt, allTopicLevelOpts -- Set(createOpt)) CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, allTopicLevelOpts -- Set(createOpt,alterOpt)) + // Topic configs cannot be changed with alterTopic + CommandLineUtils.checkInvalidArgs(parser, options, alterOpt, Set(configOpt)) if(options.has(createOpt)) CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, Set(partitionsOpt, replicationFactorOpt)) CommandLineUtils.checkInvalidArgs(parser, options, reportUnderReplicatedPartitionsOpt, diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 2649090..511d3c9 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -22,7 +22,7 @@ import kafka.utils.CoreUtils.{inReadLock,inWriteLock} import kafka.admin.AdminUtils import kafka.api.{PartitionStateInfo, LeaderAndIsr} import kafka.log.LogConfig -import kafka.server.{TopicPartitionOperationKey, LogOffsetMetadata, LogReadResult, ReplicaManager} +import kafka.server._ import kafka.metrics.KafkaMetricsGroup import kafka.controller.KafkaController import kafka.message.ByteBufferMessageSet @@ -86,7 +86,8 @@ class Partition(val topic: String, case Some(replica) => replica case None => if (isReplicaLocal(replicaId)) { - val config = LogConfig.fromProps(logManager.defaultConfig.originals, AdminUtils.fetchTopicConfig(zkClient, topic)) + val config = LogConfig.fromProps(logManager.defaultConfig.originals, + AdminUtils.fetchEntityConfig(zkClient, ConfigType.Topic, topic)) val log = logManager.createLog(TopicAndPartition(topic, partitionId), config) val checkpoint = replicaManager.highWatermarkCheckpoints(log.dir.getParentFile.getAbsolutePath) val offsetMap = checkpoint.read diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index b4fc755..6844602 100755 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -1037,8 +1037,8 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt // if the replica to be removed from the ISR is the last surviving member of the ISR and unclean leader election // is disallowed for the corresponding topic, then we must preserve the ISR membership so that the replica can // eventually be restored as the leader. - if (newIsr.isEmpty && !LogConfig.fromProps(config.originals, AdminUtils.fetchTopicConfig(zkClient, - topicAndPartition.topic)).uncleanLeaderElectionEnable) { + if (newIsr.isEmpty && !LogConfig.fromProps(config.originals, AdminUtils.fetchEntityConfig(zkClient, + ConfigType.Topic, topicAndPartition.topic)).uncleanLeaderElectionEnable) { info("Retaining last ISR %d of partition %s since unclean leader election is disabled".format(replicaId, topicAndPartition)) newIsr = leaderAndIsr.isr } @@ -1322,7 +1322,8 @@ class IsrChangeNotificationListener(controller: KafkaController) extends IZkChil processUpdateNotifications(topicAndPartitions) // delete processed children - childrenAsScala.map(x => ZkUtils.deletePath(controller.controllerContext.zkClient, ZkUtils.TopicConfigChangesPath + "/" + x)) + childrenAsScala.map(x => ZkUtils.deletePath(controller.controllerContext.zkClient, + ZkUtils.getEntityConfigPath(ConfigType.Topic, x))) } } diff --git a/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala b/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala index bb6b5c8..4ebeb5a 100644 --- a/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala +++ b/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala @@ -21,7 +21,7 @@ import kafka.api.LeaderAndIsr import kafka.log.LogConfig import kafka.utils.Logging import kafka.common.{LeaderElectionNotNeededException, TopicAndPartition, StateChangeFailedException, NoReplicaOnlineException} -import kafka.server.KafkaConfig +import kafka.server.{ConfigType, KafkaConfig} trait PartitionLeaderSelector { @@ -61,8 +61,8 @@ class OfflinePartitionLeaderSelector(controllerContext: ControllerContext, confi case true => // Prior to electing an unclean (i.e. non-ISR) leader, ensure that doing so is not disallowed by the configuration // for unclean leader election. - if (!LogConfig.fromProps(config.originals, AdminUtils.fetchTopicConfig(controllerContext.zkClient, - topicAndPartition.topic)).uncleanLeaderElectionEnable) { + if (!LogConfig.fromProps(config.originals, AdminUtils.fetchEntityConfig(controllerContext.zkClient, + ConfigType.Topic, topicAndPartition.topic)).uncleanLeaderElectionEnable) { throw new NoReplicaOnlineException(("No broker in ISR for partition " + "%s is alive. Live brokers are: [%s],".format(topicAndPartition, controllerContext.liveBrokerIds)) + " ISR brokers are: [%s]".format(currentLeaderAndIsr.isr.mkString(","))) diff --git a/core/src/main/scala/kafka/controller/TopicDeletionManager.scala b/core/src/main/scala/kafka/controller/TopicDeletionManager.scala index 64ecb49..64b11df 100755 --- a/core/src/main/scala/kafka/controller/TopicDeletionManager.scala +++ b/core/src/main/scala/kafka/controller/TopicDeletionManager.scala @@ -16,6 +16,9 @@ */ package kafka.controller + +import kafka.server.ConfigType + import collection.mutable import kafka.utils.{ShutdownableThread, Logging, ZkUtils} import kafka.utils.CoreUtils._ @@ -284,7 +287,7 @@ class TopicDeletionManager(controller: KafkaController, topicsToBeDeleted -= topic partitionsToBeDeleted.retain(_.topic != topic) controllerContext.zkClient.deleteRecursive(ZkUtils.getTopicPath(topic)) - controllerContext.zkClient.deleteRecursive(ZkUtils.getTopicConfigPath(topic)) + controllerContext.zkClient.deleteRecursive(ZkUtils.getEntityConfigPath(ConfigType.Topic, topic)) controllerContext.zkClient.delete(ZkUtils.getDeleteTopicPath(topic)) controllerContext.removeTopic(topic) } diff --git a/core/src/main/scala/kafka/server/ConfigHandler.scala b/core/src/main/scala/kafka/server/ConfigHandler.scala new file mode 100644 index 0000000..8347a69 --- /dev/null +++ b/core/src/main/scala/kafka/server/ConfigHandler.scala @@ -0,0 +1,69 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server + +import java.util.Properties + +import kafka.common.TopicAndPartition +import kafka.log.{Log, LogConfig, LogManager} +import kafka.utils.Pool + +import scala.collection.mutable + +/** + * The ConfigHandler is used to process config change notifications received by the DynamicConfigManager + */ +trait ConfigHandler { + def processConfigChanges(entityName : String, value : Properties) +} + +/** + * The TopicConfigHandler will process topic config changes in ZK. + * The callback provides the topic name and the full properties set read from ZK + */ +class TopicConfigHandler(private val logManager: LogManager) extends ConfigHandler{ + + def processConfigChanges(topic : String, topicConfig : Properties) { + val logs: mutable.Buffer[(TopicAndPartition, Log)] = logManager.logsByTopicPartition.toBuffer + val logsByTopic: Map[String, mutable.Buffer[Log]] = logs.groupBy{ case (topicAndPartition, log) => topicAndPartition.topic } + .mapValues{ case v: mutable.Buffer[(TopicAndPartition, Log)] => v.map(_._2) } + + if (logsByTopic.contains(topic)) { + /* combine the default properties with the overrides in zk to create the new LogConfig */ + val props = new Properties() + props.putAll(logManager.defaultConfig.originals) + props.putAll(topicConfig) + val logConfig = LogConfig(props) + for (log <- logsByTopic(topic)) + log.config = logConfig + } + } +} + +/** + * The ClientIdConfigHandler will process clientId config changes in ZK. + * The callback provides the clientId and the full properties set read from ZK. + * This implementation does nothing currently. In the future, it will change quotas per client + */ +class ClientIdConfigHandler extends ConfigHandler { + val configPool = new Pool[String, Properties]() + + def processConfigChanges(clientId : String, clientConfig : Properties): Unit = { + configPool.put(clientId, clientConfig) + } +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/DynamicConfigManager.scala b/core/src/main/scala/kafka/server/DynamicConfigManager.scala new file mode 100644 index 0000000..a66fb75 --- /dev/null +++ b/core/src/main/scala/kafka/server/DynamicConfigManager.scala @@ -0,0 +1,183 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server + +import kafka.utils.Json +import kafka.utils.Logging +import kafka.utils.SystemTime +import kafka.utils.Time +import kafka.utils.ZkUtils + +import scala.collection._ +import kafka.admin.AdminUtils +import org.I0Itec.zkclient.{IZkChildListener, ZkClient} + + +/** + * Represents all the entities that can be configured via ZK + */ +object ConfigType { + val Topic = "topic" + val Client = "client" +} + +/** + * This class initiates and carries out config changes for all entities defined in ConfigType. + * + * It works as follows. + * + * Config is stored under the path: /config/entityType/entityName + * E.g. /config/topics/ and /config/clients/ + * This znode stores the overrides for this entity (but no defaults) in properties format. + * + * To avoid watching all topics for changes instead we have a notification path + * /config/changes + * The DynamicConfigManager has a child watch on this path. + * + * To update a config we first update the config properties. Then we create a new sequential + * znode under the change path which contains the name of the entityType and entityName that was updated, say + * /config/changes/config_change_13321 + * The sequential znode contains data in this format: {"version" : 1, "entityType":"topic/client", "entityName" : "topic_name/client_id"} + * This is just a notification--the actual config change is stored only once under the /config/entityType/entityName path. + * + * This will fire a watcher on all brokers. This watcher works as follows. It reads all the config change notifications. + * It keeps track of the highest config change suffix number it has applied previously. For any previously applied change it finds + * it checks if this notification is larger than a static expiration time (say 10mins) and if so it deletes this notification. + * For any new changes it reads the new configuration, combines it with the defaults, and updates the existing config. + * + * Note that config is always read from the config path in zk, the notification is just a trigger to do so. So if a broker is + * down and misses a change that is fine--when it restarts it will be loading the full config anyway. Note also that + * if there are two consecutive config changes it is possible that only the last one will be applied (since by the time the + * broker reads the config the both changes may have been made). In this case the broker would needlessly refresh the config twice, + * but that is harmless. + * + * On restart the config manager re-processes all notifications. This will usually be wasted work, but avoids any race conditions + * on startup where a change might be missed between the initial config load and registering for change notifications. + * + */ +class DynamicConfigManager(private val zkClient: ZkClient, + private val configHandler : Map[String, ConfigHandler], + private val changeExpirationMs: Long = 15*60*1000, + private val time: Time = SystemTime) extends Logging { + private var lastExecutedChange = -1L + + /** + * Begin watching for config changes + */ + def startup() { + ZkUtils.makeSurePersistentPathExists(zkClient, ZkUtils.EntityConfigChangesPath) + zkClient.subscribeChildChanges(ZkUtils.EntityConfigChangesPath, ConfigChangeListener) + processAllConfigChanges() + } + + /** + * Process all config changes + */ + private def processAllConfigChanges() { + val configChanges = zkClient.getChildren(ZkUtils.EntityConfigChangesPath) + import JavaConversions._ + processConfigChanges((configChanges: mutable.Buffer[String]).sorted) + } + + /** + * Process the given list of config changes + */ + private def processConfigChanges(notifications: Seq[String]) { + if (notifications.size > 0) { + info("Processing config change notification(s)...") + val now = time.milliseconds + for (notification <- notifications) { + val changeId = changeNumber(notification) + + if (changeId > lastExecutedChange) { + val changeZnode = ZkUtils.EntityConfigChangesPath + "/" + notification + + val (jsonOpt, stat) = ZkUtils.readDataMaybeNull(zkClient, changeZnode) + processNotification(jsonOpt) + } + lastExecutedChange = changeId + } + purgeObsoleteNotifications(now, notifications) + } + } + + def processNotification(jsonOpt: Option[String]) = { + if(jsonOpt.isDefined) { + val json = jsonOpt.get + Json.parseFull(json) match { + case None => // There are no config overrides. + // Ignore non-json notifications because they can be from the deprecated TopicConfigManager + case Some(mapAnon: Map[_, _]) => + val map = mapAnon collect + { case (k: String, v: Any) => k -> v } + require(map("version") == 1) + + val entityType = map.get("entity_type") match { + case Some(ConfigType.Topic) => ConfigType.Topic + case Some(ConfigType.Client) => ConfigType.Client + case _ => throw new IllegalArgumentException("Config change notification must have 'entity_type' set to either 'client' or 'topic'." + + " Received: " + json) + } + + val entity = map.get("entity_name") match { + case Some(value: String) => value + case _ => throw new IllegalArgumentException("Config change notification does not specify 'entity_name'. Received: " + json) + } + configHandler(entityType).processConfigChanges(entity, AdminUtils.fetchEntityConfig(zkClient, entityType, entity)) + + case o => throw new IllegalArgumentException("Config change notification has an unexpected value. The format is:" + + "{\"version\" : 1," + + " \"entity_type\":\"topic/client\"," + + " \"entity_name\" : \"topic_name/client_id\"}." + + " Received: " + json) + } + } + } + + private def purgeObsoleteNotifications(now: Long, notifications: Seq[String]) { + for(notification <- notifications.sorted) { + val (jsonOpt, stat) = ZkUtils.readDataMaybeNull(zkClient, ZkUtils.EntityConfigChangesPath + "/" + notification) + if(jsonOpt.isDefined) { + val changeZnode = ZkUtils.EntityConfigChangesPath + "/" + notification + if (now - stat.getCtime > changeExpirationMs) { + debug("Purging config change notification " + notification) + ZkUtils.deletePath(zkClient, changeZnode) + } else { + return + } + } + } + } + + /* get the change number from a change notification znode */ + private def changeNumber(name: String): Long = name.substring(AdminUtils.EntityConfigChangeZnodePrefix.length).toLong + + /** + * A listener that applies config changes to logs + */ + object ConfigChangeListener extends IZkChildListener { + override def handleChildChange(path: String, chillins: java.util.List[String]) { + try { + import JavaConversions._ + processConfigChanges(chillins: mutable.Buffer[String]) + } catch { + case e: Exception => error("Error processing config change:", e) + } + } + } +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 18917bc..84d4730 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -31,8 +31,9 @@ import java.io.File import kafka.utils._ import org.apache.kafka.common.metrics._ import org.apache.kafka.common.network.NetworkReceive +import org.apache.kafka.common.metrics.{JmxReporter, Metrics} -import scala.collection.{JavaConversions, mutable} +import scala.collection.mutable import org.I0Itec.zkclient.ZkClient import kafka.controller.{ControllerStats, KafkaController} import kafka.cluster.{EndPoint, Broker} @@ -77,7 +78,9 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg var replicaManager: ReplicaManager = null - var topicConfigManager: TopicConfigManager = null + var dynamicConfigHandlers: Map[String, ConfigHandler] = null + var dynamicConfigManager: DynamicConfigManager = null + val metrics: Metrics = new Metrics() var consumerCoordinator: ConsumerCoordinator = null @@ -171,9 +174,11 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg Mx4jLoader.maybeLoad() - /* start topic config manager */ - topicConfigManager = new TopicConfigManager(zkClient, logManager) - topicConfigManager.startup() + /* start dynamic config manager */ + dynamicConfigHandlers = Map[String, ConfigHandler](ConfigType.Topic -> new TopicConfigHandler(logManager), + ConfigType.Client -> new ClientIdConfigHandler) + dynamicConfigManager = new DynamicConfigManager(zkClient, dynamicConfigHandlers) + dynamicConfigManager.startup() /* tell everyone we are alive */ val listeners = config.advertisedListeners.map {case(protocol, endpoint) => diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index c89d00b..fae22d2 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -90,8 +90,8 @@ class ReplicaFetcherThread(name:String, // Prior to truncating the follower's log, ensure that doing so is not disallowed by the configuration for unclean leader election. // This situation could only happen if the unclean election configuration for a topic changes while a replica is down. Otherwise, // we should never encounter this situation since a non-ISR leader cannot be elected if disallowed by the broker configuration. - if (!LogConfig.fromProps(brokerConfig.originals, AdminUtils.fetchTopicConfig(replicaMgr.zkClient, - topicAndPartition.topic)).uncleanLeaderElectionEnable) { + if (!LogConfig.fromProps(brokerConfig.originals, AdminUtils.fetchEntityConfig(replicaMgr.zkClient, + ConfigType.Topic, topicAndPartition.topic)).uncleanLeaderElectionEnable) { // Log a fatal error and shutdown the broker to ensure that data loss does not unexpectedly occur. fatal("Halting because log truncation is not allowed for topic %s,".format(topicAndPartition.topic) + " Current leader %d's latest offset %d is less than replica %d's latest offset %d" diff --git a/core/src/main/scala/kafka/server/TopicConfigManager.scala b/core/src/main/scala/kafka/server/TopicConfigManager.scala deleted file mode 100644 index 01b1b0a..0000000 --- a/core/src/main/scala/kafka/server/TopicConfigManager.scala +++ /dev/null @@ -1,152 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.server - -import java.util.Properties -import scala.collection._ -import kafka.log._ -import kafka.utils._ -import kafka.admin.AdminUtils -import org.I0Itec.zkclient.{IZkChildListener, ZkClient} - -/** - * This class initiates and carries out topic config changes. - * - * It works as follows. - * - * Config is stored under the path - * /config/topics/ - * This znode stores the topic-overrides for this topic (but no defaults) in properties format. - * - * To avoid watching all topics for changes instead we have a notification path - * /config/changes - * The TopicConfigManager has a child watch on this path. - * - * To update a topic config we first update the topic config properties. Then we create a new sequential - * znode under the change path which contains the name of the topic that was updated, say - * /config/changes/config_change_13321 - * This is just a notification--the actual config change is stored only once under the /config/topics/ path. - * - * This will fire a watcher on all brokers. This watcher works as follows. It reads all the config change notifications. - * It keeps track of the highest config change suffix number it has applied previously. For any previously applied change it finds - * it checks if this notification is larger than a static expiration time (say 10mins) and if so it deletes this notification. - * For any new changes it reads the new configuration, combines it with the defaults, and updates the log config - * for all logs for that topic (if any) that it has. - * - * Note that config is always read from the config path in zk, the notification is just a trigger to do so. So if a broker is - * down and misses a change that is fine--when it restarts it will be loading the full config anyway. Note also that - * if there are two consecutive config changes it is possible that only the last one will be applied (since by the time the - * broker reads the config the both changes may have been made). In this case the broker would needlessly refresh the config twice, - * but that is harmless. - * - * On restart the config manager re-processes all notifications. This will usually be wasted work, but avoids any race conditions - * on startup where a change might be missed between the initial config load and registering for change notifications. - * - */ -class TopicConfigManager(private val zkClient: ZkClient, - private val logManager: LogManager, - private val changeExpirationMs: Long = 15*60*1000, - private val time: Time = SystemTime) extends Logging { - private var lastExecutedChange = -1L - - /** - * Begin watching for config changes - */ - def startup() { - ZkUtils.makeSurePersistentPathExists(zkClient, ZkUtils.TopicConfigChangesPath) - zkClient.subscribeChildChanges(ZkUtils.TopicConfigChangesPath, ConfigChangeListener) - processAllConfigChanges() - } - - /** - * Process all config changes - */ - private def processAllConfigChanges() { - val configChanges = zkClient.getChildren(ZkUtils.TopicConfigChangesPath) - import JavaConversions._ - processConfigChanges((configChanges: mutable.Buffer[String]).sorted) - } - - /** - * Process the given list of config changes - */ - private def processConfigChanges(notifications: Seq[String]) { - if (notifications.size > 0) { - info("Processing config change notification(s)...") - val now = time.milliseconds - val logs = logManager.logsByTopicPartition.toBuffer - val logsByTopic = logs.groupBy(_._1.topic).mapValues(_.map(_._2)) - for (notification <- notifications) { - val changeId = changeNumber(notification) - if (changeId > lastExecutedChange) { - val changeZnode = ZkUtils.TopicConfigChangesPath + "/" + notification - val (jsonOpt, stat) = ZkUtils.readDataMaybeNull(zkClient, changeZnode) - if(jsonOpt.isDefined) { - val json = jsonOpt.get - val topic = json.substring(1, json.length - 1) // hacky way to dequote - if (logsByTopic.contains(topic)) { - /* combine the default properties with the overrides in zk to create the new LogConfig */ - val props = new Properties() - props.putAll(logManager.defaultConfig.originals) - props.putAll(AdminUtils.fetchTopicConfig(zkClient, topic)) - val logConfig = LogConfig(props) - for (log <- logsByTopic(topic)) - log.config = logConfig - info("Processed topic config change %d for topic %s, setting new config to %s.".format(changeId, topic, props)) - purgeObsoleteNotifications(now, notifications) - } - } - lastExecutedChange = changeId - } - } - } - } - - private def purgeObsoleteNotifications(now: Long, notifications: Seq[String]) { - for(notification <- notifications.sorted) { - val (jsonOpt, stat) = ZkUtils.readDataMaybeNull(zkClient, ZkUtils.TopicConfigChangesPath + "/" + notification) - if(jsonOpt.isDefined) { - val changeZnode = ZkUtils.TopicConfigChangesPath + "/" + notification - if (now - stat.getCtime > changeExpirationMs) { - debug("Purging config change notification " + notification) - ZkUtils.deletePath(zkClient, changeZnode) - } else { - return - } - } - } - } - - /* get the change number from a change notification znode */ - private def changeNumber(name: String): Long = name.substring(AdminUtils.TopicConfigChangeZnodePrefix.length).toLong - - /** - * A listener that applies config changes to logs - */ - object ConfigChangeListener extends IZkChildListener { - override def handleChildChange(path: String, chillins: java.util.List[String]) { - try { - import JavaConversions._ - processConfigChanges(chillins: mutable.Buffer[String]) - } catch { - case e: Exception => error("Error processing config change:", e) - } - } - } - -} \ No newline at end of file diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index 166814c..4ae310e 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -19,6 +19,7 @@ package kafka.utils import kafka.cluster._ import kafka.consumer.{ConsumerThreadId, TopicCount} +import kafka.server.ConfigType import org.I0Itec.zkclient.ZkClient import org.I0Itec.zkclient.exception.{ZkNodeExistsException, ZkNoNodeException, ZkMarshallingError, ZkBadVersionException} @@ -39,8 +40,6 @@ object ZkUtils extends Logging { val ConsumersPath = "/consumers" val BrokerIdsPath = "/brokers/ids" val BrokerTopicsPath = "/brokers/topics" - val TopicConfigPath = "/config/topics" - val TopicConfigChangesPath = "/config/changes" val ControllerPath = "/controller" val ControllerEpochPath = "/controller_epoch" val ReassignPartitionsPath = "/admin/reassign_partitions" @@ -48,6 +47,8 @@ object ZkUtils extends Logging { val PreferredReplicaLeaderElectionPath = "/admin/preferred_replica_election" val BrokerSequenceIdPath = "/brokers/seqid" val IsrChangeNotificationPath = "/isr_change_notification" + val EntityConfigPath = "/config" + val EntityConfigChangesPath = "/config/changes" def getTopicPath(topic: String): String = { BrokerTopicsPath + "/" + topic @@ -57,8 +58,11 @@ object ZkUtils extends Logging { getTopicPath(topic) + "/partitions" } - def getTopicConfigPath(topic: String): String = - TopicConfigPath + "/" + topic + def getEntityConfigRootPath(entityType: String): String = + EntityConfigPath + "/" + entityType + + def getEntityConfigPath(entityType: String, entity: String): String = + getEntityConfigRootPath(entityType) + "/" + entity def getDeleteTopicPath(topic: String): String = DeleteTopicsPath + "/" + topic @@ -93,8 +97,14 @@ object ZkUtils extends Logging { } def setupCommonPaths(zkClient: ZkClient) { - for(path <- Seq(ConsumersPath, BrokerIdsPath, BrokerTopicsPath, TopicConfigChangesPath, TopicConfigPath, - DeleteTopicsPath, BrokerSequenceIdPath)) + for(path <- Seq(ConsumersPath, + BrokerIdsPath, + BrokerTopicsPath, + EntityConfigChangesPath, + ZkUtils.getEntityConfigRootPath(ConfigType.Topic), + ZkUtils.getEntityConfigRootPath(ConfigType.Client), + DeleteTopicsPath, + BrokerSequenceIdPath)) makeSurePersistentPathExists(zkClient, path) } @@ -753,6 +763,17 @@ object ZkUtils extends Logging { topics } + /** + * Returns all the entities whose configs have been overridden. + */ + def getAllEntitiesWithConfig(zkClient: ZkClient, entityType: String): Seq[String] = { + val entities = ZkUtils.getChildrenParentMayNotExist(zkClient, getEntityConfigRootPath(entityType)) + if(entities == null) + Seq.empty[String] + else + entities + } + def getAllPartitions(zkClient: ZkClient): Set[TopicAndPartition] = { val topics = ZkUtils.getChildrenParentMayNotExist(zkClient, BrokerTopicsPath) if(topics == null) Set.empty[TopicAndPartition] diff --git a/core/src/test/scala/unit/kafka/admin/AdminTest.scala b/core/src/test/scala/unit/kafka/admin/AdminTest.scala index 93f200e..86dcc4c 100755 --- a/core/src/test/scala/unit/kafka/admin/AdminTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AdminTest.scala @@ -25,7 +25,7 @@ import kafka.log._ import kafka.zk.ZooKeeperTestHarness import kafka.utils.{Logging, ZkUtils, TestUtils} import kafka.common.{InvalidTopicException, TopicExistsException, TopicAndPartition} -import kafka.server.{KafkaServer, KafkaConfig} +import kafka.server.{ConfigType, KafkaServer, KafkaConfig} import java.io.File import TestUtils._ @@ -407,12 +407,16 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { checkConfig(maxMessageSize, retentionMs) // now double the config values for the topic and check that it is applied + val newConfig: Properties = makeConfig(2*maxMessageSize, 2 * retentionMs) AdminUtils.changeTopicConfig(server.zkClient, topic, makeConfig(2*maxMessageSize, 2 * retentionMs)) checkConfig(2*maxMessageSize, 2 * retentionMs) + + // Verify that the same config can be read from ZK + val configInZk = AdminUtils.fetchEntityConfig(server.zkClient, ConfigType.Topic, topic) + assertEquals(newConfig, configInZk) } finally { server.shutdown() server.config.logDirs.foreach(CoreUtils.rm(_)) } } - } diff --git a/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala b/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala new file mode 100644 index 0000000..cfe0ec3 --- /dev/null +++ b/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala @@ -0,0 +1,73 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package kafka.admin + +import junit.framework.Assert._ +import kafka.admin.ConfigCommand.ConfigCommandOptions +import org.junit.Test +import org.scalatest.junit.JUnit3Suite +import kafka.utils.Logging +import kafka.utils.TestUtils +import kafka.zk.ZooKeeperTestHarness +import kafka.server.{ConfigType, OffsetManager, KafkaConfig} +import kafka.admin.TopicCommand.TopicCommandOptions +import kafka.utils.ZkUtils + +class ConfigCommandTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { + @Test + def testArgumentParse() { + // Should parse correctly + var createOpts = new ConfigCommandOptions(Array("--zookeeper", zkConnect, + "--entity-name", "x", + "--entity-type", "client", + "--describe")) + createOpts.checkArgs() + + // For --alter and added config + createOpts = new ConfigCommandOptions(Array("--zookeeper", zkConnect, + "--entity-name", "x", + "--entity-type", "client", + "--alter", + "--added-config", "a=b,c=d")) + createOpts.checkArgs() + + // For alter and deleted config + createOpts = new ConfigCommandOptions(Array("--zookeeper", zkConnect, + "--entity-name", "x", + "--entity-type", "client", + "--alter", + "--deleted-config", "a,b,c")) + createOpts.checkArgs() + + // For alter and both added, deleted config + createOpts = new ConfigCommandOptions(Array("--zookeeper", zkConnect, + "--entity-name", "x", + "--entity-type", "client", + "--alter", + "--added-config", "a=b,c=d", + "--deleted-config", "a")) + createOpts.checkArgs() + val addedProps = ConfigCommand.parseConfigsToBeAdded(createOpts) + assertEquals(2, addedProps.size()) + assertEquals("b", addedProps.getProperty("a")) + assertEquals("d", addedProps.getProperty("c")) + + val deletedProps = ConfigCommand.parseConfigsToBeDeleted(createOpts) + assertEquals(1, deletedProps.size) + assertEquals("a", deletedProps(0)) + } +} \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala index dcd6988..58adef6 100644 --- a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala @@ -22,6 +22,7 @@ import org.scalatest.junit.JUnit3Suite import kafka.utils.Logging import kafka.utils.TestUtils import kafka.zk.ZooKeeperTestHarness +import kafka.server.{ConfigType, OffsetManager, KafkaConfig} import kafka.admin.TopicCommand.TopicCommandOptions import kafka.utils.ZkUtils import kafka.coordinator.ConsumerCoordinator @@ -43,20 +44,18 @@ class TopicCommandTest extends JUnit3Suite with ZooKeeperTestHarness with Loggin "--config", cleanupKey + "=" + cleanupVal, "--topic", topic)) TopicCommand.createTopic(zkClient, createOpts) - val props = AdminUtils.fetchTopicConfig(zkClient, topic) + val props = AdminUtils.fetchEntityConfig(zkClient, ConfigType.Topic, topic) assertTrue("Properties after creation don't contain " + cleanupKey, props.containsKey(cleanupKey)) assertTrue("Properties after creation have incorrect value", props.getProperty(cleanupKey).equals(cleanupVal)) // pre-create the topic config changes path to avoid a NoNodeException - ZkUtils.createPersistentPath(zkClient, ZkUtils.TopicConfigChangesPath) + ZkUtils.createPersistentPath(zkClient, ZkUtils.EntityConfigChangesPath) // modify the topic to add new partitions val numPartitionsModified = 3 - val alterOpts = new TopicCommandOptions(Array("--partitions", numPartitionsModified.toString, - "--config", cleanupKey + "=" + cleanupVal, - "--topic", topic)) + val alterOpts = new TopicCommandOptions(Array("--partitions", numPartitionsModified.toString, "--topic", topic)) TopicCommand.alterTopic(zkClient, alterOpts) - val newProps = AdminUtils.fetchTopicConfig(zkClient, topic) + val newProps = AdminUtils.fetchEntityConfig(zkClient, ConfigType.Topic, topic) assertTrue("Updated properties do not contain " + cleanupKey, newProps.containsKey(cleanupKey)) assertTrue("Updated properties have incorrect value", newProps.getProperty(cleanupKey).equals(cleanupVal)) } @@ -99,4 +98,4 @@ class TopicCommandTest extends JUnit3Suite with ZooKeeperTestHarness with Loggin } assertFalse("Delete path for topic shouldn't exist after deletion.", zkClient.exists(deleteOffsetTopicPath)) } -} \ No newline at end of file +} diff --git a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala index 8a871cf..7c45393 100644 --- a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala @@ -19,6 +19,7 @@ package kafka.server import java.util.Properties import junit.framework.Assert._ +import org.easymock.{Capture, EasyMock} import org.junit.Test import kafka.integration.KafkaServerTestHarness import kafka.utils._ @@ -32,8 +33,10 @@ class DynamicConfigChangeTest extends JUnit3Suite with KafkaServerTestHarness { @Test def testConfigChange() { - val oldVal: java.lang.Long = 100000 - val newVal: java.lang.Long = 200000 + assertTrue("Should contain a ConfigHandler for topics", + this.servers(0).dynamicConfigHandlers.contains(ConfigType.Topic)) + val oldVal: java.lang.Long = 100000L + val newVal: java.lang.Long = 200000L val tp = TopicAndPartition("test", 0) val logProps = new Properties() logProps.put(LogConfig.FlushMessagesProp, oldVal.toString) @@ -50,6 +53,25 @@ class DynamicConfigChangeTest extends JUnit3Suite with KafkaServerTestHarness { } } + // For now client config changes do not do anything. Simply verify that the call was made + @Test + def testClientConfigChange() { + assertTrue("Should contain a ConfigHandler for topics", + this.servers(0).dynamicConfigHandlers.contains(ConfigType.Client)) + val clientId = "testClient" + val props = new Properties() + props.put("a.b", "c") + props.put("x.y", "z") + AdminUtils.changeClientIdConfig(zkClient, clientId, props) + TestUtils.retry(10000) { + val configHandler = this.servers(0).dynamicConfigHandlers(ConfigType.Client).asInstanceOf[ClientIdConfigHandler] + assertTrue("ClientId testClient must exist", configHandler.configPool.contains(clientId)) + assertEquals("ClientId testClient must be the only override", 1, configHandler.configPool.size) + assertEquals("c", configHandler.configPool.get(clientId).getProperty("a.b")) + assertEquals("z", configHandler.configPool.get(clientId).getProperty("x.y")) + } + } + @Test def testConfigChangeOnNonExistingTopic() { val topic = TestUtils.tempTopic @@ -63,4 +85,59 @@ class DynamicConfigChangeTest extends JUnit3Suite with KafkaServerTestHarness { } } -} \ No newline at end of file + @Test + def testProcessNotification { + val props = new Properties() + props.put("a.b", "10") + + // Create a mock ConfigHandler to record config changes it is asked to process + val entityArgument = new Capture[String]() + val propertiesArgument = new Capture[Properties]() + val handler = EasyMock.createNiceMock(classOf[ConfigHandler]) + handler.processConfigChanges( + EasyMock.and(EasyMock.capture(entityArgument), EasyMock.isA(classOf[String])), + EasyMock.and(EasyMock.capture(propertiesArgument), EasyMock.isA(classOf[Properties]))) + EasyMock.expectLastCall().once() + EasyMock.replay(handler) + + val configManager = new DynamicConfigManager(zkClient, Map(ConfigType.Topic -> handler)) + // Notifications created using the old TopicConfigManager are ignored. + configManager.processNotification(Some("not json")) + + // Incorrect Map. No version + try { + val jsonMap = Map("v" -> 1, "x" -> 2) + configManager.processNotification(Some(Json.encode(jsonMap))) + fail("Should have thrown an Exception while parsing incorrect notification " + jsonMap) + } + catch { + case t: Throwable => + } + // Version is provided. EntityType is incorrect + try { + val jsonMap = Map("version" -> 1, "entity_type" -> "garbage", "entity_name" -> "x") + configManager.processNotification(Some(Json.encode(jsonMap))) + fail("Should have thrown an Exception while parsing incorrect notification " + jsonMap) + } + catch { + case t: Throwable => + } + + // EntityName isn't provided + try { + val jsonMap = Map("version" -> 1, "entity_type" -> ConfigType.Topic) + configManager.processNotification(Some(Json.encode(jsonMap))) + fail("Should have thrown an Exception while parsing incorrect notification " + jsonMap) + } + catch { + case t: Throwable => + } + + // Everything is provided + val jsonMap = Map("version" -> 1, "entity_type" -> ConfigType.Topic, "entity_name" -> "x") + configManager.processNotification(Some(Json.encode(jsonMap))) + + // Verify that processConfigChanges was only called once + EasyMock.verify(handler) + } +} -- 1.7.12.4 From 7a666f7aa8b1be927579817187e0b5b93543b5e2 Mon Sep 17 00:00:00 2001 From: Jaikiran Pai Date: Tue, 4 Aug 2015 17:10:02 -0700 Subject: [PATCH 084/120] KAFKA-2405; Don't kill the JVM on session establishment failure As noted in the JIRA https://issues.apache.org/jira/browse/KAFKA-2405 currently the KafkaHealthCheck causes the JVM to terminate in cases where session establishment with Zookeeper fails. I don't know if retrying (after a while) is a better way to fix this but at least, IMO, the session establishment failure shouldn't kill the JVM. This commit removes the `System.exit()` call. Author: Jaikiran Pai Reviewers: Gwen Shapira Closes #111 from jaikiran/kafka-2405 and squashes the following commits: 0255fdb [Jaikiran Pai] KAFKA-2405 Don't kill the JVM on session establishment failure --- core/src/main/scala/kafka/controller/KafkaController.scala | 2 +- core/src/main/scala/kafka/server/KafkaHealthcheck.scala | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 6844602..b19e57f 100755 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -1138,7 +1138,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt } override def handleSessionEstablishmentError(error: Throwable): Unit = { - //no-op handleSessionEstablishmentError in KafkaHealthCheck should System.exit and log the error. + //no-op handleSessionEstablishmentError in KafkaHealthCheck should handle this error in its handleSessionEstablishmentError } } diff --git a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala index ea0c996..e6e270b 100644 --- a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala +++ b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala @@ -92,7 +92,6 @@ class KafkaHealthcheck(private val brokerId: Int, override def handleSessionEstablishmentError(error: Throwable): Unit = { fatal("Could not establish session with zookeeper", error) - System.exit(-1) } } -- 1.7.12.4 From 9cefb2a0fb7852d35cfe0f051bc6eadb8e9c4c80 Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Tue, 4 Aug 2015 19:04:58 -0700 Subject: [PATCH 085/120] KAFKA-2288; Follow-up to KAFKA-2249 - reduce logging and testing; Reviewd by Jun Rao --- .../apache/kafka/common/config/AbstractConfig.java | 23 +- core/src/main/scala/kafka/log/LogConfig.scala | 2 +- .../test/scala/unit/kafka/log/LogConfigTest.scala | 22 -- .../kafka/server/KafkaConfigConfigDefTest.scala | 403 --------------------- .../scala/unit/kafka/server/KafkaConfigTest.scala | 154 +++++++- 5 files changed, 175 insertions(+), 429 deletions(-) delete mode 100644 core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala diff --git a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java index ec3ae15..6c31748 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java @@ -39,7 +39,7 @@ public class AbstractConfig { private final Map values; @SuppressWarnings("unchecked") - public AbstractConfig(ConfigDef definition, Map originals) { + public AbstractConfig(ConfigDef definition, Map originals, Boolean doLog) { /* check that all the keys are really strings */ for (Object key : originals.keySet()) if (!(key instanceof String)) @@ -47,7 +47,12 @@ public class AbstractConfig { this.originals = (Map) originals; this.values = definition.parse(this.originals); this.used = Collections.synchronizedSet(new HashSet()); - logAll(); + if (doLog) + logAll(); + } + + public AbstractConfig(ConfigDef definition, Map originals) { + this(definition, originals, true); } protected Object get(String key) { @@ -167,4 +172,18 @@ public class AbstractConfig { return objects; } + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + AbstractConfig that = (AbstractConfig) o; + + return originals.equals(that.originals); + } + + @Override + public int hashCode() { + return originals.hashCode(); + } } diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala index fc41132..c969d16 100755 --- a/core/src/main/scala/kafka/log/LogConfig.scala +++ b/core/src/main/scala/kafka/log/LogConfig.scala @@ -46,7 +46,7 @@ object Defaults { val PreAllocateEnable = kafka.server.Defaults.LogPreAllocateEnable } -case class LogConfig(props: java.util.Map[_, _]) extends AbstractConfig(LogConfig.configDef, props) { +case class LogConfig(props: java.util.Map[_, _]) extends AbstractConfig(LogConfig.configDef, props, false) { val segmentSize = getInt(LogConfig.SegmentBytesProp) val segmentMs = getLong(LogConfig.SegmentMsProp) diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala index 19dcb47..72e98b3 100644 --- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala @@ -33,28 +33,6 @@ class LogConfigTest extends JUnit3Suite { } @Test - def testFromPropsToProps() { - import scala.util.Random._ - val expected = new Properties() - LogConfig.configNames().foreach((name) => { - name match { - case LogConfig.UncleanLeaderElectionEnableProp => expected.setProperty(name, randFrom("true", "false")) - case LogConfig.CompressionTypeProp => expected.setProperty(name, randFrom("producer", "uncompressed", "gzip")) - case LogConfig.CleanupPolicyProp => expected.setProperty(name, randFrom(LogConfig.Compact, LogConfig.Delete)) - case LogConfig.MinCleanableDirtyRatioProp => expected.setProperty(name, "%.1f".format(nextDouble * .9 + .1)) - case LogConfig.MinInSyncReplicasProp => expected.setProperty(name, (nextInt(Int.MaxValue - 1) + 1).toString) - case LogConfig.RetentionBytesProp => expected.setProperty(name, nextInt().toString) - case LogConfig.RetentionMsProp => expected.setProperty(name, nextLong().toString) - case LogConfig.PreAllocateEnableProp => expected.setProperty(name, randFrom("true", "false")) - case positiveIntProperty => expected.setProperty(name, nextInt(Int.MaxValue).toString) - } - }) - - val actual = LogConfig(expected).originals - Assert.assertEquals(expected, actual) - } - - @Test def testFromPropsInvalid() { LogConfig.configNames().foreach((name) => { name match { diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala deleted file mode 100644 index 04a02e0..0000000 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala +++ /dev/null @@ -1,403 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package kafka.server - -import java.util.Properties - -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 scala.collection.Map -import scala.util.Random._ - -class KafkaConfigConfigDefTest extends JUnit3Suite { - - @Test - def testFromPropsEmpty() { - // only required - val p = new Properties() - p.put(KafkaConfig.ZkConnectProp, "127.0.0.1:2181") - val actualConfig = KafkaConfig.fromProps(p) - - val expectedConfig = new KafkaConfig(p) - - Assert.assertEquals(expectedConfig.zkConnect, actualConfig.zkConnect) - Assert.assertEquals(expectedConfig.zkSessionTimeoutMs, actualConfig.zkSessionTimeoutMs) - Assert.assertEquals(expectedConfig.zkConnectionTimeoutMs, actualConfig.zkConnectionTimeoutMs) - Assert.assertEquals(expectedConfig.zkSyncTimeMs, actualConfig.zkSyncTimeMs) - Assert.assertEquals(expectedConfig.maxReservedBrokerId, actualConfig.maxReservedBrokerId) - Assert.assertEquals(expectedConfig.brokerId, actualConfig.brokerId) - Assert.assertEquals(expectedConfig.messageMaxBytes, actualConfig.messageMaxBytes) - Assert.assertEquals(expectedConfig.numNetworkThreads, actualConfig.numNetworkThreads) - Assert.assertEquals(expectedConfig.numIoThreads, actualConfig.numIoThreads) - Assert.assertEquals(expectedConfig.backgroundThreads, actualConfig.backgroundThreads) - Assert.assertEquals(expectedConfig.queuedMaxRequests, actualConfig.queuedMaxRequests) - - Assert.assertEquals(expectedConfig.port, actualConfig.port) - Assert.assertEquals(expectedConfig.hostName, actualConfig.hostName) - Assert.assertEquals(expectedConfig.advertisedHostName, actualConfig.advertisedHostName) - Assert.assertEquals(expectedConfig.advertisedPort, actualConfig.advertisedPort) - Assert.assertEquals(expectedConfig.socketSendBufferBytes, actualConfig.socketSendBufferBytes) - Assert.assertEquals(expectedConfig.socketReceiveBufferBytes, actualConfig.socketReceiveBufferBytes) - Assert.assertEquals(expectedConfig.socketRequestMaxBytes, actualConfig.socketRequestMaxBytes) - Assert.assertEquals(expectedConfig.maxConnectionsPerIp, actualConfig.maxConnectionsPerIp) - Assert.assertEquals(expectedConfig.maxConnectionsPerIpOverrides, actualConfig.maxConnectionsPerIpOverrides) - Assert.assertEquals(expectedConfig.connectionsMaxIdleMs, actualConfig.connectionsMaxIdleMs) - - Assert.assertEquals(expectedConfig.numPartitions, actualConfig.numPartitions) - Assert.assertEquals(expectedConfig.logDirs, actualConfig.logDirs) - - Assert.assertEquals(expectedConfig.logSegmentBytes, actualConfig.logSegmentBytes) - - Assert.assertEquals(expectedConfig.logRollTimeMillis, actualConfig.logRollTimeMillis) - Assert.assertEquals(expectedConfig.logRollTimeJitterMillis, actualConfig.logRollTimeJitterMillis) - Assert.assertEquals(expectedConfig.logRetentionTimeMillis, actualConfig.logRetentionTimeMillis) - - Assert.assertEquals(expectedConfig.logRetentionBytes, actualConfig.logRetentionBytes) - Assert.assertEquals(expectedConfig.logCleanupIntervalMs, actualConfig.logCleanupIntervalMs) - Assert.assertEquals(expectedConfig.logCleanupPolicy, actualConfig.logCleanupPolicy) - Assert.assertEquals(expectedConfig.logCleanerThreads, actualConfig.logCleanerThreads) - Assert.assertEquals(expectedConfig.logCleanerIoMaxBytesPerSecond, actualConfig.logCleanerIoMaxBytesPerSecond, 0.0) - Assert.assertEquals(expectedConfig.logCleanerDedupeBufferSize, actualConfig.logCleanerDedupeBufferSize) - Assert.assertEquals(expectedConfig.logCleanerIoBufferSize, actualConfig.logCleanerIoBufferSize) - Assert.assertEquals(expectedConfig.logCleanerDedupeBufferLoadFactor, actualConfig.logCleanerDedupeBufferLoadFactor, 0.0) - Assert.assertEquals(expectedConfig.logCleanerBackoffMs, actualConfig.logCleanerBackoffMs) - Assert.assertEquals(expectedConfig.logCleanerMinCleanRatio, actualConfig.logCleanerMinCleanRatio, 0.0) - Assert.assertEquals(expectedConfig.logCleanerEnable, actualConfig.logCleanerEnable) - Assert.assertEquals(expectedConfig.logCleanerDeleteRetentionMs, actualConfig.logCleanerDeleteRetentionMs) - Assert.assertEquals(expectedConfig.logIndexSizeMaxBytes, actualConfig.logIndexSizeMaxBytes) - Assert.assertEquals(expectedConfig.logIndexIntervalBytes, actualConfig.logIndexIntervalBytes) - Assert.assertEquals(expectedConfig.logFlushIntervalMessages, actualConfig.logFlushIntervalMessages) - Assert.assertEquals(expectedConfig.logDeleteDelayMs, actualConfig.logDeleteDelayMs) - Assert.assertEquals(expectedConfig.logFlushSchedulerIntervalMs, actualConfig.logFlushSchedulerIntervalMs) - Assert.assertEquals(expectedConfig.logFlushIntervalMs, actualConfig.logFlushIntervalMs) - Assert.assertEquals(expectedConfig.logFlushOffsetCheckpointIntervalMs, actualConfig.logFlushOffsetCheckpointIntervalMs) - Assert.assertEquals(expectedConfig.numRecoveryThreadsPerDataDir, actualConfig.numRecoveryThreadsPerDataDir) - Assert.assertEquals(expectedConfig.autoCreateTopicsEnable, actualConfig.autoCreateTopicsEnable) - - Assert.assertEquals(expectedConfig.minInSyncReplicas, actualConfig.minInSyncReplicas) - - Assert.assertEquals(expectedConfig.controllerSocketTimeoutMs, actualConfig.controllerSocketTimeoutMs) - Assert.assertEquals(expectedConfig.defaultReplicationFactor, actualConfig.defaultReplicationFactor) - Assert.assertEquals(expectedConfig.replicaLagTimeMaxMs, actualConfig.replicaLagTimeMaxMs) - Assert.assertEquals(expectedConfig.replicaSocketTimeoutMs, actualConfig.replicaSocketTimeoutMs) - Assert.assertEquals(expectedConfig.replicaSocketReceiveBufferBytes, actualConfig.replicaSocketReceiveBufferBytes) - Assert.assertEquals(expectedConfig.replicaFetchMaxBytes, actualConfig.replicaFetchMaxBytes) - Assert.assertEquals(expectedConfig.replicaFetchWaitMaxMs, actualConfig.replicaFetchWaitMaxMs) - Assert.assertEquals(expectedConfig.replicaFetchMinBytes, actualConfig.replicaFetchMinBytes) - Assert.assertEquals(expectedConfig.replicaFetchBackoffMs, actualConfig.replicaFetchBackoffMs) - Assert.assertEquals(expectedConfig.numReplicaFetchers, actualConfig.numReplicaFetchers) - Assert.assertEquals(expectedConfig.replicaHighWatermarkCheckpointIntervalMs, actualConfig.replicaHighWatermarkCheckpointIntervalMs) - Assert.assertEquals(expectedConfig.fetchPurgatoryPurgeIntervalRequests, actualConfig.fetchPurgatoryPurgeIntervalRequests) - Assert.assertEquals(expectedConfig.producerPurgatoryPurgeIntervalRequests, actualConfig.producerPurgatoryPurgeIntervalRequests) - Assert.assertEquals(expectedConfig.autoLeaderRebalanceEnable, actualConfig.autoLeaderRebalanceEnable) - Assert.assertEquals(expectedConfig.leaderImbalancePerBrokerPercentage, actualConfig.leaderImbalancePerBrokerPercentage) - Assert.assertEquals(expectedConfig.leaderImbalanceCheckIntervalSeconds, actualConfig.leaderImbalanceCheckIntervalSeconds) - Assert.assertEquals(expectedConfig.uncleanLeaderElectionEnable, actualConfig.uncleanLeaderElectionEnable) - - Assert.assertEquals(expectedConfig.controlledShutdownMaxRetries, actualConfig.controlledShutdownMaxRetries) - Assert.assertEquals(expectedConfig.controlledShutdownRetryBackoffMs, actualConfig.controlledShutdownRetryBackoffMs) - Assert.assertEquals(expectedConfig.controlledShutdownEnable, actualConfig.controlledShutdownEnable) - - Assert.assertEquals(expectedConfig.consumerMinSessionTimeoutMs, actualConfig.consumerMinSessionTimeoutMs) - Assert.assertEquals(expectedConfig.consumerMaxSessionTimeoutMs, actualConfig.consumerMaxSessionTimeoutMs) - - Assert.assertEquals(expectedConfig.offsetMetadataMaxSize, actualConfig.offsetMetadataMaxSize) - Assert.assertEquals(expectedConfig.offsetsLoadBufferSize, actualConfig.offsetsLoadBufferSize) - Assert.assertEquals(expectedConfig.offsetsTopicReplicationFactor, actualConfig.offsetsTopicReplicationFactor) - Assert.assertEquals(expectedConfig.offsetsTopicPartitions, actualConfig.offsetsTopicPartitions) - Assert.assertEquals(expectedConfig.offsetsTopicSegmentBytes, actualConfig.offsetsTopicSegmentBytes) - Assert.assertEquals(expectedConfig.offsetsTopicCompressionCodec, actualConfig.offsetsTopicCompressionCodec) - Assert.assertEquals(expectedConfig.offsetsRetentionMinutes, actualConfig.offsetsRetentionMinutes) - Assert.assertEquals(expectedConfig.offsetsRetentionCheckIntervalMs, actualConfig.offsetsRetentionCheckIntervalMs) - Assert.assertEquals(expectedConfig.offsetCommitTimeoutMs, actualConfig.offsetCommitTimeoutMs) - Assert.assertEquals(expectedConfig.offsetCommitRequiredAcks, actualConfig.offsetCommitRequiredAcks) - - Assert.assertEquals(expectedConfig.deleteTopicEnable, actualConfig.deleteTopicEnable) - Assert.assertEquals(expectedConfig.compressionType, actualConfig.compressionType) - } - - private def atLeastXIntProp(x: Int): String = (nextInt(Int.MaxValue - x) + x).toString - - private def atLeastOneIntProp: String = atLeastXIntProp(1) - - private def inRangeIntProp(fromInc: Int, toInc: Int): String = (nextInt(toInc + 1 - fromInc) + fromInc).toString - - @Test - def testFromPropsToProps() { - import scala.util.Random._ - val expected = new Properties() - KafkaConfig.configNames().foreach(name => { - name match { - case KafkaConfig.ZkConnectProp => expected.setProperty(name, "127.0.0.1:2181") - case KafkaConfig.ZkSessionTimeoutMsProp => expected.setProperty(name, atLeastOneIntProp) - case KafkaConfig.ZkConnectionTimeoutMsProp => expected.setProperty(name, atLeastOneIntProp) - case KafkaConfig.ZkSyncTimeMsProp => expected.setProperty(name, atLeastOneIntProp) - - case KafkaConfig.NumNetworkThreadsProp => expected.setProperty(name, atLeastOneIntProp) - case KafkaConfig.NumIoThreadsProp => expected.setProperty(name, atLeastOneIntProp) - case KafkaConfig.BackgroundThreadsProp => expected.setProperty(name, atLeastOneIntProp) - case KafkaConfig.QueuedMaxRequestsProp => expected.setProperty(name, atLeastOneIntProp) - - case KafkaConfig.PortProp => expected.setProperty(name, "1234") - case KafkaConfig.HostNameProp => expected.setProperty(name, "hostname") - case KafkaConfig.ListenersProp => expected.setProperty(name, "PLAINTEXT://:9092") - case KafkaConfig.AdvertisedHostNameProp => expected.setProperty(name, "advertised.hostname") - case KafkaConfig.AdvertisedPortProp => expected.setProperty(name, "4321") - case KafkaConfig.AdvertisedListenersProp => expected.setProperty(name, "PLAINTEXT://:2909") - case KafkaConfig.SocketRequestMaxBytesProp => expected.setProperty(name, atLeastOneIntProp) - case KafkaConfig.MaxConnectionsPerIpProp => expected.setProperty(name, atLeastOneIntProp) - case KafkaConfig.MaxConnectionsPerIpOverridesProp => expected.setProperty(name, "127.0.0.1:2, 127.0.0.2:3") - - case KafkaConfig.NumPartitionsProp => expected.setProperty(name, "2") - case KafkaConfig.LogDirsProp => expected.setProperty(name, "/tmp/logs,/tmp/logs2") - case KafkaConfig.LogDirProp => expected.setProperty(name, "/tmp/log") - case KafkaConfig.LogSegmentBytesProp => expected.setProperty(name, atLeastXIntProp(Message.MinHeaderSize)) - - case KafkaConfig.LogRollTimeMillisProp => expected.setProperty(name, atLeastOneIntProp) - case KafkaConfig.LogRollTimeHoursProp => expected.setProperty(name, atLeastOneIntProp) - - case KafkaConfig.LogRetentionTimeMillisProp => expected.setProperty(name, atLeastOneIntProp) - case KafkaConfig.LogRetentionTimeMinutesProp => expected.setProperty(name, atLeastOneIntProp) - case KafkaConfig.LogRetentionTimeHoursProp => expected.setProperty(name, atLeastOneIntProp) - - case KafkaConfig.LogCleanupIntervalMsProp => expected.setProperty(name, atLeastOneIntProp) - case KafkaConfig.LogCleanupPolicyProp => expected.setProperty(name, randFrom(Defaults.Compact, Defaults.Delete)) - case KafkaConfig.LogCleanerIoMaxBytesPerSecondProp => expected.setProperty(name, "%.1f".format(nextDouble * .9 + .1)) - case KafkaConfig.LogCleanerDedupeBufferLoadFactorProp => expected.setProperty(name, "%.1f".format(nextDouble * .9 + .1)) - case KafkaConfig.LogCleanerMinCleanRatioProp => expected.setProperty(name, "%.1f".format(nextDouble * .9 + .1)) - case KafkaConfig.LogCleanerEnableProp => expected.setProperty(name, randFrom("true", "false")) - case KafkaConfig.LogIndexSizeMaxBytesProp => expected.setProperty(name, atLeastXIntProp(4)) - case KafkaConfig.LogFlushIntervalMessagesProp => expected.setProperty(name, atLeastOneIntProp) - - case KafkaConfig.NumRecoveryThreadsPerDataDirProp => expected.setProperty(name, atLeastOneIntProp) - case KafkaConfig.AutoCreateTopicsEnableProp => expected.setProperty(name, randFrom("true", "false")) - case KafkaConfig.MinInSyncReplicasProp => expected.setProperty(name, atLeastOneIntProp) - case KafkaConfig.AutoLeaderRebalanceEnableProp => expected.setProperty(name, randFrom("true", "false")) - case KafkaConfig.UncleanLeaderElectionEnableProp => expected.setProperty(name, randFrom("true", "false")) - case KafkaConfig.LogPreAllocateProp => expected.setProperty(name, randFrom("true", "false")) - case KafkaConfig.InterBrokerSecurityProtocolProp => expected.setProperty(name, SecurityProtocol.PLAINTEXT.toString) - case KafkaConfig.InterBrokerProtocolVersionProp => expected.setProperty(name, ApiVersion.latestVersion.toString) - - case KafkaConfig.ControlledShutdownEnableProp => expected.setProperty(name, randFrom("true", "false")) - case KafkaConfig.OffsetsLoadBufferSizeProp => expected.setProperty(name, atLeastOneIntProp) - case KafkaConfig.OffsetsTopicPartitionsProp => expected.setProperty(name, atLeastOneIntProp) - case KafkaConfig.OffsetsTopicSegmentBytesProp => expected.setProperty(name, atLeastOneIntProp) - case KafkaConfig.OffsetsTopicCompressionCodecProp => expected.setProperty(name, randFrom(GZIPCompressionCodec.codec.toString, - SnappyCompressionCodec.codec.toString, LZ4CompressionCodec.codec.toString, NoCompressionCodec.codec.toString)) - case KafkaConfig.OffsetsRetentionMinutesProp => expected.setProperty(name, atLeastOneIntProp) - case KafkaConfig.OffsetsRetentionCheckIntervalMsProp => expected.setProperty(name, atLeastOneIntProp) - case KafkaConfig.OffsetCommitTimeoutMsProp => expected.setProperty(name, atLeastOneIntProp) - case KafkaConfig.DeleteTopicEnableProp => expected.setProperty(name, randFrom("true", "false")) - - // explicit, non trivial validations or with transient dependencies - - // require(brokerId >= -1 && brokerId <= maxReservedBrokerId) - case KafkaConfig.MaxReservedBrokerIdProp => expected.setProperty(name, "100") - case KafkaConfig.BrokerIdProp => expected.setProperty(name, inRangeIntProp(0, 100)) - // require(logCleanerDedupeBufferSize / logCleanerThreads > 1024 * 1024) - case KafkaConfig.LogCleanerThreadsProp => expected.setProperty(name, "2") - case KafkaConfig.LogCleanerDedupeBufferSizeProp => expected.setProperty(name, (1024 * 1024 * 3 + 1).toString) - // require(replicaFetchWaitMaxMs <= replicaSocketTimeoutMs) - case KafkaConfig.ReplicaFetchWaitMaxMsProp => expected.setProperty(name, "321") - case KafkaConfig.ReplicaSocketTimeoutMsProp => expected.setProperty(name, atLeastXIntProp(321)) - // require(replicaFetchMaxBytes >= messageMaxBytes) - case KafkaConfig.MessageMaxBytesProp => expected.setProperty(name, "1234") - case KafkaConfig.ReplicaFetchMaxBytesProp => expected.setProperty(name, atLeastXIntProp(1234)) - // require(replicaFetchWaitMaxMs <= replicaLagTimeMaxMs) - case KafkaConfig.ReplicaLagTimeMaxMsProp => expected.setProperty(name, atLeastXIntProp(321)) - //require(offsetCommitRequiredAcks >= -1 && offsetCommitRequiredAcks <= offsetsTopicReplicationFactor) - case KafkaConfig.OffsetCommitRequiredAcksProp => expected.setProperty(name, "-1") - case KafkaConfig.OffsetsTopicReplicationFactorProp => expected.setProperty(name, inRangeIntProp(1, Short.MaxValue)) - //BrokerCompressionCodec.isValid(compressionType) - case KafkaConfig.CompressionTypeProp => expected.setProperty(name, randFrom(BrokerCompressionCodec.brokerCompressionOptions)) - - case KafkaConfig.MetricNumSamplesProp => expected.setProperty(name, "2") - case KafkaConfig.MetricSampleWindowMsProp => expected.setProperty(name, "1000") - case KafkaConfig.MetricReporterClassesProp => expected.setProperty(name, "") - - case nonNegativeIntProperty => expected.setProperty(name, nextInt(Int.MaxValue).toString) - } - }) - - val actual = KafkaConfig.fromProps(expected).originals - Assert.assertEquals(expected, actual) - } - - @Test - def testFromPropsInvalid() { - def getBaseProperties(): Properties = { - val validRequiredProperties = new Properties() - validRequiredProperties.put(KafkaConfig.ZkConnectProp, "127.0.0.1") - validRequiredProperties - } - // to ensure a basis is valid - bootstraps all needed validation - KafkaConfig.fromProps(getBaseProperties()) - - KafkaConfig.configNames().foreach(name => { - name match { - case KafkaConfig.ZkConnectProp => // ignore string - case KafkaConfig.ZkSessionTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") - case KafkaConfig.ZkConnectionTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") - case KafkaConfig.ZkSyncTimeMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") - - case KafkaConfig.BrokerIdProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") - case KafkaConfig.NumNetworkThreadsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") - case KafkaConfig.NumIoThreadsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") - case KafkaConfig.BackgroundThreadsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") - case KafkaConfig.QueuedMaxRequestsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") - - case KafkaConfig.PortProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") - case KafkaConfig.HostNameProp => // ignore string - case KafkaConfig.AdvertisedHostNameProp => //ignore string - case KafkaConfig.AdvertisedPortProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") - case KafkaConfig.SocketSendBufferBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") - case KafkaConfig.SocketReceiveBufferBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") - case KafkaConfig.MaxConnectionsPerIpOverridesProp => - assertPropertyInvalid(getBaseProperties(), name, "127.0.0.1:not_a_number") - case KafkaConfig.ConnectionsMaxIdleMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") - - case KafkaConfig.NumPartitionsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") - case KafkaConfig.LogDirsProp => // ignore string - case KafkaConfig.LogDirProp => // ignore string - case KafkaConfig.LogSegmentBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", Message.MinHeaderSize - 1) - - case KafkaConfig.LogRollTimeMillisProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") - case KafkaConfig.LogRollTimeHoursProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") - - case KafkaConfig.LogRetentionTimeMillisProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") - case KafkaConfig.LogRetentionTimeMinutesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") - case KafkaConfig.LogRetentionTimeHoursProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") - - case KafkaConfig.LogRetentionBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") - case KafkaConfig.LogCleanupIntervalMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") - case KafkaConfig.LogCleanupPolicyProp => assertPropertyInvalid(getBaseProperties(), name, "unknown_policy", "0") - case KafkaConfig.LogCleanerIoMaxBytesPerSecondProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") - case KafkaConfig.LogCleanerDedupeBufferSizeProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "1024") - case KafkaConfig.LogCleanerDedupeBufferLoadFactorProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") - case KafkaConfig.LogCleanerEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean") - case KafkaConfig.LogCleanerDeleteRetentionMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") - case KafkaConfig.LogCleanerMinCleanRatioProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") - case KafkaConfig.LogIndexSizeMaxBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "3") - case KafkaConfig.LogFlushIntervalMessagesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") - case KafkaConfig.LogFlushSchedulerIntervalMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") - case KafkaConfig.LogFlushIntervalMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") - case KafkaConfig.NumRecoveryThreadsPerDataDirProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") - case KafkaConfig.AutoCreateTopicsEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean", "0") - case KafkaConfig.MinInSyncReplicasProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") - case KafkaConfig.ControllerSocketTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") - case KafkaConfig.DefaultReplicationFactorProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") - case KafkaConfig.ReplicaLagTimeMaxMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") - case KafkaConfig.ReplicaSocketTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "-2") - case KafkaConfig.ReplicaSocketReceiveBufferBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") - case KafkaConfig.ReplicaFetchMaxBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") - case KafkaConfig.ReplicaFetchWaitMaxMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") - case KafkaConfig.ReplicaFetchMinBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") - case KafkaConfig.NumReplicaFetchersProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") - case KafkaConfig.ReplicaHighWatermarkCheckpointIntervalMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") - case KafkaConfig.FetchPurgatoryPurgeIntervalRequestsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") - case KafkaConfig.ProducerPurgatoryPurgeIntervalRequestsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") - case KafkaConfig.AutoLeaderRebalanceEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean", "0") - case KafkaConfig.LeaderImbalancePerBrokerPercentageProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") - case KafkaConfig.LeaderImbalanceCheckIntervalSecondsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") - case KafkaConfig.UncleanLeaderElectionEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean", "0") - case KafkaConfig.ControlledShutdownMaxRetriesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") - case KafkaConfig.ControlledShutdownRetryBackoffMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") - case KafkaConfig.ControlledShutdownEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean", "0") - case KafkaConfig.ConsumerMinSessionTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") - case KafkaConfig.ConsumerMaxSessionTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") - case KafkaConfig.OffsetMetadataMaxSizeProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") - case KafkaConfig.OffsetsLoadBufferSizeProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") - case KafkaConfig.OffsetsTopicReplicationFactorProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") - case KafkaConfig.OffsetsTopicPartitionsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") - case KafkaConfig.OffsetsTopicSegmentBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") - case KafkaConfig.OffsetsTopicCompressionCodecProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "-1") - case KafkaConfig.OffsetsRetentionMinutesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") - case KafkaConfig.OffsetsRetentionCheckIntervalMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") - case KafkaConfig.OffsetCommitTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") - case KafkaConfig.OffsetCommitRequiredAcksProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "-2") - - case KafkaConfig.DeleteTopicEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean", "0") - - case KafkaConfig.MetricNumSamplesProp => assertPropertyInvalid(getBaseProperties, name, "not_a_number", "-1", "0") - case KafkaConfig.MetricSampleWindowMsProp => assertPropertyInvalid(getBaseProperties, name, "not_a_number", "-1", "0") - case KafkaConfig.MetricReporterClassesProp => // ignore string - - case nonNegativeIntProperty => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "-1") - } - }) - } - - @Test - def testSpecificProperties(): Unit = { - val defaults = new Properties() - defaults.put(KafkaConfig.ZkConnectProp, "127.0.0.1:2181") - // For ZkConnectionTimeoutMs - defaults.put(KafkaConfig.ZkSessionTimeoutMsProp, "1234") - defaults.put(KafkaConfig.MaxReservedBrokerIdProp, "1") - defaults.put(KafkaConfig.BrokerIdProp, "1") - defaults.put(KafkaConfig.HostNameProp, "127.0.0.1") - defaults.put(KafkaConfig.PortProp, "1122") - defaults.put(KafkaConfig.MaxConnectionsPerIpOverridesProp, "127.0.0.1:2, 127.0.0.2:3") - defaults.put(KafkaConfig.LogDirProp, "/tmp1,/tmp2") - defaults.put(KafkaConfig.LogRollTimeHoursProp, "12") - defaults.put(KafkaConfig.LogRollTimeJitterHoursProp, "11") - defaults.put(KafkaConfig.LogRetentionTimeHoursProp, "10") - //For LogFlushIntervalMsProp - defaults.put(KafkaConfig.LogFlushSchedulerIntervalMsProp, "123") - defaults.put(KafkaConfig.OffsetsTopicCompressionCodecProp, SnappyCompressionCodec.codec.toString) - - val config = KafkaConfig.fromProps(defaults) - Assert.assertEquals("127.0.0.1:2181", config.zkConnect) - Assert.assertEquals(1234, config.zkConnectionTimeoutMs) - Assert.assertEquals(1, config.maxReservedBrokerId) - Assert.assertEquals(1, config.brokerId) - Assert.assertEquals("127.0.0.1", config.hostName) - Assert.assertEquals(1122, config.advertisedPort) - Assert.assertEquals("127.0.0.1", config.advertisedHostName) - Assert.assertEquals(Map("127.0.0.1" -> 2, "127.0.0.2" -> 3), config.maxConnectionsPerIpOverrides) - Assert.assertEquals(List("/tmp1", "/tmp2"), config.logDirs) - Assert.assertEquals(12 * 60L * 1000L * 60, config.logRollTimeMillis) - Assert.assertEquals(11 * 60L * 1000L * 60, config.logRollTimeJitterMillis) - Assert.assertEquals(10 * 60L * 1000L * 60, config.logRetentionTimeMillis) - Assert.assertEquals(123L, config.logFlushIntervalMs) - Assert.assertEquals(SnappyCompressionCodec, config.offsetsTopicCompressionCodec) - } - - private def assertPropertyInvalid(validRequiredProps: => Properties, name: String, values: Any*) { - values.foreach((value) => { - val props = validRequiredProps - props.setProperty(name, value.toString) - intercept[Exception] { - KafkaConfig.fromProps(props) - } - }) - } - - private def randFrom[T](choices: T*): T = { - import scala.util.Random - choices(Random.nextInt(choices.size)) - } - - private def randFrom[T](choices: List[T]): T = { - import scala.util.Random - choices(Random.nextInt(choices.size)) - } -} diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index d354452..f32d206 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -21,10 +21,11 @@ import java.util.Properties import junit.framework.Assert._ import kafka.api.{ApiVersion, KAFKA_082} +import kafka.message._ 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.junit.{Assert, Test} import org.scalatest.junit.JUnit3Suite class KafkaConfigTest extends JUnit3Suite { @@ -380,4 +381,155 @@ class KafkaConfigTest extends JUnit3Suite { KafkaConfig.fromProps(props) } } + + @Test + def testFromPropsInvalid() { + def getBaseProperties(): Properties = { + val validRequiredProperties = new Properties() + validRequiredProperties.put(KafkaConfig.ZkConnectProp, "127.0.0.1:2181") + validRequiredProperties + } + // to ensure a basis is valid - bootstraps all needed validation + KafkaConfig.fromProps(getBaseProperties()) + + KafkaConfig.configNames().foreach(name => { + name match { + case KafkaConfig.ZkConnectProp => // ignore string + case KafkaConfig.ZkSessionTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.ZkConnectionTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.ZkSyncTimeMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + + case KafkaConfig.BrokerIdProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.NumNetworkThreadsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.NumIoThreadsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.BackgroundThreadsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.QueuedMaxRequestsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + + case KafkaConfig.PortProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.HostNameProp => // ignore string + case KafkaConfig.AdvertisedHostNameProp => //ignore string + case KafkaConfig.AdvertisedPortProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.SocketSendBufferBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.SocketReceiveBufferBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.MaxConnectionsPerIpOverridesProp => + assertPropertyInvalid(getBaseProperties(), name, "127.0.0.1:not_a_number") + case KafkaConfig.ConnectionsMaxIdleMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + + case KafkaConfig.NumPartitionsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.LogDirsProp => // ignore string + case KafkaConfig.LogDirProp => // ignore string + case KafkaConfig.LogSegmentBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", Message.MinHeaderSize - 1) + + case KafkaConfig.LogRollTimeMillisProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.LogRollTimeHoursProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + + case KafkaConfig.LogRetentionTimeMillisProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.LogRetentionTimeMinutesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.LogRetentionTimeHoursProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + + case KafkaConfig.LogRetentionBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.LogCleanupIntervalMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.LogCleanupPolicyProp => assertPropertyInvalid(getBaseProperties(), name, "unknown_policy", "0") + case KafkaConfig.LogCleanerIoMaxBytesPerSecondProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.LogCleanerDedupeBufferSizeProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "1024") + case KafkaConfig.LogCleanerDedupeBufferLoadFactorProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.LogCleanerEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean") + case KafkaConfig.LogCleanerDeleteRetentionMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.LogCleanerMinCleanRatioProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.LogIndexSizeMaxBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "3") + case KafkaConfig.LogFlushIntervalMessagesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.LogFlushSchedulerIntervalMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.LogFlushIntervalMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.NumRecoveryThreadsPerDataDirProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.AutoCreateTopicsEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean", "0") + case KafkaConfig.MinInSyncReplicasProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.ControllerSocketTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.DefaultReplicationFactorProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.ReplicaLagTimeMaxMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.ReplicaSocketTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "-2") + case KafkaConfig.ReplicaSocketReceiveBufferBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.ReplicaFetchMaxBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.ReplicaFetchWaitMaxMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.ReplicaFetchMinBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.NumReplicaFetchersProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.ReplicaHighWatermarkCheckpointIntervalMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.FetchPurgatoryPurgeIntervalRequestsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.ProducerPurgatoryPurgeIntervalRequestsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.AutoLeaderRebalanceEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean", "0") + case KafkaConfig.LeaderImbalancePerBrokerPercentageProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.LeaderImbalanceCheckIntervalSecondsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.UncleanLeaderElectionEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean", "0") + case KafkaConfig.ControlledShutdownMaxRetriesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.ControlledShutdownRetryBackoffMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.ControlledShutdownEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean", "0") + case KafkaConfig.ConsumerMinSessionTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.ConsumerMaxSessionTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.OffsetMetadataMaxSizeProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.OffsetsLoadBufferSizeProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.OffsetsTopicReplicationFactorProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.OffsetsTopicPartitionsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.OffsetsTopicSegmentBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.OffsetsTopicCompressionCodecProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "-1") + case KafkaConfig.OffsetsRetentionMinutesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.OffsetsRetentionCheckIntervalMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.OffsetCommitTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.OffsetCommitRequiredAcksProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "-2") + + case KafkaConfig.DeleteTopicEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean", "0") + + case KafkaConfig.MetricNumSamplesProp => assertPropertyInvalid(getBaseProperties, name, "not_a_number", "-1", "0") + case KafkaConfig.MetricSampleWindowMsProp => assertPropertyInvalid(getBaseProperties, name, "not_a_number", "-1", "0") + case KafkaConfig.MetricReporterClassesProp => // ignore string + + case nonNegativeIntProperty => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "-1") + } + }) + } + + @Test + def testSpecificProperties(): Unit = { + val defaults = new Properties() + defaults.put(KafkaConfig.ZkConnectProp, "127.0.0.1:2181") + // For ZkConnectionTimeoutMs + defaults.put(KafkaConfig.ZkSessionTimeoutMsProp, "1234") + defaults.put(KafkaConfig.MaxReservedBrokerIdProp, "1") + defaults.put(KafkaConfig.BrokerIdProp, "1") + defaults.put(KafkaConfig.HostNameProp, "127.0.0.1") + defaults.put(KafkaConfig.PortProp, "1122") + defaults.put(KafkaConfig.MaxConnectionsPerIpOverridesProp, "127.0.0.1:2, 127.0.0.2:3") + defaults.put(KafkaConfig.LogDirProp, "/tmp1,/tmp2") + defaults.put(KafkaConfig.LogRollTimeHoursProp, "12") + defaults.put(KafkaConfig.LogRollTimeJitterHoursProp, "11") + defaults.put(KafkaConfig.LogRetentionTimeHoursProp, "10") + //For LogFlushIntervalMsProp + defaults.put(KafkaConfig.LogFlushSchedulerIntervalMsProp, "123") + defaults.put(KafkaConfig.OffsetsTopicCompressionCodecProp, SnappyCompressionCodec.codec.toString) + + val config = KafkaConfig.fromProps(defaults) + Assert.assertEquals("127.0.0.1:2181", config.zkConnect) + Assert.assertEquals(1234, config.zkConnectionTimeoutMs) + Assert.assertEquals(1, config.maxReservedBrokerId) + Assert.assertEquals(1, config.brokerId) + Assert.assertEquals("127.0.0.1", config.hostName) + Assert.assertEquals(1122, config.advertisedPort) + Assert.assertEquals("127.0.0.1", config.advertisedHostName) + Assert.assertEquals(Map("127.0.0.1" -> 2, "127.0.0.2" -> 3), config.maxConnectionsPerIpOverrides) + Assert.assertEquals(List("/tmp1", "/tmp2"), config.logDirs) + Assert.assertEquals(12 * 60L * 1000L * 60, config.logRollTimeMillis) + Assert.assertEquals(11 * 60L * 1000L * 60, config.logRollTimeJitterMillis) + Assert.assertEquals(10 * 60L * 1000L * 60, config.logRetentionTimeMillis) + Assert.assertEquals(123L, config.logFlushIntervalMs) + Assert.assertEquals(SnappyCompressionCodec, config.offsetsTopicCompressionCodec) + } + + private def assertPropertyInvalid(validRequiredProps: => Properties, name: String, values: Any*) { + values.foreach((value) => { + val props = validRequiredProps + props.setProperty(name, value.toString) + intercept[Exception] { + KafkaConfig.fromProps(props) + } + }) + } + } -- 1.7.12.4 From fc40016cf3cc4143f2e8486722737da3461771e6 Mon Sep 17 00:00:00 2001 From: Grant Henke Date: Wed, 5 Aug 2015 11:46:11 -0700 Subject: [PATCH 086/120] KAFKA-2407: Only create log directory when it will be used Author: Grant Henke Reviewers: Gwen Shapira Closes #115 from granthenke/log-fix and squashes the following commits: de36138 [Grant Henke] Small comment fix 49a8dd4 [Grant Henke] KAFKA-2407: Only create log directory when it will be used --- bin/kafka-run-class.sh | 20 +++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/bin/kafka-run-class.sh b/bin/kafka-run-class.sh index ebe7409..2f00f68 100755 --- a/bin/kafka-run-class.sh +++ b/bin/kafka-run-class.sh @@ -22,15 +22,6 @@ fi base_dir=$(dirname $0)/.. -# create logs directory -if [ "x$LOG_DIR" = "x" ]; then - LOG_DIR="$base_dir/logs" -fi - -if [ ! -d "$LOG_DIR" ]; then - mkdir -p "$LOG_DIR" -fi - if [ -z "$SCALA_VERSION" ]; then SCALA_VERSION=2.10.5 fi @@ -96,9 +87,20 @@ if [ $JMX_PORT ]; then KAFKA_JMX_OPTS="$KAFKA_JMX_OPTS -Dcom.sun.management.jmxremote.port=$JMX_PORT " fi +# Log directory to use +if [ "x$LOG_DIR" = "x" ]; then + LOG_DIR="$base_dir/logs" +fi + # Log4j settings if [ -z "$KAFKA_LOG4J_OPTS" ]; then + # Log to console. This is a tool. KAFKA_LOG4J_OPTS="-Dlog4j.configuration=file:$base_dir/config/tools-log4j.properties" +else + # create logs directory + if [ ! -d "$LOG_DIR" ]; then + mkdir -p "$LOG_DIR" + fi fi KAFKA_LOG4J_OPTS="-Dkafka.logs.dir=$LOG_DIR $KAFKA_LOG4J_OPTS" -- 1.7.12.4 From 85d8218ef41381c58eb16d7542e41416c82bdf4a Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Wed, 5 Aug 2015 12:50:05 -0700 Subject: [PATCH 087/120] KAFKA-2401: Fix transient failure in ProducerSendTest.testCloseWithZeroTimeoutFromSenderThread Author: Jiangjie Qin Reviewers: Guozhang Closes #113 from becketqin/KAFKA-2401 and squashes the following commits: 7d4223d [Jiangjie Qin] KAFKA-2401: fix transient failure in ProducerSendTest.testCloseWithZeroTimeoutFromSenderThread --- .../integration/kafka/api/ProducerSendTest.scala | 25 ++++++---------------- 1 file changed, 7 insertions(+), 18 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala index 9ce4bd5..5c6ccbc 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala @@ -372,20 +372,18 @@ class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness { var producer: KafkaProducer[Array[Byte],Array[Byte]] = null try { // create topic - val leaders = TestUtils.createTopic(zkClient, topic, 2, 2, servers) - val leader0 = leaders(0) - val leader1 = leaders(1) + val leaders = TestUtils.createTopic(zkClient, topic, 1, 2, servers) + val leader = leaders(0) // create record - val record0 = new ProducerRecord[Array[Byte], Array[Byte]](topic, 0, null, "value".getBytes) - val record1 = new ProducerRecord[Array[Byte], Array[Byte]](topic, 1, null, "value".getBytes) + val record = new ProducerRecord[Array[Byte], Array[Byte]](topic, 0, null, "value".getBytes) // Test closing from sender thread. class CloseCallback(producer: KafkaProducer[Array[Byte], Array[Byte]]) extends Callback { override def onCompletion(metadata: RecordMetadata, exception: Exception) { // Trigger another batch in accumulator before close the producer. These messages should // not be sent. - (0 until numRecords) map (i => producer.send(record1)) + (0 until numRecords) map (i => producer.send(record)) // The close call will be called by all the message callbacks. This tests idempotence of the close call. producer.close(0, TimeUnit.MILLISECONDS) // Test close with non zero timeout. Should not block at all. @@ -395,29 +393,20 @@ class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness { for(i <- 0 until 50) { producer = TestUtils.createNewProducer(brokerList, lingerMs = Long.MaxValue) // send message to partition 0 - var responses = (0 until numRecords) map (i => producer.send(record0)) - // send message to partition 1 - responses ++= ((0 until numRecords) map (i => producer.send(record1, new CloseCallback(producer)))) + val responses = ((0 until numRecords) map (i => producer.send(record, new CloseCallback(producer)))) assertTrue("No request is complete.", responses.forall(!_.isDone())) // flush the messages. producer.flush() assertTrue("All request are complete.", responses.forall(_.isDone())) // Check the messages received by broker. - val fetchResponse0 = if (leader0.get == configs(0).brokerId) { + val fetchResponse = if (leader.get == configs(0).brokerId) { consumer1.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, Int.MaxValue).build()) } else { consumer2.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, Int.MaxValue).build()) } - val fetchResponse1 = if (leader1.get == configs(0).brokerId) { - consumer1.fetch(new FetchRequestBuilder().addFetch(topic, 1, 0, Int.MaxValue).build()) - } else { - consumer2.fetch(new FetchRequestBuilder().addFetch(topic, 1, 0, Int.MaxValue).build()) - } val expectedNumRecords = (i + 1) * numRecords assertEquals("Fetch response to partition 0 should have %d messages.".format(expectedNumRecords), - expectedNumRecords, fetchResponse0.messageSet(topic, 0).size) - assertEquals("Fetch response to partition 1 should have %d messages.".format(expectedNumRecords), - expectedNumRecords, fetchResponse1.messageSet(topic, 1).size) + expectedNumRecords, fetchResponse.messageSet(topic, 0).size) } } finally { if (producer != null) -- 1.7.12.4 From 27d499fe66fc034669a1852ca33ee2f486e1b83c Mon Sep 17 00:00:00 2001 From: Grant Henke Date: Wed, 5 Aug 2015 15:55:48 -0700 Subject: [PATCH 088/120] =?UTF-8?q?KAFKA-2393:=20Correctly=20Handle=20Invali?= =?UTF-8?q?dTopicException=20in=20KafkaApis.getTo=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit …picMetadata() Author: Grant Henke Reviewers: Jun Rao Closes #117 from granthenke/invalid-topic and squashes the following commits: 0abda5f [Grant Henke] KAFKA-2393: Correctly Handle InvalidTopicException in KafkaApis.getTopicMetadata() --- core/src/main/scala/kafka/server/KafkaApis.scala | 5 +++- .../unit/kafka/integration/TopicMetadataTest.scala | 28 ++++++++++++++++++++++ 2 files changed, 32 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 18f5b5b..7ea509c 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -462,10 +462,13 @@ class KafkaApis(val requestChannel: RequestChannel, info("Auto creation of topic %s with %d partitions and replication factor %d is successful!" .format(topic, config.numPartitions, config.defaultReplicationFactor)) } + new TopicMetadata(topic, Seq.empty[PartitionMetadata], ErrorMapping.LeaderNotAvailableCode) } catch { case e: TopicExistsException => // let it go, possibly another broker created this topic + new TopicMetadata(topic, Seq.empty[PartitionMetadata], ErrorMapping.LeaderNotAvailableCode) + case itex: InvalidTopicException => + new TopicMetadata(topic, Seq.empty[PartitionMetadata], ErrorMapping.InvalidTopicCode) } - new TopicMetadata(topic, Seq.empty[PartitionMetadata], ErrorMapping.LeaderNotAvailableCode) } else { new TopicMetadata(topic, Seq.empty[PartitionMetadata], ErrorMapping.UnknownTopicOrPartitionCode) } diff --git a/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala b/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala index 5b6c9d6..9aebec4 100644 --- a/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala +++ b/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala @@ -135,6 +135,34 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { assertTrue(partitionMetadata.head.leader.isDefined) } + def testAutoCreateTopicWithCollision { + // auto create topic + val topic1 = "testAutoCreate_Topic" + val topic2 = "testAutoCreate.Topic" + var topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic1, topic2), brokerEndPoints, "TopicMetadataTest-testAutoCreateTopic", + 2000,0).topicsMetadata + assertEquals("Expecting metadata for 2 topics", 2, topicsMetadata.size) + assertEquals("Expecting metadata for topic1", topic1, topicsMetadata.head.topic) + assertEquals(ErrorMapping.LeaderNotAvailableCode, topicsMetadata.head.errorCode) + assertEquals("Expecting metadata for topic2", topic2, topicsMetadata(1).topic) + assertEquals("Expecting InvalidTopicCode for topic2 metadata", ErrorMapping.InvalidTopicCode, topicsMetadata(1).errorCode) + + // wait for leader to be elected + TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic1, 0) + TestUtils.waitUntilMetadataIsPropagated(Seq(server1), topic1, 0) + + // retry the metadata for the first auto created topic + topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic1), brokerEndPoints, "TopicMetadataTest-testBasicTopicMetadata", + 2000,0).topicsMetadata + assertEquals(ErrorMapping.NoError, topicsMetadata.head.errorCode) + assertEquals(ErrorMapping.NoError, topicsMetadata.head.partitionsMetadata.head.errorCode) + var partitionMetadata = topicsMetadata.head.partitionsMetadata + assertEquals("Expecting metadata for 1 partition", 1, partitionMetadata.size) + assertEquals("Expecting partition id to be 0", 0, partitionMetadata.head.partitionId) + assertEquals(1, partitionMetadata.head.replicas.size) + assertTrue(partitionMetadata.head.leader.isDefined) + } + private def checkIsr(servers: Seq[KafkaServer]): Unit = { val activeBrokers: Seq[KafkaServer] = servers.filter(x => x.brokerState.currentState != NotRunning.state) val expectedIsr: Seq[BrokerEndPoint] = activeBrokers.map( -- 1.7.12.4 From 95c626dc04e033c4349e68d47a3bd6e21b77706c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Stevo=20Slavi=C4=87?= Date: Wed, 5 Aug 2015 16:32:25 -0700 Subject: [PATCH 089/120] MINOR: ConsumerRecords are organized per topic partition MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ConsumerRecords has records organized per topic partition, not per topic as ConsumerRecords javadoc suggested. Author: Stevo Slavić Reviewers: Jason, Guozhang Closes #92 from sslavic/patch-6 and squashes the following commits: b08a58d [Stevo Slavić] MINOR: ConsumerRecords are organized per topic partition --- .../main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java index 16a8357..8e6fef4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java @@ -23,8 +23,8 @@ import java.util.Map; /** * A container that holds the list {@link ConsumerRecord} per partition for a - * particular topic. There is one for every topic returned by a - * {@link Consumer#poll(long)} operation. + * particular topic. There is one {@link ConsumerRecord} list for every topic + * partition returned by a {@link Consumer#poll(long)} operation. */ public class ConsumerRecords implements Iterable> { public static final ConsumerRecords EMPTY = -- 1.7.12.4 From 4b798f0dee004ac6b96ff563a0e2df65e016c694 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Stevo=20Slavi=C4=87?= Date: Wed, 5 Aug 2015 16:33:55 -0700 Subject: [PATCH 090/120] MINOR: auto.offset.reset docs not in sync with validation MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit In this commit https://github.com/apache/kafka/commit/0699ff2ce60abb466cab5315977a224f1a70a4da#diff-5533ddc72176acd1c32f5abbe94aa672 among other things auto.offset.reset possible options were changed from smallest to earliest and from largest to latest, but not in documentation for that configuration property. This patch fixes documentation for auto.offset.reset consumer configuration property so it is in sync with validation logic. Author: Stevo Slavić Reviewers: Jason, Ismael, Guozhang Closes #91 from sslavic/patch-5 and squashes the following commits: f4c9656 [Stevo Slavić] MINOR: auto.offset.reset docs not in sync with validation --- .../src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 70377ae..8adabb6 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 @@ -77,7 +77,7 @@ public class ConsumerConfig extends AbstractConfig { * auto.offset.reset */ public static final String AUTO_OFFSET_RESET_CONFIG = "auto.offset.reset"; - private static final String AUTO_OFFSET_RESET_DOC = "What to do when there is no initial offset in Kafka or if the current offset does not exist any more on the server (e.g. because that data has been deleted):
    • smallest: automatically reset the offset to the smallest offset
    • largest: automatically reset the offset to the largest offset
    • none: throw exception to the consumer if no previous offset is found for the consumer's group
    • anything else: throw exception to the consumer.
    "; + private static final String AUTO_OFFSET_RESET_DOC = "What to do when there is no initial offset in Kafka or if the current offset does not exist any more on the server (e.g. because that data has been deleted):
    • earliest: automatically reset the offset to the earliest offset
    • latest: automatically reset the offset to the latest offset
    • none: throw exception to the consumer if no previous offset is found for the consumer's group
    • anything else: throw exception to the consumer.
    "; /** * fetch.min.bytes -- 1.7.12.4 From 907ee7b5eae9e163e7abb24f97dedbaa78bc9d8c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Stevo=20Slavi=C4=87?= Date: Wed, 5 Aug 2015 16:34:57 -0700 Subject: [PATCH 091/120] MINOR: Fixed javadoc for committed return value MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit If no offset has been committed, then committed method does not return (null) value, instead NoOffsetForPartitionException is thrown in that case. Author: Stevo Slavić Reviewers: Ismael, Guozhang Closes #89 from sslavic/patch-4 and squashes the following commits: 5c0a152 [Stevo Slavić] MINOR: Fixed javadoc for committed return value --- .../src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 7851644..c57bba0 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 @@ -960,7 +960,7 @@ public class KafkaConsumer implements Consumer { * consumer hasn't yet initialized it's cache of committed offsets. * * @param partition The partition to check - * @return The last committed offset or null if no offset has been committed + * @return The last committed offset * @throws NoOffsetForPartitionException If no offset has ever been committed by any process for the given * partition. */ -- 1.7.12.4 From 9b1c52d52e4e9bbbd89b125fd510bf8b247643ac Mon Sep 17 00:00:00 2001 From: Andrew Otto Date: Wed, 5 Aug 2015 16:51:25 -0700 Subject: [PATCH 092/120] MINOR: Fix typo in ReplicaVerificationTool output Author: Andrew Otto Reviewers: Ismael Juma, Guozhang Wang Closes #101 from ottomata/trunk and squashes the following commits: 10b76f3 [Andrew Otto] MINOR - Fix typo in ReplicaVerificationTool output --- core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala b/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala index 1366172..fd9daec 100644 --- a/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala +++ b/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala @@ -92,7 +92,7 @@ object ReplicaVerificationTool extends Logging { .describedAs("ms") .ofType(classOf[java.lang.Long]) .defaultsTo(30 * 1000L) - + if(args.length == 0) CommandLineUtils.printUsageAndDie(parser, "Validate that all replicas for a set of topics have the same data.") @@ -329,7 +329,7 @@ private class ReplicaBuffer(expectedReplicasPerTopicAndPartition: Map[TopicAndPa if (currentTimeMs - lastReportTime > reportInterval) { println(ReplicaVerificationTool.dateFormat.format(new Date(currentTimeMs)) + ": max lag is " + maxLag + " for partition " + maxLagTopicAndPartition + " at offset " + offsetWithMaxLag - + " among " + messageSetCache.size + " paritions") + + " among " + messageSetCache.size + " partitions") lastReportTime = currentTimeMs } } -- 1.7.12.4 From 006b45c7e5b94e70b3a4200f4646042f3557c48e Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Thu, 6 Aug 2015 14:17:30 -0700 Subject: [PATCH 093/120] KAFKA-2400: Expose heartbeat interval in KafkaConsumer configuration Author: Jason Gustafson Reviewers: Guozhang Wang Closes #116 from hachikuji/KAFKA-2400 and squashes the following commits: 3c1b1dd [Jason Gustafson] KAFKA-2400; expose heartbeat interval in KafkaConsumer configuration --- .../kafka/clients/consumer/ConsumerConfig.java | 11 ++++++++++ .../kafka/clients/consumer/KafkaConsumer.java | 1 + .../clients/consumer/internals/Coordinator.java | 3 ++- .../clients/consumer/internals/Heartbeat.java | 24 +++++++++++----------- .../consumer/internals/CoordinatorTest.java | 2 ++ .../clients/consumer/internals/HeartbeatTest.java | 7 ++++--- .../integration/kafka/api/ConsumerBounceTest.scala | 3 ++- .../scala/integration/kafka/api/ConsumerTest.scala | 3 +++ 8 files changed, 37 insertions(+), 17 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 8adabb6..d35b421 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 @@ -51,6 +51,12 @@ public class ConsumerConfig extends AbstractConfig { private static final String SESSION_TIMEOUT_MS_DOC = "The timeout used to detect failures when using Kafka's group management facilities."; /** + * heartbeat.interval.ms + */ + public static final String HEARTBEAT_INTERVAL_MS_CONFIG = "heartbeat.interval.ms"; + private static final String HEARTBEAT_INTERVAL_MS_DOC = "The expected time between heartbeats to the consumer coordinator when using Kafka's group management facilities. Heartbeats are used to ensure that the consumer's session stays active and to facilitate rebalancing when new consumers join or leave the group. The value must be set lower than session.timeout.ms, but typically should be set no higher than 1/3 of that value. It can be adjusted even lower to control the expected time for normal rebalances."; + + /** * bootstrap.servers */ public static final String BOOTSTRAP_SERVERS_CONFIG = CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG; @@ -171,6 +177,11 @@ public class ConsumerConfig extends AbstractConfig { 30000, Importance.HIGH, SESSION_TIMEOUT_MS_DOC) + .define(HEARTBEAT_INTERVAL_MS_CONFIG, + Type.INT, + 3000, + Importance.HIGH, + HEARTBEAT_INTERVAL_MS_DOC) .define(PARTITION_ASSIGNMENT_STRATEGY_CONFIG, Type.STRING, "range", 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 c57bba0..ed99e9b 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 @@ -540,6 +540,7 @@ public class KafkaConsumer implements Consumer { this.coordinator = new Coordinator(this.client, config.getString(ConsumerConfig.GROUP_ID_CONFIG), config.getInt(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG), + config.getInt(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG), config.getString(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG), this.subscriptions, metrics, 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 cd5cdc3..70442aa 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 @@ -84,6 +84,7 @@ public final class Coordinator { public Coordinator(ConsumerNetworkClient client, String groupId, int sessionTimeoutMs, + int heartbeatIntervalMs, String assignmentStrategy, SubscriptionState subscriptions, Metrics metrics, @@ -103,7 +104,7 @@ public final class Coordinator { this.subscriptions = subscriptions; this.sessionTimeoutMs = sessionTimeoutMs; this.assignmentStrategy = assignmentStrategy; - this.heartbeat = new Heartbeat(this.sessionTimeoutMs, time.milliseconds()); + this.heartbeat = new Heartbeat(this.sessionTimeoutMs, heartbeatIntervalMs, time.milliseconds()); this.heartbeatTask = new HeartbeatTask(); this.sensors = new CoordinatorMetrics(metrics, metricGrpPrefix, metricTags); this.requestTimeoutMs = requestTimeoutMs; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java index 6da8936..79e17e2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java @@ -16,20 +16,21 @@ package org.apache.kafka.clients.consumer.internals; * A helper class for managing the heartbeat to the coordinator */ public final class Heartbeat { - - /* The number of heartbeats to attempt to complete per session timeout interval. - * so, e.g., with a session timeout of 3 seconds we would attempt a heartbeat - * once per second. - */ - public final static int HEARTBEATS_PER_SESSION_INTERVAL = 3; - private final long timeout; + private final long interval; + private long lastHeartbeatSend; private long lastHeartbeatReceive; private long lastSessionReset; - public Heartbeat(long timeout, long now) { + public Heartbeat(long timeout, + long interval, + long now) { + if (interval >= timeout) + throw new IllegalArgumentException("Heartbeat must be set lower than the session timeout"); + this.timeout = timeout; + this.interval = interval; this.lastSessionReset = now; } @@ -52,11 +53,10 @@ public final class Heartbeat { public long timeToNextHeartbeat(long now) { long timeSinceLastHeartbeat = now - Math.max(lastHeartbeatSend, lastSessionReset); - long hbInterval = timeout / HEARTBEATS_PER_SESSION_INTERVAL; - if (timeSinceLastHeartbeat > hbInterval) + if (timeSinceLastHeartbeat > interval) return 0; else - return hbInterval - timeSinceLastHeartbeat; + return interval - timeSinceLastHeartbeat; } public boolean sessionTimeoutExpired(long now) { @@ -64,7 +64,7 @@ public final class Heartbeat { } public long interval() { - return timeout / HEARTBEATS_PER_SESSION_INTERVAL; + return interval; } public void resetSessionTimeout(long now) { 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 ca832be..a23b8e7 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 @@ -59,6 +59,7 @@ public class CoordinatorTest { private String groupId = "test-group"; private TopicPartition tp = new TopicPartition(topicName, 0); private int sessionTimeoutMs = 10; + private int heartbeatIntervalMs = 2; private long retryBackoffMs = 100; private long requestTimeoutMs = 5000; private String rebalanceStrategy = "not-matter"; @@ -89,6 +90,7 @@ public class CoordinatorTest { this.coordinator = new Coordinator(consumerClient, groupId, sessionTimeoutMs, + heartbeatIntervalMs, rebalanceStrategy, subscriptions, metrics, diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatTest.java index b587e14..75e68cc 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatTest.java @@ -27,20 +27,21 @@ import static org.junit.Assert.assertTrue; public class HeartbeatTest { private long timeout = 300L; + private long interval = 100L; private MockTime time = new MockTime(); - private Heartbeat heartbeat = new Heartbeat(timeout, -1L); + private Heartbeat heartbeat = new Heartbeat(timeout, interval, -1L); @Test public void testShouldHeartbeat() { heartbeat.sentHeartbeat(time.milliseconds()); - time.sleep((long) ((float) timeout / Heartbeat.HEARTBEATS_PER_SESSION_INTERVAL * 1.1)); + time.sleep((long) ((float) interval * 1.1)); assertTrue(heartbeat.shouldHeartbeat(time.milliseconds())); } @Test public void testShouldNotHeartbeat() { heartbeat.sentHeartbeat(time.milliseconds()); - time.sleep(timeout / (2 * Heartbeat.HEARTBEATS_PER_SESSION_INTERVAL)); + time.sleep(interval / 2); assertFalse(heartbeat.shouldHeartbeat(time.milliseconds())); } diff --git a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala index 93f9468..1d07391 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala @@ -43,7 +43,8 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging { this.producerConfig.setProperty(ProducerConfig.ACKS_CONFIG, "all") this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "my-test") this.consumerConfig.setProperty(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, 4096.toString) - this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "20") + this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "100") + this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "30") this.consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") override def generateConfigs() = { diff --git a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala index 4ea49f2..f9e22ba 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala @@ -207,6 +207,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging { def testPartitionReassignmentCallback() { val callback = new TestConsumerReassignmentCallback() this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "100"); // timeout quickly to avoid slow test + this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "30"); val consumer0 = new KafkaConsumer(this.consumerConfig, callback, new ByteArrayDeserializer(), new ByteArrayDeserializer()) consumer0.subscribe(topic) @@ -238,6 +239,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging { def testUnsubscribeTopic() { val callback = new TestConsumerReassignmentCallback() this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "100"); // timeout quickly to avoid slow test + this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "30"); val consumer0 = new KafkaConsumer(this.consumerConfig, callback, new ByteArrayDeserializer(), new ByteArrayDeserializer()) try { @@ -267,6 +269,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging { def testPauseStateNotPreservedByRebalance() { this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "100"); // timeout quickly to avoid slow test + this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "30"); val consumer0 = new KafkaConsumer(this.consumerConfig, null, new ByteArrayDeserializer(), new ByteArrayDeserializer()) sendRecords(5) -- 1.7.12.4 From f6373e4d9929d123a8474ab7673ee701f63ac593 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Thu, 6 Aug 2015 15:52:43 -0700 Subject: [PATCH 094/120] KAFKA-2340: Improve KafkaConsumer Fetcher test coverage Author: Jason Gustafson Reviewers: Guozhang Wang Closes #112 from hachikuji/KAFKA-2340 and squashes the following commits: cc49ca2 [Jason Gustafson] KAFKA-2340; improve KafkaConsumer Fetcher test coverage --- .../kafka/clients/consumer/internals/Fetcher.java | 4 +- .../java/org/apache/kafka/clients/MockClient.java | 49 +++++++- .../clients/consumer/internals/FetcherTest.java | 133 +++++++++++++++++++-- 3 files changed, 169 insertions(+), 17 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index 956197b..9dc6697 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -61,8 +61,8 @@ import java.util.Set; * This class manage the fetching process with the brokers. */ public class Fetcher { - private static final long EARLIEST_OFFSET_TIMESTAMP = -2L; - private static final long LATEST_OFFSET_TIMESTAMP = -1L; + public static final long EARLIEST_OFFSET_TIMESTAMP = -2L; + public static final long LATEST_OFFSET_TIMESTAMP = -1L; private static final Logger log = LoggerFactory.getLogger(Fetcher.class); diff --git a/clients/src/test/java/org/apache/kafka/clients/MockClient.java b/clients/src/test/java/org/apache/kafka/clients/MockClient.java index d9c97e9..9133d85 100644 --- a/clients/src/test/java/org/apache/kafka/clients/MockClient.java +++ b/clients/src/test/java/org/apache/kafka/clients/MockClient.java @@ -34,15 +34,24 @@ import org.apache.kafka.common.utils.Time; * A mock network client for use testing code */ public class MockClient implements KafkaClient { + public static final RequestMatcher ALWAYS_TRUE = new RequestMatcher() { + @Override + public boolean matches(ClientRequest request) { + return true; + } + }; private class FutureResponse { public final Struct responseBody; public final boolean disconnected; + public final RequestMatcher requestMatcher; - public FutureResponse(Struct responseBody, boolean disconnected) { + public FutureResponse(Struct responseBody, boolean disconnected, RequestMatcher requestMatcher) { this.responseBody = responseBody; this.disconnected = disconnected; + this.requestMatcher = requestMatcher; } + } private final Time time; @@ -94,6 +103,9 @@ public class MockClient implements KafkaClient { public void send(ClientRequest request) { if (!futureResponses.isEmpty()) { FutureResponse futureResp = futureResponses.poll(); + if (!futureResp.requestMatcher.matches(request)) + throw new IllegalStateException("Next in line response did not match expected request"); + ClientResponse resp = new ClientResponse(request, time.milliseconds(), futureResp.disconnected, futureResp.responseBody); responses.add(resp); } else { @@ -141,11 +153,32 @@ public class MockClient implements KafkaClient { } public void prepareResponse(Struct body) { - prepareResponse(body, false); + prepareResponse(ALWAYS_TRUE, body, false); + } + + /** + * Prepare a response for a request matching the provided matcher. If the matcher does not + * match, {@link #send(ClientRequest)} will throw IllegalStateException + * @param matcher The matcher to apply + * @param body The response body + */ + public void prepareResponse(RequestMatcher matcher, Struct body) { + prepareResponse(matcher, body, false); } public void prepareResponse(Struct body, boolean disconnected) { - futureResponses.add(new FutureResponse(body, disconnected)); + prepareResponse(ALWAYS_TRUE, body, disconnected); + } + + /** + * Prepare a response for a request matching the provided matcher. If the matcher does not + * match, {@link #send(ClientRequest)} will throw IllegalStateException + * @param matcher The matcher to apply + * @param body The response body + * @param disconnected Whether the request was disconnected + */ + public void prepareResponse(RequestMatcher matcher, Struct body, boolean disconnected) { + futureResponses.add(new FutureResponse(body, disconnected, matcher)); } public void setNode(Node node) { @@ -180,4 +213,14 @@ public class MockClient implements KafkaClient { return this.node; } + /** + * The RequestMatcher provides a way to match a particular request to a response prepared + * through {@link #prepareResponse(RequestMatcher, Struct)}. Basically this allows testers + * to inspect the request body for the type of the request or for specific fields that should be set, + * and to fail the test if it doesn't match. + */ + public interface RequestMatcher { + boolean matches(ClientRequest request); + } + } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java index 56850bb..a7c83ca 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.clients.consumer.internals; +import org.apache.kafka.clients.ClientRequest; import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.MockClient; import org.apache.kafka.clients.consumer.ConsumerRecord; @@ -30,6 +31,8 @@ import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.requests.FetchResponse; +import org.apache.kafka.common.requests.ListOffsetRequest; +import org.apache.kafka.common.requests.ListOffsetResponse; import org.apache.kafka.common.requests.MetadataResponse; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.utils.MockTime; @@ -40,11 +43,13 @@ import org.junit.Test; import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; @@ -158,26 +163,34 @@ public class FetcherTest { } @Test - public void testFetchFailed() { + public void testFetchNotLeaderForPartition() { subscriptions.subscribe(tp); subscriptions.seek(tp, 0); - // fetch with not leader fetcher.initFetches(cluster); client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NOT_LEADER_FOR_PARTITION.code(), 100L)); consumerClient.poll(0); assertEquals(0, fetcher.fetchedRecords().size()); assertEquals(0L, metadata.timeToNextUpdate(time.milliseconds())); + } + + @Test + public void testFetchUnknownTopicOrPartition() { + subscriptions.subscribe(tp); + subscriptions.seek(tp, 0); - // fetch with unknown topic partition fetcher.initFetches(cluster); client.prepareResponse(fetchResponse(this.records.buffer(), Errors.UNKNOWN_TOPIC_OR_PARTITION.code(), 100L)); consumerClient.poll(0); assertEquals(0, fetcher.fetchedRecords().size()); assertEquals(0L, metadata.timeToNextUpdate(time.milliseconds())); + } + + @Test + public void testFetchOffsetOutOfRange() { + subscriptions.subscribe(tp); + subscriptions.seek(tp, 0); - // fetch with out of range - subscriptions.fetched(tp, 5); fetcher.initFetches(cluster); client.prepareResponse(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L)); consumerClient.poll(0); @@ -188,18 +201,94 @@ public class FetcherTest { } @Test - public void testFetchOutOfRange() { + public void testFetchDisconnected() { subscriptions.subscribe(tp); - subscriptions.seek(tp, 5); + subscriptions.seek(tp, 0); - // fetch with out of range fetcher.initFetches(cluster); - client.prepareResponse(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L)); + client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L), true); consumerClient.poll(0); - assertTrue(subscriptions.isOffsetResetNeeded(tp)); assertEquals(0, fetcher.fetchedRecords().size()); - assertEquals(null, subscriptions.fetched(tp)); - assertEquals(null, subscriptions.consumed(tp)); + + // disconnects should have no affect on subscription state + assertFalse(subscriptions.isOffsetResetNeeded(tp)); + assertTrue(subscriptions.isFetchable(tp)); + assertEquals(0, (long) subscriptions.fetched(tp)); + assertEquals(0, (long) subscriptions.consumed(tp)); + } + + @Test + public void testUpdateFetchPositionToCommitted() { + // unless a specific reset is expected, the default behavior is to reset to the committed + // position if one is present + subscriptions.subscribe(tp); + subscriptions.committed(tp, 5); + + fetcher.updateFetchPositions(Collections.singleton(tp)); + assertTrue(subscriptions.isFetchable(tp)); + assertEquals(5, (long) subscriptions.fetched(tp)); + assertEquals(5, (long) subscriptions.consumed(tp)); + } + + @Test + public void testUpdateFetchPositionResetToDefaultOffset() { + subscriptions.subscribe(tp); + // with no commit position, we should reset using the default strategy defined above (EARLIEST) + + client.prepareResponse(listOffsetRequestMatcher(Fetcher.EARLIEST_OFFSET_TIMESTAMP), + listOffsetResponse(Errors.NONE, Arrays.asList(5L))); + fetcher.updateFetchPositions(Collections.singleton(tp)); + assertFalse(subscriptions.isOffsetResetNeeded(tp)); + assertTrue(subscriptions.isFetchable(tp)); + assertEquals(5, (long) subscriptions.fetched(tp)); + assertEquals(5, (long) subscriptions.consumed(tp)); + } + + @Test + public void testUpdateFetchPositionResetToLatestOffset() { + subscriptions.subscribe(tp); + subscriptions.needOffsetReset(tp, OffsetResetStrategy.LATEST); + + client.prepareResponse(listOffsetRequestMatcher(Fetcher.LATEST_OFFSET_TIMESTAMP), + listOffsetResponse(Errors.NONE, Arrays.asList(5L))); + fetcher.updateFetchPositions(Collections.singleton(tp)); + assertFalse(subscriptions.isOffsetResetNeeded(tp)); + assertTrue(subscriptions.isFetchable(tp)); + assertEquals(5, (long) subscriptions.fetched(tp)); + assertEquals(5, (long) subscriptions.consumed(tp)); + } + + @Test + public void testUpdateFetchPositionResetToEarliestOffset() { + subscriptions.subscribe(tp); + subscriptions.needOffsetReset(tp, OffsetResetStrategy.EARLIEST); + + client.prepareResponse(listOffsetRequestMatcher(Fetcher.EARLIEST_OFFSET_TIMESTAMP), + listOffsetResponse(Errors.NONE, Arrays.asList(5L))); + fetcher.updateFetchPositions(Collections.singleton(tp)); + assertFalse(subscriptions.isOffsetResetNeeded(tp)); + assertTrue(subscriptions.isFetchable(tp)); + assertEquals(5, (long) subscriptions.fetched(tp)); + assertEquals(5, (long) subscriptions.consumed(tp)); + } + + @Test + public void testUpdateFetchPositionDisconnect() { + subscriptions.subscribe(tp); + subscriptions.needOffsetReset(tp, OffsetResetStrategy.LATEST); + + // First request gets a disconnect + client.prepareResponse(listOffsetRequestMatcher(Fetcher.LATEST_OFFSET_TIMESTAMP), + listOffsetResponse(Errors.NONE, Arrays.asList(5L)), true); + + // Next one succeeds + client.prepareResponse(listOffsetRequestMatcher(Fetcher.LATEST_OFFSET_TIMESTAMP), + listOffsetResponse(Errors.NONE, Arrays.asList(5L))); + fetcher.updateFetchPositions(Collections.singleton(tp)); + assertFalse(subscriptions.isOffsetResetNeeded(tp)); + assertTrue(subscriptions.isFetchable(tp)); + assertEquals(5, (long) subscriptions.fetched(tp)); + assertEquals(5, (long) subscriptions.consumed(tp)); } @Test @@ -213,6 +302,26 @@ public class FetcherTest { assertEquals(cluster.topics().size(), allTopics.size()); } + private MockClient.RequestMatcher listOffsetRequestMatcher(final long timestamp) { + // matches any list offset request with the provided timestamp + return new MockClient.RequestMatcher() { + @Override + public boolean matches(ClientRequest request) { + ListOffsetRequest req = new ListOffsetRequest(request.request().body()); + ListOffsetRequest.PartitionData partitionData = req.offsetData().get(tp); + return partitionData != null && partitionData.timestamp == timestamp; + } + }; + } + + private Struct listOffsetResponse(Errors error, List offsets) { + ListOffsetResponse.PartitionData partitionData = new ListOffsetResponse.PartitionData(error.code(), offsets); + Map allPartitionData = new HashMap<>(); + allPartitionData.put(tp, partitionData); + ListOffsetResponse response = new ListOffsetResponse(allPartitionData); + return response.toStruct(); + } + private Struct fetchResponse(ByteBuffer buffer, short error, long hw) { FetchResponse response = new FetchResponse(Collections.singletonMap(tp, new FetchResponse.PartitionData(error, hw, buffer))); return response.toStruct(); -- 1.7.12.4 From 63b820c592aba3ec6f26cc99c0c470795029b10c Mon Sep 17 00:00:00 2001 From: Onur Karaman Date: Fri, 7 Aug 2015 15:20:09 -0700 Subject: [PATCH 095/120] KAFKA-2413; New consumer's subscribe(Topic...) api fails if called more than once Author: Onur Karaman Reviewers: Ashish Singh, Ismael Juma, Jason Gustafson Closes #122 from onurkaraman/KAFKA-2413 and squashes the following commits: cc340fc [Onur Karaman] fix ConsumerCoordinator updateConsumer --- .../kafka/coordinator/ConsumerCoordinator.scala | 2 +- .../scala/integration/kafka/api/ConsumerTest.scala | 60 +++++++++++++++++----- 2 files changed, 49 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala b/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala index 6c2df4c..1bceb43 100644 --- a/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala @@ -326,7 +326,7 @@ class ConsumerCoordinator(val brokerId: Int, private def updateConsumer(group: ConsumerGroupMetadata, consumer: ConsumerMetadata, topics: Set[String]) { val topicsToBind = topics -- group.topics group.remove(consumer.consumerId) - val topicsToUnbind = consumer.topics -- group.topics + val topicsToUnbind = consumer.topics -- (group.topics ++ topics) group.add(consumer.consumerId, consumer) consumer.topics = topics coordinatorMetadata.bindAndUnbindGroupFromTopics(group.groupId, topicsToBind, topicsToUnbind) diff --git a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala index f9e22ba..79f1640 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala @@ -18,7 +18,7 @@ import org.apache.kafka.clients.producer.ProducerConfig import org.apache.kafka.clients.producer.ProducerRecord import org.apache.kafka.clients.consumer._ import org.apache.kafka.common.serialization.ByteArrayDeserializer -import org.apache.kafka.common.{PartitionInfo, TopicPartition} +import org.apache.kafka.common.TopicPartition import kafka.utils.{TestUtils, Logging} import kafka.server.KafkaConfig @@ -26,7 +26,7 @@ import kafka.server.KafkaConfig import java.util.ArrayList import org.junit.Assert._ -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import kafka.coordinator.ConsumerCoordinator @@ -95,7 +95,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging { this.consumers(0).poll(50) val pos1 = this.consumers(0).position(tp) val pos2 = this.consumers(0).position(tp2) - this.consumers(0).commit(Map[TopicPartition,java.lang.Long]((tp, 3L)), CommitType.SYNC) + this.consumers(0).commit(Map[TopicPartition,java.lang.Long]((tp, 3L)).asJava, CommitType.SYNC) assertEquals(3, this.consumers(0).committed(tp)) intercept[NoOffsetForPartitionException] { this.consumers(0).committed(tp2) @@ -103,13 +103,13 @@ class ConsumerTest extends IntegrationTestHarness with Logging { // positions should not change assertEquals(pos1, this.consumers(0).position(tp)) assertEquals(pos2, this.consumers(0).position(tp2)) - this.consumers(0).commit(Map[TopicPartition,java.lang.Long]((tp2, 5L)), CommitType.SYNC) + this.consumers(0).commit(Map[TopicPartition,java.lang.Long]((tp2, 5L)).asJava, CommitType.SYNC) assertEquals(3, this.consumers(0).committed(tp)) assertEquals(5, this.consumers(0).committed(tp2)) // Using async should pick up the committed changes after commit completes val commitCallback = new CountConsumerCommitCallback() - this.consumers(0).commit(Map[TopicPartition,java.lang.Long]((tp2, 7L)), CommitType.ASYNC, commitCallback) + this.consumers(0).commit(Map[TopicPartition,java.lang.Long]((tp2, 7L)).asJava, CommitType.ASYNC, commitCallback) awaitCommitCallback(this.consumers(0), commitCallback) assertEquals(7, this.consumers(0).committed(tp2)) } @@ -182,7 +182,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging { TestUtils.createTopic(this.zkClient, "part-test", numParts, 1, this.servers) val parts = this.consumers(0).partitionsFor("part-test") assertNotNull(parts) - assertEquals(2, parts.length) + assertEquals(2, parts.size) assertNull(this.consumers(0).partitionsFor("non-exist-topic")) } @@ -199,9 +199,9 @@ class ConsumerTest extends IntegrationTestHarness with Logging { assertNotNull(topics) assertEquals(5, topics.size()) assertEquals(5, topics.keySet().size()) - assertEquals(2, topics.get(topic1).length) - assertEquals(2, topics.get(topic2).length) - assertEquals(2, topics.get(topic3).length) + assertEquals(2, topics.get(topic1).size) + assertEquals(2, topics.get(topic2).size) + assertEquals(2, topics.get(topic3).size) } def testPartitionReassignmentCallback() { @@ -216,9 +216,9 @@ class ConsumerTest extends IntegrationTestHarness with Logging { consumer0.poll(50) // get metadata for the topic - var parts = consumer0.partitionsFor(ConsumerCoordinator.OffsetsTopicName) + var parts = consumer0.partitionsFor(ConsumerCoordinator.OffsetsTopicName).asScala while(parts == null) - parts = consumer0.partitionsFor(ConsumerCoordinator.OffsetsTopicName) + parts = consumer0.partitionsFor(ConsumerCoordinator.OffsetsTopicName).asScala assertEquals(1, parts.size) assertNotNull(parts(0).leader()) @@ -256,6 +256,42 @@ class ConsumerTest extends IntegrationTestHarness with Logging { } } + def testExpandingTopicSubscriptions() { + val otherTopic = "other" + val subscriptions = Set(new TopicPartition(topic, 0), new TopicPartition(topic, 1)) + val expandedSubscriptions = subscriptions ++ Set(new TopicPartition(otherTopic, 0), new TopicPartition(otherTopic, 1)) + this.consumers(0).subscribe(topic) + TestUtils.waitUntilTrue(() => { + this.consumers(0).poll(50) + this.consumers(0).subscriptions == subscriptions.asJava + }, s"Expected partitions ${subscriptions.asJava} but actually got ${this.consumers(0).subscriptions}") + + TestUtils.createTopic(this.zkClient, otherTopic, 2, serverCount, this.servers) + this.consumers(0).subscribe(otherTopic) + TestUtils.waitUntilTrue(() => { + this.consumers(0).poll(50) + this.consumers(0).subscriptions == expandedSubscriptions.asJava + }, s"Expected partitions ${expandedSubscriptions.asJava} but actually got ${this.consumers(0).subscriptions}") + } + + def testShrinkingTopicSubscriptions() { + val otherTopic = "other" + TestUtils.createTopic(this.zkClient, otherTopic, 2, serverCount, this.servers) + val subscriptions = Set(new TopicPartition(topic, 0), new TopicPartition(topic, 1), new TopicPartition(otherTopic, 0), new TopicPartition(otherTopic, 1)) + val shrunkenSubscriptions = Set(new TopicPartition(topic, 0), new TopicPartition(topic, 1)) + this.consumers(0).subscribe(topic, otherTopic) + TestUtils.waitUntilTrue(() => { + this.consumers(0).poll(50) + this.consumers(0).subscriptions == subscriptions.asJava + }, s"Expected partitions ${subscriptions.asJava} but actually got ${this.consumers(0).subscriptions}") + + this.consumers(0).unsubscribe(otherTopic) + TestUtils.waitUntilTrue(() => { + this.consumers(0).poll(50) + this.consumers(0).subscriptions == shrunkenSubscriptions.asJava + }, s"Expected partitions ${shrunkenSubscriptions.asJava} but actually got ${this.consumers(0).subscriptions}") + } + def testPartitionPauseAndResume() { sendRecords(5) this.consumers(0).subscribe(tp) @@ -314,7 +350,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging { val maxIters = numRecords * 300 var iters = 0 while (records.size < numRecords) { - for (record <- consumer.poll(50)) + for (record <- consumer.poll(50).asScala) records.add(record) if(iters > maxIters) throw new IllegalStateException("Failed to consume the expected records after " + iters + " iterations.") -- 1.7.12.4 From 47c99f38726dd5bd3bf2c2f8a8a999d36b53ffcf Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Fri, 7 Aug 2015 16:07:45 -0700 Subject: [PATCH 096/120] Minor: Fixes to Selector's javadoc Author: Ismael Juma Closes #126 from ijuma/minor-selector-javadoc-fixes and squashes the following commits: a26f529 [Ismael Juma] Minor fixes to Selector's javadoc --- .../java/org/apache/kafka/common/network/Selector.java | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) 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 aaf60c9..ce20111 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 @@ -45,7 +45,7 @@ import org.slf4j.LoggerFactory; * A connection can be added to the nioSelector associated with an integer id by doing * *
    - * nioSelector.connect(42, new InetSocketAddress("google.com", server.port), 64000, 64000);
    + * nioSelector.connect("42", new InetSocketAddress("google.com", server.port), 64000, 64000);
      * 
    * * The connect call does not block on the creation of the TCP connection, so the connect method only begins initiating @@ -55,8 +55,9 @@ import org.slf4j.LoggerFactory; * connections are all done using the poll() call. * *
    - * List<NetworkRequest> requestsToSend = Arrays.asList(new NetworkRequest(0, myBytes), new NetworkRequest(1, myOtherBytes));
    - * nioSelector.poll(TIMEOUT_MS, requestsToSend);
    + * nioSelector.send(new NetworkSend(myDestination, myBytes));
    + * nioSelector.send(new NetworkSend(myOtherDestination, myOtherBytes));
    + * nioSelector.poll(TIMEOUT_MS);
      * 
    * * The nioSelector maintains several lists that are reset by each call to poll() which are available via @@ -123,7 +124,7 @@ public class Selector implements Selectable { * Begin connecting to the given address and add the connection to this nioSelector associated with the given id * number. *

    - * Note that this call only initiates the connection, which will be completed on a future {@link #poll(long, List)} + * Note that this call only initiates the connection, which will be completed on a future {@link #poll(long)} * call. Check {@link #connected()} to see which (if any) connections have completed after a given poll call. * @param id The id for the new connection * @param address The address to connect to @@ -171,7 +172,7 @@ public class Selector implements Selectable { /** * Disconnect any connections for the given id (if there are any). The disconnection is asynchronous and will not be - * processed until the next {@link #poll(long, List) poll()} call. + * processed until the next {@link #poll(long) poll()} call. */ @Override public void disconnect(String id) { @@ -228,8 +229,8 @@ public class Selector implements Selectable { * * When this call is completed the user can check for completed sends, receives, connections or disconnects using * {@link #completedSends()}, {@link #completedReceives()}, {@link #connected()}, {@link #disconnected()}. These - * lists will be cleared at the beginning of each {@link #poll(long, List)} call and repopulated by the call if any - * completed I/O. + * lists will be cleared at the beginning of each {@link #poll(long)} call and repopulated by the call if there is + * any completed I/O. * * @param timeout The amount of time to wait, in milliseconds. If negative, wait indefinitely. * @throws IllegalStateException If a send is given for which we have no existing connection or for which there is -- 1.7.12.4 From a36befd81ed97c18e3e676559617bffdbb191d65 Mon Sep 17 00:00:00 2001 From: Dave Cromberge Date: Fri, 7 Aug 2015 16:10:25 -0700 Subject: [PATCH 097/120] MINOR: documentation fix in StringDecoder Author: Dave Cromberge Closes #124 from davecromberge/documentation-fix and squashes the following commits: 7b9b1ce [Dave Cromberge] minor documentation fix to StringDecoder --- core/src/main/scala/kafka/serializer/Decoder.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/kafka/serializer/Decoder.scala b/core/src/main/scala/kafka/serializer/Decoder.scala index 54d0b93..164c3fa 100644 --- a/core/src/main/scala/kafka/serializer/Decoder.scala +++ b/core/src/main/scala/kafka/serializer/Decoder.scala @@ -5,7 +5,7 @@ * The ASF licenses this file to You under the Apache License, Version 2.0 * (the "License"); you may not use this file except in compliance with * the License. You may obtain a copy of the License at - * + * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software @@ -36,16 +36,16 @@ class DefaultDecoder(props: VerifiableProperties = null) extends Decoder[Array[B } /** - * The string encoder translates strings into bytes. It uses UTF8 by default but takes + * The string decoder translates bytes into strings. It uses UTF8 by default but takes * an optional property serializer.encoding to control this. */ class StringDecoder(props: VerifiableProperties = null) extends Decoder[String] { - val encoding = + val encoding = if(props == null) - "UTF8" + "UTF8" else props.getString("serializer.encoding", "UTF8") - + def fromBytes(bytes: Array[Byte]): String = { new String(bytes, encoding) } -- 1.7.12.4 From 68ad80f8524931222d78a8125cd012321d784337 Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Fri, 7 Aug 2015 16:16:34 -0700 Subject: [PATCH 098/120] KAFKA-2415; Fix transient failure in LogRecoveryTest Author: Jiangjie Qin Author: Jiangjie Qin Reviewers: Ismael Juma, Gwen Shapira Closes #121 from becketqin/KAFKA-2415 and squashes the following commits: 7a9f453 [Jiangjie Qin] Addressed Ismael's comment 346103c [Jiangjie Qin] KAFKA-2415: Fix transient failure in LogRecoveryTest --- core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala b/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala index 7688f26..21081ce 100755 --- a/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala @@ -210,6 +210,9 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { sendMessages(2) hw += 2 + // allow some time for the follower to create replica + TestUtils.waitUntilTrue(() => server1.replicaManager.getReplica(topic, 0).nonEmpty, + "Failed to create replica in follower after timeout") // allow some time for the follower to get the leader HW TestUtils.waitUntilTrue(() => server1.replicaManager.getReplica(topic, 0).get.highWatermark.messageOffset == hw, -- 1.7.12.4 From 1ed88f0eba860bce4be0269cdd4d38c2b57db7f7 Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Fri, 7 Aug 2015 23:42:42 -0700 Subject: [PATCH 099/120] KAFKA-1997: Follow-up patch, hardcode key/value serializer in mirror maker to byte serializer. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Hardcode the key/value serializer to ByteArraySerializer according to Jun’s comments. Author: Jiangjie Qin Reviewers: Guozhang Wang Closes #120 from becketqin/KAFKA-1997 and squashes the following commits: 7f2e5a6 [Jiangjie Qin] KAFKA-1997: Follow-up patch, hardcode key/value serializer in mirror maker to byte serializer. --- core/src/main/scala/kafka/tools/MirrorMaker.scala | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/tools/MirrorMaker.scala b/core/src/main/scala/kafka/tools/MirrorMaker.scala index 797b4bb..fbe0c83 100755 --- a/core/src/main/scala/kafka/tools/MirrorMaker.scala +++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala @@ -18,8 +18,8 @@ package kafka.tools import java.util -import java.util.concurrent.{TimeUnit, CountDownLatch} import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger} +import java.util.concurrent.{CountDownLatch, TimeUnit} import java.util.{Collections, Properties} import com.yammer.metrics.core.Gauge @@ -29,7 +29,7 @@ import kafka.javaapi.consumer.ConsumerRebalanceListener import kafka.message.MessageAndMetadata import kafka.metrics.KafkaMetricsGroup import kafka.serializer.DefaultDecoder -import kafka.utils.{CommandLineUtils, Logging, CoreUtils} +import kafka.utils.{CommandLineUtils, CoreUtils, Logging} import org.apache.kafka.clients.producer.internals.ErrorLoggingCallback import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord, RecordMetadata} import org.apache.kafka.common.utils.Utils @@ -185,6 +185,9 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { maybeSetDefaultProperty(producerProps, ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG, "true") maybeSetDefaultProperty(producerProps, ProducerConfig.ACKS_CONFIG, "all") maybeSetDefaultProperty(producerProps, ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, "1") + // Always set producer key and value serializer to ByteArraySerializer. + producerProps.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") + producerProps.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") producer = new MirrorMakerProducer(producerProps) // Create consumer connector -- 1.7.12.4 From 1fdb758f286868a00cbebcc5bbcfc8195529158e Mon Sep 17 00:00:00 2001 From: Manikumar Reddy Date: Mon, 10 Aug 2015 20:58:20 -0700 Subject: [PATCH 100/120] KAFKA-2202: fix consumerTimeoutMs computation on ConsumerPerformance; reviewed by Guozhang Wang --- core/src/main/scala/kafka/tools/ConsumerPerformance.scala | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/kafka/tools/ConsumerPerformance.scala b/core/src/main/scala/kafka/tools/ConsumerPerformance.scala index 903318d..7797dee 100644 --- a/core/src/main/scala/kafka/tools/ConsumerPerformance.scala +++ b/core/src/main/scala/kafka/tools/ConsumerPerformance.scala @@ -30,6 +30,7 @@ import kafka.consumer.ConsumerConnector import kafka.consumer.KafkaStream import kafka.consumer.ConsumerTimeoutException import java.text.SimpleDateFormat +import java.util.concurrent.atomic.AtomicBoolean /** * Performance test for the full zookeeper consumer @@ -43,6 +44,7 @@ object ConsumerPerformance { logger.info("Starting consumer...") val totalMessagesRead = new AtomicLong(0) val totalBytesRead = new AtomicLong(0) + val consumerTimeout = new AtomicBoolean(false) if (!config.hideHeader) { if (!config.showDetailedStats) @@ -67,7 +69,7 @@ object ConsumerPerformance { var threadList = List[ConsumerPerfThread]() for ((topic, streamList) <- topicMessageStreams) for (i <- 0 until streamList.length) - threadList ::= new ConsumerPerfThread(i, "kafka-zk-consumer-" + i, streamList(i), config, totalMessagesRead, totalBytesRead) + threadList ::= new ConsumerPerfThread(i, "kafka-zk-consumer-" + i, streamList(i), config, totalMessagesRead, totalBytesRead, consumerTimeout) logger.info("Sleeping for 1 second.") Thread.sleep(1000) @@ -77,7 +79,10 @@ object ConsumerPerformance { thread.start for (thread <- threadList) thread.join - endMs = System.currentTimeMillis - consumerConfig.consumerTimeoutMs + if(consumerTimeout.get()) + endMs = System.currentTimeMillis - consumerConfig.consumerTimeoutMs + else + endMs = System.currentTimeMillis consumerConnector.shutdown() } val elapsedSecs = (endMs - startMs) / 1000.0 @@ -209,7 +214,7 @@ object ConsumerPerformance { } class ConsumerPerfThread(threadId: Int, name: String, stream: KafkaStream[Array[Byte], Array[Byte]], - config: ConsumerPerfConfig, totalMessagesRead: AtomicLong, totalBytesRead: AtomicLong) + config: ConsumerPerfConfig, totalMessagesRead: AtomicLong, totalBytesRead: AtomicLong, consumerTimeout: AtomicBoolean) extends Thread(name) { override def run() { @@ -238,7 +243,9 @@ object ConsumerPerformance { } catch { case _: InterruptedException => case _: ClosedByInterruptException => - case _: ConsumerTimeoutException => + case _: ConsumerTimeoutException => { + consumerTimeout.set(true); + } case e: Throwable => e.printStackTrace() } totalMessagesRead.addAndGet(messagesRead) -- 1.7.12.4 From 96534a7d502be58026feec3c2012f022bf330049 Mon Sep 17 00:00:00 2001 From: Grant Henke Date: Tue, 11 Aug 2015 15:07:40 -0700 Subject: [PATCH 101/120] KAFKA-2336: Changing offsets.topic.num.partitions after the offset topic is created breaks consumer group partition assignment; Reviewed by Jiangjie Qin, Gwen Shapira --- core/src/main/scala/kafka/server/OffsetManager.scala | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala index 47b6ce9..0e613e7 100755 --- a/core/src/main/scala/kafka/server/OffsetManager.scala +++ b/core/src/main/scala/kafka/server/OffsetManager.scala @@ -96,6 +96,7 @@ class OffsetManager(val config: OffsetManagerConfig, private val loadingPartitions: mutable.Set[Int] = mutable.Set() private val cleanupOrLoadMutex = new Object private val shuttingDown = new AtomicBoolean(false) + private val offsetsTopicPartitionCount = getOffsetsTopicPartitionCount this.logIdent = "[Offset Manager on Broker " + replicaManager.config.brokerId + "]: " @@ -170,7 +171,7 @@ class OffsetManager(val config: OffsetManagerConfig, } - def partitionFor(group: String): Int = Utils.abs(group.hashCode) % config.offsetsTopicNumPartitions + def partitionFor(group: String): Int = Utils.abs(group.hashCode) % offsetsTopicPartitionCount /** * Fetch the current offset for the given group/topic/partition from the underlying offsets storage. @@ -436,13 +437,24 @@ class OffsetManager(val config: OffsetManagerConfig, if (numRemoved > 0) info("Removed %d cached offsets for %s on follower transition." .format(numRemoved, TopicAndPartition(ConsumerCoordinator.OffsetsTopicName, offsetsPartition))) - } def shutdown() { shuttingDown.set(true) } + /** + * Gets the partition count of the offsets topic from ZooKeeper. + * If the topic does not exist, the configured partition count is returned. + */ + private def getOffsetsTopicPartitionCount = { + val topic = ConsumerCoordinator.OffsetsTopicName + val topicData = ZkUtils.getPartitionAssignmentForTopics(zkClient, Seq(topic)) + if (topicData(topic).nonEmpty) + topicData(topic).size + else + config.offsetsTopicNumPartitions + } } object OffsetManager { -- 1.7.12.4 From be633a713e311c90ebfafc650eb3dcfb94ce372d Mon Sep 17 00:00:00 2001 From: Geoff Anderson Date: Tue, 11 Aug 2015 15:24:52 -0700 Subject: [PATCH 102/120] KAFKA-2408: ConsoleConsumerService direct log output to file console consumer writes to System.out, while (some) log4j loggers operate in other threads. This occasionally led to funky interleaved output which disrupted parsing of consumed messages by ConsoleConsumerService, leading to spurious test failures. This fix directs log output to a separate file. Author: Geoff Anderson Reviewers: Ewen Cheslack-Postava Closes #123 from granders/KAFKA-2408 and squashes the following commits: 247b0e0 [Geoff Anderson] Updated line counting to use wc -l 66d6f4f [Geoff Anderson] lower -> uperrcase constants e67f554 [Geoff Anderson] Changed incorrect license header af67e01 [Geoff Anderson] Merged in upstream trunk 8f89044 [Geoff Anderson] Added another lifecycle check. Wait for log file to exist before exmaning contents. 521a84b [Geoff Anderson] Updated console consumer to directo log output directly to file rather than stdout --- tests/kafkatest/sanity_checks/__init__.py | 14 ++++ .../sanity_checks/test_console_consumer.py | 80 ++++++++++++++++++++++ tests/kafkatest/services/console_consumer.py | 61 ++++++++++++++--- .../services/templates/console_consumer.properties | 2 +- .../templates/console_consumer_log4j.properties | 26 +++++++ 5 files changed, 171 insertions(+), 12 deletions(-) create mode 100644 tests/kafkatest/sanity_checks/__init__.py create mode 100644 tests/kafkatest/sanity_checks/test_console_consumer.py create mode 100644 tests/kafkatest/services/templates/console_consumer_log4j.properties diff --git a/tests/kafkatest/sanity_checks/__init__.py b/tests/kafkatest/sanity_checks/__init__.py new file mode 100644 index 0000000..91eacc9 --- /dev/null +++ b/tests/kafkatest/sanity_checks/__init__.py @@ -0,0 +1,14 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. \ No newline at end of file diff --git a/tests/kafkatest/sanity_checks/test_console_consumer.py b/tests/kafkatest/sanity_checks/test_console_consumer.py new file mode 100644 index 0000000..cd8c8f9 --- /dev/null +++ b/tests/kafkatest/sanity_checks/test_console_consumer.py @@ -0,0 +1,80 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from ducktape.tests.test import Test +from ducktape.utils.util import wait_until + +from kafkatest.services.zookeeper import ZookeeperService +from kafkatest.services.kafka import KafkaService +from kafkatest.services.console_consumer import ConsoleConsumer + +import time + + +def file_exists(node, file): + """Quick and dirty check for existence of remote file.""" + try: + node.account.ssh("cat " + file, allow_fail=False) + return True + except: + return False + + +def line_count(node, file): + """Return the line count of file on node""" + out = [line for line in node.account.ssh_capture("wc -l %s" % file)] + if len(out) != 1: + raise Exception("Expected single line of output from wc -l") + + return int(out[0].strip().split(" ")[0]) + + +class ConsoleConsumerTest(Test): + """Sanity checks on console consumer service class.""" + def __init__(self, test_context): + super(ConsoleConsumerTest, self).__init__(test_context) + + self.topic = "topic" + self.zk = ZookeeperService(test_context, num_nodes=1) + self.kafka = KafkaService(test_context, num_nodes=1, zk=self.zk, + topics={self.topic: {"partitions": 1, "replication-factor": 1}}) + self.consumer = ConsoleConsumer(test_context, num_nodes=1, kafka=self.kafka, topic=self.topic) + + def setUp(self): + self.zk.start() + self.kafka.start() + + def test_lifecycle(self): + t0 = time.time() + self.consumer.start() + node = self.consumer.nodes[0] + + if not wait_until(lambda: self.consumer.alive(node), timeout_sec=10, backoff_sec=.2): + raise Exception("Consumer was too slow to start") + self.logger.info("consumer started in %s seconds " % str(time.time() - t0)) + + # Verify that log output is happening + if not wait_until(lambda: file_exists(node, ConsoleConsumer.LOG_FILE), timeout_sec=10): + raise Exception("Timed out waiting for log file to exist") + assert line_count(node, ConsoleConsumer.LOG_FILE) > 0 + + # Verify no consumed messages + assert line_count(node, ConsoleConsumer.STDOUT_CAPTURE) == 0 + + self.consumer.stop_node(node) + if not wait_until(lambda: not self.consumer.alive(node), timeout_sec=10, backoff_sec=.2): + raise Exception("Took too long for consumer to die.") + + diff --git a/tests/kafkatest/services/console_consumer.py b/tests/kafkatest/services/console_consumer.py index 33ef4ea..18c9f63 100644 --- a/tests/kafkatest/services/console_consumer.py +++ b/tests/kafkatest/services/console_consumer.py @@ -15,6 +15,8 @@ from ducktape.services.background_thread import BackgroundThreadService +import os + def is_int(msg): """Default method used to check whether text pulled from console consumer is a message. @@ -26,7 +28,6 @@ def is_int(msg): except: return None - """ 0.8.2.1 ConsoleConsumer options @@ -69,9 +70,24 @@ Option Description class ConsoleConsumer(BackgroundThreadService): + # Root directory for persistent output + PERSISTENT_ROOT = "/mnt/console_consumer" + STDOUT_CAPTURE = os.path.join(PERSISTENT_ROOT, "console_consumer.stdout") + STDERR_CAPTURE = os.path.join(PERSISTENT_ROOT, "console_consumer.stderr") + LOG_DIR = os.path.join(PERSISTENT_ROOT, "logs") + LOG_FILE = os.path.join(LOG_DIR, "console_consumer.log") + LOG4J_CONFIG = os.path.join(PERSISTENT_ROOT, "tools-log4j.properties") + CONFIG_FILE = os.path.join(PERSISTENT_ROOT, "console_consumer.properties") + logs = { + "consumer_stdout": { + "path": STDOUT_CAPTURE, + "collect_default": False}, + "consumer_stderr": { + "path": STDERR_CAPTURE, + "collect_default": False}, "consumer_log": { - "path": "/mnt/consumer.log", + "path": LOG_FILE, "collect_default": True} } @@ -104,18 +120,37 @@ class ConsoleConsumer(BackgroundThreadService): @property def start_cmd(self): args = self.args.copy() - args.update({'zk_connect': self.kafka.zk.connect_setting()}) - cmd = "/opt/kafka/bin/kafka-console-consumer.sh --topic %(topic)s --zookeeper %(zk_connect)s" \ - " --consumer.config /mnt/console_consumer.properties" % args + args['zk_connect'] = self.kafka.zk.connect_setting() + args['stdout'] = ConsoleConsumer.STDOUT_CAPTURE + args['stderr'] = ConsoleConsumer.STDERR_CAPTURE + args['config_file'] = ConsoleConsumer.CONFIG_FILE + + cmd = "export LOG_DIR=%s;" % ConsoleConsumer.LOG_DIR + cmd += " export KAFKA_LOG4J_OPTS=\"-Dlog4j.configuration=file:%s\";" % ConsoleConsumer.LOG4J_CONFIG + cmd += " /opt/kafka/bin/kafka-console-consumer.sh --topic %(topic)s --zookeeper %(zk_connect)s" \ + " --consumer.config %(config_file)s" % args if self.from_beginning: cmd += " --from-beginning" - cmd += " 2>> /mnt/consumer.log | tee -a /mnt/consumer.log &" + cmd += " 2>> %(stderr)s | tee -a %(stdout)s &" % args return cmd + def pids(self, node): + try: + cmd = "ps ax | grep -i console_consumer | grep java | grep -v grep | awk '{print $1}'" + pid_arr = [pid for pid in node.account.ssh_capture(cmd, allow_fail=True, callback=int)] + return pid_arr + except: + return [] + + def alive(self, node): + return len(self.pids(node)) > 0 + def _worker(self, idx, node): - # form config file + node.account.ssh("mkdir -p %s" % ConsoleConsumer.PERSISTENT_ROOT, allow_fail=False) + + # Create and upload config file if self.consumer_timeout_ms is not None: prop_file = self.render('console_consumer.properties', consumer_timeout_ms=self.consumer_timeout_ms) else: @@ -123,12 +158,16 @@ class ConsoleConsumer(BackgroundThreadService): self.logger.info("console_consumer.properties:") self.logger.info(prop_file) - node.account.create_file("/mnt/console_consumer.properties", prop_file) + node.account.create_file(ConsoleConsumer.CONFIG_FILE, prop_file) + + # Create and upload log properties + log_config = self.render('console_consumer_log4j.properties', log_file=ConsoleConsumer.LOG_FILE) + node.account.create_file(ConsoleConsumer.LOG4J_CONFIG, log_config) # Run and capture output cmd = self.start_cmd self.logger.debug("Console consumer %d command: %s", idx, cmd) - for line in node.account.ssh_capture(cmd): + for line in node.account.ssh_capture(cmd, allow_fail=False): msg = line.strip() msg = self.message_validator(msg) if msg is not None: @@ -139,8 +178,8 @@ class ConsoleConsumer(BackgroundThreadService): super(ConsoleConsumer, self).start_node(node) def stop_node(self, node): - node.account.kill_process("java", allow_fail=False) + node.account.kill_process("java", allow_fail=True) def clean_node(self, node): - node.account.ssh("rm -rf /mnt/console_consumer.properties /mnt/consumer.log", allow_fail=False) + node.account.ssh("rm -rf %s" % ConsoleConsumer.PERSISTENT_ROOT, allow_fail=False) diff --git a/tests/kafkatest/services/templates/console_consumer.properties b/tests/kafkatest/services/templates/console_consumer.properties index 63782fc..944c2c9 100644 --- a/tests/kafkatest/services/templates/console_consumer.properties +++ b/tests/kafkatest/services/templates/console_consumer.properties @@ -14,6 +14,6 @@ # limitations under the License. # see kafka.server.KafkaConfig for additional details and defaults -{% if consumer_timeout_ms is defined %} +{% if consumer_timeout_ms is not none %} consumer.timeout.ms={{ consumer_timeout_ms }} {% endif %} \ No newline at end of file diff --git a/tests/kafkatest/services/templates/console_consumer_log4j.properties b/tests/kafkatest/services/templates/console_consumer_log4j.properties new file mode 100644 index 0000000..e63e6d6 --- /dev/null +++ b/tests/kafkatest/services/templates/console_consumer_log4j.properties @@ -0,0 +1,26 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Define the root logger with appender file +log4j.rootLogger = INFO, FILE + +log4j.appender.FILE=org.apache.log4j.FileAppender +log4j.appender.FILE.File={{ log_file }} +log4j.appender.FILE.ImmediateFlush=true +log4j.appender.FILE.Threshold=debug +# Set the append to false, overwrite +log4j.appender.FILE.Append=false +log4j.appender.FILE.layout=org.apache.log4j.PatternLayout +log4j.appender.FILE.layout.conversionPattern=[%d] %p %m (%c)%n \ No newline at end of file -- 1.7.12.4 From a62d63007c89c0c6f7ad62fe4643f7adc7fbc661 Mon Sep 17 00:00:00 2001 From: flavio junqueira Date: Wed, 12 Aug 2015 14:31:39 -0700 Subject: [PATCH 103/120] KAFKA-2300: Error in controller log when broker tries to rejoin cluster Author: flavio junqueira Reviewers: Ismael Juma, Guozhang Wang Closes #102 from fpj/2300 and squashes the following commits: 7bd2edb [flavio junqueira] KAFKA-2300: Removed unnecessary s" occurrences. aa6ec90 [flavio junqueira] KAFKA-2300: Wrapped all occurences of sendRequestToBrokers with try/catch and fixed string typo. f1261b1 [flavio junqueira] Fixed some style issues. 9b6390a [flavio junqueira] Updated package name and removed unnecessary imports. dbd1bf3 [flavio junqueira] KAFKA-2300: Error in controller log when broker tries to rejoin cluster --- .../controller/ControllerChannelManager.scala | 102 ++++++----- .../scala/kafka/controller/KafkaController.scala | 52 ++++-- .../kafka/controller/ControllerFailoverTest.scala | 187 +++++++++++++++++++++ 3 files changed, 288 insertions(+), 53 deletions(-) create mode 100644 core/src/test/scala/unit/kafka/controller/ControllerFailoverTest.scala diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index 9f521fa..4396b6e 100755 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -30,7 +30,7 @@ import kafka.api.RequestOrResponse import collection.Set class ControllerChannelManager (private val controllerContext: ControllerContext, config: KafkaConfig) extends Logging { - private val brokerStateInfo = new HashMap[Int, ControllerBrokerStateInfo] + protected val brokerStateInfo = new HashMap[Int, ControllerBrokerStateInfo] private val brokerLock = new Object this.logIdent = "[Channel manager on controller " + config.brokerId + "]: " @@ -100,7 +100,7 @@ class ControllerChannelManager (private val controllerContext: ControllerContext } } - private def startRequestSendThread(brokerId: Int) { + protected def startRequestSendThread(brokerId: Int) { val requestThread = brokerStateInfo(brokerId).requestSendThread if(requestThread.getState == Thread.State.NEW) requestThread.start() @@ -280,49 +280,67 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging } def sendRequestsToBrokers(controllerEpoch: Int, correlationId: Int) { - leaderAndIsrRequestMap.foreach { m => - val broker = m._1 - val partitionStateInfos = m._2.toMap - val leaderIds = partitionStateInfos.map(_._2.leaderIsrAndControllerEpoch.leaderAndIsr.leader).toSet - val leaders = controllerContext.liveOrShuttingDownBrokers.filter(b => leaderIds.contains(b.id)).map(b => b.getBrokerEndPoint(controller.config.interBrokerSecurityProtocol)) - val leaderAndIsrRequest = new LeaderAndIsrRequest(partitionStateInfos, leaders, controllerId, controllerEpoch, correlationId, clientId) - for (p <- partitionStateInfos) { - val typeOfRequest = if (broker == p._2.leaderIsrAndControllerEpoch.leaderAndIsr.leader) "become-leader" else "become-follower" - stateChangeLogger.trace(("Controller %d epoch %d sending %s LeaderAndIsr request %s with correlationId %d to broker %d " + - "for partition [%s,%d]").format(controllerId, controllerEpoch, typeOfRequest, - p._2.leaderIsrAndControllerEpoch, correlationId, broker, - p._1._1, p._1._2)) + try { + leaderAndIsrRequestMap.foreach { m => + val broker = m._1 + val partitionStateInfos = m._2.toMap + val leaderIds = partitionStateInfos.map(_._2.leaderIsrAndControllerEpoch.leaderAndIsr.leader).toSet + val leaders = controllerContext.liveOrShuttingDownBrokers.filter(b => leaderIds.contains(b.id)).map(b => b.getBrokerEndPoint(controller.config.interBrokerSecurityProtocol)) + val leaderAndIsrRequest = new LeaderAndIsrRequest(partitionStateInfos, leaders, controllerId, controllerEpoch, correlationId, clientId) + for (p <- partitionStateInfos) { + val typeOfRequest = if (broker == p._2.leaderIsrAndControllerEpoch.leaderAndIsr.leader) "become-leader" else "become-follower" + stateChangeLogger.trace(("Controller %d epoch %d sending %s LeaderAndIsr request %s with correlationId %d to broker %d " + + "for partition [%s,%d]").format(controllerId, controllerEpoch, typeOfRequest, + p._2.leaderIsrAndControllerEpoch, correlationId, broker, + p._1._1, p._1._2)) + } + controller.sendRequest(broker, leaderAndIsrRequest, null) } - controller.sendRequest(broker, leaderAndIsrRequest, null) - } - leaderAndIsrRequestMap.clear() - updateMetadataRequestMap.foreach { m => - val broker = m._1 - val partitionStateInfos = m._2.toMap - - val versionId = if (controller.config.interBrokerProtocolVersion.onOrAfter(KAFKA_083)) 1 else 0 - val updateMetadataRequest = new UpdateMetadataRequest(versionId = versionId.toShort, controllerId = controllerId, controllerEpoch = controllerEpoch, - correlationId = correlationId, clientId = clientId, partitionStateInfos = partitionStateInfos, aliveBrokers = controllerContext.liveOrShuttingDownBrokers) - partitionStateInfos.foreach(p => stateChangeLogger.trace(("Controller %d epoch %d sending UpdateMetadata request %s with " + - "correlationId %d to broker %d for partition %s").format(controllerId, controllerEpoch, p._2.leaderIsrAndControllerEpoch, - correlationId, broker, p._1))) - controller.sendRequest(broker, updateMetadataRequest, null) - } - updateMetadataRequestMap.clear() - stopReplicaRequestMap foreach { case(broker, replicaInfoList) => - val stopReplicaWithDelete = replicaInfoList.filter(_.deletePartition).map(_.replica).toSet - val stopReplicaWithoutDelete = replicaInfoList.filterNot(_.deletePartition).map(_.replica).toSet - debug("The stop replica request (delete = true) sent to broker %d is %s" - .format(broker, stopReplicaWithDelete.mkString(","))) - debug("The stop replica request (delete = false) sent to broker %d is %s" - .format(broker, stopReplicaWithoutDelete.mkString(","))) - replicaInfoList.foreach { r => - val stopReplicaRequest = new StopReplicaRequest(r.deletePartition, - Set(TopicAndPartition(r.replica.topic, r.replica.partition)), controllerId, controllerEpoch, correlationId) - controller.sendRequest(broker, stopReplicaRequest, r.callback) + leaderAndIsrRequestMap.clear() + updateMetadataRequestMap.foreach { m => + val broker = m._1 + val partitionStateInfos = m._2.toMap + + val versionId = if (controller.config.interBrokerProtocolVersion.onOrAfter(KAFKA_083)) 1 else 0 + val updateMetadataRequest = new UpdateMetadataRequest(versionId = versionId.toShort, controllerId = controllerId, controllerEpoch = controllerEpoch, + correlationId = correlationId, clientId = clientId, partitionStateInfos = partitionStateInfos, aliveBrokers = controllerContext.liveOrShuttingDownBrokers) + partitionStateInfos.foreach(p => stateChangeLogger.trace(("Controller %d epoch %d sending UpdateMetadata request %s with " + + "correlationId %d to broker %d for partition %s").format(controllerId, controllerEpoch, p._2.leaderIsrAndControllerEpoch, + correlationId, broker, p._1))) + controller.sendRequest(broker, updateMetadataRequest, null) + } + updateMetadataRequestMap.clear() + stopReplicaRequestMap foreach { case(broker, replicaInfoList) => + val stopReplicaWithDelete = replicaInfoList.filter(_.deletePartition).map(_.replica).toSet + val stopReplicaWithoutDelete = replicaInfoList.filterNot(_.deletePartition).map(_.replica).toSet + debug("The stop replica request (delete = true) sent to broker %d is %s" + .format(broker, stopReplicaWithDelete.mkString(","))) + debug("The stop replica request (delete = false) sent to broker %d is %s" + .format(broker, stopReplicaWithoutDelete.mkString(","))) + replicaInfoList.foreach { r => + val stopReplicaRequest = new StopReplicaRequest(r.deletePartition, + Set(TopicAndPartition(r.replica.topic, r.replica.partition)), controllerId, controllerEpoch, correlationId) + controller.sendRequest(broker, stopReplicaRequest, r.callback) + } + } + stopReplicaRequestMap.clear() + } catch { + case e : Throwable => { + if(leaderAndIsrRequestMap.size > 0) { + error("Haven't been able to send leader and isr requests, current state of " + + s"the map is $leaderAndIsrRequestMap") + } + if(updateMetadataRequestMap.size > 0) { + error("Haven't been able to send metadata update requests, current state of " + + s"the map is $updateMetadataRequestMap") + } + if(stopReplicaRequestMap.size > 0) { + error("Haven't been able to send stop replica requests, current state of " + + s"the map is $stopReplicaRequestMap") + } + throw new IllegalStateException(e) } } - stopReplicaRequestMap.clear() } } diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index b19e57f..68536f5 100755 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -263,11 +263,20 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt } else { // Stop the replica first. The state change below initiates ZK changes which should take some time // before which the stop replica request should be completed (in most cases) - brokerRequestBatch.newBatch() - brokerRequestBatch.addStopReplicaRequestForBrokers(Seq(id), topicAndPartition.topic, - topicAndPartition.partition, deletePartition = false) - brokerRequestBatch.sendRequestsToBrokers(epoch, controllerContext.correlationId.getAndIncrement) - + try { + brokerRequestBatch.newBatch() + brokerRequestBatch.addStopReplicaRequestForBrokers(Seq(id), topicAndPartition.topic, + topicAndPartition.partition, deletePartition = false) + brokerRequestBatch.sendRequestsToBrokers(epoch, controllerContext.correlationId.getAndIncrement) + } catch { + case e : IllegalStateException => { + // Resign if the controller is in an illegal state + error("Forcing the controller to resign") + controllerElector.resign() + + throw e + } + } // If the broker is a follower, updates the isr in ZK and notifies the current leader replicaStateMachine.handleStateChanges(Set(PartitionAndReplica(topicAndPartition.topic, topicAndPartition.partition, id)), OfflineReplica) @@ -341,6 +350,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt * required to clean up internal controller data structures */ def onControllerResignation() { + debug("Controller resigning, broker id %d".format(config.brokerId)) // de-register listeners deregisterIsrChangeNotificationListener() deregisterReassignedPartitionsListener() @@ -888,9 +898,19 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt brokerRequestBatch.newBatch() updateLeaderEpoch(topicAndPartition.topic, topicAndPartition.partition) match { case Some(updatedLeaderIsrAndControllerEpoch) => - brokerRequestBatch.addLeaderAndIsrRequestForBrokers(replicasToReceiveRequest, topicAndPartition.topic, - topicAndPartition.partition, updatedLeaderIsrAndControllerEpoch, newAssignedReplicas) - brokerRequestBatch.sendRequestsToBrokers(controllerContext.epoch, controllerContext.correlationId.getAndIncrement) + try { + brokerRequestBatch.addLeaderAndIsrRequestForBrokers(replicasToReceiveRequest, topicAndPartition.topic, + topicAndPartition.partition, updatedLeaderIsrAndControllerEpoch, newAssignedReplicas) + brokerRequestBatch.sendRequestsToBrokers(controllerContext.epoch, controllerContext.correlationId.getAndIncrement) + } catch { + case e : IllegalStateException => { + // Resign if the controller is in an illegal state + error("Forcing the controller to resign") + controllerElector.resign() + + throw e + } + } stateChangeLogger.trace(("Controller %d epoch %d sent LeaderAndIsr request %s with new assigned replica list %s " + "to leader %d for partition being reassigned %s").format(config.brokerId, controllerContext.epoch, updatedLeaderIsrAndControllerEpoch, newAssignedReplicas.mkString(","), updatedLeaderIsrAndControllerEpoch.leaderAndIsr.leader, topicAndPartition)) @@ -998,9 +1018,19 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt * @param brokers The brokers that the update metadata request should be sent to */ def sendUpdateMetadataRequest(brokers: Seq[Int], partitions: Set[TopicAndPartition] = Set.empty[TopicAndPartition]) { - brokerRequestBatch.newBatch() - brokerRequestBatch.addUpdateMetadataRequestForBrokers(brokers, partitions) - brokerRequestBatch.sendRequestsToBrokers(epoch, controllerContext.correlationId.getAndIncrement) + try { + brokerRequestBatch.newBatch() + brokerRequestBatch.addUpdateMetadataRequestForBrokers(brokers, partitions) + brokerRequestBatch.sendRequestsToBrokers(epoch, controllerContext.correlationId.getAndIncrement) + } catch { + case e : IllegalStateException => { + // Resign if the controller is in an illegal state + error("Forcing the controller to resign") + controllerElector.resign() + + throw e + } + } } /** diff --git a/core/src/test/scala/unit/kafka/controller/ControllerFailoverTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerFailoverTest.scala new file mode 100644 index 0000000..206a7c3 --- /dev/null +++ b/core/src/test/scala/unit/kafka/controller/ControllerFailoverTest.scala @@ -0,0 +1,187 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.controller + +import java.util.concurrent.LinkedBlockingQueue +import java.util.Properties + +import junit.framework.Assert._ +import org.scalatest.junit.JUnit3Suite + +import org.junit.{Test, After, Before} +import org.I0Itec.zkclient.{IZkDataListener, IZkStateListener, ZkClient} +import org.I0Itec.zkclient.serialize.ZkSerializer +import org.apache.log4j.{Logger, Level} + +import kafka.api.RequestOrResponse +import kafka.common.TopicAndPartition +import kafka.integration.KafkaServerTestHarness +import kafka.server.BrokerState +import kafka.server.KafkaConfig +import kafka.server.KafkaServer +import kafka.server.RunningAsController +import kafka.utils._ +import kafka.utils.TestUtils._ + +import scala.collection.Map +import scala.collection.mutable + + +class ControllerFailoverTest extends KafkaServerTestHarness with Logging { + val log = Logger.getLogger(classOf[ControllerFailoverTest]) + val numNodes = 2 + val numParts = 1 + val msgQueueSize = 1 + val topic = "topic1" + val overridingProps = new Properties() + overridingProps.put(KafkaConfig.NumPartitionsProp, numParts.toString) + + override def generateConfigs() = TestUtils.createBrokerConfigs(numNodes, zkConnect) + .map(KafkaConfig.fromProps(_, overridingProps)) + + override def setUp() { + super.setUp() + } + + override def tearDown() { + super.tearDown() + } + + /** + * See @link{https://issues.apache.org/jira/browse/KAFKA-2300} + * for the background of this test case + */ + def testMetadataUpdate() { + log.setLevel(Level.INFO) + var controller: KafkaServer = this.servers.head; + // Find the current controller + val epochMap: mutable.Map[Int, Int] = mutable.Map.empty + for (server <- this.servers) { + epochMap += (server.config.brokerId -> server.kafkaController.epoch) + if(server.kafkaController.isActive()) { + controller = server + } + } + // Create topic with one partition + kafka.admin.AdminUtils.createTopic(controller.zkClient, topic, 1, 1) + val topicPartition = TopicAndPartition("topic1", 0) + var partitions = controller.kafkaController.partitionStateMachine.partitionsInState(OnlinePartition) + while (!partitions.contains(topicPartition)) { + partitions = controller.kafkaController.partitionStateMachine.partitionsInState(OnlinePartition) + Thread.sleep(100) + } + // Replace channel manager with our mock manager + controller.kafkaController.controllerContext.controllerChannelManager.shutdown() + val channelManager = new MockChannelManager(controller.kafkaController.controllerContext, + controller.kafkaController.config) + channelManager.startup() + controller.kafkaController.controllerContext.controllerChannelManager = channelManager + channelManager.shrinkBlockingQueue(0) + channelManager.stopSendThread(0) + // Spawn a new thread to block on the outgoing channel + // queue + val thread = new Thread(new Runnable { + def run() { + try { + controller.kafkaController.sendUpdateMetadataRequest(Seq(0), Set(topicPartition)) + log.info("Queue state %d %d".format(channelManager.queueCapacity(0), channelManager.queueSize(0))) + controller.kafkaController.sendUpdateMetadataRequest(Seq(0), Set(topicPartition)) + log.info("Queue state %d %d".format(channelManager.queueCapacity(0), channelManager.queueSize(0))) + } catch { + case e : Exception => { + log.info("Thread interrupted") + } + } + } + }) + thread.setName("mythread") + thread.start() + while (thread.getState() != Thread.State.WAITING) { + Thread.sleep(100) + } + // Assume that the thread is WAITING because it is + // blocked on the queue, so interrupt and move forward + thread.interrupt() + thread.join() + channelManager.resumeSendThread(0) + // Wait and find current controller + var found = false + var counter = 0 + while (!found && counter < 10) { + for (server <- this.servers) { + val previousEpoch = (epochMap get server.config.brokerId) match { + case Some(epoch) => + epoch + case None => + val msg = String.format("Missing element in epoch map %s", epochMap.mkString(", ")) + throw new IllegalStateException(msg) + } + + if (server.kafkaController.isActive + && (previousEpoch) < server.kafkaController.epoch) { + controller = server + found = true + } + } + if (!found) { + Thread.sleep(100) + counter += 1 + } + } + // Give it a shot to make sure that sending isn't blocking + try { + controller.kafkaController.sendUpdateMetadataRequest(Seq(0), Set(topicPartition)) + } catch { + case e : Throwable => { + fail(e) + } + } + } +} + +class MockChannelManager(private val controllerContext: ControllerContext, + config: KafkaConfig) + extends ControllerChannelManager(controllerContext, config) { + def stopSendThread(brokerId: Int) { + val requestThread = brokerStateInfo(brokerId).requestSendThread + requestThread.isRunning.set(false) + requestThread.interrupt + requestThread.join + } + + def shrinkBlockingQueue(brokerId: Int) { + val messageQueue = new LinkedBlockingQueue[(RequestOrResponse, RequestOrResponse => Unit)](1) + val brokerInfo = this.brokerStateInfo(brokerId) + this.brokerStateInfo.put(brokerId, new ControllerBrokerStateInfo(brokerInfo.channel, + brokerInfo.broker, + messageQueue, + brokerInfo.requestSendThread)) + } + + def resumeSendThread (brokerId: Int) { + this.startRequestSendThread(0) + } + + def queueCapacity(brokerId: Int): Int = { + this.brokerStateInfo(brokerId).messageQueue.remainingCapacity + } + + def queueSize(brokerId: Int): Int = { + this.brokerStateInfo(brokerId).messageQueue.size + } +} \ No newline at end of file -- 1.7.12.4 From 3902dc024595baf715cdfc8fa4b66aacbeca72d1 Mon Sep 17 00:00:00 2001 From: Grant Henke Date: Wed, 12 Aug 2015 14:34:30 -0700 Subject: [PATCH 104/120] MINOR: Fix hard coded strings in ProduceResponse Author: Grant Henke Reviewers: Ismael Juma, Ewen Cheslack-Postava and Guozhang Wang Closes #131 from granthenke/minor-string and squashes the following commits: 3c6250d [Grant Henke] MINOR: Fix hard coded strings in ProduceResponse --- .../org/apache/kafka/common/requests/ProduceResponse.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java index 37ec0b7..febfc70 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java @@ -76,14 +76,14 @@ public class ProduceResponse extends AbstractRequestResponse { public ProduceResponse(Struct struct) { super(struct); responses = new HashMap(); - for (Object topicResponse : struct.getArray("responses")) { + for (Object topicResponse : struct.getArray(RESPONSES_KEY_NAME)) { Struct topicRespStruct = (Struct) topicResponse; - String topic = topicRespStruct.getString("topic"); - for (Object partResponse : topicRespStruct.getArray("partition_responses")) { + String topic = topicRespStruct.getString(TOPIC_KEY_NAME); + for (Object partResponse : topicRespStruct.getArray(PARTITION_RESPONSES_KEY_NAME)) { Struct partRespStruct = (Struct) partResponse; - int partition = partRespStruct.getInt("partition"); - short errorCode = partRespStruct.getShort("error_code"); - long offset = partRespStruct.getLong("base_offset"); + int partition = partRespStruct.getInt(PARTITION_KEY_NAME); + short errorCode = partRespStruct.getShort(ERROR_CODE_KEY_NAME); + long offset = partRespStruct.getLong(BASE_OFFSET_KEY_NAME); TopicPartition tp = new TopicPartition(topic, partition); responses.put(tp, new PartitionResponse(errorCode, offset)); } -- 1.7.12.4 From 04b0d870b263117e301584bfc00dd8e81486617a Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Wed, 12 Aug 2015 14:57:42 -0700 Subject: [PATCH 105/120] KAFKA-2429: Add annotations to mark classes as stable/unstable This also marks the consumer as unstable to show an example of using these annotations. Author: Ewen Cheslack-Postava Reviewers: Gwen Shapira Closes #133 from ewencp/stability-annotations and squashes the following commits: 09c15c3 [Ewen Cheslack-Postava] KAFKA-2429: Add annotations to mark classes as stable/unstable --- .../apache/kafka/clients/consumer/Consumer.java | 2 + .../kafka/clients/consumer/KafkaConsumer.java | 2 + .../common/annotation/InterfaceStability.java | 48 ++++++++++++++++++++++ 3 files changed, 52 insertions(+) create mode 100644 clients/src/main/java/org/apache/kafka/common/annotation/InterfaceStability.java diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java index 158e1ea..76834ad 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java @@ -21,11 +21,13 @@ import org.apache.kafka.common.Metric; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.annotation.InterfaceStability; /** * @see KafkaConsumer * @see MockConsumer */ +@InterfaceStability.Unstable public interface Consumer extends Closeable { /** 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 ed99e9b..be46b6c 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 @@ -26,6 +26,7 @@ import org.apache.kafka.common.Metric; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.common.metrics.JmxReporter; import org.apache.kafka.common.metrics.MetricConfig; import org.apache.kafka.common.metrics.Metrics; @@ -390,6 +391,7 @@ import static org.apache.kafka.common.utils.Utils.min; * commit. * */ +@InterfaceStability.Unstable public class KafkaConsumer implements Consumer { private static final Logger log = LoggerFactory.getLogger(KafkaConsumer.class); diff --git a/clients/src/main/java/org/apache/kafka/common/annotation/InterfaceStability.java b/clients/src/main/java/org/apache/kafka/common/annotation/InterfaceStability.java new file mode 100644 index 0000000..0d38f56 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/annotation/InterfaceStability.java @@ -0,0 +1,48 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package org.apache.kafka.common.annotation; + +import java.lang.annotation.Documented; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; + +/** + * Annotation to inform users of how much to rely on a particular package, + * class or method not changing over time. Currently the stability can be + * {@link Stable}, {@link Evolving} or {@link Unstable}.
    + */ +@InterfaceStability.Evolving +public class InterfaceStability { + /** + * Can evolve while retaining compatibility for minor release boundaries.; + * can break compatibility only at major release (ie. at m.0). + */ + @Documented + @Retention(RetentionPolicy.RUNTIME) + public @interface Stable { } + + /** + * Evolving, but can break compatibility at minor release (i.e. m.x) + */ + @Documented + @Retention(RetentionPolicy.RUNTIME) + public @interface Evolving { } + + /** + * No guarantee is provided as to reliability or stability across any + * level of release granularity. + */ + @Documented + @Retention(RetentionPolicy.RUNTIME) + public @interface Unstable { } +} \ No newline at end of file -- 1.7.12.4 From 78685dc162a867c0b8870faf5baa6a098953ddc8 Mon Sep 17 00:00:00 2001 From: Alexander Pakulov Date: Wed, 12 Aug 2015 19:58:44 -0700 Subject: [PATCH 106/120] KAFKA-1782: fix JUnit3 Misuse Author: Ewen Cheslack-Postava Reviewers: Ewen Cheslack-Postava, Guozhang Wang Closes #135 from ewencp/kafka-1782-junit3-misusage and squashes the following commits: 0ae6258 [Ewen Cheslack-Postava] KAFKA-1782: Junit3 Misusage --- .../integration/kafka/api/ConsumerBounceTest.scala | 2 ++ .../scala/integration/kafka/api/ConsumerTest.scala | 2 ++ .../kafka/api/IntegrationTestHarness.scala | 5 ++- .../integration/kafka/api/ProducerBounceTest.scala | 4 ++- .../kafka/api/ProducerCompressionTest.scala | 3 +- .../kafka/api/ProducerFailureHandlingTest.scala | 14 ++++---- .../integration/kafka/api/ProducerSendTest.scala | 8 ++--- .../test/scala/unit/kafka/KafkaConfigTest.scala | 2 +- .../scala/unit/kafka/admin/AddPartitionsTest.scala | 8 +++-- .../test/scala/unit/kafka/admin/AdminTest.scala | 3 +- .../scala/unit/kafka/admin/ConfigCommandTest.scala | 7 +--- .../unit/kafka/admin/DeleteConsumerGroupTest.scala | 3 +- .../scala/unit/kafka/admin/DeleteTopicTest.scala | 3 +- .../scala/unit/kafka/admin/TopicCommandTest.scala | 5 ++- .../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 +++-- .../ConsumerCoordinatorResponseTest.scala | 6 ++-- .../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 | 12 ++++--- .../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 | 10 +++--- .../integration/UncleanLeaderElectionTest.scala | 12 +++---- .../consumer/ZookeeperConsumerConnectorTest.scala | 6 ++-- .../javaapi/message/BaseMessageSetTestCases.scala | 2 +- .../javaapi/message/ByteBufferMessageSetTest.scala | 2 +- .../test/scala/unit/kafka/log/CleanerTest.scala | 17 ++++----- .../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 | 18 +++++----- .../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 +- .../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 | 9 +++-- .../unit/kafka/network/SocketServerTest.scala | 27 +++++++-------- .../unit/kafka/producer/AsyncProducerTest.scala | 15 ++------ .../scala/unit/kafka/producer/ProducerTest.scala | 28 +++++++-------- .../unit/kafka/producer/SyncProducerTest.scala | 5 ++- .../unit/kafka/server/AdvertiseBrokerTest.scala | 8 +++-- .../unit/kafka/server/DelayedOperationTest.scala | 15 ++++---- .../kafka/server/DynamicConfigChangeTest.scala | 3 +- .../server/HighwatermarkPersistenceTest.scala | 3 +- .../unit/kafka/server/ISRExpirationTest.scala | 12 +++---- .../scala/unit/kafka/server/KafkaConfigTest.scala | 4 +-- .../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 | 10 +++--- .../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 | 13 +++---- 87 files changed, 276 insertions(+), 293 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala index 1d07391..cf65f12 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala @@ -19,6 +19,7 @@ import org.apache.kafka.clients.consumer._ import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord} import org.apache.kafka.common.TopicPartition import org.junit.Assert._ +import org.junit.Before import scala.collection.JavaConversions._ @@ -52,6 +53,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 79f1640..b46070a 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala @@ -25,6 +25,7 @@ import kafka.server.KafkaConfig import java.util.ArrayList import org.junit.Assert._ +import org.junit.Before import scala.collection.JavaConverters._ import kafka.coordinator.ConsumerCoordinator @@ -56,6 +57,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging { this.consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") this.consumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") + @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 afcc349..8080b08 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 import kafka.coordinator.ConsumerCoordinator @@ -49,6 +50,7 @@ trait IntegrationTestHarness extends KafkaServerTestHarness { cfgs.map(KafkaConfig.fromProps) } + @Before override def setUp() { super.setUp() producerConfig.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, this.bootstrapUrl) @@ -70,7 +72,8 @@ trait IntegrationTestHarness extends KafkaServerTestHarness { servers, servers(0).consumerCoordinator.offsetsTopicConfigs) } - + + @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..1198df0 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala @@ -17,22 +17,20 @@ package kafka.api -import org.junit.Test -import org.junit.Assert._ - +import java.util.concurrent.{ExecutionException, TimeUnit, TimeoutException} import java.util.{Properties, Random} -import java.util.concurrent.{TimeoutException, TimeUnit, ExecutionException} import kafka.common.Topic import kafka.consumer.SimpleConsumer import kafka.integration.KafkaServerTestHarness import kafka.server.KafkaConfig import kafka.utils.{ShutdownableThread, TestUtils} - -import org.apache.kafka.common.KafkaException -import org.apache.kafka.common.errors.{InvalidTopicException, NotEnoughReplicasException, NotEnoughReplicasAfterAppendException} import org.apache.kafka.clients.producer._ import org.apache.kafka.clients.producer.internals.ErrorLoggingCallback +import org.apache.kafka.common.KafkaException +import org.apache.kafka.common.errors.{InvalidTopicException, NotEnoughReplicasAfterAppendException, NotEnoughReplicasException} +import org.junit.Assert._ +import org.junit.{After, Before, Test} class ProducerFailureHandlingTest extends KafkaServerTestHarness { private val producerBufferSize = 30000 @@ -61,6 +59,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { private val topic1 = "topic-1" private val topic2 = "topic-2" + @Before override def setUp() { super.setUp() @@ -69,6 +68,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 diff --git a/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala index 5c6ccbc..4b6358c 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala @@ -30,11 +30,9 @@ import org.apache.kafka.common.config.ConfigException import org.apache.kafka.common.errors.SerializationException import org.apache.kafka.common.serialization.ByteArraySerializer import org.junit.Assert._ -import org.junit.Test -import org.scalatest.junit.JUnit3Suite +import org.junit.{After, Before, Test} - -class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness { +class ProducerSendTest extends KafkaServerTestHarness { val numServers = 2 val overridingProps = new Properties() @@ -49,6 +47,7 @@ class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness { private val topic = "topic" private val numRecords = 100 + @Before override def setUp() { super.setUp() @@ -57,6 +56,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() 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 8b14bcf..08c170b 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 86dcc4c..9bd8171 100755 --- a/core/src/test/scala/unit/kafka/admin/AdminTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AdminTest.scala @@ -18,7 +18,6 @@ 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._ @@ -30,7 +29,7 @@ import java.io.File import TestUtils._ -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/ConfigCommandTest.scala b/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala index cfe0ec3..6524ae7 100644 --- a/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala @@ -19,15 +19,10 @@ package kafka.admin import junit.framework.Assert._ import kafka.admin.ConfigCommand.ConfigCommandOptions import org.junit.Test -import org.scalatest.junit.JUnit3Suite import kafka.utils.Logging -import kafka.utils.TestUtils import kafka.zk.ZooKeeperTestHarness -import kafka.server.{ConfigType, OffsetManager, KafkaConfig} -import kafka.admin.TopicCommand.TopicCommandOptions -import kafka.utils.ZkUtils -class ConfigCommandTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { +class ConfigCommandTest extends ZooKeeperTestHarness with Logging { @Test def testArgumentParse() { // Should parse correctly 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 c38df80..fbae398 100644 --- a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala +++ b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala @@ -17,7 +17,6 @@ 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} @@ -26,7 +25,7 @@ import org.junit.Test import java.util.Properties import kafka.common.{TopicAlreadyMarkedForDeletionException, 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 58adef6..9bfec72 100644 --- a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala @@ -18,16 +18,15 @@ 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.{ConfigType, OffsetManager, KafkaConfig} +import kafka.server.ConfigType import kafka.admin.TopicCommand.TopicCommandOptions import kafka.utils.ZkUtils import kafka.coordinator.ConsumerCoordinator -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 17525fe..0482bf1 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 4b326d0..2e18e92 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/ConsumerCoordinatorResponseTest.scala b/core/src/test/scala/unit/kafka/coordinator/ConsumerCoordinatorResponseTest.scala index 058daef..42ffdde 100644 --- a/core/src/test/scala/unit/kafka/coordinator/ConsumerCoordinatorResponseTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/ConsumerCoordinatorResponseTest.scala @@ -20,10 +20,10 @@ package kafka.coordinator import java.util.concurrent.TimeUnit -import junit.framework.Assert._ +import org.junit.Assert._ import kafka.common.TopicAndPartition -import kafka.server.{OffsetManager, ReplicaManager, KafkaConfig} -import kafka.utils.{KafkaScheduler, TestUtils} +import kafka.server.{OffsetManager, KafkaConfig} +import kafka.utils.TestUtils import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.JoinGroupRequest import org.easymock.EasyMock 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 2cbf6e2..d8a7948 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..bca0dcc 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 scala.collection.mutable.Buffer +import kafka.common.KafkaException import kafka.server._ import kafka.utils.{CoreUtils, TestUtils} -import org.scalatest.junit.JUnit3Suite import kafka.zk.ZooKeeperTestHarness -import kafka.common.KafkaException +import org.junit.{After, Before} + +import scala.collection.mutable.Buffer /** * 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 9aebec4..61451a2 100644 --- a/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala +++ b/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala @@ -19,9 +19,8 @@ package kafka.integration import java.nio.ByteBuffer -import junit.framework.Assert._ import kafka.admin.AdminUtils -import kafka.api.{TopicMetadataResponse, TopicMetadataRequest} +import kafka.api.{TopicMetadataRequest, TopicMetadataResponse} import kafka.client.ClientUtils import kafka.cluster.{Broker, BrokerEndPoint} import kafka.common.ErrorMapping @@ -30,14 +29,16 @@ import kafka.utils.TestUtils import kafka.utils.TestUtils._ import kafka.zk.ZooKeeperTestHarness import org.apache.kafka.common.protocol.SecurityProtocol -import org.scalatest.junit.JUnit3Suite +import org.junit.Assert._ +import org.junit.{After, Before} -class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { +class TopicMetadataTest extends ZooKeeperTestHarness { private var server1: KafkaServer = null var brokerEndPoints: Seq[BrokerEndPoint] = null var adHocConfigs: Seq[KafkaConfig] = null val numConfigs: Int = 4 + @Before override def setUp() { super.setUp() val props = createBrokerConfigs(numConfigs, zkConnect) @@ -47,6 +48,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..28f6cc3 100755 --- a/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala +++ b/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala @@ -18,24 +18,22 @@ 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} -import kafka.producer.{KeyedMessage, Producer} +import kafka.consumer.{Consumer, ConsumerConfig} import kafka.serializer.StringDecoder import kafka.server.{KafkaConfig, KafkaServer} import kafka.utils.CoreUtils import kafka.utils.TestUtils._ import kafka.zk.ZooKeeperTestHarness +import org.junit.Assert._ -class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { +class UncleanLeaderElectionTest extends ZooKeeperTestHarness { val brokerId1 = 0 val brokerId2 = 1 @@ -58,6 +56,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 +76,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 0e2a6a1..9c4518c 100755 --- a/core/src/test/scala/unit/kafka/log/CleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/CleanerTest.scala @@ -17,19 +17,20 @@ package kafka.log +import java.io.File +import java.nio._ import java.util.Properties +import java.util.concurrent.atomic.AtomicLong -import junit.framework.Assert._ -import org.scalatest.junit.JUnitSuite -import org.junit.{After, Test} -import java.nio._ -import java.io.File -import scala.collection._ import kafka.common._ -import kafka.utils._ import kafka.message._ -import java.util.concurrent.atomic.AtomicLong +import kafka.utils._ import org.apache.kafka.common.utils.Utils +import org.junit.Assert._ +import org.junit.{After, Test} +import org.scalatest.junit.JUnitSuite + +import scala.collection._ /** * Unit tests for the log cleaning logic diff --git a/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala b/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala index 02cf668..95085f4 100644 --- a/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala +++ b/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala @@ -20,7 +20,7 @@ package kafka.log import java.io._ 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 381e9aa..70beb5f 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala @@ -30,16 +30,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 72e98b3..348b012 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 testFromPropsEmpty() { diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index a13f2be..816354f 100755 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -19,14 +19,14 @@ package kafka.log import java.io._ import java.util.Properties -import junit.framework.Assert._ -import org.junit.Test -import org.scalatest.junit.JUnit3Suite -import kafka.server.{BrokerState, OffsetCheckpoint} + import kafka.common._ +import kafka.server.OffsetCheckpoint import kafka.utils._ +import org.junit.Assert._ +import org.junit.{After, Before, Test} -class LogManagerTest extends JUnit3Suite { +class LogManagerTest { val time: MockTime = new MockTime() val maxRollInterval = 100 @@ -41,20 +41,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 abcd1f0..fa982b1 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 9e26190..7f0d9d6 100755 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -20,7 +20,7 @@ package kafka.log import java.io._ import java.util.Properties 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/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..6ceb17b 100644 --- a/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala +++ b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala @@ -21,11 +21,10 @@ 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.{After, Test} +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" @@ -48,6 +46,7 @@ class MetricsTest extends JUnit3Suite with KafkaServerTestHarness with Logging { val nMessages = 2 + @After override def tearDown() { super.tearDown() } diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index 7dc2fad..d08b8b8 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -17,27 +17,26 @@ package kafka.network; -import java.net._ import java.io._ +import java.net._ +import java.nio.ByteBuffer +import java.util.Random + +import kafka.api.ProducerRequest import kafka.cluster.EndPoint +import kafka.common.TopicAndPartition +import kafka.message.ByteBufferMessageSet +import kafka.producer.SyncProducerConfig import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.network.NetworkSend import org.apache.kafka.common.protocol.SecurityProtocol import org.apache.kafka.common.utils.SystemTime +import org.junit.Assert._ import org.junit._ -import org.scalatest.junit.JUnitSuite -import java.util.Random -import junit.framework.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 scala.collection.Map -class SocketServerTest extends JUnitSuite { +class SocketServerTest { val server: SocketServer = new SocketServer(0, Map(SecurityProtocol.PLAINTEXT -> EndPoint(null, 0, SecurityProtocol.PLAINTEXT), @@ -84,11 +83,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 +174,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..dd96d29 100755 --- a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala @@ -17,26 +17,24 @@ package kafka.producer -import org.scalatest.TestFailedException -import org.scalatest.junit.JUnit3Suite -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 kafka.utils._ import java.util +import java.util.Properties + 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.consumer.SimpleConsumer +import kafka.message.Message import kafka.serializer.StringEncoder +import kafka.server.{KafkaConfig, KafkaRequestHandler, KafkaServer} +import kafka.utils._ +import kafka.zk.ZooKeeperTestHarness +import org.apache.log4j.{Level, Logger} +import org.junit.Assert._ +import org.junit.{After, Before, Test} +import org.scalatest.exceptions.TestFailedException -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 +58,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 +80,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) 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 7c45393..16ac40d 100644 --- a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala @@ -26,9 +26,8 @@ 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/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index f32d206..e26a730 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -26,9 +26,9 @@ import kafka.utils.{TestUtils, CoreUtils} import org.apache.kafka.common.config.ConfigException import org.apache.kafka.common.protocol.SecurityProtocol import org.junit.{Assert, Test} -import org.scalatest.junit.JUnit3Suite +import org.scalatest.Assertions.intercept -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 21081ce..7a0d0b2 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 39a6852..f846698 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..2a8da0c 100755 --- a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala @@ -27,18 +27,18 @@ 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._ -class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness { +class ServerShutdownTest extends ZooKeeperTestHarness { var config: KafkaConfig = null val host = "localhost" val topic = "test" val sent1 = List("hello", "there") val sent2 = List("more", "messages") - override def setUp(): Unit = { + @Before + override def setUp() { super.setUp() 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 b9de8d6..b3835f0 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 eb169d8..f00f00a 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 @@ -595,7 +594,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..e4bfb48 100755 --- a/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala +++ b/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala @@ -17,11 +17,12 @@ package kafka.zk -import org.scalatest.junit.JUnit3Suite import org.I0Itec.zkclient.ZkClient import kafka.utils.{ZkUtils, CoreUtils} +import org.junit.{After, Before} +import org.scalatest.junit.JUnitSuite -trait ZooKeeperTestHarness extends JUnit3Suite { +trait ZooKeeperTestHarness extends JUnitSuite { var zkPort: Int = -1 var zookeeper: EmbeddedZookeeper = null var zkClient: ZkClient = null @@ -30,17 +31,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 } } -- 1.7.12.4 From e2ebae80982c45c65f709efaedd89d10e109600f Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 13 Aug 2015 10:35:09 -0700 Subject: [PATCH 107/120] KAFKA-2430; Listing of PR commits in commit message should be optional If there is a single commit in the PR, then it's never listed. Author: Ismael Juma Reviewers: Guozhang Wang Closes #136 from ijuma/kafka-2430-optional-listing-commits and squashes the following commits: 64f1aec [Ismael Juma] Listing of PR commits in commit message should be optional --- kafka-merge-pr.py | 23 +++++++++++++++++------ 1 file changed, 17 insertions(+), 6 deletions(-) diff --git a/kafka-merge-pr.py b/kafka-merge-pr.py index 5e8cbf5..7f48bf9 100644 --- a/kafka-merge-pr.py +++ b/kafka-merge-pr.py @@ -140,7 +140,16 @@ def merge_pr(pr_num, target_ref, title, body, pr_repo_desc): "Enter reviewers in the format of \"name1 , name2 \": ").strip() commits = run_cmd(['git', 'log', 'HEAD..%s' % pr_branch_name, - '--pretty=format:%h [%an] %s']).split("\n\n") + '--pretty=format:%h [%an] %s']).split("\n") + + if len(commits) > 1: + result = raw_input("List pull request commits in squashed commit message? (y/n): ") + if result.lower() == "y": + should_list_commits = True + else: + should_list_commits = False + else: + should_list_commits = False merge_message_flags = [] @@ -165,11 +174,13 @@ def merge_pr(pr_num, target_ref, title, body, pr_repo_desc): merge_message_flags += ["-m", message] # The string "Closes #%s" string is required for GitHub to correctly close the PR - merge_message_flags += [ - "-m", - "Closes #%s from %s and squashes the following commits:" % (pr_num, pr_repo_desc)] - for c in commits: - merge_message_flags += ["-m", c] + close_line = "Closes #%s from %s" % (pr_num, pr_repo_desc) + if should_list_commits: + close_line += " and squashes the following commits:" + merge_message_flags += ["-m", close_line] + + if should_list_commits: + merge_message_flags += ["-m", "\n".join(commits)] run_cmd(['git', 'commit', '--author="%s"' % primary_author] + merge_message_flags) -- 1.7.12.4 From 2c55bd8aa2666ea11ee2814b04f782eef8fa52c6 Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Thu, 13 Aug 2015 17:54:36 -0400 Subject: [PATCH 108/120] KAFKA-2406: Throttle ISR propagation This is a follow up patch for KAFKA-2406. Further test to verify if this change alone is enough to solve the problem or not. Author: Jiangjie Qin Author: Jiangjie Qin Reviewers: Jun Rao Closes #114 from becketqin/KAFKA-2406 --- core/src/main/scala/kafka/cluster/Partition.scala | 2 + .../scala/kafka/common/TopicAndPartition.scala | 4 -- .../scala/kafka/controller/KafkaController.scala | 47 +++++++++++++--------- core/src/main/scala/kafka/server/KafkaConfig.scala | 7 ++-- .../main/scala/kafka/server/ReplicaManager.scala | 43 ++++++++++++-------- .../main/scala/kafka/utils/ReplicationUtils.scala | 23 ++++++----- core/src/main/scala/kafka/utils/ZkUtils.scala | 19 +++++---- .../unit/kafka/integration/TopicMetadataTest.scala | 2 +- 8 files changed, 88 insertions(+), 59 deletions(-) diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 511d3c9..ee332ed 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -428,7 +428,9 @@ class Partition(val topic: String, val newLeaderAndIsr = new LeaderAndIsr(localBrokerId, leaderEpoch, newIsr.map(r => r.brokerId).toList, zkVersion) val (updateSucceeded,newVersion) = ReplicationUtils.updateLeaderAndIsr(zkClient, topic, partitionId, newLeaderAndIsr, controllerEpoch, zkVersion) + if(updateSucceeded) { + replicaManager.recordIsrChange(new TopicAndPartition(topic, partitionId)) inSyncReplicas = newIsr zkVersion = newVersion trace("ISR updated to [%s] and zkVersion updated to [%d]".format(newIsr.mkString(","), zkVersion)) diff --git a/core/src/main/scala/kafka/common/TopicAndPartition.scala b/core/src/main/scala/kafka/common/TopicAndPartition.scala index 13a3f28..95db1dc 100644 --- a/core/src/main/scala/kafka/common/TopicAndPartition.scala +++ b/core/src/main/scala/kafka/common/TopicAndPartition.scala @@ -25,8 +25,6 @@ import kafka.utils.Json */ case class TopicAndPartition(topic: String, partition: Int) { - private val version: Long = 1L - def this(tuple: (String, Int)) = this(tuple._1, tuple._2) def this(partition: Partition) = this(partition.topic, partition.partitionId) @@ -36,6 +34,4 @@ case class TopicAndPartition(topic: String, partition: Int) { def asTuple = (topic, partition) override def toString = "[%s,%d]".format(topic, partition) - - def toJson = Json.encode(Map("version" -> version, "topic" -> topic, "partition" -> partition)) } \ No newline at end of file diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 68536f5..4c37616 100755 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -923,7 +923,6 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt private def registerIsrChangeNotificationListener() = { debug("Registering IsrChangeNotificationListener") - ZkUtils.makeSurePersistentPathExists(zkClient, ZkUtils.IsrChangeNotificationPath) zkClient.subscribeChildChanges(ZkUtils.IsrChangeNotificationPath, isrChangeNotificationListener) } @@ -1339,7 +1338,6 @@ class ReassignedPartitionsIsrChangeListener(controller: KafkaController, topic: * @param controller */ class IsrChangeNotificationListener(controller: KafkaController) extends IZkChildListener with Logging { - var topicAndPartitionSet: Set[TopicAndPartition] = Set() override def handleChildChange(parentPath: String, currentChildren: util.List[String]): Unit = { import scala.collection.JavaConverters._ @@ -1347,23 +1345,25 @@ class IsrChangeNotificationListener(controller: KafkaController) extends IZkChil inLock(controller.controllerContext.controllerLock) { debug("[IsrChangeNotificationListener] Fired!!!") val childrenAsScala: mutable.Buffer[String] = currentChildren.asScala - val topicAndPartitions: immutable.Set[TopicAndPartition] = childrenAsScala.map(x => getTopicAndPartition(x)).flatten.toSet - controller.updateLeaderAndIsrCache(topicAndPartitions) - processUpdateNotifications(topicAndPartitions) - - // delete processed children - childrenAsScala.map(x => ZkUtils.deletePath(controller.controllerContext.zkClient, - ZkUtils.getEntityConfigPath(ConfigType.Topic, x))) + try { + val topicAndPartitions: immutable.Set[TopicAndPartition] = childrenAsScala.map(x => getTopicAndPartition(x)).flatten.toSet + controller.updateLeaderAndIsrCache(topicAndPartitions) + processUpdateNotifications(topicAndPartitions) + } finally { + // delete processed children + childrenAsScala.map(x => ZkUtils.deletePath(controller.controllerContext.zkClient, + ZkUtils.IsrChangeNotificationPath + "/" + x)) + } } } private def processUpdateNotifications(topicAndPartitions: immutable.Set[TopicAndPartition]) { val liveBrokers: Seq[Int] = controller.controllerContext.liveOrShuttingDownBrokerIds.toSeq - controller.sendUpdateMetadataRequest(liveBrokers, topicAndPartitions) debug("Sending MetadataRequest to Brokers:" + liveBrokers + " for TopicAndPartitions:" + topicAndPartitions) + controller.sendUpdateMetadataRequest(liveBrokers, topicAndPartitions) } - private def getTopicAndPartition(child: String): Option[TopicAndPartition] = { + private def getTopicAndPartition(child: String): Set[TopicAndPartition] = { val changeZnode: String = ZkUtils.IsrChangeNotificationPath + "/" + child val (jsonOpt, stat) = ZkUtils.readDataMaybeNull(controller.controllerContext.zkClient, changeZnode) if (jsonOpt.isDefined) { @@ -1371,20 +1371,31 @@ class IsrChangeNotificationListener(controller: KafkaController) extends IZkChil json match { case Some(m) => - val topicAndPartition = m.asInstanceOf[Map[String, Any]] - val topic = topicAndPartition("topic").asInstanceOf[String] - val partition = topicAndPartition("partition").asInstanceOf[Int] - Some(TopicAndPartition(topic, partition)) + val topicAndPartitions: mutable.Set[TopicAndPartition] = new mutable.HashSet[TopicAndPartition]() + val isrChanges = m.asInstanceOf[Map[String, Any]] + val topicAndPartitionList = isrChanges("partitions").asInstanceOf[List[Any]] + topicAndPartitionList.foreach { + case tp => + val topicAndPartition = tp.asInstanceOf[Map[String, Any]] + val topic = topicAndPartition("topic").asInstanceOf[String] + val partition = topicAndPartition("partition").asInstanceOf[Int] + topicAndPartitions += TopicAndPartition(topic, partition) + } + topicAndPartitions case None => - error("Invalid topic and partition JSON: " + json + " in ZK: " + changeZnode) - None + error("Invalid topic and partition JSON: " + jsonOpt.get + " in ZK: " + changeZnode) + Set.empty } } else { - None + Set.empty } } } +object IsrChangeNotificationListener { + val version: Long = 1L +} + /** * Starts the preferred replica leader election for the list of partitions specified under * /admin/preferred_replica_election - diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index dbe170f..a06f0bd 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -26,10 +26,11 @@ import kafka.consumer.ConsumerConfig import kafka.message.{BrokerCompressionCodec, CompressionCodec, Message, MessageSet} import kafka.utils.CoreUtils import org.apache.kafka.clients.CommonClientConfigs -import org.apache.kafka.common.config.{ConfigException, AbstractConfig, ConfigDef} +import org.apache.kafka.common.config.{AbstractConfig, ConfigDef} import org.apache.kafka.common.metrics.MetricsReporter import org.apache.kafka.common.protocol.SecurityProtocol -import scala.collection.{mutable, immutable, JavaConversions, Map} + +import scala.collection.{Map, immutable} object Defaults { /** ********* Zookeeper Configuration ***********/ @@ -531,7 +532,7 @@ object KafkaConfig { * Check that property names are valid */ def validateNames(props: Properties) { - import JavaConversions._ + import scala.collection.JavaConversions._ val names = configDef.names() for (name <- props.keys) require(names.contains(name), "Unknown configuration \"%s\".".format(name)) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 795220e..2e0bbcd 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -16,30 +16,24 @@ */ package kafka.server +import java.io.{File, IOException} +import java.util.concurrent.TimeUnit +import java.util.concurrent.atomic.AtomicBoolean + +import com.yammer.metrics.core.Gauge import kafka.api._ -import kafka.common._ -import kafka.utils._ import kafka.cluster.{BrokerEndPoint, Partition, Replica} -import kafka.log.{LogAppendInfo, LogManager} -import kafka.metrics.KafkaMetricsGroup +import kafka.common._ import kafka.controller.KafkaController +import kafka.log.{LogAppendInfo, LogManager} import kafka.message.{ByteBufferMessageSet, MessageSet} -import kafka.api.ProducerResponseStatus -import kafka.common.TopicAndPartition -import kafka.api.PartitionFetchInfo - +import kafka.metrics.KafkaMetricsGroup +import kafka.utils._ +import org.I0Itec.zkclient.ZkClient import org.apache.kafka.common.protocol.Errors -import java.util.concurrent.atomic.AtomicBoolean -import java.io.{IOException, File} -import java.util.concurrent.TimeUnit - -import scala.Some import scala.collection._ -import org.I0Itec.zkclient.ZkClient -import com.yammer.metrics.core.Gauge - /* * Result metadata of a log append operation on the log */ @@ -116,6 +110,7 @@ class ReplicaManager(val config: KafkaConfig, private var hwThreadInitialized = false this.logIdent = "[Replica Manager on Broker " + localBrokerId + "]: " val stateChangeLogger = KafkaController.stateChangeLogger + private val isrChangeSet: mutable.Set[TopicAndPartition] = new mutable.HashSet[TopicAndPartition]() val delayedProducePurgatory = new DelayedOperationPurgatory[DelayedProduce]( purgatoryName = "Produce", config.brokerId, config.producerPurgatoryPurgeIntervalRequests) @@ -154,6 +149,21 @@ class ReplicaManager(val config: KafkaConfig, scheduler.schedule("highwatermark-checkpoint", checkpointHighWatermarks, period = config.replicaHighWatermarkCheckpointIntervalMs, unit = TimeUnit.MILLISECONDS) } + def recordIsrChange(topicAndPartition: TopicAndPartition) { + isrChangeSet synchronized { + isrChangeSet += topicAndPartition + } + } + + def maybePropagateIsrChanges() { + isrChangeSet synchronized { + if (isrChangeSet.nonEmpty) { + ReplicationUtils.propagateIsrChanges(zkClient, isrChangeSet) + isrChangeSet.clear() + } + } + } + /** * Try to complete some delayed produce requests with the request key; * this can be triggered when: @@ -181,6 +191,7 @@ class ReplicaManager(val config: KafkaConfig, def startup() { // start ISR expiration thread scheduler.schedule("isr-expiration", maybeShrinkIsr, period = config.replicaLagTimeMaxMs, unit = TimeUnit.MILLISECONDS) + scheduler.schedule("isr-change-propagation", maybePropagateIsrChanges, period = 5000, unit = TimeUnit.MILLISECONDS) } def stopReplica(topic: String, partitionId: Int, deletePartition: Boolean): Short = { diff --git a/core/src/main/scala/kafka/utils/ReplicationUtils.scala b/core/src/main/scala/kafka/utils/ReplicationUtils.scala index 783ba10..d99629a 100644 --- a/core/src/main/scala/kafka/utils/ReplicationUtils.scala +++ b/core/src/main/scala/kafka/utils/ReplicationUtils.scala @@ -19,7 +19,7 @@ package kafka.utils import kafka.api.LeaderAndIsr import kafka.common.TopicAndPartition -import kafka.controller.LeaderIsrAndControllerEpoch +import kafka.controller.{IsrChangeNotificationListener, LeaderIsrAndControllerEpoch} import org.I0Itec.zkclient.ZkClient import org.apache.zookeeper.data.Stat @@ -27,7 +27,7 @@ import scala.collection._ object ReplicationUtils extends Logging { - val IsrChangeNotificationPrefix = "isr_change_" + private val IsrChangeNotificationPrefix = "isr_change_" def updateLeaderAndIsr(zkClient: ZkClient, topic: String, partitionId: Int, newLeaderAndIsr: LeaderAndIsr, controllerEpoch: Int, zkVersion: Int): (Boolean,Int) = { @@ -36,16 +36,16 @@ object ReplicationUtils extends Logging { val newLeaderData = ZkUtils.leaderAndIsrZkData(newLeaderAndIsr, controllerEpoch) // use the epoch of the controller that made the leadership decision, instead of the current controller epoch val updatePersistentPath: (Boolean, Int) = ZkUtils.conditionalUpdatePersistentPath(zkClient, path, newLeaderData, zkVersion, Some(checkLeaderAndIsrZkData)) - if (updatePersistentPath._1) { - val topicAndPartition: TopicAndPartition = TopicAndPartition(topic, partitionId) - val isrChangeNotificationPath: String = ZkUtils.createSequentialPersistentPath( - zkClient, ZkUtils.IsrChangeNotificationPath + "/" + IsrChangeNotificationPrefix, - topicAndPartition.toJson) - debug("Added " + isrChangeNotificationPath + " for " + topicAndPartition) - } updatePersistentPath } + def propagateIsrChanges(zkClient: ZkClient, isrChangeSet: Set[TopicAndPartition]): Unit = { + val isrChangeNotificationPath: String = ZkUtils.createSequentialPersistentPath( + zkClient, ZkUtils.IsrChangeNotificationPath + "/" + IsrChangeNotificationPrefix, + generateIsrChangeJson(isrChangeSet)) + debug("Added " + isrChangeNotificationPath + " for " + isrChangeSet) + } + def checkLeaderAndIsrZkData(zkClient: ZkClient, path: String, expectedLeaderAndIsrInfo: String): (Boolean,Int) = { try { val writtenLeaderAndIsrInfo = ZkUtils.readDataMaybeNull(zkClient, path) @@ -89,4 +89,9 @@ object ReplicationUtils extends Logging { Some(LeaderIsrAndControllerEpoch(LeaderAndIsr(leader, epoch, isr, zkPathVersion), controllerEpoch))} } + private def generateIsrChangeJson(isrChanges: Set[TopicAndPartition]): String = { + val partitions = isrChanges.map(tp => Map("topic" -> tp.topic, "partition" -> tp.partition)).toArray + Json.encode(Map("version" -> IsrChangeNotificationListener.version, "partitions" -> partitions)) + } + } diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index 4ae310e..74b587e 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -49,6 +49,16 @@ object ZkUtils extends Logging { val IsrChangeNotificationPath = "/isr_change_notification" val EntityConfigPath = "/config" val EntityConfigChangesPath = "/config/changes" + // These are persistent ZK paths that should exist on kafka broker startup. + val persistentZkPaths = Seq(ConsumersPath, + BrokerIdsPath, + BrokerTopicsPath, + EntityConfigChangesPath, + ZkUtils.getEntityConfigRootPath(ConfigType.Topic), + ZkUtils.getEntityConfigRootPath(ConfigType.Client), + DeleteTopicsPath, + BrokerSequenceIdPath, + IsrChangeNotificationPath) def getTopicPath(topic: String): String = { BrokerTopicsPath + "/" + topic @@ -97,14 +107,7 @@ object ZkUtils extends Logging { } def setupCommonPaths(zkClient: ZkClient) { - for(path <- Seq(ConsumersPath, - BrokerIdsPath, - BrokerTopicsPath, - EntityConfigChangesPath, - ZkUtils.getEntityConfigRootPath(ConfigType.Topic), - ZkUtils.getEntityConfigRootPath(ConfigType.Client), - DeleteTopicsPath, - BrokerSequenceIdPath)) + for(path <- persistentZkPaths) makeSurePersistentPathExists(zkClient, path) } diff --git a/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala b/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala index 61451a2..24f0a07 100644 --- a/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala +++ b/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala @@ -195,7 +195,7 @@ class TopicMetadataTest extends ZooKeeperTestHarness { metadata.topicsMetadata.head.partitionsMetadata.nonEmpty) metadata.topicsMetadata.head.partitionsMetadata.head.isr else - "")) + ""), 6000L) }) } -- 1.7.12.4 From c8e62c9818a04c01a4eb16cf30772dfbd30d3a26 Mon Sep 17 00:00:00 2001 From: Geoff Anderson Date: Thu, 13 Aug 2015 15:22:54 -0700 Subject: [PATCH 109/120] MINOR: expose vagrant base box as variable Added base_box variable to Vagrantfile. This makes it possible to override the base box in Vagrantfile.local. Author: Geoff Anderson Reviewers: Guozhang Wang Closes #137 from granders/minor-expose-vagrant-box and squashes the following commits: 44936f7 [Geoff Anderson] Added base_box variable to Vagrantfile. This makes it possible to override the base box in Vagrantfile.local. --- Vagrantfile | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/Vagrantfile b/Vagrantfile index ee8b352..28bf24a 100644 --- a/Vagrantfile +++ b/Vagrantfile @@ -27,6 +27,7 @@ num_zookeepers = 1 num_brokers = 3 num_workers = 0 # Generic workers that get the code, but don't start any services ram_megabytes = 1280 +base_box = "ubuntu/trusty64" # EC2 ec2_access_key = ENV['AWS_ACCESS_KEY'] @@ -77,7 +78,7 @@ Vagrant.configure(VAGRANTFILE_API_VERSION) do |config| ## Provider-specific global configs config.vm.provider :virtualbox do |vb,override| - override.vm.box = "ubuntu/trusty64" + override.vm.box = base_box override.hostmanager.ignore_private_ip = false -- 1.7.12.4 From f6acfb08917946f15cb8de2fee786019124af212 Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Fri, 14 Aug 2015 16:00:51 -0700 Subject: [PATCH 110/120] KAFKA-2366; Initial patch for Copycat This is an initial patch implementing the basics of Copycat for KIP-26. The intent here is to start a review of the key pieces of the core API and get a reasonably functional, baseline, non-distributed implementation of Copycat in place to get things rolling. The current patch has a number of known issues that need to be addressed before a final version: * Some build-related issues. Specifically, requires some locally-installed dependencies (see below), ignores checkstyle for the runtime data library because it's lifted from Avro currently and likely won't last in its current form, and some Gradle task dependencies aren't quite right because I haven't gotten rid of the dependency on `core` (which should now be an easy patch since new consumer groups are in a much better state). * This patch currently depends on some Confluent trunk code because I prototyped with our Avro serializers w/ schema-registry support. We need to figure out what we want to provide as an example built-in set of serializers. Unlike core Kafka where we could ignore the issue, providing only ByteArray or String serializers, this is pretty central to how Copycat works. * This patch uses a hacked up version of Avro as its runtime data format. Not sure if we want to go through the entire API discussion just to get some basic code committed, so I filed KAFKA-2367 to handle that separately. The core connector APIs and the runtime data APIs are entirely orthogonal. * This patch needs some updates to get aligned with recent new consumer changes (specifically, I'm aware of the ConcurrentModificationException issue on exit). More generally, the new consumer is in flux but Copycat depends on it, so there are likely to be some negative interactions. * The layout feels a bit awkward to me right now because I ported it from a Maven layout. We don't have nearly the same level of granularity in Kafka currently (core and clients, plus the mostly ignored examples, log4j-appender, and a couple of contribs). We might want to reorganize, although keeping data+api separate from runtime and connector plugins is useful for minimizing dependencies. * There are a variety of other things (e.g., I'm not happy with the exception hierarchy/how they are currently handled, TopicPartition doesn't really need to be duplicated unless we want Copycat entirely isolated from the Kafka APIs, etc), but I expect those we'll cover in the review. Before commenting on the patch, it's probably worth reviewing https://issues.apache.org/jira/browse/KAFKA-2365 and https://issues.apache.org/jira/browse/KAFKA-2366 to get an idea of what I had in mind for a) what we ultimately want with all the Copycat patches and b) what we aim to cover in this initial patch. My hope is that we can use a WIP patch (after the current obvious deficiencies are addressed) while recognizing that we want to make iterative progress with a bunch of subsequent PRs. Author: Ewen Cheslack-Postava Reviewers: Ismael Juma, Gwen Shapira Closes #99 from ewencp/copycat and squashes the following commits: a3a47a6 [Ewen Cheslack-Postava] Simplify Copycat exceptions, make them a subclass of KafkaException. 8c108b0 [Ewen Cheslack-Postava] Rename Coordinator to Herder to avoid confusion with the consumer coordinator. 7bf8075 [Ewen Cheslack-Postava] Make Copycat CLI speific to standalone mode, clean up some config and get rid of config storage in standalone mode. 656a003 [Ewen Cheslack-Postava] Clarify and expand the explanation of the Copycat Coordinator interface. c0e5fdc [Ewen Cheslack-Postava] Merge remote-tracking branch 'origin/trunk' into copycat 0fa7a36 [Ewen Cheslack-Postava] Mark Copycat classes as unstable and reduce visibility of some classes where possible. d55d31e [Ewen Cheslack-Postava] Reorganize Copycat code to put it all under one top-level directory. b29cb2c [Ewen Cheslack-Postava] Merge remote-tracking branch 'origin/trunk' into copycat d713a21 [Ewen Cheslack-Postava] Address Gwen's review comments. 6787a85 [Ewen Cheslack-Postava] Make Converter generic to match serializers since some serialization formats do not require a base class of Object; update many other classes to have generic key and value class type parameters to match this change. b194c73 [Ewen Cheslack-Postava] Split Copycat converter option into two options for key and value. 0b5a1a0 [Ewen Cheslack-Postava] Normalize naming to use partition for both source and Kafka, adjusting naming in CopycatRecord classes to clearly differentiate. e345142 [Ewen Cheslack-Postava] Remove Copycat reflection utils, use existing Utils and ConfigDef functionality from clients package. be5c387 [Ewen Cheslack-Postava] Minor cleanup 122423e [Ewen Cheslack-Postava] Style cleanup 6ba87de [Ewen Cheslack-Postava] Remove most of the Avro-based mock runtime data API, only preserving enough schema functionality to support basic primitive types for an initial patch. 4674d13 [Ewen Cheslack-Postava] Address review comments, clean up some code styling. 25b5739 [Ewen Cheslack-Postava] Fix sink task offset commit concurrency issue by moving it to the worker thread and waking up the consumer to ensure it exits promptly. 0aefe21 [Ewen Cheslack-Postava] Add log4j settings for Copycat. 220e42d [Ewen Cheslack-Postava] Replace Avro serializer with JSON serializer. 1243a7c [Ewen Cheslack-Postava] Merge remote-tracking branch 'origin/trunk' into copycat 5a618c6 [Ewen Cheslack-Postava] Remove offset serializers, instead reusing the existing serializers and removing schema projection support. e849e10 [Ewen Cheslack-Postava] Remove duplicated TopicPartition implementation. dec1379 [Ewen Cheslack-Postava] Switch to using new consumer coordinator instead of manually assigning partitions. Remove dependency of copycat-runtime on core. 4a9b4f3 [Ewen Cheslack-Postava] Add some helpful Copycat-specific build and test targets that cover all Copycat packages. 31cd1ca [Ewen Cheslack-Postava] Add CLI tools for Copycat. e14942c [Ewen Cheslack-Postava] Add Copycat file connector. 0233456 [Ewen Cheslack-Postava] Add copycat-avro and copycat-runtime 11981d2 [Ewen Cheslack-Postava] Add copycat-data and copycat-api --- bin/copycat-standalone.sh | 23 + bin/kafka-run-class.sh | 8 + build.gradle | 269 ++- checkstyle/import-control.xml | 57 + .../java/org/apache/kafka/common/utils/Utils.java | 15 +- config/copycat-console-sink.properties | 19 + config/copycat-console-source.properties | 19 + config/copycat-file-sink.properties | 20 + config/copycat-file-source.properties | 20 + config/copycat-log4j.properties | 23 + config/copycat-standalone.properties | 28 + .../apache/kafka/copycat/connector/Connector.java | 117 + .../kafka/copycat/connector/ConnectorContext.java | 33 + .../kafka/copycat/connector/CopycatRecord.java | 103 + .../org/apache/kafka/copycat/connector/Task.java | 49 + .../kafka/copycat/errors/CopycatException.java | 40 + .../apache/kafka/copycat/sink/SinkConnector.java | 40 + .../org/apache/kafka/copycat/sink/SinkRecord.java | 71 + .../org/apache/kafka/copycat/sink/SinkTask.java | 64 + .../apache/kafka/copycat/sink/SinkTaskContext.java | 59 + .../kafka/copycat/source/SourceConnector.java | 29 + .../apache/kafka/copycat/source/SourceRecord.java | 103 + .../apache/kafka/copycat/source/SourceTask.java | 62 + .../kafka/copycat/source/SourceTaskContext.java | 40 + .../apache/kafka/copycat/storage/Converter.java | 45 + .../kafka/copycat/storage/OffsetStorageReader.java | 59 + .../apache/kafka/copycat/util/ConnectorUtils.java | 66 + .../connector/ConnectorReconfigurationTest.java | 76 + .../kafka/copycat/util/ConnectorUtilsTest.java | 67 + .../kafka/copycat/data/DataRuntimeException.java | 36 + .../kafka/copycat/data/DataTypeException.java | 33 + .../kafka/copycat/data/ObjectProperties.java | 85 + .../java/org/apache/kafka/copycat/data/Schema.java | 1054 +++++++++ .../apache/kafka/copycat/data/SchemaBuilder.java | 2415 ++++++++++++++++++++ .../kafka/copycat/data/SchemaParseException.java | 32 + .../copycat/file/FileStreamSinkConnector.java | 62 + .../kafka/copycat/file/FileStreamSinkTask.java | 79 + .../copycat/file/FileStreamSourceConnector.java | 70 + .../kafka/copycat/file/FileStreamSourceTask.java | 176 ++ .../copycat/file/FileStreamSinkConnectorTest.java | 85 + .../kafka/copycat/file/FileStreamSinkTaskTest.java | 67 + .../file/FileStreamSourceConnectorTest.java | 104 + .../copycat/file/FileStreamSourceTaskTest.java | 140 ++ .../apache/kafka/copycat/json/JsonConverter.java | 265 +++ .../kafka/copycat/json/JsonDeserializer.java | 87 + .../org/apache/kafka/copycat/json/JsonSchema.java | 114 + .../apache/kafka/copycat/json/JsonSerializer.java | 72 + .../kafka/copycat/json/JsonConverterTest.java | 173 ++ .../kafka/copycat/cli/CopycatStandalone.java | 87 + .../org/apache/kafka/copycat/cli/WorkerConfig.java | 141 ++ .../kafka/copycat/runtime/ConnectorConfig.java | 87 + .../org/apache/kafka/copycat/runtime/Copycat.java | 94 + .../org/apache/kafka/copycat/runtime/Herder.java | 67 + .../kafka/copycat/runtime/SinkTaskContextImpl.java | 24 + .../copycat/runtime/SourceTaskOffsetCommitter.java | 103 + .../org/apache/kafka/copycat/runtime/Worker.java | 236 ++ .../kafka/copycat/runtime/WorkerSinkTask.java | 226 ++ .../copycat/runtime/WorkerSinkTaskThread.java | 112 + .../kafka/copycat/runtime/WorkerSourceTask.java | 310 +++ .../apache/kafka/copycat/runtime/WorkerTask.java | 54 + .../standalone/StandaloneConnectorContext.java | 42 + .../runtime/standalone/StandaloneHerder.java | 257 +++ .../copycat/storage/FileOffsetBackingStore.java | 111 + .../copycat/storage/MemoryOffsetBackingStore.java | 113 + .../kafka/copycat/storage/OffsetBackingStore.java | 74 + .../copycat/storage/OffsetStorageReaderImpl.java | 114 + .../kafka/copycat/storage/OffsetStorageWriter.java | 208 ++ .../org/apache/kafka/copycat/util/Callback.java | 31 + .../apache/kafka/copycat/util/ConnectorTaskId.java | 71 + .../apache/kafka/copycat/util/FutureCallback.java | 76 + .../kafka/copycat/util/ShutdownableThread.java | 145 ++ .../kafka/copycat/runtime/WorkerSinkTaskTest.java | 367 +++ .../copycat/runtime/WorkerSourceTaskTest.java | 279 +++ .../apache/kafka/copycat/runtime/WorkerTest.java | 179 ++ .../runtime/standalone/StandaloneHerderTest.java | 186 ++ .../storage/FileOffsetBackingStoreTest.java | 117 + .../copycat/storage/OffsetStorageWriterTest.java | 242 ++ .../org/apache/kafka/copycat/util/MockTime.java | 49 + .../kafka/copycat/util/ShutdownableThreadTest.java | 72 + .../util/TestBackgroundThreadExceptionHandler.java | 37 + .../apache/kafka/copycat/util/ThreadedTest.java | 43 + settings.gradle | 4 +- 82 files changed, 11116 insertions(+), 15 deletions(-) create mode 100755 bin/copycat-standalone.sh create mode 100644 config/copycat-console-sink.properties create mode 100644 config/copycat-console-source.properties create mode 100644 config/copycat-file-sink.properties create mode 100644 config/copycat-file-source.properties create mode 100644 config/copycat-log4j.properties create mode 100644 config/copycat-standalone.properties create mode 100644 copycat/api/src/main/java/org/apache/kafka/copycat/connector/Connector.java create mode 100644 copycat/api/src/main/java/org/apache/kafka/copycat/connector/ConnectorContext.java create mode 100644 copycat/api/src/main/java/org/apache/kafka/copycat/connector/CopycatRecord.java create mode 100644 copycat/api/src/main/java/org/apache/kafka/copycat/connector/Task.java create mode 100644 copycat/api/src/main/java/org/apache/kafka/copycat/errors/CopycatException.java create mode 100644 copycat/api/src/main/java/org/apache/kafka/copycat/sink/SinkConnector.java create mode 100644 copycat/api/src/main/java/org/apache/kafka/copycat/sink/SinkRecord.java create mode 100644 copycat/api/src/main/java/org/apache/kafka/copycat/sink/SinkTask.java create mode 100644 copycat/api/src/main/java/org/apache/kafka/copycat/sink/SinkTaskContext.java create mode 100644 copycat/api/src/main/java/org/apache/kafka/copycat/source/SourceConnector.java create mode 100644 copycat/api/src/main/java/org/apache/kafka/copycat/source/SourceRecord.java create mode 100644 copycat/api/src/main/java/org/apache/kafka/copycat/source/SourceTask.java create mode 100644 copycat/api/src/main/java/org/apache/kafka/copycat/source/SourceTaskContext.java create mode 100644 copycat/api/src/main/java/org/apache/kafka/copycat/storage/Converter.java create mode 100644 copycat/api/src/main/java/org/apache/kafka/copycat/storage/OffsetStorageReader.java create mode 100644 copycat/api/src/main/java/org/apache/kafka/copycat/util/ConnectorUtils.java create mode 100644 copycat/api/src/test/java/org/apache/kafka/copycat/connector/ConnectorReconfigurationTest.java create mode 100644 copycat/api/src/test/java/org/apache/kafka/copycat/util/ConnectorUtilsTest.java create mode 100644 copycat/data/src/main/java/org/apache/kafka/copycat/data/DataRuntimeException.java create mode 100644 copycat/data/src/main/java/org/apache/kafka/copycat/data/DataTypeException.java create mode 100644 copycat/data/src/main/java/org/apache/kafka/copycat/data/ObjectProperties.java create mode 100644 copycat/data/src/main/java/org/apache/kafka/copycat/data/Schema.java create mode 100644 copycat/data/src/main/java/org/apache/kafka/copycat/data/SchemaBuilder.java create mode 100644 copycat/data/src/main/java/org/apache/kafka/copycat/data/SchemaParseException.java create mode 100644 copycat/file/src/main/java/org/apache/kafka/copycat/file/FileStreamSinkConnector.java create mode 100644 copycat/file/src/main/java/org/apache/kafka/copycat/file/FileStreamSinkTask.java create mode 100644 copycat/file/src/main/java/org/apache/kafka/copycat/file/FileStreamSourceConnector.java create mode 100644 copycat/file/src/main/java/org/apache/kafka/copycat/file/FileStreamSourceTask.java create mode 100644 copycat/file/src/test/java/org/apache/kafka/copycat/file/FileStreamSinkConnectorTest.java create mode 100644 copycat/file/src/test/java/org/apache/kafka/copycat/file/FileStreamSinkTaskTest.java create mode 100644 copycat/file/src/test/java/org/apache/kafka/copycat/file/FileStreamSourceConnectorTest.java create mode 100644 copycat/file/src/test/java/org/apache/kafka/copycat/file/FileStreamSourceTaskTest.java create mode 100644 copycat/json/src/main/java/org/apache/kafka/copycat/json/JsonConverter.java create mode 100644 copycat/json/src/main/java/org/apache/kafka/copycat/json/JsonDeserializer.java create mode 100644 copycat/json/src/main/java/org/apache/kafka/copycat/json/JsonSchema.java create mode 100644 copycat/json/src/main/java/org/apache/kafka/copycat/json/JsonSerializer.java create mode 100644 copycat/json/src/test/java/org/apache/kafka/copycat/json/JsonConverterTest.java create mode 100644 copycat/runtime/src/main/java/org/apache/kafka/copycat/cli/CopycatStandalone.java create mode 100644 copycat/runtime/src/main/java/org/apache/kafka/copycat/cli/WorkerConfig.java create mode 100644 copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/ConnectorConfig.java create mode 100644 copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/Copycat.java create mode 100644 copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/Herder.java create mode 100644 copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/SinkTaskContextImpl.java create mode 100644 copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/SourceTaskOffsetCommitter.java create mode 100644 copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/Worker.java create mode 100644 copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/WorkerSinkTask.java create mode 100644 copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/WorkerSinkTaskThread.java create mode 100644 copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/WorkerSourceTask.java create mode 100644 copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/WorkerTask.java create mode 100644 copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/standalone/StandaloneConnectorContext.java create mode 100644 copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/standalone/StandaloneHerder.java create mode 100644 copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/FileOffsetBackingStore.java create mode 100644 copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/MemoryOffsetBackingStore.java create mode 100644 copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/OffsetBackingStore.java create mode 100644 copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/OffsetStorageReaderImpl.java create mode 100644 copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/OffsetStorageWriter.java create mode 100644 copycat/runtime/src/main/java/org/apache/kafka/copycat/util/Callback.java create mode 100644 copycat/runtime/src/main/java/org/apache/kafka/copycat/util/ConnectorTaskId.java create mode 100644 copycat/runtime/src/main/java/org/apache/kafka/copycat/util/FutureCallback.java create mode 100644 copycat/runtime/src/main/java/org/apache/kafka/copycat/util/ShutdownableThread.java create mode 100644 copycat/runtime/src/test/java/org/apache/kafka/copycat/runtime/WorkerSinkTaskTest.java create mode 100644 copycat/runtime/src/test/java/org/apache/kafka/copycat/runtime/WorkerSourceTaskTest.java create mode 100644 copycat/runtime/src/test/java/org/apache/kafka/copycat/runtime/WorkerTest.java create mode 100644 copycat/runtime/src/test/java/org/apache/kafka/copycat/runtime/standalone/StandaloneHerderTest.java create mode 100644 copycat/runtime/src/test/java/org/apache/kafka/copycat/storage/FileOffsetBackingStoreTest.java create mode 100644 copycat/runtime/src/test/java/org/apache/kafka/copycat/storage/OffsetStorageWriterTest.java create mode 100644 copycat/runtime/src/test/java/org/apache/kafka/copycat/util/MockTime.java create mode 100644 copycat/runtime/src/test/java/org/apache/kafka/copycat/util/ShutdownableThreadTest.java create mode 100644 copycat/runtime/src/test/java/org/apache/kafka/copycat/util/TestBackgroundThreadExceptionHandler.java create mode 100644 copycat/runtime/src/test/java/org/apache/kafka/copycat/util/ThreadedTest.java diff --git a/bin/copycat-standalone.sh b/bin/copycat-standalone.sh new file mode 100755 index 0000000..b219f8a --- /dev/null +++ b/bin/copycat-standalone.sh @@ -0,0 +1,23 @@ +#!/bin/sh +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +base_dir=$(dirname $0) + +if [ "x$KAFKA_LOG4J_OPTS" = "x" ]; then + export KAFKA_LOG4J_OPTS="-Dlog4j.configuration=file:$base_dir/../config/copycat-log4j.properties" +fi + +exec $(dirname $0)/kafka-run-class.sh org.apache.kafka.copycat.cli.CopycatStandalone "$@" diff --git a/bin/kafka-run-class.sh b/bin/kafka-run-class.sh index 2f00f68..b689b2e 100755 --- a/bin/kafka-run-class.sh +++ b/bin/kafka-run-class.sh @@ -66,6 +66,14 @@ do CLASSPATH=$CLASSPATH:$file done +for cc_pkg in "data" "api" "runtime" "file" "json" +do + for file in $base_dir/copycat/${cc_pkg}/build/libs/copycat-${cc_pkg}*.jar $base_dir/copycat/${cc_pkg}/build/dependant-libs/*.jar; + do + CLASSPATH=$CLASSPATH:$file + done +done + # classpath addition for release for file in $base_dir/libs/*.jar; do diff --git a/build.gradle b/build.gradle index 1b67e62..864427b 100644 --- a/build.gradle +++ b/build.gradle @@ -28,6 +28,11 @@ buildscript { } def slf4jlog4j='org.slf4j:slf4j-log4j12:1.7.6' +def slf4japi="org.slf4j:slf4j-api:1.7.6" +def junit='junit:junit:4.6' +def easymock='org.easymock:easymock:3.3.1' +def powermock='org.powermock:powermock-module-junit4:1.6.2' +def powermock_easymock='org.powermock:powermock-api-easymock:1.6.2' allprojects { apply plugin: 'idea' @@ -59,7 +64,7 @@ rat { // And some of the files that we have checked in should also be excluded from this check excludes.addAll([ '**/.git/**', - 'build/**', + '**/build/**', 'CONTRIBUTING.md', 'gradlew', 'gradlew.bat', @@ -204,20 +209,25 @@ for ( sv in ['2_10_5', '2_11_7'] ) { } } -tasks.create(name: "jarAll", dependsOn: ['jar_core_2_10_5', 'jar_core_2_11_7', 'clients:jar', 'examples:jar', 'contrib:hadoop-consumer:jar', 'contrib:hadoop-producer:jar', 'log4j-appender:jar', 'tools:jar']) { +def copycatPkgs = ['copycat:data', 'copycat:api', 'copycat:runtime', 'copycat:json', 'copycat:file'] +def pkgs = ['clients', 'examples', 'contrib:hadoop-consumer', 'contrib:hadoop-producer', 'log4j-appender', 'tools'] + copycatPkgs + +tasks.create(name: "jarCopycat", dependsOn: copycatPkgs.collect { it + ":jar" }) {} +tasks.create(name: "jarAll", dependsOn: ['jar_core_2_10_5', 'jar_core_2_11_7'] + pkgs.collect { it + ":jar" }) { } -tasks.create(name: "srcJarAll", dependsOn: ['srcJar_2_10_5', 'srcJar_2_11_7', 'clients:srcJar', 'examples:srcJar', 'contrib:hadoop-consumer:srcJar', 'contrib:hadoop-producer:srcJar', 'log4j-appender:srcJar', 'tools:srcJar']) { } +tasks.create(name: "srcJarAll", dependsOn: ['srcJar_2_10_5', 'srcJar_2_11_7'] + pkgs.collect { it + ":srcJar" }) { } -tasks.create(name: "docsJarAll", dependsOn: ['docsJar_2_10_5', 'docsJar_2_11_7', 'clients:docsJar', 'examples:docsJar', 'contrib:hadoop-consumer:docsJar', 'contrib:hadoop-producer:docsJar', 'log4j-appender:docsJar', 'tools:docsJar']) { } +tasks.create(name: "docsJarAll", dependsOn: ['docsJar_2_10_5', 'docsJar_2_11_7'] + pkgs.collect { it + ":docsJar" }) { } -tasks.create(name: "testAll", dependsOn: ['test_core_2_10_5', 'test_core_2_11_7', 'clients:test', 'log4j-appender:test', 'tools:test']) { +tasks.create(name: "testCopycat", dependsOn: copycatPkgs.collect { it + ":test" }) {} +tasks.create(name: "testAll", dependsOn: ['test_core_2_10_5', 'test_core_2_11_7'] + pkgs.collect { it + ":test" }) { } tasks.create(name: "releaseTarGzAll", dependsOn: ['releaseTarGz_2_10_5', 'releaseTarGz_2_11_7']) { } -tasks.create(name: "uploadArchivesAll", dependsOn: ['uploadCoreArchives_2_10_5', 'uploadCoreArchives_2_11_7', 'clients:uploadArchives', 'examples:uploadArchives', 'contrib:hadoop-consumer:uploadArchives', 'contrib:hadoop-producer:uploadArchives', 'log4j-appender:uploadArchives', 'tools:uploadArchives']) { +tasks.create(name: "uploadArchivesAll", dependsOn: ['uploadCoreArchives_2_10_5', 'uploadCoreArchives_2_11_7'] + pkgs.collect { it + ":uploadArchives" }) { } project(':core') { @@ -239,8 +249,8 @@ project(':core') { compile 'org.scala-lang.modules:scala-parser-combinators_2.11:1.0.4' } - testCompile 'junit:junit:4.6' - testCompile 'org.easymock:easymock:3.0' + testCompile "$junit" + testCompile "$easymock" testCompile 'org.objenesis:objenesis:1.2' testCompile "org.scalatest:scalatest_$baseScalaVersion:2.2.5" @@ -371,11 +381,11 @@ project(':clients') { archivesBaseName = "kafka-clients" dependencies { - compile "org.slf4j:slf4j-api:1.7.6" + compile "$slf4japi" compile 'org.xerial.snappy:snappy-java:1.1.1.7' compile 'net.jpountz.lz4:lz4:1.2.0' - testCompile 'junit:junit:4.6' + testCompile "$junit" testRuntime "$slf4jlog4j" } @@ -423,7 +433,7 @@ project(':tools') { compile 'com.fasterxml.jackson.core:jackson-databind:2.5.4' compile "$slf4jlog4j" - testCompile 'junit:junit:4.6' + testCompile "$junit" testCompile project(path: ':clients', configuration: 'archives') } @@ -471,7 +481,7 @@ project(':log4j-appender') { compile project(':clients') compile "$slf4jlog4j" - testCompile 'junit:junit:4.6' + testCompile "$junit" testCompile project(path: ':clients', configuration: 'archives') } @@ -496,3 +506,238 @@ project(':log4j-appender') { } test.dependsOn('checkstyleMain', 'checkstyleTest') } + +project(':copycat:data') { + apply plugin: 'checkstyle' + archivesBaseName = "copycat-data" + + dependencies { + compile project(':clients') + compile "$slf4japi" + + testCompile "$junit" + testRuntime "$slf4jlog4j" + } + + task testJar(type: Jar) { + classifier = 'test' + from sourceSets.test.output + } + + test { + testLogging { + events "passed", "skipped", "failed" + exceptionFormat = 'full' + } + } + + javadoc { + include "**/org/apache/kafka/copycat/data/*" + } + + artifacts { + archives testJar + } + + configurations { + archives.extendsFrom (testCompile) + } + + /* FIXME Re-enable this with KAFKA-2367 when the placeholder data API is replaced + checkstyle { + configFile = new File(rootDir, "checkstyle/checkstyle.xml") + } + test.dependsOn('checkstyleMain', 'checkstyleTest') */ +} + +project(':copycat:api') { + apply plugin: 'checkstyle' + archivesBaseName = "copycat-api" + + dependencies { + compile project(':copycat:data') + compile "$slf4japi" + + testCompile "$junit" + testRuntime "$slf4jlog4j" + } + + task testJar(type: Jar) { + classifier = 'test' + from sourceSets.test.output + } + + test { + testLogging { + events "passed", "skipped", "failed" + exceptionFormat = 'full' + } + } + + javadoc { + include "**/org/apache/kafka/copycat/*" + } + + artifacts { + archives testJar + } + + configurations { + archives.extendsFrom (testCompile) + } + + checkstyle { + configFile = new File(rootDir, "checkstyle/checkstyle.xml") + } + test.dependsOn('checkstyleMain', 'checkstyleTest') +} + +project(':copycat:json') { + apply plugin: 'checkstyle' + archivesBaseName = "copycat-json" + + dependencies { + compile project(':copycat:api') + compile "$slf4japi" + compile 'com.fasterxml.jackson.core:jackson-databind:2.5.4' + + testCompile "$junit" + testCompile "$easymock" + testCompile "$powermock" + testCompile "$powermock_easymock" + testRuntime "$slf4jlog4j" + } + + task testJar(type: Jar) { + classifier = 'test' + from sourceSets.test.output + } + + test { + testLogging { + events "passed", "skipped", "failed" + exceptionFormat = 'full' + } + } + + javadoc { + include "**/org/apache/kafka/copycat/*" + } + + artifacts { + archives testJar + } + + configurations { + archives.extendsFrom(testCompile) + } + + checkstyle { + configFile = new File(rootDir, "checkstyle/checkstyle.xml") + } + test.dependsOn('checkstyleMain', 'checkstyleTest') + + tasks.create(name: "copyDependantLibs", type: Copy) { + from (configurations.runtime) { + exclude('kafka-clients*') + exclude('copycat-*') + } + into "$buildDir/dependant-libs" + } + + jar { + dependsOn copyDependantLibs + } +} + +project(':copycat:runtime') { + apply plugin: 'checkstyle' + archivesBaseName = "copycat-runtime" + + dependencies { + compile project(':copycat:api') + compile project(':clients') + compile "$slf4japi" + + testCompile "$junit" + testCompile "$easymock" + testCompile "$powermock" + testCompile "$powermock_easymock" + testRuntime "$slf4jlog4j" + testRuntime project(":copycat:json") + } + + task testJar(type: Jar) { + classifier = 'test' + from sourceSets.test.output + } + + test { + testLogging { + events "passed", "skipped", "failed" + exceptionFormat = 'full' + } + } + + javadoc { + include "**/org/apache/kafka/copycat/*" + } + + artifacts { + archives testJar + } + + configurations { + archives.extendsFrom(testCompile) + } + + checkstyle { + configFile = new File(rootDir, "checkstyle/checkstyle.xml") + } + test.dependsOn('checkstyleMain', 'checkstyleTest') +} + +project(':copycat:file') { + apply plugin: 'checkstyle' + archivesBaseName = "copycat-file" + + dependencies { + compile project(':copycat:api') + compile "$slf4japi" + + testCompile "$junit" + testCompile "$easymock" + testCompile "$powermock" + testCompile "$powermock_easymock" + testRuntime "$slf4jlog4j" + } + + task testJar(type: Jar) { + classifier = 'test' + from sourceSets.test.output + } + + test { + testLogging { + events "passed", "skipped", "failed" + exceptionFormat = 'full' + } + } + + javadoc { + include "**/org/apache/kafka/copycat/*" + } + + artifacts { + archives testJar + } + + configurations { + archives.extendsFrom(testCompile) + } + + checkstyle { + configFile = new File(rootDir, "checkstyle/checkstyle.xml") + } + test.dependsOn('checkstyleMain', 'checkstyleTest') +} diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 18be1bb..e3f4f84 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -108,4 +108,61 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java index af9993c..80a914e 100755 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -271,17 +271,30 @@ public class Utils { /** * Instantiate the class */ - public static Object newInstance(Class c) { + public static T newInstance(Class c) { try { return c.newInstance(); } catch (IllegalAccessException e) { throw new KafkaException("Could not instantiate class " + c.getName(), e); } catch (InstantiationException e) { throw new KafkaException("Could not instantiate class " + c.getName() + " Does it have a public no-argument constructor?", e); + } catch (NullPointerException e) { + throw new KafkaException("Requested class was null", e); } } /** + * Look up the class by name and instantiate it. + * @param klass class name + * @param base super class of the class to be instantiated + * @param + * @return the new instance + */ + public static T newInstance(String klass, Class base) throws ClassNotFoundException { + return Utils.newInstance(Class.forName(klass).asSubclass(base)); + } + + /** * Generates 32 bit murmur2 hash from byte array * @param data byte array to hash * @return 32 bit hash of the given array diff --git a/config/copycat-console-sink.properties b/config/copycat-console-sink.properties new file mode 100644 index 0000000..4cd4c33 --- /dev/null +++ b/config/copycat-console-sink.properties @@ -0,0 +1,19 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name=local-console-sink +connector.class=org.apache.kafka.copycat.file.FileStreamSinkConnector +tasks.max=1 +topics=test \ No newline at end of file diff --git a/config/copycat-console-source.properties b/config/copycat-console-source.properties new file mode 100644 index 0000000..17dbbf9 --- /dev/null +++ b/config/copycat-console-source.properties @@ -0,0 +1,19 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name=local-console-source +connector.class=org.apache.kafka.copycat.file.FileStreamSourceConnector +tasks.max=1 +topic=test \ No newline at end of file diff --git a/config/copycat-file-sink.properties b/config/copycat-file-sink.properties new file mode 100644 index 0000000..3cc0d62 --- /dev/null +++ b/config/copycat-file-sink.properties @@ -0,0 +1,20 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name=local-file-sink +connector.class=org.apache.kafka.copycat.file.FileStreamSinkConnector +tasks.max=1 +file=test.sink.txt +topics=test \ No newline at end of file diff --git a/config/copycat-file-source.properties b/config/copycat-file-source.properties new file mode 100644 index 0000000..7512e50 --- /dev/null +++ b/config/copycat-file-source.properties @@ -0,0 +1,20 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name=local-file-source +connector.class=org.apache.kafka.copycat.file.FileStreamSourceConnector +tasks.max=1 +file=test.txt +topic=test \ No newline at end of file diff --git a/config/copycat-log4j.properties b/config/copycat-log4j.properties new file mode 100644 index 0000000..158daed --- /dev/null +++ b/config/copycat-log4j.properties @@ -0,0 +1,23 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +log4j.rootLogger=INFO, stdout + +log4j.appender.stdout=org.apache.log4j.ConsoleAppender +log4j.appender.stdout.layout=org.apache.log4j.PatternLayout +log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c:%L)%n + +log4j.logger.org.apache.zookeeper=ERROR +log4j.logger.org.I0Itec.zkclient=ERROR \ No newline at end of file diff --git a/config/copycat-standalone.properties b/config/copycat-standalone.properties new file mode 100644 index 0000000..cf3b268 --- /dev/null +++ b/config/copycat-standalone.properties @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# These are defaults. This file just demonstrates how to override some settings. +bootstrap.servers=localhost:9092 + +key.converter=org.apache.kafka.copycat.json.JsonConverter +value.converter=org.apache.kafka.copycat.json.JsonConverter +key.serializer=org.apache.kafka.copycat.json.JsonSerializer +value.serializer=org.apache.kafka.copycat.json.JsonSerializer +key.deserializer=org.apache.kafka.copycat.json.JsonDeserializer +value.deserializer=org.apache.kafka.copycat.json.JsonDeserializer + +offset.storage.file.filename=/tmp/copycat.offsets +# Flush much faster than normal, which is useful for testing/debugging +offset.flush.interval.ms=10000 diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/connector/Connector.java b/copycat/api/src/main/java/org/apache/kafka/copycat/connector/Connector.java new file mode 100644 index 0000000..2ea3c95 --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/connector/Connector.java @@ -0,0 +1,117 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.connector; + +import org.apache.kafka.common.annotation.InterfaceStability; + +import java.util.List; +import java.util.Properties; + +/** + *

    + * Connectors manage integration of Copycat with another system, either as an input that ingests + * data into Kafka or an output that passes data to an external system. Implementations should + * not use this class directly; they should inherit from SourceConnector or SinkConnector. + *

    + *

    + * Connectors have two primary tasks. First, given some configuration, they are responsible for + * creating configurations for a set of {@link Task}s that split up the data processing. For + * example, a database Connector might create Tasks by dividing the set of tables evenly among + * tasks. Second, they are responsible for monitoring inputs for changes that require + * reconfiguration and notifying the Copycat runtime via the ConnectorContext. Continuing the + * previous example, the connector might periodically check for new tables and notify Copycat of + * additions and deletions. Copycat will then request new configurations and update the running + * Tasks. + *

    + */ +@InterfaceStability.Unstable +public abstract class Connector { + + protected ConnectorContext context; + + /** + * Initialize this connector, using the provided ConnectorContext to notify the runtime of + * input configuration changes. + * @param ctx context object used to interact with the Copycat runtime + */ + public void initialize(ConnectorContext ctx) { + context = ctx; + } + + /** + *

    + * Initialize this connector, using the provided ConnectorContext to notify the runtime of + * input configuration changes and using the provided set of Task configurations. + * This version is only used to recover from failures. + *

    + *

    + * The default implementation ignores the provided Task configurations. During recovery, Copycat will request + * an updated set of configurations and update the running Tasks appropriately. However, Connectors should + * implement special handling of this case if it will avoid unnecessary changes to running Tasks. + *

    + * + * @param ctx context object used to interact with the Copycat runtime + * @param taskConfigs existing task configurations, which may be used when generating new task configs to avoid + * churn in partition to task assignments + */ + public void initialize(ConnectorContext ctx, List taskConfigs) { + context = ctx; + // Ignore taskConfigs. May result in more churn of tasks during recovery if updated configs + // are very different, but reduces the difficulty of implementing a Connector + } + + /** + * Start this Connector. This method will only be called on a clean Connector, i.e. it has + * either just been instantiated and initialized or {@link #stop()} has been invoked. + * + * @param props configuration settings + */ + public abstract void start(Properties props); + + /** + * Reconfigure this Connector. Most implementations will not override this, using the default + * implementation that calls {@link #stop()} followed by {@link #start(Properties)}. + * Implementations only need to override this if they want to handle this process more + * efficiently, e.g. without shutting down network connections to the external system. + * + * @param props new configuration settings + */ + public void reconfigure(Properties props) { + stop(); + start(props); + } + + /** + * Returns the Task implementation for this Connector. + */ + public abstract Class getTaskClass(); + + /** + * Returns a set of configurations for Tasks based on the current configuration, + * producing at most count configurations. + * + * @param maxTasks maximum number of configurations to generate + * @return configurations for Tasks + */ + public abstract List getTaskConfigs(int maxTasks); + + /** + * Stop this connector. + */ + public abstract void stop(); +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/connector/ConnectorContext.java b/copycat/api/src/main/java/org/apache/kafka/copycat/connector/ConnectorContext.java new file mode 100644 index 0000000..ecba69a --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/connector/ConnectorContext.java @@ -0,0 +1,33 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.connector; + +import org.apache.kafka.common.annotation.InterfaceStability; + +/** + * ConnectorContext allows Connectors to proactively interact with the Copycat runtime. + */ +@InterfaceStability.Unstable +public interface ConnectorContext { + /** + * Requests that the runtime reconfigure the Tasks for this source. This should be used to + * indicate to the runtime that something about the input/output has changed (e.g. partitions + * added/removed) and the running Tasks will need to be modified. + */ + void requestTaskReconfiguration(); +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/connector/CopycatRecord.java b/copycat/api/src/main/java/org/apache/kafka/copycat/connector/CopycatRecord.java new file mode 100644 index 0000000..576904a --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/connector/CopycatRecord.java @@ -0,0 +1,103 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.connector; + +import org.apache.kafka.common.annotation.InterfaceStability; + +/** + *

    + * Base class for records containing data to be copied to/from Kafka. This corresponds closely to + * Kafka's ProducerRecord and ConsumerRecord classes, and holds the data that may be used by both + * sources and sinks (topic, kafkaPartition, key, value). Although both implementations include a + * notion of offset, it is not included here because they differ in type. + *

    + */ +@InterfaceStability.Unstable +public abstract class CopycatRecord { + private final String topic; + private final Integer kafkaPartition; + private final Object key; + private final Object value; + + public CopycatRecord(String topic, Integer kafkaPartition, Object value) { + this(topic, kafkaPartition, null, value); + } + + public CopycatRecord(String topic, Integer kafkaPartition, Object key, Object value) { + this.topic = topic; + this.kafkaPartition = kafkaPartition; + this.key = key; + this.value = value; + } + + public String getTopic() { + return topic; + } + + public Integer getKafkaPartition() { + return kafkaPartition; + } + + public Object getKey() { + return key; + } + + public Object getValue() { + return value; + } + + @Override + public String toString() { + return "CopycatRecord{" + + "topic='" + topic + '\'' + + ", kafkaPartition=" + kafkaPartition + + ", key=" + key + + ", value=" + value + + '}'; + } + + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + CopycatRecord that = (CopycatRecord) o; + + if (key != null ? !key.equals(that.key) : that.key != null) + return false; + if (kafkaPartition != null ? !kafkaPartition.equals(that.kafkaPartition) : that.kafkaPartition != null) + return false; + if (topic != null ? !topic.equals(that.topic) : that.topic != null) + return false; + if (value != null ? !value.equals(that.value) : that.value != null) + return false; + + return true; + } + + @Override + public int hashCode() { + int result = topic != null ? topic.hashCode() : 0; + result = 31 * result + (kafkaPartition != null ? kafkaPartition.hashCode() : 0); + result = 31 * result + (key != null ? key.hashCode() : 0); + result = 31 * result + (value != null ? value.hashCode() : 0); + return result; + } +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/connector/Task.java b/copycat/api/src/main/java/org/apache/kafka/copycat/connector/Task.java new file mode 100644 index 0000000..cdaba08 --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/connector/Task.java @@ -0,0 +1,49 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.connector; + +import org.apache.kafka.common.annotation.InterfaceStability; + +import java.util.Properties; + +/** + *

    + * Tasks contain the code that actually copies data to/from another system. They receive + * a configuration from their parent Connector, assigning them a fraction of a Copycat job's work. + * The Copycat framework then pushes/pulls data from the Task. The Task must also be able to + * respond to reconfiguration requests. + *

    + *

    + * Task only contains the minimal shared functionality between + * {@link org.apache.kafka.copycat.source.SourceTask} and + * {@link org.apache.kafka.copycat.sink.SinkTask}. + *

    + */ +@InterfaceStability.Unstable +public interface Task { + /** + * Start the Task + * @param props initial configuration + */ + void start(Properties props); + + /** + * Stop this task. + */ + void stop(); +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/errors/CopycatException.java b/copycat/api/src/main/java/org/apache/kafka/copycat/errors/CopycatException.java new file mode 100644 index 0000000..c8f1bad --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/errors/CopycatException.java @@ -0,0 +1,40 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.errors; + +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.annotation.InterfaceStability; + +/** + * CopycatException is the top-level exception type generated by Copycat and connectors. + */ +@InterfaceStability.Unstable +public class CopycatException extends KafkaException { + + public CopycatException(String s) { + super(s); + } + + public CopycatException(String s, Throwable throwable) { + super(s, throwable); + } + + public CopycatException(Throwable throwable) { + super(throwable); + } +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/sink/SinkConnector.java b/copycat/api/src/main/java/org/apache/kafka/copycat/sink/SinkConnector.java new file mode 100644 index 0000000..fb2e694 --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/sink/SinkConnector.java @@ -0,0 +1,40 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.sink; + +import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.copycat.connector.Connector; + +/** + * SinkConnectors implement the Connector interface to send Kafka data to another system. + */ +@InterfaceStability.Unstable +public abstract class SinkConnector extends Connector { + + /** + *

    + * Configuration key for the list of input topics for this connector. + *

    + *

    + * Usually this setting is only relevant to the Copycat framework, but is provided here for + * the convenience of Connector developers if they also need to know the set of topics. + *

    + */ + public static final String TOPICS_CONFIG = "topics"; + +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/sink/SinkRecord.java b/copycat/api/src/main/java/org/apache/kafka/copycat/sink/SinkRecord.java new file mode 100644 index 0000000..e3775b3 --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/sink/SinkRecord.java @@ -0,0 +1,71 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.sink; + +import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.copycat.connector.CopycatRecord; + +/** + * SinkRecord is a CopycatRecord that has been read from Kafka and includes the kafkaOffset of + * the record in the Kafka topic-partition in addition to the standard fields. This information + * should be used by the SinkTask to coordinate kafkaOffset commits. + */ +@InterfaceStability.Unstable +public class SinkRecord extends CopycatRecord { + private final long kafkaOffset; + + public SinkRecord(String topic, int partition, Object key, Object value, long kafkaOffset) { + super(topic, partition, key, value); + this.kafkaOffset = kafkaOffset; + } + + public long getKafkaOffset() { + return kafkaOffset; + } + + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + if (!super.equals(o)) + return false; + + SinkRecord that = (SinkRecord) o; + + if (kafkaOffset != that.kafkaOffset) + return false; + + return true; + } + + @Override + public int hashCode() { + int result = super.hashCode(); + result = 31 * result + (int) (kafkaOffset ^ (kafkaOffset >>> 32)); + return result; + } + + @Override + public String toString() { + return "SinkRecord{" + + "kafkaOffset=" + kafkaOffset + + "} " + super.toString(); + } +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/sink/SinkTask.java b/copycat/api/src/main/java/org/apache/kafka/copycat/sink/SinkTask.java new file mode 100644 index 0000000..49fbbe9 --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/sink/SinkTask.java @@ -0,0 +1,64 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ +package org.apache.kafka.copycat.sink; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.copycat.connector.Task; + +import java.util.Collection; +import java.util.Map; + +/** + * SinkTask is a Task takes records loaded from Kafka and sends them to another system. In + * addition to the basic {@link #put} interface, SinkTasks must also implement {@link #flush} + * to support offset commits. + */ +@InterfaceStability.Unstable +public abstract class SinkTask implements Task { + + /** + *

    + * The configuration key that provides the list of topics that are inputs for this + * SinkTask. + *

    + */ + public static final String TOPICS_CONFIG = "topics"; + + protected SinkTaskContext context; + + public void initialize(SinkTaskContext context) { + this.context = context; + } + + /** + * Put the records in the sink. Usually this should send the records to the sink asynchronously + * and immediately return. + * + * @param records the set of records to send + */ + public abstract void put(Collection records); + + /** + * Flush all records that have been {@link #put} for the specified topic-partitions. The + * offsets are provided for convenience, but could also be determined by tracking all offsets + * included in the SinkRecords passed to {@link #put}. + * + * @param offsets mapping of TopicPartition to committed offset + */ + public abstract void flush(Map offsets); +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/sink/SinkTaskContext.java b/copycat/api/src/main/java/org/apache/kafka/copycat/sink/SinkTaskContext.java new file mode 100644 index 0000000..7cc6109 --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/sink/SinkTaskContext.java @@ -0,0 +1,59 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.sink; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.annotation.InterfaceStability; + +import java.util.HashMap; +import java.util.Map; + +/** + * Context passed to SinkTasks, allowing them to access utilities in the copycat runtime. + */ +@InterfaceStability.Unstable +public abstract class SinkTaskContext { + private Map offsets; + + public SinkTaskContext() { + offsets = new HashMap<>(); + } + + /** + * Reset the consumer offsets for the given topic partitions. SinkTasks should use this when they are started + * if they manage offsets in the sink data store rather than using Kafka consumer offsets. For example, an HDFS + * connector might record offsets in HDFS to provide exactly once delivery. When the SinkTask is started or + * a rebalance occurs, the task would reload offsets from HDFS and use this method to reset the consumer to those + * offsets. + * + * SinkTasks that do not manage their own offsets do not need to use this method. + * + * @param offsets map of offsets for topic partitions + */ + public void resetOffset(Map offsets) { + this.offsets = offsets; + } + + /** + * Get offsets that the SinkTask has submitted to be reset. Used by the Copycat framework. + * @return the map of offsets + */ + public Map getOffsets() { + return offsets; + } +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/source/SourceConnector.java b/copycat/api/src/main/java/org/apache/kafka/copycat/source/SourceConnector.java new file mode 100644 index 0000000..7258cdf --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/source/SourceConnector.java @@ -0,0 +1,29 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ +package org.apache.kafka.copycat.source; + +import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.copycat.connector.Connector; + +/** + * SourceConnectors implement the connector interface to pull data from another system and send + * it to Kafka. + */ +@InterfaceStability.Unstable +public abstract class SourceConnector extends Connector { + +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/source/SourceRecord.java b/copycat/api/src/main/java/org/apache/kafka/copycat/source/SourceRecord.java new file mode 100644 index 0000000..2085f66 --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/source/SourceRecord.java @@ -0,0 +1,103 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.source; + +import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.copycat.connector.CopycatRecord; + +/** + *

    + * SourceRecords are generated by SourceTasks and passed to Copycat for storage in + * Kafka. In addition to the standard fields in CopycatRecord which specify where data is stored + * in Kafka, they also include a sourcePartition and sourceOffset. + *

    + *

    + * The sourcePartition represents a single input sourcePartition that the record came from (e.g. a filename, table + * name, or topic-partition). The sourceOffset represents a position in that sourcePartition which can be used + * to resume consumption of data. + *

    + *

    + * These values can have arbitrary structure and should be represented using + * org.apache.kafka.copycat.data objects (or primitive values). For example, a database connector + * might specify the sourcePartition as a record containing { "db": "database_name", "table": + * "table_name"} and the sourceOffset as a Long containing the timestamp of the row. + *

    + */ +@InterfaceStability.Unstable +public class SourceRecord extends CopycatRecord { + private final Object sourcePartition; + private final Object sourceOffset; + + public SourceRecord(Object sourcePartition, Object sourceOffset, String topic, Integer partition, Object value) { + this(sourcePartition, sourceOffset, topic, partition, null, value); + } + + public SourceRecord(Object sourcePartition, Object sourceOffset, String topic, Object value) { + this(sourcePartition, sourceOffset, topic, null, null, value); + } + + public SourceRecord(Object sourcePartition, Object sourceOffset, String topic, Integer partition, + Object key, Object value) { + super(topic, partition, key, value); + this.sourcePartition = sourcePartition; + this.sourceOffset = sourceOffset; + } + + public Object getSourcePartition() { + return sourcePartition; + } + + public Object getSourceOffset() { + return sourceOffset; + } + + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + if (!super.equals(o)) + return false; + + SourceRecord that = (SourceRecord) o; + + if (sourceOffset != null ? !sourceOffset.equals(that.sourceOffset) : that.sourceOffset != null) + return false; + if (sourcePartition != null ? !sourcePartition.equals(that.sourcePartition) : that.sourcePartition != null) + return false; + + return true; + } + + @Override + public int hashCode() { + int result = super.hashCode(); + result = 31 * result + (sourcePartition != null ? sourcePartition.hashCode() : 0); + result = 31 * result + (sourceOffset != null ? sourceOffset.hashCode() : 0); + return result; + } + + @Override + public String toString() { + return "SourceRecord{" + + "sourcePartition=" + sourcePartition + + ", sourceOffset=" + sourceOffset + + "} " + super.toString(); + } +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/source/SourceTask.java b/copycat/api/src/main/java/org/apache/kafka/copycat/source/SourceTask.java new file mode 100644 index 0000000..1e1da34 --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/source/SourceTask.java @@ -0,0 +1,62 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.source; + +import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.copycat.connector.Task; + +import java.util.List; + +/** + * SourceTask is a Task that pulls records from another system for storage in Kafka. + */ +@InterfaceStability.Unstable +public abstract class SourceTask implements Task { + + protected SourceTaskContext context; + + /** + * Initialize this SourceTask with the specified context object. + */ + public void initialize(SourceTaskContext context) { + this.context = context; + } + + /** + * Poll this SourceTask for new records. This method should block if no data is currently + * available. + * + * @return a list of source records + */ + public abstract List poll() throws InterruptedException; + + /** + *

    + * Commit the offsets, up to the offsets that have been returned by {@link #poll()}. This + * method should block until the commit is complete. + *

    + *

    + * SourceTasks are not required to implement this functionality; Copycat will record offsets + * automatically. This hook is provided for systems that also need to store offsets internally + * in their own system. + *

    + */ + public void commit() throws InterruptedException { + // This space intentionally left blank. + } +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/source/SourceTaskContext.java b/copycat/api/src/main/java/org/apache/kafka/copycat/source/SourceTaskContext.java new file mode 100644 index 0000000..d52fd62 --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/source/SourceTaskContext.java @@ -0,0 +1,40 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ +package org.apache.kafka.copycat.source; + +import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.copycat.storage.OffsetStorageReader; + +/** + * SourceTaskContext is provided to SourceTasks to allow them to interact with the underlying + * runtime. + */ +@InterfaceStability.Unstable +public class SourceTaskContext { + private final OffsetStorageReader reader; + + public SourceTaskContext(OffsetStorageReader reader) { + this.reader = reader; + } + + /** + * Get the OffsetStorageReader for this SourceTask. + */ + public OffsetStorageReader getOffsetStorageReader() { + return reader; + } +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/storage/Converter.java b/copycat/api/src/main/java/org/apache/kafka/copycat/storage/Converter.java new file mode 100644 index 0000000..c50aee7 --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/storage/Converter.java @@ -0,0 +1,45 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.storage; + +import org.apache.kafka.common.annotation.InterfaceStability; + +/** + * The Converter interface provides support for translating between Copycat's runtime data format + * and the "native" runtime format used by the serialization layer. This is used to translate + * two types of data: records and offsets. The (de)serialization is performed by a separate + * component -- the producer or consumer serializer or deserializer for records or a Copycat + * serializer or deserializer for offsets. + */ +@InterfaceStability.Unstable +public interface Converter { + + /** + * Convert a Copycat data object to a native object for serialization. + * @param value + * @return + */ + T fromCopycatData(Object value); + + /** + * Convert a native object to a Copycat data object. + * @param value + * @return + */ + Object toCopycatData(T value); +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/storage/OffsetStorageReader.java b/copycat/api/src/main/java/org/apache/kafka/copycat/storage/OffsetStorageReader.java new file mode 100644 index 0000000..785660d --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/storage/OffsetStorageReader.java @@ -0,0 +1,59 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.storage; + +import org.apache.kafka.common.annotation.InterfaceStability; + +import java.util.Collection; +import java.util.Map; + +/** + * OffsetStorageReader provides access to the offset storage used by sources. This can be used by + * connectors to determine offsets to start consuming data from. This is most commonly used during + * initialization of a task, but can also be used during runtime, e.g. when reconfiguring a task. + */ +@InterfaceStability.Unstable +public interface OffsetStorageReader { + /** + * Get the offset for the specified partition. If the data isn't already available locally, this + * gets it from the backing store, which may require some network round trips. + * + * @param partition object uniquely identifying the partition of data + * @return object uniquely identifying the offset in the partition of data + */ + Object getOffset(Object partition); + + /** + *

    + * Get a set of offsets for the specified partition identifiers. This may be more efficient + * than calling {@link #getOffset(Object)} repeatedly. + *

    + *

    + * Note that when errors occur, this method omits the associated data and tries to return as + * many of the requested values as possible. This allows a task that's managing many partitions to + * still proceed with any available data. Therefore, implementations should take care to check + * that the data is actually available in the returned response. The only case when an + * exception will be thrown is if the entire request failed, e.g. because the underlying + * storage was unavailable. + *

    + * + * @param partitions set of identifiers for partitions of data + * @return a map of partition identifiers to decoded offsets + */ + Map getOffsets(Collection partitions); +} diff --git a/copycat/api/src/main/java/org/apache/kafka/copycat/util/ConnectorUtils.java b/copycat/api/src/main/java/org/apache/kafka/copycat/util/ConnectorUtils.java new file mode 100644 index 0000000..f9dd53a --- /dev/null +++ b/copycat/api/src/main/java/org/apache/kafka/copycat/util/ConnectorUtils.java @@ -0,0 +1,66 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.util; + +import org.apache.kafka.common.annotation.InterfaceStability; + +import java.util.ArrayList; +import java.util.List; + +/** + * Utilities that connector implementations might find useful. Contains common building blocks + * for writing connectors. + */ +@InterfaceStability.Unstable +public class ConnectorUtils { + /** + * Given a list of elements and a target number of groups, generates list of groups of + * elements to match the target number of groups, spreading them evenly among the groups. + * This generates groups with contiguous elements, which results in intuitive ordering if + * your elements are also ordered (e.g. alphabetical lists of table names if you sort + * table names alphabetically to generate the raw partitions) or can result in efficient + * partitioning if elements are sorted according to some criteria that affects performance + * (e.g. topic partitions with the same leader). + * + * @param elements list of elements to partition + * @param numGroups the number of output groups to generate. + */ + public static List> groupPartitions(List elements, int numGroups) { + if (numGroups <= 0) + throw new IllegalArgumentException("Number of groups must be positive."); + + List> result = new ArrayList<>(numGroups); + + // Each group has either n+1 or n raw partitions + int perGroup = elements.size() / numGroups; + int leftover = elements.size() - (numGroups * perGroup); + + int assigned = 0; + for (int group = 0; group < numGroups; group++) { + int numThisGroup = group < leftover ? perGroup + 1 : perGroup; + List groupList = new ArrayList<>(numThisGroup); + for (int i = 0; i < numThisGroup; i++) { + groupList.add(elements.get(assigned)); + assigned++; + } + result.add(groupList); + } + + return result; + } +} diff --git a/copycat/api/src/test/java/org/apache/kafka/copycat/connector/ConnectorReconfigurationTest.java b/copycat/api/src/test/java/org/apache/kafka/copycat/connector/ConnectorReconfigurationTest.java new file mode 100644 index 0000000..e7ad2f3 --- /dev/null +++ b/copycat/api/src/test/java/org/apache/kafka/copycat/connector/ConnectorReconfigurationTest.java @@ -0,0 +1,76 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.connector; + +import org.apache.kafka.copycat.errors.CopycatException; +import org.junit.Test; + +import java.util.List; +import java.util.Properties; + +import static org.junit.Assert.assertEquals; + +public class ConnectorReconfigurationTest { + + @Test + public void testDefaultReconfigure() throws Exception { + TestConnector conn = new TestConnector(false); + conn.reconfigure(new Properties()); + assertEquals(conn.stopOrder, 0); + assertEquals(conn.configureOrder, 1); + } + + @Test(expected = CopycatException.class) + public void testReconfigureStopException() throws Exception { + TestConnector conn = new TestConnector(true); + conn.reconfigure(new Properties()); + } + + private static class TestConnector extends Connector { + private boolean stopException; + private int order = 0; + public int stopOrder = -1; + public int configureOrder = -1; + + public TestConnector(boolean stopException) { + this.stopException = stopException; + } + + @Override + public void start(Properties props) { + configureOrder = order++; + } + + @Override + public Class getTaskClass() { + return null; + } + + @Override + public List getTaskConfigs(int count) { + return null; + } + + @Override + public void stop() { + stopOrder = order++; + if (stopException) + throw new CopycatException("error"); + } + } +} \ No newline at end of file diff --git a/copycat/api/src/test/java/org/apache/kafka/copycat/util/ConnectorUtilsTest.java b/copycat/api/src/test/java/org/apache/kafka/copycat/util/ConnectorUtilsTest.java new file mode 100644 index 0000000..e46967b --- /dev/null +++ b/copycat/api/src/test/java/org/apache/kafka/copycat/util/ConnectorUtilsTest.java @@ -0,0 +1,67 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.util; + +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static org.junit.Assert.assertEquals; + +public class ConnectorUtilsTest { + + private static final List FIVE_ELEMENTS = Arrays.asList(1, 2, 3, 4, 5); + + @Test + public void testGroupPartitions() { + + List> grouped = ConnectorUtils.groupPartitions(FIVE_ELEMENTS, 1); + assertEquals(Arrays.asList(FIVE_ELEMENTS), grouped); + + grouped = ConnectorUtils.groupPartitions(FIVE_ELEMENTS, 2); + assertEquals(Arrays.asList(Arrays.asList(1, 2, 3), Arrays.asList(4, 5)), grouped); + + grouped = ConnectorUtils.groupPartitions(FIVE_ELEMENTS, 3); + assertEquals(Arrays.asList(Arrays.asList(1, 2), + Arrays.asList(3, 4), + Arrays.asList(5)), grouped); + + grouped = ConnectorUtils.groupPartitions(FIVE_ELEMENTS, 5); + assertEquals(Arrays.asList(Arrays.asList(1), + Arrays.asList(2), + Arrays.asList(3), + Arrays.asList(4), + Arrays.asList(5)), grouped); + + grouped = ConnectorUtils.groupPartitions(FIVE_ELEMENTS, 7); + assertEquals(Arrays.asList(Arrays.asList(1), + Arrays.asList(2), + Arrays.asList(3), + Arrays.asList(4), + Arrays.asList(5), + Collections.EMPTY_LIST, + Collections.EMPTY_LIST), grouped); + } + + @Test(expected = IllegalArgumentException.class) + public void testGroupPartitionsInvalidCount() { + ConnectorUtils.groupPartitions(FIVE_ELEMENTS, 0); + } +} diff --git a/copycat/data/src/main/java/org/apache/kafka/copycat/data/DataRuntimeException.java b/copycat/data/src/main/java/org/apache/kafka/copycat/data/DataRuntimeException.java new file mode 100644 index 0000000..855c0fd --- /dev/null +++ b/copycat/data/src/main/java/org/apache/kafka/copycat/data/DataRuntimeException.java @@ -0,0 +1,36 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + + + +package org.apache.kafka.copycat.data; + +/** Base Avro exception. */ +public class DataRuntimeException extends RuntimeException { + public DataRuntimeException(Throwable cause) { + super(cause); + } + + public DataRuntimeException(String message) { + super(message); + } + + public DataRuntimeException(String message, Throwable cause) { + super(message, cause); + } +} + diff --git a/copycat/data/src/main/java/org/apache/kafka/copycat/data/DataTypeException.java b/copycat/data/src/main/java/org/apache/kafka/copycat/data/DataTypeException.java new file mode 100644 index 0000000..6a74d88 --- /dev/null +++ b/copycat/data/src/main/java/org/apache/kafka/copycat/data/DataTypeException.java @@ -0,0 +1,33 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + + + +package org.apache.kafka.copycat.data; + + +/** Thrown when an illegal type is used. */ +public class DataTypeException extends DataRuntimeException { + public DataTypeException(String message) { + super(message); + } + + public DataTypeException(String message, Throwable cause) { + super(message, cause); + } +} + diff --git a/copycat/data/src/main/java/org/apache/kafka/copycat/data/ObjectProperties.java b/copycat/data/src/main/java/org/apache/kafka/copycat/data/ObjectProperties.java new file mode 100644 index 0000000..e995b7f --- /dev/null +++ b/copycat/data/src/main/java/org/apache/kafka/copycat/data/ObjectProperties.java @@ -0,0 +1,85 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + + + +package org.apache.kafka.copycat.data; + +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.Set; + +/** + * Base class for objects that have Object-valued properties. + */ +public abstract class ObjectProperties { + public static class Null { + private Null() { + } + } + + /** A value representing a JSON null. */ + public static final Null NULL_VALUE = new Null(); + + Map props = new LinkedHashMap(1); + + private Set reserved; + + ObjectProperties(Set reserved) { + this.reserved = reserved; + } + + /** + * Returns the value of the named, string-valued property in this schema. + * Returns null if there is no string-valued property with that name. + */ + public String getProp(String name) { + Object value = getObjectProp(name); + return (value instanceof String) ? (String) value : null; + } + + /** + * Returns the value of the named property in this schema. + * Returns null if there is no property with that name. + */ + public synchronized Object getObjectProp(String name) { + return props.get(name); + } + + /** + * Adds a property with the given name name and + * value value. Neither name nor value can be + * null. It is illegal to add a property if another with + * the same name but different value already exists in this schema. + * + * @param name The name of the property to add + * @param value The value for the property to add + */ + public synchronized void addProp(String name, Object value) { + if (reserved.contains(name)) + throw new DataRuntimeException("Can't set reserved property: " + name); + + if (value == null) + throw new DataRuntimeException("Can't set a property to null: " + name); + + Object old = props.get(name); + if (old == null) + props.put(name, value); + else if (!old.equals(value)) + throw new DataRuntimeException("Can't overwrite property: " + name); + } +} diff --git a/copycat/data/src/main/java/org/apache/kafka/copycat/data/Schema.java b/copycat/data/src/main/java/org/apache/kafka/copycat/data/Schema.java new file mode 100644 index 0000000..04906c3 --- /dev/null +++ b/copycat/data/src/main/java/org/apache/kafka/copycat/data/Schema.java @@ -0,0 +1,1054 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + + + +package org.apache.kafka.copycat.data; + +import java.nio.ByteBuffer; +import java.util.*; + +/** An abstract data type. + *

    A schema may be one of: + *

      + *
    • A record, mapping field names to field value data; + *
    • An enum, containing one of a small set of symbols; + *
    • An array of values, all of the same schema; + *
    • A map, containing string/value pairs, of a declared schema; + *
    • A union of other schemas; + *
    • A fixed sized binary object; + *
    • A unicode string; + *
    • A sequence of bytes; + *
    • A 32-bit signed int; + *
    • A 64-bit signed long; + *
    • A 32-bit IEEE single-float; or + *
    • A 64-bit IEEE double-float; or + *
    • A boolean; or + *
    • null. + *
    + * + * A schema can be constructed using one of its static createXXX + * methods, or more conveniently using {@link SchemaBuilder}. The schema objects are + * logically immutable. + * There are only two mutating methods - {@link #setFields(List)} and + * {@link #addProp(String, Object)}. The following restrictions apply on these + * two methods. + *
      + *
    • {@link #setFields(List)}, can be called at most once. This method exists + * in order to enable clients to build recursive schemas. + *
    • {@link #addProp(String, Object)} can be called with property names + * that are not present already. It is not possible to change or delete an + * existing property. + *
    + */ +public abstract class Schema extends ObjectProperties { + private static final int NO_HASHCODE = Integer.MIN_VALUE; + + /** The type of a schema. */ + public enum Type { + ENUM { + @Override + public Object defaultValue(Schema schema) { + return null; + } + }, + ARRAY { + @Override + public Object defaultValue(Schema schema) { + return new ArrayList<>(); + } + }, + MAP { + @Override + public Object defaultValue(Schema schema) { + return new HashMap(); + } + }, + UNION { + @Override + public Object defaultValue(Schema schema) { + Schema firstSchema = schema.getTypes().get(0); + return firstSchema.getType().defaultValue(firstSchema); + } + }, + STRING { + @Override + public Object defaultValue(Schema schema) { + return ""; + } + }, + BYTES { + @Override + public Object defaultValue(Schema schema) { + return new byte[0]; + } + }, + INT { + @Override + public Object defaultValue(Schema schema) { + return 0; + } + }, + LONG { + @Override + public Object defaultValue(Schema schema) { + return 0; + } + }, + FLOAT { + @Override + public Object defaultValue(Schema schema) { + return 0; + } + }, + DOUBLE { + @Override + public Object defaultValue(Schema schema) { + return 0; + } + }, + BOOLEAN { + @Override + public Object defaultValue(Schema schema) { + return false; + } + }, + NULL { + @Override + public Object defaultValue(Schema schema) { + return null; + } + }; + private String name; + + private Type() { + this.name = this.name().toLowerCase(); + } + + public String getName() { + return name; + } + + public abstract Object defaultValue(Schema schema); + } + + private final Type type; + + Schema(Type type) { + super(SCHEMA_RESERVED); + this.type = type; + } + + /** Create a schema for a primitive type. */ + public static Schema create(Type type) { + switch (type) { + case STRING: + return new StringSchema(); + case BYTES: + return new BytesSchema(); + case INT: + return new IntSchema(); + case LONG: + return new LongSchema(); + case FLOAT: + return new FloatSchema(); + case DOUBLE: + return new DoubleSchema(); + case BOOLEAN: + return new BooleanSchema(); + case NULL: + return new NullSchema(); + default: + throw new DataRuntimeException("Can't create a: " + type); + } + } + + private static final Set SCHEMA_RESERVED = new HashSet(); + + static { + Collections.addAll(SCHEMA_RESERVED, + "doc", "fields", "items", "name", "namespace", + "size", "symbols", "values", "type", "aliases"); + } + + int hashCode = NO_HASHCODE; + + @Override + public void addProp(String name, Object value) { + super.addProp(name, value); + hashCode = NO_HASHCODE; + } + + /** Create an enum schema. */ + public static Schema createEnum(String name, String doc, String namespace, + List values) { + return new EnumSchema(new Name(name, namespace), doc, + new LockableArrayList(values)); + } + + /** Create an array schema. */ + public static Schema createArray(Schema elementType) { + return new ArraySchema(elementType); + } + + /** Create a map schema. */ + public static Schema createMap(Schema valueType) { + return new MapSchema(valueType); + } + + /** Create a union schema. */ + public static Schema createUnion(List types) { + return new UnionSchema(new LockableArrayList(types)); + } + + /** Create a union schema. */ + public static Schema createUnion(Schema... types) { + return createUnion(new LockableArrayList(types)); + } + + /** Return the type of this schema. */ + public Type getType() { + return type; + } + + /** + * If this is a record, returns the Field with the + * given name fieldName. If there is no field by that name, a + * null is returned. + */ + public Field getField(String fieldname) { + throw new DataRuntimeException("Not a record: " + this); + } + + /** + * If this is a record, returns the fields in it. The returned + * list is in the order of their positions. + */ + public List getFields() { + throw new DataRuntimeException("Not a record: " + this); + } + + /** + * If this is a record, set its fields. The fields can be set + * only once in a schema. + */ + public void setFields(List fields) { + throw new DataRuntimeException("Not a record: " + this); + } + + /** If this is an enum, return its symbols. */ + public List getEnumSymbols() { + throw new DataRuntimeException("Not an enum: " + this); + } + + /** If this is an enum, return a symbol's ordinal value. */ + public int getEnumOrdinal(String symbol) { + throw new DataRuntimeException("Not an enum: " + this); + } + + /** If this is an enum, returns true if it contains given symbol. */ + public boolean hasEnumSymbol(String symbol) { + throw new DataRuntimeException("Not an enum: " + this); + } + + /** If this is a record, enum or fixed, returns its name, otherwise the name + * of the primitive type. */ + public String getName() { + return type.name; + } + + /** If this is a record, enum, or fixed, returns its docstring, + * if available. Otherwise, returns null. */ + public String getDoc() { + return null; + } + + /** If this is a record, enum or fixed, returns its namespace, if any. */ + public String getNamespace() { + throw new DataRuntimeException("Not a named type: " + this); + } + + /** If this is a record, enum or fixed, returns its namespace-qualified name, + * otherwise returns the name of the primitive type. */ + public String getFullName() { + return getName(); + } + + /** If this is a record, enum or fixed, add an alias. */ + public void addAlias(String alias) { + throw new DataRuntimeException("Not a named type: " + this); + } + + /** If this is a record, enum or fixed, add an alias. */ + public void addAlias(String alias, String space) { + throw new DataRuntimeException("Not a named type: " + this); + } + + /** If this is a record, enum or fixed, return its aliases, if any. */ + public Set getAliases() { + throw new DataRuntimeException("Not a named type: " + this); + } + + /** Returns true if this record is an error type. */ + public boolean isError() { + throw new DataRuntimeException("Not a record: " + this); + } + + /** If this is an array, returns its element type. */ + public Schema getElementType() { + throw new DataRuntimeException("Not an array: " + this); + } + + /** If this is a map, returns its value type. */ + public Schema getValueType() { + throw new DataRuntimeException("Not a map: " + this); + } + + /** If this is a union, returns its types. */ + public List getTypes() { + throw new DataRuntimeException("Not a union: " + this); + } + + /** If this is a union, return the branch with the provided full name. */ + public Integer getIndexNamed(String name) { + throw new DataRuntimeException("Not a union: " + this); + } + + /** If this is fixed, returns its size. */ + public int getFixedSize() { + throw new DataRuntimeException("Not fixed: " + this); + } + + @Override + public String toString() { + // FIXME A more JSON-like output showing the details would be nice + return "Schema:" + this.getType() + ":" + getFullName(); + } + + public boolean equals(Object o) { + if (o == this) return true; + if (!(o instanceof Schema)) return false; + Schema that = (Schema) o; + if (!(this.type == that.type)) return false; + return equalCachedHash(that) && props.equals(that.props); + } + + public final int hashCode() { + if (hashCode == NO_HASHCODE) + hashCode = computeHash(); + return hashCode; + } + + int computeHash() { + return getType().hashCode() + props.hashCode(); + } + + final boolean equalCachedHash(Schema other) { + return (hashCode == other.hashCode) + || (hashCode == NO_HASHCODE) + || (other.hashCode == NO_HASHCODE); + } + + private static final Set FIELD_RESERVED = new HashSet(); + + static { + Collections.addAll(FIELD_RESERVED, + "default", "doc", "name", "order", "type", "aliases"); + } + + /** A field within a record. */ + public static class Field extends ObjectProperties { + + /** How values of this field should be ordered when sorting records. */ + public enum Order { + ASCENDING, DESCENDING, IGNORE; + private String name; + + private Order() { + this.name = this.name().toLowerCase(); + } + } + + + private final String name; // name of the field. + private int position = -1; + private final Schema schema; + private final String doc; + private final Object defaultValue; + private final Order order; + private Set aliases; + + public Field(String name, Schema schema, String doc, + Object defaultValue) { + this(name, schema, doc, defaultValue, Order.ASCENDING); + } + + public Field(String name, Schema schema, String doc, + Object defaultValue, Order order) { + super(FIELD_RESERVED); + this.name = validateName(name); + this.schema = schema; + this.doc = doc; + this.defaultValue = validateDefault(name, schema, defaultValue); + this.order = order; + } + + public String name() { + return name; + } + + + /** The position of this field within the record. */ + public int pos() { + return position; + } + + /** This field's {@link Schema}. */ + public Schema schema() { + return schema; + } + + /** Field's documentation within the record, if set. May return null. */ + public String doc() { + return doc; + } + + public Object defaultValue() { + return defaultValue; + } + + public Order order() { + return order; + } + + public void addAlias(String alias) { + if (aliases == null) + this.aliases = new LinkedHashSet(); + aliases.add(alias); + } + + /** Return the defined aliases as an unmodifieable Set. */ + public Set aliases() { + if (aliases == null) + return Collections.emptySet(); + return Collections.unmodifiableSet(aliases); + } + + public boolean equals(Object other) { + if (other == this) return true; + if (!(other instanceof Field)) return false; + Field that = (Field) other; + return (name.equals(that.name)) && + (schema.equals(that.schema)) && + defaultValueEquals(that.defaultValue) && + (order == that.order) && + props.equals(that.props); + } + + public int hashCode() { + return name.hashCode() + schema.computeHash(); + } + + /** Do any possible implicit conversions to double, or return 0 if there isn't a + * valid conversion */ + private double doubleValue(Object v) { + if (v instanceof Integer) + return (double) (Integer) v; + else if (v instanceof Long) + return (double) (Long) v; + else if (v instanceof Float) + return (double) (Float) v; + else if (v instanceof Double) + return (double) (Double) v; + else + return 0; + } + + private boolean defaultValueEquals(Object thatDefaultValue) { + if (defaultValue == null) + return thatDefaultValue == null; + if (Double.isNaN(doubleValue(defaultValue))) + return Double.isNaN(doubleValue(thatDefaultValue)); + return defaultValue.equals(thatDefaultValue); + } + + @Override + public String toString() { + return name + " type:" + schema.type + " pos:" + position; + } + } + + static class Name { + private final String name; + private final String space; + private final String full; + + public Name(String name, String space) { + if (name == null) { // anonymous + this.name = this.space = this.full = null; + return; + } + int lastDot = name.lastIndexOf('.'); + if (lastDot < 0) { // unqualified name + this.name = validateName(name); + } else { // qualified name + space = name.substring(0, lastDot); // get space from name + this.name = validateName(name.substring(lastDot + 1, name.length())); + } + if ("".equals(space)) + space = null; + this.space = space; + this.full = (this.space == null) ? this.name : this.space + "." + this.name; + } + + public boolean equals(Object o) { + if (o == this) return true; + if (!(o instanceof Name)) return false; + Name that = (Name) o; + return full == null ? that.full == null : full.equals(that.full); + } + + public int hashCode() { + return full == null ? 0 : full.hashCode(); + } + + public String toString() { + return full; + } + + public String getQualified(String defaultSpace) { + return (space == null || space.equals(defaultSpace)) ? name : full; + } + } + + private static abstract class NamedSchema extends Schema { + final Name name; + final String doc; + Set aliases; + + public NamedSchema(Type type, Name name, String doc) { + super(type); + this.name = name; + this.doc = doc; + if (PRIMITIVES.containsKey(name.full)) { + throw new DataTypeException("Schemas may not be named after primitives: " + name.full); + } + } + + public String getName() { + return name.name; + } + + public String getDoc() { + return doc; + } + + public String getNamespace() { + return name.space; + } + + public String getFullName() { + return name.full; + } + + public void addAlias(String alias) { + addAlias(alias, null); + } + + public void addAlias(String name, String space) { + if (aliases == null) + this.aliases = new LinkedHashSet(); + if (space == null) + space = this.name.space; + aliases.add(new Name(name, space)); + } + + public Set getAliases() { + Set result = new LinkedHashSet(); + if (aliases != null) + for (Name alias : aliases) + result.add(alias.full); + return result; + } + + public boolean equalNames(NamedSchema that) { + return this.name.equals(that.name); + } + + @Override + int computeHash() { + return super.computeHash() + name.hashCode(); + } + } + + private static class SeenPair { + private Object s1; + private Object s2; + + private SeenPair(Object s1, Object s2) { + this.s1 = s1; + this.s2 = s2; + } + + public boolean equals(Object o) { + return this.s1 == ((SeenPair) o).s1 && this.s2 == ((SeenPair) o).s2; + } + + public int hashCode() { + return System.identityHashCode(s1) + System.identityHashCode(s2); + } + } + + private static final ThreadLocal SEEN_EQUALS = new ThreadLocal() { + protected Set initialValue() { + return new HashSet(); + } + }; + private static final ThreadLocal SEEN_HASHCODE = new ThreadLocal() { + protected Map initialValue() { + return new IdentityHashMap(); + } + }; + + private static class EnumSchema extends NamedSchema { + private final List symbols; + private final Map ordinals; + + public EnumSchema(Name name, String doc, + LockableArrayList symbols) { + super(Type.ENUM, name, doc); + this.symbols = symbols.lock(); + this.ordinals = new HashMap(); + int i = 0; + for (String symbol : symbols) + if (ordinals.put(validateName(symbol), i++) != null) + throw new SchemaParseException("Duplicate enum symbol: " + symbol); + } + + public List getEnumSymbols() { + return symbols; + } + + public boolean hasEnumSymbol(String symbol) { + return ordinals.containsKey(symbol); + } + + public int getEnumOrdinal(String symbol) { + return ordinals.get(symbol); + } + + public boolean equals(Object o) { + if (o == this) return true; + if (!(o instanceof EnumSchema)) return false; + EnumSchema that = (EnumSchema) o; + return equalCachedHash(that) + && equalNames(that) + && symbols.equals(that.symbols) + && props.equals(that.props); + } + + @Override + int computeHash() { + return super.computeHash() + symbols.hashCode(); + } + } + + private static class ArraySchema extends Schema { + private final Schema elementType; + + public ArraySchema(Schema elementType) { + super(Type.ARRAY); + this.elementType = elementType; + } + + public Schema getElementType() { + return elementType; + } + + public boolean equals(Object o) { + if (o == this) return true; + if (!(o instanceof ArraySchema)) return false; + ArraySchema that = (ArraySchema) o; + return equalCachedHash(that) + && elementType.equals(that.elementType) + && props.equals(that.props); + } + + @Override + int computeHash() { + return super.computeHash() + elementType.computeHash(); + } + } + + private static class MapSchema extends Schema { + private final Schema valueType; + + public MapSchema(Schema valueType) { + super(Type.MAP); + this.valueType = valueType; + } + + public Schema getValueType() { + return valueType; + } + + public boolean equals(Object o) { + if (o == this) return true; + if (!(o instanceof MapSchema)) return false; + MapSchema that = (MapSchema) o; + return equalCachedHash(that) + && valueType.equals(that.valueType) + && props.equals(that.props); + } + + @Override + int computeHash() { + return super.computeHash() + valueType.computeHash(); + } + } + + private static class UnionSchema extends Schema { + private final List types; + private final Map indexByName + = new HashMap(); + + public UnionSchema(LockableArrayList types) { + super(Type.UNION); + this.types = types.lock(); + int index = 0; + for (Schema type : types) { + if (type.getType() == Type.UNION) + throw new DataRuntimeException("Nested union: " + this); + String name = type.getFullName(); + if (name == null) + throw new DataRuntimeException("Nameless in union:" + this); + if (indexByName.put(name, index++) != null) + throw new DataRuntimeException("Duplicate in union:" + name); + } + } + + public List getTypes() { + return types; + } + + public Integer getIndexNamed(String name) { + return indexByName.get(name); + } + + public boolean equals(Object o) { + if (o == this) return true; + if (!(o instanceof UnionSchema)) return false; + UnionSchema that = (UnionSchema) o; + return equalCachedHash(that) + && types.equals(that.types) + && props.equals(that.props); + } + + @Override + int computeHash() { + int hash = super.computeHash(); + for (Schema type : types) + hash += type.computeHash(); + return hash; + } + } + + private static class StringSchema extends Schema { + public StringSchema() { + super(Type.STRING); + } + } + + private static class BytesSchema extends Schema { + public BytesSchema() { + super(Type.BYTES); + } + } + + private static class IntSchema extends Schema { + public IntSchema() { + super(Type.INT); + } + } + + private static class LongSchema extends Schema { + public LongSchema() { + super(Type.LONG); + } + } + + private static class FloatSchema extends Schema { + public FloatSchema() { + super(Type.FLOAT); + } + } + + private static class DoubleSchema extends Schema { + public DoubleSchema() { + super(Type.DOUBLE); + } + } + + private static class BooleanSchema extends Schema { + public BooleanSchema() { + super(Type.BOOLEAN); + } + } + + private static class NullSchema extends Schema { + public NullSchema() { + super(Type.NULL); + } + } + + static final Map PRIMITIVES = new HashMap(); + + static { + PRIMITIVES.put("string", Type.STRING); + PRIMITIVES.put("bytes", Type.BYTES); + PRIMITIVES.put("int", Type.INT); + PRIMITIVES.put("long", Type.LONG); + PRIMITIVES.put("float", Type.FLOAT); + PRIMITIVES.put("double", Type.DOUBLE); + PRIMITIVES.put("boolean", Type.BOOLEAN); + PRIMITIVES.put("null", Type.NULL); + } + + static class Names extends LinkedHashMap { + private String space; // default namespace + + public Names() { + } + + public Names(String space) { + this.space = space; + } + + public String space() { + return space; + } + + public void space(String space) { + this.space = space; + } + + @Override + public Schema get(Object o) { + Name name; + if (o instanceof String) { + Type primitive = PRIMITIVES.get((String) o); + if (primitive != null) return Schema.create(primitive); + name = new Name((String) o, space); + if (!containsKey(name)) // if not in default + name = new Name((String) o, ""); // try anonymous + } else { + name = (Name) o; + } + return super.get(name); + } + + public boolean contains(Schema schema) { + return get(((NamedSchema) schema).name) != null; + } + + public void add(Schema schema) { + put(((NamedSchema) schema).name, schema); + } + + @Override + public Schema put(Name name, Schema schema) { + if (containsKey(name)) + throw new SchemaParseException("Can't redefine: " + name); + return super.put(name, schema); + } + } + + private static ThreadLocal validateNames + = new ThreadLocal() { + @Override + protected Boolean initialValue() { + return true; + } + }; + + private static String validateName(String name) { + if (!validateNames.get()) return name; // not validating names + int length = name.length(); + if (length == 0) + throw new SchemaParseException("Empty name"); + char first = name.charAt(0); + if (!(Character.isLetter(first) || first == '_')) + throw new SchemaParseException("Illegal initial character: " + name); + for (int i = 1; i < length; i++) { + char c = name.charAt(i); + if (!(Character.isLetterOrDigit(c) || c == '_')) + throw new SchemaParseException("Illegal character in: " + name); + } + return name; + } + + private static final ThreadLocal VALIDATE_DEFAULTS + = new ThreadLocal() { + @Override + protected Boolean initialValue() { + return false; + } + }; + + private static Object validateDefault(String fieldName, Schema schema, + Object defaultValue) { + if ((defaultValue != null) + && !isValidDefault(schema, defaultValue)) { // invalid default + String message = "Invalid default for field " + fieldName + + ": " + defaultValue + " not a " + schema; + if (VALIDATE_DEFAULTS.get()) + throw new DataTypeException(message); // throw exception + System.err.println("[WARNING] Avro: " + message); // or log warning + } + return defaultValue; + } + + private static boolean isValidDefault(Schema schema, Object defaultValue) { + switch (schema.getType()) { + case STRING: + case ENUM: + return (defaultValue instanceof String); + case BYTES: + case INT: + return (defaultValue instanceof Integer); + case LONG: + return (defaultValue instanceof Long); + case FLOAT: + return (defaultValue instanceof Float); + case DOUBLE: + return (defaultValue instanceof Double); + case BOOLEAN: + return (defaultValue instanceof Boolean); + case NULL: + return defaultValue == null; + case ARRAY: + if (!(defaultValue instanceof Collection)) + return false; + for (Object element : (Collection) defaultValue) + if (!isValidDefault(schema.getElementType(), element)) + return false; + return true; + case MAP: + if (!(defaultValue instanceof Map)) + return false; + for (Object value : ((Map) defaultValue).values()) + if (!isValidDefault(schema.getValueType(), value)) + return false; + return true; + case UNION: // union default: first branch + return isValidDefault(schema.getTypes().get(0), defaultValue); + default: + return false; + } + } + + /** + * No change is permitted on LockableArrayList once lock() has been + * called on it. + * @param + */ + + /* + * This class keeps a boolean variable locked which is set + * to true in the lock() method. It's legal to call + * lock() any number of times. Any lock() other than the first one + * is a no-op. + * + * This class throws IllegalStateException if a mutating + * operation is performed after being locked. Since modifications through + * iterator also use the list's mutating operations, this effectively + * blocks all modifications. + */ + static class LockableArrayList extends ArrayList { + private static final long serialVersionUID = 1L; + private boolean locked = false; + + public LockableArrayList() { + } + + public LockableArrayList(int size) { + super(size); + } + + public LockableArrayList(List types) { + super(types); + } + + public LockableArrayList(E... types) { + super(types.length); + Collections.addAll(this, types); + } + + public List lock() { + locked = true; + return this; + } + + private void ensureUnlocked() { + if (locked) { + throw new IllegalStateException(); + } + } + + public boolean add(E e) { + ensureUnlocked(); + return super.add(e); + } + + public boolean remove(Object o) { + ensureUnlocked(); + return super.remove(o); + } + + public E remove(int index) { + ensureUnlocked(); + return super.remove(index); + } + + public boolean addAll(Collection c) { + ensureUnlocked(); + return super.addAll(c); + } + + public boolean addAll(int index, Collection c) { + ensureUnlocked(); + return super.addAll(index, c); + } + + public boolean removeAll(Collection c) { + ensureUnlocked(); + return super.removeAll(c); + } + + public boolean retainAll(Collection c) { + ensureUnlocked(); + return super.retainAll(c); + } + + public void clear() { + ensureUnlocked(); + super.clear(); + } + + } + +} diff --git a/copycat/data/src/main/java/org/apache/kafka/copycat/data/SchemaBuilder.java b/copycat/data/src/main/java/org/apache/kafka/copycat/data/SchemaBuilder.java new file mode 100644 index 0000000..f4a76a1 --- /dev/null +++ b/copycat/data/src/main/java/org/apache/kafka/copycat/data/SchemaBuilder.java @@ -0,0 +1,2415 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + + + +package org.apache.kafka.copycat.data; + +import org.apache.kafka.copycat.data.Schema.Field; + +import java.nio.ByteBuffer; +import java.util.*; + +/** + *

    + * A fluent interface for building {@link Schema} instances. The flow of the API + * is designed to mimic the Avro Schema + * Specification + *

    + * For example, the below JSON schema and the fluent builder code to create it + * are very similar: + * + *
    + * {
    + *   "type": "record",
    + *   "name": "HandshakeRequest", "namespace":"org.apache.avro.ipc",
    + *   "fields": [
    + *     {"name": "clientHash",
    + *      "type": {"type": "fixed", "name": "MD5", "size": 16}},
    + *     {"name": "clientProtocol", "type": ["null", "string"]},
    + *     {"name": "serverHash", "type": "MD5"},
    + *     {"name": "meta", "type": ["null", {"type": "map", "values": "bytes"}]}
    + *   ]
    + * }
    + * 
    + * + *
    + *   Schema schema = SchemaBuilder
    + *   .record("HandshakeRequest").namespace("org.apache.avro.ipc)
    + *   .fields()
    + *     .name("clientHash").type().fixed("MD5").size(16).noDefault()
    + *     .name("clientProtocol").type().nullable().stringType().noDefault()
    + *     .name("serverHash").type("MD5")
    + *     .name("meta").type().nullable().map().values().bytesType().noDefault()
    + *   .endRecord();
    + * 
    + *

    + * + *

    Usage Guide
    + * SchemaBuilder chains together many smaller builders and maintains nested + * context in order to mimic the Avro Schema specification. Every Avro type in + * JSON has required and optional JSON properties, as well as user-defined + * properties. + *

    + *

    Selecting and Building an Avro Type
    + * The API analogy for the right hand side of the Avro Schema JSON + *
    + * "type":
    + * 
    + * is a {@link TypeBuilder}, {@link FieldTypeBuilder}, or + * {@link UnionFieldTypeBuilder}, depending on the context. These types all + * share a similar API for selecting and building types. + *

    + *

    Primitive Types
    + * All Avro primitive types are trivial to configure. A primitive type in + * Avro JSON can be declared two ways, one that supports custom properties + * and one that does not: + *
    + * {"type":"int"}
    + * {"type":{"name":"int"}}
    + * {"type":{"name":"int", "customProp":"val"}}
    + * 
    + * The analogous code form for the above three JSON lines are the below + * three lines: + *
    + *  .intType()
    + *  .intBuilder().endInt()
    + *  .intBuilder().prop("customProp", "val").endInt()
    + * 
    + * Every primitive type has a shortcut to create the trivial type, and + * a builder when custom properties are required. The first line above is + * a shortcut for the second, analogous to the JSON case. + *
    Named Types
    + * Avro named types have names, namespace, aliases, and doc. In this API + * these share a common parent, {@link NamespacedBuilder}. + * The builders for named types require a name to be constructed, and optional + * configuration via: + *
  1. {@link NamespacedBuilder#doc()}
  2. + *
  3. {@link NamespacedBuilder#namespace(String)}
  4. + *
  5. {@link NamespacedBuilder#aliases(String...)}
  6. + *
  7. {@link PropBuilder#prop(String, Object)}
  8. + *

    + * Each named type completes configuration of the optional properties + * with its own method: + *

  9. {@link FixedBuilder#size(int)}
  10. + *
  11. {@link EnumBuilder#symbols(String...)}
  12. + *
  13. {@link RecordBuilder#fields()}
  14. + * Example use of a named type with all optional parameters: + *
    + * .enumeration("Suit").namespace("org.apache.test")
    + *   .aliases("org.apache.test.OldSuit")
    + *   .doc("CardSuits")
    + *   .prop("customProp", "val")
    + *   .symbols("SPADES", "HEARTS", "DIAMONDS", "CLUBS")
    + * 
    + * Which is equivalent to the JSON: + *
    + * { "type":"enum",
    + *   "name":"Suit", "namespace":"org.apache.test",
    + *   "aliases":["org.apache.test.OldSuit"],
    + *   "doc":"Card Suits",
    + *   "customProp":"val",
    + *   "symbols":["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
    + * }
    + * 
    + *
    Nested Types
    + * The Avro nested types, map and array, can have custom properties like + * all avro types, are not named, and must specify a nested type. + * After configuration of optional properties, an array or map + * builds or selects its nested type with {@link ArrayBuilder#items()} + * and {@link MapBuilder#values()}, respectively. + * + *
    Fields
    + * {@link RecordBuilder#fields()} returns a {@link FieldAssembler} for + * defining the fields of the record and completing it. + * Each field must have a name, specified via {@link FieldAssembler#name(String)}, + * which returns a {@link FieldBuilder} for defining aliases, custom properties, + * and documentation of the field. After configuring these optional values for + * a field, the type is selected or built with {@link FieldBuilder#type()}. + *

    + * Fields have default values that must be specified to complete the field. + * {@link FieldDefault#noDefault()} is available for all field types, and + * a specific method is available for each type to use a default, for example + * {@link IntDefault#intDefault(int)} + *

    + * There are field shortcut methods on {@link FieldAssembler} for primitive types. + * These shortcuts create required, optional, and nullable fields, but do not + * support field aliases, doc, or custom properties. + * + *

    Unions
    + * Union types are built via {@link TypeBuilder#unionOf()} or + * {@link FieldTypeBuilder#unionOf()} in the context of type selection. + * This chains together multiple types, in union order. For example: + *
    + * .unionOf()
    + *   .fixed("IPv4").size(4).and()
    + *   .fixed("IPv6").size(16).and()
    + *   .nullType().endUnion()
    + * 
    + * is equivalent to the Avro schema JSON: + *
    + * [
    + *   {"type":"fixed", "name":"IPv4", "size":4},
    + *   {"type":"fixed", "name":"IPv6", "size":16},
    + *   "null"
    + * ]
    + * 
    + * In a field context, the first type of a union defines what default type + * is allowed. + *

    + * Unions have two shortcuts for common cases. nullable() + * creates a union of a type and null. In a field type context, optional() + * is available and creates a union of null and a type, with a null default. + * The below two are equivalent: + *
    + *   .unionOf().intType().and().nullType().endUnion()
    + *   .nullable().intType()
    + * 
    + * The below two field declarations are equivalent: + *
    + *   .name("f").type().unionOf().nullType().and().longType().endUnion().nullDefault()
    + *   .name("f").type().optional().longType()
    + * 
    + * + *
    Explicit Types and Types by Name
    + * Types can also be specified explicitly by passing in a Schema, or by name: + *
    + *   .type(Schema.create(Schema.Type.INT)) // explicitly specified
    + *   .type("MD5")                       // reference by full name or short name
    + *   .type("MD5", "org.apache.avro.test")  // reference by name and namespace
    + * 
    + * When a type is specified by name, and the namespace is absent or null, the + * namespace is inherited from the enclosing context. A namespace will + * propagate as a default to child fields, nested types, or later defined types + * in a union. To specify a name that has no namespace and ignore the inherited + * namespace, set the namespace to "". + *

    + * {@link SchemaBuilder#builder(String)} returns a type builder with a default + * namespace. {@link SchemaBuilder#builder()} returns a type builder with no + * default namespace. + */ +public class SchemaBuilder { + + private SchemaBuilder() { + } + + /** + * Create a builder for Avro schemas. + */ + public static TypeBuilder builder() { + return new TypeBuilder(new SchemaCompletion(), new NameContext()); + } + + /** + * Create a builder for Avro schemas with a default namespace. Types created + * without namespaces will inherit the namespace provided. + */ + public static TypeBuilder builder(String namespace) { + return new TypeBuilder(new SchemaCompletion(), + new NameContext().namespace(namespace)); + } + + /** + * Create a builder for an Avro enum with the specified name and symbols (values). + * This is equivalent to: + *

    +     *   builder().enumeration(name);
    +     * 
    + * @param name the enum name + */ + public static EnumBuilder enumeration(String name) { + return builder().enumeration(name); + } + + /** + * Create a builder for an Avro array + * This is equivalent to: + *
    +     *   builder().array();
    +     * 
    + */ + public static ArrayBuilder array() { + return builder().array(); + } + + /** + * Create a builder for an Avro map + * This is equivalent to: + *
    +     *   builder().map();
    +     * 
    + */ + public static MapBuilder map() { + return builder().map(); + } + + /** + * Create a builder for an Avro union + * This is equivalent to: + *
    +     *   builder().unionOf();
    +     * 
    + */ + public static BaseTypeBuilder> unionOf() { + return builder().unionOf(); + } + + /** + * Create a builder for a union of a type and null. + * This is a shortcut for: + *
    +     *   builder().nullable();
    +     * 
    + * and the following two lines are equivalent: + *
    +     *   nullable().intType();
    +     * 
    + *
    +     *   unionOf().intType().and().nullType().endUnion();
    +     * 
    + */ + public static BaseTypeBuilder nullable() { + return builder().nullable(); + } + + + /** + * An abstract builder for all Avro types. All Avro types + * can have arbitrary string key-value properties. + */ + public static abstract class PropBuilder> { + private Map props = null; + + protected PropBuilder() { + } + + public final S prop(String name, Object val) { + if (!hasProps()) { + props = new HashMap(); + } + props.put(name, val); + return self(); + } + + private boolean hasProps() { + return (props != null); + } + + final T addPropsTo(T jsonable) { + if (hasProps()) { + for (Map.Entry prop : props.entrySet()) { + jsonable.addProp(prop.getKey(), prop.getValue()); + } + } + return jsonable; + } + + /** a self-type for chaining builder subclasses. Concrete subclasses + * must return 'this' **/ + protected abstract S self(); + } + + /** + * An abstract type that provides builder methods for configuring the name, + * doc, and aliases of all Avro types that have names (fields, Fixed, Record, + * and Enum). + *

    + * All Avro named types and fields have 'doc', 'aliases', and 'name' + * components. 'name' is required, and provided to this builder. 'doc' and + * 'alises' are optional. + */ + public static abstract class NamedBuilder> extends + PropBuilder { + private final String name; + private final NameContext names; + private String doc; + private String[] aliases; + + protected NamedBuilder(NameContext names, String name) { + checkRequired(name, "Type must have a name"); + this.names = names; + this.name = name; + } + + /** configure this type's optional documentation string **/ + public final S doc(String doc) { + this.doc = doc; + return self(); + } + + /** configure this type's optional name aliases **/ + public final S aliases(String... aliases) { + this.aliases = aliases; + return self(); + } + + final String doc() { + return doc; + } + + final String name() { + return name; + } + + final NameContext names() { + return names; + } + + final Schema addAliasesTo(Schema schema) { + if (null != aliases) { + for (String alias : aliases) { + schema.addAlias(alias); + } + } + return schema; + } + + final Field addAliasesTo(Field field) { + if (null != aliases) { + for (String alias : aliases) { + field.addAlias(alias); + } + } + return field; + } + } + + /** + * An abstract type that provides builder methods for configuring the + * namespace for all Avro types that have namespaces (Fixed, Record, and + * Enum). + */ + public static abstract class NamespacedBuilder> + extends NamedBuilder { + private final Completion context; + private String namespace; + + protected NamespacedBuilder(Completion context, NameContext names, + String name) { + super(names, name); + this.context = context; + } + + /** + * Set the namespace of this type. To clear the namespace, set empty string. + *

    + * When the namespace is null or unset, the namespace of the type defaults + * to the namespace of the enclosing context. + **/ + public final S namespace(String namespace) { + this.namespace = namespace; + return self(); + } + + final String space() { + if (null == namespace) { + return names().namespace; + } + return namespace; + } + + final Schema completeSchema(Schema schema) { + addPropsTo(schema); + addAliasesTo(schema); + names().put(schema); + return schema; + } + + final Completion context() { + return context; + } + } + + /** + * An abstraction for sharing code amongst all primitive type builders. + */ + private static abstract class PrimitiveBuilder> + extends PropBuilder

    { + private final Completion context; + private final Schema immutable; + + protected PrimitiveBuilder(Completion context, NameContext names, + Schema.Type type) { + this.context = context; + this.immutable = names.getFullname(type.getName()); + } + + private R end() { + Schema schema = immutable; + if (super.hasProps()) { + schema = Schema.create(immutable.getType()); + addPropsTo(schema); + } + return context.complete(schema); + } + } + + /** + * Builds an Avro boolean type with optional properties. Set properties with + * {@link #prop(String, Object)}, and finalize with {@link #endBoolean()} + **/ + public static final class BooleanBuilder extends + PrimitiveBuilder> { + private BooleanBuilder(Completion context, NameContext names) { + super(context, names, Schema.Type.BOOLEAN); + } + + private static BooleanBuilder create(Completion context, + NameContext names) { + return new BooleanBuilder(context, names); + } + + @Override + protected BooleanBuilder self() { + return this; + } + + /** complete building this type, return control to context **/ + public R endBoolean() { + return super.end(); + } + } + + /** + * Builds an Avro int type with optional properties. Set properties with + * {@link #prop(String, Object)}, and finalize with {@link #endInt()} + **/ + public static final class IntBuilder extends + PrimitiveBuilder> { + private IntBuilder(Completion context, NameContext names) { + super(context, names, Schema.Type.INT); + } + + private static IntBuilder create(Completion context, + NameContext names) { + return new IntBuilder(context, names); + } + + @Override + protected IntBuilder self() { + return this; + } + + /** complete building this type, return control to context **/ + public R endInt() { + return super.end(); + } + } + + /** + * Builds an Avro long type with optional properties. Set properties with + * {@link #prop(String, Object)}, and finalize with {@link #endLong()} + **/ + public static final class LongBuilder extends + PrimitiveBuilder> { + private LongBuilder(Completion context, NameContext names) { + super(context, names, Schema.Type.LONG); + } + + private static LongBuilder create(Completion context, + NameContext names) { + return new LongBuilder(context, names); + } + + @Override + protected LongBuilder self() { + return this; + } + + /** complete building this type, return control to context **/ + public R endLong() { + return super.end(); + } + } + + /** + * Builds an Avro float type with optional properties. Set properties with + * {@link #prop(String, Object)}, and finalize with {@link #endFloat()} + **/ + public static final class FloatBuilder extends + PrimitiveBuilder> { + private FloatBuilder(Completion context, NameContext names) { + super(context, names, Schema.Type.FLOAT); + } + + private static FloatBuilder create(Completion context, + NameContext names) { + return new FloatBuilder(context, names); + } + + @Override + protected FloatBuilder self() { + return this; + } + + /** complete building this type, return control to context **/ + public R endFloat() { + return super.end(); + } + } + + /** + * Builds an Avro double type with optional properties. Set properties with + * {@link #prop(String, Object)}, and finalize with {@link #endDouble()} + **/ + public static final class DoubleBuilder extends + PrimitiveBuilder> { + private DoubleBuilder(Completion context, NameContext names) { + super(context, names, Schema.Type.DOUBLE); + } + + private static DoubleBuilder create(Completion context, + NameContext names) { + return new DoubleBuilder(context, names); + } + + @Override + protected DoubleBuilder self() { + return this; + } + + /** complete building this type, return control to context **/ + public R endDouble() { + return super.end(); + } + } + + /** + * Builds an Avro string type with optional properties. Set properties with + * {@link #prop(String, Object)}, and finalize with {@link #endString()} + **/ + public static final class StringBldr extends + PrimitiveBuilder> { + private StringBldr(Completion context, NameContext names) { + super(context, names, Schema.Type.STRING); + } + + private static StringBldr create(Completion context, + NameContext names) { + return new StringBldr(context, names); + } + + @Override + protected StringBldr self() { + return this; + } + + /** complete building this type, return control to context **/ + public R endString() { + return super.end(); + } + } + + /** + * Builds an Avro bytes type with optional properties. Set properties with + * {@link #prop(String, Object)}, and finalize with {@link #endBytes()} + **/ + public static final class BytesBuilder extends + PrimitiveBuilder> { + private BytesBuilder(Completion context, NameContext names) { + super(context, names, Schema.Type.BYTES); + } + + private static BytesBuilder create(Completion context, + NameContext names) { + return new BytesBuilder(context, names); + } + + @Override + protected BytesBuilder self() { + return this; + } + + /** complete building this type, return control to context **/ + public R endBytes() { + return super.end(); + } + } + + /** + * Builds an Avro null type with optional properties. Set properties with + * {@link #prop(String, Object)}, and finalize with {@link #endNull()} + **/ + public static final class NullBuilder extends + PrimitiveBuilder> { + private NullBuilder(Completion context, NameContext names) { + super(context, names, Schema.Type.NULL); + } + + private static NullBuilder create(Completion context, + NameContext names) { + return new NullBuilder(context, names); + } + + @Override + protected NullBuilder self() { + return this; + } + + /** complete building this type, return control to context **/ + public R endNull() { + return super.end(); + } + } + + /** + * Builds an Avro Enum type with optional properties, namespace, doc, and + * aliases. + *

    + * Set properties with {@link #prop(String, Object)}, namespace with + * {@link #namespace(String)}, doc with {@link #doc(String)}, and aliases with + * {@link #aliases(String[])}. + *

    + * The Enum schema is finalized when its required symbols are set via + * {@link #symbols(String[])}. + **/ + public static final class EnumBuilder extends + NamespacedBuilder> { + private EnumBuilder(Completion context, NameContext names, String name) { + super(context, names, name); + } + + private static EnumBuilder create(Completion context, + NameContext names, String name) { + return new EnumBuilder(context, names, name); + } + + @Override + protected EnumBuilder self() { + return this; + } + + /** Configure this enum type's symbols, and end its configuration. **/ + public R symbols(String... symbols) { + Schema schema = Schema.createEnum(name(), doc(), space(), + Arrays.asList(symbols)); + completeSchema(schema); + return context().complete(schema); + } + + } + + /** + * Builds an Avro Map type with optional properties. + *

    + * Set properties with {@link #prop(String, Object)}. + *

    + * The Map schema's properties are finalized when {@link #values()} or + * {@link #values(Schema)} is called. + **/ + public static final class MapBuilder extends PropBuilder> { + private final Completion context; + private final NameContext names; + + private MapBuilder(Completion context, NameContext names) { + this.context = context; + this.names = names; + } + + private static MapBuilder create(Completion context, + NameContext names) { + return new MapBuilder(context, names); + } + + @Override + protected MapBuilder self() { + return this; + } + + /** + * Return a type builder for configuring the map's nested values schema. + * This builder will return control to the map's enclosing context when + * complete. + **/ + public TypeBuilder values() { + return new TypeBuilder(new MapCompletion(this, context), names); + } + + /** + * Complete configuration of this map, setting the schema of the map values + * to the schema provided. Returns control to the enclosing context. + **/ + public R values(Schema valueSchema) { + return new MapCompletion(this, context).complete(valueSchema); + } + } + + /** + * Builds an Avro Array type with optional properties. + *

    + * Set properties with {@link #prop(String, Object)}. + *

    + * The Array schema's properties are finalized when {@link #items()} or + * {@link #items(Schema)} is called. + **/ + public static final class ArrayBuilder extends + PropBuilder> { + private final Completion context; + private final NameContext names; + + public ArrayBuilder(Completion context, NameContext names) { + this.context = context; + this.names = names; + } + + private static ArrayBuilder create(Completion context, + NameContext names) { + return new ArrayBuilder(context, names); + } + + @Override + protected ArrayBuilder self() { + return this; + } + + /** + * Return a type builder for configuring the array's nested items schema. + * This builder will return control to the array's enclosing context when + * complete. + **/ + public TypeBuilder items() { + return new TypeBuilder(new ArrayCompletion(this, context), names); + } + + /** + * Complete configuration of this array, setting the schema of the array + * items to the schema provided. Returns control to the enclosing context. + **/ + public R items(Schema itemsSchema) { + return new ArrayCompletion(this, context).complete(itemsSchema); + } + } + + /** + * internal class for passing the naming context around. This allows for the + * following: + *

  15. Cache and re-use primitive schemas when they do not set + * properties.
  16. + *
  17. Provide a default namespace for nested contexts (as + * the JSON Schema spec does).
  18. + *
  19. Allow previously defined named types or primitive types + * to be referenced by name.
  20. + **/ + private static class NameContext { + private static final Set PRIMITIVES = new HashSet(); + + { + PRIMITIVES.add("null"); + PRIMITIVES.add("boolean"); + PRIMITIVES.add("int"); + PRIMITIVES.add("long"); + PRIMITIVES.add("float"); + PRIMITIVES.add("double"); + PRIMITIVES.add("bytes"); + PRIMITIVES.add("string"); + } + + private final HashMap schemas; + private final String namespace; + + private NameContext() { + this.schemas = new HashMap(); + this.namespace = null; + schemas.put("null", Schema.create(Schema.Type.NULL)); + schemas.put("boolean", Schema.create(Schema.Type.BOOLEAN)); + schemas.put("int", Schema.create(Schema.Type.INT)); + schemas.put("long", Schema.create(Schema.Type.LONG)); + schemas.put("float", Schema.create(Schema.Type.FLOAT)); + schemas.put("double", Schema.create(Schema.Type.DOUBLE)); + schemas.put("bytes", Schema.create(Schema.Type.BYTES)); + schemas.put("string", Schema.create(Schema.Type.STRING)); + } + + private NameContext(HashMap schemas, String namespace) { + this.schemas = schemas; + this.namespace = "".equals(namespace) ? null : namespace; + } + + private NameContext namespace(String namespace) { + return new NameContext(schemas, namespace); + } + + private Schema get(String name, String namespace) { + return getFullname(resolveName(name, namespace)); + } + + private Schema getFullname(String fullName) { + Schema schema = schemas.get(fullName); + if (schema == null) { + throw new SchemaParseException("Undefined name: " + fullName); + } + return schema; + } + + private void put(Schema schema) { + String fullName = schema.getFullName(); + if (schemas.containsKey(fullName)) { + throw new SchemaParseException("Can't redefine: " + fullName); + } + schemas.put(fullName, schema); + } + + private String resolveName(String name, String space) { + if (PRIMITIVES.contains(name) && space == null) { + return name; + } + int lastDot = name.lastIndexOf('.'); + if (lastDot < 0) { // short name + if (space == null) { + space = namespace; + } + if (space != null && !"".equals(space)) { + return space + "." + name; + } + } + return name; + } + } + + /** + * A common API for building types within a context. BaseTypeBuilder can build + * all types other than Unions. {@link TypeBuilder} can additionally build + * Unions. + *

    + * The builder has two contexts: + *

  21. A naming context provides a default namespace and allows for previously + * defined named types to be referenced from {@link #type(String)}
  22. + *
  23. A completion context representing the scope that the builder was + * created in. A builder created in a nested context (for example, + * {@link MapBuilder#values()} will have a completion context assigned by the + * {@link MapBuilder}
  24. + **/ + public static class BaseTypeBuilder { + private final Completion context; + private final NameContext names; + + private BaseTypeBuilder(Completion context, NameContext names) { + this.context = context; + this.names = names; + } + + /** Use the schema provided as the type. **/ + public final R type(Schema schema) { + return context.complete(schema); + } + + /** + * Look up the type by name. This type must be previously defined in the + * context of this builder. + *

    + * The name may be fully qualified or a short name. If it is a short name, + * the default namespace of the current context will additionally be + * searched. + **/ + public final R type(String name) { + return type(name, null); + } + + /** + * Look up the type by name and namespace. This type must be previously + * defined in the context of this builder. + *

    + * The name may be fully qualified or a short name. If it is a fully + * qualified name, the namespace provided is ignored. If it is a short name, + * the namespace provided is used if not null, else the default namespace of + * the current context will be used. + **/ + public final R type(String name, String namespace) { + return type(names.get(name, namespace)); + } + + /** + * A plain boolean type without custom properties. This is equivalent to: + *

    +         * booleanBuilder().endBoolean();
    +         * 
    + */ + public final R booleanType() { + return booleanBuilder().endBoolean(); + } + + /** + * Build a boolean type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #booleanType()}. + */ + public final BooleanBuilder booleanBuilder() { + return BooleanBuilder.create(context, names); + } + + /** + * A plain int type without custom properties. This is equivalent to: + *
    +         * intBuilder().endInt();
    +         * 
    + */ + public final R intType() { + return intBuilder().endInt(); + } + + /** + * Build an int type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #intType()}. + */ + public final IntBuilder intBuilder() { + return IntBuilder.create(context, names); + } + + /** + * A plain long type without custom properties. This is equivalent to: + *
    +         * longBuilder().endLong();
    +         * 
    + */ + public final R longType() { + return longBuilder().endLong(); + } + + /** + * Build a long type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #longType()}. + */ + public final LongBuilder longBuilder() { + return LongBuilder.create(context, names); + } + + /** + * A plain float type without custom properties. This is equivalent to: + *
    +         * floatBuilder().endFloat();
    +         * 
    + */ + public final R floatType() { + return floatBuilder().endFloat(); + } + + /** + * Build a float type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #floatType()}. + */ + public final FloatBuilder floatBuilder() { + return FloatBuilder.create(context, names); + } + + /** + * A plain double type without custom properties. This is equivalent to: + *
    +         * doubleBuilder().endDouble();
    +         * 
    + */ + public final R doubleType() { + return doubleBuilder().endDouble(); + } + + /** + * Build a double type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #doubleType()}. + */ + public final DoubleBuilder doubleBuilder() { + return DoubleBuilder.create(context, names); + } + + /** + * A plain string type without custom properties. This is equivalent to: + *
    +         * stringBuilder().endString();
    +         * 
    + */ + public final R stringType() { + return stringBuilder().endString(); + } + + /** + * Build a string type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #stringType()}. + */ + public final StringBldr stringBuilder() { + return StringBldr.create(context, names); + } + + /** + * A plain bytes type without custom properties. This is equivalent to: + *
    +         * bytesBuilder().endBytes();
    +         * 
    + */ + public final R bytesType() { + return bytesBuilder().endBytes(); + } + + /** + * Build a bytes type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #bytesType()}. + */ + public final BytesBuilder bytesBuilder() { + return BytesBuilder.create(context, names); + } + + /** + * A plain null type without custom properties. This is equivalent to: + *
    +         * nullBuilder().endNull();
    +         * 
    + */ + public final R nullType() { + return nullBuilder().endNull(); + } + + /** + * Build a null type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #nullType()}. + */ + public final NullBuilder nullBuilder() { + return NullBuilder.create(context, names); + } + + /** Build an Avro map type Example usage: + *
    +         * map().values().intType()
    +         * 
    + * Equivalent to Avro JSON Schema: + *
    +         * {"type":"map", "values":"int"}
    +         * 
    + **/ + public final MapBuilder map() { + return MapBuilder.create(context, names); + } + + /** Build an Avro array type Example usage: + *
    +         * array().items().longType()
    +         * 
    + * Equivalent to Avro JSON Schema: + *
    +         * {"type":"array", "values":"long"}
    +         * 
    + **/ + public final ArrayBuilder array() { + return ArrayBuilder.create(context, names); + } + + /** Build an Avro enum type. Example usage: + *
    +         * enumeration("Suits").namespace("org.cards").doc("card suit names")
    +         *   .symbols("HEART", "SPADE", "DIAMOND", "CLUB")
    +         * 
    + * Equivalent to Avro JSON Schema: + *
    +         * {"type":"enum", "name":"Suits", "namespace":"org.cards",
    +         *  "doc":"card suit names", "symbols":[
    +         *    "HEART", "SPADE", "DIAMOND", "CLUB"]}
    +         * 
    + **/ + public final EnumBuilder enumeration(String name) { + return EnumBuilder.create(context, names, name); + } + + /** Build an Avro union schema type. Example usage: + *
    unionOf().stringType().and().bytesType().endUnion()
    + **/ + protected BaseTypeBuilder> unionOf() { + return UnionBuilder.create(context, names); + } + + /** A shortcut for building a union of a type and null. + *

    + * For example, the code snippets below are equivalent: + *

    nullable().booleanType()
    + *
    unionOf().booleanType().and().nullType().endUnion()
    + **/ + protected BaseTypeBuilder nullable() { + return new BaseTypeBuilder(new NullableCompletion(context), names); + } + + } + + /** A Builder for creating any Avro schema type. + **/ + public static final class TypeBuilder extends BaseTypeBuilder { + private TypeBuilder(Completion context, NameContext names) { + super(context, names); + } + + @Override + public BaseTypeBuilder> unionOf() { + return super.unionOf(); + } + + @Override + public BaseTypeBuilder nullable() { + return super.nullable(); + } + } + + /** A special builder for unions. Unions cannot nest unions directly **/ + private static final class UnionBuilder extends + BaseTypeBuilder> { + private UnionBuilder(Completion context, NameContext names) { + this(context, names, new ArrayList()); + } + + private static UnionBuilder create(Completion context, NameContext names) { + return new UnionBuilder(context, names); + } + + private UnionBuilder(Completion context, NameContext names, List schemas) { + super(new UnionCompletion(context, names, schemas), names); + } + } + + /** + * A special Builder for Record fields. The API is very similar to + * {@link BaseTypeBuilder}. However, fields have their own names, properties, + * and default values. + *

    + * The methods on this class create builder instances that return their + * control to the {@link FieldAssembler} of the enclosing record context after + * configuring a default for the field. + *

    + * For example, an int field with default value 1: + *

    +     * intSimple().withDefault(1);
    +     * 
    + * or an array with items that are optional int types: + *
    +     * array().items().optional().intType();
    +     * 
    + */ + public static class BaseFieldTypeBuilder { + protected final FieldBuilder bldr; + protected final NameContext names; + private final CompletionWrapper wrapper; + + protected BaseFieldTypeBuilder(FieldBuilder bldr, CompletionWrapper wrapper) { + this.bldr = bldr; + this.names = bldr.names(); + this.wrapper = wrapper; + } + + /** + * A plain boolean type without custom properties. This is equivalent to: + *
    +         * booleanBuilder().endBoolean();
    +         * 
    + */ + public final BooleanDefault booleanType() { + return booleanBuilder().endBoolean(); + } + + /** + * Build a boolean type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #booleanType()}. + */ + public final BooleanBuilder> booleanBuilder() { + return BooleanBuilder.create(wrap(new BooleanDefault(bldr)), names); + } + + /** + * A plain int type without custom properties. This is equivalent to: + *
    +         * intBuilder().endInt();
    +         * 
    + */ + public final IntDefault intType() { + return intBuilder().endInt(); + } + + /** + * Build an int type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #intType()}. + */ + public final IntBuilder> intBuilder() { + return IntBuilder.create(wrap(new IntDefault(bldr)), names); + } + + /** + * A plain long type without custom properties. This is equivalent to: + *
    +         * longBuilder().endLong();
    +         * 
    + */ + public final LongDefault longType() { + return longBuilder().endLong(); + } + + /** + * Build a long type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #longType()}. + */ + public final LongBuilder> longBuilder() { + return LongBuilder.create(wrap(new LongDefault(bldr)), names); + } + + /** + * A plain float type without custom properties. This is equivalent to: + *
    +         * floatBuilder().endFloat();
    +         * 
    + */ + public final FloatDefault floatType() { + return floatBuilder().endFloat(); + } + + /** + * Build a float type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #floatType()}. + */ + public final FloatBuilder> floatBuilder() { + return FloatBuilder.create(wrap(new FloatDefault(bldr)), names); + } + + /** + * A plain double type without custom properties. This is equivalent to: + *
    +         * doubleBuilder().endDouble();
    +         * 
    + */ + public final DoubleDefault doubleType() { + return doubleBuilder().endDouble(); + } + + /** + * Build a double type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #doubleType()}. + */ + public final DoubleBuilder> doubleBuilder() { + return DoubleBuilder.create(wrap(new DoubleDefault(bldr)), names); + } + + /** + * A plain string type without custom properties. This is equivalent to: + *
    +         * stringBuilder().endString();
    +         * 
    + */ + public final StringDefault stringType() { + return stringBuilder().endString(); + } + + /** + * Build a string type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #stringType()}. + */ + public final StringBldr> stringBuilder() { + return StringBldr.create(wrap(new StringDefault(bldr)), names); + } + + /** + * A plain bytes type without custom properties. This is equivalent to: + *
    +         * bytesBuilder().endBytes();
    +         * 
    + */ + public final BytesDefault bytesType() { + return bytesBuilder().endBytes(); + } + + /** + * Build a bytes type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #bytesType()}. + */ + public final BytesBuilder> bytesBuilder() { + return BytesBuilder.create(wrap(new BytesDefault(bldr)), names); + } + + /** + * A plain null type without custom properties. This is equivalent to: + *
    +         * nullBuilder().endNull();
    +         * 
    + */ + public final NullDefault nullType() { + return nullBuilder().endNull(); + } + + /** + * Build a null type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #nullType()}. + */ + public final NullBuilder> nullBuilder() { + return NullBuilder.create(wrap(new NullDefault(bldr)), names); + } + + /** Build an Avro map type **/ + public final MapBuilder> map() { + return MapBuilder.create(wrap(new MapDefault(bldr)), names); + } + + /** Build an Avro array type **/ + public final ArrayBuilder> array() { + return ArrayBuilder.create(wrap(new ArrayDefault(bldr)), names); + } + + /** Build an Avro enum type. **/ + public final EnumBuilder> enumeration(String name) { + return EnumBuilder.create(wrap(new EnumDefault(bldr)), names, name); + } + + private Completion wrap( + Completion completion) { + if (wrapper != null) { + return wrapper.wrap(completion); + } + return completion; + } + } + + /** FieldTypeBuilder adds {@link #unionOf()}, {@link #nullable()}, and {@link #optional()} + * to BaseFieldTypeBuilder. **/ + public static final class FieldTypeBuilder extends BaseFieldTypeBuilder { + private FieldTypeBuilder(FieldBuilder bldr) { + super(bldr, null); + } + + /** Build an Avro union schema type. **/ + public UnionFieldTypeBuilder unionOf() { + return new UnionFieldTypeBuilder(bldr); + } + + /** + * A shortcut for building a union of a type and null, with an optional default + * value of the non-null type. + *

    + * For example, the two code snippets below are equivalent: + *

    nullable().booleanType().booleanDefault(true)
    + *
    unionOf().booleanType().and().nullType().endUnion().booleanDefault(true)
    + **/ + public BaseFieldTypeBuilder nullable() { + return new BaseFieldTypeBuilder(bldr, new NullableCompletionWrapper()); + } + + /** + * A shortcut for building a union of null and a type, with a null default. + *

    + * For example, the two code snippets below are equivalent: + *

    optional().booleanType()
    + *
    unionOf().nullType().and().booleanType().endUnion().nullDefault()
    + */ + public BaseTypeBuilder> optional() { + return new BaseTypeBuilder>( + new OptionalCompletion(bldr), names); + } + } + + /** Builder for a union field. The first type in the union corresponds + * to the possible default value type. + */ + public static final class UnionFieldTypeBuilder { + private final FieldBuilder bldr; + private final NameContext names; + + private UnionFieldTypeBuilder(FieldBuilder bldr) { + this.bldr = bldr; + this.names = bldr.names(); + } + + /** + * A plain boolean type without custom properties. This is equivalent to: + *
    +         * booleanBuilder().endBoolean();
    +         * 
    + */ + public UnionAccumulator> booleanType() { + return booleanBuilder().endBoolean(); + } + + /** + * Build a boolean type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #booleanType()}. + */ + public BooleanBuilder>> booleanBuilder() { + return BooleanBuilder.create(completion(new BooleanDefault(bldr)), names); + } + + /** + * A plain int type without custom properties. This is equivalent to: + *
    +         * intBuilder().endInt();
    +         * 
    + */ + public UnionAccumulator> intType() { + return intBuilder().endInt(); + } + + /** + * Build an int type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #intType()}. + */ + public IntBuilder>> intBuilder() { + return IntBuilder.create(completion(new IntDefault(bldr)), names); + } + + /** + * A plain long type without custom properties. This is equivalent to: + *
    +         * longBuilder().endLong();
    +         * 
    + */ + public UnionAccumulator> longType() { + return longBuilder().endLong(); + } + + /** + * Build a long type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #longType()}. + */ + public LongBuilder>> longBuilder() { + return LongBuilder.create(completion(new LongDefault(bldr)), names); + } + + /** + * A plain float type without custom properties. This is equivalent to: + *
    +         * floatBuilder().endFloat();
    +         * 
    + */ + public UnionAccumulator> floatType() { + return floatBuilder().endFloat(); + } + + /** + * Build a float type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #floatType()}. + */ + public FloatBuilder>> floatBuilder() { + return FloatBuilder.create(completion(new FloatDefault(bldr)), names); + } + + /** + * A plain double type without custom properties. This is equivalent to: + *
    +         * doubleBuilder().endDouble();
    +         * 
    + */ + public UnionAccumulator> doubleType() { + return doubleBuilder().endDouble(); + } + + /** + * Build a double type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #doubleType()}. + */ + public DoubleBuilder>> doubleBuilder() { + return DoubleBuilder.create(completion(new DoubleDefault(bldr)), names); + } + + /** + * A plain string type without custom properties. This is equivalent to: + *
    +         * stringBuilder().endString();
    +         * 
    + */ + public UnionAccumulator> stringType() { + return stringBuilder().endString(); + } + + /** + * Build a string type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #stringType()}. + */ + public StringBldr>> stringBuilder() { + return StringBldr.create(completion(new StringDefault(bldr)), names); + } + + /** + * A plain bytes type without custom properties. This is equivalent to: + *
    +         * bytesBuilder().endBytes();
    +         * 
    + */ + public UnionAccumulator> bytesType() { + return bytesBuilder().endBytes(); + } + + /** + * Build a bytes type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #bytesType()}. + */ + public BytesBuilder>> bytesBuilder() { + return BytesBuilder.create(completion(new BytesDefault(bldr)), names); + } + + /** + * A plain null type without custom properties. This is equivalent to: + *
    +         * nullBuilder().endNull();
    +         * 
    + */ + public UnionAccumulator> nullType() { + return nullBuilder().endNull(); + } + + /** + * Build a null type that can set custom properties. If custom properties + * are not needed it is simpler to use {@link #nullType()}. + */ + public NullBuilder>> nullBuilder() { + return NullBuilder.create(completion(new NullDefault(bldr)), names); + } + + /** Build an Avro map type **/ + public MapBuilder>> map() { + return MapBuilder.create(completion(new MapDefault(bldr)), names); + } + + /** Build an Avro array type **/ + public ArrayBuilder>> array() { + return ArrayBuilder.create(completion(new ArrayDefault(bldr)), names); + } + + /** Build an Avro enum type. **/ + public EnumBuilder>> enumeration(String name) { + return EnumBuilder.create(completion(new EnumDefault(bldr)), names, name); + } + + private UnionCompletion completion(Completion context) { + return new UnionCompletion(context, names, new ArrayList()); + } + } + + public final static class FieldAssembler { + private final List fields = new ArrayList(); + private final Completion context; + private final NameContext names; + private final Schema record; + + private FieldAssembler(Completion context, NameContext names, Schema record) { + this.context = context; + this.names = names; + this.record = record; + } + + /** + * Add a field with the given name. + * @return A {@link FieldBuilder} for the given name. + */ + public FieldBuilder name(String fieldName) { + return new FieldBuilder(this, names, fieldName); + } + + /** + * Shortcut for creating a boolean field with the given name and no default. + *

    This is equivalent to: + *

    +         *   name(fieldName).type().booleanType().noDefault()
    +         * 
    + */ + public FieldAssembler requiredBoolean(String fieldName) { + return name(fieldName).type().booleanType().noDefault(); + } + + /** + * Shortcut for creating an optional boolean field: a union of null and + * boolean with null default.

    + * This is equivalent to: + *

    +         *   name(fieldName).type().optional().booleanType()
    +         * 
    + */ + public FieldAssembler optionalBoolean(String fieldName) { + return name(fieldName).type().optional().booleanType(); + } + + /** + * Shortcut for creating a nullable boolean field: a union of boolean and + * null with an boolean default. + *

    + * This is equivalent to: + * + *

    +         * name(fieldName).type().nullable().booleanType().booleanDefault(defaultVal)
    +         * 
    + */ + public FieldAssembler nullableBoolean(String fieldName, boolean defaultVal) { + return name(fieldName) + .type().nullable().booleanType().booleanDefault(defaultVal); + } + + /** + * Shortcut for creating an int field with the given name and no default. + *

    This is equivalent to: + *

    +         *   name(fieldName).type().intType().noDefault()
    +         * 
    + */ + public FieldAssembler requiredInt(String fieldName) { + return name(fieldName).type().intType().noDefault(); + } + + /** + * Shortcut for creating an optional int field: a union of null and int + * with null default.

    + * This is equivalent to: + *

    +         *   name(fieldName).type().optional().intType()
    +         * 
    + */ + public FieldAssembler optionalInt(String fieldName) { + return name(fieldName).type().optional().intType(); + } + + /** + * Shortcut for creating a nullable int field: a union of int and null + * with an int default.

    + * This is equivalent to: + *

    +         *   name(fieldName).type().nullable().intType().intDefault(defaultVal)
    +         * 
    + */ + public FieldAssembler nullableInt(String fieldName, int defaultVal) { + return name(fieldName).type().nullable().intType().intDefault(defaultVal); + } + + /** + * Shortcut for creating a long field with the given name and no default. + *

    This is equivalent to: + *

    +         *   name(fieldName).type().longType().noDefault()
    +         * 
    + */ + public FieldAssembler requiredLong(String fieldName) { + return name(fieldName).type().longType().noDefault(); + } + + /** + * Shortcut for creating an optional long field: a union of null and long + * with null default.

    + * This is equivalent to: + *

    +         *   name(fieldName).type().optional().longType()
    +         * 
    + */ + public FieldAssembler optionalLong(String fieldName) { + return name(fieldName).type().optional().longType(); + } + + /** + * Shortcut for creating a nullable long field: a union of long and null + * with a long default.

    + * This is equivalent to: + *

    +         *   name(fieldName).type().nullable().longType().longDefault(defaultVal)
    +         * 
    + */ + public FieldAssembler nullableLong(String fieldName, long defaultVal) { + return name(fieldName).type().nullable().longType().longDefault(defaultVal); + } + + /** + * Shortcut for creating a float field with the given name and no default. + *

    This is equivalent to: + *

    +         *   name(fieldName).type().floatType().noDefault()
    +         * 
    + */ + public FieldAssembler requiredFloat(String fieldName) { + return name(fieldName).type().floatType().noDefault(); + } + + /** + * Shortcut for creating an optional float field: a union of null and float + * with null default.

    + * This is equivalent to: + *

    +         *   name(fieldName).type().optional().floatType()
    +         * 
    + */ + public FieldAssembler optionalFloat(String fieldName) { + return name(fieldName).type().optional().floatType(); + } + + /** + * Shortcut for creating a nullable float field: a union of float and null + * with a float default.

    + * This is equivalent to: + *

    +         *   name(fieldName).type().nullable().floatType().floatDefault(defaultVal)
    +         * 
    + */ + public FieldAssembler nullableFloat(String fieldName, float defaultVal) { + return name(fieldName).type().nullable().floatType().floatDefault(defaultVal); + } + + /** + * Shortcut for creating a double field with the given name and no default. + *

    This is equivalent to: + *

    +         *   name(fieldName).type().doubleType().noDefault()
    +         * 
    + */ + public FieldAssembler requiredDouble(String fieldName) { + return name(fieldName).type().doubleType().noDefault(); + } + + /** + * Shortcut for creating an optional double field: a union of null and double + * with null default.

    + * This is equivalent to: + *

    +         *   name(fieldName).type().optional().doubleType()
    +         * 
    + */ + public FieldAssembler optionalDouble(String fieldName) { + return name(fieldName).type().optional().doubleType(); + } + + /** + * Shortcut for creating a nullable double field: a union of double and null + * with a double default.

    + * This is equivalent to: + *

    +         *   name(fieldName).type().nullable().doubleType().doubleDefault(defaultVal)
    +         * 
    + */ + public FieldAssembler nullableDouble(String fieldName, double defaultVal) { + return name(fieldName).type().nullable().doubleType().doubleDefault(defaultVal); + } + + /** + * Shortcut for creating a string field with the given name and no default. + *

    This is equivalent to: + *

    +         *   name(fieldName).type().stringType().noDefault()
    +         * 
    + */ + public FieldAssembler requiredString(String fieldName) { + return name(fieldName).type().stringType().noDefault(); + } + + /** + * Shortcut for creating an optional string field: a union of null and string + * with null default.

    + * This is equivalent to: + *

    +         *   name(fieldName).type().optional().stringType()
    +         * 
    + */ + public FieldAssembler optionalString(String fieldName) { + return name(fieldName).type().optional().stringType(); + } + + /** + * Shortcut for creating a nullable string field: a union of string and null + * with a string default.

    + * This is equivalent to: + *

    +         *   name(fieldName).type().nullable().stringType().stringDefault(defaultVal)
    +         * 
    + */ + public FieldAssembler nullableString(String fieldName, String defaultVal) { + return name(fieldName).type().nullable().stringType().stringDefault(defaultVal); + } + + /** + * Shortcut for creating a bytes field with the given name and no default. + *

    This is equivalent to: + *

    +         *   name(fieldName).type().bytesType().noDefault()
    +         * 
    + */ + public FieldAssembler requiredBytes(String fieldName) { + return name(fieldName).type().bytesType().noDefault(); + } + + /** + * Shortcut for creating an optional bytes field: a union of null and bytes + * with null default.

    + * This is equivalent to: + *

    +         *   name(fieldName).type().optional().bytesType()
    +         * 
    + */ + public FieldAssembler optionalBytes(String fieldName) { + return name(fieldName).type().optional().bytesType(); + } + + /** + * Shortcut for creating a nullable bytes field: a union of bytes and null + * with a bytes default.

    + * This is equivalent to: + *

    +         *   name(fieldName).type().nullable().bytesType().bytesDefault(defaultVal)
    +         * 
    + */ + public FieldAssembler nullableBytes(String fieldName, byte[] defaultVal) { + return name(fieldName).type().nullable().bytesType().bytesDefault(defaultVal); + } + + /** + * End adding fields to this record, returning control + * to the context that this record builder was created in. + */ + public R endRecord() { + record.setFields(fields); + return context.complete(record); + } + + private FieldAssembler addField(Field field) { + fields.add(field); + return this; + } + + } + + /** + * Builds a Field in the context of a {@link FieldAssembler}. + * + * Usage is to first configure any of the optional parameters and then to call one + * of the type methods to complete the field. For example + *
    +     *   .namespace("org.apache.example").orderDecending().type()
    +     * 
    + * Optional parameters for a field are namespace, doc, order, and aliases. + */ + public final static class FieldBuilder extends + NamedBuilder> { + private final FieldAssembler fields; + private Schema.Field.Order order = Schema.Field.Order.ASCENDING; + + private FieldBuilder(FieldAssembler fields, NameContext names, String name) { + super(names, name); + this.fields = fields; + } + + /** Set this field to have ascending order. Ascending is the default **/ + public FieldBuilder orderAscending() { + order = Schema.Field.Order.ASCENDING; + return self(); + } + + /** Set this field to have decending order. Decending is the default **/ + public FieldBuilder orderDescending() { + order = Schema.Field.Order.DESCENDING; + return self(); + } + + /** Set this field to ignore order. **/ + public FieldBuilder orderIgnore() { + order = Schema.Field.Order.IGNORE; + return self(); + } + + /** + * Final step in configuring this field, finalizing name, namespace, alias, + * and order. + * @return A builder for the field's type and default value. + */ + public FieldTypeBuilder type() { + return new FieldTypeBuilder(this); + } + + /** + * Final step in configuring this field, finalizing name, namespace, alias, + * and order. Sets the field's type to the provided schema, returns a + * {@link GenericDefault}. + */ + public GenericDefault type(Schema type) { + return new GenericDefault(this, type); + } + + /** + * Final step in configuring this field, finalizing name, namespace, alias, + * and order. Sets the field's type to the schema by name reference. + *

    + * The name must correspond with a named schema that has already been + * created in the context of this builder. The name may be a fully qualified + * name, or a short name. If it is a short name, the namespace context of + * this builder will be used. + *

    + * The name and namespace context rules are the same as the Avro schema JSON + * specification. + */ + public GenericDefault type(String name) { + return type(name, null); + } + + /** + * Final step in configuring this field, finalizing name, namespace, alias, + * and order. Sets the field's type to the schema by name reference. + *

    + * The name must correspond with a named schema that has already been + * created in the context of this builder. The name may be a fully qualified + * name, or a short name. If it is a full name, the namespace is ignored. If + * it is a short name, the namespace provided is used. If the namespace + * provided is null, the namespace context of this builder will be used. + *

    + * The name and namespace context rules are the same as the Avro schema JSON + * specification. + */ + public GenericDefault type(String name, String namespace) { + Schema schema = names().get(name, namespace); + return type(schema); + } + + private FieldAssembler completeField(Schema schema) { + return completeField(schema, null); + } + + private FieldAssembler completeField(Schema schema, Object defaultVal) { + Field field = new Field(name(), schema, doc(), defaultVal, order); + addPropsTo(field); + addAliasesTo(field); + return fields.addField(field); + } + + @Override + protected FieldBuilder self() { + return this; + } + } + + /** Abstract base class for field defaults. **/ + public static abstract class FieldDefault> extends Completion { + private final FieldBuilder field; + private Schema schema; + + FieldDefault(FieldBuilder field) { + this.field = field; + } + + /** Completes this field with no default value **/ + public final FieldAssembler noDefault() { + return field.completeField(schema); + } + + private FieldAssembler usingDefault(Object defaultVal) { + return field.completeField(schema, defaultVal); + } + + @Override + final S complete(Schema schema) { + this.schema = schema; + return self(); + } + + abstract S self(); + } + + /** Choose whether to use a default value for the field or not. **/ + public static class BooleanDefault extends FieldDefault> { + private BooleanDefault(FieldBuilder field) { + super(field); + } + + /** Completes this field with the default value provided **/ + public final FieldAssembler booleanDefault(boolean defaultVal) { + return super.usingDefault(defaultVal); + } + + @Override + final BooleanDefault self() { + return this; + } + } + + /** Choose whether to use a default value for the field or not. **/ + public static class IntDefault extends FieldDefault> { + private IntDefault(FieldBuilder field) { + super(field); + } + + /** Completes this field with the default value provided **/ + public final FieldAssembler intDefault(int defaultVal) { + return super.usingDefault(defaultVal); + } + + @Override + final IntDefault self() { + return this; + } + } + + /** Choose whether to use a default value for the field or not. **/ + public static class LongDefault extends FieldDefault> { + private LongDefault(FieldBuilder field) { + super(field); + } + + /** Completes this field with the default value provided **/ + public final FieldAssembler longDefault(long defaultVal) { + return super.usingDefault(defaultVal); + } + + @Override + final LongDefault self() { + return this; + } + } + + /** Choose whether to use a default value for the field or not. **/ + public static class FloatDefault extends FieldDefault> { + private FloatDefault(FieldBuilder field) { + super(field); + } + + /** Completes this field with the default value provided **/ + public final FieldAssembler floatDefault(float defaultVal) { + return super.usingDefault(defaultVal); + } + + @Override + final FloatDefault self() { + return this; + } + } + + /** Choose whether to use a default value for the field or not. **/ + public static class DoubleDefault extends FieldDefault> { + private DoubleDefault(FieldBuilder field) { + super(field); + } + + /** Completes this field with the default value provided **/ + public final FieldAssembler doubleDefault(double defaultVal) { + return super.usingDefault(defaultVal); + } + + @Override + final DoubleDefault self() { + return this; + } + } + + /** Choose whether to use a default value for the field or not. **/ + public static class StringDefault extends FieldDefault> { + private StringDefault(FieldBuilder field) { + super(field); + } + + /** Completes this field with the default value provided. Cannot be null. **/ + public final FieldAssembler stringDefault(String defaultVal) { + return super.usingDefault(defaultVal); + } + + @Override + final StringDefault self() { + return this; + } + } + + /** Choose whether to use a default value for the field or not. **/ + public static class BytesDefault extends FieldDefault> { + private BytesDefault(FieldBuilder field) { + super(field); + } + + /** Completes this field with the default value provided, cannot be null **/ + public final FieldAssembler bytesDefault(byte[] defaultVal) { + return super.usingDefault(ByteBuffer.wrap(defaultVal)); + } + + /** Completes this field with the default value provided, cannot be null **/ + public final FieldAssembler bytesDefault(ByteBuffer defaultVal) { + return super.usingDefault(defaultVal); + } + + /** Completes this field with the default value provided, cannot be null. + * The string is interpreted as a byte[], with each character code point + * value equalling the byte value, as in the Avro spec JSON default. **/ + public final FieldAssembler bytesDefault(String defaultVal) { + return super.usingDefault(defaultVal); + } + + @Override + final BytesDefault self() { + return this; + } + } + + /** Choose whether to use a default value for the field or not. **/ + public static class NullDefault extends FieldDefault> { + private NullDefault(FieldBuilder field) { + super(field); + } + + /** Completes this field with a default value of null **/ + public final FieldAssembler nullDefault() { + return super.usingDefault(null); + } + + @Override + final NullDefault self() { + return this; + } + } + + /** Choose whether to use a default value for the field or not. **/ + public static class MapDefault extends FieldDefault> { + private MapDefault(FieldBuilder field) { + super(field); + } + + /** Completes this field with the default value provided, cannot be null **/ + public final FieldAssembler mapDefault(Map defaultVal) { + return super.usingDefault(defaultVal); + } + + @Override + final MapDefault self() { + return this; + } + } + + /** Choose whether to use a default value for the field or not. **/ + public static class ArrayDefault extends FieldDefault> { + private ArrayDefault(FieldBuilder field) { + super(field); + } + + /** Completes this field with the default value provided, cannot be null **/ + public final FieldAssembler arrayDefault(List defaultVal) { + return super.usingDefault(defaultVal); + } + + @Override + final ArrayDefault self() { + return this; + } + } + + /** Choose whether to use a default value for the field or not. **/ + public static class FixedDefault extends FieldDefault> { + private FixedDefault(FieldBuilder field) { + super(field); + } + + /** Completes this field with the default value provided, cannot be null **/ + public final FieldAssembler fixedDefault(byte[] defaultVal) { + return super.usingDefault(ByteBuffer.wrap(defaultVal)); + } + + /** Completes this field with the default value provided, cannot be null **/ + public final FieldAssembler fixedDefault(ByteBuffer defaultVal) { + return super.usingDefault(defaultVal); + } + + /** Completes this field with the default value provided, cannot be null. + * The string is interpreted as a byte[], with each character code point + * value equalling the byte value, as in the Avro spec JSON default. **/ + public final FieldAssembler fixedDefault(String defaultVal) { + return super.usingDefault(defaultVal); + } + + @Override + final FixedDefault self() { + return this; + } + } + + /** Choose whether to use a default value for the field or not. **/ + public static class EnumDefault extends FieldDefault> { + private EnumDefault(FieldBuilder field) { + super(field); + } + + /** Completes this field with the default value provided, cannot be null **/ + public final FieldAssembler enumDefault(String defaultVal) { + return super.usingDefault(defaultVal); + } + + @Override + final EnumDefault self() { + return this; + } + } + + /** Choose whether to use a default value for the field or not. **/ + public static class RecordDefault extends FieldDefault> { + private RecordDefault(FieldBuilder field) { + super(field); + } + + @Override + final RecordDefault self() { + return this; + } + } + + public final static class GenericDefault { + private final FieldBuilder field; + private final Schema schema; + + private GenericDefault(FieldBuilder field, Schema schema) { + this.field = field; + this.schema = schema; + } + + /** Do not use a default value for this field. **/ + public FieldAssembler noDefault() { + return field.completeField(schema); + } + + /** Completes this field with the default value provided. + * The value must conform to the schema of the field. **/ + public FieldAssembler withDefault(Object defaultVal) { + return field.completeField(schema, defaultVal); + } + } + + /** + * Completion is for internal builder use, all subclasses are private. + * + * Completion is an object that takes a Schema and returns some result. + */ + private abstract static class Completion { + abstract R complete(Schema schema); + } + + private static class SchemaCompletion extends Completion { + @Override + protected Schema complete(Schema schema) { + return schema; + } + } + + private static final Schema NULL_SCHEMA = Schema.create(Schema.Type.NULL); + + private static class NullableCompletion extends Completion { + private final Completion context; + + private NullableCompletion(Completion context) { + this.context = context; + } + + @Override + protected R complete(Schema schema) { + // wrap the schema as a union of the schema and null + Schema nullable = Schema.createUnion(Arrays.asList(schema, NULL_SCHEMA)); + return context.complete(nullable); + } + } + + private static class OptionalCompletion extends Completion> { + private final FieldBuilder bldr; + + public OptionalCompletion(FieldBuilder bldr) { + this.bldr = bldr; + } + + @Override + protected FieldAssembler complete(Schema schema) { + // wrap the schema as a union of null and the schema + Schema optional = Schema.createUnion(Arrays.asList(NULL_SCHEMA, schema)); + return bldr.completeField(optional, (Object) null); + } + } + + private abstract static class CompletionWrapper { + abstract Completion wrap(Completion completion); + } + + private static final class NullableCompletionWrapper extends CompletionWrapper { + @Override + Completion wrap(Completion completion) { + return new NullableCompletion(completion); + } + } + + private static abstract class NestedCompletion extends Completion { + private final Completion context; + private final PropBuilder assembler; + + private NestedCompletion(PropBuilder assembler, Completion context) { + this.context = context; + this.assembler = assembler; + } + + @Override + protected final R complete(Schema schema) { + Schema outer = outerSchema(schema); + assembler.addPropsTo(outer); + return context.complete(outer); + } + + protected abstract Schema outerSchema(Schema inner); + } + + private static class MapCompletion extends NestedCompletion { + private MapCompletion(MapBuilder assembler, Completion context) { + super(assembler, context); + } + + @Override + protected Schema outerSchema(Schema inner) { + return Schema.createMap(inner); + } + } + + private static class ArrayCompletion extends NestedCompletion { + private ArrayCompletion(ArrayBuilder assembler, Completion context) { + super(assembler, context); + } + + @Override + protected Schema outerSchema(Schema inner) { + return Schema.createArray(inner); + } + } + + private static class UnionCompletion extends + Completion> { + private final Completion context; + private final NameContext names; + private final List schemas; + + private UnionCompletion(Completion context, NameContext names, List schemas) { + this.context = context; + this.names = names; + this.schemas = schemas; + } + + @Override + protected UnionAccumulator complete(Schema schema) { + List updated = new ArrayList(this.schemas); + updated.add(schema); + return new UnionAccumulator(context, names, updated); + } + } + + /** Accumulates all of the types in a union. Add an additional type with + * {@link #and()}. Complete the union with {@link #endUnion()} + */ + public static final class UnionAccumulator { + private final Completion context; + private final NameContext names; + private final List schemas; + + private UnionAccumulator(Completion context, NameContext names, List schemas) { + this.context = context; + this.names = names; + this.schemas = schemas; + } + + /** Add an additional type to this union **/ + public BaseTypeBuilder> and() { + return new UnionBuilder(context, names, schemas); + } + + /** Complete this union **/ + public R endUnion() { + Schema schema = Schema.createUnion(schemas); + return context.complete(schema); + } + } + + private static void checkRequired(Object reference, String errorMessage) { + if (reference == null) { + throw new NullPointerException(errorMessage); + } + } +} diff --git a/copycat/data/src/main/java/org/apache/kafka/copycat/data/SchemaParseException.java b/copycat/data/src/main/java/org/apache/kafka/copycat/data/SchemaParseException.java new file mode 100644 index 0000000..0982791 --- /dev/null +++ b/copycat/data/src/main/java/org/apache/kafka/copycat/data/SchemaParseException.java @@ -0,0 +1,32 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + + + +package org.apache.kafka.copycat.data; + +/** Thrown for errors parsing schemas and protocols. */ +public class SchemaParseException extends DataRuntimeException { + public SchemaParseException(Throwable cause) { + super(cause); + } + + public SchemaParseException(String message) { + super(message); + } +} + diff --git a/copycat/file/src/main/java/org/apache/kafka/copycat/file/FileStreamSinkConnector.java b/copycat/file/src/main/java/org/apache/kafka/copycat/file/FileStreamSinkConnector.java new file mode 100644 index 0000000..e41364e --- /dev/null +++ b/copycat/file/src/main/java/org/apache/kafka/copycat/file/FileStreamSinkConnector.java @@ -0,0 +1,62 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.file; + +import org.apache.kafka.copycat.connector.Task; +import org.apache.kafka.copycat.sink.SinkConnector; + +import java.util.ArrayList; +import java.util.List; +import java.util.Properties; + +/** + * Very simple connector that works with the console. This connector supports both source and + * sink modes via its 'mode' setting. + */ +public class FileStreamSinkConnector extends SinkConnector { + public static final String FILE_CONFIG = "file"; + + private String filename; + + @Override + public void start(Properties props) { + filename = props.getProperty(FILE_CONFIG); + } + + @Override + public Class getTaskClass() { + return FileStreamSinkTask.class; + } + + @Override + public List getTaskConfigs(int maxTasks) { + ArrayList configs = new ArrayList<>(); + for (int i = 0; i < maxTasks; i++) { + Properties config = new Properties(); + if (filename != null) + config.setProperty(FILE_CONFIG, filename); + configs.add(config); + } + return configs; + } + + @Override + public void stop() { + // Nothing to do since FileStreamSinkConnector has no background monitoring. + } +} diff --git a/copycat/file/src/main/java/org/apache/kafka/copycat/file/FileStreamSinkTask.java b/copycat/file/src/main/java/org/apache/kafka/copycat/file/FileStreamSinkTask.java new file mode 100644 index 0000000..7e4ca7e --- /dev/null +++ b/copycat/file/src/main/java/org/apache/kafka/copycat/file/FileStreamSinkTask.java @@ -0,0 +1,79 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.file; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.copycat.errors.CopycatException; +import org.apache.kafka.copycat.sink.SinkRecord; +import org.apache.kafka.copycat.sink.SinkTask; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.FileNotFoundException; +import java.io.PrintStream; +import java.util.Collection; +import java.util.Map; +import java.util.Properties; + +/** + * FileStreamSinkTask writes records to stdout or a file. + */ +public class FileStreamSinkTask extends SinkTask { + private static final Logger log = LoggerFactory.getLogger(FileStreamSinkTask.class); + + private PrintStream outputStream; + + public FileStreamSinkTask() { + } + + // for testing + public FileStreamSinkTask(PrintStream outputStream) { + this.outputStream = outputStream; + } + + @Override + public void start(Properties props) { + String filename = props.getProperty(FileStreamSinkConnector.FILE_CONFIG); + if (filename == null) { + outputStream = System.out; + } else { + try { + outputStream = new PrintStream(new File(filename)); + } catch (FileNotFoundException e) { + throw new CopycatException("Couldn't find or create file for FileStreamSinkTask: {}", e); + } + } + } + + @Override + public void put(Collection sinkRecords) { + for (SinkRecord record : sinkRecords) { + outputStream.println(record.getValue()); + } + } + + @Override + public void flush(Map offsets) { + outputStream.flush(); + } + + @Override + public void stop() { + } +} diff --git a/copycat/file/src/main/java/org/apache/kafka/copycat/file/FileStreamSourceConnector.java b/copycat/file/src/main/java/org/apache/kafka/copycat/file/FileStreamSourceConnector.java new file mode 100644 index 0000000..4f9d8d0 --- /dev/null +++ b/copycat/file/src/main/java/org/apache/kafka/copycat/file/FileStreamSourceConnector.java @@ -0,0 +1,70 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.file; + +import org.apache.kafka.copycat.connector.Task; +import org.apache.kafka.copycat.errors.CopycatException; +import org.apache.kafka.copycat.source.SourceConnector; + +import java.util.ArrayList; +import java.util.List; +import java.util.Properties; + +/** + * Very simple connector that works with the console. This connector supports both source and + * sink modes via its 'mode' setting. + */ +public class FileStreamSourceConnector extends SourceConnector { + public static final String TOPIC_CONFIG = "topic"; + public static final String FILE_CONFIG = "file"; + + private String filename; + private String topic; + + @Override + public void start(Properties props) { + filename = props.getProperty(FILE_CONFIG); + topic = props.getProperty(TOPIC_CONFIG); + if (topic == null || topic.isEmpty()) + throw new CopycatException("FileStreamSourceConnector configuration must include 'topic' setting"); + if (topic.contains(",")) + throw new CopycatException("FileStreamSourceConnector should only have a single topic when used as a source."); + } + + @Override + public Class getTaskClass() { + return FileStreamSourceTask.class; + } + + @Override + public List getTaskConfigs(int maxTasks) { + ArrayList configs = new ArrayList<>(); + // Only one input stream makes sense. + Properties config = new Properties(); + if (filename != null) + config.setProperty(FILE_CONFIG, filename); + config.setProperty(TOPIC_CONFIG, topic); + configs.add(config); + return configs; + } + + @Override + public void stop() { + // Nothing to do since FileStreamSourceConnector has no background monitoring. + } +} diff --git a/copycat/file/src/main/java/org/apache/kafka/copycat/file/FileStreamSourceTask.java b/copycat/file/src/main/java/org/apache/kafka/copycat/file/FileStreamSourceTask.java new file mode 100644 index 0000000..572ae1f --- /dev/null +++ b/copycat/file/src/main/java/org/apache/kafka/copycat/file/FileStreamSourceTask.java @@ -0,0 +1,176 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.file; + +import org.apache.kafka.copycat.errors.CopycatException; +import org.apache.kafka.copycat.source.SourceRecord; +import org.apache.kafka.copycat.source.SourceTask; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.*; +import java.util.ArrayList; +import java.util.List; +import java.util.Properties; + +/** + * FileStreamSourceTask reads from stdin or a file. + */ +public class FileStreamSourceTask extends SourceTask { + private static final Logger log = LoggerFactory.getLogger(FileStreamSourceTask.class); + + private InputStream stream; + private BufferedReader reader = null; + private char[] buffer = new char[1024]; + private int offset = 0; + private String topic = null; + + private Long streamOffset; + + @Override + public void start(Properties props) { + String filename = props.getProperty(FileStreamSourceConnector.FILE_CONFIG); + if (filename == null) { + stream = System.in; + // Tracking offset for stdin doesn't make sense + streamOffset = null; + } else { + try { + stream = new FileInputStream(filename); + Long lastRecordedOffset = (Long) context.getOffsetStorageReader().getOffset(null); + if (lastRecordedOffset != null) { + log.debug("Found previous offset, trying to skip to file offset {}", lastRecordedOffset); + long skipLeft = lastRecordedOffset; + while (skipLeft > 0) { + try { + long skipped = stream.skip(skipLeft); + skipLeft -= skipped; + } catch (IOException e) { + log.error("Error while trying to seek to previous offset in file: ", e); + throw new CopycatException(e); + } + } + log.debug("Skipped to offset {}", lastRecordedOffset); + } + streamOffset = (lastRecordedOffset != null) ? lastRecordedOffset : 0L; + } catch (FileNotFoundException e) { + throw new CopycatException("Couldn't find file for FileStreamSourceTask: {}", e); + } + } + topic = props.getProperty(FileStreamSourceConnector.TOPIC_CONFIG); + if (topic == null) + throw new CopycatException("ConsoleSourceTask config missing topic setting"); + reader = new BufferedReader(new InputStreamReader(stream)); + } + + @Override + public List poll() throws InterruptedException { + // Unfortunately we can't just use readLine() because it blocks in an uninterruptible way. + // Instead we have to manage splitting lines ourselves, using simple backoff when no new data + // is available. + try { + final BufferedReader readerCopy; + synchronized (this) { + readerCopy = reader; + } + if (readerCopy == null) + return null; + + ArrayList records = null; + + int nread = 0; + while (readerCopy.ready()) { + nread = readerCopy.read(buffer, offset, buffer.length - offset); + + if (nread > 0) { + offset += nread; + if (offset == buffer.length) { + char[] newbuf = new char[buffer.length * 2]; + System.arraycopy(buffer, 0, newbuf, 0, buffer.length); + buffer = newbuf; + } + + String line; + do { + line = extractLine(); + if (line != null) { + if (records == null) + records = new ArrayList<>(); + records.add(new SourceRecord(null, streamOffset, topic, line)); + } + new ArrayList(); + } while (line != null); + } + } + + if (nread <= 0) + Thread.sleep(1); + + return records; + } catch (IOException e) { + // Underlying stream was killed, probably as a result of calling stop. Allow to return + // null, and driving thread will handle any shutdown if necessary. + } + return null; + } + + private String extractLine() { + int until = -1, newStart = -1; + for (int i = 0; i < offset; i++) { + if (buffer[i] == '\n') { + until = i; + newStart = i + 1; + break; + } else if (buffer[i] == '\r') { + // We need to check for \r\n, so we must skip this if we can't check the next char + if (i + 1 >= offset) + return null; + + until = i; + newStart = (buffer[i + 1] == '\n') ? i + 2 : i + 1; + break; + } + } + + if (until != -1) { + String result = new String(buffer, 0, until); + System.arraycopy(buffer, newStart, buffer, 0, buffer.length - newStart); + offset = offset - newStart; + if (streamOffset != null) + streamOffset += newStart; + return result; + } else { + return null; + } + } + + @Override + public void stop() { + log.trace("Stopping"); + synchronized (this) { + try { + stream.close(); + log.trace("Closed input stream"); + } catch (IOException e) { + log.error("Failed to close ConsoleSourceTask stream: ", e); + } + reader = null; + stream = null; + } + } +} diff --git a/copycat/file/src/test/java/org/apache/kafka/copycat/file/FileStreamSinkConnectorTest.java b/copycat/file/src/test/java/org/apache/kafka/copycat/file/FileStreamSinkConnectorTest.java new file mode 100644 index 0000000..643fb43 --- /dev/null +++ b/copycat/file/src/test/java/org/apache/kafka/copycat/file/FileStreamSinkConnectorTest.java @@ -0,0 +1,85 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.file; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.copycat.connector.ConnectorContext; +import org.apache.kafka.copycat.sink.SinkConnector; +import org.junit.Before; +import org.junit.Test; +import org.powermock.api.easymock.PowerMock; + +import java.util.Arrays; +import java.util.List; +import java.util.Properties; + +import static org.junit.Assert.assertEquals; + +public class FileStreamSinkConnectorTest { + + private static final String MULTIPLE_TOPICS = "test1,test2"; + private static final String[] MULTIPLE_TOPICS_LIST + = MULTIPLE_TOPICS.split(","); + private static final List MULTIPLE_TOPICS_PARTITIONS = Arrays.asList( + new TopicPartition("test1", 1), new TopicPartition("test2", 2) + ); + private static final String FILENAME = "/afilename"; + + private FileStreamSinkConnector connector; + private ConnectorContext ctx; + private Properties sinkProperties; + + @Before + public void setup() { + connector = new FileStreamSinkConnector(); + ctx = PowerMock.createMock(ConnectorContext.class); + connector.initialize(ctx); + + sinkProperties = new Properties(); + sinkProperties.setProperty(SinkConnector.TOPICS_CONFIG, MULTIPLE_TOPICS); + sinkProperties.setProperty(FileStreamSinkConnector.FILE_CONFIG, FILENAME); + } + + @Test + public void testSinkTasks() { + PowerMock.replayAll(); + + connector.start(sinkProperties); + List taskConfigs = connector.getTaskConfigs(1); + assertEquals(1, taskConfigs.size()); + assertEquals(FILENAME, taskConfigs.get(0).getProperty(FileStreamSinkConnector.FILE_CONFIG)); + + taskConfigs = connector.getTaskConfigs(2); + assertEquals(2, taskConfigs.size()); + for (int i = 0; i < 2; i++) { + assertEquals(FILENAME, taskConfigs.get(0).getProperty(FileStreamSinkConnector.FILE_CONFIG)); + } + + PowerMock.verifyAll(); + } + + @Test + public void testTaskClass() { + PowerMock.replayAll(); + + connector.start(sinkProperties); + assertEquals(FileStreamSinkTask.class, connector.getTaskClass()); + + PowerMock.verifyAll(); + } +} diff --git a/copycat/file/src/test/java/org/apache/kafka/copycat/file/FileStreamSinkTaskTest.java b/copycat/file/src/test/java/org/apache/kafka/copycat/file/FileStreamSinkTaskTest.java new file mode 100644 index 0000000..b4e1b0c --- /dev/null +++ b/copycat/file/src/test/java/org/apache/kafka/copycat/file/FileStreamSinkTaskTest.java @@ -0,0 +1,67 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.file; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.copycat.sink.SinkRecord; +import org.junit.Before; +import org.junit.Test; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import java.util.Arrays; +import java.util.HashMap; + +import static org.junit.Assert.assertEquals; + +public class FileStreamSinkTaskTest { + + private FileStreamSinkTask task; + private ByteArrayOutputStream os; + private PrintStream printStream; + + @Before + public void setup() { + os = new ByteArrayOutputStream(); + printStream = new PrintStream(os); + task = new FileStreamSinkTask(printStream); + } + + @Test + public void testPutFlush() { + HashMap offsets = new HashMap<>(); + + // We do not call task.start() since it would override the output stream + + task.put(Arrays.asList( + new SinkRecord("topic1", 0, null, "line1", 1) + )); + offsets.put(new TopicPartition("topic1", 0), 1L); + task.flush(offsets); + assertEquals("line1\n", os.toString()); + + task.put(Arrays.asList( + new SinkRecord("topic1", 0, null, "line2", 2), + new SinkRecord("topic2", 0, null, "line3", 1) + )); + offsets.put(new TopicPartition("topic1", 0), 2L); + offsets.put(new TopicPartition("topic2", 0), 1L); + task.flush(offsets); + assertEquals("line1\nline2\nline3\n", os.toString()); + } +} diff --git a/copycat/file/src/test/java/org/apache/kafka/copycat/file/FileStreamSourceConnectorTest.java b/copycat/file/src/test/java/org/apache/kafka/copycat/file/FileStreamSourceConnectorTest.java new file mode 100644 index 0000000..e23055c --- /dev/null +++ b/copycat/file/src/test/java/org/apache/kafka/copycat/file/FileStreamSourceConnectorTest.java @@ -0,0 +1,104 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.file; + +import org.apache.kafka.copycat.connector.ConnectorContext; +import org.apache.kafka.copycat.errors.CopycatException; +import org.junit.Before; +import org.junit.Test; +import org.powermock.api.easymock.PowerMock; + +import java.util.List; +import java.util.Properties; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +public class FileStreamSourceConnectorTest { + + private static final String SINGLE_TOPIC = "test"; + private static final String MULTIPLE_TOPICS = "test1,test2"; + private static final String FILENAME = "/somefilename"; + + private FileStreamSourceConnector connector; + private ConnectorContext ctx; + private Properties sourceProperties; + + @Before + public void setup() { + connector = new FileStreamSourceConnector(); + ctx = PowerMock.createMock(ConnectorContext.class); + connector.initialize(ctx); + + sourceProperties = new Properties(); + sourceProperties.setProperty(FileStreamSourceConnector.TOPIC_CONFIG, SINGLE_TOPIC); + sourceProperties.setProperty(FileStreamSourceConnector.FILE_CONFIG, FILENAME); + } + + @Test + public void testSourceTasks() { + PowerMock.replayAll(); + + connector.start(sourceProperties); + List taskConfigs = connector.getTaskConfigs(1); + assertEquals(1, taskConfigs.size()); + assertEquals(FILENAME, + taskConfigs.get(0).getProperty(FileStreamSourceConnector.FILE_CONFIG)); + assertEquals(SINGLE_TOPIC, + taskConfigs.get(0).getProperty(FileStreamSourceConnector.TOPIC_CONFIG)); + + // Should be able to return fewer than requested # + taskConfigs = connector.getTaskConfigs(2); + assertEquals(1, taskConfigs.size()); + assertEquals(FILENAME, + taskConfigs.get(0).getProperty(FileStreamSourceConnector.FILE_CONFIG)); + assertEquals(SINGLE_TOPIC, + taskConfigs.get(0).getProperty(FileStreamSourceConnector.TOPIC_CONFIG)); + + PowerMock.verifyAll(); + } + + @Test + public void testSourceTasksStdin() { + PowerMock.replayAll(); + + sourceProperties.remove(FileStreamSourceConnector.FILE_CONFIG); + connector.start(sourceProperties); + List taskConfigs = connector.getTaskConfigs(1); + assertEquals(1, taskConfigs.size()); + assertNull(taskConfigs.get(0).getProperty(FileStreamSourceConnector.FILE_CONFIG)); + + PowerMock.verifyAll(); + } + + @Test(expected = CopycatException.class) + public void testMultipleSourcesInvalid() { + sourceProperties.setProperty(FileStreamSourceConnector.TOPIC_CONFIG, MULTIPLE_TOPICS); + connector.start(sourceProperties); + } + + @Test + public void testTaskClass() { + PowerMock.replayAll(); + + connector.start(sourceProperties); + assertEquals(FileStreamSourceTask.class, connector.getTaskClass()); + + PowerMock.verifyAll(); + } +} diff --git a/copycat/file/src/test/java/org/apache/kafka/copycat/file/FileStreamSourceTaskTest.java b/copycat/file/src/test/java/org/apache/kafka/copycat/file/FileStreamSourceTaskTest.java new file mode 100644 index 0000000..0ec71d3 --- /dev/null +++ b/copycat/file/src/test/java/org/apache/kafka/copycat/file/FileStreamSourceTaskTest.java @@ -0,0 +1,140 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.file; + +import org.apache.kafka.copycat.errors.CopycatException; +import org.apache.kafka.copycat.source.SourceRecord; +import org.apache.kafka.copycat.source.SourceTaskContext; +import org.apache.kafka.copycat.storage.OffsetStorageReader; +import org.easymock.EasyMock; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.powermock.api.easymock.PowerMock; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.util.List; +import java.util.Properties; + +import static org.junit.Assert.assertEquals; + +public class FileStreamSourceTaskTest { + + private static final String TOPIC = "test"; + + private File tempFile; + private Properties config; + private OffsetStorageReader offsetStorageReader; + private FileStreamSourceTask task; + + private boolean verifyMocks = false; + + @Before + public void setup() throws IOException { + tempFile = File.createTempFile("file-stream-source-task-test", null); + config = new Properties(); + config.setProperty(FileStreamSourceConnector.FILE_CONFIG, tempFile.getAbsolutePath()); + config.setProperty(FileStreamSourceConnector.TOPIC_CONFIG, TOPIC); + task = new FileStreamSourceTask(); + offsetStorageReader = PowerMock.createMock(OffsetStorageReader.class); + task.initialize(new SourceTaskContext(offsetStorageReader)); + } + + @After + public void teardown() { + tempFile.delete(); + + if (verifyMocks) + PowerMock.verifyAll(); + } + + private void replay() { + PowerMock.replayAll(); + verifyMocks = true; + } + + @Test + public void testNormalLifecycle() throws InterruptedException, IOException { + expectOffsetLookupReturnNone(); + replay(); + + task.start(config); + + FileOutputStream os = new FileOutputStream(tempFile); + assertEquals(null, task.poll()); + os.write("partial line".getBytes()); + os.flush(); + assertEquals(null, task.poll()); + os.write(" finished\n".getBytes()); + os.flush(); + List records = task.poll(); + assertEquals(1, records.size()); + assertEquals(TOPIC, records.get(0).getTopic()); + assertEquals("partial line finished", records.get(0).getValue()); + assertEquals(22L, records.get(0).getSourceOffset()); + assertEquals(null, task.poll()); + + // Different line endings, and make sure the final \r doesn't result in a line until we can + // read the subsequent byte. + os.write("line1\rline2\r\nline3\nline4\n\r".getBytes()); + os.flush(); + records = task.poll(); + assertEquals(4, records.size()); + assertEquals("line1", records.get(0).getValue()); + assertEquals(28L, records.get(0).getSourceOffset()); + assertEquals("line2", records.get(1).getValue()); + assertEquals(35L, records.get(1).getSourceOffset()); + assertEquals("line3", records.get(2).getValue()); + assertEquals(41L, records.get(2).getSourceOffset()); + assertEquals("line4", records.get(3).getValue()); + assertEquals(47L, records.get(3).getSourceOffset()); + + os.write("subsequent text".getBytes()); + os.flush(); + records = task.poll(); + assertEquals(1, records.size()); + assertEquals("", records.get(0).getValue()); + assertEquals(48L, records.get(0).getSourceOffset()); + + task.stop(); + } + + @Test(expected = CopycatException.class) + public void testMissingTopic() { + expectOffsetLookupReturnNone(); + replay(); + + config.remove(FileStreamSourceConnector.TOPIC_CONFIG); + task.start(config); + } + + @Test(expected = CopycatException.class) + public void testInvalidFile() { + config.setProperty(FileStreamSourceConnector.FILE_CONFIG, "bogusfilename"); + task.start(config); + } + + + private void expectOffsetLookupReturnNone() { + EasyMock.expect( + offsetStorageReader.getOffset(EasyMock.anyObject(Object.class))) + .andReturn(null); + } +} \ No newline at end of file diff --git a/copycat/json/src/main/java/org/apache/kafka/copycat/json/JsonConverter.java b/copycat/json/src/main/java/org/apache/kafka/copycat/json/JsonConverter.java new file mode 100644 index 0000000..36a6ca8 --- /dev/null +++ b/copycat/json/src/main/java/org/apache/kafka/copycat/json/JsonConverter.java @@ -0,0 +1,265 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.json; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.ArrayNode; +import com.fasterxml.jackson.databind.node.JsonNodeFactory; +import com.fasterxml.jackson.databind.node.ObjectNode; +import org.apache.kafka.copycat.data.*; +import org.apache.kafka.copycat.errors.CopycatException; +import org.apache.kafka.copycat.storage.Converter; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.*; + +/** + * Implementation of Converter that uses JSON to store schemas and objects. + */ +public class JsonConverter implements Converter { + + private static final HashMap TO_COPYCAT_CONVERTERS + = new HashMap<>(); + + static { + TO_COPYCAT_CONVERTERS.put(JsonSchema.BOOLEAN_TYPE_NAME, new JsonToCopycatTypeConverter() { + @Override + public Object convert(JsonNode jsonSchema, JsonNode value) { + return value.booleanValue(); + } + }); + TO_COPYCAT_CONVERTERS.put(JsonSchema.INT_TYPE_NAME, new JsonToCopycatTypeConverter() { + @Override + public Object convert(JsonNode jsonSchema, JsonNode value) { + return value.intValue(); + } + }); + TO_COPYCAT_CONVERTERS.put(JsonSchema.LONG_TYPE_NAME, new JsonToCopycatTypeConverter() { + @Override + public Object convert(JsonNode jsonSchema, JsonNode value) { + return value.longValue(); + } + }); + TO_COPYCAT_CONVERTERS.put(JsonSchema.FLOAT_TYPE_NAME, new JsonToCopycatTypeConverter() { + @Override + public Object convert(JsonNode jsonSchema, JsonNode value) { + return value.floatValue(); + } + }); + TO_COPYCAT_CONVERTERS.put(JsonSchema.DOUBLE_TYPE_NAME, new JsonToCopycatTypeConverter() { + @Override + public Object convert(JsonNode jsonSchema, JsonNode value) { + return value.doubleValue(); + } + }); + TO_COPYCAT_CONVERTERS.put(JsonSchema.BYTES_TYPE_NAME, new JsonToCopycatTypeConverter() { + @Override + public Object convert(JsonNode jsonSchema, JsonNode value) { + try { + return value.binaryValue(); + } catch (IOException e) { + throw new CopycatException("Invalid bytes field", e); + } + } + }); + TO_COPYCAT_CONVERTERS.put(JsonSchema.STRING_TYPE_NAME, new JsonToCopycatTypeConverter() { + @Override + public Object convert(JsonNode jsonSchema, JsonNode value) { + return value.textValue(); + } + }); + TO_COPYCAT_CONVERTERS.put(JsonSchema.ARRAY_TYPE_NAME, new JsonToCopycatTypeConverter() { + @Override + public Object convert(JsonNode jsonSchema, JsonNode value) { + JsonNode elemSchema = jsonSchema.get(JsonSchema.ARRAY_ITEMS_FIELD_NAME); + if (elemSchema == null) + throw new CopycatException("Array schema did not specify the element type"); + ArrayList result = new ArrayList<>(); + for (JsonNode elem : value) { + result.add(convertToCopycat(elemSchema, elem)); + } + return result; + } + }); + + } + + @Override + public JsonNode fromCopycatData(Object value) { + return convertToJsonWithSchemaEnvelope(value); + } + + @Override + public Object toCopycatData(JsonNode value) { + if (!value.isObject() || value.size() != 2 || !value.has(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME) || !value.has(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME)) + throw new CopycatException("JSON value converted to Copycat must be in envelope containing schema"); + + return convertToCopycat(value.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME), value.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME)); + } + + + private static JsonNode asJsonSchema(Schema schema) { + switch (schema.getType()) { + case BOOLEAN: + return JsonSchema.BOOLEAN_SCHEMA; + case BYTES: + return JsonSchema.BYTES_SCHEMA; + case DOUBLE: + return JsonSchema.DOUBLE_SCHEMA; + case FLOAT: + return JsonSchema.FLOAT_SCHEMA; + case INT: + return JsonSchema.INT_SCHEMA; + case LONG: + return JsonSchema.LONG_SCHEMA; + case NULL: + throw new UnsupportedOperationException("null schema not supported"); + case STRING: + return JsonSchema.STRING_SCHEMA; + case UNION: { + throw new UnsupportedOperationException("union schema not supported"); + } + case ARRAY: + return JsonNodeFactory.instance.objectNode().put(JsonSchema.SCHEMA_TYPE_FIELD_NAME, JsonSchema.ARRAY_TYPE_NAME) + .set(JsonSchema.ARRAY_ITEMS_FIELD_NAME, asJsonSchema(schema.getElementType())); + case ENUM: + throw new UnsupportedOperationException("enum schema not supported"); + case MAP: + throw new UnsupportedOperationException("map schema not supported"); + default: + throw new CopycatException("Couldn't translate unsupported schema type " + schema.getType().getName() + "."); + } + } + + + private static Schema asCopycatSchema(JsonNode jsonSchema) { + if (jsonSchema.isNull()) + return null; + + JsonNode schemaTypeNode = jsonSchema.get(JsonSchema.SCHEMA_TYPE_FIELD_NAME); + if (schemaTypeNode == null || !schemaTypeNode.isTextual()) + throw new CopycatException("Schema must contain 'type' field"); + + switch (schemaTypeNode.textValue()) { + case JsonSchema.BOOLEAN_TYPE_NAME: + return SchemaBuilder.builder().booleanType(); + case JsonSchema.INT_TYPE_NAME: + return SchemaBuilder.builder().intType(); + case JsonSchema.LONG_TYPE_NAME: + return SchemaBuilder.builder().longType(); + case JsonSchema.FLOAT_TYPE_NAME: + return SchemaBuilder.builder().floatType(); + case JsonSchema.DOUBLE_TYPE_NAME: + return SchemaBuilder.builder().doubleType(); + case JsonSchema.BYTES_TYPE_NAME: + return SchemaBuilder.builder().bytesType(); + case JsonSchema.STRING_TYPE_NAME: + return SchemaBuilder.builder().stringType(); + case JsonSchema.ARRAY_TYPE_NAME: + JsonNode elemSchema = jsonSchema.get(JsonSchema.ARRAY_ITEMS_FIELD_NAME); + if (elemSchema == null) + throw new CopycatException("Array schema did not specify the element type"); + return Schema.createArray(asCopycatSchema(elemSchema)); + default: + throw new CopycatException("Unknown schema type: " + schemaTypeNode.textValue()); + } + } + + + /** + * Convert this object, in org.apache.kafka.copycat.data format, into a JSON object with an envelope object + * containing schema and payload fields. + * @param value + * @return + */ + private static JsonNode convertToJsonWithSchemaEnvelope(Object value) { + return convertToJson(value).toJsonNode(); + } + + /** + * Convert this object, in the org.apache.kafka.copycat.data format, into a JSON object, returning both the schema + * and the converted object. + */ + private static JsonSchema.Envelope convertToJson(Object value) { + if (value == null) { + return JsonSchema.nullEnvelope(); + } else if (value instanceof Boolean) { + return JsonSchema.booleanEnvelope((Boolean) value); + } else if (value instanceof Byte) { + return JsonSchema.intEnvelope((Byte) value); + } else if (value instanceof Short) { + return JsonSchema.intEnvelope((Short) value); + } else if (value instanceof Integer) { + return JsonSchema.intEnvelope((Integer) value); + } else if (value instanceof Long) { + return JsonSchema.longEnvelope((Long) value); + } else if (value instanceof Float) { + return JsonSchema.floatEnvelope((Float) value); + } else if (value instanceof Double) { + return JsonSchema.doubleEnvelope((Double) value); + } else if (value instanceof byte[]) { + return JsonSchema.bytesEnvelope((byte[]) value); + } else if (value instanceof ByteBuffer) { + return JsonSchema.bytesEnvelope(((ByteBuffer) value).array()); + } else if (value instanceof CharSequence) { + return JsonSchema.stringEnvelope(value.toString()); + } else if (value instanceof Collection) { + Collection collection = (Collection) value; + ObjectNode schema = JsonNodeFactory.instance.objectNode().put(JsonSchema.SCHEMA_TYPE_FIELD_NAME, JsonSchema.ARRAY_TYPE_NAME); + ArrayNode list = JsonNodeFactory.instance.arrayNode(); + JsonNode itemSchema = null; + for (Object elem : collection) { + JsonSchema.Envelope fieldSchemaAndValue = convertToJson(elem); + if (itemSchema == null) { + itemSchema = fieldSchemaAndValue.schema; + schema.set(JsonSchema.ARRAY_ITEMS_FIELD_NAME, itemSchema); + } else { + if (!itemSchema.equals(fieldSchemaAndValue.schema)) + throw new CopycatException("Mismatching schemas found in a list."); + } + + list.add(fieldSchemaAndValue.payload); + } + return new JsonSchema.Envelope(schema, list); + } + + throw new CopycatException("Couldn't convert " + value + " to Avro."); + } + + + private static Object convertToCopycat(JsonNode jsonSchema, JsonNode jsonValue) { + if (jsonSchema.isNull()) + return null; + + JsonNode schemaTypeNode = jsonSchema.get(JsonSchema.SCHEMA_TYPE_FIELD_NAME); + if (schemaTypeNode == null || !schemaTypeNode.isTextual()) + throw new CopycatException("Schema must contain 'type' field. Schema: " + jsonSchema.toString()); + + JsonToCopycatTypeConverter typeConverter = TO_COPYCAT_CONVERTERS.get(schemaTypeNode.textValue()); + if (typeConverter != null) + return typeConverter.convert(jsonSchema, jsonValue); + + throw new CopycatException("Unknown schema type: " + schemaTypeNode); + } + + + private interface JsonToCopycatTypeConverter { + Object convert(JsonNode schema, JsonNode value); + } +} diff --git a/copycat/json/src/main/java/org/apache/kafka/copycat/json/JsonDeserializer.java b/copycat/json/src/main/java/org/apache/kafka/copycat/json/JsonDeserializer.java new file mode 100644 index 0000000..29c7bac --- /dev/null +++ b/copycat/json/src/main/java/org/apache/kafka/copycat/json/JsonDeserializer.java @@ -0,0 +1,87 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ +package org.apache.kafka.copycat.json; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ArrayNode; +import com.fasterxml.jackson.databind.node.JsonNodeFactory; +import com.fasterxml.jackson.databind.node.ObjectNode; +import org.apache.kafka.common.errors.SerializationException; +import org.apache.kafka.common.serialization.Deserializer; + +import java.util.Map; + +/** + * JSON deserializer for Jackson's JsonNode tree model. Using the tree model allows it to work with arbitrarily + * structured data without having associated Java classes. This deserializer also supports Copycat schemas. + */ +public class JsonDeserializer implements Deserializer { + private static final ObjectNode CATCH_ALL_OBJECT_SCHEMA = JsonNodeFactory.instance.objectNode(); + private static final ObjectNode CATCH_ALL_ARRAY_SCHEMA = JsonNodeFactory.instance.objectNode(); + private static final ArrayNode ALL_SCHEMAS_LIST = JsonNodeFactory.instance.arrayNode(); + private static final ObjectNode CATCH_ALL_SCHEMA = JsonNodeFactory.instance.objectNode(); + static { + CATCH_ALL_OBJECT_SCHEMA.put("type", "object") + .putArray("field").add(JsonNodeFactory.instance.objectNode().put("*", "all")); + + CATCH_ALL_ARRAY_SCHEMA.put("type", "array").put("items", "all"); + + ALL_SCHEMAS_LIST.add("boolean").add("int").add("long").add("float").add("double").add("bytes").add("string") + .add(CATCH_ALL_ARRAY_SCHEMA).add(CATCH_ALL_OBJECT_SCHEMA); + + CATCH_ALL_SCHEMA.put("name", "all").set("type", ALL_SCHEMAS_LIST); + } + + private ObjectMapper objectMapper = new ObjectMapper(); + + /** + * Default constructor needed by Kafka + */ + public JsonDeserializer() { + } + + @Override + public void configure(Map props, boolean isKey) { + } + + @Override + public JsonNode deserialize(String topic, byte[] bytes) { + JsonNode data; + try { + data = objectMapper.readTree(bytes); + } catch (Exception e) { + throw new SerializationException(e); + } + + // The deserialized data should either be an envelope object containing the schema and the payload or the schema + // was stripped during serialization and we need to fill in an all-encompassing schema. + if (!data.isObject() || data.size() != 2 || !data.has("schema") || !data.has("payload")) { + ObjectNode envelope = JsonNodeFactory.instance.objectNode(); + envelope.set("schema", CATCH_ALL_SCHEMA); + envelope.set("payload", data); + data = envelope; + } + + return data; + } + + @Override + public void close() { + + } +} diff --git a/copycat/json/src/main/java/org/apache/kafka/copycat/json/JsonSchema.java b/copycat/json/src/main/java/org/apache/kafka/copycat/json/JsonSchema.java new file mode 100644 index 0000000..a807e0f --- /dev/null +++ b/copycat/json/src/main/java/org/apache/kafka/copycat/json/JsonSchema.java @@ -0,0 +1,114 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.json; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.JsonNodeFactory; +import com.fasterxml.jackson.databind.node.ObjectNode; + +import java.nio.ByteBuffer; + +public class JsonSchema { + + static final String ENVELOPE_SCHEMA_FIELD_NAME = "schema"; + static final String ENVELOPE_PAYLOAD_FIELD_NAME = "payload"; + static final String SCHEMA_TYPE_FIELD_NAME = "type"; + static final String SCHEMA_NAME_FIELD_NAME = "name"; + static final String ARRAY_ITEMS_FIELD_NAME = "items"; + static final String BOOLEAN_TYPE_NAME = "boolean"; + static final JsonNode BOOLEAN_SCHEMA = JsonNodeFactory.instance.objectNode().put(SCHEMA_TYPE_FIELD_NAME, BOOLEAN_TYPE_NAME); + static final String INT_TYPE_NAME = "int"; + static final JsonNode INT_SCHEMA = JsonNodeFactory.instance.objectNode().put(SCHEMA_TYPE_FIELD_NAME, INT_TYPE_NAME); + static final String LONG_TYPE_NAME = "long"; + static final JsonNode LONG_SCHEMA = JsonNodeFactory.instance.objectNode().put(SCHEMA_TYPE_FIELD_NAME, LONG_TYPE_NAME); + static final String FLOAT_TYPE_NAME = "float"; + static final JsonNode FLOAT_SCHEMA = JsonNodeFactory.instance.objectNode().put(SCHEMA_TYPE_FIELD_NAME, FLOAT_TYPE_NAME); + static final String DOUBLE_TYPE_NAME = "double"; + static final JsonNode DOUBLE_SCHEMA = JsonNodeFactory.instance.objectNode().put(SCHEMA_TYPE_FIELD_NAME, DOUBLE_TYPE_NAME); + static final String BYTES_TYPE_NAME = "bytes"; + static final JsonNode BYTES_SCHEMA = JsonNodeFactory.instance.objectNode().put(SCHEMA_TYPE_FIELD_NAME, BYTES_TYPE_NAME); + static final String STRING_TYPE_NAME = "string"; + static final JsonNode STRING_SCHEMA = JsonNodeFactory.instance.objectNode().put(SCHEMA_TYPE_FIELD_NAME, STRING_TYPE_NAME); + static final String ARRAY_TYPE_NAME = "array"; + + public static ObjectNode envelope(JsonNode schema, JsonNode payload) { + ObjectNode result = JsonNodeFactory.instance.objectNode(); + result.set(ENVELOPE_SCHEMA_FIELD_NAME, schema); + result.set(ENVELOPE_PAYLOAD_FIELD_NAME, payload); + return result; + } + + static class Envelope { + public JsonNode schema; + public JsonNode payload; + + public Envelope(JsonNode schema, JsonNode payload) { + this.schema = schema; + this.payload = payload; + } + + public ObjectNode toJsonNode() { + return envelope(schema, payload); + } + } + + + public static Envelope nullEnvelope() { + return new Envelope(null, null); + } + + public static Envelope booleanEnvelope(boolean value) { + return new Envelope(JsonSchema.BOOLEAN_SCHEMA, JsonNodeFactory.instance.booleanNode(value)); + } + + public static Envelope intEnvelope(byte value) { + return new Envelope(JsonSchema.INT_SCHEMA, JsonNodeFactory.instance.numberNode(value)); + } + + public static Envelope intEnvelope(short value) { + return new Envelope(JsonSchema.INT_SCHEMA, JsonNodeFactory.instance.numberNode(value)); + } + + public static Envelope intEnvelope(int value) { + return new Envelope(JsonSchema.INT_SCHEMA, JsonNodeFactory.instance.numberNode(value)); + } + + public static Envelope longEnvelope(long value) { + return new Envelope(JsonSchema.LONG_SCHEMA, JsonNodeFactory.instance.numberNode(value)); + } + + public static Envelope floatEnvelope(float value) { + return new Envelope(JsonSchema.FLOAT_SCHEMA, JsonNodeFactory.instance.numberNode(value)); + } + + public static Envelope doubleEnvelope(double value) { + return new Envelope(JsonSchema.DOUBLE_SCHEMA, JsonNodeFactory.instance.numberNode(value)); + } + + public static Envelope bytesEnvelope(byte[] value) { + return new Envelope(JsonSchema.BYTES_SCHEMA, JsonNodeFactory.instance.binaryNode(value)); + } + + public static Envelope bytesEnvelope(ByteBuffer value) { + return new Envelope(JsonSchema.BYTES_SCHEMA, JsonNodeFactory.instance.binaryNode(value.array())); + } + + public static Envelope stringEnvelope(CharSequence value) { + return new Envelope(JsonSchema.STRING_SCHEMA, JsonNodeFactory.instance.textNode(value.toString())); + } +} diff --git a/copycat/json/src/main/java/org/apache/kafka/copycat/json/JsonSerializer.java b/copycat/json/src/main/java/org/apache/kafka/copycat/json/JsonSerializer.java new file mode 100644 index 0000000..dcac270 --- /dev/null +++ b/copycat/json/src/main/java/org/apache/kafka/copycat/json/JsonSerializer.java @@ -0,0 +1,72 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ +package org.apache.kafka.copycat.json; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.kafka.common.errors.SerializationException; +import org.apache.kafka.common.serialization.Serializer; + +import java.util.Map; + +/** + * Serialize Jackson JsonNode tree model objects to UTF-8 JSON. Using the tree model allows handling arbitrarily + * structured data without corresponding Java classes. This serializer also supports Copycat schemas. + */ +public class JsonSerializer implements Serializer { + + private static final String SCHEMAS_ENABLE_CONFIG = "schemas.enable"; + private static final boolean SCHEMAS_ENABLE_DEFAULT = true; + + private final ObjectMapper objectMapper = new ObjectMapper(); + private boolean enableSchemas = SCHEMAS_ENABLE_DEFAULT; + + /** + * Default constructor needed by Kafka + */ + public JsonSerializer() { + + } + + @Override + public void configure(Map config, boolean isKey) { + Object enableConfigsVal = config.get(SCHEMAS_ENABLE_CONFIG); + if (enableConfigsVal != null) + enableSchemas = enableConfigsVal.toString().equals("true"); + } + + @Override + public byte[] serialize(String topic, JsonNode data) { + // This serializer works for Copycat data that requires a schema to be included, so we expect it to have a + // specific format: { "schema": {...}, "payload": ... }. + if (!data.isObject() || data.size() != 2 || !data.has("schema") || !data.has("payload")) + throw new SerializationException("JsonSerializer requires \"schema\" and \"payload\" fields and may not contain additional fields"); + + try { + if (!enableSchemas) + data = data.get("payload"); + return objectMapper.writeValueAsBytes(data); + } catch (Exception e) { + throw new SerializationException("Error serializing JSON message", e); + } + } + + @Override + public void close() { + } + +} diff --git a/copycat/json/src/test/java/org/apache/kafka/copycat/json/JsonConverterTest.java b/copycat/json/src/test/java/org/apache/kafka/copycat/json/JsonConverterTest.java new file mode 100644 index 0000000..1a725c9 --- /dev/null +++ b/copycat/json/src/test/java/org/apache/kafka/copycat/json/JsonConverterTest.java @@ -0,0 +1,173 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.json; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.JsonNodeFactory; +import org.junit.Test; + +import java.io.IOException; +import java.io.UnsupportedEncodingException; +import java.nio.ByteBuffer; +import java.util.Arrays; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +public class JsonConverterTest { + + ObjectMapper objectMapper = new ObjectMapper(); + JsonConverter converter = new JsonConverter(); + + @Test + public void booleanToCopycat() { + assertEquals(true, converter.toCopycatData(parse("{ \"schema\": { \"type\": \"boolean\" }, \"payload\": true }"))); + assertEquals(false, converter.toCopycatData(parse("{ \"schema\": { \"type\": \"boolean\" }, \"payload\": false }"))); + } + + @Test + public void intToCopycat() { + assertEquals(12, converter.toCopycatData(parse("{ \"schema\": { \"type\": \"int\" }, \"payload\": 12 }"))); + } + + @Test + public void longToCopycat() { + assertEquals(12L, converter.toCopycatData(parse("{ \"schema\": { \"type\": \"long\" }, \"payload\": 12 }"))); + assertEquals(4398046511104L, converter.toCopycatData(parse("{ \"schema\": { \"type\": \"long\" }, \"payload\": 4398046511104 }"))); + } + + @Test + public void floatToCopycat() { + assertEquals(12.34f, converter.toCopycatData(parse("{ \"schema\": { \"type\": \"float\" }, \"payload\": 12.34 }"))); + } + + @Test + public void doubleToCopycat() { + assertEquals(12.34, converter.toCopycatData(parse("{ \"schema\": { \"type\": \"double\" }, \"payload\": 12.34 }"))); + } + + + @Test + public void bytesToCopycat() throws UnsupportedEncodingException { + ByteBuffer reference = ByteBuffer.wrap("test-string".getBytes("UTF-8")); + String msg = "{ \"schema\": { \"type\": \"bytes\" }, \"payload\": \"dGVzdC1zdHJpbmc=\" }"; + ByteBuffer converted = ByteBuffer.wrap((byte[]) converter.toCopycatData(parse(msg))); + assertEquals(reference, converted); + } + + @Test + public void stringToCopycat() { + assertEquals("foo-bar-baz", converter.toCopycatData(parse("{ \"schema\": { \"type\": \"string\" }, \"payload\": \"foo-bar-baz\" }"))); + } + + @Test + public void arrayToCopycat() { + JsonNode arrayJson = parse("{ \"schema\": { \"type\": \"array\", \"items\": { \"type\" : \"int\" } }, \"payload\": [1, 2, 3] }"); + assertEquals(Arrays.asList(1, 2, 3), converter.toCopycatData(arrayJson)); + } + + + @Test + public void booleanToJson() { + JsonNode converted = converter.fromCopycatData(true); + validateEnvelope(converted); + assertEquals(parse("{ \"type\": \"boolean\" }"), converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME)); + assertEquals(true, converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).booleanValue()); + } + + @Test + public void intToJson() { + JsonNode converted = converter.fromCopycatData(12); + validateEnvelope(converted); + assertEquals(parse("{ \"type\": \"int\" }"), converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME)); + assertEquals(12, converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).intValue()); + } + + @Test + public void longToJson() { + JsonNode converted = converter.fromCopycatData(4398046511104L); + validateEnvelope(converted); + assertEquals(parse("{ \"type\": \"long\" }"), converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME)); + assertEquals(4398046511104L, converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).longValue()); + } + + @Test + public void floatToJson() { + JsonNode converted = converter.fromCopycatData(12.34f); + validateEnvelope(converted); + assertEquals(parse("{ \"type\": \"float\" }"), converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME)); + assertEquals(12.34f, converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).floatValue(), 0.001); + } + + @Test + public void doubleToJson() { + JsonNode converted = converter.fromCopycatData(12.34); + validateEnvelope(converted); + assertEquals(parse("{ \"type\": \"double\" }"), converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME)); + assertEquals(12.34, converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).doubleValue(), 0.001); + } + + @Test + public void bytesToJson() throws IOException { + JsonNode converted = converter.fromCopycatData("test-string".getBytes()); + validateEnvelope(converted); + assertEquals(parse("{ \"type\": \"bytes\" }"), converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME)); + assertEquals(ByteBuffer.wrap("test-string".getBytes()), + ByteBuffer.wrap(converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).binaryValue())); + } + + @Test + public void stringToJson() { + JsonNode converted = converter.fromCopycatData("test-string"); + validateEnvelope(converted); + assertEquals(parse("{ \"type\": \"string\" }"), converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME)); + assertEquals("test-string", converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).textValue()); + } + + @Test + public void arrayToJson() { + JsonNode converted = converter.fromCopycatData(Arrays.asList(1, 2, 3)); + validateEnvelope(converted); + assertEquals(parse("{ \"type\": \"array\", \"items\": { \"type\": \"int\" } }"), + converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME)); + assertEquals(JsonNodeFactory.instance.arrayNode().add(1).add(2).add(3), + converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME)); + } + + + private JsonNode parse(String json) { + try { + return objectMapper.readTree(json); + } catch (IOException e) { + fail("IOException during JSON parse: " + e.getMessage()); + throw new RuntimeException("failed"); + } + } + + private void validateEnvelope(JsonNode env) { + assertNotNull(env); + assertTrue(env.isObject()); + assertEquals(2, env.size()); + assertTrue(env.has(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME)); + assertTrue(env.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME).isObject()); + assertTrue(env.has(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME)); + } +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/cli/CopycatStandalone.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/cli/CopycatStandalone.java new file mode 100644 index 0000000..130a529 --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/cli/CopycatStandalone.java @@ -0,0 +1,87 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.cli; + +import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.copycat.runtime.Herder; +import org.apache.kafka.copycat.runtime.Worker; +import org.apache.kafka.copycat.runtime.standalone.StandaloneHerder; +import org.apache.kafka.copycat.util.Callback; +import org.apache.kafka.copycat.util.FutureCallback; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Arrays; +import java.util.Properties; + +/** + *

    + * Command line utility that runs Copycat as a standalone process. In this mode, work is not + * distributed. Instead, all the normal Copycat machinery works within a single process. This is + * useful for ad hoc, small, or experimental jobs. + *

    + *

    + * By default, no job configs or offset data is persistent. You can make jobs persistent and + * fault tolerant by overriding the settings to use file storage for both. + *

    + */ +@InterfaceStability.Unstable +public class CopycatStandalone { + private static final Logger log = LoggerFactory.getLogger(CopycatStandalone.class); + + public static void main(String[] args) throws Exception { + Properties workerProps; + Properties connectorProps; + + if (args.length < 2) { + log.info("Usage: CopycatStandalone worker.properties connector1.properties [connector2.properties ...]"); + System.exit(1); + } + + String workerPropsFile = args[0]; + workerProps = !workerPropsFile.isEmpty() ? Utils.loadProps(workerPropsFile) : new Properties(); + + WorkerConfig workerConfig = new WorkerConfig(workerProps); + Worker worker = new Worker(workerConfig); + Herder herder = new StandaloneHerder(worker); + final org.apache.kafka.copycat.runtime.Copycat copycat = new org.apache.kafka.copycat.runtime.Copycat(worker, herder); + copycat.start(); + + try { + for (final String connectorPropsFile : Arrays.copyOfRange(args, 1, args.length)) { + connectorProps = Utils.loadProps(connectorPropsFile); + FutureCallback cb = new FutureCallback<>(new Callback() { + @Override + public void onCompletion(Throwable error, String id) { + if (error != null) + log.error("Failed to create job for {}", connectorPropsFile); + } + }); + herder.addConnector(connectorProps, cb); + cb.get(); + } + } catch (Throwable t) { + log.error("Stopping after connector error", t); + copycat.stop(); + } + + // Shutdown will be triggered by Ctrl-C or via HTTP shutdown request + copycat.awaitStop(); + } +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/cli/WorkerConfig.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/cli/WorkerConfig.java new file mode 100644 index 0000000..46229db --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/cli/WorkerConfig.java @@ -0,0 +1,141 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.cli; + +import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.common.config.AbstractConfig; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigDef.Importance; +import org.apache.kafka.common.config.ConfigDef.Type; + +import java.util.Properties; +import java.util.Set; + +/** + * Configuration for standalone workers. + */ +@InterfaceStability.Unstable +public class WorkerConfig extends AbstractConfig { + + public static final String CLUSTER_CONFIG = "cluster"; + private static final String + CLUSTER_CONFIG_DOC = + "ID for this cluster, which is used to provide a namespace so multiple Copycat clusters " + + "or instances may co-exist while sharing a single Kafka cluster."; + public static final String CLUSTER_DEFAULT = "copycat"; + + public static final String BOOTSTRAP_SERVERS_CONFIG = "bootstrap.servers"; + public static final String BOOTSTRAP_SERVERS_DOC + = "A list of host/port pairs to use for establishing the initial connection to the Kafka " + + "cluster. The client will make use of all servers irrespective of which servers are " + + "specified here for bootstrapping—this list only impacts the initial hosts used " + + "to discover the full set of servers. This list should be in the form " + + "host1:port1,host2:port2,.... Since these servers are just used for the " + + "initial connection to discover the full cluster membership (which may change " + + "dynamically), this list need not contain the full set of servers (you may want more " + + "than one, though, in case a server is down)."; + public static final String BOOTSTRAP_SERVERS_DEFAULT = "localhost:9092"; + + public static final String KEY_CONVERTER_CLASS_CONFIG = "key.converter"; + public static final String KEY_CONVERTER_CLASS_DOC = + "Converter class for key Copycat data that implements the Converter interface."; + + public static final String VALUE_CONVERTER_CLASS_CONFIG = "value.converter"; + public static final String VALUE_CONVERTER_CLASS_DOC = + "Converter class for value Copycat data that implements the Converter interface."; + + public static final String KEY_SERIALIZER_CLASS_CONFIG = "key.serializer"; + public static final String KEY_SERIALIZER_CLASS_DOC = + "Serializer class for key that implements the Serializer interface."; + + public static final String VALUE_SERIALIZER_CLASS_CONFIG = "value.serializer"; + public static final String VALUE_SERIALIZER_CLASS_DOC = + "Serializer class for value that implements the Serializer interface."; + + public static final String KEY_DESERIALIZER_CLASS_CONFIG = "key.deserializer"; + public static final String KEY_DESERIALIZER_CLASS_DOC = + "Serializer class for key that implements the Deserializer interface."; + + public static final String VALUE_DESERIALIZER_CLASS_CONFIG = "value.deserializer"; + public static final String VALUE_DESERIALIZER_CLASS_DOC = + "Deserializer class for value that implements the Deserializer interface."; + + public static final String TASK_SHUTDOWN_GRACEFUL_TIMEOUT_MS_CONFIG + = "task.shutdown.graceful.timeout.ms"; + private static final String TASK_SHUTDOWN_GRACEFUL_TIMEOUT_MS_DOC = + "Amount of time to wait for tasks to shutdown gracefully. This is the total amount of time," + + " not per task. All task have shutdown triggered, then they are waited on sequentially."; + private static final String TASK_SHUTDOWN_GRACEFUL_TIMEOUT_MS_DEFAULT = "5000"; + + public static final String OFFSET_COMMIT_INTERVAL_MS_CONFIG = "offset.flush.interval.ms"; + private static final String OFFSET_COMMIT_INTERVAL_MS_DOC + = "Interval at which to try committing offsets for tasks."; + public static final long OFFSET_COMMIT_INTERVAL_MS_DEFAULT = 60000L; + + public static final String OFFSET_COMMIT_TIMEOUT_MS_CONFIG = "offset.flush.timeout.ms"; + private static final String OFFSET_COMMIT_TIMEOUT_MS_DOC + = "Maximum number of milliseconds to wait for records to flush and partition offset data to be" + + " committed to offset storage before cancelling the process and restoring the offset " + + "data to be committed in a future attempt."; + public static final long OFFSET_COMMIT_TIMEOUT_MS_DEFAULT = 5000L; + + private static ConfigDef config; + + static { + config = new ConfigDef() + .define(CLUSTER_CONFIG, Type.STRING, CLUSTER_DEFAULT, Importance.HIGH, CLUSTER_CONFIG_DOC) + .define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, BOOTSTRAP_SERVERS_DEFAULT, + Importance.HIGH, BOOTSTRAP_SERVERS_DOC) + .define(KEY_CONVERTER_CLASS_CONFIG, Type.CLASS, + Importance.HIGH, KEY_CONVERTER_CLASS_DOC) + .define(VALUE_CONVERTER_CLASS_CONFIG, Type.CLASS, + Importance.HIGH, VALUE_CONVERTER_CLASS_DOC) + .define(KEY_SERIALIZER_CLASS_CONFIG, Type.CLASS, + Importance.HIGH, KEY_SERIALIZER_CLASS_DOC) + .define(VALUE_SERIALIZER_CLASS_CONFIG, Type.CLASS, + Importance.HIGH, VALUE_SERIALIZER_CLASS_DOC) + .define(KEY_DESERIALIZER_CLASS_CONFIG, Type.CLASS, + Importance.HIGH, KEY_DESERIALIZER_CLASS_DOC) + .define(VALUE_DESERIALIZER_CLASS_CONFIG, Type.CLASS, + Importance.HIGH, VALUE_DESERIALIZER_CLASS_DOC) + .define(TASK_SHUTDOWN_GRACEFUL_TIMEOUT_MS_CONFIG, Type.LONG, + TASK_SHUTDOWN_GRACEFUL_TIMEOUT_MS_DEFAULT, Importance.LOW, + TASK_SHUTDOWN_GRACEFUL_TIMEOUT_MS_DOC) + .define(OFFSET_COMMIT_INTERVAL_MS_CONFIG, Type.LONG, OFFSET_COMMIT_INTERVAL_MS_DEFAULT, + Importance.LOW, OFFSET_COMMIT_INTERVAL_MS_DOC) + .define(OFFSET_COMMIT_TIMEOUT_MS_CONFIG, Type.LONG, OFFSET_COMMIT_TIMEOUT_MS_DEFAULT, + Importance.LOW, OFFSET_COMMIT_TIMEOUT_MS_DOC); + } + + public WorkerConfig() { + this(new Properties()); + } + + public WorkerConfig(Properties props) { + super(config, props); + } + + public Properties getUnusedProperties() { + Set unusedKeys = this.unused(); + Properties unusedProps = new Properties(); + for (String key : unusedKeys) { + unusedProps.put(key, this.originals().get(key)); + } + return unusedProps; + } +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/ConnectorConfig.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/ConnectorConfig.java new file mode 100644 index 0000000..e3fcc1c --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/ConnectorConfig.java @@ -0,0 +1,87 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.runtime; + +import org.apache.kafka.common.config.AbstractConfig; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigDef.Importance; +import org.apache.kafka.common.config.ConfigDef.Type; + +import java.util.Properties; +import java.util.Set; + +/** + *

    + * Configuration options for Connectors. These only include Copycat system-level configuration + * options (e.g. Connector class name, timeouts used by Copycat to control the connector) but does + * not include Connector-specific options (e.g. database connection settings). + *

    + *

    + * Note that some of these options are not required for all connectors. For example TOPICS_CONFIG + * is sink-specific. + *

    + */ +public class ConnectorConfig extends AbstractConfig { + + public static final String NAME_CONFIG = "name"; + private static final String NAME_DOC = "Globally unique name to use for this connector."; + + public static final String CONNECTOR_CLASS_CONFIG = "connector.class"; + private static final String CONNECTOR_CLASS_DOC = + "Name of the class for this connector. Must be a subclass of org.apache.kafka.copycat.connector" + + ".Connector"; + + public static final String TASKS_MAX_CONFIG = "tasks.max"; + private static final String TASKS_MAX_DOC = "Maximum number of tasks to use for this connector."; + public static final int TASKS_MAX_DEFAULT = 1; + + public static final String TOPICS_CONFIG = "topics"; + private static final String TOPICS_DOC = ""; + public static final String TOPICS_DEFAULT = ""; + + private static ConfigDef config; + + static { + config = new ConfigDef() + .define(NAME_CONFIG, Type.STRING, Importance.HIGH, NAME_DOC) + .define(CONNECTOR_CLASS_CONFIG, Type.STRING, Importance.HIGH, CONNECTOR_CLASS_DOC) + .define(TASKS_MAX_CONFIG, Type.INT, TASKS_MAX_DEFAULT, Importance.HIGH, TASKS_MAX_DOC) + .define(TOPICS_CONFIG, Type.LIST, TOPICS_DEFAULT, Importance.HIGH, TOPICS_DOC); + } + + private Properties originalProperties; + + public ConnectorConfig() { + this(new Properties()); + } + + public ConnectorConfig(Properties props) { + super(config, props); + this.originalProperties = props; + } + + public Properties getUnusedProperties() { + Set unusedKeys = this.unused(); + Properties unusedProps = new Properties(); + for (String key : unusedKeys) { + unusedProps.setProperty(key, originalProperties.getProperty(key)); + } + return unusedProps; + } + +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/Copycat.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/Copycat.java new file mode 100644 index 0000000..e8dfe14 --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/Copycat.java @@ -0,0 +1,94 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.runtime; + +import org.apache.kafka.common.annotation.InterfaceStability; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * This class ties together all the components of a Copycat process (herder, worker, + * storage, command interface), managing their lifecycle. + */ +@InterfaceStability.Unstable +public class Copycat { + private static final Logger log = LoggerFactory.getLogger(Copycat.class); + + private final Worker worker; + private final Herder herder; + private final CountDownLatch startLatch = new CountDownLatch(1); + private final CountDownLatch stopLatch = new CountDownLatch(1); + private final AtomicBoolean shutdown = new AtomicBoolean(false); + private final ShutdownHook shutdownHook; + + public Copycat(Worker worker, Herder herder) { + log.debug("Copycat created"); + this.worker = worker; + this.herder = herder; + shutdownHook = new ShutdownHook(); + } + + public void start() { + log.info("Copycat starting"); + Runtime.getRuntime().addShutdownHook(shutdownHook); + + worker.start(); + herder.start(); + + log.info("Copycat started"); + + startLatch.countDown(); + } + + public void stop() { + boolean wasShuttingDown = shutdown.getAndSet(true); + if (!wasShuttingDown) { + log.info("Copycat stopping"); + + herder.stop(); + worker.stop(); + + log.info("Copycat stopped"); + } + + stopLatch.countDown(); + } + + public void awaitStop() { + try { + stopLatch.await(); + } catch (InterruptedException e) { + log.error("Interrupted waiting for Copycat to shutdown"); + } + } + + private class ShutdownHook extends Thread { + @Override + public void run() { + try { + startLatch.await(); + Copycat.this.stop(); + } catch (InterruptedException e) { + log.error("Interrupted in shutdown hook while waiting for copycat startup to finish"); + } + } + } +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/Herder.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/Herder.java new file mode 100644 index 0000000..7f8b7c2 --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/Herder.java @@ -0,0 +1,67 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.runtime; + +import org.apache.kafka.copycat.util.Callback; + +import java.util.Properties; + +/** + *

    + * The herder interface tracks and manages workers and connectors. It is the main interface for external components + * to make changes to the state of the cluster. For example, in distributed mode, an implementation of this class + * knows how to accept a connector configuration, may need to route it to the current leader worker for the cluster so + * the config can be written to persistent storage, and then ensures the new connector is correctly instantiated on one + * of the workers. + *

    + *

    + * This class must implement all the actions that can be taken on the cluster (add/remove connectors, pause/resume tasks, + * get state of connectors and tasks, etc). The non-Java interfaces to the cluster (REST API and CLI) are very simple + * wrappers of the functionality provided by this interface. + *

    + *

    + * In standalone mode, this implementation of this class will be trivial because no coordination is needed. In that case, + * the implementation will mainly be delegating tasks directly to other components. For example, when creating a new + * connector in standalone mode, there is no need to persist the config and the connector and its tasks must run in the + * same process, so the standalone herder implementation can immediately instantiate and start the connector and its + * tasks. + *

    + */ +public interface Herder { + + void start(); + + void stop(); + + /** + * Submit a connector job to the cluster. This works from any node by forwarding the request to + * the leader herder if necessary. + * + * @param connectorProps user-specified properties for this job + * @param callback callback to invoke when the request completes + */ + void addConnector(Properties connectorProps, Callback callback); + + /** + * Delete a connector job by name. + * + * @param name name of the connector job to shutdown and delete + * @param callback callback to invoke when the request completes + */ + void deleteConnector(String name, Callback callback); +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/SinkTaskContextImpl.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/SinkTaskContextImpl.java new file mode 100644 index 0000000..f47c984 --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/SinkTaskContextImpl.java @@ -0,0 +1,24 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.runtime; + +import org.apache.kafka.copycat.sink.SinkTaskContext; + +class SinkTaskContextImpl extends SinkTaskContext { + +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/SourceTaskOffsetCommitter.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/SourceTaskOffsetCommitter.java new file mode 100644 index 0000000..953cfa5 --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/SourceTaskOffsetCommitter.java @@ -0,0 +1,103 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.runtime; + +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.copycat.cli.WorkerConfig; +import org.apache.kafka.copycat.util.ConnectorTaskId; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; + +/** + *

    + * Manages offset commit scheduling and execution for SourceTasks. + *

    + *

    + * Unlike sink tasks which directly manage their offset commits in the main poll() thread since + * they drive the event loop and control (for all intents and purposes) the timeouts, source + * tasks are at the whim of the connector and cannot be guaranteed to wake up on the necessary + * schedule. Instead, this class tracks all the active tasks, their schedule for commits, and + * ensures they are invoked in a timely fashion. + *

    + *

    + * The current implementation uses a single thread to process commits and + *

    + */ +class SourceTaskOffsetCommitter { + private static final Logger log = LoggerFactory.getLogger(SourceTaskOffsetCommitter.class); + + private Time time; + private WorkerConfig config; + private ScheduledExecutorService commitExecutorService = null; + private HashMap> committers = new HashMap<>(); + + SourceTaskOffsetCommitter(Time time, WorkerConfig config) { + this.time = time; + this.config = config; + commitExecutorService = Executors.newSingleThreadScheduledExecutor(); + } + + public void close(long timeoutMs) { + commitExecutorService.shutdown(); + try { + if (!commitExecutorService.awaitTermination(timeoutMs, TimeUnit.MILLISECONDS)) { + log.error("Graceful shutdown of offset commitOffsets thread timed out."); + } + } catch (InterruptedException e) { + // ignore and allow to exit immediately + } + } + + public void schedule(final ConnectorTaskId id, final WorkerSourceTask workerTask) { + long commitIntervalMs = config.getLong(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG); + ScheduledFuture commitFuture = commitExecutorService.scheduleWithFixedDelay(new Runnable() { + @Override + public void run() { + commit(workerTask); + } + }, commitIntervalMs, commitIntervalMs, TimeUnit.MILLISECONDS); + committers.put(id, commitFuture); + } + + public void remove(ConnectorTaskId id) { + ScheduledFuture commitFuture = committers.remove(id); + commitFuture.cancel(false); + } + + public void commit(WorkerSourceTask workerTask) { + try { + log.debug("Committing offsets for {}", workerTask); + boolean success = workerTask.commitOffsets(); + if (!success) { + log.error("Failed to commit offsets for {}", workerTask); + } + } catch (Throwable t) { + // We're very careful about exceptions here since any uncaught exceptions in the commit + // thread would cause the fixed interval schedule on the ExecutorService to stop running + // for that task + log.error("Unhandled exception when committing {}: ", workerTask, t); + } + } + +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/Worker.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/Worker.java new file mode 100644 index 0000000..55d0784 --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/Worker.java @@ -0,0 +1,236 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.runtime; + +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.utils.SystemTime; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.copycat.cli.WorkerConfig; +import org.apache.kafka.copycat.connector.Task; +import org.apache.kafka.copycat.errors.CopycatException; +import org.apache.kafka.copycat.sink.SinkTask; +import org.apache.kafka.copycat.source.SourceTask; +import org.apache.kafka.copycat.storage.*; +import org.apache.kafka.copycat.util.ConnectorTaskId; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +/** + *

    + * Worker runs a (dynamic) set of tasks in a set of threads, doing the work of actually moving + * data to/from Kafka. + *

    + *

    + * Since each task has a dedicated thread, this is mainly just a container for them. + *

    + */ +public class Worker { + private static final Logger log = LoggerFactory.getLogger(Worker.class); + + private Time time; + private WorkerConfig config; + private Converter keyConverter; + private Converter valueConverter; + private OffsetBackingStore offsetBackingStore; + private Serializer offsetKeySerializer; + private Serializer offsetValueSerializer; + private Deserializer offsetKeyDeserializer; + private Deserializer offsetValueDeserializer; + private HashMap tasks = new HashMap<>(); + private KafkaProducer producer; + private SourceTaskOffsetCommitter sourceTaskOffsetCommitter; + + public Worker(WorkerConfig config) { + this(new SystemTime(), config, null, null, null, null, null); + } + + @SuppressWarnings("unchecked") + public Worker(Time time, WorkerConfig config, OffsetBackingStore offsetBackingStore, + Serializer offsetKeySerializer, Serializer offsetValueSerializer, + Deserializer offsetKeyDeserializer, Deserializer offsetValueDeserializer) { + this.time = time; + this.config = config; + this.keyConverter = config.getConfiguredInstance(WorkerConfig.KEY_CONVERTER_CLASS_CONFIG, Converter.class); + this.valueConverter = config.getConfiguredInstance(WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, Converter.class); + + if (offsetBackingStore != null) { + this.offsetBackingStore = offsetBackingStore; + } else { + this.offsetBackingStore = new FileOffsetBackingStore(); + this.offsetBackingStore.configure(config.originals()); + } + + if (offsetKeySerializer != null) { + this.offsetKeySerializer = offsetKeySerializer; + } else { + this.offsetKeySerializer = config.getConfiguredInstance(WorkerConfig.KEY_SERIALIZER_CLASS_CONFIG, Serializer.class); + this.offsetKeySerializer.configure(config.originals(), true); + } + + if (offsetValueSerializer != null) { + this.offsetValueSerializer = offsetValueSerializer; + } else { + this.offsetValueSerializer = config.getConfiguredInstance(WorkerConfig.VALUE_SERIALIZER_CLASS_CONFIG, Serializer.class); + this.offsetValueSerializer.configure(config.originals(), false); + } + + if (offsetKeyDeserializer != null) { + this.offsetKeyDeserializer = offsetKeyDeserializer; + } else { + this.offsetKeyDeserializer = config.getConfiguredInstance(WorkerConfig.KEY_DESERIALIZER_CLASS_CONFIG, Deserializer.class); + this.offsetKeyDeserializer.configure(config.originals(), true); + } + + if (offsetValueDeserializer != null) { + this.offsetValueDeserializer = offsetValueDeserializer; + } else { + this.offsetValueDeserializer = config.getConfiguredInstance(WorkerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, Deserializer.class); + this.offsetValueDeserializer.configure(config.originals(), false); + } + } + + public void start() { + log.info("Worker starting"); + + Properties unusedConfigs = config.getUnusedProperties(); + + Map producerProps = new HashMap<>(); + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, Utils.join(config.getList(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG), ",")); + producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, config.getClass(WorkerConfig.KEY_SERIALIZER_CLASS_CONFIG).getName()); + producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, config.getClass(WorkerConfig.VALUE_SERIALIZER_CLASS_CONFIG).getName()); + for (String propName : unusedConfigs.stringPropertyNames()) { + producerProps.put(propName, unusedConfigs.getProperty(propName)); + } + producer = new KafkaProducer<>(producerProps); + + offsetBackingStore.start(); + sourceTaskOffsetCommitter = new SourceTaskOffsetCommitter(time, config); + + log.info("Worker started"); + } + + public void stop() { + log.info("Worker stopping"); + + long started = time.milliseconds(); + long limit = started + config.getLong(WorkerConfig.TASK_SHUTDOWN_GRACEFUL_TIMEOUT_MS_CONFIG); + + for (Map.Entry entry : tasks.entrySet()) { + WorkerTask task = entry.getValue(); + log.warn("Shutting down task {} uncleanly; herder should have shut down " + + "tasks before the Worker is stopped.", task); + try { + task.stop(); + } catch (CopycatException e) { + log.error("Error while shutting down task " + task, e); + } + } + + for (Map.Entry entry : tasks.entrySet()) { + WorkerTask task = entry.getValue(); + log.debug("Waiting for task {} to finish shutting down", task); + if (!task.awaitStop(Math.max(limit - time.milliseconds(), 0))) + log.error("Graceful shutdown of task {} failed.", task); + task.close(); + } + + long timeoutMs = limit - time.milliseconds(); + sourceTaskOffsetCommitter.close(timeoutMs); + + offsetBackingStore.start(); + + log.info("Worker stopped"); + } + + /** + * Add a new task. + * @param id Globally unique ID for this task. + * @param taskClassName name of the {@link org.apache.kafka.copycat.connector.Task} + * class to instantiate. Must be a subclass of either + * {@link org.apache.kafka.copycat.source.SourceTask} or + * {@link org.apache.kafka.copycat.sink.SinkTask}. + * @param props configuration options for the task + */ + public void addTask(ConnectorTaskId id, String taskClassName, Properties props) { + if (tasks.containsKey(id)) { + String msg = "Task already exists in this worker; the herder should not have requested " + + "that this : " + id; + log.error(msg); + throw new CopycatException(msg); + } + + final Task task = instantiateTask(taskClassName); + + // Decide which type of worker task we need based on the type of task. + final WorkerTask workerTask; + if (task instanceof SourceTask) { + SourceTask sourceTask = (SourceTask) task; + OffsetStorageReader offsetReader = new OffsetStorageReaderImpl<>(offsetBackingStore, id.getConnector(), + keyConverter, valueConverter, offsetKeySerializer, offsetValueDeserializer); + OffsetStorageWriter offsetWriter = new OffsetStorageWriter<>(offsetBackingStore, id.getConnector(), + keyConverter, valueConverter, offsetKeySerializer, offsetValueSerializer); + workerTask = new WorkerSourceTask<>(id, sourceTask, keyConverter, valueConverter, producer, + offsetReader, offsetWriter, config, time); + } else if (task instanceof SinkTask) { + workerTask = new WorkerSinkTask<>(id, (SinkTask) task, config, keyConverter, valueConverter, time); + } else { + log.error("Tasks must be a subclass of either SourceTask or SinkTask", task); + throw new CopycatException("Tasks must be a subclass of either SourceTask or SinkTask"); + } + + // Start the task before adding modifying any state, any exceptions are caught higher up the + // call chain and there's no cleanup to do here + workerTask.start(props); + tasks.put(id, workerTask); + } + + private static Task instantiateTask(String taskClassName) { + try { + return Utils.newInstance(Class.forName(taskClassName).asSubclass(Task.class)); + } catch (ClassNotFoundException e) { + throw new CopycatException("Task class not found", e); + } + } + + public void stopTask(ConnectorTaskId id) { + WorkerTask task = getTask(id); + task.stop(); + if (!task.awaitStop(config.getLong(WorkerConfig.TASK_SHUTDOWN_GRACEFUL_TIMEOUT_MS_CONFIG))) + log.error("Graceful stop of task {} failed.", task); + task.close(); + tasks.remove(id); + } + + private WorkerTask getTask(ConnectorTaskId id) { + WorkerTask task = tasks.get(id); + if (task == null) { + log.error("Task not found: " + id); + throw new CopycatException("Task not found: " + id); + } + return task; + } + +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/WorkerSinkTask.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/WorkerSinkTask.java new file mode 100644 index 0000000..4eaf756 --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/WorkerSinkTask.java @@ -0,0 +1,226 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.runtime; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.clients.consumer.*; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.copycat.cli.WorkerConfig; +import org.apache.kafka.copycat.errors.CopycatException; +import org.apache.kafka.copycat.sink.SinkRecord; +import org.apache.kafka.copycat.sink.SinkTask; +import org.apache.kafka.copycat.sink.SinkTaskContext; +import org.apache.kafka.copycat.storage.Converter; +import org.apache.kafka.copycat.util.ConnectorTaskId; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.*; +import java.util.concurrent.TimeUnit; + +/** + * WorkerTask that uses a SinkTask to export data from Kafka. + */ +class WorkerSinkTask implements WorkerTask { + private static final Logger log = LoggerFactory.getLogger(WorkerSinkTask.class); + + private final ConnectorTaskId id; + private final SinkTask task; + private final WorkerConfig workerConfig; + private final Time time; + private final Converter keyConverter; + private final Converter valueConverter; + private WorkerSinkTaskThread workThread; + private KafkaConsumer consumer; + private final SinkTaskContext context; + + public WorkerSinkTask(ConnectorTaskId id, SinkTask task, WorkerConfig workerConfig, + Converter keyConverter, Converter valueConverter, Time time) { + this.id = id; + this.task = task; + this.workerConfig = workerConfig; + this.keyConverter = keyConverter; + this.valueConverter = valueConverter; + context = new SinkTaskContextImpl(); + this.time = time; + } + + @Override + public void start(Properties props) { + task.initialize(context); + task.start(props); + consumer = createConsumer(props); + workThread = createWorkerThread(); + workThread.start(); + } + + @Override + public void stop() { + // Offset commit is handled upon exit in work thread + task.stop(); + if (workThread != null) + workThread.startGracefulShutdown(); + consumer.wakeup(); + } + + @Override + public boolean awaitStop(long timeoutMs) { + if (workThread != null) { + try { + boolean success = workThread.awaitShutdown(timeoutMs, TimeUnit.MILLISECONDS); + if (!success) + workThread.forceShutdown(); + return success; + } catch (InterruptedException e) { + return false; + } + } + return true; + } + + @Override + public void close() { + // FIXME Kafka needs to add a timeout parameter here for us to properly obey the timeout + // passed in + if (consumer != null) + consumer.close(); + } + + /** Poll for new messages with the given timeout. Should only be invoked by the worker thread. */ + public void poll(long timeoutMs) { + try { + log.trace("{} polling consumer with timeout {} ms", id, timeoutMs); + ConsumerRecords msgs = consumer.poll(timeoutMs); + log.trace("{} polling returned {} messages", id, msgs.count()); + deliverMessages(msgs); + } catch (ConsumerWakeupException we) { + log.trace("{} consumer woken up", id); + } + } + + /** + * Starts an offset commit by flushing outstanding messages from the task and then starting + * the write commit. This should only be invoked by the WorkerSinkTaskThread. + **/ + public void commitOffsets(long now, boolean sync, final int seqno, boolean flush) { + HashMap offsets = new HashMap<>(); + for (TopicPartition tp : consumer.subscriptions()) { + offsets.put(tp, consumer.position(tp)); + } + // We only don't flush the task in one case: when shutting down, the task has already been + // stopped and all data should have already been flushed + if (flush) { + try { + task.flush(offsets); + } catch (Throwable t) { + log.error("Commit of {} offsets failed due to exception while flushing: {}", this, t); + workThread.onCommitCompleted(t, seqno); + return; + } + } + + ConsumerCommitCallback cb = new ConsumerCommitCallback() { + @Override + public void onComplete(Map offsets, Exception error) { + workThread.onCommitCompleted(error, seqno); + } + }; + consumer.commit(offsets, sync ? CommitType.SYNC : CommitType.ASYNC, cb); + } + + public Time getTime() { + return time; + } + + public WorkerConfig getWorkerConfig() { + return workerConfig; + } + + private KafkaConsumer createConsumer(Properties taskProps) { + String topicsStr = taskProps.getProperty(SinkTask.TOPICS_CONFIG); + if (topicsStr == null || topicsStr.isEmpty()) + throw new CopycatException("Sink tasks require a list of topics."); + String[] topics = topicsStr.split(","); + + // Include any unknown worker configs so consumer configs can be set globally on the worker + // and through to the task + Properties props = workerConfig.getUnusedProperties(); + props.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "copycat-" + id.toString()); + props.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, + Utils.join(workerConfig.getList(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG), ",")); + props.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); + props.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + props.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, + workerConfig.getClass(WorkerConfig.KEY_DESERIALIZER_CLASS_CONFIG).getName()); + props.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, + workerConfig.getClass(WorkerConfig.VALUE_DESERIALIZER_CLASS_CONFIG).getName()); + + KafkaConsumer newConsumer; + try { + newConsumer = new KafkaConsumer<>(props); + } catch (Throwable t) { + throw new CopycatException("Failed to create consumer", t); + } + + log.debug("Task {} subscribing to topics {}", id, topics); + newConsumer.subscribe(topics); + + // Seek to any user-provided offsets. This is useful if offsets are tracked in the downstream system (e.g., to + // enable exactly once delivery to that system). + // + // To do this correctly, we need to first make sure we have been assigned partitions, which poll() will guarantee. + // We ask for offsets after this poll to make sure any offsets committed before the rebalance are picked up correctly. + newConsumer.poll(0); + Map offsets = context.getOffsets(); + for (TopicPartition tp : newConsumer.subscriptions()) { + Long offset = offsets.get(tp); + if (offset != null) + newConsumer.seek(tp, offset); + } + return newConsumer; + } + + private WorkerSinkTaskThread createWorkerThread() { + return new WorkerSinkTaskThread(this, "WorkerSinkTask-" + id, time, workerConfig); + } + + private void deliverMessages(ConsumerRecords msgs) { + // Finally, deliver this batch to the sink + if (msgs.count() > 0) { + List records = new ArrayList<>(); + for (ConsumerRecord msg : msgs) { + log.trace("Consuming message with key {}, value {}", msg.key(), msg.value()); + records.add( + new SinkRecord(msg.topic(), msg.partition(), + keyConverter.toCopycatData(msg.key()), + valueConverter.toCopycatData(msg.value()), + msg.offset()) + ); + } + + try { + task.put(records); + } catch (CopycatException e) { + log.error("Exception from SinkTask {}: ", id, e); + } catch (Throwable t) { + log.error("Unexpected exception from SinkTask {}: ", id, t); + } + } + } +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/WorkerSinkTaskThread.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/WorkerSinkTaskThread.java new file mode 100644 index 0000000..b946343 --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/WorkerSinkTaskThread.java @@ -0,0 +1,112 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.runtime; + +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.copycat.cli.WorkerConfig; +import org.apache.kafka.copycat.util.ShutdownableThread; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Worker thread for a WorkerSinkTask. These classes are very tightly coupled, but separated to + * simplify testing. + */ +class WorkerSinkTaskThread extends ShutdownableThread { + private static final Logger log = LoggerFactory.getLogger(WorkerSinkTask.class); + + private final WorkerSinkTask task; + private long nextCommit; + private boolean committing; + private int commitSeqno; + private long commitStarted; + private int commitFailures; + + public WorkerSinkTaskThread(WorkerSinkTask task, String name, Time time, + WorkerConfig workerConfig) { + super(name); + this.task = task; + this.nextCommit = time.milliseconds() + + workerConfig.getLong(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG); + this.committing = false; + this.commitSeqno = 0; + this.commitStarted = -1; + this.commitFailures = 0; + } + + @Override + public void execute() { + while (getRunning()) { + iteration(); + } + // Make sure any uncommitted data has committed + task.commitOffsets(task.getTime().milliseconds(), true, -1, false); + } + + public void iteration() { + long now = task.getTime().milliseconds(); + + // Maybe commit + if (!committing && now >= nextCommit) { + synchronized (this) { + committing = true; + commitSeqno += 1; + commitStarted = now; + } + task.commitOffsets(now, false, commitSeqno, true); + nextCommit += task.getWorkerConfig().getLong(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG); + } + + // Check for timed out commits + long commitTimeout = commitStarted + task.getWorkerConfig().getLong( + WorkerConfig.OFFSET_COMMIT_TIMEOUT_MS_CONFIG); + if (committing && now >= commitTimeout) { + log.warn("Commit of {} offsets timed out", this); + commitFailures++; + committing = false; + } + + // And process messages + long timeoutMs = Math.max(nextCommit - now, 0); + task.poll(timeoutMs); + } + + public void onCommitCompleted(Throwable error, long seqno) { + synchronized (this) { + if (commitSeqno != seqno) { + log.debug("Got callback for timed out commit {}: {}, but most recent commit is {}", + this, + seqno, commitSeqno); + } else { + if (error != null) { + log.error("Commit of {} offsets threw an unexpected exception: ", this, error); + commitFailures++; + } else { + log.debug("Finished {} offset commit successfully in {} ms", + this, task.getTime().milliseconds() - commitStarted); + commitFailures = 0; + } + committing = false; + } + } + } + + public int getCommitFailures() { + return commitFailures; + } +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/WorkerSourceTask.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/WorkerSourceTask.java new file mode 100644 index 0000000..c80adb4 --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/WorkerSourceTask.java @@ -0,0 +1,310 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.runtime; + +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.copycat.cli.WorkerConfig; +import org.apache.kafka.copycat.source.SourceRecord; +import org.apache.kafka.copycat.source.SourceTask; +import org.apache.kafka.copycat.source.SourceTaskContext; +import org.apache.kafka.copycat.storage.Converter; +import org.apache.kafka.copycat.storage.OffsetStorageReader; +import org.apache.kafka.copycat.storage.OffsetStorageWriter; +import org.apache.kafka.copycat.util.ConnectorTaskId; +import org.apache.kafka.copycat.util.ShutdownableThread; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.IdentityHashMap; +import java.util.List; +import java.util.Properties; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +/** + * WorkerTask that uses a SourceTask to ingest data into Kafka. + */ +class WorkerSourceTask implements WorkerTask { + private static final Logger log = LoggerFactory.getLogger(WorkerSourceTask.class); + + private ConnectorTaskId id; + private SourceTask task; + private final Converter keyConverter; + private final Converter valueConverter; + private KafkaProducer producer; + private WorkerSourceTaskThread workThread; + private OffsetStorageReader offsetReader; + private OffsetStorageWriter offsetWriter; + private final WorkerConfig workerConfig; + private final Time time; + + // Use IdentityHashMap to ensure correctness with duplicate records. This is a HashMap because + // there is no IdentityHashSet. + private IdentityHashMap, ProducerRecord> + outstandingMessages; + // A second buffer is used while an offset flush is running + private IdentityHashMap, ProducerRecord> + outstandingMessagesBacklog; + private boolean flushing; + + public WorkerSourceTask(ConnectorTaskId id, SourceTask task, + Converter keyConverter, Converter valueConverter, + KafkaProducer producer, + OffsetStorageReader offsetReader, OffsetStorageWriter offsetWriter, + WorkerConfig workerConfig, Time time) { + this.id = id; + this.task = task; + this.keyConverter = keyConverter; + this.valueConverter = valueConverter; + this.producer = producer; + this.offsetReader = offsetReader; + this.offsetWriter = offsetWriter; + this.workerConfig = workerConfig; + this.time = time; + + this.outstandingMessages = new IdentityHashMap<>(); + this.outstandingMessagesBacklog = new IdentityHashMap<>(); + this.flushing = false; + } + + @Override + public void start(Properties props) { + task.initialize(new SourceTaskContext(offsetReader)); + task.start(props); + workThread = new WorkerSourceTaskThread("WorkerSourceTask-" + id); + workThread.start(); + } + + @Override + public void stop() { + task.stop(); + commitOffsets(); + if (workThread != null) + workThread.startGracefulShutdown(); + } + + @Override + public boolean awaitStop(long timeoutMs) { + if (workThread != null) { + try { + boolean success = workThread.awaitShutdown(timeoutMs, TimeUnit.MILLISECONDS); + if (!success) + workThread.forceShutdown(); + return success; + } catch (InterruptedException e) { + return false; + } + } + return true; + } + + @Override + public void close() { + // Nothing to do + } + + /** + * Send a batch of records. This is atomic up to the point of getting the messages into the + * Producer and recorded in our set of outstanding messages, so either all or none will be sent + * @param records + */ + private synchronized void sendRecords(List records) { + for (SourceRecord record : records) { + final ProducerRecord producerRecord + = new ProducerRecord<>(record.getTopic(), record.getKafkaPartition(), + keyConverter.fromCopycatData(record.getKey()), + valueConverter.fromCopycatData(record.getValue())); + log.trace("Appending record with key {}, value {}", record.getKey(), record.getValue()); + if (!flushing) { + outstandingMessages.put(producerRecord, producerRecord); + } else { + outstandingMessagesBacklog.put(producerRecord, producerRecord); + } + producer.send( + producerRecord, + new Callback() { + @Override + public void onCompletion(RecordMetadata recordMetadata, Exception e) { + if (e != null) { + log.error("Failed to send record: ", e); + } else { + log.trace("Wrote record successfully: topic {} partition {} offset {}", + recordMetadata.topic(), recordMetadata.partition(), + recordMetadata.offset()); + } + recordSent(producerRecord); + } + }); + // Offsets are converted & serialized in the OffsetWriter + offsetWriter.setOffset(record.getSourcePartition(), record.getSourceOffset()); + } + } + + private synchronized void recordSent(final ProducerRecord record) { + ProducerRecord removed = outstandingMessages.remove(record); + // While flushing, we may also see callbacks for items in the backlog + if (removed == null && flushing) + removed = outstandingMessagesBacklog.remove(record); + // But if neither one had it, something is very wrong + if (removed == null) { + log.error("Saw callback for record that was not present in the outstanding message set: " + + "{}", record); + } else if (flushing && outstandingMessages.isEmpty()) { + // flush thread may be waiting on the outstanding messages to clear + this.notifyAll(); + } + } + + public boolean commitOffsets() { + long commitTimeoutMs = workerConfig.getLong(WorkerConfig.OFFSET_COMMIT_TIMEOUT_MS_CONFIG); + + long started = time.milliseconds(); + long timeout = started + commitTimeoutMs; + + synchronized (this) { + // First we need to make sure we snapshot everything in exactly the current state. This + // means both the current set of messages we're still waiting to finish, stored in this + // class, which setting flushing = true will handle by storing any new values into a new + // buffer; and the current set of user-specified offsets, stored in the + // OffsetStorageWriter, for which we can use beginFlush() to initiate the snapshot. + flushing = true; + boolean flushStarted = offsetWriter.beginFlush(); + // Still wait for any producer records to flush, even if there aren't any offsets to write + // to persistent storage + + // Next we need to wait for all outstanding messages to finish sending + while (!outstandingMessages.isEmpty()) { + try { + long timeoutMs = timeout - time.milliseconds(); + if (timeoutMs <= 0) { + log.error( + "Failed to flush {}, timed out while waiting for producer to flush outstanding " + + "messages", this.toString()); + finishFailedFlush(); + return false; + } + this.wait(timeoutMs); + } catch (InterruptedException e) { + // ignore + } + } + + if (!flushStarted) { + // There was nothing in the offsets to process, but we still waited for the data in the + // buffer to flush. This is useful since this can feed into metrics to monitor, e.g. + // flush time, which can be used for monitoring even if the connector doesn't record any + // offsets. + finishSuccessfulFlush(); + log.debug("Finished {} offset commitOffsets successfully in {} ms", + this, time.milliseconds() - started); + return true; + } + } + + // Now we can actually flush the offsets to user storage. + Future flushFuture = offsetWriter.doFlush(new org.apache.kafka.copycat.util.Callback() { + @Override + public void onCompletion(Throwable error, Void result) { + if (error != null) { + log.error("Failed to flush {} offsets to storage: ", this, error); + } else { + log.trace("Finished flushing {} offsets to storage", this); + } + } + }); + // Very rare case: offsets were unserializable and we finished immediately, unable to store + // any data + if (flushFuture == null) { + finishFailedFlush(); + return false; + } + try { + flushFuture.get(Math.max(timeout - time.milliseconds(), 0), TimeUnit.MILLISECONDS); + // There's a small race here where we can get the callback just as this times out (and log + // success), but then catch the exception below and cancel everything. This won't cause any + // errors, is only wasteful in this minor edge case, and the worst result is that the log + // could look a little confusing. + } catch (InterruptedException e) { + log.warn("Flush of {} offsets interrupted, cancelling", this); + finishFailedFlush(); + return false; + } catch (ExecutionException e) { + log.error("Flush of {} offsets threw an unexpected exception: ", this, e); + finishFailedFlush(); + return false; + } catch (TimeoutException e) { + log.error("Timed out waiting to flush {} offsets to storage", this); + finishFailedFlush(); + return false; + } + + finishSuccessfulFlush(); + log.debug("Finished {} commitOffsets successfully in {} ms", + this, time.milliseconds() - started); + return true; + } + + private synchronized void finishFailedFlush() { + offsetWriter.cancelFlush(); + outstandingMessages.putAll(outstandingMessagesBacklog); + outstandingMessagesBacklog.clear(); + flushing = false; + } + + private void finishSuccessfulFlush() { + // If we were successful, we can just swap instead of replacing items back into the original map + IdentityHashMap, ProducerRecord> temp = outstandingMessages; + outstandingMessages = outstandingMessagesBacklog; + outstandingMessagesBacklog = temp; + flushing = false; + } + + + private class WorkerSourceTaskThread extends ShutdownableThread { + public WorkerSourceTaskThread(String name) { + super(name); + } + + @Override + public void execute() { + try { + while (getRunning()) { + List records = task.poll(); + if (records == null) + continue; + sendRecords(records); + } + } catch (InterruptedException e) { + // Ignore and allow to exit. + } + } + } + + @Override + public String toString() { + return "WorkerSourceTask{" + + "id=" + id + + '}'; + } +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/WorkerTask.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/WorkerTask.java new file mode 100644 index 0000000..af225bb --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/WorkerTask.java @@ -0,0 +1,54 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.runtime; + +import java.util.Properties; + +/** + * Handles processing for an individual task. This interface only provides the basic methods + * used by {@link Worker} to manage the tasks. Implementations combine a user-specified Task with + * Kafka to create a data flow. + */ +interface WorkerTask { + /** + * Start the Task + * @param props initial configuration + */ + void start(Properties props); + + /** + * Stop this task from processing messages. This method does not block, it only triggers + * shutdown. Use #{@link #awaitStop} to block until completion. + */ + void stop(); + + /** + * Wait for this task to finish stopping. + * + * @param timeoutMs + * @return true if successful, false if the timeout was reached + */ + boolean awaitStop(long timeoutMs); + + /** + * Close this task. This is different from #{@link #stop} and #{@link #awaitStop} in that the + * stop methods ensure processing has stopped but may leave resources allocated. This method + * should clean up all resources. + */ + void close(); +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/standalone/StandaloneConnectorContext.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/standalone/StandaloneConnectorContext.java new file mode 100644 index 0000000..0e14015 --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/standalone/StandaloneConnectorContext.java @@ -0,0 +1,42 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.runtime.standalone; + +import org.apache.kafka.copycat.connector.ConnectorContext; + +/** + * ConnectorContext for use with the StandaloneHerder, which maintains all connectors and tasks + * in a single process. + */ +class StandaloneConnectorContext implements ConnectorContext { + + private StandaloneHerder herder; + private String connectorName; + + public StandaloneConnectorContext(StandaloneHerder herder, String connectorName) { + this.herder = herder; + this.connectorName = connectorName; + } + + @Override + public void requestTaskReconfiguration() { + // This is trivial to forward since there is only one herder and it's in memory in this + // process + herder.requestTaskReconfiguration(connectorName); + } +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/standalone/StandaloneHerder.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/standalone/StandaloneHerder.java new file mode 100644 index 0000000..2ed9183 --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/runtime/standalone/StandaloneHerder.java @@ -0,0 +1,257 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.runtime.standalone; + +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.copycat.connector.Connector; +import org.apache.kafka.copycat.errors.CopycatException; +import org.apache.kafka.copycat.runtime.ConnectorConfig; +import org.apache.kafka.copycat.runtime.Herder; +import org.apache.kafka.copycat.runtime.Worker; +import org.apache.kafka.copycat.sink.SinkConnector; +import org.apache.kafka.copycat.sink.SinkTask; +import org.apache.kafka.copycat.util.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.*; + +/** + * Single process, in-memory "herder". Useful for a standalone copycat process. + */ +public class StandaloneHerder implements Herder { + private static final Logger log = LoggerFactory.getLogger(StandaloneHerder.class); + + private Worker worker; + private HashMap connectors = new HashMap<>(); + + public StandaloneHerder(Worker worker) { + this.worker = worker; + } + + public synchronized void start() { + log.info("Herder starting"); + log.info("Herder started"); + } + + public synchronized void stop() { + log.info("Herder stopping"); + + // There's no coordination/hand-off to do here since this is all standalone. Instead, we + // should just clean up the stuff we normally would, i.e. cleanly checkpoint and shutdown all + // the tasks. + for (Map.Entry entry : connectors.entrySet()) { + ConnectorState state = entry.getValue(); + stopConnector(state); + } + connectors.clear(); + + log.info("Herder stopped"); + } + + @Override + public synchronized void addConnector(Properties connectorProps, + Callback callback) { + try { + ConnectorState connState = createConnector(connectorProps); + if (callback != null) + callback.onCompletion(null, connState.name); + // This should always be a new job, create jobs from scratch + createConnectorTasks(connState); + } catch (CopycatException e) { + if (callback != null) + callback.onCompletion(e, null); + } + } + + @Override + public synchronized void deleteConnector(String name, Callback callback) { + try { + destroyConnector(name); + if (callback != null) + callback.onCompletion(null, null); + } catch (CopycatException e) { + if (callback != null) + callback.onCompletion(e, null); + } + } + + // Creates the and configures the connector. Does not setup any tasks + private ConnectorState createConnector(Properties connectorProps) { + ConnectorConfig connConfig = new ConnectorConfig(connectorProps); + String connName = connConfig.getString(ConnectorConfig.NAME_CONFIG); + String className = connConfig.getString(ConnectorConfig.CONNECTOR_CLASS_CONFIG); + log.info("Creating connector {} of type {}", connName, className); + int maxTasks = connConfig.getInt(ConnectorConfig.TASKS_MAX_CONFIG); + List topics = connConfig.getList(ConnectorConfig.TOPICS_CONFIG); // Sinks only + Properties configs = connConfig.getUnusedProperties(); + + if (connectors.containsKey(connName)) { + log.error("Ignoring request to create connector due to conflicting connector name"); + throw new CopycatException("Connector with name " + connName + " already exists"); + } + + final Connector connector; + try { + connector = instantiateConnector(className); + } catch (Throwable t) { + // Catches normal exceptions due to instantiation errors as well as any runtime errors that + // may be caused by user code + throw new CopycatException("Failed to create connector instance", t); + } + connector.initialize(new StandaloneConnectorContext(this, connName)); + try { + connector.start(configs); + } catch (CopycatException e) { + throw new CopycatException("Connector threw an exception while starting", e); + } + ConnectorState state = new ConnectorState(connName, connector, maxTasks, topics); + connectors.put(connName, state); + + log.info("Finished creating connector {}", connName); + + return state; + } + + private static Connector instantiateConnector(String className) { + try { + return Utils.newInstance(className, Connector.class); + } catch (ClassNotFoundException e) { + throw new CopycatException("Couldn't instantiate connector class", e); + } + } + + private void destroyConnector(String connName) { + log.info("Destroying connector {}", connName); + ConnectorState state = connectors.get(connName); + if (state == null) { + log.error("Failed to destroy connector {} because it does not exist", connName); + throw new CopycatException("Connector does not exist"); + } + + stopConnector(state); + connectors.remove(state.name); + + log.info("Finished destroying connector {}", connName); + } + + // Stops a connectors tasks, then the connector + private void stopConnector(ConnectorState state) { + removeConnectorTasks(state); + try { + state.connector.stop(); + } catch (CopycatException e) { + log.error("Error shutting down connector {}: ", state.connector, e); + } + } + + private void createConnectorTasks(ConnectorState state) { + String taskClassName = state.connector.getTaskClass().getName(); + + log.info("Creating tasks for connector {} of type {}", state.name, taskClassName); + + List taskConfigs = state.connector.getTaskConfigs(state.maxTasks); + + // Generate the final configs, including framework provided settings + Map taskProps = new HashMap<>(); + for (int i = 0; i < taskConfigs.size(); i++) { + ConnectorTaskId taskId = new ConnectorTaskId(state.name, i); + Properties config = taskConfigs.get(i); + // TODO: This probably shouldn't be in the Herder. It's nice to have Copycat ensure the list of topics + // is automatically provided to tasks since it is required by the framework, but this + String subscriptionTopics = Utils.join(state.inputTopics, ","); + if (state.connector instanceof SinkConnector) { + // Make sure we don't modify the original since the connector may reuse it internally + Properties configForSink = new Properties(); + configForSink.putAll(config); + configForSink.setProperty(SinkTask.TOPICS_CONFIG, subscriptionTopics); + config = configForSink; + } + taskProps.put(taskId, config); + } + + // And initiate the tasks + for (int i = 0; i < taskConfigs.size(); i++) { + ConnectorTaskId taskId = new ConnectorTaskId(state.name, i); + Properties config = taskProps.get(taskId); + try { + worker.addTask(taskId, taskClassName, config); + // We only need to store the task IDs so we can clean up. + state.tasks.add(taskId); + } catch (Throwable e) { + log.error("Failed to add task {}: ", taskId, e); + // Swallow this so we can continue updating the rest of the tasks + // FIXME what's the proper response? Kill all the tasks? Consider this the same as a task + // that died after starting successfully. + } + } + } + + private void removeConnectorTasks(ConnectorState state) { + Iterator taskIter = state.tasks.iterator(); + while (taskIter.hasNext()) { + ConnectorTaskId taskId = taskIter.next(); + try { + worker.stopTask(taskId); + taskIter.remove(); + } catch (CopycatException e) { + log.error("Failed to stop task {}: ", taskId, e); + // Swallow this so we can continue stopping the rest of the tasks + // FIXME: Forcibly kill the task? + } + } + } + + private void updateConnectorTasks(ConnectorState state) { + removeConnectorTasks(state); + createConnectorTasks(state); + } + + /** + * Requests reconfiguration of the task. This should only be triggered by + * {@link StandaloneConnectorContext}. + * + * @param connName name of the connector that should be reconfigured + */ + public synchronized void requestTaskReconfiguration(String connName) { + ConnectorState state = connectors.get(connName); + if (state == null) { + log.error("Task that requested reconfiguration does not exist: {}", connName); + return; + } + updateConnectorTasks(state); + } + + + private static class ConnectorState { + public String name; + public Connector connector; + public int maxTasks; + public List inputTopics; + Set tasks; + + public ConnectorState(String name, Connector connector, int maxTasks, + List inputTopics) { + this.name = name; + this.connector = connector; + this.maxTasks = maxTasks; + this.inputTopics = inputTopics; + this.tasks = new HashSet<>(); + } + } +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/FileOffsetBackingStore.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/FileOffsetBackingStore.java new file mode 100644 index 0000000..dfa9e78 --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/FileOffsetBackingStore.java @@ -0,0 +1,111 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.storage; + +import org.apache.kafka.copycat.errors.CopycatException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.*; +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.Map; + +/** + * Implementation of OffsetBackingStore that saves data locally to a file. To ensure this behaves + * similarly to a real backing store, operations are executed asynchronously on a background thread. + */ +public class FileOffsetBackingStore extends MemoryOffsetBackingStore { + private static final Logger log = LoggerFactory.getLogger(FileOffsetBackingStore.class); + + public final static String OFFSET_STORAGE_FILE_FILENAME_CONFIG = "offset.storage.file.filename"; + private File file; + + public FileOffsetBackingStore() { + + } + + @Override + public void configure(Map props) { + super.configure(props); + String filename = (String) props.get(OFFSET_STORAGE_FILE_FILENAME_CONFIG); + file = new File(filename); + } + + @Override + public synchronized void start() { + super.start(); + log.info("Starting FileOffsetBackingStore with file {}", file); + load(); + } + + @Override + public synchronized void stop() { + super.stop(); + // Nothing to do since this doesn't maintain any outstanding connections/data + log.info("Stopped FileOffsetBackingStore"); + } + + @SuppressWarnings("unchecked") + private void load() { + try { + ObjectInputStream is = new ObjectInputStream(new FileInputStream(file)); + Object obj = is.readObject(); + if (!(obj instanceof HashMap)) + throw new CopycatException("Expected HashMap but found " + obj.getClass()); + HashMap> raw = (HashMap>) obj; + data = new HashMap<>(); + for (Map.Entry> entry : raw.entrySet()) { + HashMap converted = new HashMap<>(); + for (Map.Entry mapEntry : entry.getValue().entrySet()) { + ByteBuffer key = (mapEntry.getKey() != null) ? ByteBuffer.wrap(mapEntry.getKey()) : null; + ByteBuffer value = (mapEntry.getValue() != null) ? ByteBuffer.wrap(mapEntry.getValue()) : + null; + converted.put(key, value); + } + data.put(entry.getKey(), converted); + } + is.close(); + } catch (FileNotFoundException | EOFException e) { + // FileNotFoundException: Ignore, may be new. + // EOFException: Ignore, this means the file was missing or corrupt + } catch (IOException | ClassNotFoundException e) { + throw new CopycatException(e); + } + } + + protected void save() { + try { + ObjectOutputStream os = new ObjectOutputStream(new FileOutputStream(file)); + HashMap> raw = new HashMap<>(); + for (Map.Entry> entry : data.entrySet()) { + HashMap converted = new HashMap<>(); + for (Map.Entry mapEntry : entry.getValue().entrySet()) { + byte[] key = (mapEntry.getKey() != null) ? mapEntry.getKey().array() : null; + byte[] value = (mapEntry.getValue() != null) ? mapEntry.getValue().array() : null; + converted.put(key, value); + } + raw.put(entry.getKey(), converted); + } + os.writeObject(raw); + os.close(); + } catch (IOException e) { + throw new CopycatException(e); + } + } +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/MemoryOffsetBackingStore.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/MemoryOffsetBackingStore.java new file mode 100644 index 0000000..6ffba58 --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/MemoryOffsetBackingStore.java @@ -0,0 +1,113 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.storage; + +import org.apache.kafka.copycat.util.Callback; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.ByteBuffer; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; + +/** + * Implementation of OffsetBackingStore that doesn't actually persist any data. To ensure this + * behaves similarly to a real backing store, operations are executed asynchronously on a + * background thread. + */ +public class MemoryOffsetBackingStore implements OffsetBackingStore { + private static final Logger log = LoggerFactory.getLogger(MemoryOffsetBackingStore.class); + + protected HashMap> data = new HashMap<>(); + protected ExecutorService executor = Executors.newSingleThreadExecutor(); + + public MemoryOffsetBackingStore() { + + } + + @Override + public void configure(Map props) { + } + + @Override + public synchronized void start() { + } + + @Override + public synchronized void stop() { + // Nothing to do since this doesn't maintain any outstanding connections/data + } + + @Override + public Future> get( + final String namespace, final Collection keys, + final Callback> callback) { + return executor.submit(new Callable>() { + @Override + public Map call() throws Exception { + Map result = new HashMap<>(); + synchronized (MemoryOffsetBackingStore.this) { + Map namespaceData = data.get(namespace); + if (namespaceData == null) + return result; + for (ByteBuffer key : keys) { + result.put(key, namespaceData.get(key)); + } + } + if (callback != null) + callback.onCompletion(null, result); + return result; + } + }); + + } + + @Override + public Future set(final String namespace, final Map values, + final Callback callback) { + return executor.submit(new Callable() { + @Override + public Void call() throws Exception { + synchronized (MemoryOffsetBackingStore.this) { + Map namespaceData = data.get(namespace); + if (namespaceData == null) { + namespaceData = new HashMap<>(); + data.put(namespace, namespaceData); + } + for (Map.Entry entry : values.entrySet()) { + namespaceData.put(entry.getKey(), entry.getValue()); + } + save(); + } + if (callback != null) + callback.onCompletion(null, null); + return null; + } + }); + } + + // Hook to allow subclasses to persist data + protected void save() { + + } +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/OffsetBackingStore.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/OffsetBackingStore.java new file mode 100644 index 0000000..e8cb2ae --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/OffsetBackingStore.java @@ -0,0 +1,74 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.storage; + +import org.apache.kafka.common.Configurable; +import org.apache.kafka.copycat.util.Callback; + +import java.nio.ByteBuffer; +import java.util.Collection; +import java.util.Map; +import java.util.concurrent.Future; + +/** + *

    + * OffsetBackingStore is an interface for storage backends that store key-value data. The backing + * store doesn't need to handle serialization or deserialization. It only needs to support + * reading/writing bytes. Since it is expected these operations will require network + * operations, only bulk operations are supported. + *

    + *

    + * Since OffsetBackingStore is a shared resource that may be used by many OffsetStorage instances + * that are associated with individual tasks, all operations include a namespace which should be + * used to isolate different key spaces. + *

    + */ +public interface OffsetBackingStore extends Configurable { + + /** + * Start this offset store. + */ + public void start(); + + /** + * Stop the backing store. Implementations should attempt to shutdown gracefully, but not block + * indefinitely. + */ + public void stop(); + + /** + * Get the values for the specified keys + * @param namespace prefix for the keys in this request + * @param keys list of keys to look up + * @param callback callback to invoke on completion + * @return future for the resulting map from key to value + */ + public Future> get( + String namespace, Collection keys, + Callback> callback); + + /** + * Set the specified keys and values. + * @param namespace prefix for the keys in this request + * @param values map from key to value + * @param callback callback to invoke on completion + * @return void future for the operation + */ + public Future set(String namespace, Map values, + Callback callback); +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/OffsetStorageReaderImpl.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/OffsetStorageReaderImpl.java new file mode 100644 index 0000000..7a050dc --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/OffsetStorageReaderImpl.java @@ -0,0 +1,114 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.storage; + +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.copycat.errors.CopycatException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; + +/** + * Implementation of OffsetStorageReader. Unlike OffsetStorageWriter which is implemented + * directly, the interface is only separate from this implementation because it needs to be + * included in the public API package. + */ +public class OffsetStorageReaderImpl implements OffsetStorageReader { + private static final Logger log = LoggerFactory.getLogger(OffsetStorageReaderImpl.class); + + private final OffsetBackingStore backingStore; + private final String namespace; + private final Converter keyConverter; + private final Converter valueConverter; + private final Serializer keySerializer; + private final Deserializer valueDeserializer; + + public OffsetStorageReaderImpl(OffsetBackingStore backingStore, String namespace, + Converter keyConverter, Converter valueConverter, + Serializer keySerializer, Deserializer valueDeserializer) { + this.backingStore = backingStore; + this.namespace = namespace; + this.keyConverter = keyConverter; + this.valueConverter = valueConverter; + this.keySerializer = keySerializer; + this.valueDeserializer = valueDeserializer; + } + + @Override + public Object getOffset(Object partition) { + return getOffsets(Arrays.asList(partition)).get(partition); + } + + @Override + public Map getOffsets(Collection partitions) { + // Serialize keys so backing store can work with them + Map serializedToOriginal = new HashMap<>(partitions.size()); + for (Object key : partitions) { + try { + byte[] keySerialized = keySerializer.serialize(namespace, keyConverter.fromCopycatData(key)); + ByteBuffer keyBuffer = (keySerialized != null) ? ByteBuffer.wrap(keySerialized) : null; + serializedToOriginal.put(keyBuffer, key); + } catch (Throwable t) { + log.error("CRITICAL: Failed to serialize partition key when getting offsets for task with " + + "namespace {}. No value for this data will be returned, which may break the " + + "task or cause it to skip some data.", namespace, t); + } + } + + // Get serialized key -> serialized value from backing store + Map raw; + try { + raw = backingStore.get(namespace, serializedToOriginal.keySet(), null).get(); + } catch (Exception e) { + log.error("Failed to fetch offsets from namespace {}: ", namespace, e); + throw new CopycatException("Failed to fetch offsets.", e); + } + + // Deserialize all the values and map back to the original keys + Map result = new HashMap<>(partitions.size()); + for (Map.Entry rawEntry : raw.entrySet()) { + try { + // Since null could be a valid key, explicitly check whether map contains the key + if (!serializedToOriginal.containsKey(rawEntry.getKey())) { + log.error("Should be able to map {} back to a requested partition-offset key, backing " + + "store may have returned invalid data", rawEntry.getKey()); + continue; + } + Object origKey = serializedToOriginal.get(rawEntry.getKey()); + Object deserializedValue = valueConverter.toCopycatData( + valueDeserializer.deserialize(namespace, rawEntry.getValue().array()) + ); + + result.put(origKey, deserializedValue); + } catch (Throwable t) { + log.error("CRITICAL: Failed to deserialize offset data when getting offsets for task with" + + " namespace {}. No value for this data will be returned, which may break the " + + "task or cause it to skip some data. This could either be due to an error in " + + "the connector implementation or incompatible schema.", namespace, t); + } + } + + return result; + } +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/OffsetStorageWriter.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/OffsetStorageWriter.java new file mode 100644 index 0000000..c6e829c --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/storage/OffsetStorageWriter.java @@ -0,0 +1,208 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.storage; + +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.copycat.errors.CopycatException; +import org.apache.kafka.copycat.util.Callback; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.Future; + +/** + *

    + * OffsetStorageWriter is a buffered writer that wraps the simple OffsetBackingStore interface. + * It maintains a copy of the key-value data in memory and buffers writes. It allows you to take + * a snapshot, which can then be asynchronously flushed to the backing store while new writes + * continue to be processed. This allows Copycat to process offset commits in the background + * while continuing to process messages. + *

    + *

    + * Copycat uses an OffsetStorage implementation to save state about the current progress of + * source (import to Kafka) jobs, which may have many input partitions and "offsets" may not be as + * simple as they are for Kafka partitions or files. Offset storage is not required for sink jobs + * because they can use Kafka's native offset storage (or the sink data store can handle offset + * storage to achieve exactly once semantics). + *

    + *

    + * Both partitions and offsets are generic data objects. This allows different connectors to use + * whatever representation they need, even arbitrarily complex records. These are translated + * internally into the serialized form the OffsetBackingStore uses. + *

    + *

    + * Note that this only provides write functionality. This is intentional to ensure stale data is + * never read. Offset data should only be read during startup or reconfiguration of a task. By + * always serving those requests by reading the values from the backing store, we ensure we never + * accidentally use stale data. (One example of how this can occur: a task is processing input + * partition A, writing offsets; reconfiguration causes partition A to be reassigned elsewhere; + * reconfiguration causes partition A to be reassigned to this node, but now the offset data is out + * of date). Since these offsets are created and managed by the connector itself, there's no way + * for the offset management layer to know which keys are "owned" by which tasks at any given + * time. + *

    + *

    + * This class is not thread-safe. It should only be accessed from a Task's processing thread. + *

    + */ +public class OffsetStorageWriter { + private static final Logger log = LoggerFactory.getLogger(OffsetStorageWriter.class); + + private final OffsetBackingStore backingStore; + private final Converter keyConverter; + private final Converter valueConverter; + private final Serializer keySerializer; + private final Serializer valueSerializer; + private final String namespace; + // Offset data in Copycat format + private Map data = new HashMap<>(); + + // Not synchronized, should only be accessed by flush thread + private Map toFlush = null; + // Unique ID for each flush request to handle callbacks after timeouts + private long currentFlushId = 0; + + public OffsetStorageWriter(OffsetBackingStore backingStore, + String namespace, Converter keyConverter, Converter valueConverter, + Serializer keySerializer, Serializer valueSerializer) { + this.backingStore = backingStore; + this.namespace = namespace; + this.keyConverter = keyConverter; + this.valueConverter = valueConverter; + this.keySerializer = keySerializer; + this.valueSerializer = valueSerializer; + } + + /** + * Set an offset for a partition using Copycat data values + * @param partition the partition to store an offset for + * @param offset the offset + */ + public synchronized void setOffset(Object partition, Object offset) { + data.put(partition, offset); + } + + private boolean flushing() { + return toFlush != null; + } + + /** + * Performs the first step of a flush operation, snapshotting the current state. This does not + * actually initiate the flush with the underlying storage. + * + * @return true if a flush was initiated, false if no data was available + */ + public synchronized boolean beginFlush() { + if (flushing()) { + log.error("Invalid call to OffsetStorageWriter flush() while already flushing, the " + + "framework should not allow this"); + throw new CopycatException("OffsetStorageWriter is already flushing"); + } + + if (data.isEmpty()) + return false; + + assert !flushing(); + toFlush = data; + data = new HashMap<>(); + return true; + } + + /** + * Flush the current offsets and clear them from this writer. This is non-blocking: it + * moves the current set of offsets out of the way, serializes the data, and asynchronously + * writes the data to the backing store. If no offsets need to be written, the callback is + * still invoked, but no Future is returned. + * + * @return a Future, or null if there are no offsets to commitOffsets + */ + public Future doFlush(final Callback callback) { + final long flushId = currentFlushId; + + // Serialize + Map offsetsSerialized; + try { + offsetsSerialized = new HashMap<>(); + for (Map.Entry entry : toFlush.entrySet()) { + byte[] key = keySerializer.serialize(namespace, keyConverter.fromCopycatData(entry.getKey())); + ByteBuffer keyBuffer = (key != null) ? ByteBuffer.wrap(key) : null; + byte[] value = valueSerializer.serialize(namespace, valueConverter.fromCopycatData(entry.getValue())); + ByteBuffer valueBuffer = (value != null) ? ByteBuffer.wrap(value) : null; + offsetsSerialized.put(keyBuffer, valueBuffer); + } + } catch (Throwable t) { + // Must handle errors properly here or the writer will be left mid-flush forever and be + // unable to make progress. + log.error("CRITICAL: Failed to serialize offset data, making it impossible to commit " + + "offsets under namespace {}. This likely won't recover unless the " + + "unserializable partition or offset information is overwritten.", namespace); + callback.onCompletion(t, null); + return null; + } + + // And submit the data + log.debug("Submitting {} entries to backing store", offsetsSerialized.size()); + return backingStore.set(namespace, offsetsSerialized, new Callback() { + @Override + public void onCompletion(Throwable error, Void result) { + boolean isCurrent = handleFinishWrite(flushId, error, result); + if (isCurrent && callback != null) + callback.onCompletion(error, result); + } + }); + } + + /** + * Cancel a flush that has been initiated by {@link #beginFlush}. This should not be called if + * {@link #doFlush} has already been invoked. It should be used if an operation performed + * between beginFlush and doFlush failed. + */ + public synchronized void cancelFlush() { + // Verify we're still flushing data to handle a race between cancelFlush() calls from up the + // call stack and callbacks from the write request to underlying storage + if (flushing()) { + // Just recombine the data and place it back in the primary storage + toFlush.putAll(data); + data = toFlush; + currentFlushId++; + toFlush = null; + } + } + + /** + * Handle completion of a write. Returns true if this callback is for the current flush + * operation, false if it's for an old operation that should now be ignored. + */ + private synchronized boolean handleFinishWrite(long flushId, Throwable error, Void result) { + // Callbacks need to be handled carefully since the flush operation may have already timed + // out and been cancelled. + if (flushId != currentFlushId) + return false; + + if (error != null) { + cancelFlush(); + } else { + currentFlushId++; + toFlush = null; + } + return true; + } +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/util/Callback.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/util/Callback.java new file mode 100644 index 0000000..5cf1423 --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/util/Callback.java @@ -0,0 +1,31 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.util; + +/** + * Generic interface for callbacks + */ +public interface Callback { + /** + * Invoked upon completion of the operation. + * + * @param error the error that caused the operation to fail, or null if no error occurred + * @param result the return value, or null if the operation failed + */ + void onCompletion(Throwable error, V result); +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/util/ConnectorTaskId.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/util/ConnectorTaskId.java new file mode 100644 index 0000000..44a9e41 --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/util/ConnectorTaskId.java @@ -0,0 +1,71 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.util; + +import java.io.Serializable; + +/** + * Unique ID for a single task. It includes a unique connector ID and a task ID that is unique within + * the connector. + */ +public class ConnectorTaskId implements Serializable { + private final String connector; + private final int task; + + public ConnectorTaskId(String job, int task) { + this.connector = job; + this.task = task; + } + + public String getConnector() { + return connector; + } + + public int getTask() { + return task; + } + + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + ConnectorTaskId that = (ConnectorTaskId) o; + + if (task != that.task) + return false; + if (connector != null ? !connector.equals(that.connector) : that.connector != null) + return false; + + return true; + } + + @Override + public int hashCode() { + int result = connector != null ? connector.hashCode() : 0; + result = 31 * result + task; + return result; + } + + @Override + public String toString() { + return connector + '-' + task; + } +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/util/FutureCallback.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/util/FutureCallback.java new file mode 100644 index 0000000..278fdd3 --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/util/FutureCallback.java @@ -0,0 +1,76 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.util; + +import java.util.concurrent.*; + +public class FutureCallback implements Callback, Future { + + private Callback underlying; + private CountDownLatch finishedLatch; + private T result = null; + private Throwable exception = null; + + public FutureCallback(Callback underlying) { + this.underlying = underlying; + this.finishedLatch = new CountDownLatch(1); + } + + @Override + public void onCompletion(Throwable error, T result) { + underlying.onCompletion(error, result); + this.exception = error; + this.result = result; + finishedLatch.countDown(); + } + + @Override + public boolean cancel(boolean b) { + return false; + } + + @Override + public boolean isCancelled() { + return false; + } + + @Override + public boolean isDone() { + return finishedLatch.getCount() == 0; + } + + @Override + public T get() throws InterruptedException, ExecutionException { + finishedLatch.await(); + return getResult(); + } + + @Override + public T get(long l, TimeUnit timeUnit) + throws InterruptedException, ExecutionException, TimeoutException { + finishedLatch.await(l, timeUnit); + return getResult(); + } + + private T getResult() throws ExecutionException { + if (exception != null) { + throw new ExecutionException(exception); + } + return result; + } +} diff --git a/copycat/runtime/src/main/java/org/apache/kafka/copycat/util/ShutdownableThread.java b/copycat/runtime/src/main/java/org/apache/kafka/copycat/util/ShutdownableThread.java new file mode 100644 index 0000000..3e23f29 --- /dev/null +++ b/copycat/runtime/src/main/java/org/apache/kafka/copycat/util/ShutdownableThread.java @@ -0,0 +1,145 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.util; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + *

    + * Thread class with support for triggering graceful and forcible shutdown. In graceful shutdown, + * a flag is set, which the thread should detect and try to exit gracefully from. In forcible + * shutdown, the thread is interrupted. These can be combined to give a thread a chance to exit + * gracefully, but then force it to exit if it takes too long. + *

    + *

    + * Implementations should override the {@link #execute} method and check {@link #getRunning} to + * determine whether they should try to gracefully exit. + *

    + */ +public abstract class ShutdownableThread extends Thread { + private static final Logger log = LoggerFactory.getLogger(ShutdownableThread.class); + + private AtomicBoolean isRunning = new AtomicBoolean(true); + private CountDownLatch shutdownLatch = new CountDownLatch(1); + + /** + * An UncaughtExceptionHandler to register on every instance of this class. This is useful for + * testing, where AssertionExceptions in the thread may not cause the test to fail. Since one + * instance is used for all threads, it must be thread-safe. + */ + volatile public static UncaughtExceptionHandler funcaughtExceptionHandler = null; + + public ShutdownableThread(String name) { + // The default is daemon=true so that these threads will not prevent shutdown. We use this + // default because threads that are running user code that may not clean up properly, even + // when we attempt to forcibly shut them down. + this(name, true); + } + + public ShutdownableThread(String name, boolean daemon) { + super(name); + this.setDaemon(daemon); + if (funcaughtExceptionHandler != null) + this.setUncaughtExceptionHandler(funcaughtExceptionHandler); + } + + /** + * Implementations should override this method with the main body for the thread. + */ + public abstract void execute(); + + /** + * Returns true if the thread hasn't exited yet and none of the shutdown methods have been + * invoked + */ + public boolean getRunning() { + return isRunning.get(); + } + + @Override + public void run() { + try { + execute(); + } catch (Error | RuntimeException e) { + log.error("Thread {} exiting with uncaught exception: ", getName(), e); + throw e; + } finally { + shutdownLatch.countDown(); + } + } + + /** + * Shutdown the thread, first trying to shut down gracefully using the specified timeout, then + * forcibly interrupting the thread. + * @param gracefulTimeout the maximum time to wait for a graceful exit + * @param unit the time unit of the timeout argument + */ + public void shutdown(long gracefulTimeout, TimeUnit unit) + throws InterruptedException { + boolean success = gracefulShutdown(gracefulTimeout, unit); + if (!success) + forceShutdown(); + } + + /** + * Attempt graceful shutdown + * @param timeout the maximum time to wait + * @param unit the time unit of the timeout argument + * @return true if successful, false if the timeout elapsed + */ + public boolean gracefulShutdown(long timeout, TimeUnit unit) throws InterruptedException { + startGracefulShutdown(); + return awaitShutdown(timeout, unit); + } + + /** + * Start shutting down this thread gracefully, but do not block waiting for it to exit. + */ + public void startGracefulShutdown() { + log.info("Starting graceful shutdown of thread {}", getName()); + isRunning.set(false); + } + + /** + * Awaits shutdown of this thread, waiting up to the timeout. + * @param timeout the maximum time to wait + * @param unit the time unit of the timeout argument + * @return true if successful, false if the timeout elapsed + * @throws InterruptedException + */ + public boolean awaitShutdown(long timeout, TimeUnit unit) throws InterruptedException { + return shutdownLatch.await(timeout, unit); + } + + /** + * Immediately tries to force the thread to shut down by interrupting it. This does not try to + * wait for the thread to truly exit because forcible shutdown is not always possible. By + * default, threads are marked as daemon threads so they will not prevent the process from + * exiting. + */ + public void forceShutdown() throws InterruptedException { + log.info("Forcing shutdown of thread {}", getName()); + isRunning.set(false); + interrupt(); + } +} diff --git a/copycat/runtime/src/test/java/org/apache/kafka/copycat/runtime/WorkerSinkTaskTest.java b/copycat/runtime/src/test/java/org/apache/kafka/copycat/runtime/WorkerSinkTaskTest.java new file mode 100644 index 0000000..0c6f950 --- /dev/null +++ b/copycat/runtime/src/test/java/org/apache/kafka/copycat/runtime/WorkerSinkTaskTest.java @@ -0,0 +1,367 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.runtime; + +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.clients.consumer.*; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.copycat.cli.WorkerConfig; +import org.apache.kafka.copycat.sink.SinkRecord; +import org.apache.kafka.copycat.sink.SinkTask; +import org.apache.kafka.copycat.sink.SinkTaskContext; +import org.apache.kafka.copycat.storage.Converter; +import org.apache.kafka.copycat.util.ConnectorTaskId; +import org.apache.kafka.copycat.util.MockTime; +import org.apache.kafka.copycat.util.ThreadedTest; +import org.easymock.*; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.easymock.PowerMock; +import org.powermock.api.easymock.annotation.Mock; +import org.powermock.core.classloader.annotations.PowerMockIgnore; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; +import org.powermock.reflect.Whitebox; + +import java.util.*; + +import static org.junit.Assert.assertEquals; + +@RunWith(PowerMockRunner.class) +@PrepareForTest(WorkerSinkTask.class) +@PowerMockIgnore("javax.management.*") +public class WorkerSinkTaskTest extends ThreadedTest { + + // These are fixed to keep this code simpler. In this example we assume byte[] raw values + // with mix of integer/string in Copycat + private static final String TOPIC = "test"; + private static final int PARTITION = 12; + private static final long FIRST_OFFSET = 45; + private static final int KEY = 12; + private static final String VALUE = "VALUE"; + private static final byte[] RAW_KEY = "key".getBytes(); + private static final byte[] RAW_VALUE = "value".getBytes(); + + private static final TopicPartition TOPIC_PARTITION = new TopicPartition(TOPIC, PARTITION); + + private ConnectorTaskId taskId = new ConnectorTaskId("job", 0); + private Time time; + @Mock private SinkTask sinkTask; + private WorkerConfig workerConfig; + @Mock private Converter keyConverter; + @Mock + private Converter valueConverter; + private WorkerSinkTask workerTask; + @Mock private KafkaConsumer consumer; + private WorkerSinkTaskThread workerThread; + + private long recordsReturned; + + @SuppressWarnings("unchecked") + @Override + public void setup() { + super.setup(); + time = new MockTime(); + Properties workerProps = new Properties(); + workerProps.setProperty("key.converter", "org.apache.kafka.copycat.json.JsonConverter"); + workerProps.setProperty("value.converter", "org.apache.kafka.copycat.json.JsonConverter"); + workerProps.setProperty("key.serializer", "org.apache.kafka.copycat.json.JsonSerializer"); + workerProps.setProperty("value.serializer", "org.apache.kafka.copycat.json.JsonSerializer"); + workerProps.setProperty("key.deserializer", "org.apache.kafka.copycat.json.JsonDeserializer"); + workerProps.setProperty("value.deserializer", "org.apache.kafka.copycat.json.JsonDeserializer"); + workerConfig = new WorkerConfig(workerProps); + workerTask = PowerMock.createPartialMock( + WorkerSinkTask.class, new String[]{"createConsumer", "createWorkerThread"}, + taskId, sinkTask, workerConfig, keyConverter, valueConverter, time); + + recordsReturned = 0; + } + + @Test + public void testPollsInBackground() throws Exception { + Properties taskProps = new Properties(); + + expectInitializeTask(taskProps); + Capture> capturedRecords = expectPolls(1L); + expectStopTask(10L); + + PowerMock.replayAll(); + + workerTask.start(taskProps); + for (int i = 0; i < 10; i++) { + workerThread.iteration(); + } + workerTask.stop(); + // No need for awaitStop since the thread is mocked + workerTask.close(); + + // Verify contents match expected values, i.e. that they were translated properly. With max + // batch size 1 and poll returns 1 message at a time, we should have a matching # of batches + assertEquals(10, capturedRecords.getValues().size()); + int offset = 0; + for (Collection recs : capturedRecords.getValues()) { + assertEquals(1, recs.size()); + for (SinkRecord rec : recs) { + SinkRecord referenceSinkRecord + = new SinkRecord(TOPIC, PARTITION, KEY, VALUE, FIRST_OFFSET + offset); + assertEquals(referenceSinkRecord, rec); + offset++; + } + } + + PowerMock.verifyAll(); + } + + @Test + public void testDeliverConvertsData() throws Exception { + // Validate conversion is performed when data is delivered + Integer record = 12; + + ConsumerRecords records = new ConsumerRecords<>( + Collections.singletonMap( + new TopicPartition("topic", 0), + Collections.singletonList(new ConsumerRecord<>("topic", 0, 0, RAW_KEY, RAW_VALUE)))); + + // Exact data doesn't matter, but should be passed directly to sink task + EasyMock.expect(keyConverter.toCopycatData(RAW_KEY)).andReturn(record); + EasyMock.expect(valueConverter.toCopycatData(RAW_VALUE)).andReturn(record); + Capture> capturedRecords + = EasyMock.newCapture(CaptureType.ALL); + sinkTask.put(EasyMock.capture(capturedRecords)); + EasyMock.expectLastCall(); + + PowerMock.replayAll(); + + Whitebox.invokeMethod(workerTask, "deliverMessages", records); + assertEquals(record, capturedRecords.getValue().iterator().next().getKey()); + assertEquals(record, capturedRecords.getValue().iterator().next().getValue()); + + PowerMock.verifyAll(); + } + + @Test + public void testCommit() throws Exception { + Properties taskProps = new Properties(); + + expectInitializeTask(taskProps); + // Make each poll() take the offset commit interval + Capture> capturedRecords + = expectPolls(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_DEFAULT); + expectOffsetFlush(1L, null, null, 0, true); + expectStopTask(2); + + PowerMock.replayAll(); + + workerTask.start(taskProps); + // First iteration gets one record + workerThread.iteration(); + // Second triggers commit, gets a second offset + workerThread.iteration(); + // Commit finishes synchronously for testing so we can check this immediately + assertEquals(0, workerThread.getCommitFailures()); + workerTask.stop(); + workerTask.close(); + + assertEquals(2, capturedRecords.getValues().size()); + + PowerMock.verifyAll(); + } + + @Test + public void testCommitTaskFlushFailure() throws Exception { + Properties taskProps = new Properties(); + + expectInitializeTask(taskProps); + Capture> capturedRecords + = expectPolls(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_DEFAULT); + expectOffsetFlush(1L, new RuntimeException(), null, 0, true); + expectStopTask(2); + + PowerMock.replayAll(); + + workerTask.start(taskProps); + // Second iteration triggers commit + workerThread.iteration(); + workerThread.iteration(); + assertEquals(1, workerThread.getCommitFailures()); + assertEquals(false, Whitebox.getInternalState(workerThread, "committing")); + workerTask.stop(); + workerTask.close(); + + PowerMock.verifyAll(); + } + + @Test + public void testCommitConsumerFailure() throws Exception { + Properties taskProps = new Properties(); + + expectInitializeTask(taskProps); + Capture> capturedRecords + = expectPolls(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_DEFAULT); + expectOffsetFlush(1L, null, new Exception(), 0, true); + expectStopTask(2); + + PowerMock.replayAll(); + + workerTask.start(taskProps); + // Second iteration triggers commit + workerThread.iteration(); + workerThread.iteration(); + // TODO Response to consistent failures? + assertEquals(1, workerThread.getCommitFailures()); + assertEquals(false, Whitebox.getInternalState(workerThread, "committing")); + workerTask.stop(); + workerTask.close(); + + PowerMock.verifyAll(); + } + + @Test + public void testCommitTimeout() throws Exception { + Properties taskProps = new Properties(); + + expectInitializeTask(taskProps); + // Cut down amount of time to pass in each poll so we trigger exactly 1 offset commit + Capture> capturedRecords + = expectPolls(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_DEFAULT / 2); + expectOffsetFlush(2L, null, null, WorkerConfig.OFFSET_COMMIT_TIMEOUT_MS_DEFAULT, false); + expectStopTask(4); + + PowerMock.replayAll(); + + workerTask.start(taskProps); + // Third iteration triggers commit, fourth gives a chance to trigger the timeout but doesn't + // trigger another commit + workerThread.iteration(); + workerThread.iteration(); + workerThread.iteration(); + workerThread.iteration(); + // TODO Response to consistent failures? + assertEquals(1, workerThread.getCommitFailures()); + assertEquals(false, Whitebox.getInternalState(workerThread, "committing")); + workerTask.stop(); + workerTask.close(); + + PowerMock.verifyAll(); + } + + private KafkaConsumer expectInitializeTask(Properties taskProps) + throws Exception { + sinkTask.initialize(EasyMock.anyObject(SinkTaskContext.class)); + PowerMock.expectLastCall(); + sinkTask.start(taskProps); + PowerMock.expectLastCall(); + + PowerMock.expectPrivate(workerTask, "createConsumer", taskProps) + .andReturn(consumer); + workerThread = PowerMock.createPartialMock(WorkerSinkTaskThread.class, new String[]{"start"}, + workerTask, "mock-worker-thread", time, + workerConfig); + PowerMock.expectPrivate(workerTask, "createWorkerThread") + .andReturn(workerThread); + workerThread.start(); + PowerMock.expectLastCall(); + return consumer; + } + + private void expectStopTask(final long expectedMessages) throws Exception { + final long finalOffset = FIRST_OFFSET + expectedMessages - 1; + + sinkTask.stop(); + PowerMock.expectLastCall(); + + // No offset commit since it happens in the mocked worker thread, but the main thread does need to wake up the + // consumer so it exits quickly + consumer.wakeup(); + PowerMock.expectLastCall(); + + consumer.close(); + PowerMock.expectLastCall(); + } + + // Note that this can only be called once per test currently + private Capture> expectPolls(final long pollDelayMs) throws Exception { + // Stub out all the consumer stream/iterator responses, which we just want to verify occur, + // but don't care about the exact details here. + EasyMock.expect(consumer.poll(EasyMock.anyLong())).andStubAnswer( + new IAnswer>() { + @Override + public ConsumerRecords answer() throws Throwable { + // "Sleep" so time will progress + time.sleep(pollDelayMs); + ConsumerRecords records = new ConsumerRecords<>( + Collections.singletonMap( + new TopicPartition(TOPIC, PARTITION), + Arrays.asList( + new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned, RAW_KEY, RAW_VALUE) + ))); + recordsReturned++; + return records; + } + }); + EasyMock.expect(keyConverter.toCopycatData(RAW_KEY)).andReturn(KEY).anyTimes(); + EasyMock.expect(valueConverter.toCopycatData(RAW_VALUE)).andReturn(VALUE).anyTimes(); + Capture> capturedRecords = EasyMock.newCapture(CaptureType.ALL); + sinkTask.put(EasyMock.capture(capturedRecords)); + EasyMock.expectLastCall().anyTimes(); + return capturedRecords; + } + + private Capture expectOffsetFlush(final long expectedMessages, + final RuntimeException flushError, + final Exception consumerCommitError, + final long consumerCommitDelayMs, + final boolean invokeCallback) + throws Exception { + final long finalOffset = FIRST_OFFSET + expectedMessages - 1; + + EasyMock.expect(consumer.subscriptions()).andReturn(Collections.singleton(TOPIC_PARTITION)); + EasyMock.expect(consumer.position(TOPIC_PARTITION)).andAnswer( + new IAnswer() { + @Override + public Long answer() throws Throwable { + return FIRST_OFFSET + recordsReturned - 1; + } + } + ); + + sinkTask.flush(Collections.singletonMap(TOPIC_PARTITION, finalOffset)); + IExpectationSetters flushExpectation = PowerMock.expectLastCall(); + if (flushError != null) { + flushExpectation.andThrow(flushError).once(); + return null; + } + + final Capture capturedCallback = EasyMock.newCapture(); + final Map offsets = Collections.singletonMap(TOPIC_PARTITION, finalOffset); + consumer.commit(EasyMock.eq(offsets), + EasyMock.eq(CommitType.ASYNC), + EasyMock.capture(capturedCallback)); + PowerMock.expectLastCall().andAnswer(new IAnswer() { + @Override + public Object answer() throws Throwable { + time.sleep(consumerCommitDelayMs); + if (invokeCallback) + capturedCallback.getValue().onComplete(offsets, consumerCommitError); + return null; + } + }); + return capturedCallback; + } + +} diff --git a/copycat/runtime/src/test/java/org/apache/kafka/copycat/runtime/WorkerSourceTaskTest.java b/copycat/runtime/src/test/java/org/apache/kafka/copycat/runtime/WorkerSourceTaskTest.java new file mode 100644 index 0000000..60e1462 --- /dev/null +++ b/copycat/runtime/src/test/java/org/apache/kafka/copycat/runtime/WorkerSourceTaskTest.java @@ -0,0 +1,279 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.runtime; + +import org.apache.kafka.common.utils.SystemTime; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.copycat.cli.WorkerConfig; +import org.apache.kafka.copycat.source.SourceRecord; +import org.apache.kafka.copycat.source.SourceTask; +import org.apache.kafka.copycat.source.SourceTaskContext; +import org.apache.kafka.copycat.storage.Converter; +import org.apache.kafka.copycat.storage.OffsetStorageReader; +import org.apache.kafka.copycat.storage.OffsetStorageWriter; +import org.apache.kafka.copycat.util.Callback; +import org.apache.kafka.copycat.util.ConnectorTaskId; +import org.apache.kafka.copycat.util.ThreadedTest; +import org.easymock.Capture; +import org.easymock.EasyMock; +import org.easymock.IAnswer; +import org.easymock.IExpectationSetters; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.easymock.PowerMock; +import org.powermock.api.easymock.annotation.Mock; +import org.powermock.modules.junit4.PowerMockRunner; +import org.powermock.reflect.Whitebox; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Properties; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import static org.junit.Assert.*; + +@RunWith(PowerMockRunner.class) +public class WorkerSourceTaskTest extends ThreadedTest { + private static final byte[] PARTITION_BYTES = "partition".getBytes(); + private static final byte[] OFFSET_BYTES = "offset-1".getBytes(); + + // Copycat-format data + private static final Integer KEY = -1; + private static final Long RECORD = 12L; + // Native-formatted data. The actual format of this data doesn't matter -- we just want to see that the right version + // is used in the right place. + private static final ByteBuffer CONVERTED_KEY = ByteBuffer.wrap("converted-key".getBytes()); + private static final String CONVERTED_RECORD = "converted-record"; + + private ConnectorTaskId taskId = new ConnectorTaskId("job", 0); + private WorkerConfig config; + @Mock private SourceTask sourceTask; + @Mock private Converter keyConverter; + @Mock private Converter valueConverter; + @Mock private KafkaProducer producer; + @Mock private OffsetStorageReader offsetReader; + @Mock private OffsetStorageWriter offsetWriter; + private WorkerSourceTask workerTask; + @Mock private Future sendFuture; + + private Capture producerCallbacks; + + private static final Properties EMPTY_TASK_PROPS = new Properties(); + private static final List RECORDS = Arrays.asList( + new SourceRecord(PARTITION_BYTES, OFFSET_BYTES, "topic", null, KEY, RECORD) + ); + + @Override + public void setup() { + super.setup(); + Properties workerProps = new Properties(); + workerProps.setProperty("key.converter", "org.apache.kafka.copycat.json.JsonConverter"); + workerProps.setProperty("value.converter", "org.apache.kafka.copycat.json.JsonConverter"); + workerProps.setProperty("key.serializer", "org.apache.kafka.copycat.json.JsonSerializer"); + workerProps.setProperty("value.serializer", "org.apache.kafka.copycat.json.JsonSerializer"); + workerProps.setProperty("key.deserializer", "org.apache.kafka.copycat.json.JsonDeserializer"); + workerProps.setProperty("value.deserializer", "org.apache.kafka.copycat.json.JsonDeserializer"); + config = new WorkerConfig(workerProps); + producerCallbacks = EasyMock.newCapture(); + } + + private void createWorkerTask() { + workerTask = new WorkerSourceTask<>(taskId, sourceTask, keyConverter, valueConverter, producer, + offsetReader, offsetWriter, config, new SystemTime()); + } + + @Test + public void testPollsInBackground() throws Exception { + createWorkerTask(); + + sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class)); + EasyMock.expectLastCall(); + sourceTask.start(EMPTY_TASK_PROPS); + EasyMock.expectLastCall(); + + final CountDownLatch pollLatch = expectPolls(10); + // In this test, we don't flush, so nothing goes any further than the offset writer + + sourceTask.stop(); + EasyMock.expectLastCall(); + expectOffsetFlush(true); + + PowerMock.replayAll(); + + workerTask.start(EMPTY_TASK_PROPS); + awaitPolls(pollLatch); + workerTask.stop(); + assertEquals(true, workerTask.awaitStop(1000)); + + PowerMock.verifyAll(); + } + + @Test + public void testCommit() throws Exception { + // Test that the task commits properly when prompted + createWorkerTask(); + + sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class)); + EasyMock.expectLastCall(); + sourceTask.start(EMPTY_TASK_PROPS); + EasyMock.expectLastCall(); + + // We'll wait for some data, then trigger a flush + final CountDownLatch pollLatch = expectPolls(1); + expectOffsetFlush(true); + + sourceTask.stop(); + EasyMock.expectLastCall(); + expectOffsetFlush(true); + + PowerMock.replayAll(); + + workerTask.start(EMPTY_TASK_PROPS); + awaitPolls(pollLatch); + assertTrue(workerTask.commitOffsets()); + workerTask.stop(); + assertEquals(true, workerTask.awaitStop(1000)); + + PowerMock.verifyAll(); + } + + @Test + public void testCommitFailure() throws Exception { + // Test that the task commits properly when prompted + createWorkerTask(); + + sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class)); + EasyMock.expectLastCall(); + sourceTask.start(EMPTY_TASK_PROPS); + EasyMock.expectLastCall(); + + // We'll wait for some data, then trigger a flush + final CountDownLatch pollLatch = expectPolls(1); + expectOffsetFlush(false); + + sourceTask.stop(); + EasyMock.expectLastCall(); + expectOffsetFlush(true); + + PowerMock.replayAll(); + + workerTask.start(EMPTY_TASK_PROPS); + awaitPolls(pollLatch); + assertFalse(workerTask.commitOffsets()); + workerTask.stop(); + assertEquals(true, workerTask.awaitStop(1000)); + + PowerMock.verifyAll(); + } + + @Test + public void testSendRecordsConvertsData() throws Exception { + createWorkerTask(); + + List records = new ArrayList<>(); + // Can just use the same record for key and value + records.add(new SourceRecord(PARTITION_BYTES, OFFSET_BYTES, "topic", null, KEY, RECORD)); + + Capture> sent = expectSendRecord(); + + PowerMock.replayAll(); + + Whitebox.invokeMethod(workerTask, "sendRecords", records); + assertEquals(CONVERTED_KEY, sent.getValue().key()); + assertEquals(CONVERTED_RECORD, sent.getValue().value()); + + PowerMock.verifyAll(); + } + + + private CountDownLatch expectPolls(int count) throws InterruptedException { + final CountDownLatch latch = new CountDownLatch(count); + // Note that we stub these to allow any number of calls because the thread will continue to + // run. The count passed in + latch returned just makes sure we get *at least* that number of + // calls + EasyMock.expect(sourceTask.poll()) + .andStubAnswer(new IAnswer>() { + @Override + public List answer() throws Throwable { + latch.countDown(); + return RECORDS; + } + }); + // Fallout of the poll() call + expectSendRecord(); + return latch; + } + + private Capture> expectSendRecord() throws InterruptedException { + EasyMock.expect(keyConverter.fromCopycatData(KEY)).andStubReturn(CONVERTED_KEY); + EasyMock.expect(valueConverter.fromCopycatData(RECORD)).andStubReturn(CONVERTED_RECORD); + + Capture> sent = EasyMock.newCapture(); + // 1. Converted data passed to the producer, which will need callbacks invoked for flush to work + EasyMock.expect( + producer.send(EasyMock.capture(sent), + EasyMock.capture(producerCallbacks))) + .andStubAnswer(new IAnswer>() { + @Override + public Future answer() throws Throwable { + synchronized (producerCallbacks) { + for (org.apache.kafka.clients.producer.Callback cb : producerCallbacks.getValues()) { + cb.onCompletion(new RecordMetadata(new TopicPartition("foo", 0), 0, 0), null); + } + producerCallbacks.reset(); + } + return sendFuture; + } + }); + // 2. Offset data is passed to the offset storage. + offsetWriter.setOffset(PARTITION_BYTES, OFFSET_BYTES); + PowerMock.expectLastCall().anyTimes(); + + return sent; + } + + private void awaitPolls(CountDownLatch latch) throws InterruptedException { + latch.await(1000, TimeUnit.MILLISECONDS); + } + + @SuppressWarnings("unchecked") + private void expectOffsetFlush(boolean succeed) throws Exception { + EasyMock.expect(offsetWriter.beginFlush()).andReturn(true); + Future flushFuture = PowerMock.createMock(Future.class); + EasyMock.expect(offsetWriter.doFlush(EasyMock.anyObject(Callback.class))).andReturn(flushFuture); + // Should throw for failure + IExpectationSetters futureGetExpect = EasyMock.expect( + flushFuture.get(EasyMock.anyLong(), EasyMock.anyObject(TimeUnit.class))); + if (succeed) { + futureGetExpect.andReturn(null); + } else { + futureGetExpect.andThrow(new TimeoutException()); + offsetWriter.cancelFlush(); + PowerMock.expectLastCall(); + } + } + +} diff --git a/copycat/runtime/src/test/java/org/apache/kafka/copycat/runtime/WorkerTest.java b/copycat/runtime/src/test/java/org/apache/kafka/copycat/runtime/WorkerTest.java new file mode 100644 index 0000000..32e7ff9 --- /dev/null +++ b/copycat/runtime/src/test/java/org/apache/kafka/copycat/runtime/WorkerTest.java @@ -0,0 +1,179 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.runtime; + +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.copycat.cli.WorkerConfig; +import org.apache.kafka.copycat.errors.CopycatException; +import org.apache.kafka.copycat.source.SourceRecord; +import org.apache.kafka.copycat.source.SourceTask; +import org.apache.kafka.copycat.storage.*; +import org.apache.kafka.copycat.util.ConnectorTaskId; +import org.apache.kafka.copycat.util.MockTime; +import org.apache.kafka.copycat.util.ThreadedTest; +import org.easymock.EasyMock; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.easymock.PowerMock; +import org.powermock.core.classloader.annotations.PowerMockIgnore; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.util.List; +import java.util.Properties; + +@RunWith(PowerMockRunner.class) +@PrepareForTest(Worker.class) +@PowerMockIgnore("javax.management.*") +public class WorkerTest extends ThreadedTest { + + private ConnectorTaskId taskId = new ConnectorTaskId("job", 0); + private Worker worker; + private OffsetBackingStore offsetBackingStore = PowerMock.createMock(OffsetBackingStore.class); + private Serializer offsetKeySerializer = PowerMock.createMock(Serializer.class); + private Serializer offsetValueSerializer = PowerMock.createMock(Serializer.class); + private Deserializer offsetKeyDeserializer = PowerMock.createMock(Deserializer.class); + private Deserializer offsetValueDeserializer = PowerMock.createMock(Deserializer.class); + + @Before + public void setup() { + super.setup(); + + Properties workerProps = new Properties(); + workerProps.setProperty("key.converter", "org.apache.kafka.copycat.json.JsonConverter"); + workerProps.setProperty("value.converter", "org.apache.kafka.copycat.json.JsonConverter"); + workerProps.setProperty("key.serializer", "org.apache.kafka.copycat.json.JsonSerializer"); + workerProps.setProperty("value.serializer", "org.apache.kafka.copycat.json.JsonSerializer"); + workerProps.setProperty("key.deserializer", "org.apache.kafka.copycat.json.JsonDeserializer"); + workerProps.setProperty("value.deserializer", "org.apache.kafka.copycat.json.JsonDeserializer"); + WorkerConfig config = new WorkerConfig(workerProps); + worker = new Worker(new MockTime(), config, offsetBackingStore, + offsetKeySerializer, offsetValueSerializer, + offsetKeyDeserializer, offsetValueDeserializer); + worker.start(); + } + + @Test + public void testAddRemoveTask() throws Exception { + ConnectorTaskId taskId = new ConnectorTaskId("job", 0); + + // Create + TestSourceTask task = PowerMock.createMock(TestSourceTask.class); + WorkerSourceTask workerTask = PowerMock.createMock(WorkerSourceTask.class); + + PowerMock.mockStatic(Worker.class); + PowerMock.expectPrivate(Worker.class, "instantiateTask", TestSourceTask.class.getName()).andReturn(task); + + PowerMock.expectNew( + WorkerSourceTask.class, EasyMock.eq(taskId), EasyMock.eq(task), + EasyMock.anyObject(Converter.class), + EasyMock.anyObject(Converter.class), + EasyMock.anyObject(KafkaProducer.class), + EasyMock.anyObject(OffsetStorageReader.class), + EasyMock.anyObject(OffsetStorageWriter.class), + EasyMock.anyObject(WorkerConfig.class), + EasyMock.anyObject(Time.class)) + .andReturn(workerTask); + Properties origProps = new Properties(); + workerTask.start(origProps); + EasyMock.expectLastCall(); + + // Remove + workerTask.stop(); + EasyMock.expectLastCall(); + EasyMock.expect(workerTask.awaitStop(EasyMock.anyLong())).andStubReturn(true); + workerTask.close(); + EasyMock.expectLastCall(); + + PowerMock.replayAll(); + + worker.addTask(taskId, TestSourceTask.class.getName(), origProps); + worker.stopTask(taskId); + // Nothing should be left, so this should effectively be a nop + worker.stop(); + + PowerMock.verifyAll(); + } + + @Test(expected = CopycatException.class) + public void testStopInvalidTask() { + worker.stopTask(taskId); + } + + @Test + public void testCleanupTasksOnStop() throws Exception { + // Create + TestSourceTask task = PowerMock.createMock(TestSourceTask.class); + WorkerSourceTask workerTask = PowerMock.createMock(WorkerSourceTask.class); + + PowerMock.mockStatic(Worker.class); + PowerMock.expectPrivate(Worker.class, "instantiateTask", TestSourceTask.class.getName()).andReturn(task); + + PowerMock.expectNew( + WorkerSourceTask.class, EasyMock.eq(taskId), EasyMock.eq(task), + EasyMock.anyObject(Converter.class), + EasyMock.anyObject(Converter.class), + EasyMock.anyObject(KafkaProducer.class), + EasyMock.anyObject(OffsetStorageReader.class), + EasyMock.anyObject(OffsetStorageWriter.class), + EasyMock.anyObject(WorkerConfig.class), + EasyMock.anyObject(Time.class)) + .andReturn(workerTask); + Properties origProps = new Properties(); + workerTask.start(origProps); + EasyMock.expectLastCall(); + + // Remove on Worker.stop() + workerTask.stop(); + EasyMock.expectLastCall(); + EasyMock.expect(workerTask.awaitStop(EasyMock.anyLong())).andReturn(true); + // Note that in this case we *do not* commit offsets since it's an unclean shutdown + workerTask.close(); + EasyMock.expectLastCall(); + + PowerMock.replayAll(); + + worker.addTask(taskId, TestSourceTask.class.getName(), origProps); + worker.stop(); + + PowerMock.verifyAll(); + } + + + private static class TestSourceTask extends SourceTask { + public TestSourceTask() { + } + + @Override + public void start(Properties props) { + } + + @Override + public List poll() throws InterruptedException { + return null; + } + + @Override + public void stop() { + } + } +} diff --git a/copycat/runtime/src/test/java/org/apache/kafka/copycat/runtime/standalone/StandaloneHerderTest.java b/copycat/runtime/src/test/java/org/apache/kafka/copycat/runtime/standalone/StandaloneHerderTest.java new file mode 100644 index 0000000..5ac7e38 --- /dev/null +++ b/copycat/runtime/src/test/java/org/apache/kafka/copycat/runtime/standalone/StandaloneHerderTest.java @@ -0,0 +1,186 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.runtime.standalone; + +import org.apache.kafka.copycat.connector.Connector; +import org.apache.kafka.copycat.connector.Task; +import org.apache.kafka.copycat.runtime.ConnectorConfig; +import org.apache.kafka.copycat.runtime.Worker; +import org.apache.kafka.copycat.sink.SinkConnector; +import org.apache.kafka.copycat.sink.SinkTask; +import org.apache.kafka.copycat.source.SourceConnector; +import org.apache.kafka.copycat.source.SourceTask; +import org.apache.kafka.copycat.util.Callback; +import org.apache.kafka.copycat.util.ConnectorTaskId; +import org.apache.kafka.copycat.util.FutureCallback; +import org.easymock.EasyMock; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.easymock.PowerMock; +import org.powermock.api.easymock.annotation.Mock; +import org.powermock.core.classloader.annotations.PowerMockIgnore; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.util.Arrays; +import java.util.Properties; +import java.util.concurrent.TimeUnit; + +@RunWith(PowerMockRunner.class) +@PrepareForTest({StandaloneHerder.class}) +@PowerMockIgnore("javax.management.*") +public class StandaloneHerderTest { + private static final String CONNECTOR_NAME = "test"; + private static final String TOPICS_LIST_STR = "topic1,topic2"; + + private StandaloneHerder herder; + @Mock protected Worker worker; + private Connector connector; + @Mock protected Callback createCallback; + + private Properties connectorProps; + private Properties taskProps; + + @Before + public void setup() { + worker = PowerMock.createMock(Worker.class); + herder = new StandaloneHerder(worker); + + connectorProps = new Properties(); + connectorProps.setProperty(ConnectorConfig.NAME_CONFIG, CONNECTOR_NAME); + connectorProps.setProperty(SinkConnector.TOPICS_CONFIG, TOPICS_LIST_STR); + PowerMock.mockStatic(StandaloneHerder.class); + + // These can be anything since connectors can pass along whatever they want. + taskProps = new Properties(); + taskProps.setProperty("foo", "bar"); + } + + @Test + public void testCreateSourceConnector() throws Exception { + connector = PowerMock.createMock(BogusSourceClass.class); + expectAdd(BogusSourceClass.class, BogusSourceTask.class, false); + PowerMock.replayAll(); + + herder.addConnector(connectorProps, createCallback); + + PowerMock.verifyAll(); + } + + @Test + public void testCreateSinkConnector() throws Exception { + connector = PowerMock.createMock(BogusSinkClass.class); + expectAdd(BogusSinkClass.class, BogusSinkTask.class, true); + + PowerMock.replayAll(); + + herder.addConnector(connectorProps, createCallback); + + PowerMock.verifyAll(); + } + + @Test + public void testDestroyConnector() throws Exception { + connector = PowerMock.createMock(BogusSourceClass.class); + expectAdd(BogusSourceClass.class, BogusSourceTask.class, false); + expectDestroy(); + PowerMock.replayAll(); + + herder.addConnector(connectorProps, createCallback); + FutureCallback futureCb = new FutureCallback<>(new Callback() { + @Override + public void onCompletion(Throwable error, Void result) { + + } + }); + herder.deleteConnector(CONNECTOR_NAME, futureCb); + futureCb.get(1000L, TimeUnit.MILLISECONDS); + PowerMock.verifyAll(); + } + + + private void expectAdd(Class connClass, + Class taskClass, + boolean sink) throws Exception { + expectCreate(connClass, taskClass, sink, true); + } + + private void expectRestore(Class connClass, + Class taskClass) throws Exception { + // Restore never uses a callback. These tests always use sources + expectCreate(connClass, taskClass, false, false); + } + + private void expectCreate(Class connClass, + Class taskClass, + boolean sink, boolean expectCallback) throws Exception { + connectorProps.setProperty(ConnectorConfig.CONNECTOR_CLASS_CONFIG, connClass.getName()); + + PowerMock.expectPrivate(StandaloneHerder.class, "instantiateConnector", connClass.getName()) + .andReturn(connector); + if (expectCallback) { + createCallback.onCompletion(null, CONNECTOR_NAME); + PowerMock.expectLastCall(); + } + + connector.initialize(EasyMock.anyObject(StandaloneConnectorContext.class)); + PowerMock.expectLastCall(); + connector.start(new Properties()); + PowerMock.expectLastCall(); + + // Just return the connector properties for the individual task we generate by default + EasyMock.>expect(connector.getTaskClass()).andReturn(taskClass); + + EasyMock.expect(connector.getTaskConfigs(ConnectorConfig.TASKS_MAX_DEFAULT)) + .andReturn(Arrays.asList(taskProps)); + // And we should instantiate the tasks. For a sink task, we should see added properties for + // the input topic partitions + Properties generatedTaskProps = new Properties(); + generatedTaskProps.putAll(taskProps); + if (sink) + generatedTaskProps.setProperty(SinkTask.TOPICS_CONFIG, TOPICS_LIST_STR); + worker.addTask(new ConnectorTaskId(CONNECTOR_NAME, 0), taskClass.getName(), generatedTaskProps); + PowerMock.expectLastCall(); + } + + private void expectStop() { + worker.stopTask(new ConnectorTaskId(CONNECTOR_NAME, 0)); + EasyMock.expectLastCall(); + connector.stop(); + EasyMock.expectLastCall(); + } + + private void expectDestroy() { + expectStop(); + } + + // We need to use a real class here due to some issue with mocking java.lang.Class + private abstract class BogusSourceClass extends SourceConnector { + } + + private abstract class BogusSourceTask extends SourceTask { + } + + private abstract class BogusSinkClass extends SinkConnector { + } + + private abstract class BogusSinkTask extends SourceTask { + } + +} diff --git a/copycat/runtime/src/test/java/org/apache/kafka/copycat/storage/FileOffsetBackingStoreTest.java b/copycat/runtime/src/test/java/org/apache/kafka/copycat/storage/FileOffsetBackingStoreTest.java new file mode 100644 index 0000000..bbcbdc9 --- /dev/null +++ b/copycat/runtime/src/test/java/org/apache/kafka/copycat/storage/FileOffsetBackingStoreTest.java @@ -0,0 +1,117 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.storage; + +import org.apache.kafka.copycat.util.Callback; +import org.easymock.EasyMock; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.powermock.api.easymock.PowerMock; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +public class FileOffsetBackingStoreTest { + + FileOffsetBackingStore store; + Map props; + File tempFile; + + private static Map firstSet = new HashMap<>(); + + static { + firstSet.put(buffer("key"), buffer("value")); + firstSet.put(null, null); + } + + @Before + public void setup() throws IOException { + store = new FileOffsetBackingStore(); + tempFile = File.createTempFile("fileoffsetbackingstore", null); + props = new HashMap<>(); + props.put(FileOffsetBackingStore.OFFSET_STORAGE_FILE_FILENAME_CONFIG, tempFile.getAbsolutePath()); + store.configure(props); + store.start(); + } + + @After + public void teardown() { + tempFile.delete(); + } + + @Test + public void testGetSet() throws Exception { + Callback setCallback = expectSuccessfulSetCallback(); + Callback> getCallback = expectSuccessfulGetCallback(); + PowerMock.replayAll(); + + store.set("namespace", firstSet, setCallback).get(); + + Map values = store.get("namespace", Arrays.asList(buffer("key"), buffer("bad")), getCallback).get(); + assertEquals(buffer("value"), values.get(buffer("key"))); + assertEquals(null, values.get(buffer("bad"))); + + PowerMock.verifyAll(); + } + + @Test + public void testSaveRestore() throws Exception { + Callback setCallback = expectSuccessfulSetCallback(); + Callback> getCallback = expectSuccessfulGetCallback(); + PowerMock.replayAll(); + + store.set("namespace", firstSet, setCallback).get(); + store.stop(); + + // Restore into a new store to ensure correct reload from scratch + FileOffsetBackingStore restore = new FileOffsetBackingStore(); + restore.configure(props); + restore.start(); + Map values = restore.get("namespace", Arrays.asList(buffer("key")), getCallback).get(); + assertEquals(buffer("value"), values.get(buffer("key"))); + + PowerMock.verifyAll(); + } + + private static ByteBuffer buffer(String v) { + return ByteBuffer.wrap(v.getBytes()); + } + + private Callback expectSuccessfulSetCallback() { + @SuppressWarnings("unchecked") + Callback setCallback = PowerMock.createMock(Callback.class); + setCallback.onCompletion(EasyMock.isNull(Throwable.class), EasyMock.isNull(Void.class)); + PowerMock.expectLastCall(); + return setCallback; + } + + @SuppressWarnings("unchecked") + private Callback> expectSuccessfulGetCallback() { + Callback> getCallback = PowerMock.createMock(Callback.class); + getCallback.onCompletion(EasyMock.isNull(Throwable.class), EasyMock.anyObject(Map.class)); + PowerMock.expectLastCall(); + return getCallback; + } +} diff --git a/copycat/runtime/src/test/java/org/apache/kafka/copycat/storage/OffsetStorageWriterTest.java b/copycat/runtime/src/test/java/org/apache/kafka/copycat/storage/OffsetStorageWriterTest.java new file mode 100644 index 0000000..3d49f05 --- /dev/null +++ b/copycat/runtime/src/test/java/org/apache/kafka/copycat/storage/OffsetStorageWriterTest.java @@ -0,0 +1,242 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.storage; + +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.copycat.errors.CopycatException; +import org.apache.kafka.copycat.util.Callback; +import org.easymock.Capture; +import org.easymock.EasyMock; +import org.easymock.IAnswer; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.easymock.PowerMock; +import org.powermock.api.easymock.annotation.Mock; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.*; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +@RunWith(PowerMockRunner.class) +public class OffsetStorageWriterTest { + private static final String NAMESPACE = "namespace"; + // Copycat format - any types should be accepted here + private static final List OFFSET_KEY = Arrays.asList("key", "key"); + private static final String OFFSET_VALUE = "value"; + // Native objects - must match serializer types + private static final int OFFSET_KEY_CONVERTED = 12; + private static final String OFFSET_VALUE_CONVERTED = "value-converted"; + // Serialized + private static final byte[] OFFSET_KEY_SERIALIZED = "key-serialized".getBytes(); + private static final byte[] OFFSET_VALUE_SERIALIZED = "value-serialized".getBytes(); + private static final Map OFFSETS_SERIALIZED + = Collections.singletonMap(ByteBuffer.wrap(OFFSET_KEY_SERIALIZED), + ByteBuffer.wrap(OFFSET_VALUE_SERIALIZED)); + + @Mock private OffsetBackingStore store; + @Mock private Converter keyConverter; + @Mock private Converter valueConverter; + @Mock private Serializer keySerializer; + @Mock private Serializer valueSerializer; + private OffsetStorageWriter writer; + + private static Exception exception = new RuntimeException("error"); + + private ExecutorService service; + + @Before + public void setup() { + writer = new OffsetStorageWriter<>(store, NAMESPACE, keyConverter, valueConverter, keySerializer, valueSerializer); + service = Executors.newFixedThreadPool(1); + } + + @After + public void teardown() { + service.shutdownNow(); + } + + @Test + public void testWriteFlush() throws Exception { + @SuppressWarnings("unchecked") + Callback callback = PowerMock.createMock(Callback.class); + expectStore(callback, false); + + PowerMock.replayAll(); + + writer.setOffset(OFFSET_KEY, OFFSET_VALUE); + + assertTrue(writer.beginFlush()); + writer.doFlush(callback).get(1000, TimeUnit.MILLISECONDS); + + PowerMock.verifyAll(); + } + + @Test + public void testNoOffsetsToFlush() { + // If no offsets are flushed, we should finish immediately and not have made any calls to the + // underlying storage layer + + PowerMock.replayAll(); + + // Should not return a future + assertFalse(writer.beginFlush()); + + PowerMock.verifyAll(); + } + + @Test + public void testFlushFailureReplacesOffsets() throws Exception { + // When a flush fails, we shouldn't just lose the offsets. Instead, they should be restored + // such that a subsequent flush will write them. + + @SuppressWarnings("unchecked") + final Callback callback = PowerMock.createMock(Callback.class); + // First time the write fails + expectStore(callback, true); + // Second time it succeeds + expectStore(callback, false); + // Third time it has no data to flush so we won't get past beginFlush() + + PowerMock.replayAll(); + + writer.setOffset(OFFSET_KEY, OFFSET_VALUE); + assertTrue(writer.beginFlush()); + writer.doFlush(callback).get(1000, TimeUnit.MILLISECONDS); + assertTrue(writer.beginFlush()); + writer.doFlush(callback).get(1000, TimeUnit.MILLISECONDS); + assertFalse(writer.beginFlush()); + + PowerMock.verifyAll(); + } + + @Test(expected = CopycatException.class) + public void testAlreadyFlushing() throws Exception { + @SuppressWarnings("unchecked") + final Callback callback = PowerMock.createMock(Callback.class); + // Trigger the send, but don't invoke the callback so we'll still be mid-flush + CountDownLatch allowStoreCompleteCountdown = new CountDownLatch(1); + expectStore(null, false, allowStoreCompleteCountdown); + + PowerMock.replayAll(); + + writer.setOffset(OFFSET_KEY, OFFSET_VALUE); + assertTrue(writer.beginFlush()); + writer.doFlush(callback); + assertTrue(writer.beginFlush()); // should throw + + PowerMock.verifyAll(); + } + + @Test + public void testCancelBeforeAwaitFlush() { + PowerMock.replayAll(); + + writer.setOffset(OFFSET_KEY, OFFSET_VALUE); + assertTrue(writer.beginFlush()); + writer.cancelFlush(); + + PowerMock.verifyAll(); + } + + @Test + public void testCancelAfterAwaitFlush() throws Exception { + @SuppressWarnings("unchecked") + Callback callback = PowerMock.createMock(Callback.class); + CountDownLatch allowStoreCompleteCountdown = new CountDownLatch(1); + // In this test, the write should be cancelled so the callback will not be invoked and is not + // passed to the expectStore call + expectStore(null, false, allowStoreCompleteCountdown); + + PowerMock.replayAll(); + + writer.setOffset(OFFSET_KEY, OFFSET_VALUE); + assertTrue(writer.beginFlush()); + // Start the flush, then immediately cancel before allowing the mocked store request to finish + Future flushFuture = writer.doFlush(callback); + writer.cancelFlush(); + allowStoreCompleteCountdown.countDown(); + flushFuture.get(1000, TimeUnit.MILLISECONDS); + + PowerMock.verifyAll(); + } + + private void expectStore(final Callback callback, final boolean fail) { + expectStore(callback, fail, null); + } + + /** + * Expect a request to store data to the underlying OffsetBackingStore. + * + * @param callback the callback to invoke when completed, or null if the callback isn't + * expected to be invoked + * @param fail if true, treat + * @param waitForCompletion if non-null, a CountDownLatch that should be awaited on before + * invoking the callback. A (generous) timeout is still imposed to + * ensure tests complete. + * @return the captured set of ByteBuffer key-value pairs passed to the storage layer + */ + private void expectStore(final Callback callback, + final boolean fail, + final CountDownLatch waitForCompletion) { + EasyMock.expect(keyConverter.fromCopycatData(OFFSET_KEY)).andReturn(OFFSET_KEY_CONVERTED); + EasyMock.expect(keySerializer.serialize(NAMESPACE, OFFSET_KEY_CONVERTED)).andReturn(OFFSET_KEY_SERIALIZED); + EasyMock.expect(valueConverter.fromCopycatData(OFFSET_VALUE)).andReturn(OFFSET_VALUE_CONVERTED); + EasyMock.expect(valueSerializer.serialize(NAMESPACE, OFFSET_VALUE_CONVERTED)).andReturn(OFFSET_VALUE_SERIALIZED); + + final Capture> storeCallback = Capture.newInstance(); + EasyMock.expect(store.set(EasyMock.eq(NAMESPACE), EasyMock.eq(OFFSETS_SERIALIZED), + EasyMock.capture(storeCallback))) + .andAnswer(new IAnswer>() { + @Override + public Future answer() throws Throwable { + return service.submit(new Callable() { + @Override + public Void call() throws Exception { + if (waitForCompletion != null) + assertTrue(waitForCompletion.await(10000, TimeUnit.MILLISECONDS)); + + if (fail) { + storeCallback.getValue().onCompletion(exception, null); + } else { + storeCallback.getValue().onCompletion(null, null); + } + return null; + } + }); + } + }); + if (callback != null) { + if (fail) { + callback.onCompletion(EasyMock.eq(exception), EasyMock.eq((Void) null)); + } else { + callback.onCompletion(null, null); + } + } + PowerMock.expectLastCall(); + } + +} diff --git a/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/MockTime.java b/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/MockTime.java new file mode 100644 index 0000000..53149db --- /dev/null +++ b/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/MockTime.java @@ -0,0 +1,49 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ +package org.apache.kafka.copycat.util; + +import org.apache.kafka.common.utils.Time; + +import java.util.concurrent.TimeUnit; + +/** + * A clock that you can manually advance by calling sleep + */ +public class MockTime implements Time { + + private long nanos = 0; + + public MockTime() { + this.nanos = System.nanoTime(); + } + + @Override + public long milliseconds() { + return TimeUnit.MILLISECONDS.convert(this.nanos, TimeUnit.NANOSECONDS); + } + + @Override + public long nanoseconds() { + return nanos; + } + + @Override + public void sleep(long ms) { + this.nanos += TimeUnit.NANOSECONDS.convert(ms, TimeUnit.MILLISECONDS); + } + +} diff --git a/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/ShutdownableThreadTest.java b/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/ShutdownableThreadTest.java new file mode 100644 index 0000000..4880ca1 --- /dev/null +++ b/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/ShutdownableThreadTest.java @@ -0,0 +1,72 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.util; + +import org.junit.Test; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class ShutdownableThreadTest { + + @Test + public void testGracefulShutdown() throws InterruptedException { + ShutdownableThread thread = new ShutdownableThread("graceful") { + @Override + public void execute() { + while (getRunning()) { + try { + Thread.sleep(1); + } catch (InterruptedException e) { + // Ignore + } + } + } + }; + thread.start(); + Thread.sleep(10); + assertTrue(thread.gracefulShutdown(1000, TimeUnit.MILLISECONDS)); + } + + @Test + public void testForcibleShutdown() throws InterruptedException { + final CountDownLatch startedLatch = new CountDownLatch(1); + ShutdownableThread thread = new ShutdownableThread("forcible") { + @Override + public void execute() { + try { + startedLatch.countDown(); + Thread.sleep(100000); + } catch (InterruptedException e) { + // Ignore + } + } + }; + thread.start(); + startedLatch.await(); + thread.forceShutdown(); + // Not all threads can be forcibly stopped since interrupt() doesn't work on threads in + // certain conditions, but in this case we know the thread is interruptible so we should be + // able join() it + thread.join(1000); + assertFalse(thread.isAlive()); + } +} diff --git a/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/TestBackgroundThreadExceptionHandler.java b/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/TestBackgroundThreadExceptionHandler.java new file mode 100644 index 0000000..5dc6d33 --- /dev/null +++ b/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/TestBackgroundThreadExceptionHandler.java @@ -0,0 +1,37 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.util; + +/** + * An UncaughtExceptionHandler that can be registered with one or more threads which tracks the + * first exception so the main thread can check for uncaught exceptions. + */ +public class TestBackgroundThreadExceptionHandler implements Thread.UncaughtExceptionHandler { + private Throwable firstException = null; + + @Override + public void uncaughtException(Thread t, Throwable e) { + if (this.firstException == null) + this.firstException = e; + } + + public void verifyNoExceptions() { + if (this.firstException != null) + throw new AssertionError(this.firstException); + } +} diff --git a/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/ThreadedTest.java b/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/ThreadedTest.java new file mode 100644 index 0000000..ed99247 --- /dev/null +++ b/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/ThreadedTest.java @@ -0,0 +1,43 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.copycat.util; + +import org.junit.After; +import org.junit.Before; + +/** + * Base class for tests that use threads. It sets up uncaught exception handlers for all known + * thread classes and checks for errors at the end of the test so that failures in background + * threads will cause the test to fail. + */ +public class ThreadedTest { + + protected TestBackgroundThreadExceptionHandler backgroundThreadExceptionHandler; + + @Before + public void setup() { + backgroundThreadExceptionHandler = new TestBackgroundThreadExceptionHandler(); + ShutdownableThread.funcaughtExceptionHandler = backgroundThreadExceptionHandler; + } + + @After + public void teardown() { + backgroundThreadExceptionHandler.verifyNoExceptions(); + ShutdownableThread.funcaughtExceptionHandler = null; + } +} diff --git a/settings.gradle b/settings.gradle index 1944917..27ae98f 100644 --- a/settings.gradle +++ b/settings.gradle @@ -14,5 +14,5 @@ // limitations under the License. apply from: file('scala.gradle') -include 'core', 'contrib:hadoop-consumer', 'contrib:hadoop-producer', 'examples', 'clients', 'tools', 'log4j-appender' - +include 'core', 'contrib:hadoop-consumer', 'contrib:hadoop-producer', 'examples', 'clients', 'tools', 'log4j-appender', + 'copycat:data', 'copycat:api', 'copycat:runtime', 'copycat:json', 'copycat:file' -- 1.7.12.4 From bbb7d97adefe5826f2e02a8e55423ea215c9f749 Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Fri, 14 Aug 2015 18:51:48 -0700 Subject: [PATCH 111/120] KAFKA-2084; Add per-client-id byte-rate metrics and quota manager; reviewed by Joel Koshy, Dong Lin, Jun Rao and Edward Ribeiro --- build.gradle | 1 + .../org/apache/kafka/common/metrics/Quota.java | 18 ++ .../common/metrics/QuotaViolationException.java | 1 - .../org/apache/kafka/common/metrics/Sensor.java | 11 +- .../apache/kafka/common/metrics/stats/Rate.java | 23 +- .../apache/kafka/common/metrics/MetricsTest.java | 32 ++- .../scala/kafka/server/ClientQuotaManager.scala | 250 +++++++++++++++++++++ core/src/main/scala/kafka/server/KafkaApis.scala | 88 ++++++-- core/src/main/scala/kafka/server/KafkaConfig.scala | 56 ++++- core/src/main/scala/kafka/server/KafkaServer.scala | 24 +- .../main/scala/kafka/server/ReplicaManager.scala | 4 +- .../scala/kafka/server/ThrottledResponse.scala | 46 ++++ .../scala/kafka/utils/ShutdownableThread.scala | 3 + .../scala/integration/kafka/api/QuotasTest.scala | 194 ++++++++++++++++ .../unit/kafka/server/ClientQuotaManagerTest.scala | 159 +++++++++++++ .../scala/unit/kafka/server/KafkaConfigTest.scala | 6 + .../server/ThrottledResponseExpirationTest.scala | 90 ++++++++ 17 files changed, 945 insertions(+), 61 deletions(-) create mode 100644 core/src/main/scala/kafka/server/ClientQuotaManager.scala create mode 100644 core/src/main/scala/kafka/server/ThrottledResponse.scala create mode 100644 core/src/test/scala/integration/kafka/api/QuotasTest.scala create mode 100644 core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala create mode 100644 core/src/test/scala/unit/kafka/server/ThrottledResponseExpirationTest.scala diff --git a/build.gradle b/build.gradle index 864427b..c7f66be 100644 --- a/build.gradle +++ b/build.gradle @@ -253,6 +253,7 @@ project(':core') { testCompile "$easymock" testCompile 'org.objenesis:objenesis:1.2' testCompile "org.scalatest:scalatest_$baseScalaVersion:2.2.5" + testCompile project(path: ':clients', configuration: 'archives') testRuntime "$slf4jlog4j" diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/Quota.java b/clients/src/main/java/org/apache/kafka/common/metrics/Quota.java index d82bb0c..a3535dc 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/Quota.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/Quota.java @@ -49,4 +49,22 @@ public final class Quota { return (upper && value <= bound) || (!upper && value >= bound); } + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + (int) this.bound; + result = prime * result + (this.upper ? 1 : 0); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (!(obj instanceof Quota)) + return false; + Quota that = (Quota) obj; + return (that.bound == this.bound) && (this.upper == this.upper); + } } diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/QuotaViolationException.java b/clients/src/main/java/org/apache/kafka/common/metrics/QuotaViolationException.java index a451e53..fbe03f5 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/QuotaViolationException.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/QuotaViolationException.java @@ -28,5 +28,4 @@ public class QuotaViolationException extends KafkaException { public QuotaViolationException(String m) { super(m); } - } diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java b/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java index ca823fd..4d55771 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java @@ -112,8 +112,14 @@ public final class Sensor { if (config != null) { Quota quota = config.quota(); if (quota != null) { - if (!quota.acceptable(metric.value(timeMs))) - throw new QuotaViolationException(metric.metricName() + " is in violation of its quota of " + quota.bound()); + double value = metric.value(timeMs); + if (!quota.acceptable(value)) { + throw new QuotaViolationException(String.format( + "(%s) violated quota. Actual: (%f), Threshold: (%f)", + metric.metricName(), + quota.bound(), + value)); + } } } } @@ -170,5 +176,4 @@ public final class Sensor { synchronized List metrics() { return Collections.unmodifiableList(this.metrics); } - } diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java index 98429da..fe43940 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java @@ -18,6 +18,7 @@ import java.util.concurrent.TimeUnit; import org.apache.kafka.common.metrics.MeasurableStat; import org.apache.kafka.common.metrics.MetricConfig; + /** * The rate of the given quantity. By default this is the total observed over a set of samples from a sampled statistic * divided by the elapsed time over the sample windows. Alternative {@link SampledStat} implementations can be provided, @@ -58,26 +59,28 @@ public class Rate implements MeasurableStat { @Override public double measure(MetricConfig config, long now) { double value = stat.measure(config, now); - double elapsed = convert(now - stat.oldest(now).lastWindowMs); - return value / elapsed; + // the elapsed time is always N-1 complete windows plus whatever fraction of the final window is complete + long elapsedCurrentWindowMs = now - stat.current(now).lastWindowMs; + long elapsedPriorWindowsMs = config.timeWindowMs() * (config.samples() - 1); + return value / convert(elapsedCurrentWindowMs + elapsedPriorWindowsMs); } - private double convert(long time) { + private double convert(long timeMs) { switch (unit) { case NANOSECONDS: - return time * 1000.0 * 1000.0; + return timeMs * 1000.0 * 1000.0; case MICROSECONDS: - return time * 1000.0; + return timeMs * 1000.0; case MILLISECONDS: - return time; + return timeMs; case SECONDS: - return time / 1000.0; + return timeMs / 1000.0; case MINUTES: - return time / (60.0 * 1000.0); + return timeMs / (60.0 * 1000.0); case HOURS: - return time / (60.0 * 60.0 * 1000.0); + return timeMs / (60.0 * 60.0 * 1000.0); case DAYS: - return time / (24.0 * 60.0 * 60.0 * 1000.0); + return timeMs / (24.0 * 60.0 * 60.0 * 1000.0); default: throw new IllegalStateException("Unknown unit: " + unit); } diff --git a/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java b/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java index 544e120..0a7dcd8 100644 --- a/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java @@ -37,9 +37,9 @@ import org.junit.Test; public class MetricsTest { private static final double EPS = 0.000001; - - MockTime time = new MockTime(); - Metrics metrics = new Metrics(new MetricConfig(), Arrays.asList((MetricsReporter) new JmxReporter()), time); + private MockTime time = new MockTime(); + private MetricConfig config = new MetricConfig(); + private Metrics metrics = new Metrics(config, Arrays.asList((MetricsReporter) new JmxReporter()), time); @Test public void testMetricName() { @@ -77,19 +77,33 @@ public class MetricsTest { s2.add(new MetricName("s2.total", "grp1"), new Total()); s2.record(5.0); - for (int i = 0; i < 10; i++) + int sum = 0; + int count = 10; + for (int i = 0; i < count; i++) { s.record(i); + sum += i; + } + // prior to any time passing + double elapsedSecs = (config.timeWindowMs() * (config.samples() - 1)) / 1000.0; + assertEquals(String.format("Occurrences(0...%d) = %f", count, count / elapsedSecs), count / elapsedSecs, + metrics.metrics().get(new MetricName("test.occurences", "grp1")).value(), EPS); // pretend 2 seconds passed... - time.sleep(2000); + long sleepTimeMs = 2; + time.sleep(sleepTimeMs * 1000); + elapsedSecs += sleepTimeMs; assertEquals("s2 reflects the constant value", 5.0, metrics.metrics().get(new MetricName("s2.total", "grp1")).value(), EPS); assertEquals("Avg(0...9) = 4.5", 4.5, metrics.metrics().get(new MetricName("test.avg", "grp1")).value(), EPS); - assertEquals("Max(0...9) = 9", 9.0, metrics.metrics().get(new MetricName("test.max", "grp1")).value(), EPS); + assertEquals("Max(0...9) = 9", count - 1, metrics.metrics().get(new MetricName("test.max", "grp1")).value(), EPS); assertEquals("Min(0...9) = 0", 0.0, metrics.metrics().get(new MetricName("test.min", "grp1")).value(), EPS); - assertEquals("Rate(0...9) = 22.5", 22.5, metrics.metrics().get(new MetricName("test.rate", "grp1")).value(), EPS); - assertEquals("Occurences(0...9) = 5", 5.0, metrics.metrics().get(new MetricName("test.occurences", "grp1")).value(), EPS); - assertEquals("Count(0...9) = 10", 10.0, metrics.metrics().get(new MetricName("test.count", "grp1")).value(), EPS); + assertEquals("Rate(0...9) = 1.40625", + sum / elapsedSecs, metrics.metrics().get(new MetricName("test.rate", "grp1")).value(), EPS); + assertEquals(String.format("Occurrences(0...%d) = %f", count, count / elapsedSecs), + count / elapsedSecs, + metrics.metrics().get(new MetricName("test.occurences", "grp1")).value(), EPS); + assertEquals("Count(0...9) = 10", + (double) count, metrics.metrics().get(new MetricName("test.count", "grp1")).value(), EPS); } @Test diff --git a/core/src/main/scala/kafka/server/ClientQuotaManager.scala b/core/src/main/scala/kafka/server/ClientQuotaManager.scala new file mode 100644 index 0000000..9f8473f --- /dev/null +++ b/core/src/main/scala/kafka/server/ClientQuotaManager.scala @@ -0,0 +1,250 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package kafka.server + +import java.util.concurrent.{DelayQueue, TimeUnit} + +import kafka.utils.{ShutdownableThread, Logging} +import org.apache.kafka.common.MetricName +import org.apache.kafka.common.metrics._ +import org.apache.kafka.common.metrics.stats.{Total, Rate, Avg} +import java.util.concurrent.locks.ReentrantReadWriteLock + +import org.apache.kafka.common.utils.Time + +/** + * Represents the sensors aggregated per client + * @param quotaSensor @Sensor that tracks the quota + * @param throttleTimeSensor @Sensor that tracks the throttle time + */ +private case class ClientSensors(quotaSensor: Sensor, throttleTimeSensor: Sensor) + +/** + * Configuration settings for quota management + * @param quotaBytesPerSecondDefault The default bytes per second quota allocated to any client + * @param quotaBytesPerSecondOverrides The comma separated overrides per client. "c1=X,c2=Y" + * @param numQuotaSamples The number of samples to retain in memory + * @param quotaWindowSizeSeconds The time span of each sample + * + */ +case class ClientQuotaManagerConfig(quotaBytesPerSecondDefault: Long = + ClientQuotaManagerConfig.QuotaBytesPerSecondDefault, + quotaBytesPerSecondOverrides: String = + ClientQuotaManagerConfig.QuotaBytesPerSecondOverrides, + numQuotaSamples: Int = + ClientQuotaManagerConfig.DefaultNumQuotaSamples, + quotaWindowSizeSeconds: Int = + ClientQuotaManagerConfig.DefaultQuotaWindowSizeSeconds) + +object ClientQuotaManagerConfig { + val QuotaBytesPerSecondDefault = Long.MaxValue + val QuotaBytesPerSecondOverrides = "" + // Always have 10 whole windows + 1 current window + val DefaultNumQuotaSamples = 11 + val DefaultQuotaWindowSizeSeconds = 1 + val MaxThrottleTimeSeconds = 30 +} + +/** + * Helper class that records per-client metrics. It is also responsible for maintaining Quota usage statistics + * for all clients. + * @param config @ClientQuotaManagerConfig quota configs + * @param metrics @Metrics Metrics instance + * @param apiKey API Key for the request + * @param time @Time object to use + */ +class ClientQuotaManager(private val config: ClientQuotaManagerConfig, + private val metrics: Metrics, + private val apiKey: String, + private val time: Time) extends Logging { + private val overriddenQuota = initQuotaMap(config.quotaBytesPerSecondOverrides) + private val defaultQuota = Quota.lessThan(config.quotaBytesPerSecondDefault) + private val lock = new ReentrantReadWriteLock() + private val delayQueue = new DelayQueue[ThrottledResponse]() + val throttledRequestReaper = new ThrottledRequestReaper(delayQueue) + throttledRequestReaper.start() + + private val delayQueueSensor = metrics.sensor(apiKey + "-delayQueue") + delayQueueSensor.add(new MetricName("queue-size", + apiKey, + "Tracks the size of the delay queue"), new Total()) + + /** + * Reaper thread that triggers callbacks on all throttled requests + * @param delayQueue DelayQueue to dequeue from + */ + class ThrottledRequestReaper(delayQueue: DelayQueue[ThrottledResponse]) extends ShutdownableThread( + "ThrottledRequestReaper-%s".format(apiKey), false) { + + override def doWork(): Unit = { + val response: ThrottledResponse = delayQueue.poll(1, TimeUnit.SECONDS) + if (response != null) { + // Decrement the size of the delay queue + delayQueueSensor.record(-1) + trace("Response throttled for: " + response.delayTimeMs + " ms") + response.execute() + } + } + } + + /** + * Records that a clientId changed some metric being throttled (produced/consumed bytes, QPS etc.) + * @param clientId clientId that produced the data + * @param value amount of data written in bytes + * @param callback Callback function. This will be triggered immediately if quota is not violated. + * If there is a quota violation, this callback will be triggered after a delay + * @return Number of milliseconds to delay the response in case of Quota violation. + * Zero otherwise + */ + def recordAndMaybeThrottle(clientId: String, value: Int, callback: => Unit): Int = { + val clientSensors = getOrCreateQuotaSensors(clientId) + var delayTimeMs = 0L + try { + clientSensors.quotaSensor.record(value) + // trigger the callback immediately if quota is not violated + callback + } catch { + case qve: QuotaViolationException => + // Compute the delay + val clientMetric = metrics.metrics().get(clientRateMetricName(clientId)) + delayTimeMs = delayTime(clientMetric.value(), getQuotaMetricConfig(quota(clientId))) + delayQueue.add(new ThrottledResponse(time, delayTimeMs, callback)) + delayQueueSensor.record() + clientSensors.throttleTimeSensor.record(delayTimeMs) + // If delayed, add the element to the delayQueue + logger.debug("Quota violated for sensor (%s). Delay time: (%d)".format(clientSensors.quotaSensor.name(), delayTimeMs)) + } + delayTimeMs.toInt + } + + /* + * This calculates the amount of time needed to bring the metric within quota + * assuming that no new metrics are recorded. + * + * Basically, if O is the observed rate and T is the target rate over a window of W, to bring O down to T, + * we need to add a delay of X to W such that O * W / (W + X) = T. + * Solving for X, we get X = (O - T)/T * W. + */ + private def delayTime(metricValue: Double, config: MetricConfig): Long = + { + val quota = config.quota() + val difference = metricValue - quota.bound + val time = difference / quota.bound * config.timeWindowMs() * config.samples() + time.round + } + + /** + * Returns the consumer quota for the specified clientId + * @return + */ + private[server] def quota(clientId: String): Quota = overriddenQuota.getOrElse(clientId, defaultQuota) + + /* + * This function either returns the sensors for a given client id or creates them if they don't exist + * First sensor of the tuple is the quota enforcement sensor. Second one is the throttle time sensor + */ + private def getOrCreateQuotaSensors(clientId: String): ClientSensors = { + + // Names of the sensors to access + val quotaSensorName = apiKey + "-" + clientId + val throttleTimeSensorName = apiKey + "ThrottleTime-" + clientId + var quotaSensor: Sensor = null + var throttleTimeSensor: Sensor = null + + /* Acquire the read lock to fetch the sensors. It is safe to call getSensor from multiple threads. + * The read lock allows a thread to create a sensor in isolation. The thread creating the sensor + * will acquire the write lock and prevent the sensors from being read while they are being created. + * It should be sufficient to simply check if the sensor is null without acquiring a read lock but the + * sensor being present doesn't mean that it is fully initialized i.e. all the Metrics may not have been added. + * This read lock waits until the writer thread has released it's lock i.e. fully initialized the sensor + * at which point it is safe to read + */ + lock.readLock().lock() + try { + quotaSensor = metrics.getSensor(quotaSensorName) + throttleTimeSensor = metrics.getSensor(throttleTimeSensorName) + } + finally { + lock.readLock().unlock() + } + + /* If the sensor is null, try to create it else return the created sensor + * Also if quota sensor is null, the throttle time sensor must be null + */ + if (quotaSensor == null) { + /* Acquire a write lock because the sensor may not have been created and we only want one thread to create it. + * Note that multiple threads may acquire the write lock if they all see a null sensor initially + * In this case, the writer checks the sensor after acquiring the lock again. + * This is safe from Double Checked Locking because the references are read + * after acquiring read locks and hence they cannot see a partially published reference + */ + lock.writeLock().lock() + try { + quotaSensor = metrics.getSensor(quotaSensorName) + if (quotaSensor == null) { + // create the throttle time sensor also + throttleTimeSensor = metrics.sensor(throttleTimeSensorName) + throttleTimeSensor.add(new MetricName("throttle-time", + apiKey, + "Tracking average throttle-time per client", + "client-id", + clientId), new Avg()) + quotaSensor = metrics.sensor(quotaSensorName, getQuotaMetricConfig(quota(clientId))) + quotaSensor.add(clientRateMetricName(clientId), new Rate()) + } + } finally { + lock.writeLock().unlock() + } + } + // return the read or created sensors + ClientSensors(quotaSensor, throttleTimeSensor) + } + + private def getQuotaMetricConfig(quota: Quota): MetricConfig = { + new MetricConfig() + .timeWindow(config.quotaWindowSizeSeconds, TimeUnit.SECONDS) + .samples(config.numQuotaSamples) + .quota(quota) + } + + /* Construct a Map of (clientId -> Quota) + * The input config is specified as a comma-separated K=V pairs + */ + private def initQuotaMap(input: String): Map[String, Quota] = { + // If empty input, return an empty map + if (input.trim.length == 0) + Map[String, Quota]() + else + input.split(",").map(entry => { + val trimmedEntry = entry.trim + val pair: Array[String] = trimmedEntry.split("=") + if (pair.length != 2) + throw new IllegalArgumentException("Incorrectly formatted override entry (%s). Format is k1=v1,k2=v2".format(entry)) + pair(0) -> new Quota(pair(1).toDouble, true) + }).toMap + } + + private def clientRateMetricName(clientId: String): MetricName = { + new MetricName("byte-rate", apiKey, + "Tracking byte-rate per client", + "client-id", clientId) + } + + def shutdown() = { + throttledRequestReaper.shutdown() + } +} diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 7ea509c..67f0cad 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -17,6 +17,7 @@ package kafka.server +import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.SecurityProtocol import org.apache.kafka.common.TopicPartition import kafka.api._ @@ -42,9 +43,12 @@ class KafkaApis(val requestChannel: RequestChannel, val zkClient: ZkClient, val brokerId: Int, val config: KafkaConfig, - val metadataCache: MetadataCache) extends Logging { + val metadataCache: MetadataCache, + val metrics: Metrics) extends Logging { this.logIdent = "[KafkaApi-%d] ".format(brokerId) + // Store all the quota managers for each type of request + private val quotaManagers = instantiateQuotaManagers(config) /** * Top-level method that handles all requests and multiplexes to the right api @@ -250,6 +254,7 @@ class KafkaApis(val requestChannel: RequestChannel, */ def handleProducerRequest(request: RequestChannel.Request) { val produceRequest = request.requestObj.asInstanceOf[ProducerRequest] + val numBytesAppended = produceRequest.sizeInBytes // the callback for sending a produce response def sendResponseCallback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus]) { @@ -265,21 +270,27 @@ class KafkaApis(val requestChannel: RequestChannel, } } - if (produceRequest.requiredAcks == 0) { - // no operation needed if producer request.required.acks = 0; however, if there is any error in handling - // the request, since no response is expected by the producer, the server will close socket server so that - // the producer client will know that some error has happened and will refresh its metadata - if (errorInResponse) { - info("Close connection due to error handling produce request with correlation id %d from client id %s with ack=0" - .format(produceRequest.correlationId, produceRequest.clientId)) - requestChannel.closeConnection(request.processor, request) + def produceResponseCallback { + if (produceRequest.requiredAcks == 0) { + // no operation needed if producer request.required.acks = 0; however, if there is any error in handling + // the request, since no response is expected by the producer, the server will close socket server so that + // the producer client will know that some error has happened and will refresh its metadata + if (errorInResponse) { + info( + "Close connection due to error handling produce request with correlation id %d from client id %s with ack=0".format( + produceRequest.correlationId, + produceRequest.clientId)) + requestChannel.closeConnection(request.processor, request) + } else { + requestChannel.noOperation(request.processor, request) + } } else { - requestChannel.noOperation(request.processor, request) + val response = ProducerResponse(produceRequest.correlationId, responseStatus) + requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, response))) } - } else { - val response = ProducerResponse(produceRequest.correlationId, responseStatus) - requestChannel.sendResponse(new RequestChannel.Response(request, new RequestOrResponseSend(request.connectionId, response))) } + + quotaManagers(RequestKeys.ProduceKey).recordAndMaybeThrottle(produceRequest.clientId, numBytesAppended, produceResponseCallback) } // only allow appending to internal topic partitions @@ -316,14 +327,27 @@ class KafkaApis(val requestChannel: RequestChannel, .format(fetchRequest.correlationId, fetchRequest.clientId, topicAndPartition, ErrorMapping.exceptionNameFor(data.error))) } - // record the bytes out metrics only when the response is being sent BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesOutRate.mark(data.messages.sizeInBytes) BrokerTopicStats.getBrokerAllTopicsStats().bytesOutRate.mark(data.messages.sizeInBytes) } val response = FetchResponse(fetchRequest.correlationId, responsePartitionData) - requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(request.connectionId, response))) + def fetchResponseCallback { + requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(request.connectionId, response))) + } + + // Do not throttle replication traffic + if (fetchRequest.isFromFollower) { + fetchResponseCallback + } else { + quotaManagers.get(RequestKeys.FetchKey) match { + case Some(quotaManager) => + quotaManager.recordAndMaybeThrottle(fetchRequest.clientId, response.sizeInBytes, fetchResponseCallback) + case None => + warn("Cannot throttle Api key %s".format(RequestKeys.nameForKey(RequestKeys.FetchKey))) + } + } } // call the replica manager to fetch messages from the local replica @@ -604,9 +628,37 @@ class KafkaApis(val requestChannel: RequestChannel, sendResponseCallback) } + /* + * Returns a Map of all quota managers configured. The request Api key is the key for the Map + */ + private def instantiateQuotaManagers(cfg: KafkaConfig): Map[Short, ClientQuotaManager] = { + val producerQuotaManagerCfg = ClientQuotaManagerConfig( + quotaBytesPerSecondDefault = cfg.producerQuotaBytesPerSecondDefault, + quotaBytesPerSecondOverrides = cfg.producerQuotaBytesPerSecondOverrides, + numQuotaSamples = cfg.numQuotaSamples, + quotaWindowSizeSeconds = cfg.quotaWindowSizeSeconds + ) + + val consumerQuotaManagerCfg = ClientQuotaManagerConfig( + quotaBytesPerSecondDefault = cfg.consumerQuotaBytesPerSecondDefault, + quotaBytesPerSecondOverrides = cfg.consumerQuotaBytesPerSecondOverrides, + numQuotaSamples = cfg.numQuotaSamples, + quotaWindowSizeSeconds = cfg.quotaWindowSizeSeconds + ) + + val quotaManagers = Map[Short, ClientQuotaManager]( + RequestKeys.ProduceKey -> + new ClientQuotaManager(producerQuotaManagerCfg, metrics, RequestKeys.nameForKey(RequestKeys.ProduceKey), new org.apache.kafka.common.utils.SystemTime), + RequestKeys.FetchKey -> + new ClientQuotaManager(consumerQuotaManagerCfg, metrics, RequestKeys.nameForKey(RequestKeys.FetchKey), new org.apache.kafka.common.utils.SystemTime) + ) + quotaManagers + } + def close() { - // TODO currently closing the API is an no-op since the API no longer maintain any modules - // maybe removing the closing call in the end when KafkaAPI becomes a pure stateless layer - debug("Shut down complete.") + quotaManagers.foreach { case(apiKey, quotaManager) => + quotaManager.shutdown() + } + info("Shutdown complete.") } } diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index a06f0bd..394f21b 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -26,7 +26,10 @@ import kafka.consumer.ConsumerConfig import kafka.message.{BrokerCompressionCodec, CompressionCodec, Message, MessageSet} import kafka.utils.CoreUtils import org.apache.kafka.clients.CommonClientConfigs -import org.apache.kafka.common.config.{AbstractConfig, ConfigDef} +import org.apache.kafka.common.config.ConfigDef.Importance._ +import org.apache.kafka.common.config.ConfigDef.Range._ +import org.apache.kafka.common.config.ConfigDef.Type._ +import org.apache.kafka.common.config.{ConfigException, AbstractConfig, ConfigDef} import org.apache.kafka.common.metrics.MetricsReporter import org.apache.kafka.common.protocol.SecurityProtocol @@ -132,12 +135,21 @@ object Defaults { val OffsetCommitTimeoutMs = OffsetManagerConfig.DefaultOffsetCommitTimeoutMs val OffsetCommitRequiredAcks = OffsetManagerConfig.DefaultOffsetCommitRequiredAcks + /** ********* Quota Configuration ***********/ + val ProducerQuotaBytesPerSecondDefault = ClientQuotaManagerConfig.QuotaBytesPerSecondDefault + val ConsumerQuotaBytesPerSecondDefault = ClientQuotaManagerConfig.QuotaBytesPerSecondDefault + val ProducerQuotaBytesPerSecondOverrides = ClientQuotaManagerConfig.QuotaBytesPerSecondOverrides + val ConsumerQuotaBytesPerSecondOverrides = ClientQuotaManagerConfig.QuotaBytesPerSecondOverrides + val NumQuotaSamples: Int = ClientQuotaManagerConfig.DefaultNumQuotaSamples + val QuotaWindowSizeSeconds: Int = ClientQuotaManagerConfig.DefaultQuotaWindowSizeSeconds + val DeleteTopicEnable = false val CompressionType = "producer" + /** ********* Kafka Metrics Configuration ***********/ val MetricNumSamples = 2 - val MetricSampleWindowMs = 1000 + val MetricSampleWindowMs = 30000 val MetricReporterClasses = "" } @@ -250,15 +262,22 @@ object KafkaConfig { val OffsetsRetentionCheckIntervalMsProp = "offsets.retention.check.interval.ms" val OffsetCommitTimeoutMsProp = "offsets.commit.timeout.ms" val OffsetCommitRequiredAcksProp = "offsets.commit.required.acks" + /** ********* Quota Configuration ***********/ + val ProducerQuotaBytesPerSecondDefaultProp = "quota.producer.default" + val ConsumerQuotaBytesPerSecondDefaultProp = "quota.consumer.default" + val ProducerQuotaBytesPerSecondOverridesProp = "quota.producer.bytes.per.second.overrides" + val ConsumerQuotaBytesPerSecondOverridesProp = "quota.consumer.bytes.per.second.overrides" + val NumQuotaSamplesProp = "quota.window.num" + val QuotaWindowSizeSecondsProp = "quota.window.size.seconds" val DeleteTopicEnableProp = "delete.topic.enable" val CompressionTypeProp = "compression.type" + /** ********* Kafka Metrics Configuration ***********/ val MetricSampleWindowMsProp = CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_CONFIG val MetricNumSamplesProp: String = CommonClientConfigs.METRICS_NUM_SAMPLES_CONFIG val MetricReporterClassesProp: String = CommonClientConfigs.METRIC_REPORTER_CLASSES_CONFIG - /* Documentation */ /** ********* Zookeeper Configuration ***********/ val ZkConnectDoc = "Zookeeper host string" @@ -388,11 +407,22 @@ object KafkaConfig { val OffsetCommitTimeoutMsDoc = "Offset commit will be delayed until all replicas for the offsets topic receive the commit " + "or this timeout is reached. This is similar to the producer request timeout." val OffsetCommitRequiredAcksDoc = "The required acks before the commit can be accepted. In general, the default (-1) should not be overridden" + /** ********* Quota Configuration ***********/ + val ProducerQuotaBytesPerSecondDefaultDoc = "Any producer distinguished by clientId will get throttled if it produces more bytes than this value per-second" + val ConsumerQuotaBytesPerSecondDefaultDoc = "Any consumer distinguished by clientId/consumer group will get throttled if it fetches more bytes than this value per-second" + val ProducerQuotaBytesPerSecondOverridesDoc = "Comma separated list of clientId:quotaBytesPerSecond to override the default producer quota. " + + "Example: clientIdX=10485760,clientIdY=10485760" + val ConsumerQuotaBytesPerSecondOverridesDoc = "Comma separated list of clientId:quotaBytesPerSecond to override the default consumer quota. " + + "Example: clientIdX=10485760,clientIdY=10485760" + val NumQuotaSamplesDoc = "The number of samples to retain in memory" + val QuotaWindowSizeSecondsDoc = "The time span of each sample" + val DeleteTopicEnableDoc = "Enables delete topic. Delete topic through the admin tool will have no effect if this config is turned off" val CompressionTypeDoc = "Specify the final compression type for a given topic. This configuration accepts the standard compression codecs " + "('gzip', 'snappy', lz4). It additionally accepts 'uncompressed' which is equivalent to no compression; and " + "'producer' which means retain the original compression codec set by the producer." + /** ********* Kafka Metrics Configuration ***********/ val MetricSampleWindowMsDoc = CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_DOC val MetricNumSamplesDoc = CommonClientConfigs.METRICS_NUM_SAMPLES_DOC val MetricReporterClassesDoc = CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC @@ -518,9 +548,19 @@ object KafkaConfig { .define(OffsetCommitRequiredAcksProp, SHORT, Defaults.OffsetCommitRequiredAcks, HIGH, OffsetCommitRequiredAcksDoc) .define(DeleteTopicEnableProp, BOOLEAN, Defaults.DeleteTopicEnable, HIGH, DeleteTopicEnableDoc) .define(CompressionTypeProp, STRING, Defaults.CompressionType, HIGH, CompressionTypeDoc) + + /** ********* Kafka Metrics Configuration ***********/ .define(MetricNumSamplesProp, INT, Defaults.MetricNumSamples, atLeast(1), LOW, MetricNumSamplesDoc) .define(MetricSampleWindowMsProp, LONG, Defaults.MetricSampleWindowMs, atLeast(1), LOW, MetricSampleWindowMsDoc) .define(MetricReporterClassesProp, LIST, Defaults.MetricReporterClasses, LOW, MetricReporterClassesDoc) + + /** ********* Quota configuration ***********/ + .define(ProducerQuotaBytesPerSecondDefaultProp, LONG, Defaults.ProducerQuotaBytesPerSecondDefault, atLeast(1), HIGH, ProducerQuotaBytesPerSecondDefaultDoc) + .define(ConsumerQuotaBytesPerSecondDefaultProp, LONG, Defaults.ConsumerQuotaBytesPerSecondDefault, atLeast(1), HIGH, ConsumerQuotaBytesPerSecondDefaultDoc) + .define(ProducerQuotaBytesPerSecondOverridesProp, STRING, Defaults.ProducerQuotaBytesPerSecondOverrides, HIGH, ProducerQuotaBytesPerSecondOverridesDoc) + .define(ConsumerQuotaBytesPerSecondOverridesProp, STRING, Defaults.ConsumerQuotaBytesPerSecondOverrides, HIGH, ConsumerQuotaBytesPerSecondOverridesDoc) + .define(NumQuotaSamplesProp, INT, Defaults.NumQuotaSamples, atLeast(1), LOW, NumQuotaSamplesDoc) + .define(QuotaWindowSizeSecondsProp, INT, Defaults.QuotaWindowSizeSeconds, atLeast(1), LOW, QuotaWindowSizeSecondsDoc) } def configNames() = { @@ -548,7 +588,6 @@ object KafkaConfig { props.putAll(overrides) fromProps(props) } - } case class KafkaConfig (props: java.util.Map[_, _]) extends AbstractConfig(KafkaConfig.configDef, props) { @@ -661,10 +700,17 @@ case class KafkaConfig (props: java.util.Map[_, _]) extends AbstractConfig(Kafka val metricSampleWindowMs = getLong(KafkaConfig.MetricSampleWindowMsProp) val metricReporterClasses: java.util.List[MetricsReporter] = getConfiguredInstances(KafkaConfig.MetricReporterClassesProp, classOf[MetricsReporter]) + /** ********* Quota Configuration **************/ + val producerQuotaBytesPerSecondDefault = getLong(KafkaConfig.ProducerQuotaBytesPerSecondDefaultProp) + val consumerQuotaBytesPerSecondDefault = getLong(KafkaConfig.ConsumerQuotaBytesPerSecondDefaultProp) + val producerQuotaBytesPerSecondOverrides = getString(KafkaConfig.ProducerQuotaBytesPerSecondOverridesProp) + val consumerQuotaBytesPerSecondOverrides = getString(KafkaConfig.ConsumerQuotaBytesPerSecondOverridesProp) + val numQuotaSamples = getInt(KafkaConfig.NumQuotaSamplesProp) + val quotaWindowSizeSeconds = getInt(KafkaConfig.QuotaWindowSizeSecondsProp) + val deleteTopicEnable = getBoolean(KafkaConfig.DeleteTopicEnableProp) val compressionType = getString(KafkaConfig.CompressionTypeProp) - val listeners = getListeners val advertisedListeners = getAdvertisedListeners val logRetentionTimeMillis = getLogRetentionTimeMillis diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 84d4730..6d65507 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -18,7 +18,6 @@ package kafka.server import java.util -import java.util.Properties import kafka.admin._ import kafka.log.LogConfig @@ -62,11 +61,11 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg private val reporters: java.util.List[MetricsReporter] = config.metricReporterClasses reporters.add(new JmxReporter(jmxPrefix)) - - - // This exists so SocketServer (which uses Client libraries) can use the client Time objects without having to convert all of Kafka to use them - // Once we get rid of kafka.utils.time, we can get rid of this too - private val socketServerTime: org.apache.kafka.common.utils.Time = new org.apache.kafka.common.utils.SystemTime() + // This exists because the Metrics package from clients has its own Time implementation. + // SocketServer/Quotas (which uses client libraries) have to use the client Time objects without having to convert all of Kafka to use them + // Eventually, we want to merge the Time objects in core and clients + private val kafkaMetricsTime: org.apache.kafka.common.utils.Time = new org.apache.kafka.common.utils.SystemTime() + var metrics: Metrics = null val brokerState: BrokerState = new BrokerState @@ -80,7 +79,6 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg var dynamicConfigHandlers: Map[String, ConfigHandler] = null var dynamicConfigManager: DynamicConfigManager = null - val metrics: Metrics = new Metrics() var consumerCoordinator: ConsumerCoordinator = null @@ -92,7 +90,6 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg val metadataCache: MetadataCache = new MetadataCache(config.brokerId) - var zkClient: ZkClient = null val correlationId: AtomicInteger = new AtomicInteger(0) val brokerMetaPropsFile = "meta.properties" @@ -121,6 +118,8 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg val canStartup = isStartingUp.compareAndSet(false, true) if (canStartup) { + metrics = new Metrics(metricConfig, reporters, kafkaMetricsTime) + brokerState.newState(Starting) /* start scheduler */ @@ -137,9 +136,6 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg config.brokerId = getBrokerId this.logIdent = "[Kafka Server " + config.brokerId + "], " - val metrics = new Metrics(metricConfig, reporters, socketServerTime) - - socketServer = new SocketServer(config.brokerId, config.listeners, config.numNetworkThreads, @@ -150,7 +146,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg config.maxConnectionsPerIp, config.connectionsMaxIdleMs, config.maxConnectionsPerIpOverrides, - socketServerTime, + kafkaMetricsTime, metrics) socketServer.startup() @@ -168,7 +164,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg /* start processing requests */ apis = new KafkaApis(socketServer.requestChannel, replicaManager, consumerCoordinator, - kafkaController, zkClient, config.brokerId, config, metadataCache) + kafkaController, zkClient, config.brokerId, config, metadataCache, metrics) requestHandlerPool = new KafkaRequestHandlerPool(config.brokerId, socketServer.requestChannel, apis, config.numIoThreads) brokerState.newState(RunningAsBroker) @@ -362,6 +358,8 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg CoreUtils.swallow(kafkaController.shutdown()) if(zkClient != null) CoreUtils.swallow(zkClient.close()) + if (metrics != null) + CoreUtils.swallow(metrics.close()) brokerState.newState(NotRunning) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 2e0bbcd..d829e18 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -31,6 +31,7 @@ import kafka.metrics.KafkaMetricsGroup import kafka.utils._ import org.I0Itec.zkclient.ZkClient import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.metrics.Metrics import scala.collection._ @@ -98,7 +99,7 @@ class ReplicaManager(val config: KafkaConfig, val zkClient: ZkClient, scheduler: Scheduler, val logManager: LogManager, - val isShuttingDown: AtomicBoolean ) extends Logging with KafkaMetricsGroup { + val isShuttingDown: AtomicBoolean) extends Logging with KafkaMetricsGroup { /* epoch of the controller that last changed the leader */ @volatile var controllerEpoch: Int = KafkaController.InitialControllerEpoch - 1 private val localBrokerId = config.brokerId @@ -440,7 +441,6 @@ class ReplicaManager(val config: KafkaConfig, fetchMinBytes: Int, fetchInfo: immutable.Map[TopicAndPartition, PartitionFetchInfo], responseCallback: Map[TopicAndPartition, FetchResponsePartitionData] => Unit) { - val isFromFollower = replicaId >= 0 val fetchOnlyFromLeader: Boolean = replicaId != Request.DebuggingConsumerId val fetchOnlyCommitted: Boolean = ! Request.isValidBrokerId(replicaId) diff --git a/core/src/main/scala/kafka/server/ThrottledResponse.scala b/core/src/main/scala/kafka/server/ThrottledResponse.scala new file mode 100644 index 0000000..1f80d54 --- /dev/null +++ b/core/src/main/scala/kafka/server/ThrottledResponse.scala @@ -0,0 +1,46 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server + +import java.util.concurrent.{TimeUnit, Delayed} + +import org.apache.kafka.common.utils.Time + + +/** + * Represents a request whose response has been delayed. + * @param time @Time instance to use + * @param delayTimeMs delay associated with this request + * @param callback Callback to trigger after delayTimeMs milliseconds + */ +private[server] class ThrottledResponse(val time: Time, val delayTimeMs: Long, callback: => Unit) extends Delayed { + val endTime = time.milliseconds + delayTimeMs + + def execute() = callback + + override def getDelay(unit: TimeUnit): Long = { + unit.convert(endTime - time.milliseconds, TimeUnit.MILLISECONDS) + } + + override def compareTo(d: Delayed): Int = { + val other = d.asInstanceOf[ThrottledResponse] + if (this.endTime < other.endTime) -1 + else if (this.endTime > other.endTime) 1 + else 0 + } +} diff --git a/core/src/main/scala/kafka/utils/ShutdownableThread.scala b/core/src/main/scala/kafka/utils/ShutdownableThread.scala index fc226c8..dc46797 100644 --- a/core/src/main/scala/kafka/utils/ShutdownableThread.scala +++ b/core/src/main/scala/kafka/utils/ShutdownableThread.scala @@ -51,6 +51,9 @@ abstract class ShutdownableThread(val name: String, val isInterruptible: Boolean info("Shutdown completed") } + /** + * This method is repeatedly invoked until the thread shuts down or this method throws an exception + */ def doWork(): Unit override def run(): Unit = { diff --git a/core/src/test/scala/integration/kafka/api/QuotasTest.scala b/core/src/test/scala/integration/kafka/api/QuotasTest.scala new file mode 100644 index 0000000..a11bf90 --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/QuotasTest.scala @@ -0,0 +1,194 @@ +/** + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package kafka.api + +import java.util.Properties + +import junit.framework.Assert +import kafka.consumer.SimpleConsumer +import kafka.integration.KafkaServerTestHarness +import kafka.server.{KafkaServer, KafkaConfig} +import kafka.utils.TestUtils +import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer} +import org.apache.kafka.clients.producer._ +import org.apache.kafka.clients.producer.internals.ErrorLoggingCallback +import org.apache.kafka.common.MetricName +import org.apache.kafka.common.metrics.KafkaMetric +import org.junit.Assert._ +import org.junit.{After, Before, Test} +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ + +import scala.collection.mutable + +class QuotasTest extends KafkaServerTestHarness { + private val producerBufferSize = 300000 + private val producerId1 = "QuotasTestProducer-1" + private val producerId2 = "QuotasTestProducer-2" + private val consumerId1 = "QuotasTestConsumer-1" + private val consumerId2 = "QuotasTestConsumer-2" + + val numServers = 2 + val overridingProps = new Properties() + + // Low enough quota that a producer sending a small payload in a tight loop should get throttled + overridingProps.put(KafkaConfig.ProducerQuotaBytesPerSecondDefaultProp, "8000") + overridingProps.put(KafkaConfig.ConsumerQuotaBytesPerSecondDefaultProp, "2500") + + // un-throttled + overridingProps.put(KafkaConfig.ProducerQuotaBytesPerSecondOverridesProp, producerId2 + "=" + Long.MaxValue) + overridingProps.put(KafkaConfig.ConsumerQuotaBytesPerSecondOverridesProp, consumerId2 + "=" + Long.MaxValue) + + override def generateConfigs() = { + FixedPortTestUtils.createBrokerConfigs(numServers, + zkConnect, + enableControlledShutdown = false) + .map(KafkaConfig.fromProps(_, overridingProps)) + } + + var producers = mutable.Buffer[KafkaProducer[Array[Byte], Array[Byte]]]() + var consumers = mutable.Buffer[KafkaConsumer[Array[Byte], Array[Byte]]]() + var replicaConsumers = mutable.Buffer[SimpleConsumer]() + + var leaderNode: KafkaServer = null + var followerNode: KafkaServer = null + private val topic1 = "topic-1" + + @Before + override def setUp() { + super.setUp() + val producerProps = new Properties() + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) + producerProps.put(ProducerConfig.ACKS_CONFIG, "0") + producerProps.put(ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG, "false") + producerProps.put(ProducerConfig.BUFFER_MEMORY_CONFIG, producerBufferSize.toString) + producerProps.put(ProducerConfig.CLIENT_ID_CONFIG, producerId1) + producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, + classOf[org.apache.kafka.common.serialization.ByteArraySerializer]) + producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, + classOf[org.apache.kafka.common.serialization.ByteArraySerializer]) + producers += new KafkaProducer[Array[Byte], Array[Byte]](producerProps) + + producerProps.put(ProducerConfig.CLIENT_ID_CONFIG, producerId2) + producers += new KafkaProducer[Array[Byte], Array[Byte]](producerProps) + + val numPartitions = 1 + val leaders = TestUtils.createTopic(zkClient, topic1, numPartitions, numServers, servers) + leaderNode = if (leaders(0).get == servers.head.config.brokerId) servers.head else servers(1) + followerNode = if (leaders(0).get != servers.head.config.brokerId) servers.head else servers(1) + assertTrue("Leader of all partitions of the topic should exist", leaders.values.forall(leader => leader.isDefined)) + + // Create consumers + val consumerProps = new Properties + consumerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) + consumerProps.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "QuotasTest") + consumerProps.setProperty(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, 4096.toString) + consumerProps.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") + consumerProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, this.bootstrapUrl) + consumerProps.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, + classOf[org.apache.kafka.common.serialization.ByteArrayDeserializer]) + consumerProps.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, + classOf[org.apache.kafka.common.serialization.ByteArrayDeserializer]) + consumerProps.put(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, "range") + + consumerProps.put(ConsumerConfig.CLIENT_ID_CONFIG, consumerId1) + consumers += new KafkaConsumer(consumerProps) + // Create replica consumers with the same clientId as the high level consumer. These requests should never be throttled + replicaConsumers += new SimpleConsumer("localhost", leaderNode.boundPort(), 1000000, 64*1024, consumerId1) + + consumerProps.put(ConsumerConfig.CLIENT_ID_CONFIG, consumerId2) + consumers += new KafkaConsumer(consumerProps) + replicaConsumers += new SimpleConsumer("localhost", leaderNode.boundPort(), 1000000, 64*1024, consumerId2) + + } + + @After + override def tearDown() { + producers.foreach( _.close ) + consumers.foreach( _.close ) + replicaConsumers.foreach( _.close ) + super.tearDown() + } + + @Test + def testThrottledProducerConsumer() { + val allMetrics: mutable.Map[MetricName, KafkaMetric] = leaderNode.metrics.metrics().asScala + + val numRecords = 1000 + produce(producers.head, numRecords) + + val producerMetricName = new MetricName("throttle-time", + RequestKeys.nameForKey(RequestKeys.ProduceKey), + "Tracking throttle-time per client", + "client-id", producerId1) + Assert.assertTrue("Should have been throttled", allMetrics(producerMetricName).value() > 0) + + // Consumer should read in a bursty manner and get throttled immediately + consume(consumers.head, numRecords) + // The replica consumer should not be throttled also. Create a fetch request which will exceed the quota immediately + val request = new FetchRequestBuilder().addFetch(topic1, 0, 0, 1024*1024).replicaId(followerNode.config.brokerId).build() + replicaConsumers.head.fetch(request) + val consumerMetricName = new MetricName("throttle-time", + RequestKeys.nameForKey(RequestKeys.FetchKey), + "Tracking throttle-time per client", + "client-id", consumerId1) + Assert.assertTrue("Should have been throttled", allMetrics(consumerMetricName).value() > 0) + } + + @Test + def testProducerConsumerOverrideUnthrottled() { + val allMetrics: mutable.Map[MetricName, KafkaMetric] = leaderNode.metrics.metrics().asScala + val numRecords = 1000 + produce(producers(1), numRecords) + val producerMetricName = new MetricName("throttle-time", + RequestKeys.nameForKey(RequestKeys.ProduceKey), + "Tracking throttle-time per client", + "client-id", producerId2) + Assert.assertEquals("Should not have been throttled", Double.NaN, allMetrics(producerMetricName).value()) + + // The "client" consumer does not get throttled. + consume(consumers(1), numRecords) + // The replica consumer should not be throttled also. Create a fetch request which will exceed the quota immediately + val request = new FetchRequestBuilder().addFetch(topic1, 0, 0, 1024*1024).replicaId(followerNode.config.brokerId).build() + replicaConsumers(1).fetch(request) + val consumerMetricName = new MetricName("throttle-time", + RequestKeys.nameForKey(RequestKeys.FetchKey), + "Tracking throttle-time per client", + "client-id", consumerId2) + Assert.assertEquals("Should not have been throttled", Double.NaN, allMetrics(consumerMetricName).value()) + } + + def produce(p: KafkaProducer[Array[Byte], Array[Byte]], count: Int): Int = { + var numBytesProduced = 0 + for (i <- 0 to count) { + val payload = i.toString.getBytes + numBytesProduced += payload.length + p.send(new ProducerRecord[Array[Byte], Array[Byte]](topic1, null, null, payload), + new ErrorLoggingCallback(topic1, null, null, true)).get() + Thread.sleep(1) + } + numBytesProduced + } + + def consume(consumer: KafkaConsumer[Array[Byte], Array[Byte]], numRecords: Int) { + consumer.subscribe(topic1) + var numConsumed = 0 + while (numConsumed < numRecords) { + for (cr <- consumer.poll(100)) { + numConsumed += 1 + } + } + } +} diff --git a/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala new file mode 100644 index 0000000..97dcca8 --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala @@ -0,0 +1,159 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package kafka.server + +import java.util.Collections + +import kafka.api.RequestKeys +import org.apache.kafka.common.MetricName +import org.apache.kafka.common.metrics.{Metrics, Quota, MetricConfig} +import org.apache.kafka.common.utils.MockTime +import org.scalatest.junit.JUnit3Suite +import org.junit.{Before, Test, Assert} + +class ClientQuotaManagerTest extends JUnit3Suite { + private val time = new MockTime + + private val config = ClientQuotaManagerConfig(quotaBytesPerSecondDefault = 500, + quotaBytesPerSecondOverrides = "p1=2000,p2=4000") + + var numCallbacks: Int = 0 + def callback { + numCallbacks += 1 + } + + @Before + def beforeMethod() { + numCallbacks = 0 + } + + @Test + def testQuotaParsing() { + val clientMetrics = new ClientQuotaManager(config, newMetrics, "producer", time) + try { + Assert.assertEquals("Default producer quota should be 500", + new Quota(500, true), clientMetrics.quota("random-client-id")) + Assert.assertEquals("Should return the overridden value (2000)", + new Quota(2000, true), clientMetrics.quota("p1")) + Assert.assertEquals("Should return the overridden value (4000)", + new Quota(4000, true), clientMetrics.quota("p2")) + } finally { + clientMetrics.shutdown() + } + } + + @Test + def testQuotaViolation() { + val metrics = newMetrics + val clientMetrics = new ClientQuotaManager(config, metrics, "producer", time) + val queueSizeMetric = metrics.metrics().get(new MetricName("queue-size", "producer", "")) + try { + /* We have 10 second windows. Make sure that there is no quota violation + * if we produce under the quota + */ + for (i <- 0 until 10) { + clientMetrics.recordAndMaybeThrottle("unknown", 400, callback) + time.sleep(1000) + } + Assert.assertEquals(10, numCallbacks) + Assert.assertEquals(0, queueSizeMetric.value().toInt) + + // Create a spike. + // 400*10 + 2000 = 6000/10 = 600 bytes per second. + // (600 - quota)/quota*window-size = (600-500)/500*11 seconds = 2200 + val sleepTime = clientMetrics.recordAndMaybeThrottle("unknown", 2000, callback) + Assert.assertEquals("Should be throttled", 2200, sleepTime) + Assert.assertEquals(1, queueSizeMetric.value().toInt) + // After a request is delayed, the callback cannot be triggered immediately + clientMetrics.throttledRequestReaper.doWork() + Assert.assertEquals(10, numCallbacks) + time.sleep(sleepTime) + + // Callback can only be triggered after the the delay time passes + clientMetrics.throttledRequestReaper.doWork() + Assert.assertEquals(0, queueSizeMetric.value().toInt) + Assert.assertEquals(11, numCallbacks) + + // Could continue to see delays until the bursty sample disappears + for (i <- 0 until 10) { + clientMetrics.recordAndMaybeThrottle("unknown", 400, callback) + time.sleep(1000) + } + + Assert.assertEquals("Should be unthrottled since bursty sample has rolled over", + 0, clientMetrics.recordAndMaybeThrottle("unknown", 0, callback)) + } finally { + clientMetrics.shutdown() + } + } + + @Test + def testOverrideParse() { + var testConfig = ClientQuotaManagerConfig() + var clientMetrics = new ClientQuotaManager(testConfig, newMetrics, "consumer", time) + + try { + // Case 1 - Default config + Assert.assertEquals(new Quota(ClientQuotaManagerConfig.QuotaBytesPerSecondDefault, true), + clientMetrics.quota("p1")) + } finally { + clientMetrics.shutdown() + } + + + // Case 2 - Empty override + testConfig = ClientQuotaManagerConfig(quotaBytesPerSecondDefault = 500, + quotaBytesPerSecondOverrides = "p1=2000,p2=4000,,") + + clientMetrics = new ClientQuotaManager(testConfig, newMetrics, "consumer", time) + try { + Assert.assertEquals(new Quota(2000, true), clientMetrics.quota("p1")) + Assert.assertEquals(new Quota(4000, true), clientMetrics.quota("p2")) + } finally { + clientMetrics.shutdown() + } + + // Case 3 - NumberFormatException for override + testConfig = ClientQuotaManagerConfig(quotaBytesPerSecondDefault = 500, + quotaBytesPerSecondOverrides = "p1=2000,p2=4000,p3=p4") + try { + clientMetrics = new ClientQuotaManager(testConfig, newMetrics, "consumer", time) + Assert.fail("Should fail to parse invalid config " + testConfig.quotaBytesPerSecondOverrides) + } + catch { + // Swallow. + case nfe: NumberFormatException => + } + + // Case 4 - IllegalArgumentException for override + testConfig = ClientQuotaManagerConfig(quotaBytesPerSecondDefault = 500, + quotaBytesPerSecondOverrides = "p1=2000=3000") + try { + clientMetrics = new ClientQuotaManager(testConfig, newMetrics, "producer", time) + Assert.fail("Should fail to parse invalid config " + testConfig.quotaBytesPerSecondOverrides) + } + catch { + // Swallow. + case nfe: IllegalArgumentException => + } + + } + + def newMetrics: Metrics = { + new Metrics(new MetricConfig(), Collections.emptyList(), time) + } +} diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index e26a730..9688b8c 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -474,6 +474,12 @@ class KafkaConfigTest { case KafkaConfig.OffsetsRetentionCheckIntervalMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") case KafkaConfig.OffsetCommitTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") case KafkaConfig.OffsetCommitRequiredAcksProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "-2") + case KafkaConfig.ProducerQuotaBytesPerSecondDefaultProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.ConsumerQuotaBytesPerSecondDefaultProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.ProducerQuotaBytesPerSecondOverridesProp => // ignore string + case KafkaConfig.ConsumerQuotaBytesPerSecondOverridesProp => // ignore string + case KafkaConfig.NumQuotaSamplesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.QuotaWindowSizeSecondsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") case KafkaConfig.DeleteTopicEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean", "0") diff --git a/core/src/test/scala/unit/kafka/server/ThrottledResponseExpirationTest.scala b/core/src/test/scala/unit/kafka/server/ThrottledResponseExpirationTest.scala new file mode 100644 index 0000000..14a7f45 --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/ThrottledResponseExpirationTest.scala @@ -0,0 +1,90 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server + + +import java.util.Collections +import java.util.concurrent.{TimeUnit, DelayQueue} + +import org.apache.kafka.common.metrics.MetricConfig +import org.apache.kafka.common.utils.MockTime +import org.junit.{AfterClass, Before, Assert, Test} +import org.scalatest.junit.JUnit3Suite + +class ThrottledResponseExpirationTest extends JUnit3Suite { + private val time = new MockTime + private var numCallbacks: Int = 0 + private val metrics = new org.apache.kafka.common.metrics.Metrics(new MetricConfig(), + Collections.emptyList(), + time) + + def callback { + numCallbacks += 1 + } + + @Before + def beforeMethod() { + numCallbacks = 0 + } + + @Test + def testExpire() { + val clientMetrics = new ClientQuotaManager(ClientQuotaManagerConfig(), metrics, "producer", time) + + val delayQueue = new DelayQueue[ThrottledResponse]() + val reaper = new clientMetrics.ThrottledRequestReaper(delayQueue) + try { + // Add 4 elements to the queue out of order. Add 2 elements with the same expire timestamp + delayQueue.add(new ThrottledResponse(time, 10, callback)) + delayQueue.add(new ThrottledResponse(time, 30, callback)) + delayQueue.add(new ThrottledResponse(time, 30, callback)) + delayQueue.add(new ThrottledResponse(time, 20, callback)) + + for(itr <- 1 to 3) { + time.sleep(10) + reaper.doWork() + Assert.assertEquals(itr, numCallbacks) + + } + reaper.doWork() + Assert.assertEquals(4, numCallbacks) + Assert.assertEquals(0, delayQueue.size()) + reaper.doWork() + Assert.assertEquals(4, numCallbacks) + } finally { + clientMetrics.shutdown() + } + } + + @Test + def testThrottledRequest() { + val t1: ThrottledResponse = new ThrottledResponse(time, 10, callback) + val t2: ThrottledResponse = new ThrottledResponse(time, 20, callback) + val t3: ThrottledResponse = new ThrottledResponse(time, 20, callback) + Assert.assertEquals(10, t1.delayTimeMs) + Assert.assertEquals(20, t2.delayTimeMs) + Assert.assertEquals(20, t3.delayTimeMs) + + for(itr <- 0 to 2) { + Assert.assertEquals(10 - 10*itr, t1.getDelay(TimeUnit.MILLISECONDS)) + Assert.assertEquals(20 - 10*itr, t2.getDelay(TimeUnit.MILLISECONDS)) + Assert.assertEquals(20 - 10*itr, t3.getDelay(TimeUnit.MILLISECONDS)) + time.sleep(10) + } + } +} -- 1.7.12.4 From 63b89658bcb5fc2d95e10d28987337c3d971163f Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Fri, 14 Aug 2015 21:55:42 -0700 Subject: [PATCH 112/120] KAFKA-1782: Follow up - add missing @Test annotations. Author: Ewen Cheslack-Postava Reviewers: Guozhang Closes #140 from ewencp/kafka-1782-followup and squashes the following commits: fe36bd1 [Ewen Cheslack-Postava] Fix missing annotation of Before and After in ControllerFailoverTest. 1dcaf39 [Ewen Cheslack-Postava] KAFKA-1782: Follow up - add missing @Test annotations. --- .../scala/integration/kafka/api/ConsumerBounceTest.scala | 4 +++- .../test/scala/integration/kafka/api/ConsumerTest.scala | 16 +++++++++++++++- .../test/scala/unit/kafka/admin/AddPartitionsTest.scala | 7 ++++++- .../unit/kafka/consumer/PartitionAssignorTest.scala | 3 +++ .../kafka/consumer/ZookeeperConsumerConnectorTest.scala | 9 +++++++-- .../unit/kafka/controller/ControllerFailoverTest.scala | 3 +++ .../test/scala/unit/kafka/integration/FetcherTest.scala | 3 ++- .../scala/unit/kafka/integration/MinIsrConfigTest.scala | 2 ++ .../scala/unit/kafka/integration/PrimitiveApiTest.scala | 9 +++++++++ .../scala/unit/kafka/integration/RollingBounceTest.scala | 3 ++- .../scala/unit/kafka/integration/TopicMetadataTest.scala | 12 ++++++++++-- .../kafka/integration/UncleanLeaderElectionTest.scala | 7 ++++++- .../consumer/ZookeeperConsumerConnectorTest.scala | 2 ++ .../scala/unit/kafka/server/AdvertiseBrokerTest.scala | 5 +++-- .../unit/kafka/server/HighwatermarkPersistenceTest.scala | 2 ++ .../test/scala/unit/kafka/server/ISRExpirationTest.scala | 5 ++++- .../scala/unit/kafka/server/LeaderElectionTest.scala | 4 +++- .../test/scala/unit/kafka/server/LogRecoveryTest.scala | 6 +++++- .../test/scala/unit/kafka/server/ReplicaFetchTest.scala | 3 ++- .../scala/unit/kafka/server/ServerShutdownTest.scala | 1 + .../test/scala/unit/kafka/server/ServerStartupTest.scala | 3 +++ .../test/scala/unit/kafka/server/SimpleFetchTest.scala | 3 ++- core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala | 3 ++- core/src/test/scala/unit/kafka/zk/ZKPathTest.scala | 9 +++++++++ 24 files changed, 106 insertions(+), 18 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala index cf65f12..2c6ee23 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala @@ -19,7 +19,7 @@ import org.apache.kafka.clients.consumer._ import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord} import org.apache.kafka.common.TopicPartition import org.junit.Assert._ -import org.junit.Before +import org.junit.{Test, Before} import scala.collection.JavaConversions._ @@ -61,6 +61,7 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging { TestUtils.createTopic(this.zkClient, topic, 1, serverCount, this.servers) } + @Test def testConsumptionWithBrokerFailures() = consumeWithBrokerFailures(10) /* @@ -96,6 +97,7 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging { scheduler.shutdown() } + @Test def testSeekAndCommitWithBrokerFailures() = seekAndCommitWithBrokerFailures(5) def seekAndCommitWithBrokerFailures(numIters: Int) { diff --git a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala index b46070a..9e8172a 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala @@ -25,7 +25,7 @@ import kafka.server.KafkaConfig import java.util.ArrayList import org.junit.Assert._ -import org.junit.Before +import org.junit.{Test, Before} import scala.collection.JavaConverters._ import kafka.coordinator.ConsumerCoordinator @@ -65,6 +65,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging { TestUtils.createTopic(this.zkClient, topic, 2, serverCount, this.servers) } + @Test def testSimpleConsumption() { val numRecords = 10000 sendRecords(numRecords) @@ -86,6 +87,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging { awaitCommitCallback(this.consumers(0), commitCallback) } + @Test def testCommitSpecifiedOffsets() { sendRecords(5, tp) sendRecords(7, tp2) @@ -116,12 +118,14 @@ class ConsumerTest extends IntegrationTestHarness with Logging { assertEquals(7, this.consumers(0).committed(tp2)) } + @Test def testAutoOffsetReset() { sendRecords(1) this.consumers(0).subscribe(tp) consumeRecords(this.consumers(0), numRecords = 1, startingOffset = 0) } + @Test def testSeek() { val consumer = this.consumers(0) val totalRecords = 50L @@ -142,12 +146,14 @@ class ConsumerTest extends IntegrationTestHarness with Logging { consumeRecords(consumer, numRecords = 1, startingOffset = mid.toInt) } + @Test def testGroupConsumption() { sendRecords(10) this.consumers(0).subscribe(topic) consumeRecords(this.consumers(0), numRecords = 1, startingOffset = 0) } + @Test def testPositionAndCommit() { sendRecords(5) @@ -179,6 +185,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging { consumeRecords(this.consumers(1), 1, 5) } + @Test def testPartitionsFor() { val numParts = 2 TestUtils.createTopic(this.zkClient, "part-test", numParts, 1, this.servers) @@ -188,6 +195,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging { assertNull(this.consumers(0).partitionsFor("non-exist-topic")) } + @Test def testListTopics() { val numParts = 2 val topic1: String = "part-test-topic-1" @@ -206,6 +214,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging { assertEquals(2, topics.get(topic3).size) } + @Test def testPartitionReassignmentCallback() { val callback = new TestConsumerReassignmentCallback() this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "100"); // timeout quickly to avoid slow test @@ -238,6 +247,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging { consumer0.close() } + @Test def testUnsubscribeTopic() { val callback = new TestConsumerReassignmentCallback() this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "100"); // timeout quickly to avoid slow test @@ -258,6 +268,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging { } } + @Test def testExpandingTopicSubscriptions() { val otherTopic = "other" val subscriptions = Set(new TopicPartition(topic, 0), new TopicPartition(topic, 1)) @@ -276,6 +287,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging { }, s"Expected partitions ${expandedSubscriptions.asJava} but actually got ${this.consumers(0).subscriptions}") } + @Test def testShrinkingTopicSubscriptions() { val otherTopic = "other" TestUtils.createTopic(this.zkClient, otherTopic, 2, serverCount, this.servers) @@ -294,6 +306,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging { }, s"Expected partitions ${shrunkenSubscriptions.asJava} but actually got ${this.consumers(0).subscriptions}") } + @Test def testPartitionPauseAndResume() { sendRecords(5) this.consumers(0).subscribe(tp) @@ -305,6 +318,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging { consumeRecords(this.consumers(0), 5, 5) } + @Test def testPauseStateNotPreservedByRebalance() { this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "100"); // timeout quickly to avoid slow test this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "30"); diff --git a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala index 08c170b..05b9a87 100755 --- a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala @@ -25,7 +25,7 @@ import kafka.utils.{ZkUtils, CoreUtils, TestUtils} import kafka.cluster.Broker import kafka.client.ClientUtils import kafka.server.{KafkaConfig, KafkaServer} -import org.junit.{After, Before} +import org.junit.{Test, After, Before} class AddPartitionsTest extends ZooKeeperTestHarness { var configs: Seq[KafkaConfig] = null @@ -62,6 +62,7 @@ class AddPartitionsTest extends ZooKeeperTestHarness { super.tearDown() } + @Test def testTopicDoesNotExist { try { AdminUtils.addPartitions(zkClient, "Blah", 1) @@ -72,6 +73,7 @@ class AddPartitionsTest extends ZooKeeperTestHarness { } } + @Test def testWrongReplicaCount { try { AdminUtils.addPartitions(zkClient, topic1, 2, "0:1,0:1:2") @@ -82,6 +84,7 @@ class AddPartitionsTest extends ZooKeeperTestHarness { } } + @Test def testIncrementPartitions { AdminUtils.addPartitions(zkClient, topic1, 3) // wait until leader is elected @@ -107,6 +110,7 @@ class AddPartitionsTest extends ZooKeeperTestHarness { assert(replicas.contains(partitionDataForTopic1(1).leader.get)) } + @Test def testManualAssignmentOfReplicas { AdminUtils.addPartitions(zkClient, topic2, 3, "1:2,0:1,2:3") // wait until leader is elected @@ -133,6 +137,7 @@ class AddPartitionsTest extends ZooKeeperTestHarness { assert(replicas(1).id == 0 || replicas(1).id == 1) } + @Test def testReplicaPlacement { AdminUtils.addPartitions(zkClient, topic3, 7) diff --git a/core/src/test/scala/unit/kafka/consumer/PartitionAssignorTest.scala b/core/src/test/scala/unit/kafka/consumer/PartitionAssignorTest.scala index c1071b8..dba1afb 100644 --- a/core/src/test/scala/unit/kafka/consumer/PartitionAssignorTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/PartitionAssignorTest.scala @@ -26,9 +26,11 @@ import kafka.common.TopicAndPartition import kafka.consumer.PartitionAssignorTest.StaticSubscriptionInfo import kafka.consumer.PartitionAssignorTest.Scenario import kafka.consumer.PartitionAssignorTest.WildcardSubscriptionInfo +import org.junit.Test class PartitionAssignorTest extends Logging { + @Test def testRoundRobinPartitionAssignor() { val assignor = new RoundRobinAssignor @@ -52,6 +54,7 @@ class PartitionAssignorTest extends Logging { }) } + @Test def testRangePartitionAssignor() { val assignor = new RangeAssignor (1 to PartitionAssignorTest.TestCaseCount).foreach (testCase => { diff --git a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala b/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala index c851e27..cb59542 100644 --- a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala @@ -30,7 +30,7 @@ import kafka.utils.TestUtils._ import kafka.utils._ import org.I0Itec.zkclient.ZkClient import org.apache.log4j.{Level, Logger} -import org.junit.{After, Before} +import org.junit.{Test, After, Before} import scala.collection._ @@ -65,6 +65,7 @@ class ZookeeperConsumerConnectorTest extends KafkaServerTestHarness with Logging super.tearDown() } + @Test def testBasic() { val requestHandlerLogger = Logger.getLogger(classOf[KafkaRequestHandler]) requestHandlerLogger.setLevel(Level.FATAL) @@ -175,7 +176,7 @@ class ZookeeperConsumerConnectorTest extends KafkaServerTestHarness with Logging requestHandlerLogger.setLevel(Level.ERROR) } - + @Test def testCompression() { val requestHandlerLogger = Logger.getLogger(classOf[kafka.server.KafkaRequestHandler]) requestHandlerLogger.setLevel(Level.FATAL) @@ -255,6 +256,7 @@ class ZookeeperConsumerConnectorTest extends KafkaServerTestHarness with Logging requestHandlerLogger.setLevel(Level.ERROR) } + @Test def testCompressionSetConsumption() { // send some messages to each broker val sentMessages = sendMessages(servers, topic, 200, 0, DefaultCompressionCodec) ++ @@ -278,6 +280,7 @@ class ZookeeperConsumerConnectorTest extends KafkaServerTestHarness with Logging zkConsumerConnector1.shutdown } + @Test def testConsumerDecoder() { val requestHandlerLogger = Logger.getLogger(classOf[kafka.server.KafkaRequestHandler]) requestHandlerLogger.setLevel(Level.FATAL) @@ -317,6 +320,7 @@ class ZookeeperConsumerConnectorTest extends KafkaServerTestHarness with Logging requestHandlerLogger.setLevel(Level.ERROR) } + @Test def testLeaderSelectionForPartition() { val zkClient = ZkUtils.createZkClient(zkConnect, 6000, 30000) @@ -348,6 +352,7 @@ class ZookeeperConsumerConnectorTest extends KafkaServerTestHarness with Logging zkClient.close() } + @Test def testConsumerRebalanceListener() { // Send messages to create topic sendMessages(servers, topic, nMessages, 0) diff --git a/core/src/test/scala/unit/kafka/controller/ControllerFailoverTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerFailoverTest.scala index 206a7c3..0e38a18 100644 --- a/core/src/test/scala/unit/kafka/controller/ControllerFailoverTest.scala +++ b/core/src/test/scala/unit/kafka/controller/ControllerFailoverTest.scala @@ -54,10 +54,12 @@ class ControllerFailoverTest extends KafkaServerTestHarness with Logging { override def generateConfigs() = TestUtils.createBrokerConfigs(numNodes, zkConnect) .map(KafkaConfig.fromProps(_, overridingProps)) + @Before override def setUp() { super.setUp() } + @After override def tearDown() { super.tearDown() } @@ -66,6 +68,7 @@ class ControllerFailoverTest extends KafkaServerTestHarness with Logging { * See @link{https://issues.apache.org/jira/browse/KAFKA-2300} * for the background of this test case */ + @Test def testMetadataUpdate() { log.setLevel(Level.INFO) var controller: KafkaServer = this.servers.head; diff --git a/core/src/test/scala/unit/kafka/integration/FetcherTest.scala b/core/src/test/scala/unit/kafka/integration/FetcherTest.scala index 92af0a1..c061597 100644 --- a/core/src/test/scala/unit/kafka/integration/FetcherTest.scala +++ b/core/src/test/scala/unit/kafka/integration/FetcherTest.scala @@ -19,7 +19,7 @@ package kafka.integration import java.util.concurrent._ import java.util.concurrent.atomic._ -import org.junit.{After, Before} +import org.junit.{Test, After, Before} import scala.collection._ import org.junit.Assert._ @@ -65,6 +65,7 @@ class FetcherTest extends KafkaServerTestHarness { super.tearDown } + @Test def testFetcher() { val perNode = 2 var count = TestUtils.sendMessages(servers, topic, perNode).size diff --git a/core/src/test/scala/unit/kafka/integration/MinIsrConfigTest.scala b/core/src/test/scala/unit/kafka/integration/MinIsrConfigTest.scala index 3c1cade..3977601 100644 --- a/core/src/test/scala/unit/kafka/integration/MinIsrConfigTest.scala +++ b/core/src/test/scala/unit/kafka/integration/MinIsrConfigTest.scala @@ -21,6 +21,7 @@ import java.util.Properties import kafka.server.KafkaConfig import kafka.utils.TestUtils +import org.junit.Test class MinIsrConfigTest extends KafkaServerTestHarness { @@ -28,6 +29,7 @@ class MinIsrConfigTest extends KafkaServerTestHarness { overridingProps.put(KafkaConfig.MinInSyncReplicasProp, "5") def generateConfigs() = TestUtils.createBrokerConfigs(1, zkConnect).map(KafkaConfig.fromProps(_, overridingProps)) + @Test def testDefaultKafkaConfig() { assert(servers.head.getLogManager().defaultConfig.minInSyncReplicas == 5) } diff --git a/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala b/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala index e05d16b..e6f0c54 100755 --- a/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala +++ b/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala @@ -24,6 +24,7 @@ import kafka.server.{KafkaRequestHandler, KafkaConfig} import kafka.producer.{KeyedMessage, Producer} import org.apache.log4j.{Level, Logger} import kafka.zk.ZooKeeperTestHarness +import org.junit.Test import scala.collection._ import kafka.common.{TopicAndPartition, ErrorMapping, UnknownTopicOrPartitionException, OffsetOutOfRangeException} import kafka.utils.{StaticPartitioner, TestUtils, CoreUtils} @@ -38,6 +39,7 @@ class PrimitiveApiTest extends ProducerConsumerTestHarness with ZooKeeperTestHar def generateConfigs() = List(KafkaConfig.fromProps(TestUtils.createBrokerConfig(0, zkConnect))) + @Test def testFetchRequestCanProperlySerialize() { val request = new FetchRequestBuilder() .clientId("test-client") @@ -54,6 +56,7 @@ class PrimitiveApiTest extends ProducerConsumerTestHarness with ZooKeeperTestHar assertEquals(request, deserializedRequest) } + @Test def testEmptyFetchRequest() { val partitionRequests = immutable.Map[TopicAndPartition, PartitionFetchInfo]() val request = new FetchRequest(requestInfo = partitionRequests) @@ -61,6 +64,7 @@ class PrimitiveApiTest extends ProducerConsumerTestHarness with ZooKeeperTestHar assertTrue(!fetched.hasError && fetched.data.size == 0) } + @Test def testDefaultEncoderProducerAndFetch() { val topic = "test-topic" @@ -84,6 +88,7 @@ class PrimitiveApiTest extends ProducerConsumerTestHarness with ZooKeeperTestHar assertEquals("test-message", TestUtils.readString(fetchedMessageAndOffset.message.payload, "UTF-8")) } + @Test def testDefaultEncoderProducerAndFetchWithCompression() { val topic = "test-topic" val props = new Properties() @@ -170,6 +175,7 @@ class PrimitiveApiTest extends ProducerConsumerTestHarness with ZooKeeperTestHar requestHandlerLogger.setLevel(Level.ERROR) } + @Test def testProduceAndMultiFetch() { produceAndMultiFetch(producer) } @@ -196,10 +202,12 @@ class PrimitiveApiTest extends ProducerConsumerTestHarness with ZooKeeperTestHar } } + @Test def testMultiProduce() { multiProduce(producer) } + @Test def testConsumerEmptyTopic() { val newTopic = "new-topic" TestUtils.createTopic(zkClient, newTopic, numPartitions = 1, replicationFactor = 1, servers = servers) @@ -208,6 +216,7 @@ class PrimitiveApiTest extends ProducerConsumerTestHarness with ZooKeeperTestHar assertFalse(fetchResponse.messageSet(newTopic, 0).iterator.hasNext) } + @Test def testPipelinedProduceRequests() { val topics = Map("test4" -> 0, "test1" -> 0, "test2" -> 0, "test3" -> 0) topics.keys.map(topic => TestUtils.createTopic(zkClient, topic, servers = servers)) diff --git a/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala b/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala index 2fd10d8..4d73be1 100755 --- a/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala +++ b/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala @@ -17,7 +17,7 @@ package kafka.integration -import org.junit.{After, Before} +import org.junit.{Test, After, Before} import kafka.zk.ZooKeeperTestHarness import kafka.utils.TestUtils._ import org.junit.Assert._ @@ -47,6 +47,7 @@ class RollingBounceTest extends ZooKeeperTestHarness { super.tearDown() } + @Test def testRollingBounce { // start all the brokers val topic1 = "new-topic1" diff --git a/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala b/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala index 24f0a07..5e32d59 100644 --- a/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala +++ b/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala @@ -30,7 +30,7 @@ import kafka.utils.TestUtils._ import kafka.zk.ZooKeeperTestHarness import org.apache.kafka.common.protocol.SecurityProtocol import org.junit.Assert._ -import org.junit.{After, Before} +import org.junit.{Test, After, Before} class TopicMetadataTest extends ZooKeeperTestHarness { private var server1: KafkaServer = null @@ -54,6 +54,7 @@ class TopicMetadataTest extends ZooKeeperTestHarness { super.tearDown() } + @Test def testTopicMetadataRequest { // create topic val topic = "test" @@ -70,6 +71,7 @@ class TopicMetadataTest extends ZooKeeperTestHarness { assertEquals(topicMetadataRequest, deserializedMetadataRequest) } + @Test def testBasicTopicMetadata { // create topic val topic = "test" @@ -87,6 +89,7 @@ class TopicMetadataTest extends ZooKeeperTestHarness { assertEquals(1, partitionMetadata.head.replicas.size) } + @Test def testGetAllTopicMetadata { // create topic val topic1 = "testGetAllTopicMetadata1" @@ -111,6 +114,7 @@ class TopicMetadataTest extends ZooKeeperTestHarness { assertEquals(1, partitionMetadataTopic2.head.replicas.size) } + @Test def testAutoCreateTopic { // auto create topic val topic = "testAutoCreateTopic" @@ -137,6 +141,7 @@ class TopicMetadataTest extends ZooKeeperTestHarness { assertTrue(partitionMetadata.head.leader.isDefined) } + @Test def testAutoCreateTopicWithCollision { // auto create topic val topic1 = "testAutoCreate_Topic" @@ -199,7 +204,7 @@ class TopicMetadataTest extends ZooKeeperTestHarness { }) } - + @Test def testIsrAfterBrokerShutDownAndJoinsBack { val numBrokers = 2 //just 2 brokers are enough for the test @@ -250,10 +255,12 @@ class TopicMetadataTest extends ZooKeeperTestHarness { } + @Test def testAliveBrokerListWithNoTopics { checkMetadata(Seq(server1), 1) } + @Test def testAliveBrokersListWithNoTopicsAfterNewBrokerStartup { var adHocServers = adHocConfigs.takeRight(adHocConfigs.size - 1).map(p => createServer(p)) @@ -267,6 +274,7 @@ class TopicMetadataTest extends ZooKeeperTestHarness { } + @Test def testAliveBrokersListWithNoTopicsAfterABrokerShutdown { val adHocServers = adHocConfigs.map(p => createServer(p)) diff --git a/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala b/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala index 28f6cc3..4dba7dc 100755 --- a/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala +++ b/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala @@ -18,7 +18,7 @@ package kafka.integration import org.apache.kafka.common.config.ConfigException -import org.junit.{After, Before} +import org.junit.{Test, After, Before} import scala.util.Random import org.apache.log4j.{Level, Logger} @@ -99,6 +99,7 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness { } } + @Test def testUncleanLeaderElectionEnabled { // unclean leader election is enabled by default startBrokers(Seq(configProps1, configProps2)) @@ -109,6 +110,7 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness { verifyUncleanLeaderElectionEnabled } + @Test def testUncleanLeaderElectionDisabled { // disable unclean leader election configProps1.put("unclean.leader.election.enable", String.valueOf(false)) @@ -121,6 +123,7 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness { verifyUncleanLeaderElectionDisabled } + @Test def testUncleanLeaderElectionEnabledByTopicOverride { // disable unclean leader election globally, but enable for our specific test topic configProps1.put("unclean.leader.election.enable", String.valueOf(false)) @@ -136,6 +139,7 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness { verifyUncleanLeaderElectionEnabled } + @Test def testCleanLeaderElectionDisabledByTopicOverride { // enable unclean leader election globally, but disable for our specific test topic configProps1.put("unclean.leader.election.enable", String.valueOf(true)) @@ -151,6 +155,7 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness { verifyUncleanLeaderElectionDisabled } + @Test def testUncleanLeaderElectionInvalidTopicOverride { startBrokers(Seq(configProps1)) 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 cf6b9a9..f4e0127 100644 --- a/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala +++ b/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala @@ -29,6 +29,7 @@ import kafka.utils.{Logging, TestUtils} import kafka.consumer.{KafkaStream, ConsumerConfig} import kafka.zk.ZooKeeperTestHarness import kafka.common.MessageStreamsExistException +import org.junit.Test import scala.collection.JavaConversions @@ -50,6 +51,7 @@ class ZookeeperConsumerConnectorTest extends KafkaServerTestHarness with ZooKeep val consumer1 = "consumer1" val nMessages = 2 + @Test def testBasic() { val requestHandlerLogger = Logger.getLogger(classOf[KafkaRequestHandler]) requestHandlerLogger.setLevel(Level.FATAL) diff --git a/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala b/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala index 066f506..7b55f79 100755 --- a/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala +++ b/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala @@ -21,7 +21,7 @@ import org.junit.Assert._ import kafka.utils.{TestUtils, CoreUtils, ZkUtils} import kafka.zk.ZooKeeperTestHarness import org.apache.kafka.common.protocol.SecurityProtocol -import org.junit.{After, Before} +import org.junit.{Test, After, Before} class AdvertiseBrokerTest extends ZooKeeperTestHarness { var server : KafkaServer = null @@ -46,7 +46,8 @@ class AdvertiseBrokerTest extends ZooKeeperTestHarness { CoreUtils.rm(server.config.logDirs) super.tearDown() } - + + @Test def testBrokerAdvertiseToZK { val brokerInfo = ZkUtils.getBrokerInfo(zkClient, brokerId) val endpoint = brokerInfo.get.endPoints.get(SecurityProtocol.PLAINTEXT).get diff --git a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala index 7f55a80..0c6d23d 100755 --- a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala +++ b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala @@ -43,6 +43,7 @@ class HighwatermarkPersistenceTest { CoreUtils.rm(dir) } + @Test def testHighWatermarkPersistenceSinglePartition() { // mock zkclient val zkClient = EasyMock.createMock(classOf[ZkClient]) @@ -78,6 +79,7 @@ class HighwatermarkPersistenceTest { replicaManager.shutdown(false) } + @Test def testHighWatermarkPersistenceMultiplePartitions() { val topic1 = "foo1" val topic2 = "foo2" diff --git a/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala b/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala index 25f0d41..977b29a 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.junit.{Before, After} +import org.junit.{Test, Before, After} import collection.mutable.HashMap import collection.mutable.Map import kafka.cluster.{Partition, Replica} @@ -59,6 +59,7 @@ class IsrExpirationTest { /* * Test the case where a follower is caught up but stops making requests to the leader. Once beyond the configured time limit, it should fall out of ISR */ + @Test def testIsrExpirationForStuckFollowers() { val log = getLogWithLogEndOffset(15L, 2) // set logEndOffset for leader to 15L @@ -89,6 +90,7 @@ class IsrExpirationTest { /* * Test the case where a follower never makes a fetch request. It should fall out of ISR because it will be declared stuck */ + @Test def testIsrExpirationIfNoFetchRequestMade() { val log = getLogWithLogEndOffset(15L, 1) // set logEndOffset for leader to 15L @@ -109,6 +111,7 @@ class IsrExpirationTest { * Test the case where a follower continually makes fetch requests but is unable to catch up. It should fall out of the ISR * However, any time it makes a request to the LogEndOffset it should be back in the ISR */ + @Test def testIsrExpirationForSlowFollowers() { // create leader replica val log = getLogWithLogEndOffset(15L, 4) diff --git a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala index f77f186..bb12a50 100755 --- a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala +++ b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala @@ -26,7 +26,7 @@ import kafka.controller.{ControllerChannelManager, ControllerContext, LeaderIsrA import kafka.utils.TestUtils._ import kafka.zk.ZooKeeperTestHarness import org.apache.kafka.common.protocol.SecurityProtocol -import org.junit.{After, Before} +import org.junit.{Test, After, Before} class LeaderElectionTest extends ZooKeeperTestHarness { val brokerId1 = 0 @@ -56,6 +56,7 @@ class LeaderElectionTest extends ZooKeeperTestHarness { super.tearDown() } + @Test def testLeaderElectionAndEpoch { // start 2 brokers val topic = "new-topic" @@ -101,6 +102,7 @@ class LeaderElectionTest extends ZooKeeperTestHarness { assertEquals("Second epoch value should be %d".format(leaderEpoch2+1) , leaderEpoch2+1, leaderEpoch3) } + @Test def testLeaderElectionWithStaleControllerEpoch() { // start 2 brokers val topic = "new-topic" diff --git a/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala b/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala index 7a0d0b2..46829b8 100755 --- a/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala @@ -27,7 +27,7 @@ import kafka.serializer.StringEncoder import java.io.File -import org.junit.{After, Before} +import org.junit.{Test, After, Before} import org.junit.Assert._ class LogRecoveryTest extends ZooKeeperTestHarness { @@ -97,6 +97,7 @@ class LogRecoveryTest extends ZooKeeperTestHarness { super.tearDown() } + @Test def testHWCheckpointNoFailuresSingleLogSegment { val numMessages = 2L sendMessages(numMessages.toInt) @@ -113,6 +114,7 @@ class LogRecoveryTest extends ZooKeeperTestHarness { assertEquals(numMessages, followerHW) } + @Test def testHWCheckpointWithFailuresSingleLogSegment { var leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId) @@ -163,6 +165,7 @@ class LogRecoveryTest extends ZooKeeperTestHarness { assertEquals(hw, hwFile2.read.getOrElse(TopicAndPartition(topic, 0), 0L)) } + @Test def testHWCheckpointNoFailuresMultipleLogSegments { sendMessages(20) val hw = 20L @@ -178,6 +181,7 @@ class LogRecoveryTest extends ZooKeeperTestHarness { assertEquals(hw, followerHW) } + @Test def testHWCheckpointWithFailuresMultipleLogSegments { var leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId) diff --git a/core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala index dead087..e40bf3b 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.junit.{After, Before} +import org.junit.{Test, After, Before} import kafka.zk.ZooKeeperTestHarness import kafka.utils.TestUtils._ import kafka.producer.KeyedMessage @@ -44,6 +44,7 @@ class ReplicaFetchTest extends ZooKeeperTestHarness { super.tearDown() } + @Test def testReplicaFetcherThread() { val partition = 0 val testMessageList1 = List("test1", "test2", "test3", "test4") diff --git a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala index 2a8da0c..102dba9 100755 --- a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala @@ -159,6 +159,7 @@ class ServerShutdownTest extends ZooKeeperTestHarness { .count(isNonDaemonKafkaThread)) } + @Test def testConsecutiveShutdown(){ val server = new KafkaServer(config) try { diff --git a/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala b/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala index 7d986ad..0adc0aa 100755 --- a/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala @@ -23,9 +23,11 @@ import kafka.utils.TestUtils import kafka.zk.ZooKeeperTestHarness import org.junit.Assert._ +import org.junit.Test class ServerStartupTest extends ZooKeeperTestHarness { + @Test def testBrokerCreatesZKChroot { val brokerId = 0 val zookeeperChroot = "/kafka-chroot-for-unittest" @@ -41,6 +43,7 @@ class ServerStartupTest extends ZooKeeperTestHarness { CoreUtils.rm(server.config.logDirs) } + @Test def testConflictBrokerRegistration { // Try starting a broker with the a conflicting broker id. // This shouldn't affect the existing broker registration. diff --git a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala index d950665..ba584a2 100644 --- a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala +++ b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala @@ -22,7 +22,7 @@ import kafka.cluster.Replica import kafka.common.TopicAndPartition import kafka.log.Log import kafka.message.{MessageSet, ByteBufferMessageSet, Message} -import org.junit.{After, Before} +import org.junit.{Test, After, Before} import java.util.{Properties, Collections} import java.util.concurrent.atomic.AtomicBoolean @@ -136,6 +136,7 @@ class SimpleFetchTest { * * This test also verifies counts of fetch requests recorded by the ReplicaManager */ + @Test def testReadFromLog() { val initialTopicCount = BrokerTopicStats.getBrokerTopicStats(topic).totalFetchRequestRate.count(); val initialAllTopicsCount = BrokerTopicStats.getBrokerAllTopicsStats().totalFetchRequestRate.count(); diff --git a/core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala b/core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala index 247aa6e..f240e89 100644 --- a/core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala +++ b/core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala @@ -20,11 +20,12 @@ package kafka.zk import kafka.consumer.ConsumerConfig import kafka.utils.ZkUtils import kafka.utils.TestUtils -import org.junit.Assert +import org.junit.{Test, Assert} class ZKEphemeralTest extends ZooKeeperTestHarness { var zkSessionTimeoutMs = 1000 + @Test def testEphemeralNodeCleanup = { val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, "test", "1")) var zkClient = ZkUtils.createZkClient(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs) diff --git a/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala b/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala index 35c635a..241eea5 100644 --- a/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala +++ b/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala @@ -21,6 +21,7 @@ import kafka.consumer.ConsumerConfig import kafka.utils.{ZkPath, TestUtils, ZkUtils} import org.apache.kafka.common.config.ConfigException import org.junit.Assert._ +import org.junit.Test class ZKPathTest extends ZooKeeperTestHarness { @@ -28,6 +29,7 @@ class ZKPathTest extends ZooKeeperTestHarness { val zkSessionTimeoutMs = 1000 def zkConnectWithInvalidRoot: String = zkConnect + "/ghost" + @Test def testCreatePersistentPathThrowsException { val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnectWithInvalidRoot, "test", "1")) @@ -43,6 +45,7 @@ class ZKPathTest extends ZooKeeperTestHarness { } } + @Test def testCreatePersistentPath { val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, "test", "1")) var zkClient = ZkUtils.createZkClient(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs) @@ -56,6 +59,7 @@ class ZKPathTest extends ZooKeeperTestHarness { assertTrue("Failed to create persistent path", ZkUtils.pathExists(zkClient, path)) } + @Test def testMakeSurePersistsPathExistsThrowsException { val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnectWithInvalidRoot, "test", "1")) @@ -71,6 +75,7 @@ class ZKPathTest extends ZooKeeperTestHarness { } } + @Test def testMakeSurePersistsPathExists { val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, "test", "1")) var zkClient = ZkUtils.createZkClient(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs) @@ -84,6 +89,7 @@ class ZKPathTest extends ZooKeeperTestHarness { assertTrue("Failed to create persistent path", ZkUtils.pathExists(zkClient, path)) } + @Test def testCreateEphemeralPathThrowsException { val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnectWithInvalidRoot, "test", "1")) @@ -99,6 +105,7 @@ class ZKPathTest extends ZooKeeperTestHarness { } } + @Test def testCreateEphemeralPathExists { val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, "test", "1")) var zkClient = ZkUtils.createZkClient(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs) @@ -112,6 +119,7 @@ class ZKPathTest extends ZooKeeperTestHarness { assertTrue("Failed to create ephemeral path", ZkUtils.pathExists(zkClient, path)) } + @Test def testCreatePersistentSequentialThrowsException { val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnectWithInvalidRoot, "test", "1")) @@ -127,6 +135,7 @@ class ZKPathTest extends ZooKeeperTestHarness { } } + @Test def testCreatePersistentSequentialExists { val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, "test", "1")) var zkClient = ZkUtils.createZkClient(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs) -- 1.7.12.4 From 786867c2e18f79fa17be120f78a253bb9822a861 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Sun, 16 Aug 2015 20:53:39 -0700 Subject: [PATCH 113/120] KAFKA-2438; add maxParallelForks to build.gradle to speedup tests. Author: Sriharsha Chintalapani Reviewers: Gwen Shapira Closes #143 from harshach/KAFKA-2438 --- build.gradle | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/build.gradle b/build.gradle index c7f66be..983587f 100644 --- a/build.gradle +++ b/build.gradle @@ -114,6 +114,10 @@ subprojects { } } + tasks.withType(Test) { + maxParallelForks = Runtime.runtime.availableProcessors() + } + jar { from '../LICENSE' from '../NOTICE' -- 1.7.12.4 From 601413c73639d6bb2c15befbb4d88fb9e023b92d Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Mon, 11 May 2015 14:50:19 -0700 Subject: [PATCH 114/120] Fixing bug --- .../kafka/clients/consumer/internals/Fetcher.java | 14 +++++ .../kafka/clients/producer/internals/Sender.java | 15 ++++++ .../org/apache/kafka/common/protocol/Protocol.java | 34 ++++++++++--- .../apache/kafka/common/requests/FetchRequest.java | 2 +- .../kafka/common/requests/FetchResponse.java | 8 ++- .../kafka/common/requests/ProduceRequest.java | 4 +- .../kafka/common/requests/ProduceResponse.java | 8 ++- .../clients/consumer/internals/FetcherTest.java | 6 +-- .../clients/producer/internals/SenderTest.java | 37 ++++++++++++-- .../kafka/common/requests/RequestResponseTest.java | 4 +- core/src/main/scala/kafka/api/FetchRequest.scala | 9 +++- core/src/main/scala/kafka/api/FetchResponse.scala | 26 +++++++--- .../src/main/scala/kafka/api/ProducerRequest.scala | 2 +- .../main/scala/kafka/api/ProducerResponse.scala | 17 +++++-- .../main/scala/kafka/consumer/SimpleConsumer.scala | 2 +- .../scala/kafka/server/AbstractFetcherThread.scala | 16 ++++-- .../src/main/scala/kafka/server/DelayedFetch.scala | 5 +- .../main/scala/kafka/server/DelayedProduce.scala | 5 +- core/src/main/scala/kafka/server/KafkaApis.scala | 59 +++++++++++++++------- .../main/scala/kafka/server/OffsetManager.scala | 2 +- .../scala/kafka/server/ReplicaFetcherThread.scala | 4 +- .../main/scala/kafka/server/ReplicaManager.scala | 13 ++--- .../api/RequestResponseSerializationTest.scala | 40 ++++++++++++++- .../unit/kafka/server/DelayedOperationTest.scala | 12 +++++ .../unit/kafka/server/ReplicaManagerTest.scala | 2 +- 25 files changed, 277 insertions(+), 69 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index 9dc6697..3c3a84b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -63,6 +63,7 @@ import java.util.Set; public class Fetcher { public static final long EARLIEST_OFFSET_TIMESTAMP = -2L; public static final long LATEST_OFFSET_TIMESTAMP = -1L; + private static final String QUOTA_DELAY_KEY_NAME = "throttle_time_ms"; private static final Logger log = LoggerFactory.getLogger(Fetcher.class); @@ -455,6 +456,7 @@ public class Fetcher { this.sensors.recordsFetched.record(totalCount); } this.sensors.fetchLatency.record(resp.requestLatencyMs()); + this.sensors.quotaDelayTimeSensor.record(resp.responseBody().getInt(QUOTA_DELAY_KEY_NAME)); } /** @@ -493,6 +495,7 @@ public class Fetcher { public final Sensor recordsFetched; public final Sensor fetchLatency; public final Sensor recordsFetchLag; + public final Sensor quotaDelayTimeSensor; public FetchManagerMetrics(Metrics metrics, String metricGrpPrefix, Map tags) { @@ -542,6 +545,17 @@ public class Fetcher { this.metricGrpName, "The maximum lag in terms of number of records for any partition in this window", tags), new Max()); + + this.quotaDelayTimeSensor = metrics.sensor("fetch-throttle-time"); + this.quotaDelayTimeSensor.add(new MetricName("throttle-time-avg", + this.metricGrpName, + "The average throttle time in ms", + tags), new Avg()); + + this.quotaDelayTimeSensor.add(new MetricName("throttle-time-max", + this.metricGrpName, + "The maximum throttle time in ms", + tags), new Max()); } public void recordTopicFetchMetrics(String topic, int bytes, int records) { 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 0baf16e..9c87879 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 @@ -55,6 +55,7 @@ import org.slf4j.LoggerFactory; public class Sender implements Runnable { private static final Logger log = LoggerFactory.getLogger(Sender.class); + private static final String QUOTA_DELAY_KEY_NAME = "throttle_time_ms"; /* the state of each nodes connection */ private final KafkaClient client; @@ -253,6 +254,8 @@ public class Sender implements Runnable { completeBatch(batch, error, partResp.baseOffset, correlationId, now); } this.sensors.recordLatency(response.request().request().destination(), response.requestLatencyMs()); + this.sensors.recordQuotaDelay(response.request().request().destination(), + response.responseBody().getInt(QUOTA_DELAY_KEY_NAME)); } else { // this is the acks = 0 case, just complete all requests for (RecordBatch batch : batches.values()) @@ -352,6 +355,7 @@ public class Sender implements Runnable { public final Sensor batchSizeSensor; public final Sensor compressionRateSensor; public final Sensor maxRecordSizeSensor; + public final Sensor quotaDelayTimeSensor; public SenderMetrics(Metrics metrics) { this.metrics = metrics; @@ -381,6 +385,12 @@ public class Sender implements Runnable { m = new MetricName("request-latency-max", metricGrpName, "The maximum request latency in ms", metricTags); this.requestTimeSensor.add(m, new Max()); + this.quotaDelayTimeSensor = metrics.sensor("produce-throttle-time"); + m = new MetricName("throttle-time-avg", metricGrpName, "The average throttle time in ms", metricTags); + this.quotaDelayTimeSensor.add(m, new Avg()); + m = new MetricName("throttle-time-max", metricGrpName, "The maximum throttle time in ms", metricTags); + this.quotaDelayTimeSensor.add(m, new Max()); + this.recordsPerRequestSensor = metrics.sensor("records-per-request"); m = new MetricName("record-send-rate", metricGrpName, "The average number of records sent per second.", metricTags); this.recordsPerRequestSensor.add(m, new Rate()); @@ -515,6 +525,11 @@ public class Sender implements Runnable { nodeRequestTime.record(latency, now); } } + + public void recordQuotaDelay(int node, long delayTimeMs) { + this.quotaDelayTimeSensor.record(delayTimeMs, time.milliseconds()); + } + } } diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java index 3dc8b01..461388e 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java @@ -107,9 +107,23 @@ public class Protocol { INT16), new Field("base_offset", INT64)))))))); - - public static final Schema[] PRODUCE_REQUEST = new Schema[] {PRODUCE_REQUEST_V0}; - public static final Schema[] PRODUCE_RESPONSE = new Schema[] {PRODUCE_RESPONSE_V0}; + // The V1 Fetch Request body is the same as V0. + // Only the version number is incremented to indicate a newer client + public static final Schema PRODUCE_REQUEST_V1 = PRODUCE_REQUEST_V0; + + public static final Schema PRODUCE_RESPONSE_V1 = new Schema(new Field("responses", + new ArrayOf(new Schema(new Field("topic", STRING), + new Field("partition_responses", + new ArrayOf(new Schema(new Field("partition", + INT32), + new Field("error_code", + INT16), + new Field("base_offset", + INT64))))))), + new Field("throttle_time_ms", INT32, "Amount of time in milliseconds the request was throttled if at all", 0)); + + public static final Schema[] PRODUCE_REQUEST = new Schema[] {PRODUCE_REQUEST_V0, PRODUCE_REQUEST_V1}; + public static final Schema[] PRODUCE_RESPONSE = new Schema[] {PRODUCE_RESPONSE_V0, PRODUCE_RESPONSE_V1}; /* Offset commit api */ public static final Schema OFFSET_COMMIT_REQUEST_PARTITION_V0 = new Schema(new Field("partition", @@ -342,6 +356,9 @@ public class Protocol { new ArrayOf(FETCH_REQUEST_TOPIC_V0), "Topics to fetch.")); + // The V1 Fetch Request body is the same as V0. + // Only the version number is incremented to indicate a newer client + public static final Schema FETCH_REQUEST_V1 = FETCH_REQUEST_V0; public static final Schema FETCH_RESPONSE_PARTITION_V0 = new Schema(new Field("partition", INT32, "Topic partition id."), @@ -357,9 +374,14 @@ public class Protocol { public static final Schema FETCH_RESPONSE_V0 = new Schema(new Field("responses", new ArrayOf(FETCH_RESPONSE_TOPIC_V0))); - - public static final Schema[] FETCH_REQUEST = new Schema[] {FETCH_REQUEST_V0}; - public static final Schema[] FETCH_RESPONSE = new Schema[] {FETCH_RESPONSE_V0}; + public static final Schema FETCH_RESPONSE_V1 = new Schema(new Field("throttle_time_ms", INT32, + "Amount of time in milliseconds the request was throttled if at all", + 0), + new Field("responses", + new ArrayOf(FETCH_RESPONSE_TOPIC_V0))); + + public static final Schema[] FETCH_REQUEST = new Schema[] {FETCH_REQUEST_V0, FETCH_REQUEST_V1}; + public static final Schema[] FETCH_RESPONSE = new Schema[] {FETCH_RESPONSE_V0, FETCH_RESPONSE_V1}; /* Consumer metadata api */ public static final Schema CONSUMER_METADATA_REQUEST_V0 = new Schema(new Field("group_id", 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 df073a0..feb4109 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 @@ -132,7 +132,7 @@ public class FetchRequest extends AbstractRequest { switch (versionId) { case 0: - return new FetchResponse(responseData); + return new FetchResponse(responseData, 0); 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))); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java index eb8951f..005ec08 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java @@ -37,6 +37,7 @@ public class FetchResponse extends AbstractRequestResponse { // topic level field names private static final String TOPIC_KEY_NAME = "topic"; private static final String PARTITIONS_KEY_NAME = "partition_responses"; + private static final String QUOTA_DELAY_KEY_NAME = "throttle_time_ms"; // partition level field names private static final String PARTITION_KEY_NAME = "partition"; @@ -59,6 +60,7 @@ public class FetchResponse extends AbstractRequestResponse { public static final ByteBuffer EMPTY_RECORD_SET = ByteBuffer.allocate(0); private final Map responseData; + private final int delayTime; public static final class PartitionData { public final short errorCode; @@ -72,8 +74,9 @@ public class FetchResponse extends AbstractRequestResponse { } } - public FetchResponse(Map responseData) { + public FetchResponse(Map responseData, int delayTime) { super(new Struct(CURRENT_SCHEMA)); + this.delayTime = delayTime; Map> topicsData = CollectionUtils.groupDataByTopic(responseData); List topicArray = new ArrayList(); @@ -94,11 +97,13 @@ public class FetchResponse extends AbstractRequestResponse { topicArray.add(topicData); } struct.set(RESPONSES_KEY_NAME, topicArray.toArray()); + struct.set(QUOTA_DELAY_KEY_NAME, delayTime); this.responseData = responseData; } public FetchResponse(Struct struct) { super(struct); + this.delayTime = struct.getInt(QUOTA_DELAY_KEY_NAME); responseData = new HashMap(); for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) { Struct topicResponse = (Struct) topicResponseObj; @@ -118,6 +123,7 @@ public class FetchResponse extends AbstractRequestResponse { public Map responseData() { return responseData; } + public int getDelayTime() { return this.delayTime; } public static FetchResponse parse(ByteBuffer buffer) { return new FetchResponse((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 715504b..5663f2c 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 @@ -27,7 +27,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -public class ProduceRequest extends AbstractRequest { +public class ProduceRequest extends AbstractRequest { private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.PRODUCE.id); private static final String ACKS_KEY_NAME = "acks"; @@ -103,7 +103,7 @@ public class ProduceRequest extends AbstractRequest { switch (versionId) { case 0: - return new ProduceResponse(responseMap); + return new ProduceResponse(responseMap, 0); 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))); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java index febfc70..8bb5bda 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java @@ -33,6 +33,7 @@ public class ProduceResponse extends AbstractRequestResponse { // topic level field names private static final String TOPIC_KEY_NAME = "topic"; private static final String PARTITION_RESPONSES_KEY_NAME = "partition_responses"; + private static final String QUOTA_DELAY_KEY_NAME = "throttle_time_ms"; // partition level field names private static final String PARTITION_KEY_NAME = "partition"; @@ -49,8 +50,9 @@ public class ProduceResponse extends AbstractRequestResponse { private static final String BASE_OFFSET_KEY_NAME = "base_offset"; private final Map responses; + private final int delayTime; - public ProduceResponse(Map responses) { + public ProduceResponse(Map responses, int delayTime) { super(new Struct(CURRENT_SCHEMA)); Map> responseByTopic = CollectionUtils.groupDataByTopic(responses); List topicDatas = new ArrayList(responseByTopic.size()); @@ -70,7 +72,9 @@ public class ProduceResponse extends AbstractRequestResponse { topicDatas.add(topicData); } struct.set(RESPONSES_KEY_NAME, topicDatas.toArray()); + struct.set(QUOTA_DELAY_KEY_NAME, delayTime); this.responses = responses; + this.delayTime = delayTime; } public ProduceResponse(Struct struct) { @@ -88,11 +92,13 @@ public class ProduceResponse extends AbstractRequestResponse { responses.put(tp, new PartitionResponse(errorCode, offset)); } } + this.delayTime = struct.getInt(QUOTA_DELAY_KEY_NAME); } public Map responses() { return this.responses; } + public int getDelayTime() { return this.delayTime; } public static final class PartitionResponse { public short errorCode; diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java index a7c83ca..c6b7feb 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -22,9 +22,11 @@ import org.apache.kafka.clients.MockClient; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.OffsetResetStrategy; import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.MetricName; import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.metrics.KafkaMetric; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.types.Struct; @@ -323,9 +325,7 @@ public class FetcherTest { } private Struct fetchResponse(ByteBuffer buffer, short error, long hw) { - FetchResponse response = new FetchResponse(Collections.singletonMap(tp, new FetchResponse.PartitionData(error, hw, buffer))); + FetchResponse response = new FetchResponse(Collections.singletonMap(tp, new FetchResponse.PartitionData(error, hw, buffer)), 0); return response.toStruct(); } - - } diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java index 8b1805d..2cc9e50 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java @@ -26,7 +26,9 @@ import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.MockClient; import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.MetricName; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.metrics.KafkaMetric; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.types.Struct; @@ -76,7 +78,7 @@ public class SenderTest { sender.run(time.milliseconds()); // connect sender.run(time.milliseconds()); // send produce request assertEquals("We should have a single produce request in flight.", 1, client.inFlightRequestCount()); - client.respond(produceResponse(tp, offset, Errors.NONE.code())); + client.respond(produceResponse(tp, offset, Errors.NONE.code(), 0)); sender.run(time.milliseconds()); assertEquals("All requests completed.", offset, (long) client.inFlightRequestCount()); sender.run(time.milliseconds()); @@ -84,6 +86,33 @@ public class SenderTest { assertEquals(offset, future.get().offset()); } + /* + * Send multiple request. Verify that the client side quota metrics have the right values + */ + @Test + public void testQuotaMetrics() throws Exception { + final long offset = 0; + for (int i = 1; i <= 3; i++) { + Future future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), null).future; + sender.run(time.milliseconds()); // send produce request + client.respond(produceResponse(tp, offset, Errors.NONE.code(), 100 * i)); + sender.run(time.milliseconds()); + } + long avg = 0; + long max = 0; + + Map allMetrics = metrics.metrics(); + for (MetricName m : allMetrics.keySet()) { + if (m.name().equals("throttle-time-avg")) { + avg = (long) allMetrics.get(m).value(); + } else if (m.name().equals("throttle-time-max")) { + max = (long) allMetrics.get(m).value(); + } + } + assertEquals(200, avg); + assertEquals(300, max); + } + @Test public void testRetries() throws Exception { // create a sender with retries = 1 @@ -110,7 +139,7 @@ public class SenderTest { sender.run(time.milliseconds()); // resend assertEquals(1, client.inFlightRequestCount()); long offset = 0; - client.respond(produceResponse(tp, offset, Errors.NONE.code())); + client.respond(produceResponse(tp, offset, Errors.NONE.code(), 0)); sender.run(time.milliseconds()); assertTrue("Request should have retried and completed", future.isDone()); assertEquals(offset, future.get().offset()); @@ -138,10 +167,10 @@ public class SenderTest { } } - private Struct produceResponse(TopicPartition tp, long offset, int error) { + private Struct produceResponse(TopicPartition tp, long offset, int error, int delayTimeMs) { ProduceResponse.PartitionResponse resp = new ProduceResponse.PartitionResponse((short) error, offset); Map partResp = Collections.singletonMap(tp, resp); - ProduceResponse response = new ProduceResponse(partResp); + ProduceResponse response = new ProduceResponse(partResp, delayTimeMs); return response.toStruct(); } 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 8b2aca8..0645bc8 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 @@ -103,7 +103,7 @@ public class RequestResponseTest { private AbstractRequestResponse createFetchResponse() { Map responseData = new HashMap(); responseData.put(new TopicPartition("test", 0), new FetchResponse.PartitionData(Errors.NONE.code(), 1000000, ByteBuffer.allocate(10))); - return new FetchResponse(responseData); + return new FetchResponse(responseData, 0); } private AbstractRequest createHeartBeatRequest() { @@ -182,6 +182,6 @@ public class RequestResponseTest { private AbstractRequestResponse createProduceResponse() { Map responseData = new HashMap(); responseData.put(new TopicPartition("test", 0), new ProduceResponse.PartitionResponse(Errors.NONE.code(), 10000)); - return new ProduceResponse(responseData); + return new ProduceResponse(responseData, 0); } } diff --git a/core/src/main/scala/kafka/api/FetchRequest.scala b/core/src/main/scala/kafka/api/FetchRequest.scala index 5b38f85..36e288f 100644 --- a/core/src/main/scala/kafka/api/FetchRequest.scala +++ b/core/src/main/scala/kafka/api/FetchRequest.scala @@ -31,7 +31,7 @@ import scala.collection.immutable.Map case class PartitionFetchInfo(offset: Long, fetchSize: Int) object FetchRequest { - val CurrentVersion = 0.shortValue + val CurrentVersion = 1.shortValue val DefaultMaxWait = 0 val DefaultMinBytes = 0 val DefaultCorrelationId = 0 @@ -170,7 +170,7 @@ case class FetchRequest(versionId: Short = FetchRequest.CurrentVersion, @nonthreadsafe class FetchRequestBuilder() { private val correlationId = new AtomicInteger(0) - private val versionId = FetchRequest.CurrentVersion + private var versionId = FetchRequest.CurrentVersion private var clientId = ConsumerConfig.DefaultClientId private var replicaId = Request.OrdinaryConsumerId private var maxWait = FetchRequest.DefaultMaxWait @@ -205,6 +205,11 @@ class FetchRequestBuilder() { this } + def requestVersion(versionId: Short): FetchRequestBuilder = { + this.versionId = versionId + this + } + def build() = { val fetchRequest = FetchRequest(versionId, correlationId.getAndIncrement, clientId, replicaId, maxWait, minBytes, requestMap.toMap) requestMap.clear() diff --git a/core/src/main/scala/kafka/api/FetchResponse.scala b/core/src/main/scala/kafka/api/FetchResponse.scala index 0b6b33a..7881a65 100644 --- a/core/src/main/scala/kafka/api/FetchResponse.scala +++ b/core/src/main/scala/kafka/api/FetchResponse.scala @@ -147,8 +147,12 @@ object FetchResponse { 4 + /* correlationId */ 4 /* topic count */ - def readFrom(buffer: ByteBuffer): FetchResponse = { + + // The request version is used to determine which fields we can expect in the response + def readFrom(buffer: ByteBuffer, requestVersion: Int): FetchResponse = { val correlationId = buffer.getInt + val delayTime = if (requestVersion > 0) buffer.getInt else 0 + val topicCount = buffer.getInt val pairs = (1 to topicCount).flatMap(_ => { val topicData = TopicData.readFrom(buffer) @@ -157,20 +161,23 @@ object FetchResponse { (TopicAndPartition(topicData.topic, partitionId), partitionData) } }) - FetchResponse(correlationId, Map(pairs:_*)) + FetchResponse(correlationId, Map(pairs:_*), requestVersion, delayTime) } } -case class FetchResponse(correlationId: Int, data: Map[TopicAndPartition, FetchResponsePartitionData]) +case class FetchResponse(correlationId: Int, + data: Map[TopicAndPartition, FetchResponsePartitionData], + requestVersion : Int = 0, + delayTime : Int = 0) extends RequestOrResponse() { /** * Partitions the data into a map of maps (one for each topic). */ lazy val dataGroupedByTopic = data.groupBy(_._1.topic) - + val delayTimeSize = if(requestVersion > 0) 4 else 0 val sizeInBytes = - FetchResponse.headerSize + + FetchResponse.headerSize + delayTimeSize + dataGroupedByTopic.foldLeft(0) ((folded, curr) => { val topicData = TopicData(curr._1, curr._2.map { case (topicAndPartition, partitionData) => (topicAndPartition.partition, partitionData) @@ -220,10 +227,17 @@ class FetchResponseSend(val dest: String, val fetchResponse: FetchResponse) exte override def destination = dest - private val buffer = ByteBuffer.allocate(4 /* for size */ + FetchResponse.headerSize) + // The delayTimeSize will be 0 if the request was made from a client sending a V0 style request + private val buffer = ByteBuffer.allocate(4 /* for size */ + FetchResponse.headerSize + fetchResponse.delayTimeSize) buffer.putInt(payloadSize) buffer.putInt(fetchResponse.correlationId) + // Include the delayTime only if the client can read it + if(fetchResponse.requestVersion > 0) { + buffer.putInt(fetchResponse.delayTime) + } + buffer.putInt(fetchResponse.dataGroupedByTopic.size) // topic count + buffer.rewind() private val sends = new MultiSend(dest, JavaConversions.seqAsJavaList(fetchResponse.dataGroupedByTopic.toList.map { diff --git a/core/src/main/scala/kafka/api/ProducerRequest.scala b/core/src/main/scala/kafka/api/ProducerRequest.scala index c866180..7fb143e 100644 --- a/core/src/main/scala/kafka/api/ProducerRequest.scala +++ b/core/src/main/scala/kafka/api/ProducerRequest.scala @@ -26,7 +26,7 @@ import kafka.network.{RequestOrResponseSend, RequestChannel} import kafka.network.RequestChannel.Response object ProducerRequest { - val CurrentVersion = 0.shortValue + val CurrentVersion = 1.shortValue def readFrom(buffer: ByteBuffer): ProducerRequest = { val versionId: Short = buffer.getShort diff --git a/core/src/main/scala/kafka/api/ProducerResponse.scala b/core/src/main/scala/kafka/api/ProducerResponse.scala index 5d1fac4..0f40a65 100644 --- a/core/src/main/scala/kafka/api/ProducerResponse.scala +++ b/core/src/main/scala/kafka/api/ProducerResponse.scala @@ -37,13 +37,17 @@ object ProducerResponse { }) }) - ProducerResponse(correlationId, Map(statusPairs:_*)) + val delayTime = buffer.getInt + ProducerResponse(correlationId, Map(statusPairs:_*), ProducerRequest.CurrentVersion, delayTime) } } case class ProducerResponseStatus(var error: Short, offset: Long) -case class ProducerResponse(correlationId: Int, status: Map[TopicAndPartition, ProducerResponseStatus]) +case class ProducerResponse(correlationId: Int, + status: Map[TopicAndPartition, ProducerResponseStatus], + requestVersion : Int = 0, + delayTime : Int = 0) extends RequestOrResponse() { /** @@ -54,6 +58,7 @@ case class ProducerResponse(correlationId: Int, status: Map[TopicAndPartition, P def hasError = status.values.exists(_.error != ErrorMapping.NoError) val sizeInBytes = { + val delayTimeSize = if(requestVersion > 0) 4 else 0 val groupedStatus = statusGroupedByTopic 4 + /* correlation id */ 4 + /* topic count */ @@ -66,7 +71,9 @@ case class ProducerResponse(correlationId: Int, status: Map[TopicAndPartition, P 2 + /* error code */ 8 /* offset */ } - }) + }) + + delayTimeSize + } def writeTo(buffer: ByteBuffer) { @@ -85,6 +92,10 @@ case class ProducerResponse(correlationId: Int, status: Map[TopicAndPartition, P buffer.putLong(nextOffset) } }) + // Delay time is only supported on V1 style requests + if(requestVersion > 0) { + buffer.putInt(delayTime) + } } override def describe(details: Boolean):String = { toString } diff --git a/core/src/main/scala/kafka/consumer/SimpleConsumer.scala b/core/src/main/scala/kafka/consumer/SimpleConsumer.scala index 7ebc040..4e1833a 100644 --- a/core/src/main/scala/kafka/consumer/SimpleConsumer.scala +++ b/core/src/main/scala/kafka/consumer/SimpleConsumer.scala @@ -131,7 +131,7 @@ class SimpleConsumer(val host: String, response = sendRequest(request) } } - val fetchResponse = FetchResponse.readFrom(response.payload()) + val fetchResponse = FetchResponse.readFrom(response.payload(), request.versionId) val fetchedSize = fetchResponse.sizeInBytes fetchRequestAndResponseStats.getFetchRequestAndResponseStats(host, port).requestSizeHist.update(fetchedSize) fetchRequestAndResponseStats.getFetchRequestAndResponseAllBrokersStats.requestSizeHist.update(fetchedSize) diff --git a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala index f843061..dca975c 100755 --- a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala @@ -37,8 +37,17 @@ import com.yammer.metrics.core.Gauge /** * Abstract class for fetching data from multiple partitions from the same broker. */ -abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroker: BrokerEndPoint, socketTimeout: Int, socketBufferSize: Int, - fetchSize: Int, fetcherBrokerId: Int = -1, maxWait: Int = 0, minBytes: Int = 1, fetchBackOffMs: Int = 0, +abstract class AbstractFetcherThread(name: String, + clientId: String, + sourceBroker: BrokerEndPoint, + socketTimeout: Int, + socketBufferSize: Int, + fetchSize: Int, + fetcherBrokerId: Int = -1, + maxWait: Int = 0, + minBytes: Int = 1, + fetchBackOffMs: Int = 0, + fetchRequestVersion: Short = FetchRequest.CurrentVersion, isInterruptible: Boolean = true) extends ShutdownableThread(name, isInterruptible) { private val partitionMap = new mutable.HashMap[TopicAndPartition, PartitionFetchState] // a (topic, partition) -> partitionFetchState map @@ -52,7 +61,8 @@ abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroke clientId(clientId). replicaId(fetcherBrokerId). maxWait(maxWait). - minBytes(minBytes) + minBytes(minBytes). + requestVersion(fetchRequestVersion) /* callbacks to be defined in subclass */ diff --git a/core/src/main/scala/kafka/server/DelayedFetch.scala b/core/src/main/scala/kafka/server/DelayedFetch.scala index de6cf5b..8e5cb39 100644 --- a/core/src/main/scala/kafka/server/DelayedFetch.scala +++ b/core/src/main/scala/kafka/server/DelayedFetch.scala @@ -55,7 +55,7 @@ case class FetchMetadata(fetchMinBytes: Int, class DelayedFetch(delayMs: Long, fetchMetadata: FetchMetadata, replicaManager: ReplicaManager, - responseCallback: Map[TopicAndPartition, FetchResponsePartitionData] => Unit) + responseCallback: (Map[TopicAndPartition, FetchResponsePartitionData], Int) => Unit) extends DelayedOperation(delayMs) { /** @@ -131,7 +131,8 @@ class DelayedFetch(delayMs: Long, val fetchPartitionData = logReadResults.mapValues(result => FetchResponsePartitionData(result.errorCode, result.hw, result.info.messageSet)) - responseCallback(fetchPartitionData) + // Zero delay time until quotas are enforced + responseCallback(fetchPartitionData, 0) } } diff --git a/core/src/main/scala/kafka/server/DelayedProduce.scala b/core/src/main/scala/kafka/server/DelayedProduce.scala index 05078b2..7a9c30b 100644 --- a/core/src/main/scala/kafka/server/DelayedProduce.scala +++ b/core/src/main/scala/kafka/server/DelayedProduce.scala @@ -53,7 +53,7 @@ case class ProduceMetadata(produceRequiredAcks: Short, class DelayedProduce(delayMs: Long, produceMetadata: ProduceMetadata, replicaManager: ReplicaManager, - responseCallback: Map[TopicAndPartition, ProducerResponseStatus] => Unit) + responseCallback: (Map[TopicAndPartition, ProducerResponseStatus], Int) => Unit) extends DelayedOperation(delayMs) { // first update the acks pending variable according to the error code @@ -126,7 +126,8 @@ class DelayedProduce(delayMs: Long, */ override def onComplete() { val responseStatus = produceMetadata.produceStatus.mapValues(status => status.responseStatus) - responseCallback(responseStatus) + // Zero delay time until quotas are enforced + responseCallback(responseStatus, 0) } } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 67f0cad..07896d0 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -257,40 +257,59 @@ class KafkaApis(val requestChannel: RequestChannel, val numBytesAppended = produceRequest.sizeInBytes // the callback for sending a produce response - def sendResponseCallback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus]) { + def sendResponseCallback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus], delayTime : Int) + { var errorInResponse = false - responseStatus.foreach { case (topicAndPartition, status) => + responseStatus.foreach + { case (topicAndPartition, status) => // we only print warnings for known errors here; if it is unknown, it will cause // an error message in the replica manager - if (status.error != ErrorMapping.NoError && status.error != ErrorMapping.UnknownCode) { - debug("Produce request with correlation id %d from client %s on partition %s failed due to %s" - .format(produceRequest.correlationId, produceRequest.clientId, - topicAndPartition, ErrorMapping.exceptionNameFor(status.error))) + if (status.error != ErrorMapping.NoError && status.error != ErrorMapping.UnknownCode) + { + debug("Produce request with correlation id %d from client %s on partition %s failed due to %s".format( + produceRequest.correlationId, + produceRequest.clientId, + topicAndPartition, + ErrorMapping.exceptionNameFor(status.error))) errorInResponse = true } } - def produceResponseCallback { - if (produceRequest.requiredAcks == 0) { + def produceResponseCallback + { + if (produceRequest.requiredAcks == 0) + { // no operation needed if producer request.required.acks = 0; however, if there is any error in handling // the request, since no response is expected by the producer, the server will close socket server so that // the producer client will know that some error has happened and will refresh its metadata - if (errorInResponse) { + if (errorInResponse) + { info( "Close connection due to error handling produce request with correlation id %d from client id %s with ack=0".format( produceRequest.correlationId, produceRequest.clientId)) requestChannel.closeConnection(request.processor, request) - } else { + } + else + { requestChannel.noOperation(request.processor, request) } - } else { - val response = ProducerResponse(produceRequest.correlationId, responseStatus) - requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, response))) } - } + else + { + val response = ProducerResponse(produceRequest.correlationId, + responseStatus, + produceRequest.versionId, + delayTime) + requestChannel.sendResponse(new RequestChannel.Response(request, + new RequestOrResponseSend(request.connectionId, + response))) + } - quotaManagers(RequestKeys.ProduceKey).recordAndMaybeThrottle(produceRequest.clientId, numBytesAppended, produceResponseCallback) + quotaManagers(RequestKeys.ProduceKey).recordAndMaybeThrottle(produceRequest.clientId, + numBytesAppended, + produceResponseCallback) + } } // only allow appending to internal topic partitions @@ -318,7 +337,7 @@ class KafkaApis(val requestChannel: RequestChannel, val fetchRequest = request.requestObj.asInstanceOf[FetchRequest] // the callback for sending a fetch response - def sendResponseCallback(responsePartitionData: Map[TopicAndPartition, FetchResponsePartitionData]) { + def sendResponseCallback(responsePartitionData: Map[TopicAndPartition, FetchResponsePartitionData], delayTime : Int) { responsePartitionData.foreach { case (topicAndPartition, data) => // we only print warnings for known errors here; if it is unknown, it will cause // an error message in the replica manager already and hence can be ignored here @@ -332,9 +351,9 @@ class KafkaApis(val requestChannel: RequestChannel, BrokerTopicStats.getBrokerAllTopicsStats().bytesOutRate.mark(data.messages.sizeInBytes) } - val response = FetchResponse(fetchRequest.correlationId, responsePartitionData) + val rr = FetchResponse(fetchRequest.correlationId, responsePartitionData) def fetchResponseCallback { - requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(request.connectionId, response))) + requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(request.connectionId, rr))) } // Do not throttle replication traffic @@ -343,11 +362,13 @@ class KafkaApis(val requestChannel: RequestChannel, } else { quotaManagers.get(RequestKeys.FetchKey) match { case Some(quotaManager) => - quotaManager.recordAndMaybeThrottle(fetchRequest.clientId, response.sizeInBytes, fetchResponseCallback) + quotaManager.recordAndMaybeThrottle(fetchRequest.clientId, rr.sizeInBytes, fetchResponseCallback) case None => warn("Cannot throttle Api key %s".format(RequestKeys.nameForKey(RequestKeys.FetchKey))) } } + val response = FetchResponse(fetchRequest.correlationId, responsePartitionData, fetchRequest.versionId, delayTime) + requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(request.connectionId, response))) } // call the replica manager to fetch messages from the local replica diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala index 0e613e7..4c5f4be 100755 --- a/core/src/main/scala/kafka/server/OffsetManager.scala +++ b/core/src/main/scala/kafka/server/OffsetManager.scala @@ -231,7 +231,7 @@ class OffsetManager(val config: OffsetManagerConfig, new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, messages:_*)) // set the callback function to insert offsets into cache after log append completed - def putCacheCallback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus]) { + def putCacheCallback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus], delayTime : Int) { // the append response should only contain the topics partition if (responseStatus.size != 1 || ! responseStatus.contains(offsetTopicPartition)) throw new IllegalStateException("Append status %s should only have one partition %s" diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index fae22d2..711d749 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -21,7 +21,7 @@ import kafka.admin.AdminUtils import kafka.cluster.BrokerEndPoint import kafka.log.LogConfig import kafka.message.ByteBufferMessageSet -import kafka.api.{OffsetRequest, FetchResponsePartitionData} +import kafka.api.{KAFKA_083, OffsetRequest, FetchResponsePartitionData} import kafka.common.{KafkaStorageException, TopicAndPartition} class ReplicaFetcherThread(name:String, @@ -38,6 +38,8 @@ class ReplicaFetcherThread(name:String, maxWait = brokerConfig.replicaFetchWaitMaxMs, minBytes = brokerConfig.replicaFetchMinBytes, fetchBackOffMs = brokerConfig.replicaFetchBackoffMs, + fetchRequestVersion = + if (brokerConfig.interBrokerProtocolVersion.onOrAfter(KAFKA_083)) 1 else 0, isInterruptible = false) { // process fetched data diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index d829e18..0426f11 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -301,10 +301,9 @@ class ReplicaManager(val config: KafkaConfig, requiredAcks: Short, internalTopicsAllowed: Boolean, messagesPerPartition: Map[TopicAndPartition, MessageSet], - responseCallback: Map[TopicAndPartition, ProducerResponseStatus] => Unit) { + responseCallback: (Map[TopicAndPartition, ProducerResponseStatus], Int) => Unit) { if (isValidRequiredAcks(requiredAcks)) { - val sTime = SystemTime.milliseconds val localProduceResults = appendToLocalLog(internalTopicsAllowed, messagesPerPartition, requiredAcks) debug("Produce to local log in %d ms".format(SystemTime.milliseconds - sTime)) @@ -332,7 +331,8 @@ class ReplicaManager(val config: KafkaConfig, } else { // we can respond immediately val produceResponseStatus = produceStatus.mapValues(status => status.responseStatus) - responseCallback(produceResponseStatus) + // Zero delay time until quotas are enforced + responseCallback(produceResponseStatus, 0) } } else { // If required.acks is outside accepted range, something is wrong with the client @@ -343,7 +343,7 @@ class ReplicaManager(val config: KafkaConfig, ProducerResponseStatus(Errors.INVALID_REQUIRED_ACKS.code, LogAppendInfo.UnknownLogAppendInfo.firstOffset)) } - responseCallback(responseStatus) + responseCallback(responseStatus, 0) } } @@ -440,7 +440,7 @@ class ReplicaManager(val config: KafkaConfig, replicaId: Int, fetchMinBytes: Int, fetchInfo: immutable.Map[TopicAndPartition, PartitionFetchInfo], - responseCallback: Map[TopicAndPartition, FetchResponsePartitionData] => Unit) { + responseCallback: (Map[TopicAndPartition, FetchResponsePartitionData], Int) => Unit) { val isFromFollower = replicaId >= 0 val fetchOnlyFromLeader: Boolean = replicaId != Request.DebuggingConsumerId val fetchOnlyCommitted: Boolean = ! Request.isValidBrokerId(replicaId) @@ -465,7 +465,8 @@ class ReplicaManager(val config: KafkaConfig, if(timeout <= 0 || fetchInfo.size <= 0 || bytesReadable >= fetchMinBytes || errorReadingData) { val fetchPartitionData = logReadResults.mapValues(result => FetchResponsePartitionData(result.errorCode, result.hw, result.info.messageSet)) - responseCallback(fetchPartitionData) + // Zero delay time until quotas are enforced + responseCallback(fetchPartitionData, 0) } else { // construct the fetch results from the read results val fetchPartitionStatus = logReadResults.map { case (topicAndPartition, result) => diff --git a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala index b4c2a22..adfd570 100644 --- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala +++ b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala @@ -18,9 +18,12 @@ package kafka.api +import java.nio.channels.GatheringByteChannel + import kafka.cluster.{BrokerEndPoint, EndPoint, Broker} import kafka.common.{OffsetAndMetadata, ErrorMapping, OffsetMetadataAndError} import kafka.common._ +import kafka.consumer.FetchRequestAndResponseStatsRegistry import kafka.message.{Message, ByteBufferMessageSet} import kafka.utils.SystemTime @@ -150,7 +153,7 @@ object SerializationTestUtils { ProducerResponse(1, Map( TopicAndPartition(topic1, 0) -> ProducerResponseStatus(0.toShort, 10001), TopicAndPartition(topic2, 0) -> ProducerResponseStatus(0.toShort, 20001) - )) + ), ProducerRequest.CurrentVersion, 100) def createTestFetchRequest: FetchRequest = { new FetchRequest(requestInfo = requestInfos) @@ -304,4 +307,39 @@ class RequestResponseSerializationTest extends JUnitSuite { assertEquals("The original and deserialized for " + original.getClass.getSimpleName + " should be the same.", original, deserialized) } } + + @Test + def testProduceResponseVersion() { + val oldClientResponse = ProducerResponse(1, Map( + TopicAndPartition("t1", 0) -> ProducerResponseStatus(0.toShort, 10001), + TopicAndPartition("t2", 0) -> ProducerResponseStatus(0.toShort, 20001) + )) + + val newClientResponse = ProducerResponse(1, Map( + TopicAndPartition("t1", 0) -> ProducerResponseStatus(0.toShort, 10001), + TopicAndPartition("t2", 0) -> ProducerResponseStatus(0.toShort, 20001) + ), 1, 100) + + // new response should have 4 bytes more than the old response since delayTime is an INT32 + assertEquals(oldClientResponse.sizeInBytes + 4, newClientResponse.sizeInBytes) + + val buffer = ByteBuffer.allocate(newClientResponse.sizeInBytes) + newClientResponse.writeTo(buffer) + buffer.rewind() + assertEquals(ProducerResponse.readFrom(buffer).delayTime, 100) + } + + @Test + def testFetchResponseVersion() { + val oldClientResponse = FetchResponse(1, Map( + TopicAndPartition("t1", 0) -> new FetchResponsePartitionData(messages = new ByteBufferMessageSet(new Message("first message".getBytes))) + ), 0) + + val newClientResponse = FetchResponse(1, Map( + TopicAndPartition("t1", 0) -> new FetchResponsePartitionData(messages = new ByteBufferMessageSet(new Message("first message".getBytes))) + ), 1, 100) + + // new response should have 4 bytes more than the old response since delayTime is an INT32 + assertEquals(oldClientResponse.sizeInBytes + 4, newClientResponse.sizeInBytes) + } } diff --git a/core/src/test/scala/unit/kafka/server/DelayedOperationTest.scala b/core/src/test/scala/unit/kafka/server/DelayedOperationTest.scala index df8d5b1..b722ce6 100644 --- a/core/src/test/scala/unit/kafka/server/DelayedOperationTest.scala +++ b/core/src/test/scala/unit/kafka/server/DelayedOperationTest.scala @@ -67,6 +67,18 @@ class DelayedOperationTest { } @Test + def testTimeoutNoKeys() { + val expiration = 20L + val r1 = new MockDelayedOperation(expiration) + val start = System.currentTimeMillis + purgatory.tryCompleteElseWatch(r1, Seq("a1")) + r1.awaitExpiration() + val elapsed = System.currentTimeMillis - start + assertTrue("r1 completed due to expiration", r1.isCompleted()) + assertTrue("Time for expiration %d should at least %d".format(elapsed, expiration), elapsed >= expiration) + } + + @Test def testRequestPurge() { val r1 = new MockDelayedOperation(100000L) val r2 = new MockDelayedOperation(100000L) diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 3770cb4..2630e71 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -77,7 +77,7 @@ class ReplicaManagerTest { val time: MockTime = new MockTime() val rm = new ReplicaManager(config, time, zkClient, new MockScheduler(time), mockLogMgr, new AtomicBoolean(false)) val produceRequest = new ProducerRequest(1, "client 1", 3, 1000, SerializationTestUtils.topicDataProducerRequest) - def callback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus]) = { + def callback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus], delayTime : Int) = { assert(responseStatus.values.head.error == Errors.INVALID_REQUIRED_ACKS.code) } -- 1.7.12.4 From 2d08980d9ba3264c65d69505f9c715b8d8edf4aa Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Mon, 8 Jun 2015 17:39:16 -0700 Subject: [PATCH 115/120] Addressing Joel's comments --- .../kafka/clients/producer/internals/Sender.java | 18 +++++------ .../org/apache/kafka/common/protocol/Protocol.java | 14 ++++++--- .../kafka/common/requests/FetchResponse.java | 14 ++++----- .../kafka/common/requests/ProduceResponse.java | 14 ++++----- .../clients/consumer/internals/FetcherTest.java | 4 +-- .../clients/producer/internals/SenderTest.java | 6 ++-- core/src/main/scala/kafka/api/FetchResponse.scala | 36 +++++++++++++++------- .../main/scala/kafka/api/ProducerResponse.scala | 17 +++++----- .../api/RequestResponseSerializationTest.scala | 2 +- .../unit/kafka/server/DelayedOperationTest.scala | 12 -------- 10 files changed, 71 insertions(+), 66 deletions(-) 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 9c87879..49a4329 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 @@ -55,7 +55,7 @@ import org.slf4j.LoggerFactory; public class Sender implements Runnable { private static final Logger log = LoggerFactory.getLogger(Sender.class); - private static final String QUOTA_DELAY_KEY_NAME = "throttle_time_ms"; + private static final String THROTTLE_TIME_KEY_NAME = "throttle_time_ms"; /* the state of each nodes connection */ private final KafkaClient client; @@ -254,8 +254,8 @@ public class Sender implements Runnable { completeBatch(batch, error, partResp.baseOffset, correlationId, now); } this.sensors.recordLatency(response.request().request().destination(), response.requestLatencyMs()); - this.sensors.recordQuotaDelay(response.request().request().destination(), - response.responseBody().getInt(QUOTA_DELAY_KEY_NAME)); + this.sensors.recordThrottleTime(response.request().request().destination(), + response.responseBody().getInt(THROTTLE_TIME_KEY_NAME)); } else { // this is the acks = 0 case, just complete all requests for (RecordBatch batch : batches.values()) @@ -355,7 +355,7 @@ public class Sender implements Runnable { public final Sensor batchSizeSensor; public final Sensor compressionRateSensor; public final Sensor maxRecordSizeSensor; - public final Sensor quotaDelayTimeSensor; + public final Sensor throttleTimeSensor; public SenderMetrics(Metrics metrics) { this.metrics = metrics; @@ -385,11 +385,11 @@ public class Sender implements Runnable { m = new MetricName("request-latency-max", metricGrpName, "The maximum request latency in ms", metricTags); this.requestTimeSensor.add(m, new Max()); - this.quotaDelayTimeSensor = metrics.sensor("produce-throttle-time"); + this.throttleTimeSensor = metrics.sensor("produce-throttle-time"); m = new MetricName("throttle-time-avg", metricGrpName, "The average throttle time in ms", metricTags); - this.quotaDelayTimeSensor.add(m, new Avg()); + this.throttleTimeSensor.add(m, new Avg()); m = new MetricName("throttle-time-max", metricGrpName, "The maximum throttle time in ms", metricTags); - this.quotaDelayTimeSensor.add(m, new Max()); + this.throttleTimeSensor.add(m, new Max()); this.recordsPerRequestSensor = metrics.sensor("records-per-request"); m = new MetricName("record-send-rate", metricGrpName, "The average number of records sent per second.", metricTags); @@ -526,8 +526,8 @@ public class Sender implements Runnable { } } - public void recordQuotaDelay(int node, long delayTimeMs) { - this.quotaDelayTimeSensor.record(delayTimeMs, time.milliseconds()); + public void recordThrottleTime(int node, long throttleTimeMs) { + this.throttleTimeSensor.record(throttleTimeMs, time.milliseconds()); } } diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java index 461388e..048d761 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java @@ -107,8 +107,6 @@ public class Protocol { INT16), new Field("base_offset", INT64)))))))); - // The V1 Fetch Request body is the same as V0. - // Only the version number is incremented to indicate a newer client public static final Schema PRODUCE_REQUEST_V1 = PRODUCE_REQUEST_V0; public static final Schema PRODUCE_RESPONSE_V1 = new Schema(new Field("responses", @@ -120,7 +118,11 @@ public class Protocol { INT16), new Field("base_offset", INT64))))))), - new Field("throttle_time_ms", INT32, "Amount of time in milliseconds the request was throttled if at all", 0)); + new Field("throttle_time_ms", + INT32, + "Duration in milliseconds for which the request was throttled" + + " due to quota violation. (Zero if the request did not violate any quota.)", + 0)); public static final Schema[] PRODUCE_REQUEST = new Schema[] {PRODUCE_REQUEST_V0, PRODUCE_REQUEST_V1}; public static final Schema[] PRODUCE_RESPONSE = new Schema[] {PRODUCE_RESPONSE_V0, PRODUCE_RESPONSE_V1}; @@ -374,8 +376,10 @@ public class Protocol { public static final Schema FETCH_RESPONSE_V0 = new Schema(new Field("responses", new ArrayOf(FETCH_RESPONSE_TOPIC_V0))); - public static final Schema FETCH_RESPONSE_V1 = new Schema(new Field("throttle_time_ms", INT32, - "Amount of time in milliseconds the request was throttled if at all", + public static final Schema FETCH_RESPONSE_V1 = new Schema(new Field("throttle_time_ms", + INT32, + "Duration in milliseconds for which the request was throttled" + + " due to quota violation. (Zero if the request did not violate any quota.)", 0), new Field("responses", new ArrayOf(FETCH_RESPONSE_TOPIC_V0))); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java index 005ec08..1ef3db7 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java @@ -37,7 +37,7 @@ public class FetchResponse extends AbstractRequestResponse { // topic level field names private static final String TOPIC_KEY_NAME = "topic"; private static final String PARTITIONS_KEY_NAME = "partition_responses"; - private static final String QUOTA_DELAY_KEY_NAME = "throttle_time_ms"; + private static final String THROTTLE_TIME_KEY_NAME = "throttle_time_ms"; // partition level field names private static final String PARTITION_KEY_NAME = "partition"; @@ -60,7 +60,7 @@ public class FetchResponse extends AbstractRequestResponse { public static final ByteBuffer EMPTY_RECORD_SET = ByteBuffer.allocate(0); private final Map responseData; - private final int delayTime; + private final int throttleTime; public static final class PartitionData { public final short errorCode; @@ -74,9 +74,9 @@ public class FetchResponse extends AbstractRequestResponse { } } - public FetchResponse(Map responseData, int delayTime) { + public FetchResponse(Map responseData, int throttleTime) { super(new Struct(CURRENT_SCHEMA)); - this.delayTime = delayTime; + this.throttleTime = throttleTime; Map> topicsData = CollectionUtils.groupDataByTopic(responseData); List topicArray = new ArrayList(); @@ -97,13 +97,13 @@ public class FetchResponse extends AbstractRequestResponse { topicArray.add(topicData); } struct.set(RESPONSES_KEY_NAME, topicArray.toArray()); - struct.set(QUOTA_DELAY_KEY_NAME, delayTime); + struct.set(THROTTLE_TIME_KEY_NAME, throttleTime); this.responseData = responseData; } public FetchResponse(Struct struct) { super(struct); - this.delayTime = struct.getInt(QUOTA_DELAY_KEY_NAME); + this.throttleTime = struct.getInt(THROTTLE_TIME_KEY_NAME); responseData = new HashMap(); for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) { Struct topicResponse = (Struct) topicResponseObj; @@ -123,7 +123,7 @@ public class FetchResponse extends AbstractRequestResponse { public Map responseData() { return responseData; } - public int getDelayTime() { return this.delayTime; } + public int getThrottleTime() { return this.throttleTime; } public static FetchResponse parse(ByteBuffer buffer) { return new FetchResponse((Struct) CURRENT_SCHEMA.read(buffer)); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java index 8bb5bda..d4a3e82 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java @@ -33,7 +33,7 @@ public class ProduceResponse extends AbstractRequestResponse { // topic level field names private static final String TOPIC_KEY_NAME = "topic"; private static final String PARTITION_RESPONSES_KEY_NAME = "partition_responses"; - private static final String QUOTA_DELAY_KEY_NAME = "throttle_time_ms"; + private static final String THROTTLE_TIME_KEY_NAME = "throttle_time_ms"; // partition level field names private static final String PARTITION_KEY_NAME = "partition"; @@ -50,9 +50,9 @@ public class ProduceResponse extends AbstractRequestResponse { private static final String BASE_OFFSET_KEY_NAME = "base_offset"; private final Map responses; - private final int delayTime; + private final int throttleTime; - public ProduceResponse(Map responses, int delayTime) { + public ProduceResponse(Map responses, int throttleTime) { super(new Struct(CURRENT_SCHEMA)); Map> responseByTopic = CollectionUtils.groupDataByTopic(responses); List topicDatas = new ArrayList(responseByTopic.size()); @@ -72,9 +72,9 @@ public class ProduceResponse extends AbstractRequestResponse { topicDatas.add(topicData); } struct.set(RESPONSES_KEY_NAME, topicDatas.toArray()); - struct.set(QUOTA_DELAY_KEY_NAME, delayTime); + struct.set(THROTTLE_TIME_KEY_NAME, throttleTime); this.responses = responses; - this.delayTime = delayTime; + this.throttleTime = throttleTime; } public ProduceResponse(Struct struct) { @@ -92,13 +92,13 @@ public class ProduceResponse extends AbstractRequestResponse { responses.put(tp, new PartitionResponse(errorCode, offset)); } } - this.delayTime = struct.getInt(QUOTA_DELAY_KEY_NAME); + this.throttleTime = struct.getInt(THROTTLE_TIME_KEY_NAME); } public Map responses() { return this.responses; } - public int getDelayTime() { return this.delayTime; } + public int getThrottleTime() { return this.throttleTime; } public static final class PartitionResponse { public short errorCode; diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java index c6b7feb..66759a9 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -324,8 +324,8 @@ public class FetcherTest { return response.toStruct(); } - private Struct fetchResponse(ByteBuffer buffer, short error, long hw) { - FetchResponse response = new FetchResponse(Collections.singletonMap(tp, new FetchResponse.PartitionData(error, hw, buffer)), 0); + private Struct fetchResponse(ByteBuffer buffer, short error, long hw, int throttleTime) { + FetchResponse response = new FetchResponse(Collections.singletonMap(tp, new FetchResponse.PartitionData(error, hw, buffer)), throttleTime); return response.toStruct(); } } diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java index 2cc9e50..2d4dac5 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java @@ -87,7 +87,7 @@ public class SenderTest { } /* - * Send multiple request. Verify that the client side quota metrics have the right values + * Send multiple requests. Verify that the client side quota metrics have the right values */ @Test public void testQuotaMetrics() throws Exception { @@ -167,10 +167,10 @@ public class SenderTest { } } - private Struct produceResponse(TopicPartition tp, long offset, int error, int delayTimeMs) { + private Struct produceResponse(TopicPartition tp, long offset, int error, int throttleTimeMs) { ProduceResponse.PartitionResponse resp = new ProduceResponse.PartitionResponse((short) error, offset); Map partResp = Collections.singletonMap(tp, resp); - ProduceResponse response = new ProduceResponse(partResp, delayTimeMs); + ProduceResponse response = new ProduceResponse(partResp, throttleTimeMs); return response.toStruct(); } diff --git a/core/src/main/scala/kafka/api/FetchResponse.scala b/core/src/main/scala/kafka/api/FetchResponse.scala index 7881a65..1a8cc84 100644 --- a/core/src/main/scala/kafka/api/FetchResponse.scala +++ b/core/src/main/scala/kafka/api/FetchResponse.scala @@ -66,13 +66,19 @@ class PartitionDataSend(val partitionId: Int, override def completed = !buffer.hasRemaining && messagesSentSize >= messageSize +<<<<<<< HEAD override def destination: String = "" override def writeTo(channel: GatheringByteChannel): Long = { var written = 0L if(buffer.hasRemaining) +======= + override def writeTo(channel: GatheringByteChannel): Int = { + var written = 0 + if (buffer.hasRemaining) +>>>>>>> Addressing Joel's comments written += channel.write(buffer) - if(!buffer.hasRemaining && messagesSentSize < messageSize) { + if (!buffer.hasRemaining && messagesSentSize < messageSize) { val bytesSent = partitionData.messages.writeTo(channel, messagesSentSize, messageSize - messagesSentSize) messagesSentSize += bytesSent written += bytesSent @@ -125,6 +131,7 @@ class TopicDataSend(val dest: String, val topicData: TopicData) extends Send { private val sends = new MultiSend(dest, JavaConversions.seqAsJavaList(topicData.partitionData.toList.map(d => new PartitionDataSend(d._1, d._2)))) +<<<<<<< HEAD override def writeTo(channel: GatheringByteChannel): Long = { if (completed) throw new KafkaException("This operation cannot be completed on a complete request.") @@ -133,6 +140,14 @@ class TopicDataSend(val dest: String, val topicData: TopicData) extends Send { if(buffer.hasRemaining) written += channel.write(buffer) if(!buffer.hasRemaining && !sends.completed) { +======= + def writeTo(channel: GatheringByteChannel): Int = { + expectIncomplete() + var written = 0 + if (buffer.hasRemaining) + written += channel.write(buffer) + if (!buffer.hasRemaining && !sends.complete) { +>>>>>>> Addressing Joel's comments written += sends.writeTo(channel) } sent += written @@ -151,8 +166,7 @@ object FetchResponse { // The request version is used to determine which fields we can expect in the response def readFrom(buffer: ByteBuffer, requestVersion: Int): FetchResponse = { val correlationId = buffer.getInt - val delayTime = if (requestVersion > 0) buffer.getInt else 0 - + val throttleTime = if (requestVersion > 0) buffer.getInt else 0 val topicCount = buffer.getInt val pairs = (1 to topicCount).flatMap(_ => { val topicData = TopicData.readFrom(buffer) @@ -161,23 +175,23 @@ object FetchResponse { (TopicAndPartition(topicData.topic, partitionId), partitionData) } }) - FetchResponse(correlationId, Map(pairs:_*), requestVersion, delayTime) + FetchResponse(correlationId, Map(pairs:_*), requestVersion, throttleTime) } } case class FetchResponse(correlationId: Int, data: Map[TopicAndPartition, FetchResponsePartitionData], requestVersion : Int = 0, - delayTime : Int = 0) + throttleTime : Int = 0) extends RequestOrResponse() { /** * Partitions the data into a map of maps (one for each topic). */ lazy val dataGroupedByTopic = data.groupBy(_._1.topic) - val delayTimeSize = if(requestVersion > 0) 4 else 0 + val throttleTimeSize = if (requestVersion > 0) 4 else 0 val sizeInBytes = - FetchResponse.headerSize + delayTimeSize + + FetchResponse.headerSize + throttleTimeSize + dataGroupedByTopic.foldLeft(0) ((folded, curr) => { val topicData = TopicData(curr._1, curr._2.map { case (topicAndPartition, partitionData) => (topicAndPartition.partition, partitionData) @@ -228,12 +242,12 @@ class FetchResponseSend(val dest: String, val fetchResponse: FetchResponse) exte override def destination = dest // The delayTimeSize will be 0 if the request was made from a client sending a V0 style request - private val buffer = ByteBuffer.allocate(4 /* for size */ + FetchResponse.headerSize + fetchResponse.delayTimeSize) + private val buffer = ByteBuffer.allocate(4 /* for size */ + FetchResponse.headerSize + fetchResponse.throttleTimeSize) buffer.putInt(payloadSize) buffer.putInt(fetchResponse.correlationId) - // Include the delayTime only if the client can read it - if(fetchResponse.requestVersion > 0) { - buffer.putInt(fetchResponse.delayTime) + // Include the throttleTime only if the client can read it + if (fetchResponse.requestVersion > 0) { + buffer.putInt(fetchResponse.throttleTime) } buffer.putInt(fetchResponse.dataGroupedByTopic.size) // topic count diff --git a/core/src/main/scala/kafka/api/ProducerResponse.scala b/core/src/main/scala/kafka/api/ProducerResponse.scala index 0f40a65..2899f20 100644 --- a/core/src/main/scala/kafka/api/ProducerResponse.scala +++ b/core/src/main/scala/kafka/api/ProducerResponse.scala @@ -37,8 +37,8 @@ object ProducerResponse { }) }) - val delayTime = buffer.getInt - ProducerResponse(correlationId, Map(statusPairs:_*), ProducerRequest.CurrentVersion, delayTime) + val throttleTime = buffer.getInt + ProducerResponse(correlationId, Map(statusPairs:_*), ProducerRequest.CurrentVersion, throttleTime) } } @@ -47,7 +47,7 @@ case class ProducerResponseStatus(var error: Short, offset: Long) case class ProducerResponse(correlationId: Int, status: Map[TopicAndPartition, ProducerResponseStatus], requestVersion : Int = 0, - delayTime : Int = 0) + throttleTime : Int = 0) extends RequestOrResponse() { /** @@ -58,7 +58,7 @@ case class ProducerResponse(correlationId: Int, def hasError = status.values.exists(_.error != ErrorMapping.NoError) val sizeInBytes = { - val delayTimeSize = if(requestVersion > 0) 4 else 0 + val throttleTimeSize = if (requestVersion > 0) 4 else 0 val groupedStatus = statusGroupedByTopic 4 + /* correlation id */ 4 + /* topic count */ @@ -72,8 +72,7 @@ case class ProducerResponse(correlationId: Int, 8 /* offset */ } }) + - delayTimeSize - + throttleTimeSize } def writeTo(buffer: ByteBuffer) { @@ -92,9 +91,9 @@ case class ProducerResponse(correlationId: Int, buffer.putLong(nextOffset) } }) - // Delay time is only supported on V1 style requests - if(requestVersion > 0) { - buffer.putInt(delayTime) + // Throttle time is only supported on V1 style requests + if (requestVersion > 0) { + buffer.putInt(throttleTime) } } diff --git a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala index adfd570..b7e7967 100644 --- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala +++ b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala @@ -326,7 +326,7 @@ class RequestResponseSerializationTest extends JUnitSuite { val buffer = ByteBuffer.allocate(newClientResponse.sizeInBytes) newClientResponse.writeTo(buffer) buffer.rewind() - assertEquals(ProducerResponse.readFrom(buffer).delayTime, 100) + assertEquals(ProducerResponse.readFrom(buffer).throttleTime, 100) } @Test diff --git a/core/src/test/scala/unit/kafka/server/DelayedOperationTest.scala b/core/src/test/scala/unit/kafka/server/DelayedOperationTest.scala index b722ce6..df8d5b1 100644 --- a/core/src/test/scala/unit/kafka/server/DelayedOperationTest.scala +++ b/core/src/test/scala/unit/kafka/server/DelayedOperationTest.scala @@ -67,18 +67,6 @@ class DelayedOperationTest { } @Test - def testTimeoutNoKeys() { - val expiration = 20L - val r1 = new MockDelayedOperation(expiration) - val start = System.currentTimeMillis - purgatory.tryCompleteElseWatch(r1, Seq("a1")) - r1.awaitExpiration() - val elapsed = System.currentTimeMillis - start - assertTrue("r1 completed due to expiration", r1.isCompleted()) - assertTrue("Time for expiration %d should at least %d".format(elapsed, expiration), elapsed >= expiration) - } - - @Test def testRequestPurge() { val r1 = new MockDelayedOperation(100000L) val r2 = new MockDelayedOperation(100000L) -- 1.7.12.4 From 835ea1518bfb3e68804f417e20d1d2dafb206e44 Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Tue, 9 Jun 2015 10:06:56 -0700 Subject: [PATCH 116/120] Merging --- .../apache/kafka/clients/producer/internals/Sender.java | 2 +- core/src/main/scala/kafka/api/FetchResponse.scala | 15 --------------- 2 files changed, 1 insertion(+), 16 deletions(-) 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 49a4329..4e13c21 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 @@ -526,7 +526,7 @@ public class Sender implements Runnable { } } - public void recordThrottleTime(int node, long throttleTimeMs) { + public void recordThrottleTime(String node, long throttleTimeMs) { this.throttleTimeSensor.record(throttleTimeMs, time.milliseconds()); } diff --git a/core/src/main/scala/kafka/api/FetchResponse.scala b/core/src/main/scala/kafka/api/FetchResponse.scala index 1a8cc84..cf98aee 100644 --- a/core/src/main/scala/kafka/api/FetchResponse.scala +++ b/core/src/main/scala/kafka/api/FetchResponse.scala @@ -66,17 +66,11 @@ class PartitionDataSend(val partitionId: Int, override def completed = !buffer.hasRemaining && messagesSentSize >= messageSize -<<<<<<< HEAD override def destination: String = "" override def writeTo(channel: GatheringByteChannel): Long = { var written = 0L if(buffer.hasRemaining) -======= - override def writeTo(channel: GatheringByteChannel): Int = { - var written = 0 - if (buffer.hasRemaining) ->>>>>>> Addressing Joel's comments written += channel.write(buffer) if (!buffer.hasRemaining && messagesSentSize < messageSize) { val bytesSent = partitionData.messages.writeTo(channel, messagesSentSize, messageSize - messagesSentSize) @@ -131,7 +125,6 @@ class TopicDataSend(val dest: String, val topicData: TopicData) extends Send { private val sends = new MultiSend(dest, JavaConversions.seqAsJavaList(topicData.partitionData.toList.map(d => new PartitionDataSend(d._1, d._2)))) -<<<<<<< HEAD override def writeTo(channel: GatheringByteChannel): Long = { if (completed) throw new KafkaException("This operation cannot be completed on a complete request.") @@ -140,14 +133,6 @@ class TopicDataSend(val dest: String, val topicData: TopicData) extends Send { if(buffer.hasRemaining) written += channel.write(buffer) if(!buffer.hasRemaining && !sends.completed) { -======= - def writeTo(channel: GatheringByteChannel): Int = { - expectIncomplete() - var written = 0 - if (buffer.hasRemaining) - written += channel.write(buffer) - if (!buffer.hasRemaining && !sends.complete) { ->>>>>>> Addressing Joel's comments written += sends.writeTo(channel) } sent += written -- 1.7.12.4 From 56c1a75b032a1983ce77edfd2e1720e51f893168 Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Tue, 9 Jun 2015 10:10:06 -0700 Subject: [PATCH 117/120] Chaning variable name --- .../java/org/apache/kafka/clients/consumer/internals/Fetcher.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index 3c3a84b..d3dbabc 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -61,9 +61,9 @@ import java.util.Set; * This class manage the fetching process with the brokers. */ public class Fetcher { - public static final long EARLIEST_OFFSET_TIMESTAMP = -2L; - public static final long LATEST_OFFSET_TIMESTAMP = -1L; - private static final String QUOTA_DELAY_KEY_NAME = "throttle_time_ms"; + private static final long EARLIEST_OFFSET_TIMESTAMP = -2L; + private static final long LATEST_OFFSET_TIMESTAMP = -1L; + private static final String THROTTLE_TIME_KEY_NAME = "throttle_time_ms"; private static final Logger log = LoggerFactory.getLogger(Fetcher.class); @@ -456,7 +456,7 @@ public class Fetcher { this.sensors.recordsFetched.record(totalCount); } this.sensors.fetchLatency.record(resp.requestLatencyMs()); - this.sensors.quotaDelayTimeSensor.record(resp.responseBody().getInt(QUOTA_DELAY_KEY_NAME)); + this.sensors.quotaDelayTimeSensor.record(resp.responseBody().getInt(THROTTLE_TIME_KEY_NAME)); } /** -- 1.7.12.4 From 2d463b8cab51af01b5331718568837cac49b907c Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Tue, 30 Jun 2015 19:43:25 -0700 Subject: [PATCH 118/120] Addressing Joel's comments --- .../kafka/clients/consumer/internals/Fetcher.java | 17 ++++++++--------- .../kafka/clients/producer/internals/Sender.java | 17 ++++++++--------- .../clients/consumer/internals/FetcherTest.java | 7 +++++++ .../clients/producer/internals/SenderTest.java | 22 +++++++++------------- 4 files changed, 32 insertions(+), 31 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index d3dbabc..f02041a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -63,7 +63,6 @@ import java.util.Set; public class Fetcher { private static final long EARLIEST_OFFSET_TIMESTAMP = -2L; private static final long LATEST_OFFSET_TIMESTAMP = -1L; - private static final String THROTTLE_TIME_KEY_NAME = "throttle_time_ms"; private static final Logger log = LoggerFactory.getLogger(Fetcher.class); @@ -454,9 +453,9 @@ public class Fetcher { } this.sensors.bytesFetched.record(totalBytes); this.sensors.recordsFetched.record(totalCount); + this.sensors.fetchThrottleTimeSensor.record(response.getThrottleTime()); } this.sensors.fetchLatency.record(resp.requestLatencyMs()); - this.sensors.quotaDelayTimeSensor.record(resp.responseBody().getInt(THROTTLE_TIME_KEY_NAME)); } /** @@ -495,7 +494,7 @@ public class Fetcher { public final Sensor recordsFetched; public final Sensor fetchLatency; public final Sensor recordsFetchLag; - public final Sensor quotaDelayTimeSensor; + public final Sensor fetchThrottleTimeSensor; public FetchManagerMetrics(Metrics metrics, String metricGrpPrefix, Map tags) { @@ -546,16 +545,16 @@ public class Fetcher { "The maximum lag in terms of number of records for any partition in this window", tags), new Max()); - this.quotaDelayTimeSensor = metrics.sensor("fetch-throttle-time"); - this.quotaDelayTimeSensor.add(new MetricName("throttle-time-avg", + this.fetchThrottleTimeSensor = metrics.sensor("fetch-throttle-time"); + this.fetchThrottleTimeSensor.add(new MetricName("fetch-throttle-time-avg", this.metricGrpName, "The average throttle time in ms", tags), new Avg()); - this.quotaDelayTimeSensor.add(new MetricName("throttle-time-max", - this.metricGrpName, - "The maximum throttle time in ms", - tags), new Max()); + this.fetchThrottleTimeSensor.add(new MetricName("fetch-throttle-time-max", + this.metricGrpName, + "The maximum throttle time in ms", + tags), new Max()); } public void recordTopicFetchMetrics(String topic, int bytes, int records) { 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 4e13c21..d2e64f7 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 @@ -55,7 +55,6 @@ import org.slf4j.LoggerFactory; public class Sender implements Runnable { private static final Logger log = LoggerFactory.getLogger(Sender.class); - private static final String THROTTLE_TIME_KEY_NAME = "throttle_time_ms"; /* the state of each nodes connection */ private final KafkaClient client; @@ -255,7 +254,7 @@ public class Sender implements Runnable { } this.sensors.recordLatency(response.request().request().destination(), response.requestLatencyMs()); this.sensors.recordThrottleTime(response.request().request().destination(), - response.responseBody().getInt(THROTTLE_TIME_KEY_NAME)); + produceResponse.getThrottleTime()); } else { // this is the acks = 0 case, just complete all requests for (RecordBatch batch : batches.values()) @@ -355,7 +354,7 @@ public class Sender implements Runnable { public final Sensor batchSizeSensor; public final Sensor compressionRateSensor; public final Sensor maxRecordSizeSensor; - public final Sensor throttleTimeSensor; + public final Sensor produceThrottleTimeSensor; public SenderMetrics(Metrics metrics) { this.metrics = metrics; @@ -385,11 +384,11 @@ public class Sender implements Runnable { m = new MetricName("request-latency-max", metricGrpName, "The maximum request latency in ms", metricTags); this.requestTimeSensor.add(m, new Max()); - this.throttleTimeSensor = metrics.sensor("produce-throttle-time"); - m = new MetricName("throttle-time-avg", metricGrpName, "The average throttle time in ms", metricTags); - this.throttleTimeSensor.add(m, new Avg()); - m = new MetricName("throttle-time-max", metricGrpName, "The maximum throttle time in ms", metricTags); - this.throttleTimeSensor.add(m, new Max()); + this.produceThrottleTimeSensor = metrics.sensor("produce-throttle-time"); + m = new MetricName("produce-throttle-time-avg", metricGrpName, "The average throttle time in ms", metricTags); + this.produceThrottleTimeSensor.add(m, new Avg()); + m = new MetricName("produce-throttle-time-max", metricGrpName, "The maximum throttle time in ms", metricTags); + this.produceThrottleTimeSensor.add(m, new Max()); this.recordsPerRequestSensor = metrics.sensor("records-per-request"); m = new MetricName("record-send-rate", metricGrpName, "The average number of records sent per second.", metricTags); @@ -527,7 +526,7 @@ public class Sender implements Runnable { } public void recordThrottleTime(String node, long throttleTimeMs) { - this.throttleTimeSensor.record(throttleTimeMs, time.milliseconds()); + this.produceThrottleTimeSensor.record(throttleTimeMs, time.milliseconds()); } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java index 66759a9..06ec707 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -59,6 +59,7 @@ public class FetcherTest { private String topicName = "test"; private String groupId = "test-group"; + private final String metricGroup = "consumer" + groupId + "-fetch-manager-metrics"; private TopicPartition tp = new TopicPartition(topicName, 0); private int minBytes = 1; private int maxWaitMs = 0; @@ -73,6 +74,7 @@ public class FetcherTest { private Metrics metrics = new Metrics(time); private Map metricTags = new LinkedHashMap(); private ConsumerNetworkClient consumerClient = new ConsumerNetworkClient(client, metadata, time, 100); + private static final double EPS = 0.0001; private MemoryRecords records = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), CompressionType.NONE); @@ -314,6 +316,11 @@ public class FetcherTest { return partitionData != null && partitionData.timestamp == timestamp; } }; + Map allMetrics = metrics.metrics(); + KafkaMetric avgMetric = allMetrics.get(new MetricName("fetch-throttle-time-avg", metricGroup, "", metricTags)); + KafkaMetric maxMetric = allMetrics.get(new MetricName("fetch-throttle-time-max", metricGroup, "", metricTags)); + assertEquals(200, avgMetric.value(), EPS); + assertEquals(300, maxMetric.value(), EPS); } private Struct listOffsetResponse(Errors error, List offsets) { diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java index 2d4dac5..aa44991 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java @@ -45,6 +45,9 @@ public class SenderTest { private static final short ACKS_ALL = -1; private static final int MAX_RETRIES = 0; private static final int REQUEST_TIMEOUT_MS = 10000; + private static final String CLIENT_ID = "clientId"; + private static final String METRIC_GROUP = "producer-metrics"; + private static final double EPS = 0.0001; private TopicPartition tp = new TopicPartition("test", 0); private MockTime time = new MockTime(); @@ -64,11 +67,12 @@ public class SenderTest { REQUEST_TIMEOUT_MS, metrics, time, - "clientId"); + CLIENT_ID); @Before public void setup() { metadata.update(cluster, time.milliseconds()); + metricTags.put("client-id", CLIENT_ID); } @Test @@ -98,19 +102,11 @@ public class SenderTest { client.respond(produceResponse(tp, offset, Errors.NONE.code(), 100 * i)); sender.run(time.milliseconds()); } - long avg = 0; - long max = 0; - Map allMetrics = metrics.metrics(); - for (MetricName m : allMetrics.keySet()) { - if (m.name().equals("throttle-time-avg")) { - avg = (long) allMetrics.get(m).value(); - } else if (m.name().equals("throttle-time-max")) { - max = (long) allMetrics.get(m).value(); - } - } - assertEquals(200, avg); - assertEquals(300, max); + KafkaMetric avgMetric = allMetrics.get(new MetricName("produce-throttle-time-avg", METRIC_GROUP, "", metricTags)); + KafkaMetric maxMetric = allMetrics.get(new MetricName("produce-throttle-time-max", METRIC_GROUP, "", metricTags)); + assertEquals(200, avgMetric.value(), EPS); + assertEquals(300, maxMetric.value(), EPS); } @Test -- 1.7.12.4 From ec9de04648347a52babf6f0075f2eb21281d4c7e Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Mon, 13 Jul 2015 13:29:45 -0700 Subject: [PATCH 119/120] Addressing Joel's comments --- .../kafka/common/requests/ProduceResponse.java | 5 ++- .../clients/consumer/internals/FetcherTest.java | 8 +++-- core/src/main/scala/kafka/api/FetchResponse.scala | 41 ++++++++++++---------- .../unit/kafka/server/ReplicaManagerTest.scala | 2 +- 4 files changed, 33 insertions(+), 23 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java index d4a3e82..24c4f33 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java @@ -98,7 +98,10 @@ public class ProduceResponse extends AbstractRequestResponse { public Map responses() { return this.responses; } - public int getThrottleTime() { return this.throttleTime; } + + public int getThrottleTime() { + return this.throttleTime; + } public static final class PartitionResponse { public short errorCode; diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java index 06ec707..d480c97 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -73,8 +73,12 @@ public class FetcherTest { private SubscriptionState subscriptions = new SubscriptionState(OffsetResetStrategy.EARLIEST); private Metrics metrics = new Metrics(time); private Map metricTags = new LinkedHashMap(); +<<<<<<< HEAD private ConsumerNetworkClient consumerClient = new ConsumerNetworkClient(client, metadata, time, 100); private static final double EPS = 0.0001; +======= + private static final double EPSILON = 0.0001; +>>>>>>> Addressing Joel's comments private MemoryRecords records = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), CompressionType.NONE); @@ -319,8 +323,8 @@ public class FetcherTest { Map allMetrics = metrics.metrics(); KafkaMetric avgMetric = allMetrics.get(new MetricName("fetch-throttle-time-avg", metricGroup, "", metricTags)); KafkaMetric maxMetric = allMetrics.get(new MetricName("fetch-throttle-time-max", metricGroup, "", metricTags)); - assertEquals(200, avgMetric.value(), EPS); - assertEquals(300, maxMetric.value(), EPS); + assertEquals(200, avgMetric.value(), EPSILON); + assertEquals(300, maxMetric.value(), EPSILON); } private Struct listOffsetResponse(Errors error, List offsets) { diff --git a/core/src/main/scala/kafka/api/FetchResponse.scala b/core/src/main/scala/kafka/api/FetchResponse.scala index cf98aee..75e7d2d 100644 --- a/core/src/main/scala/kafka/api/FetchResponse.scala +++ b/core/src/main/scala/kafka/api/FetchResponse.scala @@ -143,11 +143,6 @@ class TopicDataSend(val dest: String, val topicData: TopicData) extends Send { object FetchResponse { - val headerSize = - 4 + /* correlationId */ - 4 /* topic count */ - - // The request version is used to determine which fields we can expect in the response def readFrom(buffer: ByteBuffer, requestVersion: Int): FetchResponse = { val correlationId = buffer.getInt @@ -166,8 +161,8 @@ object FetchResponse { case class FetchResponse(correlationId: Int, data: Map[TopicAndPartition, FetchResponsePartitionData], - requestVersion : Int = 0, - throttleTime : Int = 0) + requestVersion: Int = 0, + throttleTimeMs: Int = 0) extends RequestOrResponse() { /** @@ -175,8 +170,12 @@ case class FetchResponse(correlationId: Int, */ lazy val dataGroupedByTopic = data.groupBy(_._1.topic) val throttleTimeSize = if (requestVersion > 0) 4 else 0 + val headerSize = 4 + /* correlationId */ + 4 + /* topic count */ + throttleTimeSize + val sizeInBytes = - FetchResponse.headerSize + throttleTimeSize + + headerSize + dataGroupedByTopic.foldLeft(0) ((folded, curr) => { val topicData = TopicData(curr._1, curr._2.map { case (topicAndPartition, partitionData) => (topicAndPartition.partition, partitionData) @@ -185,6 +184,18 @@ case class FetchResponse(correlationId: Int, }) /* + * Writes the header of the FetchResponse to the input buffer + */ + def writeHeaderTo(buffer: ByteBuffer) = { + buffer.putInt(sizeInBytes) + buffer.putInt(correlationId) + // Include the throttleTime only if the client can read it + if (requestVersion > 0) { + buffer.putInt(throttleTimeMs) + } + buffer.putInt(dataGroupedByTopic.size) // topic count + } + /* * FetchResponse uses [sendfile](http://man7.org/linux/man-pages/man2/sendfile.2.html) * api for data transfer through the FetchResponseSend, so `writeTo` aren't actually being used. * It is implemented as an empty function to conform to `RequestOrResponse.writeTo` @@ -226,17 +237,9 @@ class FetchResponseSend(val dest: String, val fetchResponse: FetchResponse) exte override def destination = dest - // The delayTimeSize will be 0 if the request was made from a client sending a V0 style request - private val buffer = ByteBuffer.allocate(4 /* for size */ + FetchResponse.headerSize + fetchResponse.throttleTimeSize) - buffer.putInt(payloadSize) - buffer.putInt(fetchResponse.correlationId) - // Include the throttleTime only if the client can read it - if (fetchResponse.requestVersion > 0) { - buffer.putInt(fetchResponse.throttleTime) - } - - buffer.putInt(fetchResponse.dataGroupedByTopic.size) // topic count - + // The throttleTimeSize will be 0 if the request was made from a client sending a V0 style request + private val buffer = ByteBuffer.allocate(4 /* for size */ + fetchResponse.headerSize) + fetchResponse.writeHeaderTo(buffer) buffer.rewind() private val sends = new MultiSend(dest, JavaConversions.seqAsJavaList(fetchResponse.dataGroupedByTopic.toList.map { diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 2630e71..b76b31a 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -77,7 +77,7 @@ class ReplicaManagerTest { val time: MockTime = new MockTime() val rm = new ReplicaManager(config, time, zkClient, new MockScheduler(time), mockLogMgr, new AtomicBoolean(false)) val produceRequest = new ProducerRequest(1, "client 1", 3, 1000, SerializationTestUtils.topicDataProducerRequest) - def callback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus], delayTime : Int) = { + def callback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus], delayTime: Int) = { assert(responseStatus.values.head.error == Errors.INVALID_REQUIRED_ACKS.code) } -- 1.7.12.4 From 67b0916655ac5191b9a58dd2f1d55c2202600a0e Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Tue, 18 Aug 2015 13:18:56 -0700 Subject: [PATCH 120/120] Addressing comments --- .../kafka/clients/consumer/internals/Fetcher.java | 4 +- .../kafka/common/requests/FetchResponse.java | 10 ++- .../kafka/common/requests/ProduceResponse.java | 56 +++++++++----- .../clients/consumer/internals/FetcherTest.java | 88 +++++++++++++--------- core/src/main/scala/kafka/api/FetchResponse.scala | 42 +++++++---- .../main/scala/kafka/api/ProducerResponse.scala | 8 +- .../scala/kafka/server/ClientQuotaManager.scala | 11 ++- .../src/main/scala/kafka/server/DelayedFetch.scala | 5 +- .../main/scala/kafka/server/DelayedProduce.scala | 5 +- core/src/main/scala/kafka/server/KafkaApis.scala | 55 ++++++-------- .../main/scala/kafka/server/OffsetManager.scala | 2 +- .../main/scala/kafka/server/ReplicaManager.scala | 13 ++-- .../scala/kafka/server/ThrottledResponse.scala | 4 +- .../unit/kafka/server/ClientQuotaManagerTest.scala | 2 +- .../unit/kafka/server/ReplicaManagerTest.scala | 2 +- .../server/ThrottledResponseExpirationTest.scala | 2 +- 16 files changed, 172 insertions(+), 137 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index f02041a..1ae6d03 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -61,8 +61,8 @@ import java.util.Set; * This class manage the fetching process with the brokers. */ public class Fetcher { - private static final long EARLIEST_OFFSET_TIMESTAMP = -2L; - private static final long LATEST_OFFSET_TIMESTAMP = -1L; + public static final long EARLIEST_OFFSET_TIMESTAMP = -2L; + public static final long LATEST_OFFSET_TIMESTAMP = -1L; private static final Logger log = LoggerFactory.getLogger(Fetcher.class); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java index 1ef3db7..bcb6e00 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java @@ -74,6 +74,11 @@ public class FetchResponse extends AbstractRequestResponse { } } + /** + * Constructor for Version 1 + * @param responseData fetched data grouped by topic-partition + * @param throttleTime Time in milliseconds the response was throttled + */ public FetchResponse(Map responseData, int throttleTime) { super(new Struct(CURRENT_SCHEMA)); this.throttleTime = throttleTime; @@ -123,7 +128,10 @@ public class FetchResponse extends AbstractRequestResponse { public Map responseData() { return responseData; } - public int getThrottleTime() { return this.throttleTime; } + + public int getThrottleTime() { + return this.throttleTime; + } public static FetchResponse parse(ByteBuffer buffer) { return new FetchResponse((Struct) CURRENT_SCHEMA.read(buffer)); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java index 24c4f33..60847b1 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java @@ -40,6 +40,7 @@ public class ProduceResponse extends AbstractRequestResponse { private static final String ERROR_CODE_KEY_NAME = "error_code"; public static final long INVALID_OFFSET = -1L; + private static final int DEFAULT_THROTTLE_TIME = 0; /** * Possible error code: @@ -52,26 +53,25 @@ public class ProduceResponse extends AbstractRequestResponse { private final Map responses; private final int throttleTime; + /** + * Constructor for Version 0 + * @param responses Produced data grouped by topic-partition + */ + public ProduceResponse(Map responses) { + super(new Struct(ProtoUtils.requestSchema(ApiKeys.PRODUCE.id, 0))); + initCommonFields(responses); + this.responses = responses; + this.throttleTime = DEFAULT_THROTTLE_TIME; + } + + /** + * Constructor for Version 1 + * @param responses Produced data grouped by topic-partition + * @param throttleTime Time in milliseconds the response was throttled + */ public ProduceResponse(Map responses, int throttleTime) { super(new Struct(CURRENT_SCHEMA)); - Map> responseByTopic = CollectionUtils.groupDataByTopic(responses); - List topicDatas = new ArrayList(responseByTopic.size()); - for (Map.Entry> entry : responseByTopic.entrySet()) { - Struct topicData = struct.instance(RESPONSES_KEY_NAME); - topicData.set(TOPIC_KEY_NAME, entry.getKey()); - List partitionArray = new ArrayList(); - for (Map.Entry partitionEntry : entry.getValue().entrySet()) { - PartitionResponse part = partitionEntry.getValue(); - Struct partStruct = topicData.instance(PARTITION_RESPONSES_KEY_NAME) - .set(PARTITION_KEY_NAME, partitionEntry.getKey()) - .set(ERROR_CODE_KEY_NAME, part.errorCode) - .set(BASE_OFFSET_KEY_NAME, part.baseOffset); - partitionArray.add(partStruct); - } - topicData.set(PARTITION_RESPONSES_KEY_NAME, partitionArray.toArray()); - topicDatas.add(topicData); - } - struct.set(RESPONSES_KEY_NAME, topicDatas.toArray()); + initCommonFields(responses); struct.set(THROTTLE_TIME_KEY_NAME, throttleTime); this.responses = responses; this.throttleTime = throttleTime; @@ -95,6 +95,26 @@ public class ProduceResponse extends AbstractRequestResponse { this.throttleTime = struct.getInt(THROTTLE_TIME_KEY_NAME); } + private void initCommonFields(Map responses) { + Map> responseByTopic = CollectionUtils.groupDataByTopic(responses); + List topicDatas = new ArrayList(responseByTopic.size()); + for (Map.Entry> entry : responseByTopic.entrySet()) { + Struct topicData = struct.instance(RESPONSES_KEY_NAME); + topicData.set(TOPIC_KEY_NAME, entry.getKey()); + List partitionArray = new ArrayList(); + for (Map.Entry partitionEntry : entry.getValue().entrySet()) { + PartitionResponse part = partitionEntry.getValue(); + Struct partStruct = topicData.instance(PARTITION_RESPONSES_KEY_NAME).set(PARTITION_KEY_NAME, + partitionEntry.getKey()).set( + ERROR_CODE_KEY_NAME, part.errorCode).set(BASE_OFFSET_KEY_NAME, part.baseOffset); + partitionArray.add(partStruct); + } + topicData.set(PARTITION_RESPONSES_KEY_NAME, partitionArray.toArray()); + topicDatas.add(topicData); + } + struct.set(RESPONSES_KEY_NAME, topicDatas.toArray()); + } + public Map responses() { return this.responses; } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java index d480c97..22712bb 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -56,7 +56,6 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; public class FetcherTest { - private String topicName = "test"; private String groupId = "test-group"; private final String metricGroup = "consumer" + groupId + "-fetch-manager-metrics"; @@ -73,29 +72,25 @@ public class FetcherTest { private SubscriptionState subscriptions = new SubscriptionState(OffsetResetStrategy.EARLIEST); private Metrics metrics = new Metrics(time); private Map metricTags = new LinkedHashMap(); -<<<<<<< HEAD - private ConsumerNetworkClient consumerClient = new ConsumerNetworkClient(client, metadata, time, 100); - private static final double EPS = 0.0001; -======= private static final double EPSILON = 0.0001; ->>>>>>> Addressing Joel's comments + private ConsumerNetworkClient consumerClient = new ConsumerNetworkClient(client, metadata, time, 100); private MemoryRecords records = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), CompressionType.NONE); private Fetcher fetcher = new Fetcher(consumerClient, - minBytes, - maxWaitMs, - fetchSize, - true, // check crc - new ByteArrayDeserializer(), - new ByteArrayDeserializer(), - metadata, - subscriptions, - metrics, - "consumer" + groupId, - metricTags, - time, - retryBackoffMs); + minBytes, + maxWaitMs, + fetchSize, + true, // check crc + new ByteArrayDeserializer(), + new ByteArrayDeserializer(), + metadata, + subscriptions, + metrics, + "consumer" + groupId, + metricTags, + time, + retryBackoffMs); @Before public void setup() throws Exception { @@ -117,7 +112,7 @@ public class FetcherTest { // normal fetch fetcher.initFetches(cluster); - client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L)); + client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 0)); consumerClient.poll(0); records = fetcher.fetchedRecords().get(tp); assertEquals(3, records.size()); @@ -140,7 +135,7 @@ public class FetcherTest { // Now the rebalance happens and fetch positions are cleared subscriptions.changePartitionAssignment(Arrays.asList(tp)); - client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L)); + client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 0)); consumerClient.poll(0); // The active fetch should be ignored since its position is no longer valid @@ -155,7 +150,7 @@ public class FetcherTest { fetcher.initFetches(cluster); subscriptions.pause(tp); - client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L)); + client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 0)); consumerClient.poll(0); assertNull(fetcher.fetchedRecords().get(tp)); } @@ -176,7 +171,7 @@ public class FetcherTest { subscriptions.seek(tp, 0); fetcher.initFetches(cluster); - client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NOT_LEADER_FOR_PARTITION.code(), 100L)); + client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NOT_LEADER_FOR_PARTITION.code(), 100L, 0)); consumerClient.poll(0); assertEquals(0, fetcher.fetchedRecords().size()); assertEquals(0L, metadata.timeToNextUpdate(time.milliseconds())); @@ -188,7 +183,7 @@ public class FetcherTest { subscriptions.seek(tp, 0); fetcher.initFetches(cluster); - client.prepareResponse(fetchResponse(this.records.buffer(), Errors.UNKNOWN_TOPIC_OR_PARTITION.code(), 100L)); + client.prepareResponse(fetchResponse(this.records.buffer(), Errors.UNKNOWN_TOPIC_OR_PARTITION.code(), 100L, 0)); consumerClient.poll(0); assertEquals(0, fetcher.fetchedRecords().size()); assertEquals(0L, metadata.timeToNextUpdate(time.milliseconds())); @@ -200,7 +195,7 @@ public class FetcherTest { subscriptions.seek(tp, 0); fetcher.initFetches(cluster); - client.prepareResponse(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L)); + client.prepareResponse(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L, 0)); consumerClient.poll(0); assertTrue(subscriptions.isOffsetResetNeeded(tp)); assertEquals(0, fetcher.fetchedRecords().size()); @@ -214,7 +209,7 @@ public class FetcherTest { subscriptions.seek(tp, 0); fetcher.initFetches(cluster); - client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L), true); + client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 0), true); consumerClient.poll(0); assertEquals(0, fetcher.fetchedRecords().size()); @@ -244,7 +239,7 @@ public class FetcherTest { // with no commit position, we should reset using the default strategy defined above (EARLIEST) client.prepareResponse(listOffsetRequestMatcher(Fetcher.EARLIEST_OFFSET_TIMESTAMP), - listOffsetResponse(Errors.NONE, Arrays.asList(5L))); + listOffsetResponse(Errors.NONE, Arrays.asList(5L))); fetcher.updateFetchPositions(Collections.singleton(tp)); assertFalse(subscriptions.isOffsetResetNeeded(tp)); assertTrue(subscriptions.isFetchable(tp)); @@ -258,7 +253,7 @@ public class FetcherTest { subscriptions.needOffsetReset(tp, OffsetResetStrategy.LATEST); client.prepareResponse(listOffsetRequestMatcher(Fetcher.LATEST_OFFSET_TIMESTAMP), - listOffsetResponse(Errors.NONE, Arrays.asList(5L))); + listOffsetResponse(Errors.NONE, Arrays.asList(5L))); fetcher.updateFetchPositions(Collections.singleton(tp)); assertFalse(subscriptions.isOffsetResetNeeded(tp)); assertTrue(subscriptions.isFetchable(tp)); @@ -272,7 +267,7 @@ public class FetcherTest { subscriptions.needOffsetReset(tp, OffsetResetStrategy.EARLIEST); client.prepareResponse(listOffsetRequestMatcher(Fetcher.EARLIEST_OFFSET_TIMESTAMP), - listOffsetResponse(Errors.NONE, Arrays.asList(5L))); + listOffsetResponse(Errors.NONE, Arrays.asList(5L))); fetcher.updateFetchPositions(Collections.singleton(tp)); assertFalse(subscriptions.isOffsetResetNeeded(tp)); assertTrue(subscriptions.isFetchable(tp)); @@ -287,11 +282,11 @@ public class FetcherTest { // First request gets a disconnect client.prepareResponse(listOffsetRequestMatcher(Fetcher.LATEST_OFFSET_TIMESTAMP), - listOffsetResponse(Errors.NONE, Arrays.asList(5L)), true); + listOffsetResponse(Errors.NONE, Arrays.asList(5L)), true); // Next one succeeds client.prepareResponse(listOffsetRequestMatcher(Fetcher.LATEST_OFFSET_TIMESTAMP), - listOffsetResponse(Errors.NONE, Arrays.asList(5L))); + listOffsetResponse(Errors.NONE, Arrays.asList(5L))); fetcher.updateFetchPositions(Collections.singleton(tp)); assertFalse(subscriptions.isOffsetResetNeeded(tp)); assertTrue(subscriptions.isFetchable(tp)); @@ -310,6 +305,32 @@ public class FetcherTest { assertEquals(cluster.topics().size(), allTopics.size()); } + /* + * Send multiple requests. Verify that the client side quota metrics have the right values + */ + @Test + public void testQuotaMetrics() throws Exception { + List> records; + subscriptions.subscribe(tp); + subscriptions.seek(tp, 0); + + // normal fetch + for (int i = 1; i < 4; i++) { + fetcher.initFetches(cluster); + + client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 100 * i)); + consumerClient.poll(0); + records = fetcher.fetchedRecords().get(tp); + assertEquals(3, records.size()); + } + + Map allMetrics = metrics.metrics(); + KafkaMetric avgMetric = allMetrics.get(new MetricName("fetch-throttle-time-avg", metricGroup, "", metricTags)); + KafkaMetric maxMetric = allMetrics.get(new MetricName("fetch-throttle-time-max", metricGroup, "", metricTags)); + assertEquals(200, avgMetric.value(), EPSILON); + assertEquals(300, maxMetric.value(), EPSILON); + } + private MockClient.RequestMatcher listOffsetRequestMatcher(final long timestamp) { // matches any list offset request with the provided timestamp return new MockClient.RequestMatcher() { @@ -320,11 +341,6 @@ public class FetcherTest { return partitionData != null && partitionData.timestamp == timestamp; } }; - Map allMetrics = metrics.metrics(); - KafkaMetric avgMetric = allMetrics.get(new MetricName("fetch-throttle-time-avg", metricGroup, "", metricTags)); - KafkaMetric maxMetric = allMetrics.get(new MetricName("fetch-throttle-time-max", metricGroup, "", metricTags)); - assertEquals(200, avgMetric.value(), EPSILON); - assertEquals(300, maxMetric.value(), EPSILON); } private Struct listOffsetResponse(Errors error, List offsets) { diff --git a/core/src/main/scala/kafka/api/FetchResponse.scala b/core/src/main/scala/kafka/api/FetchResponse.scala index 75e7d2d..2e6fbdd 100644 --- a/core/src/main/scala/kafka/api/FetchResponse.scala +++ b/core/src/main/scala/kafka/api/FetchResponse.scala @@ -157,6 +157,27 @@ object FetchResponse { }) FetchResponse(correlationId, Map(pairs:_*), requestVersion, throttleTime) } + + // Returns the size of the response header + def headerSize(requestVersion: Int): Int = { + val throttleTimeSize = if (requestVersion > 0) 4 else 0 + 4 + /* correlationId */ + 4 + /* topic count */ + throttleTimeSize + } + + // Returns the size of entire fetch response in bytes (including the header size) + def responseSize(data: Map[TopicAndPartition, FetchResponsePartitionData], + requestVersion: Int): Int = { + val dataGroupedByTopic = data.groupBy(_._1.topic) + headerSize(requestVersion) + + dataGroupedByTopic.foldLeft(0) ((folded, curr) => { + val topicData = TopicData(curr._1, curr._2.map { + case (topicAndPartition, partitionData) => (topicAndPartition.partition, partitionData) + }) + folded + topicData.sizeInBytes + }) + } } case class FetchResponse(correlationId: Int, @@ -169,19 +190,8 @@ case class FetchResponse(correlationId: Int, * Partitions the data into a map of maps (one for each topic). */ lazy val dataGroupedByTopic = data.groupBy(_._1.topic) - val throttleTimeSize = if (requestVersion > 0) 4 else 0 - val headerSize = 4 + /* correlationId */ - 4 + /* topic count */ - throttleTimeSize - - val sizeInBytes = - headerSize + - dataGroupedByTopic.foldLeft(0) ((folded, curr) => { - val topicData = TopicData(curr._1, curr._2.map { - case (topicAndPartition, partitionData) => (topicAndPartition.partition, partitionData) - }) - folded + topicData.sizeInBytes - }) + val headerSizeInBytes = FetchResponse.headerSize(requestVersion) + val sizeInBytes = FetchResponse.responseSize(data, requestVersion) /* * Writes the header of the FetchResponse to the input buffer @@ -190,9 +200,9 @@ case class FetchResponse(correlationId: Int, buffer.putInt(sizeInBytes) buffer.putInt(correlationId) // Include the throttleTime only if the client can read it - if (requestVersion > 0) { + if (requestVersion > 0) buffer.putInt(throttleTimeMs) - } + buffer.putInt(dataGroupedByTopic.size) // topic count } /* @@ -238,7 +248,7 @@ class FetchResponseSend(val dest: String, val fetchResponse: FetchResponse) exte override def destination = dest // The throttleTimeSize will be 0 if the request was made from a client sending a V0 style request - private val buffer = ByteBuffer.allocate(4 /* for size */ + fetchResponse.headerSize) + private val buffer = ByteBuffer.allocate(4 /* for size */ + fetchResponse.headerSizeInBytes) fetchResponse.writeHeaderTo(buffer) buffer.rewind() diff --git a/core/src/main/scala/kafka/api/ProducerResponse.scala b/core/src/main/scala/kafka/api/ProducerResponse.scala index 2899f20..7719f30 100644 --- a/core/src/main/scala/kafka/api/ProducerResponse.scala +++ b/core/src/main/scala/kafka/api/ProducerResponse.scala @@ -23,6 +23,7 @@ import kafka.common.{TopicAndPartition, ErrorMapping} import kafka.api.ApiUtils._ object ProducerResponse { + // readFrom assumes that the response is written using V1 format def readFrom(buffer: ByteBuffer): ProducerResponse = { val correlationId = buffer.getInt val topicCount = buffer.getInt @@ -46,8 +47,8 @@ case class ProducerResponseStatus(var error: Short, offset: Long) case class ProducerResponse(correlationId: Int, status: Map[TopicAndPartition, ProducerResponseStatus], - requestVersion : Int = 0, - throttleTime : Int = 0) + requestVersion: Int = 0, + throttleTime: Int = 0) extends RequestOrResponse() { /** @@ -92,9 +93,8 @@ case class ProducerResponse(correlationId: Int, } }) // Throttle time is only supported on V1 style requests - if (requestVersion > 0) { + if (requestVersion > 0) buffer.putInt(throttleTime) - } } override def describe(details: Boolean):String = { toString } diff --git a/core/src/main/scala/kafka/server/ClientQuotaManager.scala b/core/src/main/scala/kafka/server/ClientQuotaManager.scala index 9f8473f..de7f68d 100644 --- a/core/src/main/scala/kafka/server/ClientQuotaManager.scala +++ b/core/src/main/scala/kafka/server/ClientQuotaManager.scala @@ -110,13 +110,13 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, * @return Number of milliseconds to delay the response in case of Quota violation. * Zero otherwise */ - def recordAndMaybeThrottle(clientId: String, value: Int, callback: => Unit): Int = { + def recordAndMaybeThrottle(clientId: String, value: Int, callback: Int => Unit): Int = { val clientSensors = getOrCreateQuotaSensors(clientId) - var delayTimeMs = 0L + var delayTimeMs = 0 try { clientSensors.quotaSensor.record(value) // trigger the callback immediately if quota is not violated - callback + callback(delayTimeMs) } catch { case qve: QuotaViolationException => // Compute the delay @@ -139,12 +139,11 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, * we need to add a delay of X to W such that O * W / (W + X) = T. * Solving for X, we get X = (O - T)/T * W. */ - private def delayTime(metricValue: Double, config: MetricConfig): Long = - { + private def delayTime(metricValue: Double, config: MetricConfig): Int = { val quota = config.quota() val difference = metricValue - quota.bound val time = difference / quota.bound * config.timeWindowMs() * config.samples() - time.round + time.round.toInt } /** diff --git a/core/src/main/scala/kafka/server/DelayedFetch.scala b/core/src/main/scala/kafka/server/DelayedFetch.scala index 8e5cb39..de6cf5b 100644 --- a/core/src/main/scala/kafka/server/DelayedFetch.scala +++ b/core/src/main/scala/kafka/server/DelayedFetch.scala @@ -55,7 +55,7 @@ case class FetchMetadata(fetchMinBytes: Int, class DelayedFetch(delayMs: Long, fetchMetadata: FetchMetadata, replicaManager: ReplicaManager, - responseCallback: (Map[TopicAndPartition, FetchResponsePartitionData], Int) => Unit) + responseCallback: Map[TopicAndPartition, FetchResponsePartitionData] => Unit) extends DelayedOperation(delayMs) { /** @@ -131,8 +131,7 @@ class DelayedFetch(delayMs: Long, val fetchPartitionData = logReadResults.mapValues(result => FetchResponsePartitionData(result.errorCode, result.hw, result.info.messageSet)) - // Zero delay time until quotas are enforced - responseCallback(fetchPartitionData, 0) + responseCallback(fetchPartitionData) } } diff --git a/core/src/main/scala/kafka/server/DelayedProduce.scala b/core/src/main/scala/kafka/server/DelayedProduce.scala index 7a9c30b..05078b2 100644 --- a/core/src/main/scala/kafka/server/DelayedProduce.scala +++ b/core/src/main/scala/kafka/server/DelayedProduce.scala @@ -53,7 +53,7 @@ case class ProduceMetadata(produceRequiredAcks: Short, class DelayedProduce(delayMs: Long, produceMetadata: ProduceMetadata, replicaManager: ReplicaManager, - responseCallback: (Map[TopicAndPartition, ProducerResponseStatus], Int) => Unit) + responseCallback: Map[TopicAndPartition, ProducerResponseStatus] => Unit) extends DelayedOperation(delayMs) { // first update the acks pending variable according to the error code @@ -126,8 +126,7 @@ class DelayedProduce(delayMs: Long, */ override def onComplete() { val responseStatus = produceMetadata.produceStatus.mapValues(status => status.responseStatus) - // Zero delay time until quotas are enforced - responseCallback(responseStatus, 0) + responseCallback(responseStatus) } } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 07896d0..c838ca7 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -257,15 +257,13 @@ class KafkaApis(val requestChannel: RequestChannel, val numBytesAppended = produceRequest.sizeInBytes // the callback for sending a produce response - def sendResponseCallback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus], delayTime : Int) - { + def sendResponseCallback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus]) { var errorInResponse = false responseStatus.foreach { case (topicAndPartition, status) => // we only print warnings for known errors here; if it is unknown, it will cause // an error message in the replica manager - if (status.error != ErrorMapping.NoError && status.error != ErrorMapping.UnknownCode) - { + if (status.error != ErrorMapping.NoError && status.error != ErrorMapping.UnknownCode) { debug("Produce request with correlation id %d from client %s on partition %s failed due to %s".format( produceRequest.correlationId, produceRequest.clientId, @@ -275,41 +273,34 @@ class KafkaApis(val requestChannel: RequestChannel, } } - def produceResponseCallback - { - if (produceRequest.requiredAcks == 0) - { + def produceResponseCallback(delayTimeMs: Int) { + if (produceRequest.requiredAcks == 0) { // no operation needed if producer request.required.acks = 0; however, if there is any error in handling // the request, since no response is expected by the producer, the server will close socket server so that // the producer client will know that some error has happened and will refresh its metadata - if (errorInResponse) - { + if (errorInResponse) { info( "Close connection due to error handling produce request with correlation id %d from client id %s with ack=0".format( produceRequest.correlationId, produceRequest.clientId)) requestChannel.closeConnection(request.processor, request) - } - else - { + } else { requestChannel.noOperation(request.processor, request) } - } - else - { + } else { val response = ProducerResponse(produceRequest.correlationId, responseStatus, produceRequest.versionId, - delayTime) + delayTimeMs) requestChannel.sendResponse(new RequestChannel.Response(request, new RequestOrResponseSend(request.connectionId, response))) } - - quotaManagers(RequestKeys.ProduceKey).recordAndMaybeThrottle(produceRequest.clientId, - numBytesAppended, - produceResponseCallback) } + + quotaManagers(RequestKeys.ProduceKey).recordAndMaybeThrottle(produceRequest.clientId, + numBytesAppended, + produceResponseCallback) } // only allow appending to internal topic partitions @@ -337,7 +328,7 @@ class KafkaApis(val requestChannel: RequestChannel, val fetchRequest = request.requestObj.asInstanceOf[FetchRequest] // the callback for sending a fetch response - def sendResponseCallback(responsePartitionData: Map[TopicAndPartition, FetchResponsePartitionData], delayTime : Int) { + def sendResponseCallback(responsePartitionData: Map[TopicAndPartition, FetchResponsePartitionData]) { responsePartitionData.foreach { case (topicAndPartition, data) => // we only print warnings for known errors here; if it is unknown, it will cause // an error message in the replica manager already and hence can be ignored here @@ -351,24 +342,20 @@ class KafkaApis(val requestChannel: RequestChannel, BrokerTopicStats.getBrokerAllTopicsStats().bytesOutRate.mark(data.messages.sizeInBytes) } - val rr = FetchResponse(fetchRequest.correlationId, responsePartitionData) - def fetchResponseCallback { - requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(request.connectionId, rr))) + def fetchResponseCallback(delayTimeMs: Int) { + val response = FetchResponse(fetchRequest.correlationId, responsePartitionData, fetchRequest.versionId, delayTimeMs) + requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(request.connectionId, response))) } // Do not throttle replication traffic if (fetchRequest.isFromFollower) { - fetchResponseCallback + fetchResponseCallback(0) } else { - quotaManagers.get(RequestKeys.FetchKey) match { - case Some(quotaManager) => - quotaManager.recordAndMaybeThrottle(fetchRequest.clientId, rr.sizeInBytes, fetchResponseCallback) - case None => - warn("Cannot throttle Api key %s".format(RequestKeys.nameForKey(RequestKeys.FetchKey))) - } + quotaManagers(RequestKeys.FetchKey).recordAndMaybeThrottle(fetchRequest.clientId, + FetchResponse.responseSize(responsePartitionData, + fetchRequest.versionId), + fetchResponseCallback) } - val response = FetchResponse(fetchRequest.correlationId, responsePartitionData, fetchRequest.versionId, delayTime) - requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(request.connectionId, response))) } // call the replica manager to fetch messages from the local replica diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala index 4c5f4be..0e613e7 100755 --- a/core/src/main/scala/kafka/server/OffsetManager.scala +++ b/core/src/main/scala/kafka/server/OffsetManager.scala @@ -231,7 +231,7 @@ class OffsetManager(val config: OffsetManagerConfig, new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, messages:_*)) // set the callback function to insert offsets into cache after log append completed - def putCacheCallback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus], delayTime : Int) { + def putCacheCallback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus]) { // the append response should only contain the topics partition if (responseStatus.size != 1 || ! responseStatus.contains(offsetTopicPartition)) throw new IllegalStateException("Append status %s should only have one partition %s" diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 0426f11..c195536 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -31,7 +31,6 @@ import kafka.metrics.KafkaMetricsGroup import kafka.utils._ import org.I0Itec.zkclient.ZkClient import org.apache.kafka.common.protocol.Errors -import org.apache.kafka.common.metrics.Metrics import scala.collection._ @@ -301,7 +300,7 @@ class ReplicaManager(val config: KafkaConfig, requiredAcks: Short, internalTopicsAllowed: Boolean, messagesPerPartition: Map[TopicAndPartition, MessageSet], - responseCallback: (Map[TopicAndPartition, ProducerResponseStatus], Int) => Unit) { + responseCallback: Map[TopicAndPartition, ProducerResponseStatus] => Unit) { if (isValidRequiredAcks(requiredAcks)) { val sTime = SystemTime.milliseconds @@ -331,8 +330,7 @@ class ReplicaManager(val config: KafkaConfig, } else { // we can respond immediately val produceResponseStatus = produceStatus.mapValues(status => status.responseStatus) - // Zero delay time until quotas are enforced - responseCallback(produceResponseStatus, 0) + responseCallback(produceResponseStatus) } } else { // If required.acks is outside accepted range, something is wrong with the client @@ -343,7 +341,7 @@ class ReplicaManager(val config: KafkaConfig, ProducerResponseStatus(Errors.INVALID_REQUIRED_ACKS.code, LogAppendInfo.UnknownLogAppendInfo.firstOffset)) } - responseCallback(responseStatus, 0) + responseCallback(responseStatus) } } @@ -440,7 +438,7 @@ class ReplicaManager(val config: KafkaConfig, replicaId: Int, fetchMinBytes: Int, fetchInfo: immutable.Map[TopicAndPartition, PartitionFetchInfo], - responseCallback: (Map[TopicAndPartition, FetchResponsePartitionData], Int) => Unit) { + responseCallback: Map[TopicAndPartition, FetchResponsePartitionData] => Unit) { val isFromFollower = replicaId >= 0 val fetchOnlyFromLeader: Boolean = replicaId != Request.DebuggingConsumerId val fetchOnlyCommitted: Boolean = ! Request.isValidBrokerId(replicaId) @@ -465,8 +463,7 @@ class ReplicaManager(val config: KafkaConfig, if(timeout <= 0 || fetchInfo.size <= 0 || bytesReadable >= fetchMinBytes || errorReadingData) { val fetchPartitionData = logReadResults.mapValues(result => FetchResponsePartitionData(result.errorCode, result.hw, result.info.messageSet)) - // Zero delay time until quotas are enforced - responseCallback(fetchPartitionData, 0) + responseCallback(fetchPartitionData) } else { // construct the fetch results from the read results val fetchPartitionStatus = logReadResults.map { case (topicAndPartition, result) => diff --git a/core/src/main/scala/kafka/server/ThrottledResponse.scala b/core/src/main/scala/kafka/server/ThrottledResponse.scala index 1f80d54..b8b70cd 100644 --- a/core/src/main/scala/kafka/server/ThrottledResponse.scala +++ b/core/src/main/scala/kafka/server/ThrottledResponse.scala @@ -28,10 +28,10 @@ import org.apache.kafka.common.utils.Time * @param delayTimeMs delay associated with this request * @param callback Callback to trigger after delayTimeMs milliseconds */ -private[server] class ThrottledResponse(val time: Time, val delayTimeMs: Long, callback: => Unit) extends Delayed { +private[server] class ThrottledResponse(val time: Time, val delayTimeMs: Int, callback: Int => Unit) extends Delayed { val endTime = time.milliseconds + delayTimeMs - def execute() = callback + def execute() = callback(delayTimeMs) override def getDelay(unit: TimeUnit): Long = { unit.convert(endTime - time.milliseconds, TimeUnit.MILLISECONDS) diff --git a/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala index 97dcca8..e42da61 100644 --- a/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala @@ -32,7 +32,7 @@ class ClientQuotaManagerTest extends JUnit3Suite { quotaBytesPerSecondOverrides = "p1=2000,p2=4000") var numCallbacks: Int = 0 - def callback { + def callback(delayTimeMs: Int) { numCallbacks += 1 } diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index b76b31a..3770cb4 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -77,7 +77,7 @@ class ReplicaManagerTest { val time: MockTime = new MockTime() val rm = new ReplicaManager(config, time, zkClient, new MockScheduler(time), mockLogMgr, new AtomicBoolean(false)) val produceRequest = new ProducerRequest(1, "client 1", 3, 1000, SerializationTestUtils.topicDataProducerRequest) - def callback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus], delayTime: Int) = { + def callback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus]) = { assert(responseStatus.values.head.error == Errors.INVALID_REQUIRED_ACKS.code) } diff --git a/core/src/test/scala/unit/kafka/server/ThrottledResponseExpirationTest.scala b/core/src/test/scala/unit/kafka/server/ThrottledResponseExpirationTest.scala index 14a7f45..b0d5082 100644 --- a/core/src/test/scala/unit/kafka/server/ThrottledResponseExpirationTest.scala +++ b/core/src/test/scala/unit/kafka/server/ThrottledResponseExpirationTest.scala @@ -33,7 +33,7 @@ class ThrottledResponseExpirationTest extends JUnit3Suite { Collections.emptyList(), time) - def callback { + def callback(delayTimeMs: Int) { numCallbacks += 1 } -- 1.7.12.4