diff --git a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java index 2f1fe93a5ac08..750b8a19aba42 100644 --- a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java +++ b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java @@ -29,10 +29,15 @@ public class CommonClientConfigs { */ public static final String BOOTSTRAP_SERVERS_CONFIG = "bootstrap.servers"; - public static final String BOOSTRAP_SERVERS_DOC = "A list of host/port pairs to use for establishing the initial connection to the Kafka cluster. The client will make use of all servers irrespective of which servers are specified here for bootstrapping—this list only impacts the initial hosts used to discover the full set of servers. This list should be in the form " + public static final String BOOTSTRAP_SERVERS_DOC = "A list of host/port pairs to use for establishing the initial connection to the Kafka cluster. The client will make use of all servers irrespective of which servers are specified here for bootstrapping—this list only impacts the initial hosts used to discover the full set of servers. This list should be in the form " + "host1:port1,host2:port2,.... Since these servers are just used for the initial connection to " + "discover the full cluster membership (which may change dynamically), this list need not contain the full set of " + "servers (you may want more than one, though, in case a server is down)."; + /** + * @deprecated This will be removed in a future release. Please use {@link #BOOTSTRAP_SERVERS_DOC} + */ + @Deprecated + public static final String BOOSTRAP_SERVERS_DOC = BOOTSTRAP_SERVERS_DOC; public static final String METADATA_MAX_AGE_CONFIG = "metadata.max.age.ms"; public static final String METADATA_MAX_AGE_DOC = "The period of time in milliseconds after which we force a refresh of metadata even if we haven't seen any partition leadership changes to proactively discover any new brokers or partitions."; 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 322ae0fbe2315..fd44072c8d5ae 100644 --- a/clients/src/main/java/org/apache/kafka/clients/Metadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/Metadata.java @@ -12,12 +12,6 @@ */ package org.apache.kafka.clients; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Set; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; @@ -25,6 +19,13 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + /** * A class encapsulating some of the logic around metadata. *

@@ -254,7 +255,10 @@ private Cluster getClusterForCurrentTopics(Cluster cluster) { unauthorizedTopics.retainAll(this.topics); for (String topic : this.topics) { - partitionInfos.addAll(cluster.partitionsForTopic(topic)); + List partitionInfoList = cluster.partitionsForTopic(topic); + if (partitionInfoList != null) { + partitionInfos.addAll(cluster.partitionsForTopic(topic)); + } } nodes = cluster.nodes(); } 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 6523d184585c1..e58f2fd570ab1 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 @@ -191,7 +191,7 @@ public class ConsumerConfig extends AbstractConfig { CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, Importance.HIGH, - CommonClientConfigs.BOOSTRAP_SERVERS_DOC) + CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) .define(GROUP_ID_CONFIG, Type.STRING, "", Importance.HIGH, GROUP_ID_DOC) .define(SESSION_TIMEOUT_MS_CONFIG, Type.INT, diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index fd3eb096c1c19..91697c12ca8b5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -431,8 +431,7 @@ public Future send(ProducerRecord record, Callback callbac } /** - * Implementation of asynchronously send a record to a topic. Equivalent to send(record, null). - * See {@link #send(ProducerRecord, Callback)} for details. + * Implementation of asynchronously send a record to a topic. */ private Future doSend(ProducerRecord record, Callback callback) { TopicPartition tp = null; 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 4ed083b2e79a6..e505f71fd0376 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 @@ -144,11 +144,10 @@ public class ProducerConfig extends AbstractConfig { @Deprecated public static final String BLOCK_ON_BUFFER_FULL_CONFIG = "block.on.buffer.full"; private static final String BLOCK_ON_BUFFER_FULL_DOC = "When our memory buffer is exhausted we must either stop accepting new records (block) or throw errors. " - + "By default this setting is false and the producer will no longer throw a BufferExhaustException but instead will use the {@link #MAX_BLOCK_MS_CONFIG} " - + "value to block, after which it will throw a TimeoutException. Setting this property to true will set the " + MAX_BLOCK_MS_CONFIG + " to Long.MAX_VALUE." + + "By default this setting is false and the producer will no longer throw a BufferExhaustException but instead will use the " + MAX_BLOCK_MS_CONFIG + " " + + "value to block, after which it will throw a TimeoutException. Setting this property to true will set the " + MAX_BLOCK_MS_CONFIG + " to Long.MAX_VALUE. " + "Also if this property is set to true, parameter " + METADATA_FETCH_TIMEOUT_CONFIG + " is not longer honored." - + "

" - + "This parameter is deprecated and will be removed in a future release. " + + "

This parameter is deprecated and will be removed in a future release. " + "Parameter " + MAX_BLOCK_MS_CONFIG + " should be used instead."; /** buffer.memory */ @@ -218,7 +217,7 @@ public class ProducerConfig extends AbstractConfig { + "received by the producer before they are published to the Kafka cluster. By default, there are no interceptors."; static { - CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, Importance.HIGH, CommonClientConfigs.BOOSTRAP_SERVERS_DOC) + CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, Importance.HIGH, CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) .define(BUFFER_MEMORY_CONFIG, Type.LONG, 32 * 1024 * 1024L, atLeast(0L), Importance.HIGH, BUFFER_MEMORY_DOC) .define(RETRIES_CONFIG, Type.INT, 0, between(0, Integer.MAX_VALUE), Importance.HIGH, RETRIES_DOC) .define(ACKS_CONFIG, diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java index 5339096efa750..a73d882226564 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java @@ -74,7 +74,6 @@ public final class RecordAccumulator { private final Set muted; private int drainIndex; - /** * Create a new record accumulator * @@ -104,11 +103,11 @@ public RecordAccumulator(int batchSize, this.compression = compression; this.lingerMs = lingerMs; this.retryBackoffMs = retryBackoffMs; - this.batches = new CopyOnWriteMap>(); + this.batches = new CopyOnWriteMap<>(); String metricGrpName = "producer-metrics"; this.free = new BufferPool(totalSize, batchSize, metrics, time, metricGrpName); this.incomplete = new IncompleteRecordBatches(); - this.muted = new HashSet(); + this.muted = new HashSet<>(); this.time = time; registerMetrics(metrics, metricGrpName); } @@ -171,12 +170,9 @@ public RecordAppendResult append(TopicPartition tp, synchronized (dq) { if (closed) throw new IllegalStateException("Cannot send after the producer is closed."); - RecordBatch last = dq.peekLast(); - if (last != null) { - FutureRecordMetadata future = last.tryAppend(timestamp, key, value, callback, time.milliseconds()); - if (future != null) - return new RecordAppendResult(future, dq.size() > 1 || last.records.isFull(), false); - } + RecordAppendResult appendResult = tryAppend(timestamp, key, value, callback, dq); + if (appendResult != null) + return appendResult; } // we don't have an in-progress record batch try to allocate a new batch @@ -187,14 +183,12 @@ public RecordAppendResult append(TopicPartition tp, // Need to check if producer is closed again after grabbing the dequeue lock. if (closed) throw new IllegalStateException("Cannot send after the producer is closed."); - RecordBatch last = dq.peekLast(); - if (last != null) { - FutureRecordMetadata future = last.tryAppend(timestamp, key, value, callback, time.milliseconds()); - if (future != null) { - // Somebody else found us a batch, return the one we waited for! Hopefully this doesn't happen often... - free.deallocate(buffer); - return new RecordAppendResult(future, dq.size() > 1 || last.records.isFull(), false); - } + + RecordAppendResult appendResult = tryAppend(timestamp, key, value, callback, dq); + if (appendResult != null) { + // Somebody else found us a batch, return the one we waited for! Hopefully this doesn't happen often... + free.deallocate(buffer); + return appendResult; } MemoryRecords records = MemoryRecords.emptyRecords(buffer, compression, this.batchSize); RecordBatch batch = new RecordBatch(tp, records, time.milliseconds()); @@ -209,12 +203,28 @@ public RecordAppendResult append(TopicPartition tp, } } + /** + * If `RecordBatch.tryAppend` fails (i.e. the record batch is full), close its memory records to release temporary + * resources (like compression streams buffers). + */ + private RecordAppendResult tryAppend(long timestamp, byte[] key, byte[] value, Callback callback, Deque deque) { + RecordBatch last = deque.peekLast(); + if (last != null) { + FutureRecordMetadata future = last.tryAppend(timestamp, key, value, callback, time.milliseconds()); + if (future == null) + last.records.close(); + else + return new RecordAppendResult(future, deque.size() > 1 || last.records.isFull(), false); + } + return null; + } + /** * Abort the batches that have been sitting in RecordAccumulator for more than the configured requestTimeout * due to metadata being unavailable */ public List abortExpiredBatches(int requestTimeout, long now) { - List expiredBatches = new ArrayList(); + List expiredBatches = new ArrayList<>(); int count = 0; for (Map.Entry> entry : this.batches.entrySet()) { Deque dq = entry.getValue(); @@ -245,7 +255,7 @@ public List abortExpiredBatches(int requestTimeout, long now) { } } } - if (expiredBatches.size() > 0) + if (!expiredBatches.isEmpty()) log.trace("Expired {} batches in accumulator", count); return expiredBatches; @@ -287,7 +297,7 @@ public void reenqueue(RecordBatch batch, long now) { * */ public ReadyCheckResult ready(Cluster cluster, long nowMs) { - Set readyNodes = new HashSet(); + Set readyNodes = new HashSet<>(); long nextReadyCheckDelayMs = Long.MAX_VALUE; boolean unknownLeadersExist = false; @@ -333,7 +343,7 @@ public boolean hasUnsent() { for (Map.Entry> entry : this.batches.entrySet()) { Deque deque = entry.getValue(); synchronized (deque) { - if (deque.size() > 0) + if (!deque.isEmpty()) return true; } } @@ -357,11 +367,11 @@ public Map> drain(Cluster cluster, if (nodes.isEmpty()) return Collections.emptyMap(); - Map> batches = new HashMap>(); + Map> batches = new HashMap<>(); for (Node node : nodes) { int size = 0; List parts = cluster.partitionsForNode(node.id()); - List ready = new ArrayList(); + List ready = new ArrayList<>(); /* to make starvation less likely this loop doesn't start at 0 */ int start = drainIndex = drainIndex % parts.size(); do { @@ -436,6 +446,11 @@ public void deallocate(RecordBatch batch) { boolean flushInProgress() { return flushesInProgress.get() > 0; } + + /* Visible for testing */ + Map> batches() { + return Collections.unmodifiableMap(batches); + } /** * Initiate the flushing of data from the accumulator...this makes all requests immediately ready @@ -569,7 +584,7 @@ public void remove(RecordBatch batch) { public Iterable all() { synchronized (incomplete) { - return new ArrayList(this.incomplete); + return new ArrayList<>(this.incomplete); } } } 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 fcf7f446a459f..603f74b18e0e1 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 @@ -213,6 +213,11 @@ public String toString() { return builder.toString(); } + /** Visible for testing */ + public boolean isWritable() { + return writable; + } + public static class RecordsIterator extends AbstractIterator { private final ByteBuffer buffer; private final DataInputStream stream; diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java index b3a5a049a82d3..43ac15a09a4a0 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java @@ -22,6 +22,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.Deque; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -75,14 +76,27 @@ public void teardown() { @Test public void testFull() throws Exception { long now = time.milliseconds(); - RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, CompressionType.NONE, 10L, 100L, metrics, time); - int appends = 1024 / msgSize; + int batchSize = 1024; + RecordAccumulator accum = new RecordAccumulator(batchSize, 10 * batchSize, CompressionType.NONE, 10L, 100L, metrics, time); + int appends = batchSize / msgSize; for (int i = 0; i < appends; i++) { + // append to the first batch accum.append(tp1, 0L, key, value, null, maxBlockTimeMs); + Deque partitionBatches = accum.batches().get(tp1); + assertEquals(1, partitionBatches.size()); + assertTrue(partitionBatches.peekFirst().records.isWritable()); assertEquals("No partitions should be ready.", 0, accum.ready(cluster, now).readyNodes.size()); } + + // this append doesn't fit in the first batch, so a new batch is created and the first batch is closed accum.append(tp1, 0L, key, value, null, maxBlockTimeMs); + Deque partitionBatches = accum.batches().get(tp1); + assertEquals(2, partitionBatches.size()); + Iterator partitionBatchesIterator = partitionBatches.iterator(); + assertFalse(partitionBatchesIterator.next().records.isWritable()); + assertTrue(partitionBatchesIterator.next().records.isWritable()); assertEquals("Our partition's leader should be ready", Collections.singleton(node1), accum.ready(cluster, time.milliseconds()).readyNodes); + List batches = accum.drain(cluster, Collections.singleton(node1), Integer.MAX_VALUE, 0).get(node1.id()); assertEquals(1, batches.size()); RecordBatch batch = batches.get(0); diff --git a/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSourceTask.java b/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSourceTask.java index 14d944f60f365..bab3f77276dec 100644 --- a/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSourceTask.java +++ b/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSourceTask.java @@ -146,7 +146,6 @@ public List poll() throws InterruptedException { records = new ArrayList<>(); records.add(new SourceRecord(offsetKey(filename), offsetValue(streamOffset), topic, VALUE_SCHEMA, line)); } - new ArrayList(); } while (line != null); } } diff --git a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java index d9a685953d2d5..59e653eb441d2 100644 --- a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java +++ b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java @@ -304,7 +304,7 @@ public void configure(Map configs, boolean isKey) { Object cacheSizeVal = configs.get(SCHEMAS_CACHE_SIZE_CONFIG); if (cacheSizeVal != null) - cacheSize = (int) cacheSizeVal; + cacheSize = Integer.parseInt((String) cacheSizeVal); fromConnectSchemaCache = new SynchronizedCache<>(new LRUCache(cacheSize)); toConnectSchemaCache = new SynchronizedCache<>(new LRUCache(cacheSize)); } diff --git a/connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterTest.java b/connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterTest.java index c923285306771..0e7c153ddaf9e 100644 --- a/connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterTest.java +++ b/connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterTest.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.ArrayNode; import com.fasterxml.jackson.databind.node.JsonNodeFactory; +import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.data.Date; import org.apache.kafka.connect.data.Decimal; import com.fasterxml.jackson.databind.node.ObjectNode; @@ -36,10 +37,13 @@ import org.junit.Test; import org.powermock.reflect.Whitebox; +import java.io.File; import java.io.IOException; import java.io.UnsupportedEncodingException; import java.math.BigDecimal; import java.math.BigInteger; +import java.net.URISyntaxException; +import java.net.URL; import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Calendar; @@ -607,6 +611,19 @@ public void testCacheSchemaToJsonConversion() { assertEquals(2, cache.size()); } + @Test + public void testJsonSchemaCacheSizeFromConfigFile() throws URISyntaxException, IOException { + URL url = getClass().getResource("/connect-test.properties"); + File propFile = new File(url.toURI()); + String workerPropsFile = propFile.getAbsolutePath(); + Map workerProps = !workerPropsFile.isEmpty() ? + Utils.propsToStringMap(Utils.loadProps(workerPropsFile)) : Collections.emptyMap(); + + JsonConverter rc = new JsonConverter(); + rc.configure(workerProps, false); + + } + private JsonNode parse(byte[] json) { try { diff --git a/connect/json/src/test/resources/connect-test.properties b/connect/json/src/test/resources/connect-test.properties new file mode 100644 index 0000000000000..9a48f68e08182 --- /dev/null +++ b/connect/json/src/test/resources/connect-test.properties @@ -0,0 +1,17 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +schemas.cache.size=1 + diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryOffsetBackingStore.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryOffsetBackingStore.java index 669e5f5a5f695..baa8192454a64 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryOffsetBackingStore.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryOffsetBackingStore.java @@ -17,6 +17,7 @@ package org.apache.kafka.connect.storage; +import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.runtime.WorkerConfig; import org.apache.kafka.connect.util.Callback; import org.slf4j.Logger; @@ -30,6 +31,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; /** * Implementation of OffsetBackingStore that doesn't actually persist any data. To ensure this @@ -40,7 +42,7 @@ public class MemoryOffsetBackingStore implements OffsetBackingStore { private static final Logger log = LoggerFactory.getLogger(MemoryOffsetBackingStore.class); protected Map data = new HashMap<>(); - protected ExecutorService executor = Executors.newSingleThreadExecutor(); + protected ExecutorService executor; public MemoryOffsetBackingStore() { @@ -51,12 +53,26 @@ public void configure(WorkerConfig config) { } @Override - public synchronized void start() { + public void start() { + executor = Executors.newSingleThreadExecutor(); } @Override - public synchronized void stop() { - // Nothing to do since this doesn't maintain any outstanding connections/data + public void stop() { + if (executor != null) { + executor.shutdown(); + // Best effort wait for any get() and set() tasks (and caller's callbacks) to complete. + try { + executor.awaitTermination(30, TimeUnit.SECONDS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + if (!executor.shutdownNow().isEmpty()) { + throw new ConnectException("Failed to stop MemoryOffsetBackingStore. Exiting without cleanly " + + "shutting down pending tasks and/or callbacks."); + } + executor = null; + } } @Override @@ -67,10 +83,8 @@ public Future> get( @Override public Map call() throws Exception { Map result = new HashMap<>(); - synchronized (MemoryOffsetBackingStore.this) { - for (ByteBuffer key : keys) { - result.put(key, data.get(key)); - } + for (ByteBuffer key : keys) { + result.put(key, data.get(key)); } if (callback != null) callback.onCompletion(null, result); @@ -86,12 +100,10 @@ public Future set(final Map values, return executor.submit(new Callable() { @Override public Void call() throws Exception { - synchronized (MemoryOffsetBackingStore.this) { - for (Map.Entry entry : values.entrySet()) { - data.put(entry.getKey(), entry.getValue()); - } - save(); + for (Map.Entry entry : values.entrySet()) { + data.put(entry.getKey(), entry.getValue()); } + save(); if (callback != null) callback.onCompletion(null, null); 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 81e6be86d80cd..747db1af53d4f 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 @@ -81,9 +81,7 @@ public class DistributedHerderTest { HERDER_CONFIG.put(WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.json.JsonConverter"); HERDER_CONFIG.put(WorkerConfig.INTERNAL_KEY_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.json.JsonConverter"); HERDER_CONFIG.put(WorkerConfig.INTERNAL_VALUE_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.json.JsonConverter"); - HERDER_CONFIG.put(DistributedConfig.CONFIG_TOPIC_CONFIG, "connect-configs"); HERDER_CONFIG.put(DistributedConfig.OFFSET_STORAGE_TOPIC_CONFIG, "connect-offsets"); - HERDER_CONFIG.put(DistributedConfig.STATUS_STORAGE_TOPIC_CONFIG, "status-topic"); } private static final String MEMBER_URL = "memberUrl"; diff --git a/core/src/main/scala/kafka/admin/AclCommand.scala b/core/src/main/scala/kafka/admin/AclCommand.scala index 01e95ca45b351..080f8097c7281 100644 --- a/core/src/main/scala/kafka/admin/AclCommand.scala +++ b/core/src/main/scala/kafka/admin/AclCommand.scala @@ -99,10 +99,10 @@ object AclCommand { for ((resource, acls) <- resourceToAcl) { if (acls.isEmpty) { - if (confirmAction(s"Are you sure you want to delete all ACLs for resource `${resource}`? (y/n)")) + if (confirmAction(opts, s"Are you sure you want to delete all ACLs for resource `${resource}`? (y/n)")) authorizer.removeAcls(resource) } else { - if (confirmAction(s"Are you sure you want to remove ACLs: $Newline ${acls.map("\t" + _).mkString(Newline)} $Newline from resource `${resource}`? (y/n)")) + if (confirmAction(opts, s"Are you sure you want to remove ACLs: $Newline ${acls.map("\t" + _).mkString(Newline)} $Newline from resource `${resource}`? (y/n)")) authorizer.removeAcls(acls, resource) } } @@ -117,7 +117,7 @@ object AclCommand { val resourceToAcls: Iterable[(Resource, Set[Acl])] = if (resources.isEmpty) authorizer.getAcls() - else resources.map(resource => (resource -> authorizer.getAcls(resource))) + else resources.map(resource => resource -> authorizer.getAcls(resource)) for ((resource, acls) <- resourceToAcls) println(s"Current ACLs for resource `${resource}`: $Newline ${acls.map("\t" + _).mkString(Newline)} $Newline") @@ -241,7 +241,9 @@ object AclCommand { resources } - private def confirmAction(msg: String): Boolean = { + private def confirmAction(opts: AclCommandOptions, msg: String): Boolean = { + if (opts.options.has(opts.forceOpt)) + return true println(msg) Console.readLine().equalsIgnoreCase("y") } @@ -329,6 +331,8 @@ object AclCommand { val helpOpt = parser.accepts("help", "Print usage information.") + val forceOpt = parser.accepts("force", "Assume Yes to all queries and do not prompt.") + val options = parser.parse(args: _*) def checkArgs() { diff --git a/core/src/main/scala/kafka/admin/AdminClient.scala b/core/src/main/scala/kafka/admin/AdminClient.scala index ef76ffc40ee1e..556a02b9d411b 100644 --- a/core/src/main/scala/kafka/admin/AdminClient.scala +++ b/core/src/main/scala/kafka/admin/AdminClient.scala @@ -49,7 +49,7 @@ class AdminClient(val time: Time, client.poll(future) if (future.succeeded()) - return future.value().responseBody() + future.value().responseBody() else throw future.exception() } @@ -61,10 +61,10 @@ class AdminClient(val time: Time, return send(broker, api, request) } catch { case e: Exception => - debug(s"Request ${api} failed against node ${broker}", e) + debug(s"Request $api failed against node $broker", e) } } - throw new RuntimeException(s"Request ${api} failed on brokers ${bootstrapBrokers}") + throw new RuntimeException(s"Request $api failed on brokers $bootstrapBrokers") } private def findCoordinator(groupId: String): Node = { @@ -88,7 +88,7 @@ class AdminClient(val time: Time, val response = new MetadataResponse(responseBody) val errors = response.errors() if (!errors.isEmpty) - debug(s"Metadata request contained errors: ${errors}") + debug(s"Metadata request contained errors: $errors") response.cluster().nodes().asScala.toList } @@ -100,7 +100,7 @@ class AdminClient(val time: Time, listGroups(broker) } catch { case e: Exception => - debug(s"Failed to find groups from broker ${broker}", e) + debug(s"Failed to find groups from broker $broker", e) List[GroupOverview]() } } @@ -127,7 +127,7 @@ class AdminClient(val time: Time, val response = new DescribeGroupsResponse(responseBody) val metadata = response.groups().get(groupId) if (metadata == null) - throw new KafkaException(s"Response from broker contained no metadata for group ${groupId}") + throw new KafkaException(s"Response from broker contained no metadata for group $groupId") Errors.forCode(metadata.errorCode()).maybeThrow() val members = metadata.members().map { member => @@ -143,21 +143,21 @@ class AdminClient(val time: Time, clientHost: String, assignment: List[TopicPartition]) - def describeConsumerGroup(groupId: String): List[ConsumerSummary] = { + def describeConsumerGroup(groupId: String): Option[List[ConsumerSummary]] = { val group = describeGroup(groupId) if (group.state == "Dead") - return List.empty[ConsumerSummary] + return None if (group.protocolType != ConsumerProtocol.PROTOCOL_TYPE) - throw new IllegalArgumentException(s"Group ${groupId} with protocol type '${group.protocolType}' is not a valid consumer group") + throw new IllegalArgumentException(s"Group $groupId with protocol type '${group.protocolType}' is not a valid consumer group") if (group.state == "Stable") { - group.members.map { member => + Some(group.members.map { member => val assignment = ConsumerProtocol.deserializeAssignment(ByteBuffer.wrap(member.assignment)) new ConsumerSummary(member.memberId, member.clientId, member.clientHost, assignment.partitions().asScala.toList) - } + }) } else { - List.empty + Some(List.empty) } } @@ -182,7 +182,7 @@ object AdminClient { CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, Type.LIST, Importance.HIGH, - CommonClientConfigs.BOOSTRAP_SERVERS_DOC) + CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) .define( CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, ConfigDef.Type.STRING, diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala index a8a282e2c13a1..53b6dd72a0b6b 100644 --- a/core/src/main/scala/kafka/admin/AdminUtils.scala +++ b/core/src/main/scala/kafka/admin/AdminUtils.scala @@ -250,7 +250,7 @@ object AdminUtils extends Logging { checkBrokerAvailable: Boolean = true, rackAwareMode: RackAwareMode = RackAwareMode.Enforced) { val existingPartitionsReplicaList = zkUtils.getReplicaAssignmentForTopics(List(topic)) - if (existingPartitionsReplicaList.size == 0) + if (existingPartitionsReplicaList.isEmpty) throw new AdminOperationException("The topic %s does not exist".format(topic)) val existingReplicaListForPartitionZero = existingPartitionsReplicaList.find(p => p._1.partition == 0) match { @@ -274,8 +274,8 @@ object AdminUtils extends Logging { existingPartitionsReplicaList.size, checkBrokerAvailable) // check if manual assignment has the right replication factor - val unmatchedRepFactorList = newPartitionReplicaList.values.filter(p => (p.size != existingReplicaListForPartitionZero.size)) - if (unmatchedRepFactorList.size != 0) + val unmatchedRepFactorList = newPartitionReplicaList.values.filter(p => p.size != existingReplicaListForPartitionZero.size) + if (unmatchedRepFactorList.nonEmpty) throw new AdminOperationException("The replication factor in manual replication assignment " + " is not equal to the existing replication factor for the topic " + existingReplicaListForPartitionZero.size) @@ -291,9 +291,9 @@ object AdminUtils extends Logging { val ret = new mutable.HashMap[Int, List[Int]]() var partitionId = startPartitionId partitionList = partitionList.takeRight(partitionList.size - partitionId) - for (i <- 0 until partitionList.size) { + for (i <- partitionList.indices) { val brokerList = partitionList(i).split(":").map(s => s.trim().toInt) - if (brokerList.size <= 0) + if (brokerList.isEmpty) throw new AdminOperationException("replication factor must be larger than 0") if (brokerList.size != brokerList.toSet.size) throw new AdminOperationException("duplicate brokers in replica assignment: " + brokerList) @@ -443,7 +443,7 @@ object AdminUtils extends Logging { private def writeTopicPartitionAssignment(zkUtils: ZkUtils, topic: String, replicaAssignment: Map[Int, Seq[Int]], update: Boolean) { try { val zkPath = getTopicPath(topic) - val jsonPartitionData = zkUtils.replicaAssignmentZkData(replicaAssignment.map(e => (e._1.toString -> e._2))) + val jsonPartitionData = zkUtils.replicaAssignmentZkData(replicaAssignment.map(e => e._1.toString -> e._2)) if (!update) { info("Topic creation " + jsonPartitionData.toString) diff --git a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala index 414e7baee449d..b086d8f64ca91 100755 --- a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala +++ b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala @@ -312,22 +312,25 @@ object ConsumerGroupCommand { } protected def describeGroup(group: String) { - val consumerSummaries = adminClient.describeConsumerGroup(group) - if (consumerSummaries.isEmpty) - println(s"Consumer group `${group}` does not exist or is rebalancing.") - else { - val consumer = getConsumer() - printDescribeHeader() - consumerSummaries.foreach { consumerSummary => - val topicPartitions = consumerSummary.assignment.map(tp => TopicAndPartition(tp.topic, tp.partition)) - val partitionOffsets = topicPartitions.flatMap { topicPartition => - Option(consumer.committed(new TopicPartition(topicPartition.topic, topicPartition.partition))).map { offsetAndMetadata => - topicPartition -> offsetAndMetadata.offset + adminClient.describeConsumerGroup(group) match { + case None => println(s"Consumer group `${group}` does not exist.") + case Some(consumerSummaries) => + if (consumerSummaries.isEmpty) + println(s"Consumer group `${group}` is rebalancing.") + else { + val consumer = getConsumer() + printDescribeHeader() + consumerSummaries.foreach { consumerSummary => + val topicPartitions = consumerSummary.assignment.map(tp => TopicAndPartition(tp.topic, tp.partition)) + val partitionOffsets = topicPartitions.flatMap { topicPartition => + Option(consumer.committed(new TopicPartition(topicPartition.topic, topicPartition.partition))).map { offsetAndMetadata => + topicPartition -> offsetAndMetadata.offset + } + }.toMap + describeTopicPartition(group, topicPartitions, partitionOffsets.get, + _ => Some(s"${consumerSummary.clientId}_${consumerSummary.clientHost}")) } - }.toMap - describeTopicPartition(group, topicPartitions, partitionOffsets.get, - _ => Some(s"${consumerSummary.clientId}_${consumerSummary.clientHost}")) - } + } } } diff --git a/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala b/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala index 1bf351a05e0a5..fae0a4045209e 100755 --- a/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala +++ b/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala @@ -113,7 +113,7 @@ object ReassignPartitionsCommand extends Logging { val (_, replicas) = assignment.head val assignedReplicas = AdminUtils.assignReplicasToBrokers(brokerMetadatas, assignment.size, replicas.size) partitionsToBeReassigned ++= assignedReplicas.map { case (partition, replicas) => - (TopicAndPartition(topic, partition) -> replicas) + TopicAndPartition(topic, partition) -> replicas } } diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala index e6ebb96c872b1..c643a9df688d5 100755 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -118,7 +118,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 - if (topics.length == 0 && !ifExists) { + 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))) } @@ -165,7 +165,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 - if (topics.length == 0 && !ifExists) { + 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))) } diff --git a/core/src/main/scala/kafka/admin/ZkSecurityMigrator.scala b/core/src/main/scala/kafka/admin/ZkSecurityMigrator.scala index 20808796dada9..a87e5b7c836b2 100644 --- a/core/src/main/scala/kafka/admin/ZkSecurityMigrator.scala +++ b/core/src/main/scala/kafka/admin/ZkSecurityMigrator.scala @@ -18,20 +18,18 @@ package kafka.admin import java.util.concurrent.LinkedBlockingQueue -import java.util.concurrent.ThreadPoolExecutor -import java.util.concurrent.TimeUnit + import joptsimple.OptionParser import org.I0Itec.zkclient.exception.ZkException -import kafka.utils.{Logging, ZkUtils, CommandLineUtils} -import org.apache.log4j.Level +import kafka.utils.{CommandLineUtils, Logging, ZkUtils} import org.apache.kafka.common.security.JaasUtils import org.apache.zookeeper.AsyncCallback.{ChildrenCallback, StatCallback} import org.apache.zookeeper.data.Stat import org.apache.zookeeper.KeeperException import org.apache.zookeeper.KeeperException.Code + import scala.annotation.tailrec import scala.collection.JavaConverters._ -import scala.collection._ import scala.collection.mutable.Queue import scala.concurrent._ import scala.concurrent.duration._ @@ -83,9 +81,9 @@ object ZkSecurityMigrator extends Logging { if (options.has(helpOpt)) CommandLineUtils.printUsageAndDie(parser, usageMessage) - if ((jaasFile == null)) { - val errorMsg = ("No JAAS configuration file has been specified. Please make sure that you have set " + - "the system property %s".format(JaasUtils.JAVA_LOGIN_CONFIG_PARAM)) + if (jaasFile == null) { + val errorMsg = "No JAAS configuration file has been specified. Please make sure that you have set " + + "the system property %s".format(JaasUtils.JAVA_LOGIN_CONFIG_PARAM) System.out.println("ERROR: %s".format(errorMsg)) throw new IllegalArgumentException("Incorrect configuration") } diff --git a/core/src/main/scala/kafka/api/ControlledShutdownRequest.scala b/core/src/main/scala/kafka/api/ControlledShutdownRequest.scala index b875e3e0a3626..42a17e67fca33 100644 --- a/core/src/main/scala/kafka/api/ControlledShutdownRequest.scala +++ b/core/src/main/scala/kafka/api/ControlledShutdownRequest.scala @@ -19,7 +19,7 @@ package kafka.api import java.nio.ByteBuffer -import kafka.common.{TopicAndPartition} +import kafka.common.TopicAndPartition import kafka.api.ApiUtils._ import kafka.network.{RequestOrResponseSend, RequestChannel} import kafka.network.RequestChannel.Response diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 4e79bdcf388c0..ea22e87558137 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -296,46 +296,48 @@ class Partition(val topic: String, } /* - * Note that this method will only be called if requiredAcks = -1 - * and we are waiting for all replicas in ISR to be fully caught up to - * the (local) leader's offset corresponding to this produce request - * before we acknowledge the produce request. + * Returns a tuple where the first element is a boolean indicating whether enough replicas reached `requiredOffset` + * and the second element is an error (which would be `Errors.NONE` for no error). + * + * Note that this method will only be called if requiredAcks = -1 and we are waiting for all replicas in ISR to be + * fully caught up to the (local) leader's offset corresponding to this produce request before we acknowledge the + * produce request. */ - def checkEnoughReplicasReachOffset(requiredOffset: Long): (Boolean, Short) = { + def checkEnoughReplicasReachOffset(requiredOffset: Long): (Boolean, Errors) = { leaderReplicaIfLocal() match { case Some(leaderReplica) => // keep the current immutable replica list reference val curInSyncReplicas = inSyncReplicas - val numAcks = curInSyncReplicas.count(r => { + + def numAcks = curInSyncReplicas.count { r => if (!r.isLocal) if (r.logEndOffset.messageOffset >= requiredOffset) { - trace("Replica %d of %s-%d received offset %d".format(r.brokerId, topic, partitionId, requiredOffset)) + trace(s"Replica ${r.brokerId} of ${topic}-${partitionId} received offset $requiredOffset") true } else false else true /* also count the local (leader) replica */ - }) + } - trace("%d acks satisfied for %s-%d with acks = -1".format(numAcks, topic, partitionId)) + trace(s"$numAcks acks satisfied for ${topic}-${partitionId} with acks = -1") val minIsr = leaderReplica.log.get.config.minInSyncReplicas - if (leaderReplica.highWatermark.messageOffset >= requiredOffset ) { + if (leaderReplica.highWatermark.messageOffset >= requiredOffset) { /* - * The topic may be configured not to accept messages if there are not enough replicas in ISR - * in this scenario the request was already appended locally and then added to the purgatory before the ISR was shrunk - */ - if (minIsr <= curInSyncReplicas.size) { - (true, Errors.NONE.code) - } else { - (true, Errors.NOT_ENOUGH_REPLICAS_AFTER_APPEND.code) - } + * The topic may be configured not to accept messages if there are not enough replicas in ISR + * in this scenario the request was already appended locally and then added to the purgatory before the ISR was shrunk + */ + if (minIsr <= curInSyncReplicas.size) + (true, Errors.NONE) + else + (true, Errors.NOT_ENOUGH_REPLICAS_AFTER_APPEND) } else - (false, Errors.NONE.code) + (false, Errors.NONE) case None => - (false, Errors.NOT_LEADER_FOR_PARTITION.code) + (false, Errors.NOT_LEADER_FOR_PARTITION) } } diff --git a/core/src/main/scala/kafka/consumer/BaseConsumer.scala b/core/src/main/scala/kafka/consumer/BaseConsumer.scala index 3774e735089ae..8252cfc7915a9 100644 --- a/core/src/main/scala/kafka/consumer/BaseConsumer.scala +++ b/core/src/main/scala/kafka/consumer/BaseConsumer.scala @@ -27,7 +27,7 @@ import org.apache.kafka.common.record.TimestampType /** * A base consumer used to abstract both old and new consumer - * this class should be removed (along with BaseProducer) be removed + * this class should be removed (along with BaseProducer) * once we deprecate old consumer */ trait BaseConsumer { diff --git a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala index fb712541ec87c..f4457647ece83 100644 --- a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala @@ -21,7 +21,7 @@ import java.util.concurrent.atomic.AtomicBoolean import kafka.common.{OffsetAndMetadata, OffsetMetadataAndError, TopicAndPartition} import kafka.log.LogConfig -import kafka.message.UncompressedCodec +import kafka.message.ProducerCompressionCodec import kafka.server._ import kafka.utils._ import org.apache.kafka.common.TopicPartition @@ -65,7 +65,7 @@ class GroupCoordinator(val brokerId: Int, val props = new Properties props.put(LogConfig.CleanupPolicyProp, LogConfig.Compact) props.put(LogConfig.SegmentBytesProp, offsetConfig.offsetsTopicSegmentBytes.toString) - props.put(LogConfig.CompressionTypeProp, UncompressedCodec.name) + props.put(LogConfig.CompressionTypeProp, ProducerCompressionCodec.name) props } @@ -744,14 +744,16 @@ object GroupCoordinator { offsetsRetentionMs = config.offsetsRetentionMinutes * 60 * 1000L, offsetsRetentionCheckIntervalMs = config.offsetsRetentionCheckIntervalMs, offsetsTopicNumPartitions = config.offsetsTopicPartitions, + offsetsTopicSegmentBytes = config.offsetsTopicSegmentBytes, offsetsTopicReplicationFactor = config.offsetsTopicReplicationFactor, + offsetsTopicCompressionCodec = config.offsetsTopicCompressionCodec, offsetCommitTimeoutMs = config.offsetCommitTimeoutMs, offsetCommitRequiredAcks = config.offsetCommitRequiredAcks) val groupConfig = GroupConfig(groupMinSessionTimeoutMs = config.groupMinSessionTimeoutMs, groupMaxSessionTimeoutMs = config.groupMaxSessionTimeoutMs) - val groupManager = new GroupMetadataManager(config.brokerId, offsetConfig, replicaManager, zkUtils, time) - new GroupCoordinator(config.brokerId, groupConfig, offsetConfig, groupManager, heartbeatPurgatory, joinPurgatory, time) + val groupMetadataManager = new GroupMetadataManager(config.brokerId, offsetConfig, replicaManager, zkUtils, time) + new GroupCoordinator(config.brokerId, groupConfig, offsetConfig, groupMetadataManager, heartbeatPurgatory, joinPurgatory, time) } } diff --git a/core/src/main/scala/kafka/log/FileMessageSet.scala b/core/src/main/scala/kafka/log/FileMessageSet.scala index a164b4b96730c..a454f2cbe2ae3 100755 --- a/core/src/main/scala/kafka/log/FileMessageSet.scala +++ b/core/src/main/scala/kafka/log/FileMessageSet.scala @@ -83,7 +83,7 @@ class FileMessageSet private[kafka](@volatile var file: File, this(file, channel = FileMessageSet.openChannel(file, mutable = true, fileAlreadyExists, initFileSize, preallocate), start = 0, - end = ( if ( !fileAlreadyExists && preallocate ) 0 else Int.MaxValue), + end = if (!fileAlreadyExists && preallocate) 0 else Int.MaxValue, isSlice = false) /** @@ -224,7 +224,7 @@ class FileMessageSet private[kafka](@volatile var file: File, } } - if (sizeInBytes > 0 && newMessages.size == 0) { + if (sizeInBytes > 0 && newMessages.isEmpty) { // This indicates that the message is too large. We just return all the bytes in the file message set. this } else { diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index a7549dc134a79..62dc7a1788408 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -35,7 +35,7 @@ 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, false) + val UnknownLogAppendInfo = LogAppendInfo(-1, -1, Message.NoTimestamp, NoCompressionCodec, NoCompressionCodec, -1, -1, offsetsMonotonic = false) } /** @@ -228,7 +228,7 @@ class Log(val dir: File, replaceSegments(swapSegment, oldSegments.toSeq, isRecoveredSwapFile = true) } - if(logSegments.size == 0) { + if(logSegments.isEmpty) { // no existing segments, create a new mutable segment beginning at offset 0 segments.put(0L, new LogSegment(dir = dir, startOffset = 0, diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index c6636be09428e..4c0db0d9d451b 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -578,12 +578,12 @@ private[log] class Cleaner(val id: Int, private[log] def groupSegmentsBySize(segments: Iterable[LogSegment], maxSize: Int, maxIndexSize: Int): List[Seq[LogSegment]] = { var grouped = List[List[LogSegment]]() var segs = segments.toList - while(!segs.isEmpty) { + while(segs.nonEmpty) { var group = List(segs.head) var logSize = segs.head.size var indexSize = segs.head.index.sizeInBytes segs = segs.tail - while(!segs.isEmpty && + while(segs.nonEmpty && logSize + segs.head.size <= maxSize && indexSize + segs.head.index.sizeInBytes <= maxIndexSize && segs.head.index.lastOffset - group.last.index.baseOffset <= Int.MaxValue) { diff --git a/core/src/main/scala/kafka/log/LogCleanerManager.scala b/core/src/main/scala/kafka/log/LogCleanerManager.scala index f92db4ed844fb..72757c083d44d 100755 --- a/core/src/main/scala/kafka/log/LogCleanerManager.scala +++ b/core/src/main/scala/kafka/log/LogCleanerManager.scala @@ -100,7 +100,7 @@ private[log] class LogCleanerManager(val logDirs: Array[File], val logs: Pool[To LogToClean(topicAndPartition, log, firstDirtyOffset) }.filter(ltc => ltc.totalBytes > 0) // skip any empty logs - this.dirtiestLogCleanableRatio = if (!dirtyLogs.isEmpty) dirtyLogs.max.cleanableRatio else 0 + this.dirtiestLogCleanableRatio = if (dirtyLogs.nonEmpty) dirtyLogs.max.cleanableRatio else 0 // and must meet the minimum threshold for dirty byte ratio val cleanableLogs = dirtyLogs.filter(ltc => ltc.cleanableRatio > ltc.log.config.minCleanableRatio) if(cleanableLogs.isEmpty) { diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index 749c6229a7d09..4357ef4c5d36e 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -132,10 +132,9 @@ class LogManager(val logDirs: Array[File], try { recoveryPoints = this.recoveryPointCheckpoints(dir).read } catch { - case e: Exception => { + case e: Exception => warn("Error occured while reading recovery-point-offset-checkpoint file of directory " + dir, e) warn("Resetting the recovery checkpoint to 0") - } } val jobsForDir = for { @@ -282,7 +281,7 @@ class LogManager(val logDirs: Array[File], // If the log does not exist, skip it if (log != null) { //May need to abort and pause the cleaning of the log, and resume after truncation is done. - val needToStopCleaner: Boolean = (truncateOffset < log.activeSegment.baseOffset) + val needToStopCleaner: Boolean = truncateOffset < log.activeSegment.baseOffset if (needToStopCleaner && cleaner != null) cleaner.abortAndPauseCleaning(topicAndPartition) log.truncateTo(truncateOffset) diff --git a/core/src/main/scala/kafka/log/OffsetIndex.scala b/core/src/main/scala/kafka/log/OffsetIndex.scala index ce35d6874c7c0..f4327324e0637 100755 --- a/core/src/main/scala/kafka/log/OffsetIndex.scala +++ b/core/src/main/scala/kafka/log/OffsetIndex.scala @@ -24,9 +24,11 @@ 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: @@ -306,8 +308,10 @@ class OffsetIndex(@volatile private[this] var _file: File, val baseOffset: Long, */ private def forceUnmap(m: MappedByteBuffer) { try { - if(m.isInstanceOf[sun.nio.ch.DirectBuffer]) - (m.asInstanceOf[sun.nio.ch.DirectBuffer]).cleaner().clean() + m match { + case buffer: DirectBuffer => buffer.cleaner().clean() + case _ => + } } catch { case t: Throwable => warn("Error when freeing index buffer", t) } diff --git a/core/src/main/scala/kafka/log/OffsetMap.scala b/core/src/main/scala/kafka/log/OffsetMap.scala index 3893b2c7b1c28..f4530307df8a9 100755 --- a/core/src/main/scala/kafka/log/OffsetMap.scala +++ b/core/src/main/scala/kafka/log/OffsetMap.scala @@ -118,7 +118,12 @@ class SkimpyOffsetMap(val memory: Int, val hashAlgorithm: String = "MD5") extend // search for the hash of this key by repeated probing until we find the hash we are looking for or we find an empty slot var attempt = 0 var pos = 0 + //we need to guard against attempt integer overflow if the map is full + //limit attempt to number of slots once positionOf(..) enters linear search mode + val maxAttempts = slots + hashSize - 4 do { + if(attempt >= maxAttempts) + return -1L pos = positionOf(hash1, attempt) bytes.position(pos) if(isEmpty(pos)) diff --git a/core/src/main/scala/kafka/network/RequestChannel.scala b/core/src/main/scala/kafka/network/RequestChannel.scala index e2000dbfd04c9..53a2346b42a30 100644 --- a/core/src/main/scala/kafka/network/RequestChannel.scala +++ b/core/src/main/scala/kafka/network/RequestChannel.scala @@ -100,7 +100,7 @@ object RequestChannel extends Logging { buffer = null private val requestLogger = Logger.getLogger("kafka.request.logger") - private def requestDesc(details: Boolean): String = { + def requestDesc(details: Boolean): String = { if (requestObj != null) requestObj.describe(details) else diff --git a/core/src/main/scala/kafka/server/DelayedProduce.scala b/core/src/main/scala/kafka/server/DelayedProduce.scala index be1be4faf97c7..5a59d3be7bfdc 100644 --- a/core/src/main/scala/kafka/server/DelayedProduce.scala +++ b/core/src/main/scala/kafka/server/DelayedProduce.scala @@ -82,32 +82,26 @@ class DelayedProduce(delayMs: Long, override def tryComplete(): Boolean = { // check for each partition if it still has pending acks produceMetadata.produceStatus.foreach { case (topicAndPartition, status) => - trace("Checking produce satisfaction for %s, current status %s" - .format(topicAndPartition, status)) + trace(s"Checking produce satisfaction for ${topicAndPartition}, current status $status") // skip those partitions that have already been satisfied if (status.acksPending) { - val partitionOpt = replicaManager.getPartition(topicAndPartition.topic, topicAndPartition.partition) - val (hasEnough, errorCode) = partitionOpt match { + val (hasEnough, error) = replicaManager.getPartition(topicAndPartition.topic, topicAndPartition.partition) match { case Some(partition) => partition.checkEnoughReplicasReachOffset(status.requiredOffset) case None => // Case A - (false, Errors.UNKNOWN_TOPIC_OR_PARTITION.code) + (false, Errors.UNKNOWN_TOPIC_OR_PARTITION) } - if (errorCode != Errors.NONE.code) { - // Case B.1 + // Case B.1 || B.2 + if (error != Errors.NONE || hasEnough) { status.acksPending = false - status.responseStatus.errorCode = errorCode - } else if (hasEnough) { - // Case B.2 - status.acksPending = false - status.responseStatus.errorCode = Errors.NONE.code + status.responseStatus.errorCode = error.code } } } - // check if each partition has satisfied at lease one of case A and case B - if (! produceMetadata.produceStatus.values.exists(p => p.acksPending)) + // check if every partition has satisfied at least one of case A or B + if (!produceMetadata.produceStatus.values.exists(_.acksPending)) forceComplete() else false diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 086bd4b893dba..1edc16242db5c 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -71,7 +71,7 @@ class KafkaApis(val requestChannel: RequestChannel, def handle(request: RequestChannel.Request) { try { trace("Handling request:%s from connection %s;securityProtocol:%s,principal:%s". - format(request.requestObj, request.connectionId, request.securityProtocol, request.session.principal)) + format(request.requestDesc(true), request.connectionId, request.securityProtocol, request.session.principal)) ApiKeys.forId(request.requestId) match { case ApiKeys.PRODUCE => handleProducerRequest(request) case ApiKeys.FETCH => handleFetchRequest(request) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 2832ebc6266ed..f95d9ef74039d 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -123,7 +123,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime, threadNamePr var dynamicConfigHandlers: Map[String, ConfigHandler] = null var dynamicConfigManager: DynamicConfigManager = null - var consumerCoordinator: GroupCoordinator = null + var groupCoordinator: GroupCoordinator = null var kafkaController: KafkaController = null @@ -199,9 +199,9 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime, threadNamePr kafkaController = new KafkaController(config, zkUtils, brokerState, kafkaMetricsTime, metrics, threadNamePrefix) kafkaController.startup() - /* start kafka coordinator */ - consumerCoordinator = GroupCoordinator(config, zkUtils, replicaManager, kafkaMetricsTime) - consumerCoordinator.startup() + /* start group coordinator */ + groupCoordinator = GroupCoordinator(config, zkUtils, replicaManager, kafkaMetricsTime) + groupCoordinator.startup() /* Get the authorizer and initialize it if one is specified.*/ authorizer = Option(config.authorizerClassName).filter(_.nonEmpty).map { authorizerClassName => @@ -211,10 +211,9 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime, threadNamePr } /* start processing requests */ - apis = new KafkaApis(socketServer.requestChannel, replicaManager, consumerCoordinator, + apis = new KafkaApis(socketServer.requestChannel, replicaManager, groupCoordinator, kafkaController, zkUtils, config.brokerId, config, metadataCache, metrics, authorizer) requestHandlerPool = new KafkaRequestHandlerPool(config.brokerId, socketServer.requestChannel, apis, config.numIoThreads) - brokerState.newState(RunningAsBroker) Mx4jLoader.maybeLoad() @@ -249,6 +248,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime, threadNamePr /* register broker metrics */ registerStats() + brokerState.newState(RunningAsBroker) shutdownLatch = new CountDownLatch(1) startupComplete.set(true) isStartingUp.set(false) @@ -555,8 +555,8 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime, threadNamePr CoreUtils.swallow(replicaManager.shutdown()) if(logManager != null) CoreUtils.swallow(logManager.shutdown()) - if(consumerCoordinator != null) - CoreUtils.swallow(consumerCoordinator.shutdown()) + if(groupCoordinator != null) + CoreUtils.swallow(groupCoordinator.shutdown()) if(kafkaController != null) CoreUtils.swallow(kafkaController.shutdown()) if(zkUtils != null) diff --git a/core/src/main/scala/kafka/tools/ConsoleProducer.scala b/core/src/main/scala/kafka/tools/ConsoleProducer.scala index e6476015f1901..4cc7c2000bc14 100644 --- a/core/src/main/scala/kafka/tools/ConsoleProducer.scala +++ b/core/src/main/scala/kafka/tools/ConsoleProducer.scala @@ -311,7 +311,7 @@ object ConsoleProducer { line.indexOf(keySeparator) match { case -1 => if (ignoreError) new ProducerRecord(topic, line.getBytes) - else throw new KafkaException(s"No key found on line ${lineNumber}: $line") + else throw new KafkaException(s"No key found on line $lineNumber: $line") case n => val value = (if (n + keySeparator.size > line.size) "" else line.substring(n + keySeparator.size)).getBytes new ProducerRecord(topic, line.substring(0, n).getBytes, value) diff --git a/core/src/main/scala/kafka/tools/ExportZkOffsets.scala b/core/src/main/scala/kafka/tools/ExportZkOffsets.scala index ccccae5792321..907e8acd72946 100644 --- a/core/src/main/scala/kafka/tools/ExportZkOffsets.scala +++ b/core/src/main/scala/kafka/tools/ExportZkOffsets.scala @@ -25,7 +25,7 @@ import org.apache.kafka.common.security.JaasUtils /** - * A utility that retrieve the offset of broker partitions in ZK and + * A utility that retrieves the offset of broker partitions in ZK and * prints to an output file in the following format: * * /consumers/group1/offsets/topic1/1-0:286894308 diff --git a/core/src/main/scala/kafka/tools/GetOffsetShell.scala b/core/src/main/scala/kafka/tools/GetOffsetShell.scala index 30c7afe7edca8..f7207eca2fc84 100644 --- a/core/src/main/scala/kafka/tools/GetOffsetShell.scala +++ b/core/src/main/scala/kafka/tools/GetOffsetShell.scala @@ -77,7 +77,7 @@ object GetOffsetShell { val maxWaitMs = options.valueOf(maxWaitMsOpt).intValue() val topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic), metadataTargetBrokers, clientId, maxWaitMs).topicsMetadata - if(topicsMetadata.size != 1 || !topicsMetadata(0).topic.equals(topic)) { + if(topicsMetadata.size != 1 || !topicsMetadata.head.topic.equals(topic)) { System.err.println(("Error: no valid topic metadata for topic: %s, " + " probably the topic does not exist, run ").format(topic) + "kafka-list-topic.sh to verify") System.exit(1) diff --git a/core/src/main/scala/kafka/tools/JmxTool.scala b/core/src/main/scala/kafka/tools/JmxTool.scala index bd7ca0e85365e..8112f9ea66ef1 100644 --- a/core/src/main/scala/kafka/tools/JmxTool.scala +++ b/core/src/main/scala/kafka/tools/JmxTool.scala @@ -89,7 +89,7 @@ object JmxTool extends Logging { else List(null) - val names = queries.map((name: ObjectName) => mbsc.queryNames(name, null): mutable.Set[ObjectName]).flatten + val names = queries.flatMap((name: ObjectName) => mbsc.queryNames(name, null): mutable.Set[ObjectName]) val numExpectedAttributes: Map[ObjectName, Int] = attributesWhitelistExists match { @@ -101,7 +101,7 @@ object JmxTool extends Logging { // print csv header val keys = List("time") ++ queryAttributes(mbsc, names, attributesWhitelist).keys.toArray.sorted - if(keys.size == numExpectedAttributes.map(_._2).sum + 1) + if(keys.size == numExpectedAttributes.values.sum + 1) println(keys.map("\"" + _ + "\"").mkString(",")) while(true) { @@ -111,7 +111,7 @@ object JmxTool extends Logging { case Some(dFormat) => dFormat.format(new Date) case None => System.currentTimeMillis().toString } - if(attributes.keySet.size == numExpectedAttributes.map(_._2).sum + 1) + if(attributes.keySet.size == numExpectedAttributes.values.sum + 1) println(keys.map(attributes(_)).mkString(",")) val sleep = max(0, interval - (System.currentTimeMillis - start)) Thread.sleep(sleep) diff --git a/core/src/main/scala/kafka/tools/MirrorMaker.scala b/core/src/main/scala/kafka/tools/MirrorMaker.scala index 87f3cc53bafb1..9d5f7e6040c40 100755 --- a/core/src/main/scala/kafka/tools/MirrorMaker.scala +++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala @@ -494,7 +494,7 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { // Creating one stream per each connector instance val streams = connector.createMessageStreamsByFilter(filterSpec, 1, new DefaultDecoder(), new DefaultDecoder()) require(streams.size == 1) - val stream = streams(0) + val stream = streams.head iter = stream.iterator() } diff --git a/core/src/test/scala/integration/kafka/api/AdminClientTest.scala b/core/src/test/scala/integration/kafka/api/AdminClientTest.scala index 7fae81e8622dd..3d39475d3d074 100644 --- a/core/src/test/scala/integration/kafka/api/AdminClientTest.scala +++ b/core/src/test/scala/integration/kafka/api/AdminClientTest.scala @@ -106,7 +106,7 @@ class AdminClientTest extends IntegrationTestHarness with Logging { val consumerSummaries = client.describeConsumerGroup(groupId) assertEquals(1, consumerSummaries.size) - assertEquals(Set(tp, tp2), consumerSummaries.head.assignment.toSet) + assertEquals(Some(Set(tp, tp2)), consumerSummaries.map(_.head.assignment.toSet)) } @Test diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index bc705f13f10d3..2d5900f8364a7 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -148,7 +148,7 @@ class AuthorizerIntegrationTest extends KafkaServerTestHarness { 1, 1, servers, - servers.head.consumerCoordinator.offsetsTopicConfigs) + servers.head.groupCoordinator.offsetsTopicConfigs) // create the test topic with all the brokers as replicas TestUtils.createTopic(zkUtils, topic, 1, 1, this.servers) } diff --git a/core/src/test/scala/integration/kafka/api/GroupCoordinatorIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/GroupCoordinatorIntegrationTest.scala new file mode 100644 index 0000000000000..9183d0f36e28d --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/GroupCoordinatorIntegrationTest.scala @@ -0,0 +1,63 @@ +/** + * 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 integration.kafka.api + +import kafka.common.TopicAndPartition +import kafka.integration.KafkaServerTestHarness +import kafka.log.Log +import kafka.message.GZIPCompressionCodec +import kafka.server.KafkaConfig +import kafka.utils.TestUtils +import org.apache.kafka.clients.consumer.OffsetAndMetadata +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.internals.TopicConstants +import org.apache.kafka.common.protocol.SecurityProtocol +import org.junit.Test +import org.junit.Assert._ + +import scala.collection.JavaConverters._ +import java.util.Properties + +class GroupCoordinatorIntegrationTest extends KafkaServerTestHarness { + val offsetsTopicCompressionCodec = GZIPCompressionCodec + val overridingProps = new Properties() + overridingProps.put(KafkaConfig.OffsetsTopicPartitionsProp, "1") + overridingProps.put(KafkaConfig.OffsetsTopicCompressionCodecProp, offsetsTopicCompressionCodec.codec.toString) + + override def generateConfigs = TestUtils.createBrokerConfigs(1, zkConnect, enableControlledShutdown = false).map { + KafkaConfig.fromProps(_, overridingProps) + } + + @Test + def testGroupCoordinatorPropagatesOfffsetsTopicCompressionCodec() { + val consumer = TestUtils.createNewConsumer(TestUtils.getBrokerListStrFromServers(servers), + securityProtocol = SecurityProtocol.PLAINTEXT) + val offsetMap = Map( + new TopicPartition(TopicConstants.GROUP_METADATA_TOPIC_NAME, 0) -> new OffsetAndMetadata(10, "") + ).asJava + consumer.commitSync(offsetMap) + val logManager = servers.head.getLogManager + + def getGroupMetadataLogOpt: Option[Log] = + logManager.getLog(TopicAndPartition(TopicConstants.GROUP_METADATA_TOPIC_NAME, 0)) + + TestUtils.waitUntilTrue(() => getGroupMetadataLogOpt.exists(_.logSegments.exists(_.log.nonEmpty)), + "Commit message not appended in time") + + val logSegments = getGroupMetadataLogOpt.get.logSegments + val incorrectCompressionCodecs = logSegments.flatMap(_.log.map(_.message.compressionCodec)).filter(_ != offsetsTopicCompressionCodec) + assertEquals("Incorrect compression codecs should be empty", Seq.empty, incorrectCompressionCodecs) + + consumer.close() + } +} diff --git a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala index de05c9c6ea4b5..6e76f90b21a32 100644 --- a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala +++ b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala @@ -81,7 +81,7 @@ trait IntegrationTestHarness extends KafkaServerTestHarness { serverConfig.getProperty(KafkaConfig.OffsetsTopicPartitionsProp).toInt, serverConfig.getProperty(KafkaConfig.OffsetsTopicReplicationFactorProp).toInt, servers, - servers(0).consumerCoordinator.offsetsTopicConfigs) + servers(0).groupCoordinator.offsetsTopicConfigs) } @After diff --git a/core/src/test/scala/unit/kafka/admin/AclCommandTest.scala b/core/src/test/scala/unit/kafka/admin/AclCommandTest.scala index d43d0d48dd2e4..7ada48ebb8e3f 100644 --- a/core/src/test/scala/unit/kafka/admin/AclCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AclCommandTest.scala @@ -116,12 +116,10 @@ class AclCommandTest extends ZooKeeperTestHarness with Logging { private def testRemove(resources: Set[Resource], resourceCmd: Array[String], args: Array[String], brokerProps: Properties) { for (resource <- resources) { - Console.withIn(new StringReader(s"y${AclCommand.Newline}" * resources.size)) { - AclCommand.main(args ++ resourceCmd :+ "--remove") + AclCommand.main(args ++ resourceCmd :+ "--remove" :+ "--force") withAuthorizer(brokerProps) { authorizer => TestUtils.waitAndVerifyAcls(Set.empty[Acl], authorizer, resource) } - } } } diff --git a/core/src/test/scala/unit/kafka/log/OffsetMapTest.scala b/core/src/test/scala/unit/kafka/log/OffsetMapTest.scala index f50daa43ef1d7..a5bec1766da70 100644 --- a/core/src/test/scala/unit/kafka/log/OffsetMapTest.scala +++ b/core/src/test/scala/unit/kafka/log/OffsetMapTest.scala @@ -44,7 +44,19 @@ class OffsetMapTest extends JUnitSuite { assertEquals(map.get(key(i)), -1L) } - def key(key: Int) = ByteBuffer.wrap(key.toString.getBytes) + @Test + def testGetWhenFull() { + val map = new SkimpyOffsetMap(4096) + var i = 37L //any value would do + while (map.size < map.slots) { + map.put(key(i), i) + i = i + 1L + } + assertEquals(map.get(key(i)), -1L) + assertEquals(map.get(key(i-1L)), i-1L) + } + + def key(key: Long) = ByteBuffer.wrap(key.toString.getBytes) def validateMap(items: Int, loadFactor: Double = 0.5): SkimpyOffsetMap = { val map = new SkimpyOffsetMap((items/loadFactor * 24).toInt) diff --git a/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala b/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala index 9b49365c77060..b5560c36d623c 100755 --- a/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala @@ -20,8 +20,8 @@ package kafka.server import kafka.utils.ZkUtils import kafka.utils.CoreUtils import kafka.utils.TestUtils - import kafka.zk.ZooKeeperTestHarness +import org.easymock.EasyMock import org.junit.Assert._ import org.junit.Test @@ -82,4 +82,30 @@ class ServerStartupTest extends ZooKeeperTestHarness { server.shutdown() CoreUtils.delete(server.config.logDirs) } + + @Test + def testBrokerStateRunningAfterZK { + val brokerId = 0 + val mockBrokerState = EasyMock.niceMock(classOf[kafka.server.BrokerState]) + + class BrokerStateInterceptor() extends BrokerState { + override def newState(newState: BrokerStates): Unit = { + val brokers = zkUtils.getAllBrokersInCluster() + assertEquals(1, brokers.size) + assertEquals(brokerId, brokers.head.id) + } + } + + class MockKafkaServer(override val config: KafkaConfig, override val brokerState: BrokerState = mockBrokerState) extends KafkaServer(config) {} + + val props = TestUtils.createBrokerConfig(brokerId, zkConnect) + val server = new MockKafkaServer(KafkaConfig.fromProps(props)) + + EasyMock.expect(mockBrokerState.newState(RunningAsBroker)).andDelegateTo(new BrokerStateInterceptor).once() + EasyMock.replay(mockBrokerState) + + server.startup() + server.shutdown() + CoreUtils.delete(server.config.logDirs) + } } diff --git a/docs/documentation.html b/docs/documentation.html index ddc3102180182..31dc03960c4fa 100644 --- a/docs/documentation.html +++ b/docs/documentation.html @@ -123,7 +123,8 @@

Kafka 0.10.0 Documentation

  • 7.2 Encryption and Authentication using SSL
  • 7.3 Authentication using SASL
  • 7.4 Authorization and ACLs
  • -
  • 7.5 ZooKeeper Authentication
  • +
  • 7.5 Incorporating Security Features in a Running Cluster
  • +
  • 7.6 ZooKeeper Authentication