From baf70497759e636a55d50f03d0c5a9673ac71d2d Mon Sep 17 00:00:00 2001 From: Ishita Mandhan Date: Tue, 16 Aug 2016 17:32:22 -0700 Subject: [PATCH 01/35] KAFKA-3940 Log should check the return value of dir.mkdirs() This commit changes all the occurrences of dir.mkdirs() with Files.createDirectory(dir.toPath()) --- core/src/main/scala/kafka/log/Log.scala | 9 ++++++++- core/src/main/scala/kafka/log/LogManager.scala | 15 +++++++++++---- .../kafka/metrics/KafkaCSVMetricsReporter.scala | 11 +++++++++-- .../kafka/log/LogCleanerIntegrationTest.scala | 13 +++++++++---- .../scala/unit/kafka/log/LogManagerTest.scala | 7 ++++++- core/src/test/scala/unit/kafka/log/LogTest.scala | 15 ++++++++++++--- .../test/scala/unit/kafka/utils/TestUtils.scala | 10 +++++++--- 7 files changed, 62 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 1a7719a67650..7be8797af8dc 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -23,6 +23,7 @@ import kafka.common._ import kafka.metrics.KafkaMetricsGroup import kafka.server.{BrokerTopicStats, FetchDataInfo, LogOffsetMetadata} import java.io.{File, IOException} +import java.nio.file.Files import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap} import java.util.concurrent.atomic._ import java.text.NumberFormat @@ -139,7 +140,13 @@ class Log(val dir: File, /* Load the log segments from the log files on disk */ private def loadSegments() { // create the log directory if it doesn't exist - dir.mkdirs() + if (!Files.exists(dir.toPath())) { + try { + Files.createDirectory(dir.toPath()) + } catch { + case e: IOException => throw new KafkaException("Error in creating new directory.", e) + } + } var swapFiles = Set[File]() // first do a pass through the files in the log directory and remove any temporary files diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index 4357ef4c5d36..70ba51dec541 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -18,6 +18,7 @@ package kafka.log import java.io._ +import java.nio.file.Files import java.util.concurrent.TimeUnit import kafka.utils._ @@ -81,9 +82,11 @@ class LogManager(val logDirs: Array[File], for(dir <- dirs) { if(!dir.exists) { info("Log directory '" + dir.getAbsolutePath + "' not found, creating it.") - val created = dir.mkdirs() - if(!created) - throw new KafkaException("Failed to create data directory " + dir.getAbsolutePath) + try { + Files.createDirectory(dir.toPath()) + } catch { + case e: IOException => throw new KafkaException("Error in creating new directory.", e) + } } if(!dir.isDirectory || !dir.canRead) throw new KafkaException(dir.getAbsolutePath + " is not a readable log directory.") @@ -358,7 +361,11 @@ class LogManager(val logDirs: Array[File], // if not, create it val dataDir = nextLogDir() val dir = new File(dataDir, topicAndPartition.topic + "-" + topicAndPartition.partition) - dir.mkdirs() + try { + Files.createDirectory(dir.toPath()) + } catch { + case e: IOException => throw new KafkaException("Error in creating new directory.", e) + } log = new Log(dir, config, recoveryPoint = 0L, diff --git a/core/src/main/scala/kafka/metrics/KafkaCSVMetricsReporter.scala b/core/src/main/scala/kafka/metrics/KafkaCSVMetricsReporter.scala index 686f692548d9..0f8c59e7d855 100755 --- a/core/src/main/scala/kafka/metrics/KafkaCSVMetricsReporter.scala +++ b/core/src/main/scala/kafka/metrics/KafkaCSVMetricsReporter.scala @@ -21,11 +21,13 @@ package kafka.metrics import com.yammer.metrics.Metrics -import java.io.File +import java.io.{File, IOException} +import java.nio.file.Files import com.yammer.metrics.reporting.CsvReporter import java.util.concurrent.TimeUnit +import kafka.common._ import kafka.utils.{Logging, VerifiableProperties} import org.apache.kafka.common.utils.Utils @@ -50,7 +52,12 @@ private class KafkaCSVMetricsReporter extends KafkaMetricsReporter val metricsConfig = new KafkaMetricsConfig(props) csvDir = new File(props.getString("kafka.csv.metrics.dir", "kafka_metrics")) Utils.delete(csvDir) - csvDir.mkdirs() + try { + Files.createDirectory(csvDir.toPath()) + } catch { + case e: IOException => throw new KafkaException("Error in creating new directory '%s'".format(csvDir), e) + } + Files.createDirectory(csvDir.toPath()) underlying = new CsvReporter(Metrics.defaultRegistry(), csvDir) if (props.getBoolean("kafka.csv.metrics.reporter.enabled", default = false)) { initialized = true diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala index 825a55b5a8c5..7f336e906918 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala @@ -17,11 +17,12 @@ package kafka.log -import java.io.File +import java.io.{File, IOException} +import java.nio.file.Files import java.util.Properties import kafka.api.{KAFKA_0_10_0_IV1, KAFKA_0_9_0} -import kafka.common.TopicAndPartition +import kafka.common._ import kafka.message._ import kafka.server.OffsetCheckpoint import kafka.utils._ @@ -220,8 +221,12 @@ class LogCleanerIntegrationTest(compressionCodec: String) { val logs = new Pool[TopicAndPartition, Log]() for(i <- 0 until parts) { val dir = new File(logDir, "log-" + i) - dir.mkdirs() - + try { + Files.createDirectory(dir.toPath()) + } catch { + case e: IOException => throw new KafkaException("Error in creating new directory '%s'".format(dir), e) + } + val log = new Log(dir = dir, LogConfig(logConfigProperties(maxMessageSize, minCleanableDirtyRatio)), recoveryPoint = 0L, diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index 7b52a094c6c2..8fe8d2381f97 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.nio.file.Files import java.util.Properties import kafka.common._ @@ -242,7 +243,11 @@ class LogManagerTest { def testRecoveryDirectoryMappingWithRelativeDirectory() { logManager.shutdown() logDir = new File("data" + File.separator + logDir.getName) - logDir.mkdirs() + try { + Files.createDirectory(logDir.toPath()) + } catch { + case e: IOException => throw new KafkaException("Error in creating new directory '%s'".format(logDir), e) + } logDir.deleteOnExit() logManager = createLogManager() logManager.startup diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index 33dd68ef3ab4..777d6e7a97f3 100755 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -18,11 +18,12 @@ package kafka.log import java.io._ +import java.nio.file.Files import java.util.Properties import org.apache.kafka.common.errors.{CorruptRecordException, OffsetOutOfRangeException, RecordBatchTooLargeException, RecordTooLargeException} import kafka.api.ApiVersion -import kafka.common.LongRef +import kafka.common._ import org.junit.Assert._ import org.scalatest.junit.JUnitSuite import org.junit.{After, Before, Test} @@ -322,7 +323,11 @@ class LogTest extends JUnitSuite { @Test def testThatGarbageCollectingSegmentsDoesntChangeOffset() { for(messagesToAppend <- List(0, 1, 25)) { - logDir.mkdirs() + try { + Files.createDirectory(logDir.toPath()) + } catch { + case e: IOException => throw new KafkaException("Error in creating new directory '%s'".format(logDir), e) + } // first test a log segment starting at 0 val logProps = new Properties() logProps.put(LogConfig.SegmentBytesProp, 100: java.lang.Integer) @@ -799,7 +804,11 @@ class LogTest extends JUnitSuite { val recoveryPoint = 50L for(iteration <- 0 until 50) { // create a log and write some messages to it - logDir.mkdirs() + try { + Files.createDirectory(logDir.toPath()) + } catch { + case e: IOException => throw new KafkaException("Error in creating new directory '%s'".format(logDir), e) + } var log = new Log(logDir, config, recoveryPoint = 0L, diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 05b84eff38e5..cb386b079bbe 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -20,6 +20,7 @@ package kafka.utils import java.io._ import java.nio._ import java.nio.channels._ +import java.nio.file.Files import java.util.concurrent.{Callable, Executors, TimeUnit} import java.util.{Properties, Random} import java.security.cert.X509Certificate @@ -37,7 +38,7 @@ import kafka.api._ import kafka.cluster.{Broker, EndPoint} import kafka.consumer.{ConsumerConfig, ConsumerTimeoutException, KafkaStream} import kafka.serializer.{DefaultEncoder, Encoder, StringEncoder} -import kafka.common.TopicAndPartition +import kafka.common._ import kafka.admin.AdminUtils import kafka.log._ import kafka.utils.ZkUtils._ @@ -80,8 +81,11 @@ object TestUtils extends Logging { */ def tempRelativeDir(parent: String): File = { val parentFile = new File(parent) - parentFile.mkdirs() - + try { + Files.createDirectory(parentFile.toPath()) + } catch { + case e: IOException => throw new KafkaException("Error in creating new directory '%s'".format(parentFile), e) + } JTestUtils.tempDirectory(parentFile.toPath, null) } From 19997ede08bffeb95d17b2c41066e8943a391034 Mon Sep 17 00:00:00 2001 From: Manikumar Reddy O Date: Wed, 17 Aug 2016 08:05:18 -0700 Subject: [PATCH 02/35] KAFKA-4044; log actual socket send/receive buffer size after connecting in Selector Author: Manikumar Reddy O Reviewers: Jun Rao Closes #1750 from omkreddy/KAFKA-4044-LOG --- .../main/java/org/apache/kafka/common/network/Selector.java | 6 ++++++ 1 file changed, 6 insertions(+) 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 c33374181821..ab9dab998d4a 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 @@ -309,6 +309,12 @@ private void pollSelectionKeys(Iterable selectionKeys, boolean isI if (channel.finishConnect()) { this.connected.add(channel.id()); this.sensors.connectionCreated.record(); + SocketChannel socketChannel = (SocketChannel) key.channel(); + log.debug("Created socket with SO_RCVBUF = {}, SO_SNDBUF = {}, SO_TIMEOUT = {} to node {}", + socketChannel.socket().getReceiveBufferSize(), + socketChannel.socket().getSendBufferSize(), + socketChannel.socket().getSoTimeout(), + channel.id()); } else continue; } From 40b1dd3f495a59abef8a0cba5450526994c92c04 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Wed, 17 Aug 2016 11:50:04 -0700 Subject: [PATCH 03/35] KAFKA-3888: send consumer heartbeats from a background thread (KIP-62) Author: Jason Gustafson Reviewers: Ewen Cheslack-Postava , Ismael Juma , Guozhang Wang Closes #1627 from hachikuji/KAFKA-3888 --- .../consumer/CommitFailedException.java | 9 +- .../clients/consumer/ConsumerConfig.java | 39 +- .../kafka/clients/consumer/KafkaConsumer.java | 78 ++- .../internals/AbstractCoordinator.java | 532 ++++++++++++------ .../internals/ConsumerCoordinator.java | 264 +++++---- .../internals/ConsumerNetworkClient.java | 267 +++++---- .../consumer/internals/DelayedTask.java | 24 - .../consumer/internals/DelayedTaskQueue.java | 96 ---- .../clients/consumer/internals/Fetcher.java | 21 +- .../clients/consumer/internals/Heartbeat.java | 56 +- .../consumer/internals/RequestFuture.java | 2 +- .../kafka/common/protocol/Protocol.java | 25 +- .../common/requests/JoinGroupRequest.java | 49 +- .../common/requests/JoinGroupResponse.java | 15 +- .../common/requests/OffsetFetchResponse.java | 2 - .../clients/consumer/KafkaConsumerTest.java | 49 +- .../internals/AbstractCoordinatorTest.java | 68 ++- .../internals/ConsumerCoordinatorTest.java | 179 +++--- .../internals/ConsumerNetworkClientTest.java | 26 +- .../internals/DelayedTaskQueueTest.java | 89 --- .../consumer/internals/FetcherTest.java | 1 + .../consumer/internals/HeartbeatTest.java | 6 +- .../common/requests/RequestResponseTest.java | 14 +- .../distributed/DistributedConfig.java | 29 +- .../distributed/WorkerCoordinator.java | 38 +- .../distributed/WorkerGroupMember.java | 15 +- .../distributed/WorkerCoordinatorTest.java | 2 + .../src/main/scala/kafka/api/ApiVersion.scala | 11 +- .../kafka/coordinator/GroupCoordinator.scala | 90 +-- .../kafka/coordinator/GroupMetadata.scala | 4 +- .../coordinator/GroupMetadataManager.scala | 150 +++-- .../kafka/coordinator/MemberMetadata.scala | 1 + .../main/scala/kafka/server/KafkaApis.scala | 6 +- .../kafka/api/AuthorizerIntegrationTest.scala | 2 +- .../kafka/api/BaseConsumerTest.scala | 170 +----- .../kafka/api/ConsumerBounceTest.scala | 13 +- .../kafka/api/PlaintextConsumerTest.scala | 223 ++++++-- ...aslPlainSslEndToEndAuthorizationTest.scala | 1 - .../GroupCoordinatorResponseTest.scala | 213 ++++--- .../GroupMetadataManagerTest.scala | 11 +- .../kafka/coordinator/GroupMetadataTest.scala | 59 +- .../coordinator/MemberMetadataTest.scala | 16 +- .../unit/kafka/utils/timer/MockTimer.scala | 2 +- 43 files changed, 1686 insertions(+), 1281 deletions(-) delete mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedTask.java delete mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedTaskQueue.java delete mode 100644 clients/src/test/java/org/apache/kafka/clients/consumer/internals/DelayedTaskQueueTest.java diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/CommitFailedException.java b/clients/src/main/java/org/apache/kafka/clients/consumer/CommitFailedException.java index 26ef48e0b410..5695be83d6cb 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/CommitFailedException.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/CommitFailedException.java @@ -28,7 +28,12 @@ public class CommitFailedException extends KafkaException { private static final long serialVersionUID = 1L; - public CommitFailedException(String message) { - super(message); + public CommitFailedException() { + super("Commit cannot be completed since the group has already " + + "rebalanced and assigned the partitions to another member. This means that the time " + + "between subsequent calls to poll() was longer than the configured max.poll.interval.ms, " + + "which typically implies that the poll loop is spending too much time message processing. " + + "You can address this either by increasing the session timeout or by reducing the maximum " + + "size of batches returned in poll() with max.poll.records."); } } 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 de10bed20aed..509c3a1d7fac 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 @@ -48,24 +48,33 @@ public class ConsumerConfig extends AbstractConfig { public static final String MAX_POLL_RECORDS_CONFIG = "max.poll.records"; private static final String MAX_POLL_RECORDS_DOC = "The maximum number of records returned in a single call to poll()."; + /** max.poll.interval.ms */ + public static final String MAX_POLL_INTERVAL_MS_CONFIG = "max.poll.interval.ms"; + private static final String MAX_POLL_INTERVAL_MS_DOC = "The maximum delay between invocations of poll() when using " + + "consumer group management. This places an upper bound on the amount of time that the consumer can be idle " + + "before fetching more records. If poll() is not called before expiration of this timeout, then the consumer " + + "is considered failed and the group will rebalance in order to reassign the partitions to another member. "; + /** * session.timeout.ms */ public static final String SESSION_TIMEOUT_MS_CONFIG = "session.timeout.ms"; - private static final String SESSION_TIMEOUT_MS_DOC = "The timeout used to detect failures when using Kafka's " + - "group management facilities. When a consumer's heartbeat is not received within the session timeout, " + - "the broker will mark the consumer as failed and rebalance the group. Since heartbeats are sent only " + - "when poll() is invoked, a higher session timeout allows more time for message processing in the consumer's " + - "poll loop at the cost of a longer time to detect hard failures. See also " + MAX_POLL_RECORDS_CONFIG + " for " + - "another option to control the processing time in the poll loop. Note that the value must be in the " + - "allowable range as configured in the broker configuration by group.min.session.timeout.ms " + + private static final String SESSION_TIMEOUT_MS_DOC = "The timeout used to detect consumer failures when using " + + "Kafka's group management facility. The consumer sends periodic heartbeats to indicate its liveness " + + "to the broker. If no heartbeats are received by the broker before the expiration of this session timeout, " + + "then the broker will remove this consumer from the group and initiate a rebalance. Note that the value " + + "must be in the allowable range as configured in the broker configuration by group.min.session.timeout.ms " + "and group.max.session.timeout.ms."; /** * 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."; + 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 @@ -196,7 +205,7 @@ public class ConsumerConfig extends AbstractConfig { .define(GROUP_ID_CONFIG, Type.STRING, "", Importance.HIGH, GROUP_ID_DOC) .define(SESSION_TIMEOUT_MS_CONFIG, Type.INT, - 30000, + 10000, Importance.HIGH, SESSION_TIMEOUT_MS_DOC) .define(HEARTBEAT_INTERVAL_MS_CONFIG, @@ -221,7 +230,7 @@ public class ConsumerConfig extends AbstractConfig { Importance.MEDIUM, ENABLE_AUTO_COMMIT_DOC) .define(AUTO_COMMIT_INTERVAL_MS_CONFIG, - Type.LONG, + Type.INT, 5000, atLeast(0), Importance.LOW, @@ -311,7 +320,7 @@ public class ConsumerConfig extends AbstractConfig { VALUE_DESERIALIZER_CLASS_DOC) .define(REQUEST_TIMEOUT_MS_CONFIG, Type.INT, - 40 * 1000, + 305000, // chosen to be higher than the default of max.poll.interval.ms atLeast(0), Importance.MEDIUM, REQUEST_TIMEOUT_MS_DOC) @@ -328,10 +337,16 @@ public class ConsumerConfig extends AbstractConfig { INTERCEPTOR_CLASSES_DOC) .define(MAX_POLL_RECORDS_CONFIG, Type.INT, - Integer.MAX_VALUE, + 500, atLeast(1), Importance.MEDIUM, MAX_POLL_RECORDS_DOC) + .define(MAX_POLL_INTERVAL_MS_CONFIG, + Type.INT, + 300000, + atLeast(1), + Importance.MEDIUM, + MAX_POLL_INTERVAL_MS_DOC) .define(EXCLUDE_INTERNAL_TOPICS_CONFIG, Type.BOOLEAN, DEFAULT_EXCLUDE_INTERNAL_TOPICS, 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 522cfdee7bb3..ef913027361b 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 @@ -137,32 +137,31 @@ * After subscribing to a set of topics, the consumer will automatically join the group when {@link #poll(long)} is * invoked. The poll API is designed to ensure consumer liveness. As long as you continue to call poll, the consumer * will stay in the group and continue to receive messages from the partitions it was assigned. Underneath the covers, - * the poll API sends periodic heartbeats to the server; when you stop calling poll (perhaps because an exception was thrown), - * then no heartbeats will be sent. If a period of the configured session timeout elapses before the server - * has received a heartbeat, then the consumer will be kicked out of the group and its partitions will be reassigned. - * This is designed to prevent situations where the consumer has failed, yet continues to hold onto the partitions - * it was assigned (thus preventing active consumers in the group from taking them). To stay in the group, you - * have to prove you are still alive by calling poll. + * the consumer sends periodic heartbeats to the server. If the consumer crashes or is unable to send heartbeats for + * a duration of session.timeout.ms, then the consumer will be considered dead and its partitions will + * be reassigned. It is also possible that the consumer could encounter a "livelock" situation where it is continuing + * to send heartbeats, but no progress is being made. To prevent the consumer from holding onto its partitions + * indefinitely in this case, we provide a liveness detection mechanism: basically if you don't call poll at least + * as frequently as the configured poll.interval.ms, then the client will proactively leave the group + * so that another consumer can take over its partitions. So to stay in the group, you must continue to call poll *

* The implication of this design is that message processing time in the poll loop must be bounded so that - * heartbeats can be sent before expiration of the session timeout. What typically happens when processing time - * exceeds the session timeout is that the consumer won't be able to commit offsets for any of the processed records. - * For example, this is indicated by a {@link CommitFailedException} thrown from {@link #commitSync()}. This - * guarantees that only active members of the group are allowed to commit offsets. If the consumer - * has been kicked out of the group, then its partitions will have been assigned to another member, which will be - * committing its own offsets as it handles new records. This gives offset commits an isolation guarantee. + * you always ensure that poll() is called at least once every poll interval. If not, then the consumer leaves + * the group, which typically results in an offset commit failure when the processing of the polled records + * finally completes (this is indicated by a {@link CommitFailedException} thrown from {@link #commitSync()}). + * This is a safety mechanism which guarantees that only active members of the group are able to commit offsets. + * If the consumer has been kicked out of the group, then its partitions will have been assigned to another member, + * which will be committing its own offsets as it handles new records. This gives offset commits an isolation guarantee. *

- * The consumer provides two configuration settings to control this behavior: + * The consumer provides two configuration settings to control the behavior of the poll loop: *

    - *
  1. session.timeout.ms: By increasing the session timeout, you can give the consumer more - * time to handle a batch of records returned from {@link #poll(long)}. The only drawback is that it - * will take longer for the server to detect hard consumer failures, which can cause a delay before - * a rebalance can be completed. However, clean shutdown with {@link #close()} is not impacted since - * the consumer will send an explicit message to the server to leave the group and cause an immediate - * rebalance.
  2. - *
  3. max.poll.records: Processing time in the poll loop is typically proportional to the number - * of records processed, so it's natural to want to set a limit on the number of records handled at once. - * This setting provides that. By default, there is essentially no limit.
  4. + *
  5. max.poll.interval.ms: By increasing the interval between expected polls, you can give + * the consumer more time to handle a batch of records returned from {@link #poll(long)}. The drawback + * is that increasing this value may delay a group rebalance since the consumer will only join the rebalance + * inside the call to poll.
  6. + *
  7. max.poll.records: Use this setting to limit the total records returned from a single + * call to poll. This can make it easier to predict the maximum that must be handled within each poll + * interval.
  8. *
*

* For use cases where message processing time varies unpredictably, neither of these options may be viable. @@ -187,7 +186,6 @@ * props.put("group.id", "test"); * props.put("enable.auto.commit", "true"); * props.put("auto.commit.interval.ms", "1000"); - * props.put("session.timeout.ms", "30000"); * 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<>(props); @@ -210,13 +208,6 @@ * In this example the client is subscribing to the topics foo and bar as part of a group of consumers * called test as described above. *

- * The broker will automatically detect failed processes in the test group by using a heartbeat mechanism. The - * consumer will automatically ping the cluster periodically, which lets the cluster know that it is alive. Note that - * the consumer is single-threaded, so periodic heartbeats can only be sent when {@link #poll(long)} is called. As long as - * the consumer is able to do this it is considered alive and retains the right to consume from the partitions assigned - * to it. If it stops heartbeating by failing to call {@link #poll(long)} for a period of time longer than session.timeout.ms - * then it will be considered dead and its partitions will be assigned to another process. - *

* 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. * @@ -242,7 +233,6 @@ * props.put("bootstrap.servers", "localhost:9092"); * props.put("group.id", "test"); * props.put("enable.auto.commit", "false"); - * props.put("session.timeout.ms", "30000"); * 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<>(props); @@ -645,6 +635,7 @@ private KafkaConsumer(ConsumerConfig config, this.interceptors = interceptorList.isEmpty() ? null : new ConsumerInterceptors<>(interceptorList); this.coordinator = new ConsumerCoordinator(this.client, config.getString(ConsumerConfig.GROUP_ID_CONFIG), + config.getInt(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG), config.getInt(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG), config.getInt(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG), assignors, @@ -656,7 +647,7 @@ private KafkaConsumer(ConsumerConfig config, retryBackoffMs, new ConsumerCoordinator.DefaultOffsetCommitCallback(), config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG), - config.getLong(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG), + config.getInt(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG), this.interceptors, config.getBoolean(ConsumerConfig.EXCLUDE_INTERNAL_TOPICS_CONFIG)); if (keyDeserializer == null) { @@ -715,6 +706,9 @@ private KafkaConsumer(ConsumerConfig config, Metrics metrics, SubscriptionState subscriptions, Metadata metadata, + boolean autoCommitEnabled, + int autoCommitIntervalMs, + int heartbeatIntervalMs, long retryBackoffMs, long requestTimeoutMs) { this.clientId = clientId; @@ -970,7 +964,6 @@ public ConsumerRecords poll(long timeout) { // // NOTE: since the consumed position has already been updated, we must not allow // wakeups or any other errors to be triggered prior to returning the fetched records. - // Additionally, pollNoWakeup does not allow automatic commits to get triggered. fetcher.sendFetches(); client.pollNoWakeup(); @@ -997,30 +990,23 @@ public ConsumerRecords poll(long timeout) { * @return The fetched records (may be empty) */ private Map>> pollOnce(long timeout) { - // ensure we have partitions assigned if we expect to - if (subscriptions.partitionsAutoAssigned()) - coordinator.ensurePartitionAssignment(); + coordinator.poll(time.milliseconds()); // 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()); - long now = time.milliseconds(); - - // execute delayed tasks (e.g. autocommits and heartbeats) prior to fetching records - client.executeDelayedTasks(now); - - // init any new fetches (won't resend pending fetches) + // if data is available already, return it immediately Map>> records = fetcher.fetchedRecords(); - - // if data is available already, e.g. from a previous network client poll() call to commit, - // then just return it immediately if (!records.isEmpty()) return records; + // send any new fetches (won't resend pending fetches) fetcher.sendFetches(); - client.poll(timeout, now); + + long now = time.milliseconds(); + client.poll(Math.min(coordinator.timeToNextPoll(now), timeout), now); return fetcher.fetchedRecords(); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java index e957856536e6..690df2600d55 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.errors.GroupCoordinatorNotAvailableException; import org.apache.kafka.common.errors.IllegalGenerationException; import org.apache.kafka.common.errors.RebalanceInProgressException; +import org.apache.kafka.common.errors.RetriableException; import org.apache.kafka.common.errors.UnknownMemberIdException; import org.apache.kafka.common.metrics.Measurable; import org.apache.kafka.common.metrics.MetricConfig; @@ -53,6 +54,7 @@ import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; /** * AbstractCoordinator implements group management for a single group member by interacting with @@ -77,26 +79,38 @@ * by the leader in {@link #performAssignment(String, String, Map)} and becomes available to members in * {@link #onJoinComplete(int, String, String, ByteBuffer)}. * + * Note on locking: this class shares state between the caller and a background thread which is + * used for sending heartbeats after the client has joined the group. All mutable state as well as + * state transitions are protected with the class's monitor. Generally this means acquiring the lock + * before reading or writing the state of the group (e.g. generation, memberId) and holding the lock + * when sending a request that affects the state of the group (e.g. JoinGroup, LeaveGroup). */ public abstract class AbstractCoordinator implements Closeable { private static final Logger log = LoggerFactory.getLogger(AbstractCoordinator.class); - private final Heartbeat heartbeat; - private final HeartbeatTask heartbeatTask; + private enum MemberState { + UNJOINED, // the client is not part of a group + REBALANCING, // the client has begun rebalancing + STABLE, // the client has joined and is sending heartbeats + } + + private final int rebalanceTimeoutMs; private final int sessionTimeoutMs; private final GroupCoordinatorMetrics sensors; + private final Heartbeat heartbeat; protected final String groupId; protected final ConsumerNetworkClient client; protected final Time time; protected final long retryBackoffMs; - private boolean needsJoinPrepare = true; + private HeartbeatThread heartbeatThread = null; private boolean rejoinNeeded = true; - protected Node coordinator; - protected String memberId; - protected String protocol; - protected int generation; + private boolean needsJoinPrepare = true; + private MemberState state = MemberState.UNJOINED; + private RequestFuture joinFuture = null; + private Node coordinator = null; + private Generation generation = Generation.NO_GENERATION; private RequestFuture findCoordinatorFuture = null; @@ -105,6 +119,7 @@ public abstract class AbstractCoordinator implements Closeable { */ public AbstractCoordinator(ConsumerNetworkClient client, String groupId, + int rebalanceTimeoutMs, int sessionTimeoutMs, int heartbeatIntervalMs, Metrics metrics, @@ -113,19 +128,16 @@ public AbstractCoordinator(ConsumerNetworkClient client, long retryBackoffMs) { this.client = client; this.time = time; - this.generation = OffsetCommitRequest.DEFAULT_GENERATION_ID; - this.memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID; this.groupId = groupId; - this.coordinator = null; + this.rebalanceTimeoutMs = rebalanceTimeoutMs; this.sessionTimeoutMs = sessionTimeoutMs; - this.heartbeat = new Heartbeat(this.sessionTimeoutMs, heartbeatIntervalMs, time.milliseconds()); - this.heartbeatTask = new HeartbeatTask(); + this.heartbeat = new Heartbeat(sessionTimeoutMs, heartbeatIntervalMs, rebalanceTimeoutMs, retryBackoffMs); this.sensors = new GroupCoordinatorMetrics(metrics, metricGrpPrefix); this.retryBackoffMs = retryBackoffMs; } /** - * Unique identifier for the class of protocols implements (e.g. "consumer" or "connect"). + * Unique identifier for the class of supported protocols (e.g. "consumer" or "connect"). * @return Non-null protocol type name */ protected abstract String protocolType(); @@ -175,7 +187,7 @@ protected abstract void onJoinComplete(int generation, /** * Block until the coordinator for this group is known and is ready to receive requests. */ - public void ensureCoordinatorReady() { + public synchronized void ensureCoordinatorReady() { while (coordinatorUnknown()) { RequestFuture future = lookupCoordinator(); client.poll(future); @@ -216,14 +228,44 @@ public void onFailure(RuntimeException e) { * Check whether the group should be rejoined (e.g. if metadata changes) * @return true if it should, false otherwise */ - protected boolean needRejoin() { + protected synchronized boolean needRejoin() { return rejoinNeeded; } + /** + * Check the status of the heartbeat thread (if it is active) and indicate the liveness + * of the client. This must be called periodically after joining with {@link #ensureActiveGroup()} + * to ensure that the member stays in the group. If an interval of time longer than the + * provided rebalance timeout expires without calling this method, then the client will proactively + * leave the group. + * @param now current time in milliseconds + * @throws RuntimeException for unexpected errors raised from the heartbeat thread + */ + protected synchronized void pollHeartbeat(long now) { + if (heartbeatThread != null) { + if (heartbeatThread.hasFailed()) { + // set the heartbeat thread to null and raise an exception. If the user catches it, + // the next call to ensureActiveGroup() will spawn a new heartbeat thread. + RuntimeException cause = heartbeatThread.failureCause(); + heartbeatThread = null; + throw cause; + } + + heartbeat.poll(now); + } + } + + protected synchronized long timeToNextHeartbeat(long now) { + // if we have not joined the group, we don't need to send heartbeats + if (state == MemberState.UNJOINED) + return Long.MAX_VALUE; + return heartbeat.timeToNextHeartbeat(now); + } + /** * Ensure that the group is active (i.e. joined and synced) */ - public void ensureActiveGroup() { + public synchronized void ensureActiveGroup() { // always ensure that the coordinator is ready because we may have been disconnected // when sending heartbeats and does not necessarily require us to rejoin the group. ensureCoordinatorReady(); @@ -231,11 +273,18 @@ public void ensureActiveGroup() { if (!needRejoin()) return; + // call onJoinPrepare if needed. We set a flag to make sure that we do not call it a second + // time if the client is woken up before a pending rebalance completes. if (needsJoinPrepare) { - onJoinPrepare(generation, memberId); + onJoinPrepare(generation.generationId, generation.memberId); needsJoinPrepare = false; } + if (heartbeatThread == null) { + heartbeatThread = new HeartbeatThread(); + heartbeatThread.start(); + } + while (needRejoin()) { ensureCoordinatorReady(); @@ -246,23 +295,41 @@ public void ensureActiveGroup() { continue; } - RequestFuture future = sendJoinGroupRequest(); - future.addListener(new RequestFutureListener() { - @Override - public void onSuccess(ByteBuffer value) { - // handle join completion in the callback so that the callback will be invoked - // even if the consumer is woken up before finishing the rebalance - onJoinComplete(generation, memberId, protocol, value); - needsJoinPrepare = true; - heartbeatTask.reset(); - } + // we store the join future in case we are woken up by the user after beginning the + // rebalance in the call to poll below. This ensures that we do not mistakenly attempt + // to rejoin before the pending rebalance has completed. + if (joinFuture == null) { + state = MemberState.REBALANCING; + joinFuture = sendJoinGroupRequest(); + joinFuture.addListener(new RequestFutureListener() { + @Override + public void onSuccess(ByteBuffer value) { + // handle join completion in the callback so that the callback will be invoked + // even if the consumer is woken up before finishing the rebalance + synchronized (AbstractCoordinator.this) { + log.info("Successfully joined group {} with generation {}", groupId, generation.generationId); + joinFuture = null; + state = MemberState.STABLE; + needsJoinPrepare = true; + heartbeatThread.enable(); + } - @Override - public void onFailure(RuntimeException e) { - // we handle failures below after the request finishes. if the join completes - // after having been woken up, the exception is ignored and we will rejoin - } - }); + onJoinComplete(generation.generationId, generation.memberId, generation.protocol, value); + } + + @Override + public void onFailure(RuntimeException e) { + // we handle failures below after the request finishes. if the join completes + // after having been woken up, the exception is ignored and we will rejoin + synchronized (AbstractCoordinator.this) { + joinFuture = null; + state = MemberState.UNJOINED; + } + } + }); + } + + RequestFuture future = joinFuture; client.poll(future); if (future.failed()) { @@ -278,63 +345,6 @@ else if (!future.isRetriable()) } } - private class HeartbeatTask implements DelayedTask { - - private boolean requestInFlight = false; - - 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); - - if (!requestInFlight) - client.schedule(this, now); - } - - @Override - public void run(final long now) { - if (generation < 0 || needRejoin() || coordinatorUnknown()) { - // no need to send the heartbeat 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); - requestInFlight = true; - - RequestFuture future = sendHeartbeatRequest(); - future.addListener(new RequestFutureListener() { - @Override - public void onSuccess(Void value) { - requestInFlight = false; - long now = time.milliseconds(); - heartbeat.receiveHeartbeat(now); - long nextHeartbeatTime = now + heartbeat.timeToNextHeartbeat(now); - client.schedule(HeartbeatTask.this, nextHeartbeatTime); - } - - @Override - public void onFailure(RuntimeException e) { - requestInFlight = false; - client.schedule(HeartbeatTask.this, time.milliseconds() + retryBackoffMs); - } - }); - } - } - } - /** * Join the group and return the assignment for the next generation. This function handles both * JoinGroup and SyncGroup, delegating to {@link #performAssignment(String, String, Map)} if @@ -350,7 +360,8 @@ private RequestFuture sendJoinGroupRequest() { JoinGroupRequest request = new JoinGroupRequest( groupId, this.sessionTimeoutMs, - this.memberId, + this.rebalanceTimeoutMs, + this.generation.memberId, protocolType(), metadata()); @@ -359,7 +370,6 @@ private RequestFuture sendJoinGroupRequest() { .compose(new JoinGroupResponseHandler()); } - private class JoinGroupResponseHandler extends CoordinatorResponseHandler { @Override @@ -372,24 +382,32 @@ public void handle(JoinGroupResponse joinResponse, RequestFuture fut Errors error = Errors.forCode(joinResponse.errorCode()); if (error == Errors.NONE) { log.debug("Received successful join group response for group {}: {}", groupId, joinResponse.toStruct()); - AbstractCoordinator.this.memberId = joinResponse.memberId(); - AbstractCoordinator.this.generation = joinResponse.generationId(); - AbstractCoordinator.this.rejoinNeeded = false; - AbstractCoordinator.this.protocol = joinResponse.groupProtocol(); sensors.joinLatency.record(response.requestLatencyMs()); - if (joinResponse.isLeader()) { - onJoinLeader(joinResponse).chain(future); - } else { - onJoinFollower().chain(future); + + synchronized (AbstractCoordinator.this) { + if (state != MemberState.REBALANCING) { + // if the consumer was woken up before a rebalance completes, we may have already left + // the group. In this case, we do not want to continue with the sync group. + future.raise(new UnjoinedGroupException()); + } else { + AbstractCoordinator.this.generation = new Generation(joinResponse.generationId(), + joinResponse.memberId(), joinResponse.groupProtocol()); + AbstractCoordinator.this.rejoinNeeded = false; + if (joinResponse.isLeader()) { + onJoinLeader(joinResponse).chain(future); + } else { + onJoinFollower().chain(future); + } + } } } else if (error == Errors.GROUP_LOAD_IN_PROGRESS) { log.debug("Attempt to join group {} rejected since coordinator {} is loading the group.", groupId, - coordinator); + coordinator()); // backoff and retry future.raise(error); } else if (error == Errors.UNKNOWN_MEMBER_ID) { // reset the member id and retry immediately - AbstractCoordinator.this.memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID; + resetGeneration(); log.debug("Attempt to join group {} failed due to unknown member id.", groupId); future.raise(Errors.UNKNOWN_MEMBER_ID); } else if (error == Errors.GROUP_COORDINATOR_NOT_AVAILABLE @@ -415,8 +433,8 @@ public void handle(JoinGroupResponse joinResponse, RequestFuture fut private RequestFuture onJoinFollower() { // send follower's sync group with an empty assignment - SyncGroupRequest request = new SyncGroupRequest(groupId, generation, - memberId, Collections.emptyMap()); + SyncGroupRequest request = new SyncGroupRequest(groupId, generation.generationId, + generation.memberId, Collections.emptyMap()); log.debug("Sending follower SyncGroup for group {} to coordinator {}: {}", groupId, this.coordinator, request); return sendSyncGroupRequest(request); } @@ -427,7 +445,7 @@ private RequestFuture onJoinLeader(JoinGroupResponse joinResponse) { Map groupAssignment = performAssignment(joinResponse.leaderId(), joinResponse.groupProtocol(), joinResponse.members()); - SyncGroupRequest request = new SyncGroupRequest(groupId, generation, memberId, groupAssignment); + SyncGroupRequest request = new SyncGroupRequest(groupId, generation.generationId, generation.memberId, groupAssignment); log.debug("Sending leader SyncGroup for group {} to coordinator {}: {}", groupId, this.coordinator, request); return sendSyncGroupRequest(request); } catch (RuntimeException e) { @@ -454,11 +472,11 @@ public void handle(SyncGroupResponse syncResponse, RequestFuture future) { Errors error = Errors.forCode(syncResponse.errorCode()); if (error == Errors.NONE) { - log.info("Successfully joined group {} with generation {}", groupId, generation); sensors.syncLatency.record(response.requestLatencyMs()); future.complete(syncResponse.memberAssignment()); } else { - AbstractCoordinator.this.rejoinNeeded = true; + requestRejoin(); + if (error == Errors.GROUP_AUTHORIZATION_FAILED) { future.raise(new GroupAuthorizationException(groupId)); } else if (error == Errors.REBALANCE_IN_PROGRESS) { @@ -467,7 +485,7 @@ public void handle(SyncGroupResponse syncResponse, } else if (error == Errors.UNKNOWN_MEMBER_ID || error == Errors.ILLEGAL_GENERATION) { log.debug("SyncGroup for group {} failed due to {}", groupId, error); - AbstractCoordinator.this.memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID; + resetGeneration(); future.raise(error); } else if (error == Errors.GROUP_COORDINATOR_NOT_AVAILABLE || error == Errors.NOT_COORDINATOR_FOR_GROUP) { @@ -499,43 +517,36 @@ private RequestFuture sendGroupCoordinatorRequest() { log.debug("Sending coordinator request for group {} to broker {}", groupId, node); GroupCoordinatorRequest metadataRequest = new GroupCoordinatorRequest(this.groupId); return client.send(node, ApiKeys.GROUP_COORDINATOR, metadataRequest) - .compose(new RequestFutureAdapter() { - @Override - public void onSuccess(ClientResponse response, RequestFuture future) { - handleGroupMetadataResponse(response, future); - } - }); + .compose(new GroupCoordinatorResponseHandler()); } } - private void handleGroupMetadataResponse(ClientResponse resp, RequestFuture future) { - log.debug("Received group coordinator response {}", resp); + private class GroupCoordinatorResponseHandler extends RequestFutureAdapter { + + @Override + public void onSuccess(ClientResponse resp, RequestFuture future) { + log.debug("Received group coordinator response {}", resp); - if (!coordinatorUnknown()) { - // We already found the coordinator, so ignore the request - future.complete(null); - } else { GroupCoordinatorResponse groupCoordinatorResponse = new GroupCoordinatorResponse(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 Errors error = Errors.forCode(groupCoordinatorResponse.errorCode()); if (error == Errors.NONE) { - this.coordinator = new Node(Integer.MAX_VALUE - groupCoordinatorResponse.node().id(), - groupCoordinatorResponse.node().host(), - groupCoordinatorResponse.node().port()); - - log.info("Discovered coordinator {} for group {}.", coordinator, groupId); - - client.tryConnect(coordinator); - - // start sending heartbeats only if we have a valid generation - if (generation > 0) - heartbeatTask.reset(); + synchronized (AbstractCoordinator.this) { + AbstractCoordinator.this.coordinator = new Node( + Integer.MAX_VALUE - groupCoordinatorResponse.node().id(), + groupCoordinatorResponse.node().host(), + groupCoordinatorResponse.node().port()); + log.info("Discovered coordinator {} for group {}.", coordinator, groupId); + client.tryConnect(coordinator); + heartbeat.resetTimeouts(time.milliseconds()); + } future.complete(null); } else if (error == Errors.GROUP_AUTHORIZATION_FAILED) { future.raise(new GroupAuthorizationException(groupId)); } else { + log.debug("Group coordinator lookup for group {} failed: {}", groupId, error.message()); future.raise(error); } } @@ -546,21 +557,25 @@ private void handleGroupMetadataResponse(ClientResponse resp, RequestFuture 0) { + public synchronized void maybeLeaveGroup() { + if (!coordinatorUnknown() && state != MemberState.UNJOINED && generation != Generation.NO_GENERATION) { // this is a minimal effort attempt to leave the group. we do not // attempt any resending if the request fails or times out. - sendLeaveGroupRequest(); + LeaveGroupRequest request = new LeaveGroupRequest(groupId, generation.memberId); + client.send(coordinator, ApiKeys.LEAVE_GROUP, request) + .compose(new LeaveGroupResponseHandler()); + client.pollNoWakeup(); } - this.generation = OffsetCommitRequest.DEFAULT_GENERATION_ID; - this.memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID; - rejoinNeeded = true; - } - - private void sendLeaveGroupRequest() { - LeaveGroupRequest request = new LeaveGroupRequest(groupId, memberId); - RequestFuture future = client.send(coordinator, ApiKeys.LEAVE_GROUP, request) - .compose(new LeaveGroupResponseHandler()); - - future.addListener(new RequestFutureListener() { - @Override - public void onSuccess(Void value) {} - - @Override - public void onFailure(RuntimeException e) { - log.debug("LeaveGroup request for group {} failed with error", groupId, e); - } - }); - - client.poll(future, 0); + resetGeneration(); } private class LeaveGroupResponseHandler extends CoordinatorResponseHandler { + @Override public LeaveGroupResponse parse(ClientResponse response) { return new LeaveGroupResponse(response.responseBody()); @@ -620,25 +641,26 @@ public LeaveGroupResponse parse(ClientResponse response) { @Override public void handle(LeaveGroupResponse leaveResponse, RequestFuture future) { - // process the response - short errorCode = leaveResponse.errorCode(); - if (errorCode == Errors.NONE.code()) + Errors error = Errors.forCode(leaveResponse.errorCode()); + if (error == Errors.NONE) { + log.debug("LeaveGroup request for group {} returned successfully", groupId); future.complete(null); - else - future.raise(Errors.forCode(errorCode)); + } else { + log.debug("LeaveGroup request for group {} failed with error: {}", groupId, error.message()); + future.raise(error); + } } } - /** - * Send a heartbeat request now (visible only for testing). - */ - public RequestFuture sendHeartbeatRequest() { - HeartbeatRequest req = new HeartbeatRequest(this.groupId, this.generation, this.memberId); + // visible for testing + synchronized RequestFuture sendHeartbeatRequest() { + HeartbeatRequest req = new HeartbeatRequest(this.groupId, this.generation.generationId, this.generation.memberId); return client.send(coordinator, ApiKeys.HEARTBEAT, req) - .compose(new HeartbeatCompletionHandler()); + .compose(new HeartbeatResponseHandler()); } - private class HeartbeatCompletionHandler extends CoordinatorResponseHandler { + private class HeartbeatResponseHandler extends CoordinatorResponseHandler { + @Override public HeartbeatResponse parse(ClientResponse response) { return new HeartbeatResponse(response.responseBody()); @@ -654,21 +676,20 @@ public void handle(HeartbeatResponse heartbeatResponse, RequestFuture futu } else if (error == Errors.GROUP_COORDINATOR_NOT_AVAILABLE || error == Errors.NOT_COORDINATOR_FOR_GROUP) { log.debug("Attempt to heart beat failed for group {} since coordinator {} is either not started or not valid.", - groupId, coordinator); + groupId, coordinator()); coordinatorDead(); future.raise(error); } else if (error == Errors.REBALANCE_IN_PROGRESS) { log.debug("Attempt to heart beat failed for group {} since it is rebalancing.", groupId); - AbstractCoordinator.this.rejoinNeeded = true; + requestRejoin(); future.raise(Errors.REBALANCE_IN_PROGRESS); } else if (error == Errors.ILLEGAL_GENERATION) { log.debug("Attempt to heart beat failed for group {} since generation id is not legal.", groupId); - AbstractCoordinator.this.rejoinNeeded = true; + resetGeneration(); future.raise(Errors.ILLEGAL_GENERATION); } else if (error == Errors.UNKNOWN_MEMBER_ID) { log.debug("Attempt to heart beat failed for group {} since member id is not valid.", groupId); - memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID; - AbstractCoordinator.this.rejoinNeeded = true; + resetGeneration(); future.raise(Errors.UNKNOWN_MEMBER_ID); } else if (error == Errors.GROUP_AUTHORIZATION_FAILED) { future.raise(new GroupAuthorizationException(groupId)); @@ -678,8 +699,7 @@ public void handle(HeartbeatResponse heartbeatResponse, RequestFuture futu } } - protected abstract class CoordinatorResponseHandler - extends RequestFutureAdapter { + protected abstract class CoordinatorResponseHandler extends RequestFutureAdapter { protected ClientResponse response; public abstract R parse(ClientResponse response); @@ -758,9 +778,149 @@ public double measure(MetricConfig config, long now) { }; metrics.addMetric(metrics.metricName("last-heartbeat-seconds-ago", this.metricGrpName, - "The number of seconds since the last controller heartbeat"), + "The number of seconds since the last controller heartbeat was sent"), lastHeartbeat); } } + private class HeartbeatThread extends Thread { + private boolean enabled = false; + private boolean closed = false; + private AtomicReference failed = new AtomicReference<>(null); + + public void enable() { + synchronized (AbstractCoordinator.this) { + this.enabled = true; + heartbeat.resetTimeouts(time.milliseconds()); + AbstractCoordinator.this.notify(); + } + } + + public void disable() { + synchronized (AbstractCoordinator.this) { + this.enabled = false; + } + } + + public void close() { + synchronized (AbstractCoordinator.this) { + this.closed = true; + AbstractCoordinator.this.notify(); + } + } + + private boolean hasFailed() { + return failed.get() != null; + } + + private RuntimeException failureCause() { + return failed.get(); + } + + @Override + public void run() { + try { + RequestFuture findCoordinatorFuture = null; + + while (true) { + synchronized (AbstractCoordinator.this) { + if (closed) + return; + + if (!enabled) { + AbstractCoordinator.this.wait(); + continue; + } + + if (state != MemberState.STABLE) { + // the group is not stable (perhaps because we left the group or because the coordinator + // kicked us out), so disable heartbeats and wait for the main thread to rejoin. + disable(); + continue; + } + + client.pollNoWakeup(); + long now = time.milliseconds(); + + if (coordinatorUnknown()) { + if (findCoordinatorFuture == null || findCoordinatorFuture.isDone()) + findCoordinatorFuture = lookupCoordinator(); + else + AbstractCoordinator.this.wait(retryBackoffMs); + } else if (heartbeat.sessionTimeoutExpired(now)) { + // the session timeout has expired without seeing a successful heartbeat, so we should + // probably make sure the coordinator is still healthy. + coordinatorDead(); + } else if (heartbeat.pollTimeoutExpired(now)) { + // the poll timeout has expired, which means that the foreground thread has stalled + // in between calls to poll(), so we explicitly leave the group. + maybeLeaveGroup(); + } else if (!heartbeat.shouldHeartbeat(now)) { + // poll again after waiting for the retry backoff in case the heartbeat failed or the + // coordinator disconnected + AbstractCoordinator.this.wait(retryBackoffMs); + } else { + heartbeat.sentHeartbeat(now); + + sendHeartbeatRequest().addListener(new RequestFutureListener() { + @Override + public void onSuccess(Void value) { + synchronized (AbstractCoordinator.this) { + heartbeat.receiveHeartbeat(time.milliseconds()); + } + } + + @Override + public void onFailure(RuntimeException e) { + synchronized (AbstractCoordinator.this) { + if (e instanceof RebalanceInProgressException) { + // it is valid to continue heartbeating while the group is rebalancing. This + // ensures that the coordinator keeps the member in the group for as long + // as the duration of the rebalance timeout. If we stop sending heartbeats, + // however, then the session timeout may expire before we can rejoin. + heartbeat.receiveHeartbeat(time.milliseconds()); + } else { + heartbeat.failHeartbeat(); + + // wake up the thread if it's sleeping to reschedule the heartbeat + AbstractCoordinator.this.notify(); + } + } + } + }); + } + } + } + } catch (InterruptedException e) { + log.error("Unexpected interrupt received in heartbeat thread for group {}", groupId, e); + this.failed.set(new RuntimeException(e)); + } catch (RuntimeException e) { + log.error("Heartbeat thread for group {} failed due to unexpected error" , groupId, e); + this.failed.set(e); + } + } + + } + + protected static class Generation { + public static final Generation NO_GENERATION = new Generation( + OffsetCommitRequest.DEFAULT_GENERATION_ID, + JoinGroupRequest.UNKNOWN_MEMBER_ID, + null); + + public final int generationId; + public final String memberId; + public final String protocol; + + public Generation(int generationId, String memberId, String protocol) { + this.generationId = generationId; + this.memberId = memberId; + this.protocol = protocol; + } + } + + private static class UnjoinedGroupException extends RetriableException { + + } + } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java index 81a40f132a8c..5fee45afe831 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java @@ -18,12 +18,13 @@ import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.OffsetCommitCallback; +import org.apache.kafka.clients.consumer.RetriableCommitFailedException; import org.apache.kafka.clients.consumer.internals.PartitionAssignor.Assignment; import org.apache.kafka.clients.consumer.internals.PartitionAssignor.Subscription; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.clients.consumer.RetriableCommitFailedException; import org.apache.kafka.common.errors.GroupAuthorizationException; import org.apache.kafka.common.errors.RetriableException; import org.apache.kafka.common.errors.TopicAuthorizationException; @@ -54,6 +55,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.ConcurrentLinkedQueue; /** * This class manages the coordination process with the consumer coordinator. @@ -68,18 +70,24 @@ public final class ConsumerCoordinator extends AbstractCoordinator { private final SubscriptionState subscriptions; private final OffsetCommitCallback defaultOffsetCommitCallback; private final boolean autoCommitEnabled; - private final AutoCommitTask autoCommitTask; + private final int autoCommitIntervalMs; private final ConsumerInterceptors interceptors; private final boolean excludeInternalTopics; + // this collection must be thread-safe because it is modified from the response handler + // of offset commit requests, which may be invoked from the heartbeat thread + private final ConcurrentLinkedQueue completedOffsetCommits; + private MetadataSnapshot metadataSnapshot; private MetadataSnapshot assignmentSnapshot; + private long nextAutoCommitDeadline; /** * Initialize the coordination manager. */ public ConsumerCoordinator(ConsumerNetworkClient client, String groupId, + int rebalanceTimeoutMs, int sessionTimeoutMs, int heartbeatIntervalMs, List assignors, @@ -91,11 +99,12 @@ public ConsumerCoordinator(ConsumerNetworkClient client, long retryBackoffMs, OffsetCommitCallback defaultOffsetCommitCallback, boolean autoCommitEnabled, - long autoCommitIntervalMs, + int autoCommitIntervalMs, ConsumerInterceptors interceptors, boolean excludeInternalTopics) { super(client, groupId, + rebalanceTimeoutMs, sessionTimeoutMs, heartbeatIntervalMs, metrics, @@ -103,26 +112,22 @@ public ConsumerCoordinator(ConsumerNetworkClient client, time, retryBackoffMs); this.metadata = metadata; - - this.metadata.requestUpdate(); this.metadataSnapshot = new MetadataSnapshot(subscriptions, metadata.fetch()); this.subscriptions = subscriptions; this.defaultOffsetCommitCallback = defaultOffsetCommitCallback; this.autoCommitEnabled = autoCommitEnabled; + this.autoCommitIntervalMs = autoCommitIntervalMs; this.assignors = assignors; - - addMetadataListener(); - - if (autoCommitEnabled) { - this.autoCommitTask = new AutoCommitTask(autoCommitIntervalMs); - this.autoCommitTask.reschedule(); - } else { - this.autoCommitTask = null; - } - + this.completedOffsetCommits = new ConcurrentLinkedQueue<>(); this.sensors = new ConsumerCoordinatorMetrics(metrics, metricGrpPrefix); this.interceptors = interceptors; this.excludeInternalTopics = excludeInternalTopics; + + if (autoCommitEnabled) + this.nextAutoCommitDeadline = time.milliseconds() + autoCommitIntervalMs; + + this.metadata.requestUpdate(); + addMetadataListener(); } @Override @@ -210,8 +215,7 @@ protected void onJoinComplete(int generation, assignor.onAssignment(assignment); // reschedule the auto commit starting from now - if (autoCommitEnabled) - autoCommitTask.reschedule(); + this.nextAutoCommitDeadline = time.milliseconds() + autoCommitIntervalMs; // execute the user's callback after rebalance ConsumerRebalanceListener listener = subscriptions.listener(); @@ -227,6 +231,54 @@ protected void onJoinComplete(int generation, } } + /** + * Poll for coordinator events. This ensures that the coordinator is known and that the consumer + * has joined the group (if it is using group management). This also handles periodic offset commits + * if they are enabled. + * + * @param now current time in milliseconds + */ + public void poll(long now) { + invokeCompletedOffsetCommitCallbacks(); + + if (subscriptions.partitionsAutoAssigned() && coordinatorUnknown()) { + ensureCoordinatorReady(); + now = time.milliseconds(); + } + + if (subscriptions.partitionsAutoAssigned() && needRejoin()) { + // due to a race condition between the initial metadata fetch and the initial rebalance, we need to ensure that + // the metadata is fresh before joining initially, and then request the metadata update. If metadata update arrives + // while the rebalance is still pending (for example, when the join group is still inflight), then we will lose + // track of the fact that we need to rebalance again to reflect the change to the topic subscription. Without + // ensuring that the metadata is fresh, any metadata update that changes the topic subscriptions and arrives with a + // rebalance in progress will essentially be ignored. See KAFKA-3949 for the complete description of the problem. + if (subscriptions.hasPatternSubscription()) + client.ensureFreshMetadata(); + + ensureActiveGroup(); + now = time.milliseconds(); + } + + pollHeartbeat(now); + maybeAutoCommitOffsetsAsync(now); + } + + /** + * Return the time to the next needed invocation of {@link #poll(long)}. + * @param now current time in milliseconds + * @return the maximum time in milliseconds the caller should wait before the next invocation of poll() + */ + public long timeToNextPoll(long now) { + if (!autoCommitEnabled) + return timeToNextHeartbeat(now); + + if (now > nextAutoCommitDeadline) + return 0; + + return Math.min(nextAutoCommitDeadline - now, timeToNextHeartbeat(now)); + } + @Override protected Map performAssignment(String leaderId, String assignmentStrategy, @@ -292,7 +344,7 @@ protected void onJoinPrepare(int generation, String memberId) { } @Override - public boolean needRejoin() { + protected boolean needRejoin() { return subscriptions.partitionsAutoAssigned() && (super.needRejoin() || subscriptions.partitionAssignmentNeeded()); } @@ -336,24 +388,6 @@ public Map fetchCommittedOffsets(Set offsets, final OffsetCommitCallback callback) { + invokeCompletedOffsetCommitCallbacks(); + if (!coordinatorUnknown()) { doCommitOffsetsAsync(offsets, callback); } else { @@ -384,7 +430,7 @@ public void onSuccess(Void value) { @Override public void onFailure(RuntimeException e) { - callback.onComplete(offsets, new RetriableCommitFailedException(e)); + completedOffsetCommits.add(new OffsetCommitCompletion(callback, offsets, new RetriableCommitFailedException(e))); } }); } @@ -404,16 +450,18 @@ private void doCommitOffsetsAsync(final Map o public void onSuccess(Void value) { if (interceptors != null) interceptors.onCommit(offsets); - cb.onComplete(offsets, null); + + completedOffsetCommits.add(new OffsetCommitCompletion(cb, offsets, null)); } @Override public void onFailure(RuntimeException e) { - if (e instanceof RetriableException) { - cb.onComplete(offsets, new RetriableCommitFailedException(e)); - } else { - cb.onComplete(offsets, e); - } + Exception commitException = e; + + if (e instanceof RetriableException) + commitException = new RetriableCommitFailedException(e); + + completedOffsetCommits.add(new OffsetCommitCompletion(cb, offsets, commitException)); } }); } @@ -427,6 +475,8 @@ public void onFailure(RuntimeException e) { * @throws CommitFailedException if an unrecoverable error occurs before the commit can be completed */ public void commitOffsetsSync(Map offsets) { + invokeCompletedOffsetCommitCallbacks(); + if (offsets.isEmpty()) return; @@ -449,46 +499,25 @@ public void commitOffsetsSync(Map offsets) { } } - private class AutoCommitTask implements DelayedTask { - private final long interval; - - public AutoCommitTask(long interval) { - this.interval = interval; - } - - private void reschedule() { - client.schedule(this, time.milliseconds() + interval); - } - - private void reschedule(long at) { - client.schedule(this, at); - } - - public void run(final long now) { + private void maybeAutoCommitOffsetsAsync(long now) { + if (autoCommitEnabled) { if (coordinatorUnknown()) { - log.debug("Cannot auto-commit offsets for group {} since the coordinator is unknown", groupId); - reschedule(now + retryBackoffMs); - return; - } - - if (needRejoin()) { - // skip the commit when we're rejoining since we'll commit offsets synchronously - // before the revocation callback is invoked - reschedule(now + interval); - return; - } - - commitOffsetsAsync(subscriptions.allConsumed(), new OffsetCommitCallback() { - @Override - public void onComplete(Map offsets, Exception exception) { - if (exception == null) { - reschedule(now + interval); - } else { - log.warn("Auto offset commit failed for group {}: {}", groupId, exception.getMessage()); - reschedule(now + interval); + this.nextAutoCommitDeadline = now + retryBackoffMs; + } else if (now >= nextAutoCommitDeadline) { + this.nextAutoCommitDeadline = now + autoCommitIntervalMs; + commitOffsetsAsync(subscriptions.allConsumed(), new OffsetCommitCallback() { + @Override + public void onComplete(Map offsets, Exception exception) { + if (exception != null) { + log.warn("Auto offset commit failed for group {}: {}", groupId, exception.getMessage()); + if (exception instanceof RetriableException) + nextAutoCommitDeadline = Math.min(time.milliseconds() + retryBackoffMs, nextAutoCommitDeadline); + } else { + log.debug("Completed autocommit of offsets {} for group {}", offsets, groupId); + } } - } - }); + }); + } } } @@ -506,6 +535,14 @@ private void maybeAutoCommitOffsetsSync() { } } + public static class DefaultOffsetCommitCallback implements OffsetCommitCallback { + @Override + public void onComplete(Map offsets, Exception exception) { + if (exception != null) + log.error("Offset commit failed.", exception); + } + } + /** * 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 @@ -515,12 +552,13 @@ private void maybeAutoCommitOffsetsSync() { * @return A request future whose value indicates whether the commit was successful or not */ private RequestFuture sendOffsetCommitRequest(final Map offsets) { - if (coordinatorUnknown()) - return RequestFuture.coordinatorNotAvailable(); - if (offsets.isEmpty()) return RequestFuture.voidSuccess(); + Node coordinator = coordinator(); + if (coordinator == null) + return RequestFuture.coordinatorNotAvailable(); + // create the offset commit request Map offsetData = new HashMap<>(offsets.size()); for (Map.Entry entry : offsets.entrySet()) { @@ -529,9 +567,21 @@ private RequestFuture sendOffsetCommitRequest(final Map sendOffsetCommitRequest(final Map offsets, Exception exception) { - if (exception != null) - log.error("Offset commit failed.", exception); - } - } - private class OffsetCommitResponseHandler extends CoordinatorResponseHandler { private final Map offsets; @@ -607,13 +649,8 @@ public void handle(OffsetCommitResponse commitResponse, RequestFuture futu || error == Errors.REBALANCE_IN_PROGRESS) { // need to re-join group log.debug("Offset commit for group {} failed: {}", groupId, error.message()); - subscriptions.needReassignment(); - future.raise(new CommitFailedException("Commit cannot be completed since the group has already " + - "rebalanced and assigned the partitions to another member. This means that the time " + - "between subsequent calls to poll() was longer than the configured session.timeout.ms, " + - "which typically implies that the poll loop is spending too much time message processing. " + - "You can address this either by increasing the session timeout or by reducing the maximum " + - "size of batches returned in poll() with max.poll.records.")); + resetGeneration(); + future.raise(new CommitFailedException()); return; } else { log.error("Group {} failed to commit partition {} at offset {}: {}", groupId, tp, offset, error.message()); @@ -639,7 +676,8 @@ public void handle(OffsetCommitResponse commitResponse, RequestFuture futu * @return A request future containing the committed offsets. */ private RequestFuture> sendOffsetFetchRequest(Set partitions) { - if (coordinatorUnknown()) + Node coordinator = coordinator(); + if (coordinator == null) return RequestFuture.coordinatorNotAvailable(); log.debug("Group {} fetching committed offsets for partitions: {}", groupId, partitions); @@ -675,11 +713,6 @@ public void handle(OffsetFetchResponse response, RequestFuture offsets; + private final Exception exception; + + public OffsetCommitCompletion(OffsetCommitCallback callback, Map offsets, Exception exception) { + this.callback = callback; + this.offsets = offsets; + this.exception = exception; + } + + public void invoke() { + callback.onComplete(offsets, exception); + } + } } 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 index b65a5b7b203e..07edd3c8986c 100644 --- 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 @@ -22,6 +22,7 @@ import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.WakeupException; import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.requests.AbstractRequest; import org.apache.kafka.common.requests.RequestHeader; import org.apache.kafka.common.requests.RequestSend; @@ -36,27 +37,34 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.atomic.AtomicBoolean; /** - * Higher level consumer access to the network layer with basic support for futures and - * task scheduling. This class is not thread-safe, except for wakeup(). + * Higher level consumer access to the network layer with basic support for request futures. This class + * is thread-safe, but provides no synchronization for response callbacks. This guarantees that no locks + * are held when they are invoked. */ public class ConsumerNetworkClient implements Closeable { private static final Logger log = LoggerFactory.getLogger(ConsumerNetworkClient.class); + // the mutable state of this class is protected by the object's monitor (excluding the wakeup + // flag and the request completion queue below). 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; private final long unsentExpiryMs; - - // this count is only accessed from the consumer's main thread private int wakeupDisabledCount = 0; + // when requests complete, they are transferred to this queue prior to invocation. The purpose + // is to avoid invoking them while holding the lock above. + private final ConcurrentLinkedQueue pendingCompletion = new ConcurrentLinkedQueue<>(); + + // this flag allows the client to be safely woken up without waiting on the lock above. It is + // atomic to avoid the need to acquire the lock above in order to enable it concurrently. + private final AtomicBoolean wakeup = new AtomicBoolean(false); public ConsumerNetworkClient(KafkaClient client, Metadata metadata, @@ -70,25 +78,6 @@ public ConsumerNetworkClient(KafkaClient client, this.unsentExpiryMs = requestTimeoutMs; } - /** - * 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 @@ -104,25 +93,36 @@ public void unschedule(DelayedTask task) { public RequestFuture send(Node node, ApiKeys api, AbstractRequest request) { + return send(node, api, ProtoUtils.latestVersion(api.id), request); + } + + private RequestFuture send(Node node, + ApiKeys api, + short version, + AbstractRequest request) { long now = time.milliseconds(); - RequestFutureCompletionHandler future = new RequestFutureCompletionHandler(); - RequestHeader header = client.nextRequestHeader(api); + RequestFutureCompletionHandler completionHandler = new RequestFutureCompletionHandler(); + RequestHeader header = client.nextRequestHeader(api, version); RequestSend send = new RequestSend(node.idString(), header, request.toStruct()); - put(node, new ClientRequest(now, true, send, future)); - return future; + put(node, new ClientRequest(now, true, send, completionHandler)); + return completionHandler.future; } private void put(Node node, ClientRequest request) { - List nodeUnsent = unsent.get(node); - if (nodeUnsent == null) { - nodeUnsent = new ArrayList<>(); - unsent.put(node, nodeUnsent); + synchronized (this) { + List nodeUnsent = unsent.get(node); + if (nodeUnsent == null) { + nodeUnsent = new ArrayList<>(); + unsent.put(node, nodeUnsent); + } + nodeUnsent.add(request); } - nodeUnsent.add(request); } public Node leastLoadedNode() { - return client.leastLoadedNode(time.milliseconds()); + synchronized (this) { + return client.leastLoadedNode(time.milliseconds()); + } } /** @@ -149,6 +149,8 @@ public void ensureFreshMetadata() { * on the current poll if one is active, or the next poll. */ public void wakeup() { + // wakeup should be safe without holding the client lock since it simply delegates to + // Selector's wakeup, which is threadsafe this.wakeup.set(true); this.client.wakeup(); } @@ -175,7 +177,7 @@ public boolean poll(RequestFuture future, long timeout) { long remaining = timeout; long now = begin; do { - poll(remaining, now, true); + poll(remaining, now); now = time.milliseconds(); long elapsed = now - begin; remaining = timeout - elapsed; @@ -189,7 +191,7 @@ public boolean poll(RequestFuture future, long timeout) { * @throws WakeupException if {@link #wakeup()} is called from another thread */ public void poll(long timeout) { - poll(timeout, time.milliseconds(), true); + poll(timeout, time.milliseconds()); } /** @@ -198,7 +200,37 @@ public void poll(long timeout) { * @param now current time in milliseconds */ public void poll(long timeout, long now) { - poll(timeout, now, true); + // there may be handlers which need to be invoked if we woke up the previous call to poll + firePendingCompletedRequests(); + + synchronized (this) { + // send all the requests we can send now + trySend(now); + + // ensure we don't poll any longer than the deadline for + // the next scheduled task + client.poll(timeout, now); + now = time.milliseconds(); + + // handle any disconnects by failing the active requests. note that disconnects must + // be checked immediately following poll since any subsequent call to client.ready() + // will reset the disconnect status + checkDisconnects(now); + + // trigger wakeups after checking for disconnects so that the callbacks will be ready + // to be fired on the next call to poll() + maybeTriggerWakeup(); + + // try again to send requests since buffer space may have been + // cleared or a connect finished in the poll + trySend(now); + + // fail requests that couldn't be sent if they have expired + failExpiredRequests(now); + } + + // called without the lock to avoid deadlock potential if handlers need to acquire locks + firePendingCompletedRequests(); } /** @@ -208,49 +240,12 @@ public void poll(long timeout, long now) { public void pollNoWakeup() { disableWakeups(); try { - poll(0, time.milliseconds(), false); + poll(0, time.milliseconds()); } finally { enableWakeups(); } } - private void poll(long timeout, long now, boolean executeDelayedTasks) { - // send all the requests we can send now - trySend(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); - now = time.milliseconds(); - - // handle any disconnects by failing the active requests. note that disconnects must - // be checked immediately following poll since any subsequent call to client.ready() - // will reset the disconnect status - checkDisconnects(now); - - // execute scheduled tasks - if (executeDelayedTasks) - delayedTasks.poll(now); - - // try again to send requests since buffer space may have been - // cleared or a connect finished in the poll - trySend(now); - - // fail requests that couldn't be sent if they have expired - failExpiredRequests(now); - } - - /** - * Execute delayed tasks now. - * @param now current time in milliseconds - * @throws WakeupException if a wakeup has been requested - */ - public void executeDelayedTasks(long now) { - delayedTasks.poll(now); - maybeTriggerWakeup(); - } - /** * Block until all pending requests from the given node have finished. * @param node The node to await requests from @@ -267,9 +262,11 @@ public void awaitPendingRequests(Node node) { * @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()); + synchronized (this) { + List pending = unsent.get(node); + int unsentCount = pending == null ? 0 : pending.size(); + return unsentCount + client.inFlightRequestCount(node.idString()); + } } /** @@ -278,10 +275,22 @@ public int pendingRequestCount(Node node) { * @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(); + synchronized (this) { + int total = 0; + for (List requests: unsent.values()) + total += requests.size(); + return total + client.inFlightRequestCount(); + } + } + + private void firePendingCompletedRequests() { + for (;;) { + RequestFutureCompletionHandler completionHandler = pendingCompletion.poll(); + if (completionHandler == null) + break; + + completionHandler.fireCompletion(); + } } private void checkDisconnects(long now) { @@ -315,9 +324,8 @@ private void failExpiredRequests(long now) { while (requestIterator.hasNext()) { ClientRequest request = requestIterator.next(); if (request.createdTimeMs() < now - unsentExpiryMs) { - RequestFutureCompletionHandler handler = - (RequestFutureCompletionHandler) request.callback(); - handler.raise(new TimeoutException("Failed to send request after " + unsentExpiryMs + " ms.")); + RequestFutureCompletionHandler handler = (RequestFutureCompletionHandler) request.callback(); + handler.onFailure(new TimeoutException("Failed to send request after " + unsentExpiryMs + " ms.")); requestIterator.remove(); } else break; @@ -327,15 +335,20 @@ private void failExpiredRequests(long now) { } } - protected void failUnsentRequests(Node node, RuntimeException e) { + public void failUnsentRequests(Node node, RuntimeException e) { // clear unsent requests to node and fail their corresponding futures - List unsentRequests = unsent.remove(node); - if (unsentRequests != null) { - for (ClientRequest request : unsentRequests) { - RequestFutureCompletionHandler handler = (RequestFutureCompletionHandler) request.callback(); - handler.raise(e); + synchronized (this) { + List unsentRequests = unsent.remove(node); + if (unsentRequests != null) { + for (ClientRequest request : unsentRequests) { + RequestFutureCompletionHandler handler = (RequestFutureCompletionHandler) request.callback(); + handler.onFailure(e); + } } } + + // called without the lock to avoid deadlock potential + firePendingCompletedRequests(); } private boolean trySend(long now) { @@ -356,11 +369,6 @@ private boolean trySend(long now) { return requestsSent; } - private void clientPoll(long timeout, long now) { - client.poll(timeout, now); - maybeTriggerWakeup(); - } - private void maybeTriggerWakeup() { if (wakeupDisabledCount == 0 && wakeup.get()) { wakeup.set(false); @@ -369,24 +377,30 @@ private void maybeTriggerWakeup() { } public void disableWakeups() { - wakeupDisabledCount++; + synchronized (this) { + wakeupDisabledCount++; + } } public void enableWakeups() { - if (wakeupDisabledCount <= 0) - throw new IllegalStateException("Cannot enable wakeups since they were never disabled"); + synchronized (this) { + if (wakeupDisabledCount <= 0) + throw new IllegalStateException("Cannot enable wakeups since they were never disabled"); - wakeupDisabledCount--; + wakeupDisabledCount--; - // re-wakeup the client if the flag was set since previous wake-up call - // could be cleared by poll(0) while wakeups were disabled - if (wakeupDisabledCount == 0 && wakeup.get()) - this.client.wakeup(); + // re-wakeup the client if the flag was set since previous wake-up call + // could be cleared by poll(0) while wakeups were disabled + if (wakeupDisabledCount == 0 && wakeup.get()) + this.client.wakeup(); + } } @Override public void close() throws IOException { - client.close(); + synchronized (this) { + client.close(); + } } /** @@ -395,7 +409,9 @@ public void close() throws IOException { * @param node Node to connect to if possible */ public boolean connectionFailed(Node node) { - return client.connectionFailed(node); + synchronized (this) { + return client.connectionFailed(node); + } } /** @@ -405,26 +421,45 @@ public boolean connectionFailed(Node node) { * @param node The node to connect to */ public void tryConnect(Node node) { - client.ready(node, time.milliseconds()); + synchronized (this) { + client.ready(node, time.milliseconds()); + } } - public static class RequestFutureCompletionHandler - extends RequestFuture - implements RequestCompletionHandler { + public class RequestFutureCompletionHandler implements RequestCompletionHandler { + private final RequestFuture future; + private ClientResponse response; + private RuntimeException e; - @Override - public void onComplete(ClientResponse response) { - if (response.wasDisconnected()) { + public RequestFutureCompletionHandler() { + this.future = new RequestFuture<>(); + } + + public void fireCompletion() { + if (e != null) { + future.raise(e); + } else if (response.wasDisconnected()) { ClientRequest request = response.request(); RequestSend send = request.request(); ApiKeys api = ApiKeys.forId(send.header().apiKey()); int correlation = send.header().correlationId(); log.debug("Cancelled {} request {} with correlation id {} due to node {} being disconnected", api, request, correlation, send.destination()); - raise(DisconnectException.INSTANCE); + future.raise(DisconnectException.INSTANCE); } else { - complete(response); + future.complete(response); } } + + public void onFailure(RuntimeException e) { + this.e = e; + pendingCompletion.add(this); + } + + @Override + public void onComplete(ClientResponse response) { + this.response = response; + pendingCompletion.add(this); + } } } 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 deleted file mode 100644 index 61663f8ba8cc..000000000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedTask.java +++ /dev/null @@ -1,24 +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.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 deleted file mode 100644 index 61cab20af4a1..000000000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedTaskQueue.java +++ /dev/null @@ -1,96 +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.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 913ce9e422ed..84278c6cd557 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 @@ -65,6 +65,7 @@ import java.util.Locale; import java.util.Map; import java.util.Set; +import java.util.concurrent.ConcurrentLinkedQueue; /** * This class manage the fetching process with the brokers. @@ -84,7 +85,7 @@ public class Fetcher { private final Metadata metadata; private final FetchManagerMetrics sensors; private final SubscriptionState subscriptions; - private final List completedFetches; + private final ConcurrentLinkedQueue completedFetches; private final Deserializer keyDeserializer; private final Deserializer valueDeserializer; @@ -115,7 +116,7 @@ public Fetcher(ConsumerNetworkClient client, this.checkCrcs = checkCrcs; this.keyDeserializer = keyDeserializer; this.valueDeserializer = valueDeserializer; - this.completedFetches = new ArrayList<>(); + this.completedFetches = new ConcurrentLinkedQueue<>(); this.sensors = new FetchManagerMetrics(metrics, metricGrpPrefix); this.retryBackoffMs = retryBackoffMs; } @@ -127,7 +128,8 @@ public Fetcher(ConsumerNetworkClient client, public void sendFetches() { for (Map.Entry fetchEntry: createFetchRequests().entrySet()) { final FetchRequest request = fetchEntry.getValue(); - client.send(fetchEntry.getKey(), ApiKeys.FETCH, request) + final Node fetchTarget = fetchEntry.getKey(); + client.send(fetchTarget, ApiKeys.FETCH, request) .addListener(new RequestFutureListener() { @Override public void onSuccess(ClientResponse resp) { @@ -148,7 +150,7 @@ public void onSuccess(ClientResponse resp) { @Override public void onFailure(RuntimeException e) { - log.debug("Fetch failed", e); + log.debug("Fetch request to {} failed", fetchTarget, e); } }); } @@ -353,16 +355,14 @@ public Map>> fetchedRecords() { } else { Map>> drained = new HashMap<>(); int recordsRemaining = maxPollRecords; - Iterator completedFetchesIterator = completedFetches.iterator(); while (recordsRemaining > 0) { if (nextInLineRecords == null || nextInLineRecords.isEmpty()) { - if (!completedFetchesIterator.hasNext()) + CompletedFetch completedFetch = completedFetches.poll(); + if (completedFetch == null) break; - CompletedFetch completion = completedFetchesIterator.next(); - completedFetchesIterator.remove(); - nextInLineRecords = parseFetchedData(completion); + nextInLineRecords = parseFetchedData(completedFetch); } else { recordsRemaining -= append(drained, nextInLineRecords, recordsRemaining); } @@ -510,6 +510,8 @@ private Map createFetchRequests() { long position = this.subscriptions.position(partition); fetch.put(partition, new FetchRequest.PartitionData(position, this.fetchSize)); log.trace("Added fetch request for partition {} at offset {}", partition, position); + } else { + log.trace("Skipping fetch for partition {} because there is an inflight request to {}", partition, node); } } @@ -845,4 +847,5 @@ public void recordTopicFetchMetrics(String topic, int bytes, int records) { recordsFetched.record(records); } } + } 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 79e17e219f09..dff100601034 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,26 +16,41 @@ * A helper class for managing the heartbeat to the coordinator */ public final class Heartbeat { - private final long timeout; - private final long interval; + private final long sessionTimeout; + private final long heartbeatInterval; + private final long maxPollInterval; + private final long retryBackoffMs; - private long lastHeartbeatSend; + private volatile long lastHeartbeatSend; // volatile since it is read by metrics private long lastHeartbeatReceive; private long lastSessionReset; + private long lastPoll; + private boolean heartbeatFailed; - public Heartbeat(long timeout, - long interval, - long now) { - if (interval >= timeout) + public Heartbeat(long sessionTimeout, + long heartbeatInterval, + long maxPollInterval, + long retryBackoffMs) { + if (heartbeatInterval >= sessionTimeout) throw new IllegalArgumentException("Heartbeat must be set lower than the session timeout"); - this.timeout = timeout; - this.interval = interval; - this.lastSessionReset = now; + this.sessionTimeout = sessionTimeout; + this.heartbeatInterval = heartbeatInterval; + this.maxPollInterval = maxPollInterval; + this.retryBackoffMs = retryBackoffMs; + } + + public void poll(long now) { + this.lastPoll = now; } public void sentHeartbeat(long now) { this.lastHeartbeatSend = now; + this.heartbeatFailed = false; + } + + public void failHeartbeat() { + this.heartbeatFailed = true; } public void receiveHeartbeat(long now) { @@ -52,23 +67,34 @@ public long lastHeartbeatSend() { public long timeToNextHeartbeat(long now) { long timeSinceLastHeartbeat = now - Math.max(lastHeartbeatSend, lastSessionReset); + final long delayToNextHeartbeat; + if (heartbeatFailed) + delayToNextHeartbeat = retryBackoffMs; + else + delayToNextHeartbeat = heartbeatInterval; - if (timeSinceLastHeartbeat > interval) + if (timeSinceLastHeartbeat > delayToNextHeartbeat) return 0; else - return interval - timeSinceLastHeartbeat; + return delayToNextHeartbeat - timeSinceLastHeartbeat; } public boolean sessionTimeoutExpired(long now) { - return now - Math.max(lastSessionReset, lastHeartbeatReceive) > timeout; + return now - Math.max(lastSessionReset, lastHeartbeatReceive) > sessionTimeout; } public long interval() { - return interval; + return heartbeatInterval; } - public void resetSessionTimeout(long now) { + public void resetTimeouts(long now) { this.lastSessionReset = now; + this.lastPoll = now; + this.heartbeatFailed = false; + } + + public boolean pollTimeoutExpired(long now) { + return now - lastPoll > maxPollInterval; } } \ 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 index 71c16faf9527..b21d13e3e54d 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 @@ -196,7 +196,7 @@ public static RequestFuture failure(RuntimeException e) { } public static RequestFuture voidSuccess() { - RequestFuture future = new RequestFuture(); + RequestFuture future = new RequestFuture<>(); future.complete(null); return future; } 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 d27ec8a1ec09..313477fd3f8f 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 @@ -572,9 +572,28 @@ public class Protocol { new ArrayOf(JOIN_GROUP_REQUEST_PROTOCOL_V0), "List of protocols that the member supports")); + public static final Schema JOIN_GROUP_REQUEST_V1 = new Schema(new Field("group_id", + STRING, + "The group id."), + new Field("session_timeout", + INT32, + "The coordinator considers the consumer dead if it receives no heartbeat after this timeout in ms."), + new Field("rebalance_timeout", + INT32, + "The maximum time that the coordinator will wait for each member to rejoin when rebalancing the group"), + new Field("member_id", + STRING, + "The assigned consumer id or an empty string for a new consumer."), + new Field("protocol_type", + STRING, + "Unique name for class of protocols implemented by group"), + new Field("group_protocols", + new ArrayOf(JOIN_GROUP_REQUEST_PROTOCOL_V0), + "List of protocols that the member supports")); public static final Schema JOIN_GROUP_RESPONSE_MEMBER_V0 = new Schema(new Field("member_id", STRING), new Field("member_metadata", BYTES)); + public static final Schema JOIN_GROUP_RESPONSE_V0 = new Schema(new Field("error_code", INT16), new Field("generation_id", INT32, @@ -591,8 +610,10 @@ public class Protocol { new Field("members", new ArrayOf(JOIN_GROUP_RESPONSE_MEMBER_V0))); - public static final Schema[] JOIN_GROUP_REQUEST = new Schema[] {JOIN_GROUP_REQUEST_V0}; - public static final Schema[] JOIN_GROUP_RESPONSE = new Schema[] {JOIN_GROUP_RESPONSE_V0}; + public static final Schema JOIN_GROUP_RESPONSE_V1 = JOIN_GROUP_RESPONSE_V0; + + public static final Schema[] JOIN_GROUP_REQUEST = new Schema[] {JOIN_GROUP_REQUEST_V0, JOIN_GROUP_REQUEST_V1}; + public static final Schema[] JOIN_GROUP_RESPONSE = new Schema[] {JOIN_GROUP_RESPONSE_V0, JOIN_GROUP_RESPONSE_V1}; /* SyncGroup api */ public static final Schema SYNC_GROUP_REQUEST_MEMBER_V0 = new Schema(new Field("member_id", STRING), 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 14a6c1de4364..2845ee079b6b 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 @@ -24,10 +24,11 @@ import java.util.List; public class JoinGroupRequest extends AbstractRequest { - + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.JOIN_GROUP.id); private static final String GROUP_ID_KEY_NAME = "group_id"; private static final String SESSION_TIMEOUT_KEY_NAME = "session_timeout"; + private static final String REBALANCE_TIMEOUT_KEY_NAME = "rebalance_timeout"; private static final String MEMBER_ID_KEY_NAME = "member_id"; private static final String PROTOCOL_TYPE_KEY_NAME = "protocol_type"; private static final String GROUP_PROTOCOLS_KEY_NAME = "group_protocols"; @@ -38,6 +39,7 @@ public class JoinGroupRequest extends AbstractRequest { private final String groupId; private final int sessionTimeout; + private final int rebalanceTimeout; private final String memberId; private final String protocolType; private final List groupProtocols; @@ -60,14 +62,40 @@ public ByteBuffer metadata() { } } + // v0 constructor + @Deprecated + public JoinGroupRequest(String groupId, + int sessionTimeout, + String memberId, + String protocolType, + List groupProtocols) { + this(0, groupId, sessionTimeout, sessionTimeout, memberId, protocolType, groupProtocols); + } + public JoinGroupRequest(String groupId, int sessionTimeout, + int rebalanceTimeout, String memberId, String protocolType, List groupProtocols) { - super(new Struct(CURRENT_SCHEMA)); + this(1, groupId, sessionTimeout, rebalanceTimeout, memberId, protocolType, groupProtocols); + } + + private JoinGroupRequest(int version, + String groupId, + int sessionTimeout, + int rebalanceTimeout, + String memberId, + String protocolType, + List groupProtocols) { + super(new Struct(ProtoUtils.requestSchema(ApiKeys.JOIN_GROUP.id, version))); + struct.set(GROUP_ID_KEY_NAME, groupId); struct.set(SESSION_TIMEOUT_KEY_NAME, sessionTimeout); + + if (version >= 1) + struct.set(REBALANCE_TIMEOUT_KEY_NAME, rebalanceTimeout); + struct.set(MEMBER_ID_KEY_NAME, memberId); struct.set(PROTOCOL_TYPE_KEY_NAME, protocolType); @@ -82,6 +110,7 @@ public JoinGroupRequest(String groupId, struct.set(GROUP_PROTOCOLS_KEY_NAME, groupProtocolsList.toArray()); this.groupId = groupId; this.sessionTimeout = sessionTimeout; + this.rebalanceTimeout = rebalanceTimeout; this.memberId = memberId; this.protocolType = protocolType; this.groupProtocols = groupProtocols; @@ -89,8 +118,17 @@ public JoinGroupRequest(String groupId, public JoinGroupRequest(Struct struct) { super(struct); + groupId = struct.getString(GROUP_ID_KEY_NAME); sessionTimeout = struct.getInt(SESSION_TIMEOUT_KEY_NAME); + + if (struct.hasField(REBALANCE_TIMEOUT_KEY_NAME)) + // rebalance timeout is added in v1 + rebalanceTimeout = struct.getInt(REBALANCE_TIMEOUT_KEY_NAME); + else + // v0 had no rebalance timeout but used session timeout implicitly + rebalanceTimeout = sessionTimeout; + memberId = struct.getString(MEMBER_ID_KEY_NAME); protocolType = struct.getString(PROTOCOL_TYPE_KEY_NAME); @@ -107,13 +145,16 @@ public JoinGroupRequest(Struct struct) { public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { switch (versionId) { case 0: + case 1: return new JoinGroupResponse( + versionId, Errors.forException(e).code(), JoinGroupResponse.UNKNOWN_GENERATION_ID, JoinGroupResponse.UNKNOWN_PROTOCOL, JoinGroupResponse.UNKNOWN_MEMBER_ID, // memberId JoinGroupResponse.UNKNOWN_MEMBER_ID, // leaderId Collections.emptyMap()); + 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))); @@ -128,6 +169,10 @@ public int sessionTimeout() { return sessionTimeout; } + public int rebalanceTimeout() { + return rebalanceTimeout; + } + public String memberId() { return memberId; } 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 dd829ed72fc2..8895ace37d95 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 @@ -24,7 +24,8 @@ import java.util.Map; public class JoinGroupResponse extends AbstractRequestResponse { - + + private static final short CURRENT_VERSION = ProtoUtils.latestVersion(ApiKeys.JOIN_GROUP.id); private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.JOIN_GROUP.id); private static final String ERROR_CODE_KEY_NAME = "error_code"; @@ -65,7 +66,17 @@ public JoinGroupResponse(short errorCode, String memberId, String leaderId, Map groupMembers) { - super(new Struct(CURRENT_SCHEMA)); + this(CURRENT_VERSION, errorCode, generationId, groupProtocol, memberId, leaderId, groupMembers); + } + + public JoinGroupResponse(int version, + short errorCode, + int generationId, + String groupProtocol, + String memberId, + String leaderId, + Map groupMembers) { + super(new Struct(ProtoUtils.responseSchema(ApiKeys.JOIN_GROUP.id, version))); struct.set(ERROR_CODE_KEY_NAME, errorCode); struct.set(GENERATION_ID_KEY_NAME, generationId); 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 a76f48ea9991..6cf93a00ef94 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 @@ -50,8 +50,6 @@ public class OffsetFetchResponse extends AbstractRequestResponse { * UNKNOWN_TOPIC_OR_PARTITION (3) <- only for request v0 * GROUP_LOAD_IN_PROGRESS (14) * NOT_COORDINATOR_FOR_GROUP (16) - * ILLEGAL_GENERATION (22) - * UNKNOWN_MEMBER_ID (25) * TOPIC_AUTHORIZATION_FAILED (29) * GROUP_AUTHORIZATION_FAILED (30) */ diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 8b526641b028..8d2ac0068e8a 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -90,8 +90,7 @@ public void testConstructorClose() throws Exception { final int oldInitCount = MockMetricsReporter.INIT_COUNT.get(); final int oldCloseCount = MockMetricsReporter.CLOSE_COUNT.get(); try { - KafkaConsumer consumer = new KafkaConsumer<>( - props, new ByteArrayDeserializer(), new ByteArrayDeserializer()); + new KafkaConsumer<>(props, new ByteArrayDeserializer(), new ByteArrayDeserializer()); } catch (KafkaException e) { assertEquals(oldInitCount + 1, MockMetricsReporter.INIT_COUNT.get()); assertEquals(oldCloseCount + 1, MockMetricsReporter.CLOSE_COUNT.get()); @@ -314,17 +313,17 @@ private KafkaConsumer newConsumer() { props.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999"); props.setProperty(ConsumerConfig.METRIC_REPORTER_CLASSES_CONFIG, MockMetricsReporter.class.getName()); - return new KafkaConsumer( - props, new ByteArrayDeserializer(), new ByteArrayDeserializer()); + return new KafkaConsumer<>(props, new ByteArrayDeserializer(), new ByteArrayDeserializer()); } @Test - public void verifyHeartbeatSent() { + public void verifyHeartbeatSent() throws Exception { String topic = "topic"; TopicPartition partition = new TopicPartition(topic, 0); + int rebalanceTimeoutMs = 60000; int sessionTimeoutMs = 30000; - int heartbeatIntervalMs = 3000; + int heartbeatIntervalMs = 1000; int autoCommitIntervalMs = 10000; Time time = new MockTime(); @@ -337,7 +336,7 @@ public void verifyHeartbeatSent() { PartitionAssignor assignor = new RoundRobinAssignor(); final KafkaConsumer consumer = newConsumer(time, client, metadata, assignor, - sessionTimeoutMs, heartbeatIntervalMs, autoCommitIntervalMs); + rebalanceTimeoutMs, sessionTimeoutMs, heartbeatIntervalMs, autoCommitIntervalMs); consumer.subscribe(Arrays.asList(topic), new ConsumerRebalanceListener() { @Override @@ -370,9 +369,6 @@ public void onPartitionsAssigned(Collection partitions) { consumer.poll(0); assertEquals(Collections.singleton(partition), consumer.assignment()); - // heartbeat interval is 2 seconds - time.sleep(heartbeatIntervalMs); - final AtomicBoolean heartbeatReceived = new AtomicBoolean(false); client.prepareResponseFrom(new MockClient.RequestMatcher() { @Override @@ -382,18 +378,23 @@ public boolean matches(ClientRequest request) { } }, new HeartbeatResponse(Errors.NONE.code()).toStruct(), coordinator); + // heartbeat interval is 2 seconds + time.sleep(heartbeatIntervalMs); + Thread.sleep(heartbeatIntervalMs); + consumer.poll(0); assertTrue(heartbeatReceived.get()); } @Test - public void verifyHeartbeatSentWhenFetchedDataReady() { + public void verifyHeartbeatSentWhenFetchedDataReady() throws Exception { String topic = "topic"; TopicPartition partition = new TopicPartition(topic, 0); + int rebalanceTimeoutMs = 60000; int sessionTimeoutMs = 30000; - int heartbeatIntervalMs = 3000; + int heartbeatIntervalMs = 1000; int autoCommitIntervalMs = 10000; Time time = new MockTime(); @@ -406,7 +407,7 @@ public void verifyHeartbeatSentWhenFetchedDataReady() { PartitionAssignor assignor = new RoundRobinAssignor(); final KafkaConsumer consumer = newConsumer(time, client, metadata, assignor, - sessionTimeoutMs, heartbeatIntervalMs, autoCommitIntervalMs); + rebalanceTimeoutMs, sessionTimeoutMs, heartbeatIntervalMs, autoCommitIntervalMs); consumer.subscribe(Arrays.asList(topic), new ConsumerRebalanceListener() { @Override public void onPartitionsRevoked(Collection partitions) { @@ -438,8 +439,6 @@ public void onPartitionsAssigned(Collection partitions) { client.respondFrom(fetchResponse(partition, 0, 5), node); client.poll(0, time.milliseconds()); - time.sleep(heartbeatIntervalMs); - client.prepareResponseFrom(fetchResponse(partition, 5, 0), node); final AtomicBoolean heartbeatReceived = new AtomicBoolean(false); client.prepareResponseFrom(new MockClient.RequestMatcher() { @@ -450,6 +449,9 @@ public boolean matches(ClientRequest request) { } }, new HeartbeatResponse(Errors.NONE.code()).toStruct(), coordinator); + time.sleep(heartbeatIntervalMs); + Thread.sleep(heartbeatIntervalMs); + consumer.poll(0); assertTrue(heartbeatReceived.get()); @@ -459,6 +461,7 @@ public boolean matches(ClientRequest request) { public void verifyNoCoordinatorLookupForManualAssignmentWithSeek() { String topic = "topic"; final TopicPartition partition = new TopicPartition(topic, 0); + int rebalanceTimeoutMs = 60000; int sessionTimeoutMs = 3000; int heartbeatIntervalMs = 2000; int autoCommitIntervalMs = 1000; @@ -473,7 +476,7 @@ public void verifyNoCoordinatorLookupForManualAssignmentWithSeek() { PartitionAssignor assignor = new RoundRobinAssignor(); final KafkaConsumer consumer = newConsumer(time, client, metadata, assignor, - sessionTimeoutMs, heartbeatIntervalMs, autoCommitIntervalMs); + rebalanceTimeoutMs, sessionTimeoutMs, heartbeatIntervalMs, autoCommitIntervalMs); consumer.assign(Arrays.asList(partition)); consumer.seekToBeginning(Arrays.asList(partition)); @@ -496,6 +499,7 @@ public void testCommitsFetchedDuringAssign() { long offset1 = 10000; long offset2 = 20000; + int rebalanceTimeoutMs = 6000; int sessionTimeoutMs = 3000; int heartbeatIntervalMs = 2000; int autoCommitIntervalMs = 1000; @@ -510,7 +514,7 @@ public void testCommitsFetchedDuringAssign() { PartitionAssignor assignor = new RoundRobinAssignor(); final KafkaConsumer consumer = newConsumer(time, client, metadata, assignor, - sessionTimeoutMs, heartbeatIntervalMs, autoCommitIntervalMs); + rebalanceTimeoutMs, sessionTimeoutMs, heartbeatIntervalMs, autoCommitIntervalMs); consumer.assign(Arrays.asList(partition1)); // lookup coordinator @@ -541,6 +545,7 @@ public void testAutoCommitSentBeforePositionUpdate() { String topic = "topic"; final TopicPartition partition = new TopicPartition(topic, 0); + int rebalanceTimeoutMs = 60000; int sessionTimeoutMs = 30000; int heartbeatIntervalMs = 3000; @@ -558,7 +563,7 @@ public void testAutoCommitSentBeforePositionUpdate() { PartitionAssignor assignor = new RoundRobinAssignor(); final KafkaConsumer consumer = newConsumer(time, client, metadata, assignor, - sessionTimeoutMs, heartbeatIntervalMs, autoCommitIntervalMs); + rebalanceTimeoutMs, sessionTimeoutMs, heartbeatIntervalMs, autoCommitIntervalMs); consumer.subscribe(Arrays.asList(topic), new ConsumerRebalanceListener() { @Override public void onPartitionsRevoked(Collection partitions) { @@ -619,6 +624,7 @@ public void testWakeupWithFetchDataAvailable() { String topic = "topic"; final TopicPartition partition = new TopicPartition(topic, 0); + int rebalanceTimeoutMs = 60000; int sessionTimeoutMs = 30000; int heartbeatIntervalMs = 3000; @@ -636,7 +642,7 @@ public void testWakeupWithFetchDataAvailable() { PartitionAssignor assignor = new RoundRobinAssignor(); final KafkaConsumer consumer = newConsumer(time, client, metadata, assignor, - sessionTimeoutMs, heartbeatIntervalMs, autoCommitIntervalMs); + rebalanceTimeoutMs, sessionTimeoutMs, heartbeatIntervalMs, autoCommitIntervalMs); consumer.subscribe(Arrays.asList(topic), new ConsumerRebalanceListener() { @Override public void onPartitionsRevoked(Collection partitions) { @@ -725,6 +731,7 @@ private KafkaConsumer newConsumer(Time time, KafkaClient client, Metadata metadata, PartitionAssignor assignor, + int rebalanceTimeoutMs, int sessionTimeoutMs, int heartbeatIntervalMs, int autoCommitIntervalMs) { @@ -757,6 +764,7 @@ private KafkaConsumer newConsumer(Time time, ConsumerCoordinator consumerCoordinator = new ConsumerCoordinator( consumerClient, groupId, + rebalanceTimeoutMs, sessionTimeoutMs, heartbeatIntervalMs, assignors, @@ -800,6 +808,9 @@ private KafkaConsumer newConsumer(Time time, metrics, subscriptions, metadata, + autoCommitEnabled, + autoCommitIntervalMs, + heartbeatIntervalMs, retryBackoffMs, requestTimeoutMs); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java index 7a05eb1fb807..77f9df5d47f9 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java @@ -16,15 +16,20 @@ **/ 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.common.Cluster; import org.apache.kafka.common.Node; import org.apache.kafka.common.metrics.Metrics; +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.GroupCoordinatorResponse; +import org.apache.kafka.common.requests.HeartbeatResponse; import org.apache.kafka.common.requests.JoinGroupRequest; +import org.apache.kafka.common.requests.JoinGroupResponse; +import org.apache.kafka.common.requests.SyncGroupResponse; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; import org.apache.kafka.test.TestUtils; @@ -37,12 +42,15 @@ import java.util.List; import java.util.Map; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; public class AbstractCoordinatorTest { private static final ByteBuffer EMPTY_DATA = ByteBuffer.wrap(new byte[0]); - private static final int SESSION_TIMEOUT_MS = 30000; + private static final int REBALANCE_TIMEOUT_MS = 60000; + private static final int SESSION_TIMEOUT_MS = 10000; private static final int HEARTBEAT_INTERVAL_MS = 3000; private static final long RETRY_BACKOFF_MS = 100; private static final long REQUEST_TIMEOUT_MS = 40000; @@ -77,8 +85,8 @@ public void setupCoordinator() { @Test public void testCoordinatorDiscoveryBackoff() { - mockClient.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); - mockClient.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); + mockClient.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); + mockClient.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); // blackout the coordinator for 50 milliseconds to simulate a disconnect. // after backing off, we should be able to connect. @@ -91,17 +99,65 @@ public void testCoordinatorDiscoveryBackoff() { assertTrue(endTime - initialTime >= RETRY_BACKOFF_MS); } - private Struct groupCoordinatorResponse(Node node, short error) { - GroupCoordinatorResponse response = new GroupCoordinatorResponse(error, node); + @Test + public void testUncaughtExceptionInHeartbeatThread() throws Exception { + mockClient.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); + mockClient.prepareResponse(joinGroupFollowerResponse(1, "memberId", "leaderId", Errors.NONE)); + mockClient.prepareResponse(syncGroupResponse(Errors.NONE)); + + + final RuntimeException e = new RuntimeException(); + + // raise the error when the background thread tries to send a heartbeat + mockClient.prepareResponse(new MockClient.RequestMatcher() { + @Override + public boolean matches(ClientRequest request) { + if (request.request().header().apiKey() == ApiKeys.HEARTBEAT.id) + throw e; + return false; + } + }, heartbeatResponse(Errors.UNKNOWN)); + + try { + coordinator.ensureActiveGroup(); + mockTime.sleep(HEARTBEAT_INTERVAL_MS); + synchronized (coordinator) { + coordinator.notify(); + } + Thread.sleep(100); + + coordinator.pollHeartbeat(mockTime.milliseconds()); + fail("Expected pollHeartbeat to raise an error"); + } catch (RuntimeException exception) { + assertEquals(exception, e); + } + } + + private Struct groupCoordinatorResponse(Node node, Errors error) { + GroupCoordinatorResponse response = new GroupCoordinatorResponse(error.code(), node); return response.toStruct(); } + private Struct heartbeatResponse(Errors error) { + HeartbeatResponse response = new HeartbeatResponse(error.code()); + return response.toStruct(); + } + + private Struct joinGroupFollowerResponse(int generationId, String memberId, String leaderId, Errors error) { + return new JoinGroupResponse(error.code(), generationId, "dummy-subprotocol", memberId, leaderId, + Collections.emptyMap()).toStruct(); + } + + private Struct syncGroupResponse(Errors error) { + return new SyncGroupResponse(error.code(), ByteBuffer.allocate(0)).toStruct(); + } + public class DummyCoordinator extends AbstractCoordinator { public DummyCoordinator(ConsumerNetworkClient client, Metrics metrics, Time time) { - super(client, GROUP_ID, SESSION_TIMEOUT_MS, HEARTBEAT_INTERVAL_MS, metrics, + super(client, GROUP_ID, REBALANCE_TIMEOUT_MS, SESSION_TIMEOUT_MS, HEARTBEAT_INTERVAL_MS, metrics, METRIC_GROUP_PREFIX, time, RETRY_BACKOFF_MS); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java index 176571cca5f1..8ec8b7578b9c 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java @@ -26,13 +26,13 @@ import org.apache.kafka.clients.consumer.OffsetCommitCallback; import org.apache.kafka.clients.consumer.OffsetResetStrategy; import org.apache.kafka.clients.consumer.RangeAssignor; +import org.apache.kafka.clients.consumer.RetriableCommitFailedException; import org.apache.kafka.clients.consumer.RoundRobinAssignor; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.ApiException; -import org.apache.kafka.clients.consumer.RetriableCommitFailedException; import org.apache.kafka.common.errors.DisconnectException; import org.apache.kafka.common.errors.GroupAuthorizationException; import org.apache.kafka.common.errors.OffsetMetadataTooLarge; @@ -79,11 +79,12 @@ public class ConsumerCoordinatorTest { private String topicName = "test"; private String groupId = "test-group"; private TopicPartition tp = new TopicPartition(topicName, 0); + private int rebalanceTimeoutMs = 60000; private int sessionTimeoutMs = 10000; private int heartbeatIntervalMs = 5000; private long retryBackoffMs = 100; private boolean autoCommitEnabled = false; - private long autoCommitIntervalMs = 2000; + private int autoCommitIntervalMs = 2000; private MockPartitionAssignor partitionAssignor = new MockPartitionAssignor(); private List assignors = Collections.singletonList(partitionAssignor); private MockTime time; @@ -123,7 +124,7 @@ public void teardown() { @Test public void testNormalHeartbeat() { - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); // normal heartbeat @@ -141,7 +142,7 @@ public void testNormalHeartbeat() { @Test(expected = GroupAuthorizationException.class) public void testGroupDescribeUnauthorized() { - client.prepareResponse(consumerMetadataResponse(node, Errors.GROUP_AUTHORIZATION_FAILED.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.GROUP_AUTHORIZATION_FAILED.code())); coordinator.ensureCoordinatorReady(); } @@ -149,17 +150,17 @@ public void testGroupDescribeUnauthorized() { public void testGroupReadUnauthorized() { subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener); - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); client.prepareResponse(joinGroupLeaderResponse(0, "memberId", Collections.>emptyMap(), Errors.GROUP_AUTHORIZATION_FAILED.code())); - coordinator.ensurePartitionAssignment(); + coordinator.poll(time.milliseconds()); } @Test public void testCoordinatorNotAvailable() { - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); // GROUP_COORDINATOR_NOT_AVAILABLE will mark coordinator as unknown @@ -180,7 +181,7 @@ public void testCoordinatorNotAvailable() { @Test public void testNotCoordinator() { - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); // not_coordinator will mark coordinator as unknown @@ -201,7 +202,7 @@ public void testNotCoordinator() { @Test public void testIllegalGeneration() { - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); // illegal_generation will cause re-partition @@ -225,7 +226,7 @@ public void testIllegalGeneration() { @Test public void testUnknownConsumerId() { - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); // illegal_generation will cause re-partition @@ -249,7 +250,7 @@ public void testUnknownConsumerId() { @Test public void testCoordinatorDisconnect() { - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); // coordinator disconnect will mark coordinator as unknown @@ -279,12 +280,12 @@ public void testJoinGroupInvalidGroupId() { metadata.setTopics(Arrays.asList(topicName)); metadata.update(cluster, time.milliseconds()); - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); client.prepareResponse(joinGroupLeaderResponse(0, consumerId, Collections.>emptyMap(), Errors.INVALID_GROUP_ID.code())); - coordinator.ensurePartitionAssignment(); + coordinator.poll(time.milliseconds()); } @Test @@ -298,7 +299,7 @@ public void testNormalJoinGroupLeader() { metadata.setTopics(Arrays.asList(topicName)); metadata.update(cluster, time.milliseconds()); - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); // normal join group @@ -315,7 +316,7 @@ public boolean matches(ClientRequest request) { sync.groupAssignment().containsKey(consumerId); } }, syncGroupResponse(Arrays.asList(tp), Errors.NONE.code())); - coordinator.ensurePartitionAssignment(); + coordinator.poll(time.milliseconds()); assertFalse(subscriptions.partitionAssignmentNeeded()); assertEquals(Collections.singleton(tp), subscriptions.assignedPartitions()); @@ -336,7 +337,7 @@ public void testWakeupDuringJoin() { metadata.setTopics(Arrays.asList(topicName)); metadata.update(cluster, time.milliseconds()); - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); Map> memberSubscriptions = Collections.singletonMap(consumerId, Arrays.asList(topicName)); @@ -347,14 +348,14 @@ public void testWakeupDuringJoin() { consumerClient.wakeup(); try { - coordinator.ensurePartitionAssignment(); + coordinator.poll(time.milliseconds()); } catch (WakeupException e) { // ignore } // now complete the second half client.prepareResponse(syncGroupResponse(Arrays.asList(tp), Errors.NONE.code())); - coordinator.ensurePartitionAssignment(); + coordinator.poll(time.milliseconds()); assertFalse(subscriptions.partitionAssignmentNeeded()); assertEquals(Collections.singleton(tp), subscriptions.assignedPartitions()); @@ -371,7 +372,7 @@ public void testNormalJoinGroupFollower() { subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener); subscriptions.needReassignment(); - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); // normal join group @@ -386,7 +387,7 @@ public boolean matches(ClientRequest request) { } }, syncGroupResponse(Arrays.asList(tp), Errors.NONE.code())); - coordinator.ensurePartitionAssignment(); + coordinator.poll(time.milliseconds()); assertFalse(subscriptions.partitionAssignmentNeeded()); assertEquals(Collections.singleton(tp), subscriptions.assignedPartitions()); @@ -402,12 +403,12 @@ public void testLeaveGroupOnClose() { subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener); subscriptions.needReassignment(); - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE.code())); client.prepareResponse(syncGroupResponse(Arrays.asList(tp), Errors.NONE.code())); - coordinator.ensurePartitionAssignment(); + coordinator.poll(time.milliseconds()); final AtomicBoolean received = new AtomicBoolean(false); client.prepareResponse(new MockClient.RequestMatcher() { @@ -430,12 +431,12 @@ public void testMaybeLeaveGroup() { subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener); subscriptions.needReassignment(); - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE.code())); client.prepareResponse(syncGroupResponse(Arrays.asList(tp), Errors.NONE.code())); - coordinator.ensurePartitionAssignment(); + coordinator.poll(time.milliseconds()); final AtomicBoolean received = new AtomicBoolean(false); client.prepareResponse(new MockClient.RequestMatcher() { @@ -449,8 +450,9 @@ public boolean matches(ClientRequest request) { }, new LeaveGroupResponse(Errors.NONE.code()).toStruct()); coordinator.maybeLeaveGroup(); assertTrue(received.get()); - assertEquals(JoinGroupRequest.UNKNOWN_MEMBER_ID, coordinator.memberId); - assertEquals(OffsetCommitRequest.DEFAULT_GENERATION_ID, coordinator.generation); + + AbstractCoordinator.Generation generation = coordinator.generation(); + assertNull(generation); } @Test(expected = KafkaException.class) @@ -460,13 +462,13 @@ public void testUnexpectedErrorOnSyncGroup() { subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener); subscriptions.needReassignment(); - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); // join initially, but let coordinator rebalance on sync client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE.code())); client.prepareResponse(syncGroupResponse(Collections.emptyList(), Errors.UNKNOWN.code())); - coordinator.ensurePartitionAssignment(); + coordinator.poll(time.milliseconds()); } @Test @@ -476,7 +478,7 @@ public void testUnknownMemberIdOnSyncGroup() { subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener); subscriptions.needReassignment(); - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); // join initially, but let coordinator returns unknown member id @@ -493,7 +495,7 @@ public boolean matches(ClientRequest request) { }, joinGroupFollowerResponse(2, consumerId, "leader", Errors.NONE.code())); client.prepareResponse(syncGroupResponse(Arrays.asList(tp), Errors.NONE.code())); - coordinator.ensurePartitionAssignment(); + coordinator.poll(time.milliseconds()); assertFalse(subscriptions.partitionAssignmentNeeded()); assertEquals(Collections.singleton(tp), subscriptions.assignedPartitions()); @@ -506,7 +508,7 @@ public void testRebalanceInProgressOnSyncGroup() { subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener); subscriptions.needReassignment(); - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); // join initially, but let coordinator rebalance on sync @@ -517,7 +519,7 @@ public void testRebalanceInProgressOnSyncGroup() { client.prepareResponse(joinGroupFollowerResponse(2, consumerId, "leader", Errors.NONE.code())); client.prepareResponse(syncGroupResponse(Arrays.asList(tp), Errors.NONE.code())); - coordinator.ensurePartitionAssignment(); + coordinator.poll(time.milliseconds()); assertFalse(subscriptions.partitionAssignmentNeeded()); assertEquals(Collections.singleton(tp), subscriptions.assignedPartitions()); @@ -530,7 +532,7 @@ public void testIllegalGenerationOnSyncGroup() { subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener); subscriptions.needReassignment(); - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); // join initially, but let coordinator rebalance on sync @@ -547,7 +549,7 @@ public boolean matches(ClientRequest request) { }, joinGroupFollowerResponse(2, consumerId, "leader", Errors.NONE.code())); client.prepareResponse(syncGroupResponse(Arrays.asList(tp), Errors.NONE.code())); - coordinator.ensurePartitionAssignment(); + coordinator.poll(time.milliseconds()); assertFalse(subscriptions.partitionAssignmentNeeded()); assertEquals(Collections.singleton(tp), subscriptions.assignedPartitions()); @@ -560,13 +562,13 @@ public void testMetadataChangeTriggersRebalance() { subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener); subscriptions.needReassignment(); - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE.code())); client.prepareResponse(syncGroupResponse(Arrays.asList(tp), Errors.NONE.code())); - coordinator.ensurePartitionAssignment(); + coordinator.poll(time.milliseconds()); assertFalse(subscriptions.partitionAssignmentNeeded()); @@ -595,7 +597,7 @@ public void testUpdateMetadataDuringRebalance() { // we only have metadata for one topic initially metadata.update(TestUtils.singletonCluster(topic1, 1), time.milliseconds()); - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); // prepare initial rebalance @@ -625,7 +627,7 @@ public boolean matches(ClientRequest request) { client.prepareResponse(joinGroupLeaderResponse(2, consumerId, memberSubscriptions, Errors.NONE.code())); client.prepareResponse(syncGroupResponse(Arrays.asList(tp1, tp2), Errors.NONE.code())); - coordinator.ensurePartitionAssignment(); + coordinator.poll(time.milliseconds()); assertFalse(subscriptions.partitionAssignmentNeeded()); assertEquals(new HashSet<>(Arrays.asList(tp1, tp2)), subscriptions.assignedPartitions()); @@ -656,13 +658,13 @@ public void testRejoinGroup() { subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener); subscriptions.needReassignment(); - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); // join the group once client.prepareResponse(joinGroupFollowerResponse(1, "consumer", "leader", Errors.NONE.code())); client.prepareResponse(syncGroupResponse(Arrays.asList(tp), Errors.NONE.code())); - coordinator.ensurePartitionAssignment(); + coordinator.poll(time.milliseconds()); assertEquals(1, rebalanceListener.revokedCount); assertEquals(1, rebalanceListener.assignedCount); @@ -671,7 +673,7 @@ public void testRejoinGroup() { subscriptions.needReassignment(); client.prepareResponse(joinGroupFollowerResponse(2, "consumer", "leader", Errors.NONE.code())); client.prepareResponse(syncGroupResponse(Arrays.asList(tp), Errors.NONE.code())); - coordinator.ensurePartitionAssignment(); + coordinator.poll(time.milliseconds()); assertEquals(2, rebalanceListener.revokedCount); assertEquals(Collections.singleton(tp), rebalanceListener.revoked); @@ -684,15 +686,15 @@ public void testDisconnectInJoin() { subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener); subscriptions.needReassignment(); - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); // disconnected from original coordinator will cause re-discover and join again client.prepareResponse(joinGroupFollowerResponse(1, "consumer", "leader", Errors.NONE.code()), true); - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); client.prepareResponse(joinGroupFollowerResponse(1, "consumer", "leader", Errors.NONE.code())); client.prepareResponse(syncGroupResponse(Arrays.asList(tp), Errors.NONE.code())); - coordinator.ensurePartitionAssignment(); + coordinator.poll(time.milliseconds()); assertFalse(subscriptions.partitionAssignmentNeeded()); assertEquals(Collections.singleton(tp), subscriptions.assignedPartitions()); assertEquals(1, rebalanceListener.revokedCount); @@ -705,25 +707,26 @@ public void testInvalidSessionTimeout() { subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener); subscriptions.needReassignment(); - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); // coordinator doesn't like the session timeout client.prepareResponse(joinGroupFollowerResponse(0, "consumer", "", Errors.INVALID_SESSION_TIMEOUT.code())); - coordinator.ensurePartitionAssignment(); + coordinator.poll(time.milliseconds()); } @Test public void testCommitOffsetOnly() { subscriptions.assignFromUser(Arrays.asList(tp)); - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); AtomicBoolean success = new AtomicBoolean(false); coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), callback(success)); + coordinator.invokeCompletedOffsetCommitCallbacks(); assertTrue(success.get()); assertEquals(100L, subscriptions.committed(tp).offset()); @@ -739,18 +742,18 @@ public void testAutoCommitDynamicAssignment() { subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener); subscriptions.needReassignment(); - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE.code())); client.prepareResponse(syncGroupResponse(Arrays.asList(tp), Errors.NONE.code())); - coordinator.ensurePartitionAssignment(); + coordinator.poll(time.milliseconds()); subscriptions.seek(tp, 100); client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); time.sleep(autoCommitIntervalMs); - consumerClient.poll(0); + coordinator.poll(time.milliseconds()); assertEquals(100L, subscriptions.committed(tp).offset()); } @@ -765,7 +768,7 @@ public void testAutoCommitDynamicAssignmentRebalance() { subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener); subscriptions.needReassignment(); - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); // haven't joined, so should not cause a commit @@ -774,13 +777,13 @@ public void testAutoCommitDynamicAssignmentRebalance() { client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE.code())); client.prepareResponse(syncGroupResponse(Arrays.asList(tp), Errors.NONE.code())); - coordinator.ensurePartitionAssignment(); + coordinator.poll(time.milliseconds()); subscriptions.seek(tp, 100); client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); time.sleep(autoCommitIntervalMs); - consumerClient.poll(0); + coordinator.poll(time.milliseconds()); assertEquals(100L, subscriptions.committed(tp).offset()); } @@ -793,12 +796,12 @@ public void testAutoCommitManualAssignment() { subscriptions.assignFromUser(Arrays.asList(tp)); subscriptions.seek(tp, 100); - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); time.sleep(autoCommitIntervalMs); - consumerClient.poll(0); + coordinator.poll(time.milliseconds()); assertEquals(100L, subscriptions.committed(tp).offset()); } @@ -819,13 +822,13 @@ public void testAutoCommitManualAssignmentCoordinatorUnknown() { assertNull(subscriptions.committed(tp)); // now find the coordinator - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); // sleep only for the retry backoff time.sleep(retryBackoffMs); client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); - consumerClient.poll(0); + coordinator.poll(time.milliseconds()); assertEquals(100L, subscriptions.committed(tp).offset()); } @@ -834,13 +837,14 @@ public void testAutoCommitManualAssignmentCoordinatorUnknown() { public void testCommitOffsetMetadata() { subscriptions.assignFromUser(Arrays.asList(tp)); - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); AtomicBoolean success = new AtomicBoolean(false); coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L, "hello")), callback(success)); + coordinator.invokeCompletedOffsetCommitCallbacks(); assertTrue(success.get()); assertEquals(100L, subscriptions.committed(tp).offset()); @@ -850,10 +854,11 @@ public void testCommitOffsetMetadata() { @Test public void testCommitOffsetAsyncWithDefaultCallback() { int invokedBeforeTest = defaultOffsetCommitCallback.invoked; - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), null); + coordinator.invokeCompletedOffsetCommitCallbacks(); assertEquals(invokedBeforeTest + 1, defaultOffsetCommitCallback.invoked); assertNull(defaultOffsetCommitCallback.exception); } @@ -863,12 +868,12 @@ public void testCommitAfterLeaveGroup() { // enable auto-assignment subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener); - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); client.prepareResponse(joinGroupFollowerResponse(1, "consumer", "leader", Errors.NONE.code())); client.prepareResponse(syncGroupResponse(Arrays.asList(tp), Errors.NONE.code())); - coordinator.ensurePartitionAssignment(); + coordinator.poll(time.milliseconds()); // now switch to manual assignment client.prepareResponse(new LeaveGroupResponse(Errors.NONE.code()).toStruct()); @@ -888,29 +893,32 @@ public boolean matches(ClientRequest request) { AtomicBoolean success = new AtomicBoolean(false); coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), callback(success)); + coordinator.invokeCompletedOffsetCommitCallbacks(); assertTrue(success.get()); } @Test public void testCommitOffsetAsyncFailedWithDefaultCallback() { int invokedBeforeTest = defaultOffsetCommitCallback.invoked; - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code()))); coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), null); + coordinator.invokeCompletedOffsetCommitCallbacks(); assertEquals(invokedBeforeTest + 1, defaultOffsetCommitCallback.invoked); assertTrue(defaultOffsetCommitCallback.exception instanceof RetriableCommitFailedException); } @Test public void testCommitOffsetAsyncCoordinatorNotAvailable() { - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); // async commit with coordinator not available MockCommitCallback cb = new MockCommitCallback(); client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code()))); coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), cb); + coordinator.invokeCompletedOffsetCommitCallbacks(); assertTrue(coordinator.coordinatorUnknown()); assertEquals(1, cb.invoked); @@ -919,13 +927,14 @@ public void testCommitOffsetAsyncCoordinatorNotAvailable() { @Test public void testCommitOffsetAsyncNotCoordinator() { - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); // async commit with not coordinator MockCommitCallback cb = new MockCommitCallback(); client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NOT_COORDINATOR_FOR_GROUP.code()))); coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), cb); + coordinator.invokeCompletedOffsetCommitCallbacks(); assertTrue(coordinator.coordinatorUnknown()); assertEquals(1, cb.invoked); @@ -934,13 +943,14 @@ public void testCommitOffsetAsyncNotCoordinator() { @Test public void testCommitOffsetAsyncDisconnected() { - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); // async commit with coordinator disconnected MockCommitCallback cb = new MockCommitCallback(); client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())), true); coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), cb); + coordinator.invokeCompletedOffsetCommitCallbacks(); assertTrue(coordinator.coordinatorUnknown()); assertEquals(1, cb.invoked); @@ -949,36 +959,36 @@ public void testCommitOffsetAsyncDisconnected() { @Test public void testCommitOffsetSyncNotCoordinator() { - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); // sync commit with coordinator disconnected (should connect, get metadata, and then submit the commit request) client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NOT_COORDINATOR_FOR_GROUP.code()))); - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); coordinator.commitOffsetsSync(Collections.singletonMap(tp, new OffsetAndMetadata(100L))); } @Test public void testCommitOffsetSyncCoordinatorNotAvailable() { - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); // sync commit with coordinator disconnected (should connect, get metadata, and then submit the commit request) client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code()))); - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); coordinator.commitOffsetsSync(Collections.singletonMap(tp, new OffsetAndMetadata(100L))); } @Test public void testCommitOffsetSyncCoordinatorDisconnected() { - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); // sync commit with coordinator disconnected (should connect, get metadata, and then submit the commit request) client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())), true); - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); coordinator.commitOffsetsSync(Collections.singletonMap(tp, new OffsetAndMetadata(100L))); } @@ -986,7 +996,7 @@ public void testCommitOffsetSyncCoordinatorDisconnected() { @Test(expected = OffsetMetadataTooLarge.class) public void testCommitOffsetMetadataTooLarge() { // since offset metadata is provided by the user, we have to propagate the exception so they can handle it - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.OFFSET_METADATA_TOO_LARGE.code()))); @@ -996,7 +1006,7 @@ public void testCommitOffsetMetadataTooLarge() { @Test(expected = CommitFailedException.class) public void testCommitOffsetIllegalGeneration() { // we cannot retry if a rebalance occurs before the commit completed - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.ILLEGAL_GENERATION.code()))); @@ -1006,7 +1016,7 @@ public void testCommitOffsetIllegalGeneration() { @Test(expected = CommitFailedException.class) public void testCommitOffsetUnknownMemberId() { // we cannot retry if a rebalance occurs before the commit completed - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.UNKNOWN_MEMBER_ID.code()))); @@ -1016,7 +1026,7 @@ public void testCommitOffsetUnknownMemberId() { @Test(expected = CommitFailedException.class) public void testCommitOffsetRebalanceInProgress() { // we cannot retry if a rebalance occurs before the commit completed - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.REBALANCE_IN_PROGRESS.code()))); @@ -1025,7 +1035,7 @@ public void testCommitOffsetRebalanceInProgress() { @Test(expected = KafkaException.class) public void testCommitOffsetSyncCallbackWithNonRetriableException() { - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); // sync commit with invalid partitions should throw if we have no callback @@ -1035,7 +1045,7 @@ public void testCommitOffsetSyncCallbackWithNonRetriableException() { @Test public void testRefreshOffset() { - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); subscriptions.assignFromUser(Arrays.asList(tp)); @@ -1048,7 +1058,7 @@ public void testRefreshOffset() { @Test public void testRefreshOffsetLoadInProgress() { - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); subscriptions.assignFromUser(Arrays.asList(tp)); @@ -1062,13 +1072,13 @@ public void testRefreshOffsetLoadInProgress() { @Test public void testRefreshOffsetNotCoordinatorForConsumer() { - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); subscriptions.assignFromUser(Arrays.asList(tp)); subscriptions.needRefreshCommits(); client.prepareResponse(offsetFetchResponse(tp, Errors.NOT_COORDINATOR_FOR_GROUP.code(), "", 100L)); - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", 100L)); coordinator.refreshCommittedOffsetsIfNeeded(); assertFalse(subscriptions.refreshCommitsNeeded()); @@ -1077,7 +1087,7 @@ public void testRefreshOffsetNotCoordinatorForConsumer() { @Test public void testRefreshOffsetWithNoFetchableOffsets() { - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); subscriptions.assignFromUser(Arrays.asList(tp)); @@ -1122,12 +1132,12 @@ public void testMetadataTopicsExpiryDisabled() { metadata.setTopics(topics); subscriptions.needReassignment(); - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE.code())); client.prepareResponse(syncGroupResponse(Arrays.asList(tp), Errors.NONE.code())); - coordinator.ensurePartitionAssignment(); + coordinator.poll(time.milliseconds()); metadata.update(TestUtils.singletonCluster(topicName, 2), time.milliseconds()); assertTrue("Topic not found in metadata", metadata.containsTopic(topicName)); @@ -1150,6 +1160,7 @@ private ConsumerCoordinator buildCoordinator(Metrics metrics, return new ConsumerCoordinator( consumerClient, groupId, + rebalanceTimeoutMs, sessionTimeoutMs, heartbeatIntervalMs, assignors, @@ -1166,7 +1177,7 @@ private ConsumerCoordinator buildCoordinator(Metrics metrics, excludeInternalTopics); } - private Struct consumerMetadataResponse(Node node, short error) { + private Struct groupCoordinatorResponse(Node node, short error) { GroupCoordinatorResponse response = new GroupCoordinatorResponse(error, node); return response.toStruct(); } 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 index f0f2a973294e..8dcbde2cec5d 100644 --- 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 @@ -15,9 +15,9 @@ import org.apache.kafka.clients.ClientResponse; import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.MockClient; -import org.apache.kafka.common.errors.WakeupException; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; +import org.apache.kafka.common.errors.WakeupException; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.types.Struct; @@ -75,22 +75,6 @@ public void multiSend() { 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()); @@ -175,12 +159,4 @@ private Struct heartbeatResponse(short 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/DelayedTaskQueueTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/DelayedTaskQueueTest.java deleted file mode 100644 index db87b6635c71..000000000000 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/DelayedTaskQueueTest.java +++ /dev/null @@ -1,89 +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.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 ba04cb5be10a..518661877fb7 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 @@ -438,6 +438,7 @@ public void testFetchOffsetOutOfRangeException() { fetcherNoAutoReset.sendFetches(); client.prepareResponse(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L, 0)); consumerClient.poll(0); + assertFalse(subscriptionsNoAutoReset.isOffsetResetNeeded(tp)); try { fetcherNoAutoReset.fetchedRecords(); 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 75e68cc56ba1..0177c7915167 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 @@ -28,8 +28,10 @@ public class HeartbeatTest { private long timeout = 300L; private long interval = 100L; + private long maxPollInterval = 900L; + private long retryBackoff = 10L; private MockTime time = new MockTime(); - private Heartbeat heartbeat = new Heartbeat(timeout, interval, -1L); + private Heartbeat heartbeat = new Heartbeat(timeout, interval, maxPollInterval, retryBackoff); @Test public void testShouldHeartbeat() { @@ -64,7 +66,7 @@ public void testSessionTimeoutExpired() { public void testResetSession() { heartbeat.sentHeartbeat(time.milliseconds()); time.sleep(305); - heartbeat.resetSessionTimeout(time.milliseconds()); + heartbeat.resetTimeouts(time.milliseconds()); assertFalse(heartbeat.sessionTimeoutExpired(time.milliseconds())); } } 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 be7f974a9ff0..766c7453f49f 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 @@ -55,8 +55,9 @@ public void testSerialization() throws Exception { createHeartBeatRequest(), createHeartBeatRequest().getErrorResponse(0, new UnknownServerException()), createHeartBeatResponse(), - createJoinGroupRequest(), - createJoinGroupRequest().getErrorResponse(0, new UnknownServerException()), + createJoinGroupRequest(1), + createJoinGroupRequest(0).getErrorResponse(0, new UnknownServerException()), + createJoinGroupRequest(1).getErrorResponse(1, new UnknownServerException()), createJoinGroupResponse(), createLeaveGroupRequest(), createLeaveGroupRequest().getErrorResponse(0, new UnknownServerException()), @@ -118,6 +119,7 @@ public void testSerialization() throws Exception { checkSerialization(createOffsetCommitRequest(0).getErrorResponse(0, new UnknownServerException()), 0); checkSerialization(createOffsetCommitRequest(1), 1); checkSerialization(createOffsetCommitRequest(1).getErrorResponse(1, new UnknownServerException()), 1); + checkSerialization(createJoinGroupRequest(0), 0); checkSerialization(createUpdateMetadataRequest(0, null), 0); checkSerialization(createUpdateMetadataRequest(0, null).getErrorResponse(0, new UnknownServerException()), 0); checkSerialization(createUpdateMetadataRequest(1, null), 1); @@ -236,11 +238,15 @@ private AbstractRequestResponse createHeartBeatResponse() { return new HeartbeatResponse(Errors.NONE.code()); } - private AbstractRequest createJoinGroupRequest() { + private AbstractRequest createJoinGroupRequest(int version) { ByteBuffer metadata = ByteBuffer.wrap(new byte[] {}); List protocols = new ArrayList<>(); protocols.add(new JoinGroupRequest.ProtocolMetadata("consumer-range", metadata)); - return new JoinGroupRequest("group1", 30000, "consumer1", "consumer", protocols); + if (version == 0) { + return new JoinGroupRequest("group1", 30000, "consumer1", "consumer", protocols); + } else { + return new JoinGroupRequest("group1", 10000, 60000, "consumer1", "consumer", protocols); + } } private AbstractRequestResponse createJoinGroupResponse() { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java index f5aa8ae9eabb..6e9d7b4aa4c1 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java @@ -43,13 +43,31 @@ public class DistributedConfig extends WorkerConfig { * session.timeout.ms */ public static final String SESSION_TIMEOUT_MS_CONFIG = "session.timeout.ms"; - private static final String SESSION_TIMEOUT_MS_DOC = "The timeout used to detect failures when using Kafka's group management facilities."; + private static final String SESSION_TIMEOUT_MS_DOC = "The timeout used to detect worker failures." + + "The worker sends periodic heartbeats to indicate its liveness to the broker. If no heartbeats are " + + "received by the broker before the expiration of this session timeout, then the broker will remove the " + + "worker from the group and initiate a rebalance. Note that the value must be in the allowable range as " + + "configured in the broker configuration by group.min.session.timeout.ms " + + "and group.max.session.timeout.ms."; /** * 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 group coordinator when using Kafka's group management facilities. Heartbeats are used to ensure that the worker's session stays active and to facilitate rebalancing when new members 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."; + private static final String HEARTBEAT_INTERVAL_MS_DOC = "The expected time between heartbeats to the group " + + "coordinator when using Kafka's group management facilities. Heartbeats are used to ensure that the " + + "worker's session stays active and to facilitate rebalancing when new members 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."; + + /** + * rebalance.timeout.ms + */ + public static final String REBALANCE_TIMEOUT_MS_CONFIG = "rebalance.timeout.ms"; + private static final String REBALANCE_TIMEOUT_MS_DOC = "The maximum allowed time for each worker to join the group " + + "once a rebalance has begun. This is basically a limit on the amount of time needed for all tasks to " + + "flush any pending data and commit offsets. If the timeout is exceeded, then the worker will be removed " + + "from the group, which will cause offset commit failures."; /** * worker.sync.timeout.ms @@ -90,9 +108,14 @@ public class DistributedConfig extends WorkerConfig { .define(GROUP_ID_CONFIG, ConfigDef.Type.STRING, ConfigDef.Importance.HIGH, GROUP_ID_DOC) .define(SESSION_TIMEOUT_MS_CONFIG, ConfigDef.Type.INT, - 30000, + 10000, ConfigDef.Importance.HIGH, SESSION_TIMEOUT_MS_DOC) + .define(REBALANCE_TIMEOUT_MS_CONFIG, + ConfigDef.Type.INT, + 60000, + ConfigDef.Importance.HIGH, + REBALANCE_TIMEOUT_MS_DOC) .define(HEARTBEAT_INTERVAL_MS_CONFIG, ConfigDef.Type.INT, 3000, diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinator.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinator.java index 9c7496026c8f..911455594b51 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinator.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinator.java @@ -21,6 +21,7 @@ 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.requests.JoinGroupRequest; import org.apache.kafka.common.requests.JoinGroupRequest.ProtocolMetadata; import org.apache.kafka.common.utils.CircularIterator; import org.apache.kafka.common.utils.Time; @@ -63,6 +64,7 @@ public final class WorkerCoordinator extends AbstractCoordinator implements Clos */ public WorkerCoordinator(ConsumerNetworkClient client, String groupId, + int rebalanceTimeoutMs, int sessionTimeoutMs, int heartbeatIntervalMs, Metrics metrics, @@ -74,6 +76,7 @@ public WorkerCoordinator(ConsumerNetworkClient client, WorkerRebalanceListener listener) { super(client, groupId, + rebalanceTimeoutMs, sessionTimeoutMs, heartbeatIntervalMs, metrics, @@ -97,6 +100,32 @@ public String protocolType() { return "connect"; } + public void poll(long timeout) { + // poll for io until the timeout expires + long now = time.milliseconds(); + long deadline = now + timeout; + + while (now <= deadline) { + if (coordinatorUnknown()) { + ensureCoordinatorReady(); + now = time.milliseconds(); + } + + if (needRejoin()) { + ensureActiveGroup(); + now = time.milliseconds(); + } + + pollHeartbeat(now); + + // Note that because the network client is shared with the background heartbeat thread, + // we do not want to block in poll longer than the time to the next heartbeat. + long remaining = Math.max(0, deadline - now); + client.poll(Math.min(remaining, timeToNextHeartbeat(now))); + now = time.milliseconds(); + } + } + @Override public List metadata() { configSnapshot = configStorage.snapshot(); @@ -238,12 +267,15 @@ protected void onJoinPrepare(int generation, String memberId) { } @Override - public boolean needRejoin() { + protected boolean needRejoin() { return super.needRejoin() || (assignmentSnapshot == null || assignmentSnapshot.failed()) || rejoinRequested; } public String memberId() { - return this.memberId; + Generation generation = generation(); + if (generation != null) + return generation.memberId; + return JoinGroupRequest.UNKNOWN_MEMBER_ID; } @Override @@ -252,7 +284,7 @@ public void close() { } private boolean isLeader() { - return assignmentSnapshot != null && memberId.equals(assignmentSnapshot.leader()); + return assignmentSnapshot != null && memberId().equals(assignmentSnapshot.leader()); } public String ownerUrl(String connector) { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java index c21b9bfe1870..a5213db8bb29 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java @@ -104,6 +104,7 @@ public WorkerGroupMember(DistributedConfig config, config.getInt(CommonClientConfigs.REQUEST_TIMEOUT_MS_CONFIG)); this.coordinator = new WorkerCoordinator(this.client, config.getString(DistributedConfig.GROUP_ID_CONFIG), + config.getInt(DistributedConfig.REBALANCE_TIMEOUT_MS_CONFIG), config.getInt(DistributedConfig.SESSION_TIMEOUT_MS_CONFIG), config.getInt(DistributedConfig.HEARTBEAT_INTERVAL_MS_CONFIG), metrics, @@ -131,23 +132,13 @@ public void stop() { } public void ensureActive() { - coordinator.ensureCoordinatorReady(); - coordinator.ensureActiveGroup(); + coordinator.poll(0); } public void poll(long timeout) { 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(); - coordinator.ensureCoordinatorReady(); - coordinator.ensureActiveGroup(); - client.poll(remaining); - remaining -= time.milliseconds() - start; - } + coordinator.poll(timeout); } /** diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java index 4c2ac4002b10..3bfa83f0f55b 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java @@ -67,6 +67,7 @@ public class WorkerCoordinatorTest { private String groupId = "test-group"; private int sessionTimeoutMs = 10; + private int rebalanceTimeoutMs = 60; private int heartbeatIntervalMs = 2; private long retryBackoffMs = 100; private MockTime time; @@ -98,6 +99,7 @@ public void setup() { this.coordinator = new WorkerCoordinator(consumerClient, groupId, + rebalanceTimeoutMs, sessionTimeoutMs, heartbeatIntervalMs, metrics, diff --git a/core/src/main/scala/kafka/api/ApiVersion.scala b/core/src/main/scala/kafka/api/ApiVersion.scala index 666d0e79bba1..d955225a351f 100644 --- a/core/src/main/scala/kafka/api/ApiVersion.scala +++ b/core/src/main/scala/kafka/api/ApiVersion.scala @@ -51,7 +51,10 @@ object ApiVersion { "0.10.0-IV0" -> KAFKA_0_10_0_IV0, // 0.10.0-IV1 is introduced for KIP-36(rack awareness) and KIP-43(SASL handshake). "0.10.0-IV1" -> KAFKA_0_10_0_IV1, - "0.10.0" -> KAFKA_0_10_0_IV1 + "0.10.0" -> KAFKA_0_10_0_IV1, + + // introduced for JoinGroup protocol change in KIP-62 + "0.10.1-IV0" -> KAFKA_0_10_1_IV0 ) private val versionPattern = "\\.".r @@ -111,3 +114,9 @@ case object KAFKA_0_10_0_IV1 extends ApiVersion { val messageFormatVersion: Byte = Message.MagicValue_V1 val id: Int = 5 } + +case object KAFKA_0_10_1_IV0 extends ApiVersion { + val version: String = "0.10.1-IV0" + val messageFormatVersion: Byte = Message.MagicValue_V1 + val id: Int = 6 +} diff --git a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala index 0d02a4c3403b..726426ae9bff 100644 --- a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala @@ -93,6 +93,7 @@ class GroupCoordinator(val brokerId: Int, memberId: String, clientId: String, clientHost: String, + rebalanceTimeoutMs: Int, sessionTimeoutMs: Int, protocolType: String, protocols: List[(String, Array[Byte])], @@ -118,11 +119,11 @@ class GroupCoordinator(val brokerId: Int, responseCallback(joinError(memberId, Errors.UNKNOWN_MEMBER_ID.code)) } else { val group = groupManager.addGroup(new GroupMetadata(groupId)) - doJoinGroup(group, memberId, clientId, clientHost, sessionTimeoutMs, protocolType, protocols, responseCallback) + doJoinGroup(group, memberId, clientId, clientHost, rebalanceTimeoutMs, sessionTimeoutMs, protocolType, protocols, responseCallback) } case Some(group) => - doJoinGroup(group, memberId, clientId, clientHost, sessionTimeoutMs, protocolType, protocols, responseCallback) + doJoinGroup(group, memberId, clientId, clientHost, rebalanceTimeoutMs, sessionTimeoutMs, protocolType, protocols, responseCallback) } } } @@ -131,6 +132,7 @@ class GroupCoordinator(val brokerId: Int, memberId: String, clientId: String, clientHost: String, + rebalanceTimeoutMs: Int, sessionTimeoutMs: Int, protocolType: String, protocols: List[(String, Array[Byte])], @@ -154,7 +156,7 @@ class GroupCoordinator(val brokerId: Int, case PreparingRebalance => if (memberId == JoinGroupRequest.UNKNOWN_MEMBER_ID) { - addMemberAndRebalance(sessionTimeoutMs, clientId, clientHost, protocolType, protocols, group, responseCallback) + addMemberAndRebalance(rebalanceTimeoutMs, sessionTimeoutMs, clientId, clientHost, protocolType, protocols, group, responseCallback) } else { val member = group.get(memberId) updateMemberAndRebalance(group, member, protocols, responseCallback) @@ -162,7 +164,7 @@ class GroupCoordinator(val brokerId: Int, case AwaitingSync => if (memberId == JoinGroupRequest.UNKNOWN_MEMBER_ID) { - addMemberAndRebalance(sessionTimeoutMs, clientId, clientHost, protocolType, protocols, group, responseCallback) + addMemberAndRebalance(rebalanceTimeoutMs, sessionTimeoutMs, clientId, clientHost, protocolType, protocols, group, responseCallback) } else { val member = group.get(memberId) if (member.matches(protocols)) { @@ -189,7 +191,7 @@ class GroupCoordinator(val brokerId: Int, case Empty | Stable => if (memberId == JoinGroupRequest.UNKNOWN_MEMBER_ID) { // if the member id is unknown, register the member to the group - addMemberAndRebalance(sessionTimeoutMs, clientId, clientHost, protocolType, protocols, group, responseCallback) + addMemberAndRebalance(rebalanceTimeoutMs, sessionTimeoutMs, clientId, clientHost, protocolType, protocols, group, responseCallback) } else { val member = group.get(memberId) if (memberId == group.leaderId || !member.matches(protocols)) { @@ -256,7 +258,6 @@ class GroupCoordinator(val brokerId: Int, case AwaitingSync => group.get(memberId).awaitingSyncCallback = responseCallback - completeAndScheduleNextHeartbeatExpiration(group, group.get(memberId)) // if this is the leader, then we can attempt to persist state and transition to stable if (memberId == group.leaderId) { @@ -299,7 +300,7 @@ class GroupCoordinator(val brokerId: Int, delayedGroupStore.foreach(groupManager.store) } - def handleLeaveGroup(groupId: String, consumerId: String, responseCallback: Short => Unit) { + def handleLeaveGroup(groupId: String, memberId: String, responseCallback: Short => Unit) { if (!isActive.get) { responseCallback(Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code) } else if (!isCoordinatorForGroup(groupId)) { @@ -317,10 +318,10 @@ class GroupCoordinator(val brokerId: Int, case Some(group) => group synchronized { - if (group.is(Dead) || !group.has(consumerId)) { + if (group.is(Dead) || !group.has(memberId)) { responseCallback(Errors.UNKNOWN_MEMBER_ID.code) } else { - val member = group.get(consumerId) + val member = group.get(memberId) removeHeartbeatForLeavingMember(group, member) onMemberFailure(group, member) responseCallback(Errors.NONE.code) @@ -343,27 +344,49 @@ class GroupCoordinator(val brokerId: Int, responseCallback(Errors.NONE.code) } else { groupManager.getGroup(groupId) match { - case None => responseCallback(Errors.UNKNOWN_MEMBER_ID.code) + case None => + responseCallback(Errors.UNKNOWN_MEMBER_ID.code) + case Some(group) => group synchronized { - if (group.is(Empty)) { - responseCallback(Errors.UNKNOWN_MEMBER_ID.code) - } else if (group.is(Dead)) { - // if the group is marked as dead, it means some other thread has just removed the group - // from the coordinator metadata; this is likely that the group has migrated to some other - // coordinator OR the group is in a transient unstable phase. Let the member retry - // joining without the specified member id, - responseCallback(Errors.UNKNOWN_MEMBER_ID.code) - } else if (!group.is(Stable)) { - responseCallback(Errors.REBALANCE_IN_PROGRESS.code) - } else if (!group.has(memberId)) { - responseCallback(Errors.UNKNOWN_MEMBER_ID.code) - } else if (generationId != group.generationId) { - responseCallback(Errors.ILLEGAL_GENERATION.code) - } else { - val member = group.get(memberId) - completeAndScheduleNextHeartbeatExpiration(group, member) - responseCallback(Errors.NONE.code) + group.currentState match { + case Dead => + // if the group is marked as dead, it means some other thread has just removed the group + // from the coordinator metadata; this is likely that the group has migrated to some other + // coordinator OR the group is in a transient unstable phase. Let the member retry + // joining without the specified member id, + responseCallback(Errors.UNKNOWN_MEMBER_ID.code) + + case Empty => + responseCallback(Errors.UNKNOWN_MEMBER_ID.code) + + case AwaitingSync => + if (!group.has(memberId)) + responseCallback(Errors.UNKNOWN_MEMBER_ID.code) + else + responseCallback(Errors.REBALANCE_IN_PROGRESS.code) + + case PreparingRebalance => + if (!group.has(memberId)) { + responseCallback(Errors.UNKNOWN_MEMBER_ID.code) + } else if (generationId != group.generationId) { + responseCallback(Errors.ILLEGAL_GENERATION.code) + } else { + val member = group.get(memberId) + completeAndScheduleNextHeartbeatExpiration(group, member) + responseCallback(Errors.REBALANCE_IN_PROGRESS.code) + } + + case Stable => + if (!group.has(memberId)) { + responseCallback(Errors.UNKNOWN_MEMBER_ID.code) + } else if (generationId != group.generationId) { + responseCallback(Errors.ILLEGAL_GENERATION.code) + } else { + val member = group.get(memberId) + completeAndScheduleNextHeartbeatExpiration(group, member) + responseCallback(Errors.NONE.code) + } } } } @@ -585,7 +608,8 @@ class GroupCoordinator(val brokerId: Int, heartbeatPurgatory.checkAndComplete(memberKey) } - private def addMemberAndRebalance(sessionTimeoutMs: Int, + private def addMemberAndRebalance(rebalanceTimeoutMs: Int, + sessionTimeoutMs: Int, clientId: String, clientHost: String, protocolType: String, @@ -594,7 +618,8 @@ class GroupCoordinator(val brokerId: Int, callback: JoinCallback) = { // use the client-id with a random id suffix as the member-id val memberId = clientId + "-" + group.generateMemberIdSuffix - val member = new MemberMetadata(memberId, group.groupId, clientId, clientHost, sessionTimeoutMs, protocolType, protocols) + val member = new MemberMetadata(memberId, group.groupId, clientId, clientHost, rebalanceTimeoutMs, + sessionTimeoutMs, protocolType, protocols) member.awaitingJoinCallback = callback group.add(member.memberId, member) maybePrepareRebalance(group) @@ -625,7 +650,7 @@ class GroupCoordinator(val brokerId: Int, group.transitionTo(PreparingRebalance) info("Preparing to restabilize group %s with old generation %s".format(group.groupId, group.generationId)) - val rebalanceTimeout = group.rebalanceTimeout + val rebalanceTimeout = group.rebalanceTimeoutMs val delayedRebalance = new DelayedJoin(this, group, rebalanceTimeout) val groupKey = GroupKey(group.groupId) joinPurgatory.tryCompleteElseWatch(delayedRebalance, Seq(groupKey)) @@ -770,7 +795,8 @@ object GroupCoordinator { val groupConfig = GroupConfig(groupMinSessionTimeoutMs = config.groupMinSessionTimeoutMs, groupMaxSessionTimeoutMs = config.groupMaxSessionTimeoutMs) - val groupMetadataManager = new GroupMetadataManager(config.brokerId, offsetConfig, replicaManager, zkUtils, time) + val groupMetadataManager = new GroupMetadataManager(config.brokerId, config.interBrokerProtocolVersion, + offsetConfig, replicaManager, zkUtils, time) new GroupCoordinator(config.brokerId, groupConfig, offsetConfig, groupMetadataManager, heartbeatPurgatory, joinPurgatory, time) } diff --git a/core/src/main/scala/kafka/coordinator/GroupMetadata.scala b/core/src/main/scala/kafka/coordinator/GroupMetadata.scala index b4559646e5d1..c86c7f8d7b96 100644 --- a/core/src/main/scala/kafka/coordinator/GroupMetadata.scala +++ b/core/src/main/scala/kafka/coordinator/GroupMetadata.scala @@ -190,8 +190,8 @@ private[coordinator] class GroupMetadata(val groupId: String, initialState: Grou def allMemberMetadata = members.values.toList - def rebalanceTimeout = members.values.foldLeft(0) {(timeout, member) => - timeout.max(member.sessionTimeoutMs) + def rebalanceTimeoutMs = members.values.foldLeft(0) { (timeout, member) => + timeout.max(member.rebalanceTimeoutMs) } // TODO: decide if ids should be predictable or random diff --git a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala index ef8b2951bc19..cf8ae917b7ac 100644 --- a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala +++ b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala @@ -47,10 +47,12 @@ import java.util.concurrent.TimeUnit import java.util.concurrent.locks.ReentrantLock import com.yammer.metrics.core.Gauge +import kafka.api.{ApiVersion, KAFKA_0_10_1_IV0} import kafka.utils.CoreUtils.inLock class GroupMetadataManager(val brokerId: Int, + val interBrokerProtocolVersion: ApiVersion, val config: OffsetConfig, replicaManager: ReplicaManager, zkUtils: ZkUtils, @@ -175,9 +177,11 @@ class GroupMetadataManager(val brokerId: Int, groupAssignment: Map[String, Array[Byte]], responseCallback: Short => Unit): DelayedStore = { val (magicValue, timestamp) = getMessageFormatVersionAndTimestamp(partitionFor(group.groupId)) + val groupMetadataValueVersion = if (interBrokerProtocolVersion < KAFKA_0_10_1_IV0) 0.toShort else GroupMetadataManager.CURRENT_GROUP_VALUE_SCHEMA_VERSION + val message = new Message( key = GroupMetadataManager.groupMetadataKey(group.groupId), - bytes = GroupMetadataManager.groupMetadataValue(group, groupAssignment), + bytes = GroupMetadataManager.groupMetadataValue(group, groupAssignment, version = groupMetadataValueVersion), timestamp = timestamp, magicValue = magicValue) @@ -704,30 +708,51 @@ object GroupMetadataManager { private val GROUP_METADATA_KEY_SCHEMA = new Schema(new Field("group", STRING)) private val GROUP_KEY_GROUP_FIELD = GROUP_METADATA_KEY_SCHEMA.get("group") - private val MEMBER_METADATA_V0 = new Schema(new Field("member_id", STRING), - new Field("client_id", STRING), - new Field("client_host", STRING), - new Field("session_timeout", INT32), - new Field("subscription", BYTES), - new Field("assignment", BYTES)) - private val MEMBER_METADATA_MEMBER_ID_V0 = MEMBER_METADATA_V0.get("member_id") - private val MEMBER_METADATA_CLIENT_ID_V0 = MEMBER_METADATA_V0.get("client_id") - private val MEMBER_METADATA_CLIENT_HOST_V0 = MEMBER_METADATA_V0.get("client_host") - private val MEMBER_METADATA_SESSION_TIMEOUT_V0 = MEMBER_METADATA_V0.get("session_timeout") - private val MEMBER_METADATA_SUBSCRIPTION_V0 = MEMBER_METADATA_V0.get("subscription") - private val MEMBER_METADATA_ASSIGNMENT_V0 = MEMBER_METADATA_V0.get("assignment") - - - private val GROUP_METADATA_VALUE_SCHEMA_V0 = new Schema(new Field("protocol_type", STRING), - new Field("generation", INT32), - new Field("protocol", NULLABLE_STRING), - new Field("leader", NULLABLE_STRING), - new Field("members", new ArrayOf(MEMBER_METADATA_V0))) - private val GROUP_METADATA_PROTOCOL_TYPE_V0 = GROUP_METADATA_VALUE_SCHEMA_V0.get("protocol_type") - private val GROUP_METADATA_GENERATION_V0 = GROUP_METADATA_VALUE_SCHEMA_V0.get("generation") - private val GROUP_METADATA_PROTOCOL_V0 = GROUP_METADATA_VALUE_SCHEMA_V0.get("protocol") - private val GROUP_METADATA_LEADER_V0 = GROUP_METADATA_VALUE_SCHEMA_V0.get("leader") - private val GROUP_METADATA_MEMBERS_V0 = GROUP_METADATA_VALUE_SCHEMA_V0.get("members") + private val MEMBER_ID_KEY = "member_id" + private val CLIENT_ID_KEY = "client_id" + private val CLIENT_HOST_KEY = "client_host" + private val REBALANCE_TIMEOUT_KEY = "rebalance_timeout" + private val SESSION_TIMEOUT_KEY = "session_timeout" + private val SUBSCRIPTION_KEY = "subscription" + private val ASSIGNMENT_KEY = "assignment" + + private val MEMBER_METADATA_V0 = new Schema( + new Field(MEMBER_ID_KEY, STRING), + new Field(CLIENT_ID_KEY, STRING), + new Field(CLIENT_HOST_KEY, STRING), + new Field(SESSION_TIMEOUT_KEY, INT32), + new Field(SUBSCRIPTION_KEY, BYTES), + new Field(ASSIGNMENT_KEY, BYTES)) + + private val MEMBER_METADATA_V1 = new Schema( + new Field(MEMBER_ID_KEY, STRING), + new Field(CLIENT_ID_KEY, STRING), + new Field(CLIENT_HOST_KEY, STRING), + new Field(REBALANCE_TIMEOUT_KEY, INT32), + new Field(SESSION_TIMEOUT_KEY, INT32), + new Field(SUBSCRIPTION_KEY, BYTES), + new Field(ASSIGNMENT_KEY, BYTES)) + + private val PROTOCOL_TYPE_KEY = "protocol_type" + private val GENERATION_KEY = "generation" + private val PROTOCOL_KEY = "protocol" + private val LEADER_KEY = "leader" + private val MEMBERS_KEY = "members" + + private val GROUP_METADATA_VALUE_SCHEMA_V0 = new Schema( + new Field(PROTOCOL_TYPE_KEY, STRING), + new Field(GENERATION_KEY, INT32), + new Field(PROTOCOL_KEY, NULLABLE_STRING), + new Field(LEADER_KEY, NULLABLE_STRING), + new Field(MEMBERS_KEY, new ArrayOf(MEMBER_METADATA_V0))) + + private val GROUP_METADATA_VALUE_SCHEMA_V1 = new Schema( + new Field(PROTOCOL_TYPE_KEY, STRING), + new Field(GENERATION_KEY, INT32), + new Field(PROTOCOL_KEY, NULLABLE_STRING), + new Field(LEADER_KEY, NULLABLE_STRING), + new Field(MEMBERS_KEY, new ArrayOf(MEMBER_METADATA_V1))) + // map of versions to key schemas as data types private val MESSAGE_TYPE_SCHEMAS = Map( @@ -742,8 +767,10 @@ object GroupMetadataManager { private val CURRENT_OFFSET_VALUE_SCHEMA_VERSION = 1.toShort // map of version of group metadata value schemas - private val GROUP_VALUE_SCHEMAS = Map(0 -> GROUP_METADATA_VALUE_SCHEMA_V0) - private val CURRENT_GROUP_VALUE_SCHEMA_VERSION = 0.toShort + private val GROUP_VALUE_SCHEMAS = Map( + 0 -> GROUP_METADATA_VALUE_SCHEMA_V0, + 1 -> GROUP_METADATA_VALUE_SCHEMA_V1) + private val CURRENT_GROUP_VALUE_SCHEMA_VERSION = 1.toShort private val CURRENT_OFFSET_KEY_SCHEMA = schemaForKey(CURRENT_OFFSET_KEY_SCHEMA_VERSION) private val CURRENT_GROUP_KEY_SCHEMA = schemaForKey(CURRENT_GROUP_KEY_SCHEMA_VERSION) @@ -830,40 +857,47 @@ object GroupMetadataManager { * Generates the payload for group metadata message from given offset and metadata * assuming the generation id, selected protocol, leader and member assignment are all available * - * @param groupMetadata + * @param groupMetadata current group metadata + * @param assignment the assignment for the rebalancing generation + * @param version the version of the value message to use * @return payload for offset commit message */ - def groupMetadataValue(groupMetadata: GroupMetadata, assignment: Map[String, Array[Byte]]): Array[Byte] = { - // generate commit value with schema version 1 - val value = new Struct(CURRENT_GROUP_VALUE_SCHEMA) - value.set(GROUP_METADATA_PROTOCOL_TYPE_V0, groupMetadata.protocolType.getOrElse("")) - value.set(GROUP_METADATA_GENERATION_V0, groupMetadata.generationId) - value.set(GROUP_METADATA_PROTOCOL_V0, groupMetadata.protocol) - value.set(GROUP_METADATA_LEADER_V0, groupMetadata.leaderId) + def groupMetadataValue(groupMetadata: GroupMetadata, + assignment: Map[String, Array[Byte]], + version: Short = 0): Array[Byte] = { + val value = if (version == 0) new Struct(GROUP_METADATA_VALUE_SCHEMA_V0) else new Struct(CURRENT_GROUP_VALUE_SCHEMA) + + value.set(PROTOCOL_TYPE_KEY, groupMetadata.protocolType.getOrElse("")) + value.set(GENERATION_KEY, groupMetadata.generationId) + value.set(PROTOCOL_KEY, groupMetadata.protocol) + value.set(LEADER_KEY, groupMetadata.leaderId) val memberArray = groupMetadata.allMemberMetadata.map { case memberMetadata => - val memberStruct = value.instance(GROUP_METADATA_MEMBERS_V0) - memberStruct.set(MEMBER_METADATA_MEMBER_ID_V0, memberMetadata.memberId) - memberStruct.set(MEMBER_METADATA_CLIENT_ID_V0, memberMetadata.clientId) - memberStruct.set(MEMBER_METADATA_CLIENT_HOST_V0, memberMetadata.clientHost) - memberStruct.set(MEMBER_METADATA_SESSION_TIMEOUT_V0, memberMetadata.sessionTimeoutMs) + val memberStruct = value.instance(MEMBERS_KEY) + memberStruct.set(MEMBER_ID_KEY, memberMetadata.memberId) + memberStruct.set(CLIENT_ID_KEY, memberMetadata.clientId) + memberStruct.set(CLIENT_HOST_KEY, memberMetadata.clientHost) + memberStruct.set(SESSION_TIMEOUT_KEY, memberMetadata.sessionTimeoutMs) + + if (version > 0) + memberStruct.set(REBALANCE_TIMEOUT_KEY, memberMetadata.rebalanceTimeoutMs) val metadata = memberMetadata.metadata(groupMetadata.protocol) - memberStruct.set(MEMBER_METADATA_SUBSCRIPTION_V0, ByteBuffer.wrap(metadata)) + memberStruct.set(SUBSCRIPTION_KEY, ByteBuffer.wrap(metadata)) val memberAssignment = assignment(memberMetadata.memberId) assert(memberAssignment != null) - memberStruct.set(MEMBER_METADATA_ASSIGNMENT_V0, ByteBuffer.wrap(memberAssignment)) + memberStruct.set(ASSIGNMENT_KEY, ByteBuffer.wrap(memberAssignment)) memberStruct } - value.set(GROUP_METADATA_MEMBERS_V0, memberArray.toArray) + value.set(MEMBERS_KEY, memberArray.toArray) val byteBuffer = ByteBuffer.allocate(2 /* version */ + value.sizeOf) - byteBuffer.putShort(CURRENT_GROUP_VALUE_SCHEMA_VERSION) + byteBuffer.putShort(version) value.writeTo(byteBuffer) byteBuffer.array() } @@ -944,31 +978,33 @@ object GroupMetadataManager { val valueSchema = schemaForGroup(version) val value = valueSchema.read(buffer) - if (version == 0) { - val protocolType = value.get(GROUP_METADATA_PROTOCOL_TYPE_V0).asInstanceOf[String] + if (version == 0 || version == 1) { + val protocolType = value.get(PROTOCOL_TYPE_KEY).asInstanceOf[String] - val memberMetadataArray = value.getArray(GROUP_METADATA_MEMBERS_V0) + val memberMetadataArray = value.getArray(MEMBERS_KEY) val initialState = if (memberMetadataArray.isEmpty) Empty else Stable val group = new GroupMetadata(groupId, initialState) - group.generationId = value.get(GROUP_METADATA_GENERATION_V0).asInstanceOf[Int] - group.leaderId = value.get(GROUP_METADATA_LEADER_V0).asInstanceOf[String] - group.protocol = value.get(GROUP_METADATA_PROTOCOL_V0).asInstanceOf[String] + group.generationId = value.get(GENERATION_KEY).asInstanceOf[Int] + group.leaderId = value.get(LEADER_KEY).asInstanceOf[String] + group.protocol = value.get(PROTOCOL_KEY).asInstanceOf[String] memberMetadataArray.foreach { case memberMetadataObj => val memberMetadata = memberMetadataObj.asInstanceOf[Struct] - val memberId = memberMetadata.get(MEMBER_METADATA_MEMBER_ID_V0).asInstanceOf[String] - val clientId = memberMetadata.get(MEMBER_METADATA_CLIENT_ID_V0).asInstanceOf[String] - val clientHost = memberMetadata.get(MEMBER_METADATA_CLIENT_HOST_V0).asInstanceOf[String] - val sessionTimeout = memberMetadata.get(MEMBER_METADATA_SESSION_TIMEOUT_V0).asInstanceOf[Int] - val subscription = Utils.toArray(memberMetadata.get(MEMBER_METADATA_SUBSCRIPTION_V0).asInstanceOf[ByteBuffer]) + val memberId = memberMetadata.get(MEMBER_ID_KEY).asInstanceOf[String] + val clientId = memberMetadata.get(CLIENT_ID_KEY).asInstanceOf[String] + val clientHost = memberMetadata.get(CLIENT_HOST_KEY).asInstanceOf[String] + val sessionTimeout = memberMetadata.get(SESSION_TIMEOUT_KEY).asInstanceOf[Int] + val rebalanceTimeout = if (version == 0) sessionTimeout else memberMetadata.get(REBALANCE_TIMEOUT_KEY).asInstanceOf[Int] + + val subscription = Utils.toArray(memberMetadata.get(SUBSCRIPTION_KEY).asInstanceOf[ByteBuffer]) - val member = new MemberMetadata(memberId, groupId, clientId, clientHost, sessionTimeout, + val member = new MemberMetadata(memberId, groupId, clientId, clientHost, rebalanceTimeout, sessionTimeout, protocolType, List((group.protocol, subscription))) - member.assignment = Utils.toArray(memberMetadata.get(MEMBER_METADATA_ASSIGNMENT_V0).asInstanceOf[ByteBuffer]) + member.assignment = Utils.toArray(memberMetadata.get(ASSIGNMENT_KEY).asInstanceOf[ByteBuffer]) group.add(memberId, member) } diff --git a/core/src/main/scala/kafka/coordinator/MemberMetadata.scala b/core/src/main/scala/kafka/coordinator/MemberMetadata.scala index 19c9e8ece8dd..61492768b600 100644 --- a/core/src/main/scala/kafka/coordinator/MemberMetadata.scala +++ b/core/src/main/scala/kafka/coordinator/MemberMetadata.scala @@ -55,6 +55,7 @@ private[coordinator] class MemberMetadata(val memberId: String, val groupId: String, val clientId: String, val clientHost: String, + val rebalanceTimeoutMs: Int, val sessionTimeoutMs: Int, val protocolType: String, var supportedProtocols: List[(String, Array[Byte])]) { diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 6d38f85fccaa..bb219ca2f0c9 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -890,8 +890,8 @@ class KafkaApis(val requestChannel: RequestChannel, // the callback for sending a join-group response def sendResponseCallback(joinResult: JoinGroupResult) { val members = joinResult.members map { case (memberId, metadataArray) => (memberId, ByteBuffer.wrap(metadataArray)) } - val responseBody = new JoinGroupResponse(joinResult.errorCode, joinResult.generationId, joinResult.subProtocol, - joinResult.memberId, joinResult.leaderId, members) + val responseBody = new JoinGroupResponse(request.header.apiVersion, joinResult.errorCode, joinResult.generationId, + joinResult.subProtocol, joinResult.memberId, joinResult.leaderId, members) trace("Sending join group response %s for correlation id %d to client %s." .format(responseBody, request.header.correlationId, request.header.clientId)) @@ -900,6 +900,7 @@ class KafkaApis(val requestChannel: RequestChannel, if (!authorize(request.session, Read, new Resource(Group, joinGroupRequest.groupId()))) { val responseBody = new JoinGroupResponse( + request.header.apiVersion, Errors.GROUP_AUTHORIZATION_FAILED.code, JoinGroupResponse.UNKNOWN_GENERATION_ID, JoinGroupResponse.UNKNOWN_PROTOCOL, @@ -916,6 +917,7 @@ class KafkaApis(val requestChannel: RequestChannel, joinGroupRequest.memberId, request.header.clientId, request.session.clientAddress.toString, + joinGroupRequest.rebalanceTimeout, joinGroupRequest.sessionTimeout, joinGroupRequest.protocolType, protocols, diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index 817cdf7502f4..1a5f187eaba5 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -199,7 +199,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { } private def createJoinGroupRequest = { - new JoinGroupRequest(group, 30000, "", "consumer", + new JoinGroupRequest(group, 10000, 60000, "", "consumer", List( new JoinGroupRequest.ProtocolMetadata("consumer-range",ByteBuffer.wrap("test".getBytes()))).asJava) } diff --git a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala index f039750f9900..c13bf5804ebd 100644 --- a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala @@ -13,6 +13,7 @@ package kafka.api import java.util + import org.apache.kafka.clients.consumer._ import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord} import org.apache.kafka.common.record.TimestampType @@ -22,11 +23,14 @@ import kafka.utils.{TestUtils, Logging, ShutdownableThread} import kafka.common.Topic import kafka.server.KafkaConfig import java.util.ArrayList + import org.junit.Assert._ import org.junit.{Before, Test} + import scala.collection.JavaConverters._ import scala.collection.mutable.Buffer import org.apache.kafka.clients.producer.KafkaProducer +import org.apache.kafka.common.errors.WakeupException /** * Integration tests for the new consumer that cover basic usage as well as server failures @@ -82,112 +86,19 @@ abstract class BaseConsumerTest extends IntegrationTestHarness with Logging { } @Test - def testAutoCommitOnRebalance() { - val topic2 = "topic2" - TestUtils.createTopic(this.zkUtils, topic2, 2, serverCount, this.servers) - - this.consumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true") - val consumer0 = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer()) - consumers += consumer0 - - val numRecords = 10000 - sendRecords(numRecords) - - val rebalanceListener = new ConsumerRebalanceListener { - override def onPartitionsAssigned(partitions: util.Collection[TopicPartition]) = { - // keep partitions paused in this test so that we can verify the commits based on specific seeks - consumer0.pause(partitions) - } - - override def onPartitionsRevoked(partitions: util.Collection[TopicPartition]) = {} - } - - consumer0.subscribe(List(topic).asJava, rebalanceListener) - - val assignment = Set(tp, tp2) - TestUtils.waitUntilTrue(() => { - consumer0.poll(50) - consumer0.assignment() == assignment.asJava - }, s"Expected partitions ${assignment.asJava} but actually got ${consumer0.assignment()}") - - consumer0.seek(tp, 300) - consumer0.seek(tp2, 500) - - // change subscription to trigger rebalance - consumer0.subscribe(List(topic, topic2).asJava, rebalanceListener) - - val newAssignment = Set(tp, tp2, new TopicPartition(topic2, 0), new TopicPartition(topic2, 1)) - TestUtils.waitUntilTrue(() => { - val records = consumer0.poll(50) - consumer0.assignment() == newAssignment.asJava - }, s"Expected partitions ${newAssignment.asJava} but actually got ${consumer0.assignment()}") - - // after rebalancing, we should have reset to the committed positions - assertEquals(300, consumer0.committed(tp).offset) - assertEquals(500, consumer0.committed(tp2).offset) - } - - @Test - def testCommitSpecifiedOffsets() { - sendRecords(5, tp) - sendRecords(7, tp2) - - this.consumers.head.assign(List(tp, tp2).asJava) - - // Need to poll to join the group - this.consumers.head.poll(50) - val pos1 = this.consumers.head.position(tp) - val pos2 = this.consumers.head.position(tp2) - this.consumers.head.commitSync(Map[TopicPartition, OffsetAndMetadata]((tp, new OffsetAndMetadata(3L))).asJava) - assertEquals(3, this.consumers.head.committed(tp).offset) - assertNull(this.consumers.head.committed(tp2)) - - // Positions should not change - assertEquals(pos1, this.consumers.head.position(tp)) - assertEquals(pos2, this.consumers.head.position(tp2)) - this.consumers.head.commitSync(Map[TopicPartition, OffsetAndMetadata]((tp2, new OffsetAndMetadata(5L))).asJava) - assertEquals(3, this.consumers.head.committed(tp).offset) - assertEquals(5, this.consumers.head.committed(tp2).offset) - - // Using async should pick up the committed changes after commit completes - val commitCallback = new CountConsumerCommitCallback() - this.consumers.head.commitAsync(Map[TopicPartition, OffsetAndMetadata]((tp2, new OffsetAndMetadata(7L))).asJava, commitCallback) - awaitCommitCallback(this.consumers.head, commitCallback) - assertEquals(7, this.consumers.head.committed(tp2).offset) - } - - @Test - def testListTopics() { - val numParts = 2 - val topic1 = "part-test-topic-1" - val topic2 = "part-test-topic-2" - val topic3 = "part-test-topic-3" - TestUtils.createTopic(this.zkUtils, topic1, numParts, 1, this.servers) - TestUtils.createTopic(this.zkUtils, topic2, numParts, 1, this.servers) - TestUtils.createTopic(this.zkUtils, 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).size) - assertEquals(2, topics.get(topic2).size) - assertEquals(2, topics.get(topic3).size) - } - - @Test - def testPartitionReassignmentCallback() { + def testCoordinatorFailover() { val listener = new TestConsumerReassignmentListener() - this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "100") // timeout quickly to avoid slow test - this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "30") + this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "5000") + this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "2000") val consumer0 = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer()) consumers += consumer0 consumer0.subscribe(List(topic).asJava, listener) // the initial subscription should cause a callback execution - while (listener.callsToAssigned == 0) - consumer0.poll(50) + consumer0.poll(2000) + + assertEquals(1, listener.callsToAssigned) // get metadata for the topic var parts: Seq[PartitionInfo] = null @@ -200,54 +111,13 @@ abstract class BaseConsumerTest extends IntegrationTestHarness with Logging { val coordinator = parts.head.leader().id() this.servers(coordinator).shutdown() - // this should cause another callback execution - while (listener.callsToAssigned < 2) - consumer0.poll(50) - - assertEquals(2, listener.callsToAssigned) - - // only expect one revocation since revoke is not invoked on initial membership - assertEquals(2, listener.callsToRevoked) - } - - @Test - def testUnsubscribeTopic() { - - 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, new ByteArrayDeserializer(), new ByteArrayDeserializer()) - consumers += consumer0 - - val listener = new TestConsumerReassignmentListener() - consumer0.subscribe(List(topic).asJava, listener) - - // the initial subscription should cause a callback execution - while (listener.callsToAssigned == 0) - consumer0.poll(50) + consumer0.poll(5000) - consumer0.subscribe(List[String]().asJava) - assertEquals(0, consumer0.assignment.size()) + // the failover should not cause a rebalance + assertEquals(1, listener.callsToAssigned) + assertEquals(1, listener.callsToRevoked) } - @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") - val consumer0 = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer()) - consumers += consumer0 - - sendRecords(5) - consumer0.subscribe(List(topic).asJava) - consumeAndVerifyRecords(consumer = consumer0, numRecords = 5, startingOffset = 0) - consumer0.pause(List(tp).asJava) - - // subscribe to a new topic to trigger a rebalance - consumer0.subscribe(List("topic2").asJava) - - // after rebalance, our position should be reset and our pause state lost, - // so we should be able to consume from the beginning - consumeAndVerifyRecords(consumer = consumer0, numRecords = 0, startingOffset = 5) - } protected class TestConsumerReassignmentListener extends ConsumerRebalanceListener { var callsToAssigned = 0 @@ -394,12 +264,22 @@ abstract class BaseConsumerTest extends IntegrationTestHarness with Logging { !subscriptionChanged } + override def initiateShutdown(): Boolean = { + val res = super.initiateShutdown() + consumer.wakeup() + res + } + override def doWork(): Unit = { if (subscriptionChanged) { consumer.subscribe(topicsSubscription.asJava, rebalanceListener) subscriptionChanged = false } - consumer.poll(50) + try { + consumer.poll(50) + } catch { + case e: WakeupException => // ignore for shutdown + } } } diff --git a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala index 706405267d48..0900d437fd20 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala @@ -49,8 +49,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, "100") - this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "30") + this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "10000") + this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "3000") this.consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") override def generateConfigs() = { @@ -81,14 +81,7 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging { var consumed = 0L val consumer = this.consumers.head - consumer.subscribe(List(topic), new ConsumerRebalanceListener { - override def onPartitionsAssigned(partitions: util.Collection[TopicPartition]) { - // TODO: until KAFKA-2017 is merged, we have to handle the case in which - // the commit fails on prior to rebalancing on coordinator fail-over. - consumer.seek(tp, consumed) - } - override def onPartitionsRevoked(partitions: util.Collection[TopicPartition]) {} - }) + consumer.subscribe(List(topic)) val scheduler = new BounceBrokerScheduler(numIters) scheduler.start() diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index b1e96768e7cf..243f9138f72f 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -57,6 +57,31 @@ class PlaintextConsumerTest extends BaseConsumerTest { maxPollRecords = maxPollRecords) } + @Test + def testMaxPollIntervalMs() { + this.consumerConfig.setProperty(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, 3000.toString) + this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 500.toString) + this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 2000.toString) + + val consumer0 = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer()) + consumers += consumer0 + + val listener = new TestConsumerReassignmentListener() + consumer0.subscribe(List(topic).asJava, listener) + + // poll once to get the initial assignment + consumer0.poll(0) + assertEquals(1, listener.callsToAssigned) + assertEquals(1, listener.callsToRevoked) + + Thread.sleep(3500) + + // we should fall out of the group and need to rebalance + consumer0.poll(0) + assertEquals(2, listener.callsToAssigned) + assertEquals(2, listener.callsToRevoked) + } + @Test def testAutoCommitOnClose() { this.consumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true") @@ -593,16 +618,14 @@ class PlaintextConsumerTest extends BaseConsumerTest { // create a group of consumers, subscribe the consumers to all the topics and start polling // for the topic partition assignment val (rrConsumers, consumerPollers) = createConsumerGroupAndWaitForAssignment(10, List(topic1, topic2), subscriptions) + try { + validateGroupAssignment(consumerPollers, subscriptions, s"Did not get valid initial assignment for partitions ${subscriptions.asJava}") - // add one more consumer and validate re-assignment - addConsumersToGroupAndWaitForGroupAssignment(1, consumers, consumerPollers, List(topic1, topic2), subscriptions) - - // done with pollers and consumers - for (poller <- consumerPollers) - poller.shutdown() - - for (consumer <- consumers) - consumer.unsubscribe() + // add one more consumer and validate re-assignment + addConsumersToGroupAndWaitForGroupAssignment(1, consumers, consumerPollers, List(topic1, topic2), subscriptions) + } finally { + consumerPollers.foreach(_.shutdown()) + } } /** @@ -618,25 +641,25 @@ class PlaintextConsumerTest extends BaseConsumerTest { val subscriptions = Set(tp, tp2) ++ createTopicAndSendRecords(topic1, 5, 100) // subscribe all consumers to all topics and validate the assignment - val consumerPollers = subscribeConsumersAndWaitForAssignment(consumers, List(topic, topic1), subscriptions) + val consumerPollers = subscribeConsumers(consumers, List(topic, topic1)) - // add 2 more consumers and validate re-assignment - addConsumersToGroupAndWaitForGroupAssignment(2, consumers, consumerPollers, List(topic, topic1), subscriptions) + try { + validateGroupAssignment(consumerPollers, subscriptions, s"Did not get valid initial assignment for partitions ${subscriptions.asJava}") - // add one more topic and validate partition re-assignment - val topic2 = "topic2" - val expandedSubscriptions = subscriptions ++ createTopicAndSendRecords(topic2, 3, 100) - changeConsumerGroupSubscriptionAndValidateAssignment(consumerPollers, List(topic, topic1, topic2), expandedSubscriptions) + // add 2 more consumers and validate re-assignment + addConsumersToGroupAndWaitForGroupAssignment(2, consumers, consumerPollers, List(topic, topic1), subscriptions) - // remove the topic we just added and validate re-assignment - changeConsumerGroupSubscriptionAndValidateAssignment(consumerPollers, List(topic, topic1), subscriptions) + // add one more topic and validate partition re-assignment + val topic2 = "topic2" + val expandedSubscriptions = subscriptions ++ createTopicAndSendRecords(topic2, 3, 100) + changeConsumerGroupSubscriptionAndValidateAssignment(consumerPollers, List(topic, topic1, topic2), expandedSubscriptions) - // done with pollers and consumers - for (poller <- consumerPollers) - poller.shutdown() + // remove the topic we just added and validate re-assignment + changeConsumerGroupSubscriptionAndValidateAssignment(consumerPollers, List(topic, topic1), subscriptions) - for (consumer <- consumers) - consumer.unsubscribe() + } finally { + consumerPollers.foreach(_.shutdown()) + } } @Test @@ -830,6 +853,138 @@ class PlaintextConsumerTest extends BaseConsumerTest { startingTimestamp = startTime, timestampType = TimestampType.LOG_APPEND_TIME) } + @Test + def testListTopics() { + val numParts = 2 + val topic1 = "part-test-topic-1" + val topic2 = "part-test-topic-2" + val topic3 = "part-test-topic-3" + TestUtils.createTopic(this.zkUtils, topic1, numParts, 1, this.servers) + TestUtils.createTopic(this.zkUtils, topic2, numParts, 1, this.servers) + TestUtils.createTopic(this.zkUtils, 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).size) + assertEquals(2, topics.get(topic2).size) + assertEquals(2, topics.get(topic3).size) + } + + @Test + def testUnsubscribeTopic() { + 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, new ByteArrayDeserializer(), new ByteArrayDeserializer()) + consumers += consumer0 + + val listener = new TestConsumerReassignmentListener() + consumer0.subscribe(List(topic).asJava, listener) + + // the initial subscription should cause a callback execution + while (listener.callsToAssigned == 0) + consumer0.poll(50) + + consumer0.subscribe(List[String]().asJava) + assertEquals(0, consumer0.assignment.size()) + } + + @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") + val consumer0 = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer()) + consumers += consumer0 + + sendRecords(5) + consumer0.subscribe(List(topic).asJava) + consumeAndVerifyRecords(consumer = consumer0, numRecords = 5, startingOffset = 0) + consumer0.pause(List(tp).asJava) + + // subscribe to a new topic to trigger a rebalance + consumer0.subscribe(List("topic2").asJava) + + // after rebalance, our position should be reset and our pause state lost, + // so we should be able to consume from the beginning + consumeAndVerifyRecords(consumer = consumer0, numRecords = 0, startingOffset = 5) + } + + @Test + def testCommitSpecifiedOffsets() { + sendRecords(5, tp) + sendRecords(7, tp2) + + this.consumers.head.assign(List(tp, tp2).asJava) + + // Need to poll to join the group + this.consumers.head.poll(50) + val pos1 = this.consumers.head.position(tp) + val pos2 = this.consumers.head.position(tp2) + this.consumers.head.commitSync(Map[TopicPartition, OffsetAndMetadata]((tp, new OffsetAndMetadata(3L))).asJava) + assertEquals(3, this.consumers.head.committed(tp).offset) + assertNull(this.consumers.head.committed(tp2)) + + // Positions should not change + assertEquals(pos1, this.consumers.head.position(tp)) + assertEquals(pos2, this.consumers.head.position(tp2)) + this.consumers.head.commitSync(Map[TopicPartition, OffsetAndMetadata]((tp2, new OffsetAndMetadata(5L))).asJava) + assertEquals(3, this.consumers.head.committed(tp).offset) + assertEquals(5, this.consumers.head.committed(tp2).offset) + + // Using async should pick up the committed changes after commit completes + val commitCallback = new CountConsumerCommitCallback() + this.consumers.head.commitAsync(Map[TopicPartition, OffsetAndMetadata]((tp2, new OffsetAndMetadata(7L))).asJava, commitCallback) + awaitCommitCallback(this.consumers.head, commitCallback) + assertEquals(7, this.consumers.head.committed(tp2).offset) + } + + @Test + def testAutoCommitOnRebalance() { + val topic2 = "topic2" + TestUtils.createTopic(this.zkUtils, topic2, 2, serverCount, this.servers) + + this.consumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true") + val consumer0 = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer()) + consumers += consumer0 + + val numRecords = 10000 + sendRecords(numRecords) + + val rebalanceListener = new ConsumerRebalanceListener { + override def onPartitionsAssigned(partitions: util.Collection[TopicPartition]) = { + // keep partitions paused in this test so that we can verify the commits based on specific seeks + consumer0.pause(partitions) + } + + override def onPartitionsRevoked(partitions: util.Collection[TopicPartition]) = {} + } + + consumer0.subscribe(List(topic).asJava, rebalanceListener) + + val assignment = Set(tp, tp2) + TestUtils.waitUntilTrue(() => { + consumer0.poll(50) + consumer0.assignment() == assignment.asJava + }, s"Expected partitions ${assignment.asJava} but actually got ${consumer0.assignment()}") + + consumer0.seek(tp, 300) + consumer0.seek(tp2, 500) + + // change subscription to trigger rebalance + consumer0.subscribe(List(topic, topic2).asJava, rebalanceListener) + + val newAssignment = Set(tp, tp2, new TopicPartition(topic2, 0), new TopicPartition(topic2, 1)) + TestUtils.waitUntilTrue(() => { + val records = consumer0.poll(50) + consumer0.assignment() == newAssignment.asJava + }, s"Expected partitions ${newAssignment.asJava} but actually got ${consumer0.assignment()}") + + // after rebalancing, we should have reset to the committed positions + assertEquals(300, consumer0.committed(tp).offset) + assertEquals(500, consumer0.committed(tp2).offset) + } + def runMultiConsumerSessionTimeoutTest(closeConsumer: Boolean): Unit = { // use consumers defined in this class plus one additional consumer // Use topic defined in this class + one additional topic @@ -887,7 +1042,8 @@ class PlaintextConsumerTest extends BaseConsumerTest { * Subscribes consumer 'consumer' to a given list of topics 'topicsToSubscribe', creates * consumer poller and starts polling. * Assumes that the consumer is not subscribed to any topics yet - * @param consumer consumer + * + * @param consumer consumer * @param topicsToSubscribe topics that this consumer will subscribe to * @return consumer poller for the given consumer */ @@ -901,34 +1057,25 @@ class PlaintextConsumerTest extends BaseConsumerTest { /** * Creates consumer pollers corresponding to a given consumer group, one per consumer; subscribes consumers to - * 'topicsToSubscribe' topics, waits until consumers get topics assignment, and validates the assignment - * Currently, assignment validation requires that total number of partitions is greater or equal to - * number of consumers (i.e. subscriptions.size >= consumerGroup.size) - * Assumes that topics are already created with partitions corresponding to a given set of topic partitions ('subscriptions') + * 'topicsToSubscribe' topics, waits until consumers get topics assignment. * * When the function returns, consumer pollers will continue to poll until shutdown is called on every poller. * * @param consumerGroup consumer group * @param topicsToSubscribe topics to which consumers will subscribe to - * @param subscriptions set of all topic partitions * @return collection of consumer pollers */ - def subscribeConsumersAndWaitForAssignment(consumerGroup: Buffer[KafkaConsumer[Array[Byte], Array[Byte]]], - topicsToSubscribe: List[String], - subscriptions: Set[TopicPartition]): Buffer[ConsumerAssignmentPoller] = { + def subscribeConsumers(consumerGroup: Buffer[KafkaConsumer[Array[Byte], Array[Byte]]], + topicsToSubscribe: List[String]): Buffer[ConsumerAssignmentPoller] = { val consumerPollers = Buffer[ConsumerAssignmentPoller]() for (consumer <- consumerGroup) consumerPollers += subscribeConsumerAndStartPolling(consumer, topicsToSubscribe) - validateGroupAssignment(consumerPollers, subscriptions, s"Did not get valid initial assignment for partitions ${subscriptions.asJava}") consumerPollers } /** * Creates 'consumerCount' consumers and consumer pollers, one per consumer; subscribes consumers to - * 'topicsToSubscribe' topics, waits until consumers get topics assignment, and validates the assignment - * Currently, assignment validation requires that total number of partitions is greater or equal to - * number of consumers (i.e. subscriptions.size >= consumerCount) - * Assumes that topics are already created with partitions corresponding to a given set of topic partitions ('subscriptions') + * 'topicsToSubscribe' topics, waits until consumers get topics assignment. * * When the function returns, consumer pollers will continue to poll until shutdown is called on every poller. * @@ -947,7 +1094,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { consumers ++= consumerGroup // create consumer pollers, wait for assignment and validate it - val consumerPollers = subscribeConsumersAndWaitForAssignment(consumerGroup, topicsToSubscribe, subscriptions) + val consumerPollers = subscribeConsumers(consumerGroup, topicsToSubscribe) (consumerGroup, consumerPollers) } diff --git a/core/src/test/scala/integration/kafka/api/SaslPlainSslEndToEndAuthorizationTest.scala b/core/src/test/scala/integration/kafka/api/SaslPlainSslEndToEndAuthorizationTest.scala index 63636c013f6a..591479ec2844 100644 --- a/core/src/test/scala/integration/kafka/api/SaslPlainSslEndToEndAuthorizationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslPlainSslEndToEndAuthorizationTest.scala @@ -16,7 +16,6 @@ */ package kafka.api -import kafka.server.KafkaConfig import org.apache.kafka.common.protocol.SecurityProtocol class SaslPlainSslEndToEndAuthorizationTest extends EndToEndAuthorizationTest { diff --git a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala index c917ca4a41b1..a981e68ff176 100644 --- a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala @@ -54,6 +54,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val ClientHost = "localhost" val ConsumerMinSessionTimeout = 10 val ConsumerMaxSessionTimeout = 1000 + val DefaultRebalanceTimeout = 500 val DefaultSessionTimeout = 500 var timer: MockTimer = null var groupCoordinator: GroupCoordinator = null @@ -113,7 +114,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { def testJoinGroupWrongCoordinator() { val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val joinGroupResult = joinGroup(otherGroupId, memberId, DefaultSessionTimeout, protocolType, protocols) + val joinGroupResult = joinGroup(otherGroupId, memberId, protocolType, protocols) val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NOT_COORDINATOR_FOR_GROUP.code, joinGroupErrorCode) } @@ -122,7 +123,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { def testJoinGroupSessionTimeoutTooSmall() { val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val joinGroupResult = joinGroup(groupId, memberId, ConsumerMinSessionTimeout - 1, protocolType, protocols) + val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols, sessionTimeout = ConsumerMinSessionTimeout - 1) val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.INVALID_SESSION_TIMEOUT.code, joinGroupErrorCode) } @@ -131,14 +132,14 @@ class GroupCoordinatorResponseTest extends JUnitSuite { def testJoinGroupSessionTimeoutTooLarge() { val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val joinGroupResult = joinGroup(groupId, memberId, ConsumerMaxSessionTimeout + 1, protocolType, protocols) + val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols, sessionTimeout = ConsumerMaxSessionTimeout + 1) val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.INVALID_SESSION_TIMEOUT.code, joinGroupErrorCode) } @Test def testJoinGroupUnknownConsumerNewGroup() { - val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols) + val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols) val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.UNKNOWN_MEMBER_ID.code, joinGroupErrorCode) } @@ -148,7 +149,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val groupId = "" val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols) + val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols) assertEquals(Errors.INVALID_GROUP_ID.code, joinGroupResult.errorCode) } @@ -156,8 +157,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { def testValidJoinGroup() { val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, - protocols) + val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols) val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NONE.code, joinGroupErrorCode) } @@ -167,12 +167,11 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID val otherMemberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, - protocols) + val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols) assertEquals(Errors.NONE.code, joinGroupResult.errorCode) EasyMock.reset(replicaManager) - val otherJoinGroupResult = joinGroup(groupId, otherMemberId, DefaultSessionTimeout, "connect", protocols) + val otherJoinGroupResult = joinGroup(groupId, otherMemberId, "connect", protocols) assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code, otherJoinGroupResult.errorCode) } @@ -182,12 +181,11 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val otherMemberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, List(("range", metadata))) + val joinGroupResult = joinGroup(groupId, memberId, protocolType, List(("range", metadata))) assertEquals(Errors.NONE.code, joinGroupResult.errorCode) EasyMock.reset(replicaManager) - val otherJoinGroupResult = joinGroup(groupId, otherMemberId, DefaultSessionTimeout, protocolType, - List(("roundrobin", metadata))) + val otherJoinGroupResult = joinGroup(groupId, otherMemberId, protocolType, List(("roundrobin", metadata))) assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code, otherJoinGroupResult.errorCode) } @@ -196,11 +194,11 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID val otherMemberId = "memberId" - val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols) + val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols) assertEquals(Errors.NONE.code, joinGroupResult.errorCode) EasyMock.reset(replicaManager) - val otherJoinGroupResult = joinGroup(groupId, otherMemberId, DefaultSessionTimeout, protocolType, protocols) + val otherJoinGroupResult = joinGroup(groupId, otherMemberId, protocolType, protocols) assertEquals(Errors.UNKNOWN_MEMBER_ID.code, otherJoinGroupResult.errorCode) } @@ -223,7 +221,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID val otherMemberId = "memberId" - val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols) + val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols) val assignedMemberId = joinGroupResult.memberId val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NONE.code, joinGroupErrorCode) @@ -242,7 +240,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { def testHeartbeatRebalanceInProgress() { val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols) + val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols) val assignedMemberId = joinGroupResult.memberId val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NONE.code, joinGroupErrorCode) @@ -256,7 +254,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { def testHeartbeatIllegalGeneration() { val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols) + val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols) val assignedMemberId = joinGroupResult.memberId val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NONE.code, joinGroupErrorCode) @@ -275,7 +273,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { def testValidHeartbeat() { val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols) + val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols) val assignedConsumerId = joinGroupResult.memberId val generationId = joinGroupResult.generationId val joinGroupErrorCode = joinGroupResult.errorCode @@ -295,7 +293,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { def testSessionTimeout() { val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols) + val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols) val assignedConsumerId = joinGroupResult.memberId val generationId = joinGroupResult.generationId val joinGroupErrorCode = joinGroupResult.errorCode @@ -322,7 +320,8 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID val sessionTimeout = 1000 - val joinGroupResult = joinGroup(groupId, memberId, sessionTimeout, protocolType, protocols) + val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols, + rebalanceTimeout = sessionTimeout, sessionTimeout = sessionTimeout) val assignedConsumerId = joinGroupResult.memberId val generationId = joinGroupResult.generationId val joinGroupErrorCode = joinGroupResult.errorCode @@ -352,7 +351,8 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val tp = new TopicPartition("topic", 0) val offset = OffsetAndMetadata(0) - val joinGroupResult = joinGroup(groupId, memberId, sessionTimeout, protocolType, protocols) + val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols, + rebalanceTimeout = sessionTimeout, sessionTimeout = sessionTimeout) val assignedConsumerId = joinGroupResult.memberId val generationId = joinGroupResult.generationId val joinGroupErrorCode = joinGroupResult.errorCode @@ -375,11 +375,83 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(Errors.NONE.code, heartbeatResult) } + @Test + def testSessionTimeoutDuringRebalance() { + // create a group with a single member + val firstJoinResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols, + rebalanceTimeout = 2000, sessionTimeout = 1000) + val firstMemberId = firstJoinResult.memberId + val firstGenerationId = firstJoinResult.generationId + assertEquals(firstMemberId, firstJoinResult.leaderId) + assertEquals(Errors.NONE.code, firstJoinResult.errorCode) + + EasyMock.reset(replicaManager) + val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map(firstMemberId -> Array[Byte]())) + assertEquals(Errors.NONE.code, firstSyncResult._2) + + // now have a new member join to trigger a rebalance + EasyMock.reset(replicaManager) + val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols) + + timer.advanceClock(500) + + EasyMock.reset(replicaManager) + var heartbeatResult = heartbeat(groupId, firstMemberId, firstGenerationId) + assertEquals(Errors.REBALANCE_IN_PROGRESS.code, heartbeatResult) + + // letting the session expire should make the member fall out of the group + timer.advanceClock(1100) + + EasyMock.reset(replicaManager) + heartbeatResult = heartbeat(groupId, firstMemberId, firstGenerationId) + assertEquals(Errors.UNKNOWN_MEMBER_ID.code, heartbeatResult) + + // and the rebalance should complete with only the new member + val otherJoinResult = await(otherJoinFuture, DefaultSessionTimeout+100) + assertEquals(Errors.NONE.code, otherJoinResult.errorCode) + } + + @Test + def testRebalanceCompletesBeforeMemberJoins() { + // create a group with a single member + val firstJoinResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols, + rebalanceTimeout = 1200, sessionTimeout = 1000) + val firstMemberId = firstJoinResult.memberId + val firstGenerationId = firstJoinResult.generationId + assertEquals(firstMemberId, firstJoinResult.leaderId) + assertEquals(Errors.NONE.code, firstJoinResult.errorCode) + + EasyMock.reset(replicaManager) + val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map(firstMemberId -> Array[Byte]())) + assertEquals(Errors.NONE.code, firstSyncResult._2) + + // now have a new member join to trigger a rebalance + EasyMock.reset(replicaManager) + val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols) + + // send a couple heartbeats to keep the member alive while the rebalance finishes + timer.advanceClock(500) + EasyMock.reset(replicaManager) + var heartbeatResult = heartbeat(groupId, firstMemberId, firstGenerationId) + assertEquals(Errors.REBALANCE_IN_PROGRESS.code, heartbeatResult) + + timer.advanceClock(500) + EasyMock.reset(replicaManager) + heartbeatResult = heartbeat(groupId, firstMemberId, firstGenerationId) + assertEquals(Errors.REBALANCE_IN_PROGRESS.code, heartbeatResult) + + // now timeout the rebalance, which should kick the unjoined member out of the group + // and let the rebalance finish with only the new member + timer.advanceClock(500) + val otherJoinResult = await(otherJoinFuture, DefaultSessionTimeout+100) + assertEquals(Errors.NONE.code, otherJoinResult.errorCode) + } + @Test def testSyncGroupEmptyAssignment() { val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols) + val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols) val assignedConsumerId = joinGroupResult.memberId val generationId = joinGroupResult.generationId val joinGroupErrorCode = joinGroupResult.errorCode @@ -416,7 +488,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { def testSyncGroupFromUnknownMember() { val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols) + val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols) val assignedConsumerId = joinGroupResult.memberId val generationId = joinGroupResult.generationId assertEquals(Errors.NONE.code, joinGroupResult.errorCode) @@ -436,7 +508,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { def testSyncGroupFromIllegalGeneration() { val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols) + val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols) val assignedConsumerId = joinGroupResult.memberId val generationId = joinGroupResult.generationId assertEquals(Errors.NONE.code, joinGroupResult.errorCode) @@ -453,8 +525,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { // 1. join and sync with a single member (because we can't immediately join with two members) // 2. join and sync with the first member and a new member - val firstJoinResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, - protocolType, protocols) + val firstJoinResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols) val firstMemberId = firstJoinResult.memberId val firstGenerationId = firstJoinResult.generationId assertEquals(firstMemberId, firstJoinResult.leaderId) @@ -465,11 +536,10 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(Errors.NONE.code, firstSyncResult._2) EasyMock.reset(replicaManager) - val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, - protocolType, protocols) + val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols) EasyMock.reset(replicaManager) - val joinFuture = sendJoinGroup(groupId, firstMemberId, DefaultSessionTimeout, protocolType, protocols) + val joinFuture = sendJoinGroup(groupId, firstMemberId, protocolType, protocols) val joinResult = await(joinFuture, DefaultSessionTimeout+100) val otherJoinResult = await(otherJoinFuture, DefaultSessionTimeout+100) @@ -484,7 +554,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { // this shouldn't cause a rebalance since protocol information hasn't changed EasyMock.reset(replicaManager) - val followerJoinResult = joinGroup(groupId, otherJoinResult.memberId, DefaultSessionTimeout, protocolType, protocols) + val followerJoinResult = joinGroup(groupId, otherJoinResult.memberId, protocolType, protocols) assertEquals(Errors.NONE.code, followerJoinResult.errorCode) assertEquals(nextGenerationId, followerJoinResult.generationId) @@ -492,8 +562,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { @Test def testJoinGroupFromUnchangedLeaderShouldRebalance() { - val firstJoinResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, - protocolType, protocols) + val firstJoinResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols) val firstMemberId = firstJoinResult.memberId val firstGenerationId = firstJoinResult.generationId assertEquals(firstMemberId, firstJoinResult.leaderId) @@ -507,7 +576,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { // leader to push new assignments when local metadata changes EasyMock.reset(replicaManager) - val secondJoinResult = joinGroup(groupId, firstMemberId, DefaultSessionTimeout, protocolType, protocols) + val secondJoinResult = joinGroup(groupId, firstMemberId, protocolType, protocols) assertEquals(Errors.NONE.code, secondJoinResult.errorCode) assertNotEquals(firstGenerationId, secondJoinResult.generationId) @@ -519,8 +588,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { // 1. join and sync with a single member (because we can't immediately join with two members) // 2. join and sync with the first member and a new member - val firstJoinResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, - protocolType, protocols) + val firstJoinResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols) val firstMemberId = firstJoinResult.memberId val firstGenerationId = firstJoinResult.generationId assertEquals(firstMemberId, firstJoinResult.leaderId) @@ -531,11 +599,10 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(Errors.NONE.code, firstSyncResult._2) EasyMock.reset(replicaManager) - val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, - protocolType, protocols) + val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols) EasyMock.reset(replicaManager) - val joinFuture = sendJoinGroup(groupId, firstMemberId, DefaultSessionTimeout, protocolType, protocols) + val joinFuture = sendJoinGroup(groupId, firstMemberId, protocolType, protocols) val joinResult = await(joinFuture, DefaultSessionTimeout+100) val otherJoinResult = await(otherJoinFuture, DefaultSessionTimeout+100) @@ -565,8 +632,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { // 1. join and sync with a single member (because we can't immediately join with two members) // 2. join and sync with the first member and a new member - val firstJoinResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, - protocolType, protocols) + val firstJoinResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols) val firstMemberId = firstJoinResult.memberId val firstGenerationId = firstJoinResult.generationId assertEquals(firstMemberId, firstJoinResult.leaderId) @@ -577,11 +643,10 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(Errors.NONE.code, firstSyncResult._2) EasyMock.reset(replicaManager) - val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, - protocolType, protocols) + val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols) EasyMock.reset(replicaManager) - val joinFuture = sendJoinGroup(groupId, firstMemberId, DefaultSessionTimeout, protocolType, protocols) + val joinFuture = sendJoinGroup(groupId, firstMemberId, protocolType, protocols) val joinResult = await(joinFuture, DefaultSessionTimeout+100) val otherJoinResult = await(otherJoinFuture, DefaultSessionTimeout+100) @@ -616,8 +681,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { // 1. join and sync with a single member (because we can't immediately join with two members) // 2. join and sync with the first member and a new member - val joinGroupResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, - protocolType, protocols) + val joinGroupResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols) val firstMemberId = joinGroupResult.memberId val firstGenerationId = joinGroupResult.generationId assertEquals(firstMemberId, joinGroupResult.leaderId) @@ -629,11 +693,10 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(Errors.NONE.code, syncGroupErrorCode) EasyMock.reset(replicaManager) - val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, - protocolType, protocols) + val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols) EasyMock.reset(replicaManager) - val joinFuture = sendJoinGroup(groupId, firstMemberId, DefaultSessionTimeout, protocolType, protocols) + val joinFuture = sendJoinGroup(groupId, firstMemberId, protocolType, protocols) val joinResult = await(joinFuture, DefaultSessionTimeout+100) val otherJoinResult = await(otherJoinFuture, DefaultSessionTimeout+100) @@ -690,7 +753,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val tp = new TopicPartition("topic", 0) val offset = OffsetAndMetadata(0) - val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols) + val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols) val assignedMemberId = joinGroupResult.memberId val generationId = joinGroupResult.generationId val joinGroupErrorCode = joinGroupResult.errorCode @@ -704,8 +767,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { @Test def testHeartbeatDuringRebalanceCausesRebalanceInProgress() { // 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_MEMBER_ID, DefaultSessionTimeout, - protocolType, protocols) + val joinGroupResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols) val assignedConsumerId = joinGroupResult.memberId val initialGenerationId = joinGroupResult.generationId val joinGroupErrorCode = joinGroupResult.errorCode @@ -713,7 +775,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { // Then join with a new consumer to trigger a rebalance EasyMock.reset(replicaManager) - sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, protocolType, protocols) + sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols) // We should be in the middle of a rebalance, so the heartbeat should return rebalance in progress EasyMock.reset(replicaManager) @@ -723,7 +785,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { @Test def testGenerationIdIncrementsOnRebalance() { - val joinGroupResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, protocolType, protocols) + val joinGroupResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols) val initialGenerationId = joinGroupResult.generationId val joinGroupErrorCode = joinGroupResult.errorCode val memberId = joinGroupResult.memberId @@ -736,7 +798,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(Errors.NONE.code, syncGroupErrorCode) EasyMock.reset(replicaManager) - val otherJoinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols) + val otherJoinGroupResult = joinGroup(groupId, memberId, protocolType, protocols) val nextGenerationId = otherJoinGroupResult.generationId val otherJoinGroupErrorCode = otherJoinGroupResult.errorCode assertEquals(2, nextGenerationId) @@ -763,7 +825,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID val otherMemberId = "consumerId" - val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols) + val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols) val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NONE.code, joinGroupErrorCode) @@ -776,7 +838,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { def testValidLeaveGroup() { val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols) + val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols) val assignedMemberId = joinGroupResult.memberId val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NONE.code, joinGroupErrorCode) @@ -789,7 +851,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { @Test def testListGroupsIncludesStableGroups() { val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols) + val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols) val assignedMemberId = joinGroupResult.memberId val generationId = joinGroupResult.generationId assertEquals(Errors.NONE.code, joinGroupResult.errorCode) @@ -808,7 +870,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { @Test def testListGroupsIncludesRebalancingGroups() { val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols) + val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols) assertEquals(Errors.NONE.code, joinGroupResult.errorCode) val (error, groups) = groupCoordinator.handleListGroups() @@ -835,14 +897,15 @@ class GroupCoordinatorResponseTest extends JUnitSuite { @Test def testDescribeGroupStable() { val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols) + val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols) val assignedMemberId = joinGroupResult.memberId val generationId = joinGroupResult.generationId val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NONE.code, joinGroupErrorCode) EasyMock.reset(replicaManager) - val syncGroupResult = syncGroupLeader(groupId, generationId, assignedMemberId, Map(assignedMemberId -> Array[Byte]())) + val syncGroupResult = syncGroupLeader(groupId, generationId, assignedMemberId, Map(assignedMemberId -> Array[Byte]())) + val syncGroupErrorCode = syncGroupResult._2 assertEquals(Errors.NONE.code, syncGroupErrorCode) @@ -857,7 +920,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { @Test def testDescribeGroupRebalancing() { val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols) + val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols) val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NONE.code, joinGroupErrorCode) @@ -903,14 +966,15 @@ class GroupCoordinatorResponseTest extends JUnitSuite { private def sendJoinGroup(groupId: String, memberId: String, - sessionTimeout: Int, protocolType: String, - protocols: List[(String, Array[Byte])]): Future[JoinGroupResult] = { + protocols: List[(String, Array[Byte])], + rebalanceTimeout: Int = DefaultRebalanceTimeout, + sessionTimeout: Int = DefaultSessionTimeout): Future[JoinGroupResult] = { val (responseFuture, responseCallback) = setupJoinGroupCallback EasyMock.replay(replicaManager) - groupCoordinator.handleJoinGroup(groupId, memberId, "clientId", "clientHost", sessionTimeout, + groupCoordinator.handleJoinGroup(groupId, memberId, "clientId", "clientHost", rebalanceTimeout, sessionTimeout, protocolType, protocols, responseCallback) responseFuture } @@ -954,29 +1018,32 @@ class GroupCoordinatorResponseTest extends JUnitSuite { private def joinGroup(groupId: String, memberId: String, - sessionTimeout: Int, protocolType: String, - protocols: List[(String, Array[Byte])]): JoinGroupResult = { - val responseFuture = sendJoinGroup(groupId, memberId, sessionTimeout, protocolType, protocols) + protocols: List[(String, Array[Byte])], + sessionTimeout: Int = DefaultSessionTimeout, + rebalanceTimeout: Int = DefaultRebalanceTimeout): JoinGroupResult = { + val responseFuture = sendJoinGroup(groupId, memberId, protocolType, protocols, rebalanceTimeout, sessionTimeout) timer.advanceClock(10) // 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)) + Await.result(responseFuture, Duration(rebalanceTimeout + 100, TimeUnit.MILLISECONDS)) } private def syncGroupFollower(groupId: String, generationId: Int, - memberId: String): SyncGroupCallbackParams = { + memberId: String, + sessionTimeout: Int = DefaultSessionTimeout): SyncGroupCallbackParams = { val responseFuture = sendSyncGroupFollower(groupId, generationId, memberId) - Await.result(responseFuture, Duration(DefaultSessionTimeout+100, TimeUnit.MILLISECONDS)) + Await.result(responseFuture, Duration(sessionTimeout + 100, TimeUnit.MILLISECONDS)) } private def syncGroupLeader(groupId: String, generationId: Int, memberId: String, - assignment: Map[String, Array[Byte]]): SyncGroupCallbackParams = { + assignment: Map[String, Array[Byte]], + sessionTimeout: Int = DefaultSessionTimeout): SyncGroupCallbackParams = { val responseFuture = sendSyncGroupLeader(groupId, generationId, memberId, assignment) - Await.result(responseFuture, Duration(DefaultSessionTimeout+100, TimeUnit.MILLISECONDS)) + Await.result(responseFuture, Duration(sessionTimeout + 100, TimeUnit.MILLISECONDS)) } private def heartbeat(groupId: String, diff --git a/core/src/test/scala/unit/kafka/coordinator/GroupMetadataManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/GroupMetadataManagerTest.scala index b9569ca09206..b4f9ba31929c 100644 --- a/core/src/test/scala/unit/kafka/coordinator/GroupMetadataManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/GroupMetadataManagerTest.scala @@ -17,6 +17,7 @@ package kafka.coordinator +import kafka.api.ApiVersion import kafka.cluster.Partition import kafka.common.{OffsetAndMetadata, Topic} import kafka.log.LogAppendInfo @@ -46,7 +47,8 @@ class GroupMetadataManagerTest { val groupId = "foo" val groupPartitionId = 0 val protocolType = "protocolType" - val sessionTimeout = 30000 + val rebalanceTimeout = 60000 + val sessionTimeout = 10000 @Before @@ -74,9 +76,8 @@ class GroupMetadataManagerTest { time = new MockTime replicaManager = EasyMock.createNiceMock(classOf[ReplicaManager]) - groupMetadataManager = new GroupMetadataManager(0, offsetConfig, replicaManager, zkUtils, time) + groupMetadataManager = new GroupMetadataManager(0, ApiVersion.latestVersion, offsetConfig, replicaManager, zkUtils, time) partition = EasyMock.niceMock(classOf[Partition]) - } @After @@ -119,7 +120,7 @@ class GroupMetadataManagerTest { val group = new GroupMetadata(groupId) groupMetadataManager.addGroup(group) - val member = new MemberMetadata(memberId, groupId, clientId, clientHost, sessionTimeout, + val member = new MemberMetadata(memberId, groupId, clientId, clientHost, rebalanceTimeout, sessionTimeout, protocolType, List(("protocol", Array[Byte]()))) member.awaitingJoinCallback = (joinGroupResult: JoinGroupResult) => {} group.add(memberId, member) @@ -337,7 +338,7 @@ class GroupMetadataManagerTest { val group = new GroupMetadata(groupId) groupMetadataManager.addGroup(group) - val member = new MemberMetadata(memberId, groupId, clientId, clientHost, sessionTimeout, + val member = new MemberMetadata(memberId, groupId, clientId, clientHost, rebalanceTimeout, sessionTimeout, protocolType, List(("protocol", Array[Byte]()))) member.awaitingJoinCallback = (joinGroupResult: JoinGroupResult) => {} group.add(memberId, member) diff --git a/core/src/test/scala/unit/kafka/coordinator/GroupMetadataTest.scala b/core/src/test/scala/unit/kafka/coordinator/GroupMetadataTest.scala index 18dd1432ed11..853934017210 100644 --- a/core/src/test/scala/unit/kafka/coordinator/GroupMetadataTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/GroupMetadataTest.scala @@ -27,7 +27,14 @@ import org.scalatest.junit.JUnitSuite * Test group state transitions and other GroupMetadata functionality */ class GroupMetadataTest extends JUnitSuite { - var group: GroupMetadata = null + private val protocolType = "consumer" + private val groupId = "groupId" + private val clientId = "clientId" + private val clientHost = "clientHost" + private val rebalanceTimeoutMs = 60000 + private val sessionTimeoutMs = 10000 + + private var group: GroupMetadata = null @Before def setUp() { @@ -169,30 +176,24 @@ class GroupMetadataTest extends JUnitSuite { @Test def testSelectProtocol() { - val protocolType = "consumer" - val groupId = "groupId" - val clientId = "clientId" - val clientHost = "clientHost" - val sessionTimeoutMs = 10000 - val memberId = "memberId" - val member = new MemberMetadata(memberId, groupId, clientId, clientHost, sessionTimeoutMs, + val member = new MemberMetadata(memberId, groupId, clientId, clientHost, rebalanceTimeoutMs, sessionTimeoutMs, protocolType, List(("range", Array.empty[Byte]), ("roundrobin", Array.empty[Byte]))) group.add(memberId, member) assertEquals("range", group.selectProtocol) val otherMemberId = "otherMemberId" - val otherMember = new MemberMetadata(otherMemberId, groupId, clientId, clientHost, sessionTimeoutMs, - protocolType, List(("roundrobin", Array.empty[Byte]), ("range", Array.empty[Byte]))) + val otherMember = new MemberMetadata(otherMemberId, groupId, clientId, clientHost, rebalanceTimeoutMs, + sessionTimeoutMs, protocolType, List(("roundrobin", Array.empty[Byte]), ("range", Array.empty[Byte]))) group.add(otherMemberId, otherMember) // now could be either range or robin since there is no majority preference assertTrue(Set("range", "roundrobin")(group.selectProtocol)) val lastMemberId = "lastMemberId" - val lastMember = new MemberMetadata(lastMemberId, groupId, clientId, clientHost, sessionTimeoutMs, - protocolType, List(("roundrobin", Array.empty[Byte]), ("range", Array.empty[Byte]))) + val lastMember = new MemberMetadata(lastMemberId, groupId, clientId, clientHost, rebalanceTimeoutMs, + sessionTimeoutMs, protocolType, List(("roundrobin", Array.empty[Byte]), ("range", Array.empty[Byte]))) group.add(lastMemberId, lastMember) // now we should prefer 'roundrobin' @@ -207,19 +208,13 @@ class GroupMetadataTest extends JUnitSuite { @Test def testSelectProtocolChoosesCompatibleProtocol() { - val protocolType = "consumer" - val groupId = "groupId" - val clientId = "clientId" - val clientHost = "clientHost" - val sessionTimeoutMs = 10000 - val memberId = "memberId" - val member = new MemberMetadata(memberId, groupId, clientId, clientHost, sessionTimeoutMs, + val member = new MemberMetadata(memberId, groupId, clientId, clientHost, rebalanceTimeoutMs, sessionTimeoutMs, protocolType, List(("range", Array.empty[Byte]), ("roundrobin", Array.empty[Byte]))) val otherMemberId = "otherMemberId" - val otherMember = new MemberMetadata(otherMemberId, groupId, clientId, clientHost, sessionTimeoutMs, - protocolType, List(("roundrobin", Array.empty[Byte]), ("blah", Array.empty[Byte]))) + val otherMember = new MemberMetadata(otherMemberId, groupId, clientId, clientHost, rebalanceTimeoutMs, + sessionTimeoutMs, protocolType, List(("roundrobin", Array.empty[Byte]), ("blah", Array.empty[Byte]))) group.add(memberId, member) group.add(otherMemberId, otherMember) @@ -228,18 +223,12 @@ class GroupMetadataTest extends JUnitSuite { @Test def testSupportsProtocols() { - val protocolType = "consumer" - val groupId = "groupId" - val clientId = "clientId" - val clientHost = "clientHost" - val sessionTimeoutMs = 10000 - // by default, the group supports everything assertTrue(group.supportsProtocols(Set("roundrobin", "range"))) val memberId = "memberId" - val member = new MemberMetadata(memberId, groupId, clientId, clientHost, sessionTimeoutMs, - protocolType, List(("range", Array.empty[Byte]), ("roundrobin", Array.empty[Byte]))) + val member = new MemberMetadata(memberId, groupId, clientId, clientHost, rebalanceTimeoutMs, + sessionTimeoutMs, protocolType, List(("range", Array.empty[Byte]), ("roundrobin", Array.empty[Byte]))) group.add(memberId, member) assertTrue(group.supportsProtocols(Set("roundrobin", "foo"))) @@ -247,8 +236,8 @@ class GroupMetadataTest extends JUnitSuite { assertFalse(group.supportsProtocols(Set("foo", "bar"))) val otherMemberId = "otherMemberId" - val otherMember = new MemberMetadata(otherMemberId, groupId, clientId, clientHost, sessionTimeoutMs, - protocolType, List(("roundrobin", Array.empty[Byte]), ("blah", Array.empty[Byte]))) + val otherMember = new MemberMetadata(otherMemberId, groupId, clientId, clientHost, rebalanceTimeoutMs, + sessionTimeoutMs, protocolType, List(("roundrobin", Array.empty[Byte]), ("blah", Array.empty[Byte]))) group.add(otherMemberId, otherMember) @@ -258,14 +247,8 @@ class GroupMetadataTest extends JUnitSuite { @Test def testInitNextGeneration() { - val protocolType = "consumer" - val groupId = "groupId" - val clientId = "clientId" - val clientHost = "clientHost" - val sessionTimeoutMs = 10000 val memberId = "memberId" - - val member = new MemberMetadata(memberId, groupId, clientId, clientHost, sessionTimeoutMs, + val member = new MemberMetadata(memberId, groupId, clientId, clientHost, rebalanceTimeoutMs, sessionTimeoutMs, protocolType, List(("roundrobin", Array.empty[Byte]))) group.transitionTo(PreparingRebalance) diff --git a/core/src/test/scala/unit/kafka/coordinator/MemberMetadataTest.scala b/core/src/test/scala/unit/kafka/coordinator/MemberMetadataTest.scala index 06884246e835..257dde7a13af 100644 --- a/core/src/test/scala/unit/kafka/coordinator/MemberMetadataTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/MemberMetadataTest.scala @@ -28,6 +28,7 @@ class MemberMetadataTest extends JUnitSuite { val clientHost = "clientHost" val memberId = "memberId" val protocolType = "consumer" + val rebalanceTimeoutMs = 60000 val sessionTimeoutMs = 10000 @@ -35,7 +36,8 @@ class MemberMetadataTest extends JUnitSuite { def testMatchesSupportedProtocols { val protocols = List(("range", Array.empty[Byte])) - val member = new MemberMetadata(memberId, groupId, clientId, clientHost, sessionTimeoutMs, protocolType, protocols) + val member = new MemberMetadata(memberId, groupId, clientId, clientHost, rebalanceTimeoutMs, sessionTimeoutMs, + protocolType, protocols) assertTrue(member.matches(protocols)) assertFalse(member.matches(List(("range", Array[Byte](0))))) assertFalse(member.matches(List(("roundrobin", Array.empty[Byte])))) @@ -46,7 +48,8 @@ class MemberMetadataTest extends JUnitSuite { def testVoteForPreferredProtocol { val protocols = List(("range", Array.empty[Byte]), ("roundrobin", Array.empty[Byte])) - val member = new MemberMetadata(memberId, groupId, clientId, clientHost, sessionTimeoutMs, protocolType, protocols) + val member = new MemberMetadata(memberId, groupId, clientId, clientHost, rebalanceTimeoutMs, sessionTimeoutMs, + protocolType, protocols) assertEquals("range", member.vote(Set("range", "roundrobin"))) assertEquals("roundrobin", member.vote(Set("blah", "roundrobin"))) } @@ -55,7 +58,8 @@ class MemberMetadataTest extends JUnitSuite { def testMetadata { val protocols = List(("range", Array[Byte](0)), ("roundrobin", Array[Byte](1))) - val member = new MemberMetadata(memberId, groupId, clientId, clientHost, sessionTimeoutMs, protocolType, protocols) + val member = new MemberMetadata(memberId, groupId, clientId, clientHost, rebalanceTimeoutMs, sessionTimeoutMs, + protocolType, protocols) assertTrue(util.Arrays.equals(Array[Byte](0), member.metadata("range"))) assertTrue(util.Arrays.equals(Array[Byte](1), member.metadata("roundrobin"))) } @@ -64,7 +68,8 @@ class MemberMetadataTest extends JUnitSuite { def testMetadataRaisesOnUnsupportedProtocol { val protocols = List(("range", Array.empty[Byte]), ("roundrobin", Array.empty[Byte])) - val member = new MemberMetadata(memberId, groupId, clientId, clientHost, sessionTimeoutMs, protocolType, protocols) + val member = new MemberMetadata(memberId, groupId, clientId, clientHost, rebalanceTimeoutMs, sessionTimeoutMs, + protocolType, protocols) member.metadata("blah") fail() } @@ -73,7 +78,8 @@ class MemberMetadataTest extends JUnitSuite { def testVoteRaisesOnNoSupportedProtocols { val protocols = List(("range", Array.empty[Byte]), ("roundrobin", Array.empty[Byte])) - val member = new MemberMetadata(memberId, groupId, clientId, clientHost, sessionTimeoutMs, protocolType, protocols) + val member = new MemberMetadata(memberId, groupId, clientId, clientHost, rebalanceTimeoutMs, sessionTimeoutMs, + protocolType, protocols) member.vote(Set("blah")) fail() } diff --git a/core/src/test/scala/unit/kafka/utils/timer/MockTimer.scala b/core/src/test/scala/unit/kafka/utils/timer/MockTimer.scala index d18a06097600..e4ac4fa1ec70 100644 --- a/core/src/test/scala/unit/kafka/utils/timer/MockTimer.scala +++ b/core/src/test/scala/unit/kafka/utils/timer/MockTimer.scala @@ -23,7 +23,7 @@ import scala.collection.mutable class MockTimer extends Timer { val time = new MockTime - private val taskQueue = mutable.PriorityQueue[TimerTaskEntry]() + private val taskQueue = mutable.PriorityQueue[TimerTaskEntry]()(Ordering[TimerTaskEntry].reverse) def add(timerTask: TimerTask) { if (timerTask.delayMs <= 0) From c813091659336a7985353a518b4284ca1bd47cf9 Mon Sep 17 00:00:00 2001 From: Ishita Mandhan Date: Wed, 17 Aug 2016 13:30:46 -0700 Subject: [PATCH 04/35] Kafka 3940: Log should check the return value of dir.mkdirs() Fixed the failing errors in the previous commit that changed occurrences of dir.mkdirs() to Files.createDirectory() --- .../test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala | 2 +- core/src/test/scala/unit/kafka/log/LogManagerTest.scala | 2 +- core/src/test/scala/unit/kafka/log/LogTest.scala | 4 ++-- core/src/test/scala/unit/kafka/utils/TestUtils.scala | 2 +- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala index 7f336e906918..d3bf0303c02d 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala @@ -224,7 +224,7 @@ class LogCleanerIntegrationTest(compressionCodec: String) { try { Files.createDirectory(dir.toPath()) } catch { - case e: IOException => throw new KafkaException("Error in creating new directory '%s'".format(dir), e) + case e: IOException => //this is good } val log = new Log(dir = dir, diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index 8fe8d2381f97..d2cd2f78226e 100755 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -246,7 +246,7 @@ class LogManagerTest { try { Files.createDirectory(logDir.toPath()) } catch { - case e: IOException => throw new KafkaException("Error in creating new directory '%s'".format(logDir), e) + case e: IOException => //this is good } logDir.deleteOnExit() logManager = createLogManager() diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index 777d6e7a97f3..d90ea71d9269 100755 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -326,7 +326,7 @@ class LogTest extends JUnitSuite { try { Files.createDirectory(logDir.toPath()) } catch { - case e: IOException => throw new KafkaException("Error in creating new directory '%s'".format(logDir), e) + case e: IOException => //this is good } // first test a log segment starting at 0 val logProps = new Properties() @@ -807,7 +807,7 @@ class LogTest extends JUnitSuite { try { Files.createDirectory(logDir.toPath()) } catch { - case e: IOException => throw new KafkaException("Error in creating new directory '%s'".format(logDir), e) + case e: IOException => //this is good } var log = new Log(logDir, config, diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index cb386b079bbe..26b7217a0e41 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -84,7 +84,7 @@ object TestUtils extends Logging { try { Files.createDirectory(parentFile.toPath()) } catch { - case e: IOException => throw new KafkaException("Error in creating new directory '%s'".format(parentFile), e) + case e: IOException => //this is good } JTestUtils.tempDirectory(parentFile.toPath, null) } From 9af2e69ef4323dc6c8daf479226f25cc821b1bb6 Mon Sep 17 00:00:00 2001 From: Grant Henke Date: Thu, 18 Aug 2016 12:31:38 +0100 Subject: [PATCH 05/35] KAFKA-4038; Transient failure in DeleteTopicsRequestTest.testErrorDeleteTopicRequests Author: Grant Henke Reviewers: Ismael Juma Closes #1737 from granthenke/transient-delete --- .../test/scala/unit/kafka/server/DeleteTopicsRequestTest.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestTest.scala b/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestTest.scala index 4ef1af166aca..a59316b575b9 100644 --- a/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestTest.scala @@ -73,7 +73,8 @@ class DeleteTopicsRequestTest extends BaseRequestTest { // Timeout TestUtils.createTopic(zkUtils, timeoutTopic, 5, 2, servers) - validateErrorDeleteTopicRequests(new DeleteTopicsRequest(Set(timeoutTopic).asJava, 1), + // Must be a 0ms timeout to avoid transient test failures. Even a timeout of 1ms has succeeded in the past. + validateErrorDeleteTopicRequests(new DeleteTopicsRequest(Set(timeoutTopic).asJava, 0), Map(timeoutTopic -> Errors.REQUEST_TIMED_OUT)) // The topic should still get deleted eventually TestUtils.waitUntilTrue(() => !servers.head.metadataCache.contains(timeoutTopic), s"Topic $timeoutTopic is never deleted") From 59cfa84801c67de9729385a8f9b536721e0c37b9 Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Thu, 18 Aug 2016 15:29:56 -0700 Subject: [PATCH 06/35] KAFKA-4037: Make Connect REST API retries aware of 409 CONFLICT errors Author: Ewen Cheslack-Postava Reviewers: Jason Gustafson , Ismael Juma Closes #1733 from ewencp/rest-api-retries --- tests/kafkatest/services/connect.py | 62 ++++++++++++------- .../tests/connect/connect_rest_test.py | 12 ++-- tests/kafkatest/utils/util.py | 10 --- 3 files changed, 46 insertions(+), 38 deletions(-) diff --git a/tests/kafkatest/services/connect.py b/tests/kafkatest/services/connect.py index 7f36854f9a7c..ebc19b00f443 100644 --- a/tests/kafkatest/services/connect.py +++ b/tests/kafkatest/services/connect.py @@ -17,12 +17,12 @@ import os.path import random import signal +import time import requests from ducktape.errors import DucktapeError from ducktape.services.service import Service from ducktape.utils.util import wait_until -from kafkatest.utils.util import retry_on_exception from kafkatest.directory_layout.kafka_path import KafkaPathResolverMixin @@ -107,45 +107,49 @@ def clean_node(self, node): def config_filenames(self): return [os.path.join(self.PERSISTENT_ROOT, "connect-connector-" + str(idx) + ".properties") for idx, template in enumerate(self.connector_config_templates or [])] - def list_connectors(self, node=None, retries=0, retry_backoff=.01): - return self._rest_with_retry('/connectors', node=node, retries=retries, retry_backoff=retry_backoff) + def list_connectors(self, node=None, **kwargs): + return self._rest_with_retry('/connectors', node=node, **kwargs) - def create_connector(self, config, node=None, retries=0, retry_backoff=.01): + def create_connector(self, config, node=None, **kwargs): create_request = { 'name': config['name'], 'config': config } - return self._rest_with_retry('/connectors', create_request, node=node, method="POST", retries=retries, retry_backoff=retry_backoff) + return self._rest_with_retry('/connectors', create_request, node=node, method="POST", **kwargs) - def get_connector(self, name, node=None, retries=0, retry_backoff=.01): - return self._rest_with_retry('/connectors/' + name, node=node, retries=retries, retry_backoff=retry_backoff) + def get_connector(self, name, node=None, **kwargs): + return self._rest_with_retry('/connectors/' + name, node=node, **kwargs) - def get_connector_config(self, name, node=None, retries=0, retry_backoff=.01): - return self._rest_with_retry('/connectors/' + name + '/config', node=node, retries=retries, retry_backoff=retry_backoff) + def get_connector_config(self, name, node=None, **kwargs): + return self._rest_with_retry('/connectors/' + name + '/config', node=node, **kwargs) - def set_connector_config(self, name, config, node=None, retries=0, retry_backoff=.01): - return self._rest_with_retry('/connectors/' + name + '/config', config, node=node, method="PUT", retries=retries, retry_backoff=retry_backoff) + def set_connector_config(self, name, config, node=None, **kwargs): + # Unlike many other calls, a 409 when setting a connector config is expected if the connector already exists. + # However, we also might see 409s for other reasons (e.g. rebalancing). So we still perform retries at the cost + # of tests possibly taking longer to ultimately fail. Tests that care about this can explicitly override the + # number of retries. + return self._rest_with_retry('/connectors/' + name + '/config', config, node=node, method="PUT", **kwargs) - def get_connector_tasks(self, name, node=None, retries=0, retry_backoff=.01): - return self._rest_with_retry('/connectors/' + name + '/tasks', node=node, retries=retries, retry_backoff=retry_backoff) + def get_connector_tasks(self, name, node=None, **kwargs): + return self._rest_with_retry('/connectors/' + name + '/tasks', node=node, **kwargs) - def delete_connector(self, name, node=None, retries=0, retry_backoff=.01): - return self._rest_with_retry('/connectors/' + name, node=node, method="DELETE", retries=retries, retry_backoff=retry_backoff) + def delete_connector(self, name, node=None, **kwargs): + return self._rest_with_retry('/connectors/' + name, node=node, method="DELETE", **kwargs) def get_connector_status(self, name, node=None): return self._rest('/connectors/' + name + '/status', node=node) - def restart_connector(self, name, node=None): - return self._rest('/connectors/' + name + '/restart', method="POST") + def restart_connector(self, name, node=None, **kwargs): + return self._rest_with_retry('/connectors/' + name + '/restart', node=node, method="POST", **kwargs) def restart_task(self, connector_name, task_id, node=None): - return self._rest('/connectors/' + connector_name + '/tasks/' + str(task_id) + '/restart', method="POST") + return self._rest('/connectors/' + connector_name + '/tasks/' + str(task_id) + '/restart', node=node, method="POST") def pause_connector(self, name, node=None): - return self._rest('/connectors/' + name + '/pause', method="PUT") + return self._rest('/connectors/' + name + '/pause', node=node, method="PUT") def resume_connector(self, name, node=None): - return self._rest('/connectors/' + name + '/resume', method="PUT") + return self._rest('/connectors/' + name + '/resume', node=node, method="PUT") def list_connector_plugins(self, node=None): return self._rest('/connector-plugins/', node=node) @@ -163,14 +167,28 @@ def _rest(self, path, body=None, node=None, method="GET"): resp = meth(url, json=body) self.logger.debug("%s %s response: %d", url, method, resp.status_code) if resp.status_code > 400: + self.logger.debug("Connect REST API error for %s: %d %s", resp.url, resp.status_code, resp.text) raise ConnectRestError(resp.status_code, resp.text, resp.url) if resp.status_code == 204 or resp.status_code == 202: return None else: return resp.json() - def _rest_with_retry(self, path, body=None, node=None, method="GET", retries=0, retry_backoff=.01): - return retry_on_exception(lambda: self._rest(path, body, node, method), ConnectRestError, retries, retry_backoff) + def _rest_with_retry(self, path, body=None, node=None, method="GET", retries=40, retry_backoff=.25): + """ + Invokes a REST API with retries for errors that may occur during normal operation (notably 409 CONFLICT + responses that can occur due to rebalancing). + """ + exception_to_throw = None + for i in range(0, retries + 1): + try: + return self._rest(path, body, node, method) + except ConnectRestError as e: + exception_to_throw = e + if e.status != 409: + break + time.sleep(retry_backoff) + raise exception_to_throw def _base_url(self, node): return 'http://' + node.account.externally_routable_ip + ':' + '8083' diff --git a/tests/kafkatest/tests/connect/connect_rest_test.py b/tests/kafkatest/tests/connect/connect_rest_test.py index c32b8e179c91..0b004996cafd 100644 --- a/tests/kafkatest/tests/connect/connect_rest_test.py +++ b/tests/kafkatest/tests/connect/connect_rest_test.py @@ -84,11 +84,11 @@ def test_rest_api(self): self.verify_config(self.FILE_SINK_CONNECTOR, self.FILE_SINK_CONFIGS, configs) self.logger.info("Creating connectors") - self.cc.create_connector(source_connector_config, retries=120, retry_backoff=1) - self.cc.create_connector(sink_connector_config, retries=120, retry_backoff=1) + self.cc.create_connector(source_connector_config) + self.cc.create_connector(sink_connector_config) # We should see the connectors appear - wait_until(lambda: set(self.cc.list_connectors(retries=5, retry_backoff=1)) == set(["local-file-source", "local-file-sink"]), + wait_until(lambda: set(self.cc.list_connectors()) == set(["local-file-source", "local-file-sink"]), timeout_sec=10, err_msg="Connectors that were just created did not appear in connector listing") # We'll only do very simple validation that the connectors and tasks really ran. @@ -157,9 +157,9 @@ def test_rest_api(self): node.account.ssh("echo -e -n " + repr(self.LONER_INPUTS) + " >> " + self.INPUT_FILE2) wait_until(lambda: self.validate_output(self.LONGER_INPUT_LIST), timeout_sec=120, err_msg="Data added to input file was not seen in the output file in a reasonable amount of time.") - self.cc.delete_connector("local-file-source", retries=120, retry_backoff=1) - self.cc.delete_connector("local-file-sink", retries=120, retry_backoff=1) - wait_until(lambda: len(self.cc.list_connectors(retries=5, retry_backoff=1)) == 0, timeout_sec=10, err_msg="Deleted connectors did not disappear from REST listing") + self.cc.delete_connector("local-file-source") + self.cc.delete_connector("local-file-sink") + wait_until(lambda: len(self.cc.list_connectors()) == 0, timeout_sec=10, err_msg="Deleted connectors did not disappear from REST listing") def validate_output(self, input): input_set = set(input) diff --git a/tests/kafkatest/utils/util.py b/tests/kafkatest/utils/util.py index c043bec743fb..f004ece25d7f 100644 --- a/tests/kafkatest/utils/util.py +++ b/tests/kafkatest/utils/util.py @@ -73,13 +73,3 @@ def is_int_with_prefix(msg): "prefix dot integer value, but one of the two parts (before or after dot) " "are not integers. Message: %s" % (msg)) - -def retry_on_exception(fun, exception, retries, retry_backoff=.01): - exception_to_throw = None - for i in range(0, retries + 1): - try: - return fun() - except exception as e: - exception_to_throw = e - time.sleep(retry_backoff) - raise exception_to_throw From c6e6fadeed3e060da34b0fc7ec4c8540f8dffa90 Mon Sep 17 00:00:00 2001 From: Vahid Hashemian Date: Thu, 18 Aug 2016 20:39:57 -0700 Subject: [PATCH 07/35] MINOR: Clarification in producer config documentation Author: Vahid Hashemian Reviewers: Dana Powers, Gwen Shapira Closes #1732 from vahidhashemian/minor/clarify_producer_config_documentation --- .../org/apache/kafka/clients/producer/ProducerConfig.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index 3293aabfd183..dbbde06009db 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java @@ -51,8 +51,9 @@ public class ProducerConfig extends AbstractConfig { */ @Deprecated public static final String METADATA_FETCH_TIMEOUT_CONFIG = "metadata.fetch.timeout.ms"; - private static final String METADATA_FETCH_TIMEOUT_DOC = "The first time data is sent to a topic we must fetch metadata about that topic to know which servers host the topic's partitions. This " - + "fetch to succeed before throwing an exception back to the client."; + private static final String METADATA_FETCH_TIMEOUT_DOC = "The first time data is sent to a topic we must fetch metadata about that topic to know which servers " + + "host the topic's partitions. This config specifies the maximum time, in milliseconds, for this fetch " + + "to succeed before throwing an exception back to the client."; /** metadata.max.age.ms */ public static final String METADATA_MAX_AGE_CONFIG = CommonClientConfigs.METADATA_MAX_AGE_CONFIG; From e3ede843443ddb98b249d8ce1d111cc18aec6ed5 Mon Sep 17 00:00:00 2001 From: "P. Thorpe" Date: Thu, 18 Aug 2016 20:50:20 -0700 Subject: [PATCH 08/35] MINOR: Remove # from .bat start script On Windows, the following output is seen when starting Zookeeper and Kafka servers: ``` '#' is not recognized as an internal or external command, operable program or batch file. ``` This pull request makes a minor correction to the Windows `kafka-run-class.bat` script to replace the use of `#` with `rem`. Author: P. Thorpe Reviewers: Vahid Hashemian, Gwen Shapira Closes #1740 from p-thorpe/trunk --- bin/windows/kafka-run-class.bat | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/windows/kafka-run-class.bat b/bin/windows/kafka-run-class.bat index b16717c97fec..030c9b78d6c2 100644 --- a/bin/windows/kafka-run-class.bat +++ b/bin/windows/kafka-run-class.bat @@ -81,7 +81,7 @@ rem Log4j settings IF ["%KAFKA_LOG4J_OPTS%"] EQU [""] ( set KAFKA_LOG4J_OPTS=-Dlog4j.configuration=file:%BASE_DIR%/config/tools-log4j.properties ) ELSE ( - # create logs directory + rem create logs directory IF not exist %LOG_DIR% ( mkdir %LOG_DIR% ) From 05ed54bf2b80691d413dbfa05065eb3afe33972f Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Thu, 18 Aug 2016 20:56:31 -0700 Subject: [PATCH 09/35] KAFKA-3845: KIP-75: Add per-connector converters Author: Ewen Cheslack-Postava Reviewers: Shikhar Bhushan, Gwen Shapira Closes #1721 from ewencp/kafka-3845-per-connector-converters --- build.gradle | 2 +- .../connect/runtime/ConnectorConfig.java | 12 +- .../apache/kafka/connect/runtime/Worker.java | 31 +++-- .../distributed/DistributedHerder.java | 6 +- .../runtime/standalone/StandaloneConfig.java | 2 +- .../runtime/standalone/StandaloneHerder.java | 15 ++- .../kafka/connect/runtime/WorkerTest.java | 113 ++++++++++++++++- .../distributed/DistributedHerderTest.java | 20 +-- .../standalone/StandaloneHerderTest.java | 117 ++++++++++-------- docs/connect.html | 16 ++- .../tests/connect/connect_rest_test.py | 4 +- tests/kafkatest/tests/connect/connect_test.py | 13 +- .../templates/connect-file-sink.properties | 10 +- .../templates/connect-file-source.properties | 10 +- 14 files changed, 272 insertions(+), 99 deletions(-) diff --git a/build.gradle b/build.gradle index 6edcfdd330cc..32141081572f 100644 --- a/build.gradle +++ b/build.gradle @@ -874,8 +874,8 @@ project(':connect:runtime') { testCompile libs.junit testCompile libs.powermock testCompile libs.powermockEasymock + testCompile project(":connect:json") - testRuntime project(":connect:json") testRuntime libs.slf4jlog4j } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java index 9569b4beae0b..30869a41d0be 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java @@ -53,6 +53,14 @@ public class ConnectorConfig extends AbstractConfig { " or use \"FileStreamSink\" or \"FileStreamSinkConnector\" to make the configuration a bit shorter"; private static final String CONNECTOR_CLASS_DISPLAY = "Connector class"; + public static final String KEY_CONVERTER_CLASS_CONFIG = WorkerConfig.KEY_CONVERTER_CLASS_CONFIG; + public static final String KEY_CONVERTER_CLASS_DOC = WorkerConfig.KEY_CONVERTER_CLASS_DOC; + public static final String KEY_CONVERTER_CLASS_DISPLAY = "Key converter class"; + + public static final String VALUE_CONVERTER_CLASS_CONFIG = WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG; + public static final String VALUE_CONVERTER_CLASS_DOC = WorkerConfig.VALUE_CONVERTER_CLASS_DOC; + public static final String VALUE_CONVERTER_CLASS_DISPLAY = "Value converter class"; + 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; @@ -64,7 +72,9 @@ public static ConfigDef configDef() { return new ConfigDef() .define(NAME_CONFIG, Type.STRING, Importance.HIGH, NAME_DOC, COMMON_GROUP, 1, Width.MEDIUM, NAME_DISPLAY) .define(CONNECTOR_CLASS_CONFIG, Type.STRING, Importance.HIGH, CONNECTOR_CLASS_DOC, COMMON_GROUP, 2, Width.LONG, CONNECTOR_CLASS_DISPLAY) - .define(TASKS_MAX_CONFIG, Type.INT, TASKS_MAX_DEFAULT, atLeast(TASKS_MIN_CONFIG), Importance.HIGH, TASKS_MAX_DOC, COMMON_GROUP, 3, Width.SHORT, TASK_MAX_DISPLAY); + .define(TASKS_MAX_CONFIG, Type.INT, TASKS_MAX_DEFAULT, atLeast(TASKS_MIN_CONFIG), Importance.HIGH, TASKS_MAX_DOC, COMMON_GROUP, 3, Width.SHORT, TASK_MAX_DISPLAY) + .define(KEY_CONVERTER_CLASS_CONFIG, Type.CLASS, null, Importance.LOW, KEY_CONVERTER_CLASS_DOC, COMMON_GROUP, 4, Width.SHORT, KEY_CONVERTER_CLASS_DISPLAY) + .define(VALUE_CONVERTER_CLASS_CONFIG, Type.CLASS, null, Importance.LOW, VALUE_CONVERTER_CLASS_DOC, COMMON_GROUP, 5, Width.SHORT, VALUE_CONVERTER_CLASS_DISPLAY); } public ConnectorConfig() { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java index a0664adff8f1..d39806a9f6b8 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java @@ -66,8 +66,8 @@ public class Worker { private final Time time; private final String workerId; private final WorkerConfig config; - private final Converter keyConverter; - private final Converter valueConverter; + private final Converter defaultKeyConverter; + private final Converter defaultValueConverter; private final Converter internalKeyConverter; private final Converter internalValueConverter; private final OffsetBackingStore offsetBackingStore; @@ -85,10 +85,10 @@ public Worker(String workerId, this.workerId = workerId; this.time = time; this.config = config; - this.keyConverter = config.getConfiguredInstance(WorkerConfig.KEY_CONVERTER_CLASS_CONFIG, Converter.class); - this.keyConverter.configure(config.originalsWithPrefix("key.converter."), true); - this.valueConverter = config.getConfiguredInstance(WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, Converter.class); - this.valueConverter.configure(config.originalsWithPrefix("value.converter."), false); + this.defaultKeyConverter = config.getConfiguredInstance(WorkerConfig.KEY_CONVERTER_CLASS_CONFIG, Converter.class); + this.defaultKeyConverter.configure(config.originalsWithPrefix("key.converter."), true); + this.defaultValueConverter = config.getConfiguredInstance(WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, Converter.class); + this.defaultValueConverter.configure(config.originalsWithPrefix("value.converter."), false); this.internalKeyConverter = config.getConfiguredInstance(WorkerConfig.INTERNAL_KEY_CONVERTER_CLASS_CONFIG, Converter.class); this.internalKeyConverter.configure(config.originalsWithPrefix("internal.key.converter."), true); this.internalValueConverter = config.getConfiguredInstance(WorkerConfig.INTERNAL_VALUE_CONVERTER_CLASS_CONFIG, Converter.class); @@ -302,11 +302,13 @@ public boolean isRunning(String connName) { * Add a new task. * @param id Globally unique ID for this task. * @param taskConfig the parsed task configuration + * @param connConfig the parsed connector configuration * @param statusListener listener for notifications of task status changes * @param initialState the initial target state that the task should be initialized to */ public void startTask(ConnectorTaskId id, TaskConfig taskConfig, + ConnectorConfig connConfig, TaskStatus.Listener statusListener, TargetState initialState) { log.info("Creating task {}", id); @@ -322,7 +324,18 @@ public void startTask(ConnectorTaskId id, final Task task = instantiateTask(taskClass); log.info("Instantiated task {} with version {} of type {}", id, task.version(), taskClass.getName()); - final WorkerTask workerTask = buildWorkerTask(id, task, statusListener, initialState); + Converter keyConverter = connConfig.getConfiguredInstance(WorkerConfig.KEY_CONVERTER_CLASS_CONFIG, Converter.class); + if (keyConverter != null) + keyConverter.configure(connConfig.originalsWithPrefix("key.converter."), true); + else + keyConverter = defaultKeyConverter; + Converter valueConverter = connConfig.getConfiguredInstance(WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, Converter.class); + if (valueConverter != null) + valueConverter.configure(connConfig.originalsWithPrefix("value.converter."), false); + else + valueConverter = defaultValueConverter; + + final WorkerTask workerTask = buildWorkerTask(id, task, statusListener, initialState, keyConverter, valueConverter); // 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 @@ -339,7 +352,9 @@ public void startTask(ConnectorTaskId id, private WorkerTask buildWorkerTask(ConnectorTaskId id, Task task, TaskStatus.Listener statusListener, - TargetState initialState) { + TargetState initialState, + Converter keyConverter, + Converter valueConverter) { // Decide which type of worker task we need based on the type of task. if (task instanceof SourceTask) { OffsetStorageReader offsetReader = new OffsetStorageReaderImpl(offsetBackingStore, id.connector(), diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java index afabbeb1e37f..6232187d4ae0 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java @@ -772,9 +772,9 @@ private void startWork() { private void startTask(ConnectorTaskId taskId) { log.info("Starting task {}", taskId); TargetState initialState = configState.targetState(taskId.connector()); - Map configs = configState.taskConfig(taskId); - TaskConfig taskConfig = new TaskConfig(configs); - worker.startTask(taskId, taskConfig, this, initialState); + TaskConfig taskConfig = new TaskConfig(configState.taskConfig(taskId)); + ConnectorConfig connConfig = new ConnectorConfig(configState.connectorConfig(taskId.connector())); + worker.startTask(taskId, taskConfig, connConfig, this, initialState); } // Helper for starting a connector with the given name, which will extract & parse the config, generate connector diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneConfig.java index 8014b3aa87df..5637e05a9c79 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneConfig.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneConfig.java @@ -29,7 +29,7 @@ public class StandaloneConfig extends WorkerConfig { * offset.storage.file.filename */ public static final String OFFSET_STORAGE_FILE_FILENAME_CONFIG = "offset.storage.file.filename"; - private static final String OFFSET_STORAGE_FILE_FILENAME_DOC = "file to store offset data in"; + private static final String OFFSET_STORAGE_FILE_FILENAME_DOC = "File to store offset data in"; static { CONFIG = baseConfigDef() diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java index 8dbda74b8e7a..cac8d18f486b 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java @@ -206,14 +206,16 @@ public synchronized void restartTask(ConnectorTaskId taskId, Callback cb) if (!configState.contains(taskId.connector())) cb.onCompletion(new NotFoundException("Connector " + taskId.connector() + " not found", null), null); - Map taskConfig = configState.taskConfig(taskId); - if (taskConfig == null) + Map taskConfigProps = configState.taskConfig(taskId); + if (taskConfigProps == null) cb.onCompletion(new NotFoundException("Task " + taskId + " not found", null), null); + TaskConfig taskConfig = new TaskConfig(taskConfigProps); + ConnectorConfig connConfig = new ConnectorConfig(configState.connectorConfig(taskId.connector())); TargetState targetState = configState.targetState(taskId.connector()); try { worker.stopAndAwaitTask(taskId); - worker.startTask(taskId, new TaskConfig(taskConfig), this, targetState); + worker.startTask(taskId, taskConfig, connConfig, this, targetState); cb.onCompletion(null, null); } catch (Exception e) { log.error("Failed to restart task {}", taskId, e); @@ -270,11 +272,14 @@ private List> recomputeTaskConfigs(String connName) { } private void createConnectorTasks(String connName, TargetState initialState) { + Map connConfigs = configState.connectorConfig(connName); + ConnectorConfig connConfig = new ConnectorConfig(connConfigs); + for (ConnectorTaskId taskId : configState.tasks(connName)) { Map taskConfigMap = configState.taskConfig(taskId); - TaskConfig config = new TaskConfig(taskConfigMap); + TaskConfig taskConfig = new TaskConfig(taskConfigMap); try { - worker.startTask(taskId, config, this, initialState); + worker.startTask(taskId, taskConfig, connConfig, this, initialState); } catch (Throwable e) { log.error("Failed to add task {}: ", taskId, e); // Swallow this so we can continue updating the rest of the tasks diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java index ec4f0253c1e3..f9839f5fb64c 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java @@ -23,7 +23,10 @@ import org.apache.kafka.connect.connector.Connector; import org.apache.kafka.connect.connector.ConnectorContext; import org.apache.kafka.connect.connector.Task; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaAndValue; import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.json.JsonConverter; import org.apache.kafka.connect.runtime.standalone.StandaloneConfig; import org.apache.kafka.connect.sink.SinkTask; import org.apache.kafka.connect.source.SourceRecord; @@ -35,6 +38,7 @@ import org.apache.kafka.connect.util.ConnectorTaskId; import org.apache.kafka.connect.util.MockTime; import org.apache.kafka.connect.util.ThreadedTest; +import org.easymock.Capture; import org.easymock.EasyMock; import org.junit.Before; import org.junit.Test; @@ -352,8 +356,8 @@ public void testAddRemoveTask() throws Exception { EasyMock.eq(task), EasyMock.anyObject(TaskStatus.Listener.class), EasyMock.eq(TargetState.STARTED), - EasyMock.anyObject(Converter.class), - EasyMock.anyObject(Converter.class), + EasyMock.anyObject(JsonConverter.class), + EasyMock.anyObject(JsonConverter.class), EasyMock.anyObject(KafkaProducer.class), EasyMock.anyObject(OffsetStorageReader.class), EasyMock.anyObject(OffsetStorageWriter.class), @@ -380,7 +384,7 @@ public void testAddRemoveTask() throws Exception { worker = new Worker(WORKER_ID, new MockTime(), config, offsetBackingStore); worker.start(); assertEquals(Collections.emptySet(), worker.taskIds()); - worker.startTask(TASK_ID, new TaskConfig(origProps), taskStatusListener, TargetState.STARTED); + worker.startTask(TASK_ID, new TaskConfig(origProps), anyConnectorConfig(), taskStatusListener, TargetState.STARTED); assertEquals(new HashSet<>(Arrays.asList(TASK_ID)), worker.taskIds()); worker.stopAndAwaitTask(TASK_ID); assertEquals(Collections.emptySet(), worker.taskIds()); @@ -420,8 +424,8 @@ public void testCleanupTasksOnStop() throws Exception { EasyMock.eq(task), EasyMock.anyObject(TaskStatus.Listener.class), EasyMock.eq(TargetState.STARTED), - EasyMock.anyObject(Converter.class), - EasyMock.anyObject(Converter.class), + EasyMock.anyObject(JsonConverter.class), + EasyMock.anyObject(JsonConverter.class), EasyMock.anyObject(KafkaProducer.class), EasyMock.anyObject(OffsetStorageReader.class), EasyMock.anyObject(OffsetStorageWriter.class), @@ -449,12 +453,79 @@ public void testCleanupTasksOnStop() throws Exception { worker = new Worker(WORKER_ID, new MockTime(), config, offsetBackingStore); worker.start(); - worker.startTask(TASK_ID, new TaskConfig(origProps), taskStatusListener, TargetState.STARTED); + worker.startTask(TASK_ID, new TaskConfig(origProps), anyConnectorConfig(), taskStatusListener, TargetState.STARTED); worker.stop(); PowerMock.verifyAll(); } + @Test + public void testConverterOverrides() throws Exception { + expectStartStorage(); + + TestSourceTask task = PowerMock.createMock(TestSourceTask.class); + WorkerSourceTask workerTask = PowerMock.createMock(WorkerSourceTask.class); + EasyMock.expect(workerTask.id()).andStubReturn(TASK_ID); + + PowerMock.mockStaticPartial(Worker.class, "instantiateTask"); + PowerMock.expectPrivate(Worker.class, "instantiateTask", new Object[]{TestSourceTask.class}).andReturn(task); + EasyMock.expect(task.version()).andReturn("1.0"); + + Capture keyConverter = EasyMock.newCapture(); + Capture valueConverter = EasyMock.newCapture(); + + PowerMock.expectNew( + WorkerSourceTask.class, EasyMock.eq(TASK_ID), + EasyMock.eq(task), + EasyMock.anyObject(TaskStatus.Listener.class), + EasyMock.eq(TargetState.STARTED), + EasyMock.capture(keyConverter), + EasyMock.capture(valueConverter), + EasyMock.anyObject(KafkaProducer.class), + EasyMock.anyObject(OffsetStorageReader.class), + EasyMock.anyObject(OffsetStorageWriter.class), + EasyMock.anyObject(WorkerConfig.class), + EasyMock.anyObject(Time.class)) + .andReturn(workerTask); + Map origProps = new HashMap<>(); + origProps.put(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName()); + workerTask.initialize(new TaskConfig(origProps)); + EasyMock.expectLastCall(); + workerTask.run(); + EasyMock.expectLastCall(); + + // Remove + workerTask.stop(); + EasyMock.expectLastCall(); + EasyMock.expect(workerTask.awaitStop(EasyMock.anyLong())).andStubReturn(true); + EasyMock.expectLastCall(); + + expectStopStorage(); + + PowerMock.replayAll(); + + worker = new Worker(WORKER_ID, new MockTime(), config, offsetBackingStore); + worker.start(); + assertEquals(Collections.emptySet(), worker.taskIds()); + Map connProps = anyConnectorConfigMap(); + connProps.put(ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG, TestConverter.class.getName()); + connProps.put("key.converter.extra.config", "foo"); + connProps.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, TestConverter.class.getName()); + connProps.put("value.converter.extra.config", "bar"); + worker.startTask(TASK_ID, new TaskConfig(origProps), new ConnectorConfig(connProps), taskStatusListener, TargetState.STARTED); + assertEquals(new HashSet<>(Arrays.asList(TASK_ID)), worker.taskIds()); + worker.stopAndAwaitTask(TASK_ID); + assertEquals(Collections.emptySet(), worker.taskIds()); + // Nothing should be left, so this should effectively be a nop + worker.stop(); + + // Validate extra configs got passed through to overridden converters + assertEquals("foo", keyConverter.getValue().configs.get("extra.config")); + assertEquals("bar", valueConverter.getValue().configs.get("extra.config")); + + PowerMock.verifyAll(); + } + private void expectStartStorage() { offsetBackingStore.configure(EasyMock.anyObject(WorkerConfig.class)); EasyMock.expectLastCall(); @@ -467,6 +538,17 @@ private void expectStopStorage() { EasyMock.expectLastCall(); } + private Map anyConnectorConfigMap() { + Map props = new HashMap<>(); + props.put(ConnectorConfig.NAME_CONFIG, CONNECTOR_ID); + props.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, WorkerTestConnector.class.getName()); + props.put(ConnectorConfig.TASKS_MAX_CONFIG, "1"); + return props; + } + + private ConnectorConfig anyConnectorConfig() { + return new ConnectorConfig(anyConnectorConfigMap()); + } /* Name here needs to be unique as we are testing the aliasing mechanism */ public static class WorkerTestConnector extends Connector { @@ -527,4 +609,23 @@ public List poll() throws InterruptedException { public void stop() { } } + + public static class TestConverter implements Converter { + public Map configs; + + @Override + public void configure(Map configs, boolean isKey) { + this.configs = configs; + } + + @Override + public byte[] fromConnectData(String topic, Schema schema, Object value) { + return new byte[0]; + } + + @Override + public SchemaAndValue toConnectData(String topic, byte[] value) { + return null; + } + } } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java index 747db1af53d4..8fc6dbd3562e 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java @@ -175,7 +175,7 @@ public void testJoinAssignment() throws Exception { EasyMock.expect(worker.isRunning(CONN1)).andReturn(true); EasyMock.expect(worker.connectorTaskConfigs(CONN1, MAX_TASKS, null)).andReturn(TASK_CONFIGS); - worker.startTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); + worker.startTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall(); member.poll(EasyMock.anyInt()); PowerMock.expectLastCall(); @@ -198,7 +198,7 @@ public void testRebalance() throws Exception { PowerMock.expectLastCall(); EasyMock.expect(worker.isRunning(CONN1)).andReturn(true); EasyMock.expect(worker.connectorTaskConfigs(CONN1, MAX_TASKS, null)).andReturn(TASK_CONFIGS); - worker.startTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); + worker.startTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall(); member.poll(EasyMock.anyInt()); PowerMock.expectLastCall(); @@ -234,7 +234,7 @@ public void testRebalanceFailedConnector() throws Exception { PowerMock.expectLastCall(); EasyMock.expect(worker.isRunning(CONN1)).andReturn(true); EasyMock.expect(worker.connectorTaskConfigs(CONN1, MAX_TASKS, null)).andReturn(TASK_CONFIGS); - worker.startTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); + worker.startTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall(); member.poll(EasyMock.anyInt()); PowerMock.expectLastCall(); @@ -530,7 +530,7 @@ public void testRestartTask() throws Exception { expectPostRebalanceCatchup(SNAPSHOT); member.poll(EasyMock.anyInt()); PowerMock.expectLastCall(); - worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); + worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall(); // now handle the task restart @@ -545,7 +545,7 @@ public void testRestartTask() throws Exception { worker.stopAndAwaitTask(TASK0); PowerMock.expectLastCall(); - worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); + worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall(); PowerMock.replayAll(); @@ -841,7 +841,7 @@ public void testUnknownConnectorPaused() throws Exception { // join expectRebalance(1, Collections.emptyList(), Collections.singletonList(TASK0)); expectPostRebalanceCatchup(SNAPSHOT); - worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); + worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall(); member.poll(EasyMock.anyInt()); PowerMock.expectLastCall(); @@ -877,7 +877,7 @@ public void testConnectorPausedRunningTaskOnly() throws Exception { // join expectRebalance(1, Collections.emptyList(), Collections.singletonList(TASK0)); expectPostRebalanceCatchup(SNAPSHOT); - worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); + worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall(); member.poll(EasyMock.anyInt()); PowerMock.expectLastCall(); @@ -918,7 +918,7 @@ public void testConnectorResumedRunningTaskOnly() throws Exception { // join expectRebalance(1, Collections.emptyList(), Collections.singletonList(TASK0)); expectPostRebalanceCatchup(SNAPSHOT_PAUSED_CONN1); - worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.PAUSED)); + worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.PAUSED)); PowerMock.expectLastCall(); member.poll(EasyMock.anyInt()); PowerMock.expectLastCall(); @@ -970,7 +970,7 @@ public void testTaskConfigAdded() { expectRebalance(Collections.emptyList(), Collections.emptyList(), ConnectProtocol.Assignment.NO_ERROR, 1, Collections.emptyList(), Arrays.asList(TASK0)); - worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); + worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall(); member.poll(EasyMock.anyInt()); PowerMock.expectLastCall(); @@ -1008,7 +1008,7 @@ public void testJoinLeaderCatchUpFails() throws Exception { EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall(); EasyMock.expect(worker.connectorTaskConfigs(CONN1, MAX_TASKS, null)).andReturn(TASK_CONFIGS); - worker.startTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); + worker.startTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall(); EasyMock.expect(worker.isRunning(CONN1)).andReturn(true); member.poll(EasyMock.anyInt()); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java index e70b968d420c..3772586bbe58 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java @@ -73,6 +73,10 @@ public class StandaloneHerderTest { private static final int DEFAULT_MAX_TASKS = 1; private static final String WORKER_ID = "localhost:8083"; + private enum SourceSink { + SOURCE, SINK + }; + private StandaloneHerder herder; private Connector connector; @@ -88,11 +92,11 @@ public void setup() { @Test public void testCreateSourceConnector() throws Exception { connector = PowerMock.createMock(BogusSourceConnector.class); - expectAdd(CONNECTOR_NAME, BogusSourceConnector.class, BogusSourceTask.class, false); + expectAdd(SourceSink.SOURCE); PowerMock.replayAll(); - herder.putConnectorConfig(CONNECTOR_NAME, connectorConfig(CONNECTOR_NAME, BogusSourceConnector.class, false), false, createCallback); + herder.putConnectorConfig(CONNECTOR_NAME, connectorConfig(SourceSink.SOURCE), false, createCallback); PowerMock.verifyAll(); } @@ -101,7 +105,7 @@ public void testCreateSourceConnector() throws Exception { public void testCreateConnectorAlreadyExists() throws Exception { connector = PowerMock.createMock(BogusSourceConnector.class); // First addition should succeed - expectAdd(CONNECTOR_NAME, BogusSourceConnector.class, BogusSourceTask.class, false); + expectAdd(SourceSink.SOURCE); // Second should fail createCallback.onCompletion(EasyMock.anyObject(), EasyMock.>isNull()); @@ -109,8 +113,8 @@ public void testCreateConnectorAlreadyExists() throws Exception { PowerMock.replayAll(); - herder.putConnectorConfig(CONNECTOR_NAME, connectorConfig(CONNECTOR_NAME, BogusSourceConnector.class, false), false, createCallback); - herder.putConnectorConfig(CONNECTOR_NAME, connectorConfig(CONNECTOR_NAME, BogusSourceConnector.class, false), false, createCallback); + herder.putConnectorConfig(CONNECTOR_NAME, connectorConfig(SourceSink.SOURCE), false, createCallback); + herder.putConnectorConfig(CONNECTOR_NAME, connectorConfig(SourceSink.SOURCE), false, createCallback); PowerMock.verifyAll(); } @@ -118,11 +122,11 @@ public void testCreateConnectorAlreadyExists() throws Exception { @Test public void testCreateSinkConnector() throws Exception { connector = PowerMock.createMock(BogusSinkConnector.class); - expectAdd(CONNECTOR_NAME, BogusSinkConnector.class, BogusSinkTask.class, true); + expectAdd(SourceSink.SINK); PowerMock.replayAll(); - herder.putConnectorConfig(CONNECTOR_NAME, connectorConfig(CONNECTOR_NAME, BogusSinkConnector.class, true), false, createCallback); + herder.putConnectorConfig(CONNECTOR_NAME, connectorConfig(SourceSink.SINK), false, createCallback); PowerMock.verifyAll(); } @@ -130,7 +134,7 @@ public void testCreateSinkConnector() throws Exception { @Test public void testDestroyConnector() throws Exception { connector = PowerMock.createMock(BogusSourceConnector.class); - expectAdd(CONNECTOR_NAME, BogusSourceConnector.class, BogusSourceTask.class, false); + expectAdd(SourceSink.SOURCE); EasyMock.expect(statusBackingStore.getAll(CONNECTOR_NAME)).andReturn(Collections.emptyList()); statusBackingStore.put(new ConnectorStatus(CONNECTOR_NAME, AbstractStatus.State.DESTROYED, WORKER_ID, 0)); @@ -139,7 +143,7 @@ public void testDestroyConnector() throws Exception { PowerMock.replayAll(); - herder.putConnectorConfig(CONNECTOR_NAME, connectorConfig(CONNECTOR_NAME, BogusSourceConnector.class, false), false, createCallback); + herder.putConnectorConfig(CONNECTOR_NAME, connectorConfig(SourceSink.SOURCE), false, createCallback); FutureCallback> futureCb = new FutureCallback<>(); herder.putConnectorConfig(CONNECTOR_NAME, null, true, futureCb); futureCb.get(1000L, TimeUnit.MILLISECONDS); @@ -159,18 +163,18 @@ public void testDestroyConnector() throws Exception { @Test public void testRestartConnector() throws Exception { - expectAdd(CONNECTOR_NAME, BogusSourceConnector.class, BogusSourceTask.class, false); + expectAdd(SourceSink.SOURCE); worker.stopConnector(CONNECTOR_NAME); EasyMock.expectLastCall(); - worker.startConnector(EasyMock.eq(new ConnectorConfig(connectorConfig(CONNECTOR_NAME, BogusSourceConnector.class, false))), + worker.startConnector(EasyMock.eq(new ConnectorConfig(connectorConfig(SourceSink.SOURCE))), EasyMock.anyObject(HerderConnectorContext.class), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); EasyMock.expectLastCall(); PowerMock.replayAll(); - herder.putConnectorConfig(CONNECTOR_NAME, connectorConfig(CONNECTOR_NAME, BogusSourceConnector.class, false), false, createCallback); + herder.putConnectorConfig(CONNECTOR_NAME, connectorConfig(SourceSink.SOURCE), false, createCallback); FutureCallback cb = new FutureCallback<>(); herder.restartConnector(CONNECTOR_NAME, cb); @@ -181,7 +185,7 @@ public void testRestartConnector() throws Exception { @Test public void testRestartConnectorFailureOnStop() throws Exception { - expectAdd(CONNECTOR_NAME, BogusSourceConnector.class, BogusSourceTask.class, false); + expectAdd(SourceSink.SOURCE); RuntimeException e = new RuntimeException(); worker.stopConnector(CONNECTOR_NAME); @@ -191,7 +195,7 @@ public void testRestartConnectorFailureOnStop() throws Exception { PowerMock.replayAll(); - herder.putConnectorConfig(CONNECTOR_NAME, connectorConfig(CONNECTOR_NAME, BogusSourceConnector.class, false), false, createCallback); + herder.putConnectorConfig(CONNECTOR_NAME, connectorConfig(SourceSink.SOURCE), false, createCallback); FutureCallback cb = new FutureCallback<>(); herder.restartConnector(CONNECTOR_NAME, cb); @@ -207,19 +211,19 @@ public void testRestartConnectorFailureOnStop() throws Exception { @Test public void testRestartConnectorFailureOnStart() throws Exception { - expectAdd(CONNECTOR_NAME, BogusSourceConnector.class, BogusSourceTask.class, false); + expectAdd(SourceSink.SOURCE); worker.stopConnector(CONNECTOR_NAME); EasyMock.expectLastCall(); RuntimeException e = new RuntimeException(); - worker.startConnector(EasyMock.eq(new ConnectorConfig(connectorConfig(CONNECTOR_NAME, BogusSourceConnector.class, false))), + worker.startConnector(EasyMock.eq(new ConnectorConfig(connectorConfig(SourceSink.SOURCE))), EasyMock.anyObject(HerderConnectorContext.class), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); EasyMock.expectLastCall().andThrow(e); PowerMock.replayAll(); - herder.putConnectorConfig(CONNECTOR_NAME, connectorConfig(CONNECTOR_NAME, BogusSourceConnector.class, false), false, createCallback); + herder.putConnectorConfig(CONNECTOR_NAME, connectorConfig(SourceSink.SOURCE), false, createCallback); FutureCallback cb = new FutureCallback<>(); herder.restartConnector(CONNECTOR_NAME, cb); @@ -236,18 +240,19 @@ public void testRestartConnectorFailureOnStart() throws Exception { @Test public void testRestartTask() throws Exception { ConnectorTaskId taskId = new ConnectorTaskId(CONNECTOR_NAME, 0); - expectAdd(CONNECTOR_NAME, BogusSourceConnector.class, BogusSourceTask.class, false); + expectAdd(SourceSink.SOURCE); worker.stopAndAwaitTask(taskId); EasyMock.expectLastCall(); - Map generatedTaskProps = taskConfig(BogusSourceTask.class, false); - worker.startTask(taskId, new TaskConfig(generatedTaskProps), herder, TargetState.STARTED); + ConnectorConfig connConfig = new ConnectorConfig(connectorConfig(SourceSink.SOURCE)); + TaskConfig taskConfig = new TaskConfig(taskConfig(SourceSink.SOURCE)); + worker.startTask(taskId, taskConfig, connConfig, herder, TargetState.STARTED); EasyMock.expectLastCall(); PowerMock.replayAll(); - herder.putConnectorConfig(CONNECTOR_NAME, connectorConfig(CONNECTOR_NAME, BogusSourceConnector.class, false), false, createCallback); + herder.putConnectorConfig(CONNECTOR_NAME, connectorConfig(SourceSink.SOURCE), false, createCallback); FutureCallback cb = new FutureCallback<>(); herder.restartTask(taskId, cb); @@ -259,7 +264,7 @@ public void testRestartTask() throws Exception { @Test public void testRestartTaskFailureOnStop() throws Exception { ConnectorTaskId taskId = new ConnectorTaskId(CONNECTOR_NAME, 0); - expectAdd(CONNECTOR_NAME, BogusSourceConnector.class, BogusSourceTask.class, false); + expectAdd(SourceSink.SOURCE); RuntimeException e = new RuntimeException(); worker.stopAndAwaitTask(taskId); @@ -269,7 +274,7 @@ public void testRestartTaskFailureOnStop() throws Exception { PowerMock.replayAll(); - herder.putConnectorConfig(CONNECTOR_NAME, connectorConfig(CONNECTOR_NAME, BogusSourceConnector.class, false), false, createCallback); + herder.putConnectorConfig(CONNECTOR_NAME, connectorConfig(SourceSink.SOURCE), false, createCallback); FutureCallback cb = new FutureCallback<>(); herder.restartTask(taskId, cb); @@ -285,19 +290,20 @@ public void testRestartTaskFailureOnStop() throws Exception { @Test public void testRestartTaskFailureOnStart() throws Exception { ConnectorTaskId taskId = new ConnectorTaskId(CONNECTOR_NAME, 0); - expectAdd(CONNECTOR_NAME, BogusSourceConnector.class, BogusSourceTask.class, false); + expectAdd(SourceSink.SOURCE); worker.stopAndAwaitTask(taskId); EasyMock.expectLastCall(); RuntimeException e = new RuntimeException(); - Map generatedTaskProps = taskConfig(BogusSourceTask.class, false); - worker.startTask(taskId, new TaskConfig(generatedTaskProps), herder, TargetState.STARTED); + ConnectorConfig connConfig = new ConnectorConfig(connectorConfig(SourceSink.SOURCE)); + TaskConfig taskConfig = new TaskConfig(taskConfig(SourceSink.SOURCE)); + worker.startTask(taskId, taskConfig, connConfig, herder, TargetState.STARTED); EasyMock.expectLastCall().andThrow(e); PowerMock.replayAll(); - herder.putConnectorConfig(CONNECTOR_NAME, connectorConfig(CONNECTOR_NAME, BogusSourceConnector.class, false), false, createCallback); + herder.putConnectorConfig(CONNECTOR_NAME, connectorConfig(SourceSink.SOURCE), false, createCallback); FutureCallback cb = new FutureCallback<>(); herder.restartTask(taskId, cb); @@ -314,7 +320,7 @@ public void testRestartTaskFailureOnStart() throws Exception { @Test public void testCreateAndStop() throws Exception { connector = PowerMock.createMock(BogusSourceConnector.class); - expectAdd(CONNECTOR_NAME, BogusSourceConnector.class, BogusSourceTask.class, false); + expectAdd(SourceSink.SOURCE); // herder.stop() should stop any running connectors and tasks even if destroyConnector was not invoked expectStop(); @@ -325,7 +331,7 @@ public void testCreateAndStop() throws Exception { PowerMock.replayAll(); - herder.putConnectorConfig(CONNECTOR_NAME, connectorConfig(CONNECTOR_NAME, BogusSourceConnector.class, false), false, createCallback); + herder.putConnectorConfig(CONNECTOR_NAME, connectorConfig(SourceSink.SOURCE), false, createCallback); herder.stop(); PowerMock.verifyAll(); @@ -333,7 +339,7 @@ public void testCreateAndStop() throws Exception { @Test public void testAccessors() throws Exception { - Map connConfig = connectorConfig(CONNECTOR_NAME, BogusSourceConnector.class, false); + Map connConfig = connectorConfig(SourceSink.SOURCE); Callback> listConnectorsCb = PowerMock.createMock(Callback.class); Callback connectorInfoCb = PowerMock.createMock(Callback.class); @@ -353,7 +359,7 @@ public void testAccessors() throws Exception { // Create connector connector = PowerMock.createMock(BogusSourceConnector.class); - expectAdd(CONNECTOR_NAME, BogusSourceConnector.class, BogusSourceTask.class, false); + expectAdd(SourceSink.SOURCE); // Validate accessors with 1 connector listConnectorsCb.onCompletion(null, Collections.singleton(CONNECTOR_NAME)); @@ -364,7 +370,7 @@ public void testAccessors() throws Exception { connectorConfigCb.onCompletion(null, connConfig); EasyMock.expectLastCall(); - TaskInfo taskInfo = new TaskInfo(new ConnectorTaskId(CONNECTOR_NAME, 0), taskConfig(BogusSourceTask.class, false)); + TaskInfo taskInfo = new TaskInfo(new ConnectorTaskId(CONNECTOR_NAME, 0), taskConfig(SourceSink.SOURCE)); taskConfigsCb.onCompletion(null, Arrays.asList(taskInfo)); EasyMock.expectLastCall(); @@ -388,7 +394,7 @@ public void testAccessors() throws Exception { @Test public void testPutConnectorConfig() throws Exception { - Map connConfig = connectorConfig(CONNECTOR_NAME, BogusSourceConnector.class, false); + Map connConfig = connectorConfig(SourceSink.SOURCE); Map newConnConfig = new HashMap<>(connConfig); newConnConfig.put("foo", "bar"); @@ -397,7 +403,7 @@ public void testPutConnectorConfig() throws Exception { // Create connector = PowerMock.createMock(BogusSourceConnector.class); - expectAdd(CONNECTOR_NAME, BogusSourceConnector.class, BogusSourceTask.class, false); + expectAdd(SourceSink.SOURCE); // Should get first config connectorConfigCb.onCompletion(null, connConfig); EasyMock.expectLastCall(); @@ -411,7 +417,7 @@ public void testPutConnectorConfig() throws Exception { EasyMock.expect(worker.isRunning(CONNECTOR_NAME)).andReturn(true); // Generate same task config, which should result in no additional action to restart tasks EasyMock.expect(worker.connectorTaskConfigs(CONNECTOR_NAME, DEFAULT_MAX_TASKS, null)) - .andReturn(Collections.singletonList(taskConfig(BogusSourceTask.class, false))); + .andReturn(Collections.singletonList(taskConfig(SourceSink.SOURCE))); worker.isSinkConnector(CONNECTOR_NAME); EasyMock.expectLastCall().andReturn(false); ConnectorInfo newConnInfo = new ConnectorInfo(CONNECTOR_NAME, newConnConfig, Arrays.asList(new ConnectorTaskId(CONNECTOR_NAME, 0))); @@ -446,17 +452,14 @@ public void testPutTaskConfigs() { PowerMock.verifyAll(); } - private void expectAdd(String name, - Class connClass, - Class taskClass, - boolean sink) throws Exception { + private void expectAdd(SourceSink sourceSink) throws Exception { - Map connectorProps = connectorConfig(name, connClass, sink); + Map connectorProps = connectorConfig(sourceSink); worker.startConnector(EasyMock.eq(new ConnectorConfig(connectorProps)), EasyMock.anyObject(HerderConnectorContext.class), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); EasyMock.expectLastCall(); - EasyMock.expect(worker.isRunning(name)).andReturn(true); + EasyMock.expect(worker.isRunning(CONNECTOR_NAME)).andReturn(true); ConnectorInfo connInfo = new ConnectorInfo(CONNECTOR_NAME, connectorProps, Arrays.asList(new ConnectorTaskId(CONNECTOR_NAME, 0))); createCallback.onCompletion(null, new Herder.Created<>(true, connInfo)); @@ -464,16 +467,18 @@ private void expectAdd(String name, // And we should instantiate the tasks. For a sink task, we should see added properties for // the input topic partitions - Map generatedTaskProps = taskConfig(taskClass, sink); + ConnectorConfig connConfig = new ConnectorConfig(connectorConfig(sourceSink)); + Map generatedTaskProps = taskConfig(sourceSink); + TaskConfig taskConfig = new TaskConfig(generatedTaskProps); - EasyMock.expect(worker.connectorTaskConfigs(CONNECTOR_NAME, DEFAULT_MAX_TASKS, sink ? TOPICS_LIST : null)) + EasyMock.expect(worker.connectorTaskConfigs(CONNECTOR_NAME, DEFAULT_MAX_TASKS, sourceSink == SourceSink.SINK ? TOPICS_LIST : null)) .andReturn(Collections.singletonList(generatedTaskProps)); - worker.startTask(new ConnectorTaskId(CONNECTOR_NAME, 0), new TaskConfig(generatedTaskProps), herder, TargetState.STARTED); + worker.startTask(new ConnectorTaskId(CONNECTOR_NAME, 0), taskConfig, connConfig, herder, TargetState.STARTED); EasyMock.expectLastCall(); worker.isSinkConnector(CONNECTOR_NAME); - PowerMock.expectLastCall().andReturn(sink); + PowerMock.expectLastCall().andReturn(sourceSink == SourceSink.SINK); } private void expectStop() { @@ -490,22 +495,24 @@ private void expectDestroy() { expectStop(); } - private static HashMap connectorConfig(String name, Class connClass, boolean sink) { - HashMap connectorProps = new HashMap<>(); - connectorProps.put(ConnectorConfig.NAME_CONFIG, name); - connectorProps.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, connClass.getName()); - if (sink) { - connectorProps.put(SinkConnector.TOPICS_CONFIG, TOPICS_LIST_STR); - } - return connectorProps; + private static Map connectorConfig(SourceSink sourceSink) { + Map props = new HashMap<>(); + props.put(ConnectorConfig.NAME_CONFIG, CONNECTOR_NAME); + Class connectorClass = sourceSink == SourceSink.SINK ? BogusSinkConnector.class : BogusSourceConnector.class; + props.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, connectorClass.getName()); + props.put(ConnectorConfig.TASKS_MAX_CONFIG, "1"); + if (sourceSink == SourceSink.SINK) + props.put(SinkTask.TOPICS_CONFIG, TOPICS_LIST_STR); + return props; } - private static Map taskConfig(Class taskClass, boolean sink) { + private static Map taskConfig(SourceSink sourceSink) { HashMap generatedTaskProps = new HashMap<>(); // Connectors can add any settings, so these are arbitrary generatedTaskProps.put("foo", "bar"); + Class taskClass = sourceSink == SourceSink.SINK ? BogusSinkTask.class : BogusSourceTask.class; generatedTaskProps.put(TaskConfig.TASK_CLASS_CONFIG, taskClass.getName()); - if (sink) + if (sourceSink == SourceSink.SINK) generatedTaskProps.put(SinkTask.TOPICS_CONFIG, TOPICS_LIST_STR); return generatedTaskProps; } diff --git a/docs/connect.html b/docs/connect.html index e5a4ad2b55c8..de3b5aaae237 100644 --- a/docs/connect.html +++ b/docs/connect.html @@ -43,7 +43,17 @@

Running Kafka Connect -The first parameter is the configuration for the worker. This includes settings such as the Kafka connection parameters, serialization format, and how frequently to commit offsets. The provided example should work well with a local cluster running with the default configuration provided by config/server.properties. It will require tweaking to use with a different configuration or production deployment. +The first parameter is the configuration for the worker. This includes settings such as the Kafka connection parameters, serialization format, and how frequently to commit offsets. The provided example should work well with a local cluster running with the default configuration provided by config/server.properties. It will require tweaking to use with a different configuration or production deployment. All workers (both standalone and distributed) require a few configs: +
    +
  • bootstrap.servers - List of Kafka servers used to bootstrap connections to Kafka
  • +
  • key.converter - Converter class used to convert between Kafka Connect format and the serialized form that is written to Kafka. This controls the format of the keys in messages written to or read from Kafka, and since this is independent of connectors it allows any connector to work with any serialization format. Examples of common formats include JSON and Avro.
  • +
  • value.converter - Converter class used to convert between Kafka Connect format and the serialized form that is written to Kafka. This controls the format of the values in messages written to or read from Kafka, and since this is independent of connectors it allows any connector to work with any serialization format. Examples of common formats include JSON and Avro.
  • +
+ +The important configuration options specific to standalone mode are: +
    +
  • offset.storage.file.filename - File to store offset data in
  • +
The remaining parameters are connector configuration files. You may include as many as you want, but all will execute within the same process (on different threads). @@ -55,7 +65,7 @@

Running Kafka Connect
  • group.id (default connect-cluster) - unique name for the cluster, used in forming the Connect cluster group; note that this must not conflict with consumer group IDs
  • config.storage.topic (default connect-configs) - topic to use for storing connector and task configurations; note that this should be a single partition, highly replicated topic. You may need to manually create the topic to ensure single partition for the config topic as auto created topics may have multiple partitions.
  • @@ -76,6 +86,8 @@

    Configuring Connecto
  • name - Unique name for the connector. Attempting to register again with the same name will fail.
  • connector.class - The Java class for the connector
  • tasks.max - The maximum number of tasks that should be created for this connector. The connector may create fewer tasks if it cannot achieve this level of parallelism.
  • +
  • key.converter - (optional) Override the default key converter set by the worker.
  • +
  • value.converter - (optional) Override the default value converter set by the worker.
  • The connector.class config supports several formats: the full name or alias of the class for this connector. If the connector is org.apache.kafka.connect.file.FileStreamSinkConnector, you can either specify this full name or use FileStreamSink or FileStreamSinkConnector to make the configuration a bit shorter. diff --git a/tests/kafkatest/tests/connect/connect_rest_test.py b/tests/kafkatest/tests/connect/connect_rest_test.py index 0b004996cafd..70bc32c9d5b9 100644 --- a/tests/kafkatest/tests/connect/connect_rest_test.py +++ b/tests/kafkatest/tests/connect/connect_rest_test.py @@ -29,8 +29,8 @@ class ConnectRestApiTest(KafkaTest): FILE_SOURCE_CONNECTOR = 'org.apache.kafka.connect.file.FileStreamSourceConnector' FILE_SINK_CONNECTOR = 'org.apache.kafka.connect.file.FileStreamSinkConnector' - FILE_SOURCE_CONFIGS = {'name', 'connector.class', 'tasks.max', 'topic', 'file'} - FILE_SINK_CONFIGS = {'name', 'connector.class', 'tasks.max', 'topics', 'file'} + FILE_SOURCE_CONFIGS = {'name', 'connector.class', 'tasks.max', 'key.converter', 'value.converter', 'topic', 'file'} + FILE_SINK_CONFIGS = {'name', 'connector.class', 'tasks.max', 'key.converter', 'value.converter', 'topics', 'file'} INPUT_FILE = "/mnt/connect.input" INPUT_FILE2 = "/mnt/connect.input2" diff --git a/tests/kafkatest/tests/connect/connect_test.py b/tests/kafkatest/tests/connect/connect_test.py index 91843900e73d..93f57344fb9a 100644 --- a/tests/kafkatest/tests/connect/connect_test.py +++ b/tests/kafkatest/tests/connect/connect_test.py @@ -63,10 +63,17 @@ def __init__(self, test_context): @parametrize(converter="org.apache.kafka.connect.storage.StringConverter", schemas=None) @matrix(security_protocol=[SecurityConfig.PLAINTEXT, SecurityConfig.SASL_SSL]) def test_file_source_and_sink(self, converter="org.apache.kafka.connect.json.JsonConverter", schemas=True, security_protocol='PLAINTEXT'): + """ + Validates basic end-to-end functionality of Connect standalone using the file source and sink converters. Includes + parameterizations to test different converters (which also test per-connector converter overrides), schema/schemaless + modes, and security support. + """ assert converter != None, "converter type must be set" - # Template parameters - self.key_converter = converter - self.value_converter = converter + # Template parameters. Note that we don't set key/value.converter. These default to JsonConverter and we validate + # converter overrides via the connector configuration. + if converter != "org.apache.kafka.connect.json.JsonConverter": + self.override_key_converter = converter + self.override_value_converter = converter self.schemas = schemas self.kafka = KafkaService(self.test_context, self.num_brokers, self.zk, diff --git a/tests/kafkatest/tests/connect/templates/connect-file-sink.properties b/tests/kafkatest/tests/connect/templates/connect-file-sink.properties index 216dab55592b..bff002bbdb07 100644 --- a/tests/kafkatest/tests/connect/templates/connect-file-sink.properties +++ b/tests/kafkatest/tests/connect/templates/connect-file-sink.properties @@ -17,4 +17,12 @@ name=local-file-sink connector.class={{ FILE_SINK_CONNECTOR }} tasks.max=1 file={{ OUTPUT_FILE }} -topics={{ TOPIC }} \ No newline at end of file +topics={{ TOPIC }} + +# For testing per-connector converters +{% if override_key_converter is defined %} +key.converter={{ override_key_converter }} +{% endif %} +{% if override_key_converter is defined %} +value.converter={{ override_value_converter }} +{% endif %} \ No newline at end of file diff --git a/tests/kafkatest/tests/connect/templates/connect-file-source.properties b/tests/kafkatest/tests/connect/templates/connect-file-source.properties index bff9720b8bfb..800d6a0751ba 100644 --- a/tests/kafkatest/tests/connect/templates/connect-file-source.properties +++ b/tests/kafkatest/tests/connect/templates/connect-file-source.properties @@ -17,4 +17,12 @@ name=local-file-source connector.class={{ FILE_SOURCE_CONNECTOR }} tasks.max=1 file={{ INPUT_FILE }} -topic={{ TOPIC }} \ No newline at end of file +topic={{ TOPIC }} + +# For testing per-connector converters +{% if override_key_converter is defined %} +key.converter={{ override_key_converter }} +{% endif %} +{% if override_key_converter is defined %} +value.converter={{ override_value_converter }} +{% endif %} From 79d3fd2bf0e5c89ff74a2988c403882ae8a9852e Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Fri, 19 Aug 2016 10:07:07 -0700 Subject: [PATCH 10/35] KAFKA-3163; Add time based index to Kafka. This patch is for KIP-33. https://cwiki.apache.org/confluence/display/KAFKA/KIP-33+-+Add+a+time+based+log+index Author: Jiangjie Qin Reviewers: Ismael Juma , Jun Rao , Liquan Pei Closes #1215 from becketqin/KAFKA-3163 --- .../main/scala/kafka/log/AbstractIndex.scala | 287 ++++++++++++++++ .../main/scala/kafka/log/FileMessageSet.scala | 78 ++++- ...{OffsetPosition.scala => IndexEntry.scala} | 23 +- core/src/main/scala/kafka/log/Log.scala | 196 ++++++++--- .../src/main/scala/kafka/log/LogCleaner.scala | 41 ++- .../src/main/scala/kafka/log/LogManager.scala | 6 +- .../src/main/scala/kafka/log/LogSegment.scala | 169 ++++++++-- .../main/scala/kafka/log/OffsetIndex.scala | 306 +++--------------- core/src/main/scala/kafka/log/TimeIndex.scala | 208 ++++++++++++ .../kafka/message/ByteBufferMessageSet.scala | 91 ++++-- .../kafka/message/MessageAndOffset.scala | 8 + .../main/scala/kafka/server/KafkaApis.scala | 3 +- .../scala/kafka/server/ReplicaManager.scala | 2 +- .../scala/kafka/tools/DumpLogSegments.scala | 122 ++++++- .../scala/unit/kafka/log/CleanerTest.scala | 5 +- .../unit/kafka/log/FileMessageSetTest.scala | 16 +- .../scala/unit/kafka/log/LogManagerTest.scala | 4 +- .../scala/unit/kafka/log/LogSegmentTest.scala | 114 +++++-- .../test/scala/unit/kafka/log/LogTest.scala | 152 +++++++-- .../scala/unit/kafka/log/TimeIndexTest.scala | 97 ++++++ .../message/ByteBufferMessageSetTest.scala | 135 +++++--- .../scala/unit/kafka/utils/TestUtils.scala | 3 +- docs/upgrade.html | 16 +- 23 files changed, 1607 insertions(+), 475 deletions(-) create mode 100644 core/src/main/scala/kafka/log/AbstractIndex.scala rename core/src/main/scala/kafka/log/{OffsetPosition.scala => IndexEntry.scala} (57%) create mode 100644 core/src/main/scala/kafka/log/TimeIndex.scala create mode 100644 core/src/test/scala/unit/kafka/log/TimeIndexTest.scala diff --git a/core/src/main/scala/kafka/log/AbstractIndex.scala b/core/src/main/scala/kafka/log/AbstractIndex.scala new file mode 100644 index 000000000000..d594f188f464 --- /dev/null +++ b/core/src/main/scala/kafka/log/AbstractIndex.scala @@ -0,0 +1,287 @@ +/** + * 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.log + +import java.io.{File, RandomAccessFile} +import java.nio.{ByteBuffer, MappedByteBuffer} +import java.nio.channels.FileChannel +import java.util.concurrent.locks.{Lock, ReentrantLock} + +import kafka.log.IndexSearchType.IndexSearchEntity +import kafka.utils.CoreUtils.inLock +import kafka.utils.{CoreUtils, Logging, Os} +import org.apache.kafka.common.utils.Utils +import sun.nio.ch.DirectBuffer + +import scala.math.ceil + +/** + * The abstract index class which holds entry format agnostic methods. + * + * @param _file The index file + * @param baseOffset the base offset of the segment that this index is corresponding to. + * @param maxIndexSize The maximum index size in bytes. + */ +abstract class AbstractIndex[K, V](@volatile private[this] var _file: File, val baseOffset: Long, val maxIndexSize: Int = -1) + extends Logging { + + protected def entrySize: Int + + protected val lock = new ReentrantLock + + @volatile + protected var mmap: MappedByteBuffer = { + val newlyCreated = _file.createNewFile() + val raf = new RandomAccessFile(_file, "rw") + try { + /* pre-allocate the file if necessary */ + if(newlyCreated) { + if(maxIndexSize < entrySize) + throw new IllegalArgumentException("Invalid max index size: " + maxIndexSize) + raf.setLength(roundDownToExactMultiple(maxIndexSize, entrySize)) + } + + /* memory-map the file */ + val len = raf.length() + val idx = raf.getChannel.map(FileChannel.MapMode.READ_WRITE, 0, len) + + /* set the position in the index for the next entry */ + if(newlyCreated) + idx.position(0) + else + // if this is a pre-existing index, assume it is valid and set position to last entry + idx.position(roundDownToExactMultiple(idx.limit, entrySize)) + idx + } finally { + CoreUtils.swallow(raf.close()) + } + } + + /** + * The maximum number of entries this index can hold + */ + @volatile + private[this] var _maxEntries = mmap.limit / entrySize + + /** The number of entries in this index */ + @volatile + protected var _entries = mmap.position / entrySize + + /** + * True iff there are no more slots available in this index + */ + def isFull: Boolean = _entries >= _maxEntries + + def maxEntries: Int = _maxEntries + + def entries: Int = _entries + + /** + * The index file + */ + def file: File = _file + + /** + * Reset the size of the memory map and the underneath file. This is used in two kinds of cases: (1) in + * trimToValidSize() which is called at closing the segment or new segment being rolled; (2) at + * loading segments from disk or truncating back to an old segment where a new log segment became active; + * we want to reset the index size to maximum index size to avoid rolling new segment. + */ + def resize(newSize: Int) { + inLock(lock) { + val raf = new RandomAccessFile(_file, "rw") + val roundedNewSize = roundDownToExactMultiple(newSize, entrySize) + val position = mmap.position + + /* Windows won't let us modify the file length while the file is mmapped :-( */ + if(Os.isWindows) + forceUnmap(mmap) + try { + raf.setLength(roundedNewSize) + mmap = raf.getChannel().map(FileChannel.MapMode.READ_WRITE, 0, roundedNewSize) + _maxEntries = mmap.limit / entrySize + mmap.position(position) + } finally { + CoreUtils.swallow(raf.close()) + } + } + } + + /** + * Rename the file that backs this offset index + * + * @throws IOException if rename fails + */ + def renameTo(f: File) { + try Utils.atomicMoveWithFallback(_file.toPath, f.toPath) + finally _file = f + } + + /** + * Flush the data in the index to disk + */ + def flush() { + inLock(lock) { + mmap.force() + } + } + + /** + * Delete this index file + */ + def delete(): Boolean = { + info(s"Deleting index ${_file.getAbsolutePath}") + if(Os.isWindows) + CoreUtils.swallow(forceUnmap(mmap)) + _file.delete() + } + + /** + * Trim this segment to fit just the valid entries, deleting all trailing unwritten bytes from + * the file. + */ + def trimToValidSize() { + inLock(lock) { + resize(entrySize * _entries) + } + } + + /** + * The number of bytes actually used by this index + */ + def sizeInBytes = entrySize * _entries + + /** Close the index */ + def close() { + trimToValidSize() + } + + /** + * Do a basic sanity check on this index to detect obvious problems + * + * @throws IllegalArgumentException if any problems are found + */ + def sanityCheck(): Unit + + /** + * Remove all the entries from the index. + */ + def truncate(): Unit + + /** + * Remove all entries from the index which have an offset greater than or equal to the given offset. + * Truncating to an offset larger than the largest in the index has no effect. + */ + def truncateTo(offset: Long): Unit + + /** + * Forcefully free the buffer's mmap. We do this only on windows. + */ + protected def forceUnmap(m: MappedByteBuffer) { + try { + m match { + case buffer: DirectBuffer => + val bufferCleaner = buffer.cleaner() + /* cleaner can be null if the mapped region has size 0 */ + if (bufferCleaner != null) + bufferCleaner.clean() + case _ => + } + } catch { + case t: Throwable => error("Error when freeing index buffer", t) + } + } + + /** + * Execute the given function in a lock only if we are running on windows. We do this + * because Windows won't let us resize a file while it is mmapped. As a result we have to force unmap it + * and this requires synchronizing reads. + */ + protected def maybeLock[T](lock: Lock)(fun: => T): T = { + if(Os.isWindows) + lock.lock() + try { + fun + } finally { + if(Os.isWindows) + lock.unlock() + } + } + + /** + * To parse an entry in the index. + * + * @param buffer the buffer of this memory mapped index. + * @param n the slot + * @return the index entry stored in the given slot. + */ + protected def parseEntry(buffer: ByteBuffer, n: Int): IndexEntry + + /** + * Find the slot in which the largest entry less than or equal to the given target key or value is stored. + * The comparison is made using the `IndexEntry.compareTo()` method. + * + * @param idx The index buffer + * @param target The index key to look for + * @return The slot found or -1 if the least entry in the index is larger than the target key or the index is empty + */ + protected def indexSlotFor(idx: ByteBuffer, target: Long, searchEntity: IndexSearchEntity): Int = { + // check if the index is empty + if(_entries == 0) + return -1 + + // check if the target offset is smaller than the least offset + if(compareIndexEntry(parseEntry(idx, 0), target, searchEntity) > 0) + return -1 + + // binary search for the entry + var lo = 0 + var hi = _entries - 1 + while(lo < hi) { + val mid = ceil(hi/2.0 + lo/2.0).toInt + val found = parseEntry(idx, mid) + val compareResult = compareIndexEntry(found, target, searchEntity) + if(compareResult > 0) + hi = mid - 1 + else if(compareResult < 0) + lo = mid + else + return mid + } + lo + } + + private def compareIndexEntry(indexEntry: IndexEntry, target: Long, searchEntity: IndexSearchEntity): Int = { + searchEntity match { + case IndexSearchType.KEY => indexEntry.indexKey.compareTo(target) + case IndexSearchType.VALUE => indexEntry.indexValue.compareTo(target) + } + } + + /** + * Round a number to the greatest exact multiple of the given factor less than the given number. + * E.g. roundDownToExactMultiple(67, 8) == 64 + */ + private def roundDownToExactMultiple(number: Int, factor: Int) = factor * (number / factor) + +} + +object IndexSearchType extends Enumeration { + type IndexSearchEntity = Value + val KEY, VALUE = Value +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/log/FileMessageSet.scala b/core/src/main/scala/kafka/log/FileMessageSet.scala index 8e92f954fed4..57630429e341 100755 --- a/core/src/main/scala/kafka/log/FileMessageSet.scala +++ b/core/src/main/scala/kafka/log/FileMessageSet.scala @@ -117,7 +117,13 @@ class FileMessageSet private[kafka](@volatile var file: File, new FileMessageSet(file, channel, start = this.start + position, - end = math.min(this.start + position + size, sizeInBytes())) + end = { + // Handle the integer overflow + if (this.start + position + size < 0) + sizeInBytes() + else + math.min(this.start + position + size, sizeInBytes()) + }) } /** @@ -126,7 +132,7 @@ class FileMessageSet private[kafka](@volatile var file: File, * @param targetOffset The offset to search for. * @param startingPosition The starting position in the file to begin searching from. */ - def searchFor(targetOffset: Long, startingPosition: Int): OffsetPosition = { + def searchForOffset(targetOffset: Long, startingPosition: Int): OffsetPosition = { var position = startingPosition val buffer = ByteBuffer.allocate(MessageSet.LogOverhead) val size = sizeInBytes() @@ -135,7 +141,7 @@ class FileMessageSet private[kafka](@volatile var file: File, channel.read(buffer, position) if(buffer.hasRemaining) throw new IllegalStateException("Failed to read complete buffer for targetOffset %d startPosition %d in %s" - .format(targetOffset, startingPosition, file.getAbsolutePath)) + .format(targetOffset, startingPosition, file.getAbsolutePath)) buffer.rewind() val offset = buffer.getLong() if(offset >= targetOffset) @@ -148,6 +154,72 @@ class FileMessageSet private[kafka](@volatile var file: File, null } + /** + * Search forward for the message whose timestamp is greater than or equals to the target timestamp. + * + * The search will stop immediately when it sees a message in format version before 0.10.0. This is to avoid + * scanning the entire log when all the messages are still in old format. + * + * @param targetTimestamp The timestamp to search for. + * @param startingPosition The starting position to search. + * @return None, if no message exists at or after the starting position. + * Some(the_next_offset_to_read) otherwise. + */ + def searchForTimestamp(targetTimestamp: Long, startingPosition: Int): Option[Long] = { + var maxTimestampChecked = Message.NoTimestamp + var lastOffsetChecked = -1L + val messagesToSearch = read(startingPosition, sizeInBytes) + for (messageAndOffset <- messagesToSearch) { + val message = messageAndOffset.message + lastOffsetChecked = messageAndOffset.offset + // Stop searching once we see message format before 0.10.0. + // This equivalent as treating message without timestamp has the largest timestamp. + // We do this to avoid scanning the entire log if no message has a timestamp. + if (message.magic == Message.MagicValue_V0) + return Some(messageAndOffset.offset) + else if (message.timestamp >= targetTimestamp) { + // We found a message + message.compressionCodec match { + case NoCompressionCodec => + return Some(messageAndOffset.offset) + case _ => + // Iterate over the inner messages to get the exact offset. + for (innerMessage <- ByteBufferMessageSet.deepIterator(messageAndOffset)) { + val timestamp = innerMessage.message.timestamp + if (timestamp >= targetTimestamp) + return Some(innerMessage.offset) + } + throw new IllegalStateException(s"The message set (max timestamp = ${message.timestamp}, max offset = ${messageAndOffset.offset}" + + s" should contain target timestamp $targetTimestamp but it does not.") + } + } else + maxTimestampChecked = math.max(maxTimestampChecked, message.timestamp) + } + + if (lastOffsetChecked >= 0) + Some(lastOffsetChecked + 1) + else + None + } + + /** + * Return the largest timestamp of the messages after a given position in this file message set. + * @param startingPosition The starting position. + * @return The largest timestamp of the messages after the given position. + */ + def largestTimestampAfter(startingPosition: Int): TimestampOffset = { + var maxTimestamp = Message.NoTimestamp + var offsetOfMaxTimestamp = -1L + val messagesToSearch = read(startingPosition, Int.MaxValue) + for (messageAndOffset <- messagesToSearch) { + if (messageAndOffset.message.timestamp > maxTimestamp) { + maxTimestamp = messageAndOffset.message.timestamp + offsetOfMaxTimestamp = messageAndOffset.offset + } + } + TimestampOffset(maxTimestamp, offsetOfMaxTimestamp) + } + /** * Write some of this set to the given channel. * @param destChannel The channel to write to. diff --git a/core/src/main/scala/kafka/log/OffsetPosition.scala b/core/src/main/scala/kafka/log/IndexEntry.scala similarity index 57% rename from core/src/main/scala/kafka/log/OffsetPosition.scala rename to core/src/main/scala/kafka/log/IndexEntry.scala index 24b6dcf0bb78..2f5a6a79693e 100644 --- a/core/src/main/scala/kafka/log/OffsetPosition.scala +++ b/core/src/main/scala/kafka/log/IndexEntry.scala @@ -17,9 +17,30 @@ package kafka.log +sealed trait IndexEntry { + // We always use Long for both key and value to avoid boxing. + def indexKey: Long + def indexValue: Long +} + /** * The mapping between a logical log offset and the physical position * in some log file of the beginning of the message set entry with the * given offset. */ -case class OffsetPosition(offset: Long, position: Int) +case class OffsetPosition(offset: Long, position: Int) extends IndexEntry { + override def indexKey = offset + override def indexValue = position.toLong +} + + +/** + * The mapping between a timestamp to a message offset. The entry means that any message whose timestamp is greater + * than that timestamp must be at or after that offset. + * @param timestamp The max timestamp before the given offset. + * @param offset The message offset. + */ +case class TimestampOffset(timestamp: Long, offset: Long) extends IndexEntry { + override def indexKey = timestamp + override def indexValue = offset +} diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 1a7719a67650..b4aa470fa28b 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -17,6 +17,7 @@ package kafka.log +import kafka.api.OffsetRequest import kafka.utils._ import kafka.message._ import kafka.common._ @@ -30,19 +31,22 @@ import java.text.NumberFormat import org.apache.kafka.common.errors.{CorruptRecordException, OffsetOutOfRangeException, RecordBatchTooLargeException, RecordTooLargeException} import org.apache.kafka.common.record.TimestampType -import scala.collection.JavaConversions +import scala.collection.{Seq, JavaConversions} import com.yammer.metrics.core.Gauge import org.apache.kafka.common.utils.Utils object LogAppendInfo { - val UnknownLogAppendInfo = LogAppendInfo(-1, -1, Message.NoTimestamp, NoCompressionCodec, NoCompressionCodec, -1, -1, offsetsMonotonic = false) + val UnknownLogAppendInfo = LogAppendInfo(-1, -1, Message.NoTimestamp, -1L, Message.NoTimestamp, NoCompressionCodec, NoCompressionCodec, -1, -1, offsetsMonotonic = false) } /** * Struct to hold various quantities we compute about each message set before appending to the log + * * @param firstOffset The first offset in the message set * @param lastOffset The last offset in the message set - * @param timestamp The log append time (if used) of the message set, otherwise Message.NoTimestamp + * @param maxTimestamp The maximum timestamp of the message set. + * @param offsetOfMaxTimestamp The offset of the message with the maximum timestamp. + * @param logAppendTime The log append time (if used) of the message set, otherwise Message.NoTimestamp * @param sourceCodec The source codec used in the message set (send by the producer) * @param targetCodec The target codec of the message set(after applying the broker compression configuration if any) * @param shallowCount The number of shallow messages @@ -51,7 +55,9 @@ object LogAppendInfo { */ case class LogAppendInfo(var firstOffset: Long, var lastOffset: Long, - var timestamp: Long, + var maxTimestamp: Long, + var offsetOfMaxTimestamp: Long, + var logAppendTime: Long, sourceCodec: CompressionCodec, targetCodec: CompressionCodec, shallowCount: Int, @@ -95,7 +101,7 @@ class Log(val dir: File, else 0 } - + val t = time.milliseconds /* the actual segments of the log */ private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment] loadSegments() @@ -105,7 +111,8 @@ class Log(val dir: File, val topicAndPartition: TopicAndPartition = Log.parseTopicPartitionName(dir) - info("Completed load of log %s with log end offset %d".format(name, logEndOffset)) + info("Completed load of log %s with %d log segments and log end offset %d in %d ms" + .format(name, segments.size(), logEndOffset, time.milliseconds - t)) val tags = Map("topic" -> topicAndPartition.topic, "partition" -> topicAndPartition.partition.toString) @@ -167,12 +174,17 @@ class Log(val dir: File, } } - // now do a second pass and load all the .log and .index files + // now do a second pass and load all the .log and all index files for(file <- dir.listFiles if file.isFile) { val filename = file.getName - if(filename.endsWith(IndexFileSuffix)) { + if(filename.endsWith(IndexFileSuffix) || filename.endsWith(TimeIndexFileSuffix)) { // if it is an index file, make sure it has a corresponding .log file - val logFile = new File(file.getAbsolutePath.replace(IndexFileSuffix, LogFileSuffix)) + val logFile = + if (filename.endsWith(TimeIndexFileSuffix)) + new File(file.getAbsolutePath.replace(TimeIndexFileSuffix, LogFileSuffix)) + else + new File(file.getAbsolutePath.replace(IndexFileSuffix, LogFileSuffix)) + if(!logFile.exists) { warn("Found an orphaned index file, %s, with no corresponding log file.".format(file.getAbsolutePath)) file.delete() @@ -181,6 +193,9 @@ class Log(val dir: File, // if its a log file, load the corresponding log segment val start = filename.substring(0, filename.length - LogFileSuffix.length).toLong val indexFile = Log.indexFilename(dir, start) + val timeIndexFile = Log.timeIndexFilename(dir, start) + + val indexFileExists = indexFile.exists() val segment = new LogSegment(dir = dir, startOffset = start, indexIntervalBytes = config.indexInterval, @@ -189,20 +204,23 @@ class Log(val dir: File, time = time, fileAlreadyExists = true) - if(indexFile.exists()) { + if (indexFileExists) { try { - segment.index.sanityCheck() + segment.index.sanityCheck() + segment.timeIndex.sanityCheck() } catch { case e: java.lang.IllegalArgumentException => - warn("Found a corrupted index file, %s, deleting and rebuilding index. Error Message: %s".format(indexFile.getAbsolutePath, e.getMessage)) + warn(s"Found a corrupted index file due to ${e.getMessage}}. deleting ${timeIndexFile.getAbsolutePath}, " + + s"${indexFile.getAbsolutePath} and rebuilding index...") indexFile.delete() + timeIndexFile.delete() segment.recover(config.maxMessageSize) } - } - else { + } else { error("Could not find index file corresponding to log file %s, rebuilding index...".format(segment.log.file.getAbsolutePath)) segment.recover(config.maxMessageSize) } + segments.put(start, segment) } } @@ -216,8 +234,11 @@ class Log(val dir: File, val startOffset = fileName.substring(0, fileName.length - LogFileSuffix.length).toLong val indexFile = new File(CoreUtils.replaceSuffix(logFile.getPath, LogFileSuffix, IndexFileSuffix) + SwapFileSuffix) val index = new OffsetIndex(indexFile, baseOffset = startOffset, maxIndexSize = config.maxIndexSize) + val timeIndexFile = new File(CoreUtils.replaceSuffix(logFile.getPath, LogFileSuffix, TimeIndexFileSuffix) + SwapFileSuffix) + val timeIndex = new TimeIndex(timeIndexFile, baseOffset = startOffset, maxIndexSize = config.maxIndexSize) val swapSegment = new LogSegment(new FileMessageSet(file = swapFile), index = index, + timeIndex = timeIndex, baseOffset = startOffset, indexIntervalBytes = config.indexInterval, rollJitterMs = config.randomSegmentJitter, @@ -243,6 +264,7 @@ class Log(val dir: File, recoverLog() // reset the index size of the currently active log segment to allow more entries activeSegment.index.resize(config.maxIndexSize) + activeSegment.timeIndex.resize(config.maxIndexSize) } } @@ -298,8 +320,7 @@ class Log(val dir: File, def close() { debug("Closing log " + name) lock synchronized { - for(seg <- logSegments) - seg.close() + logSegments.foreach(_.close()) } } @@ -311,9 +332,7 @@ class Log(val dir: File, * * @param messages The message set to append * @param assignOffsets Should the log assign offsets to this message set or blindly apply what it is given - * * @throws KafkaStorageException If the append fails due to an I/O error. - * * @return Information about the appended messages including the first and last offset. */ def append(messages: ByteBufferMessageSet, assignOffsets: Boolean = true): LogAppendInfo = { @@ -335,7 +354,7 @@ class Log(val dir: File, val offset = new LongRef(nextOffsetMetadata.messageOffset) appendInfo.firstOffset = offset.value val now = time.milliseconds - val (validatedMessages, messageSizesMaybeChanged) = try { + val validateAndOffsetAssignResult = try { validMessages.validateMessagesAndAssignOffsets(offset, now, appendInfo.sourceCodec, @@ -347,14 +366,16 @@ class Log(val dir: File, } catch { case e: IOException => throw new KafkaException("Error in validating messages while appending to log '%s'".format(name), e) } - validMessages = validatedMessages + validMessages = validateAndOffsetAssignResult.validatedMessages + appendInfo.maxTimestamp = validateAndOffsetAssignResult.maxTimestamp + appendInfo.offsetOfMaxTimestamp = validateAndOffsetAssignResult.offsetOfMaxTimestamp appendInfo.lastOffset = offset.value - 1 if (config.messageTimestampType == TimestampType.LOG_APPEND_TIME) - appendInfo.timestamp = now + appendInfo.logAppendTime = now // re-validate message sizes if there's a possibility that they have changed (due to re-compression or message // format conversion) - if (messageSizesMaybeChanged) { + if (validateAndOffsetAssignResult.messageSizeMaybeChanged) { for (messageAndOffset <- validMessages.shallowIterator) { if (MessageSet.entrySize(messageAndOffset.message) > config.maxMessageSize) { // we record the original message set size instead of the trimmed size @@ -383,7 +404,8 @@ class Log(val dir: File, val segment = maybeRoll(validMessages.sizeInBytes) // now append to the log - segment.append(appendInfo.firstOffset, validMessages) + segment.append(firstOffset = appendInfo.firstOffset, largestTimestamp = appendInfo.maxTimestamp, + offsetOfLargestTimestamp = appendInfo.offsetOfMaxTimestamp, messages = validMessages) // increment the log end offset updateLogEndOffset(appendInfo.lastOffset + 1) @@ -424,6 +446,8 @@ class Log(val dir: File, var firstOffset, lastOffset = -1L var sourceCodec: CompressionCodec = NoCompressionCodec var monotonic = true + var maxTimestamp = Message.NoTimestamp + var offsetOfMaxTimestamp = -1L for(messageAndOffset <- messages.shallowIterator) { // update the first offset if on the first message if(firstOffset < 0) @@ -447,7 +471,10 @@ class Log(val dir: File, // check the validity of the message by checking CRC m.ensureValid() - + if (m.timestamp > maxTimestamp) { + maxTimestamp = m.timestamp + offsetOfMaxTimestamp = lastOffset + } shallowMessageCount += 1 validBytesCount += messageSize @@ -459,11 +486,12 @@ class Log(val dir: File, // Apply broker-side compression if any val targetCodec = BrokerCompressionCodec.getTargetCompressionCodec(config.compressionType, sourceCodec) - LogAppendInfo(firstOffset, lastOffset, Message.NoTimestamp, sourceCodec, targetCodec, shallowMessageCount, validBytesCount, monotonic) + LogAppendInfo(firstOffset, lastOffset, maxTimestamp, offsetOfMaxTimestamp, Message.NoTimestamp, sourceCodec, targetCodec, shallowMessageCount, validBytesCount, monotonic) } /** * Trim any invalid bytes from the end of this message set (if there are any) + * * @param messages The message set to trim * @param info The general information of the message set * @return A trimmed message set. This may be the same as what was passed in or it may not. @@ -543,6 +571,71 @@ class Log(val dir: File, FetchDataInfo(nextOffsetMetadata, MessageSet.Empty) } + /** + * Get an offset based on the given timestamp + * The offset returned is the offset of the first message whose timestamp is greater than or equals to the + * given timestamp. + * + * If no such message is found, the log end offset is returned. + * + * `NOTE:` OffsetRequest V0 does not use this method, the behavior of OffsetRequest V0 remains the same as before + * , i.e. it only gives back the timestamp based on the last modification time of the log segments. + * + * @param timestamp The given timestamp for offset fetching. + * @return The offset of the first message whose timestamp is greater than or equals to the given timestamp. + */ + def fetchOffsetsByTimestamp(timestamp: Long): Long = { + debug(s"Searching offset for timestamp $timestamp") + val segsArray = logSegments.toArray + if (timestamp == OffsetRequest.EarliestTime) + return segsArray(0).baseOffset + + // set the target timestamp to be Long.MaxValue if we need to find from the latest. + val targetTimestamp = timestamp match { + case OffsetRequest.LatestTime => Long.MaxValue + case _ => timestamp + } + + var foundOffset: Long = -1L + // We have this while loop here to make sure we are returning the valid offsets to our best knowledge. + // This while loop is to handle the case where the log is truncated during the timestamp search and we did not + // find any message. In this case, we need to retry the search. + do { + val targetSeg = { + // Get all the segments whose largest timestamp is smaller than target timestamp + val earlierSegs = segsArray.takeWhile(_.largestTimestamp < targetTimestamp) + // We need to search the first segment whose largest timestamp is greater than the target timestamp if there is one. + if (earlierSegs.length < segsArray.length) + segsArray(earlierSegs.length) + else + earlierSegs.last + } + + // First cache the current log end offset + val leo = logEndOffset + foundOffset = { + // Use the cached log end offsets if + // 1. user is asking for latest messages, or, + // 2. we are searching on the active segment and the target timestamp is greater than the largestTimestamp + // after we cached the log end offset. (We have to use the cached log end offsets because it is possible that + // some messages with a larger timestamp are appended after we check the largest timestamp. Using log end offset + // after the timestamp check might skip those messages.) + if (targetTimestamp == Long.MaxValue + || (targetTimestamp > targetSeg.largestTimestamp && targetSeg == activeSegment)) + leo + else + // The findOffsetByTimestamp() method may return None when the log is truncated during the timestamp search. + // In that case we simply set the foundOffset to -1 so that we will search the timestamp again in the + // while loop. + targetSeg.findOffsetByTimestamp(targetTimestamp) match { + case Some(offset) => offset + case None => -1L + } + } + } while (foundOffset < 0) + foundOffset + } + /** * Given a message offset, find its corresponding offset metadata in the log. * If the message offset is out of range, return unknown offset metadata @@ -559,6 +652,7 @@ class Log(val dir: File, /** * Delete any log segments matching the given predicate function, * starting with the oldest segment and moving forward until a segment doesn't match. + * * @param predicate A function that takes in a single log segment and returns true iff it is deletable * @return The number of segments deleted */ @@ -609,24 +703,22 @@ class Log(val dir: File, * logSegment will be rolled if one of the following conditions met *
      *
    1. The logSegment is full - *
    2. The maxTime has elapsed + *
    3. The maxTime has elapsed since the timestamp of first message in the segment (or since the create time if + * the first message does not have a timestamp) *
    4. The index is full *
    * @return The currently active segment after (perhaps) rolling to a new segment */ private def maybeRoll(messagesSize: Int): LogSegment = { val segment = activeSegment + val reachedRollMs = segment.timeWaitedForRoll(time.milliseconds) > config.segmentMs - segment.rollJitterMs if (segment.size > config.segmentSize - messagesSize || - segment.size > 0 && time.milliseconds - segment.created > config.segmentMs - segment.rollJitterMs || - segment.index.isFull) { - debug("Rolling new log segment in %s (log_size = %d/%d, index_size = %d/%d, age_ms = %d/%d)." - .format(name, - segment.size, - config.segmentSize, - segment.index.entries, - segment.index.maxEntries, - time.milliseconds - segment.created, - config.segmentMs - segment.rollJitterMs)) + (segment.size > 0 && reachedRollMs) || + segment.index.isFull || segment.timeIndex.isFull) { + debug(s"Rolling new log segment in $name (log_size = ${segment.size}/${config.segmentSize}}, " + + s"index_size = ${segment.index.entries}/${segment.index.maxEntries}, " + + s"time_index_size = ${segment.timeIndex.entries}/${segment.timeIndex.maxEntries}, " + + s"inactive_time_ms = ${segment.timeWaitedForRoll(time.milliseconds)}/${config.segmentMs - segment.rollJitterMs}).") roll() } else { segment @@ -636,6 +728,7 @@ class Log(val dir: File, /** * Roll the log over to a new active segment starting with the current logEndOffset. * This will trim the index to the exact size of the number of entries it currently contains. + * * @return The newly rolled segment */ def roll(): LogSegment = { @@ -644,7 +737,8 @@ class Log(val dir: File, val newOffset = logEndOffset val logFile = logFilename(dir, newOffset) val indexFile = indexFilename(dir, newOffset) - for(file <- List(logFile, indexFile); if file.exists) { + val timeIndexFile = timeIndexFilename(dir, newOffset) + for(file <- List(logFile, indexFile, timeIndexFile); if file.exists) { warn("Newly rolled segment file " + file.getName + " already exists; deleting it first") file.delete() } @@ -652,8 +746,11 @@ class Log(val dir: File, segments.lastEntry() match { case null => case entry => { - entry.getValue.index.trimToValidSize() - entry.getValue.log.trim() + val seg = entry.getValue + seg.onBecomeInactiveSegment() + seg.index.trimToValidSize() + seg.timeIndex.trimToValidSize() + seg.log.trim() } } val segment = new LogSegment(dir, @@ -692,6 +789,7 @@ class Log(val dir: File, /** * Flush log segments for all offsets up to offset-1 + * * @param offset The offset to flush up to (non-inclusive); the new recovery point */ def flush(offset: Long) : Unit = { @@ -723,6 +821,7 @@ class Log(val dir: File, /** * Truncate this log so that it ends with the greatest offset < targetOffset. + * * @param targetOffset The offset to truncate to, an upper bound on all offsets in the log after truncation is complete. */ private[log] def truncateTo(targetOffset: Long) { @@ -748,6 +847,7 @@ class Log(val dir: File, /** * Delete all data in the log and start at the new offset + * * @param newOffset The new offset to start the log with */ private[log] def truncateFullyAndStartAt(newOffset: Long) { @@ -826,6 +926,7 @@ class Log(val dir: File, /** * Perform an asynchronous delete on the given file if it exists (otherwise do nothing) + * * @throws KafkaStorageException if the file can't be renamed and still exists */ private def asyncDeleteSegment(segment: LogSegment) { @@ -893,6 +994,7 @@ class Log(val dir: File, } /** * Add the given segment to the segments in this log. If this segment replaces an existing segment, delete it. + * * @param segment The segment to add */ def addSegment(segment: LogSegment) = this.segments.put(segment.baseOffset, segment) @@ -910,6 +1012,9 @@ object Log { /** an index file */ val IndexFileSuffix = ".index" + /** a time index file */ + val TimeIndexFileSuffix = ".timeindex" + /** a file that is scheduled to be deleted */ val DeletedFileSuffix = ".deleted" @@ -920,13 +1025,14 @@ object Log { val SwapFileSuffix = ".swap" /** Clean shutdown file that indicates the broker was cleanly shutdown in 0.8. This is required to maintain backwards compatibility - * with 0.8 and avoid unnecessary log recovery when upgrading from 0.8 to 0.8.1 */ + * with 0.8 and avoid unnecessary log recovery when upgrading from 0.8 to 0.8.1 */ /** TODO: Get rid of CleanShutdownFile in 0.8.2 */ val CleanShutdownFile = ".kafka_cleanshutdown" /** * Make log segment file name from offset bytes. All this does is pad out the offset number with zeros * so that ls sorts the files numerically. + * * @param offset The offset to use in the file name * @return The filename */ @@ -940,6 +1046,7 @@ object Log { /** * Construct a log file name in the given dir with the given base offset + * * @param dir The directory in which the log will reside * @param offset The base offset of the log file */ @@ -948,12 +1055,21 @@ object Log { /** * Construct an index file name in the given dir using the given base offset + * * @param dir The directory in which the log will reside * @param offset The base offset of the log file */ def indexFilename(dir: File, offset: Long) = new File(dir, filenamePrefixFromOffset(offset) + IndexFileSuffix) + /** + * Construct a time index file name in the given dir using the given base offset + * + * @param dir The directory in which the log will reside + * @param offset The base offset of the log file + */ + def timeIndexFilename(dir: File, offset: Long) = + new File(dir, filenamePrefixFromOffset(offset) + TimeIndexFileSuffix) /** * Parse the topic and partition out of the directory name of a log diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index 25c36e72ff51..d4bb1f2c6eff 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -334,7 +334,7 @@ private[log] class Cleaner(val id: Int, val deleteHorizonMs = log.logSegments(0, cleanable.firstDirtyOffset).lastOption match { case None => 0L - case Some(seg) => seg.lastModified - log.config.deleteRetentionMs + case Some(seg) => seg.largestTimestamp - log.config.deleteRetentionMs } // group the segments and clean the groups @@ -366,23 +366,32 @@ private[log] class Cleaner(val id: Int, val logFile = new File(segments.head.log.file.getPath + Log.CleanedFileSuffix) logFile.delete() val indexFile = new File(segments.head.index.file.getPath + Log.CleanedFileSuffix) + val timeIndexFile = new File(segments.head.timeIndex.file.getPath + Log.CleanedFileSuffix) indexFile.delete() + timeIndexFile.delete() 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) + val timeIndex = new TimeIndex(timeIndexFile, segments.head.baseOffset, segments.head.timeIndex.maxIndexSize) + val cleaned = new LogSegment(messages, index, timeIndex, segments.head.baseOffset, segments.head.indexIntervalBytes, log.config.randomSegmentJitter, time) try { // clean segments into the new destination segment for (old <- segments) { - val retainDeletes = old.lastModified > deleteHorizonMs - info("Cleaning segment %s in log %s (last modified %s) into %s, %s deletes." - .format(old.baseOffset, log.name, new Date(old.lastModified), cleaned.baseOffset, if(retainDeletes) "retaining" else "discarding")) + val retainDeletes = old.largestTimestamp > deleteHorizonMs + info("Cleaning segment %s in log %s (largest timestamp %s) into %s, %s deletes." + .format(old.baseOffset, log.name, new Date(old.largestTimestamp), cleaned.baseOffset, if(retainDeletes) "retaining" else "discarding")) cleanInto(log.topicAndPartition, old, cleaned, map, retainDeletes, log.config.messageFormatVersion.messageFormatVersion) } // trim excess index index.trimToValidSize() + // Append the last index entry + cleaned.onBecomeInactiveSegment() + + // trim time index + timeIndex.trimToValidSize() + // flush new segment to disk before swap cleaned.flush() @@ -422,6 +431,8 @@ private[log] class Cleaner(val id: Int, // read a chunk of messages and copy any that are to be retained to the write buffer to be written out readBuffer.clear() writeBuffer.clear() + var maxTimestamp = Message.NoTimestamp + var offsetOfMaxTimestamp = -1L val messages = new ByteBufferMessageSet(source.log.readInto(readBuffer, position)) throttler.maybeThrottle(messages.sizeInBytes) // check each message to see if it is to be retained @@ -433,6 +444,10 @@ private[log] class Cleaner(val id: Int, if (shouldRetainMessage(source, map, retainDeletes, entry)) { ByteBufferMessageSet.writeMessage(writeBuffer, entry.message, entry.offset) stats.recopyMessage(size) + if (entry.message.timestamp > maxTimestamp) { + maxTimestamp = entry.message.timestamp + offsetOfMaxTimestamp = entry.offset + } } messagesRead += 1 } else { @@ -443,12 +458,16 @@ private[log] class Cleaner(val id: Int, val retainedMessages = new mutable.ArrayBuffer[MessageAndOffset] messages.foreach { messageAndOffset => messagesRead += 1 - if (shouldRetainMessage(source, map, retainDeletes, messageAndOffset)) + if (shouldRetainMessage(source, map, retainDeletes, messageAndOffset)) { retainedMessages += messageAndOffset + // We need the max timestamp and last offset for time index + if (messageAndOffset.message.timestamp > maxTimestamp) + maxTimestamp = messageAndOffset.message.timestamp + } else writeOriginalMessageSet = false } - - // There are no messages compacted out, write the original message set back + offsetOfMaxTimestamp = if (retainedMessages.nonEmpty) retainedMessages.last.offset else -1L + // There are no messages compacted out and no message format conversion, write the original message set back if (writeOriginalMessageSet) ByteBufferMessageSet.writeMessage(writeBuffer, entry.message, entry.offset) else @@ -461,7 +480,8 @@ private[log] class Cleaner(val id: Int, if (writeBuffer.position > 0) { writeBuffer.flip() val retained = new ByteBufferMessageSet(writeBuffer) - dest.append(retained.head.offset, retained) + dest.append(firstOffset = retained.head.offset, largestTimestamp = maxTimestamp, + offsetOfLargestTimestamp = offsetOfMaxTimestamp, messages = retained) throttler.maybeThrottle(writeBuffer.limit) } @@ -569,14 +589,17 @@ private[log] class Cleaner(val id: Int, var group = List(segs.head) var logSize = segs.head.size var indexSize = segs.head.index.sizeInBytes + var timeIndexSize = segs.head.timeIndex.sizeInBytes segs = segs.tail while(segs.nonEmpty && logSize + segs.head.size <= maxSize && indexSize + segs.head.index.sizeInBytes <= maxIndexSize && + timeIndexSize + segs.head.timeIndex.sizeInBytes <= maxIndexSize && segs.head.index.lastOffset - group.last.index.baseOffset <= Int.MaxValue) { group = segs.head :: group logSize += segs.head.size indexSize += segs.head.index.sizeInBytes + timeIndexSize += segs.head.timeIndex.sizeInBytes segs = segs.tail } grouped ::= group.reverse diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index 4357ef4c5d36..e6c60b9bf96c 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -108,7 +108,7 @@ class LogManager(val logDirs: Array[File], */ private def loadLogs(): Unit = { info("Loading logs.") - + val startMs = time.milliseconds val threadPools = mutable.ArrayBuffer.empty[ExecutorService] val jobs = mutable.Map.empty[File, Seq[Future[_]]] @@ -177,7 +177,7 @@ class LogManager(val logDirs: Array[File], threadPools.foreach(_.shutdown()) } - info("Logs loading complete.") + info(s"Logs loading complete in ${time.milliseconds - startMs} ms.") } /** @@ -423,7 +423,7 @@ class LogManager(val logDirs: Array[File], if (log.config.retentionMs < 0) return 0 val startMs = time.milliseconds - log.deleteOldSegments(startMs - _.lastModified > log.config.retentionMs) + log.deleteOldSegments(startMs - _.largestTimestamp > log.config.retentionMs) } /** diff --git a/core/src/main/scala/kafka/log/LogSegment.scala b/core/src/main/scala/kafka/log/LogSegment.scala index 6bbc50c7c3c0..d894020d5013 100755 --- a/core/src/main/scala/kafka/log/LogSegment.scala +++ b/core/src/main/scala/kafka/log/LogSegment.scala @@ -36,6 +36,7 @@ import java.io.{IOException, File} * * @param log The message set containing log entries * @param index The offset index + * @param timeIndex The timestamp index * @param baseOffset A lower bound on the offsets in this segment * @param indexIntervalBytes The approximate number of bytes between entries in the index * @param time The time instance @@ -43,6 +44,7 @@ import java.io.{IOException, File} @nonthreadsafe class LogSegment(val log: FileMessageSet, val index: OffsetIndex, + val timeIndex: TimeIndex, val baseOffset: Long, val indexIntervalBytes: Int, val rollJitterMs: Long, @@ -53,9 +55,17 @@ class LogSegment(val log: FileMessageSet, /* the number of bytes since we last added an entry in the offset index */ private var bytesSinceLastIndexEntry = 0 + /* The timestamp we used for time based log rolling */ + private var rollingBasedTimestamp: Option[Long] = None + + /* The maximum timestamp we see so far */ + @volatile private var maxTimestampSoFar = timeIndex.lastEntry.timestamp + @volatile private var offsetOfMaxTimestamp = timeIndex.lastEntry.offset + 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(Log.indexFilename(dir, startOffset), baseOffset = startOffset, maxIndexSize = maxIndexSize), + new TimeIndex(Log.timeIndexFilename(dir, startOffset), baseOffset = startOffset, maxIndexSize = maxIndexSize), startOffset, indexIntervalBytes, rollJitterMs, @@ -70,21 +80,33 @@ class LogSegment(val log: FileMessageSet, * * It is assumed this method is being called from within a lock. * - * @param offset The first offset in the message set. + * @param firstOffset The first offset in the message set. + * @param largestTimestamp The largest timestamp in the message set. + * @param offsetOfLargestTimestamp The offset of the message that has the largest timestamp in the messages to append. * @param messages The messages to append. */ @nonthreadsafe - def append(offset: Long, messages: ByteBufferMessageSet) { + def append(firstOffset: Long, largestTimestamp: Long, offsetOfLargestTimestamp: Long, messages: ByteBufferMessageSet) { if (messages.sizeInBytes > 0) { - trace("Inserting %d bytes at offset %d at position %d".format(messages.sizeInBytes, offset, log.sizeInBytes())) + trace("Inserting %d bytes at offset %d at position %d with largest timestamp %d at offset %d" + .format(messages.sizeInBytes, firstOffset, log.sizeInBytes(), largestTimestamp, offsetOfLargestTimestamp)) + val physicalPosition = log.sizeInBytes() + if (physicalPosition == 0) + rollingBasedTimestamp = Some(largestTimestamp) + // append the messages + log.append(messages) + // Update the in memory max timestamp and corresponding offset. + if (largestTimestamp > maxTimestampSoFar) { + maxTimestampSoFar = largestTimestamp + offsetOfMaxTimestamp = offsetOfLargestTimestamp + } // append an entry to the index (if needed) if(bytesSinceLastIndexEntry > indexIntervalBytes) { - index.append(offset, log.sizeInBytes()) - this.bytesSinceLastIndexEntry = 0 + index.append(firstOffset, physicalPosition) + timeIndex.maybeAppend(maxTimestampSoFar, offsetOfMaxTimestamp) + bytesSinceLastIndexEntry = 0 } - // append the messages - log.append(messages) - this.bytesSinceLastIndexEntry += messages.sizeInBytes + bytesSinceLastIndexEntry += messages.sizeInBytes } } @@ -97,13 +119,12 @@ class LogSegment(val log: FileMessageSet, * @param offset The offset we want to translate * @param startingFilePosition A lower bound on the file position from which to begin the search. This is purely an optimization and * when omitted, the search will begin at the position in the offset index. - * * @return The position in the log storing the message with the least offset >= the requested offset or null if no message meets this criteria. */ @threadsafe private[log] def translateOffset(offset: Long, startingFilePosition: Int = 0): OffsetPosition = { val mapping = index.lookup(offset) - log.searchFor(offset, max(mapping.position, startingFilePosition)) + log.searchForOffset(offset, max(mapping.position, startingFilePosition)) } /** @@ -165,30 +186,34 @@ class LogSegment(val log: FileMessageSet, * * @param maxMessageSize A bound the memory allocation in the case of a corrupt message size--we will assume any message larger than this * is corrupt. - * * @return The number of bytes truncated from the log */ @nonthreadsafe def recover(maxMessageSize: Int): Int = { index.truncate() index.resize(index.maxIndexSize) + timeIndex.truncate() + timeIndex.resize(timeIndex.maxIndexSize) var validBytes = 0 var lastIndexEntry = 0 val iter = log.iterator(maxMessageSize) + maxTimestampSoFar = Message.NoTimestamp try { while(iter.hasNext) { val entry = iter.next entry.message.ensureValid() + + // The max timestamp should have been put in the outer message, so we don't need to iterate over the inner messages. + if (entry.message.timestamp > maxTimestampSoFar) { + maxTimestampSoFar = entry.message.timestamp + offsetOfMaxTimestamp = entry.offset + } + + // Build offset index if(validBytes - lastIndexEntry > indexIntervalBytes) { - // we need to decompress the message, if required, to get the offset of the first uncompressed message - val startOffset = - entry.message.compressionCodec match { - case NoCompressionCodec => - entry.offset - case _ => - ByteBufferMessageSet.deepIterator(entry).next().offset - } + val startOffset = entry.firstOffset index.append(startOffset, validBytes) + timeIndex.maybeAppend(maxTimestampSoFar, offsetOfMaxTimestamp) lastIndexEntry = validBytes } validBytes += MessageSet.entrySize(entry.message) @@ -200,14 +225,35 @@ class LogSegment(val log: FileMessageSet, val truncated = log.sizeInBytes - validBytes log.truncateTo(validBytes) index.trimToValidSize() + // A normally closed segment always appends the biggest timestamp ever seen into log segment, we do this as well. + timeIndex.maybeAppend(maxTimestampSoFar, offsetOfMaxTimestamp, skipFullCheck = true) + timeIndex.trimToValidSize() truncated } + def loadLargestTimestamp(readToLogEnd: Boolean = false) { + // Get the last time index entry. If the time index is empty, it will return (-1, baseOffset) + val lastTimeIndexEntry = timeIndex.lastEntry + maxTimestampSoFar = lastTimeIndexEntry.timestamp + offsetOfMaxTimestamp = lastTimeIndexEntry.offset + if (readToLogEnd) { + val offsetPosition = index.lookup(lastTimeIndexEntry.offset) + // Scan the rest of the messages to see if there is a larger timestamp after the last time index entry. + val maxTimestampOffsetAfterLastEntry = log.largestTimestampAfter(offsetPosition.position) + if (maxTimestampOffsetAfterLastEntry.timestamp > lastTimeIndexEntry.timestamp) { + maxTimestampSoFar = maxTimestampOffsetAfterLastEntry.timestamp + offsetOfMaxTimestamp = maxTimestampOffsetAfterLastEntry.offset + } + } + } + + override def toString = "LogSegment(baseOffset=" + baseOffset + ", size=" + size + ")" /** * Truncate off all index and log entries with offsets >= the given offset. * If the given offset is larger than the largest message in this segment, do nothing. + * * @param offset The offset to truncate to * @return The number of log bytes truncated */ @@ -217,12 +263,19 @@ class LogSegment(val log: FileMessageSet, if(mapping == null) return 0 index.truncateTo(offset) + timeIndex.truncateTo(offset) // after truncation, reset and allocate more space for the (new currently active) index index.resize(index.maxIndexSize) + timeIndex.resize(timeIndex.maxIndexSize) val bytesTruncated = log.truncateTo(mapping.position) - if(log.sizeInBytes == 0) + if(log.sizeInBytes == 0) { created = time.milliseconds + rollingBasedTimestamp = None + } bytesSinceLastIndexEntry = 0 + // We may need to reload the max timestamp after truncation. + if (maxTimestampSoFar >= 0) + loadLargestTimestamp(readToLogEnd = true) bytesTruncated } @@ -251,6 +304,7 @@ class LogSegment(val log: FileMessageSet, LogFlushStats.logFlushTimer.time { log.flush() index.flush() + timeIndex.flush() } } @@ -270,27 +324,96 @@ class LogSegment(val log: FileMessageSet, catch { case e: IOException => throw kafkaStorageException("index", e) } + try timeIndex.renameTo(new File(CoreUtils.replaceSuffix(timeIndex.file.getPath, oldSuffix, newSuffix))) + catch { + case e: IOException => throw kafkaStorageException("timeindex", e) + } + } + + /** + * Append the largest time index entry to the time index when this log segment become inactive segment. + * This entry will be used to decide when to delete the segment. + */ + def onBecomeInactiveSegment() { + timeIndex.maybeAppend(maxTimestampSoFar, offsetOfMaxTimestamp, skipFullCheck = true) + } + + /** + * The time this segment has waited to be rolled. If the first message in the segment does not have a timestamp, + * the time is based on the create time of the segment. Otherwise the time is based on the timestamp of that message. + */ + def timeWaitedForRoll(now: Long) : Long= { + // Load the timestamp of the first message into memory + if (!rollingBasedTimestamp.isDefined) { + val iter = log.iterator + if (iter.hasNext) + rollingBasedTimestamp = Some(iter.next.message.timestamp) + else + // If the log is empty, we return 0 as time waited. + return now - created + } + now - {if (rollingBasedTimestamp.get >= 0) rollingBasedTimestamp.get else created} + } + + /** + * Search the message offset based on timestamp. + * This method returns an option of TimestampOffset. The offset is the offset of the first message whose timestamp is + * greater than or equals to the target timestamp. + * + * If all the message in the segment have smaller timestamps, the returned offset will be last offset + 1 and the + * timestamp will be max timestamp in the segment. + * + * If all the messages in the segment have larger timestamps, or no message in the segment has a timestamp, + * the returned the offset will be the base offset of the segment and the timestamp will be Message.NoTimestamp. + * + * This methods only returns None when the log is not empty but we did not see any messages when scanning the log + * from the indexed position. This could happen if the log is truncated after we get the indexed position but + * before we scan the log from there. In this case we simply return None and the caller will need to check on + * the truncated log and maybe retry or even do the search on another log segment. + * + * @param timestamp The timestamp to search for. + * @return an offset which points to the first message whose timestamp is larger than or equals to the + * target timestamp. + * None maybe returned when the log is truncated. + */ + def findOffsetByTimestamp(timestamp: Long): Option[Long] = { + if (log.end == log.start) { + // The log segment is empty, just return base offset with no timestamp. + Some(baseOffset) + } else { + // Get the index entry with a timestamp less than or equal to the target timestamp + val timestampOffset = timeIndex.lookup(timestamp) + val position = index.lookup(timestampOffset.offset).position + // Search the timestamp + log.searchForTimestamp(timestamp, position) + } } /** * Close this log segment */ def close() { + timeIndex.maybeAppend(maxTimestampSoFar, offsetOfMaxTimestamp, skipFullCheck = true) CoreUtils.swallow(index.close) + CoreUtils.swallow(timeIndex.close()) CoreUtils.swallow(log.close) } /** * Delete this log segment from the filesystem. + * * @throws KafkaStorageException if the delete fails. */ def delete() { val deletedLog = log.delete() val deletedIndex = index.delete() + val deletedTimeIndex = timeIndex.delete() if(!deletedLog && log.file.exists) throw new KafkaStorageException("Delete of log " + log.file.getName + " failed.") if(!deletedIndex && index.file.exists) throw new KafkaStorageException("Delete of index " + index.file.getName + " failed.") + if(!deletedTimeIndex && timeIndex.file.exists) + throw new KafkaStorageException("Delete of time index " + timeIndex.file.getName + " failed.") } /** @@ -298,11 +421,17 @@ class LogSegment(val log: FileMessageSet, */ def lastModified = log.file.lastModified + /** + * The largest timestamp this segment contains. + */ + def largestTimestamp = if (maxTimestampSoFar >= 0) maxTimestampSoFar else lastModified + /** * Change the last modified time for this log segment */ def lastModified_=(ms: Long) = { log.file.setLastModified(ms) index.file.setLastModified(ms) + timeIndex.file.setLastModified(ms) } } diff --git a/core/src/main/scala/kafka/log/OffsetIndex.scala b/core/src/main/scala/kafka/log/OffsetIndex.scala index 848fe3b53e7c..ad1b1969c234 100755 --- a/core/src/main/scala/kafka/log/OffsetIndex.scala +++ b/core/src/main/scala/kafka/log/OffsetIndex.scala @@ -17,18 +17,11 @@ package kafka.log -import org.apache.kafka.common.utils.Utils +import java.io.File +import java.nio.ByteBuffer -import scala.math._ -import java.io._ -import java.nio._ -import java.nio.channels._ -import java.util.concurrent.locks._ - -import kafka.utils._ import kafka.utils.CoreUtils.inLock import kafka.common.InvalidOffsetException -import sun.nio.ch.DirectBuffer /** * An index that maps offsets to physical file locations for a particular log segment. This index may be sparse: @@ -55,137 +48,58 @@ import sun.nio.ch.DirectBuffer * All external APIs translate from relative offsets to full offsets, so users of this class do not interact with the internal * storage format. */ -class OffsetIndex(@volatile private[this] var _file: File, val baseOffset: Long, val maxIndexSize: Int = -1) extends Logging { - - private val lock = new ReentrantLock - - /* initialize the memory mapping for this index */ - @volatile - private[this] var mmap: MappedByteBuffer = { - val newlyCreated = _file.createNewFile() - val raf = new RandomAccessFile(_file, "rw") - try { - /* pre-allocate the file if necessary */ - if (newlyCreated) { - if (maxIndexSize < 8) - throw new IllegalArgumentException("Invalid max index size: " + maxIndexSize) - raf.setLength(roundToExactMultiple(maxIndexSize, 8)) - } - - /* memory-map the file */ - val len = raf.length() - val idx = raf.getChannel.map(FileChannel.MapMode.READ_WRITE, 0, len) - - /* set the position in the index for the next entry */ - if (newlyCreated) - idx.position(0) - else - // if this is a pre-existing index, assume it is all valid and set position to last entry - idx.position(roundToExactMultiple(idx.limit, 8)) - idx - } finally { - CoreUtils.swallow(raf.close()) - } - } +class OffsetIndex(file: File, baseOffset: Long, maxIndexSize: Int = -1) + extends AbstractIndex[Long, Int](file, baseOffset, maxIndexSize) { - /* the number of eight-byte entries currently in the index */ - @volatile - private[this] var _entries = mmap.position / 8 - - /* The maximum number of eight-byte entries this index can hold */ - @volatile - private[this] var _maxEntries = mmap.limit / 8 - - @volatile - private[this] var _lastOffset = readLastEntry.offset + override def entrySize = 8 + + /* the last offset in the index */ + private[this] var _lastOffset = lastEntry.offset debug("Loaded index file %s with maxEntries = %d, maxIndexSize = %d, entries = %d, lastOffset = %d, file position = %d" - .format(_file.getAbsolutePath, _maxEntries, maxIndexSize, _entries, _lastOffset, mmap.position)) - - /** The maximum number of entries this index can hold */ - def maxEntries: Int = _maxEntries - - /** The last offset in the index */ - def lastOffset: Long = _lastOffset - - /** The index file */ - def file: File = _file + .format(file.getAbsolutePath, maxEntries, maxIndexSize, _entries, _lastOffset, mmap.position)) /** * The last entry in the index */ - def readLastEntry(): OffsetPosition = { + private def lastEntry: OffsetPosition = { inLock(lock) { _entries match { case 0 => OffsetPosition(baseOffset, 0) - case s => OffsetPosition(baseOffset + relativeOffset(mmap, s - 1), physical(mmap, s - 1)) + case s => parseEntry(mmap, s - 1).asInstanceOf[OffsetPosition] } } } + def lastOffset: Long = _lastOffset + /** * Find the largest offset less than or equal to the given targetOffset * and return a pair holding this offset and its corresponding physical file position. * * @param targetOffset The offset to look up. - * - * @return The offset found and the corresponding file position for this offset. - * If the target offset is smaller than the least entry in the index (or the index is empty), - * the pair (baseOffset, 0) is returned. + * @return The offset found and the corresponding file position for this offset + * If the target offset is smaller than the least entry in the index (or the index is empty), + * the pair (baseOffset, 0) is returned. */ def lookup(targetOffset: Long): OffsetPosition = { maybeLock(lock) { val idx = mmap.duplicate - val slot = indexSlotFor(idx, targetOffset) + val slot = indexSlotFor(idx, targetOffset, IndexSearchType.KEY) if(slot == -1) OffsetPosition(baseOffset, 0) else - OffsetPosition(baseOffset + relativeOffset(idx, slot), physical(idx, slot)) - } - } - - /** - * Find the slot in which the largest offset less than or equal to the given - * target offset is stored. - * - * @param idx The index buffer - * @param targetOffset The offset to look for - * - * @return The slot found or -1 if the least entry in the index is larger than the target offset or the index is empty - */ - private def indexSlotFor(idx: ByteBuffer, targetOffset: Long): Int = { - // we only store the difference from the base offset so calculate that - val relOffset = targetOffset - baseOffset - - // check if the index is empty - if (_entries == 0) - return -1 - - // check if the target offset is smaller than the least offset - if (relativeOffset(idx, 0) > relOffset) - return -1 - - // binary search for the entry - var lo = 0 - var hi = _entries - 1 - while (lo < hi) { - val mid = ceil(hi/2.0 + lo/2.0).toInt - val found = relativeOffset(idx, mid) - if (found == relOffset) - return mid - else if (found < relOffset) - lo = mid - else - hi = mid - 1 + parseEntry(idx, slot).asInstanceOf[OffsetPosition] } - lo } - - /* return the nth offset relative to the base offset */ - private def relativeOffset(buffer: ByteBuffer, n: Int): Int = buffer.getInt(n * 8) - - /* return the nth physical position */ - private def physical(buffer: ByteBuffer, n: Int): Int = buffer.getInt(n * 8 + 4) + + private def relativeOffset(buffer: ByteBuffer, n: Int): Int = buffer.getInt(n * entrySize) + + private def physical(buffer: ByteBuffer, n: Int): Int = buffer.getInt(n * entrySize + 4) + + override def parseEntry(buffer: ByteBuffer, n: Int): IndexEntry = { + OffsetPosition(baseOffset + relativeOffset(buffer, n), physical(buffer, n)) + } /** * Get the nth offset mapping from the index @@ -208,37 +122,25 @@ class OffsetIndex(@volatile private[this] var _file: File, val baseOffset: Long, inLock(lock) { require(!isFull, "Attempt to append to a full index (size = " + _entries + ").") if (_entries == 0 || offset > _lastOffset) { - debug("Adding index entry %d => %d to %s.".format(offset, position, _file.getName)) + debug("Adding index entry %d => %d to %s.".format(offset, position, file.getName)) mmap.putInt((offset - baseOffset).toInt) mmap.putInt(position) _entries += 1 _lastOffset = offset - require(_entries * 8 == mmap.position, _entries + " entries but file position in index is " + mmap.position + ".") + require(_entries * entrySize == mmap.position, entries + " entries but file position in index is " + mmap.position + ".") } else { throw new InvalidOffsetException("Attempt to append an offset (%d) to position %d no larger than the last offset appended (%d) to %s." - .format(offset, _entries, _lastOffset, _file.getAbsolutePath)) + .format(offset, entries, _lastOffset, file.getAbsolutePath)) } } } - - /** - * True iff there are no more slots available in this index - */ - def isFull: Boolean = _entries >= _maxEntries - - /** - * Truncate the entire index, deleting all entries - */ - def truncate() = truncateToEntries(0) - - /** - * Remove all entries from the index which have an offset greater than or equal to the given offset. - * Truncating to an offset larger than the largest in the index has no effect. - */ - def truncateTo(offset: Long) { + + override def truncate() = truncateToEntries(0) + + override def truncateTo(offset: Long) { inLock(lock) { val idx = mmap.duplicate - val slot = indexSlotFor(idx, offset) + val slot = indexSlotFor(idx, offset, IndexSearchType.KEY) /* There are 3 cases for choosing the new size * 1) if there is no entry in the index <= the offset, delete everything @@ -262,139 +164,19 @@ class OffsetIndex(@volatile private[this] var _file: File, val baseOffset: Long, private def truncateToEntries(entries: Int) { inLock(lock) { _entries = entries - mmap.position(_entries * 8) - _lastOffset = readLastEntry.offset - } - } - - /** - * Trim this segment to fit just the valid entries, deleting all trailing unwritten bytes from - * the file. - */ - def trimToValidSize() { - inLock(lock) { - resize(_entries * 8) + mmap.position(_entries * entrySize) + _lastOffset = lastEntry.offset } } - /** - * Reset the size of the memory map and the underneath file. This is used in two kinds of cases: (1) in - * trimToValidSize() which is called at closing the segment or new segment being rolled; (2) at - * loading segments from disk or truncating back to an old segment where a new log segment became active; - * we want to reset the index size to maximum index size to avoid rolling new segment. - */ - def resize(newSize: Int) { - inLock(lock) { - val raf = new RandomAccessFile(_file, "rw") - val roundedNewSize = roundToExactMultiple(newSize, 8) - val position = mmap.position - - /* Windows won't let us modify the file length while the file is mmapped :-( */ - if (Os.isWindows) - forceUnmap(mmap) - try { - raf.setLength(roundedNewSize) - mmap = raf.getChannel().map(FileChannel.MapMode.READ_WRITE, 0, roundedNewSize) - _maxEntries = mmap.limit / 8 - mmap.position(position) - } finally { - CoreUtils.swallow(raf.close()) - } - } - } - - /** - * Forcefully free the buffer's mmap. We do this only on windows. - */ - private def forceUnmap(m: MappedByteBuffer) { - try { - m match { - case buffer: DirectBuffer => - val bufferCleaner = buffer.cleaner() - /* cleaner can be null if the mapped region has size 0 */ - if (bufferCleaner != null) - bufferCleaner.clean() - case _ => - } - } catch { - case t: Throwable => warn("Error when freeing index buffer", t) - } - } - - /** - * Flush the data in the index to disk - */ - def flush() { - inLock(lock) { - mmap.force() - } - } - - /** - * Delete this index file - */ - def delete(): Boolean = { - info("Deleting index " + _file.getAbsolutePath) - if (Os.isWindows) - CoreUtils.swallow(forceUnmap(mmap)) - _file.delete() - } - - /** The number of entries in this index */ - def entries = _entries - - /** - * The number of bytes actually used by this index - */ - def sizeInBytes() = 8 * _entries - - /** Close the index */ - def close() { - trimToValidSize() - } - - /** - * Rename the file that backs this offset index - * @throws IOException if rename fails - */ - def renameTo(f: File) { - try Utils.atomicMoveWithFallback(_file.toPath, f.toPath) - finally _file = f - } - - /** - * Do a basic sanity check on this index to detect obvious problems - * @throws IllegalArgumentException if any problems are found - */ - def sanityCheck() { - require(_entries == 0 || lastOffset > baseOffset, - "Corrupt index found, index file (%s) has non-zero size but the last offset is %d and the base offset is %d" - .format(_file.getAbsolutePath, lastOffset, baseOffset)) - val len = _file.length() - require(len % 8 == 0, - "Index file " + _file.getName + " is corrupt, found " + len + + override def sanityCheck() { + require(_entries == 0 || _lastOffset > baseOffset, + s"Corrupt index found, index file (${file.getAbsolutePath}) has non-zero size but the last offset " + + s"is ${_lastOffset} which is no larger than the base offset $baseOffset.") + val len = file.length() + require(len % entrySize == 0, + "Index file " + file.getAbsolutePath + " is corrupt, found " + len + " bytes which is not positive or not a multiple of 8.") } - - /** - * Round a number to the greatest exact multiple of the given factor less than the given number. - * E.g. roundToExactMultiple(67, 8) == 64 - */ - private def roundToExactMultiple(number: Int, factor: Int) = factor * (number / factor) - - /** - * Execute the given function in a lock only if we are running on windows. We do this - * because Windows won't let us resize a file while it is mmapped. As a result we have to force unmap it - * and this requires synchronizing reads. - */ - private def maybeLock[T](lock: Lock)(fun: => T): T = { - if(Os.isWindows) - lock.lock() - try { - fun - } finally { - if(Os.isWindows) - lock.unlock() - } - } + } diff --git a/core/src/main/scala/kafka/log/TimeIndex.scala b/core/src/main/scala/kafka/log/TimeIndex.scala new file mode 100644 index 000000000000..7f2408172195 --- /dev/null +++ b/core/src/main/scala/kafka/log/TimeIndex.scala @@ -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 kafka.log + +import java.io.File +import java.nio.ByteBuffer + +import kafka.common.InvalidOffsetException +import kafka.message.Message +import kafka.utils.CoreUtils._ +import kafka.utils.Logging + +/** + * An index that maps from the timestamp to the logical offsets of the messages in a segment. This index might be + * sparse, i.e. it may not hold an entry for all the messages in the segment. + * + * The index is stored in a file that is preallocated to hold a fixed maximum amount of 12-byte time index entries. + * The file format is a series of time index entries. The physical format is a 8 bytes timestamp and a 4 bytes "relative" + * offset used in the [[OffsetIndex]]. A time index entry (TIMESTAMP, OFFSET) means that the biggest timestamp seen + * before OFFSET is TIMESTAMP. i.e. Any message whose timestamp is greater than TIMESTAMP must come after OFFSET. + * + * All external APIs translate from relative offsets to full offsets, so users of this class do not interact with the internal + * storage format. + * + * The timestamps in the same time index file are guaranteed to be monotonically increasing. + * + * The index support timestamp lookup for a memory map of this file. The lookup is done using a binary search to find + * the offset of the message whose indexed timestamp is closest but smaller or equals to the target timestamp. + * + * Time index files can be opened in two ways: either as an empty, mutable index that allows appends or + * an immutable read-only index file that has previously been populated. The makeReadOnly method will turn a mutable file into an + * immutable one and truncate off any extra bytes. This is done when the index file is rolled over. + * + * No attempt is made to checksum the contents of this file, in the event of a crash it is rebuilt. + * + */ +class TimeIndex(file: File, + baseOffset: Long, + maxIndexSize: Int = -1) + extends AbstractIndex[Long, Long](file, baseOffset, maxIndexSize) with Logging { + + override def entrySize = 12 + + // We override the full check to reserve the last time index entry slot for the on roll call. + override def isFull: Boolean = entries >= maxEntries - 1 + + private def timestamp(buffer: ByteBuffer, n: Int): Long = buffer.getLong(n * entrySize) + + private def relativeOffset(buffer: ByteBuffer, n: Int): Int = buffer.getInt(n * entrySize + 8) + + /** + * The last entry in the index + */ + def lastEntry: TimestampOffset = { + inLock(lock) { + _entries match { + case 0 => TimestampOffset(Message.NoTimestamp, baseOffset) + case s => parseEntry(mmap, s - 1).asInstanceOf[TimestampOffset] + } + } + } + + /** + * Get the nth timestamp mapping from the time index + * @param n The entry number in the time index + * @return The timestamp/offset pair at that entry + */ + def entry(n: Int): TimestampOffset = { + maybeLock(lock) { + if(n >= _entries) + throw new IllegalArgumentException("Attempt to fetch the %dth entry from a time index of size %d.".format(n, _entries)) + val idx = mmap.duplicate + TimestampOffset(timestamp(idx, n), relativeOffset(idx, n)) + } + } + + override def parseEntry(buffer: ByteBuffer, n: Int): IndexEntry = { + TimestampOffset(timestamp(buffer, n), baseOffset + relativeOffset(buffer, n)) + } + + /** + * Attempt to append a time index entry to the time index. + * The new entry is appended only if both the timestamp and offsets are greater than the last appended timestamp and + * the last appended offset. + * + * @param timestamp The timestamp of the new time index entry + * @param offset The offset of the new time index entry + * @param skipFullCheck To skip checking whether the segment is full or not. We only skip the check when the segment + * gets rolled or the segment is closed. + */ + def maybeAppend(timestamp: Long, offset: Long, skipFullCheck: Boolean = false) { + inLock(lock) { + if (!skipFullCheck) + require(!isFull, "Attempt to append to a full time index (size = " + _entries + ").") + // We do not throw exception when the offset equals to the offset of last entry. That means we are trying + // to insert the same time index entry as the last entry. + // If the timestamp index entry to be inserted is the same as the last entry, we simply ignore the insertion + // because that could happen in the following two scenarios: + // 1. An log segment is closed. + // 2. LogSegment.onBecomeInactiveSegment() is called when an active log segment is rolled. + if (_entries != 0 && offset < lastEntry.offset) + throw new InvalidOffsetException("Attempt to append an offset (%d) to slot %d no larger than the last offset appended (%d) to %s." + .format(offset, _entries, lastEntry.offset, file.getAbsolutePath)) + if (_entries != 0 && timestamp < lastEntry.timestamp) + throw new IllegalStateException("Attempt to append an timestamp (%d) to slot %d no larger than the last timestamp appended (%d) to %s." + .format(timestamp, _entries, lastEntry.timestamp, file.getAbsolutePath)) + // We only append to the time index when the timestamp is greater than the last inserted timestamp. + // If all the messages are in message format v0, the timestamp will always be NoTimestamp. In that case, the time + // index will be empty. + if (timestamp > lastEntry.timestamp) { + debug("Adding index entry %d => %d to %s.".format(timestamp, offset, file.getName)) + mmap.putLong(timestamp) + mmap.putInt((offset - baseOffset).toInt) + _entries += 1 + require(_entries * entrySize == mmap.position, _entries + " entries but file position in index is " + mmap.position + ".") + } + } + } + + /** + * Find the time index entry whose timestamp is less than or equal to the given timestamp. + * If the target timestamp is smaller than the least timestamp in the time index, (NoTimestamp, baseOffset) is + * returned. + * + * @param targetTimestamp The timestamp to look up. + * @return The time index entry found. + */ + def lookup(targetTimestamp: Long): TimestampOffset = { + maybeLock(lock) { + val idx = mmap.duplicate + val slot = indexSlotFor(idx, targetTimestamp, IndexSearchType.KEY) + if (slot == -1) + TimestampOffset(Message.NoTimestamp, baseOffset) + else { + val entry = parseEntry(idx, slot).asInstanceOf[TimestampOffset] + TimestampOffset(entry.timestamp, entry.offset) + } + } + } + + override def truncate() = truncateToEntries(0) + + /** + * Remove all entries from the index which have an offset greater than or equal to the given offset. + * Truncating to an offset larger than the largest in the index has no effect. + */ + override def truncateTo(offset: Long) { + inLock(lock) { + val idx = mmap.duplicate + val slot = indexSlotFor(idx, offset, IndexSearchType.VALUE) + + /* There are 3 cases for choosing the new size + * 1) if there is no entry in the index <= the offset, delete everything + * 2) if there is an entry for this exact offset, delete it and everything larger than it + * 3) if there is no entry for this offset, delete everything larger than the next smallest + */ + val newEntries = + if(slot < 0) + 0 + else if(relativeOffset(idx, slot) == offset - baseOffset) + slot + else + slot + 1 + truncateToEntries(newEntries) + } + } + + /** + * Truncates index to a known number of entries. + */ + private def truncateToEntries(entries: Int) { + inLock(lock) { + _entries = entries + mmap.position(_entries * entrySize) + } + } + + override def sanityCheck() { + val entry = lastEntry + val lastTimestamp = entry.timestamp + val lastOffset = entry.offset + require(_entries == 0 || (lastTimestamp >= timestamp(mmap, 0)), + s"Corrupt time index found, time index file (${file.getAbsolutePath}) has non-zero size but the last timestamp " + + s"is $lastTimestamp which is no larger than the first timestamp ${timestamp(mmap, 0)}") + require(_entries == 0 || lastOffset >= baseOffset, + s"Corrupt time index found, time index file (${file.getAbsolutePath}) has non-zero size but the last offset " + + s"is $lastOffset which is smaller than the first offset $baseOffset") + val len = file.length() + require(len % entrySize == 0, + "Time index file " + file.getAbsolutePath + " is corrupt, found " + len + + " bytes which is not positive or not a multiple of 12.") + } +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala index 15d4eea455e3..aadda86cf0e2 100644 --- a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala +++ b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala @@ -400,7 +400,8 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi * If no format conversion or value overwriting is required for messages, this method will perform in-place * operations and avoid re-compression. * - * Returns the message set and a boolean indicating whether the message sizes may have changed. + * Returns a ValidationAndOffsetAssignResult containing the validated message set, maximum timestamp, the offset + * of the shallow message with the max timestamp and a boolean indicating whether the message sizes may have changed. */ private[kafka] def validateMessagesAndAssignOffsets(offsetCounter: LongRef, now: Long, @@ -409,18 +410,16 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi compactedTopic: Boolean = false, messageFormatVersion: Byte = Message.CurrentMagicValue, messageTimestampType: TimestampType, - messageTimestampDiffMaxMs: Long): (ByteBufferMessageSet, Boolean) = { + messageTimestampDiffMaxMs: Long): ValidationAndOffsetAssignResult = { if (sourceCodec == NoCompressionCodec && targetCodec == NoCompressionCodec) { // check the magic value - if (!isMagicValueInAllWrapperMessages(messageFormatVersion)) { - // Message format conversion - (convertNonCompressedMessages(offsetCounter, compactedTopic, now, messageTimestampType, messageTimestampDiffMaxMs, - messageFormatVersion), true) - } else { + if (!isMagicValueInAllWrapperMessages(messageFormatVersion)) + convertNonCompressedMessages(offsetCounter, compactedTopic, now, messageTimestampType, messageTimestampDiffMaxMs, + messageFormatVersion) + else // Do in-place validation, offset assignment and maybe set timestamp - (validateNonCompressedMessagesAndAssignOffsetInPlace(offsetCounter, now, compactedTopic, messageTimestampType, - messageTimestampDiffMaxMs), false) - } + validateNonCompressedMessagesAndAssignOffsetInPlace(offsetCounter, now, compactedTopic, messageTimestampType, + messageTimestampDiffMaxMs) } else { // Deal with compressed messages // We cannot do in place assignment in one of the following situations: @@ -433,6 +432,7 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi var inPlaceAssignment = sourceCodec == targetCodec && messageFormatVersion > Message.MagicValue_V0 var maxTimestamp = Message.NoTimestamp + var offsetOfMaxTimestamp = -1L val expectedInnerOffset = new LongRef(0) val validatedMessages = new mutable.ArrayBuffer[Message] this.internalIterator(isShallow = false).foreach { messageAndOffset => @@ -446,7 +446,10 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi // Check if we need to overwrite offset if (messageAndOffset.offset != expectedInnerOffset.getAndIncrement()) inPlaceAssignment = false - maxTimestamp = math.max(maxTimestamp, message.timestamp) + if (message.timestamp > maxTimestamp) { + maxTimestamp = message.timestamp + offsetOfMaxTimestamp = offsetCounter.value + expectedInnerOffset.value - 1 + } } if (sourceCodec != NoCompressionCodec && message.compressionCodec != NoCompressionCodec) @@ -462,20 +465,23 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi if (!inPlaceAssignment) { // Cannot do in place assignment. - val wrapperMessageTimestamp = { + val (largestTimestampOfMessageSet, offsetOfMaxTimestampInMessageSet) = { if (messageFormatVersion == Message.MagicValue_V0) - Some(Message.NoTimestamp) - else if (messageFormatVersion > Message.MagicValue_V0 && messageTimestampType == TimestampType.CREATE_TIME) - Some(maxTimestamp) + (Some(Message.NoTimestamp), -1L) + else if (messageTimestampType == TimestampType.CREATE_TIME) + (Some(maxTimestamp), {if (targetCodec == NoCompressionCodec) offsetOfMaxTimestamp else offsetCounter.value + validatedMessages.length - 1}) else // Log append time - Some(now) + (Some(now), {if (targetCodec == NoCompressionCodec) offsetCounter.value else offsetCounter.value + validatedMessages.length - 1}) } - (new ByteBufferMessageSet(compressionCodec = targetCodec, - offsetCounter = offsetCounter, - wrapperMessageTimestamp = wrapperMessageTimestamp, - timestampType = messageTimestampType, - messages = validatedMessages: _*), true) + ValidationAndOffsetAssignResult(validatedMessages = new ByteBufferMessageSet(compressionCodec = targetCodec, + offsetCounter = offsetCounter, + wrapperMessageTimestamp = largestTimestampOfMessageSet, + timestampType = messageTimestampType, + messages = validatedMessages: _*), + maxTimestamp = largestTimestampOfMessageSet.get, + offsetOfMaxTimestamp = offsetOfMaxTimestampInMessageSet, + messageSizeMaybeChanged = true) } else { // Do not do re-compression but simply update the offset, timestamp and attributes field of the wrapper message. buffer.putLong(0, offsetCounter.addAndGet(validatedMessages.size) - 1) @@ -487,6 +493,7 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi val attributeOffset = MessageSet.LogOverhead + Message.AttributesOffset val timestamp = buffer.getLong(timestampOffset) val attributes = buffer.get(attributeOffset) + buffer.putLong(timestampOffset, maxTimestamp) if (messageTimestampType == TimestampType.CREATE_TIME && timestamp == maxTimestamp) // We don't need to recompute crc if the timestamp is not updated. crcUpdateNeeded = false @@ -503,7 +510,11 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi Utils.writeUnsignedInt(buffer, MessageSet.LogOverhead + Message.CrcOffset, wrapperMessage.computeChecksum) } buffer.rewind() - (this, false) + // For compressed messages, + ValidationAndOffsetAssignResult(validatedMessages = this, + maxTimestamp = buffer.getLong(timestampOffset), + offsetOfMaxTimestamp = buffer.getLong(0), + messageSizeMaybeChanged = false) } } } @@ -516,12 +527,14 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi now: Long, timestampType: TimestampType, messageTimestampDiffMaxMs: Long, - toMagicValue: Byte): ByteBufferMessageSet = { + toMagicValue: Byte): ValidationAndOffsetAssignResult = { val sizeInBytesAfterConversion = shallowValidBytes + this.internalIterator(isShallow = true).map { messageAndOffset => Message.headerSizeDiff(messageAndOffset.message.magic, toMagicValue) }.sum val newBuffer = ByteBuffer.allocate(sizeInBytesAfterConversion) var newMessagePosition = 0 + var maxTimestamp = Message.NoTimestamp + var offsetOfMaxTimestamp = -1L this.internalIterator(isShallow = true).foreach { case MessageAndOffset(message, _) => validateMessageKey(message, compactedTopic) validateTimestamp(message, now, timestampType, messageTimestampDiffMaxMs) @@ -532,20 +545,31 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi val newMessageBuffer = newBuffer.slice() newMessageBuffer.limit(newMessageSize) message.convertToBuffer(toMagicValue, newMessageBuffer, now, timestampType) - + if (toMagicValue > Message.MagicValue_V0) { + val timestamp = newMessageBuffer.getLong(Message.TimestampOffset) + if (maxTimestamp < timestamp) { + maxTimestamp = timestamp + offsetOfMaxTimestamp = offsetCounter.value - 1 + } + } newMessagePosition += MessageSet.LogOverhead + newMessageSize } newBuffer.rewind() - new ByteBufferMessageSet(newBuffer) + new ValidationAndOffsetAssignResult(validatedMessages = new ByteBufferMessageSet(newBuffer), + maxTimestamp = maxTimestamp, + offsetOfMaxTimestamp = offsetOfMaxTimestamp, + messageSizeMaybeChanged = true) } private def validateNonCompressedMessagesAndAssignOffsetInPlace(offsetCounter: LongRef, now: Long, compactedTopic: Boolean, timestampType: TimestampType, - timestampDiffMaxMs: Long): ByteBufferMessageSet = { + timestampDiffMaxMs: Long): ValidationAndOffsetAssignResult = { // do in-place validation and offset assignment var messagePosition = 0 + var maxTimestamp = Message.NoTimestamp + var offsetOfMaxTimestamp = -1L buffer.mark() while (messagePosition < sizeInBytes - MessageSet.LogOverhead) { buffer.position(messagePosition) @@ -562,11 +586,19 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi message.buffer.put(Message.AttributesOffset, timestampType.updateAttributes(message.attributes)) Utils.writeUnsignedInt(message.buffer, Message.CrcOffset, message.computeChecksum) } + if (message.timestamp > maxTimestamp) { + maxTimestamp = message.timestamp + offsetOfMaxTimestamp = offsetCounter.value - 1 + } } + messagePosition += MessageSet.LogOverhead + messageSize } buffer.reset() - this + ValidationAndOffsetAssignResult(validatedMessages = this, + maxTimestamp = maxTimestamp, + offsetOfMaxTimestamp = offsetOfMaxTimestamp, + messageSizeMaybeChanged = false) } private def validateMessageKey(message: Message, compactedTopic: Boolean) { @@ -614,3 +646,8 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi override def hashCode: Int = buffer.hashCode } + +case class ValidationAndOffsetAssignResult(validatedMessages: ByteBufferMessageSet, + maxTimestamp: Long, + offsetOfMaxTimestamp: Long, + messageSizeMaybeChanged: Boolean) diff --git a/core/src/main/scala/kafka/message/MessageAndOffset.scala b/core/src/main/scala/kafka/message/MessageAndOffset.scala index 51edf9f5da9d..fab6898298f8 100644 --- a/core/src/main/scala/kafka/message/MessageAndOffset.scala +++ b/core/src/main/scala/kafka/message/MessageAndOffset.scala @@ -24,5 +24,13 @@ case class MessageAndOffset(message: Message, offset: Long) { * Compute the offset of the next message in the log */ def nextOffset: Long = offset + 1 + + /** + * We need to decompress the message, if required, to get the offset of the first uncompressed message. + */ + def firstOffset: Long = message.compressionCodec match { + case NoCompressionCodec => offset + case _ => ByteBufferMessageSet.deepIterator(this).next().offset + } } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index bb219ca2f0c9..6eb574f575db 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -38,10 +38,11 @@ import kafka.utils.{Logging, SystemTime, ZKGroupTopicDirs, ZkUtils} import org.apache.kafka.common.errors.{ClusterAuthorizationException, InvalidTopicException, NotLeaderForPartitionException, UnknownTopicOrPartitionException, TopicExistsException} import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.{ApiKeys, Errors, Protocol, SecurityProtocol} -import org.apache.kafka.common.requests.{ApiVersionsResponse, DescribeGroupsRequest, DescribeGroupsResponse, GroupCoordinatorRequest, GroupCoordinatorResponse, HeartbeatRequest, HeartbeatResponse, JoinGroupRequest, JoinGroupResponse, LeaderAndIsrRequest, LeaderAndIsrResponse, LeaveGroupRequest, LeaveGroupResponse, ListGroupsResponse, ListOffsetRequest, ListOffsetResponse, MetadataRequest, MetadataResponse, OffsetCommitRequest, OffsetCommitResponse, OffsetFetchRequest, OffsetFetchResponse, ProduceRequest, ProduceResponse, ResponseHeader, ResponseSend, SaslHandshakeResponse, StopReplicaRequest, StopReplicaResponse, SyncGroupRequest, SyncGroupResponse, UpdateMetadataRequest, UpdateMetadataResponse, CreateTopicsRequest, CreateTopicsResponse, DeleteTopicsRequest, DeleteTopicsResponse} +import org.apache.kafka.common.requests.{ApiVersionsResponse, DescribeGroupsRequest, DescribeGroupsResponse, GroupCoordinatorRequest, GroupCoordinatorResponse, HeartbeatRequest, HeartbeatResponse, JoinGroupRequest, JoinGroupResponse, LeaderAndIsrRequest, LeaderAndIsrResponse, LeaveGroupRequest, LeaveGroupResponse, ListGroupsResponse, ListOffsetRequest, ListOffsetResponse, MetadataRequest, MetadataResponse, OffsetCommitRequest, OffsetCommitResponse, OffsetFetchRequest, OffsetFetchResponse, ProduceRequest, ProduceResponse, ResponseHeader, ResponseSend, StopReplicaRequest, StopReplicaResponse, SyncGroupRequest, SyncGroupResponse, UpdateMetadataRequest, UpdateMetadataResponse, CreateTopicsRequest, CreateTopicsResponse, DeleteTopicsRequest, DeleteTopicsResponse} import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.{Node, TopicPartition} +import org.apache.kafka.common.requests.SaslHandshakeResponse import scala.collection._ import scala.collection.JavaConverters._ diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 2b977839e1c3..f94cbf998aba 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -335,7 +335,7 @@ class ReplicaManager(val config: KafkaConfig, topicPartition -> ProducePartitionStatus( result.info.lastOffset + 1, // required offset - new PartitionResponse(result.errorCode, result.info.firstOffset, result.info.timestamp)) // response status + new PartitionResponse(result.errorCode, result.info.firstOffset, result.info.logAppendTime)) // response status } if (delayedRequestRequired(requiredAcks, messagesPerPartition, localProduceResults)) { diff --git a/core/src/main/scala/kafka/tools/DumpLogSegments.scala b/core/src/main/scala/kafka/tools/DumpLogSegments.scala index dc99672136b2..0a659f49db55 100755 --- a/core/src/main/scala/kafka/tools/DumpLogSegments.scala +++ b/core/src/main/scala/kafka/tools/DumpLogSegments.scala @@ -25,12 +25,13 @@ import kafka.coordinator.{GroupMetadataKey, GroupMetadataManager, OffsetKey} import kafka.log._ import kafka.message._ import kafka.serializer.Decoder -import kafka.utils.{VerifiableProperties, _} +import kafka.utils._ import org.apache.kafka.clients.consumer.internals.ConsumerProtocol import org.apache.kafka.common.KafkaException import org.apache.kafka.common.utils.Utils import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer object DumpLogSegments { @@ -85,6 +86,7 @@ object DumpLogSegments { } val misMatchesForIndexFilesMap = new mutable.HashMap[String, List[(Long, Long)]] + val timeIndexDumpErrors = new TimeIndexDumpErrors val nonConsecutivePairsForLogFilesMap = new mutable.HashMap[String, List[(Long, Long)]] for(arg <- files) { @@ -95,8 +97,12 @@ object DumpLogSegments { } else if(file.getName.endsWith(Log.IndexFileSuffix)) { println("Dumping " + file) dumpIndex(file, indexSanityOnly, verifyOnly, misMatchesForIndexFilesMap, maxMessageSize) + } else if(file.getName.endsWith(Log.TimeIndexFileSuffix)) { + println("Dumping " + file) + dumpTimeIndex(file, indexSanityOnly, verifyOnly, timeIndexDumpErrors, maxMessageSize) } } + misMatchesForIndexFilesMap.foreach { case (fileName, listOfMismatches) => { System.err.println("Mismatches in :" + fileName) @@ -105,6 +111,9 @@ object DumpLogSegments { }) } } + + timeIndexDumpErrors.printErrors() + nonConsecutivePairsForLogFilesMap.foreach { case (fileName, listOfNonConsecutivePairs) => { System.err.println("Non-secutive offsets in :" + fileName) @@ -150,6 +159,58 @@ object DumpLogSegments { } } + private def dumpTimeIndex(file: File, + indexSanityOnly: Boolean, + verifyOnly: Boolean, + timeIndexDumpErrors: TimeIndexDumpErrors, + maxMessageSize: Int) { + val startOffset = file.getName().split("\\.")(0).toLong + val logFile = new File(file.getAbsoluteFile.getParent, file.getName.split("\\.")(0) + Log.LogFileSuffix) + val messageSet = new FileMessageSet(logFile, false) + val indexFile = new File(file.getAbsoluteFile.getParent, file.getName.split("\\.")(0) + Log.IndexFileSuffix) + val index = new OffsetIndex(indexFile, baseOffset = startOffset) + val timeIndex = new TimeIndex(file, baseOffset = startOffset) + + //Check that index passes sanityCheck, this is the check that determines if indexes will be rebuilt on startup or not. + if (indexSanityOnly) { + timeIndex.sanityCheck + println(s"$file passed sanity check.") + return + } + + var prevTimestamp = Message.NoTimestamp + for(i <- 0 until timeIndex.entries) { + val entry = timeIndex.entry(i) + val position = index.lookup(entry.offset + timeIndex.baseOffset).position + val partialFileMessageSet: FileMessageSet = messageSet.read(position, Int.MaxValue) + val shallowIter = partialFileMessageSet.iterator + var maxTimestamp = Message.NoTimestamp + // We first find the message by offset then check if the timestamp is correct. + val wrapperMessageOpt = shallowIter.find(_.offset >= entry.offset + timeIndex.baseOffset) + if (!wrapperMessageOpt.isDefined || wrapperMessageOpt.get.offset != entry.offset + timeIndex.baseOffset) { + timeIndexDumpErrors.recordShallowOffsetNotFound(file, entry.offset + timeIndex.baseOffset, + {if (wrapperMessageOpt.isDefined) wrapperMessageOpt.get.offset else -1}) + } else { + val deepIter = getIterator(wrapperMessageOpt.get, isDeepIteration = true) + for (messageAndOffset <- deepIter) + maxTimestamp = math.max(maxTimestamp, messageAndOffset.message.timestamp) + + if (maxTimestamp != entry.timestamp) + timeIndexDumpErrors.recordMismatchTimeIndex(file, entry.timestamp, maxTimestamp) + + if (prevTimestamp >= entry.timestamp) + timeIndexDumpErrors.recordOutOfOrderIndexTimestamp(file, entry.timestamp, prevTimestamp) + + // since it is a sparse file, in the event of a crash there may be many zero entries, stop if we see one + if (entry.offset == 0 && i > 0) + return + } + if (!verifyOnly) + println("timestamp: %s offset: %s".format(entry.timestamp, timeIndex.baseOffset + entry.offset)) + prevTimestamp = entry.timestamp + } + } + private trait MessageParser[K, V] { def parse(message: Message): (Option[K], Option[V]) } @@ -261,7 +322,8 @@ object DumpLogSegments { } lastOffset = messageAndOffset.offset - print("offset: " + messageAndOffset.offset + " position: " + validBytes + " isvalid: " + msg.isValid + + print("offset: " + messageAndOffset.offset + " position: " + validBytes + + " " + msg.timestampType + ": " + msg.timestamp + " isvalid: " + msg.isValid + " payloadsize: " + msg.payloadSize + " magic: " + msg.magic + " compresscodec: " + msg.compressionCodec + " crc: " + msg.checksum) if(msg.hasKey) @@ -307,4 +369,60 @@ object DumpLogSegments { } } + class TimeIndexDumpErrors { + val misMatchesForTimeIndexFilesMap = new mutable.HashMap[String, ArrayBuffer[(Long, Long)]] + val outOfOrderTimestamp = new mutable.HashMap[String, ArrayBuffer[(Long, Long)]] + val shallowOffsetNotFound = new mutable.HashMap[String, ArrayBuffer[(Long, Long)]] + + def recordMismatchTimeIndex(file: File, indexTimestamp: Long, logTimestamp: Long) { + var misMatchesSeq = misMatchesForTimeIndexFilesMap.getOrElse(file.getAbsolutePath, new ArrayBuffer[(Long, Long)]()) + if (misMatchesSeq.isEmpty) + misMatchesForTimeIndexFilesMap.put(file.getAbsolutePath, misMatchesSeq) + misMatchesSeq += ((indexTimestamp, logTimestamp)) + } + + def recordOutOfOrderIndexTimestamp(file: File, indexTimestamp: Long, prevIndexTimestamp: Long) { + var outOfOrderSeq = outOfOrderTimestamp.getOrElse(file.getAbsolutePath, new ArrayBuffer[(Long, Long)]()) + if (outOfOrderSeq.isEmpty) + outOfOrderTimestamp.put(file.getAbsolutePath, outOfOrderSeq) + outOfOrderSeq += ((indexTimestamp, prevIndexTimestamp)) + } + + def recordShallowOffsetNotFound(file: File, indexOffset: Long, logOffset: Long) { + var shallowOffsetNotFoundSeq = shallowOffsetNotFound.getOrElse(file.getAbsolutePath, new ArrayBuffer[(Long, Long)]()) + if (shallowOffsetNotFoundSeq.isEmpty) + shallowOffsetNotFound.put(file.getAbsolutePath, shallowOffsetNotFoundSeq) + shallowOffsetNotFoundSeq += ((indexOffset, logOffset)) + } + + def printErrors() { + misMatchesForTimeIndexFilesMap.foreach { + case (fileName, listOfMismatches) => { + System.err.println("Found timestamp mismatch in :" + fileName) + listOfMismatches.foreach(m => { + System.err.println(" Index timestamp: %d, log timestamp: %d".format(m._1, m._2)) + }) + } + } + + outOfOrderTimestamp.foreach { + case (fileName, outOfOrderTimestamps) => { + System.err.println("Found out of order timestamp in :" + fileName) + outOfOrderTimestamps.foreach(m => { + System.err.println(" Index timestamp: %d, Previously indexed timestamp: %d".format(m._1, m._2)) + }) + } + } + + shallowOffsetNotFound.foreach { + case (fileName, listOfShallowOffsetNotFound) => { + System.err.println("The following indexed offsets are not found in the log.") + listOfShallowOffsetNotFound.foreach(m => { + System.err.println("Indexed offset: %s, found log offset: %s".format(m._1, m._2)) + }) + } + } + } + } + } diff --git a/core/src/test/scala/unit/kafka/log/CleanerTest.scala b/core/src/test/scala/unit/kafka/log/CleanerTest.scala index 82121218928a..15920ad8b8c8 100755 --- a/core/src/test/scala/unit/kafka/log/CleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/CleanerTest.scala @@ -233,7 +233,7 @@ class CleanerTest extends JUnitSuite { assertTrue("All but the last group should be the target size.", groups.dropRight(1).forall(_.size == groupSize)) // check grouping by index size - val indexSize = log.logSegments.take(groupSize).map(_.index.sizeInBytes()).sum + 1 + val indexSize = log.logSegments.take(groupSize).map(_.index.sizeInBytes).sum + 1 groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = indexSize) checkSegmentOrder(groups) assertTrue("All but the last group should be the target size.", groups.dropRight(1).forall(_.size == groupSize)) @@ -391,8 +391,9 @@ class CleanerTest extends JUnitSuite { for (file <- dir.listFiles if file.getName.endsWith(Log.DeletedFileSuffix)) { Utils.atomicMoveWithFallback(file.toPath, Paths.get(CoreUtils.replaceSuffix(file.getPath, Log.DeletedFileSuffix, ""))) } + System.out.println("here") log = recoverAndCheck(config, cleanedKeys) - + // add some more messages and clean the log again while(log.numberOfSegments < 10) { log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt)) diff --git a/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala b/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala index a64454d3165f..82496f2a8498 100644 --- a/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala +++ b/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala @@ -111,21 +111,21 @@ class FileMessageSetTest extends BaseMessageSetTestCases { var position = 0 assertEquals("Should be able to find the first message by its offset", OffsetPosition(0L, position), - messageSet.searchFor(0, 0)) + messageSet.searchForOffset(0, 0)) position += MessageSet.entrySize(messageSet.head.message) assertEquals("Should be able to find second message when starting from 0", OffsetPosition(1L, position), - messageSet.searchFor(1, 0)) + messageSet.searchForOffset(1, 0)) assertEquals("Should be able to find second message starting from its offset", OffsetPosition(1L, position), - messageSet.searchFor(1, position)) + messageSet.searchForOffset(1, position)) position += MessageSet.entrySize(messageSet.tail.head.message) + MessageSet.entrySize(messageSet.tail.tail.head.message) assertEquals("Should be able to find fourth message from a non-existant offset", OffsetPosition(50L, position), - messageSet.searchFor(3, position)) + messageSet.searchForOffset(3, position)) assertEquals("Should be able to find fourth message by correct offset", OffsetPosition(50L, position), - messageSet.searchFor(50, position)) + messageSet.searchForOffset(50, position)) } /** @@ -134,7 +134,7 @@ class FileMessageSetTest extends BaseMessageSetTestCases { @Test def testIteratorWithLimits() { val message = messageSet.toList(1) - val start = messageSet.searchFor(1, 0).position + val start = messageSet.searchForOffset(1, 0).position val size = message.message.size + 12 val slice = messageSet.read(start, size) assertEquals(List(message), slice.toList) @@ -148,7 +148,7 @@ class FileMessageSetTest extends BaseMessageSetTestCases { @Test def testTruncate() { val message = messageSet.toList.head - val end = messageSet.searchFor(1, 0).position + val end = messageSet.searchForOffset(1, 0).position messageSet.truncateTo(end) assertEquals(List(message), messageSet.toList) assertEquals(MessageSet.entrySize(message.message), messageSet.sizeInBytes) @@ -272,7 +272,7 @@ class FileMessageSetTest extends BaseMessageSetTestCases { @Test def testFormatConversionWithPartialMessage() { val message = messageSet.toList(1) - val start = messageSet.searchFor(1, 0).position + val start = messageSet.searchForOffset(1, 0).position val size = message.message.size + 12 val slice = messageSet.read(start, size - 1) val messageV0 = slice.toMessageFormat(Message.MagicValue_V0) diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index 7b52a094c6c2..dc4cc7974fdf 100755 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -100,7 +100,7 @@ class LogManagerTest { time.sleep(maxLogAgeMs + 1) assertEquals("Now there should only be only one segment in the index.", 1, log.numberOfSegments) time.sleep(log.config.fileDeleteDelayMs + 1) - assertEquals("Files should have been deleted", log.numberOfSegments * 2, log.dir.list.length) + assertEquals("Files should have been deleted", log.numberOfSegments * 3, log.dir.list.length) assertEquals("Should get empty fetch off new log.", 0, log.read(offset+1, 1024).messageSet.sizeInBytes) try { @@ -146,7 +146,7 @@ class LogManagerTest { time.sleep(logManager.InitialTaskDelayMs) assertEquals("Now there should be exactly 6 segments", 6, log.numberOfSegments) time.sleep(log.config.fileDeleteDelayMs + 1) - assertEquals("Files should have been deleted", log.numberOfSegments * 2, log.dir.list.length) + assertEquals("Files should have been deleted", log.numberOfSegments * 3, log.dir.list.length) assertEquals("Should get empty fetch off new log.", 0, log.read(offset + 1, 1024).messageSet.sizeInBytes) try { log.read(0, 1024) diff --git a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala index edbfd99482d0..64140e824ed6 100644 --- a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala @@ -26,19 +26,23 @@ import kafka.message._ import kafka.utils.SystemTime import scala.collection._ + import scala.collection.mutable.ListBuffer class LogSegmentTest { val segments = mutable.ArrayBuffer[LogSegment]() /* create a segment with the given base offset */ - def createSegment(offset: Long): LogSegment = { + def createSegment(offset: Long, indexIntervalBytes: Int = 10): LogSegment = { val msFile = TestUtils.tempFile() val ms = new FileMessageSet(msFile) val idxFile = TestUtils.tempFile() + val timeIdxFile = TestUtils.tempFile() idxFile.delete() + timeIdxFile.delete() val idx = new OffsetIndex(idxFile, offset, 1000) - val seg = new LogSegment(ms, idx, offset, 10, 0, SystemTime) + val timeIdx = new TimeIndex(timeIdxFile, offset, 1500) + val seg = new LogSegment(ms, idx, timeIdx, offset, indexIntervalBytes, 0, SystemTime) segments += seg seg } @@ -47,7 +51,7 @@ class LogSegmentTest { def messages(offset: Long, messages: String*): ByteBufferMessageSet = { new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, offsetCounter = new LongRef(offset), - messages = messages.map(s => new Message(s.getBytes)):_*) + messages = messages.map(s => new Message(s.getBytes, offset * 10, Message.MagicValue_V1)):_*) } @After @@ -76,7 +80,7 @@ class LogSegmentTest { def testReadBeforeFirstOffset() { val seg = createSegment(40) val ms = messages(50, "hello", "there", "little", "bee") - seg.append(50, ms) + seg.append(50, Message.NoTimestamp, -1L, ms) val read = seg.read(startOffset = 41, maxSize = 300, maxOffset = None).messageSet assertEquals(ms.toList, read.toList) } @@ -90,7 +94,7 @@ class LogSegmentTest { val baseOffset = 50 val seg = createSegment(baseOffset) val ms = messages(baseOffset, "hello", "there", "beautiful") - seg.append(baseOffset, ms) + seg.append(baseOffset, Message.NoTimestamp, -1L, ms) def validate(offset: Long) = assertEquals(ms.filter(_.offset == offset).toList, seg.read(startOffset = offset, maxSize = 1024, maxOffset = Some(offset+1)).messageSet.toList) @@ -106,7 +110,7 @@ class LogSegmentTest { def testReadAfterLast() { val seg = createSegment(40) val ms = messages(50, "hello", "there") - seg.append(50, ms) + seg.append(50, Message.NoTimestamp, -1L, ms) val read = seg.read(startOffset = 52, maxSize = 200, maxOffset = None) assertNull("Read beyond the last offset in the segment should give null", read) } @@ -119,9 +123,9 @@ class LogSegmentTest { def testReadFromGap() { val seg = createSegment(40) val ms = messages(50, "hello", "there") - seg.append(50, ms) + seg.append(50, Message.NoTimestamp, -1L, ms) val ms2 = messages(60, "alpha", "beta") - seg.append(60, ms2) + seg.append(60, Message.NoTimestamp, -1L, ms2) val read = seg.read(startOffset = 55, maxSize = 200, maxOffset = None) assertEquals(ms2.toList, read.messageSet.toList) } @@ -136,9 +140,9 @@ class LogSegmentTest { var offset = 40 for(i <- 0 until 30) { val ms1 = messages(offset, "hello") - seg.append(offset, ms1) - val ms2 = messages(offset+1, "hello") - seg.append(offset+1, ms2) + seg.append(offset, Message.NoTimestamp, -1L, ms1) + val ms2 = messages(offset + 1, "hello") + seg.append(offset + 1, Message.NoTimestamp, -1L, ms2) // check that we can read back both messages val read = seg.read(offset, None, 10000) assertEquals(List(ms1.head, ms2.head), read.messageSet.toList) @@ -150,7 +154,25 @@ class LogSegmentTest { offset += 1 } } - + + @Test + def testReloadLargestTimestampAfterTruncation() { + val numMessages = 30 + val seg = createSegment(40, 2 * messages(0, "hello").sizeInBytes - 1) + var offset = 40 + for (i <- 0 until numMessages) { + seg.append(offset, offset, offset, messages(offset, "hello")) + offset += 1 + } + val expectedNumEntries = numMessages / 2 - 1 + assertEquals(s"Should have $expectedNumEntries time indexes", expectedNumEntries, seg.timeIndex.entries) + + seg.truncateTo(41) + assertEquals(s"Should have 0 time indexes", 0, seg.timeIndex.entries) + assertEquals(s"Largest timestamp should be 400", 400L, seg.largestTimestamp) + + } + /** * Test truncating the whole segment, and check that we can reappend with the original offset. */ @@ -158,12 +180,38 @@ class LogSegmentTest { def testTruncateFull() { // test the case where we fully truncate the log val seg = createSegment(40) - seg.append(40, messages(40, "hello", "there")) + seg.append(40, Message.NoTimestamp, -1L, messages(40, "hello", "there")) seg.truncateTo(0) assertNull("Segment should be empty.", seg.read(0, None, 1024)) - seg.append(40, messages(40, "hello", "there")) + seg.append(40, Message.NoTimestamp, -1L, messages(40, "hello", "there")) } - + + /** + * Append messages with timestamp and search message by timestamp. + */ + @Test + def testFindOffsetByTimestamp() { + val messageSize = messages(0, s"msg00").sizeInBytes + val seg = createSegment(40, messageSize * 2 - 1) + // Produce some messages + for (i <- 40 until 50) + seg.append(i, i * 10, i, messages(i, s"msg$i")) + + assertEquals(490, seg.largestTimestamp) + // Search for an indexed timestamp + assertEquals(42, seg.findOffsetByTimestamp(420).get) + assertEquals(43, seg.findOffsetByTimestamp(421).get) + // Search for an un-indexed timestamp + assertEquals(43, seg.findOffsetByTimestamp(430).get) + assertEquals(44, seg.findOffsetByTimestamp(431).get) + // Search beyond the last timestamp + assertEquals(50, seg.findOffsetByTimestamp(491).get) + // Search before the first indexed timestamp + assertEquals(41, seg.findOffsetByTimestamp(401).get) + // Search before the first timestamp + assertEquals(40, seg.findOffsetByTimestamp(399).get) + } + /** * Test that offsets are assigned sequentially and that the nextOffset variable is incremented */ @@ -171,7 +219,7 @@ class LogSegmentTest { def testNextOffsetCalculation() { val seg = createSegment(40) assertEquals(40, seg.nextOffset) - seg.append(50, messages(50, "hello", "there", "you")) + seg.append(50, Message.NoTimestamp, -1L, messages(50, "hello", "there", "you")) assertEquals(53, seg.nextOffset()) } @@ -198,13 +246,31 @@ class LogSegmentTest { def testRecoveryFixesCorruptIndex() { val seg = createSegment(0) for(i <- 0 until 100) - seg.append(i, messages(i, i.toString)) + seg.append(i, Message.NoTimestamp, -1L, messages(i, i.toString)) val indexFile = seg.index.file TestUtils.writeNonsenseToFile(indexFile, 5, indexFile.length.toInt) seg.recover(64*1024) for(i <- 0 until 100) assertEquals(i, seg.read(i, Some(i+1), 1024).messageSet.head.offset) } + + /** + * Create a segment with some data and an index. Then corrupt the index, + * and recover the segment, the entries should all be readable. + */ + @Test + def testRecoveryFixesCorruptTimeIndex() { + val seg = createSegment(0) + for(i <- 0 until 100) + seg.append(i, i * 10, i, messages(i, i.toString)) + val timeIndexFile = seg.timeIndex.file + TestUtils.writeNonsenseToFile(timeIndexFile, 5, timeIndexFile.length.toInt) + seg.recover(64*1024) + for(i <- 0 until 100) { + assertEquals(i, seg.findOffsetByTimestamp(i * 10).get) + assertEquals(i + 1, seg.findOffsetByTimestamp(i * 10 + 1).get) + } + } /** * Randomly corrupt a log a number of times and attempt recovery. @@ -215,10 +281,10 @@ class LogSegmentTest { for(iteration <- 0 until 10) { val seg = createSegment(0) for(i <- 0 until messagesAppended) - seg.append(i, messages(i, i.toString)) + seg.append(i, Message.NoTimestamp, -1L, messages(i, i.toString)) val offsetToBeginCorruption = TestUtils.random.nextInt(messagesAppended) // start corrupting somewhere in the middle of the chosen record all the way to the end - val position = seg.log.searchFor(offsetToBeginCorruption, 0).position + TestUtils.random.nextInt(15) + val position = seg.log.searchForOffset(offsetToBeginCorruption, 0).position + TestUtils.random.nextInt(15) TestUtils.writeNonsenseToFile(seg.log.file, position, seg.log.file.length.toInt - position) seg.recover(64*1024) assertEquals("Should have truncated off bad messages.", (0 until offsetToBeginCorruption).toList, seg.log.map(_.offset).toList) @@ -227,7 +293,7 @@ class LogSegmentTest { } /* create a segment with pre allocate */ - def createSegment(offset: Long, fileAlreadyExists: Boolean = false, initFileSize: Int = 0, preallocate: Boolean = false): LogSegment = { + def createSegment(offset: Long, fileAlreadyExists: Boolean, initFileSize: Int, preallocate: Boolean): LogSegment = { val tempDir = TestUtils.tempDir() val seg = new LogSegment(tempDir, offset, 10, 1000, 0, SystemTime, fileAlreadyExists = fileAlreadyExists, initFileSize = initFileSize, preallocate = preallocate) segments += seg @@ -239,9 +305,9 @@ class LogSegmentTest { def testCreateWithInitFileSizeAppendMessage() { val seg = createSegment(40, false, 512*1024*1024, true) val ms = messages(50, "hello", "there") - seg.append(50, ms) + seg.append(50, Message.NoTimestamp, -1L, ms) val ms2 = messages(60, "alpha", "beta") - seg.append(60, ms2) + seg.append(60, Message.NoTimestamp, -1L, ms2) val read = seg.read(startOffset = 55, maxSize = 200, maxOffset = None) assertEquals(ms2.toList, read.messageSet.toList) } @@ -253,9 +319,9 @@ class LogSegmentTest { 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) + seg.append(50, Message.NoTimestamp, -1L, ms) val ms2 = messages(60, "alpha", "beta") - seg.append(60, ms2) + seg.append(60, Message.NoTimestamp, -1L, ms2) val read = seg.read(startOffset = 55, maxSize = 200, maxOffset = None) assertEquals(ms2.toList, read.messageSet.toList) val oldSize = seg.log.sizeInBytes() diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index 33dd68ef3ab4..2466ef265b13 100755 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -35,7 +35,7 @@ class LogTest extends JUnitSuite { val tmpDir = TestUtils.tempDir() val logDir = TestUtils.randomPartitionLogDir(tmpDir) - val time = new MockTime(0) + val time = new MockTime(100) var config: KafkaConfig = null val logConfig = LogConfig() @@ -88,6 +88,20 @@ class LogTest extends JUnitSuite { assertEquals("Changing time beyond rollMs and appending should create a new segment.", numSegments, log.numberOfSegments) } + time.sleep(log.config.segmentMs + 1) + val setWithTimestamp = + TestUtils.singleMessageSet(payload = "test".getBytes, timestamp = time.milliseconds + log.config.segmentMs + 1) + log.append(setWithTimestamp) + assertEquals("A new segment should have been rolled out", 5, log.numberOfSegments) + + time.sleep(log.config.segmentMs + 1) + log.append(set) + assertEquals("Log should not roll because the roll should depend on the index of the first time index entry.", 5, log.numberOfSegments) + + time.sleep(log.config.segmentMs + 1) + log.append(set) + assertEquals("Log should roll because the time since the timestamp of first time index entry has expired.", 6, log.numberOfSegments) + val numSegments = log.numberOfSegments time.sleep(log.config.segmentMs + 1) log.append(new ByteBufferMessageSet()) @@ -457,27 +471,63 @@ class LogTest extends JUnitSuite { 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))) + log.append(TestUtils.singleMessageSet(payload = TestUtils.randomBytes(messageSize), + timestamp = time.milliseconds + i * 10)) assertEquals("After appending %d messages to an empty log, the log end offset should be %d".format(numMessages, numMessages), numMessages, log.logEndOffset) val lastIndexOffset = log.activeSegment.index.lastOffset val numIndexEntries = log.activeSegment.index.entries val lastOffset = log.logEndOffset + // After segment is closed, the last entry in the time index should be (largest timestamp -> last offset). + val lastTimeIndexOffset = log.logEndOffset - 1 + val lastTimeIndexTimestamp = log.activeSegment.largestTimestamp + // Depending on when the last time index entry is inserted, an entry may or may not be inserted into the time index. + val numTimeIndexEntries = log.activeSegment.timeIndex.entries + { + if (log.activeSegment.timeIndex.lastEntry.offset == log.logEndOffset - 1) 0 else 1 + } log.close() + def verifyRecoveredLog(log: Log) { + assertEquals(s"Should have $numMessages messages when log is reopened w/o recovery", numMessages, log.logEndOffset) + assertEquals("Should have same last index offset as before.", lastIndexOffset, log.activeSegment.index.lastOffset) + assertEquals("Should have same number of index entries as before.", numIndexEntries, log.activeSegment.index.entries) + assertEquals("Should have same last time index timestamp", lastTimeIndexTimestamp, log.activeSegment.timeIndex.lastEntry.timestamp) + assertEquals("Should have same last time index offset", lastTimeIndexOffset, log.activeSegment.timeIndex.lastEntry.offset) + assertEquals("Should have same number of time index entries as before.", numTimeIndexEntries, log.activeSegment.timeIndex.entries) + } + log = new Log(logDir, config, recoveryPoint = lastOffset, time.scheduler, time) - assertEquals("Should have %d messages when log is reopened w/o recovery".format(numMessages), numMessages, log.logEndOffset) - assertEquals("Should have same last index offset as before.", lastIndexOffset, log.activeSegment.index.lastOffset) - assertEquals("Should have same number of index entries as before.", numIndexEntries, log.activeSegment.index.entries) + verifyRecoveredLog(log) log.close() // test recovery case log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time) - assertEquals("Should have %d messages when log is reopened with recovery".format(numMessages), numMessages, log.logEndOffset) - assertEquals("Should have same last index offset as before.", lastIndexOffset, log.activeSegment.index.lastOffset) - assertEquals("Should have same number of index entries as before.", numIndexEntries, log.activeSegment.index.entries) + verifyRecoveredLog(log) log.close() } + /** + * Test building the time index on the follower by setting assignOffsets to false. + */ + @Test + def testBuildTimeIndexWhenNotAssigningOffsets() { + val numMessages = 100 + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 10000: java.lang.Integer) + logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer) + + val config = LogConfig(logProps) + val log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time) + + val messages = (0 until numMessages).map { i => + new ByteBufferMessageSet(NoCompressionCodec, new LongRef(100 + i), new Message(i.toString.getBytes(), time.milliseconds + i, Message.MagicValue_V1)) + } + messages.foreach(log.append(_, assignOffsets = false)) + val timeIndexEntries = log.logSegments.foldLeft(0) { (entries, segment) => entries + segment.timeIndex.entries } + assertEquals(s"There should be ${numMessages - 1} time index entries", numMessages - 1, timeIndexEntries) + assertEquals(s"The last time index entry should have timestamp ${time.milliseconds + numMessages - 1}", + time.milliseconds + numMessages - 1, log.activeSegment.timeIndex.lastEntry.timestamp) + } + /** * Test that if we manually delete an index segment it is rebuilt when the log is re-opened */ @@ -492,19 +542,58 @@ class LogTest extends JUnitSuite { 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))) + log.append(TestUtils.singleMessageSet(payload = TestUtils.randomBytes(10), timestamp = time.milliseconds + i * 10)) val indexFiles = log.logSegments.map(_.index.file) + val timeIndexFiles = log.logSegments.map(_.timeIndex.file) log.close() // delete all the index files indexFiles.foreach(_.delete()) + timeIndexFiles.foreach(_.delete()) // reopen the log log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time) assertEquals("Should have %d messages when log is reopened".format(numMessages), numMessages, log.logEndOffset) - for(i <- 0 until numMessages) + assertTrue("The index should have been rebuilt", log.logSegments.head.index.entries > 0) + assertTrue("The time index should have been rebuilt", log.logSegments.head.timeIndex.entries > 0) + for(i <- 0 until numMessages) { assertEquals(i, log.read(i, 100, None).messageSet.head.offset) + if (i == 0) + assertEquals(log.logSegments.head.baseOffset, log.fetchOffsetsByTimestamp(time.milliseconds + i * 10)) + else + assertEquals(i, log.fetchOffsetsByTimestamp(time.milliseconds + i * 10)) + } + log.close() + } + + /** + * Test that if messages format version of the messages in a segment is before 0.10.0, the time index should be empty. + */ + @Test + def testRebuildTimeIndexForOldMessages() { + val numMessages = 200 + val segmentSize = 200 + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, segmentSize: java.lang.Integer) + logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer) + logProps.put(LogConfig.MessageFormatVersionProp, "0.9.0") + + val config = LogConfig(logProps) + var log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time) + for(i <- 0 until numMessages) + log.append(TestUtils.singleMessageSet(payload = TestUtils.randomBytes(10), timestamp = time.milliseconds + i * 10)) + val timeIndexFiles = log.logSegments.map(_.timeIndex.file) log.close() + + // Delete the time index. + timeIndexFiles.foreach(_.delete()) + + // The rebuilt time index should be empty + log = new Log(logDir, config, recoveryPoint = numMessages + 1, time.scheduler, time) + val segArray = log.logSegments.toArray + for (i <- 0 until segArray.size - 1) + assertEquals("The time index should be empty", 0, segArray(i).timeIndex.entries) + } /** @@ -521,8 +610,9 @@ class LogTest extends JUnitSuite { 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))) + log.append(TestUtils.singleMessageSet(payload = TestUtils.randomBytes(10), timestamp = time.milliseconds + i * 10)) val indexFiles = log.logSegments.map(_.index.file) + val timeIndexFiles = log.logSegments.map(_.timeIndex.file) log.close() // corrupt all the index files @@ -532,11 +622,23 @@ class LogTest extends JUnitSuite { bw.close() } + // corrupt all the index files + for( file <- timeIndexFiles) { + 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) + for(i <- 0 until numMessages) { assertEquals(i, log.read(i, 100, None).messageSet.head.offset) + if (i == 0) + assertEquals(log.logSegments.head.baseOffset, log.fetchOffsetsByTimestamp(time.milliseconds + i * 10)) + else + assertEquals(i, log.fetchOffsetsByTimestamp(time.milliseconds + i * 10)) + } log.close() } @@ -602,27 +704,37 @@ class LogTest extends JUnitSuite { */ @Test def testIndexResizingAtTruncation() { - val set = TestUtils.singleMessageSet("test".getBytes) - val setSize = set.sizeInBytes + val setSize = TestUtils.singleMessageSet(payload = "test".getBytes).sizeInBytes 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) + logProps.put(LogConfig.IndexIntervalBytesProp, (setSize - 1): 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) - log.append(set) + log.append(TestUtils.singleMessageSet(payload = "test".getBytes, timestamp = time.milliseconds + i)) assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments) + + time.sleep(msgPerSeg) for (i<- 1 to msgPerSeg) - log.append(set) + log.append(TestUtils.singleMessageSet(payload = "test".getBytes, timestamp = time.milliseconds + i)) assertEquals("There should be exactly 2 segment.", 2, log.numberOfSegments) - assertEquals("The index of the first segment should be trimmed to empty", 0, log.logSegments.toList.head.index.maxEntries) + val expectedEntries = msgPerSeg - 1 + + assertEquals(s"The index of the first segment should have $expectedEntries entries", expectedEntries, log.logSegments.toList.head.index.maxEntries) + assertEquals(s"The time index of the first segment should have $expectedEntries entries", expectedEntries, log.logSegments.toList.head.timeIndex.maxEntries) + log.truncateTo(0) assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments) assertEquals("The index of segment 1 should be resized to maxIndexSize", log.config.maxIndexSize/8, log.logSegments.toList.head.index.maxEntries) + assertEquals("The time index of segment 1 should be resized to maxIndexSize", log.config.maxIndexSize/12, log.logSegments.toList.head.timeIndex.maxEntries) + + time.sleep(msgPerSeg) for (i<- 1 to msgPerSeg) - log.append(set) + log.append(TestUtils.singleMessageSet(payload = "test".getBytes, timestamp = time.milliseconds + i)) assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments) } @@ -632,7 +744,9 @@ class LogTest extends JUnitSuite { @Test def testBogusIndexSegmentsAreRemoved() { val bogusIndex1 = Log.indexFilename(logDir, 0) + val bogusTimeIndex1 = Log.timeIndexFilename(logDir, 0) val bogusIndex2 = Log.indexFilename(logDir, 5) + val bogusTimeIndex2 = Log.timeIndexFilename(logDir, 5) val set = TestUtils.singleMessageSet("test".getBytes) val logProps = new Properties() @@ -646,7 +760,9 @@ class LogTest extends JUnitSuite { time) assertTrue("The first index file should have been replaced with a larger file", bogusIndex1.length > 0) + assertTrue("The first time index file should have been replaced with a larger file", bogusTimeIndex1.length > 0) assertFalse("The second index file should have been deleted.", bogusIndex2.exists) + assertFalse("The second time index file should have been deleted.", bogusTimeIndex2.exists) // check that we can append to the log for(i <- 0 until 10) diff --git a/core/src/test/scala/unit/kafka/log/TimeIndexTest.scala b/core/src/test/scala/unit/kafka/log/TimeIndexTest.scala new file mode 100644 index 000000000000..bc60c7236da2 --- /dev/null +++ b/core/src/test/scala/unit/kafka/log/TimeIndexTest.scala @@ -0,0 +1,97 @@ +/** + * 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.log + +import java.io.File + +import kafka.common.InvalidOffsetException +import kafka.utils.TestUtils +import org.junit.{Test, After, Before} +import org.junit.Assert.{assertEquals} +import org.scalatest.junit.JUnitSuite + +/** + * Unit test for time index. + */ +class TimeIndexTest extends JUnitSuite { + var idx: TimeIndex = null + val maxEntries = 30 + val baseOffset = 45L + + @Before + def setup() { + this.idx = new TimeIndex(file = nonExistantTempFile(), baseOffset = baseOffset, maxIndexSize = maxEntries * 12) + } + + @After + def teardown() { + if(this.idx != null) + this.idx.file.delete() + } + + @Test + def testLookUp() { + // Empty time index + assertEquals(TimestampOffset(-1L, baseOffset), idx.lookup(100L)) + + // Add several time index entries. + appendEntries(maxEntries - 1) + + // look for timestamp smaller than the earliest entry + assertEquals(TimestampOffset(-1L, baseOffset), idx.lookup(9)) + // look for timestamp in the middle of two entries. + assertEquals(TimestampOffset(20L, 65L), idx.lookup(25)) + // look for timestamp same as the one in the entry + assertEquals(TimestampOffset(30L, 75L), idx.lookup(30)) + } + + @Test + def testTruncate() { + appendEntries(maxEntries - 1) + idx.truncate() + assertEquals(0, idx.entries) + + appendEntries(maxEntries - 1) + idx.truncateTo(10 + baseOffset) + assertEquals(0, idx.entries) + } + + @Test + def testAppend() { + appendEntries(maxEntries - 1) + intercept[IllegalArgumentException] { + idx.maybeAppend(10000L, 1000L) + } + intercept[InvalidOffsetException] { + idx.maybeAppend(10000L, (maxEntries - 2) * 10, true) + } + idx.maybeAppend(10000L, 1000L, true) + } + + private def appendEntries(numEntries: Int) { + for (i <- 1 to numEntries) + idx.maybeAppend(i * 10, i * 10 + baseOffset) + } + + def nonExistantTempFile(): File = { + val file = TestUtils.tempFile() + file.delete() + file + } +} + diff --git a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala b/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala index 481000946a45..39eb84c8fd48 100644 --- a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala +++ b/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala @@ -152,56 +152,69 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { @Test def testLogAppendTime() { - val startTime = System.currentTimeMillis() + val now = System.currentTimeMillis() // The timestamps should be overwritten val messages = getMessages(magicValue = Message.MagicValue_V1, timestamp = 0L, codec = NoCompressionCodec) val compressedMessagesWithRecompresion = getMessages(magicValue = Message.MagicValue_V0, codec = DefaultCompressionCodec) val compressedMessagesWithoutRecompression = - getMessages(magicValue = Message.MagicValue_V1, timestamp = -1L, codec = DefaultCompressionCodec) - - val (validatedMessages, _) = messages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(0), - now = System.currentTimeMillis(), - sourceCodec = NoCompressionCodec, - targetCodec = NoCompressionCodec, - messageFormatVersion = 1, - messageTimestampType = TimestampType.LOG_APPEND_TIME, - messageTimestampDiffMaxMs = 1000L) - - val (validatedCompressedMessages, _) = + getMessages(magicValue = Message.MagicValue_V1, timestamp = 0L, codec = DefaultCompressionCodec) + + val validatingResults = messages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(0), + now = now, + sourceCodec = NoCompressionCodec, + targetCodec = NoCompressionCodec, + messageFormatVersion = 1, + messageTimestampType = TimestampType.LOG_APPEND_TIME, + messageTimestampDiffMaxMs = 1000L) + val validatedMessages = validatingResults.validatedMessages + + val validatingCompressedMessagesResults = compressedMessagesWithRecompresion.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(0), - now = System.currentTimeMillis(), + now = now, sourceCodec = DefaultCompressionCodec, targetCodec = DefaultCompressionCodec, messageFormatVersion = 1, messageTimestampType = TimestampType.LOG_APPEND_TIME, messageTimestampDiffMaxMs = 1000L) + val validatedCompressedMessages = validatingCompressedMessagesResults.validatedMessages - val (validatedCompressedMessagesWithoutRecompression, _) = + val validatingCompressedMessagesWithoutRecompressionResults = compressedMessagesWithoutRecompression.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(0), - now = System.currentTimeMillis(), + now = now, sourceCodec = DefaultCompressionCodec, targetCodec = DefaultCompressionCodec, messageFormatVersion = 1, messageTimestampType = TimestampType.LOG_APPEND_TIME, messageTimestampDiffMaxMs = 1000L) - val now = System.currentTimeMillis() + val validatedCompressedMessagesWithoutRecompression = validatingCompressedMessagesWithoutRecompressionResults.validatedMessages + assertEquals("message set size should not change", messages.size, validatedMessages.size) validatedMessages.foreach(messageAndOffset => validateLogAppendTime(messageAndOffset.message)) + assertEquals(s"Max timestamp should be $now", now, validatingResults.maxTimestamp) + assertEquals(s"The offset of max timestamp should be 0", 0, validatingResults.offsetOfMaxTimestamp) + assertFalse("Message size should not have been changed", validatingResults.messageSizeMaybeChanged) assertEquals("message set size should not change", compressedMessagesWithRecompresion.size, validatedCompressedMessages.size) validatedCompressedMessages.foreach(messageAndOffset => validateLogAppendTime(messageAndOffset.message)) assertTrue("MessageSet should still valid", validatedCompressedMessages.shallowIterator.next().message.isValid) + assertEquals(s"Max timestamp should be $now", now, validatingCompressedMessagesResults.maxTimestamp) + assertEquals(s"The offset of max timestamp should be ${compressedMessagesWithRecompresion.size - 1}", + compressedMessagesWithRecompresion.size - 1, validatingCompressedMessagesResults.offsetOfMaxTimestamp) + assertTrue("Message size may have been changed", validatingCompressedMessagesResults.messageSizeMaybeChanged) assertEquals("message set size should not change", compressedMessagesWithoutRecompression.size, validatedCompressedMessagesWithoutRecompression.size) validatedCompressedMessagesWithoutRecompression.foreach(messageAndOffset => validateLogAppendTime(messageAndOffset.message)) assertTrue("MessageSet should still valid", validatedCompressedMessagesWithoutRecompression.shallowIterator.next().message.isValid) + assertEquals(s"Max timestamp should be $now", now, validatingCompressedMessagesWithoutRecompressionResults.maxTimestamp) + assertEquals(s"The offset of max timestamp should be ${compressedMessagesWithoutRecompression.size - 1}", + compressedMessagesWithoutRecompression.size - 1, validatingCompressedMessagesWithoutRecompressionResults.offsetOfMaxTimestamp) + assertFalse("Message size should not have been changed", validatingCompressedMessagesWithoutRecompressionResults.messageSizeMaybeChanged) def validateLogAppendTime(message: Message) { message.ensureValid() - assertTrue(s"Timestamp of message $message should be between $startTime and $now", - message.timestamp >= startTime && message.timestamp <= now) + assertEquals(s"Timestamp of message $message should be $now", now, message.timestamp) assertEquals(TimestampType.LOG_APPEND_TIME, message.timestampType) } } @@ -209,18 +222,28 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { @Test def testCreateTime() { val now = System.currentTimeMillis() - val messages = getMessages(magicValue = Message.MagicValue_V1, timestamp = now, codec = NoCompressionCodec) - val compressedMessages = getMessages(magicValue = Message.MagicValue_V1, timestamp = now, codec = DefaultCompressionCodec) - - val (validatedMessages, _) = messages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(0), - now = System.currentTimeMillis(), - sourceCodec = NoCompressionCodec, - targetCodec = NoCompressionCodec, - messageFormatVersion = 1, - messageTimestampType = TimestampType.CREATE_TIME, - messageTimestampDiffMaxMs = 1000L) - - val (validatedCompressedMessages, _) = + val timestampSeq = Seq(now - 1, now + 1, now) + val messages = + new ByteBufferMessageSet(NoCompressionCodec, + new Message("hello".getBytes, timestamp = timestampSeq(0), magicValue = Message.MagicValue_V1), + new Message("there".getBytes, timestamp = timestampSeq(1), magicValue = Message.MagicValue_V1), + new Message("beautiful".getBytes, timestamp = timestampSeq(2), magicValue = Message.MagicValue_V1)) + val compressedMessages = + new ByteBufferMessageSet(DefaultCompressionCodec, + new Message("hello".getBytes, timestamp = timestampSeq(0), magicValue = Message.MagicValue_V1), + new Message("there".getBytes, timestamp = timestampSeq(1), magicValue = Message.MagicValue_V1), + new Message("beautiful".getBytes, timestamp = timestampSeq(2), magicValue = Message.MagicValue_V1)) + + val validatingResults = messages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(0), + now = System.currentTimeMillis(), + sourceCodec = NoCompressionCodec, + targetCodec = NoCompressionCodec, + messageFormatVersion = 1, + messageTimestampType = TimestampType.CREATE_TIME, + messageTimestampDiffMaxMs = 1000L) + val validatedMessages = validatingResults.validatedMessages + + val validatingCompressedMessagesResults = compressedMessages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(0), now = System.currentTimeMillis(), sourceCodec = DefaultCompressionCodec, @@ -228,17 +251,29 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { messageFormatVersion = 1, messageTimestampType = TimestampType.CREATE_TIME, messageTimestampDiffMaxMs = 1000L) + val validatedCompressedMessages = validatingCompressedMessagesResults.validatedMessages + var i = 0 for (messageAndOffset <- validatedMessages) { messageAndOffset.message.ensureValid() - assertEquals(messageAndOffset.message.timestamp, now) + assertEquals(messageAndOffset.message.timestamp, timestampSeq(i)) assertEquals(messageAndOffset.message.timestampType, TimestampType.CREATE_TIME) + i += 1 } + assertEquals(s"Max timestamp should be ${now + 1}", now + 1, validatingResults.maxTimestamp) + assertEquals(s"Offset of max timestamp should be 1", 1, validatingResults.offsetOfMaxTimestamp) + assertFalse("Message size should not have been changed", validatingResults.messageSizeMaybeChanged) + i = 0 for (messageAndOffset <- validatedCompressedMessages) { messageAndOffset.message.ensureValid() - assertEquals(messageAndOffset.message.timestamp, now) + assertEquals(messageAndOffset.message.timestamp, timestampSeq(i)) assertEquals(messageAndOffset.message.timestampType, TimestampType.CREATE_TIME) + i += 1 } + assertEquals(s"Max timestamp should be ${now + 1}", now + 1, validatingResults.maxTimestamp) + assertEquals(s"Offset of max timestamp should be ${validatedCompressedMessages.size - 1}", + validatedCompressedMessages.size - 1, validatingCompressedMessagesResults.offsetOfMaxTimestamp) + assertFalse("Message size should not have been changed", validatingCompressedMessagesResults.messageSizeMaybeChanged) } @Test @@ -287,7 +322,7 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { targetCodec = NoCompressionCodec, messageFormatVersion = 0, messageTimestampType = TimestampType.CREATE_TIME, - messageTimestampDiffMaxMs = 1000L)._1, offset) + messageTimestampDiffMaxMs = 1000L).validatedMessages, offset) // check compressed messages checkOffsets(compressedMessages, 0) @@ -297,7 +332,7 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { targetCodec = DefaultCompressionCodec, messageFormatVersion = 0, messageTimestampType = TimestampType.CREATE_TIME, - messageTimestampDiffMaxMs = 1000L)._1, offset) + messageTimestampDiffMaxMs = 1000L).validatedMessages, offset) } @@ -310,22 +345,22 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { // check uncompressed offsets checkOffsets(messages, 0) val offset = 1234567 - val (messageWithOffset, _) = messages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(offset), - now = System.currentTimeMillis(), - sourceCodec = NoCompressionCodec, - targetCodec = NoCompressionCodec, - messageTimestampType = TimestampType.CREATE_TIME, - messageTimestampDiffMaxMs = 5000L) + val messageWithOffset = messages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(offset), + now = System.currentTimeMillis(), + sourceCodec = NoCompressionCodec, + targetCodec = NoCompressionCodec, + messageTimestampType = TimestampType.CREATE_TIME, + messageTimestampDiffMaxMs = 5000L).validatedMessages checkOffsets(messageWithOffset, offset) // check compressed messages checkOffsets(compressedMessages, 0) - val (compressedMessagesWithOffset, _) = compressedMessages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(offset), - now = System.currentTimeMillis(), - sourceCodec = DefaultCompressionCodec, - targetCodec = DefaultCompressionCodec, - messageTimestampType = TimestampType.CREATE_TIME, - messageTimestampDiffMaxMs = 5000L) + val compressedMessagesWithOffset = compressedMessages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(offset), + now = System.currentTimeMillis(), + sourceCodec = DefaultCompressionCodec, + targetCodec = DefaultCompressionCodec, + messageTimestampType = TimestampType.CREATE_TIME, + messageTimestampDiffMaxMs = 5000L).validatedMessages checkOffsets(compressedMessagesWithOffset, offset) } @@ -343,7 +378,7 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { targetCodec = NoCompressionCodec, messageFormatVersion = 1, messageTimestampType = TimestampType.LOG_APPEND_TIME, - messageTimestampDiffMaxMs = 1000L)._1, offset) + messageTimestampDiffMaxMs = 1000L).validatedMessages, offset) // check compressed messages checkOffsets(compressedMessagesV0, 0) @@ -353,7 +388,7 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { targetCodec = DefaultCompressionCodec, messageFormatVersion = 1, messageTimestampType = TimestampType.LOG_APPEND_TIME, - messageTimestampDiffMaxMs = 1000L)._1, offset) + messageTimestampDiffMaxMs = 1000L).validatedMessages, offset) // Check down conversion val now = System.currentTimeMillis() @@ -368,7 +403,7 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { targetCodec = NoCompressionCodec, messageFormatVersion = 0, messageTimestampType = TimestampType.CREATE_TIME, - messageTimestampDiffMaxMs = 5000L)._1, offset) + messageTimestampDiffMaxMs = 5000L).validatedMessages, offset) // check compressed messages checkOffsets(compressedMessagesV1, 0) @@ -378,7 +413,7 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { targetCodec = DefaultCompressionCodec, messageFormatVersion = 0, messageTimestampType = TimestampType.CREATE_TIME, - messageTimestampDiffMaxMs = 5000L)._1, offset) + messageTimestampDiffMaxMs = 5000L).validatedMessages, offset) } @Test diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 05b84eff38e5..131a24a7f75f 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -274,8 +274,9 @@ object TestUtils extends Logging { def singleMessageSet(payload: Array[Byte], codec: CompressionCodec = NoCompressionCodec, key: Array[Byte] = null, + timestamp: Long = Message.NoTimestamp, magicValue: Byte = Message.CurrentMagicValue) = - new ByteBufferMessageSet(compressionCodec = codec, messages = new Message(payload, key, Message.NoTimestamp, magicValue)) + new ByteBufferMessageSet(compressionCodec = codec, messages = new Message(payload, key, timestamp, magicValue)) /** * Generate an array of random bytes diff --git a/docs/upgrade.html b/docs/upgrade.html index dfd20f446b9d..eef21cf3b6a8 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -15,10 +15,24 @@ limitations under the License. --> + +

    1.5 Upgrading From Previous Versions

    -

    Notable changes in 0.10.1.0
    +

    Upgrading from 0.10.0.X to 0.10.1.0

    +0.10.1.0 is compatible with 0.10.0.X in terms of wire protocol. The upgrade can be done one broker at a time by simply bringing it down, updating the code, and restarting it. +However, please notice the Potential breaking changes in 0.10.1.0 before upgrade. +
    Potential breaking changes in 0.10.1.0
    +
      +
    • The log retention time is no longer based on last modified time of the log segments. Instead it will be based on the largest timestamp of the messages in a log segment.
    • +
    • The log rolling time is no longer depending on log segment create time. Instead it is now based on the timestamp of the first message in a log segment. i.e. if the timestamp of the first message in the segment is T, the log will be rolled out at T + log.roll.ms
    • +
    • The open file handlers of 0.10.0 will increase by ~33% because of the addition of time index files for each segment.
    • +
    • The time index and offset index share the same index size configuration. Since each time index entry is 1.5x the size of offset index entry. User may need to increase log.index.size.max.bytes to avoid potential frequent log rolling.
    • +
    • Due to the increased number of index files, on some brokers with large amount the log segments (e.g. >15K), the log loading process during the broker startup could be longer. Based on our experiment, setting the num.recovery.threads.per.data.dir to one may reduce the log loading time.
    • +
    + +
    Notable changes in 0.10.1.0
    • The BrokerState "RunningAsController" (value 4) has been removed. Due to a bug, a broker would only be in this state briefly before transitioning out of it and hence the impact of the removal should be minimal. The recommended way to detect if a given broker is the controller is via the kafka.controller:type=KafkaController,name=ActiveControllerCount metric.
    From 104d2154b635c50efc80331ee2a4779cc3658414 Mon Sep 17 00:00:00 2001 From: Todd Palino Date: Fri, 19 Aug 2016 11:05:39 -0700 Subject: [PATCH 11/35] KAFKA-4050; Allow configuration of the PRNG used for SSL Add an optional configuration for the SecureRandom PRNG implementation, with the default behavior being the same (use the default implementation in the JDK/JRE). Author: Todd Palino Reviewers: Grant Henke , Ismael Juma , Joel Koshy , Jiangjie Qin , Rajini Sivaram Closes #1747 from toddpalino/trunk --- .../apache/kafka/common/config/SslConfigs.java | 6 +++++- .../kafka/common/security/ssl/SslFactory.java | 13 ++++++++++++- .../common/network/SslTransportLayerTest.java | 17 ++++++++++++++++- .../main/scala/kafka/server/KafkaConfig.scala | 3 +++ .../unit/kafka/server/KafkaConfigTest.scala | 1 + docs/security.html | 9 +++++++++ .../kafkatest/services/kafka/config_property.py | 1 + 7 files changed, 47 insertions(+), 3 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/SslConfigs.java b/clients/src/main/java/org/apache/kafka/common/config/SslConfigs.java index 1ccd039fb12b..ba1ff6b23126 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/SslConfigs.java +++ b/clients/src/main/java/org/apache/kafka/common/config/SslConfigs.java @@ -85,6 +85,9 @@ public class SslConfigs { public static final String SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG = "ssl.endpoint.identification.algorithm"; public static final String SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_DOC = "The endpoint identification algorithm to validate server hostname using server certificate. "; + public static final String SSL_SECURE_RANDOM_IMPLEMENTATION_CONFIG = "ssl.secure.random.implementation"; + public static final String SSL_SECURE_RANDOM_IMPLEMENTATION_DOC = "The SecureRandom PRNG implementation to use for SSL cryptography operations. "; + public static final String SSL_CLIENT_AUTH_CONFIG = "ssl.client.auth"; public static final String SSL_CLIENT_AUTH_DOC = "Configures kafka broker to request client authentication." + " The following settings are common: " @@ -109,6 +112,7 @@ public static void addClientSslSupport(ConfigDef config) { .define(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, ConfigDef.Type.PASSWORD, null, ConfigDef.Importance.HIGH, SslConfigs.SSL_TRUSTSTORE_PASSWORD_DOC) .define(SslConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG, ConfigDef.Type.STRING, SslConfigs.DEFAULT_SSL_KEYMANGER_ALGORITHM, ConfigDef.Importance.LOW, SslConfigs.SSL_KEYMANAGER_ALGORITHM_DOC) .define(SslConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG, ConfigDef.Type.STRING, SslConfigs.DEFAULT_SSL_TRUSTMANAGER_ALGORITHM, ConfigDef.Importance.LOW, SslConfigs.SSL_TRUSTMANAGER_ALGORITHM_DOC) - .define(SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, ConfigDef.Type.STRING, null, ConfigDef.Importance.LOW, SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_DOC); + .define(SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, ConfigDef.Type.STRING, null, ConfigDef.Importance.LOW, SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_DOC) + .define(SslConfigs.SSL_SECURE_RANDOM_IMPLEMENTATION_CONFIG, ConfigDef.Type.STRING, null, ConfigDef.Importance.LOW, SslConfigs.SSL_SECURE_RANDOM_IMPLEMENTATION_DOC); } } diff --git a/clients/src/main/java/org/apache/kafka/common/security/ssl/SslFactory.java b/clients/src/main/java/org/apache/kafka/common/security/ssl/SslFactory.java index d0fe2e8694fc..ee7b65b2c680 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/ssl/SslFactory.java +++ b/clients/src/main/java/org/apache/kafka/common/security/ssl/SslFactory.java @@ -26,6 +26,7 @@ import java.io.IOException; import java.security.GeneralSecurityException; import java.security.KeyStore; +import java.security.SecureRandom; import java.util.List; import java.util.Map; @@ -51,6 +52,7 @@ public class SslFactory implements Configurable { private String[] cipherSuites; private String[] enabledProtocols; private String endpointIdentification; + private SecureRandom secureRandomImplementation; private SSLContext sslContext; private boolean needClientAuth; private boolean wantClientAuth; @@ -83,6 +85,15 @@ public void configure(Map configs) throws KafkaException { if (endpointIdentification != null) this.endpointIdentification = endpointIdentification; + String secureRandomImplementation = (String) configs.get(SslConfigs.SSL_SECURE_RANDOM_IMPLEMENTATION_CONFIG); + if (secureRandomImplementation != null) { + try { + this.secureRandomImplementation = SecureRandom.getInstance(secureRandomImplementation); + } catch (GeneralSecurityException e) { + throw new KafkaException(e); + } + } + String clientAuthConfig = clientAuthConfigOverride; if (clientAuthConfig == null) clientAuthConfig = (String) configs.get(SslConfigs.SSL_CLIENT_AUTH_CONFIG); @@ -134,7 +145,7 @@ private SSLContext createSSLContext() throws GeneralSecurityException, IOExcepti KeyStore ts = truststore == null ? null : truststore.load(); tmf.init(ts); - sslContext.init(keyManagers, tmf.getTrustManagers(), null); + sslContext.init(keyManagers, tmf.getTrustManagers(), this.secureRandomImplementation); return sslContext; } diff --git a/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java b/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java index 4e96411c362a..a044dc97b1b2 100644 --- a/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java +++ b/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java @@ -252,7 +252,22 @@ public void testClientAuthenticationRequestedNotProvided() throws Exception { NetworkTestUtils.checkClientConnection(selector, node, 100, 10); } - + + /** + * Tests that an invalid SecureRandom implementation cannot be configured + */ + @Test + public void testInvalidSecureRandomImplementation() throws Exception { + SslChannelBuilder channelBuilder = new SslChannelBuilder(Mode.CLIENT); + try { + sslClientConfigs.put(SslConfigs.SSL_SECURE_RANDOM_IMPLEMENTATION_CONFIG, "invalid"); + channelBuilder.configure(sslClientConfigs); + fail("SSL channel configured with invalid SecureRandom implementation"); + } catch (KafkaException e) { + // Expected exception + } + } + /** * Tests that channels cannot be created if truststore cannot be loaded */ diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 665568115b32..b31f5965a4b8 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -336,6 +336,7 @@ object KafkaConfig { val SslKeyManagerAlgorithmProp = SslConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG val SslTrustManagerAlgorithmProp = SslConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG val SslEndpointIdentificationAlgorithmProp = SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG + val SslSecureRandomImplementationProp = SslConfigs.SSL_SECURE_RANDOM_IMPLEMENTATION_CONFIG val SslClientAuthProp = SslConfigs.SSL_CLIENT_AUTH_CONFIG /** ********* SASL Configuration ****************/ @@ -544,6 +545,7 @@ object KafkaConfig { val SslKeyManagerAlgorithmDoc = SslConfigs.SSL_KEYMANAGER_ALGORITHM_DOC val SslTrustManagerAlgorithmDoc = SslConfigs.SSL_TRUSTMANAGER_ALGORITHM_DOC val SslEndpointIdentificationAlgorithmDoc = SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_DOC + val SslSecureRandomImplementationDoc = SslConfigs.SSL_SECURE_RANDOM_IMPLEMENTATION_DOC val SslClientAuthDoc = SslConfigs.SSL_CLIENT_AUTH_DOC /** ********* Sasl Configuration ****************/ @@ -716,6 +718,7 @@ object KafkaConfig { .define(SslKeyManagerAlgorithmProp, STRING, Defaults.SslKeyManagerAlgorithm, MEDIUM, SslKeyManagerAlgorithmDoc) .define(SslTrustManagerAlgorithmProp, STRING, Defaults.SslTrustManagerAlgorithm, MEDIUM, SslTrustManagerAlgorithmDoc) .define(SslEndpointIdentificationAlgorithmProp, STRING, null, LOW, SslEndpointIdentificationAlgorithmDoc) + .define(SslSecureRandomImplementationProp, STRING, null, LOW, SslSecureRandomImplementationDoc) .define(SslClientAuthProp, STRING, Defaults.SslClientAuth, in(Defaults.SslClientAuthRequired, Defaults.SslClientAuthRequested, Defaults.SslClientAuthNone), MEDIUM, SslClientAuthDoc) .define(SslCipherSuitesProp, LIST, null, MEDIUM, SslCipherSuitesDoc) diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index f8476cd303be..eb4c0eab4fd1 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -547,6 +547,7 @@ class KafkaConfigTest { case KafkaConfig.SslTrustManagerAlgorithmProp => case KafkaConfig.SslClientAuthProp => // ignore string case KafkaConfig.SslEndpointIdentificationAlgorithmProp => // ignore string + case KafkaConfig.SslSecureRandomImplementationProp => // ignore string case KafkaConfig.SslCipherSuitesProp => // ignore string //Sasl Configs diff --git a/docs/security.html b/docs/security.html index 53d6cf1f1192..0a5e5617f45b 100644 --- a/docs/security.html +++ b/docs/security.html @@ -145,6 +145,7 @@

    7.2 Encryption and Authentication
  • ssl.enabled.protocols=TLSv1.2,TLSv1.1,TLSv1 (list out the SSL protocols that you are going to accept from clients. Do note that SSL is deprecated in favor of TLS and using SSL in production is not recommended)
  • ssl.keystore.type=JKS
  • ssl.truststore.type=JKS
  • +
  • ssl.secure.random.implementation=SHA1PRNG
  • If you want to enable SSL for inter-broker communication, add the following to the broker properties file (it defaults to PLAINTEXT)
    @@ -155,6 +156,14 @@ 

    7.2 Encryption and Authentication JCA Providers Documentation for more information.

    +

    + The JRE/JDK will have a default pseudo-random number generator (PRNG) that is used for cryptography operations, so it is not required to configure the + implementation used with the

    ssl.secure.random.implementation
    . However, there are performance issues with some implementations (notably, the + default chosen on Linux systems,
    NativePRNG
    , utilizes a global lock). In cases where performance of SSL connections becomes an issue, + consider explicitly setting the implementation to be used. The
    SHA1PRNG
    implementation is non-blocking, and has shown very good performance + characteristics under heavy load (50 MB/sec of produced messages, plus replication traffic, per-broker). +

    + Once you start the broker you should be able to see in the server.log
             with addresses: PLAINTEXT -> EndPoint(192.168.64.1,9092,PLAINTEXT),SSL -> EndPoint(192.168.64.1,9093,SSL)
    diff --git a/tests/kafkatest/services/kafka/config_property.py b/tests/kafkatest/services/kafka/config_property.py index e1801efa9293..217e9703e883 100644 --- a/tests/kafkatest/services/kafka/config_property.py +++ b/tests/kafkatest/services/kafka/config_property.py @@ -173,6 +173,7 @@ val SSLKeyManagerAlgorithmProp = SSLConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG val SSLTrustManagerAlgorithmProp = SSLConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG val SSLEndpointIdentificationAlgorithmProp = SSLConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG + val SSLSecureRandomImplementationProp = SSLConfigs.SSL_SECURE_RANDOM_IMPLEMENTATION_CONFIG val SSLClientAuthProp = SSLConfigs.SSL_CLIENT_AUTH_CONFIG """ From 880bde5ac5644c904eacc1c4a53159a58d107254 Mon Sep 17 00:00:00 2001 From: Mickael Maison Date: Fri, 19 Aug 2016 11:29:54 -0700 Subject: [PATCH 12/35] KAFKA-4056: Kafka logs values of sensitive configs like passwords In case of unknown configs, only list the name without the value Author: Mickael Maison Reviewers: Jaikiran, Gwen Shapira, Grant Henke, Ryan Pridgeon, Dustin Cote Closes #1759 from mimaison/KAFKA-4056 --- .../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 91c5a9f4d93a..9f3cba4faa36 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 @@ -185,7 +185,7 @@ private void logAll() { */ public void logUnused() { for (String key : unused()) - log.warn("The configuration {} = {} was supplied but isn't a known config.", key, this.originals.get(key)); + log.warn("The configuration '{}' was supplied but isn't a known config.", key); } /** From 33447cb4beb8dd710a20556a6a2baf8307f03860 Mon Sep 17 00:00:00 2001 From: Shuai Zhang Date: Fri, 19 Aug 2016 11:59:11 -0700 Subject: [PATCH 13/35] KAFKA-4053: remove redundant if/else statements in TopicCommand Author: Shuai Zhang Reviewers: Gwen Shapira Closes #1751 from sh-z/KAFKA-4053 --- core/src/main/scala/kafka/admin/TopicCommand.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala index 657f26cd46f3..421486c07aa6 100755 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -91,7 +91,7 @@ object TopicCommand extends Logging { def createTopic(zkUtils: ZkUtils, opts: TopicCommandOptions) { val topic = opts.options.valueOf(opts.topicOpt) val configs = parseTopicConfigsToBeAdded(opts) - val ifNotExists = if (opts.options.has(opts.ifNotExistsOpt)) true else false + val ifNotExists = opts.options.has(opts.ifNotExistsOpt) 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.") try { @@ -116,7 +116,7 @@ object TopicCommand extends Logging { def alterTopic(zkUtils: ZkUtils, opts: TopicCommandOptions) { val topics = getTopics(zkUtils, opts) - val ifExists = if (opts.options.has(opts.ifExistsOpt)) true else false + val ifExists = opts.options.has(opts.ifExistsOpt) if (topics.isEmpty && !ifExists) { throw new IllegalArgumentException("Topic %s does not exist on ZK path %s".format(opts.options.valueOf(opts.topicOpt), opts.options.valueOf(opts.zkConnectOpt))) @@ -163,7 +163,7 @@ object TopicCommand extends Logging { def deleteTopic(zkUtils: ZkUtils, opts: TopicCommandOptions) { val topics = getTopics(zkUtils, opts) - val ifExists = if (opts.options.has(opts.ifExistsOpt)) true else false + val ifExists = opts.options.has(opts.ifExistsOpt) if (topics.isEmpty && !ifExists) { throw new IllegalArgumentException("Topic %s does not exist on ZK path %s".format(opts.options.valueOf(opts.topicOpt), opts.options.valueOf(opts.zkConnectOpt))) @@ -190,9 +190,9 @@ object TopicCommand extends Logging { def describeTopic(zkUtils: ZkUtils, opts: TopicCommandOptions) { val topics = getTopics(zkUtils, opts) - val reportUnderReplicatedPartitions = if (opts.options.has(opts.reportUnderReplicatedPartitionsOpt)) true else false - val reportUnavailablePartitions = if (opts.options.has(opts.reportUnavailablePartitionsOpt)) true else false - val reportOverriddenConfigs = if (opts.options.has(opts.topicsWithOverridesOpt)) true else false + val reportUnderReplicatedPartitions = opts.options.has(opts.reportUnderReplicatedPartitionsOpt) + val reportUnavailablePartitions = opts.options.has(opts.reportUnavailablePartitionsOpt) + val reportOverriddenConfigs = opts.options.has(opts.topicsWithOverridesOpt) val liveBrokers = zkUtils.getAllBrokersInCluster().map(_.id).toSet for (topic <- topics) { zkUtils.getPartitionAssignmentForTopics(List(topic)).get(topic) match { From c5d26c4829583c95af7ca9e961a4d3954f8e09eb Mon Sep 17 00:00:00 2001 From: Eno Thereska Date: Fri, 19 Aug 2016 15:23:30 -0700 Subject: [PATCH 14/35] KAFKA-4016: Added join benchmarks Author: Eno Thereska Reviewers: Ismael Juma, Damian Guy, Guozhang Wang Closes #1700 from enothereska/join-benchmarks --- .../kafka/streams/perf/SimpleBenchmark.java | 325 ++++++++++++++---- .../streams/streams_simple_benchmark_test.py | 2 +- .../performance/streams_performance.py | 6 +- 3 files changed, 270 insertions(+), 63 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java b/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java index a4b5345ad411..54954167759d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java +++ b/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java @@ -28,14 +28,18 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.serialization.ByteArraySerializer; -import org.apache.kafka.common.serialization.LongDeserializer; -import org.apache.kafka.common.serialization.LongSerializer; +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.kstream.ForeachAction; +import org.apache.kafka.streams.kstream.JoinWindows; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.streams.kstream.KTable; +import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.ProcessorSupplier; @@ -47,6 +51,7 @@ import java.util.List; import java.util.concurrent.CountDownLatch; import java.util.Properties; +import java.util.Random; public class SimpleBenchmark { @@ -57,14 +62,33 @@ public class SimpleBenchmark { private static final String SOURCE_TOPIC = "simpleBenchmarkSourceTopic"; private static final String SINK_TOPIC = "simpleBenchmarkSinkTopic"; - private static final long NUM_RECORDS = 10000000L; - private static final Long END_KEY = NUM_RECORDS - 1; + private static final String JOIN_TOPIC_1_PREFIX = "joinSourceTopic1"; + private static final String JOIN_TOPIC_2_PREFIX = "joinSourceTopic2"; + private static final ValueJoiner VALUE_JOINER = new ValueJoiner() { + @Override + public byte[] apply(final byte[] value1, final byte[] value2) { + if (value1 == null && value2 == null) + return new byte[VALUE_SIZE]; + if (value1 == null && value2 != null) + return value2; + if (value1 != null && value2 == null) + return value1; + + byte[] tmp = new byte[value1.length + value2.length]; + System.arraycopy(value1, 0, tmp, 0, value1.length); + System.arraycopy(value2, 0, tmp, value1.length, value2.length); + return tmp; + } + }; + + private static int numRecords; + private static Integer endKey; private static final int KEY_SIZE = 8; private static final int VALUE_SIZE = 100; private static final int RECORD_SIZE = KEY_SIZE + VALUE_SIZE; private static final Serde BYTE_SERDE = Serdes.ByteArray(); - private static final Serde LONG_SERDE = Serdes.Long(); + private static final Serde INTEGER_SERDE = Serdes.Integer(); public SimpleBenchmark(File stateDir, String kafka, String zookeeper) { super(); @@ -77,6 +101,8 @@ public static void main(String[] args) throws Exception { String kafka = args.length > 0 ? args[0] : "localhost:9092"; String zookeeper = args.length > 1 ? args[1] : "localhost:2181"; String stateDirStr = args.length > 2 ? args[2] : "/tmp/kafka-streams-simple-benchmark"; + numRecords = args.length > 3 ? Integer.parseInt(args[3]) : 10000000; + endKey = numRecords - 1; final File stateDir = new File(stateDirStr); stateDir.mkdir(); @@ -88,25 +114,130 @@ public static void main(String[] args) throws Exception { System.out.println("kafka=" + kafka); System.out.println("zookeeper=" + zookeeper); System.out.println("stateDir=" + stateDir); + System.out.println("numRecords=" + numRecords); SimpleBenchmark benchmark = new SimpleBenchmark(stateDir, kafka, zookeeper); // producer performance - benchmark.produce(); + benchmark.produce(SOURCE_TOPIC, VALUE_SIZE, "simple-benchmark-produce", numRecords, true, numRecords, true); // consumer performance - benchmark.consume(); + benchmark.consume(SOURCE_TOPIC); // simple stream performance source->process - benchmark.processStream(); + benchmark.processStream(SOURCE_TOPIC); // simple stream performance source->sink - benchmark.processStreamWithSink(); + benchmark.processStreamWithSink(SOURCE_TOPIC); // simple stream performance source->store - benchmark.processStreamWithStateStore(); + benchmark.processStreamWithStateStore(SOURCE_TOPIC); + // simple streams performance KSTREAM-KTABLE join + benchmark.kStreamKTableJoin(JOIN_TOPIC_1_PREFIX + "kStreamKTable", JOIN_TOPIC_2_PREFIX + "kStreamKTable"); + // simple streams performance KSTREAM-KSTREAM join + benchmark.kStreamKStreamJoin(JOIN_TOPIC_1_PREFIX + "kStreamKStream", JOIN_TOPIC_2_PREFIX + "kStreamKStream"); + // simple streams performance KTABLE-KTABLE join + benchmark.kTableKTableJoin(JOIN_TOPIC_1_PREFIX + "kTableKTable", JOIN_TOPIC_2_PREFIX + "kTableKTable"); + } + + private Properties setJoinProperties(final String applicationId) { + Properties props = new Properties(); + props.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId); + props.put(StreamsConfig.STATE_DIR_CONFIG, stateDir.toString()); + props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); + props.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, zookeeper); + props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1); + props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + props.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); + props.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serdes.ByteArray().getClass()); + return props; + } + + /** + * Measure the performance of a KStream-KTable left join. The setup is such that each + * KStream record joins to exactly one element in the KTable + */ + public void kStreamKTableJoin(String kStreamTopic, String kTableTopic) throws Exception { + CountDownLatch latch = new CountDownLatch(numRecords); + + // initialize topics + System.out.println("Initializing kStreamTopic " + kStreamTopic); + produce(kStreamTopic, VALUE_SIZE, "simple-benchmark-produce-kstream", numRecords, false, numRecords, false); + System.out.println("Initializing kTableTopic " + kTableTopic); + produce(kTableTopic, VALUE_SIZE, "simple-benchmark-produce-ktable", numRecords, true, numRecords, false); + + // setup join + Properties props = setJoinProperties("simple-benchmark-kstream-ktable-join"); + final KafkaStreams streams = createKafkaStreamsKStreamKTableJoin(props, kStreamTopic, kTableTopic, latch); + + // run benchmark + runJoinBenchmark(streams, "Streams KStreamKTable LeftJoin Performance [MB/s joined]: ", latch); + } + + /** + * Measure the performance of a KStream-KStream left join. The setup is such that each + * KStream record joins to exactly one element in the other KStream + */ + public void kStreamKStreamJoin(String kStreamTopic1, String kStreamTopic2) throws Exception { + CountDownLatch latch = new CountDownLatch(numRecords); + + // initialize topics + System.out.println("Initializing kStreamTopic " + kStreamTopic1); + produce(kStreamTopic1, VALUE_SIZE, "simple-benchmark-produce-kstream-topic1", numRecords, true, numRecords, false); + System.out.println("Initializing kStreamTopic " + kStreamTopic2); + produce(kStreamTopic2, VALUE_SIZE, "simple-benchmark-produce-kstream-topic2", numRecords, true, numRecords, false); + + // setup join + Properties props = setJoinProperties("simple-benchmark-kstream-kstream-join"); + final KafkaStreams streams = createKafkaStreamsKStreamKStreamJoin(props, kStreamTopic1, kStreamTopic2, latch); + + // run benchmark + runJoinBenchmark(streams, "Streams KStreamKStream LeftJoin Performance [MB/s joined]: ", latch); + } + + /** + * Measure the performance of a KTable-KTable left join. The setup is such that each + * KTable record joins to exactly one element in the other KTable + */ + public void kTableKTableJoin(String kTableTopic1, String kTableTopic2) throws Exception { + CountDownLatch latch = new CountDownLatch(numRecords); + + // initialize topics + System.out.println("Initializing kTableTopic " + kTableTopic1); + produce(kTableTopic1, VALUE_SIZE, "simple-benchmark-produce-ktable-topic1", numRecords, true, numRecords, false); + System.out.println("Initializing kTableTopic " + kTableTopic2); + produce(kTableTopic2, VALUE_SIZE, "simple-benchmark-produce-ktable-topic2", numRecords, true, numRecords, false); + + // setup join + Properties props = setJoinProperties("simple-benchmark-ktable-ktable-join"); + final KafkaStreams streams = createKafkaStreamsKTableKTableJoin(props, kTableTopic1, kTableTopic2, latch); + + // run benchmark + runJoinBenchmark(streams, "Streams KTableKTable LeftJoin Performance [MB/s joined]: ", latch); + } + + private void runJoinBenchmark(final KafkaStreams streams, final String nameOfBenchmark, final CountDownLatch latch) { + streams.start(); + + long startTime = System.currentTimeMillis(); + + while (latch.getCount() > 0) { + try { + latch.await(); + } catch (InterruptedException ex) { + //ignore + } + } + long endTime = System.currentTimeMillis(); + + + System.out.println(nameOfBenchmark + megaBytePerSec(endTime - startTime, numRecords, KEY_SIZE + VALUE_SIZE)); + + streams.close(); } - public void processStream() { + + + public void processStream(String topic) { CountDownLatch latch = new CountDownLatch(1); - final KafkaStreams streams = createKafkaStreams(stateDir, kafka, zookeeper, latch); + final KafkaStreams streams = createKafkaStreams(topic, stateDir, kafka, zookeeper, latch); Thread thread = new Thread() { public void run() { @@ -137,10 +268,10 @@ public void run() { } } - public void processStreamWithSink() { + public void processStreamWithSink(String topic) { CountDownLatch latch = new CountDownLatch(1); - final KafkaStreams streams = createKafkaStreamsWithSink(stateDir, kafka, zookeeper, latch); + final KafkaStreams streams = createKafkaStreamsWithSink(topic, stateDir, kafka, zookeeper, latch); Thread thread = new Thread() { public void run() { @@ -171,10 +302,10 @@ public void run() { } } - public void processStreamWithStateStore() { + public void processStreamWithStateStore(String topic) { CountDownLatch latch = new CountDownLatch(1); - final KafkaStreams streams = createKafkaStreamsWithStateStore(stateDir, kafka, zookeeper, latch); + final KafkaStreams streams = createKafkaStreamsWithStateStore(topic, stateDir, kafka, zookeeper, latch); Thread thread = new Thread() { public void run() { @@ -205,54 +336,76 @@ public void run() { } } - public void produce() { + /** + * Produce values to a topic + * @param topic Topic to produce to + * @param valueSizeBytes Size of value in bytes + * @param clientId String specifying client ID + * @param numRecords Number of records to produce + * @param sequential if True, then keys are produced sequentially from 0 to upperRange. In this case upperRange must be >= numRecords. + * if False, then keys are produced randomly in range [0, upperRange) + * @param printStats if True, print stats on how long producing took. If False, don't print stats. False can be used + * when this produce step is part of another benchmark that produces its own stats + */ + public void produce(String topic, int valueSizeBytes, String clientId, int numRecords, boolean sequential, + int upperRange, boolean printStats) throws Exception { + + if (sequential) { + if (upperRange < numRecords) throw new Exception("UpperRange must be >= numRecords"); + } Properties props = new Properties(); - props.put(ProducerConfig.CLIENT_ID_CONFIG, "simple-benchmark-produce"); + props.put(ProducerConfig.CLIENT_ID_CONFIG, clientId); props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); - props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, LongSerializer.class); + props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, IntegerSerializer.class); props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); + int key = 0; + Random rand = new Random(); + KafkaProducer producer = new KafkaProducer<>(props); - KafkaProducer producer = new KafkaProducer<>(props); - - byte[] value = new byte[VALUE_SIZE]; + byte[] value = new byte[valueSizeBytes]; long startTime = System.currentTimeMillis(); - for (int i = 0; i < NUM_RECORDS; i++) { - producer.send(new ProducerRecord<>(SOURCE_TOPIC, (long) i, value)); + if (sequential) key = 0; + else key = rand.nextInt(upperRange); + for (int i = 0; i < numRecords; i++) { + producer.send(new ProducerRecord<>(topic, key, value)); + if (sequential) key++; + else key = rand.nextInt(upperRange); } producer.close(); long endTime = System.currentTimeMillis(); - System.out.println("Producer Performance [MB/sec write]: " + megaBytePerSec(endTime - startTime)); + if (printStats) + System.out.println("Producer Performance [MB/sec write]: " + megaBytePerSec(endTime - startTime, numRecords, KEY_SIZE + valueSizeBytes)); } - public void consume() { + public void consume(String topic) { Properties props = new Properties(); props.put(ConsumerConfig.CLIENT_ID_CONFIG, "simple-benchmark-consumer"); props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); - props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, LongDeserializer.class); + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, IntegerDeserializer.class); props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class); props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); - KafkaConsumer consumer = new KafkaConsumer<>(props); + KafkaConsumer consumer = new KafkaConsumer<>(props); - List partitions = getAllPartitions(consumer, SOURCE_TOPIC); + List partitions = getAllPartitions(consumer, topic); consumer.assign(partitions); consumer.seekToBeginning(partitions); - Long key = null; + Integer key = null; long startTime = System.currentTimeMillis(); while (true) { - ConsumerRecords records = consumer.poll(500); + ConsumerRecords records = consumer.poll(500); if (records.isEmpty()) { - if (END_KEY.equals(key)) + if (endKey.equals(key)) break; } else { - for (ConsumerRecord record : records) { - Long recKey = record.key(); + for (ConsumerRecord record : records) { + Integer recKey = record.key(); if (key == null || key < recKey) key = recKey; @@ -266,7 +419,7 @@ public void consume() { System.out.println("Consumer Performance [MB/sec read]: " + megaBytePerSec(endTime - startTime)); } - private KafkaStreams createKafkaStreams(File stateDir, String kafka, String zookeeper, final CountDownLatch latch) { + private KafkaStreams createKafkaStreams(String topic, File stateDir, String kafka, String zookeeper, final CountDownLatch latch) { Properties props = new Properties(); props.put(StreamsConfig.APPLICATION_ID_CONFIG, "simple-benchmark-streams"); props.put(StreamsConfig.STATE_DIR_CONFIG, stateDir.toString()); @@ -277,20 +430,20 @@ private KafkaStreams createKafkaStreams(File stateDir, String kafka, String zook KStreamBuilder builder = new KStreamBuilder(); - KStream source = builder.stream(LONG_SERDE, BYTE_SERDE, SOURCE_TOPIC); + KStream source = builder.stream(INTEGER_SERDE, BYTE_SERDE, topic); - source.process(new ProcessorSupplier() { + source.process(new ProcessorSupplier() { @Override - public Processor get() { - return new Processor() { + public Processor get() { + return new Processor() { @Override public void init(ProcessorContext context) { } @Override - public void process(Long key, byte[] value) { - if (END_KEY.equals(key)) { + public void process(Integer key, byte[] value) { + if (endKey.equals(key)) { latch.countDown(); } } @@ -309,7 +462,7 @@ public void close() { return new KafkaStreams(builder, props); } - private KafkaStreams createKafkaStreamsWithSink(File stateDir, String kafka, String zookeeper, final CountDownLatch latch) { + private KafkaStreams createKafkaStreamsWithSink(String topic, File stateDir, String kafka, String zookeeper, final CountDownLatch latch) { Properties props = new Properties(); props.put(StreamsConfig.APPLICATION_ID_CONFIG, "simple-benchmark-streams-with-sink"); props.put(StreamsConfig.STATE_DIR_CONFIG, stateDir.toString()); @@ -320,21 +473,21 @@ private KafkaStreams createKafkaStreamsWithSink(File stateDir, String kafka, Str KStreamBuilder builder = new KStreamBuilder(); - KStream source = builder.stream(LONG_SERDE, BYTE_SERDE, SOURCE_TOPIC); + KStream source = builder.stream(INTEGER_SERDE, BYTE_SERDE, topic); - source.to(LONG_SERDE, BYTE_SERDE, SINK_TOPIC); - source.process(new ProcessorSupplier() { + source.to(INTEGER_SERDE, BYTE_SERDE, SINK_TOPIC); + source.process(new ProcessorSupplier() { @Override - public Processor get() { - return new Processor() { + public Processor get() { + return new Processor() { @Override public void init(ProcessorContext context) { } @Override - public void process(Long key, byte[] value) { - if (END_KEY.equals(key)) { + public void process(Integer key, byte[] value) { + if (endKey.equals(key)) { latch.countDown(); } } @@ -353,8 +506,56 @@ public void close() { return new KafkaStreams(builder, props); } + private class CountDownAction implements ForeachAction { + private CountDownLatch latch; + CountDownAction(final CountDownLatch latch) { + this.latch = latch; + } + @Override + public void apply(K key, V value) { + this.latch.countDown(); + } + } + + private KafkaStreams createKafkaStreamsKStreamKTableJoin(Properties streamConfig, String kStreamTopic, + String kTableTopic, final CountDownLatch latch) { + final KStreamBuilder builder = new KStreamBuilder(); + + final KStream input1 = builder.stream(kStreamTopic); + final KTable input2 = builder.table(kTableTopic, kTableTopic + "-store"); + + input1.leftJoin(input2, VALUE_JOINER).foreach(new CountDownAction(latch)); + + return new KafkaStreams(builder, streamConfig); + } + + private KafkaStreams createKafkaStreamsKTableKTableJoin(Properties streamConfig, String kTableTopic1, + String kTableTopic2, final CountDownLatch latch) { + final KStreamBuilder builder = new KStreamBuilder(); + + final KTable input1 = builder.table(kTableTopic1, kTableTopic1 + "-store"); + final KTable input2 = builder.table(kTableTopic2, kTableTopic2 + "-store"); + + input1.leftJoin(input2, VALUE_JOINER).foreach(new CountDownAction(latch)); + + return new KafkaStreams(builder, streamConfig); + } + + private KafkaStreams createKafkaStreamsKStreamKStreamJoin(Properties streamConfig, String kStreamTopic1, + String kStreamTopic2, final CountDownLatch latch) { + final KStreamBuilder builder = new KStreamBuilder(); + + final KStream input1 = builder.stream(kStreamTopic1); + final KStream input2 = builder.stream(kStreamTopic2); + final long timeDifferenceMs = 10000L; - private KafkaStreams createKafkaStreamsWithStateStore(File stateDir, String kafka, String zookeeper, final CountDownLatch latch) { + input1.leftJoin(input2, VALUE_JOINER, JoinWindows.of(timeDifferenceMs)).foreach(new CountDownAction(latch)); + + return new KafkaStreams(builder, streamConfig); + } + + private KafkaStreams createKafkaStreamsWithStateStore(String topic, File stateDir, String kafka, String zookeeper, + final CountDownLatch latch) { Properties props = new Properties(); props.put(StreamsConfig.APPLICATION_ID_CONFIG, "simple-benchmark-streams-with-store"); props.put(StreamsConfig.STATE_DIR_CONFIG, stateDir.toString()); @@ -365,28 +566,28 @@ private KafkaStreams createKafkaStreamsWithStateStore(File stateDir, String kafk KStreamBuilder builder = new KStreamBuilder(); - builder.addStateStore(Stores.create("store").withLongKeys().withByteArrayValues().persistent().build()); + builder.addStateStore(Stores.create("store").withIntegerKeys().withByteArrayValues().persistent().build()); - KStream source = builder.stream(LONG_SERDE, BYTE_SERDE, SOURCE_TOPIC); + KStream source = builder.stream(INTEGER_SERDE, BYTE_SERDE, topic); - source.process(new ProcessorSupplier() { + source.process(new ProcessorSupplier() { @Override - public Processor get() { - return new Processor() { + public Processor get() { + return new Processor() { - KeyValueStore store; + KeyValueStore store; @SuppressWarnings("unchecked") @Override public void init(ProcessorContext context) { - store = (KeyValueStore) context.getStateStore("store"); + store = (KeyValueStore) context.getStateStore("store"); } @Override - public void process(Long key, byte[] value) { + public void process(Integer key, byte[] value) { store.put(key, value); - if (END_KEY.equals(key)) { + if (endKey.equals(key)) { latch.countDown(); } } @@ -406,7 +607,11 @@ public void close() { } private double megaBytePerSec(long time) { - return (double) (RECORD_SIZE * NUM_RECORDS / 1024 / 1024) / ((double) time / 1000); + return (double) (RECORD_SIZE * numRecords / 1024 / 1024) / ((double) time / 1000); + } + + private double megaBytePerSec(long time, int numRecords, int recordSizeBytes) { + return (double) (recordSizeBytes * numRecords / 1024 / 1024) / ((double) time / 1000); } private List getAllPartitions(KafkaConsumer consumer, String... topics) { diff --git a/tests/kafkatest/benchmarks/streams/streams_simple_benchmark_test.py b/tests/kafkatest/benchmarks/streams/streams_simple_benchmark_test.py index 5eb266397820..de687e69b6f9 100644 --- a/tests/kafkatest/benchmarks/streams/streams_simple_benchmark_test.py +++ b/tests/kafkatest/benchmarks/streams/streams_simple_benchmark_test.py @@ -27,7 +27,7 @@ class StreamsSimpleBenchmarkTest(KafkaTest): def __init__(self, test_context): super(StreamsSimpleBenchmarkTest, self).__init__(test_context, num_zk=1, num_brokers=1) - self.driver = StreamsSimpleBenchmarkService(test_context, self.kafka) + self.driver = StreamsSimpleBenchmarkService(test_context, self.kafka, 1000000L) def test_simple_benchmark(self): """ diff --git a/tests/kafkatest/services/performance/streams_performance.py b/tests/kafkatest/services/performance/streams_performance.py index 289bccbefc00..b7d6b892a43e 100644 --- a/tests/kafkatest/services/performance/streams_performance.py +++ b/tests/kafkatest/services/performance/streams_performance.py @@ -47,9 +47,10 @@ class StreamsSimpleBenchmarkService(KafkaPathResolverMixin, Service): "collect_default": True}, } - def __init__(self, context, kafka): + def __init__(self, context, kafka, numrecs): super(StreamsSimpleBenchmarkService, self).__init__(context, 1) self.kafka = kafka + self.numrecs = numrecs @property def node(self): @@ -88,6 +89,7 @@ def start_cmd(self, node): args['kafka'] = self.kafka.bootstrap_servers() args['zk'] = self.kafka.zk.connect_setting() args['state_dir'] = self.PERSISTENT_ROOT + args['numrecs'] = self.numrecs args['stdout'] = self.STDOUT_FILE args['stderr'] = self.STDERR_FILE args['pidfile'] = self.PID_FILE @@ -96,7 +98,7 @@ def start_cmd(self, node): cmd = "( export KAFKA_LOG4J_OPTS=\"-Dlog4j.configuration=file:%(log4j)s\"; " \ "INCLUDE_TEST_JARS=true %(kafka_run_class)s org.apache.kafka.streams.perf.SimpleBenchmark " \ - " %(kafka)s %(zk)s %(state_dir)s " \ + " %(kafka)s %(zk)s %(state_dir)s %(numrecs)s " \ " & echo $! >&3 ) 1>> %(stdout)s 2>> %(stderr)s 3> %(pidfile)s" % args return cmd From 317c4fdede41f2026b34f473af1ad69f8ee62a1d Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Fri, 19 Aug 2016 21:59:55 -0700 Subject: [PATCH 15/35] KAFKA-3949: Fix race condition when metadata update arrives during rebalance Author: Jason Gustafson Reviewers: Vahid Hashemian, Guozhang Wang Closes #1762 from hachikuji/KAFKA-3949 --- .../org/apache/kafka/clients/Metadata.java | 13 +- .../kafka/clients/consumer/KafkaConsumer.java | 10 +- .../kafka/clients/consumer/MockConsumer.java | 8 +- .../internals/AbstractCoordinator.java | 20 +- .../internals/ConsumerCoordinator.java | 59 ++-- .../clients/consumer/internals/Fetcher.java | 28 +- .../consumer/internals/SubscriptionState.java | 100 ++++--- .../org/apache/kafka/clients/MockClient.java | 2 +- .../internals/ConsumerCoordinatorTest.java | 280 ++++++++++-------- .../consumer/internals/FetcherTest.java | 65 ++-- .../internals/SubscriptionStateTest.java | 77 +++-- 11 files changed, 354 insertions(+), 308 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 0fd5d63da54d..a4cf730b99d5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/Metadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/Metadata.java @@ -211,8 +211,15 @@ else if (expireMs <= now) { for (Listener listener: listeners) listener.onMetadataUpdate(cluster); - // Do this after notifying listeners as subscribed topics' list can be changed by listeners - this.cluster = this.needMetadataForAllTopics ? getClusterForCurrentTopics(cluster) : cluster; + + if (this.needMetadataForAllTopics) { + // the listener may change the interested topics, which could cause another metadata refresh. + // If we have already fetched all topics, however, another fetch should be unnecessary. + this.needUpdate = false; + this.cluster = getClusterForCurrentTopics(cluster); + } else { + this.cluster = cluster; + } notifyAll(); log.debug("Updated cluster metadata version {} to {}", this.version, this.cluster); @@ -287,7 +294,7 @@ private Cluster getClusterForCurrentTopics(Cluster cluster) { Set unauthorizedTopics = new HashSet<>(); Collection partitionInfos = new ArrayList<>(); List nodes = Collections.emptyList(); - Set internalTopics = Collections.emptySet(); + Set internalTopics = Collections.emptySet(); if (cluster != null) { internalTopics = cluster.internalTopics(); unauthorizedTopics.addAll(cluster.unauthorizedTopics()); 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 ef913027361b..85d519418854 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 @@ -801,7 +801,7 @@ public void subscribe(Collection topics, ConsumerRebalanceListener liste throw new IllegalArgumentException("Topic collection to subscribe to cannot contain null or empty topic"); } log.debug("Subscribed to topic(s): {}", Utils.join(topics, ", ")); - this.subscriptions.subscribe(topics, listener); + this.subscriptions.subscribe(new HashSet<>(topics), listener); metadata.setTopics(subscriptions.groupSubscription()); } } finally { @@ -914,7 +914,7 @@ public void assign(Collection partitions) { } log.debug("Subscribed to partition(s): {}", Utils.join(partitions, ", ")); - this.subscriptions.assignFromUser(partitions); + this.subscriptions.assignFromUser(new HashSet<>(partitions)); metadata.setTopics(topics); } } finally { @@ -1007,6 +1007,12 @@ private Map>> pollOnce(long timeout) { long now = time.milliseconds(); client.poll(Math.min(coordinator.timeToNextPoll(now), timeout), now); + + // after the long poll, we should check whether the group needs to rebalance + // prior to returning data so that the group can stabilize faster + if (coordinator.needRejoin()) + return Collections.emptyMap(); + return fetcher.fetchedRecords(); } 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 9ab4c29493da..62eb77dd6e72 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 @@ -94,26 +94,26 @@ public void subscribe(Collection topics) { public void subscribe(Pattern pattern, final ConsumerRebalanceListener listener) { ensureNotClosed(); this.subscriptions.subscribe(pattern, listener); - List topicsToSubscribe = new ArrayList<>(); + Set topicsToSubscribe = new HashSet<>(); for (String topic: partitions.keySet()) { if (pattern.matcher(topic).matches() && !subscriptions.subscription().contains(topic)) topicsToSubscribe.add(topic); } ensureNotClosed(); - this.subscriptions.changeSubscription(topicsToSubscribe); + this.subscriptions.subscribeFromPattern(topicsToSubscribe); } @Override public void subscribe(Collection topics, final ConsumerRebalanceListener listener) { ensureNotClosed(); - this.subscriptions.subscribe(topics, listener); + this.subscriptions.subscribe(new HashSet<>(topics), listener); } @Override public void assign(Collection partitions) { ensureNotClosed(); - this.subscriptions.assignFromUser(partitions); + this.subscriptions.assignFromUser(new HashSet<>(partitions)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java index 690df2600d55..bf6b920fdae4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java @@ -270,16 +270,6 @@ public synchronized void ensureActiveGroup() { // when sending heartbeats and does not necessarily require us to rejoin the group. ensureCoordinatorReady(); - if (!needRejoin()) - return; - - // call onJoinPrepare if needed. We set a flag to make sure that we do not call it a second - // time if the client is woken up before a pending rebalance completes. - if (needsJoinPrepare) { - onJoinPrepare(generation.generationId, generation.memberId); - needsJoinPrepare = false; - } - if (heartbeatThread == null) { heartbeatThread = new HeartbeatThread(); heartbeatThread.start(); @@ -288,6 +278,16 @@ public synchronized void ensureActiveGroup() { while (needRejoin()) { ensureCoordinatorReady(); + // call onJoinPrepare if needed. We set a flag to make sure that we do not call it a second + // time if the client is woken up before a pending rebalance completes. This must be called + // on each iteration of the loop because an event requiring a rebalance (such as a metadata + // refresh which changes the matched subscription set) can occur while another rebalance is + // still in progress. + if (needsJoinPrepare) { + onJoinPrepare(generation.generationId, generation.memberId); + needsJoinPrepare = false; + } + // 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.coordinator) > 0) { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java index 5fee45afe831..b8df50e9070d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java @@ -78,6 +78,8 @@ public final class ConsumerCoordinator extends AbstractCoordinator { // of offset commit requests, which may be invoked from the heartbeat thread private final ConcurrentLinkedQueue completedOffsetCommits; + private boolean isLeader = false; + private Set joinedSubscription; private MetadataSnapshot metadataSnapshot; private MetadataSnapshot assignmentSnapshot; private long nextAutoCommitDeadline; @@ -137,9 +139,10 @@ public String protocolType() { @Override public List metadata() { + this.joinedSubscription = subscriptions.subscription(); List metadataList = new ArrayList<>(); for (PartitionAssignor assignor : assignors) { - Subscription subscription = assignor.subscription(subscriptions.subscription()); + Subscription subscription = assignor.subscription(joinedSubscription); ByteBuffer metadata = ConsumerProtocol.serializeSubscription(subscription); metadataList.add(new ProtocolMetadata(assignor.name(), metadata)); } @@ -155,26 +158,26 @@ public void onMetadataUpdate(Cluster cluster) { throw new TopicAuthorizationException(new HashSet<>(cluster.unauthorizedTopics())); if (subscriptions.hasPatternSubscription()) { - final List topicsToSubscribe = new ArrayList<>(); + final Set topicsToSubscribe = new HashSet<>(); for (String topic : cluster.topics()) if (subscriptions.getSubscribedPattern().matcher(topic).matches() && !(excludeInternalTopics && cluster.internalTopics().contains(topic))) topicsToSubscribe.add(topic); - subscriptions.changeSubscription(topicsToSubscribe); + subscriptions.subscribeFromPattern(topicsToSubscribe); + + // note we still need to update the topics contained in the metadata. Although we have + // specified that all topics should be fetched, only those set explicitly will be retained metadata.setTopics(subscriptions.groupSubscription()); } // check if there are any changes to the metadata which should trigger a rebalance if (subscriptions.partitionsAutoAssigned()) { MetadataSnapshot snapshot = new MetadataSnapshot(subscriptions, cluster); - if (!snapshot.equals(metadataSnapshot)) { + if (!snapshot.equals(metadataSnapshot)) metadataSnapshot = snapshot; - subscriptions.needReassignment(); - } } - } }); } @@ -192,12 +195,9 @@ protected void onJoinComplete(int generation, String memberId, String assignmentStrategy, ByteBuffer assignmentBuffer) { - // if we were the assignor, then we need to make sure that there have been no metadata updates - // since the rebalance begin. Otherwise, we won't rebalance again until the next metadata change - if (assignmentSnapshot != null && !assignmentSnapshot.equals(metadataSnapshot)) { - subscriptions.needReassignment(); - return; - } + // only the leader is responsible for monitoring for metadata changes (i.e. partition changes) + if (!isLeader) + assignmentSnapshot = null; PartitionAssignor assignor = lookupAssignor(assignmentStrategy); if (assignor == null) @@ -246,13 +246,10 @@ public void poll(long now) { now = time.milliseconds(); } - if (subscriptions.partitionsAutoAssigned() && needRejoin()) { - // due to a race condition between the initial metadata fetch and the initial rebalance, we need to ensure that - // the metadata is fresh before joining initially, and then request the metadata update. If metadata update arrives - // while the rebalance is still pending (for example, when the join group is still inflight), then we will lose - // track of the fact that we need to rebalance again to reflect the change to the topic subscription. Without - // ensuring that the metadata is fresh, any metadata update that changes the topic subscriptions and arrives with a - // rebalance in progress will essentially be ignored. See KAFKA-3949 for the complete description of the problem. + if (needRejoin()) { + // due to a race condition between the initial metadata fetch and the initial rebalance, + // we need to ensure that the metadata is fresh before joining initially. This ensures + // that we have matched the pattern against the cluster's topics at least once before joining. if (subscriptions.hasPatternSubscription()) client.ensureFreshMetadata(); @@ -303,6 +300,8 @@ protected Map performAssignment(String leaderId, // update metadata (if needed) and keep track of the metadata used for assignment so that // we can check after rebalance completion whether anything has changed client.ensureFreshMetadata(); + + isLeader = true; assignmentSnapshot = metadataSnapshot; log.debug("Performing assignment for group {} using strategy {} with subscriptions {}", @@ -339,14 +338,24 @@ protected void onJoinPrepare(int generation, String memberId) { listener.getClass().getName(), groupId, e); } - assignmentSnapshot = null; - subscriptions.needReassignment(); + isLeader = false; + subscriptions.resetGroupSubscription(); } @Override - protected boolean needRejoin() { - return subscriptions.partitionsAutoAssigned() && - (super.needRejoin() || subscriptions.partitionAssignmentNeeded()); + public boolean needRejoin() { + if (!subscriptions.partitionsAutoAssigned()) + return false; + + // we need to rejoin if we performed the assignment and metadata has changed + if (assignmentSnapshot != null && !assignmentSnapshot.equals(metadataSnapshot)) + return true; + + // we need to join if our subscription has changed since the last join + if (joinedSubscription != null && !joinedSubscription.equals(subscriptions.subscription())) + return true; + + return super.needRejoin(); } /** 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 84278c6cd557..aa5cdbeb6875 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 @@ -350,26 +350,22 @@ private long listOffset(TopicPartition partition, long timestamp) { * the defaultResetPolicy is NONE */ public Map>> fetchedRecords() { - if (this.subscriptions.partitionAssignmentNeeded()) { - return Collections.emptyMap(); - } else { - Map>> drained = new HashMap<>(); - int recordsRemaining = maxPollRecords; + Map>> drained = new HashMap<>(); + int recordsRemaining = maxPollRecords; - while (recordsRemaining > 0) { - if (nextInLineRecords == null || nextInLineRecords.isEmpty()) { - CompletedFetch completedFetch = completedFetches.poll(); - if (completedFetch == null) - break; + while (recordsRemaining > 0) { + if (nextInLineRecords == null || nextInLineRecords.isEmpty()) { + CompletedFetch completedFetch = completedFetches.poll(); + if (completedFetch == null) + break; - nextInLineRecords = parseFetchedData(completedFetch); - } else { - recordsRemaining -= append(drained, nextInLineRecords, recordsRemaining); - } + nextInLineRecords = parseFetchedData(completedFetch); + } else { + recordsRemaining -= append(drained, nextInLineRecords, recordsRemaining); } - - return drained; } + + return drained; } private int append(Map>> drained, 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 e9b2eb24ba3e..6d4c01b59a6a 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 @@ -18,6 +18,7 @@ import org.apache.kafka.common.TopicPartition; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -46,6 +47,8 @@ * to set the initial fetch position (e.g. {@link Fetcher#resetOffset(TopicPartition)}. */ public class SubscriptionState { + private static final String SUBSCRIPTION_EXCEPTION_MESSAGE = + "Subscription to topics, partitions and pattern are mutually exclusive"; private enum SubscriptionType { NONE, AUTO_TOPICS, AUTO_PATTERN, USER_ASSIGNED @@ -58,20 +61,17 @@ private enum SubscriptionType { private Pattern subscribedPattern; /* the list of topics the user has requested */ - private final Set subscription; + private Set subscription; + + /* the list of partitions the user has requested */ + private Set userAssignment; /* the list of topics the group has subscribed to (set only for the leader on join group completion) */ private final Set groupSubscription; - /* the list of partitions the user has requested */ - private final Set userAssignment; - /* the list of partitions currently assigned */ private final Map assignment; - /* do we need to request a partition assignment from the coordinator? */ - private boolean needsPartitionAssignment; - /* do we need to request the latest committed offsets from the coordinator? */ private boolean needsFetchCommittedOffsets; @@ -81,8 +81,16 @@ private enum SubscriptionType { /* Listener to be invoked when assignment changes */ private ConsumerRebalanceListener listener; - private static final String SUBSCRIPTION_EXCEPTION_MESSAGE = - "Subscription to topics, partitions and pattern are mutually exclusive"; + public SubscriptionState(OffsetResetStrategy defaultResetStrategy) { + this.defaultResetStrategy = defaultResetStrategy; + this.subscription = Collections.emptySet(); + this.userAssignment = Collections.emptySet(); + this.assignment = new HashMap<>(); + this.groupSubscription = new HashSet<>(); + this.needsFetchCommittedOffsets = true; // initialize to true for the consumers to fetch offset upon starting up + this.subscribedPattern = null; + this.subscriptionType = SubscriptionType.NONE; + } /** * This method sets the subscription type if it is not already set (i.e. when it is NONE), @@ -97,19 +105,7 @@ else if (this.subscriptionType != type) throw new IllegalStateException(SUBSCRIPTION_EXCEPTION_MESSAGE); } - public SubscriptionState(OffsetResetStrategy defaultResetStrategy) { - this.defaultResetStrategy = defaultResetStrategy; - this.subscription = new HashSet<>(); - this.userAssignment = new HashSet<>(); - this.assignment = new HashMap<>(); - this.groupSubscription = new HashSet<>(); - this.needsPartitionAssignment = false; - this.needsFetchCommittedOffsets = true; // initialize to true for the consumers to fetch offset upon starting up - this.subscribedPattern = null; - this.subscriptionType = SubscriptionType.NONE; - } - - public void subscribe(Collection topics, ConsumerRebalanceListener listener) { + public void subscribe(Set topics, ConsumerRebalanceListener listener) { if (listener == null) throw new IllegalArgumentException("RebalanceListener cannot be null"); @@ -120,12 +116,18 @@ public void subscribe(Collection topics, ConsumerRebalanceListener liste changeSubscription(topics); } - public void changeSubscription(Collection topicsToSubscribe) { - if (!this.subscription.equals(new HashSet<>(topicsToSubscribe))) { - this.subscription.clear(); - this.subscription.addAll(topicsToSubscribe); + public void subscribeFromPattern(Set topics) { + if (subscriptionType != SubscriptionType.AUTO_PATTERN) + throw new IllegalArgumentException("Attempt to subscribe from pattern while subscription type set to " + + subscriptionType); + + changeSubscription(topics); + } + + private void changeSubscription(Set topicsToSubscribe) { + if (!this.subscription.equals(topicsToSubscribe)) { + this.subscription = topicsToSubscribe; this.groupSubscription.addAll(topicsToSubscribe); - this.needsPartitionAssignment = true; // Remove any assigned partitions which are no longer subscribed to for (Iterator it = assignment.keySet().iterator(); it.hasNext(); ) { @@ -147,9 +149,11 @@ public void groupSubscribe(Collection topics) { this.groupSubscription.addAll(topics); } - public void needReassignment() { + /** + * Reset the group's subscription to only contain topics subscribed by this consumer. + */ + public void resetGroupSubscription() { this.groupSubscription.retainAll(subscription); - this.needsPartitionAssignment = true; } /** @@ -157,34 +161,37 @@ public void needReassignment() { * note this is different from {@link #assignFromSubscribed(Collection)} * whose input partitions are provided from the subscribed topics. */ - public void assignFromUser(Collection partitions) { + public void assignFromUser(Set partitions) { setSubscriptionType(SubscriptionType.USER_ASSIGNED); - this.userAssignment.clear(); - this.userAssignment.addAll(partitions); + if (!this.assignment.keySet().equals(partitions)) { + this.userAssignment = partitions; - for (TopicPartition partition : partitions) - if (!assignment.containsKey(partition)) - addAssignedPartition(partition); - - this.assignment.keySet().retainAll(this.userAssignment); - - this.needsPartitionAssignment = false; - this.needsFetchCommittedOffsets = true; + for (TopicPartition partition : partitions) + if (!assignment.containsKey(partition)) + addAssignedPartition(partition); + this.assignment.keySet().retainAll(this.userAssignment); + this.needsFetchCommittedOffsets = true; + } } /** * Change the assignment to the specified partitions returned from the coordinator, - * note this is different from {@link #assignFromUser(Collection)} which directly set the assignment from user inputs + * note this is different from {@link #assignFromUser(Set)} which directly set the assignment from user inputs */ public void assignFromSubscribed(Collection assignments) { + if (!this.partitionsAutoAssigned()) + throw new IllegalArgumentException("Attempt to dynamically assign partitions while manual assignment in use"); + for (TopicPartition tp : assignments) if (!this.subscription.contains(tp.topic())) throw new IllegalArgumentException("Assigned partition " + tp + " for non-subscribed topic."); + + // after rebalancing, we always reinitialize the assignment state this.assignment.clear(); for (TopicPartition tp: assignments) addAssignedPartition(tp); - this.needsPartitionAssignment = false; + this.needsFetchCommittedOffsets = true; } public void subscribe(Pattern pattern, ConsumerRebalanceListener listener) { @@ -202,10 +209,9 @@ public boolean hasPatternSubscription() { } public void unsubscribe() { - this.subscription.clear(); - this.userAssignment.clear(); + this.subscription = Collections.emptySet(); + this.userAssignment = Collections.emptySet(); this.assignment.clear(); - this.needsPartitionAssignment = true; this.subscribedPattern = null; this.subscriptionType = SubscriptionType.NONE; } @@ -346,10 +352,6 @@ public Set missingFetchPositions() { return missing; } - public boolean partitionAssignmentNeeded() { - return this.needsPartitionAssignment; - } - public boolean isAssigned(TopicPartition tp) { return assignment.containsKey(tp); } 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 9fbbb88c48c5..8881f829b3ce 100644 --- a/clients/src/test/java/org/apache/kafka/clients/MockClient.java +++ b/clients/src/test/java/org/apache/kafka/clients/MockClient.java @@ -147,7 +147,7 @@ public void send(ClientRequest request, long now) { @Override public List poll(long timeoutMs, long now) { - List copy = new ArrayList(this.responses); + List copy = new ArrayList<>(this.responses); while (!this.responses.isEmpty()) { ClientResponse response = this.responses.poll(); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java index 8ec8b7578b9c..0486e6c0c4e2 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java @@ -66,9 +66,13 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; import java.util.regex.Pattern; +import static java.util.Collections.singleton; +import static java.util.Collections.singletonList; +import static java.util.Collections.singletonMap; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNull; @@ -148,7 +152,7 @@ public void testGroupDescribeUnauthorized() { @Test(expected = GroupAuthorizationException.class) public void testGroupReadUnauthorized() { - subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topicName), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); @@ -206,7 +210,7 @@ public void testIllegalGeneration() { coordinator.ensureCoordinatorReady(); // illegal_generation will cause re-partition - subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topicName), rebalanceListener); subscriptions.assignFromSubscribed(Collections.singletonList(tp)); time.sleep(sessionTimeoutMs); @@ -230,7 +234,7 @@ public void testUnknownConsumerId() { coordinator.ensureCoordinatorReady(); // illegal_generation will cause re-partition - subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topicName), rebalanceListener); subscriptions.assignFromSubscribed(Collections.singletonList(tp)); time.sleep(sessionTimeoutMs); @@ -273,8 +277,7 @@ public void testCoordinatorDisconnect() { public void testJoinGroupInvalidGroupId() { final String consumerId = "leader"; - subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener); - subscriptions.needReassignment(); + subscriptions.subscribe(singleton(topicName), rebalanceListener); // ensure metadata is up-to-date for leader metadata.setTopics(Arrays.asList(topicName)); @@ -292,8 +295,7 @@ public void testJoinGroupInvalidGroupId() { public void testNormalJoinGroupLeader() { final String consumerId = "leader"; - subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener); - subscriptions.needReassignment(); + subscriptions.subscribe(singleton(topicName), rebalanceListener); // ensure metadata is up-to-date for leader metadata.setTopics(Arrays.asList(topicName)); @@ -304,7 +306,7 @@ public void testNormalJoinGroupLeader() { // normal join group Map> memberSubscriptions = Collections.singletonMap(consumerId, Arrays.asList(topicName)); - partitionAssignor.prepare(Collections.singletonMap(consumerId, Arrays.asList(tp))); + partitionAssignor.prepare(Collections.singletonMap(consumerId, singletonList(tp))); client.prepareResponse(joinGroupLeaderResponse(1, consumerId, memberSubscriptions, Errors.NONE.code())); client.prepareResponse(new MockClient.RequestMatcher() { @@ -315,23 +317,86 @@ public boolean matches(ClientRequest request) { sync.generationId() == 1 && sync.groupAssignment().containsKey(consumerId); } - }, syncGroupResponse(Arrays.asList(tp), Errors.NONE.code())); + }, syncGroupResponse(singletonList(tp), Errors.NONE.code())); coordinator.poll(time.milliseconds()); - assertFalse(subscriptions.partitionAssignmentNeeded()); - assertEquals(Collections.singleton(tp), subscriptions.assignedPartitions()); + assertFalse(coordinator.needRejoin()); + assertEquals(singleton(tp), subscriptions.assignedPartitions()); assertEquals(1, rebalanceListener.revokedCount); assertEquals(Collections.emptySet(), rebalanceListener.revoked); assertEquals(1, rebalanceListener.assignedCount); - assertEquals(Collections.singleton(tp), rebalanceListener.assigned); + assertEquals(singleton(tp), rebalanceListener.assigned); + } + + @Test + public void testMetadataRefreshDuringRebalance() { + final String consumerId = "leader"; + final String otherTopicName = "otherTopic"; + TopicPartition otherPartition = new TopicPartition(otherTopicName, 0); + + subscriptions.subscribe(Pattern.compile(".*"), rebalanceListener); + metadata.needMetadataForAllTopics(true); + metadata.update(cluster, time.milliseconds()); + + assertEquals(singleton(topicName), subscriptions.subscription()); + + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorReady(); + + Map> initialSubscription = singletonMap(consumerId, Arrays.asList(topicName)); + partitionAssignor.prepare(singletonMap(consumerId, singletonList(tp))); + + // the metadata will be updated in flight with a new topic added + final List updatedSubscription = Arrays.asList(topicName, otherTopicName); + final Set updatedSubscriptionSet = new HashSet<>(updatedSubscription); + + client.prepareResponse(joinGroupLeaderResponse(1, consumerId, initialSubscription, Errors.NONE.code())); + client.prepareResponse(new MockClient.RequestMatcher() { + @Override + public boolean matches(ClientRequest request) { + final Map updatedPartitions = new HashMap<>(); + for (String topic : updatedSubscription) + updatedPartitions.put(topic, 1); + metadata.update(TestUtils.clusterWith(1, updatedPartitions), time.milliseconds()); + return true; + } + }, syncGroupResponse(singletonList(tp), Errors.NONE.code())); + + List newAssignment = Arrays.asList(tp, otherPartition); + Set newAssignmentSet = new HashSet<>(newAssignment); + + Map> updatedSubscriptions = singletonMap(consumerId, Arrays.asList(topicName, otherTopicName)); + partitionAssignor.prepare(singletonMap(consumerId, newAssignment)); + + // we expect to see a second rebalance with the new-found topics + client.prepareResponse(new MockClient.RequestMatcher() { + @Override + public boolean matches(ClientRequest request) { + JoinGroupRequest join = new JoinGroupRequest(request.request().body()); + ProtocolMetadata protocolMetadata = join.groupProtocols().iterator().next(); + PartitionAssignor.Subscription subscription = ConsumerProtocol.deserializeSubscription(protocolMetadata.metadata()); + protocolMetadata.metadata().rewind(); + return subscription.topics().containsAll(updatedSubscriptionSet); + } + }, joinGroupLeaderResponse(2, consumerId, updatedSubscriptions, Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(newAssignment, Errors.NONE.code())); + + coordinator.poll(time.milliseconds()); + + assertFalse(coordinator.needRejoin()); + assertEquals(updatedSubscriptionSet, subscriptions.subscription()); + assertEquals(newAssignmentSet, subscriptions.assignedPartitions()); + assertEquals(2, rebalanceListener.revokedCount); + assertEquals(singleton(tp), rebalanceListener.revoked); + assertEquals(2, rebalanceListener.assignedCount); + assertEquals(newAssignmentSet, rebalanceListener.assigned); } @Test public void testWakeupDuringJoin() { final String consumerId = "leader"; - subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener); - subscriptions.needReassignment(); + subscriptions.subscribe(singleton(topicName), rebalanceListener); // ensure metadata is up-to-date for leader metadata.setTopics(Arrays.asList(topicName)); @@ -341,7 +406,7 @@ public void testWakeupDuringJoin() { coordinator.ensureCoordinatorReady(); Map> memberSubscriptions = Collections.singletonMap(consumerId, Arrays.asList(topicName)); - partitionAssignor.prepare(Collections.singletonMap(consumerId, Arrays.asList(tp))); + partitionAssignor.prepare(Collections.singletonMap(consumerId, singletonList(tp))); // prepare only the first half of the join and then trigger the wakeup client.prepareResponse(joinGroupLeaderResponse(1, consumerId, memberSubscriptions, Errors.NONE.code())); @@ -354,23 +419,22 @@ public void testWakeupDuringJoin() { } // now complete the second half - client.prepareResponse(syncGroupResponse(Arrays.asList(tp), Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); coordinator.poll(time.milliseconds()); - assertFalse(subscriptions.partitionAssignmentNeeded()); - assertEquals(Collections.singleton(tp), subscriptions.assignedPartitions()); + assertFalse(coordinator.needRejoin()); + assertEquals(singleton(tp), subscriptions.assignedPartitions()); assertEquals(1, rebalanceListener.revokedCount); assertEquals(Collections.emptySet(), rebalanceListener.revoked); assertEquals(1, rebalanceListener.assignedCount); - assertEquals(Collections.singleton(tp), rebalanceListener.assigned); + assertEquals(singleton(tp), rebalanceListener.assigned); } @Test public void testNormalJoinGroupFollower() { final String consumerId = "consumer"; - subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener); - subscriptions.needReassignment(); + subscriptions.subscribe(singleton(topicName), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); @@ -385,29 +449,28 @@ public boolean matches(ClientRequest request) { sync.generationId() == 1 && sync.groupAssignment().isEmpty(); } - }, syncGroupResponse(Arrays.asList(tp), Errors.NONE.code())); + }, syncGroupResponse(singletonList(tp), Errors.NONE.code())); coordinator.poll(time.milliseconds()); - assertFalse(subscriptions.partitionAssignmentNeeded()); - assertEquals(Collections.singleton(tp), subscriptions.assignedPartitions()); + assertFalse(coordinator.needRejoin()); + assertEquals(singleton(tp), subscriptions.assignedPartitions()); assertEquals(1, rebalanceListener.revokedCount); assertEquals(1, rebalanceListener.assignedCount); - assertEquals(Collections.singleton(tp), rebalanceListener.assigned); + assertEquals(singleton(tp), rebalanceListener.assigned); } @Test public void testLeaveGroupOnClose() { final String consumerId = "consumer"; - subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener); - subscriptions.needReassignment(); + subscriptions.subscribe(singleton(topicName), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE.code())); - client.prepareResponse(syncGroupResponse(Arrays.asList(tp), Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); coordinator.poll(time.milliseconds()); final AtomicBoolean received = new AtomicBoolean(false); @@ -428,14 +491,13 @@ public boolean matches(ClientRequest request) { public void testMaybeLeaveGroup() { final String consumerId = "consumer"; - subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener); - subscriptions.needReassignment(); + subscriptions.subscribe(singleton(topicName), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE.code())); - client.prepareResponse(syncGroupResponse(Arrays.asList(tp), Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); coordinator.poll(time.milliseconds()); final AtomicBoolean received = new AtomicBoolean(false); @@ -459,8 +521,7 @@ public boolean matches(ClientRequest request) { public void testUnexpectedErrorOnSyncGroup() { final String consumerId = "consumer"; - subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener); - subscriptions.needReassignment(); + subscriptions.subscribe(singleton(topicName), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); @@ -475,8 +536,7 @@ public void testUnexpectedErrorOnSyncGroup() { public void testUnknownMemberIdOnSyncGroup() { final String consumerId = "consumer"; - subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener); - subscriptions.needReassignment(); + subscriptions.subscribe(singleton(topicName), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); @@ -493,20 +553,19 @@ public boolean matches(ClientRequest request) { return joinRequest.memberId().equals(JoinGroupRequest.UNKNOWN_MEMBER_ID); } }, joinGroupFollowerResponse(2, consumerId, "leader", Errors.NONE.code())); - client.prepareResponse(syncGroupResponse(Arrays.asList(tp), Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); coordinator.poll(time.milliseconds()); - assertFalse(subscriptions.partitionAssignmentNeeded()); - assertEquals(Collections.singleton(tp), subscriptions.assignedPartitions()); + assertFalse(coordinator.needRejoin()); + assertEquals(singleton(tp), subscriptions.assignedPartitions()); } @Test public void testRebalanceInProgressOnSyncGroup() { final String consumerId = "consumer"; - subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener); - subscriptions.needReassignment(); + subscriptions.subscribe(singleton(topicName), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); @@ -517,20 +576,19 @@ public void testRebalanceInProgressOnSyncGroup() { // then let the full join/sync finish successfully client.prepareResponse(joinGroupFollowerResponse(2, consumerId, "leader", Errors.NONE.code())); - client.prepareResponse(syncGroupResponse(Arrays.asList(tp), Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); coordinator.poll(time.milliseconds()); - assertFalse(subscriptions.partitionAssignmentNeeded()); - assertEquals(Collections.singleton(tp), subscriptions.assignedPartitions()); + assertFalse(coordinator.needRejoin()); + assertEquals(singleton(tp), subscriptions.assignedPartitions()); } @Test public void testIllegalGenerationOnSyncGroup() { final String consumerId = "consumer"; - subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener); - subscriptions.needReassignment(); + subscriptions.subscribe(singleton(topicName), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); @@ -547,39 +605,45 @@ public boolean matches(ClientRequest request) { return joinRequest.memberId().equals(JoinGroupRequest.UNKNOWN_MEMBER_ID); } }, joinGroupFollowerResponse(2, consumerId, "leader", Errors.NONE.code())); - client.prepareResponse(syncGroupResponse(Arrays.asList(tp), Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); coordinator.poll(time.milliseconds()); - assertFalse(subscriptions.partitionAssignmentNeeded()); - assertEquals(Collections.singleton(tp), subscriptions.assignedPartitions()); + assertFalse(coordinator.needRejoin()); + assertEquals(singleton(tp), subscriptions.assignedPartitions()); } @Test public void testMetadataChangeTriggersRebalance() { final String consumerId = "consumer"; - subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener); - subscriptions.needReassignment(); + // ensure metadata is up-to-date for leader + metadata.setTopics(Arrays.asList(topicName)); + metadata.update(cluster, time.milliseconds()); + + subscriptions.subscribe(singleton(topicName), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE.code())); - client.prepareResponse(syncGroupResponse(Arrays.asList(tp), Errors.NONE.code())); + Map> memberSubscriptions = Collections.singletonMap(consumerId, Arrays.asList(topicName)); + partitionAssignor.prepare(Collections.singletonMap(consumerId, singletonList(tp))); + + // the leader is responsible for picking up metadata changes and forcing a group rebalance + client.prepareResponse(joinGroupLeaderResponse(1, consumerId, memberSubscriptions, Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); coordinator.poll(time.milliseconds()); - assertFalse(subscriptions.partitionAssignmentNeeded()); + assertFalse(coordinator.needRejoin()); // a new partition is added to the topic metadata.update(TestUtils.singletonCluster(topicName, 2), time.milliseconds()); // we should detect the change and ask for reassignment - assertTrue(subscriptions.partitionAssignmentNeeded()); + assertTrue(coordinator.needRejoin()); } - @Test public void testUpdateMetadataDuringRebalance() { final String topic1 = "topic1"; @@ -590,9 +654,8 @@ public void testUpdateMetadataDuringRebalance() { List topics = Arrays.asList(topic1, topic2); - subscriptions.subscribe(topics, rebalanceListener); + subscriptions.subscribe(new HashSet<>(topics), rebalanceListener); metadata.setTopics(topics); - subscriptions.needReassignment(); // we only have metadata for one topic initially metadata.update(TestUtils.singletonCluster(topic1, 1), time.milliseconds()); @@ -629,7 +692,7 @@ public boolean matches(ClientRequest request) { coordinator.poll(time.milliseconds()); - assertFalse(subscriptions.partitionAssignmentNeeded()); + assertFalse(coordinator.needRejoin()); assertEquals(new HashSet<>(Arrays.asList(tp1, tp2)), subscriptions.assignedPartitions()); } @@ -640,7 +703,7 @@ public void testExcludeInternalTopicsConfigOption() { metadata.update(TestUtils.singletonCluster(TestUtils.GROUP_METADATA_TOPIC_NAME, 2), time.milliseconds()); - assertFalse(subscriptions.partitionAssignmentNeeded()); + assertFalse(subscriptions.subscription().contains(TestUtils.GROUP_METADATA_TOPIC_NAME)); } @Test @@ -650,41 +713,43 @@ public void testIncludeInternalTopicsConfigOption() { metadata.update(TestUtils.singletonCluster(TestUtils.GROUP_METADATA_TOPIC_NAME, 2), time.milliseconds()); - assertTrue(subscriptions.partitionAssignmentNeeded()); + assertTrue(subscriptions.subscription().contains(TestUtils.GROUP_METADATA_TOPIC_NAME)); } @Test public void testRejoinGroup() { - subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener); - subscriptions.needReassignment(); + String otherTopic = "otherTopic"; + + subscriptions.subscribe(singleton(topicName), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); // join the group once client.prepareResponse(joinGroupFollowerResponse(1, "consumer", "leader", Errors.NONE.code())); - client.prepareResponse(syncGroupResponse(Arrays.asList(tp), Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); coordinator.poll(time.milliseconds()); assertEquals(1, rebalanceListener.revokedCount); + assertTrue(rebalanceListener.revoked.isEmpty()); assertEquals(1, rebalanceListener.assignedCount); + assertEquals(singleton(tp), rebalanceListener.assigned); // and join the group again - subscriptions.needReassignment(); + subscriptions.subscribe(new HashSet<>(Arrays.asList(topicName, otherTopic)), rebalanceListener); client.prepareResponse(joinGroupFollowerResponse(2, "consumer", "leader", Errors.NONE.code())); - client.prepareResponse(syncGroupResponse(Arrays.asList(tp), Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); coordinator.poll(time.milliseconds()); assertEquals(2, rebalanceListener.revokedCount); - assertEquals(Collections.singleton(tp), rebalanceListener.revoked); + assertEquals(singleton(tp), rebalanceListener.revoked); assertEquals(2, rebalanceListener.assignedCount); - assertEquals(Collections.singleton(tp), rebalanceListener.assigned); + assertEquals(singleton(tp), rebalanceListener.assigned); } @Test public void testDisconnectInJoin() { - subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener); - subscriptions.needReassignment(); + subscriptions.subscribe(singleton(topicName), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); @@ -693,19 +758,18 @@ public void testDisconnectInJoin() { client.prepareResponse(joinGroupFollowerResponse(1, "consumer", "leader", Errors.NONE.code()), true); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); client.prepareResponse(joinGroupFollowerResponse(1, "consumer", "leader", Errors.NONE.code())); - client.prepareResponse(syncGroupResponse(Arrays.asList(tp), Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); coordinator.poll(time.milliseconds()); - assertFalse(subscriptions.partitionAssignmentNeeded()); - assertEquals(Collections.singleton(tp), subscriptions.assignedPartitions()); + assertFalse(coordinator.needRejoin()); + assertEquals(singleton(tp), subscriptions.assignedPartitions()); assertEquals(1, rebalanceListener.revokedCount); assertEquals(1, rebalanceListener.assignedCount); - assertEquals(Collections.singleton(tp), rebalanceListener.assigned); + assertEquals(singleton(tp), rebalanceListener.assigned); } @Test(expected = ApiException.class) public void testInvalidSessionTimeout() { - subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener); - subscriptions.needReassignment(); + subscriptions.subscribe(singleton(topicName), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); @@ -717,7 +781,7 @@ public void testInvalidSessionTimeout() { @Test public void testCommitOffsetOnly() { - subscriptions.assignFromUser(Arrays.asList(tp)); + subscriptions.assignFromUser(singleton(tp)); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); @@ -739,14 +803,13 @@ public void testAutoCommitDynamicAssignment() { ConsumerCoordinator coordinator = buildCoordinator(new Metrics(), assignors, ConsumerConfig.DEFAULT_EXCLUDE_INTERNAL_TOPICS, true); - subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener); - subscriptions.needReassignment(); + subscriptions.subscribe(singleton(topicName), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE.code())); - client.prepareResponse(syncGroupResponse(Arrays.asList(tp), Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); coordinator.poll(time.milliseconds()); subscriptions.seek(tp, 100); @@ -765,8 +828,7 @@ public void testAutoCommitDynamicAssignmentRebalance() { ConsumerCoordinator coordinator = buildCoordinator(new Metrics(), assignors, ConsumerConfig.DEFAULT_EXCLUDE_INTERNAL_TOPICS, true); - subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener); - subscriptions.needReassignment(); + subscriptions.subscribe(singleton(topicName), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); @@ -776,7 +838,7 @@ public void testAutoCommitDynamicAssignmentRebalance() { consumerClient.poll(0); client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE.code())); - client.prepareResponse(syncGroupResponse(Arrays.asList(tp), Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); coordinator.poll(time.milliseconds()); subscriptions.seek(tp, 100); @@ -793,7 +855,7 @@ public void testAutoCommitManualAssignment() { ConsumerCoordinator coordinator = buildCoordinator(new Metrics(), assignors, ConsumerConfig.DEFAULT_EXCLUDE_INTERNAL_TOPICS, true); - subscriptions.assignFromUser(Arrays.asList(tp)); + subscriptions.assignFromUser(singleton(tp)); subscriptions.seek(tp, 100); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); @@ -811,7 +873,7 @@ public void testAutoCommitManualAssignmentCoordinatorUnknown() { ConsumerCoordinator coordinator = buildCoordinator(new Metrics(), assignors, ConsumerConfig.DEFAULT_EXCLUDE_INTERNAL_TOPICS, true); - subscriptions.assignFromUser(Arrays.asList(tp)); + subscriptions.assignFromUser(singleton(tp)); subscriptions.seek(tp, 100); // no commit initially since coordinator is unknown @@ -835,7 +897,7 @@ public void testAutoCommitManualAssignmentCoordinatorUnknown() { @Test public void testCommitOffsetMetadata() { - subscriptions.assignFromUser(Arrays.asList(tp)); + subscriptions.assignFromUser(singleton(tp)); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); @@ -866,20 +928,20 @@ public void testCommitOffsetAsyncWithDefaultCallback() { @Test public void testCommitAfterLeaveGroup() { // enable auto-assignment - subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topicName), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); client.prepareResponse(joinGroupFollowerResponse(1, "consumer", "leader", Errors.NONE.code())); - client.prepareResponse(syncGroupResponse(Arrays.asList(tp), Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); coordinator.poll(time.milliseconds()); // now switch to manual assignment client.prepareResponse(new LeaveGroupResponse(Errors.NONE.code()).toStruct()); subscriptions.unsubscribe(); coordinator.maybeLeaveGroup(); - subscriptions.assignFromUser(Arrays.asList(tp)); + subscriptions.assignFromUser(singleton(tp)); // the client should not reuse generation/memberId from auto-subscribed generation client.prepareResponse(new MockClient.RequestMatcher() { @@ -1048,7 +1110,7 @@ public void testRefreshOffset() { client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - subscriptions.assignFromUser(Arrays.asList(tp)); + subscriptions.assignFromUser(singleton(tp)); subscriptions.needRefreshCommits(); client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", 100L)); coordinator.refreshCommittedOffsetsIfNeeded(); @@ -1061,7 +1123,7 @@ public void testRefreshOffsetLoadInProgress() { client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - subscriptions.assignFromUser(Arrays.asList(tp)); + subscriptions.assignFromUser(singleton(tp)); subscriptions.needRefreshCommits(); client.prepareResponse(offsetFetchResponse(tp, Errors.GROUP_LOAD_IN_PROGRESS.code(), "", 100L)); client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", 100L)); @@ -1075,7 +1137,7 @@ public void testRefreshOffsetNotCoordinatorForConsumer() { client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - subscriptions.assignFromUser(Arrays.asList(tp)); + subscriptions.assignFromUser(singleton(tp)); subscriptions.needRefreshCommits(); client.prepareResponse(offsetFetchResponse(tp, Errors.NOT_COORDINATOR_FOR_GROUP.code(), "", 100L)); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); @@ -1090,7 +1152,7 @@ public void testRefreshOffsetWithNoFetchableOffsets() { client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - subscriptions.assignFromUser(Arrays.asList(tp)); + subscriptions.assignFromUser(singleton(tp)); subscriptions.needRefreshCommits(); client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", -1L)); coordinator.refreshCommittedOffsetsIfNeeded(); @@ -1122,37 +1184,6 @@ public void testProtocolMetadataOrder() { } } - @Test - public void testMetadataTopicsExpiryDisabled() { - final String consumerId = "consumer"; - - subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener); - HashSet topics = new HashSet<>(); - topics.add(topicName); - metadata.setTopics(topics); - subscriptions.needReassignment(); - - client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); - coordinator.ensureCoordinatorReady(); - - client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE.code())); - client.prepareResponse(syncGroupResponse(Arrays.asList(tp), Errors.NONE.code())); - coordinator.poll(time.milliseconds()); - - metadata.update(TestUtils.singletonCluster(topicName, 2), time.milliseconds()); - assertTrue("Topic not found in metadata", metadata.containsTopic(topicName)); - time.sleep(Metadata.TOPIC_EXPIRY_MS * 2); - metadata.update(TestUtils.singletonCluster(topicName, 2), time.milliseconds()); - assertTrue("Topic expired", metadata.containsTopic(topicName)); - metadata.update(TestUtils.singletonCluster(topicName, 2), time.milliseconds()); - metadata.update(Cluster.empty(), time.milliseconds()); - assertTrue("Topic expired", metadata.containsTopic(topicName)); - - assertTrue(subscriptions.partitionAssignmentNeeded()); - metadata.update(TestUtils.singletonCluster(topicName, 2), time.milliseconds()); - assertTrue(subscriptions.partitionAssignmentNeeded()); - } - private ConsumerCoordinator buildCoordinator(Metrics metrics, List assignors, boolean excludeInternalTopics, @@ -1187,7 +1218,8 @@ private Struct heartbeatResponse(short error) { return response.toStruct(); } - private Struct joinGroupLeaderResponse(int generationId, String memberId, + private Struct joinGroupLeaderResponse(int generationId, + String memberId, Map> subscriptions, short error) { Map metadata = new HashMap<>(); 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 518661877fb7..5c0b49c94bb3 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 @@ -66,6 +66,7 @@ import java.util.Map; import java.util.Random; +import static java.util.Collections.singleton; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNull; @@ -123,7 +124,7 @@ public void teardown() { @Test public void testFetchNormal() { List> records; - subscriptions.assignFromUser(Arrays.asList(tp)); + subscriptions.assignFromUser(singleton(tp)); subscriptions.seek(tp, 0); // normal fetch @@ -167,7 +168,7 @@ public byte[] deserialize(String topic, byte[] data) { Fetcher fetcher = createFetcher(subscriptions, new Metrics(time), deserializer, deserializer); - subscriptions.assignFromUser(Collections.singleton(tp)); + subscriptions.assignFromUser(singleton(tp)); subscriptions.seek(tp, 1); client.prepareResponse(matchesOffset(tp, 1), fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 0)); @@ -209,7 +210,7 @@ public void testParseInvalidRecord() { compressor.close(); buffer.flip(); - subscriptions.assignFromUser(Arrays.asList(tp)); + subscriptions.assignFromUser(singleton(tp)); subscriptions.seek(tp, 0); // normal fetch @@ -230,7 +231,7 @@ public void testFetchMaxPollRecords() { Fetcher fetcher = createFetcher(subscriptions, new Metrics(time), 2); List> records; - subscriptions.assignFromUser(Arrays.asList(tp)); + subscriptions.assignFromUser(singleton(tp)); subscriptions.seek(tp, 1); client.prepareResponse(matchesOffset(tp, 1), fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 0)); @@ -272,7 +273,7 @@ public void testFetchNonContinuousRecords() { records.close(); List> consumerRecords; - subscriptions.assignFromUser(Arrays.asList(tp)); + subscriptions.assignFromUser(singleton(tp)); subscriptions.seek(tp, 0); // normal fetch @@ -290,7 +291,7 @@ public void testFetchNonContinuousRecords() { @Test(expected = RecordTooLargeException.class) public void testFetchRecordTooLarge() { - subscriptions.assignFromUser(Arrays.asList(tp)); + subscriptions.assignFromUser(singleton(tp)); subscriptions.seek(tp, 0); // prepare large record @@ -309,7 +310,7 @@ public void testFetchRecordTooLarge() { @Test public void testUnauthorizedTopic() { - subscriptions.assignFromUser(Arrays.asList(tp)); + subscriptions.assignFromUser(singleton(tp)); subscriptions.seek(tp, 0); // resize the limit of the buffer to pretend it is only fetch-size large @@ -320,20 +321,20 @@ public void testUnauthorizedTopic() { fetcher.fetchedRecords(); fail("fetchedRecords should have thrown"); } catch (TopicAuthorizationException e) { - assertEquals(Collections.singleton(topicName), e.unauthorizedTopics()); + assertEquals(singleton(topicName), e.unauthorizedTopics()); } } @Test public void testFetchDuringRebalance() { - subscriptions.subscribe(Arrays.asList(topicName), listener); - subscriptions.assignFromSubscribed(Arrays.asList(tp)); + subscriptions.subscribe(singleton(topicName), listener); + subscriptions.assignFromSubscribed(singleton(tp)); subscriptions.seek(tp, 0); fetcher.sendFetches(); // Now the rebalance happens and fetch positions are cleared - subscriptions.assignFromSubscribed(Arrays.asList(tp)); + subscriptions.assignFromSubscribed(singleton(tp)); client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 0)); consumerClient.poll(0); @@ -343,7 +344,7 @@ public void testFetchDuringRebalance() { @Test public void testInFlightFetchOnPausedPartition() { - subscriptions.assignFromUser(Arrays.asList(tp)); + subscriptions.assignFromUser(singleton(tp)); subscriptions.seek(tp, 0); fetcher.sendFetches(); @@ -356,7 +357,7 @@ public void testInFlightFetchOnPausedPartition() { @Test public void testFetchOnPausedPartition() { - subscriptions.assignFromUser(Arrays.asList(tp)); + subscriptions.assignFromUser(singleton(tp)); subscriptions.seek(tp, 0); subscriptions.pause(tp); @@ -366,7 +367,7 @@ public void testFetchOnPausedPartition() { @Test public void testFetchNotLeaderForPartition() { - subscriptions.assignFromUser(Arrays.asList(tp)); + subscriptions.assignFromUser(singleton(tp)); subscriptions.seek(tp, 0); fetcher.sendFetches(); @@ -378,7 +379,7 @@ public void testFetchNotLeaderForPartition() { @Test public void testFetchUnknownTopicOrPartition() { - subscriptions.assignFromUser(Arrays.asList(tp)); + subscriptions.assignFromUser(singleton(tp)); subscriptions.seek(tp, 0); fetcher.sendFetches(); @@ -390,7 +391,7 @@ public void testFetchUnknownTopicOrPartition() { @Test public void testFetchOffsetOutOfRange() { - subscriptions.assignFromUser(Arrays.asList(tp)); + subscriptions.assignFromUser(singleton(tp)); subscriptions.seek(tp, 0); fetcher.sendFetches(); @@ -405,7 +406,7 @@ public void testFetchOffsetOutOfRange() { public void testStaleOutOfRangeError() { // verify that an out of range error which arrives after a seek // does not cause us to reset our position or throw an exception - subscriptions.assignFromUser(Arrays.asList(tp)); + subscriptions.assignFromUser(singleton(tp)); subscriptions.seek(tp, 0); fetcher.sendFetches(); @@ -419,7 +420,7 @@ public void testStaleOutOfRangeError() { @Test public void testFetchedRecordsAfterSeek() { - subscriptionsNoAutoReset.assignFromUser(Arrays.asList(tp)); + subscriptionsNoAutoReset.assignFromUser(singleton(tp)); subscriptionsNoAutoReset.seek(tp, 0); fetcherNoAutoReset.sendFetches(); @@ -432,7 +433,7 @@ public void testFetchedRecordsAfterSeek() { @Test public void testFetchOffsetOutOfRangeException() { - subscriptionsNoAutoReset.assignFromUser(Arrays.asList(tp)); + subscriptionsNoAutoReset.assignFromUser(singleton(tp)); subscriptionsNoAutoReset.seek(tp, 0); fetcherNoAutoReset.sendFetches(); @@ -452,7 +453,7 @@ public void testFetchOffsetOutOfRangeException() { @Test public void testFetchDisconnected() { - subscriptions.assignFromUser(Arrays.asList(tp)); + subscriptions.assignFromUser(singleton(tp)); subscriptions.seek(tp, 0); fetcher.sendFetches(); @@ -470,22 +471,22 @@ public void testFetchDisconnected() { public void testUpdateFetchPositionToCommitted() { // unless a specific reset is expected, the default behavior is to reset to the committed // position if one is present - subscriptions.assignFromUser(Arrays.asList(tp)); + subscriptions.assignFromUser(singleton(tp)); subscriptions.committed(tp, new OffsetAndMetadata(5)); - fetcher.updateFetchPositions(Collections.singleton(tp)); + fetcher.updateFetchPositions(singleton(tp)); assertTrue(subscriptions.isFetchable(tp)); assertEquals(5, subscriptions.position(tp).longValue()); } @Test public void testUpdateFetchPositionResetToDefaultOffset() { - subscriptions.assignFromUser(Arrays.asList(tp)); + subscriptions.assignFromUser(singleton(tp)); // with no commit position, we should reset using the default strategy defined above (EARLIEST) client.prepareResponse(listOffsetRequestMatcher(ListOffsetRequest.EARLIEST_TIMESTAMP), listOffsetResponse(Errors.NONE, Arrays.asList(5L))); - fetcher.updateFetchPositions(Collections.singleton(tp)); + fetcher.updateFetchPositions(singleton(tp)); assertFalse(subscriptions.isOffsetResetNeeded(tp)); assertTrue(subscriptions.isFetchable(tp)); assertEquals(5, subscriptions.position(tp).longValue()); @@ -493,12 +494,12 @@ public void testUpdateFetchPositionResetToDefaultOffset() { @Test public void testUpdateFetchPositionResetToLatestOffset() { - subscriptions.assignFromUser(Arrays.asList(tp)); + subscriptions.assignFromUser(singleton(tp)); subscriptions.needOffsetReset(tp, OffsetResetStrategy.LATEST); client.prepareResponse(listOffsetRequestMatcher(ListOffsetRequest.LATEST_TIMESTAMP), listOffsetResponse(Errors.NONE, Arrays.asList(5L))); - fetcher.updateFetchPositions(Collections.singleton(tp)); + fetcher.updateFetchPositions(singleton(tp)); assertFalse(subscriptions.isOffsetResetNeeded(tp)); assertTrue(subscriptions.isFetchable(tp)); assertEquals(5, subscriptions.position(tp).longValue()); @@ -506,12 +507,12 @@ public void testUpdateFetchPositionResetToLatestOffset() { @Test public void testUpdateFetchPositionResetToEarliestOffset() { - subscriptions.assignFromUser(Arrays.asList(tp)); + subscriptions.assignFromUser(singleton(tp)); subscriptions.needOffsetReset(tp, OffsetResetStrategy.EARLIEST); client.prepareResponse(listOffsetRequestMatcher(ListOffsetRequest.EARLIEST_TIMESTAMP), listOffsetResponse(Errors.NONE, Arrays.asList(5L))); - fetcher.updateFetchPositions(Collections.singleton(tp)); + fetcher.updateFetchPositions(singleton(tp)); assertFalse(subscriptions.isOffsetResetNeeded(tp)); assertTrue(subscriptions.isFetchable(tp)); assertEquals(5, subscriptions.position(tp).longValue()); @@ -519,7 +520,7 @@ public void testUpdateFetchPositionResetToEarliestOffset() { @Test public void testUpdateFetchPositionDisconnect() { - subscriptions.assignFromUser(Arrays.asList(tp)); + subscriptions.assignFromUser(singleton(tp)); subscriptions.needOffsetReset(tp, OffsetResetStrategy.LATEST); // First request gets a disconnect @@ -529,7 +530,7 @@ public void testUpdateFetchPositionDisconnect() { // Next one succeeds client.prepareResponse(listOffsetRequestMatcher(ListOffsetRequest.LATEST_TIMESTAMP), listOffsetResponse(Errors.NONE, Arrays.asList(5L))); - fetcher.updateFetchPositions(Collections.singleton(tp)); + fetcher.updateFetchPositions(singleton(tp)); assertFalse(subscriptions.isOffsetResetNeeded(tp)); assertTrue(subscriptions.isFetchable(tp)); assertEquals(5, subscriptions.position(tp).longValue()); @@ -567,7 +568,7 @@ public void testGetAllTopicsUnauthorized() { fetcher.getAllTopicMetadata(10L); fail(); } catch (TopicAuthorizationException e) { - assertEquals(Collections.singleton(topicName), e.unauthorizedTopics()); + assertEquals(singleton(topicName), e.unauthorizedTopics()); } } @@ -600,7 +601,7 @@ public void testGetTopicMetadataLeaderNotAvailable() { @Test public void testQuotaMetrics() throws Exception { List> records; - subscriptions.assignFromUser(Arrays.asList(tp)); + subscriptions.assignFromUser(singleton(tp)); subscriptions.seek(tp, 0); // normal fetch 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 3b4b10e7b0c1..783f0e63c8b7 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 @@ -16,21 +16,23 @@ */ 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; +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.common.TopicPartition; +import org.junit.Test; import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import java.util.HashSet; import java.util.regex.Pattern; -import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; -import org.apache.kafka.clients.consumer.OffsetAndMetadata; -import org.apache.kafka.clients.consumer.OffsetResetStrategy; -import org.apache.kafka.common.TopicPartition; -import org.junit.Test; +import static java.util.Arrays.asList; +import static java.util.Collections.singleton; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; public class SubscriptionStateTest { @@ -43,16 +45,15 @@ public class SubscriptionStateTest { @Test public void partitionAssignment() { - state.assignFromUser(Arrays.asList(tp0)); - assertEquals(Collections.singleton(tp0), state.assignedPartitions()); - assertFalse(state.partitionAssignmentNeeded()); + state.assignFromUser(singleton(tp0)); + assertEquals(singleton(tp0), state.assignedPartitions()); assertFalse(state.hasAllFetchPositions()); assertTrue(state.refreshCommitsNeeded()); state.committed(tp0, new OffsetAndMetadata(1)); state.seek(tp0, 1); assertTrue(state.isFetchable(tp0)); assertAllPositions(tp0, 1L); - state.assignFromUser(Arrays.asList()); + state.assignFromUser(Collections.emptySet()); assertTrue(state.assignedPartitions().isEmpty()); assertFalse(state.isAssigned(tp0)); assertFalse(state.isFetchable(tp0)); @@ -60,7 +61,7 @@ public void partitionAssignment() { @Test public void partitionReset() { - state.assignFromUser(Arrays.asList(tp0)); + state.assignFromUser(singleton(tp0)); state.seek(tp0, 5); assertEquals(5L, (long) state.position(tp0)); state.needOffsetReset(tp0); @@ -76,9 +77,8 @@ public void partitionReset() { @Test public void topicSubscription() { - state.subscribe(Arrays.asList(topic), rebalanceListener); + state.subscribe(singleton(topic), rebalanceListener); assertEquals(1, state.subscription().size()); - assertTrue(state.partitionAssignmentNeeded()); assertTrue(state.assignedPartitions().isEmpty()); assertTrue(state.partitionsAutoAssigned()); state.assignFromSubscribed(asList(tp0)); @@ -87,15 +87,14 @@ public void topicSubscription() { assertAllPositions(tp0, 1L); state.assignFromSubscribed(asList(tp1)); assertTrue(state.isAssigned(tp1)); - assertFalse(state.partitionAssignmentNeeded()); assertFalse(state.isAssigned(tp0)); assertFalse(state.isFetchable(tp1)); - assertEquals(Collections.singleton(tp1), state.assignedPartitions()); + assertEquals(singleton(tp1), state.assignedPartitions()); } @Test public void partitionPause() { - state.assignFromUser(Arrays.asList(tp0)); + state.assignFromUser(singleton(tp0)); state.seek(tp0, 100); assertTrue(state.isFetchable(tp0)); state.pause(tp0); @@ -106,7 +105,7 @@ public void partitionPause() { @Test public void commitOffsetMetadata() { - state.assignFromUser(Arrays.asList(tp0)); + state.assignFromUser(singleton(tp0)); state.committed(tp0, new OffsetAndMetadata(5, "hi")); assertEquals(5, state.committed(tp0).offset()); @@ -115,7 +114,7 @@ public void commitOffsetMetadata() { @Test(expected = IllegalStateException.class) public void invalidPositionUpdate() { - state.subscribe(Arrays.asList(topic), rebalanceListener); + state.subscribe(singleton(topic), rebalanceListener); state.assignFromSubscribed(asList(tp0)); state.position(tp0, 0); } @@ -132,32 +131,32 @@ public void assertAllPositions(TopicPartition tp, Long offset) { @Test(expected = IllegalStateException.class) public void cantSubscribeTopicAndPattern() { - state.subscribe(Arrays.asList(topic), rebalanceListener); + state.subscribe(singleton(topic), rebalanceListener); state.subscribe(Pattern.compile(".*"), rebalanceListener); } @Test(expected = IllegalStateException.class) public void cantSubscribePartitionAndPattern() { - state.assignFromUser(Arrays.asList(tp0)); + state.assignFromUser(singleton(tp0)); state.subscribe(Pattern.compile(".*"), rebalanceListener); } @Test(expected = IllegalStateException.class) public void cantSubscribePatternAndTopic() { state.subscribe(Pattern.compile(".*"), rebalanceListener); - state.subscribe(Arrays.asList(topic), rebalanceListener); + state.subscribe(singleton(topic), rebalanceListener); } @Test(expected = IllegalStateException.class) public void cantSubscribePatternAndPartition() { state.subscribe(Pattern.compile(".*"), rebalanceListener); - state.assignFromUser(Arrays.asList(tp0)); + state.assignFromUser(singleton(tp0)); } @Test public void patternSubscription() { state.subscribe(Pattern.compile(".*"), rebalanceListener); - state.changeSubscription(Arrays.asList(topic, topic1)); + state.subscribeFromPattern(new HashSet<>(Arrays.asList(topic, topic1))); assertEquals( "Expected subscribed topics count is incorrect", 2, state.subscription().size()); @@ -165,43 +164,37 @@ public void patternSubscription() { @Test public void unsubscribeUserAssignment() { - state.assignFromUser(Arrays.asList(tp0, tp1)); + state.assignFromUser(new HashSet<>(Arrays.asList(tp0, tp1))); state.unsubscribe(); - state.subscribe(Arrays.asList(topic), rebalanceListener); - assertEquals(Collections.singleton(topic), state.subscription()); + state.subscribe(singleton(topic), rebalanceListener); + assertEquals(singleton(topic), state.subscription()); } @Test public void unsubscribeUserSubscribe() { - state.subscribe(Arrays.asList(topic), rebalanceListener); + state.subscribe(singleton(topic), rebalanceListener); state.unsubscribe(); - state.assignFromUser(Arrays.asList(tp0)); - assertEquals(Collections.singleton(tp0), state.assignedPartitions()); + state.assignFromUser(singleton(tp0)); + assertEquals(singleton(tp0), state.assignedPartitions()); } @Test public void unsubscription() { state.subscribe(Pattern.compile(".*"), rebalanceListener); - state.changeSubscription(Arrays.asList(topic, topic1)); - assertTrue(state.partitionAssignmentNeeded()); - + state.subscribeFromPattern(new HashSet<>(Arrays.asList(topic, topic1))); state.assignFromSubscribed(asList(tp1)); - assertEquals(Collections.singleton(tp1), state.assignedPartitions()); - assertFalse(state.partitionAssignmentNeeded()); + assertEquals(singleton(tp1), state.assignedPartitions()); state.unsubscribe(); assertEquals(0, state.subscription().size()); assertTrue(state.assignedPartitions().isEmpty()); - assertTrue(state.partitionAssignmentNeeded()); - state.assignFromUser(Arrays.asList(tp0)); - assertEquals(Collections.singleton(tp0), state.assignedPartitions()); - assertFalse(state.partitionAssignmentNeeded()); + state.assignFromUser(singleton(tp0)); + assertEquals(singleton(tp0), state.assignedPartitions()); state.unsubscribe(); assertEquals(0, state.subscription().size()); assertTrue(state.assignedPartitions().isEmpty()); - assertTrue(state.partitionAssignmentNeeded()); } private static class MockRebalanceListener implements ConsumerRebalanceListener { From 113d23e8fe290878120c1384b580181169777f4f Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Mon, 22 Aug 2016 10:34:01 -0700 Subject: [PATCH 16/35] KAFKA-3163; Minor follow up for (KIP-33) junrao Could you take a look when get a chance? Thanks. Author: Jiangjie Qin Reviewers: Jun Rao Closes #1769 from becketqin/KAFKA-3163-follow-up --- core/src/main/scala/kafka/log/LogSegment.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/log/LogSegment.scala b/core/src/main/scala/kafka/log/LogSegment.scala index d894020d5013..3d94452a5d77 100755 --- a/core/src/main/scala/kafka/log/LogSegment.scala +++ b/core/src/main/scala/kafka/log/LogSegment.scala @@ -349,7 +349,7 @@ class LogSegment(val log: FileMessageSet, if (iter.hasNext) rollingBasedTimestamp = Some(iter.next.message.timestamp) else - // If the log is empty, we return 0 as time waited. + // If the log is empty, we return time elapsed since the segment is created. return now - created } now - {if (rollingBasedTimestamp.get >= 0) rollingBasedTimestamp.get else created} @@ -393,7 +393,7 @@ class LogSegment(val log: FileMessageSet, * Close this log segment */ def close() { - timeIndex.maybeAppend(maxTimestampSoFar, offsetOfMaxTimestamp, skipFullCheck = true) + CoreUtils.swallow(timeIndex.maybeAppend(maxTimestampSoFar, offsetOfMaxTimestamp, skipFullCheck = true)) CoreUtils.swallow(index.close) CoreUtils.swallow(timeIndex.close()) CoreUtils.swallow(log.close) From 6ed3e6b1cb8a73b1f5f78926ccb247a8953a554c Mon Sep 17 00:00:00 2001 From: Tom Crayford Date: Mon, 22 Aug 2016 10:58:40 -0700 Subject: [PATCH 17/35] KAFKA-3894; log cleaner can partially clean a segment As discussed in https://issues.apache.org/jira/browse/KAFKA-3894, this PR makes the log cleaner do a "partial" clean on a segment, whereby it builds a partial offset map up to a particular offset in a segment. Once cleaning resumes again, we will continue from the next dirty offset, which can now be located in the middle of a segment. Prior to this PR, segments with overly numerous keys could crash the log cleaner thread, as it was required that the log cleaner had to fit at least a single segment in the offset map. Author: Tom Crayford Reviewers: Jun Rao Closes #1725 from tcrayford/dont_crash_log_cleaner_thread_if_segment_overflows_buffer --- .../src/main/scala/kafka/log/LogCleaner.scala | 46 +++++----- core/src/main/scala/kafka/log/OffsetMap.scala | 19 ++++- .../scala/unit/kafka/log/CleanerTest.scala | 85 +++++++++++++++++-- 3 files changed, 111 insertions(+), 39 deletions(-) diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index d4bb1f2c6eff..ef880e62d44f 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -311,6 +311,8 @@ private[log] class Cleaner(val id: Int, /* buffer used for write i/o */ private var writeBuffer = ByteBuffer.allocate(ioBufferSize) + require(offsetMap.slots * dupBufferLoadFactor > 1, "offset map is too small to fit in even a single message, so log cleaning will never make progress. You can increase log.cleaner.dedupe.buffer.size or decrease log.cleaner.threads") + /** * Clean the given log * @@ -326,7 +328,8 @@ private[log] class Cleaner(val id: Int, // build the offset map info("Building offset map for %s...".format(cleanable.log.name)) val upperBoundOffset = log.activeSegment.baseOffset - val endOffset = buildOffsetMap(log, cleanable.firstDirtyOffset, upperBoundOffset, offsetMap) + 1 + buildOffsetMap(log, cleanable.firstDirtyOffset, upperBoundOffset, offsetMap) + val endOffset = offsetMap.latestOffset + 1 stats.indexDone() // figure out the timestamp below which it is safe to remove delete tombstones @@ -341,7 +344,7 @@ private[log] class Cleaner(val id: Int, info("Cleaning log %s (discarding tombstones prior to %s)...".format(log.name, new Date(deleteHorizonMs))) for (group <- groupSegmentsBySize(log.logSegments(0, endOffset), log.config.segmentSize, log.config.maxIndexSize)) cleanSegments(log, group, offsetMap, deleteHorizonMs) - + // record buffer utilization stats.bufferUtilization = offsetMap.utilization @@ -533,6 +536,10 @@ private[log] class Cleaner(val id: Int, map: kafka.log.OffsetMap, retainDeletes: Boolean, entry: kafka.message.MessageAndOffset): Boolean = { + val pastLatestOffset = entry.offset > map.latestOffset + if (pastLatestOffset) + return true + val key = entry.message.key if (key != null) { val foundOffset = map.get(key) @@ -613,34 +620,23 @@ private[log] class Cleaner(val id: Int, * @param start The offset at which dirty messages begin * @param end The ending offset for the map that is being built * @param map The map in which to store the mappings - * - * @return The final offset the map covers */ - private[log] def buildOffsetMap(log: Log, start: Long, end: Long, map: OffsetMap): Long = { + private[log] def buildOffsetMap(log: Log, start: Long, end: Long, map: OffsetMap) { map.clear() val dirty = log.logSegments(start, end).toBuffer info("Building offset map for log %s for %d segments in offset range [%d, %d).".format(log.name, dirty.size, start, end)) // Add all the dirty segments. We must take at least map.slots * load_factor, // 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)) var full = false for (segment <- dirty if !full) { checkDone(log.topicAndPartition) - val newOffset = buildOffsetMapForSegment(log.topicAndPartition, segment, map) - if (newOffset > -1L) - offset = newOffset - else { - // If not even one segment can fit in the map, compaction cannot happen - require(offset > start, "Unable to build the offset map for segment %s/%s. You can increase log.cleaner.dedupe.buffer.size or decrease log.cleaner.threads".format(log.name, segment.log.file.getName)) + full = buildOffsetMapForSegment(log.topicAndPartition, segment, map, start) + if (full) debug("Offset map is full, %d segments fully mapped, segment with base offset %d is partially mapped".format(dirty.indexOf(segment), segment.baseOffset)) - full = true - } } info("Offset map for log %s complete.".format(log.name)) - offset } /** @@ -649,11 +645,10 @@ private[log] class Cleaner(val id: Int, * @param segment The segment to index * @param map The map in which to store the key=>offset mapping * - * @return The final offset covered by the map or -1 if the map is full + * @return If the map was filled whilst loading from this segment */ - private def buildOffsetMapForSegment(topicAndPartition: TopicAndPartition, segment: LogSegment, map: OffsetMap): Long = { - var position = 0 - var offset = segment.baseOffset + private def buildOffsetMapForSegment(topicAndPartition: TopicAndPartition, segment: LogSegment, map: OffsetMap, start: Long): Boolean = { + var position = segment.index.lookup(start).position val maxDesiredMapSize = (map.slots * this.dupBufferLoadFactor).toInt while (position < segment.log.sizeInBytes) { checkDone(topicAndPartition) @@ -663,15 +658,12 @@ private[log] class Cleaner(val id: Int, val startPosition = position for (entry <- messages) { val message = entry.message - if (message.hasKey) { + if (message.hasKey && entry.offset >= start) { if (map.size < maxDesiredMapSize) map.put(message.key, entry.offset) - else { - // The map is full, stop looping and return - return -1L - } + else + return true } - offset = entry.offset stats.indexMessagesRead(1) } position += messages.validBytes @@ -682,7 +674,7 @@ private[log] class Cleaner(val id: Int, growBuffers() } restoreBuffers() - offset + return false } } diff --git a/core/src/main/scala/kafka/log/OffsetMap.scala b/core/src/main/scala/kafka/log/OffsetMap.scala index f4530307df8a..1df061515e50 100755 --- a/core/src/main/scala/kafka/log/OffsetMap.scala +++ b/core/src/main/scala/kafka/log/OffsetMap.scala @@ -30,6 +30,7 @@ trait OffsetMap { def clear() def size: Int def utilization: Double = size.toDouble / slots + def latestOffset: Long } /** @@ -60,7 +61,10 @@ class SkimpyOffsetMap(val memory: Int, val hashAlgorithm: String = "MD5") extend /* the number of probes for all lookups */ private var probes = 0L - + + /* the latest offset written into the map */ + private var lastOffset = -1L + /** * The number of bytes of space each entry uses (the number of bytes in the hash plus an 8 byte offset) */ @@ -89,6 +93,7 @@ class SkimpyOffsetMap(val memory: Int, val hashAlgorithm: String = "MD5") extend if(Arrays.equals(hash1, hash2)) { // we found an existing entry, overwrite it and return (size does not change) bytes.putLong(offset) + lastOffset = offset return } attempt += 1 @@ -98,6 +103,7 @@ class SkimpyOffsetMap(val memory: Int, val hashAlgorithm: String = "MD5") extend bytes.position(pos) bytes.put(hash1) bytes.putLong(offset) + lastOffset = offset entries += 1 } @@ -106,7 +112,7 @@ class SkimpyOffsetMap(val memory: Int, val hashAlgorithm: String = "MD5") extend */ private def isEmpty(position: Int): Boolean = bytes.getLong(position) == 0 && bytes.getLong(position + 8) == 0 && bytes.getLong(position + 16) == 0 - + /** * Get the offset associated with this key. * @param key The key @@ -136,12 +142,12 @@ class SkimpyOffsetMap(val memory: Int, val hashAlgorithm: String = "MD5") extend /** * Change the salt used for key hashing making all existing keys unfindable. - * Doesn't actually zero out the array. */ override def clear() { this.entries = 0 this.lookups = 0L this.probes = 0L + this.lastOffset = -1L Arrays.fill(bytes.array, bytes.arrayOffset, bytes.arrayOffset + bytes.limit, 0.toByte) } @@ -155,7 +161,12 @@ class SkimpyOffsetMap(val memory: Int, val hashAlgorithm: String = "MD5") extend */ def collisionRate: Double = (this.probes - this.lookups) / this.lookups.toDouble - + + /** + * The latest offset put into the map + */ + override def latestOffset: Long = lastOffset + /** * Calculate the ith probe position. We first try reading successive integers from the hash itself * then if all of those fail we degrade to linear probing. diff --git a/core/src/test/scala/unit/kafka/log/CleanerTest.scala b/core/src/test/scala/unit/kafka/log/CleanerTest.scala index 15920ad8b8c8..5b0ce9a8532d 100755 --- a/core/src/test/scala/unit/kafka/log/CleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/CleanerTest.scala @@ -107,6 +107,38 @@ class CleanerTest extends JUnitSuite { (0 until leo.toInt by 2).forall(!keys.contains(_))) } + @Test + def testPartialSegmentClean() { + // because loadFactor is 0.75, this means we can fit 2 messages in the map + var cleaner = makeCleaner(2) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) + + val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) + + log.append(message(0,0)) // offset 0 + log.append(message(1,1)) // offset 1 + log.append(message(0,0)) // offset 2 + log.append(message(1,1)) // offset 3 + log.append(message(0,0)) // offset 4 + // roll the segment, so we can clean the messages already appended + log.roll() + + // clean the log with only one message removed + cleaner.clean(LogToClean(TopicAndPartition("test", 0), log, 2)) + assertEquals(immutable.List(1,0,1,0), keysInLog(log)) + assertEquals(immutable.List(1,2,3,4), offsetsInLog(log)) + + // continue to make progress, even though we can only clean one message at a time + cleaner.clean(LogToClean(TopicAndPartition("test", 0), log, 3)) + assertEquals(immutable.List(0,1,0), keysInLog(log)) + assertEquals(immutable.List(2,3,4), offsetsInLog(log)) + + cleaner.clean(LogToClean(TopicAndPartition("test", 0), log, 4)) + assertEquals(immutable.List(1,0), keysInLog(log)) + assertEquals(immutable.List(3,4), offsetsInLog(log)) + } + @Test def testLogToClean: Unit = { // create a log with small segment size @@ -159,6 +191,10 @@ class CleanerTest extends JUnitSuite { def keysInLog(log: Log): Iterable[Int] = log.logSegments.flatMap(s => s.log.filter(!_.message.isNull).filter(_.message.hasKey).map(m => TestUtils.readString(m.message.key).toInt)) + /* extract all the offsets from a log */ + def offsetsInLog(log: Log): Iterable[Long] = + log.logSegments.flatMap(s => s.log.filter(!_.message.isNull).filter(_.message.hasKey).map(m => m.offset)) + def unkeyedMessageCountInLog(log: Log) = log.logSegments.map(s => s.log.filter(!_.message.isNull).count(m => !m.message.hasKey)).sum @@ -307,7 +343,8 @@ class CleanerTest extends JUnitSuite { val end = 500 val offsets = writeToLog(log, (start until end) zip (start until end)) def checkRange(map: FakeOffsetMap, start: Int, end: Int) { - val endOffset = cleaner.buildOffsetMap(log, start, end, map) + 1 + cleaner.buildOffsetMap(log, start, end, map) + val endOffset = map.latestOffset + 1 assertEquals("Last offset should be the end offset.", end, endOffset) assertEquals("Should have the expected number of messages in the map.", end-start, map.size) for(i <- start until end) @@ -439,13 +476,39 @@ class CleanerTest extends JUnitSuite { val end = 2 val offsetSeq = Seq(0L, 7206178L) val offsets = writeToLog(log, (start until end) zip (start until end), offsetSeq) - val endOffset = cleaner.buildOffsetMap(log, start, end, map) + cleaner.buildOffsetMap(log, start, end, map) + val endOffset = map.latestOffset assertEquals("Last offset should be the end offset.", 7206178L, endOffset) assertEquals("Should have the expected number of messages in the map.", end - start, map.size) assertEquals("Map should contain first value", 0L, map.get(key(0))) assertEquals("Map should contain second value", 7206178L, map.get(key(1))) } + /** + * Test building a partial offset map of part of a log segment + */ + @Test + def testBuildPartialOffsetMap() { + // because loadFactor is 0.75, this means we can fit 2 messages in the map + val map = new FakeOffsetMap(3) + val log = makeLog() + val cleaner = makeCleaner(2) + + log.append(message(0,0)) + log.append(message(1,1)) + log.append(message(2,2)) + log.append(message(3,3)) + log.append(message(4,4)) + log.roll() + + cleaner.buildOffsetMap(log, 2, Int.MaxValue, map) + assertEquals(2, map.size) + assertEquals(-1, map.get(key(0))) + assertEquals(2, map.get(key(2))) + assertEquals(3, map.get(key(3))) + assertEquals(-1, map.get(key(4))) + } + private def writeToLog(log: Log, keysAndValues: Iterable[(Int, Int)], offsetSeq: Iterable[Long]): Iterable[Long] = { for(((key, value), offset) <- keysAndValues.zip(offsetSeq)) yield log.append(messageWithOffset(key, value, offset), assignOffsets = false).firstOffset @@ -469,7 +532,7 @@ class CleanerTest extends JUnitSuite { offsetMap = new FakeOffsetMap(capacity), ioBufferSize = 64*1024, maxIoBufferSize = 64*1024, - dupBufferLoadFactor = 0.75, + dupBufferLoadFactor = 0.75, throttler = throttler, time = time, checkDone = checkDone ) @@ -500,12 +563,15 @@ class CleanerTest extends JUnitSuite { class FakeOffsetMap(val slots: Int) extends OffsetMap { val map = new java.util.HashMap[String, Long]() - - private def keyFor(key: ByteBuffer) = + var lastOffset = -1L + + private def keyFor(key: ByteBuffer) = new String(Utils.readBytes(key.duplicate), "UTF-8") - - def put(key: ByteBuffer, offset: Long): Unit = + + def put(key: ByteBuffer, offset: Long): Unit = { + lastOffset = offset map.put(keyFor(key), offset) + } def get(key: ByteBuffer): Long = { val k = keyFor(key) @@ -518,5 +584,8 @@ class FakeOffsetMap(val slots: Int) extends OffsetMap { def clear() = map.clear() def size: Int = map.size - + + def latestOffset: Long = lastOffset + + override def toString: String = map.toString() } From 7b16b4731666ff321fbe46828d526872ff5f56d7 Mon Sep 17 00:00:00 2001 From: Rajini Sivaram Date: Mon, 22 Aug 2016 21:34:26 +0100 Subject: [PATCH 18/35] KAFKA-4066; Fix NPE in consumer due to multi-threaded updates Author: Rajini Sivaram Reviewers: Jason Gustafson , Ismael Juma Closes #1763 from rajinisivaram/KAFKA-4066 --- .../internals/AbstractCoordinator.java | 32 +++++++++---------- 1 file changed, 15 insertions(+), 17 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java index bf6b920fdae4..f2e15ca6f943 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java @@ -206,24 +206,16 @@ public synchronized void ensureCoordinatorReady() { } } - protected RequestFuture lookupCoordinator() { - if (findCoordinatorFuture == null) { + protected synchronized RequestFuture lookupCoordinator() { + if (findCoordinatorFuture == null) findCoordinatorFuture = sendGroupCoordinatorRequest(); - findCoordinatorFuture.addListener(new RequestFutureListener() { - @Override - public void onSuccess(Void value) { - findCoordinatorFuture = null; - } - - @Override - public void onFailure(RuntimeException e) { - findCoordinatorFuture = null; - } - }); - } return findCoordinatorFuture; } + private synchronized void clearFindCoordinatorFuture() { + findCoordinatorFuture = null; + } + /** * Check whether the group should be rejoined (e.g. if metadata changes) * @return true if it should, false otherwise @@ -532,6 +524,7 @@ public void onSuccess(ClientResponse resp, RequestFuture future) { // for the coordinator in the underlying network client layer // TODO: this needs to be better handled in KAFKA-1935 Errors error = Errors.forCode(groupCoordinatorResponse.errorCode()); + clearFindCoordinatorFuture(); if (error == Errors.NONE) { synchronized (AbstractCoordinator.this) { AbstractCoordinator.this.coordinator = new Node( @@ -550,6 +543,12 @@ public void onSuccess(ClientResponse resp, RequestFuture future) { future.raise(error); } } + + @Override + public void onFailure(RuntimeException e, RequestFuture future) { + clearFindCoordinatorFuture(); + super.onFailure(e, future); + } } /** @@ -820,7 +819,6 @@ private RuntimeException failureCause() { @Override public void run() { try { - RequestFuture findCoordinatorFuture = null; while (true) { synchronized (AbstractCoordinator.this) { @@ -843,8 +841,8 @@ public void run() { long now = time.milliseconds(); if (coordinatorUnknown()) { - if (findCoordinatorFuture == null || findCoordinatorFuture.isDone()) - findCoordinatorFuture = lookupCoordinator(); + if (findCoordinatorFuture == null) + lookupCoordinator(); else AbstractCoordinator.this.wait(retryBackoffMs); } else if (heartbeat.sessionTimeoutExpired(now)) { From f153407c42716f4f4d9abe8be39ab1112f36a8be Mon Sep 17 00:00:00 2001 From: Samuel Taylor Date: Mon, 22 Aug 2016 23:16:56 +0100 Subject: [PATCH 19/35] MINOR: Fix typos in security section 1. I think the instructions in step 2 of the security section which describe adding the CA to server/client truststores are swapped. That is, the instruction that says to add the CA to the server truststore adds it to the client truststore (and vice versa). 2. "clients keys" should be possessive ("clients' keys"). This contribution is my original work, and I license the work to the project under the project's open source license. Author: Samuel Taylor Reviewers: Ismael Juma Closes #1651 from ssaamm/trunk --- docs/security.html | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/security.html b/docs/security.html index 0a5e5617f45b..d51c340eed11 100644 --- a/docs/security.html +++ b/docs/security.html @@ -75,11 +75,11 @@

    7.2 Encryption and Authentication The next step is to add the generated CA to the **clients' truststore** so that the clients can trust this CA:
    -        keytool -keystore server.truststore.jks -alias CARoot -import -file ca-cert
    + keytool -keystore client.truststore.jks -alias CARoot -import -file ca-cert

    - Note: If you configure the Kafka brokers to require client authentication by setting ssl.client.auth to be "requested" or "required" on the Kafka brokers config then you must provide a truststore for the Kafka brokers as well and it should have all the CA certificates that clients keys were signed by. + Note: If you configure the Kafka brokers to require client authentication by setting ssl.client.auth to be "requested" or "required" on the Kafka brokers config then you must provide a truststore for the Kafka brokers as well and it should have all the CA certificates that clients' keys were signed by.
    -        keytool -keystore client.truststore.jks -alias CARoot -import -file ca-cert
    + keytool -keystore server.truststore.jks -alias CARoot -import -file ca-cert In contrast to the keystore in step 1 that stores each machine's own identity, the truststore of a client stores all the certificates that the client should trust. Importing a certificate into one's truststore also means trusting all certificates that are signed by that certificate. As the analogy above, trusting the government (CA) also means trusting all passports (certificates) that it has issued. This attribute is called the chain of trust, and it is particularly useful when deploying SSL on a large Kafka cluster. You can sign all certificates in the cluster with a single CA, and have all machines share the same truststore that trusts the CA. That way all machines can authenticate all other machines. From d851ce76fdaa469dcaed7f66340574529d79a3cd Mon Sep 17 00:00:00 2001 From: "Matthias J. Sax" Date: Mon, 22 Aug 2016 15:45:29 -0700 Subject: [PATCH 20/35] MINOR: improve Streams application reset tool to make sure application is down Author: Matthias J. Sax Reviewers: Guozhang Wang Closes #1764 from mjsax/improveResetTool --- .../main/scala/kafka/tools/StreamsResetter.java | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/tools/StreamsResetter.java b/core/src/main/scala/kafka/tools/StreamsResetter.java index 8e463d125498..22b8bd60c8eb 100644 --- a/core/src/main/scala/kafka/tools/StreamsResetter.java +++ b/core/src/main/scala/kafka/tools/StreamsResetter.java @@ -20,6 +20,7 @@ import joptsimple.OptionParser; import joptsimple.OptionSet; import joptsimple.OptionSpec; +import kafka.admin.AdminClient; import kafka.admin.TopicCommand; import kafka.utils.ZkUtils; import org.apache.kafka.clients.consumer.ConsumerConfig; @@ -82,10 +83,18 @@ public int run(final String[] args, final Properties config) { int exitCode = EXIT_CODE_SUCCESS; + AdminClient adminClient = null; ZkUtils zkUtils = null; try { parseArguments(args); + adminClient = AdminClient.createSimplePlaintext(this.options.valueOf(bootstrapServerOption)); + final String groupId = this.options.valueOf(applicationIdOption); + if (adminClient.describeConsumerGroup(groupId).get().size() != 0) { + throw new IllegalStateException("Consumer group '" + groupId + "' is still active. " + + "Make sure to stop all running application instances before running the reset tool."); + } + zkUtils = ZkUtils.apply(this.options.valueOf(zookeeperOption), 30000, 30000, @@ -97,10 +106,13 @@ public int run(final String[] args, final Properties config) { resetInputAndInternalTopicOffsets(); seekToEndIntermediateTopics(); deleteInternalTopics(zkUtils); - } catch (final Exception e) { + } catch (final Throwable e) { exitCode = EXIT_CODE_ERROR; System.err.println("ERROR: " + e.getMessage()); } finally { + if (adminClient != null) { + adminClient.close(); + } if (zkUtils != null) { zkUtils.close(); } From d903babb726bdf725bd473c0590f9826b6fa2f14 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 22 Aug 2016 23:52:50 +0100 Subject: [PATCH 21/35] MINOR: add slf4jlog4j to streams example Author: Guozhang Wang Reviewers: Ismael Juma Closes #1731 from guozhangwang/Kminor-log4j-streams-examples --- build.gradle | 8 +++++++- gradle/dependencies.gradle | 2 ++ 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index 32141081572f..31a86f4b32f9 100644 --- a/build.gradle +++ b/build.gradle @@ -695,7 +695,12 @@ project(':streams') { exclude module: 'jline' exclude module: 'netty' } - compile libs.jacksonDatabind // this dependency should be removed after KIP-4 + + // zkclient has a hard log4j dependency, we set it here as well to avoid copying an older + // version to the binary package; the following two dependencies should be removed after KIP-4 + // (along with the zkclient dependency) + compile libs.log4j + compile libs.jacksonDatabind testCompile project(':clients').sourceSets.test.output testCompile project(':core') @@ -743,6 +748,7 @@ project(':streams:examples') { dependencies { compile project(':streams') compile project(':connect:json') // this dependency should be removed after we unify data API + compile libs.slf4jlog4j // this dependency should be removed after KIP-4 } javadoc { diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index 2634450d613e..ea900b6155e8 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -31,6 +31,7 @@ versions += [ jackson: "2.6.3", jetty: "9.2.15.v20160210", jersey: "2.22.2", + log4j: "1.2.17", jopt: "4.9", junit: "4.12", lz4: "1.3.0", @@ -85,6 +86,7 @@ libs += [ jettyServlets: "org.eclipse.jetty:jetty-servlets:$versions.jetty", jerseyContainerServlet: "org.glassfish.jersey.containers:jersey-container-servlet:$versions.jersey", junit: "junit:junit:$versions.junit", + log4j: "log4j:log4j:$versions.log4j", joptSimple: "net.sf.jopt-simple:jopt-simple:$versions.jopt", lz4: "net.jpountz.lz4:lz4:$versions.lz4", metrics: "com.yammer.metrics:metrics-core:$versions.metrics", From dedacd06e4d1e967261b9bca3e32ba0e44b52ba1 Mon Sep 17 00:00:00 2001 From: Rajini Sivaram Date: Mon, 22 Aug 2016 15:58:32 -0700 Subject: [PATCH 22/35] KAFKA-4051: Use nanosecond clock for timers in broker Use System.nanoseconds instead of System.currentTimeMillis in broker timer tasks to cope with changes to wall-clock time. Author: Rajini Sivaram Reviewers: Gwen Shapira Closes #1768 from rajinisivaram/KAFKA-4051 --- core/src/main/scala/kafka/utils/Time.scala | 4 ++++ core/src/main/scala/kafka/utils/timer/Timer.scala | 5 +++-- core/src/main/scala/kafka/utils/timer/TimerTaskList.scala | 2 +- 3 files changed, 8 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/kafka/utils/Time.scala b/core/src/main/scala/kafka/utils/Time.scala index 194cc1fa73b6..f562ef7301bc 100644 --- a/core/src/main/scala/kafka/utils/Time.scala +++ b/core/src/main/scala/kafka/utils/Time.scala @@ -17,6 +17,8 @@ package kafka.utils +import java.util.concurrent.TimeUnit + /** * Some common constants */ @@ -44,6 +46,8 @@ trait Time { def nanoseconds: Long + def hiResClockMs: Long = TimeUnit.NANOSECONDS.toMillis(nanoseconds) + def sleep(ms: Long) } diff --git a/core/src/main/scala/kafka/utils/timer/Timer.scala b/core/src/main/scala/kafka/utils/timer/Timer.scala index 2d7866521c87..67de27696c0a 100644 --- a/core/src/main/scala/kafka/utils/timer/Timer.scala +++ b/core/src/main/scala/kafka/utils/timer/Timer.scala @@ -22,6 +22,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock import kafka.utils.threadsafe import org.apache.kafka.common.utils.Utils +import kafka.utils.SystemTime trait Timer { /** @@ -55,7 +56,7 @@ trait Timer { class SystemTimer(executorName: String, tickMs: Long = 1, wheelSize: Int = 20, - startMs: Long = System.currentTimeMillis) extends Timer { + startMs: Long = SystemTime.hiResClockMs) extends Timer { // timeout timer private[this] val taskExecutor = Executors.newFixedThreadPool(1, new ThreadFactory() { @@ -81,7 +82,7 @@ class SystemTimer(executorName: String, def add(timerTask: TimerTask): Unit = { readLock.lock() try { - addTimerTaskEntry(new TimerTaskEntry(timerTask, timerTask.delayMs + System.currentTimeMillis())) + addTimerTaskEntry(new TimerTaskEntry(timerTask, timerTask.delayMs + SystemTime.hiResClockMs)) } finally { readLock.unlock() } diff --git a/core/src/main/scala/kafka/utils/timer/TimerTaskList.scala b/core/src/main/scala/kafka/utils/timer/TimerTaskList.scala index e862f4f11991..7a77b2749a28 100644 --- a/core/src/main/scala/kafka/utils/timer/TimerTaskList.scala +++ b/core/src/main/scala/kafka/utils/timer/TimerTaskList.scala @@ -117,7 +117,7 @@ private[timer] class TimerTaskList(taskCounter: AtomicInteger) extends Delayed { } def getDelay(unit: TimeUnit): Long = { - unit.convert(max(getExpiration - SystemTime.milliseconds, 0), TimeUnit.MILLISECONDS) + unit.convert(max(getExpiration - SystemTime.hiResClockMs, 0), TimeUnit.MILLISECONDS) } def compareTo(d: Delayed): Int = { From f903215536af06b7b79739882d9286abc2e50000 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 22 Aug 2016 16:09:02 -0700 Subject: [PATCH 23/35] KAFKA-4049: Fix transient failure in RegexSourceIntegrationTest Author: Guozhang Wang Reviewers: Ismael Juma , Ewen Cheslack-Postava Closes #1746 from guozhangwang/K4049-RegexSourceIntegrationTest-failure --- .../java/org/apache/kafka/test/TestUtils.java | 1 - .../RegexSourceIntegrationTest.java | 62 +++++++------------ 2 files changed, 21 insertions(+), 42 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/test/TestUtils.java b/clients/src/test/java/org/apache/kafka/test/TestUtils.java index 4baa63bff4c6..44026be4b4a4 100644 --- a/clients/src/test/java/org/apache/kafka/test/TestUtils.java +++ b/clients/src/test/java/org/apache/kafka/test/TestUtils.java @@ -247,7 +247,6 @@ public static void waitForCondition(TestCondition testCondition, String conditio public static void waitForCondition(TestCondition testCondition, long maxWaitMs, String conditionDetails) throws InterruptedException { long startTime = System.currentTimeMillis(); - while (!testCondition.conditionMet() && ((System.currentTimeMillis() - startTime) < maxWaitMs)) { Thread.sleep(Math.min(maxWaitMs, 100L)); } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java index 08928937dbc6..51fa06ae4ad0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java @@ -52,9 +52,7 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.Properties; import java.util.UUID; import java.util.regex.Pattern; @@ -81,9 +79,6 @@ public class RegexSourceIntegrationTest { private static final String FA_TOPIC = "fa"; private static final String FOO_TOPIC = "foo"; - private static final int FIRST_UPDATE = 0; - private static final int SECOND_UPDATE = 1; - private static final String DEFAULT_OUTPUT_TOPIC = "outputTopic"; private static final String STRING_SERDE_CLASSNAME = Serdes.String().getClass().getName(); private Properties streamsConfiguration; @@ -121,6 +116,8 @@ public void tearDown() throws Exception { public void testRegexMatchesTopicsAWhenCreated() throws Exception { final Serde stringSerde = Serdes.String(); + final List expectedFirstAssignment = Arrays.asList("TEST-TOPIC-1"); + final List expectedSecondAssignment = Arrays.asList("TEST-TOPIC-1", "TEST-TOPIC-2"); StreamsConfig streamsConfig = new StreamsConfig(streamsConfiguration); @@ -146,41 +143,35 @@ public void testRegexMatchesTopicsAWhenCreated() throws Exception { TestCondition oneTopicAdded = new TestCondition() { @Override public boolean conditionMet() { - List assignedTopics = testStreamThread.assignedTopicPartitions.get(FIRST_UPDATE); - return assignedTopics != null && assignedTopics.contains("TEST-TOPIC-1") && !assignedTopics.contains("TEST-TOPIC-2"); + return testStreamThread.assignedTopicPartitions.equals(expectedFirstAssignment); } }; streamThreads[0] = testStreamThread; streams.start(); - TestUtils.waitForCondition(oneTopicAdded, STREAM_TASKS_NOT_UPDATED); + TestUtils.waitForCondition(oneTopicAdded, STREAM_TASKS_NOT_UPDATED); CLUSTER.createTopic("TEST-TOPIC-2"); TestCondition secondTopicAdded = new TestCondition() { @Override public boolean conditionMet() { - List assignedTopics = testStreamThread.assignedTopicPartitions.get(SECOND_UPDATE); - return assignedTopics != null && assignedTopics.contains("TEST-TOPIC-1") && assignedTopics.contains("TEST-TOPIC-2"); + return testStreamThread.assignedTopicPartitions.equals(expectedSecondAssignment); } }; - TestUtils.waitForCondition(secondTopicAdded, STREAM_TASKS_NOT_UPDATED); + TestUtils.waitForCondition(secondTopicAdded, STREAM_TASKS_NOT_UPDATED); streams.close(); - - List expectedFirstAssignment = Arrays.asList("TEST-TOPIC-1"); - List expectedSecondAssignment = Arrays.asList("TEST-TOPIC-1", "TEST-TOPIC-2"); - - assertThat(testStreamThread.assignedTopicPartitions.get(FIRST_UPDATE), equalTo(expectedFirstAssignment)); - assertThat(testStreamThread.assignedTopicPartitions.get(SECOND_UPDATE), equalTo(expectedSecondAssignment)); } @Test public void testRegexMatchesTopicsAWhenDeleted() throws Exception { final Serde stringSerde = Serdes.String(); + final List expectedFirstAssignment = Arrays.asList("TEST-TOPIC-A", "TEST-TOPIC-B"); + final List expectedSecondAssignment = Arrays.asList("TEST-TOPIC-B"); StreamsConfig streamsConfig = new StreamsConfig(streamsConfiguration); @@ -209,34 +200,25 @@ public void testRegexMatchesTopicsAWhenDeleted() throws Exception { TestCondition bothTopicsAdded = new TestCondition() { @Override public boolean conditionMet() { - List assignedTopics = testStreamThread.assignedTopicPartitions.get(FIRST_UPDATE); - return assignedTopics != null && assignedTopics.contains("TEST-TOPIC-A") && assignedTopics.contains("TEST-TOPIC-B"); + return testStreamThread.assignedTopicPartitions.equals(expectedFirstAssignment); } }; streams.start(); - TestUtils.waitForCondition(bothTopicsAdded, STREAM_TASKS_NOT_UPDATED); + TestUtils.waitForCondition(bothTopicsAdded, STREAM_TASKS_NOT_UPDATED); CLUSTER.deleteTopic("TEST-TOPIC-A"); - TestCondition oneTopicRemoved = new TestCondition() { @Override public boolean conditionMet() { - List assignedTopics = testStreamThread.assignedTopicPartitions.get(SECOND_UPDATE); - return assignedTopics != null && !assignedTopics.contains("TEST-TOPIC-A") && assignedTopics.contains("TEST-TOPIC-B"); + return testStreamThread.assignedTopicPartitions.equals(expectedSecondAssignment); } }; - TestUtils.waitForCondition(oneTopicRemoved, STREAM_TASKS_NOT_UPDATED); + TestUtils.waitForCondition(oneTopicRemoved, STREAM_TASKS_NOT_UPDATED); streams.close(); - - List expectedFirstAssignment = Arrays.asList("TEST-TOPIC-A", "TEST-TOPIC-B"); - List expectedSecondAssignment = Arrays.asList("TEST-TOPIC-B"); - - assertThat(testStreamThread.assignedTopicPartitions.get(FIRST_UPDATE), equalTo(expectedFirstAssignment)); - assertThat(testStreamThread.assignedTopicPartitions.get(SECOND_UPDATE), equalTo(expectedSecondAssignment)); } @@ -291,7 +273,7 @@ public void testShouldReadFromRegexAndNamedTopics() throws Exception { assertThat(actualValues, equalTo(expectedReceivedValues)); } - //TODO should be updated to expected = TopologyBuilderException after KAFKA-3708 + // TODO should be updated to expected = TopologyBuilderException after KAFKA-3708 @Test(expected = AssertionError.class) public void testNoMessagesSentExceptionFromOverlappingPatterns() throws Exception { @@ -304,8 +286,8 @@ public void testNoMessagesSentExceptionFromOverlappingPatterns() throws Exceptio KStreamBuilder builder = new KStreamBuilder(); - // overlapping patterns here, no messages should be sent as TopologyBuilderException - // will be thrown when the processor topology is built. + // overlapping patterns here, no messages should be sent as TopologyBuilderException + // will be thrown when the processor topology is built. KStream pattern1Stream = builder.stream(Pattern.compile("foo.*")); KStream pattern2Stream = builder.stream(Pattern.compile("f.*")); @@ -334,9 +316,7 @@ public void testNoMessagesSentExceptionFromOverlappingPatterns() throws Exceptio } private class TestStreamThread extends StreamThread { - - public Map> assignedTopicPartitions = new HashMap<>(); - private int index = 0; + public volatile List assignedTopicPartitions = new ArrayList<>(); public TestStreamThread(TopologyBuilder builder, StreamsConfig config, KafkaClientSupplier clientSupplier, String applicationId, String clientId, UUID processId, Metrics metrics, Time time) { super(builder, config, clientSupplier, applicationId, clientId, processId, metrics, time, new StreamsMetadataState(builder)); @@ -344,15 +324,15 @@ public TestStreamThread(TopologyBuilder builder, StreamsConfig config, KafkaClie @Override public StreamTask createStreamTask(TaskId id, Collection partitions) { - List assignedTopics = new ArrayList<>(); + List topicPartitions = new ArrayList<>(); for (TopicPartition partition : partitions) { - assignedTopics.add(partition.topic()); + topicPartitions.add(partition.topic()); } - Collections.sort(assignedTopics); - assignedTopicPartitions.put(index++, assignedTopics); + Collections.sort(topicPartitions); + + assignedTopicPartitions = topicPartitions; return super.createStreamTask(id, partitions); } } - } From 05d00b5aca2e1e59ad685a3f051d2ab022f75acc Mon Sep 17 00:00:00 2001 From: Grant Henke Date: Tue, 23 Aug 2016 00:14:44 +0100 Subject: [PATCH 24/35] KAFKA-4032; Uncaught exceptions when autocreating topics handled by adding a catch all for any unhandled exception. Because the jira specifically mentions the InvalidReplicationFactor exception, a test was added for that specific case. Author: Grant Henke Reviewers: Jason Gustafson , Ismael Juma Closes #1739 from granthenke/create-errors --- .../main/scala/kafka/server/KafkaApis.scala | 4 +-- .../integration/BaseTopicMetadataTest.scala | 25 +++++++++++++++++++ 2 files changed, 27 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 6eb574f575db..0a5258e9fa5b 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -643,8 +643,8 @@ class KafkaApis(val requestChannel: RequestChannel, case e: TopicExistsException => // let it go, possibly another broker created this topic new MetadataResponse.TopicMetadata(Errors.LEADER_NOT_AVAILABLE, topic, Topic.isInternal(topic), java.util.Collections.emptyList()) - case itex: InvalidTopicException => - new MetadataResponse.TopicMetadata(Errors.INVALID_TOPIC_EXCEPTION, topic, Topic.isInternal(topic), + case ex: Throwable => // Catch all to prevent unhandled errors + new MetadataResponse.TopicMetadata(Errors.forException(ex), topic, Topic.isInternal(topic), java.util.Collections.emptyList()) } } diff --git a/core/src/test/scala/unit/kafka/integration/BaseTopicMetadataTest.scala b/core/src/test/scala/unit/kafka/integration/BaseTopicMetadataTest.scala index 7c9f3aee22bf..24ed9546caf4 100644 --- a/core/src/test/scala/unit/kafka/integration/BaseTopicMetadataTest.scala +++ b/core/src/test/scala/unit/kafka/integration/BaseTopicMetadataTest.scala @@ -132,6 +132,31 @@ abstract class BaseTopicMetadataTest extends ZooKeeperTestHarness { assertTrue(partitionMetadata.head.leader.isDefined) } + @Test + def testAutoCreateTopicWithInvalidReplication { + val adHocProps = createBrokerConfig(2, zkConnect, interBrokerSecurityProtocol = Some(securityProtocol), + trustStoreFile = trustStoreFile) + // Set default replication higher than the number of live brokers + adHocProps.setProperty(KafkaConfig.DefaultReplicationFactorProp, "3") + // start adHoc brokers with replication factor too high + val adHocServer = createServer(new KafkaConfig(adHocProps)) + // We are using the Scala clients and they don't support SSL. Once we move to the Java ones, we should use + // `securityProtocol` instead of PLAINTEXT below + val adHocEndpoint = new BrokerEndPoint(adHocServer.config.brokerId, adHocServer.config.hostName, + adHocServer.boundPort(SecurityProtocol.PLAINTEXT)) + + // auto create topic on "bad" endpoint + val topic = "testAutoCreateTopic" + val topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic), Seq(adHocEndpoint), "TopicMetadataTest-testAutoCreateTopic", + 2000,0).topicsMetadata + assertEquals(Errors.INVALID_REPLICATION_FACTOR.code, topicsMetadata.head.errorCode) + assertEquals("Expecting metadata only for 1 topic", 1, topicsMetadata.size) + assertEquals("Expecting metadata for the test topic", topic, topicsMetadata.head.topic) + assertEquals(0, topicsMetadata.head.partitionsMetadata.size) + + adHocServer.shutdown() + } + @Test def testAutoCreateTopicWithCollision { // auto create topic From 24fd025d407d42176847143d3a0dc416a75d8f35 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Tue, 23 Aug 2016 02:55:17 +0100 Subject: [PATCH 25/35] KAFKA-3916; Check for disconnects properly before sending from the controller Author: Jason Gustafson Reviewers: Ismael Juma Closes #1734 from hachikuji/KAFKA-3916 --- .../apache/kafka/common/network/Selector.java | 125 +++++++++++++----- .../controller/ControllerChannelManager.scala | 19 +-- .../utils/NetworkClientBlockingOps.scala | 58 ++++---- 3 files changed, 120 insertions(+), 82 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 ab9dab998d4a..524471010123 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 @@ -77,6 +77,7 @@ */ public class Selector implements Selectable { + public static final long NO_IDLE_TIMEOUT_MS = -1; private static final Logger log = LoggerFactory.getLogger(Selector.class); private final java.nio.channels.Selector nioSelector; @@ -93,25 +94,36 @@ public class Selector implements Selectable { private final String metricGrpPrefix; private final Map metricTags; private final ChannelBuilder channelBuilder; - private final Map lruConnections; - private final long connectionsMaxIdleNanos; private final int maxReceiveSize; private final boolean metricsPerConnection; - private long currentTimeNanos; - private long nextIdleCloseCheckTime; - + private final IdleExpiryManager idleExpiryManager; /** * Create a new nioSelector + * + * @param maxReceiveSize Max size in bytes of a single network receive (use {@link NetworkReceive#UNLIMITED} for no limit) + * @param connectionMaxIdleMs Max idle connection time (use {@link #NO_IDLE_TIMEOUT_MS} to disable idle timeout) + * @param metrics Registry for Selector metrics + * @param time Time implementation + * @param metricGrpPrefix Prefix for the group of metrics registered by Selector + * @param metricTags Additional tags to add to metrics registered by Selector + * @param metricsPerConnection Whether or not to enable per-connection metrics + * @param channelBuilder Channel builder for every new connection */ - public Selector(int maxReceiveSize, long connectionMaxIdleMs, Metrics metrics, Time time, String metricGrpPrefix, Map metricTags, boolean metricsPerConnection, ChannelBuilder channelBuilder) { + public Selector(int maxReceiveSize, + long connectionMaxIdleMs, + Metrics metrics, + Time time, + String metricGrpPrefix, + Map metricTags, + boolean metricsPerConnection, + ChannelBuilder channelBuilder) { try { this.nioSelector = java.nio.channels.Selector.open(); } catch (IOException e) { throw new KafkaException(e); } this.maxReceiveSize = maxReceiveSize; - this.connectionsMaxIdleNanos = connectionMaxIdleMs * 1000 * 1000; this.time = time; this.metricGrpPrefix = metricGrpPrefix; this.metricTags = metricTags; @@ -125,11 +137,8 @@ public Selector(int maxReceiveSize, long connectionMaxIdleMs, Metrics metrics, T this.failedSends = new ArrayList<>(); this.sensors = new SelectorMetrics(metrics); this.channelBuilder = channelBuilder; - // initial capacity and load factor are default, we set them explicitly because we want to set accessOrder = true - this.lruConnections = new LinkedHashMap<>(16, .75F, true); - currentTimeNanos = time.nanoseconds(); - nextIdleCloseCheckTime = currentTimeNanos + connectionsMaxIdleNanos; this.metricsPerConnection = metricsPerConnection; + this.idleExpiryManager = connectionMaxIdleMs < 0 ? null : new IdleExpiryManager(time, connectionMaxIdleMs); } public Selector(long connectionMaxIdleMS, Metrics metrics, Time time, String metricGrpPrefix, ChannelBuilder channelBuilder) { @@ -276,22 +285,26 @@ public void poll(long timeout) throws IOException { long startSelect = time.nanoseconds(); int readyKeys = select(timeout); long endSelect = time.nanoseconds(); - currentTimeNanos = endSelect; this.sensors.selectTime.record(endSelect - startSelect, time.milliseconds()); if (readyKeys > 0 || !immediatelyConnectedKeys.isEmpty()) { - pollSelectionKeys(this.nioSelector.selectedKeys(), false); - pollSelectionKeys(immediatelyConnectedKeys, true); + pollSelectionKeys(this.nioSelector.selectedKeys(), false, endSelect); + pollSelectionKeys(immediatelyConnectedKeys, true, endSelect); } addToCompletedReceives(); long endIo = time.nanoseconds(); this.sensors.ioTime.record(endIo - endSelect, time.milliseconds()); - maybeCloseOldestConnection(); + + // we use the time at the end of select to ensure that we don't close any connections that + // have just been processed in pollSelectionKeys + maybeCloseOldestConnection(endSelect); } - private void pollSelectionKeys(Iterable selectionKeys, boolean isImmediatelyConnected) { + private void pollSelectionKeys(Iterable selectionKeys, + boolean isImmediatelyConnected, + long currentTimeNanos) { Iterator iterator = selectionKeys.iterator(); while (iterator.hasNext()) { SelectionKey key = iterator.next(); @@ -300,7 +313,8 @@ private void pollSelectionKeys(Iterable selectionKeys, boolean isI // register all per-connection metrics at once sensors.maybeRegisterConnectionMetrics(channel.id()); - lruConnections.put(channel.id(), currentTimeNanos); + if (idleExpiryManager != null) + idleExpiryManager.update(channel.id(), currentTimeNanos); try { @@ -409,24 +423,20 @@ public void unmuteAll() { unmute(channel); } - private void maybeCloseOldestConnection() { - if (currentTimeNanos > nextIdleCloseCheckTime) { - if (lruConnections.isEmpty()) { - nextIdleCloseCheckTime = currentTimeNanos + connectionsMaxIdleNanos; - } else { - Map.Entry oldestConnectionEntry = lruConnections.entrySet().iterator().next(); - Long connectionLastActiveTime = oldestConnectionEntry.getValue(); - nextIdleCloseCheckTime = connectionLastActiveTime + connectionsMaxIdleNanos; - if (currentTimeNanos > nextIdleCloseCheckTime) { - String connectionId = oldestConnectionEntry.getKey(); - 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); - } - } + private void maybeCloseOldestConnection(long currentTimeNanos) { + if (idleExpiryManager == null) + return; + + Map.Entry expiredConnection = idleExpiryManager.pollExpiredConnection(currentTimeNanos); + if (expiredConnection != null) { + String connectionId = expiredConnection.getKey(); + + if (log.isTraceEnabled()) + log.trace("About to close the idle connection from {} due to being idle for {} millis", + connectionId, (currentTimeNanos - expiredConnection.getValue()) / 1000 / 1000); + + disconnected.add(connectionId); + close(connectionId); } } @@ -480,8 +490,10 @@ private void close(KafkaChannel channel) { } this.stagedReceives.remove(channel); this.channels.remove(channel.id()); - this.lruConnections.remove(channel.id()); this.sensors.connectionClosed.record(); + + if (idleExpiryManager != null) + idleExpiryManager.remove(channel.id()); } @@ -726,4 +738,45 @@ public void close() { } } + // helper class for tracking least recently used connections to enable idle connection closing + private static class IdleExpiryManager { + private final Map lruConnections; + private final long connectionsMaxIdleNanos; + private long nextIdleCloseCheckTime; + + public IdleExpiryManager(Time time, long connectionsMaxIdleMs) { + this.connectionsMaxIdleNanos = connectionsMaxIdleMs * 1000 * 1000; + // initial capacity and load factor are default, we set them explicitly because we want to set accessOrder = true + this.lruConnections = new LinkedHashMap<>(16, .75F, true); + this.nextIdleCloseCheckTime = time.nanoseconds() + this.connectionsMaxIdleNanos; + } + + public void update(String connectionId, long currentTimeNanos) { + lruConnections.put(connectionId, currentTimeNanos); + } + + public Map.Entry pollExpiredConnection(long currentTimeNanos) { + if (currentTimeNanos <= nextIdleCloseCheckTime) + return null; + + if (lruConnections.isEmpty()) { + nextIdleCloseCheckTime = currentTimeNanos + connectionsMaxIdleNanos; + return null; + } + + Map.Entry oldestConnectionEntry = lruConnections.entrySet().iterator().next(); + Long connectionLastActiveTime = oldestConnectionEntry.getValue(); + nextIdleCloseCheckTime = connectionLastActiveTime + connectionsMaxIdleNanos; + + if (currentTimeNanos > nextIdleCloseCheckTime) + return oldestConnectionEntry; + else + return null; + } + + public void remove(String connectionId) { + lruConnections.remove(connectionId); + } + } + } diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index 30070040a395..c46a536fea96 100755 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -100,7 +100,7 @@ class ControllerChannelManager(controllerContext: ControllerContext, config: Kaf ) val selector = new Selector( NetworkReceive.UNLIMITED, - config.connectionsMaxIdleMs, + Selector.NO_IDLE_TIMEOUT_MS, metrics, time, "controller-channel", @@ -167,7 +167,7 @@ class RequestSendThread(val controllerId: Int, override def doWork(): Unit = { - def backoff(): Unit = CoreUtils.swallowTrace(Thread.sleep(300)) + def backoff(): Unit = CoreUtils.swallowTrace(Thread.sleep(100)) val QueueItem(apiKey, apiVersion, request, callback) = queue.take() import NetworkClientBlockingOps._ @@ -226,18 +226,13 @@ class RequestSendThread(val controllerId: Int, private def brokerReady(): Boolean = { import NetworkClientBlockingOps._ try { + val ready = networkClient.blockingReady(brokerNode, socketTimeoutMs)(time) - if (networkClient.isReady(brokerNode, time.milliseconds())) - true - else { - val ready = networkClient.blockingReady(brokerNode, socketTimeoutMs)(time) + if (!ready) + throw new SocketTimeoutException(s"Failed to connect within $socketTimeoutMs ms") - if (!ready) - throw new SocketTimeoutException(s"Failed to connect within $socketTimeoutMs ms") - - info("Controller %d connected to %s for sending state change requests".format(controllerId, brokerNode.toString())) - true - } + info("Controller %d connected to %s for sending state change requests".format(controllerId, brokerNode.toString())) + true } catch { case e: Throwable => warn("Controller %d's connection to broker %s was unsuccessful".format(controllerId, brokerNode.toString()), e) diff --git a/core/src/main/scala/kafka/utils/NetworkClientBlockingOps.scala b/core/src/main/scala/kafka/utils/NetworkClientBlockingOps.scala index fd4af6e949b6..9aca6639b7cd 100644 --- a/core/src/main/scala/kafka/utils/NetworkClientBlockingOps.scala +++ b/core/src/main/scala/kafka/utils/NetworkClientBlockingOps.scala @@ -45,24 +45,42 @@ object NetworkClientBlockingOps { class NetworkClientBlockingOps(val client: NetworkClient) extends AnyVal { /** - * Invokes `client.ready` followed by 0 or more `client.poll` invocations until the connection to `node` is ready, - * the timeout expires or the connection fails. + * Invokes `client.poll` to discard pending disconnects, followed by `client.ready` and 0 or more `client.poll` + * invocations until the connection to `node` is ready, the timeout expires or the connection fails. * * It returns `true` if the call completes normally or `false` if the timeout expires. If the connection fails, - * an `IOException` is thrown instead. + * an `IOException` is thrown instead. Note that if the `NetworkClient` has been configured with a positive + * connection timeout, it is possible for this method to raise an `IOException` for a previous connection which + * has recently disconnected. * * This method is useful for implementing blocking behaviour on top of the non-blocking `NetworkClient`, use it with * care. */ def blockingReady(node: Node, timeout: Long)(implicit time: JTime): Boolean = { require(timeout >=0, "timeout should be >= 0") - client.ready(node, time.milliseconds()) || pollUntil(timeout) { (_, now) => - if (client.isReady(node, now)) + + val startTime = time.milliseconds() + val expiryTime = startTime + timeout + + @tailrec + def awaitReady(iterationStartTime: Long): Boolean = { + if (client.isReady(node, iterationStartTime)) true else if (client.connectionFailed(node)) throw new IOException(s"Connection to $node failed") - else false + else { + val pollTimeout = expiryTime - iterationStartTime + client.poll(pollTimeout, iterationStartTime) + val afterPollTime = time.milliseconds() + if (afterPollTime < expiryTime) awaitReady(afterPollTime) + else false + } } + + // poll once to receive pending disconnects + client.poll(0, startTime) + + client.ready(node, startTime) || awaitReady(startTime) } /** @@ -92,34 +110,6 @@ class NetworkClientBlockingOps(val client: NetworkClient) extends AnyVal { } - /** - * Invokes `client.poll` until `predicate` returns `true` or the timeout expires. - * - * It returns `true` if the call completes normally or `false` if the timeout expires. Exceptions thrown via - * `predicate` are not handled and will bubble up. - * - * This method is useful for implementing blocking behaviour on top of the non-blocking `NetworkClient`, use it with - * care. - */ - private def pollUntil(timeout: Long)(predicate: (Seq[ClientResponse], Long) => Boolean)(implicit time: JTime): Boolean = { - val methodStartTime = time.milliseconds() - val timeoutExpiryTime = methodStartTime + timeout - - @tailrec - def recursivePoll(iterationStartTime: Long): Boolean = { - val pollTimeout = timeoutExpiryTime - iterationStartTime - val responses = client.poll(pollTimeout, iterationStartTime).asScala - if (predicate(responses, iterationStartTime)) true - else { - val afterPollTime = time.milliseconds() - if (afterPollTime < timeoutExpiryTime) recursivePoll(afterPollTime) - else false - } - } - - recursivePoll(methodStartTime) - } - /** * Invokes `client.poll` until `collect` returns `Some`. The value inside `Some` is returned. * From a2bac70a6634e9a78734d23158fb7e45f290ea26 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 22 Aug 2016 21:33:47 -0700 Subject: [PATCH 26/35] MINOR: Refactor TopologyBuilder with ApplicationID Prefix Author: Guozhang Wang Reviewers: Damian Guy , Ewen Cheslack-Postava Closes #1736 from guozhangwang/Kminor-topology-applicationID --- .../kafka/streams/kstream/KStreamBuilder.java | 1 - .../streams/processor/TopologyBuilder.java | 393 +++++++++--------- .../processor/internals/StreamThread.java | 4 +- .../kstream/internals/KStreamBranchTest.java | 1 + .../kstream/internals/KStreamImplTest.java | 2 +- .../processor/TopologyBuilderTest.java | 29 +- .../internals/ProcessorTopologyTest.java | 4 +- .../processor/internals/StreamThreadTest.java | 14 +- .../StreamThreadStateStoreProviderTest.java | 3 +- .../apache/kafka/test/KStreamTestDriver.java | 3 +- .../test/ProcessorTopologyTestDriver.java | 2 +- 11 files changed, 232 insertions(+), 224 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java index 08e984253426..f9544cc7f1b9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java @@ -92,7 +92,6 @@ public KStream stream(Serde keySerde, Serde valSerde, String. return new KStreamImpl<>(this, name, Collections.singleton(name), false); } - /** * Create a {@link KStream} instance from the specified Pattern. *

    diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java index 7b79236da83b..bcdb54a8f10a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java @@ -60,23 +60,45 @@ public class TopologyBuilder { // state factories private final Map stateFactories = new HashMap<>(); + // all topics subscribed from source processors (without application-id prefix for internal topics) private final Set sourceTopicNames = new HashSet<>(); + + // all internal topics auto-created by the topology builder and used in source / sink processors private final Set internalTopicNames = new HashSet<>(); - private final QuickUnion nodeGrouper = new QuickUnion<>(); + + // groups of source processors that need to be copartitioned private final List> copartitionSourceGroups = new ArrayList<>(); + + // map from source processor names to subscribed topics (without application-id prefix for internal topics) private final HashMap nodeToSourceTopics = new HashMap<>(); + + // map from source processor names to regex subscription patterns private final HashMap nodeToSourcePatterns = new LinkedHashMap<>(); - private final HashMap topicToPatterns = new HashMap<>(); + + // map from sink processor names to subscribed topic (without application-id prefix for internal topics) private final HashMap nodeToSinkTopic = new HashMap<>(); + + // map from topics to their matched regex patterns, this is to ensure one topic is passed through on source node + // even if it can be matched by multiple regex patterns + private final HashMap topicToPatterns = new HashMap<>(); + + // map from state store names to all the topics subscribed from source processors that + // are connected to these state stores private final Map> stateStoreNameToSourceTopics = new HashMap<>(); + + // map from state store names that are directly associated with source processors to their subscribed topics, + // this is used in the extended KStreamBuilder. private final HashMap sourceStoreToSourceTopic = new HashMap<>(); + + private final QuickUnion nodeGrouper = new QuickUnion<>(); + private SubscriptionUpdates subscriptionUpdates = new SubscriptionUpdates(); - private String applicationId; - private Map> nodeGroups = null; - private Pattern topicPattern; + private String applicationId = null; + private Pattern topicPattern = null; + private Map> nodeGroups = null; private static class StateStoreFactory { public final Set users; @@ -98,7 +120,7 @@ private static abstract class NodeFactory { this.name = name; } - public abstract ProcessorNode build(String applicationId); + public abstract ProcessorNode build(); } private static class ProcessorNodeFactory extends NodeFactory { @@ -118,7 +140,7 @@ public void addStateStore(String stateStoreName) { @SuppressWarnings("unchecked") @Override - public ProcessorNode build(String applicationId) { + public ProcessorNode build() { return new ProcessorNode(name, supplier.get(), stateStoreNames); } } @@ -146,9 +168,12 @@ public String[] getTopics(Collection subscribedTopics) { for (String update : subscribedTopics) { if (this.pattern == topicToPatterns.get(update)) { matchedTopics.add(update); - //not same pattern instance,but still matches not allowed } else if (topicToPatterns.containsKey(update) && isMatch(update)) { - throw new TopologyBuilderException("Topic " + update + " already matched check for overlapping regex patterns"); + // the same topic cannot be matched to more than one pattern + // TODO: we should lift this requirement in the future + throw new TopologyBuilderException("Topic " + update + + " is already matched for another regex pattern " + topicToPatterns.get(update) + + " and hence cannot be matched to this regex pattern " + pattern + " any more."); } else if (isMatch(update)) { topicToPatterns.put(update, this.pattern); matchedTopics.add(update); @@ -159,7 +184,7 @@ public String[] getTopics(Collection subscribedTopics) { @SuppressWarnings("unchecked") @Override - public ProcessorNode build(String applicationId) { + public ProcessorNode build() { return new SourceNode(name, nodeToSourceTopics.get(name), keyDeserializer, valDeserializer); } @@ -186,10 +211,10 @@ private SinkNodeFactory(String name, String[] parents, String topic, Serializer @SuppressWarnings("unchecked") @Override - public ProcessorNode build(String applicationId) { + public ProcessorNode build() { if (internalTopicNames.contains(topic)) { // prefix the internal topic name with the application id - return new SinkNode(name, applicationId + "-" + topic, keySerializer, valSerializer, partitioner); + return new SinkNode(name, decorateTopic(topic), keySerializer, valSerializer, partitioner); } else { return new SinkNode(name, topic, keySerializer, valSerializer, partitioner); } @@ -231,6 +256,22 @@ public int hashCode() { */ public TopologyBuilder() {} + /** + * Set the applicationId to be used for auto-generated internal topics. + * + * This is required before calling {@link #sourceTopics}, {@link #topicGroups}, + * {@link #copartitionSources}, {@link #stateStoreNameToSourceTopics} and {@link #build(Integer)}. + * + * @param applicationId the streams applicationId. Should be the same as set by + * {@link org.apache.kafka.streams.StreamsConfig#APPLICATION_ID_CONFIG} + */ + public synchronized final TopologyBuilder setApplicationId(String applicationId) { + Objects.requireNonNull(applicationId, "applicationId can't be null"); + this.applicationId = applicationId; + + return this; + } + /** * Add a new source that consumes the named topics and forward the records to child processor and/or sink nodes. * The source will use the {@link org.apache.kafka.streams.StreamsConfig#KEY_SERDE_CLASS_CONFIG default key deserializer} and @@ -342,8 +383,8 @@ public synchronized final TopologyBuilder addSource(String name, Deserializer ke } } - nodeToSourcePatterns.put(name, topicPattern); nodeFactories.put(name, new SourceNodeFactory(name, null, topicPattern, keyDeserializer, valDeserializer)); + nodeToSourcePatterns.put(name, topicPattern); nodeGrouper.add(name); return this; @@ -549,14 +590,10 @@ public synchronized final TopologyBuilder connectProcessorAndStateStores(String } protected synchronized final TopologyBuilder connectSourceStoreAndTopic(String sourceStoreName, String topic) { - if (sourceStoreToSourceTopic != null) { - if (sourceStoreToSourceTopic.containsKey(sourceStoreName)) { - throw new TopologyBuilderException("Source store " + sourceStoreName + " is already added."); - } - sourceStoreToSourceTopic.put(sourceStoreName, topic); - } else { - throw new TopologyBuilderException("sourceStoreToSourceTopic is null"); + if (sourceStoreToSourceTopic.containsKey(sourceStoreName)) { + throw new TopologyBuilderException("Source store " + sourceStoreName + " is already added."); } + sourceStoreToSourceTopic.put(sourceStoreName, topic); return this; } @@ -601,6 +638,17 @@ public synchronized final TopologyBuilder addInternalTopic(String topicName) { return this; } + /** + * Asserts that the streams of the specified source nodes must be copartitioned. + * + * @param sourceNodes a set of source node names + * @return this builder instance so methods can be chained together; never null + */ + public synchronized final TopologyBuilder copartitionSources(Collection sourceNodes) { + copartitionSourceGroups.add(Collections.unmodifiableSet(new HashSet<>(sourceNodes))); + return this; + } + private void connectProcessorAndStateStore(String processorName, String stateStoreName) { if (!stateFactories.containsKey(stateStoreName)) throw new TopologyBuilderException("StateStore " + stateStoreName + " is not added yet."); @@ -625,7 +673,6 @@ private void connectProcessorAndStateStore(String processorName, String stateSto } } - private Set findSourceTopicsForProcessorParents(String [] parents) { final Set sourceTopics = new HashSet<>(); for (String parent : parents) { @@ -651,85 +698,6 @@ private void connectStateStoreNameToSourceTopics(final String stateStoreName, Collections.unmodifiableSet(sourceTopics)); } - /** - * Returns the map of topic groups keyed by the group id. - * A topic group is a group of topics in the same task. - * - * @return groups of topic names - */ - public synchronized Map topicGroups() { - Map topicGroups = new LinkedHashMap<>(); - - - if (subscriptionUpdates.hasUpdates()) { - for (Map.Entry stringPatternEntry : nodeToSourcePatterns.entrySet()) { - SourceNodeFactory sourceNode = (SourceNodeFactory) nodeFactories.get(stringPatternEntry.getKey()); - //need to update nodeToSourceTopics with topics matched from given regex - nodeToSourceTopics.put(stringPatternEntry.getKey(), sourceNode.getTopics(subscriptionUpdates.getUpdates())); - } - } - - if (nodeGroups == null) - nodeGroups = makeNodeGroups(); - - - for (Map.Entry> entry : nodeGroups.entrySet()) { - Set sinkTopics = new HashSet<>(); - Set sourceTopics = new HashSet<>(); - Set internalSourceTopics = new HashSet<>(); - Set stateChangelogTopics = new HashSet<>(); - for (String node : entry.getValue()) { - // if the node is a source node, add to the source topics - String[] topics = nodeToSourceTopics.get(node); - if (topics != null) { - // if some of the topics are internal, add them to the internal topics - for (String topic : topics) { - if (this.internalTopicNames.contains(topic)) { - if (applicationId == null) { - throw new TopologyBuilderException("There are internal topics and" - + " applicationId hasn't been " - + "set. Call setApplicationId " - + "first"); - } - // prefix the internal topic name with the application id - String internalTopic = applicationId + "-" + topic; - internalSourceTopics.add(internalTopic); - sourceTopics.add(internalTopic); - } else { - sourceTopics.add(topic); - } - } - } - - // if the node is a sink node, add to the sink topics - String topic = nodeToSinkTopic.get(node); - if (topic != null) { - if (internalTopicNames.contains(topic)) { - // prefix the change log topic name with the application id - sinkTopics.add(applicationId + "-" + topic); - } else { - sinkTopics.add(topic); - } - } - - // if the node is connected to a state, add to the state topics - for (StateStoreFactory stateFactory : stateFactories.values()) { - if (stateFactory.isInternal && stateFactory.users.contains(node)) { - // prefix the change log topic name with the application id - stateChangelogTopics.add(applicationId + "-" + stateFactory.supplier.name() + ProcessorStateManager.STATE_CHANGELOG_TOPIC_SUFFIX); - } - } - } - topicGroups.put(entry.getKey(), new TopicsInfo( - Collections.unmodifiableSet(sinkTopics), - Collections.unmodifiableSet(sourceTopics), - Collections.unmodifiableSet(internalSourceTopics), - Collections.unmodifiableSet(stateChangelogTopics))); - } - - return Collections.unmodifiableMap(topicGroups); - } - /** * Returns the map of node groups keyed by the topic group id. * @@ -777,63 +745,13 @@ private Map> makeNodeGroups() { return nodeGroups; } - /** - * Asserts that the streams of the specified source nodes must be copartitioned. - * - * @param sourceNodes a set of source node names - * @return this builder instance so methods can be chained together; never null - */ - public synchronized final TopologyBuilder copartitionSources(Collection sourceNodes) { - copartitionSourceGroups.add(Collections.unmodifiableSet(new HashSet<>(sourceNodes))); - return this; - } - - /** - * Returns the copartition groups. - * A copartition group is a group of source topics that are required to be copartitioned. - * - * @return groups of topic names - */ - public synchronized Collection> copartitionGroups() { - List> list = new ArrayList<>(copartitionSourceGroups.size()); - for (Set nodeNames : copartitionSourceGroups) { - Set copartitionGroup = new HashSet<>(); - for (String node : nodeNames) { - String[] topics = nodeToSourceTopics.get(node); - if (topics != null) - copartitionGroup.addAll(convertInternalTopicNames(topics)); - } - list.add(Collections.unmodifiableSet(copartitionGroup)); - } - return Collections.unmodifiableList(list); - } - - private List convertInternalTopicNames(String...topics) { - final List topicNames = new ArrayList<>(); - for (String topic : topics) { - if (internalTopicNames.contains(topic)) { - if (applicationId == null) { - throw new TopologyBuilderException("there are internal topics " - + "and applicationId hasn't been set. Call " - + "setApplicationId first"); - } - topicNames.add(applicationId + "-" + topic); - } else { - topicNames.add(topic); - } - } - return topicNames; - } - - /** * Build the topology for the specified topic group. This is called automatically when passing this builder into the * {@link org.apache.kafka.streams.KafkaStreams#KafkaStreams(TopologyBuilder, org.apache.kafka.streams.StreamsConfig)} constructor. * * @see org.apache.kafka.streams.KafkaStreams#KafkaStreams(TopologyBuilder, org.apache.kafka.streams.StreamsConfig) */ - public synchronized ProcessorTopology build(String applicationId, Integer topicGroupId) { - Objects.requireNonNull(applicationId, "applicationId can't be null"); + public synchronized ProcessorTopology build(Integer topicGroupId) { Set nodeGroup; if (topicGroupId != null) { nodeGroup = nodeGroups().get(topicGroupId); @@ -841,11 +759,11 @@ public synchronized ProcessorTopology build(String applicationId, Integer topicG // when nodeGroup is null, we build the full topology. this is used in some tests. nodeGroup = null; } - return build(applicationId, nodeGroup); + return build(nodeGroup); } @SuppressWarnings("unchecked") - private ProcessorTopology build(String applicationId, Set nodeGroup) { + private ProcessorTopology build(Set nodeGroup) { List processorNodes = new ArrayList<>(nodeFactories.size()); Map processorMap = new HashMap<>(); Map topicSourceMap = new HashMap<>(); @@ -855,7 +773,7 @@ private ProcessorTopology build(String applicationId, Set nodeGroup) { // create processor nodes in a topological order ("nodeFactories" is already topologically sorted) for (NodeFactory factory : nodeFactories.values()) { if (nodeGroup == null || nodeGroup.contains(factory.name)) { - ProcessorNode node = factory.build(applicationId); + ProcessorNode node = factory.build(); processorNodes.add(node); processorMap.put(node.name(), node); @@ -870,11 +788,13 @@ private ProcessorTopology build(String applicationId, Set nodeGroup) { } } else if (factory instanceof SourceNodeFactory) { SourceNodeFactory sourceNodeFactory = (SourceNodeFactory) factory; - String[] topics = (sourceNodeFactory.pattern != null) ? sourceNodeFactory.getTopics(subscriptionUpdates.getUpdates()) : sourceNodeFactory.getTopics(); + String[] topics = (sourceNodeFactory.pattern != null) ? + sourceNodeFactory.getTopics(subscriptionUpdates.getUpdates()) : + sourceNodeFactory.getTopics(); for (String topic : topics) { if (internalTopicNames.contains(topic)) { // prefix the internal topic name with the application id - topicSourceMap.put(applicationId + "-" + topic, (SourceNode) node); + topicSourceMap.put(decorateTopic(topic), (SourceNode) node); } else { topicSourceMap.put(topic, (SourceNode) node); } @@ -885,7 +805,7 @@ private ProcessorTopology build(String applicationId, Set nodeGroup) { processorMap.get(parent).addChild(node); if (internalTopicNames.contains(sinkNodeFactory.topic)) { // prefix the internal topic name with the application id - topicSinkMap.put(applicationId + "-" + sinkNodeFactory.topic, (SinkNode) node); + topicSinkMap.put(decorateTopic(sinkNodeFactory.topic), (SinkNode) node); } else { topicSinkMap.put(sinkNodeFactory.topic, (SinkNode) node); } @@ -899,6 +819,78 @@ private ProcessorTopology build(String applicationId, Set nodeGroup) { return new ProcessorTopology(processorNodes, topicSourceMap, topicSinkMap, new ArrayList<>(stateStoreMap.values()), sourceStoreToSourceTopic); } + /** + * Returns the map of topic groups keyed by the group id. + * A topic group is a group of topics in the same task. + * + * @return groups of topic names + */ + public synchronized Map topicGroups() { + Map topicGroups = new LinkedHashMap<>(); + + if (subscriptionUpdates.hasUpdates()) { + for (Map.Entry stringPatternEntry : nodeToSourcePatterns.entrySet()) { + SourceNodeFactory sourceNode = (SourceNodeFactory) nodeFactories.get(stringPatternEntry.getKey()); + //need to update nodeToSourceTopics with topics matched from given regex + nodeToSourceTopics.put(stringPatternEntry.getKey(), sourceNode.getTopics(subscriptionUpdates.getUpdates())); + } + } + + if (nodeGroups == null) + nodeGroups = makeNodeGroups(); + + for (Map.Entry> entry : nodeGroups.entrySet()) { + Set sinkTopics = new HashSet<>(); + Set sourceTopics = new HashSet<>(); + Set internalSourceTopics = new HashSet<>(); + Set stateChangelogTopics = new HashSet<>(); + for (String node : entry.getValue()) { + // if the node is a source node, add to the source topics + String[] topics = nodeToSourceTopics.get(node); + if (topics != null) { + // if some of the topics are internal, add them to the internal topics + for (String topic : topics) { + if (this.internalTopicNames.contains(topic)) { + // prefix the internal topic name with the application id + String internalTopic = decorateTopic(topic); + internalSourceTopics.add(internalTopic); + sourceTopics.add(internalTopic); + } else { + sourceTopics.add(topic); + } + } + } + + // if the node is a sink node, add to the sink topics + String topic = nodeToSinkTopic.get(node); + if (topic != null) { + if (internalTopicNames.contains(topic)) { + // prefix the change log topic name with the application id + sinkTopics.add(decorateTopic(topic)); + } else { + sinkTopics.add(topic); + } + } + + // if the node is connected to a state, add to the state topics + for (StateStoreFactory stateFactory : stateFactories.values()) { + if (stateFactory.isInternal && stateFactory.users.contains(node)) { + // prefix the change log topic name with the application id + stateChangelogTopics.add(ProcessorStateManager.storeChangelogTopic(applicationId, stateFactory.supplier.name())); + } + } + } + topicGroups.put(entry.getKey(), new TopicsInfo( + Collections.unmodifiableSet(sinkTopics), + Collections.unmodifiableSet(sourceTopics), + Collections.unmodifiableSet(internalSourceTopics), + Collections.unmodifiableSet(stateChangelogTopics))); + } + + return Collections.unmodifiableMap(topicGroups); + } + + /** * Get the names of topics that are to be consumed by the source nodes created by this builder. * @return the unmodifiable set of topic names used by source nodes, which changes as new sources are added; never null @@ -908,21 +900,62 @@ public synchronized Set sourceTopics() { return Collections.unmodifiableSet(topics); } - private Set maybeDecorateInternalSourceTopics(final Set sourceTopicNames) { - Set topics = new HashSet<>(); - for (String topic : sourceTopicNames) { + /** + * @return a mapping from state store name to a Set of source Topics. + */ + public Map> stateStoreNameToSourceTopics() { + final Map> results = new HashMap<>(); + for (Map.Entry> entry : stateStoreNameToSourceTopics.entrySet()) { + results.put(entry.getKey(), maybeDecorateInternalSourceTopics(entry.getValue())); + + } + return results; + } + + /** + * Returns the copartition groups. + * A copartition group is a group of source topics that are required to be copartitioned. + * + * @return groups of topic names + */ + public synchronized Collection> copartitionGroups() { + List> list = new ArrayList<>(copartitionSourceGroups.size()); + for (Set nodeNames : copartitionSourceGroups) { + Set copartitionGroup = new HashSet<>(); + for (String node : nodeNames) { + String[] topics = nodeToSourceTopics.get(node); + if (topics != null) + copartitionGroup.addAll(maybeDecorateInternalSourceTopics(topics)); + } + list.add(Collections.unmodifiableSet(copartitionGroup)); + } + return Collections.unmodifiableList(list); + } + + private Set maybeDecorateInternalSourceTopics(final Set sourceTopics) { + return maybeDecorateInternalSourceTopics(sourceTopics.toArray(new String[sourceTopics.size()])); + } + + private Set maybeDecorateInternalSourceTopics(String ... sourceTopics) { + final Set decoratedTopics = new HashSet<>(); + for (String topic : sourceTopics) { if (internalTopicNames.contains(topic)) { - if (applicationId == null) { - throw new TopologyBuilderException("there are internal topics and " - + "applicationId is null. Call " - + "setApplicationId first"); - } - topics.add(applicationId + "-" + topic); + decoratedTopics.add(decorateTopic(topic)); } else { - topics.add(topic); + decoratedTopics.add(topic); } } - return topics; + return decoratedTopics; + } + + private String decorateTopic(String topic) { + if (applicationId == null) { + throw new TopologyBuilderException("there are internal topics and " + + "applicationId hasn't been set. Call " + + "setApplicationId first"); + } + + return applicationId + "-" + topic; } public synchronized Pattern sourceTopicPattern() { @@ -948,28 +981,4 @@ public synchronized Pattern sourceTopicPattern() { public synchronized void updateSubscriptions(SubscriptionUpdates subscriptionUpdates) { this.subscriptionUpdates = subscriptionUpdates; } - - /** - * Set the applicationId. This is required before calling - * {@link #sourceTopics}, {@link #topicGroups}, {@link #copartitionSources}, and - * {@link #stateStoreNameToSourceTopics} - * @param applicationId the streams applicationId. Should be the same as set by - * {@link org.apache.kafka.streams.StreamsConfig#APPLICATION_ID_CONFIG} - */ - public synchronized void setApplicationId(String applicationId) { - Objects.requireNonNull(applicationId, "applicationId can't be null"); - this.applicationId = applicationId; - } - - /** - * @return a mapping from state store name to a Set of source Topics. - */ - public Map> stateStoreNameToSourceTopics() { - final Map> results = new HashMap<>(); - for (Map.Entry> entry : stateStoreNameToSourceTopics.entrySet()) { - results.put(entry.getKey(), maybeDecorateInternalSourceTopics(entry.getValue())); - - } - return results; - } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index 50d77c364012..c0e54b9737a0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -542,7 +542,7 @@ public Set cachedTasks() { protected StreamTask createStreamTask(TaskId id, Collection partitions) { sensors.taskCreationSensor.record(); - ProcessorTopology topology = builder.build(applicationId, id.topicGroupId); + ProcessorTopology topology = builder.build(id.topicGroupId); return new StreamTask(id, applicationId, partitions, topology, consumer, producer, restoreConsumer, config, sensors, stateDirectory); } @@ -612,7 +612,7 @@ private void closeOne(AbstractTask task) { protected StandbyTask createStandbyTask(TaskId id, Collection partitions) { sensors.taskCreationSensor.record(); - ProcessorTopology topology = builder.build(applicationId, id.topicGroupId); + ProcessorTopology topology = builder.build(id.topicGroupId); if (!topology.stateStoreSuppliers().isEmpty()) { return new StandbyTask(id, applicationId, partitions, topology, consumer, restoreConsumer, config, sensors, stateDirectory); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java index 0650b95fef86..fb19c0f8e6c8 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java @@ -48,6 +48,7 @@ public void cleanup() { @Test public void testKStreamBranch() { KStreamBuilder builder = new KStreamBuilder(); + builder.setApplicationId("X"); Predicate isEven = new Predicate() { @Override diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java index 9cbc156f80a7..fb2afec67526 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java @@ -143,7 +143,7 @@ public Integer apply(Integer value1, Integer value2) { 1 + // to 2 + // through 1, // process - builder.build("X", null).processors().size()); + builder.setApplicationId("X").build(null).processors().size()); } @Test diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuilderTest.java index 6f047b07332a..fe66acb8f087 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuilderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuilderTest.java @@ -227,17 +227,18 @@ public void testAddStateStoreWithDuplicates() { @Test public void testAddStateStore() { final TopologyBuilder builder = new TopologyBuilder(); - List suppliers; StateStoreSupplier supplier = new MockStateStoreSupplier("store-1", false); builder.addStateStore(supplier); - suppliers = builder.build("X", null).stateStoreSuppliers(); - assertEquals(0, suppliers.size()); + builder.setApplicationId("X"); + + assertEquals(0, builder.build(null).stateStoreSuppliers().size()); builder.addSource("source-1", "topic-1"); builder.addProcessor("processor-1", new MockProcessorSupplier(), "source-1"); builder.connectProcessorAndStateStores("processor-1", "store-1"); - suppliers = builder.build("X", null).stateStoreSuppliers(); + + List suppliers = builder.build(null).stateStoreSuppliers(); assertEquals(1, suppliers.size()); assertEquals(supplier.name(), suppliers.get(0).name()); } @@ -245,7 +246,8 @@ public void testAddStateStore() { @Test public void testTopicGroups() { final TopologyBuilder builder = new TopologyBuilder(); - + builder.setApplicationId("X"); + builder.addInternalTopic("topic-1x"); builder.addSource("source-1", "topic-1", "topic-1x"); builder.addSource("source-2", "topic-2"); builder.addSource("source-3", "topic-3"); @@ -262,7 +264,7 @@ public void testTopicGroups() { Map topicGroups = builder.topicGroups(); Map expectedTopicGroups = new HashMap<>(); - expectedTopicGroups.put(0, new TopicsInfo(Collections.emptySet(), mkSet("topic-1", "topic-1x", "topic-2"), Collections.emptySet(), Collections.emptySet())); + expectedTopicGroups.put(0, new TopicsInfo(Collections.emptySet(), mkSet("topic-1", "X-topic-1x", "topic-2"), Collections.emptySet(), Collections.emptySet())); expectedTopicGroups.put(1, new TopicsInfo(Collections.emptySet(), mkSet("topic-3", "topic-4"), Collections.emptySet(), Collections.emptySet())); expectedTopicGroups.put(2, new TopicsInfo(Collections.emptySet(), mkSet("topic-5"), Collections.emptySet(), Collections.emptySet())); @@ -271,7 +273,7 @@ public void testTopicGroups() { Collection> copartitionGroups = builder.copartitionGroups(); - assertEquals(mkSet(mkSet("topic-1", "topic-1x", "topic-2")), new HashSet<>(copartitionGroups)); + assertEquals(mkSet(mkSet("topic-1", "X-topic-1x", "topic-2")), new HashSet<>(copartitionGroups)); } @Test @@ -322,9 +324,10 @@ public void testBuild() { builder.addProcessor("processor-2", new MockProcessorSupplier(), "source-2", "processor-1"); builder.addProcessor("processor-3", new MockProcessorSupplier(), "source-3", "source-4"); - ProcessorTopology topology0 = builder.build("X", 0); - ProcessorTopology topology1 = builder.build("X", 1); - ProcessorTopology topology2 = builder.build("X", 2); + builder.setApplicationId("X"); + ProcessorTopology topology0 = builder.build(0); + ProcessorTopology topology1 = builder.build(1); + ProcessorTopology topology2 = builder.build(2); assertEquals(mkSet("source-1", "source-2", "processor-1", "processor-2"), nodeNames(topology0.processors())); assertEquals(mkSet("source-3", "source-4", "processor-3"), nodeNames(topology1.processors())); @@ -378,12 +381,6 @@ public void shouldNotAddNullInternalTopic() throws Exception { builder.addInternalTopic(null); } - @Test(expected = NullPointerException.class) - public void shouldNotAllowNullApplicationIdOnBuild() throws Exception { - final TopologyBuilder builder = new TopologyBuilder(); - builder.build(null, 1); - } - @Test(expected = NullPointerException.class) public void shouldNotSetApplicationIdToNull() throws Exception { final TopologyBuilder builder = new TopologyBuilder(); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java index d08780b5421e..f7ef7f72204f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java @@ -87,7 +87,7 @@ public void cleanup() { @Test public void testTopologyMetadata() { - final TopologyBuilder builder = new TopologyBuilder(); + final TopologyBuilder builder = new TopologyBuilder().setApplicationId("X"); builder.addSource("source-1", "topic-1"); builder.addSource("source-2", "topic-2", "topic-3"); @@ -96,7 +96,7 @@ public void testTopologyMetadata() { builder.addSink("sink-1", "topic-3", "processor-1"); builder.addSink("sink-2", "topic-4", "processor-1", "processor-2"); - final ProcessorTopology topology = builder.build("X", null); + final ProcessorTopology topology = builder.build(null); assertEquals(6, topology.processors().size()); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java index 1a66d321147b..1da759241ad2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java @@ -153,7 +153,7 @@ protected void initializeOffsetLimits() { public void testPartitionAssignmentChange() throws Exception { StreamsConfig config = new StreamsConfig(configProps()); - TopologyBuilder builder = new TopologyBuilder(); + TopologyBuilder builder = new TopologyBuilder().setApplicationId("X"); builder.addSource("source1", "topic1"); builder.addSource("source2", "topic2"); builder.addSource("source3", "topic3"); @@ -162,7 +162,7 @@ public void testPartitionAssignmentChange() throws Exception { StreamThread thread = new StreamThread(builder, config, new MockClientSupplier(), applicationId, clientId, processId, new Metrics(), new SystemTime(), new StreamsMetadataState(builder)) { @Override protected StreamTask createStreamTask(TaskId id, Collection partitionsForTask) { - ProcessorTopology topology = builder.build("X", id.topicGroupId); + ProcessorTopology topology = builder.build(id.topicGroupId); return new TestStreamTask(id, applicationId, partitionsForTask, topology, consumer, producer, restoreConsumer, config, stateDirectory); } }; @@ -275,7 +275,7 @@ public void testMaybeClean() throws Exception { MockTime mockTime = new MockTime(); - TopologyBuilder builder = new TopologyBuilder(); + TopologyBuilder builder = new TopologyBuilder().setApplicationId("X"); builder.addSource("source1", "topic1"); StreamThread thread = new StreamThread(builder, config, new MockClientSupplier(), applicationId, clientId, processId, new Metrics(), mockTime, new StreamsMetadataState(builder)) { @@ -286,7 +286,7 @@ public void maybeClean() { @Override protected StreamTask createStreamTask(TaskId id, Collection partitionsForTask) { - ProcessorTopology topology = builder.build("X", id.topicGroupId); + ProcessorTopology topology = builder.build(id.topicGroupId); return new TestStreamTask(id, applicationId, partitionsForTask, topology, consumer, producer, restoreConsumer, config, stateDirectory); } }; @@ -394,7 +394,7 @@ public void testMaybeCommit() throws Exception { MockTime mockTime = new MockTime(); - TopologyBuilder builder = new TopologyBuilder(); + TopologyBuilder builder = new TopologyBuilder().setApplicationId("X"); builder.addSource("source1", "topic1"); StreamThread thread = new StreamThread(builder, config, new MockClientSupplier(), applicationId, clientId, processId, new Metrics(), mockTime, new StreamsMetadataState(builder)) { @@ -405,7 +405,7 @@ public void maybeCommit() { @Override protected StreamTask createStreamTask(TaskId id, Collection partitionsForTask) { - ProcessorTopology topology = builder.build("X", id.topicGroupId); + ProcessorTopology topology = builder.build(id.topicGroupId); return new TestStreamTask(id, applicationId, partitionsForTask, topology, consumer, producer, restoreConsumer, config, stateDirectory); } }; @@ -465,7 +465,7 @@ protected StreamTask createStreamTask(TaskId id, Collection part @Test public void testInjectClients() { - TopologyBuilder builder = new TopologyBuilder(); + TopologyBuilder builder = new TopologyBuilder().setApplicationId("X"); StreamsConfig config = new StreamsConfig(configProps()); MockClientSupplier clientSupplier = new MockClientSupplier(); StreamThread thread = new StreamThread(builder, config, clientSupplier, applicationId, diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java index 1baedbb51463..795d7daaccb8 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java @@ -92,7 +92,8 @@ public void before() throws IOException { configureRestoreConsumer(clientSupplier, "applicationId-kv-store-changelog"); configureRestoreConsumer(clientSupplier, "applicationId-window-store-changelog"); - final ProcessorTopology topology = builder.build("X", null); + builder.setApplicationId(applicationId); + final ProcessorTopology topology = builder.build(null); final Map tasks = new HashMap<>(); stateDirectory = new StateDirectory(applicationId, stateConfigDir); taskOne = createStreamsTask(applicationId, streamsConfig, clientSupplier, topology, diff --git a/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java b/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java index 73168048f5e1..3901d3a9bbb7 100644 --- a/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java @@ -57,7 +57,8 @@ public KStreamTestDriver(KStreamBuilder builder, File stateDir, Serde keySerde, Serde valSerde) { - this.topology = builder.build("X", null); + builder.setApplicationId("TestDriver"); + this.topology = builder.build(null); this.stateDir = stateDir; this.context = new MockProcessorContext(this, stateDir, keySerde, valSerde, new MockRecordCollector()); this.context.setTime(0L); diff --git a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java index d2d9668fcbc0..6b8d969c3a89 100644 --- a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java @@ -148,7 +148,7 @@ public class ProcessorTopologyTestDriver { */ public ProcessorTopologyTestDriver(StreamsConfig config, TopologyBuilder builder, String... storeNames) { id = new TaskId(0, 0); - topology = builder.build("X", null); + topology = builder.setApplicationId("ProcessorTopologyTestDriver").build(null); // Set up the consumer and producer ... consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); From a1e0b2240dba0740135621d959441eefa6fd3124 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Mon, 22 Aug 2016 21:49:40 -0700 Subject: [PATCH 27/35] KAFKA-4073; MirrorMaker should handle messages without timestamp correctly Author: Ismael Juma Reviewers: Jun Rao Closes #1773 from ijuma/kafka-4073-mirror-maker-timestamps --- .../main/scala/kafka/tools/MirrorMaker.scala | 4 +++- .../unit/kafka/tools/MirrorMakerTest.scala | 18 +++++++++++++++++- 2 files changed, 20 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/tools/MirrorMaker.scala b/core/src/main/scala/kafka/tools/MirrorMaker.scala index f8000327a973..434607425e1e 100755 --- a/core/src/main/scala/kafka/tools/MirrorMaker.scala +++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala @@ -38,6 +38,7 @@ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.serialization.ByteArrayDeserializer import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.errors.WakeupException +import org.apache.kafka.common.record.Record import scala.collection.JavaConversions._ import scala.collection.mutable.HashMap @@ -680,7 +681,8 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { private[tools] object defaultMirrorMakerMessageHandler extends MirrorMakerMessageHandler { override def handle(record: BaseConsumerRecord): util.List[ProducerRecord[Array[Byte], Array[Byte]]] = { - Collections.singletonList(new ProducerRecord[Array[Byte], Array[Byte]](record.topic, null, record.timestamp, record.key, record.value)) + val timestamp: java.lang.Long = if (record.timestamp == Record.NO_TIMESTAMP) null else record.timestamp + Collections.singletonList(new ProducerRecord[Array[Byte], Array[Byte]](record.topic, null, timestamp, record.key, record.value)) } } diff --git a/core/src/test/scala/unit/kafka/tools/MirrorMakerTest.scala b/core/src/test/scala/unit/kafka/tools/MirrorMakerTest.scala index 39a0ac9d6d12..d6a5470ce43f 100644 --- a/core/src/test/scala/unit/kafka/tools/MirrorMakerTest.scala +++ b/core/src/test/scala/unit/kafka/tools/MirrorMakerTest.scala @@ -18,7 +18,7 @@ package kafka.tools import kafka.consumer.BaseConsumerRecord -import org.apache.kafka.common.record.TimestampType +import org.apache.kafka.common.record.{Record, TimestampType} import org.junit.Assert._ import org.junit.Test @@ -39,4 +39,20 @@ class MirrorMakerTest { assertEquals("key", new String(producerRecord.key)) assertEquals("value", new String(producerRecord.value)) } + + @Test + def testDefaultMirrorMakerMessageHandlerWithNoTimestampInSourceMessage() { + val consumerRecord = BaseConsumerRecord("topic", 0, 1L, Record.NO_TIMESTAMP, TimestampType.CREATE_TIME, "key".getBytes, "value".getBytes) + + val result = MirrorMaker.defaultMirrorMakerMessageHandler.handle(consumerRecord) + assertEquals(1, result.size) + + val producerRecord = result.get(0) + assertNull(producerRecord.timestamp) + assertEquals("topic", producerRecord.topic) + assertNull(producerRecord.partition) + assertEquals("key", new String(producerRecord.key)) + assertEquals("value", new String(producerRecord.value)) + } + } From 313aad423cb19618b8693af6f04b68138059c585 Mon Sep 17 00:00:00 2001 From: Rajini Sivaram Date: Tue, 23 Aug 2016 10:31:29 +0100 Subject: [PATCH 28/35] KAFKA-3680; Enable Kafka clients to run in any classloader env Configure default classes using class objects instead of class names, enable configurable lists of classes to be specified as class objects, add tests for different classloader configurations. Author: Rajini Sivaram Reviewers: Sriharsha Chintalapani , Ismael Juma Closes #1421 from rajinisivaram/KAFKA-3680 --- .../clients/consumer/ConsumerConfig.java | 3 +- .../clients/producer/ProducerConfig.java | 2 +- .../kafka/common/config/AbstractConfig.java | 17 ++- .../common/config/AbstractConfigTest.java | 125 +++++++++++++++++- 4 files changed, 133 insertions(+), 14 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 509c3a1d7fac..4ce908e38c37 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 @@ -19,6 +19,7 @@ import org.apache.kafka.common.config.ConfigDef.Type; import org.apache.kafka.common.serialization.Deserializer; +import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.Properties; @@ -215,7 +216,7 @@ public class ConsumerConfig extends AbstractConfig { HEARTBEAT_INTERVAL_MS_DOC) .define(PARTITION_ASSIGNMENT_STRATEGY_CONFIG, Type.LIST, - RangeAssignor.class.getName(), + Collections.singletonList(RangeAssignor.class), Importance.MEDIUM, PARTITION_ASSIGNMENT_STRATEGY_DOC) .define(METADATA_MAX_AGE_CONFIG, diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index dbbde06009db..2ca21832de6c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java @@ -293,7 +293,7 @@ public class ProducerConfig extends AbstractConfig { CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_DOC) .define(PARTITIONER_CLASS_CONFIG, Type.CLASS, - DefaultPartitioner.class.getName(), + DefaultPartitioner.class, Importance.MEDIUM, PARTITIONER_CLASS_DOC) .define(INTERCEPTOR_CLASSES_CONFIG, Type.LIST, 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 9f3cba4faa36..096047f39574 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 @@ -221,13 +221,18 @@ public List getConfiguredInstances(String key, Class t) { List objects = new ArrayList(); if (klasses == null) return objects; - for (String klass : klasses) { + for (Object klass : klasses) { Object o; - try { - o = Utils.newInstance(klass, t); - } catch (ClassNotFoundException e) { - throw new KafkaException(klass + " ClassNotFoundException exception occured", e); - } + if (klass instanceof String) { + try { + o = Utils.newInstance((String) klass, t); + } catch (ClassNotFoundException e) { + throw new KafkaException(klass + " ClassNotFoundException exception occured", e); + } + } else if (klass instanceof Class) { + o = Utils.newInstance((Class) klass); + } else + throw new KafkaException("List contains element of type " + klass.getClass() + ", expected String or Class"); if (!t.isInstance(o)) throw new KafkaException(klass + " is not an instance of " + t.getName()); if (o instanceof Configurable) diff --git a/clients/src/test/java/org/apache/kafka/common/config/AbstractConfigTest.java b/clients/src/test/java/org/apache/kafka/common/config/AbstractConfigTest.java index d9404c277bea..d483ef0591da 100644 --- a/clients/src/test/java/org/apache/kafka/common/config/AbstractConfigTest.java +++ b/clients/src/test/java/org/apache/kafka/common/config/AbstractConfigTest.java @@ -16,10 +16,13 @@ import org.apache.kafka.common.config.ConfigDef.Importance; import org.apache.kafka.common.config.ConfigDef.Type; import org.apache.kafka.common.metrics.FakeMetricsReporter; +import org.apache.kafka.common.metrics.JmxReporter; import org.apache.kafka.common.metrics.MetricsReporter; import org.junit.Test; +import java.util.Arrays; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Properties; @@ -96,6 +99,120 @@ private void testInvalidInputs(String configValue) { } } + @Test + public void testClassConfigs() { + class RestrictedClassLoader extends ClassLoader { + public RestrictedClassLoader() { + super(null); + } + @Override + protected Class findClass(String name) throws ClassNotFoundException { + if (name.equals(ClassTestConfig.DEFAULT_CLASS.getName()) || name.equals(ClassTestConfig.RESTRICTED_CLASS.getName())) + return null; + else + return ClassTestConfig.class.getClassLoader().loadClass(name); + } + } + + ClassLoader restrictedClassLoader = new RestrictedClassLoader(); + ClassLoader defaultClassLoader = AbstractConfig.class.getClassLoader(); + + // Test default classloading where all classes are visible to thread context classloader + Thread.currentThread().setContextClassLoader(defaultClassLoader); + ClassTestConfig testConfig = new ClassTestConfig(); + testConfig.checkInstances(ClassTestConfig.DEFAULT_CLASS, ClassTestConfig.DEFAULT_CLASS); + + // Test default classloading where default classes are not visible to thread context classloader + // Static classloading is used for default classes, so instance creation should succeed. + Thread.currentThread().setContextClassLoader(restrictedClassLoader); + testConfig = new ClassTestConfig(); + testConfig.checkInstances(ClassTestConfig.DEFAULT_CLASS, ClassTestConfig.DEFAULT_CLASS); + + // Test class overrides with names or classes where all classes are visible to thread context classloader + Thread.currentThread().setContextClassLoader(defaultClassLoader); + ClassTestConfig.testOverrides(); + + // Test class overrides with names or classes where all classes are visible to Kafka classloader, context classloader is null + Thread.currentThread().setContextClassLoader(null); + ClassTestConfig.testOverrides(); + + // Test class overrides where some classes are not visible to thread context classloader + Thread.currentThread().setContextClassLoader(restrictedClassLoader); + // Properties specified as classes should succeed + testConfig = new ClassTestConfig(ClassTestConfig.RESTRICTED_CLASS, Arrays.asList(ClassTestConfig.RESTRICTED_CLASS)); + testConfig.checkInstances(ClassTestConfig.RESTRICTED_CLASS, ClassTestConfig.RESTRICTED_CLASS); + testConfig = new ClassTestConfig(ClassTestConfig.RESTRICTED_CLASS, Arrays.asList(ClassTestConfig.VISIBLE_CLASS, ClassTestConfig.RESTRICTED_CLASS)); + testConfig.checkInstances(ClassTestConfig.RESTRICTED_CLASS, ClassTestConfig.VISIBLE_CLASS, ClassTestConfig.RESTRICTED_CLASS); + // Properties specified as classNames should fail to load classes + try { + new ClassTestConfig(ClassTestConfig.RESTRICTED_CLASS.getName(), null); + fail("Config created with class property that cannot be loaded"); + } catch (ConfigException e) { + // Expected Exception + } + try { + testConfig = new ClassTestConfig(null, Arrays.asList(ClassTestConfig.VISIBLE_CLASS.getName(), ClassTestConfig.RESTRICTED_CLASS.getName())); + testConfig.getConfiguredInstances("list.prop", MetricsReporter.class); + fail("Should have failed to load class"); + } catch (KafkaException e) { + // Expected Exception + } + try { + testConfig = new ClassTestConfig(null, ClassTestConfig.VISIBLE_CLASS.getName() + "," + ClassTestConfig.RESTRICTED_CLASS.getName()); + testConfig.getConfiguredInstances("list.prop", MetricsReporter.class); + fail("Should have failed to load class"); + } catch (KafkaException e) { + // Expected Exception + } + } + + private static class ClassTestConfig extends AbstractConfig { + static final Class DEFAULT_CLASS = FakeMetricsReporter.class; + static final Class VISIBLE_CLASS = JmxReporter.class; + static final Class RESTRICTED_CLASS = ConfiguredFakeMetricsReporter.class; + + private static final ConfigDef CONFIG; + static { + CONFIG = new ConfigDef().define("class.prop", Type.CLASS, DEFAULT_CLASS, Importance.HIGH, "docs") + .define("list.prop", Type.LIST, Arrays.asList(DEFAULT_CLASS), Importance.HIGH, "docs"); + } + + public ClassTestConfig() { + super(CONFIG, new Properties()); + } + + public ClassTestConfig(Object classPropOverride, Object listPropOverride) { + super(CONFIG, overrideProps(classPropOverride, listPropOverride)); + } + + void checkInstances(Class expectedClassPropClass, Class... expectedListPropClasses) { + assertEquals(expectedClassPropClass, getConfiguredInstance("class.prop", MetricsReporter.class).getClass()); + List list = getConfiguredInstances("list.prop", MetricsReporter.class); + for (int i = 0; i < list.size(); i++) + assertEquals(expectedListPropClasses[i], list.get(i).getClass()); + } + + static void testOverrides() { + ClassTestConfig testConfig1 = new ClassTestConfig(RESTRICTED_CLASS, Arrays.asList(VISIBLE_CLASS, RESTRICTED_CLASS)); + testConfig1.checkInstances(RESTRICTED_CLASS, VISIBLE_CLASS, RESTRICTED_CLASS); + + ClassTestConfig testConfig2 = new ClassTestConfig(RESTRICTED_CLASS.getName(), Arrays.asList(VISIBLE_CLASS.getName(), RESTRICTED_CLASS.getName())); + testConfig2.checkInstances(RESTRICTED_CLASS, VISIBLE_CLASS, RESTRICTED_CLASS); + + ClassTestConfig testConfig3 = new ClassTestConfig(RESTRICTED_CLASS.getName(), VISIBLE_CLASS.getName() + "," + RESTRICTED_CLASS.getName()); + testConfig3.checkInstances(RESTRICTED_CLASS, VISIBLE_CLASS, RESTRICTED_CLASS); + } + + private static Map overrideProps(Object classProp, Object listProp) { + Map props = new HashMap<>(); + if (classProp != null) + props.put("class.prop", classProp); + if (listProp != null) + props.put("list.prop", listProp); + return props; + } + } + private static class TestConfig extends AbstractConfig { private static final ConfigDef CONFIG; @@ -127,20 +244,16 @@ public void configure(Map configs) { } public static class FakeMetricsReporterConfig extends AbstractConfig { - private static final ConfigDef CONFIG; public static final String EXTRA_CONFIG = "metric.extra_config"; private static final String EXTRA_CONFIG_DOC = "An extraneous configuration string."; - - static { - CONFIG = new ConfigDef().define( + private static final ConfigDef CONFIG = new ConfigDef().define( EXTRA_CONFIG, ConfigDef.Type.STRING, "", ConfigDef.Importance.LOW, EXTRA_CONFIG_DOC); - } + public FakeMetricsReporterConfig(Map props) { super(CONFIG, props); } } - } From cec2769e2281ac731e8cfd12b5d8e34e0ebe38cd Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Tue, 23 Aug 2016 10:19:07 -0700 Subject: [PATCH 29/35] KAFKA-2894: WorkerSinkTask should rewind offsets on rebalance Author: Konstantine Karantasis Reviewers: Ewen Cheslack-Postava Closes #1771 from kkonstantine/KAFKA-2894-rewind-offsets-on-rebalance --- .../kafka/connect/runtime/WorkerSinkTask.java | 9 +- .../runtime/WorkerSinkTaskThreadedTest.java | 90 +++++++++++++++++-- 2 files changed, 88 insertions(+), 11 deletions(-) diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java index 2a16934a0e87..624b0326db40 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java @@ -412,13 +412,16 @@ private void rewind() { if (offsets.isEmpty()) { return; } - for (TopicPartition tp: offsets.keySet()) { - Long offset = offsets.get(tp); + for (Map.Entry entry: offsets.entrySet()) { + TopicPartition tp = entry.getKey(); + Long offset = entry.getValue(); if (offset != null) { log.trace("Rewind {} to offset {}.", tp, offset); consumer.seek(tp, offset); lastCommittedOffsets.put(tp, new OffsetAndMetadata(offset)); currentOffsets.put(tp, new OffsetAndMetadata(offset)); + } else { + log.warn("Cannot rewind {} to null offset.", tp); } } context.clearOffsets(); @@ -462,6 +465,8 @@ else if (!context.pausedPartitions().isEmpty()) if (rebalanceException == null || rebalanceException instanceof WakeupException) { try { openPartitions(partitions); + // Rewind should be applied only if openPartitions succeeds. + rewind(); } catch (RuntimeException e) { // The consumer swallows exceptions raised in the rebalance listener, so we need to store // exceptions and rethrow when poll() returns. diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java index 392985b815d4..52a86ab75196 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java @@ -139,7 +139,7 @@ public void testPollsInBackground() throws Exception { expectPollInitialAssignment(); Capture> capturedRecords = expectPolls(1L); - expectStopTask(10L); + expectStopTask(); PowerMock.replayAll(); @@ -182,7 +182,7 @@ public void testCommit() throws Exception { Capture> capturedRecords = expectPolls(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_DEFAULT); expectOffsetFlush(1L, null, null, 0, true); - expectStopTask(2); + expectStopTask(); PowerMock.replayAll(); @@ -221,7 +221,7 @@ public void testCommitTaskFlushFailure() throws Exception { PowerMock.expectLastCall(); consumer.seek(TOPIC_PARTITION3, FIRST_OFFSET); PowerMock.expectLastCall(); - expectStopTask(2); + expectStopTask(); PowerMock.replayAll(); @@ -259,7 +259,7 @@ public void testCommitTaskSuccessAndFlushFailure() throws Exception { PowerMock.expectLastCall(); consumer.seek(TOPIC_PARTITION3, FIRST_OFFSET); PowerMock.expectLastCall(); - expectStopTask(2); + expectStopTask(); PowerMock.replayAll(); @@ -291,7 +291,7 @@ public void testCommitConsumerFailure() throws Exception { Capture> capturedRecords = expectPolls(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_DEFAULT); expectOffsetFlush(1L, null, new Exception(), 0, true); - expectStopTask(2); + expectStopTask(); PowerMock.replayAll(); @@ -323,7 +323,7 @@ public void testCommitTimeout() throws Exception { Capture> capturedRecords = expectPolls(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_DEFAULT / 2); expectOffsetFlush(2L, null, null, WorkerConfig.OFFSET_COMMIT_TIMEOUT_MS_DEFAULT, false); - expectStopTask(4); + expectStopTask(); PowerMock.replayAll(); @@ -402,7 +402,7 @@ public Object answer() throws Throwable { consumer.resume(Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2)); PowerMock.expectLastCall(); - expectStopTask(0); + expectStopTask(); PowerMock.replayAll(); @@ -446,7 +446,7 @@ public Object answer() throws Throwable { } }); - expectStopTask(3); + expectStopTask(); PowerMock.replayAll(); workerTask.initialize(TASK_CONFIG); @@ -460,6 +460,33 @@ public Object answer() throws Throwable { PowerMock.verifyAll(); } + @Test + public void testRewindOnRebalanceDuringPoll() throws Exception { + expectInitializeTask(); + expectPollInitialAssignment(); + + expectRebalanceDuringPoll().andAnswer(new IAnswer() { + @Override + public Object answer() throws Throwable { + Map offsets = sinkTaskContext.getValue().offsets(); + assertEquals(0, offsets.size()); + return null; + } + }); + + expectStopTask(); + PowerMock.replayAll(); + + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + workerTask.iteration(); + workerTask.iteration(); + workerTask.stop(); + workerTask.close(); + + PowerMock.verifyAll(); + } + private void expectInitializeTask() throws Exception { PowerMock.expectPrivate(workerTask, "createConsumer").andReturn(consumer); @@ -493,7 +520,7 @@ public ConsumerRecords answer() throws Throwable { EasyMock.expectLastCall(); } - private void expectStopTask(final long expectedMessages) throws Exception { + private void expectStopTask() throws Exception { sinkTask.stop(); PowerMock.expectLastCall(); @@ -561,6 +588,51 @@ public ConsumerRecords answer() throws Throwable { return EasyMock.expectLastCall(); } + @SuppressWarnings("unchecked") + private IExpectationSetters expectRebalanceDuringPoll() throws Exception { + final List partitions = Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2, TOPIC_PARTITION3); + + final long startOffset = 40L; + final Map offsets = new HashMap<>(); + offsets.put(TOPIC_PARTITION, startOffset); + + EasyMock.expect(consumer.poll(EasyMock.anyLong())).andAnswer( + new IAnswer>() { + @Override + public ConsumerRecords answer() throws Throwable { + // "Sleep" so time will progress + time.sleep(1L); + + sinkTaskContext.getValue().offset(offsets); + rebalanceListener.getValue().onPartitionsAssigned(partitions); + + ConsumerRecords records = new ConsumerRecords<>( + Collections.singletonMap( + new TopicPartition(TOPIC, PARTITION), + Arrays.asList( + new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned, TIMESTAMP, TIMESTAMP_TYPE, 0L, 0, 0, RAW_KEY, RAW_VALUE) + ))); + recordsReturned++; + return records; + } + }); + + EasyMock.expect(consumer.position(TOPIC_PARTITION)).andReturn(FIRST_OFFSET); + EasyMock.expect(consumer.position(TOPIC_PARTITION2)).andReturn(FIRST_OFFSET); + EasyMock.expect(consumer.position(TOPIC_PARTITION3)).andReturn(FIRST_OFFSET); + + sinkTask.open(partitions); + EasyMock.expectLastCall(); + + consumer.seek(TOPIC_PARTITION, startOffset); + EasyMock.expectLastCall(); + + EasyMock.expect(keyConverter.toConnectData(TOPIC, RAW_KEY)).andReturn(new SchemaAndValue(KEY_SCHEMA, KEY)); + EasyMock.expect(valueConverter.toConnectData(TOPIC, RAW_VALUE)).andReturn(new SchemaAndValue(VALUE_SCHEMA, VALUE)); + sinkTask.put(EasyMock.anyObject(Collection.class)); + return EasyMock.expectLastCall(); + } + private Capture expectOffsetFlush(final long expectedMessages, final RuntimeException flushError, final Exception consumerCommitError, From d85aaf95febca62cb03f4c2a2f84401cae4800d2 Mon Sep 17 00:00:00 2001 From: William Yu Date: Tue, 23 Aug 2016 22:20:14 +0100 Subject: [PATCH 30/35] KAFKA-3937; Kafka Clients Leak Native Memory For Longer Than Needed With Compressed Messages ijuma - Making the change against trunk based on your suggestions to have the stream closing handled in the private RecordIterator constructor which I understand is only to be used only if the block of message(s) are compressed. Author: William Yu Author: Ismael Juma Reviewers: Ismael Juma Closes #1760 from wiyu/compressor_memory_leak_in_fetcher --- .../kafka/common/record/MemoryRecords.java | 39 ++++++++++++------- .../org/apache/kafka/common/utils/Utils.java | 13 +++++++ 2 files changed, 37 insertions(+), 15 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 603f74b18e0e..3848ea99c8e1 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 @@ -21,6 +21,7 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.utils.AbstractIterator; +import org.apache.kafka.common.utils.Utils; /** * A {@link Records} implementation backed by a ByteBuffer. @@ -245,30 +246,38 @@ private RecordsIterator(LogEntry entry) { this.shallow = true; this.stream = Compressor.wrapForInput(new ByteBufferInputStream(this.buffer), type, entry.record().magic()); long wrapperRecordOffset = entry.offset(); + + long wrapperRecordTimestamp = entry.record().timestamp(); + this.logEntries = new ArrayDeque<>(); // If relative offset is used, we need to decompress the entire message first to compute - // the absolute offset. - if (entry.record().magic() > Record.MAGIC_VALUE_V0) { - this.logEntries = new ArrayDeque<>(); - long wrapperRecordTimestamp = entry.record().timestamp(); + // the absolute offset. For simplicity and because it's a format that is on its way out, we + // do the same for message format version 0 + try { while (true) { try { LogEntry logEntry = getNextEntryFromStream(); - Record recordWithTimestamp = new Record(logEntry.record().buffer(), - wrapperRecordTimestamp, - entry.record().timestampType()); - logEntries.add(new LogEntry(logEntry.offset(), recordWithTimestamp)); + if (entry.record().magic() > Record.MAGIC_VALUE_V0) { + Record recordWithTimestamp = new Record( + logEntry.record().buffer(), + wrapperRecordTimestamp, + entry.record().timestampType() + ); + logEntry = new LogEntry(logEntry.offset(), recordWithTimestamp); + } + logEntries.add(logEntry); } catch (EOFException e) { break; - } catch (IOException e) { - throw new KafkaException(e); } } - this.absoluteBaseOffset = wrapperRecordOffset - logEntries.getLast().offset(); - } else { - this.logEntries = null; - this.absoluteBaseOffset = -1; + if (entry.record().magic() > Record.MAGIC_VALUE_V0) + this.absoluteBaseOffset = wrapperRecordOffset - logEntries.getLast().offset(); + else + this.absoluteBaseOffset = -1; + } catch (IOException e) { + throw new KafkaException(e); + } finally { + Utils.closeQuietly(stream, "records iterator stream"); } - } /* 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 4629bafd75b9..8d7014a705a1 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 @@ -699,6 +699,19 @@ public static void closeAll(Closeable... closeables) throws IOException { throw exception; } + /** + * Closes {@code closeable} and if an exception is thrown, it is logged at the WARN level. + */ + public static void closeQuietly(Closeable closeable, String name) { + if (closeable != null) { + try { + closeable.close(); + } catch (Throwable t) { + log.warn("Failed to close " + name, t); + } + } + } + /** * A cheap way to deterministically convert a number to a positive value. When the input is * positive, the original value is returned. When the input number is negative, the returned From 7a877895a1c8494d48bf2c18e2f74fb913ed036a Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Tue, 23 Aug 2016 23:56:36 +0100 Subject: [PATCH 31/35] KAFKA-4082; Upgrade to Gradle 3.0 Also upgrade scoverage (required for compatibility) and remove usage of `useAnt` which doesn't exist in Gradle 3.0 It turns out that one cannot even run `gradle` to download the project Gradle version if `useAnt` is used in the build. This is unfortunate (the SBT launcher has much saner behaviour). Release notes: https://docs.gradle.org/3.0/release-notes Author: Ismael Juma Reviewers: Grant Henke , Ewen Cheslack-Postava , Sriharsha Chintalapani Closes #1774 from ijuma/kafka-4082-support-gradle-3.0 --- build.gradle | 6 ++---- gradlew | 5 +++++ gradlew.bat | 6 ------ 3 files changed, 7 insertions(+), 10 deletions(-) diff --git a/build.gradle b/build.gradle index 31a86f4b32f9..64c389e31779 100644 --- a/build.gradle +++ b/build.gradle @@ -26,7 +26,7 @@ buildscript { // For Apache Rat plugin to ignore non-Git files classpath "org.ajoberstar:grgit:1.5.0" classpath 'com.github.ben-manes:gradle-versions-plugin:0.12.0' - classpath 'org.scoverage:gradle-scoverage:2.0.1' + classpath 'org.scoverage:gradle-scoverage:2.1.0' } } @@ -68,7 +68,7 @@ allprojects { } ext { - gradleVersion = "2.14.1" + gradleVersion = "3.0" buildVersionFileName = "kafka-version.properties" userMaxForks = project.hasProperty('maxParallelForks') ? maxParallelForks.toInteger() : null @@ -236,8 +236,6 @@ subprojects { } tasks.withType(ScalaCompile) { - scalaCompileOptions.useAnt = false - scalaCompileOptions.additionalParameters = [ "-deprecation", "-unchecked", diff --git a/gradlew b/gradlew index 27309d92314c..9aa616c273d8 100755 --- a/gradlew +++ b/gradlew @@ -161,4 +161,9 @@ function splitJvmOpts() { eval splitJvmOpts $DEFAULT_JVM_OPTS $JAVA_OPTS $GRADLE_OPTS JVM_OPTS[${#JVM_OPTS[*]}]="-Dorg.gradle.appname=$APP_BASE_NAME" +# by default we should be in the correct project dir, but when run from Finder on Mac, the cwd is wrong +if [[ "$(uname)" == "Darwin" ]] && [[ "$HOME" == "$PWD" ]]; then + cd "$(dirname "$0")" +fi + exec "$JAVACMD" "${JVM_OPTS[@]}" -classpath "$CLASSPATH" org.gradle.wrapper.GradleWrapperMain "$@" diff --git a/gradlew.bat b/gradlew.bat index f6d5974e72fd..e95643d6a2ca 100644 --- a/gradlew.bat +++ b/gradlew.bat @@ -49,7 +49,6 @@ goto fail @rem Get command-line arguments, handling Windows variants if not "%OS%" == "Windows_NT" goto win9xME_args -if "%@eval[2+2]" == "4" goto 4NT_args :win9xME_args @rem Slurp the command line arguments. @@ -60,11 +59,6 @@ set _SKIP=2 if "x%~1" == "x" goto execute set CMD_LINE_ARGS=%* -goto execute - -:4NT_args -@rem Get arguments from the 4NT Shell from JP Software -set CMD_LINE_ARGS=%$ :execute @rem Setup the command line From 825c5e9a70b9781a09e033e6a8951a6319c29ae7 Mon Sep 17 00:00:00 2001 From: Ishita Mandhan Date: Tue, 23 Aug 2016 18:59:16 -0700 Subject: [PATCH 32/35] KAFKA-3940 Log should check the return value of dir.mkdirs() --- core/src/main/scala/kafka/log/Log.scala | 6 +++--- core/src/main/scala/kafka/log/LogManager.scala | 8 ++++---- .../scala/kafka/metrics/KafkaCSVMetricsReporter.scala | 5 ++--- .../unit/kafka/log/LogCleanerIntegrationTest.scala | 11 +++-------- 4 files changed, 12 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 7be8797af8dc..9af5c045161b 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -140,11 +140,11 @@ class Log(val dir: File, /* Load the log segments from the log files on disk */ private def loadSegments() { // create the log directory if it doesn't exist - if (!Files.exists(dir.toPath())) { + if (!Files.exists(dir.toPath)) { try { - Files.createDirectory(dir.toPath()) + Files.createDirectories(dir.toPath) } catch { - case e: IOException => throw new KafkaException("Error in creating new directory.", e) + case e: IOException => throw new KafkaException("Error creating log directory ${dir.toPath}", e) } } var swapFiles = Set[File]() diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index 70ba51dec541..cd359a452f7e 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -83,9 +83,9 @@ class LogManager(val logDirs: Array[File], if(!dir.exists) { info("Log directory '" + dir.getAbsolutePath + "' not found, creating it.") try { - Files.createDirectory(dir.toPath()) + Files.createDirectories(dir.toPath()) } catch { - case e: IOException => throw new KafkaException("Error in creating new directory.", e) + case e: IOException => throw new KafkaException("Error in creating log directory ${dir.toPath}", e) } } if(!dir.isDirectory || !dir.canRead) @@ -362,9 +362,9 @@ class LogManager(val logDirs: Array[File], val dataDir = nextLogDir() val dir = new File(dataDir, topicAndPartition.topic + "-" + topicAndPartition.partition) try { - Files.createDirectory(dir.toPath()) + Files.createDirectories(dir.toPath()) } catch { - case e: IOException => throw new KafkaException("Error in creating new directory.", e) + case e: IOException => throw new KafkaException("Error in creating log directory ${dir.toPath}", e) } log = new Log(dir, config, diff --git a/core/src/main/scala/kafka/metrics/KafkaCSVMetricsReporter.scala b/core/src/main/scala/kafka/metrics/KafkaCSVMetricsReporter.scala index 0f8c59e7d855..8638cd2e0ec9 100755 --- a/core/src/main/scala/kafka/metrics/KafkaCSVMetricsReporter.scala +++ b/core/src/main/scala/kafka/metrics/KafkaCSVMetricsReporter.scala @@ -53,11 +53,10 @@ private class KafkaCSVMetricsReporter extends KafkaMetricsReporter csvDir = new File(props.getString("kafka.csv.metrics.dir", "kafka_metrics")) Utils.delete(csvDir) try { - Files.createDirectory(csvDir.toPath()) + Files.createDirectories(csvDir.toPath()) } catch { - case e: IOException => throw new KafkaException("Error in creating new directory '%s'".format(csvDir), e) + case e: IOException => throw new KafkaException("Error in creating metrics directory ${csvDir}", e) } - Files.createDirectory(csvDir.toPath()) underlying = new CsvReporter(Metrics.defaultRegistry(), csvDir) if (props.getBoolean("kafka.csv.metrics.reporter.enabled", default = false)) { initialized = true diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala index d3bf0303c02d..d1987bc9570a 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala @@ -17,12 +17,11 @@ package kafka.log -import java.io.{File, IOException} -import java.nio.file.Files +import java.io.File import java.util.Properties import kafka.api.{KAFKA_0_10_0_IV1, KAFKA_0_9_0} -import kafka.common._ +import kafka.common.TopicAndPartition import kafka.message._ import kafka.server.OffsetCheckpoint import kafka.utils._ @@ -221,11 +220,7 @@ class LogCleanerIntegrationTest(compressionCodec: String) { val logs = new Pool[TopicAndPartition, Log]() for(i <- 0 until parts) { val dir = new File(logDir, "log-" + i) - try { - Files.createDirectory(dir.toPath()) - } catch { - case e: IOException => //this is good - } + dir.mkdirs() val log = new Log(dir = dir, LogConfig(logConfigProperties(maxMessageSize, minCleanableDirtyRatio)), From 1261b0038f94b45280c46b1c34b3c22c8d32bed1 Mon Sep 17 00:00:00 2001 From: Ishita Mandhan Date: Tue, 16 Aug 2016 17:32:22 -0700 Subject: [PATCH 33/35] KAFKA-3940 Log should check the return value of dir.mkdirs() This commit changes all the occurrences of dir.mkdirs() with Files.createDirectory(dir.toPath()) --- core/src/main/scala/kafka/log/Log.scala | 9 ++++++++- core/src/main/scala/kafka/log/LogManager.scala | 15 +++++++++++---- .../kafka/metrics/KafkaCSVMetricsReporter.scala | 11 +++++++++-- .../kafka/log/LogCleanerIntegrationTest.scala | 13 +++++++++---- .../scala/unit/kafka/log/LogManagerTest.scala | 7 ++++++- core/src/test/scala/unit/kafka/log/LogTest.scala | 15 ++++++++++++--- .../test/scala/unit/kafka/utils/TestUtils.scala | 10 +++++++--- 7 files changed, 62 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index b4aa470fa28b..31ae236942dc 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -24,6 +24,7 @@ import kafka.common._ import kafka.metrics.KafkaMetricsGroup import kafka.server.{BrokerTopicStats, FetchDataInfo, LogOffsetMetadata} import java.io.{File, IOException} +import java.nio.file.Files import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap} import java.util.concurrent.atomic._ import java.text.NumberFormat @@ -146,7 +147,13 @@ class Log(val dir: File, /* Load the log segments from the log files on disk */ private def loadSegments() { // create the log directory if it doesn't exist - dir.mkdirs() + if (!Files.exists(dir.toPath())) { + try { + Files.createDirectory(dir.toPath()) + } catch { + case e: IOException => throw new KafkaException("Error in creating new directory.", e) + } + } var swapFiles = Set[File]() // first do a pass through the files in the log directory and remove any temporary files diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index e6c60b9bf96c..e7fae174551d 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -18,6 +18,7 @@ package kafka.log import java.io._ +import java.nio.file.Files import java.util.concurrent.TimeUnit import kafka.utils._ @@ -81,9 +82,11 @@ class LogManager(val logDirs: Array[File], for(dir <- dirs) { if(!dir.exists) { info("Log directory '" + dir.getAbsolutePath + "' not found, creating it.") - val created = dir.mkdirs() - if(!created) - throw new KafkaException("Failed to create data directory " + dir.getAbsolutePath) + try { + Files.createDirectory(dir.toPath()) + } catch { + case e: IOException => throw new KafkaException("Error in creating new directory.", e) + } } if(!dir.isDirectory || !dir.canRead) throw new KafkaException(dir.getAbsolutePath + " is not a readable log directory.") @@ -358,7 +361,11 @@ class LogManager(val logDirs: Array[File], // if not, create it val dataDir = nextLogDir() val dir = new File(dataDir, topicAndPartition.topic + "-" + topicAndPartition.partition) - dir.mkdirs() + try { + Files.createDirectory(dir.toPath()) + } catch { + case e: IOException => throw new KafkaException("Error in creating new directory.", e) + } log = new Log(dir, config, recoveryPoint = 0L, diff --git a/core/src/main/scala/kafka/metrics/KafkaCSVMetricsReporter.scala b/core/src/main/scala/kafka/metrics/KafkaCSVMetricsReporter.scala index 686f692548d9..0f8c59e7d855 100755 --- a/core/src/main/scala/kafka/metrics/KafkaCSVMetricsReporter.scala +++ b/core/src/main/scala/kafka/metrics/KafkaCSVMetricsReporter.scala @@ -21,11 +21,13 @@ package kafka.metrics import com.yammer.metrics.Metrics -import java.io.File +import java.io.{File, IOException} +import java.nio.file.Files import com.yammer.metrics.reporting.CsvReporter import java.util.concurrent.TimeUnit +import kafka.common._ import kafka.utils.{Logging, VerifiableProperties} import org.apache.kafka.common.utils.Utils @@ -50,7 +52,12 @@ private class KafkaCSVMetricsReporter extends KafkaMetricsReporter val metricsConfig = new KafkaMetricsConfig(props) csvDir = new File(props.getString("kafka.csv.metrics.dir", "kafka_metrics")) Utils.delete(csvDir) - csvDir.mkdirs() + try { + Files.createDirectory(csvDir.toPath()) + } catch { + case e: IOException => throw new KafkaException("Error in creating new directory '%s'".format(csvDir), e) + } + Files.createDirectory(csvDir.toPath()) underlying = new CsvReporter(Metrics.defaultRegistry(), csvDir) if (props.getBoolean("kafka.csv.metrics.reporter.enabled", default = false)) { initialized = true diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala index 825a55b5a8c5..7f336e906918 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala @@ -17,11 +17,12 @@ package kafka.log -import java.io.File +import java.io.{File, IOException} +import java.nio.file.Files import java.util.Properties import kafka.api.{KAFKA_0_10_0_IV1, KAFKA_0_9_0} -import kafka.common.TopicAndPartition +import kafka.common._ import kafka.message._ import kafka.server.OffsetCheckpoint import kafka.utils._ @@ -220,8 +221,12 @@ class LogCleanerIntegrationTest(compressionCodec: String) { val logs = new Pool[TopicAndPartition, Log]() for(i <- 0 until parts) { val dir = new File(logDir, "log-" + i) - dir.mkdirs() - + try { + Files.createDirectory(dir.toPath()) + } catch { + case e: IOException => throw new KafkaException("Error in creating new directory '%s'".format(dir), e) + } + val log = new Log(dir = dir, LogConfig(logConfigProperties(maxMessageSize, minCleanableDirtyRatio)), recoveryPoint = 0L, diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index dc4cc7974fdf..45fc6de7566e 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.nio.file.Files import java.util.Properties import kafka.common._ @@ -242,7 +243,11 @@ class LogManagerTest { def testRecoveryDirectoryMappingWithRelativeDirectory() { logManager.shutdown() logDir = new File("data" + File.separator + logDir.getName) - logDir.mkdirs() + try { + Files.createDirectory(logDir.toPath()) + } catch { + case e: IOException => throw new KafkaException("Error in creating new directory '%s'".format(logDir), e) + } logDir.deleteOnExit() logManager = createLogManager() logManager.startup diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index 2466ef265b13..55f288f075af 100755 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -18,11 +18,12 @@ package kafka.log import java.io._ +import java.nio.file.Files import java.util.Properties import org.apache.kafka.common.errors.{CorruptRecordException, OffsetOutOfRangeException, RecordBatchTooLargeException, RecordTooLargeException} import kafka.api.ApiVersion -import kafka.common.LongRef +import kafka.common._ import org.junit.Assert._ import org.scalatest.junit.JUnitSuite import org.junit.{After, Before, Test} @@ -336,7 +337,11 @@ class LogTest extends JUnitSuite { @Test def testThatGarbageCollectingSegmentsDoesntChangeOffset() { for(messagesToAppend <- List(0, 1, 25)) { - logDir.mkdirs() + try { + Files.createDirectory(logDir.toPath()) + } catch { + case e: IOException => throw new KafkaException("Error in creating new directory '%s'".format(logDir), e) + } // first test a log segment starting at 0 val logProps = new Properties() logProps.put(LogConfig.SegmentBytesProp, 100: java.lang.Integer) @@ -915,7 +920,11 @@ class LogTest extends JUnitSuite { val recoveryPoint = 50L for(iteration <- 0 until 50) { // create a log and write some messages to it - logDir.mkdirs() + try { + Files.createDirectory(logDir.toPath()) + } catch { + case e: IOException => throw new KafkaException("Error in creating new directory '%s'".format(logDir), e) + } var log = new Log(logDir, config, recoveryPoint = 0L, diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 131a24a7f75f..2e544b91e0e5 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -20,6 +20,7 @@ package kafka.utils import java.io._ import java.nio._ import java.nio.channels._ +import java.nio.file.Files import java.util.concurrent.{Callable, Executors, TimeUnit} import java.util.{Properties, Random} import java.security.cert.X509Certificate @@ -37,7 +38,7 @@ import kafka.api._ import kafka.cluster.{Broker, EndPoint} import kafka.consumer.{ConsumerConfig, ConsumerTimeoutException, KafkaStream} import kafka.serializer.{DefaultEncoder, Encoder, StringEncoder} -import kafka.common.TopicAndPartition +import kafka.common._ import kafka.admin.AdminUtils import kafka.log._ import kafka.utils.ZkUtils._ @@ -80,8 +81,11 @@ object TestUtils extends Logging { */ def tempRelativeDir(parent: String): File = { val parentFile = new File(parent) - parentFile.mkdirs() - + try { + Files.createDirectory(parentFile.toPath()) + } catch { + case e: IOException => throw new KafkaException("Error in creating new directory '%s'".format(parentFile), e) + } JTestUtils.tempDirectory(parentFile.toPath, null) } From 1ea1090a354fcdb3b026998aa210760442d5a212 Mon Sep 17 00:00:00 2001 From: Ishita Mandhan Date: Wed, 17 Aug 2016 13:30:46 -0700 Subject: [PATCH 34/35] Kafka 3940: Log should check the return value of dir.mkdirs() Fixed the failing errors in the previous commit that changed occurrences of dir.mkdirs() to Files.createDirectory() --- .../test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala | 2 +- core/src/test/scala/unit/kafka/log/LogManagerTest.scala | 2 +- core/src/test/scala/unit/kafka/log/LogTest.scala | 4 ++-- core/src/test/scala/unit/kafka/utils/TestUtils.scala | 2 +- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala index 7f336e906918..d3bf0303c02d 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala @@ -224,7 +224,7 @@ class LogCleanerIntegrationTest(compressionCodec: String) { try { Files.createDirectory(dir.toPath()) } catch { - case e: IOException => throw new KafkaException("Error in creating new directory '%s'".format(dir), e) + case e: IOException => //this is good } val log = new Log(dir = dir, diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index 45fc6de7566e..bf62a4689530 100755 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -246,7 +246,7 @@ class LogManagerTest { try { Files.createDirectory(logDir.toPath()) } catch { - case e: IOException => throw new KafkaException("Error in creating new directory '%s'".format(logDir), e) + case e: IOException => //this is good } logDir.deleteOnExit() logManager = createLogManager() diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index 55f288f075af..89647f4a6e34 100755 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -340,7 +340,7 @@ class LogTest extends JUnitSuite { try { Files.createDirectory(logDir.toPath()) } catch { - case e: IOException => throw new KafkaException("Error in creating new directory '%s'".format(logDir), e) + case e: IOException => //this is good } // first test a log segment starting at 0 val logProps = new Properties() @@ -923,7 +923,7 @@ class LogTest extends JUnitSuite { try { Files.createDirectory(logDir.toPath()) } catch { - case e: IOException => throw new KafkaException("Error in creating new directory '%s'".format(logDir), e) + case e: IOException => //this is good } var log = new Log(logDir, config, diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 2e544b91e0e5..f137b85fee26 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -84,7 +84,7 @@ object TestUtils extends Logging { try { Files.createDirectory(parentFile.toPath()) } catch { - case e: IOException => throw new KafkaException("Error in creating new directory '%s'".format(parentFile), e) + case e: IOException => //this is good } JTestUtils.tempDirectory(parentFile.toPath, null) } From 6c6a4bee509b4617b326948c3e92a91086d9fea4 Mon Sep 17 00:00:00 2001 From: Ishita Mandhan Date: Tue, 23 Aug 2016 18:59:16 -0700 Subject: [PATCH 35/35] KAFKA-3940 Log should check the return value of dir.mkdirs() --- core/src/main/scala/kafka/log/Log.scala | 6 +++--- core/src/main/scala/kafka/log/LogManager.scala | 8 ++++---- .../scala/kafka/metrics/KafkaCSVMetricsReporter.scala | 5 ++--- .../unit/kafka/log/LogCleanerIntegrationTest.scala | 11 +++-------- 4 files changed, 12 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 31ae236942dc..972980609e5d 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -147,11 +147,11 @@ class Log(val dir: File, /* Load the log segments from the log files on disk */ private def loadSegments() { // create the log directory if it doesn't exist - if (!Files.exists(dir.toPath())) { + if (!Files.exists(dir.toPath)) { try { - Files.createDirectory(dir.toPath()) + Files.createDirectories(dir.toPath) } catch { - case e: IOException => throw new KafkaException("Error in creating new directory.", e) + case e: IOException => throw new KafkaException("Error creating log directory ${dir.toPath}", e) } } var swapFiles = Set[File]() diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index e7fae174551d..b0ec90ea1b55 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -83,9 +83,9 @@ class LogManager(val logDirs: Array[File], if(!dir.exists) { info("Log directory '" + dir.getAbsolutePath + "' not found, creating it.") try { - Files.createDirectory(dir.toPath()) + Files.createDirectories(dir.toPath()) } catch { - case e: IOException => throw new KafkaException("Error in creating new directory.", e) + case e: IOException => throw new KafkaException("Error in creating log directory ${dir.toPath}", e) } } if(!dir.isDirectory || !dir.canRead) @@ -362,9 +362,9 @@ class LogManager(val logDirs: Array[File], val dataDir = nextLogDir() val dir = new File(dataDir, topicAndPartition.topic + "-" + topicAndPartition.partition) try { - Files.createDirectory(dir.toPath()) + Files.createDirectories(dir.toPath()) } catch { - case e: IOException => throw new KafkaException("Error in creating new directory.", e) + case e: IOException => throw new KafkaException("Error in creating log directory ${dir.toPath}", e) } log = new Log(dir, config, diff --git a/core/src/main/scala/kafka/metrics/KafkaCSVMetricsReporter.scala b/core/src/main/scala/kafka/metrics/KafkaCSVMetricsReporter.scala index 0f8c59e7d855..8638cd2e0ec9 100755 --- a/core/src/main/scala/kafka/metrics/KafkaCSVMetricsReporter.scala +++ b/core/src/main/scala/kafka/metrics/KafkaCSVMetricsReporter.scala @@ -53,11 +53,10 @@ private class KafkaCSVMetricsReporter extends KafkaMetricsReporter csvDir = new File(props.getString("kafka.csv.metrics.dir", "kafka_metrics")) Utils.delete(csvDir) try { - Files.createDirectory(csvDir.toPath()) + Files.createDirectories(csvDir.toPath()) } catch { - case e: IOException => throw new KafkaException("Error in creating new directory '%s'".format(csvDir), e) + case e: IOException => throw new KafkaException("Error in creating metrics directory ${csvDir}", e) } - Files.createDirectory(csvDir.toPath()) underlying = new CsvReporter(Metrics.defaultRegistry(), csvDir) if (props.getBoolean("kafka.csv.metrics.reporter.enabled", default = false)) { initialized = true diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala index d3bf0303c02d..d1987bc9570a 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala @@ -17,12 +17,11 @@ package kafka.log -import java.io.{File, IOException} -import java.nio.file.Files +import java.io.File import java.util.Properties import kafka.api.{KAFKA_0_10_0_IV1, KAFKA_0_9_0} -import kafka.common._ +import kafka.common.TopicAndPartition import kafka.message._ import kafka.server.OffsetCheckpoint import kafka.utils._ @@ -221,11 +220,7 @@ class LogCleanerIntegrationTest(compressionCodec: String) { val logs = new Pool[TopicAndPartition, Log]() for(i <- 0 until parts) { val dir = new File(logDir, "log-" + i) - try { - Files.createDirectory(dir.toPath()) - } catch { - case e: IOException => //this is good - } + dir.mkdirs() val log = new Log(dir = dir, LogConfig(logConfigProperties(maxMessageSize, minCleanableDirtyRatio)),