Merge remote-tracking branch 'origin/trunk' into copycat

This commit is contained in:
Ewen Cheslack-Postava 2015-08-12 19:43:12 -07:00
commit b29cb2c5cd
65 changed files with 2086 additions and 900 deletions

1
.gitignore vendored
View File

@ -24,6 +24,7 @@ kafka.ipr
kafka.iws
.vagrant
Vagrantfile.local
/logs
config/server-*
config/zookeeper-*

17
bin/kafka-configs.sh Executable file
View File

@ -0,0 +1,17 @@
#!/bin/bash
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
exec $(dirname $0)/kafka-run-class.sh kafka.admin.ConfigCommand $@

View File

@ -22,15 +22,6 @@ fi
base_dir=$(dirname $0)/..
# create logs directory
if [ "x$LOG_DIR" = "x" ]; then
LOG_DIR="$base_dir/logs"
fi
if [ ! -d "$LOG_DIR" ]; then
mkdir -p "$LOG_DIR"
fi
if [ -z "$SCALA_VERSION" ]; then
SCALA_VERSION=2.10.5
fi
@ -104,9 +95,20 @@ if [ $JMX_PORT ]; then
KAFKA_JMX_OPTS="$KAFKA_JMX_OPTS -Dcom.sun.management.jmxremote.port=$JMX_PORT "
fi
# Log directory to use
if [ "x$LOG_DIR" = "x" ]; then
LOG_DIR="$base_dir/logs"
fi
# Log4j settings
if [ -z "$KAFKA_LOG4J_OPTS" ]; then
# Log to console. This is a tool.
KAFKA_LOG4J_OPTS="-Dlog4j.configuration=file:$base_dir/config/tools-log4j.properties"
else
# create logs directory
if [ ! -d "$LOG_DIR" ]; then
mkdir -p "$LOG_DIR"
fi
fi
KAFKA_LOG4J_OPTS="-Dkafka.logs.dir=$LOG_DIR $KAFKA_LOG4J_OPTS"

View File

@ -21,11 +21,13 @@ import org.apache.kafka.common.Metric;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.annotation.InterfaceStability;
/**
* @see KafkaConsumer
* @see MockConsumer
*/
@InterfaceStability.Unstable
public interface Consumer<K, V> extends Closeable {
/**
@ -118,6 +120,16 @@ public interface Consumer<K, V> extends Closeable {
*/
public Map<String, List<PartitionInfo>> listTopics();
/**
* @see KafkaConsumer#pause(TopicPartition...)
*/
public void pause(TopicPartition... partitions);
/**
* @see KafkaConsumer#resume(TopicPartition...)
*/
public void resume(TopicPartition... partitions);
/**
* @see KafkaConsumer#close()
*/

View File

@ -50,6 +50,12 @@ public class ConsumerConfig extends AbstractConfig {
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.";
/**
* <code>heartbeat.interval.ms</code>
*/
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 <code>session.timeout.ms</code>, 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.";
/**
* <code>bootstrap.servers</code>
*/
@ -77,7 +83,7 @@ public class ConsumerConfig extends AbstractConfig {
* <code>auto.offset.reset</code>
*/
public static final String AUTO_OFFSET_RESET_CONFIG = "auto.offset.reset";
private static final String AUTO_OFFSET_RESET_DOC = "What to do when there is no initial offset in Kafka or if the current offset does not exist any more on the server (e.g. because that data has been deleted): <ul><li>smallest: automatically reset the offset to the smallest offset<li>largest: automatically reset the offset to the largest offset</li><li>none: throw exception to the consumer if no previous offset is found for the consumer's group</li><li>anything else: throw exception to the consumer.</li></ul>";
private static final String AUTO_OFFSET_RESET_DOC = "What to do when there is no initial offset in Kafka or if the current offset does not exist any more on the server (e.g. because that data has been deleted): <ul><li>earliest: automatically reset the offset to the earliest offset<li>latest: automatically reset the offset to the latest offset</li><li>none: throw exception to the consumer if no previous offset is found for the consumer's group</li><li>anything else: throw exception to the consumer.</li></ul>";
/**
* <code>fetch.min.bytes</code>
@ -171,6 +177,11 @@ public class ConsumerConfig extends AbstractConfig {
30000,
Importance.HIGH,
SESSION_TIMEOUT_MS_DOC)
.define(HEARTBEAT_INTERVAL_MS_CONFIG,
Type.INT,
3000,
Importance.HIGH,
HEARTBEAT_INTERVAL_MS_DOC)
.define(PARTITION_ASSIGNMENT_STRATEGY_CONFIG,
Type.STRING,
"range",

View File

@ -24,11 +24,12 @@ public final class ConsumerRecord<K, V> {
private final V value;
/**
* Create a record with no key
*
* Creates a record to be received from a specified topic and partition
*
* @param topic The topic this record is received from
* @param partition The partition of the topic this record is received from
* @param offset The offset of this record in the corresponding Kafka partition
* @param key The key of the record, if one exists (null is allowed)
* @param value The record contents
*/
public ConsumerRecord(String topic, int partition, long offset, K key, V value) {

View File

@ -23,8 +23,8 @@ import java.util.Map;
/**
* A container that holds the list {@link ConsumerRecord} per partition for a
* particular topic. There is one for every topic returned by a
* {@link Consumer#poll(long)} operation.
* particular topic. There is one {@link ConsumerRecord} list for every topic
* partition returned by a {@link Consumer#poll(long)} operation.
*/
public class ConsumerRecords<K, V> implements Iterable<ConsumerRecord<K, V>> {
public static final ConsumerRecords<Object, Object> EMPTY =

View File

@ -26,6 +26,7 @@ import org.apache.kafka.common.Metric;
import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.annotation.InterfaceStability;
import org.apache.kafka.common.metrics.JmxReporter;
import org.apache.kafka.common.metrics.MetricConfig;
import org.apache.kafka.common.metrics.Metrics;
@ -43,7 +44,6 @@ import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.ConcurrentModificationException;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
@ -391,6 +391,7 @@ import static org.apache.kafka.common.utils.Utils.min;
* commit.
*
*/
@InterfaceStability.Unstable
public class KafkaConsumer<K, V> implements Consumer<K, V> {
private static final Logger log = LoggerFactory.getLogger(KafkaConsumer.class);
@ -541,6 +542,7 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
this.coordinator = new Coordinator(this.client,
config.getString(ConsumerConfig.GROUP_ID_CONFIG),
config.getInt(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG),
config.getInt(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG),
config.getString(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG),
this.subscriptions,
metrics,
@ -852,9 +854,7 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
public void commit(CommitType commitType, ConsumerCommitCallback callback) {
acquire();
try {
// need defensive copy to ensure offsets are not removed before completion (e.g. in rebalance)
Map<TopicPartition, Long> allConsumed = new HashMap<TopicPartition, Long>(this.subscriptions.allConsumed());
commit(allConsumed, commitType, callback);
commit(subscriptions.allConsumed(), commitType, callback);
} finally {
release();
}
@ -941,7 +941,7 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
public long position(TopicPartition partition) {
acquire();
try {
if (!this.subscriptions.assignedPartitions().contains(partition))
if (!this.subscriptions.isAssigned(partition))
throw new IllegalArgumentException("You can only check the position for partitions assigned to this consumer.");
Long offset = this.subscriptions.consumed(partition);
if (offset == null) {
@ -963,7 +963,7 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
* consumer hasn't yet initialized it's cache of committed offsets.
*
* @param partition The partition to check
* @return The last committed offset or null if no offset has been committed
* @return The last committed offset
* @throws NoOffsetForPartitionException If no offset has ever been committed by any process for the given
* partition.
*/
@ -972,7 +972,7 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
acquire();
try {
Long committed;
if (subscriptions.assignedPartitions().contains(partition)) {
if (subscriptions.isAssigned(partition)) {
committed = this.subscriptions.committed(partition);
if (committed == null) {
coordinator.refreshCommittedOffsetsIfNeeded();
@ -1040,6 +1040,45 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
}
}
/**
* Suspend fetching from the requested partitions. Future calls to {@link #poll(long)} will not return
* any records from these partitions until they have been resumed using {@link #resume(TopicPartition...)}.
* Note that this method does not affect partition subscription. In particular, it does not cause a group
* rebalance when automatic assignment is used.
* @param partitions The partitions which should be paused
*/
@Override
public void pause(TopicPartition... partitions) {
acquire();
try {
for (TopicPartition partition: partitions) {
log.debug("Pausing partition {}", partition);
subscriptions.pause(partition);
}
} finally {
release();
}
}
/**
* Resume any partitions which have been paused with {@link #pause(TopicPartition...)}. New calls to
* {@link #poll(long)} will return records from these partitions if there are any to be fetched.
* If the partitions were not previously paused, this method is a no-op.
* @param partitions The partitions which should be resumed
*/
@Override
public void resume(TopicPartition... partitions) {
acquire();
try {
for (TopicPartition partition: partitions) {
log.debug("Resuming partition {}", partition);
subscriptions.resume(partition);
}
} finally {
release();
}
}
@Override
public void close() {
acquire();

View File

@ -12,6 +12,12 @@
*/
package org.apache.kafka.clients.consumer;
import org.apache.kafka.clients.consumer.internals.SubscriptionState;
import org.apache.kafka.common.Metric;
import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
@ -20,12 +26,6 @@ import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import org.apache.kafka.clients.consumer.internals.SubscriptionState;
import org.apache.kafka.common.Metric;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.MetricName;
/**
* A mock of the {@link Consumer} interface you can use for testing code that uses Kafka. This class is <i> not
* threadsafe </i>
@ -83,9 +83,11 @@ public class MockConsumer<K, V> implements Consumer<K, V> {
ensureNotClosed();
// update the consumed offset
for (Map.Entry<TopicPartition, List<ConsumerRecord<K, V>>> entry : this.records.entrySet()) {
List<ConsumerRecord<K, V>> recs = entry.getValue();
if (!recs.isEmpty())
this.subscriptions.consumed(entry.getKey(), recs.get(recs.size() - 1).offset());
if (!subscriptions.isPaused(entry.getKey())) {
List<ConsumerRecord<K, V>> recs = entry.getValue();
if (!recs.isEmpty())
this.subscriptions.consumed(entry.getKey(), recs.get(recs.size() - 1).offset());
}
}
ConsumerRecords<K, V> copy = new ConsumerRecords<K, V>(this.records);
@ -96,7 +98,12 @@ public class MockConsumer<K, V> implements Consumer<K, V> {
public synchronized void addRecord(ConsumerRecord<K, V> record) {
ensureNotClosed();
TopicPartition tp = new TopicPartition(record.topic(), record.partition());
this.subscriptions.assignedPartitions().add(tp);
ArrayList<TopicPartition> currentAssigned = new ArrayList<>(this.subscriptions.assignedPartitions());
if (!currentAssigned.contains(tp)) {
currentAssigned.add(tp);
this.subscriptions.changePartitionAssignment(currentAssigned);
}
subscriptions.seek(tp, record.offset());
List<ConsumerRecord<K, V>> recs = this.records.get(tp);
if (recs == null) {
recs = new ArrayList<ConsumerRecord<K, V>>();
@ -188,6 +195,18 @@ public class MockConsumer<K, V> implements Consumer<K, V> {
this.partitions.put(topic, partitions);
}
@Override
public void pause(TopicPartition... partitions) {
for (TopicPartition partition : partitions)
subscriptions.pause(partition);
}
@Override
public void resume(TopicPartition... partitions) {
for (TopicPartition partition : partitions)
subscriptions.resume(partition);
}
@Override
public synchronized void close() {
ensureNotClosed();

View File

@ -84,6 +84,7 @@ public final class Coordinator {
public Coordinator(ConsumerNetworkClient client,
String groupId,
int sessionTimeoutMs,
int heartbeatIntervalMs,
String assignmentStrategy,
SubscriptionState subscriptions,
Metrics metrics,
@ -103,7 +104,7 @@ public final class Coordinator {
this.subscriptions = subscriptions;
this.sessionTimeoutMs = sessionTimeoutMs;
this.assignmentStrategy = assignmentStrategy;
this.heartbeat = new Heartbeat(this.sessionTimeoutMs, time.milliseconds());
this.heartbeat = new Heartbeat(this.sessionTimeoutMs, heartbeatIntervalMs, time.milliseconds());
this.heartbeatTask = new HeartbeatTask();
this.sensors = new CoordinatorMetrics(metrics, metricGrpPrefix, metricTags);
this.requestTimeoutMs = requestTimeoutMs;
@ -119,7 +120,9 @@ public final class Coordinator {
Map<TopicPartition, Long> offsets = fetchCommittedOffsets(subscriptions.assignedPartitions());
for (Map.Entry<TopicPartition, Long> entry : offsets.entrySet()) {
TopicPartition tp = entry.getKey();
this.subscriptions.committed(tp, entry.getValue());
// verify assignment is still active
if (subscriptions.isAssigned(tp))
this.subscriptions.committed(tp, entry.getValue());
}
this.subscriptions.commitsRefreshed();
}
@ -459,7 +462,9 @@ public final class Coordinator {
short errorCode = entry.getValue();
if (errorCode == Errors.NONE.code()) {
log.debug("Committed offset {} for partition {}", offset, tp);
subscriptions.committed(tp, offset);
if (subscriptions.isAssigned(tp))
// update the local cache only if the partition is still assigned
subscriptions.committed(tp, offset);
} else if (errorCode == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code()
|| errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) {
coordinatorDead();

View File

@ -61,8 +61,8 @@ import java.util.Set;
* This class manage the fetching process with the brokers.
*/
public class Fetcher<K, V> {
private static final long EARLIEST_OFFSET_TIMESTAMP = -2L;
private static final long LATEST_OFFSET_TIMESTAMP = -1L;
public static final long EARLIEST_OFFSET_TIMESTAMP = -2L;
public static final long LATEST_OFFSET_TIMESTAMP = -1L;
private static final Logger log = LoggerFactory.getLogger(Fetcher.class);
@ -143,8 +143,7 @@ public class Fetcher<K, V> {
public void updateFetchPositions(Set<TopicPartition> partitions) {
// reset the fetch position to the committed position
for (TopicPartition tp : partitions) {
// skip if we already have a fetch position
if (subscriptions.fetched(tp) != null)
if (!subscriptions.isAssigned(tp) || subscriptions.isFetchable(tp))
continue;
// TODO: If there are several offsets to reset, we could submit offset requests in parallel
@ -222,7 +221,10 @@ public class Fetcher<K, V> {
log.debug("Resetting offset for partition {} to {} offset.", partition, strategy.name().toLowerCase());
long offset = listOffset(partition, timestamp);
this.subscriptions.seek(partition, offset);
// we might lose the assignment while fetching the offset, so check it is still active
if (subscriptions.isAssigned(partition))
this.subscriptions.seek(partition, offset);
}
/**
@ -259,11 +261,15 @@ public class Fetcher<K, V> {
if (this.subscriptions.partitionAssignmentNeeded()) {
return Collections.emptyMap();
} else {
Map<TopicPartition, List<ConsumerRecord<K, V>>> drained = new HashMap<TopicPartition, List<ConsumerRecord<K, V>>>();
Map<TopicPartition, List<ConsumerRecord<K, V>>> drained = new HashMap<>();
for (PartitionRecords<K, V> part : this.records) {
if (!subscriptions.isFetchable(part.partition)) {
log.debug("Ignoring fetched records for {} since it is no longer fetchable", part.partition);
continue;
}
Long consumed = subscriptions.consumed(part.partition);
if (this.subscriptions.assignedPartitions().contains(part.partition)
&& consumed != null && part.fetchOffset == consumed) {
if (consumed != null && part.fetchOffset == consumed) {
List<ConsumerRecord<K, V>> records = drained.get(part.partition);
if (records == null) {
records = part.records;
@ -354,8 +360,8 @@ public class Fetcher<K, V> {
*/
private Map<Node, FetchRequest> createFetchRequests(Cluster cluster) {
// create the fetch info
Map<Node, Map<TopicPartition, FetchRequest.PartitionData>> fetchable = new HashMap<Node, Map<TopicPartition, FetchRequest.PartitionData>>();
for (TopicPartition partition : subscriptions.assignedPartitions()) {
Map<Node, Map<TopicPartition, FetchRequest.PartitionData>> fetchable = new HashMap<>();
for (TopicPartition partition : subscriptions.fetchablePartitions()) {
Node node = cluster.leaderFor(partition);
if (node == null) {
metadata.requestUpdate();
@ -363,16 +369,17 @@ public class Fetcher<K, V> {
// if there is a leader and no in-flight requests, issue a new fetch
Map<TopicPartition, FetchRequest.PartitionData> fetch = fetchable.get(node);
if (fetch == null) {
fetch = new HashMap<TopicPartition, FetchRequest.PartitionData>();
fetch = new HashMap<>();
fetchable.put(node, fetch);
}
long offset = this.subscriptions.fetched(partition);
fetch.put(partition, new FetchRequest.PartitionData(offset, this.fetchSize));
}
}
// create the fetches
Map<Node, FetchRequest> requests = new HashMap<Node, FetchRequest>();
Map<Node, FetchRequest> requests = new HashMap<>();
for (Map.Entry<Node, Map<TopicPartition, FetchRequest.PartitionData>> entry : fetchable.entrySet()) {
Node node = entry.getKey();
FetchRequest fetch = new FetchRequest(this.maxWaitMs, this.minBytes, entry.getValue());
@ -399,15 +406,7 @@ public class Fetcher<K, V> {
if (!subscriptions.assignedPartitions().contains(tp)) {
log.debug("Ignoring fetched data for partition {} which is no longer assigned.", tp);
} else if (partition.errorCode == Errors.NONE.code()) {
int bytes = 0;
ByteBuffer buffer = partition.recordSet;
MemoryRecords records = MemoryRecords.readableRecords(buffer);
long fetchOffset = request.fetchData().get(tp).offset;
List<ConsumerRecord<K, V>> parsed = new ArrayList<ConsumerRecord<K, V>>();
for (LogEntry logEntry : records) {
parsed.add(parseRecord(tp, logEntry));
bytes += logEntry.size();
}
// we are interested in this fetch only if the beginning offset matches the
// current consumed position
@ -422,7 +421,15 @@ public class Fetcher<K, V> {
continue;
}
if (parsed.size() > 0) {
int bytes = 0;
ByteBuffer buffer = partition.recordSet;
MemoryRecords records = MemoryRecords.readableRecords(buffer);
List<ConsumerRecord<K, V>> parsed = new ArrayList<ConsumerRecord<K, V>>();
for (LogEntry logEntry : records) {
parsed.add(parseRecord(tp, logEntry));
bytes += logEntry.size();
}
if (!parsed.isEmpty()) {
ConsumerRecord<K, V> record = parsed.get(parsed.size() - 1);
this.subscriptions.fetched(tp, record.offset() + 1);
this.records.add(new PartitionRecords<K, V>(fetchOffset, tp, parsed));

View File

@ -16,20 +16,21 @@ package org.apache.kafka.clients.consumer.internals;
* A helper class for managing the heartbeat to the coordinator
*/
public final class Heartbeat {
/* The number of heartbeats to attempt to complete per session timeout interval.
* so, e.g., with a session timeout of 3 seconds we would attempt a heartbeat
* once per second.
*/
public final static int HEARTBEATS_PER_SESSION_INTERVAL = 3;
private final long timeout;
private final long interval;
private long lastHeartbeatSend;
private long lastHeartbeatReceive;
private long lastSessionReset;
public Heartbeat(long timeout, long now) {
public Heartbeat(long timeout,
long interval,
long now) {
if (interval >= timeout)
throw new IllegalArgumentException("Heartbeat must be set lower than the session timeout");
this.timeout = timeout;
this.interval = interval;
this.lastSessionReset = now;
}
@ -52,11 +53,10 @@ public final class Heartbeat {
public long timeToNextHeartbeat(long now) {
long timeSinceLastHeartbeat = now - Math.max(lastHeartbeatSend, lastSessionReset);
long hbInterval = timeout / HEARTBEATS_PER_SESSION_INTERVAL;
if (timeSinceLastHeartbeat > hbInterval)
if (timeSinceLastHeartbeat > interval)
return 0;
else
return hbInterval - timeSinceLastHeartbeat;
return interval - timeSinceLastHeartbeat;
}
public boolean sessionTimeoutExpired(long now) {
@ -64,7 +64,7 @@ public final class Heartbeat {
}
public long interval() {
return timeout / HEARTBEATS_PER_SESSION_INTERVAL;
return interval;
}
public void resetSessionTimeout(long now) {

View File

@ -23,7 +23,25 @@ import java.util.Map;
import java.util.Set;
/**
* A class for tracking the topics, partitions, and offsets for the consumer
* A class for tracking the topics, partitions, and offsets for the consumer. A partition
* is "assigned" either directly with {@link #subscribe(TopicPartition)} (manual assignment)
* or with {@link #changePartitionAssignment(List)} (automatic assignment).
*
* Once assigned, the partition is not considered "fetchable" until its initial position has
* been set with {@link #seek(TopicPartition, long)}. Fetchable partitions track a fetch
* position which is used to set the offset of the next fetch, and a consumed position
* which is the last offset that has been returned to the user. You can suspend fetching
* from a partition through {@link #pause(TopicPartition)} without affecting the fetched/consumed
* offsets. The partition will remain unfetchable until the {@link #resume(TopicPartition)} is
* used. You can also query the pause state independently with {@link #isPaused(TopicPartition)}.
*
* Note that pause state as well as fetch/consumed positions are not preserved when partition
* assignment is changed either with {@link #unsubscribe(TopicPartition)} or
* {@link #changePartitionAssignment(List)}.
*
* This class also maintains a cache of the latest commit position for each of the assigned
* partitions. This is updated through {@link #committed(TopicPartition, long)} and can be used
* to set the initial fetch position (e.g. {@link Fetcher#resetOffset(TopicPartition)}.
*/
public class SubscriptionState {
@ -34,16 +52,7 @@ public class SubscriptionState {
private final Set<TopicPartition> subscribedPartitions;
/* the list of partitions currently assigned */
private final Set<TopicPartition> assignedPartitions;
/* the offset exposed to the user */
private final Map<TopicPartition, Long> consumed;
/* the current point we have fetched up to */
private final Map<TopicPartition, Long> fetched;
/* the last committed offset for each partition */
private final Map<TopicPartition, Long> committed;
private final Map<TopicPartition, TopicPartitionState> assignedPartitions;
/* do we need to request a partition assignment from the coordinator? */
private boolean needsPartitionAssignment;
@ -51,28 +60,21 @@ public class SubscriptionState {
/* do we need to request the latest committed offsets from the coordinator? */
private boolean needsFetchCommittedOffsets;
/* Partitions that need to be reset before fetching */
private Map<TopicPartition, OffsetResetStrategy> resetPartitions;
/* Default offset reset strategy */
private OffsetResetStrategy offsetResetStrategy;
private final OffsetResetStrategy defaultResetStrategy;
public SubscriptionState(OffsetResetStrategy offsetResetStrategy) {
this.offsetResetStrategy = offsetResetStrategy;
this.subscribedTopics = new HashSet<String>();
this.subscribedPartitions = new HashSet<TopicPartition>();
this.assignedPartitions = new HashSet<TopicPartition>();
this.consumed = new HashMap<TopicPartition, Long>();
this.fetched = new HashMap<TopicPartition, Long>();
this.committed = new HashMap<TopicPartition, Long>();
public SubscriptionState(OffsetResetStrategy defaultResetStrategy) {
this.defaultResetStrategy = defaultResetStrategy;
this.subscribedTopics = new HashSet<>();
this.subscribedPartitions = new HashSet<>();
this.assignedPartitions = new HashMap<>();
this.needsPartitionAssignment = false;
this.needsFetchCommittedOffsets = true; // initialize to true for the consumers to fetch offset upon starting up
this.resetPartitions = new HashMap<TopicPartition, OffsetResetStrategy>();
}
public void subscribe(String topic) {
if (this.subscribedPartitions.size() > 0)
throw new IllegalStateException("Subcription to topics and partitions are mutually exclusive");
if (!this.subscribedPartitions.isEmpty())
throw new IllegalStateException("Subscription to topics and partitions are mutually exclusive");
if (!this.subscribedTopics.contains(topic)) {
this.subscribedTopics.add(topic);
this.needsPartitionAssignment = true;
@ -95,10 +97,10 @@ public class SubscriptionState {
}
public void subscribe(TopicPartition tp) {
if (this.subscribedTopics.size() > 0)
throw new IllegalStateException("Subcription to topics and partitions are mutually exclusive");
if (!this.subscribedTopics.isEmpty())
throw new IllegalStateException("Subscription to topics and partitions are mutually exclusive");
this.subscribedPartitions.add(tp);
this.assignedPartitions.add(tp);
addAssignedPartition(tp);
}
public void unsubscribe(TopicPartition partition) {
@ -110,17 +112,10 @@ public class SubscriptionState {
private void clearPartition(TopicPartition tp) {
this.assignedPartitions.remove(tp);
this.committed.remove(tp);
this.fetched.remove(tp);
this.consumed.remove(tp);
this.resetPartitions.remove(tp);
}
public void clearAssignment() {
this.assignedPartitions.clear();
this.committed.clear();
this.fetched.clear();
this.consumed.clear();
this.needsPartitionAssignment = !subscribedTopics().isEmpty();
}
@ -129,21 +124,26 @@ public class SubscriptionState {
}
public Long fetched(TopicPartition tp) {
return this.fetched.get(tp);
return assignedState(tp).fetched;
}
public void fetched(TopicPartition tp, long offset) {
if (!this.assignedPartitions.contains(tp))
throw new IllegalArgumentException("Can't change the fetch position for a partition you are not currently subscribed to.");
this.fetched.put(tp, offset);
assignedState(tp).fetched(offset);
}
private TopicPartitionState assignedState(TopicPartition tp) {
TopicPartitionState state = this.assignedPartitions.get(tp);
if (state == null)
throw new IllegalStateException("No current assignment for partition " + tp);
return state;
}
public void committed(TopicPartition tp, long offset) {
this.committed.put(tp, offset);
assignedState(tp).committed(offset);
}
public Long committed(TopicPartition tp) {
return this.committed.get(tp);
return assignedState(tp).committed;
}
public void needRefreshCommits() {
@ -157,15 +157,22 @@ public class SubscriptionState {
public void commitsRefreshed() {
this.needsFetchCommittedOffsets = false;
}
public void seek(TopicPartition tp, long offset) {
fetched(tp, offset);
consumed(tp, offset);
resetPartitions.remove(tp);
assignedState(tp).seek(offset);
}
public Set<TopicPartition> assignedPartitions() {
return this.assignedPartitions;
return this.assignedPartitions.keySet();
}
public Set<TopicPartition> fetchablePartitions() {
Set<TopicPartition> fetchable = new HashSet<>();
for (Map.Entry<TopicPartition, TopicPartitionState> entry : assignedPartitions.entrySet()) {
if (entry.getValue().isFetchable())
fetchable.add(entry.getKey());
}
return fetchable;
}
public boolean partitionsAutoAssigned() {
@ -173,49 +180,52 @@ public class SubscriptionState {
}
public void consumed(TopicPartition tp, long offset) {
if (!this.assignedPartitions.contains(tp))
throw new IllegalArgumentException("Can't change the consumed position for a partition you are not currently subscribed to.");
this.consumed.put(tp, offset);
assignedState(tp).consumed(offset);
}
public Long consumed(TopicPartition partition) {
return this.consumed.get(partition);
public Long consumed(TopicPartition tp) {
return assignedState(tp).consumed;
}
public Map<TopicPartition, Long> allConsumed() {
return this.consumed;
Map<TopicPartition, Long> allConsumed = new HashMap<>();
for (Map.Entry<TopicPartition, TopicPartitionState> entry : assignedPartitions.entrySet()) {
TopicPartitionState state = entry.getValue();
if (state.hasValidPosition)
allConsumed.put(entry.getKey(), state.consumed);
}
return allConsumed;
}
public void needOffsetReset(TopicPartition partition, OffsetResetStrategy offsetResetStrategy) {
this.resetPartitions.put(partition, offsetResetStrategy);
this.fetched.remove(partition);
this.consumed.remove(partition);
assignedState(partition).awaitReset(offsetResetStrategy);
}
public void needOffsetReset(TopicPartition partition) {
needOffsetReset(partition, offsetResetStrategy);
needOffsetReset(partition, defaultResetStrategy);
}
public boolean isOffsetResetNeeded(TopicPartition partition) {
return resetPartitions.containsKey(partition);
}
public boolean isOffsetResetNeeded() {
return !resetPartitions.isEmpty();
return assignedState(partition).awaitingReset;
}
public OffsetResetStrategy resetStrategy(TopicPartition partition) {
return resetPartitions.get(partition);
return assignedState(partition).resetStrategy;
}
public boolean hasAllFetchPositions() {
return this.fetched.size() >= this.assignedPartitions.size();
for (TopicPartitionState state : assignedPartitions.values())
if (!state.hasValidPosition)
return false;
return true;
}
public Set<TopicPartition> missingFetchPositions() {
Set<TopicPartition> copy = new HashSet<TopicPartition>(this.assignedPartitions);
copy.removeAll(this.fetched.keySet());
return copy;
Set<TopicPartition> missing = new HashSet<>(this.assignedPartitions.keySet());
for (Map.Entry<TopicPartition, TopicPartitionState> entry : assignedPartitions.entrySet())
if (!entry.getValue().hasValidPosition)
missing.add(entry.getKey());
return missing;
}
public boolean partitionAssignmentNeeded() {
@ -227,9 +237,99 @@ public class SubscriptionState {
if (!this.subscribedTopics.contains(tp.topic()))
throw new IllegalArgumentException("Assigned partition " + tp + " for non-subscribed topic.");
this.clearAssignment();
this.assignedPartitions.addAll(assignments);
for (TopicPartition tp: assignments)
addAssignedPartition(tp);
this.needsPartitionAssignment = false;
}
public boolean isAssigned(TopicPartition tp) {
return assignedPartitions.containsKey(tp);
}
public boolean isPaused(TopicPartition tp) {
return isAssigned(tp) && assignedState(tp).paused;
}
public boolean isFetchable(TopicPartition tp) {
return isAssigned(tp) && assignedState(tp).isFetchable();
}
public void pause(TopicPartition tp) {
assignedState(tp).pause();
}
public void resume(TopicPartition tp) {
assignedState(tp).resume();
}
private void addAssignedPartition(TopicPartition tp) {
this.assignedPartitions.put(tp, new TopicPartitionState());
}
private static class TopicPartitionState {
private Long consumed; // offset exposed to the user
private Long fetched; // current fetch position
private Long committed; // last committed position
private boolean hasValidPosition; // whether we have valid consumed and fetched positions
private boolean paused; // whether this partition has been paused by the user
private boolean awaitingReset; // whether we are awaiting reset
private OffsetResetStrategy resetStrategy; // the reset strategy if awaitingReset is set
public TopicPartitionState() {
this.paused = false;
this.consumed = null;
this.fetched = null;
this.committed = null;
this.awaitingReset = false;
this.hasValidPosition = false;
this.resetStrategy = null;
}
private void awaitReset(OffsetResetStrategy strategy) {
this.awaitingReset = true;
this.resetStrategy = strategy;
this.consumed = null;
this.fetched = null;
this.hasValidPosition = false;
}
private void seek(long offset) {
this.consumed = offset;
this.fetched = offset;
this.awaitingReset = false;
this.resetStrategy = null;
this.hasValidPosition = true;
}
private void fetched(long offset) {
if (!hasValidPosition)
throw new IllegalStateException("Cannot update fetch position without valid consumed/fetched positions");
this.fetched = offset;
}
private void consumed(long offset) {
if (!hasValidPosition)
throw new IllegalStateException("Cannot update consumed position without valid consumed/fetched positions");
this.consumed = offset;
}
private void committed(Long offset) {
this.committed = offset;
}
private void pause() {
this.paused = true;
}
private void resume() {
this.paused = false;
}
private boolean isFetchable() {
return !paused && hasValidPosition;
}
}
}

View File

@ -0,0 +1,48 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
* file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
* to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
* License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*/
package org.apache.kafka.common.annotation;
import java.lang.annotation.Documented;
import java.lang.annotation.Retention;
import java.lang.annotation.RetentionPolicy;
/**
* Annotation to inform users of how much to rely on a particular package,
* class or method not changing over time. Currently the stability can be
* {@link Stable}, {@link Evolving} or {@link Unstable}. <br>
*/
@InterfaceStability.Evolving
public class InterfaceStability {
/**
* Can evolve while retaining compatibility for minor release boundaries.;
* can break compatibility only at major release (ie. at m.0).
*/
@Documented
@Retention(RetentionPolicy.RUNTIME)
public @interface Stable { }
/**
* Evolving, but can break compatibility at minor release (i.e. m.x)
*/
@Documented
@Retention(RetentionPolicy.RUNTIME)
public @interface Evolving { }
/**
* No guarantee is provided as to reliability or stability across any
* level of release granularity.
*/
@Documented
@Retention(RetentionPolicy.RUNTIME)
public @interface Unstable { }
}

View File

@ -39,7 +39,7 @@ public class AbstractConfig {
private final Map<String, Object> values;
@SuppressWarnings("unchecked")
public AbstractConfig(ConfigDef definition, Map<?, ?> originals) {
public AbstractConfig(ConfigDef definition, Map<?, ?> originals, Boolean doLog) {
/* check that all the keys are really strings */
for (Object key : originals.keySet())
if (!(key instanceof String))
@ -47,7 +47,12 @@ public class AbstractConfig {
this.originals = (Map<String, ?>) originals;
this.values = definition.parse(this.originals);
this.used = Collections.synchronizedSet(new HashSet<String>());
logAll();
if (doLog)
logAll();
}
public AbstractConfig(ConfigDef definition, Map<?, ?> originals) {
this(definition, originals, true);
}
protected Object get(String key) {
@ -122,7 +127,7 @@ public class AbstractConfig {
*/
public void logUnused() {
for (String key : unused())
log.warn("The configuration {} = {} was supplied but isn't a known config.", key, this.values.get(key));
log.warn("The configuration {} = {} was supplied but isn't a known config.", key, this.originals.get(key));
}
/**
@ -167,4 +172,18 @@ public class AbstractConfig {
return objects;
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
AbstractConfig that = (AbstractConfig) o;
return originals.equals(that.originals);
}
@Override
public int hashCode() {
return originals.hashCode();
}
}

View File

@ -45,7 +45,7 @@ import org.slf4j.LoggerFactory;
* A connection can be added to the nioSelector associated with an integer id by doing
*
* <pre>
* nioSelector.connect(42, new InetSocketAddress(&quot;google.com&quot;, server.port), 64000, 64000);
* nioSelector.connect(&quot;42&quot;, new InetSocketAddress(&quot;google.com&quot;, server.port), 64000, 64000);
* </pre>
*
* The connect call does not block on the creation of the TCP connection, so the connect method only begins initiating
@ -55,8 +55,9 @@ import org.slf4j.LoggerFactory;
* connections are all done using the <code>poll()</code> call.
*
* <pre>
* List&lt;NetworkRequest&gt; requestsToSend = Arrays.asList(new NetworkRequest(0, myBytes), new NetworkRequest(1, myOtherBytes));
* nioSelector.poll(TIMEOUT_MS, requestsToSend);
* nioSelector.send(new NetworkSend(myDestination, myBytes));
* nioSelector.send(new NetworkSend(myOtherDestination, myOtherBytes));
* nioSelector.poll(TIMEOUT_MS);
* </pre>
*
* The nioSelector maintains several lists that are reset by each call to <code>poll()</code> which are available via
@ -123,7 +124,7 @@ public class Selector implements Selectable {
* Begin connecting to the given address and add the connection to this nioSelector associated with the given id
* number.
* <p>
* Note that this call only initiates the connection, which will be completed on a future {@link #poll(long, List)}
* Note that this call only initiates the connection, which will be completed on a future {@link #poll(long)}
* call. Check {@link #connected()} to see which (if any) connections have completed after a given poll call.
* @param id The id for the new connection
* @param address The address to connect to
@ -171,7 +172,7 @@ public class Selector implements Selectable {
/**
* Disconnect any connections for the given id (if there are any). The disconnection is asynchronous and will not be
* processed until the next {@link #poll(long, List) poll()} call.
* processed until the next {@link #poll(long) poll()} call.
*/
@Override
public void disconnect(String id) {
@ -228,8 +229,8 @@ public class Selector implements Selectable {
*
* When this call is completed the user can check for completed sends, receives, connections or disconnects using
* {@link #completedSends()}, {@link #completedReceives()}, {@link #connected()}, {@link #disconnected()}. These
* lists will be cleared at the beginning of each {@link #poll(long, List)} call and repopulated by the call if any
* completed I/O.
* lists will be cleared at the beginning of each {@link #poll(long)} call and repopulated by the call if there is
* any completed I/O.
*
* @param timeout The amount of time to wait, in milliseconds. If negative, wait indefinitely.
* @throws IllegalStateException If a send is given for which we have no existing connection or for which there is

View File

@ -76,14 +76,14 @@ public class ProduceResponse extends AbstractRequestResponse {
public ProduceResponse(Struct struct) {
super(struct);
responses = new HashMap<TopicPartition, PartitionResponse>();
for (Object topicResponse : struct.getArray("responses")) {
for (Object topicResponse : struct.getArray(RESPONSES_KEY_NAME)) {
Struct topicRespStruct = (Struct) topicResponse;
String topic = topicRespStruct.getString("topic");
for (Object partResponse : topicRespStruct.getArray("partition_responses")) {
String topic = topicRespStruct.getString(TOPIC_KEY_NAME);
for (Object partResponse : topicRespStruct.getArray(PARTITION_RESPONSES_KEY_NAME)) {
Struct partRespStruct = (Struct) partResponse;
int partition = partRespStruct.getInt("partition");
short errorCode = partRespStruct.getShort("error_code");
long offset = partRespStruct.getLong("base_offset");
int partition = partRespStruct.getInt(PARTITION_KEY_NAME);
short errorCode = partRespStruct.getShort(ERROR_CODE_KEY_NAME);
long offset = partRespStruct.getLong(BASE_OFFSET_KEY_NAME);
TopicPartition tp = new TopicPartition(topic, partition);
responses.put(tp, new PartitionResponse(errorCode, offset));
}

View File

@ -34,15 +34,24 @@ import org.apache.kafka.common.utils.Time;
* A mock network client for use testing code
*/
public class MockClient implements KafkaClient {
public static final RequestMatcher ALWAYS_TRUE = new RequestMatcher() {
@Override
public boolean matches(ClientRequest request) {
return true;
}
};
private class FutureResponse {
public final Struct responseBody;
public final boolean disconnected;
public final RequestMatcher requestMatcher;
public FutureResponse(Struct responseBody, boolean disconnected) {
public FutureResponse(Struct responseBody, boolean disconnected, RequestMatcher requestMatcher) {
this.responseBody = responseBody;
this.disconnected = disconnected;
this.requestMatcher = requestMatcher;
}
}
private final Time time;
@ -94,6 +103,9 @@ public class MockClient implements KafkaClient {
public void send(ClientRequest request) {
if (!futureResponses.isEmpty()) {
FutureResponse futureResp = futureResponses.poll();
if (!futureResp.requestMatcher.matches(request))
throw new IllegalStateException("Next in line response did not match expected request");
ClientResponse resp = new ClientResponse(request, time.milliseconds(), futureResp.disconnected, futureResp.responseBody);
responses.add(resp);
} else {
@ -141,11 +153,32 @@ public class MockClient implements KafkaClient {
}
public void prepareResponse(Struct body) {
prepareResponse(body, false);
prepareResponse(ALWAYS_TRUE, body, false);
}
/**
* Prepare a response for a request matching the provided matcher. If the matcher does not
* match, {@link #send(ClientRequest)} will throw IllegalStateException
* @param matcher The matcher to apply
* @param body The response body
*/
public void prepareResponse(RequestMatcher matcher, Struct body) {
prepareResponse(matcher, body, false);
}
public void prepareResponse(Struct body, boolean disconnected) {
futureResponses.add(new FutureResponse(body, disconnected));
prepareResponse(ALWAYS_TRUE, body, disconnected);
}
/**
* Prepare a response for a request matching the provided matcher. If the matcher does not
* match, {@link #send(ClientRequest)} will throw IllegalStateException
* @param matcher The matcher to apply
* @param body The response body
* @param disconnected Whether the request was disconnected
*/
public void prepareResponse(RequestMatcher matcher, Struct body, boolean disconnected) {
futureResponses.add(new FutureResponse(body, disconnected, matcher));
}
public void setNode(Node node) {
@ -180,4 +213,14 @@ public class MockClient implements KafkaClient {
return this.node;
}
/**
* The RequestMatcher provides a way to match a particular request to a response prepared
* through {@link #prepareResponse(RequestMatcher, Struct)}. Basically this allows testers
* to inspect the request body for the type of the request or for specific fields that should be set,
* and to fail the test if it doesn't match.
*/
public interface RequestMatcher {
boolean matches(ClientRequest request);
}
}

View File

@ -29,7 +29,7 @@ public class MockConsumerTest {
@Test
public void testSimpleMock() {
consumer.subscribe("topic");
consumer.subscribe("test");
assertEquals(0, consumer.poll(1000).count());
ConsumerRecord<String, String> rec1 = new ConsumerRecord<String, String>("test", 0, 0, "key1", "value1");
ConsumerRecord<String, String> rec2 = new ConsumerRecord<String, String>("test", 0, 1, "key2", "value2");

View File

@ -59,6 +59,7 @@ public class CoordinatorTest {
private String groupId = "test-group";
private TopicPartition tp = new TopicPartition(topicName, 0);
private int sessionTimeoutMs = 10;
private int heartbeatIntervalMs = 2;
private long retryBackoffMs = 100;
private long requestTimeoutMs = 5000;
private String rebalanceStrategy = "not-matter";
@ -89,6 +90,7 @@ public class CoordinatorTest {
this.coordinator = new Coordinator(consumerClient,
groupId,
sessionTimeoutMs,
heartbeatIntervalMs,
rebalanceStrategy,
subscriptions,
metrics,

View File

@ -16,6 +16,7 @@
*/
package org.apache.kafka.clients.consumer.internals;
import org.apache.kafka.clients.ClientRequest;
import org.apache.kafka.clients.Metadata;
import org.apache.kafka.clients.MockClient;
import org.apache.kafka.clients.consumer.ConsumerRecord;
@ -30,6 +31,8 @@ import org.apache.kafka.common.protocol.types.Struct;
import org.apache.kafka.common.record.CompressionType;
import org.apache.kafka.common.record.MemoryRecords;
import org.apache.kafka.common.requests.FetchResponse;
import org.apache.kafka.common.requests.ListOffsetRequest;
import org.apache.kafka.common.requests.ListOffsetResponse;
import org.apache.kafka.common.requests.MetadataResponse;
import org.apache.kafka.common.serialization.ByteArrayDeserializer;
import org.apache.kafka.common.utils.MockTime;
@ -40,11 +43,14 @@ import org.junit.Test;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
public class FetcherTest {
@ -99,8 +105,7 @@ public class FetcherTest {
public void testFetchNormal() {
List<ConsumerRecord<byte[], byte[]>> records;
subscriptions.subscribe(tp);
subscriptions.fetched(tp, 0);
subscriptions.consumed(tp, 0);
subscriptions.seek(tp, 0);
// normal fetch
fetcher.initFetches(cluster);
@ -121,8 +126,7 @@ public class FetcherTest {
public void testFetchDuringRebalance() {
subscriptions.subscribe(topicName);
subscriptions.changePartitionAssignment(Arrays.asList(tp));
subscriptions.fetched(tp, 0);
subscriptions.consumed(tp, 0);
subscriptions.seek(tp, 0);
fetcher.initFetches(cluster);
@ -136,27 +140,57 @@ public class FetcherTest {
}
@Test
public void testFetchFailed() {
public void testInFlightFetchOnPausedPartition() {
subscriptions.subscribe(tp);
subscriptions.fetched(tp, 0);
subscriptions.consumed(tp, 0);
subscriptions.seek(tp, 0);
fetcher.initFetches(cluster);
subscriptions.pause(tp);
client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L));
consumerClient.poll(0);
assertNull(fetcher.fetchedRecords().get(tp));
}
@Test
public void testFetchOnPausedPartition() {
subscriptions.subscribe(tp);
subscriptions.seek(tp, 0);
subscriptions.pause(tp);
fetcher.initFetches(cluster);
assertTrue(client.requests().isEmpty());
}
@Test
public void testFetchNotLeaderForPartition() {
subscriptions.subscribe(tp);
subscriptions.seek(tp, 0);
// fetch with not leader
fetcher.initFetches(cluster);
client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NOT_LEADER_FOR_PARTITION.code(), 100L));
consumerClient.poll(0);
assertEquals(0, fetcher.fetchedRecords().size());
assertEquals(0L, metadata.timeToNextUpdate(time.milliseconds()));
}
@Test
public void testFetchUnknownTopicOrPartition() {
subscriptions.subscribe(tp);
subscriptions.seek(tp, 0);
// fetch with unknown topic partition
fetcher.initFetches(cluster);
client.prepareResponse(fetchResponse(this.records.buffer(), Errors.UNKNOWN_TOPIC_OR_PARTITION.code(), 100L));
consumerClient.poll(0);
assertEquals(0, fetcher.fetchedRecords().size());
assertEquals(0L, metadata.timeToNextUpdate(time.milliseconds()));
}
@Test
public void testFetchOffsetOutOfRange() {
subscriptions.subscribe(tp);
subscriptions.seek(tp, 0);
// fetch with out of range
subscriptions.fetched(tp, 5);
fetcher.initFetches(cluster);
client.prepareResponse(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L));
consumerClient.poll(0);
@ -167,19 +201,94 @@ public class FetcherTest {
}
@Test
public void testFetchOutOfRange() {
public void testFetchDisconnected() {
subscriptions.subscribe(tp);
subscriptions.fetched(tp, 5);
subscriptions.consumed(tp, 5);
subscriptions.seek(tp, 0);
// fetch with out of range
fetcher.initFetches(cluster);
client.prepareResponse(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L));
client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L), true);
consumerClient.poll(0);
assertTrue(subscriptions.isOffsetResetNeeded(tp));
assertEquals(0, fetcher.fetchedRecords().size());
assertEquals(null, subscriptions.fetched(tp));
assertEquals(null, subscriptions.consumed(tp));
// disconnects should have no affect on subscription state
assertFalse(subscriptions.isOffsetResetNeeded(tp));
assertTrue(subscriptions.isFetchable(tp));
assertEquals(0, (long) subscriptions.fetched(tp));
assertEquals(0, (long) subscriptions.consumed(tp));
}
@Test
public void testUpdateFetchPositionToCommitted() {
// unless a specific reset is expected, the default behavior is to reset to the committed
// position if one is present
subscriptions.subscribe(tp);
subscriptions.committed(tp, 5);
fetcher.updateFetchPositions(Collections.singleton(tp));
assertTrue(subscriptions.isFetchable(tp));
assertEquals(5, (long) subscriptions.fetched(tp));
assertEquals(5, (long) subscriptions.consumed(tp));
}
@Test
public void testUpdateFetchPositionResetToDefaultOffset() {
subscriptions.subscribe(tp);
// with no commit position, we should reset using the default strategy defined above (EARLIEST)
client.prepareResponse(listOffsetRequestMatcher(Fetcher.EARLIEST_OFFSET_TIMESTAMP),
listOffsetResponse(Errors.NONE, Arrays.asList(5L)));
fetcher.updateFetchPositions(Collections.singleton(tp));
assertFalse(subscriptions.isOffsetResetNeeded(tp));
assertTrue(subscriptions.isFetchable(tp));
assertEquals(5, (long) subscriptions.fetched(tp));
assertEquals(5, (long) subscriptions.consumed(tp));
}
@Test
public void testUpdateFetchPositionResetToLatestOffset() {
subscriptions.subscribe(tp);
subscriptions.needOffsetReset(tp, OffsetResetStrategy.LATEST);
client.prepareResponse(listOffsetRequestMatcher(Fetcher.LATEST_OFFSET_TIMESTAMP),
listOffsetResponse(Errors.NONE, Arrays.asList(5L)));
fetcher.updateFetchPositions(Collections.singleton(tp));
assertFalse(subscriptions.isOffsetResetNeeded(tp));
assertTrue(subscriptions.isFetchable(tp));
assertEquals(5, (long) subscriptions.fetched(tp));
assertEquals(5, (long) subscriptions.consumed(tp));
}
@Test
public void testUpdateFetchPositionResetToEarliestOffset() {
subscriptions.subscribe(tp);
subscriptions.needOffsetReset(tp, OffsetResetStrategy.EARLIEST);
client.prepareResponse(listOffsetRequestMatcher(Fetcher.EARLIEST_OFFSET_TIMESTAMP),
listOffsetResponse(Errors.NONE, Arrays.asList(5L)));
fetcher.updateFetchPositions(Collections.singleton(tp));
assertFalse(subscriptions.isOffsetResetNeeded(tp));
assertTrue(subscriptions.isFetchable(tp));
assertEquals(5, (long) subscriptions.fetched(tp));
assertEquals(5, (long) subscriptions.consumed(tp));
}
@Test
public void testUpdateFetchPositionDisconnect() {
subscriptions.subscribe(tp);
subscriptions.needOffsetReset(tp, OffsetResetStrategy.LATEST);
// First request gets a disconnect
client.prepareResponse(listOffsetRequestMatcher(Fetcher.LATEST_OFFSET_TIMESTAMP),
listOffsetResponse(Errors.NONE, Arrays.asList(5L)), true);
// Next one succeeds
client.prepareResponse(listOffsetRequestMatcher(Fetcher.LATEST_OFFSET_TIMESTAMP),
listOffsetResponse(Errors.NONE, Arrays.asList(5L)));
fetcher.updateFetchPositions(Collections.singleton(tp));
assertFalse(subscriptions.isOffsetResetNeeded(tp));
assertTrue(subscriptions.isFetchable(tp));
assertEquals(5, (long) subscriptions.fetched(tp));
assertEquals(5, (long) subscriptions.consumed(tp));
}
@Test
@ -193,6 +302,26 @@ public class FetcherTest {
assertEquals(cluster.topics().size(), allTopics.size());
}
private MockClient.RequestMatcher listOffsetRequestMatcher(final long timestamp) {
// matches any list offset request with the provided timestamp
return new MockClient.RequestMatcher() {
@Override
public boolean matches(ClientRequest request) {
ListOffsetRequest req = new ListOffsetRequest(request.request().body());
ListOffsetRequest.PartitionData partitionData = req.offsetData().get(tp);
return partitionData != null && partitionData.timestamp == timestamp;
}
};
}
private Struct listOffsetResponse(Errors error, List<Long> offsets) {
ListOffsetResponse.PartitionData partitionData = new ListOffsetResponse.PartitionData(error.code(), offsets);
Map<TopicPartition, ListOffsetResponse.PartitionData> allPartitionData = new HashMap<>();
allPartitionData.put(tp, partitionData);
ListOffsetResponse response = new ListOffsetResponse(allPartitionData);
return response.toStruct();
}
private Struct fetchResponse(ByteBuffer buffer, short error, long hw) {
FetchResponse response = new FetchResponse(Collections.singletonMap(tp, new FetchResponse.PartitionData(error, hw, buffer)));
return response.toStruct();

View File

@ -27,20 +27,21 @@ import static org.junit.Assert.assertTrue;
public class HeartbeatTest {
private long timeout = 300L;
private long interval = 100L;
private MockTime time = new MockTime();
private Heartbeat heartbeat = new Heartbeat(timeout, -1L);
private Heartbeat heartbeat = new Heartbeat(timeout, interval, -1L);
@Test
public void testShouldHeartbeat() {
heartbeat.sentHeartbeat(time.milliseconds());
time.sleep((long) ((float) timeout / Heartbeat.HEARTBEATS_PER_SESSION_INTERVAL * 1.1));
time.sleep((long) ((float) interval * 1.1));
assertTrue(heartbeat.shouldHeartbeat(time.milliseconds()));
}
@Test
public void testShouldNotHeartbeat() {
heartbeat.sentHeartbeat(time.milliseconds());
time.sleep(timeout / (2 * Heartbeat.HEARTBEATS_PER_SESSION_INTERVAL));
time.sleep(interval / 2);
assertFalse(heartbeat.shouldHeartbeat(time.milliseconds()));
}

View File

@ -17,6 +17,7 @@
package org.apache.kafka.clients.consumer.internals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static java.util.Arrays.asList;
@ -37,12 +38,13 @@ public class SubscriptionStateTest {
state.subscribe(tp0);
assertEquals(Collections.singleton(tp0), state.assignedPartitions());
state.committed(tp0, 1);
state.fetched(tp0, 1);
state.consumed(tp0, 1);
state.seek(tp0, 1);
assertTrue(state.isFetchable(tp0));
assertAllPositions(tp0, 1L);
state.unsubscribe(tp0);
assertTrue(state.assignedPartitions().isEmpty());
assertAllPositions(tp0, null);
assertFalse(state.isAssigned(tp0));
assertFalse(state.isFetchable(tp0));
}
@Test
@ -52,10 +54,15 @@ public class SubscriptionStateTest {
assertEquals(5L, (long) state.fetched(tp0));
assertEquals(5L, (long) state.consumed(tp0));
state.needOffsetReset(tp0);
assertTrue(state.isOffsetResetNeeded());
assertFalse(state.isFetchable(tp0));
assertTrue(state.isOffsetResetNeeded(tp0));
assertEquals(null, state.fetched(tp0));
assertEquals(null, state.consumed(tp0));
// seek should clear the reset and make the partition fetchable
state.seek(tp0, 0);
assertTrue(state.isFetchable(tp0));
assertFalse(state.isOffsetResetNeeded(tp0));
}
@Test
@ -65,15 +72,27 @@ public class SubscriptionStateTest {
assertTrue(state.assignedPartitions().isEmpty());
assertTrue(state.partitionsAutoAssigned());
state.changePartitionAssignment(asList(tp0));
state.seek(tp0, 1);
state.committed(tp0, 1);
state.fetched(tp0, 1);
state.consumed(tp0, 1);
assertAllPositions(tp0, 1L);
state.changePartitionAssignment(asList(tp1));
assertAllPositions(tp0, null);
assertTrue(state.isAssigned(tp1));
assertFalse(state.isAssigned(tp0));
assertFalse(state.isFetchable(tp1));
assertEquals(Collections.singleton(tp1), state.assignedPartitions());
}
@Test
public void partitionPause() {
state.subscribe(tp0);
state.seek(tp0, 100);
assertTrue(state.isFetchable(tp0));
state.pause(tp0);
assertFalse(state.isFetchable(tp0));
state.resume(tp0);
assertTrue(state.isFetchable(tp0));
}
@Test
public void topicUnsubscription() {
final String topic = "test";
@ -83,24 +102,37 @@ public class SubscriptionStateTest {
assertTrue(state.partitionsAutoAssigned());
state.changePartitionAssignment(asList(tp0));
state.committed(tp0, 1);
state.fetched(tp0, 1);
state.consumed(tp0, 1);
state.seek(tp0, 1);
assertAllPositions(tp0, 1L);
state.changePartitionAssignment(asList(tp1));
assertAllPositions(tp0, null);
assertFalse(state.isAssigned(tp0));
assertEquals(Collections.singleton(tp1), state.assignedPartitions());
state.unsubscribe(topic);
assertEquals(0, state.subscribedTopics().size());
assertTrue(state.assignedPartitions().isEmpty());
}
@Test(expected = IllegalArgumentException.class)
@Test(expected = IllegalStateException.class)
public void invalidConsumedPositionUpdate() {
state.subscribe("test");
state.changePartitionAssignment(asList(tp0));
state.consumed(tp0, 0);
}
@Test(expected = IllegalStateException.class)
public void invalidFetchPositionUpdate() {
state.subscribe("test");
state.changePartitionAssignment(asList(tp0));
state.fetched(tp0, 0);
}
@Test(expected = IllegalStateException.class)
public void cantChangeFetchPositionForNonAssignedPartition() {
state.fetched(tp0, 1);
}
@Test(expected = IllegalArgumentException.class)
@Test(expected = IllegalStateException.class)
public void cantChangeConsumedPositionForNonAssignedPartition() {
state.consumed(tp0, 1);
}

View File

@ -21,6 +21,7 @@ import kafka.common._
import kafka.cluster.{BrokerEndPoint, Broker}
import kafka.log.LogConfig
import kafka.server.ConfigType
import kafka.utils._
import kafka.api.{TopicMetadata, PartitionMetadata}
@ -40,10 +41,8 @@ import org.I0Itec.zkclient.exception.ZkNodeExistsException
object AdminUtils extends Logging {
val rand = new Random
val AdminClientId = "__admin_client"
val TopicConfigChangeZnodePrefix = "config_change_"
val EntityConfigChangeZnodePrefix = "config_change_"
/**
* There are 2 goals of replica assignment:
@ -103,14 +102,12 @@ object AdminUtils extends Logging {
* @param numPartitions Number of partitions to be set
* @param replicaAssignmentStr Manual replica assignment
* @param checkBrokerAvailable Ignore checking if assigned replica broker is available. Only used for testing
* @param config Pre-existing properties that should be preserved
*/
def addPartitions(zkClient: ZkClient,
topic: String,
numPartitions: Int = 1,
replicaAssignmentStr: String = "",
checkBrokerAvailable: Boolean = true,
config: Properties = new Properties) {
checkBrokerAvailable: Boolean = true) {
val existingPartitionsReplicaList = ZkUtils.getReplicaAssignmentForTopics(zkClient, List(topic))
if (existingPartitionsReplicaList.size == 0)
throw new AdminOperationException("The topic %s does not exist".format(topic))
@ -137,7 +134,7 @@ object AdminUtils extends Logging {
val partitionReplicaList = existingPartitionsReplicaList.map(p => p._1.partition -> p._2)
// add the new list
partitionReplicaList ++= newPartitionReplicaList
AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, partitionReplicaList, config, true)
AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, partitionReplicaList, update = true)
}
def getManualReplicaAssignment(replicaAssignmentList: String, availableBrokerList: Set[Int], startPartitionId: Int, checkBrokerAvailable: Boolean = true): Map[Int, List[Int]] = {
@ -238,7 +235,7 @@ object AdminUtils extends Logging {
val replicaAssignment = AdminUtils.assignReplicasToBrokers(brokerList, partitions, replicationFactor)
AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, replicaAssignment, topicConfig)
}
def createOrUpdateTopicPartitionAssignmentPathInZK(zkClient: ZkClient,
topic: String,
partitionReplicaAssignment: Map[Int, Seq[Int]],
@ -246,7 +243,6 @@ object AdminUtils extends Logging {
update: Boolean = false) {
// validate arguments
Topic.validate(topic)
LogConfig.validate(config)
require(partitionReplicaAssignment.values.map(_.size).toSet.size == 1, "All partitions should have the same number of replicas.")
val topicPath = ZkUtils.getTopicPath(topic)
@ -264,10 +260,14 @@ object AdminUtils extends Logging {
}
partitionReplicaAssignment.values.foreach(reps => require(reps.size == reps.toSet.size, "Duplicate replica assignment found: " + partitionReplicaAssignment))
// write out the config if there is any, this isn't transactional with the partition assignments
writeTopicConfig(zkClient, topic, config)
// Configs only matter if a topic is being created. Changing configs via AlterTopic is not supported
if (!update) {
// write out the config if there is any, this isn't transactional with the partition assignments
LogConfig.validate(config)
writeEntityConfig(zkClient, ConfigType.Topic, topic, config)
}
// create the partition assignment
writeTopicPartitionAssignment(zkClient, topic, partitionReplicaAssignment, update)
}
@ -290,7 +290,19 @@ object AdminUtils extends Logging {
case e2: Throwable => throw new AdminOperationException(e2.toString)
}
}
/**
* Update the config for a client and create a change notification so the change will propagate to other brokers
* @param zkClient: The ZkClient handle used to write the new config to zookeeper
* @param clientId: The clientId for which configs are being changed
* @param configs: The final set of configs that will be applied to the topic. If any new configs need to be added or
* existing configs need to be deleted, it should be done prior to invoking this API
*
*/
def changeClientIdConfig(zkClient: ZkClient, clientId: String, configs: Properties) {
changeEntityConfig(zkClient, ConfigType.Client, clientId, configs)
}
/**
* Update the config for an existing topic and create a change notification so the change will propagate to other brokers
* @param zkClient: The ZkClient handle used to write the new config to zookeeper
@ -302,34 +314,42 @@ object AdminUtils extends Logging {
def changeTopicConfig(zkClient: ZkClient, topic: String, configs: Properties) {
if(!topicExists(zkClient, topic))
throw new AdminOperationException("Topic \"%s\" does not exist.".format(topic))
// remove the topic overrides
LogConfig.validate(configs)
// write the new config--may not exist if there were previously no overrides
writeTopicConfig(zkClient, topic, configs)
// create the change notification
zkClient.createPersistentSequential(ZkUtils.TopicConfigChangesPath + "/" + TopicConfigChangeZnodePrefix, Json.encode(topic))
changeEntityConfig(zkClient, ConfigType.Topic, topic, configs)
}
private def changeEntityConfig(zkClient: ZkClient, entityType: String, entityName: String, configs: Properties) {
// write the new config--may not exist if there were previously no overrides
writeEntityConfig(zkClient, entityType, entityName, configs)
// create the change notification
val seqNode = ZkUtils.EntityConfigChangesPath + "/" + EntityConfigChangeZnodePrefix
val content = Json.encode(getConfigChangeZnodeData(entityType, entityName))
zkClient.createPersistentSequential(seqNode, content)
}
def getConfigChangeZnodeData(entityType: String, entityName: String) : Map[String, Any] = {
Map("version" -> 1, "entity_type" -> entityType, "entity_name" -> entityName)
}
/**
* Write out the topic config to zk, if there is any
*/
private def writeTopicConfig(zkClient: ZkClient, topic: String, config: Properties) {
private def writeEntityConfig(zkClient: ZkClient, entityType: String, entityName: String, config: Properties) {
val configMap: mutable.Map[String, String] = {
import JavaConversions._
config
}
val map = Map("version" -> 1, "config" -> configMap)
ZkUtils.updatePersistentPath(zkClient, ZkUtils.getTopicConfigPath(topic), Json.encode(map))
ZkUtils.updatePersistentPath(zkClient, ZkUtils.getEntityConfigPath(entityType, entityName), Json.encode(map))
}
/**
* Read the topic config (if any) from zk
* Read the entity (topic or client) config (if any) from zk
*/
def fetchTopicConfig(zkClient: ZkClient, topic: String): Properties = {
val str: String = zkClient.readData(ZkUtils.getTopicConfigPath(topic), true)
def fetchEntityConfig(zkClient: ZkClient, entityType: String, entity: String): Properties = {
val str: String = zkClient.readData(ZkUtils.getEntityConfigPath(entityType, entity), true)
val props = new Properties()
if(str != null) {
Json.parseFull(str) match {
@ -343,19 +363,20 @@ object AdminUtils extends Logging {
configTup match {
case (k: String, v: String) =>
props.setProperty(k, v)
case _ => throw new IllegalArgumentException("Invalid topic config: " + str)
case _ => throw new IllegalArgumentException("Invalid " + entityType + " config: " + str)
}
case _ => throw new IllegalArgumentException("Invalid topic config: " + str)
case _ => throw new IllegalArgumentException("Invalid " + entityType + " config: " + str)
}
case o => throw new IllegalArgumentException("Unexpected value in config: " + str)
case o => throw new IllegalArgumentException("Unexpected value in config:(%s), entity_type: (%s), entity: (%s)"
.format(str, entityType, entity))
}
}
props
}
def fetchAllTopicConfigs(zkClient: ZkClient): Map[String, Properties] =
ZkUtils.getAllTopics(zkClient).map(topic => (topic, fetchTopicConfig(zkClient, topic))).toMap
ZkUtils.getAllTopics(zkClient).map(topic => (topic, fetchEntityConfig(zkClient, ConfigType.Topic, topic))).toMap
def fetchTopicMetadataFromZk(topic: String, zkClient: ZkClient): TopicMetadata =
fetchTopicMetadataFromZk(topic, zkClient, new mutable.HashMap[Int, Broker])

View File

@ -0,0 +1,174 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.admin
import joptsimple._
import java.util.Properties
import kafka.log.LogConfig
import kafka.server.ConfigType
import kafka.utils.{ZkUtils, CommandLineUtils}
import org.I0Itec.zkclient.ZkClient
import scala.collection._
import scala.collection.JavaConversions._
import org.apache.kafka.common.utils.Utils
/**
* This script can be used to change configs for topics/clients dynamically
*/
object ConfigCommand {
def main(args: Array[String]): Unit = {
val opts = new ConfigCommandOptions(args)
if(args.length == 0)
CommandLineUtils.printUsageAndDie(opts.parser, "Add/Remove entity (topic/client) configs")
opts.checkArgs()
val zkClient = ZkUtils.createZkClient(opts.options.valueOf(opts.zkConnectOpt), 30000, 30000)
try {
if (opts.options.has(opts.alterOpt))
alterConfig(zkClient, opts)
else if (opts.options.has(opts.describeOpt))
describeConfig(zkClient, opts)
} catch {
case e: Throwable =>
println("Error while executing topic command " + e.getMessage)
println(Utils.stackTrace(e))
} finally {
zkClient.close()
}
}
private def alterConfig(zkClient: ZkClient, opts: ConfigCommandOptions) {
val configsToBeAdded = parseConfigsToBeAdded(opts)
val configsToBeDeleted = parseConfigsToBeDeleted(opts)
val entityType = opts.options.valueOf(opts.entityType)
val entityName = opts.options.valueOf(opts.entityName)
// compile the final set of configs
val configs = AdminUtils.fetchEntityConfig(zkClient, entityType, entityName)
configs.putAll(configsToBeAdded)
configsToBeDeleted.foreach(config => configs.remove(config))
if (entityType.equals(ConfigType.Topic)) {
AdminUtils.changeTopicConfig(zkClient, entityName, configs)
println("Updated config for topic: \"%s\".".format(entityName))
} else {
AdminUtils.changeClientIdConfig(zkClient, entityName, configs)
println("Updated config for clientId: \"%s\".".format(entityName))
}
}
private def describeConfig(zkClient: ZkClient, opts: ConfigCommandOptions) {
val entityType = opts.options.valueOf(opts.entityType)
val entityNames: Seq[String] =
if (opts.options.has(opts.entityName))
Seq(opts.options.valueOf(opts.entityName))
else
ZkUtils.getAllEntitiesWithConfig(zkClient, entityType)
for (entityName <- entityNames) {
val configs = AdminUtils.fetchEntityConfig(zkClient, entityType, entityName)
println("Configs for %s:%s are %s"
.format(entityType, entityName, configs.map(kv => kv._1 + "=" + kv._2).mkString(",")))
}
}
private[admin] def parseConfigsToBeAdded(opts: ConfigCommandOptions): Properties = {
val configsToBeAdded = opts.options.valuesOf(opts.addedConfig).map(_.split("""\s*=\s*"""))
require(configsToBeAdded.forall(config => config.length == 2),
"Invalid entity config: all configs to be added must be in the format \"key=val\".")
val props = new Properties
configsToBeAdded.foreach(pair => props.setProperty(pair(0).trim, pair(1).trim))
props
}
private[admin] def parseConfigsToBeDeleted(opts: ConfigCommandOptions): Seq[String] = {
if (opts.options.has(opts.deletedConfig)) {
val configsToBeDeleted = opts.options.valuesOf(opts.deletedConfig).map(_.trim())
val propsToBeDeleted = new Properties
configsToBeDeleted.foreach(propsToBeDeleted.setProperty(_, ""))
configsToBeDeleted
}
else
Seq.empty
}
class ConfigCommandOptions(args: Array[String]) {
val parser = new OptionParser
val zkConnectOpt = parser.accepts("zookeeper", "REQUIRED: The connection string for the zookeeper connection in the form host:port. " +
"Multiple URLS can be given to allow fail-over.")
.withRequiredArg
.describedAs("urls")
.ofType(classOf[String])
val alterOpt = parser.accepts("alter", "Alter the configuration for the entity.")
val describeOpt = parser.accepts("describe", "List configs for the given entity.")
val entityType = parser.accepts("entity-type", "Type of entity (topic/client)")
.withRequiredArg
.ofType(classOf[String])
val entityName = parser.accepts("entity-name", "Name of entity (topic name/client id)")
.withRequiredArg
.ofType(classOf[String])
val nl = System.getProperty("line.separator")
val addedConfig = parser.accepts("added-config", "Key Value pairs configs to add 'k1=v1,k2=v2'. The following is a list of valid configurations: " +
"For entity_type '" + ConfigType.Topic + "': " + nl + LogConfig.configNames.map("\t" + _).mkString(nl) + nl +
"For entity_type '" + ConfigType.Client + "' currently no configs are processed by the brokers")
.withRequiredArg
.ofType(classOf[String])
.withValuesSeparatedBy(',')
val deletedConfig = parser.accepts("deleted-config", "config keys to remove 'k1,k2'")
.withRequiredArg
.ofType(classOf[String])
.withValuesSeparatedBy(',')
val helpOpt = parser.accepts("help", "Print usage information.")
val options = parser.parse(args : _*)
val allOpts: Set[OptionSpec[_]] = Set(alterOpt, describeOpt, entityType, entityName, addedConfig, deletedConfig, helpOpt)
def checkArgs() {
// should have exactly one action
val actions = Seq(alterOpt, describeOpt).count(options.has _)
if(actions != 1)
CommandLineUtils.printUsageAndDie(parser, "Command must include exactly one action: --describe, --alter")
// check required args
CommandLineUtils.checkRequiredArgs(parser, options, zkConnectOpt, entityType)
CommandLineUtils.checkInvalidArgs(parser, options, alterOpt, Set(describeOpt))
CommandLineUtils.checkInvalidArgs(parser, options, describeOpt, Set(alterOpt, addedConfig, deletedConfig))
if(options.has(alterOpt)) {
if(! options.has(entityName))
throw new IllegalArgumentException("--entity-name must be specified with --alter")
val isAddedPresent: Boolean = options.has(addedConfig)
val isDeletedPresent: Boolean = options.has(deletedConfig)
if(! isAddedPresent && ! isDeletedPresent)
throw new IllegalArgumentException("At least one of --added-config or --deleted-config must be specified with --alter")
}
val entityTypeVal = options.valueOf(entityType)
if(! entityTypeVal.equals(ConfigType.Topic) && ! entityTypeVal.equals(ConfigType.Client)) {
throw new IllegalArgumentException("--entity-type must be '%s' or '%s'".format(ConfigType.Topic, ConfigType.Client))
}
}
}
}

View File

@ -20,6 +20,7 @@ package kafka.admin
import joptsimple._
import java.util.Properties
import kafka.common.{Topic, AdminCommandFailedException}
import kafka.utils.CommandLineUtils
import kafka.utils._
import org.I0Itec.zkclient.ZkClient
import org.I0Itec.zkclient.exception.ZkNodeExistsException
@ -27,6 +28,7 @@ import scala.collection._
import scala.collection.JavaConversions._
import kafka.log.LogConfig
import kafka.consumer.Whitelist
import kafka.server.{ConfigType, OffsetManager}
import org.apache.kafka.common.utils.Utils
import kafka.coordinator.ConsumerCoordinator
@ -106,16 +108,6 @@ object TopicCommand extends Logging {
opts.options.valueOf(opts.zkConnectOpt)))
}
topics.foreach { topic =>
val configs = AdminUtils.fetchTopicConfig(zkClient, topic)
if(opts.options.has(opts.configOpt) || opts.options.has(opts.deleteConfigOpt)) {
val configsToBeAdded = parseTopicConfigsToBeAdded(opts)
val configsToBeDeleted = parseTopicConfigsToBeDeleted(opts)
// compile the final set of configs
configs.putAll(configsToBeAdded)
configsToBeDeleted.foreach(config => configs.remove(config))
AdminUtils.changeTopicConfig(zkClient, topic, configs)
println("Updated config for topic \"%s\".".format(topic))
}
if(opts.options.has(opts.partitionsOpt)) {
if (topic == ConsumerCoordinator.OffsetsTopicName) {
throw new IllegalArgumentException("The number of partitions for the offsets topic cannot be changed.")
@ -124,7 +116,7 @@ object TopicCommand extends Logging {
"logic or ordering of the messages will be affected")
val nPartitions = opts.options.valueOf(opts.partitionsOpt).intValue
val replicaAssignmentStr = opts.options.valueOf(opts.replicaAssignmentOpt)
AdminUtils.addPartitions(zkClient, topic, nPartitions, replicaAssignmentStr, config = configs)
AdminUtils.addPartitions(zkClient, topic, nPartitions, replicaAssignmentStr)
println("Adding partitions succeeded!")
}
}
@ -180,7 +172,7 @@ object TopicCommand extends Logging {
val describePartitions: Boolean = !reportOverriddenConfigs
val sortedPartitions = topicPartitionAssignment.toList.sortWith((m1, m2) => m1._1 < m2._1)
if (describeConfigs) {
val configs = AdminUtils.fetchTopicConfig(zkClient, topic)
val configs = AdminUtils.fetchEntityConfig(zkClient, ConfigType.Topic, topic)
if (!reportOverriddenConfigs || configs.size() != 0) {
val numPartitions = topicPartitionAssignment.size
val replicationFactor = topicPartitionAssignment.head._2.size
@ -219,18 +211,6 @@ object TopicCommand extends Logging {
props
}
def parseTopicConfigsToBeDeleted(opts: TopicCommandOptions): Seq[String] = {
if (opts.options.has(opts.deleteConfigOpt)) {
val configsToBeDeleted = opts.options.valuesOf(opts.deleteConfigOpt).map(_.trim())
val propsToBeDeleted = new Properties
configsToBeDeleted.foreach(propsToBeDeleted.setProperty(_, ""))
LogConfig.validateNames(propsToBeDeleted)
configsToBeDeleted
}
else
Seq.empty
}
def parseReplicaAssignment(replicaAssignmentList: String): Map[Int, List[Int]] = {
val partitionList = replicaAssignmentList.split(",")
val ret = new mutable.HashMap[Int, List[Int]]()
@ -256,7 +236,7 @@ object TopicCommand extends Logging {
val listOpt = parser.accepts("list", "List all available topics.")
val createOpt = parser.accepts("create", "Create a new topic.")
val deleteOpt = parser.accepts("delete", "Delete a topic")
val alterOpt = parser.accepts("alter", "Alter the configuration for the topic.")
val alterOpt = parser.accepts("alter", "Alter the number of partitions and/or replica assignment for a topic")
val describeOpt = parser.accepts("describe", "List details for the given topics.")
val helpOpt = parser.accepts("help", "Print usage information.")
val topicOpt = parser.accepts("topic", "The topic to be create, alter or describe. Can also accept a regular " +
@ -265,16 +245,12 @@ object TopicCommand extends Logging {
.describedAs("topic")
.ofType(classOf[String])
val nl = System.getProperty("line.separator")
val configOpt = parser.accepts("config", "A topic configuration override for the topic being created or altered." +
val configOpt = parser.accepts("config", "A configuration override for the topic being created." +
"The following is a list of valid configurations: " + nl + LogConfig.configNames.map("\t" + _).mkString(nl) + nl +
"See the Kafka documentation for full details on the topic configs.")
.withRequiredArg
.describedAs("name=value")
.ofType(classOf[String])
val deleteConfigOpt = parser.accepts("delete-config", "A topic configuration override to be removed for an existing topic (see the list of configurations under the --config option).")
.withRequiredArg
.describedAs("name")
.ofType(classOf[String])
val partitionsOpt = parser.accepts("partitions", "The number of partitions for the topic being created or " +
"altered (WARNING: If partitions are increased for a topic that has a key, the partition logic or ordering of the messages will be affected")
.withRequiredArg
@ -308,10 +284,11 @@ object TopicCommand extends Logging {
// check invalid args
CommandLineUtils.checkInvalidArgs(parser, options, configOpt, allTopicLevelOpts -- Set(alterOpt, createOpt))
CommandLineUtils.checkInvalidArgs(parser, options, deleteConfigOpt, allTopicLevelOpts -- Set(alterOpt))
CommandLineUtils.checkInvalidArgs(parser, options, partitionsOpt, allTopicLevelOpts -- Set(alterOpt, createOpt))
CommandLineUtils.checkInvalidArgs(parser, options, replicationFactorOpt, allTopicLevelOpts -- Set(createOpt))
CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, allTopicLevelOpts -- Set(createOpt,alterOpt))
// Topic configs cannot be changed with alterTopic
CommandLineUtils.checkInvalidArgs(parser, options, alterOpt, Set(configOpt))
if(options.has(createOpt))
CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, Set(partitionsOpt, replicationFactorOpt))
CommandLineUtils.checkInvalidArgs(parser, options, reportUnderReplicatedPartitionsOpt,

View File

@ -22,7 +22,7 @@ import kafka.utils.CoreUtils.{inReadLock,inWriteLock}
import kafka.admin.AdminUtils
import kafka.api.{PartitionStateInfo, LeaderAndIsr}
import kafka.log.LogConfig
import kafka.server.{TopicPartitionOperationKey, LogOffsetMetadata, LogReadResult, ReplicaManager}
import kafka.server._
import kafka.metrics.KafkaMetricsGroup
import kafka.controller.KafkaController
import kafka.message.ByteBufferMessageSet
@ -86,7 +86,8 @@ class Partition(val topic: String,
case Some(replica) => replica
case None =>
if (isReplicaLocal(replicaId)) {
val config = LogConfig.fromProps(logManager.defaultConfig.originals, AdminUtils.fetchTopicConfig(zkClient, topic))
val config = LogConfig.fromProps(logManager.defaultConfig.originals,
AdminUtils.fetchEntityConfig(zkClient, ConfigType.Topic, topic))
val log = logManager.createLog(TopicAndPartition(topic, partitionId), config)
val checkpoint = replicaManager.highWatermarkCheckpoints(log.dir.getParentFile.getAbsolutePath)
val offsetMap = checkpoint.read

View File

@ -30,7 +30,7 @@ import kafka.api.RequestOrResponse
import collection.Set
class ControllerChannelManager (private val controllerContext: ControllerContext, config: KafkaConfig) extends Logging {
private val brokerStateInfo = new HashMap[Int, ControllerBrokerStateInfo]
protected val brokerStateInfo = new HashMap[Int, ControllerBrokerStateInfo]
private val brokerLock = new Object
this.logIdent = "[Channel manager on controller " + config.brokerId + "]: "
@ -100,7 +100,7 @@ class ControllerChannelManager (private val controllerContext: ControllerContext
}
}
private def startRequestSendThread(brokerId: Int) {
protected def startRequestSendThread(brokerId: Int) {
val requestThread = brokerStateInfo(brokerId).requestSendThread
if(requestThread.getState == Thread.State.NEW)
requestThread.start()
@ -280,49 +280,67 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging
}
def sendRequestsToBrokers(controllerEpoch: Int, correlationId: Int) {
leaderAndIsrRequestMap.foreach { m =>
val broker = m._1
val partitionStateInfos = m._2.toMap
val leaderIds = partitionStateInfos.map(_._2.leaderIsrAndControllerEpoch.leaderAndIsr.leader).toSet
val leaders = controllerContext.liveOrShuttingDownBrokers.filter(b => leaderIds.contains(b.id)).map(b => b.getBrokerEndPoint(controller.config.interBrokerSecurityProtocol))
val leaderAndIsrRequest = new LeaderAndIsrRequest(partitionStateInfos, leaders, controllerId, controllerEpoch, correlationId, clientId)
for (p <- partitionStateInfos) {
val typeOfRequest = if (broker == p._2.leaderIsrAndControllerEpoch.leaderAndIsr.leader) "become-leader" else "become-follower"
stateChangeLogger.trace(("Controller %d epoch %d sending %s LeaderAndIsr request %s with correlationId %d to broker %d " +
"for partition [%s,%d]").format(controllerId, controllerEpoch, typeOfRequest,
p._2.leaderIsrAndControllerEpoch, correlationId, broker,
p._1._1, p._1._2))
try {
leaderAndIsrRequestMap.foreach { m =>
val broker = m._1
val partitionStateInfos = m._2.toMap
val leaderIds = partitionStateInfos.map(_._2.leaderIsrAndControllerEpoch.leaderAndIsr.leader).toSet
val leaders = controllerContext.liveOrShuttingDownBrokers.filter(b => leaderIds.contains(b.id)).map(b => b.getBrokerEndPoint(controller.config.interBrokerSecurityProtocol))
val leaderAndIsrRequest = new LeaderAndIsrRequest(partitionStateInfos, leaders, controllerId, controllerEpoch, correlationId, clientId)
for (p <- partitionStateInfos) {
val typeOfRequest = if (broker == p._2.leaderIsrAndControllerEpoch.leaderAndIsr.leader) "become-leader" else "become-follower"
stateChangeLogger.trace(("Controller %d epoch %d sending %s LeaderAndIsr request %s with correlationId %d to broker %d " +
"for partition [%s,%d]").format(controllerId, controllerEpoch, typeOfRequest,
p._2.leaderIsrAndControllerEpoch, correlationId, broker,
p._1._1, p._1._2))
}
controller.sendRequest(broker, leaderAndIsrRequest, null)
}
controller.sendRequest(broker, leaderAndIsrRequest, null)
}
leaderAndIsrRequestMap.clear()
updateMetadataRequestMap.foreach { m =>
val broker = m._1
val partitionStateInfos = m._2.toMap
leaderAndIsrRequestMap.clear()
updateMetadataRequestMap.foreach { m =>
val broker = m._1
val partitionStateInfos = m._2.toMap
val versionId = if (controller.config.interBrokerProtocolVersion.onOrAfter(KAFKA_083)) 1 else 0
val updateMetadataRequest = new UpdateMetadataRequest(versionId = versionId.toShort, controllerId = controllerId, controllerEpoch = controllerEpoch,
correlationId = correlationId, clientId = clientId, partitionStateInfos = partitionStateInfos, aliveBrokers = controllerContext.liveOrShuttingDownBrokers)
partitionStateInfos.foreach(p => stateChangeLogger.trace(("Controller %d epoch %d sending UpdateMetadata request %s with " +
"correlationId %d to broker %d for partition %s").format(controllerId, controllerEpoch, p._2.leaderIsrAndControllerEpoch,
correlationId, broker, p._1)))
controller.sendRequest(broker, updateMetadataRequest, null)
}
updateMetadataRequestMap.clear()
stopReplicaRequestMap foreach { case(broker, replicaInfoList) =>
val stopReplicaWithDelete = replicaInfoList.filter(_.deletePartition).map(_.replica).toSet
val stopReplicaWithoutDelete = replicaInfoList.filterNot(_.deletePartition).map(_.replica).toSet
debug("The stop replica request (delete = true) sent to broker %d is %s"
.format(broker, stopReplicaWithDelete.mkString(",")))
debug("The stop replica request (delete = false) sent to broker %d is %s"
.format(broker, stopReplicaWithoutDelete.mkString(",")))
replicaInfoList.foreach { r =>
val stopReplicaRequest = new StopReplicaRequest(r.deletePartition,
Set(TopicAndPartition(r.replica.topic, r.replica.partition)), controllerId, controllerEpoch, correlationId)
controller.sendRequest(broker, stopReplicaRequest, r.callback)
val versionId = if (controller.config.interBrokerProtocolVersion.onOrAfter(KAFKA_083)) 1 else 0
val updateMetadataRequest = new UpdateMetadataRequest(versionId = versionId.toShort, controllerId = controllerId, controllerEpoch = controllerEpoch,
correlationId = correlationId, clientId = clientId, partitionStateInfos = partitionStateInfos, aliveBrokers = controllerContext.liveOrShuttingDownBrokers)
partitionStateInfos.foreach(p => stateChangeLogger.trace(("Controller %d epoch %d sending UpdateMetadata request %s with " +
"correlationId %d to broker %d for partition %s").format(controllerId, controllerEpoch, p._2.leaderIsrAndControllerEpoch,
correlationId, broker, p._1)))
controller.sendRequest(broker, updateMetadataRequest, null)
}
updateMetadataRequestMap.clear()
stopReplicaRequestMap foreach { case(broker, replicaInfoList) =>
val stopReplicaWithDelete = replicaInfoList.filter(_.deletePartition).map(_.replica).toSet
val stopReplicaWithoutDelete = replicaInfoList.filterNot(_.deletePartition).map(_.replica).toSet
debug("The stop replica request (delete = true) sent to broker %d is %s"
.format(broker, stopReplicaWithDelete.mkString(",")))
debug("The stop replica request (delete = false) sent to broker %d is %s"
.format(broker, stopReplicaWithoutDelete.mkString(",")))
replicaInfoList.foreach { r =>
val stopReplicaRequest = new StopReplicaRequest(r.deletePartition,
Set(TopicAndPartition(r.replica.topic, r.replica.partition)), controllerId, controllerEpoch, correlationId)
controller.sendRequest(broker, stopReplicaRequest, r.callback)
}
}
stopReplicaRequestMap.clear()
} catch {
case e : Throwable => {
if(leaderAndIsrRequestMap.size > 0) {
error("Haven't been able to send leader and isr requests, current state of " +
s"the map is $leaderAndIsrRequestMap")
}
if(updateMetadataRequestMap.size > 0) {
error("Haven't been able to send metadata update requests, current state of " +
s"the map is $updateMetadataRequestMap")
}
if(stopReplicaRequestMap.size > 0) {
error("Haven't been able to send stop replica requests, current state of " +
s"the map is $stopReplicaRequestMap")
}
throw new IllegalStateException(e)
}
}
stopReplicaRequestMap.clear()
}
}

View File

@ -263,11 +263,20 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt
} else {
// Stop the replica first. The state change below initiates ZK changes which should take some time
// before which the stop replica request should be completed (in most cases)
brokerRequestBatch.newBatch()
brokerRequestBatch.addStopReplicaRequestForBrokers(Seq(id), topicAndPartition.topic,
topicAndPartition.partition, deletePartition = false)
brokerRequestBatch.sendRequestsToBrokers(epoch, controllerContext.correlationId.getAndIncrement)
try {
brokerRequestBatch.newBatch()
brokerRequestBatch.addStopReplicaRequestForBrokers(Seq(id), topicAndPartition.topic,
topicAndPartition.partition, deletePartition = false)
brokerRequestBatch.sendRequestsToBrokers(epoch, controllerContext.correlationId.getAndIncrement)
} catch {
case e : IllegalStateException => {
// Resign if the controller is in an illegal state
error("Forcing the controller to resign")
controllerElector.resign()
throw e
}
}
// If the broker is a follower, updates the isr in ZK and notifies the current leader
replicaStateMachine.handleStateChanges(Set(PartitionAndReplica(topicAndPartition.topic,
topicAndPartition.partition, id)), OfflineReplica)
@ -341,6 +350,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt
* required to clean up internal controller data structures
*/
def onControllerResignation() {
debug("Controller resigning, broker id %d".format(config.brokerId))
// de-register listeners
deregisterIsrChangeNotificationListener()
deregisterReassignedPartitionsListener()
@ -888,9 +898,19 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt
brokerRequestBatch.newBatch()
updateLeaderEpoch(topicAndPartition.topic, topicAndPartition.partition) match {
case Some(updatedLeaderIsrAndControllerEpoch) =>
brokerRequestBatch.addLeaderAndIsrRequestForBrokers(replicasToReceiveRequest, topicAndPartition.topic,
topicAndPartition.partition, updatedLeaderIsrAndControllerEpoch, newAssignedReplicas)
brokerRequestBatch.sendRequestsToBrokers(controllerContext.epoch, controllerContext.correlationId.getAndIncrement)
try {
brokerRequestBatch.addLeaderAndIsrRequestForBrokers(replicasToReceiveRequest, topicAndPartition.topic,
topicAndPartition.partition, updatedLeaderIsrAndControllerEpoch, newAssignedReplicas)
brokerRequestBatch.sendRequestsToBrokers(controllerContext.epoch, controllerContext.correlationId.getAndIncrement)
} catch {
case e : IllegalStateException => {
// Resign if the controller is in an illegal state
error("Forcing the controller to resign")
controllerElector.resign()
throw e
}
}
stateChangeLogger.trace(("Controller %d epoch %d sent LeaderAndIsr request %s with new assigned replica list %s " +
"to leader %d for partition being reassigned %s").format(config.brokerId, controllerContext.epoch, updatedLeaderIsrAndControllerEpoch,
newAssignedReplicas.mkString(","), updatedLeaderIsrAndControllerEpoch.leaderAndIsr.leader, topicAndPartition))
@ -998,9 +1018,19 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt
* @param brokers The brokers that the update metadata request should be sent to
*/
def sendUpdateMetadataRequest(brokers: Seq[Int], partitions: Set[TopicAndPartition] = Set.empty[TopicAndPartition]) {
brokerRequestBatch.newBatch()
brokerRequestBatch.addUpdateMetadataRequestForBrokers(brokers, partitions)
brokerRequestBatch.sendRequestsToBrokers(epoch, controllerContext.correlationId.getAndIncrement)
try {
brokerRequestBatch.newBatch()
brokerRequestBatch.addUpdateMetadataRequestForBrokers(brokers, partitions)
brokerRequestBatch.sendRequestsToBrokers(epoch, controllerContext.correlationId.getAndIncrement)
} catch {
case e : IllegalStateException => {
// Resign if the controller is in an illegal state
error("Forcing the controller to resign")
controllerElector.resign()
throw e
}
}
}
/**
@ -1037,8 +1067,8 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt
// if the replica to be removed from the ISR is the last surviving member of the ISR and unclean leader election
// is disallowed for the corresponding topic, then we must preserve the ISR membership so that the replica can
// eventually be restored as the leader.
if (newIsr.isEmpty && !LogConfig.fromProps(config.originals, AdminUtils.fetchTopicConfig(zkClient,
topicAndPartition.topic)).uncleanLeaderElectionEnable) {
if (newIsr.isEmpty && !LogConfig.fromProps(config.originals, AdminUtils.fetchEntityConfig(zkClient,
ConfigType.Topic, topicAndPartition.topic)).uncleanLeaderElectionEnable) {
info("Retaining last ISR %d of partition %s since unclean leader election is disabled".format(replicaId, topicAndPartition))
newIsr = leaderAndIsr.isr
}
@ -1138,7 +1168,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt
}
override def handleSessionEstablishmentError(error: Throwable): Unit = {
//no-op handleSessionEstablishmentError in KafkaHealthCheck should System.exit and log the error.
//no-op handleSessionEstablishmentError in KafkaHealthCheck should handle this error in its handleSessionEstablishmentError
}
}
@ -1322,7 +1352,8 @@ class IsrChangeNotificationListener(controller: KafkaController) extends IZkChil
processUpdateNotifications(topicAndPartitions)
// delete processed children
childrenAsScala.map(x => ZkUtils.deletePath(controller.controllerContext.zkClient, ZkUtils.TopicConfigChangesPath + "/" + x))
childrenAsScala.map(x => ZkUtils.deletePath(controller.controllerContext.zkClient,
ZkUtils.getEntityConfigPath(ConfigType.Topic, x)))
}
}

View File

@ -21,7 +21,7 @@ import kafka.api.LeaderAndIsr
import kafka.log.LogConfig
import kafka.utils.Logging
import kafka.common.{LeaderElectionNotNeededException, TopicAndPartition, StateChangeFailedException, NoReplicaOnlineException}
import kafka.server.KafkaConfig
import kafka.server.{ConfigType, KafkaConfig}
trait PartitionLeaderSelector {
@ -61,8 +61,8 @@ class OfflinePartitionLeaderSelector(controllerContext: ControllerContext, confi
case true =>
// Prior to electing an unclean (i.e. non-ISR) leader, ensure that doing so is not disallowed by the configuration
// for unclean leader election.
if (!LogConfig.fromProps(config.originals, AdminUtils.fetchTopicConfig(controllerContext.zkClient,
topicAndPartition.topic)).uncleanLeaderElectionEnable) {
if (!LogConfig.fromProps(config.originals, AdminUtils.fetchEntityConfig(controllerContext.zkClient,
ConfigType.Topic, topicAndPartition.topic)).uncleanLeaderElectionEnable) {
throw new NoReplicaOnlineException(("No broker in ISR for partition " +
"%s is alive. Live brokers are: [%s],".format(topicAndPartition, controllerContext.liveBrokerIds)) +
" ISR brokers are: [%s]".format(currentLeaderAndIsr.isr.mkString(",")))

View File

@ -16,6 +16,9 @@
*/
package kafka.controller
import kafka.server.ConfigType
import collection.mutable
import kafka.utils.{ShutdownableThread, Logging, ZkUtils}
import kafka.utils.CoreUtils._
@ -284,7 +287,7 @@ class TopicDeletionManager(controller: KafkaController,
topicsToBeDeleted -= topic
partitionsToBeDeleted.retain(_.topic != topic)
controllerContext.zkClient.deleteRecursive(ZkUtils.getTopicPath(topic))
controllerContext.zkClient.deleteRecursive(ZkUtils.getTopicConfigPath(topic))
controllerContext.zkClient.deleteRecursive(ZkUtils.getEntityConfigPath(ConfigType.Topic, topic))
controllerContext.zkClient.delete(ZkUtils.getDeleteTopicPath(topic))
controllerContext.removeTopic(topic)
}

View File

@ -326,7 +326,7 @@ class ConsumerCoordinator(val brokerId: Int,
private def updateConsumer(group: ConsumerGroupMetadata, consumer: ConsumerMetadata, topics: Set[String]) {
val topicsToBind = topics -- group.topics
group.remove(consumer.consumerId)
val topicsToUnbind = consumer.topics -- group.topics
val topicsToUnbind = consumer.topics -- (group.topics ++ topics)
group.add(consumer.consumerId, consumer)
consumer.topics = topics
coordinatorMetadata.bindAndUnbindGroupFromTopics(group.groupId, topicsToBind, topicsToUnbind)

View File

@ -46,7 +46,7 @@ object Defaults {
val PreAllocateEnable = kafka.server.Defaults.LogPreAllocateEnable
}
case class LogConfig(props: java.util.Map[_, _]) extends AbstractConfig(LogConfig.configDef, props) {
case class LogConfig(props: java.util.Map[_, _]) extends AbstractConfig(LogConfig.configDef, props, false) {
val segmentSize = getInt(LogConfig.SegmentBytesProp)
val segmentMs = getLong(LogConfig.SegmentMsProp)

View File

@ -5,7 +5,7 @@
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
@ -36,16 +36,16 @@ class DefaultDecoder(props: VerifiableProperties = null) extends Decoder[Array[B
}
/**
* The string encoder translates strings into bytes. It uses UTF8 by default but takes
* The string decoder translates bytes into strings. It uses UTF8 by default but takes
* an optional property serializer.encoding to control this.
*/
class StringDecoder(props: VerifiableProperties = null) extends Decoder[String] {
val encoding =
val encoding =
if(props == null)
"UTF8"
"UTF8"
else
props.getString("serializer.encoding", "UTF8")
def fromBytes(bytes: Array[Byte]): String = {
new String(bytes, encoding)
}

View File

@ -0,0 +1,69 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.server
import java.util.Properties
import kafka.common.TopicAndPartition
import kafka.log.{Log, LogConfig, LogManager}
import kafka.utils.Pool
import scala.collection.mutable
/**
* The ConfigHandler is used to process config change notifications received by the DynamicConfigManager
*/
trait ConfigHandler {
def processConfigChanges(entityName : String, value : Properties)
}
/**
* The TopicConfigHandler will process topic config changes in ZK.
* The callback provides the topic name and the full properties set read from ZK
*/
class TopicConfigHandler(private val logManager: LogManager) extends ConfigHandler{
def processConfigChanges(topic : String, topicConfig : Properties) {
val logs: mutable.Buffer[(TopicAndPartition, Log)] = logManager.logsByTopicPartition.toBuffer
val logsByTopic: Map[String, mutable.Buffer[Log]] = logs.groupBy{ case (topicAndPartition, log) => topicAndPartition.topic }
.mapValues{ case v: mutable.Buffer[(TopicAndPartition, Log)] => v.map(_._2) }
if (logsByTopic.contains(topic)) {
/* combine the default properties with the overrides in zk to create the new LogConfig */
val props = new Properties()
props.putAll(logManager.defaultConfig.originals)
props.putAll(topicConfig)
val logConfig = LogConfig(props)
for (log <- logsByTopic(topic))
log.config = logConfig
}
}
}
/**
* The ClientIdConfigHandler will process clientId config changes in ZK.
* The callback provides the clientId and the full properties set read from ZK.
* This implementation does nothing currently. In the future, it will change quotas per client
*/
class ClientIdConfigHandler extends ConfigHandler {
val configPool = new Pool[String, Properties]()
def processConfigChanges(clientId : String, clientConfig : Properties): Unit = {
configPool.put(clientId, clientConfig)
}
}

View File

@ -17,67 +17,79 @@
package kafka.server
import java.util.Properties
import kafka.utils.Json
import kafka.utils.Logging
import kafka.utils.SystemTime
import kafka.utils.Time
import kafka.utils.ZkUtils
import scala.collection._
import kafka.log._
import kafka.utils._
import kafka.admin.AdminUtils
import org.I0Itec.zkclient.{IZkChildListener, ZkClient}
/**
* This class initiates and carries out topic config changes.
*
* Represents all the entities that can be configured via ZK
*/
object ConfigType {
val Topic = "topic"
val Client = "client"
}
/**
* This class initiates and carries out config changes for all entities defined in ConfigType.
*
* It works as follows.
*
* Config is stored under the path
* /config/topics/<topic_name>
* This znode stores the topic-overrides for this topic (but no defaults) in properties format.
*
*
* Config is stored under the path: /config/entityType/entityName
* E.g. /config/topics/<topic_name> and /config/clients/<clientId>
* This znode stores the overrides for this entity (but no defaults) in properties format.
*
* To avoid watching all topics for changes instead we have a notification path
* /config/changes
* The TopicConfigManager has a child watch on this path.
*
* To update a topic config we first update the topic config properties. Then we create a new sequential
* znode under the change path which contains the name of the topic that was updated, say
* The DynamicConfigManager has a child watch on this path.
*
* To update a config we first update the config properties. Then we create a new sequential
* znode under the change path which contains the name of the entityType and entityName that was updated, say
* /config/changes/config_change_13321
* This is just a notification--the actual config change is stored only once under the /config/topics/<topic_name> path.
*
* The sequential znode contains data in this format: {"version" : 1, "entityType":"topic/client", "entityName" : "topic_name/client_id"}
* This is just a notification--the actual config change is stored only once under the /config/entityType/entityName path.
*
* This will fire a watcher on all brokers. This watcher works as follows. It reads all the config change notifications.
* It keeps track of the highest config change suffix number it has applied previously. For any previously applied change it finds
* it checks if this notification is larger than a static expiration time (say 10mins) and if so it deletes this notification.
* For any new changes it reads the new configuration, combines it with the defaults, and updates the log config
* for all logs for that topic (if any) that it has.
*
* it checks if this notification is larger than a static expiration time (say 10mins) and if so it deletes this notification.
* For any new changes it reads the new configuration, combines it with the defaults, and updates the existing config.
*
* Note that config is always read from the config path in zk, the notification is just a trigger to do so. So if a broker is
* down and misses a change that is fine--when it restarts it will be loading the full config anyway. Note also that
* if there are two consecutive config changes it is possible that only the last one will be applied (since by the time the
* if there are two consecutive config changes it is possible that only the last one will be applied (since by the time the
* broker reads the config the both changes may have been made). In this case the broker would needlessly refresh the config twice,
* but that is harmless.
*
*
* On restart the config manager re-processes all notifications. This will usually be wasted work, but avoids any race conditions
* on startup where a change might be missed between the initial config load and registering for change notifications.
*
*
*/
class TopicConfigManager(private val zkClient: ZkClient,
private val logManager: LogManager,
private val changeExpirationMs: Long = 15*60*1000,
private val time: Time = SystemTime) extends Logging {
class DynamicConfigManager(private val zkClient: ZkClient,
private val configHandler : Map[String, ConfigHandler],
private val changeExpirationMs: Long = 15*60*1000,
private val time: Time = SystemTime) extends Logging {
private var lastExecutedChange = -1L
/**
* Begin watching for config changes
*/
def startup() {
ZkUtils.makeSurePersistentPathExists(zkClient, ZkUtils.TopicConfigChangesPath)
zkClient.subscribeChildChanges(ZkUtils.TopicConfigChangesPath, ConfigChangeListener)
ZkUtils.makeSurePersistentPathExists(zkClient, ZkUtils.EntityConfigChangesPath)
zkClient.subscribeChildChanges(ZkUtils.EntityConfigChangesPath, ConfigChangeListener)
processAllConfigChanges()
}
/**
* Process all config changes
*/
private def processAllConfigChanges() {
val configChanges = zkClient.getChildren(ZkUtils.TopicConfigChangesPath)
val configChanges = zkClient.getChildren(ZkUtils.EntityConfigChangesPath)
import JavaConversions._
processConfigChanges((configChanges: mutable.Buffer[String]).sorted)
}
@ -89,39 +101,59 @@ class TopicConfigManager(private val zkClient: ZkClient,
if (notifications.size > 0) {
info("Processing config change notification(s)...")
val now = time.milliseconds
val logs = logManager.logsByTopicPartition.toBuffer
val logsByTopic = logs.groupBy(_._1.topic).mapValues(_.map(_._2))
for (notification <- notifications) {
val changeId = changeNumber(notification)
if (changeId > lastExecutedChange) {
val changeZnode = ZkUtils.TopicConfigChangesPath + "/" + notification
val changeZnode = ZkUtils.EntityConfigChangesPath + "/" + notification
val (jsonOpt, stat) = ZkUtils.readDataMaybeNull(zkClient, changeZnode)
if(jsonOpt.isDefined) {
val json = jsonOpt.get
val topic = json.substring(1, json.length - 1) // hacky way to dequote
if (logsByTopic.contains(topic)) {
/* combine the default properties with the overrides in zk to create the new LogConfig */
val props = new Properties()
props.putAll(logManager.defaultConfig.originals)
props.putAll(AdminUtils.fetchTopicConfig(zkClient, topic))
val logConfig = LogConfig(props)
for (log <- logsByTopic(topic))
log.config = logConfig
info("Processed topic config change %d for topic %s, setting new config to %s.".format(changeId, topic, props))
purgeObsoleteNotifications(now, notifications)
}
}
lastExecutedChange = changeId
processNotification(jsonOpt)
}
lastExecutedChange = changeId
}
purgeObsoleteNotifications(now, notifications)
}
}
def processNotification(jsonOpt: Option[String]) = {
if(jsonOpt.isDefined) {
val json = jsonOpt.get
Json.parseFull(json) match {
case None => // There are no config overrides.
// Ignore non-json notifications because they can be from the deprecated TopicConfigManager
case Some(mapAnon: Map[_, _]) =>
val map = mapAnon collect
{ case (k: String, v: Any) => k -> v }
require(map("version") == 1)
val entityType = map.get("entity_type") match {
case Some(ConfigType.Topic) => ConfigType.Topic
case Some(ConfigType.Client) => ConfigType.Client
case _ => throw new IllegalArgumentException("Config change notification must have 'entity_type' set to either 'client' or 'topic'." +
" Received: " + json)
}
val entity = map.get("entity_name") match {
case Some(value: String) => value
case _ => throw new IllegalArgumentException("Config change notification does not specify 'entity_name'. Received: " + json)
}
configHandler(entityType).processConfigChanges(entity, AdminUtils.fetchEntityConfig(zkClient, entityType, entity))
case o => throw new IllegalArgumentException("Config change notification has an unexpected value. The format is:" +
"{\"version\" : 1," +
" \"entity_type\":\"topic/client\"," +
" \"entity_name\" : \"topic_name/client_id\"}." +
" Received: " + json)
}
}
}
private def purgeObsoleteNotifications(now: Long, notifications: Seq[String]) {
for(notification <- notifications.sorted) {
val (jsonOpt, stat) = ZkUtils.readDataMaybeNull(zkClient, ZkUtils.TopicConfigChangesPath + "/" + notification)
val (jsonOpt, stat) = ZkUtils.readDataMaybeNull(zkClient, ZkUtils.EntityConfigChangesPath + "/" + notification)
if(jsonOpt.isDefined) {
val changeZnode = ZkUtils.TopicConfigChangesPath + "/" + notification
val changeZnode = ZkUtils.EntityConfigChangesPath + "/" + notification
if (now - stat.getCtime > changeExpirationMs) {
debug("Purging config change notification " + notification)
ZkUtils.deletePath(zkClient, changeZnode)
@ -131,10 +163,10 @@ class TopicConfigManager(private val zkClient: ZkClient,
}
}
}
/* get the change number from a change notification znode */
private def changeNumber(name: String): Long = name.substring(AdminUtils.TopicConfigChangeZnodePrefix.length).toLong
private def changeNumber(name: String): Long = name.substring(AdminUtils.EntityConfigChangeZnodePrefix.length).toLong
/**
* A listener that applies config changes to logs
*/
@ -148,5 +180,4 @@ class TopicConfigManager(private val zkClient: ZkClient,
}
}
}
}

View File

@ -462,10 +462,13 @@ class KafkaApis(val requestChannel: RequestChannel,
info("Auto creation of topic %s with %d partitions and replication factor %d is successful!"
.format(topic, config.numPartitions, config.defaultReplicationFactor))
}
new TopicMetadata(topic, Seq.empty[PartitionMetadata], ErrorMapping.LeaderNotAvailableCode)
} catch {
case e: TopicExistsException => // let it go, possibly another broker created this topic
new TopicMetadata(topic, Seq.empty[PartitionMetadata], ErrorMapping.LeaderNotAvailableCode)
case itex: InvalidTopicException =>
new TopicMetadata(topic, Seq.empty[PartitionMetadata], ErrorMapping.InvalidTopicCode)
}
new TopicMetadata(topic, Seq.empty[PartitionMetadata], ErrorMapping.LeaderNotAvailableCode)
} else {
new TopicMetadata(topic, Seq.empty[PartitionMetadata], ErrorMapping.UnknownTopicOrPartitionCode)
}

View File

@ -92,7 +92,6 @@ class KafkaHealthcheck(private val brokerId: Int,
override def handleSessionEstablishmentError(error: Throwable): Unit = {
fatal("Could not establish session with zookeeper", error)
System.exit(-1)
}
}

View File

@ -31,8 +31,9 @@ import java.io.File
import kafka.utils._
import org.apache.kafka.common.metrics._
import org.apache.kafka.common.network.NetworkReceive
import org.apache.kafka.common.metrics.{JmxReporter, Metrics}
import scala.collection.{JavaConversions, mutable}
import scala.collection.mutable
import org.I0Itec.zkclient.ZkClient
import kafka.controller.{ControllerStats, KafkaController}
import kafka.cluster.{EndPoint, Broker}
@ -77,7 +78,9 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg
var replicaManager: ReplicaManager = null
var topicConfigManager: TopicConfigManager = null
var dynamicConfigHandlers: Map[String, ConfigHandler] = null
var dynamicConfigManager: DynamicConfigManager = null
val metrics: Metrics = new Metrics()
var consumerCoordinator: ConsumerCoordinator = null
@ -171,9 +174,11 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg
Mx4jLoader.maybeLoad()
/* start topic config manager */
topicConfigManager = new TopicConfigManager(zkClient, logManager)
topicConfigManager.startup()
/* start dynamic config manager */
dynamicConfigHandlers = Map[String, ConfigHandler](ConfigType.Topic -> new TopicConfigHandler(logManager),
ConfigType.Client -> new ClientIdConfigHandler)
dynamicConfigManager = new DynamicConfigManager(zkClient, dynamicConfigHandlers)
dynamicConfigManager.startup()
/* tell everyone we are alive */
val listeners = config.advertisedListeners.map {case(protocol, endpoint) =>

View File

@ -96,6 +96,7 @@ class OffsetManager(val config: OffsetManagerConfig,
private val loadingPartitions: mutable.Set[Int] = mutable.Set()
private val cleanupOrLoadMutex = new Object
private val shuttingDown = new AtomicBoolean(false)
private val offsetsTopicPartitionCount = getOffsetsTopicPartitionCount
this.logIdent = "[Offset Manager on Broker " + replicaManager.config.brokerId + "]: "
@ -170,7 +171,7 @@ class OffsetManager(val config: OffsetManagerConfig,
}
def partitionFor(group: String): Int = Utils.abs(group.hashCode) % config.offsetsTopicNumPartitions
def partitionFor(group: String): Int = Utils.abs(group.hashCode) % offsetsTopicPartitionCount
/**
* Fetch the current offset for the given group/topic/partition from the underlying offsets storage.
@ -436,13 +437,24 @@ class OffsetManager(val config: OffsetManagerConfig,
if (numRemoved > 0) info("Removed %d cached offsets for %s on follower transition."
.format(numRemoved, TopicAndPartition(ConsumerCoordinator.OffsetsTopicName, offsetsPartition)))
}
def shutdown() {
shuttingDown.set(true)
}
/**
* Gets the partition count of the offsets topic from ZooKeeper.
* If the topic does not exist, the configured partition count is returned.
*/
private def getOffsetsTopicPartitionCount = {
val topic = ConsumerCoordinator.OffsetsTopicName
val topicData = ZkUtils.getPartitionAssignmentForTopics(zkClient, Seq(topic))
if (topicData(topic).nonEmpty)
topicData(topic).size
else
config.offsetsTopicNumPartitions
}
}
object OffsetManager {

View File

@ -90,8 +90,8 @@ class ReplicaFetcherThread(name:String,
// Prior to truncating the follower's log, ensure that doing so is not disallowed by the configuration for unclean leader election.
// This situation could only happen if the unclean election configuration for a topic changes while a replica is down. Otherwise,
// we should never encounter this situation since a non-ISR leader cannot be elected if disallowed by the broker configuration.
if (!LogConfig.fromProps(brokerConfig.originals, AdminUtils.fetchTopicConfig(replicaMgr.zkClient,
topicAndPartition.topic)).uncleanLeaderElectionEnable) {
if (!LogConfig.fromProps(brokerConfig.originals, AdminUtils.fetchEntityConfig(replicaMgr.zkClient,
ConfigType.Topic, topicAndPartition.topic)).uncleanLeaderElectionEnable) {
// Log a fatal error and shutdown the broker to ensure that data loss does not unexpectedly occur.
fatal("Halting because log truncation is not allowed for topic %s,".format(topicAndPartition.topic) +
" Current leader %d's latest offset %d is less than replica %d's latest offset %d"

View File

@ -30,6 +30,7 @@ import kafka.consumer.ConsumerConnector
import kafka.consumer.KafkaStream
import kafka.consumer.ConsumerTimeoutException
import java.text.SimpleDateFormat
import java.util.concurrent.atomic.AtomicBoolean
/**
* Performance test for the full zookeeper consumer
@ -43,6 +44,7 @@ object ConsumerPerformance {
logger.info("Starting consumer...")
val totalMessagesRead = new AtomicLong(0)
val totalBytesRead = new AtomicLong(0)
val consumerTimeout = new AtomicBoolean(false)
if (!config.hideHeader) {
if (!config.showDetailedStats)
@ -67,7 +69,7 @@ object ConsumerPerformance {
var threadList = List[ConsumerPerfThread]()
for ((topic, streamList) <- topicMessageStreams)
for (i <- 0 until streamList.length)
threadList ::= new ConsumerPerfThread(i, "kafka-zk-consumer-" + i, streamList(i), config, totalMessagesRead, totalBytesRead)
threadList ::= new ConsumerPerfThread(i, "kafka-zk-consumer-" + i, streamList(i), config, totalMessagesRead, totalBytesRead, consumerTimeout)
logger.info("Sleeping for 1 second.")
Thread.sleep(1000)
@ -77,7 +79,10 @@ object ConsumerPerformance {
thread.start
for (thread <- threadList)
thread.join
endMs = System.currentTimeMillis - consumerConfig.consumerTimeoutMs
if(consumerTimeout.get())
endMs = System.currentTimeMillis - consumerConfig.consumerTimeoutMs
else
endMs = System.currentTimeMillis
consumerConnector.shutdown()
}
val elapsedSecs = (endMs - startMs) / 1000.0
@ -209,7 +214,7 @@ object ConsumerPerformance {
}
class ConsumerPerfThread(threadId: Int, name: String, stream: KafkaStream[Array[Byte], Array[Byte]],
config: ConsumerPerfConfig, totalMessagesRead: AtomicLong, totalBytesRead: AtomicLong)
config: ConsumerPerfConfig, totalMessagesRead: AtomicLong, totalBytesRead: AtomicLong, consumerTimeout: AtomicBoolean)
extends Thread(name) {
override def run() {
@ -238,7 +243,9 @@ object ConsumerPerformance {
} catch {
case _: InterruptedException =>
case _: ClosedByInterruptException =>
case _: ConsumerTimeoutException =>
case _: ConsumerTimeoutException => {
consumerTimeout.set(true);
}
case e: Throwable => e.printStackTrace()
}
totalMessagesRead.addAndGet(messagesRead)

View File

@ -18,8 +18,8 @@
package kafka.tools
import java.util
import java.util.concurrent.{TimeUnit, CountDownLatch}
import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger}
import java.util.concurrent.{CountDownLatch, TimeUnit}
import java.util.{Collections, Properties}
import com.yammer.metrics.core.Gauge
@ -29,7 +29,7 @@ import kafka.javaapi.consumer.ConsumerRebalanceListener
import kafka.message.MessageAndMetadata
import kafka.metrics.KafkaMetricsGroup
import kafka.serializer.DefaultDecoder
import kafka.utils.{CommandLineUtils, Logging, CoreUtils}
import kafka.utils.{CommandLineUtils, CoreUtils, Logging}
import org.apache.kafka.clients.producer.internals.ErrorLoggingCallback
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord, RecordMetadata}
import org.apache.kafka.common.utils.Utils
@ -185,6 +185,9 @@ object MirrorMaker extends Logging with KafkaMetricsGroup {
maybeSetDefaultProperty(producerProps, ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG, "true")
maybeSetDefaultProperty(producerProps, ProducerConfig.ACKS_CONFIG, "all")
maybeSetDefaultProperty(producerProps, ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, "1")
// Always set producer key and value serializer to ByteArraySerializer.
producerProps.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer")
producerProps.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer")
producer = new MirrorMakerProducer(producerProps)
// Create consumer connector

View File

@ -92,7 +92,7 @@ object ReplicaVerificationTool extends Logging {
.describedAs("ms")
.ofType(classOf[java.lang.Long])
.defaultsTo(30 * 1000L)
if(args.length == 0)
CommandLineUtils.printUsageAndDie(parser, "Validate that all replicas for a set of topics have the same data.")
@ -329,7 +329,7 @@ private class ReplicaBuffer(expectedReplicasPerTopicAndPartition: Map[TopicAndPa
if (currentTimeMs - lastReportTime > reportInterval) {
println(ReplicaVerificationTool.dateFormat.format(new Date(currentTimeMs)) + ": max lag is "
+ maxLag + " for partition " + maxLagTopicAndPartition + " at offset " + offsetWithMaxLag
+ " among " + messageSetCache.size + " paritions")
+ " among " + messageSetCache.size + " partitions")
lastReportTime = currentTimeMs
}
}

View File

@ -19,6 +19,7 @@ package kafka.utils
import kafka.cluster._
import kafka.consumer.{ConsumerThreadId, TopicCount}
import kafka.server.ConfigType
import org.I0Itec.zkclient.ZkClient
import org.I0Itec.zkclient.exception.{ZkNodeExistsException, ZkNoNodeException,
ZkMarshallingError, ZkBadVersionException}
@ -39,8 +40,6 @@ object ZkUtils extends Logging {
val ConsumersPath = "/consumers"
val BrokerIdsPath = "/brokers/ids"
val BrokerTopicsPath = "/brokers/topics"
val TopicConfigPath = "/config/topics"
val TopicConfigChangesPath = "/config/changes"
val ControllerPath = "/controller"
val ControllerEpochPath = "/controller_epoch"
val ReassignPartitionsPath = "/admin/reassign_partitions"
@ -48,6 +47,8 @@ object ZkUtils extends Logging {
val PreferredReplicaLeaderElectionPath = "/admin/preferred_replica_election"
val BrokerSequenceIdPath = "/brokers/seqid"
val IsrChangeNotificationPath = "/isr_change_notification"
val EntityConfigPath = "/config"
val EntityConfigChangesPath = "/config/changes"
def getTopicPath(topic: String): String = {
BrokerTopicsPath + "/" + topic
@ -57,8 +58,11 @@ object ZkUtils extends Logging {
getTopicPath(topic) + "/partitions"
}
def getTopicConfigPath(topic: String): String =
TopicConfigPath + "/" + topic
def getEntityConfigRootPath(entityType: String): String =
EntityConfigPath + "/" + entityType
def getEntityConfigPath(entityType: String, entity: String): String =
getEntityConfigRootPath(entityType) + "/" + entity
def getDeleteTopicPath(topic: String): String =
DeleteTopicsPath + "/" + topic
@ -93,8 +97,14 @@ object ZkUtils extends Logging {
}
def setupCommonPaths(zkClient: ZkClient) {
for(path <- Seq(ConsumersPath, BrokerIdsPath, BrokerTopicsPath, TopicConfigChangesPath, TopicConfigPath,
DeleteTopicsPath, BrokerSequenceIdPath))
for(path <- Seq(ConsumersPath,
BrokerIdsPath,
BrokerTopicsPath,
EntityConfigChangesPath,
ZkUtils.getEntityConfigRootPath(ConfigType.Topic),
ZkUtils.getEntityConfigRootPath(ConfigType.Client),
DeleteTopicsPath,
BrokerSequenceIdPath))
makeSurePersistentPathExists(zkClient, path)
}
@ -753,6 +763,17 @@ object ZkUtils extends Logging {
topics
}
/**
* Returns all the entities whose configs have been overridden.
*/
def getAllEntitiesWithConfig(zkClient: ZkClient, entityType: String): Seq[String] = {
val entities = ZkUtils.getChildrenParentMayNotExist(zkClient, getEntityConfigRootPath(entityType))
if(entities == null)
Seq.empty[String]
else
entities
}
def getAllPartitions(zkClient: ZkClient): Set[TopicAndPartition] = {
val topics = ZkUtils.getChildrenParentMayNotExist(zkClient, BrokerTopicsPath)
if(topics == null) Set.empty[TopicAndPartition]

View File

@ -43,7 +43,8 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging {
this.producerConfig.setProperty(ProducerConfig.ACKS_CONFIG, "all")
this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "my-test")
this.consumerConfig.setProperty(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, 4096.toString)
this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "20")
this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "100")
this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "30")
this.consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest")
override def generateConfigs() = {
@ -104,6 +105,11 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging {
consumer.subscribe(tp)
consumer.seek(tp, 0)
// wait until all the followers have synced the last HW with leader
TestUtils.waitUntilTrue(() => servers.forall(server =>
server.replicaManager.getReplica(tp.topic(), tp.partition()).get.highWatermark.messageOffset == numRecords
), "Failed to update high watermark for followers after timeout")
val scheduler = new BounceBrokerScheduler(numIters)
scheduler.start()

View File

@ -26,7 +26,7 @@ import kafka.server.KafkaConfig
import java.util.ArrayList
import org.junit.Assert._
import scala.collection.JavaConversions._
import scala.collection.JavaConverters._
import kafka.coordinator.ConsumerCoordinator
@ -95,7 +95,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging {
this.consumers(0).poll(50)
val pos1 = this.consumers(0).position(tp)
val pos2 = this.consumers(0).position(tp2)
this.consumers(0).commit(Map[TopicPartition,java.lang.Long]((tp, 3L)), CommitType.SYNC)
this.consumers(0).commit(Map[TopicPartition,java.lang.Long]((tp, 3L)).asJava, CommitType.SYNC)
assertEquals(3, this.consumers(0).committed(tp))
intercept[NoOffsetForPartitionException] {
this.consumers(0).committed(tp2)
@ -103,13 +103,13 @@ class ConsumerTest extends IntegrationTestHarness with Logging {
// positions should not change
assertEquals(pos1, this.consumers(0).position(tp))
assertEquals(pos2, this.consumers(0).position(tp2))
this.consumers(0).commit(Map[TopicPartition,java.lang.Long]((tp2, 5L)), CommitType.SYNC)
this.consumers(0).commit(Map[TopicPartition,java.lang.Long]((tp2, 5L)).asJava, CommitType.SYNC)
assertEquals(3, this.consumers(0).committed(tp))
assertEquals(5, this.consumers(0).committed(tp2))
// Using async should pick up the committed changes after commit completes
val commitCallback = new CountConsumerCommitCallback()
this.consumers(0).commit(Map[TopicPartition,java.lang.Long]((tp2, 7L)), CommitType.ASYNC, commitCallback)
this.consumers(0).commit(Map[TopicPartition,java.lang.Long]((tp2, 7L)).asJava, CommitType.ASYNC, commitCallback)
awaitCommitCallback(this.consumers(0), commitCallback)
assertEquals(7, this.consumers(0).committed(tp2))
}
@ -182,7 +182,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging {
TestUtils.createTopic(this.zkClient, "part-test", numParts, 1, this.servers)
val parts = this.consumers(0).partitionsFor("part-test")
assertNotNull(parts)
assertEquals(2, parts.length)
assertEquals(2, parts.size)
assertNull(this.consumers(0).partitionsFor("non-exist-topic"))
}
@ -199,14 +199,15 @@ class ConsumerTest extends IntegrationTestHarness with Logging {
assertNotNull(topics)
assertEquals(5, topics.size())
assertEquals(5, topics.keySet().size())
assertEquals(2, topics.get(topic1).length)
assertEquals(2, topics.get(topic2).length)
assertEquals(2, topics.get(topic3).length)
assertEquals(2, topics.get(topic1).size)
assertEquals(2, topics.get(topic2).size)
assertEquals(2, topics.get(topic3).size)
}
def testPartitionReassignmentCallback() {
val callback = new TestConsumerReassignmentCallback()
this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "100"); // timeout quickly to avoid slow test
this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "30");
val consumer0 = new KafkaConsumer(this.consumerConfig, callback, new ByteArrayDeserializer(), new ByteArrayDeserializer())
consumer0.subscribe(topic)
@ -215,9 +216,9 @@ class ConsumerTest extends IntegrationTestHarness with Logging {
consumer0.poll(50)
// get metadata for the topic
var parts = consumer0.partitionsFor(ConsumerCoordinator.OffsetsTopicName)
var parts = consumer0.partitionsFor(ConsumerCoordinator.OffsetsTopicName).asScala
while(parts == null)
parts = consumer0.partitionsFor(ConsumerCoordinator.OffsetsTopicName)
parts = consumer0.partitionsFor(ConsumerCoordinator.OffsetsTopicName).asScala
assertEquals(1, parts.size)
assertNotNull(parts(0).leader())
@ -238,6 +239,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging {
def testUnsubscribeTopic() {
val callback = new TestConsumerReassignmentCallback()
this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "100"); // timeout quickly to avoid slow test
this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "30");
val consumer0 = new KafkaConsumer(this.consumerConfig, callback, new ByteArrayDeserializer(), new ByteArrayDeserializer())
try {
@ -254,6 +256,71 @@ class ConsumerTest extends IntegrationTestHarness with Logging {
}
}
def testExpandingTopicSubscriptions() {
val otherTopic = "other"
val subscriptions = Set(new TopicPartition(topic, 0), new TopicPartition(topic, 1))
val expandedSubscriptions = subscriptions ++ Set(new TopicPartition(otherTopic, 0), new TopicPartition(otherTopic, 1))
this.consumers(0).subscribe(topic)
TestUtils.waitUntilTrue(() => {
this.consumers(0).poll(50)
this.consumers(0).subscriptions == subscriptions.asJava
}, s"Expected partitions ${subscriptions.asJava} but actually got ${this.consumers(0).subscriptions}")
TestUtils.createTopic(this.zkClient, otherTopic, 2, serverCount, this.servers)
this.consumers(0).subscribe(otherTopic)
TestUtils.waitUntilTrue(() => {
this.consumers(0).poll(50)
this.consumers(0).subscriptions == expandedSubscriptions.asJava
}, s"Expected partitions ${expandedSubscriptions.asJava} but actually got ${this.consumers(0).subscriptions}")
}
def testShrinkingTopicSubscriptions() {
val otherTopic = "other"
TestUtils.createTopic(this.zkClient, otherTopic, 2, serverCount, this.servers)
val subscriptions = Set(new TopicPartition(topic, 0), new TopicPartition(topic, 1), new TopicPartition(otherTopic, 0), new TopicPartition(otherTopic, 1))
val shrunkenSubscriptions = Set(new TopicPartition(topic, 0), new TopicPartition(topic, 1))
this.consumers(0).subscribe(topic, otherTopic)
TestUtils.waitUntilTrue(() => {
this.consumers(0).poll(50)
this.consumers(0).subscriptions == subscriptions.asJava
}, s"Expected partitions ${subscriptions.asJava} but actually got ${this.consumers(0).subscriptions}")
this.consumers(0).unsubscribe(otherTopic)
TestUtils.waitUntilTrue(() => {
this.consumers(0).poll(50)
this.consumers(0).subscriptions == shrunkenSubscriptions.asJava
}, s"Expected partitions ${shrunkenSubscriptions.asJava} but actually got ${this.consumers(0).subscriptions}")
}
def testPartitionPauseAndResume() {
sendRecords(5)
this.consumers(0).subscribe(tp)
consumeRecords(this.consumers(0), 5, 0)
this.consumers(0).pause(tp)
sendRecords(5)
assertTrue(this.consumers(0).poll(0).isEmpty)
this.consumers(0).resume(tp)
consumeRecords(this.consumers(0), 5, 5)
}
def testPauseStateNotPreservedByRebalance() {
this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "100"); // timeout quickly to avoid slow test
this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "30");
val consumer0 = new KafkaConsumer(this.consumerConfig, null, new ByteArrayDeserializer(), new ByteArrayDeserializer())
sendRecords(5)
consumer0.subscribe(topic)
consumeRecords(consumer0, 5, 0)
consumer0.pause(tp)
// subscribe to a new topic to trigger a rebalance
consumer0.subscribe("topic2")
// after rebalance, our position should be reset and our pause state lost,
// so we should be able to consume from the beginning
consumeRecords(consumer0, 0, 5)
}
private class TestConsumerReassignmentCallback extends ConsumerRebalanceCallback {
var callsToAssigned = 0
var callsToRevoked = 0
@ -264,7 +331,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging {
def onPartitionsRevoked(consumer: Consumer[_,_], partitions: java.util.Collection[TopicPartition]) {
info("onPartitionsRevoked called.")
callsToRevoked += 1
}
}
}
private def sendRecords(numRecords: Int): Unit = {
@ -283,7 +350,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging {
val maxIters = numRecords * 300
var iters = 0
while (records.size < numRecords) {
for (record <- consumer.poll(50))
for (record <- consumer.poll(50).asScala)
records.add(record)
if(iters > maxIters)
throw new IllegalStateException("Failed to consume the expected records after " + iters + " iterations.")

View File

@ -372,20 +372,18 @@ class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness {
var producer: KafkaProducer[Array[Byte],Array[Byte]] = null
try {
// create topic
val leaders = TestUtils.createTopic(zkClient, topic, 2, 2, servers)
val leader0 = leaders(0)
val leader1 = leaders(1)
val leaders = TestUtils.createTopic(zkClient, topic, 1, 2, servers)
val leader = leaders(0)
// create record
val record0 = new ProducerRecord[Array[Byte], Array[Byte]](topic, 0, null, "value".getBytes)
val record1 = new ProducerRecord[Array[Byte], Array[Byte]](topic, 1, null, "value".getBytes)
val record = new ProducerRecord[Array[Byte], Array[Byte]](topic, 0, null, "value".getBytes)
// Test closing from sender thread.
class CloseCallback(producer: KafkaProducer[Array[Byte], Array[Byte]]) extends Callback {
override def onCompletion(metadata: RecordMetadata, exception: Exception) {
// Trigger another batch in accumulator before close the producer. These messages should
// not be sent.
(0 until numRecords) map (i => producer.send(record1))
(0 until numRecords) map (i => producer.send(record))
// The close call will be called by all the message callbacks. This tests idempotence of the close call.
producer.close(0, TimeUnit.MILLISECONDS)
// Test close with non zero timeout. Should not block at all.
@ -395,29 +393,20 @@ class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness {
for(i <- 0 until 50) {
producer = TestUtils.createNewProducer(brokerList, lingerMs = Long.MaxValue)
// send message to partition 0
var responses = (0 until numRecords) map (i => producer.send(record0))
// send message to partition 1
responses ++= ((0 until numRecords) map (i => producer.send(record1, new CloseCallback(producer))))
val responses = ((0 until numRecords) map (i => producer.send(record, new CloseCallback(producer))))
assertTrue("No request is complete.", responses.forall(!_.isDone()))
// flush the messages.
producer.flush()
assertTrue("All request are complete.", responses.forall(_.isDone()))
// Check the messages received by broker.
val fetchResponse0 = if (leader0.get == configs(0).brokerId) {
val fetchResponse = if (leader.get == configs(0).brokerId) {
consumer1.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, Int.MaxValue).build())
} else {
consumer2.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, Int.MaxValue).build())
}
val fetchResponse1 = if (leader1.get == configs(0).brokerId) {
consumer1.fetch(new FetchRequestBuilder().addFetch(topic, 1, 0, Int.MaxValue).build())
} else {
consumer2.fetch(new FetchRequestBuilder().addFetch(topic, 1, 0, Int.MaxValue).build())
}
val expectedNumRecords = (i + 1) * numRecords
assertEquals("Fetch response to partition 0 should have %d messages.".format(expectedNumRecords),
expectedNumRecords, fetchResponse0.messageSet(topic, 0).size)
assertEquals("Fetch response to partition 1 should have %d messages.".format(expectedNumRecords),
expectedNumRecords, fetchResponse1.messageSet(topic, 1).size)
expectedNumRecords, fetchResponse.messageSet(topic, 0).size)
}
} finally {
if (producer != null)

View File

@ -25,7 +25,7 @@ import kafka.log._
import kafka.zk.ZooKeeperTestHarness
import kafka.utils.{Logging, ZkUtils, TestUtils}
import kafka.common.{InvalidTopicException, TopicExistsException, TopicAndPartition}
import kafka.server.{KafkaServer, KafkaConfig}
import kafka.server.{ConfigType, KafkaServer, KafkaConfig}
import java.io.File
import TestUtils._
@ -407,12 +407,16 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging {
checkConfig(maxMessageSize, retentionMs)
// now double the config values for the topic and check that it is applied
val newConfig: Properties = makeConfig(2*maxMessageSize, 2 * retentionMs)
AdminUtils.changeTopicConfig(server.zkClient, topic, makeConfig(2*maxMessageSize, 2 * retentionMs))
checkConfig(2*maxMessageSize, 2 * retentionMs)
// Verify that the same config can be read from ZK
val configInZk = AdminUtils.fetchEntityConfig(server.zkClient, ConfigType.Topic, topic)
assertEquals(newConfig, configInZk)
} finally {
server.shutdown()
server.config.logDirs.foreach(CoreUtils.rm(_))
}
}
}

View File

@ -0,0 +1,73 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.admin
import junit.framework.Assert._
import kafka.admin.ConfigCommand.ConfigCommandOptions
import org.junit.Test
import org.scalatest.junit.JUnit3Suite
import kafka.utils.Logging
import kafka.utils.TestUtils
import kafka.zk.ZooKeeperTestHarness
import kafka.server.{ConfigType, OffsetManager, KafkaConfig}
import kafka.admin.TopicCommand.TopicCommandOptions
import kafka.utils.ZkUtils
class ConfigCommandTest extends JUnit3Suite with ZooKeeperTestHarness with Logging {
@Test
def testArgumentParse() {
// Should parse correctly
var createOpts = new ConfigCommandOptions(Array("--zookeeper", zkConnect,
"--entity-name", "x",
"--entity-type", "client",
"--describe"))
createOpts.checkArgs()
// For --alter and added config
createOpts = new ConfigCommandOptions(Array("--zookeeper", zkConnect,
"--entity-name", "x",
"--entity-type", "client",
"--alter",
"--added-config", "a=b,c=d"))
createOpts.checkArgs()
// For alter and deleted config
createOpts = new ConfigCommandOptions(Array("--zookeeper", zkConnect,
"--entity-name", "x",
"--entity-type", "client",
"--alter",
"--deleted-config", "a,b,c"))
createOpts.checkArgs()
// For alter and both added, deleted config
createOpts = new ConfigCommandOptions(Array("--zookeeper", zkConnect,
"--entity-name", "x",
"--entity-type", "client",
"--alter",
"--added-config", "a=b,c=d",
"--deleted-config", "a"))
createOpts.checkArgs()
val addedProps = ConfigCommand.parseConfigsToBeAdded(createOpts)
assertEquals(2, addedProps.size())
assertEquals("b", addedProps.getProperty("a"))
assertEquals("d", addedProps.getProperty("c"))
val deletedProps = ConfigCommand.parseConfigsToBeDeleted(createOpts)
assertEquals(1, deletedProps.size)
assertEquals("a", deletedProps(0))
}
}

View File

@ -22,6 +22,7 @@ import org.scalatest.junit.JUnit3Suite
import kafka.utils.Logging
import kafka.utils.TestUtils
import kafka.zk.ZooKeeperTestHarness
import kafka.server.{ConfigType, OffsetManager, KafkaConfig}
import kafka.admin.TopicCommand.TopicCommandOptions
import kafka.utils.ZkUtils
import kafka.coordinator.ConsumerCoordinator
@ -43,20 +44,18 @@ class TopicCommandTest extends JUnit3Suite with ZooKeeperTestHarness with Loggin
"--config", cleanupKey + "=" + cleanupVal,
"--topic", topic))
TopicCommand.createTopic(zkClient, createOpts)
val props = AdminUtils.fetchTopicConfig(zkClient, topic)
val props = AdminUtils.fetchEntityConfig(zkClient, ConfigType.Topic, topic)
assertTrue("Properties after creation don't contain " + cleanupKey, props.containsKey(cleanupKey))
assertTrue("Properties after creation have incorrect value", props.getProperty(cleanupKey).equals(cleanupVal))
// pre-create the topic config changes path to avoid a NoNodeException
ZkUtils.createPersistentPath(zkClient, ZkUtils.TopicConfigChangesPath)
ZkUtils.createPersistentPath(zkClient, ZkUtils.EntityConfigChangesPath)
// modify the topic to add new partitions
val numPartitionsModified = 3
val alterOpts = new TopicCommandOptions(Array("--partitions", numPartitionsModified.toString,
"--config", cleanupKey + "=" + cleanupVal,
"--topic", topic))
val alterOpts = new TopicCommandOptions(Array("--partitions", numPartitionsModified.toString, "--topic", topic))
TopicCommand.alterTopic(zkClient, alterOpts)
val newProps = AdminUtils.fetchTopicConfig(zkClient, topic)
val newProps = AdminUtils.fetchEntityConfig(zkClient, ConfigType.Topic, topic)
assertTrue("Updated properties do not contain " + cleanupKey, newProps.containsKey(cleanupKey))
assertTrue("Updated properties have incorrect value", newProps.getProperty(cleanupKey).equals(cleanupVal))
}
@ -99,4 +98,4 @@ class TopicCommandTest extends JUnit3Suite with ZooKeeperTestHarness with Loggin
}
assertFalse("Delete path for topic shouldn't exist after deletion.", zkClient.exists(deleteOffsetTopicPath))
}
}
}

View File

@ -0,0 +1,187 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.controller
import java.util.concurrent.LinkedBlockingQueue
import java.util.Properties
import junit.framework.Assert._
import org.scalatest.junit.JUnit3Suite
import org.junit.{Test, After, Before}
import org.I0Itec.zkclient.{IZkDataListener, IZkStateListener, ZkClient}
import org.I0Itec.zkclient.serialize.ZkSerializer
import org.apache.log4j.{Logger, Level}
import kafka.api.RequestOrResponse
import kafka.common.TopicAndPartition
import kafka.integration.KafkaServerTestHarness
import kafka.server.BrokerState
import kafka.server.KafkaConfig
import kafka.server.KafkaServer
import kafka.server.RunningAsController
import kafka.utils._
import kafka.utils.TestUtils._
import scala.collection.Map
import scala.collection.mutable
class ControllerFailoverTest extends KafkaServerTestHarness with Logging {
val log = Logger.getLogger(classOf[ControllerFailoverTest])
val numNodes = 2
val numParts = 1
val msgQueueSize = 1
val topic = "topic1"
val overridingProps = new Properties()
overridingProps.put(KafkaConfig.NumPartitionsProp, numParts.toString)
override def generateConfigs() = TestUtils.createBrokerConfigs(numNodes, zkConnect)
.map(KafkaConfig.fromProps(_, overridingProps))
override def setUp() {
super.setUp()
}
override def tearDown() {
super.tearDown()
}
/**
* See @link{https://issues.apache.org/jira/browse/KAFKA-2300}
* for the background of this test case
*/
def testMetadataUpdate() {
log.setLevel(Level.INFO)
var controller: KafkaServer = this.servers.head;
// Find the current controller
val epochMap: mutable.Map[Int, Int] = mutable.Map.empty
for (server <- this.servers) {
epochMap += (server.config.brokerId -> server.kafkaController.epoch)
if(server.kafkaController.isActive()) {
controller = server
}
}
// Create topic with one partition
kafka.admin.AdminUtils.createTopic(controller.zkClient, topic, 1, 1)
val topicPartition = TopicAndPartition("topic1", 0)
var partitions = controller.kafkaController.partitionStateMachine.partitionsInState(OnlinePartition)
while (!partitions.contains(topicPartition)) {
partitions = controller.kafkaController.partitionStateMachine.partitionsInState(OnlinePartition)
Thread.sleep(100)
}
// Replace channel manager with our mock manager
controller.kafkaController.controllerContext.controllerChannelManager.shutdown()
val channelManager = new MockChannelManager(controller.kafkaController.controllerContext,
controller.kafkaController.config)
channelManager.startup()
controller.kafkaController.controllerContext.controllerChannelManager = channelManager
channelManager.shrinkBlockingQueue(0)
channelManager.stopSendThread(0)
// Spawn a new thread to block on the outgoing channel
// queue
val thread = new Thread(new Runnable {
def run() {
try {
controller.kafkaController.sendUpdateMetadataRequest(Seq(0), Set(topicPartition))
log.info("Queue state %d %d".format(channelManager.queueCapacity(0), channelManager.queueSize(0)))
controller.kafkaController.sendUpdateMetadataRequest(Seq(0), Set(topicPartition))
log.info("Queue state %d %d".format(channelManager.queueCapacity(0), channelManager.queueSize(0)))
} catch {
case e : Exception => {
log.info("Thread interrupted")
}
}
}
})
thread.setName("mythread")
thread.start()
while (thread.getState() != Thread.State.WAITING) {
Thread.sleep(100)
}
// Assume that the thread is WAITING because it is
// blocked on the queue, so interrupt and move forward
thread.interrupt()
thread.join()
channelManager.resumeSendThread(0)
// Wait and find current controller
var found = false
var counter = 0
while (!found && counter < 10) {
for (server <- this.servers) {
val previousEpoch = (epochMap get server.config.brokerId) match {
case Some(epoch) =>
epoch
case None =>
val msg = String.format("Missing element in epoch map %s", epochMap.mkString(", "))
throw new IllegalStateException(msg)
}
if (server.kafkaController.isActive
&& (previousEpoch) < server.kafkaController.epoch) {
controller = server
found = true
}
}
if (!found) {
Thread.sleep(100)
counter += 1
}
}
// Give it a shot to make sure that sending isn't blocking
try {
controller.kafkaController.sendUpdateMetadataRequest(Seq(0), Set(topicPartition))
} catch {
case e : Throwable => {
fail(e)
}
}
}
}
class MockChannelManager(private val controllerContext: ControllerContext,
config: KafkaConfig)
extends ControllerChannelManager(controllerContext, config) {
def stopSendThread(brokerId: Int) {
val requestThread = brokerStateInfo(brokerId).requestSendThread
requestThread.isRunning.set(false)
requestThread.interrupt
requestThread.join
}
def shrinkBlockingQueue(brokerId: Int) {
val messageQueue = new LinkedBlockingQueue[(RequestOrResponse, RequestOrResponse => Unit)](1)
val brokerInfo = this.brokerStateInfo(brokerId)
this.brokerStateInfo.put(brokerId, new ControllerBrokerStateInfo(brokerInfo.channel,
brokerInfo.broker,
messageQueue,
brokerInfo.requestSendThread))
}
def resumeSendThread (brokerId: Int) {
this.startRequestSendThread(0)
}
def queueCapacity(brokerId: Int): Int = {
this.brokerStateInfo(brokerId).messageQueue.remainingCapacity
}
def queueSize(brokerId: Int): Int = {
this.brokerStateInfo(brokerId).messageQueue.size
}
}

View File

@ -43,8 +43,8 @@ class ConsumerCoordinatorResponseTest extends JUnitSuite {
type HeartbeatCallback = Short => Unit
val ConsumerMinSessionTimeout = 10
val ConsumerMaxSessionTimeout = 100
val DefaultSessionTimeout = 20
val ConsumerMaxSessionTimeout = 200
val DefaultSessionTimeout = 100
var consumerCoordinator: ConsumerCoordinator = null
var offsetManager : OffsetManager = null
@ -238,7 +238,7 @@ class ConsumerCoordinatorResponseTest extends JUnitSuite {
// First start up a group (with a slightly larger timeout to give us time to heartbeat when the rebalance starts)
val joinGroupResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_CONSUMER_ID, partitionAssignmentStrategy,
100, isCoordinatorForGroup = true)
DefaultSessionTimeout, isCoordinatorForGroup = true)
val assignedConsumerId = joinGroupResult._2
val initialGenerationId = joinGroupResult._3
val joinGroupErrorCode = joinGroupResult._4
@ -310,7 +310,8 @@ class ConsumerCoordinatorResponseTest extends JUnitSuite {
sessionTimeout: Int,
isCoordinatorForGroup: Boolean): JoinGroupCallbackParams = {
val responseFuture = sendJoinGroup(groupId, consumerId, partitionAssignmentStrategy, sessionTimeout, isCoordinatorForGroup)
Await.result(responseFuture, Duration(40, TimeUnit.MILLISECONDS))
// should only have to wait as long as session timeout, but allow some extra time in case of an unexpected delay
Await.result(responseFuture, Duration(sessionTimeout+100, TimeUnit.MILLISECONDS))
}
private def heartbeat(groupId: String,

View File

@ -135,6 +135,34 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness {
assertTrue(partitionMetadata.head.leader.isDefined)
}
def testAutoCreateTopicWithCollision {
// auto create topic
val topic1 = "testAutoCreate_Topic"
val topic2 = "testAutoCreate.Topic"
var topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic1, topic2), brokerEndPoints, "TopicMetadataTest-testAutoCreateTopic",
2000,0).topicsMetadata
assertEquals("Expecting metadata for 2 topics", 2, topicsMetadata.size)
assertEquals("Expecting metadata for topic1", topic1, topicsMetadata.head.topic)
assertEquals(ErrorMapping.LeaderNotAvailableCode, topicsMetadata.head.errorCode)
assertEquals("Expecting metadata for topic2", topic2, topicsMetadata(1).topic)
assertEquals("Expecting InvalidTopicCode for topic2 metadata", ErrorMapping.InvalidTopicCode, topicsMetadata(1).errorCode)
// wait for leader to be elected
TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic1, 0)
TestUtils.waitUntilMetadataIsPropagated(Seq(server1), topic1, 0)
// retry the metadata for the first auto created topic
topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic1), brokerEndPoints, "TopicMetadataTest-testBasicTopicMetadata",
2000,0).topicsMetadata
assertEquals(ErrorMapping.NoError, topicsMetadata.head.errorCode)
assertEquals(ErrorMapping.NoError, topicsMetadata.head.partitionsMetadata.head.errorCode)
var partitionMetadata = topicsMetadata.head.partitionsMetadata
assertEquals("Expecting metadata for 1 partition", 1, partitionMetadata.size)
assertEquals("Expecting partition id to be 0", 0, partitionMetadata.head.partitionId)
assertEquals(1, partitionMetadata.head.replicas.size)
assertTrue(partitionMetadata.head.leader.isDefined)
}
private def checkIsr(servers: Seq[KafkaServer]): Unit = {
val activeBrokers: Seq[KafkaServer] = servers.filter(x => x.brokerState.currentState != NotRunning.state)
val expectedIsr: Seq[BrokerEndPoint] = activeBrokers.map(

View File

@ -32,28 +32,6 @@ class LogConfigTest extends JUnit3Suite {
Assert.assertEquals(LogConfig(), config)
}
@Test
def testFromPropsToProps() {
import scala.util.Random._
val expected = new Properties()
LogConfig.configNames().foreach((name) => {
name match {
case LogConfig.UncleanLeaderElectionEnableProp => expected.setProperty(name, randFrom("true", "false"))
case LogConfig.CompressionTypeProp => expected.setProperty(name, randFrom("producer", "uncompressed", "gzip"))
case LogConfig.CleanupPolicyProp => expected.setProperty(name, randFrom(LogConfig.Compact, LogConfig.Delete))
case LogConfig.MinCleanableDirtyRatioProp => expected.setProperty(name, "%.1f".format(nextDouble * .9 + .1))
case LogConfig.MinInSyncReplicasProp => expected.setProperty(name, (nextInt(Int.MaxValue - 1) + 1).toString)
case LogConfig.RetentionBytesProp => expected.setProperty(name, nextInt().toString)
case LogConfig.RetentionMsProp => expected.setProperty(name, nextLong().toString)
case LogConfig.PreAllocateEnableProp => expected.setProperty(name, randFrom("true", "false"))
case positiveIntProperty => expected.setProperty(name, nextInt(Int.MaxValue).toString)
}
})
val actual = LogConfig(expected).originals
Assert.assertEquals(expected, actual)
}
@Test
def testFromPropsInvalid() {
LogConfig.configNames().foreach((name) => {

View File

@ -19,6 +19,7 @@ package kafka.server
import java.util.Properties
import junit.framework.Assert._
import org.easymock.{Capture, EasyMock}
import org.junit.Test
import kafka.integration.KafkaServerTestHarness
import kafka.utils._
@ -32,8 +33,10 @@ class DynamicConfigChangeTest extends JUnit3Suite with KafkaServerTestHarness {
@Test
def testConfigChange() {
val oldVal: java.lang.Long = 100000
val newVal: java.lang.Long = 200000
assertTrue("Should contain a ConfigHandler for topics",
this.servers(0).dynamicConfigHandlers.contains(ConfigType.Topic))
val oldVal: java.lang.Long = 100000L
val newVal: java.lang.Long = 200000L
val tp = TopicAndPartition("test", 0)
val logProps = new Properties()
logProps.put(LogConfig.FlushMessagesProp, oldVal.toString)
@ -50,6 +53,25 @@ class DynamicConfigChangeTest extends JUnit3Suite with KafkaServerTestHarness {
}
}
// For now client config changes do not do anything. Simply verify that the call was made
@Test
def testClientConfigChange() {
assertTrue("Should contain a ConfigHandler for topics",
this.servers(0).dynamicConfigHandlers.contains(ConfigType.Client))
val clientId = "testClient"
val props = new Properties()
props.put("a.b", "c")
props.put("x.y", "z")
AdminUtils.changeClientIdConfig(zkClient, clientId, props)
TestUtils.retry(10000) {
val configHandler = this.servers(0).dynamicConfigHandlers(ConfigType.Client).asInstanceOf[ClientIdConfigHandler]
assertTrue("ClientId testClient must exist", configHandler.configPool.contains(clientId))
assertEquals("ClientId testClient must be the only override", 1, configHandler.configPool.size)
assertEquals("c", configHandler.configPool.get(clientId).getProperty("a.b"))
assertEquals("z", configHandler.configPool.get(clientId).getProperty("x.y"))
}
}
@Test
def testConfigChangeOnNonExistingTopic() {
val topic = TestUtils.tempTopic
@ -63,4 +85,59 @@ class DynamicConfigChangeTest extends JUnit3Suite with KafkaServerTestHarness {
}
}
}
@Test
def testProcessNotification {
val props = new Properties()
props.put("a.b", "10")
// Create a mock ConfigHandler to record config changes it is asked to process
val entityArgument = new Capture[String]()
val propertiesArgument = new Capture[Properties]()
val handler = EasyMock.createNiceMock(classOf[ConfigHandler])
handler.processConfigChanges(
EasyMock.and(EasyMock.capture(entityArgument), EasyMock.isA(classOf[String])),
EasyMock.and(EasyMock.capture(propertiesArgument), EasyMock.isA(classOf[Properties])))
EasyMock.expectLastCall().once()
EasyMock.replay(handler)
val configManager = new DynamicConfigManager(zkClient, Map(ConfigType.Topic -> handler))
// Notifications created using the old TopicConfigManager are ignored.
configManager.processNotification(Some("not json"))
// Incorrect Map. No version
try {
val jsonMap = Map("v" -> 1, "x" -> 2)
configManager.processNotification(Some(Json.encode(jsonMap)))
fail("Should have thrown an Exception while parsing incorrect notification " + jsonMap)
}
catch {
case t: Throwable =>
}
// Version is provided. EntityType is incorrect
try {
val jsonMap = Map("version" -> 1, "entity_type" -> "garbage", "entity_name" -> "x")
configManager.processNotification(Some(Json.encode(jsonMap)))
fail("Should have thrown an Exception while parsing incorrect notification " + jsonMap)
}
catch {
case t: Throwable =>
}
// EntityName isn't provided
try {
val jsonMap = Map("version" -> 1, "entity_type" -> ConfigType.Topic)
configManager.processNotification(Some(Json.encode(jsonMap)))
fail("Should have thrown an Exception while parsing incorrect notification " + jsonMap)
}
catch {
case t: Throwable =>
}
// Everything is provided
val jsonMap = Map("version" -> 1, "entity_type" -> ConfigType.Topic, "entity_name" -> "x")
configManager.processNotification(Some(Json.encode(jsonMap)))
// Verify that processConfigChanges was only called once
EasyMock.verify(handler)
}
}

View File

@ -1,403 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.server
import java.util.Properties
import kafka.api.ApiVersion
import kafka.message._
import org.apache.kafka.common.protocol.SecurityProtocol
import org.junit.{Assert, Test}
import org.scalatest.junit.JUnit3Suite
import scala.collection.Map
import scala.util.Random._
class KafkaConfigConfigDefTest extends JUnit3Suite {
@Test
def testFromPropsEmpty() {
// only required
val p = new Properties()
p.put(KafkaConfig.ZkConnectProp, "127.0.0.1:2181")
val actualConfig = KafkaConfig.fromProps(p)
val expectedConfig = new KafkaConfig(p)
Assert.assertEquals(expectedConfig.zkConnect, actualConfig.zkConnect)
Assert.assertEquals(expectedConfig.zkSessionTimeoutMs, actualConfig.zkSessionTimeoutMs)
Assert.assertEquals(expectedConfig.zkConnectionTimeoutMs, actualConfig.zkConnectionTimeoutMs)
Assert.assertEquals(expectedConfig.zkSyncTimeMs, actualConfig.zkSyncTimeMs)
Assert.assertEquals(expectedConfig.maxReservedBrokerId, actualConfig.maxReservedBrokerId)
Assert.assertEquals(expectedConfig.brokerId, actualConfig.brokerId)
Assert.assertEquals(expectedConfig.messageMaxBytes, actualConfig.messageMaxBytes)
Assert.assertEquals(expectedConfig.numNetworkThreads, actualConfig.numNetworkThreads)
Assert.assertEquals(expectedConfig.numIoThreads, actualConfig.numIoThreads)
Assert.assertEquals(expectedConfig.backgroundThreads, actualConfig.backgroundThreads)
Assert.assertEquals(expectedConfig.queuedMaxRequests, actualConfig.queuedMaxRequests)
Assert.assertEquals(expectedConfig.port, actualConfig.port)
Assert.assertEquals(expectedConfig.hostName, actualConfig.hostName)
Assert.assertEquals(expectedConfig.advertisedHostName, actualConfig.advertisedHostName)
Assert.assertEquals(expectedConfig.advertisedPort, actualConfig.advertisedPort)
Assert.assertEquals(expectedConfig.socketSendBufferBytes, actualConfig.socketSendBufferBytes)
Assert.assertEquals(expectedConfig.socketReceiveBufferBytes, actualConfig.socketReceiveBufferBytes)
Assert.assertEquals(expectedConfig.socketRequestMaxBytes, actualConfig.socketRequestMaxBytes)
Assert.assertEquals(expectedConfig.maxConnectionsPerIp, actualConfig.maxConnectionsPerIp)
Assert.assertEquals(expectedConfig.maxConnectionsPerIpOverrides, actualConfig.maxConnectionsPerIpOverrides)
Assert.assertEquals(expectedConfig.connectionsMaxIdleMs, actualConfig.connectionsMaxIdleMs)
Assert.assertEquals(expectedConfig.numPartitions, actualConfig.numPartitions)
Assert.assertEquals(expectedConfig.logDirs, actualConfig.logDirs)
Assert.assertEquals(expectedConfig.logSegmentBytes, actualConfig.logSegmentBytes)
Assert.assertEquals(expectedConfig.logRollTimeMillis, actualConfig.logRollTimeMillis)
Assert.assertEquals(expectedConfig.logRollTimeJitterMillis, actualConfig.logRollTimeJitterMillis)
Assert.assertEquals(expectedConfig.logRetentionTimeMillis, actualConfig.logRetentionTimeMillis)
Assert.assertEquals(expectedConfig.logRetentionBytes, actualConfig.logRetentionBytes)
Assert.assertEquals(expectedConfig.logCleanupIntervalMs, actualConfig.logCleanupIntervalMs)
Assert.assertEquals(expectedConfig.logCleanupPolicy, actualConfig.logCleanupPolicy)
Assert.assertEquals(expectedConfig.logCleanerThreads, actualConfig.logCleanerThreads)
Assert.assertEquals(expectedConfig.logCleanerIoMaxBytesPerSecond, actualConfig.logCleanerIoMaxBytesPerSecond, 0.0)
Assert.assertEquals(expectedConfig.logCleanerDedupeBufferSize, actualConfig.logCleanerDedupeBufferSize)
Assert.assertEquals(expectedConfig.logCleanerIoBufferSize, actualConfig.logCleanerIoBufferSize)
Assert.assertEquals(expectedConfig.logCleanerDedupeBufferLoadFactor, actualConfig.logCleanerDedupeBufferLoadFactor, 0.0)
Assert.assertEquals(expectedConfig.logCleanerBackoffMs, actualConfig.logCleanerBackoffMs)
Assert.assertEquals(expectedConfig.logCleanerMinCleanRatio, actualConfig.logCleanerMinCleanRatio, 0.0)
Assert.assertEquals(expectedConfig.logCleanerEnable, actualConfig.logCleanerEnable)
Assert.assertEquals(expectedConfig.logCleanerDeleteRetentionMs, actualConfig.logCleanerDeleteRetentionMs)
Assert.assertEquals(expectedConfig.logIndexSizeMaxBytes, actualConfig.logIndexSizeMaxBytes)
Assert.assertEquals(expectedConfig.logIndexIntervalBytes, actualConfig.logIndexIntervalBytes)
Assert.assertEquals(expectedConfig.logFlushIntervalMessages, actualConfig.logFlushIntervalMessages)
Assert.assertEquals(expectedConfig.logDeleteDelayMs, actualConfig.logDeleteDelayMs)
Assert.assertEquals(expectedConfig.logFlushSchedulerIntervalMs, actualConfig.logFlushSchedulerIntervalMs)
Assert.assertEquals(expectedConfig.logFlushIntervalMs, actualConfig.logFlushIntervalMs)
Assert.assertEquals(expectedConfig.logFlushOffsetCheckpointIntervalMs, actualConfig.logFlushOffsetCheckpointIntervalMs)
Assert.assertEquals(expectedConfig.numRecoveryThreadsPerDataDir, actualConfig.numRecoveryThreadsPerDataDir)
Assert.assertEquals(expectedConfig.autoCreateTopicsEnable, actualConfig.autoCreateTopicsEnable)
Assert.assertEquals(expectedConfig.minInSyncReplicas, actualConfig.minInSyncReplicas)
Assert.assertEquals(expectedConfig.controllerSocketTimeoutMs, actualConfig.controllerSocketTimeoutMs)
Assert.assertEquals(expectedConfig.defaultReplicationFactor, actualConfig.defaultReplicationFactor)
Assert.assertEquals(expectedConfig.replicaLagTimeMaxMs, actualConfig.replicaLagTimeMaxMs)
Assert.assertEquals(expectedConfig.replicaSocketTimeoutMs, actualConfig.replicaSocketTimeoutMs)
Assert.assertEquals(expectedConfig.replicaSocketReceiveBufferBytes, actualConfig.replicaSocketReceiveBufferBytes)
Assert.assertEquals(expectedConfig.replicaFetchMaxBytes, actualConfig.replicaFetchMaxBytes)
Assert.assertEquals(expectedConfig.replicaFetchWaitMaxMs, actualConfig.replicaFetchWaitMaxMs)
Assert.assertEquals(expectedConfig.replicaFetchMinBytes, actualConfig.replicaFetchMinBytes)
Assert.assertEquals(expectedConfig.replicaFetchBackoffMs, actualConfig.replicaFetchBackoffMs)
Assert.assertEquals(expectedConfig.numReplicaFetchers, actualConfig.numReplicaFetchers)
Assert.assertEquals(expectedConfig.replicaHighWatermarkCheckpointIntervalMs, actualConfig.replicaHighWatermarkCheckpointIntervalMs)
Assert.assertEquals(expectedConfig.fetchPurgatoryPurgeIntervalRequests, actualConfig.fetchPurgatoryPurgeIntervalRequests)
Assert.assertEquals(expectedConfig.producerPurgatoryPurgeIntervalRequests, actualConfig.producerPurgatoryPurgeIntervalRequests)
Assert.assertEquals(expectedConfig.autoLeaderRebalanceEnable, actualConfig.autoLeaderRebalanceEnable)
Assert.assertEquals(expectedConfig.leaderImbalancePerBrokerPercentage, actualConfig.leaderImbalancePerBrokerPercentage)
Assert.assertEquals(expectedConfig.leaderImbalanceCheckIntervalSeconds, actualConfig.leaderImbalanceCheckIntervalSeconds)
Assert.assertEquals(expectedConfig.uncleanLeaderElectionEnable, actualConfig.uncleanLeaderElectionEnable)
Assert.assertEquals(expectedConfig.controlledShutdownMaxRetries, actualConfig.controlledShutdownMaxRetries)
Assert.assertEquals(expectedConfig.controlledShutdownRetryBackoffMs, actualConfig.controlledShutdownRetryBackoffMs)
Assert.assertEquals(expectedConfig.controlledShutdownEnable, actualConfig.controlledShutdownEnable)
Assert.assertEquals(expectedConfig.consumerMinSessionTimeoutMs, actualConfig.consumerMinSessionTimeoutMs)
Assert.assertEquals(expectedConfig.consumerMaxSessionTimeoutMs, actualConfig.consumerMaxSessionTimeoutMs)
Assert.assertEquals(expectedConfig.offsetMetadataMaxSize, actualConfig.offsetMetadataMaxSize)
Assert.assertEquals(expectedConfig.offsetsLoadBufferSize, actualConfig.offsetsLoadBufferSize)
Assert.assertEquals(expectedConfig.offsetsTopicReplicationFactor, actualConfig.offsetsTopicReplicationFactor)
Assert.assertEquals(expectedConfig.offsetsTopicPartitions, actualConfig.offsetsTopicPartitions)
Assert.assertEquals(expectedConfig.offsetsTopicSegmentBytes, actualConfig.offsetsTopicSegmentBytes)
Assert.assertEquals(expectedConfig.offsetsTopicCompressionCodec, actualConfig.offsetsTopicCompressionCodec)
Assert.assertEquals(expectedConfig.offsetsRetentionMinutes, actualConfig.offsetsRetentionMinutes)
Assert.assertEquals(expectedConfig.offsetsRetentionCheckIntervalMs, actualConfig.offsetsRetentionCheckIntervalMs)
Assert.assertEquals(expectedConfig.offsetCommitTimeoutMs, actualConfig.offsetCommitTimeoutMs)
Assert.assertEquals(expectedConfig.offsetCommitRequiredAcks, actualConfig.offsetCommitRequiredAcks)
Assert.assertEquals(expectedConfig.deleteTopicEnable, actualConfig.deleteTopicEnable)
Assert.assertEquals(expectedConfig.compressionType, actualConfig.compressionType)
}
private def atLeastXIntProp(x: Int): String = (nextInt(Int.MaxValue - x) + x).toString
private def atLeastOneIntProp: String = atLeastXIntProp(1)
private def inRangeIntProp(fromInc: Int, toInc: Int): String = (nextInt(toInc + 1 - fromInc) + fromInc).toString
@Test
def testFromPropsToProps() {
import scala.util.Random._
val expected = new Properties()
KafkaConfig.configNames().foreach(name => {
name match {
case KafkaConfig.ZkConnectProp => expected.setProperty(name, "127.0.0.1:2181")
case KafkaConfig.ZkSessionTimeoutMsProp => expected.setProperty(name, atLeastOneIntProp)
case KafkaConfig.ZkConnectionTimeoutMsProp => expected.setProperty(name, atLeastOneIntProp)
case KafkaConfig.ZkSyncTimeMsProp => expected.setProperty(name, atLeastOneIntProp)
case KafkaConfig.NumNetworkThreadsProp => expected.setProperty(name, atLeastOneIntProp)
case KafkaConfig.NumIoThreadsProp => expected.setProperty(name, atLeastOneIntProp)
case KafkaConfig.BackgroundThreadsProp => expected.setProperty(name, atLeastOneIntProp)
case KafkaConfig.QueuedMaxRequestsProp => expected.setProperty(name, atLeastOneIntProp)
case KafkaConfig.PortProp => expected.setProperty(name, "1234")
case KafkaConfig.HostNameProp => expected.setProperty(name, "hostname")
case KafkaConfig.ListenersProp => expected.setProperty(name, "PLAINTEXT://:9092")
case KafkaConfig.AdvertisedHostNameProp => expected.setProperty(name, "advertised.hostname")
case KafkaConfig.AdvertisedPortProp => expected.setProperty(name, "4321")
case KafkaConfig.AdvertisedListenersProp => expected.setProperty(name, "PLAINTEXT://:2909")
case KafkaConfig.SocketRequestMaxBytesProp => expected.setProperty(name, atLeastOneIntProp)
case KafkaConfig.MaxConnectionsPerIpProp => expected.setProperty(name, atLeastOneIntProp)
case KafkaConfig.MaxConnectionsPerIpOverridesProp => expected.setProperty(name, "127.0.0.1:2, 127.0.0.2:3")
case KafkaConfig.NumPartitionsProp => expected.setProperty(name, "2")
case KafkaConfig.LogDirsProp => expected.setProperty(name, "/tmp/logs,/tmp/logs2")
case KafkaConfig.LogDirProp => expected.setProperty(name, "/tmp/log")
case KafkaConfig.LogSegmentBytesProp => expected.setProperty(name, atLeastXIntProp(Message.MinHeaderSize))
case KafkaConfig.LogRollTimeMillisProp => expected.setProperty(name, atLeastOneIntProp)
case KafkaConfig.LogRollTimeHoursProp => expected.setProperty(name, atLeastOneIntProp)
case KafkaConfig.LogRetentionTimeMillisProp => expected.setProperty(name, atLeastOneIntProp)
case KafkaConfig.LogRetentionTimeMinutesProp => expected.setProperty(name, atLeastOneIntProp)
case KafkaConfig.LogRetentionTimeHoursProp => expected.setProperty(name, atLeastOneIntProp)
case KafkaConfig.LogCleanupIntervalMsProp => expected.setProperty(name, atLeastOneIntProp)
case KafkaConfig.LogCleanupPolicyProp => expected.setProperty(name, randFrom(Defaults.Compact, Defaults.Delete))
case KafkaConfig.LogCleanerIoMaxBytesPerSecondProp => expected.setProperty(name, "%.1f".format(nextDouble * .9 + .1))
case KafkaConfig.LogCleanerDedupeBufferLoadFactorProp => expected.setProperty(name, "%.1f".format(nextDouble * .9 + .1))
case KafkaConfig.LogCleanerMinCleanRatioProp => expected.setProperty(name, "%.1f".format(nextDouble * .9 + .1))
case KafkaConfig.LogCleanerEnableProp => expected.setProperty(name, randFrom("true", "false"))
case KafkaConfig.LogIndexSizeMaxBytesProp => expected.setProperty(name, atLeastXIntProp(4))
case KafkaConfig.LogFlushIntervalMessagesProp => expected.setProperty(name, atLeastOneIntProp)
case KafkaConfig.NumRecoveryThreadsPerDataDirProp => expected.setProperty(name, atLeastOneIntProp)
case KafkaConfig.AutoCreateTopicsEnableProp => expected.setProperty(name, randFrom("true", "false"))
case KafkaConfig.MinInSyncReplicasProp => expected.setProperty(name, atLeastOneIntProp)
case KafkaConfig.AutoLeaderRebalanceEnableProp => expected.setProperty(name, randFrom("true", "false"))
case KafkaConfig.UncleanLeaderElectionEnableProp => expected.setProperty(name, randFrom("true", "false"))
case KafkaConfig.LogPreAllocateProp => expected.setProperty(name, randFrom("true", "false"))
case KafkaConfig.InterBrokerSecurityProtocolProp => expected.setProperty(name, SecurityProtocol.PLAINTEXT.toString)
case KafkaConfig.InterBrokerProtocolVersionProp => expected.setProperty(name, ApiVersion.latestVersion.toString)
case KafkaConfig.ControlledShutdownEnableProp => expected.setProperty(name, randFrom("true", "false"))
case KafkaConfig.OffsetsLoadBufferSizeProp => expected.setProperty(name, atLeastOneIntProp)
case KafkaConfig.OffsetsTopicPartitionsProp => expected.setProperty(name, atLeastOneIntProp)
case KafkaConfig.OffsetsTopicSegmentBytesProp => expected.setProperty(name, atLeastOneIntProp)
case KafkaConfig.OffsetsTopicCompressionCodecProp => expected.setProperty(name, randFrom(GZIPCompressionCodec.codec.toString,
SnappyCompressionCodec.codec.toString, LZ4CompressionCodec.codec.toString, NoCompressionCodec.codec.toString))
case KafkaConfig.OffsetsRetentionMinutesProp => expected.setProperty(name, atLeastOneIntProp)
case KafkaConfig.OffsetsRetentionCheckIntervalMsProp => expected.setProperty(name, atLeastOneIntProp)
case KafkaConfig.OffsetCommitTimeoutMsProp => expected.setProperty(name, atLeastOneIntProp)
case KafkaConfig.DeleteTopicEnableProp => expected.setProperty(name, randFrom("true", "false"))
// explicit, non trivial validations or with transient dependencies
// require(brokerId >= -1 && brokerId <= maxReservedBrokerId)
case KafkaConfig.MaxReservedBrokerIdProp => expected.setProperty(name, "100")
case KafkaConfig.BrokerIdProp => expected.setProperty(name, inRangeIntProp(0, 100))
// require(logCleanerDedupeBufferSize / logCleanerThreads > 1024 * 1024)
case KafkaConfig.LogCleanerThreadsProp => expected.setProperty(name, "2")
case KafkaConfig.LogCleanerDedupeBufferSizeProp => expected.setProperty(name, (1024 * 1024 * 3 + 1).toString)
// require(replicaFetchWaitMaxMs <= replicaSocketTimeoutMs)
case KafkaConfig.ReplicaFetchWaitMaxMsProp => expected.setProperty(name, "321")
case KafkaConfig.ReplicaSocketTimeoutMsProp => expected.setProperty(name, atLeastXIntProp(321))
// require(replicaFetchMaxBytes >= messageMaxBytes)
case KafkaConfig.MessageMaxBytesProp => expected.setProperty(name, "1234")
case KafkaConfig.ReplicaFetchMaxBytesProp => expected.setProperty(name, atLeastXIntProp(1234))
// require(replicaFetchWaitMaxMs <= replicaLagTimeMaxMs)
case KafkaConfig.ReplicaLagTimeMaxMsProp => expected.setProperty(name, atLeastXIntProp(321))
//require(offsetCommitRequiredAcks >= -1 && offsetCommitRequiredAcks <= offsetsTopicReplicationFactor)
case KafkaConfig.OffsetCommitRequiredAcksProp => expected.setProperty(name, "-1")
case KafkaConfig.OffsetsTopicReplicationFactorProp => expected.setProperty(name, inRangeIntProp(1, Short.MaxValue))
//BrokerCompressionCodec.isValid(compressionType)
case KafkaConfig.CompressionTypeProp => expected.setProperty(name, randFrom(BrokerCompressionCodec.brokerCompressionOptions))
case KafkaConfig.MetricNumSamplesProp => expected.setProperty(name, "2")
case KafkaConfig.MetricSampleWindowMsProp => expected.setProperty(name, "1000")
case KafkaConfig.MetricReporterClassesProp => expected.setProperty(name, "")
case nonNegativeIntProperty => expected.setProperty(name, nextInt(Int.MaxValue).toString)
}
})
val actual = KafkaConfig.fromProps(expected).originals
Assert.assertEquals(expected, actual)
}
@Test
def testFromPropsInvalid() {
def getBaseProperties(): Properties = {
val validRequiredProperties = new Properties()
validRequiredProperties.put(KafkaConfig.ZkConnectProp, "127.0.0.1")
validRequiredProperties
}
// to ensure a basis is valid - bootstraps all needed validation
KafkaConfig.fromProps(getBaseProperties())
KafkaConfig.configNames().foreach(name => {
name match {
case KafkaConfig.ZkConnectProp => // ignore string
case KafkaConfig.ZkSessionTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.ZkConnectionTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.ZkSyncTimeMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.BrokerIdProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.NumNetworkThreadsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.NumIoThreadsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.BackgroundThreadsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.QueuedMaxRequestsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.PortProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.HostNameProp => // ignore string
case KafkaConfig.AdvertisedHostNameProp => //ignore string
case KafkaConfig.AdvertisedPortProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.SocketSendBufferBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.SocketReceiveBufferBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.MaxConnectionsPerIpOverridesProp =>
assertPropertyInvalid(getBaseProperties(), name, "127.0.0.1:not_a_number")
case KafkaConfig.ConnectionsMaxIdleMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.NumPartitionsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.LogDirsProp => // ignore string
case KafkaConfig.LogDirProp => // ignore string
case KafkaConfig.LogSegmentBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", Message.MinHeaderSize - 1)
case KafkaConfig.LogRollTimeMillisProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.LogRollTimeHoursProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.LogRetentionTimeMillisProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.LogRetentionTimeMinutesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.LogRetentionTimeHoursProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.LogRetentionBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.LogCleanupIntervalMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.LogCleanupPolicyProp => assertPropertyInvalid(getBaseProperties(), name, "unknown_policy", "0")
case KafkaConfig.LogCleanerIoMaxBytesPerSecondProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.LogCleanerDedupeBufferSizeProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "1024")
case KafkaConfig.LogCleanerDedupeBufferLoadFactorProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.LogCleanerEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean")
case KafkaConfig.LogCleanerDeleteRetentionMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.LogCleanerMinCleanRatioProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.LogIndexSizeMaxBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "3")
case KafkaConfig.LogFlushIntervalMessagesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.LogFlushSchedulerIntervalMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.LogFlushIntervalMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.NumRecoveryThreadsPerDataDirProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.AutoCreateTopicsEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean", "0")
case KafkaConfig.MinInSyncReplicasProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.ControllerSocketTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.DefaultReplicationFactorProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.ReplicaLagTimeMaxMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.ReplicaSocketTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "-2")
case KafkaConfig.ReplicaSocketReceiveBufferBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.ReplicaFetchMaxBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.ReplicaFetchWaitMaxMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.ReplicaFetchMinBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.NumReplicaFetchersProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.ReplicaHighWatermarkCheckpointIntervalMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.FetchPurgatoryPurgeIntervalRequestsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.ProducerPurgatoryPurgeIntervalRequestsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.AutoLeaderRebalanceEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean", "0")
case KafkaConfig.LeaderImbalancePerBrokerPercentageProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.LeaderImbalanceCheckIntervalSecondsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.UncleanLeaderElectionEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean", "0")
case KafkaConfig.ControlledShutdownMaxRetriesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.ControlledShutdownRetryBackoffMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.ControlledShutdownEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean", "0")
case KafkaConfig.ConsumerMinSessionTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.ConsumerMaxSessionTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.OffsetMetadataMaxSizeProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.OffsetsLoadBufferSizeProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.OffsetsTopicReplicationFactorProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.OffsetsTopicPartitionsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.OffsetsTopicSegmentBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.OffsetsTopicCompressionCodecProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "-1")
case KafkaConfig.OffsetsRetentionMinutesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.OffsetsRetentionCheckIntervalMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.OffsetCommitTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.OffsetCommitRequiredAcksProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "-2")
case KafkaConfig.DeleteTopicEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean", "0")
case KafkaConfig.MetricNumSamplesProp => assertPropertyInvalid(getBaseProperties, name, "not_a_number", "-1", "0")
case KafkaConfig.MetricSampleWindowMsProp => assertPropertyInvalid(getBaseProperties, name, "not_a_number", "-1", "0")
case KafkaConfig.MetricReporterClassesProp => // ignore string
case nonNegativeIntProperty => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "-1")
}
})
}
@Test
def testSpecificProperties(): Unit = {
val defaults = new Properties()
defaults.put(KafkaConfig.ZkConnectProp, "127.0.0.1:2181")
// For ZkConnectionTimeoutMs
defaults.put(KafkaConfig.ZkSessionTimeoutMsProp, "1234")
defaults.put(KafkaConfig.MaxReservedBrokerIdProp, "1")
defaults.put(KafkaConfig.BrokerIdProp, "1")
defaults.put(KafkaConfig.HostNameProp, "127.0.0.1")
defaults.put(KafkaConfig.PortProp, "1122")
defaults.put(KafkaConfig.MaxConnectionsPerIpOverridesProp, "127.0.0.1:2, 127.0.0.2:3")
defaults.put(KafkaConfig.LogDirProp, "/tmp1,/tmp2")
defaults.put(KafkaConfig.LogRollTimeHoursProp, "12")
defaults.put(KafkaConfig.LogRollTimeJitterHoursProp, "11")
defaults.put(KafkaConfig.LogRetentionTimeHoursProp, "10")
//For LogFlushIntervalMsProp
defaults.put(KafkaConfig.LogFlushSchedulerIntervalMsProp, "123")
defaults.put(KafkaConfig.OffsetsTopicCompressionCodecProp, SnappyCompressionCodec.codec.toString)
val config = KafkaConfig.fromProps(defaults)
Assert.assertEquals("127.0.0.1:2181", config.zkConnect)
Assert.assertEquals(1234, config.zkConnectionTimeoutMs)
Assert.assertEquals(1, config.maxReservedBrokerId)
Assert.assertEquals(1, config.brokerId)
Assert.assertEquals("127.0.0.1", config.hostName)
Assert.assertEquals(1122, config.advertisedPort)
Assert.assertEquals("127.0.0.1", config.advertisedHostName)
Assert.assertEquals(Map("127.0.0.1" -> 2, "127.0.0.2" -> 3), config.maxConnectionsPerIpOverrides)
Assert.assertEquals(List("/tmp1", "/tmp2"), config.logDirs)
Assert.assertEquals(12 * 60L * 1000L * 60, config.logRollTimeMillis)
Assert.assertEquals(11 * 60L * 1000L * 60, config.logRollTimeJitterMillis)
Assert.assertEquals(10 * 60L * 1000L * 60, config.logRetentionTimeMillis)
Assert.assertEquals(123L, config.logFlushIntervalMs)
Assert.assertEquals(SnappyCompressionCodec, config.offsetsTopicCompressionCodec)
}
private def assertPropertyInvalid(validRequiredProps: => Properties, name: String, values: Any*) {
values.foreach((value) => {
val props = validRequiredProps
props.setProperty(name, value.toString)
intercept[Exception] {
KafkaConfig.fromProps(props)
}
})
}
private def randFrom[T](choices: T*): T = {
import scala.util.Random
choices(Random.nextInt(choices.size))
}
private def randFrom[T](choices: List[T]): T = {
import scala.util.Random
choices(Random.nextInt(choices.size))
}
}

View File

@ -21,10 +21,11 @@ import java.util.Properties
import junit.framework.Assert._
import kafka.api.{ApiVersion, KAFKA_082}
import kafka.message._
import kafka.utils.{TestUtils, CoreUtils}
import org.apache.kafka.common.config.ConfigException
import org.apache.kafka.common.protocol.SecurityProtocol
import org.junit.Test
import org.junit.{Assert, Test}
import org.scalatest.junit.JUnit3Suite
class KafkaConfigTest extends JUnit3Suite {
@ -380,4 +381,155 @@ class KafkaConfigTest extends JUnit3Suite {
KafkaConfig.fromProps(props)
}
}
@Test
def testFromPropsInvalid() {
def getBaseProperties(): Properties = {
val validRequiredProperties = new Properties()
validRequiredProperties.put(KafkaConfig.ZkConnectProp, "127.0.0.1:2181")
validRequiredProperties
}
// to ensure a basis is valid - bootstraps all needed validation
KafkaConfig.fromProps(getBaseProperties())
KafkaConfig.configNames().foreach(name => {
name match {
case KafkaConfig.ZkConnectProp => // ignore string
case KafkaConfig.ZkSessionTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.ZkConnectionTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.ZkSyncTimeMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.BrokerIdProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.NumNetworkThreadsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.NumIoThreadsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.BackgroundThreadsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.QueuedMaxRequestsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.PortProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.HostNameProp => // ignore string
case KafkaConfig.AdvertisedHostNameProp => //ignore string
case KafkaConfig.AdvertisedPortProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.SocketSendBufferBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.SocketReceiveBufferBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.MaxConnectionsPerIpOverridesProp =>
assertPropertyInvalid(getBaseProperties(), name, "127.0.0.1:not_a_number")
case KafkaConfig.ConnectionsMaxIdleMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.NumPartitionsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.LogDirsProp => // ignore string
case KafkaConfig.LogDirProp => // ignore string
case KafkaConfig.LogSegmentBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", Message.MinHeaderSize - 1)
case KafkaConfig.LogRollTimeMillisProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.LogRollTimeHoursProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.LogRetentionTimeMillisProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.LogRetentionTimeMinutesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.LogRetentionTimeHoursProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.LogRetentionBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.LogCleanupIntervalMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.LogCleanupPolicyProp => assertPropertyInvalid(getBaseProperties(), name, "unknown_policy", "0")
case KafkaConfig.LogCleanerIoMaxBytesPerSecondProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.LogCleanerDedupeBufferSizeProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "1024")
case KafkaConfig.LogCleanerDedupeBufferLoadFactorProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.LogCleanerEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean")
case KafkaConfig.LogCleanerDeleteRetentionMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.LogCleanerMinCleanRatioProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.LogIndexSizeMaxBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "3")
case KafkaConfig.LogFlushIntervalMessagesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.LogFlushSchedulerIntervalMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.LogFlushIntervalMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.NumRecoveryThreadsPerDataDirProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.AutoCreateTopicsEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean", "0")
case KafkaConfig.MinInSyncReplicasProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.ControllerSocketTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.DefaultReplicationFactorProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.ReplicaLagTimeMaxMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.ReplicaSocketTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "-2")
case KafkaConfig.ReplicaSocketReceiveBufferBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.ReplicaFetchMaxBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.ReplicaFetchWaitMaxMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.ReplicaFetchMinBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.NumReplicaFetchersProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.ReplicaHighWatermarkCheckpointIntervalMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.FetchPurgatoryPurgeIntervalRequestsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.ProducerPurgatoryPurgeIntervalRequestsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.AutoLeaderRebalanceEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean", "0")
case KafkaConfig.LeaderImbalancePerBrokerPercentageProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.LeaderImbalanceCheckIntervalSecondsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.UncleanLeaderElectionEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean", "0")
case KafkaConfig.ControlledShutdownMaxRetriesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.ControlledShutdownRetryBackoffMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.ControlledShutdownEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean", "0")
case KafkaConfig.ConsumerMinSessionTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.ConsumerMaxSessionTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.OffsetMetadataMaxSizeProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.OffsetsLoadBufferSizeProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.OffsetsTopicReplicationFactorProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.OffsetsTopicPartitionsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.OffsetsTopicSegmentBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.OffsetsTopicCompressionCodecProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "-1")
case KafkaConfig.OffsetsRetentionMinutesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.OffsetsRetentionCheckIntervalMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.OffsetCommitTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
case KafkaConfig.OffsetCommitRequiredAcksProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "-2")
case KafkaConfig.DeleteTopicEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean", "0")
case KafkaConfig.MetricNumSamplesProp => assertPropertyInvalid(getBaseProperties, name, "not_a_number", "-1", "0")
case KafkaConfig.MetricSampleWindowMsProp => assertPropertyInvalid(getBaseProperties, name, "not_a_number", "-1", "0")
case KafkaConfig.MetricReporterClassesProp => // ignore string
case nonNegativeIntProperty => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "-1")
}
})
}
@Test
def testSpecificProperties(): Unit = {
val defaults = new Properties()
defaults.put(KafkaConfig.ZkConnectProp, "127.0.0.1:2181")
// For ZkConnectionTimeoutMs
defaults.put(KafkaConfig.ZkSessionTimeoutMsProp, "1234")
defaults.put(KafkaConfig.MaxReservedBrokerIdProp, "1")
defaults.put(KafkaConfig.BrokerIdProp, "1")
defaults.put(KafkaConfig.HostNameProp, "127.0.0.1")
defaults.put(KafkaConfig.PortProp, "1122")
defaults.put(KafkaConfig.MaxConnectionsPerIpOverridesProp, "127.0.0.1:2, 127.0.0.2:3")
defaults.put(KafkaConfig.LogDirProp, "/tmp1,/tmp2")
defaults.put(KafkaConfig.LogRollTimeHoursProp, "12")
defaults.put(KafkaConfig.LogRollTimeJitterHoursProp, "11")
defaults.put(KafkaConfig.LogRetentionTimeHoursProp, "10")
//For LogFlushIntervalMsProp
defaults.put(KafkaConfig.LogFlushSchedulerIntervalMsProp, "123")
defaults.put(KafkaConfig.OffsetsTopicCompressionCodecProp, SnappyCompressionCodec.codec.toString)
val config = KafkaConfig.fromProps(defaults)
Assert.assertEquals("127.0.0.1:2181", config.zkConnect)
Assert.assertEquals(1234, config.zkConnectionTimeoutMs)
Assert.assertEquals(1, config.maxReservedBrokerId)
Assert.assertEquals(1, config.brokerId)
Assert.assertEquals("127.0.0.1", config.hostName)
Assert.assertEquals(1122, config.advertisedPort)
Assert.assertEquals("127.0.0.1", config.advertisedHostName)
Assert.assertEquals(Map("127.0.0.1" -> 2, "127.0.0.2" -> 3), config.maxConnectionsPerIpOverrides)
Assert.assertEquals(List("/tmp1", "/tmp2"), config.logDirs)
Assert.assertEquals(12 * 60L * 1000L * 60, config.logRollTimeMillis)
Assert.assertEquals(11 * 60L * 1000L * 60, config.logRollTimeJitterMillis)
Assert.assertEquals(10 * 60L * 1000L * 60, config.logRetentionTimeMillis)
Assert.assertEquals(123L, config.logFlushIntervalMs)
Assert.assertEquals(SnappyCompressionCodec, config.offsetsTopicCompressionCodec)
}
private def assertPropertyInvalid(validRequiredProps: => Properties, name: String, values: Any*) {
values.foreach((value) => {
val props = validRequiredProps
props.setProperty(name, value.toString)
intercept[Exception] {
KafkaConfig.fromProps(props)
}
})
}
}

View File

@ -210,6 +210,9 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness {
sendMessages(2)
hw += 2
// allow some time for the follower to create replica
TestUtils.waitUntilTrue(() => server1.replicaManager.getReplica(topic, 0).nonEmpty,
"Failed to create replica in follower after timeout")
// allow some time for the follower to get the leader HW
TestUtils.waitUntilTrue(() =>
server1.replicaManager.getReplica(topic, 0).get.highWatermark.messageOffset == hw,

View File

@ -380,22 +380,24 @@ def main():
url = pr["url"]
pr_title = pr["title"]
commit_title = raw_input("Commit title [%s]: " % pr_title.encode("utf-8")).decode("utf-8")
if commit_title == "":
commit_title = pr_title
# Decide whether to use the modified title or not
modified_title = standardize_jira_ref(pr["title"])
if modified_title != pr["title"]:
modified_title = standardize_jira_ref(commit_title)
if modified_title != commit_title:
print "I've re-written the title as follows to match the standard format:"
print "Original: %s" % pr["title"]
print "Original: %s" % commit_title
print "Modified: %s" % modified_title
result = raw_input("Would you like to use the modified title? (y/n): ")
if result.lower() == "y":
title = modified_title
commit_title = modified_title
print "Using modified title:"
else:
title = pr["title"]
print "Using original title:"
print title
else:
title = pr["title"]
print commit_title
body = pr["body"]
target_ref = pr["base"]["ref"]
@ -428,13 +430,13 @@ def main():
continue_maybe(msg)
print ("\n=== Pull Request #%s ===" % pr_num)
print ("title\t%s\nsource\t%s\ntarget\t%s\nurl\t%s" % (
title, pr_repo_desc, target_ref, url))
print ("PR title\t%s\nCommit title\t%s\nSource\t\t%s\nTarget\t\t%s\nURL\t\t%s" % (
pr_title, commit_title, pr_repo_desc, target_ref, url))
continue_maybe("Proceed with merging pull request #%s?" % pr_num)
merged_refs = [target_ref]
merge_hash = merge_pr(pr_num, target_ref, title, body, pr_repo_desc)
merge_hash = merge_pr(pr_num, target_ref, commit_title, body, pr_repo_desc)
pick_prompt = "Would you like to pick %s into another branch?" % merge_hash
while raw_input("\n%s (y/n): " % pick_prompt).lower() == "y":
@ -444,7 +446,7 @@ def main():
if JIRA_USERNAME and JIRA_PASSWORD:
continue_maybe("Would you like to update an associated JIRA?")
jira_comment = "Issue resolved by pull request %s\n[%s/%s]" % (pr_num, GITHUB_BASE, pr_num)
resolve_jira_issues(title, merged_refs, jira_comment)
resolve_jira_issues(commit_title, merged_refs, jira_comment)
else:
print "JIRA_USERNAME and JIRA_PASSWORD not set"
print "Exiting without trying to close the associated JIRA."

View File

@ -0,0 +1,14 @@
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.

View File

@ -0,0 +1,80 @@
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
from ducktape.tests.test import Test
from ducktape.utils.util import wait_until
from kafkatest.services.zookeeper import ZookeeperService
from kafkatest.services.kafka import KafkaService
from kafkatest.services.console_consumer import ConsoleConsumer
import time
def file_exists(node, file):
"""Quick and dirty check for existence of remote file."""
try:
node.account.ssh("cat " + file, allow_fail=False)
return True
except:
return False
def line_count(node, file):
"""Return the line count of file on node"""
out = [line for line in node.account.ssh_capture("wc -l %s" % file)]
if len(out) != 1:
raise Exception("Expected single line of output from wc -l")
return int(out[0].strip().split(" ")[0])
class ConsoleConsumerTest(Test):
"""Sanity checks on console consumer service class."""
def __init__(self, test_context):
super(ConsoleConsumerTest, self).__init__(test_context)
self.topic = "topic"
self.zk = ZookeeperService(test_context, num_nodes=1)
self.kafka = KafkaService(test_context, num_nodes=1, zk=self.zk,
topics={self.topic: {"partitions": 1, "replication-factor": 1}})
self.consumer = ConsoleConsumer(test_context, num_nodes=1, kafka=self.kafka, topic=self.topic)
def setUp(self):
self.zk.start()
self.kafka.start()
def test_lifecycle(self):
t0 = time.time()
self.consumer.start()
node = self.consumer.nodes[0]
if not wait_until(lambda: self.consumer.alive(node), timeout_sec=10, backoff_sec=.2):
raise Exception("Consumer was too slow to start")
self.logger.info("consumer started in %s seconds " % str(time.time() - t0))
# Verify that log output is happening
if not wait_until(lambda: file_exists(node, ConsoleConsumer.LOG_FILE), timeout_sec=10):
raise Exception("Timed out waiting for log file to exist")
assert line_count(node, ConsoleConsumer.LOG_FILE) > 0
# Verify no consumed messages
assert line_count(node, ConsoleConsumer.STDOUT_CAPTURE) == 0
self.consumer.stop_node(node)
if not wait_until(lambda: not self.consumer.alive(node), timeout_sec=10, backoff_sec=.2):
raise Exception("Took too long for consumer to die.")

View File

@ -15,6 +15,8 @@
from ducktape.services.background_thread import BackgroundThreadService
import os
def is_int(msg):
"""Default method used to check whether text pulled from console consumer is a message.
@ -26,7 +28,6 @@ def is_int(msg):
except:
return None
"""
0.8.2.1 ConsoleConsumer options
@ -69,9 +70,24 @@ Option Description
class ConsoleConsumer(BackgroundThreadService):
# Root directory for persistent output
PERSISTENT_ROOT = "/mnt/console_consumer"
STDOUT_CAPTURE = os.path.join(PERSISTENT_ROOT, "console_consumer.stdout")
STDERR_CAPTURE = os.path.join(PERSISTENT_ROOT, "console_consumer.stderr")
LOG_DIR = os.path.join(PERSISTENT_ROOT, "logs")
LOG_FILE = os.path.join(LOG_DIR, "console_consumer.log")
LOG4J_CONFIG = os.path.join(PERSISTENT_ROOT, "tools-log4j.properties")
CONFIG_FILE = os.path.join(PERSISTENT_ROOT, "console_consumer.properties")
logs = {
"consumer_stdout": {
"path": STDOUT_CAPTURE,
"collect_default": False},
"consumer_stderr": {
"path": STDERR_CAPTURE,
"collect_default": False},
"consumer_log": {
"path": "/mnt/consumer.log",
"path": LOG_FILE,
"collect_default": True}
}
@ -104,18 +120,37 @@ class ConsoleConsumer(BackgroundThreadService):
@property
def start_cmd(self):
args = self.args.copy()
args.update({'zk_connect': self.kafka.zk.connect_setting()})
cmd = "/opt/kafka/bin/kafka-console-consumer.sh --topic %(topic)s --zookeeper %(zk_connect)s" \
" --consumer.config /mnt/console_consumer.properties" % args
args['zk_connect'] = self.kafka.zk.connect_setting()
args['stdout'] = ConsoleConsumer.STDOUT_CAPTURE
args['stderr'] = ConsoleConsumer.STDERR_CAPTURE
args['config_file'] = ConsoleConsumer.CONFIG_FILE
cmd = "export LOG_DIR=%s;" % ConsoleConsumer.LOG_DIR
cmd += " export KAFKA_LOG4J_OPTS=\"-Dlog4j.configuration=file:%s\";" % ConsoleConsumer.LOG4J_CONFIG
cmd += " /opt/kafka/bin/kafka-console-consumer.sh --topic %(topic)s --zookeeper %(zk_connect)s" \
" --consumer.config %(config_file)s" % args
if self.from_beginning:
cmd += " --from-beginning"
cmd += " 2>> /mnt/consumer.log | tee -a /mnt/consumer.log &"
cmd += " 2>> %(stderr)s | tee -a %(stdout)s &" % args
return cmd
def pids(self, node):
try:
cmd = "ps ax | grep -i console_consumer | grep java | grep -v grep | awk '{print $1}'"
pid_arr = [pid for pid in node.account.ssh_capture(cmd, allow_fail=True, callback=int)]
return pid_arr
except:
return []
def alive(self, node):
return len(self.pids(node)) > 0
def _worker(self, idx, node):
# form config file
node.account.ssh("mkdir -p %s" % ConsoleConsumer.PERSISTENT_ROOT, allow_fail=False)
# Create and upload config file
if self.consumer_timeout_ms is not None:
prop_file = self.render('console_consumer.properties', consumer_timeout_ms=self.consumer_timeout_ms)
else:
@ -123,12 +158,16 @@ class ConsoleConsumer(BackgroundThreadService):
self.logger.info("console_consumer.properties:")
self.logger.info(prop_file)
node.account.create_file("/mnt/console_consumer.properties", prop_file)
node.account.create_file(ConsoleConsumer.CONFIG_FILE, prop_file)
# Create and upload log properties
log_config = self.render('console_consumer_log4j.properties', log_file=ConsoleConsumer.LOG_FILE)
node.account.create_file(ConsoleConsumer.LOG4J_CONFIG, log_config)
# Run and capture output
cmd = self.start_cmd
self.logger.debug("Console consumer %d command: %s", idx, cmd)
for line in node.account.ssh_capture(cmd):
for line in node.account.ssh_capture(cmd, allow_fail=False):
msg = line.strip()
msg = self.message_validator(msg)
if msg is not None:
@ -139,8 +178,8 @@ class ConsoleConsumer(BackgroundThreadService):
super(ConsoleConsumer, self).start_node(node)
def stop_node(self, node):
node.account.kill_process("java", allow_fail=False)
node.account.kill_process("java", allow_fail=True)
def clean_node(self, node):
node.account.ssh("rm -rf /mnt/console_consumer.properties /mnt/consumer.log", allow_fail=False)
node.account.ssh("rm -rf %s" % ConsoleConsumer.PERSISTENT_ROOT, allow_fail=False)

View File

@ -14,6 +14,6 @@
# limitations under the License.
# see kafka.server.KafkaConfig for additional details and defaults
{% if consumer_timeout_ms is defined %}
{% if consumer_timeout_ms is not none %}
consumer.timeout.ms={{ consumer_timeout_ms }}
{% endif %}

View File

@ -0,0 +1,26 @@
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
# Define the root logger with appender file
log4j.rootLogger = INFO, FILE
log4j.appender.FILE=org.apache.log4j.FileAppender
log4j.appender.FILE.File={{ log_file }}
log4j.appender.FILE.ImmediateFlush=true
log4j.appender.FILE.Threshold=debug
# Set the append to false, overwrite
log4j.appender.FILE.Append=false
log4j.appender.FILE.layout=org.apache.log4j.PatternLayout
log4j.appender.FILE.layout.conversionPattern=[%d] %p %m (%c)%n