KAFKA-348 merge trunk to branch 1239902:1310937 patch by Joe Stein reviewed by Jun Rao

git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/branches/0.8@1344526 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Joe Stein 2012-05-31 01:51:23 +00:00
parent 904708fae8
commit 822963bb9d
107 changed files with 2466 additions and 1225 deletions

View File

@ -16,7 +16,7 @@
*/ */
/* /*
* encoder.hpp * encoder.hpp
* */
#ifndef KAFKA_ENCODER_HPP_ #ifndef KAFKA_ENCODER_HPP_
#define KAFKA_ENCODER_HPP_ #define KAFKA_ENCODER_HPP_

View File

@ -16,6 +16,11 @@
*/ */
package kafka.etl; package kafka.etl;
import java.io.IOException;
import java.net.URI;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Iterator;
import kafka.api.FetchRequest; import kafka.api.FetchRequest;
import kafka.api.FetchRequestBuilder; import kafka.api.FetchRequestBuilder;
import kafka.api.OffsetRequest; import kafka.api.OffsetRequest;
@ -23,7 +28,6 @@ import kafka.common.ErrorMapping;
import kafka.javaapi.FetchResponse; import kafka.javaapi.FetchResponse;
import kafka.javaapi.consumer.SimpleConsumer; import kafka.javaapi.consumer.SimpleConsumer;
import kafka.javaapi.message.ByteBufferMessageSet; import kafka.javaapi.message.ByteBufferMessageSet;
import kafka.javaapi.message.MessageSet;
import kafka.message.MessageAndOffset; import kafka.message.MessageAndOffset;
import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.JobConf;
@ -31,12 +35,6 @@ import org.apache.hadoop.mapred.OutputCollector;
import org.apache.hadoop.mapred.Reporter; import org.apache.hadoop.mapred.Reporter;
import org.apache.hadoop.mapred.lib.MultipleOutputs; import org.apache.hadoop.mapred.lib.MultipleOutputs;
import java.io.IOException;
import java.net.URI;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Iterator;
@SuppressWarnings({ "deprecation"}) @SuppressWarnings({ "deprecation"})
public class KafkaETLContext { public class KafkaETLContext {
@ -139,7 +137,7 @@ public class KafkaETLContext {
while ( !gotNext && _respIterator.hasNext()) { while ( !gotNext && _respIterator.hasNext()) {
ByteBufferMessageSet msgSet = _respIterator.next(); ByteBufferMessageSet msgSet = _respIterator.next();
if ( hasError(msgSet)) return false; if ( hasError(msgSet)) return false;
_messageIt = (Iterator<MessageAndOffset>) msgSet.iterator(); _messageIt = msgSet.iterator();
gotNext = get(key, value); gotNext = get(key, value);
} }
} }
@ -194,17 +192,17 @@ public class KafkaETLContext {
protected boolean get(KafkaETLKey key, BytesWritable value) throws IOException { protected boolean get(KafkaETLKey key, BytesWritable value) throws IOException {
if (_messageIt != null && _messageIt.hasNext()) { if (_messageIt != null && _messageIt.hasNext()) {
MessageAndOffset msgAndOffset = _messageIt.next(); MessageAndOffset messageAndOffset = _messageIt.next();
ByteBuffer buf = msgAndOffset.message().payload(); ByteBuffer buf = messageAndOffset.message().payload();
int origSize = buf.remaining(); int origSize = buf.remaining();
byte[] bytes = new byte[origSize]; byte[] bytes = new byte[origSize];
buf.get(bytes, buf.position(), origSize); buf.get(bytes, buf.position(), origSize);
value.set(bytes, 0, origSize); value.set(bytes, 0, origSize);
key.set(_index, _offset, msgAndOffset.message().checksum()); key.set(_index, _offset, messageAndOffset.message().checksum());
_offset = msgAndOffset.offset(); //increase offset _offset = messageAndOffset.offset(); //increase offset
_count ++; //increase count _count ++; //increase count
return true; return true;

View File

@ -16,6 +16,7 @@
*/ */
package kafka.etl; package kafka.etl;
import java.io.IOException; import java.io.IOException;
import java.net.URI; import java.net.URI;
import java.util.Map; import java.util.Map;
@ -23,13 +24,13 @@ import kafka.consumer.SimpleConsumer;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.mapred.OutputCollector;
import org.apache.hadoop.mapred.lib.MultipleOutputs;
import org.apache.hadoop.mapred.InputSplit; import org.apache.hadoop.mapred.InputSplit;
import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.OutputCollector;
import org.apache.hadoop.mapred.RecordReader; import org.apache.hadoop.mapred.RecordReader;
import org.apache.hadoop.mapred.Reporter; import org.apache.hadoop.mapred.Reporter;
import org.apache.hadoop.mapred.SequenceFileInputFormat; import org.apache.hadoop.mapred.SequenceFileInputFormat;
import org.apache.hadoop.mapred.lib.MultipleOutputs;
@SuppressWarnings("deprecation") @SuppressWarnings("deprecation")

View File

@ -16,13 +16,13 @@
*/ */
package kafka.etl; package kafka.etl;
import java.net.URI; import java.net.URI;
import org.apache.hadoop.filecache.DistributedCache; import org.apache.hadoop.filecache.DistributedCache;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.SequenceFileOutputFormat; import org.apache.hadoop.mapred.SequenceFileOutputFormat;
import org.apache.hadoop.mapred.lib.MultipleOutputs; import org.apache.hadoop.mapred.lib.MultipleOutputs;

View File

@ -16,11 +16,11 @@
*/ */
package kafka.etl; package kafka.etl;
import java.io.DataInput; import java.io.DataInput;
import java.io.DataOutput; import java.io.DataOutput;
import java.io.IOException; import java.io.IOException;
import org.apache.hadoop.io.WritableComparable; import org.apache.hadoop.io.WritableComparable;
import kafka.etl.KafkaETLKey;
public class KafkaETLKey implements WritableComparable<KafkaETLKey>{ public class KafkaETLKey implements WritableComparable<KafkaETLKey>{

View File

@ -17,6 +17,7 @@
package kafka.etl; package kafka.etl;
import java.io.BufferedReader; import java.io.BufferedReader;
import java.io.ByteArrayInputStream; import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream; import java.io.ByteArrayOutputStream;
@ -33,7 +34,6 @@ import java.util.Arrays;
import java.util.Enumeration; import java.util.Enumeration;
import java.util.List; import java.util.List;
import java.util.Properties; import java.util.Properties;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;

View File

@ -17,33 +17,25 @@
package kafka.etl.impl; package kafka.etl.impl;
import java.io.IOException;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Iterator;
import java.util.List;
import java.util.Random;
import java.util.Map.Entry;
import java.util.Properties;
import kafka.message.NoCompressionCodec; import java.net.URI;
import java.util.ArrayList;
import java.util.List;
import java.util.Properties;
import java.util.Random;
import kafka.etl.KafkaETLKey;
import kafka.etl.KafkaETLRequest;
import kafka.etl.Props;
import kafka.javaapi.message.ByteBufferMessageSet;
import kafka.javaapi.producer.SyncProducer;
import kafka.message.Message;
import kafka.producer.SyncProducerConfig;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.JobConf;
import kafka.etl.KafkaETLKey;
import kafka.etl.KafkaETLRequest;
import kafka.etl.KafkaETLUtils;
import kafka.etl.Props;
import kafka.javaapi.message.ByteBufferMessageSet;
import kafka.message.Message;
import kafka.javaapi.producer.SyncProducer;
import kafka.producer.SyncProducerConfig;
/** /**
* Use this class to produce test events to Kafka server. Each event contains a * Use this class to produce test events to Kafka server. Each event contains a
* random timestamp in text format. * random timestamp in text format.

View File

@ -1,6 +1,14 @@
Hadoop to Kafka Bridge Hadoop to Kafka Bridge
====================== ======================
What's new?
-----------
* Now supports Kafka's software load balancer (Kafka URIs are specified with
kafka+zk as the scheme, as described below)
* Supports Kafka 0.7. Now uses the new Producer API, rather than the legacy
SyncProducer.
What is it? What is it?
----------- -----------
@ -17,8 +25,10 @@ multiple times in the same push.
How do I use it? How do I use it?
---------------- ----------------
With this bridge, Kafka topics are URIs and are specified as With this bridge, Kafka topics are URIs and are specified in one of two
`kafka://<kafka-server>/<kafka-topic>`. formats: `kafka+zk://<zk-path>#<kafka-topic>`, which uses the software load
balancer, or the legacy `kafka://<kafka-server>/<kafka-topic>` to connect to a
specific Kafka broker.
### Pig ### ### Pig ###
@ -27,17 +37,19 @@ row. To push data via Kafka, store to the Kafka URI using `AvroKafkaStorage`
with the Avro schema as its first argument. You'll need to register the with the Avro schema as its first argument. You'll need to register the
appropriate Kafka JARs. Here is what an example Pig script looks like: appropriate Kafka JARs. Here is what an example Pig script looks like:
REGISTER hadoop-kafka-bridge-0.5.2.jar; REGISTER hadoop-producer_2.8.0-0.7.0.jar;
REGISTER avro-1.4.0.jar; REGISTER avro-1.4.0.jar;
REGISTER piggybank.jar; REGISTER piggybank.jar;
REGISTER kafka-0.5.2.jar; REGISTER kafka-0.7.0.jar;
REGISTER jackson-core-asl-1.5.5.jar; REGISTER jackson-core-asl-1.5.5.jar;
REGISTER jackson-mapper-asl-1.5.5.jar; REGISTER jackson-mapper-asl-1.5.5.jar;
REGISTER zkclient-20110412.jar;
REGISTER zookeeper-3.3.4.jar;
REGISTER scala-library.jar; REGISTER scala-library.jar;
member_info = LOAD 'member_info.tsv' as (member_id : int, name : chararray); member_info = LOAD 'member_info.tsv' as (member_id : int, name : chararray);
names = FOREACH member_info GENERATE name; names = FOREACH member_info GENERATE name;
STORE member_info INTO 'kafka://my-broker:9092/member_info' USING kafka.bridge.AvroKafkaStorage('"string"'); STORE member_info INTO 'kafka+zk://my-zookeeper:2181/kafka#member_info' USING kafka.bridge.AvroKafkaStorage('"string"');
That's it! The Pig StoreFunc makes use of AvroStorage in Piggybank to convert That's it! The Pig StoreFunc makes use of AvroStorage in Piggybank to convert
from Pig's data model to the specified Avro schema. from Pig's data model to the specified Avro schema.
@ -46,8 +58,8 @@ Further, multi-store is possible with KafkaStorage, so you can easily write to
multiple topics and brokers in the same job: multiple topics and brokers in the same job:
SPLIT member_info INTO early_adopters IF member_id < 1000, others IF member_id >= 1000; SPLIT member_info INTO early_adopters IF member_id < 1000, others IF member_id >= 1000;
STORE early_adopters INTO 'kafka://my-broker:9092/early_adopters' USING AvroKafkaStorage('$schema'); STORE early_adopters INTO 'kafka+zk://my-zookeeper:2181/kafka#early_adopters' USING AvroKafkaStorage('$schema');
STORE others INTO 'kafka://my-broker:9092/others' USING AvroKafkaStorage('$schema'); STORE others INTO 'kafka://my-broker:9092,my-broker2:9092/others' USING AvroKafkaStorage('$schema');
### KafkaOutputFormat ### ### KafkaOutputFormat ###
@ -126,9 +138,10 @@ Normally, you needn't change any of these parameters:
docs). Default is 64*1024 (64KB). docs). Default is 64*1024 (64KB).
* kafka.output.max_msgsize: Maximum message size in bytes (see Kafka producer * kafka.output.max_msgsize: Maximum message size in bytes (see Kafka producer
docs). Default is 1024*1024 (1MB). docs). Default is 1024*1024 (1MB).
* kafka.output.compression_codec: The compression codec to use (see Kafka producer
docs). Default is 0 (no compression).
For easier debugging, the above values as well as the Kafka URI For easier debugging, the above values as well as the Kafka broker information
(kafka.output.url), the output server (kafka.output.server), the topic (either kafka.zk.connect or kafka.broker.list), the topic (kafka.output.topic),
(kafka.output.topic), and the schema (kafka.output.schema) are injected into and the schema (kafka.output.schema) are injected into the job's configuration.
the job's configuration.

View File

@ -16,8 +16,9 @@
*/ */
package kafka.bridge.examples; package kafka.bridge.examples;
import kafka.bridge.hadoop.KafkaOutputFormat;
import java.io.IOException;
import kafka.bridge.hadoop.KafkaOutputFormat;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.NullWritable;
@ -27,8 +28,6 @@ import org.apache.hadoop.mapreduce.Mapper;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
import java.io.IOException;
public class TextPublisher public class TextPublisher
{ {
public static void main(String[] args) throws Exception public static void main(String[] args) throws Exception

View File

@ -16,30 +16,47 @@
*/ */
package kafka.bridge.hadoop; package kafka.bridge.hadoop;
import java.io.IOException;
import java.net.URI;
import java.util.Properties; import java.util.Properties;
import kafka.javaapi.producer.Producer;
import kafka.javaapi.producer.SyncProducer; import kafka.message.Message;
import kafka.producer.SyncProducerConfig; import kafka.producer.ProducerConfig;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.mapreduce.*; import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.OutputCommitter;
import org.apache.hadoop.mapreduce.OutputFormat;
import org.apache.hadoop.mapreduce.RecordWriter;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter; import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.net.URI;
public class KafkaOutputFormat<W extends BytesWritable> extends OutputFormat<NullWritable, W> public class KafkaOutputFormat<W extends BytesWritable> extends OutputFormat<NullWritable, W>
{ {
private Logger log = Logger.getLogger(KafkaOutputFormat.class);
public static final String KAFKA_URL = "kafka.output.url"; public static final String KAFKA_URL = "kafka.output.url";
public static final int KAFKA_PRODUCER_CONNECT_TIMEOUT = 30*1000; /** Bytes to buffer before the OutputFormat does a send */
public static final int KAFKA_PRODUCER_RECONNECT_INTERVAL = 1000;
public static final int KAFKA_PRODUCER_BUFFER_SIZE = 64*1024;
public static final int KAFKA_PRODUCER_MAX_MESSAGE_SIZE = 1024*1024;
public static final int KAFKA_QUEUE_SIZE = 10*1024*1024; public static final int KAFKA_QUEUE_SIZE = 10*1024*1024;
/** Default value for Kafka's connect.timeout.ms */
public static final int KAFKA_PRODUCER_CONNECT_TIMEOUT = 30*1000;
/** Default value for Kafka's reconnect.interval*/
public static final int KAFKA_PRODUCER_RECONNECT_INTERVAL = 1000;
/** Default value for Kafka's buffer.size */
public static final int KAFKA_PRODUCER_BUFFER_SIZE = 64*1024;
/** Default value for Kafka's max.message.size */
public static final int KAFKA_PRODUCER_MAX_MESSAGE_SIZE = 1024*1024;
/** Default value for Kafka's producer.type */
public static final String KAFKA_PRODUCER_PRODUCER_TYPE = "sync";
/** Default value for Kafka's compression.codec */
public static final int KAFKA_PRODUCER_COMPRESSION_CODEC = 0;
public KafkaOutputFormat() public KafkaOutputFormat()
{ {
super(); super();
@ -77,40 +94,80 @@ public class KafkaOutputFormat<W extends BytesWritable> extends OutputFormat<Nul
Path outputPath = getOutputPath(context); Path outputPath = getOutputPath(context);
if (outputPath == null) if (outputPath == null)
throw new IllegalArgumentException("no kafka output url specified"); throw new IllegalArgumentException("no kafka output url specified");
URI uri = outputPath.toUri(); URI uri = URI.create(outputPath.toString());
Configuration job = context.getConfiguration(); Configuration job = context.getConfiguration();
final String topic = uri.getPath().substring(1); // ignore the initial '/' in the path Properties props = new Properties();
String topic;
final int queueSize = job.getInt("kafka.output.queue_size", KAFKA_QUEUE_SIZE); final int queueSize = job.getInt("kafka.output.queue_size", KAFKA_QUEUE_SIZE);
final int timeout = job.getInt("kafka.output.connect_timeout", KAFKA_PRODUCER_CONNECT_TIMEOUT); final int timeout = job.getInt("kafka.output.connect_timeout", KAFKA_PRODUCER_CONNECT_TIMEOUT);
final int interval = job.getInt("kafka.output.reconnect_interval", KAFKA_PRODUCER_RECONNECT_INTERVAL); final int interval = job.getInt("kafka.output.reconnect_interval", KAFKA_PRODUCER_RECONNECT_INTERVAL);
final int bufSize = job.getInt("kafka.output.bufsize", KAFKA_PRODUCER_BUFFER_SIZE); final int bufSize = job.getInt("kafka.output.bufsize", KAFKA_PRODUCER_BUFFER_SIZE);
final int maxSize = job.getInt("kafka.output.max_msgsize", KAFKA_PRODUCER_MAX_MESSAGE_SIZE); final int maxSize = job.getInt("kafka.output.max_msgsize", KAFKA_PRODUCER_MAX_MESSAGE_SIZE);
final String producerType = job.get("kafka.output.producer_type", KAFKA_PRODUCER_PRODUCER_TYPE);
final int compressionCodec = job.getInt("kafka.output.compression_codec", KAFKA_PRODUCER_COMPRESSION_CODEC);
job.set("kafka.output.server", String.format("%s:%d", uri.getHost(), uri.getPort()));
job.set("kafka.output.topic", topic);
job.setInt("kafka.output.queue_size", queueSize); job.setInt("kafka.output.queue_size", queueSize);
job.setInt("kafka.output.connect_timeout", timeout); job.setInt("kafka.output.connect_timeout", timeout);
job.setInt("kafka.output.reconnect_interval", interval); job.setInt("kafka.output.reconnect_interval", interval);
job.setInt("kafka.output.bufsize", bufSize); job.setInt("kafka.output.bufsize", bufSize);
job.setInt("kafka.output.max_msgsize", maxSize); job.setInt("kafka.output.max_msgsize", maxSize);
job.set("kafka.output.producer_type", producerType);
job.setInt("kafka.output.compression_codec", compressionCodec);
if (uri.getHost().isEmpty()) props.setProperty("producer.type", producerType);
throw new IllegalArgumentException("missing kafka server");
if (uri.getPath().isEmpty())
throw new IllegalArgumentException("missing kafka topic");
Properties props = new Properties();
props.setProperty("host", uri.getHost());
props.setProperty("port", Integer.toString(uri.getPort()));
props.setProperty("buffer.size", Integer.toString(bufSize)); props.setProperty("buffer.size", Integer.toString(bufSize));
props.setProperty("connect.timeout.ms", Integer.toString(timeout)); props.setProperty("connect.timeout.ms", Integer.toString(timeout));
props.setProperty("reconnect.interval", Integer.toString(interval)); props.setProperty("reconnect.interval", Integer.toString(interval));
props.setProperty("max.message.size", Integer.toString(maxSize)); props.setProperty("max.message.size", Integer.toString(maxSize));
props.setProperty("compression.codec", Integer.toString(compressionCodec));
SyncProducer producer = new SyncProducer(new SyncProducerConfig(props)); if (uri.getScheme().equals("kafka+zk")) {
// Software load balancer:
// URL: kafka+zk://<zk connect path>#<kafka topic>
// e.g. kafka+zk://kafka-zk:2181/kafka#foobar
String zkConnect = uri.getAuthority() + uri.getPath();
props.setProperty("zk.connect", zkConnect);
job.set("kafka.zk.connect", zkConnect);
topic = uri.getFragment();
if (topic == null)
throw new IllegalArgumentException("no topic specified in kafka uri fragment");
log.info(String.format("using kafka zk.connect %s (topic %s)", zkConnect, topic));
} else if (uri.getScheme().equals("kafka")) {
// using the legacy direct broker list
// URL: kafka://<kafka host>/<topic>
// e.g. kafka://kafka-server:9000,kafka-server2:9000/foobar
// Just enumerate broker_ids, as it really doesn't matter what they are as long as they're unique
// (KAFKA-258 will remove the broker_id requirement)
StringBuilder brokerListBuilder = new StringBuilder();
String delim = "";
int brokerId = 0;
for (String serverPort : uri.getAuthority().split(",")) {
brokerListBuilder.append(delim).append(String.format("%d:%s", brokerId, serverPort));
delim = ",";
brokerId++;
}
String brokerList = brokerListBuilder.toString();
props.setProperty("broker.list", brokerList);
job.set("kafka.broker.list", brokerList);
if (uri.getPath() == null || uri.getPath().length() <= 1)
throw new IllegalArgumentException("no topic specified in kafka uri");
topic = uri.getPath().substring(1); // ignore the initial '/' in the path
job.set("kafka.output.topic", topic);
log.info(String.format("using kafka broker %s (topic %s)", brokerList, topic));
} else
throw new IllegalArgumentException("missing scheme from kafka uri (must be kafka:// or kafka+zk://)");
Producer<Integer, Message> producer = new Producer<Integer, Message>(new ProducerConfig(props));
return new KafkaRecordWriter<W>(producer, topic, queueSize); return new KafkaRecordWriter<W>(producer, topic, queueSize);
} }
} }

View File

@ -16,30 +16,28 @@
*/ */
package kafka.bridge.hadoop; package kafka.bridge.hadoop;
import kafka.message.Message;
import kafka.javaapi.message.ByteBufferMessageSet;
import kafka.javaapi.producer.SyncProducer;
import kafka.message.NoCompressionCodec; import java.io.IOException;
import java.util.LinkedList;
import java.util.List;
import kafka.javaapi.producer.Producer;
import kafka.javaapi.producer.ProducerData;
import kafka.message.Message;
import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.mapreduce.RecordWriter; import org.apache.hadoop.mapreduce.RecordWriter;
import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.TaskAttemptContext;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
public class KafkaRecordWriter<W extends BytesWritable> extends RecordWriter<NullWritable, W> public class KafkaRecordWriter<W extends BytesWritable> extends RecordWriter<NullWritable, W>
{ {
protected SyncProducer producer; protected Producer<Integer, Message> producer;
protected String topic; protected String topic;
protected List<Message> msgList = new ArrayList<Message>(); protected List<ProducerData<Integer, Message>> msgList = new LinkedList<ProducerData<Integer, Message>>();
protected int totalSize = 0; protected int totalSize = 0;
protected int queueSize; protected int queueSize;
public KafkaRecordWriter(SyncProducer producer, String topic, int queueSize) public KafkaRecordWriter(Producer<Integer, Message> producer, String topic, int queueSize)
{ {
this.producer = producer; this.producer = producer;
this.topic = topic; this.topic = topic;
@ -49,8 +47,7 @@ public class KafkaRecordWriter<W extends BytesWritable> extends RecordWriter<Nul
protected void sendMsgList() protected void sendMsgList()
{ {
if (msgList.size() > 0) { if (msgList.size() > 0) {
ByteBufferMessageSet msgSet = new ByteBufferMessageSet(kafka.message.NoCompressionCodec$.MODULE$, msgList); producer.send(msgList);
producer.send(topic, msgSet);
msgList.clear(); msgList.clear();
totalSize = 0; totalSize = 0;
} }
@ -60,10 +57,11 @@ public class KafkaRecordWriter<W extends BytesWritable> extends RecordWriter<Nul
public void write(NullWritable key, BytesWritable value) throws IOException, InterruptedException public void write(NullWritable key, BytesWritable value) throws IOException, InterruptedException
{ {
Message msg = new Message(value.getBytes()); Message msg = new Message(value.getBytes());
msgList.add(msg); msgList.add(new ProducerData<Integer, Message>(this.topic, msg));
totalSize += msg.size(); totalSize += msg.size();
if (totalSize > queueSize) // MultiProducerRequest only supports sending up to Short.MAX_VALUE messages in one batch
if (totalSize > queueSize || msgList.size() >= Short.MAX_VALUE)
sendMsgList(); sendMsgList();
} }

View File

@ -16,9 +16,12 @@
*/ */
package kafka.bridge.pig; package kafka.bridge.pig;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.OutputStream;
import kafka.bridge.hadoop.KafkaOutputFormat; import kafka.bridge.hadoop.KafkaOutputFormat;
import kafka.bridge.hadoop.KafkaRecordWriter; import kafka.bridge.hadoop.KafkaRecordWriter;
import org.apache.avro.io.BinaryEncoder; import org.apache.avro.io.BinaryEncoder;
import org.apache.avro.io.Encoder; import org.apache.avro.io.Encoder;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
@ -33,10 +36,6 @@ import org.apache.pig.data.Tuple;
import org.apache.pig.piggybank.storage.avro.PigAvroDatumWriter; import org.apache.pig.piggybank.storage.avro.PigAvroDatumWriter;
import org.apache.pig.piggybank.storage.avro.PigSchema2Avro; import org.apache.pig.piggybank.storage.avro.PigSchema2Avro;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.OutputStream;
public class AvroKafkaStorage extends StoreFunc public class AvroKafkaStorage extends StoreFunc
{ {
protected KafkaRecordWriter writer; protected KafkaRecordWriter writer;

View File

@ -17,8 +17,6 @@
package kafka package kafka
import consumer.ConsumerConfig
import producer.ProducerConfig
import server.{KafkaConfig, KafkaServerStartable, KafkaServer} import server.{KafkaConfig, KafkaServerStartable, KafkaServer}
import utils.{Utils, Logging} import utils.{Utils, Logging}
import org.apache.log4j.jmx.LoggerDynamicMBean import org.apache.log4j.jmx.LoggerDynamicMBean
@ -30,8 +28,8 @@ object Kafka extends Logging {
import org.apache.log4j.Logger import org.apache.log4j.Logger
Utils.registerMBean(new LoggerDynamicMBean(Logger.getRootLogger()), kafkaLog4jMBeanName) Utils.registerMBean(new LoggerDynamicMBean(Logger.getRootLogger()), kafkaLog4jMBeanName)
if (!List(1, 3).contains(args.length)) { if (args.length != 1) {
println("USAGE: java [options] %s server.properties [consumer.properties producer.properties]".format(classOf[KafkaServer].getSimpleName())) println("USAGE: java [options] %s server.properties".format(classOf[KafkaServer].getSimpleName()))
System.exit(1) System.exit(1)
} }
@ -39,14 +37,7 @@ object Kafka extends Logging {
val props = Utils.loadProps(args(0)) val props = Utils.loadProps(args(0))
val serverConfig = new KafkaConfig(props) val serverConfig = new KafkaConfig(props)
val kafkaServerStartble = args.length match { val kafkaServerStartble = new KafkaServerStartable(serverConfig)
case 3 =>
val consumerConfig = new ConsumerConfig(Utils.loadProps(args(1)))
val producerConfig = new ProducerConfig(Utils.loadProps(args(2)))
new KafkaServerStartable(serverConfig, consumerConfig, producerConfig)
case 1 =>
new KafkaServerStartable(serverConfig)
}
// attach shutdown handler to catch control-c // attach shutdown handler to catch control-c
Runtime.getRuntime().addShutdownHook(new Thread() { Runtime.getRuntime().addShutdownHook(new Thread() {

View File

@ -36,11 +36,19 @@ object ConsoleConsumer extends Logging {
def main(args: Array[String]) { def main(args: Array[String]) {
val parser = new OptionParser val parser = new OptionParser
val topicIdOpt = parser.accepts("topic", "REQUIRED: The topic id to consume on.") val topicIdOpt = parser.accepts("topic", "The topic id to consume on.")
.withRequiredArg .withRequiredArg
.describedAs("topic") .describedAs("topic")
.ofType(classOf[String]) .ofType(classOf[String])
val zkConnectOpt = parser.accepts("zookeeper", "REQUIRED: The connection string for the zookeeper connection in the form host:port. " + val whitelistOpt = parser.accepts("whitelist", "Whitelist of topics to include for consumption.")
.withRequiredArg
.describedAs("whitelist")
.ofType(classOf[String])
val blacklistOpt = parser.accepts("blacklist", "Blacklist of topics to exclude from consumption.")
.withRequiredArg
.describedAs("blacklist")
.ofType(classOf[String])
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.") "Multiple URLS can be given to allow fail-over.")
.withRequiredArg .withRequiredArg
.describedAs("urls") .describedAs("urls")
@ -90,8 +98,20 @@ object ConsoleConsumer extends Logging {
"skip it instead of halt.") "skip it instead of halt.")
val options: OptionSet = tryParse(parser, args) val options: OptionSet = tryParse(parser, args)
checkRequiredArgs(parser, options, topicIdOpt, zkConnectOpt) Utils.checkRequiredArgs(parser, options, zkConnectOpt)
val topicOrFilterOpt = List(topicIdOpt, whitelistOpt, blacklistOpt).filter(options.has)
if (topicOrFilterOpt.size != 1) {
error("Exactly one of whitelist/blacklist/topic is required.")
parser.printHelpOn(System.err)
System.exit(1)
}
val topicArg = options.valueOf(topicOrFilterOpt.head)
val filterSpec = if (options.has(blacklistOpt))
new Blacklist(topicArg)
else
new Whitelist(topicArg)
val props = new Properties() val props = new Properties()
props.put("groupid", options.valueOf(groupIdOpt)) props.put("groupid", options.valueOf(groupIdOpt))
props.put("socket.buffersize", options.valueOf(socketBufferSizeOpt).toString) props.put("socket.buffersize", options.valueOf(socketBufferSizeOpt).toString)
@ -104,7 +124,6 @@ object ConsoleConsumer extends Logging {
val config = new ConsumerConfig(props) val config = new ConsumerConfig(props)
val skipMessageOnError = if (options.has(skipMessageOnErrorOpt)) true else false val skipMessageOnError = if (options.has(skipMessageOnErrorOpt)) true else false
val topic = options.valueOf(topicIdOpt)
val messageFormatterClass = Class.forName(options.valueOf(messageFormatterOpt)) val messageFormatterClass = Class.forName(options.valueOf(messageFormatterOpt))
val formatterArgs = tryParseFormatterArgs(options.valuesOf(messageFormatterArgOpt)) val formatterArgs = tryParseFormatterArgs(options.valuesOf(messageFormatterArgOpt))
@ -123,21 +142,20 @@ object ConsoleConsumer extends Logging {
tryCleanupZookeeper(options.valueOf(zkConnectOpt), options.valueOf(groupIdOpt)) tryCleanupZookeeper(options.valueOf(zkConnectOpt), options.valueOf(groupIdOpt))
} }
}) })
var stream = connector.createMessageStreams(Map(topic -> 1)).get(topic).get.get(0) val stream = connector.createMessageStreamsByFilter(filterSpec).get(0)
val iter = val iter = if(maxMessages >= 0)
if(maxMessages >= 0) stream.slice(0, maxMessages)
stream.slice(0, maxMessages) else
else stream
stream
val formatter: MessageFormatter = messageFormatterClass.newInstance().asInstanceOf[MessageFormatter] val formatter: MessageFormatter = messageFormatterClass.newInstance().asInstanceOf[MessageFormatter]
formatter.init(formatterArgs) formatter.init(formatterArgs)
try { try {
for(message <- iter) { for(messageAndTopic <- iter) {
try { try {
formatter.writeTo(message, System.out) formatter.writeTo(messageAndTopic.message, System.out)
} catch { } catch {
case e => case e =>
if (skipMessageOnError) if (skipMessageOnError)
@ -173,16 +191,6 @@ object ConsoleConsumer extends Logging {
} }
} }
def checkRequiredArgs(parser: OptionParser, options: OptionSet, required: OptionSpec[_]*) {
for(arg <- required) {
if(!options.has(arg)) {
error("Missing required argument \"" + arg + "\"")
parser.printHelpOn(System.err)
System.exit(1)
}
}
}
def tryParseFormatterArgs(args: Iterable[String]): Properties = { def tryParseFormatterArgs(args: Iterable[String]): Properties = {
val splits = args.map(_ split "=").filterNot(_ == null).filterNot(_.length == 0) val splits = args.map(_ split "=").filterNot(_ == null).filterNot(_.length == 0)
if(!splits.forall(_.length == 2)) { if(!splits.forall(_.length == 2)) {
@ -210,9 +218,19 @@ object ConsoleConsumer extends Logging {
} }
class ChecksumMessageFormatter extends MessageFormatter { class ChecksumMessageFormatter extends MessageFormatter {
private var topicStr: String = _
override def init(props: Properties) {
topicStr = props.getProperty("topic")
if (topicStr != null)
topicStr = topicStr + "-"
else
topicStr = ""
}
def writeTo(message: Message, output: PrintStream) { def writeTo(message: Message, output: PrintStream) {
val chksum = message.checksum val chksum = message.checksum
output.println("checksum:" + chksum) output.println(topicStr + "checksum:" + chksum)
} }
} }

View File

@ -20,7 +20,6 @@ package kafka.consumer
import java.util.Properties import java.util.Properties
import kafka.utils.{ZKConfig, Utils} import kafka.utils.{ZKConfig, Utils}
import kafka.api.OffsetRequest import kafka.api.OffsetRequest
import kafka.common.InvalidConfigException
object ConsumerConfig { object ConsumerConfig {
val SocketTimeout = 30 * 1000 val SocketTimeout = 30 * 1000
val SocketBufferSize = 64*1024 val SocketBufferSize = 64*1024
@ -29,7 +28,7 @@ object ConsumerConfig {
val DefaultFetcherBackoffMs = 1000 val DefaultFetcherBackoffMs = 1000
val AutoCommit = true val AutoCommit = true
val AutoCommitInterval = 10 * 1000 val AutoCommitInterval = 10 * 1000
val MaxQueuedChunks = 100 val MaxQueuedChunks = 10
val MaxRebalanceRetries = 4 val MaxRebalanceRetries = 4
val AutoOffsetReset = OffsetRequest.SmallestTimeString val AutoOffsetReset = OffsetRequest.SmallestTimeString
val ConsumerTimeoutMs = -1 val ConsumerTimeoutMs = -1
@ -93,20 +92,10 @@ class ConsumerConfig(props: Properties) extends ZKConfig(props) {
/** throw a timeout exception to the consumer if no message is available for consumption after the specified interval */ /** throw a timeout exception to the consumer if no message is available for consumption after the specified interval */
val consumerTimeoutMs = Utils.getInt(props, "consumer.timeout.ms", ConsumerTimeoutMs) val consumerTimeoutMs = Utils.getInt(props, "consumer.timeout.ms", ConsumerTimeoutMs)
/** Whitelist of topics for this mirror's embedded consumer to consume. At /** Use shallow iterator over compressed messages directly. This feature should be used very carefully.
* most one of whitelist/blacklist may be specified. */ * Typically, it's only used for mirroring raw messages from one kafka cluster to another to save the
val mirrorTopicsWhitelist = Utils.getString( * overhead of decompression.
props, MirrorTopicsWhitelistProp, MirrorTopicsWhitelist) * */
val enableShallowIterator = Utils.getBoolean(props, "shallowiterator.enable", false)
/** Topics to skip mirroring. At most one of whitelist/blacklist may be
* specified */
val mirrorTopicsBlackList = Utils.getString(
props, MirrorTopicsBlacklistProp, MirrorTopicsBlacklist)
if (mirrorTopicsWhitelist.nonEmpty && mirrorTopicsBlackList.nonEmpty)
throw new InvalidConfigException("The embedded consumer's mirror topics configuration can only contain one of blacklist or whitelist")
val mirrorConsumerNumThreads = Utils.getInt(
props, MirrorConsumerNumThreadsProp, MirrorConsumerNumThreads)
} }

View File

@ -29,12 +29,28 @@ trait ConsumerConnector {
* Create a list of MessageStreams for each topic. * Create a list of MessageStreams for each topic.
* *
* @param topicCountMap a map of (topic, #streams) pair * @param topicCountMap a map of (topic, #streams) pair
* @return a map of (topic, list of KafkaMessageStream) pair. The number of items in the * @param decoder Decoder to decode each Message to type T
* list is #streams. Each KafkaMessageStream supports an iterator of messages. * @return a map of (topic, list of KafkaStream) pairs.
* The number of items in the list is #streams. Each stream supports
* an iterator over message/metadata pairs.
*/ */
def createMessageStreams[T](topicCountMap: Map[String,Int], def createMessageStreams[T](topicCountMap: Map[String,Int],
decoder: Decoder[T] = new DefaultDecoder) decoder: Decoder[T] = new DefaultDecoder)
: Map[String,List[KafkaMessageStream[T]]] : Map[String,List[KafkaStream[T]]]
/**
* Create a list of message streams for all topics that match a given filter.
*
* @param topicFilter Either a Whitelist or Blacklist TopicFilter object.
* @param numStreams Number of streams to return
* @param decoder Decoder to decode each Message to type T
* @return a list of KafkaStream each of which provides an
* iterator over message/metadata pairs over allowed topics.
*/
def createMessageStreamsByFilter[T](topicFilter: TopicFilter,
numStreams: Int = 1,
decoder: Decoder[T] = new DefaultDecoder)
: Seq[KafkaStream[T]]
/** /**
* Commit the offsets of all broker partitions connected by this connector. * Commit the offsets of all broker partitions connected by this connector.

View File

@ -19,36 +19,38 @@ package kafka.consumer
import kafka.utils.{IteratorTemplate, Logging} import kafka.utils.{IteratorTemplate, Logging}
import java.util.concurrent.{TimeUnit, BlockingQueue} import java.util.concurrent.{TimeUnit, BlockingQueue}
import kafka.message.MessageAndOffset
import kafka.serializer.Decoder import kafka.serializer.Decoder
import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.atomic.AtomicReference
import kafka.message.{MessageAndOffset, MessageAndMetadata}
/** /**
* An iterator that blocks until a value can be read from the supplied queue. * An iterator that blocks until a value can be read from the supplied queue.
* The iterator takes a shutdownCommand object which can be added to the queue to trigger a shutdown * The iterator takes a shutdownCommand object which can be added to the queue to trigger a shutdown
* *
*/ */
class ConsumerIterator[T](private val topic: String, class ConsumerIterator[T](private val channel: BlockingQueue[FetchedDataChunk],
private val channel: BlockingQueue[FetchedDataChunk],
consumerTimeoutMs: Int, consumerTimeoutMs: Int,
private val decoder: Decoder[T]) private val decoder: Decoder[T],
extends IteratorTemplate[T] with Logging { val enableShallowIterator: Boolean)
extends IteratorTemplate[MessageAndMetadata[T]] with Logging {
private var current: AtomicReference[Iterator[MessageAndOffset]] = new AtomicReference(null) private var current: AtomicReference[Iterator[MessageAndOffset]] = new AtomicReference(null)
private var currentTopicInfo:PartitionTopicInfo = null private var currentTopicInfo:PartitionTopicInfo = null
private var consumedOffset: Long = -1L private var consumedOffset: Long = -1L
override def next(): T = { override def next(): MessageAndMetadata[T] = {
val decodedMessage = super.next() val item = super.next()
if(consumedOffset < 0) if(consumedOffset < 0)
throw new IllegalStateException("Offset returned by the message set is invalid %d".format(consumedOffset)) throw new IllegalStateException("Offset returned by the message set is invalid %d".format(consumedOffset))
currentTopicInfo.resetConsumeOffset(consumedOffset) currentTopicInfo.resetConsumeOffset(consumedOffset)
trace("Setting consumed offset to %d".format(consumedOffset)) val topic = currentTopicInfo.topic
trace("Setting %s consumed offset to %d".format(topic, consumedOffset))
ConsumerTopicStat.getConsumerTopicStat(topic).recordMessagesPerTopic(1) ConsumerTopicStat.getConsumerTopicStat(topic).recordMessagesPerTopic(1)
decodedMessage item
} }
protected def makeNext(): T = { protected def makeNext(): MessageAndMetadata[T] = {
var currentDataChunk: FetchedDataChunk = null var currentDataChunk: FetchedDataChunk = null
// if we don't have an iterator, get one // if we don't have an iterator, get one
var localCurrent = current.get() var localCurrent = current.get()
@ -74,16 +76,18 @@ class ConsumerIterator[T](private val topic: String,
.format(currentTopicInfo.getConsumeOffset, currentDataChunk.fetchOffset, currentTopicInfo)) .format(currentTopicInfo.getConsumeOffset, currentDataChunk.fetchOffset, currentTopicInfo))
currentTopicInfo.resetConsumeOffset(currentDataChunk.fetchOffset) currentTopicInfo.resetConsumeOffset(currentDataChunk.fetchOffset)
} }
localCurrent = currentDataChunk.messages.iterator localCurrent = if (enableShallowIterator) currentDataChunk.messages.shallowIterator
else currentDataChunk.messages.iterator
current.set(localCurrent) current.set(localCurrent)
} }
} }
val item = localCurrent.next() val item = localCurrent.next()
consumedOffset = item.offset consumedOffset = item.offset
decoder.toEvent(item.message)
new MessageAndMetadata(decoder.toEvent(item.message), currentTopicInfo.topic)
} }
def clearCurrentChunk() = { def clearCurrentChunk() {
try { try {
info("Clearing the current data chunk for this consumer iterator") info("Clearing the current data chunk for this consumer iterator")
current.set(null) current.set(null)
@ -92,3 +96,4 @@ class ConsumerIterator[T](private val topic: String,
} }
class ConsumerTimeoutException() extends RuntimeException() class ConsumerTimeoutException() extends RuntimeException()

View File

@ -42,24 +42,24 @@ private [consumer] class Fetcher(val config: ConsumerConfig, val zkClient : ZkCl
fetcherThreads = EMPTY_FETCHER_THREADS fetcherThreads = EMPTY_FETCHER_THREADS
} }
def clearFetcherQueues[T](topicInfos: Iterable[PartitionTopicInfo], cluster: Cluster, def clearFetcherQueues(topicInfos: Iterable[PartitionTopicInfo], cluster: Cluster,
queuesTobeCleared: Iterable[BlockingQueue[FetchedDataChunk]], queuesTobeCleared: Iterable[BlockingQueue[FetchedDataChunk]],
kafkaMessageStreams: Map[String,List[KafkaMessageStream[T]]]) { messageStreams: Map[String,List[KafkaStream[_]]]) {
// Clear all but the currently iterated upon chunk in the consumer thread's queue // Clear all but the currently iterated upon chunk in the consumer thread's queue
queuesTobeCleared.foreach(_.clear) queuesTobeCleared.foreach(_.clear)
info("Cleared all relevant queues for this fetcher") info("Cleared all relevant queues for this fetcher")
// Also clear the currently iterated upon chunk in the consumer threads // Also clear the currently iterated upon chunk in the consumer threads
if(kafkaMessageStreams != null) if(messageStreams != null)
kafkaMessageStreams.foreach(_._2.foreach(s => s.clear())) messageStreams.foreach(_._2.foreach(s => s.clear()))
info("Cleared the data chunks in all the consumer message iterators") info("Cleared the data chunks in all the consumer message iterators")
} }
def startConnections[T](topicInfos: Iterable[PartitionTopicInfo], cluster: Cluster, def startConnections(topicInfos: Iterable[PartitionTopicInfo],
kafkaMessageStreams: Map[String,List[KafkaMessageStream[T]]]) { cluster: Cluster) {
if (topicInfos == null) if (topicInfos == null)
return return

View File

@ -5,7 +5,7 @@
* The ASF licenses this file to You under the Apache License, Version 2.0 * 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 not use this file except in compliance with
* the License. You may obtain a copy of the License at * the License. You may obtain a copy of the License at
* *
* http://www.apache.org/licenses/LICENSE-2.0 * http://www.apache.org/licenses/LICENSE-2.0
* *
* Unless required by applicable law or agreed to in writing, software * Unless required by applicable law or agreed to in writing, software
@ -17,22 +17,20 @@
package kafka.consumer package kafka.consumer
import java.util.concurrent.BlockingQueue import java.util.concurrent.BlockingQueue
import kafka.serializer.Decoder import kafka.serializer.Decoder
import kafka.message.MessageAndMetadata
/** class KafkaStream[T](private val queue: BlockingQueue[FetchedDataChunk],
* All calls to elements should produce the same thread-safe iterator? Should have a separate thread consumerTimeoutMs: Int,
* that feeds messages into a blocking queue for processing. private val decoder: Decoder[T],
*/ val enableShallowIterator: Boolean)
class KafkaMessageStream[T](val topic: String, extends Iterable[MessageAndMetadata[T]] with java.lang.Iterable[MessageAndMetadata[T]] {
private val queue: BlockingQueue[FetchedDataChunk],
consumerTimeoutMs: Int,
private val decoder: Decoder[T])
extends Iterable[T] with java.lang.Iterable[T]{
private val iter: ConsumerIterator[T] = private val iter: ConsumerIterator[T] =
new ConsumerIterator[T](topic, queue, consumerTimeoutMs, decoder) new ConsumerIterator[T](queue, consumerTimeoutMs, decoder, enableShallowIterator)
/** /**
* Create an iterator over messages in the stream. * Create an iterator over messages in the stream.
*/ */

View File

@ -19,33 +19,17 @@ package kafka.consumer
import scala.collection._ import scala.collection._
import scala.util.parsing.json.JSON import scala.util.parsing.json.JSON
import kafka.utils.Logging import org.I0Itec.zkclient.ZkClient
import java.util.regex.Pattern
import kafka.utils.{ZKGroupDirs, ZkUtils, Logging}
private[kafka] object TopicCount extends Logging {
val myConversionFunc = {input : String => input.toInt}
JSON.globalNumberParser = myConversionFunc
def constructTopicCount(consumerIdSting: String, jsonString : String) : TopicCount = { private[kafka] trait TopicCount {
var topMap : Map[String,Int] = null def getConsumerThreadIdsPerTopic: Map[String, Set[String]]
try { def dbString: String
JSON.parseFull(jsonString) match {
case Some(m) => topMap = m.asInstanceOf[Map[String,Int]] protected def makeConsumerThreadIdsPerTopic(consumerIdString: String,
case None => throw new RuntimeException("error constructing TopicCount : " + jsonString) topicCountMap: Map[String, Int]) = {
}
} catch {
case e =>
error("error parsing consumer json string " + jsonString, e)
throw e
}
new TopicCount(consumerIdSting, topMap)
}
}
private[kafka] class TopicCount(val consumerIdString: String, val topicCountMap: Map[String, Int]) {
def getConsumerThreadIdsPerTopic(): Map[String, Set[String]] = {
val consumerThreadIdsPerTopicMap = new mutable.HashMap[String, Set[String]]() val consumerThreadIdsPerTopicMap = new mutable.HashMap[String, Set[String]]()
for ((topic, nConsumers) <- topicCountMap) { for ((topic, nConsumers) <- topicCountMap) {
val consumerSet = new mutable.HashSet[String] val consumerSet = new mutable.HashSet[String]
@ -56,11 +40,96 @@ private[kafka] class TopicCount(val consumerIdString: String, val topicCountMap:
} }
consumerThreadIdsPerTopicMap consumerThreadIdsPerTopicMap
} }
}
private[kafka] object TopicCount extends Logging {
/*
* Example of whitelist topic count stored in ZooKeeper:
* Topics with whitetopic as prefix, and four streams: *4*whitetopic.*
*
* Example of blacklist topic count stored in ZooKeeper:
* Topics with blacktopic as prefix, and four streams: !4!blacktopic.*
*/
val WHITELIST_MARKER = "*"
val BLACKLIST_MARKER = "!"
private val WHITELIST_PATTERN =
Pattern.compile("""\*(\p{Digit}+)\*(.*)""")
private val BLACKLIST_PATTERN =
Pattern.compile("""!(\p{Digit}+)!(.*)""")
val myConversionFunc = {input : String => input.toInt}
JSON.globalNumberParser = myConversionFunc
def constructTopicCount(group: String,
consumerId: String,
zkClient: ZkClient) : TopicCount = {
val dirs = new ZKGroupDirs(group)
val topicCountString = ZkUtils.readData(zkClient, dirs.consumerRegistryDir + "/" + consumerId)
val hasWhitelist = topicCountString.startsWith(WHITELIST_MARKER)
val hasBlacklist = topicCountString.startsWith(BLACKLIST_MARKER)
if (hasWhitelist || hasBlacklist)
info("Constructing topic count for %s from %s using %s as pattern."
.format(consumerId, topicCountString,
if (hasWhitelist) WHITELIST_PATTERN else BLACKLIST_PATTERN))
if (hasWhitelist || hasBlacklist) {
val matcher = if (hasWhitelist)
WHITELIST_PATTERN.matcher(topicCountString)
else
BLACKLIST_PATTERN.matcher(topicCountString)
require(matcher.matches())
val numStreams = matcher.group(1).toInt
val regex = matcher.group(2)
val filter = if (hasWhitelist)
new Whitelist(regex)
else
new Blacklist(regex)
new WildcardTopicCount(zkClient, consumerId, filter, numStreams)
}
else {
var topMap : Map[String,Int] = null
try {
JSON.parseFull(topicCountString) match {
case Some(m) => topMap = m.asInstanceOf[Map[String,Int]]
case None => throw new RuntimeException("error constructing TopicCount : " + topicCountString)
}
}
catch {
case e =>
error("error parsing consumer json string " + topicCountString, e)
throw e
}
new StaticTopicCount(consumerId, topMap)
}
}
def constructTopicCount(consumerIdString: String, topicCount: Map[String, Int]) =
new StaticTopicCount(consumerIdString, topicCount)
def constructTopicCount(consumerIdString: String,
filter: TopicFilter,
numStreams: Int,
zkClient: ZkClient) =
new WildcardTopicCount(zkClient, consumerIdString, filter, numStreams)
}
private[kafka] class StaticTopicCount(val consumerIdString: String,
val topicCountMap: Map[String, Int])
extends TopicCount {
def getConsumerThreadIdsPerTopic =
makeConsumerThreadIdsPerTopic(consumerIdString, topicCountMap)
override def equals(obj: Any): Boolean = { override def equals(obj: Any): Boolean = {
obj match { obj match {
case null => false case null => false
case n: TopicCount => consumerIdString == n.consumerIdString && topicCountMap == n.topicCountMap case n: StaticTopicCount => consumerIdString == n.consumerIdString && topicCountMap == n.topicCountMap
case _ => false case _ => false
} }
} }
@ -71,7 +140,7 @@ private[kafka] class TopicCount(val consumerIdString: String, val topicCountMap:
* "topic2" : 4 * "topic2" : 4
* } * }
*/ */
def toJsonString() : String = { def dbString = {
val builder = new StringBuilder val builder = new StringBuilder
builder.append("{ ") builder.append("{ ")
var i = 0 var i = 0
@ -82,6 +151,29 @@ private[kafka] class TopicCount(val consumerIdString: String, val topicCountMap:
i += 1 i += 1
} }
builder.append(" }") builder.append(" }")
builder.toString builder.toString()
} }
} }
private[kafka] class WildcardTopicCount(zkClient: ZkClient,
consumerIdString: String,
topicFilter: TopicFilter,
numStreams: Int) extends TopicCount {
def getConsumerThreadIdsPerTopic = {
val wildcardTopics = ZkUtils.getChildrenParentMayNotExist(
zkClient, ZkUtils.BrokerTopicsPath).filter(topicFilter.isTopicAllowed(_))
makeConsumerThreadIdsPerTopic(consumerIdString,
Map(wildcardTopics.map((_, numStreams)): _*))
}
def dbString = {
val marker = topicFilter match {
case wl: Whitelist => TopicCount.WHITELIST_MARKER
case bl: Blacklist => TopicCount.BLACKLIST_MARKER
}
"%s%d%s%s".format(marker, numStreams, marker, topicFilter.regex)
}
}

View File

@ -0,0 +1,76 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.consumer
import kafka.utils.Logging
import java.util.regex.{PatternSyntaxException, Pattern}
sealed abstract class TopicFilter(rawRegex: String) extends Logging {
val regex = rawRegex
.trim
.replace(',', '|')
.replace(" ", "")
.replaceAll("""^["']+""","")
.replaceAll("""["']+$""","") // property files may bring quotes
try {
Pattern.compile(regex)
}
catch {
case e: PatternSyntaxException =>
throw new RuntimeException(regex + " is an invalid regex.")
}
override def toString = regex
def requiresTopicEventWatcher: Boolean
def isTopicAllowed(topic: String): Boolean
}
case class Whitelist(rawRegex: String) extends TopicFilter(rawRegex) {
override def requiresTopicEventWatcher = !regex.matches("""[\p{Alnum}-|]+""")
override def isTopicAllowed(topic: String) = {
val allowed = topic.matches(regex)
debug("%s %s".format(
topic, if (allowed) "allowed" else "filtered"))
allowed
}
}
case class Blacklist(rawRegex: String) extends TopicFilter(rawRegex) {
override def requiresTopicEventWatcher = true
override def isTopicAllowed(topic: String) = {
val allowed = !topic.matches(regex)
debug("%s %s".format(
topic, if (allowed) "allowed" else "filtered"))
allowed
}
}

View File

@ -19,6 +19,7 @@ package kafka.consumer
import java.util.concurrent._ import java.util.concurrent._
import java.util.concurrent.atomic._ import java.util.concurrent.atomic._
import locks.ReentrantLock
import scala.collection._ import scala.collection._
import kafka.cluster._ import kafka.cluster._
import kafka.utils._ import kafka.utils._
@ -33,6 +34,7 @@ import java.lang.IllegalStateException
import kafka.utils.ZkUtils._ import kafka.utils.ZkUtils._
import kafka.common.{NoBrokersForPartitionException, ConsumerRebalanceFailedException, InvalidConfigException} import kafka.common.{NoBrokersForPartitionException, ConsumerRebalanceFailedException, InvalidConfigException}
/** /**
* This class handles the consumers interaction with zookeeper * This class handles the consumers interaction with zookeeper
* *
@ -85,16 +87,37 @@ trait ZookeeperConsumerConnectorMBean {
private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
val enableFetcher: Boolean) // for testing only val enableFetcher: Boolean) // for testing only
extends ConsumerConnector with ZookeeperConsumerConnectorMBean with Logging { extends ConsumerConnector with ZookeeperConsumerConnectorMBean
with Logging {
private val isShuttingDown = new AtomicBoolean(false) private val isShuttingDown = new AtomicBoolean(false)
private val rebalanceLock = new Object private val rebalanceLock = new Object
private var fetcher: Option[Fetcher] = None private var fetcher: Option[Fetcher] = None
private var zkClient: ZkClient = null private var zkClient: ZkClient = null
private val topicRegistry = new Pool[String, Pool[Int, PartitionTopicInfo]] private var topicRegistry = new Pool[String, Pool[Int, PartitionTopicInfo]]
// queues : (topic,consumerThreadId) -> queue // topicThreadIdAndQueues : (topic,consumerThreadId) -> queue
private val queues = new Pool[Tuple2[String,String], BlockingQueue[FetchedDataChunk]] private val topicThreadIdAndQueues = new Pool[Tuple2[String,String], BlockingQueue[FetchedDataChunk]]
private val scheduler = new KafkaScheduler(1, "Kafka-consumer-autocommit-", false) private val scheduler = new KafkaScheduler(1, "Kafka-consumer-autocommit-", false)
private val messageStreamCreated = new AtomicBoolean(false)
private var sessionExpirationListener: ZKSessionExpireListener = null
private var loadBalancerListener: ZKRebalancerListener = null
private var wildcardTopicWatcher: ZookeeperTopicEventWatcher = null
val consumerIdString = {
var consumerUuid : String = null
config.consumerId match {
case Some(consumerId) // for testing only
=> consumerUuid = consumerId
case None // generate unique consumerId automatically
=> val uuid = UUID.randomUUID()
consumerUuid = "%s-%d-%s".format(
InetAddress.getLocalHost.getHostName, System.currentTimeMillis,
uuid.getMostSignificantBits().toHexString.substring(0,8))
}
config.groupId + "_" + consumerUuid
}
this.logIdent = consumerIdString + " "
connectZk() connectZk()
createFetcher() createFetcher()
@ -106,10 +129,18 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
def this(config: ConsumerConfig) = this(config, true) def this(config: ConsumerConfig) = this(config, true)
def createMessageStreams[T](topicCountMap: Map[String,Int], decoder: Decoder[T]) def createMessageStreams[T](topicCountMap: Map[String,Int], decoder: Decoder[T])
: Map[String,List[KafkaMessageStream[T]]] = { : Map[String,List[KafkaStream[T]]] = {
if (messageStreamCreated.getAndSet(true))
throw new RuntimeException(this.getClass.getSimpleName +
" can create message streams at most once")
consume(topicCountMap, decoder) consume(topicCountMap, decoder)
} }
def createMessageStreamsByFilter[T](topicFilter: TopicFilter, numStreams: Int, decoder: Decoder[T]) = {
val wildcardStreamsHandler = new WildcardStreamsHandler[T](topicFilter, numStreams, decoder)
wildcardStreamsHandler.streams
}
private def createFetcher() { private def createFetcher() {
if (enableFetcher) if (enableFetcher)
fetcher = Some(new Fetcher(config, zkClient)) fetcher = Some(new Fetcher(config, zkClient))
@ -124,6 +155,9 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
val canShutdown = isShuttingDown.compareAndSet(false, true); val canShutdown = isShuttingDown.compareAndSet(false, true);
if (canShutdown) { if (canShutdown) {
info("ZKConsumerConnector shutting down") info("ZKConsumerConnector shutting down")
if (wildcardTopicWatcher != null)
wildcardTopicWatcher.shutdown()
try { try {
scheduler.shutdownNow() scheduler.shutdownNow()
fetcher match { fetcher match {
@ -146,59 +180,30 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
} }
def consume[T](topicCountMap: scala.collection.Map[String,Int], decoder: Decoder[T]) def consume[T](topicCountMap: scala.collection.Map[String,Int], decoder: Decoder[T])
: Map[String,List[KafkaMessageStream[T]]] = { : Map[String,List[KafkaStream[T]]] = {
debug("entering consume ") debug("entering consume ")
if (topicCountMap == null) if (topicCountMap == null)
throw new RuntimeException("topicCountMap is null") throw new RuntimeException("topicCountMap is null")
val topicCount = TopicCount.constructTopicCount(consumerIdString, topicCountMap)
val topicThreadIds = topicCount.getConsumerThreadIdsPerTopic
// make a list of (queue,stream) pairs, one pair for each threadId
val queuesAndStreams = topicThreadIds.values.map(threadIdSet =>
threadIdSet.map(_ => {
val queue = new LinkedBlockingQueue[FetchedDataChunk](config.maxQueuedChunks)
val stream = new KafkaStream[T](
queue, config.consumerTimeoutMs, decoder, config.enableShallowIterator)
(queue, stream)
})
).flatten.toList
val dirs = new ZKGroupDirs(config.groupId) val dirs = new ZKGroupDirs(config.groupId)
var ret = new mutable.HashMap[String,List[KafkaMessageStream[T]]]
var consumerUuid : String = null
config.consumerId match {
case Some(consumerId) => // for testing only
consumerUuid = consumerId
case None => // generate unique consumerId automatically
val uuid = UUID.randomUUID()
consumerUuid = "%s-%d-%s".format( InetAddress.getLocalHost.getHostName,
System.currentTimeMillis,
uuid.getMostSignificantBits().toHexString.substring(0,8) )
}
val consumerIdString = config.groupId + "_" + consumerUuid
val topicCount = new TopicCount(consumerIdString, topicCountMap)
// create a queue per topic per consumer thread
val consumerThreadIdsPerTopic = topicCount.getConsumerThreadIdsPerTopic
for ((topic, threadIdSet) <- consumerThreadIdsPerTopic) {
var streamList: List[KafkaMessageStream[T]] = Nil
for (threadId <- threadIdSet) {
val stream = new LinkedBlockingQueue[FetchedDataChunk](config.maxQueuedChunks)
queues.put((topic, threadId), stream)
streamList ::= new KafkaMessageStream[T](topic, stream, config.consumerTimeoutMs, decoder)
}
ret += (topic -> streamList)
debug("adding topic " + topic + " and stream to map..")
}
// listener to consumer and partition changes
val loadBalancerListener = new ZKRebalancerListener[T](config.groupId, consumerIdString, ret)
registerConsumerInZK(dirs, consumerIdString, topicCount) registerConsumerInZK(dirs, consumerIdString, topicCount)
reinitializeConsumer(topicCount, queuesAndStreams)
// register listener for session expired event loadBalancerListener.kafkaMessageAndMetadataStreams.asInstanceOf[Map[String, List[KafkaStream[T]]]]
zkClient.subscribeStateChanges(
new ZKSessionExpireListener[T](dirs, consumerIdString, topicCount, loadBalancerListener))
zkClient.subscribeChildChanges(dirs.consumerRegistryDir, loadBalancerListener)
ret.foreach { topicAndStreams =>
// register on broker partition path changes
val partitionPath = BrokerTopicsPath + "/" + topicAndStreams._1
zkClient.subscribeChildChanges(partitionPath, loadBalancerListener)
}
// explicitly trigger load balancing for this consumer
loadBalancerListener.syncedRebalance()
ret
} }
// this API is used by unit tests only // this API is used by unit tests only
@ -206,12 +211,14 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
private def registerConsumerInZK(dirs: ZKGroupDirs, consumerIdString: String, topicCount: TopicCount) = { private def registerConsumerInZK(dirs: ZKGroupDirs, consumerIdString: String, topicCount: TopicCount) = {
info("begin registering consumer " + consumerIdString + " in ZK") info("begin registering consumer " + consumerIdString + " in ZK")
createEphemeralPathExpectConflict(zkClient, dirs.consumerRegistryDir + "/" + consumerIdString, topicCount.toJsonString) createEphemeralPathExpectConflict(zkClient,
dirs.consumerRegistryDir + "/" + consumerIdString,
topicCount.dbString)
info("end registering consumer " + consumerIdString + " in ZK") info("end registering consumer " + consumerIdString + " in ZK")
} }
private def sendShutdownToAllQueues() = { private def sendShutdownToAllQueues() = {
for (queue <- queues.values) { for (queue <- topicThreadIdAndQueues.values) {
debug("Clearing up queue") debug("Clearing up queue")
queue.clear() queue.clear()
queue.put(ZookeeperConsumerConnector.shutdownCommand) queue.put(ZookeeperConsumerConnector.shutdownCommand)
@ -330,10 +337,10 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
producedOffset producedOffset
} }
class ZKSessionExpireListener[T](val dirs: ZKGroupDirs, class ZKSessionExpireListener(val dirs: ZKGroupDirs,
val consumerIdString: String, val consumerIdString: String,
val topicCount: TopicCount, val topicCount: TopicCount,
val loadBalancerListener: ZKRebalancerListener[T]) val loadBalancerListener: ZKRebalancerListener)
extends IZkStateListener { extends IZkStateListener {
@throws(classOf[Exception]) @throws(classOf[Exception])
def handleStateChanged(state: KeeperState) { def handleStateChanged(state: KeeperState) {
@ -355,10 +362,10 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
* consumer in the consumer registry and trigger a rebalance. * consumer in the consumer registry and trigger a rebalance.
*/ */
info("ZK expired; release old broker parition ownership; re-register consumer " + consumerIdString) info("ZK expired; release old broker parition ownership; re-register consumer " + consumerIdString)
loadBalancerListener.resetState loadBalancerListener.resetState()
registerConsumerInZK(dirs, consumerIdString, topicCount) registerConsumerInZK(dirs, consumerIdString, topicCount)
// explicitly trigger load balancing for this consumer // explicitly trigger load balancing for this consumer
loadBalancerListener.syncedRebalance loadBalancerListener.syncedRebalance()
// There is no need to resubscribe to child and state changes. // There is no need to resubscribe to child and state changes.
// The child change watchers will be set inside rebalance when we read the children list. // The child change watchers will be set inside rebalance when we read the children list.
@ -366,45 +373,69 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
} }
class ZKRebalancerListener[T](val group: String, val consumerIdString: String, class ZKRebalancerListener(val group: String, val consumerIdString: String,
kafkaMessageStreams: Map[String,List[KafkaMessageStream[T]]]) val kafkaMessageAndMetadataStreams: mutable.Map[String,List[KafkaStream[_]]])
extends IZkChildListener { extends IZkChildListener {
private val dirs = new ZKGroupDirs(group)
private var oldPartitionsPerTopicMap: mutable.Map[String, Seq[String]] = new mutable.HashMap[String, Seq[String]]() private var oldPartitionsPerTopicMap: mutable.Map[String, Seq[String]] = new mutable.HashMap[String, Seq[String]]()
private var oldConsumersPerTopicMap: mutable.Map[String,List[String]] = new mutable.HashMap[String,List[String]]() private var oldConsumersPerTopicMap: mutable.Map[String,List[String]] = new mutable.HashMap[String,List[String]]()
private var isWatcherTriggered = false
private val lock = new ReentrantLock
private val cond = lock.newCondition()
private val watcherExecutorThread = new Thread(consumerIdString + "_watcher_executor") {
override def run() {
info("starting watcher executor thread for consumer " + consumerIdString)
var doRebalance = false
while (!isShuttingDown.get) {
try {
lock.lock()
try {
if (!isWatcherTriggered)
cond.await(1000, TimeUnit.MILLISECONDS) // wake up periodically so that it can check the shutdown flag
} finally {
doRebalance = isWatcherTriggered
isWatcherTriggered = false
lock.unlock()
}
if (doRebalance)
syncedRebalance
} catch {
case t => error("error during syncedRebalance", t)
}
}
info("stopping watcher executor thread for consumer " + consumerIdString)
}
}
watcherExecutorThread.start()
@throws(classOf[Exception]) @throws(classOf[Exception])
def handleChildChange(parentPath : String, curChilds : java.util.List[String]) { def handleChildChange(parentPath : String, curChilds : java.util.List[String]) {
syncedRebalance lock.lock()
} try {
isWatcherTriggered = true
private def releasePartitionOwnership()= { cond.signalAll()
info("Releasing partition ownership") } finally {
for ((topic, infos) <- topicRegistry) { lock.unlock()
for(partition <- infos.keys) {
val partitionOwnerPath = getConsumerPartitionOwnerPath(group, topic, partition.toString)
deletePath(zkClient, partitionOwnerPath)
debug("Consumer " + consumerIdString + " releasing " + partitionOwnerPath)
}
} }
} }
private def getRelevantTopicMap(myTopicThreadIdsMap: Map[String, Set[String]], private def deletePartitionOwnershipFromZK(topic: String, partition: String) {
newPartMap: Map[String, Seq[String]], val topicDirs = new ZKGroupTopicDirs(group, topic)
oldPartMap: Map[String, Seq[String]], val znode = topicDirs.consumerOwnerDir + "/" + partition
newConsumerMap: Map[String,List[String]], deletePath(zkClient, znode)
oldConsumerMap: Map[String,List[String]]): Map[String, Set[String]] = { debug("Consumer " + consumerIdString + " releasing " + znode)
var relevantTopicThreadIdsMap = new mutable.HashMap[String, Set[String]]() }
for ( (topic, consumerThreadIdSet) <- myTopicThreadIdsMap )
if ( oldPartMap.get(topic) != newPartMap.get(topic) || oldConsumerMap.get(topic) != newConsumerMap.get(topic)) private def releasePartitionOwnership(localTopicRegistry: Pool[String, Pool[Int, PartitionTopicInfo]])= {
relevantTopicThreadIdsMap += (topic -> consumerThreadIdSet) info("Releasing partition ownership")
relevantTopicThreadIdsMap for ((topic, infos) <- localTopicRegistry) {
for(partition <- infos.keys)
deletePartitionOwnershipFromZK(topic, partition.toString)
localTopicRegistry.remove(topic)
}
} }
def resetState() { def resetState() {
topicRegistry.clear topicRegistry.clear
oldConsumersPerTopicMap.clear
oldPartitionsPerTopicMap.clear
} }
def syncedRebalance() { def syncedRebalance() {
@ -422,8 +453,6 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
* the value of a child. Just let this go since another rebalance will be triggered. * the value of a child. Just let this go since another rebalance will be triggered.
**/ **/
info("exception during rebalance ", e) info("exception during rebalance ", e)
/* Explicitly make sure another rebalancing attempt will get triggered. */
done = false
} }
info("end rebalancing consumer " + consumerIdString + " try #" + i) info("end rebalancing consumer " + consumerIdString + " try #" + i)
if (done) { if (done) {
@ -432,15 +461,9 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
/* Here the cache is at a risk of being stale. To take future rebalancing decisions correctly, we should /* Here the cache is at a risk of being stale. To take future rebalancing decisions correctly, we should
* clear the cache */ * clear the cache */
info("Rebalancing attempt failed. Clearing the cache before the next rebalancing operation is triggered") info("Rebalancing attempt failed. Clearing the cache before the next rebalancing operation is triggered")
oldConsumersPerTopicMap.clear()
oldPartitionsPerTopicMap.clear()
} }
// commit offsets
commitOffsets()
// stop all fetchers and clear all the queues to avoid data duplication // stop all fetchers and clear all the queues to avoid data duplication
closeFetchersForQueues(cluster, kafkaMessageStreams, queues.map(q => q._2)) closeFetchersForQueues(cluster, kafkaMessageAndMetadataStreams, topicThreadIdAndQueues.map(q => q._2))
// release all partitions, reset state and retry
releasePartitionOwnership()
Thread.sleep(config.rebalanceBackoffMs) Thread.sleep(config.rebalanceBackoffMs)
} }
} }
@ -449,17 +472,9 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
} }
private def rebalance(cluster: Cluster): Boolean = { private def rebalance(cluster: Cluster): Boolean = {
val myTopicThreadIdsMap = getTopicCount(zkClient, group, consumerIdString).getConsumerThreadIdsPerTopic val myTopicThreadIdsMap = TopicCount.constructTopicCount(group, consumerIdString, zkClient).getConsumerThreadIdsPerTopic
val consumersPerTopicMap = getConsumersPerTopic(zkClient, group) val consumersPerTopicMap = getConsumersPerTopic(zkClient, group)
val partitionsPerTopicMap = getPartitionsForTopics(zkClient, myTopicThreadIdsMap.keys.iterator) val partitionsPerTopicMap = getPartitionsForTopics(zkClient, myTopicThreadIdsMap.keys.iterator)
val relevantTopicThreadIdsMap = getRelevantTopicMap(myTopicThreadIdsMap, partitionsPerTopicMap, oldPartitionsPerTopicMap, consumersPerTopicMap, oldConsumersPerTopicMap)
if (relevantTopicThreadIdsMap.size <= 0) {
info("Consumer %s with %s and topic partitions %s doesn't need to rebalance.".
format(consumerIdString, consumersPerTopicMap, partitionsPerTopicMap))
debug("Partitions per topic cache " + oldPartitionsPerTopicMap)
debug("Consumers per topic cache " + oldConsumersPerTopicMap)
return true
}
/** /**
* fetchers must be stopped to avoid data duplication, since if the current * fetchers must be stopped to avoid data duplication, since if the current
@ -467,14 +482,15 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
* But if we don't stop the fetchers first, this consumer would continue returning data for released * But if we don't stop the fetchers first, this consumer would continue returning data for released
* partitions in parallel. So, not stopping the fetchers leads to duplicate data. * partitions in parallel. So, not stopping the fetchers leads to duplicate data.
*/ */
closeFetchers(cluster, kafkaMessageStreams, relevantTopicThreadIdsMap) closeFetchers(cluster, kafkaMessageAndMetadataStreams, myTopicThreadIdsMap)
releasePartitionOwnership() releasePartitionOwnership(topicRegistry)
var partitionOwnershipDecision = new collection.mutable.HashMap[(String, String), String]() var partitionOwnershipDecision = new collection.mutable.HashMap[(String, String), String]()
for ((topic, consumerThreadIdSet) <- relevantTopicThreadIdsMap) { var currentTopicRegistry = new Pool[String, Pool[Int, PartitionTopicInfo]]
topicRegistry.remove(topic)
topicRegistry.put(topic, new Pool[Int, PartitionTopicInfo]) for ((topic, consumerThreadIdSet) <- myTopicThreadIdsMap) {
currentTopicRegistry.put(topic, new Pool[Int, PartitionTopicInfo])
val topicDirs = new ZKGroupTopicDirs(group, topic) val topicDirs = new ZKGroupTopicDirs(group, topic)
val curConsumers = consumersPerTopicMap.get(topic).get val curConsumers = consumersPerTopicMap.get(topic).get
@ -502,11 +518,9 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
for (i <- startPart until startPart + nParts) { for (i <- startPart until startPart + nParts) {
val partition = curPartitions(i) val partition = curPartitions(i)
info(consumerThreadId + " attempting to claim partition " + partition) info(consumerThreadId + " attempting to claim partition " + partition)
val ownPartition = processPartition(topicDirs, partition, topic, consumerThreadId) addPartitionTopicInfo(currentTopicRegistry, topicDirs, partition, topic, consumerThreadId)
if (!ownPartition) // record the partition ownership decision
return false partitionOwnershipDecision += ((topic, partition) -> consumerThreadId)
else // record the partition ownership decision
partitionOwnershipDecision += ((topic, partition) -> consumerThreadId)
} }
} }
} }
@ -520,21 +534,21 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
info("Updating the cache") info("Updating the cache")
debug("Partitions per topic cache " + partitionsPerTopicMap) debug("Partitions per topic cache " + partitionsPerTopicMap)
debug("Consumers per topic cache " + consumersPerTopicMap) debug("Consumers per topic cache " + consumersPerTopicMap)
oldPartitionsPerTopicMap = partitionsPerTopicMap topicRegistry = currentTopicRegistry
oldConsumersPerTopicMap = consumersPerTopicMap updateFetcher(cluster)
updateFetcher(cluster, kafkaMessageStreams)
true true
} else } else {
false false
}
} }
private def closeFetchersForQueues(cluster: Cluster, private def closeFetchersForQueues(cluster: Cluster,
kafkaMessageStreams: Map[String,List[KafkaMessageStream[T]]], messageStreams: Map[String,List[KafkaStream[_]]],
queuesToBeCleared: Iterable[BlockingQueue[FetchedDataChunk]]) { queuesToBeCleared: Iterable[BlockingQueue[FetchedDataChunk]]) {
var allPartitionInfos = topicRegistry.values.map(p => p.values).flatten var allPartitionInfos = topicRegistry.values.map(p => p.values).flatten
fetcher match { fetcher match {
case Some(f) => f.stopConnectionsToAllBrokers case Some(f) => f.stopConnectionsToAllBrokers
f.clearFetcherQueues(allPartitionInfos, cluster, queuesToBeCleared, kafkaMessageStreams) f.clearFetcherQueues(allPartitionInfos, cluster, queuesToBeCleared, messageStreams)
info("Committing all offsets after clearing the fetcher queues") info("Committing all offsets after clearing the fetcher queues")
/** /**
* here, we need to commit offsets before stopping the consumer from returning any more messages * here, we need to commit offsets before stopping the consumer from returning any more messages
@ -549,16 +563,15 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
} }
} }
private def closeFetchers(cluster: Cluster, kafkaMessageStreams: Map[String,List[KafkaMessageStream[T]]], private def closeFetchers(cluster: Cluster, messageStreams: Map[String,List[KafkaStream[_]]],
relevantTopicThreadIdsMap: Map[String, Set[String]]) { relevantTopicThreadIdsMap: Map[String, Set[String]]) {
// only clear the fetcher queues for certain topic partitions that *might* no longer be served by this consumer // only clear the fetcher queues for certain topic partitions that *might* no longer be served by this consumer
// after this rebalancing attempt // after this rebalancing attempt
val queuesTobeCleared = queues.filter(q => relevantTopicThreadIdsMap.contains(q._1._1)).map(q => q._2) val queuesTobeCleared = topicThreadIdAndQueues.filter(q => relevantTopicThreadIdsMap.contains(q._1._1)).map(q => q._2)
closeFetchersForQueues(cluster, kafkaMessageStreams, queuesTobeCleared) closeFetchersForQueues(cluster, messageStreams, queuesTobeCleared)
} }
private def updateFetcher[T](cluster: Cluster, private def updateFetcher(cluster: Cluster) {
kafkaMessageStreams: Map[String,List[KafkaMessageStream[T]]]) {
// update partitions for fetcher // update partitions for fetcher
var allPartitionInfos : List[PartitionTopicInfo] = Nil var allPartitionInfos : List[PartitionTopicInfo] = Nil
for (partitionInfos <- topicRegistry.values) for (partitionInfos <- topicRegistry.values)
@ -569,33 +582,13 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
fetcher match { fetcher match {
case Some(f) => case Some(f) =>
f.startConnections(allPartitionInfos, cluster, kafkaMessageStreams) f.startConnections(allPartitionInfos, cluster)
case None => case None =>
} }
} }
private def processPartition(topicDirs: ZKGroupTopicDirs, partition: String,
topic: String, consumerThreadId: String) : Boolean = {
val partitionOwnerPath = topicDirs.consumerOwnerDir + "/" + partition
// check if some other consumer owns this partition at this time
val currentPartitionOwner = readDataMaybeNull(zkClient, partitionOwnerPath)
if(currentPartitionOwner != null) {
if(currentPartitionOwner.equals(consumerThreadId)) {
info(partitionOwnerPath + " exists with value " + currentPartitionOwner + " during connection loss; this is ok")
addPartitionTopicInfo(topicDirs, partition, topic, consumerThreadId)
true
}
else {
info(partitionOwnerPath + " exists with value " + currentPartitionOwner)
false
}
} else {
addPartitionTopicInfo(topicDirs, partition, topic, consumerThreadId)
true
}
}
private def reflectPartitionOwnershipDecision(partitionOwnershipDecision: Map[(String, String), String]): Boolean = { private def reflectPartitionOwnershipDecision(partitionOwnershipDecision: Map[(String, String), String]): Boolean = {
var successfullyOwnedPartitions : List[(String, String)] = Nil
val partitionOwnershipSuccessful = partitionOwnershipDecision.map { partitionOwner => val partitionOwnershipSuccessful = partitionOwnershipDecision.map { partitionOwner =>
val topic = partitionOwner._1._1 val topic = partitionOwner._1._1
val partition = partitionOwner._1._2 val partition = partitionOwner._1._2
@ -604,6 +597,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
try { try {
createEphemeralPathExpectConflict(zkClient, partitionOwnerPath, consumerThreadId) createEphemeralPathExpectConflict(zkClient, partitionOwnerPath, consumerThreadId)
info(consumerThreadId + " successfully owned partition " + partition + " for topic " + topic) info(consumerThreadId + " successfully owned partition " + partition + " for topic " + topic)
successfullyOwnedPartitions ::= (topic, partition)
true true
} catch { } catch {
case e: ZkNodeExistsException => case e: ZkNodeExistsException =>
@ -613,14 +607,20 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
case e2 => throw e2 case e2 => throw e2
} }
} }
val success = partitionOwnershipSuccessful.foldLeft(0)((sum, decision) => if(decision) 0 else 1) val hasPartitionOwnershipFailed = partitionOwnershipSuccessful.foldLeft(0)((sum, decision) => sum + (if(decision) 0 else 1))
if(success > 0) false /* even if one of the partition ownership attempt has failed, return false */ /* even if one of the partition ownership attempt has failed, return false */
if(hasPartitionOwnershipFailed > 0) {
// remove all paths that we have owned in ZK
successfullyOwnedPartitions.foreach(topicAndPartition => deletePartitionOwnershipFromZK(topicAndPartition._1, topicAndPartition._2))
false
}
else true else true
} }
private def addPartitionTopicInfo(topicDirs: ZKGroupTopicDirs, partition: String, private def addPartitionTopicInfo(currentTopicRegistry: Pool[String, Pool[Int, PartitionTopicInfo]],
topicDirs: ZKGroupTopicDirs, partition: String,
topic: String, consumerThreadId: String) { topic: String, consumerThreadId: String) {
val partTopicInfoMap = topicRegistry.get(topic) val partTopicInfoMap = currentTopicRegistry.get(topic)
// find the leader for this partition // find the leader for this partition
val leaderOpt = getLeaderForPartition(zkClient, topic, partition.toInt) val leaderOpt = getLeaderForPartition(zkClient, topic, partition.toInt)
@ -646,8 +646,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
} }
else else
offset = offsetString.toLong offset = offsetString.toLong
val queue = topicThreadIdAndQueues.get((topic, consumerThreadId))
val queue = queues.get((topic, consumerThreadId))
val consumedOffset = new AtomicLong(offset) val consumedOffset = new AtomicLong(offset)
val fetchedOffset = new AtomicLong(offset) val fetchedOffset = new AtomicLong(offset)
val partTopicInfo = new PartitionTopicInfo(topic, val partTopicInfo = new PartitionTopicInfo(topic,
@ -661,5 +660,155 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
debug(partTopicInfo + " selected new offset " + offset) debug(partTopicInfo + " selected new offset " + offset)
} }
} }
private def reinitializeConsumer[T](
topicCount: TopicCount,
queuesAndStreams: List[(LinkedBlockingQueue[FetchedDataChunk],KafkaStream[T])]) {
val dirs = new ZKGroupDirs(config.groupId)
// listener to consumer and partition changes
if (loadBalancerListener == null) {
val topicStreamsMap = new mutable.HashMap[String,List[KafkaStream[T]]]
loadBalancerListener = new ZKRebalancerListener(
config.groupId, consumerIdString, topicStreamsMap.asInstanceOf[scala.collection.mutable.Map[String, List[KafkaStream[_]]]])
}
// register listener for session expired event
if (sessionExpirationListener == null)
sessionExpirationListener = new ZKSessionExpireListener(
dirs, consumerIdString, topicCount, loadBalancerListener)
val topicStreamsMap = loadBalancerListener.kafkaMessageAndMetadataStreams
// map of {topic -> Set(thread-1, thread-2, ...)}
val consumerThreadIdsPerTopic: Map[String, Set[String]] =
topicCount.getConsumerThreadIdsPerTopic
/*
* This usage of map flatten breaks up consumerThreadIdsPerTopic into
* a set of (topic, thread-id) pairs that we then use to construct
* the updated (topic, thread-id) -> queues map
*/
implicit def getTopicThreadIds(v: (String, Set[String])): Set[(String, String)] = v._2.map((v._1, _))
// iterator over (topic, thread-id) tuples
val topicThreadIds: Iterable[(String, String)] =
consumerThreadIdsPerTopic.flatten
// list of (pairs of pairs): e.g., ((topic, thread-id),(queue, stream))
val threadQueueStreamPairs = topicCount match {
case wildTopicCount: WildcardTopicCount =>
for (tt <- topicThreadIds; qs <- queuesAndStreams) yield (tt -> qs)
case statTopicCount: StaticTopicCount => {
require(topicThreadIds.size == queuesAndStreams.size,
"Mismatch between thread ID count (%d) and queue count (%d)".format(
topicThreadIds.size, queuesAndStreams.size))
topicThreadIds.zip(queuesAndStreams)
}
}
threadQueueStreamPairs.foreach(e => {
val topicThreadId = e._1
val q = e._2._1
topicThreadIdAndQueues.put(topicThreadId, q)
})
val groupedByTopic = threadQueueStreamPairs.groupBy(_._1._1)
groupedByTopic.foreach(e => {
val topic = e._1
val streams = e._2.map(_._2._2).toList
topicStreamsMap += (topic -> streams)
debug("adding topic %s and %d streams to map.".format(topic, streams.size))
})
// listener to consumer and partition changes
zkClient.subscribeStateChanges(sessionExpirationListener)
zkClient.subscribeChildChanges(dirs.consumerRegistryDir, loadBalancerListener)
topicStreamsMap.foreach { topicAndStreams =>
// register on broker partition path changes
val partitionPath = BrokerTopicsPath + "/" + topicAndStreams._1
zkClient.subscribeChildChanges(partitionPath, loadBalancerListener)
}
// explicitly trigger load balancing for this consumer
loadBalancerListener.syncedRebalance()
}
class WildcardStreamsHandler[T](topicFilter: TopicFilter,
numStreams: Int,
decoder: Decoder[T])
extends TopicEventHandler[String] {
if (messageStreamCreated.getAndSet(true))
throw new RuntimeException("Each consumer connector can create " +
"message streams by filter at most once.")
private val wildcardQueuesAndStreams = (1 to numStreams)
.map(e => {
val queue = new LinkedBlockingQueue[FetchedDataChunk](config.maxQueuedChunks)
val stream = new KafkaStream[T](
queue, config.consumerTimeoutMs, decoder, config.enableShallowIterator)
(queue, stream)
}).toList
// bootstrap with existing topics
private var wildcardTopics =
getChildrenParentMayNotExist(zkClient, BrokerTopicsPath)
.filter(topicFilter.isTopicAllowed)
private val wildcardTopicCount = TopicCount.constructTopicCount(
consumerIdString, topicFilter, numStreams, zkClient)
val dirs = new ZKGroupDirs(config.groupId)
registerConsumerInZK(dirs, consumerIdString, wildcardTopicCount)
reinitializeConsumer(wildcardTopicCount, wildcardQueuesAndStreams)
if (!topicFilter.requiresTopicEventWatcher) {
info("Not creating event watcher for trivial whitelist " + topicFilter)
}
else {
info("Creating topic event watcher for whitelist " + topicFilter)
wildcardTopicWatcher = new ZookeeperTopicEventWatcher(config, this)
/*
* Topic events will trigger subsequent synced rebalances. Also, the
* consumer will get registered only after an allowed topic becomes
* available.
*/
}
def handleTopicEvent(allTopics: Seq[String]) {
debug("Handling topic event")
val updatedTopics = allTopics.filter(topicFilter.isTopicAllowed)
val addedTopics = updatedTopics filterNot (wildcardTopics contains)
if (addedTopics.nonEmpty)
info("Topic event: added topics = %s"
.format(addedTopics))
/*
* TODO: Deleted topics are interesting (and will not be a concern until
* 0.8 release). We may need to remove these topics from the rebalance
* listener's map in reinitializeConsumer.
*/
val deletedTopics = wildcardTopics filterNot (updatedTopics contains)
if (deletedTopics.nonEmpty)
info("Topic event: deleted topics = %s"
.format(deletedTopics))
wildcardTopics = updatedTopics
info("Topics to consume = %s".format(wildcardTopics))
if (addedTopics.nonEmpty || deletedTopics.nonEmpty)
reinitializeConsumer(wildcardTopicCount, wildcardQueuesAndStreams)
}
def streams: Seq[KafkaStream[T]] =
wildcardQueuesAndStreams.map(_._2)
}
} }

View File

@ -21,11 +21,9 @@ import scala.collection.JavaConversions._
import kafka.utils.{ZkUtils, ZKStringSerializer, Logging} import kafka.utils.{ZkUtils, ZKStringSerializer, Logging}
import org.I0Itec.zkclient.{IZkStateListener, IZkChildListener, ZkClient} import org.I0Itec.zkclient.{IZkStateListener, IZkChildListener, ZkClient}
import org.apache.zookeeper.Watcher.Event.KeeperState import org.apache.zookeeper.Watcher.Event.KeeperState
import kafka.server.KafkaServerStartable
import kafka.common.ConsumerRebalanceFailedException
class ZookeeperTopicEventWatcher(val config:ConsumerConfig, class ZookeeperTopicEventWatcher(val config:ConsumerConfig,
val eventHandler: TopicEventHandler[String], kafkaServerStartable: KafkaServerStartable) extends Logging { val eventHandler: TopicEventHandler[String]) extends Logging {
val lock = new Object() val lock = new Object()
@ -35,7 +33,7 @@ class ZookeeperTopicEventWatcher(val config:ConsumerConfig,
startWatchingTopicEvents() startWatchingTopicEvents()
private def startWatchingTopicEvents() { private def startWatchingTopicEvents() {
val topicEventListener = new ZkTopicEventListener(kafkaServerStartable) val topicEventListener = new ZkTopicEventListener()
ZkUtils.makeSurePersistentPathExists(zkClient, ZkUtils.BrokerTopicsPath) ZkUtils.makeSurePersistentPathExists(zkClient, ZkUtils.BrokerTopicsPath)
zkClient.subscribeStateChanges( zkClient.subscribeStateChanges(
@ -52,6 +50,7 @@ class ZookeeperTopicEventWatcher(val config:ConsumerConfig,
def shutdown() { def shutdown() {
lock.synchronized { lock.synchronized {
info("Shutting down topic event watcher.")
if (zkClient != null) { if (zkClient != null) {
stopWatchingTopicEvents() stopWatchingTopicEvents()
zkClient.close() zkClient.close()
@ -62,7 +61,7 @@ class ZookeeperTopicEventWatcher(val config:ConsumerConfig,
} }
} }
class ZkTopicEventListener(val kafkaServerStartable: KafkaServerStartable) extends IZkChildListener { class ZkTopicEventListener extends IZkChildListener {
@throws(classOf[Exception]) @throws(classOf[Exception])
def handleChildChange(parent: String, children: java.util.List[String]) { def handleChildChange(parent: String, children: java.util.List[String]) {
@ -76,11 +75,8 @@ class ZookeeperTopicEventWatcher(val config:ConsumerConfig,
} }
} }
catch { catch {
case e: ConsumerRebalanceFailedException =>
fatal("can't rebalance in embedded consumer; proceed to shutdown", e)
kafkaServerStartable.shutdown()
case e => case e =>
error("error in handling child changes in embedded consumer", e) error("error in handling child changes", e)
} }
} }
} }

View File

@ -17,34 +17,53 @@
package kafka.javaapi.consumer; package kafka.javaapi.consumer;
import kafka.consumer.KafkaMessageStream;
import kafka.message.Message;
import kafka.serializer.Decoder;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import kafka.consumer.KafkaStream;
import kafka.consumer.TopicFilter;
import kafka.message.Message;
import kafka.serializer.Decoder;
public interface ConsumerConnector { public interface ConsumerConnector {
/** /**
* Create a list of MessageStreams of type T for each topic. * Create a list of MessageStreams of type T for each topic.
* *
* @param topicCountMap a map of (topic, #streams) pair * @param topicCountMap a map of (topic, #streams) pair
* @param decoder a decoder that converts from Message to T * @param decoder a decoder that converts from Message to T
* @return a map of (topic, list of KafkaMessageStream) pair. The number of items in the * @return a map of (topic, list of KafkaStream) pairs.
* list is #streams. Each KafkaMessageStream supports an iterator of messages. * The number of items in the list is #streams. Each stream supports
*/ * an iterator over message/metadata pairs.
public <T> Map<String, List<KafkaMessageStream<T>>> createMessageStreams( */
Map<String, Integer> topicCountMap, Decoder<T> decoder); public <T> Map<String, List<KafkaStream<T>>> createMessageStreams(
public Map<String, List<KafkaMessageStream<Message>>> createMessageStreams( Map<String, Integer> topicCountMap, Decoder<T> decoder);
Map<String, Integer> topicCountMap); public Map<String, List<KafkaStream<Message>>> createMessageStreams(
Map<String, Integer> topicCountMap);
/** /**
* Commit the offsets of all broker partitions connected by this connector. * Create a list of MessageAndTopicStreams containing messages of type T.
*/ *
public void commitOffsets(); * @param topicFilter a TopicFilter that specifies which topics to
* subscribe to (encapsulates a whitelist or a blacklist).
* @param numStreams the number of message streams to return.
* @param decoder a decoder that converts from Message to T
* @return a list of KafkaStream. Each stream supports an
* iterator over its MessageAndMetadata elements.
*/
public <T> List<KafkaStream<T>> createMessageStreamsByFilter(
TopicFilter topicFilter, int numStreams, Decoder<T> decoder);
public List<KafkaStream<Message>> createMessageStreamsByFilter(
TopicFilter topicFilter, int numStreams);
public List<KafkaStream<Message>> createMessageStreamsByFilter(
TopicFilter topicFilter);
/** /**
* Shut down the connector * Commit the offsets of all broker partitions connected by this connector.
*/ */
public void shutdown(); public void commitOffsets();
/**
* Shut down the connector
*/
public void shutdown();
} }

View File

@ -16,9 +16,11 @@
*/ */
package kafka.javaapi.consumer package kafka.javaapi.consumer
import kafka.consumer.{KafkaMessageStream, ConsumerConfig}
import kafka.message.Message import kafka.message.Message
import kafka.serializer.{DefaultDecoder, Decoder} import kafka.serializer.{DefaultDecoder, Decoder}
import kafka.consumer._
import scala.collection.JavaConversions.asList
/** /**
* This class handles the consumers interaction with zookeeper * This class handles the consumers interaction with zookeeper
@ -68,14 +70,14 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
def createMessageStreams[T]( def createMessageStreams[T](
topicCountMap: java.util.Map[String,java.lang.Integer], topicCountMap: java.util.Map[String,java.lang.Integer],
decoder: Decoder[T]) decoder: Decoder[T])
: java.util.Map[String,java.util.List[KafkaMessageStream[T]]] = { : java.util.Map[String,java.util.List[KafkaStream[T]]] = {
import scala.collection.JavaConversions._ import scala.collection.JavaConversions._
val scalaTopicCountMap: Map[String, Int] = Map.empty[String, Int] ++ asMap(topicCountMap.asInstanceOf[java.util.Map[String, Int]]) val scalaTopicCountMap: Map[String, Int] = Map.empty[String, Int] ++ asMap(topicCountMap.asInstanceOf[java.util.Map[String, Int]])
val scalaReturn = underlying.consume(scalaTopicCountMap, decoder) val scalaReturn = underlying.consume(scalaTopicCountMap, decoder)
val ret = new java.util.HashMap[String,java.util.List[KafkaMessageStream[T]]] val ret = new java.util.HashMap[String,java.util.List[KafkaStream[T]]]
for ((topic, streams) <- scalaReturn) { for ((topic, streams) <- scalaReturn) {
var javaStreamList = new java.util.ArrayList[KafkaMessageStream[T]] var javaStreamList = new java.util.ArrayList[KafkaStream[T]]
for (stream <- streams) for (stream <- streams)
javaStreamList.add(stream) javaStreamList.add(stream)
ret.put(topic, javaStreamList) ret.put(topic, javaStreamList)
@ -85,9 +87,17 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
def createMessageStreams( def createMessageStreams(
topicCountMap: java.util.Map[String,java.lang.Integer]) topicCountMap: java.util.Map[String,java.lang.Integer])
: java.util.Map[String,java.util.List[KafkaMessageStream[Message]]] = : java.util.Map[String,java.util.List[KafkaStream[Message]]] =
createMessageStreams(topicCountMap, new DefaultDecoder) createMessageStreams(topicCountMap, new DefaultDecoder)
def createMessageStreamsByFilter[T](topicFilter: TopicFilter, numStreams: Int, decoder: Decoder[T]) =
asList(underlying.createMessageStreamsByFilter(topicFilter, numStreams, decoder))
def createMessageStreamsByFilter(topicFilter: TopicFilter, numStreams: Int) =
createMessageStreamsByFilter(topicFilter, numStreams, new DefaultDecoder)
def createMessageStreamsByFilter(topicFilter: TopicFilter) =
createMessageStreamsByFilter(topicFilter, 1, new DefaultDecoder)
def commitOffsets() { def commitOffsets() {
underlying.commitOffsets underlying.commitOffsets

View File

@ -17,8 +17,10 @@
package kafka.javaapi.message package kafka.javaapi.message
import kafka.message.{MessageAndOffset, InvalidMessageException} import kafka.message.{MessageAndOffset, InvalidMessageException}
/** /**
* A set of messages. A message set has a fixed serialized form, though the container * A set of messages. A message set has a fixed serialized form, though the container
* for the bytes could be either in-memory or on disk. A The format of each message is * for the bytes could be either in-memory or on disk. A The format of each message is

View File

@ -25,6 +25,7 @@ import kafka.utils._
import kafka.common._ import kafka.common._
import kafka.api.OffsetRequest import kafka.api.OffsetRequest
import java.util._ import java.util._
import kafka.server.BrokerTopicStat
private[kafka] object Log { private[kafka] object Log {
val FileSuffix = ".kafka" val FileSuffix = ".kafka"
@ -199,7 +200,7 @@ private[kafka] class Log(val dir: File, val maxSize: Long, val flushInterval: In
* Append this message set to the active segment of the log, rolling over to a fresh segment if necessary. * Append this message set to the active segment of the log, rolling over to a fresh segment if necessary.
* Returns the offset at which the messages are written. * Returns the offset at which the messages are written.
*/ */
def append(messages: MessageSet): Unit = { def append(messages: ByteBufferMessageSet): Unit = {
// validate the messages // validate the messages
var numberOfMessages = 0 var numberOfMessages = 0
for(messageAndOffset <- messages) { for(messageAndOffset <- messages) {
@ -208,13 +209,25 @@ private[kafka] class Log(val dir: File, val maxSize: Long, val flushInterval: In
numberOfMessages += 1; numberOfMessages += 1;
} }
BrokerTopicStat.getBrokerTopicStat(getTopicName).recordMessagesIn(numberOfMessages)
BrokerTopicStat.getBrokerAllTopicStat.recordMessagesIn(numberOfMessages)
logStats.recordAppendedMessages(numberOfMessages) logStats.recordAppendedMessages(numberOfMessages)
// truncate the message set's buffer upto validbytes, before appending it to the on-disk log
val validByteBuffer = messages.getBuffer.duplicate()
val messageSetValidBytes = messages.validBytes
if(messageSetValidBytes > Int.MaxValue || messageSetValidBytes < 0)
throw new InvalidMessageSizeException("Illegal length of message set " + messageSetValidBytes +
" Message set cannot be appended to log. Possible causes are corrupted produce requests")
validByteBuffer.limit(messageSetValidBytes.asInstanceOf[Int])
val validMessages = new ByteBufferMessageSet(validByteBuffer)
// they are valid, insert them in the log // they are valid, insert them in the log
lock synchronized { lock synchronized {
try { try {
val segment = segments.view.last val segment = segments.view.last
segment.messageSet.append(messages) segment.messageSet.append(validMessages)
maybeFlush(numberOfMessages) maybeFlush(numberOfMessages)
maybeRoll(segment) maybeRoll(segment)
} }
@ -247,10 +260,18 @@ private[kafka] class Log(val dir: File, val maxSize: Long, val flushInterval: In
val deletable = view.takeWhile(predicate) val deletable = view.takeWhile(predicate)
for(seg <- deletable) for(seg <- deletable)
seg.deleted = true seg.deleted = true
val numToDelete = deletable.size var numToDelete = deletable.size
// if we are deleting everything, create a new empty segment // if we are deleting everything, create a new empty segment
if(numToDelete == view.size) if(numToDelete == view.size) {
roll() if (view(numToDelete - 1).size > 0)
roll()
else {
// If the last segment to be deleted is empty and we roll the log, the new segment will have the same
// file name. So simply reuse the last segment and reset the modified time.
view(numToDelete - 1).file.setLastModified(SystemTime.milliseconds)
numToDelete -=1
}
}
segments.trunc(numToDelete) segments.trunc(numToDelete)
} }
} }
@ -288,9 +309,12 @@ private[kafka] class Log(val dir: File, val maxSize: Long, val flushInterval: In
*/ */
def roll() { def roll() {
lock synchronized { lock synchronized {
val last = segments.view.last
val newOffset = nextAppendOffset val newOffset = nextAppendOffset
val newFile = new File(dir, Log.nameFromOffset(newOffset)) val newFile = new File(dir, Log.nameFromOffset(newOffset))
if (newFile.exists) {
warn("newly rolled logsegment " + newFile.getName + " already exists; deleting it first")
newFile.delete()
}
debug("Rolling log '" + name + "' to " + newFile.getName()) debug("Rolling log '" + name + "' to " + newFile.getName())
segments.append(new LogSegment(newFile, new FileMessageSet(newFile, true), newOffset)) segments.append(new LogSegment(newFile, new FileMessageSet(newFile, true), newOffset))
} }

View File

@ -18,10 +18,10 @@
package kafka.message package kafka.message
import kafka.utils.Logging import kafka.utils.Logging
import kafka.common.{InvalidMessageSizeException, ErrorMapping}
import java.nio.ByteBuffer import java.nio.ByteBuffer
import java.nio.channels._ import java.nio.channels._
import kafka.utils.IteratorTemplate import kafka.utils.IteratorTemplate
import kafka.common.{MessageSizeTooLargeException, InvalidMessageSizeException, ErrorMapping}
/** /**
* A sequence of messages stored in a byte buffer * A sequence of messages stored in a byte buffer
@ -36,8 +36,9 @@ import kafka.utils.IteratorTemplate
class ByteBufferMessageSet(private val buffer: ByteBuffer, class ByteBufferMessageSet(private val buffer: ByteBuffer,
private val initialOffset: Long = 0L, private val initialOffset: Long = 0L,
private val errorCode: Int = ErrorMapping.NoError) extends MessageSet with Logging { private val errorCode: Int = ErrorMapping.NoError) extends MessageSet with Logging {
private var validByteCount = -1L
private var shallowValidByteCount = -1L private var shallowValidByteCount = -1L
if(sizeInBytes > Int.MaxValue)
throw new InvalidMessageSizeException("Message set cannot be larger than " + Int.MaxValue)
def this(compressionCodec: CompressionCodec, messages: Message*) { def this(compressionCodec: CompressionCodec, messages: Message*) {
this(MessageSet.createByteBuffer(compressionCodec, messages:_*), 0L, ErrorMapping.NoError) this(MessageSet.createByteBuffer(compressionCodec, messages:_*), 0L, ErrorMapping.NoError)
@ -59,7 +60,7 @@ class ByteBufferMessageSet(private val buffer: ByteBuffer,
private def shallowValidBytes: Long = { private def shallowValidBytes: Long = {
if(shallowValidByteCount < 0) { if(shallowValidByteCount < 0) {
val iter = deepIterator val iter = this.internalIterator(true)
while(iter.hasNext) { while(iter.hasNext) {
val messageAndOffset = iter.next val messageAndOffset = iter.next
shallowValidByteCount = messageAndOffset.offset shallowValidByteCount = messageAndOffset.offset
@ -70,12 +71,31 @@ class ByteBufferMessageSet(private val buffer: ByteBuffer,
} }
/** Write the messages in this set to the given channel */ /** Write the messages in this set to the given channel */
def writeTo(channel: GatheringByteChannel, offset: Long, size: Long): Long = def writeTo(channel: GatheringByteChannel, offset: Long, size: Long): Long = {
channel.write(buffer.duplicate) buffer.mark()
val written = channel.write(buffer)
override def iterator: Iterator[MessageAndOffset] = deepIterator buffer.reset()
written
}
private def deepIterator(): Iterator[MessageAndOffset] = { /** default iterator that iterates over decompressed messages */
override def iterator: Iterator[MessageAndOffset] = internalIterator()
/** iterator over compressed messages without decompressing */
def shallowIterator: Iterator[MessageAndOffset] = internalIterator(true)
def verifyMessageSize(maxMessageSize: Int){
var shallowIter = internalIterator(true)
while(shallowIter.hasNext){
var messageAndOffset = shallowIter.next
val payloadSize = messageAndOffset.message.payloadSize
if ( payloadSize > maxMessageSize)
throw new MessageSizeTooLargeException("payload size of " + payloadSize + " larger than " + maxMessageSize)
}
}
/** When flag isShallow is set to be true, we do a shallow iteration: just traverse the first level of messages. This is used in verifyMessageSize() function **/
private def internalIterator(isShallow: Boolean = false): Iterator[MessageAndOffset] = {
ErrorMapping.maybeThrowException(errorCode) ErrorMapping.maybeThrowException(errorCode)
new IteratorTemplate[MessageAndOffset] { new IteratorTemplate[MessageAndOffset] {
var topIter = buffer.slice() var topIter = buffer.slice()
@ -106,33 +126,50 @@ class ByteBufferMessageSet(private val buffer: ByteBuffer,
message.limit(size) message.limit(size)
topIter.position(topIter.position + size) topIter.position(topIter.position + size)
val newMessage = new Message(message) val newMessage = new Message(message)
newMessage.compressionCodec match { if(!newMessage.isValid)
case NoCompressionCodec => throw new InvalidMessageException("message is invalid, compression codec: " + newMessage.compressionCodec
debug("Message is uncompressed. Valid byte count = %d".format(currValidBytes)) + " size: " + size + " curr offset: " + currValidBytes + " init offset: " + initialOffset)
innerIter = null
currValidBytes += 4 + size if(isShallow){
trace("currValidBytes = " + currValidBytes) currValidBytes += 4 + size
new MessageAndOffset(newMessage, currValidBytes) trace("shallow iterator currValidBytes = " + currValidBytes)
case _ => new MessageAndOffset(newMessage, currValidBytes)
debug("Message is compressed. Valid byte count = %d".format(currValidBytes)) }
innerIter = CompressionUtils.decompress(newMessage).deepIterator else{
if (!innerIter.hasNext) { newMessage.compressionCodec match {
currValidBytes += 4 + lastMessageSize case NoCompressionCodec =>
debug("Message is uncompressed. Valid byte count = %d".format(currValidBytes))
innerIter = null innerIter = null
} currValidBytes += 4 + size
makeNext() trace("currValidBytes = " + currValidBytes)
new MessageAndOffset(newMessage, currValidBytes)
case _ =>
debug("Message is compressed. Valid byte count = %d".format(currValidBytes))
innerIter = CompressionUtils.decompress(newMessage).internalIterator()
if (!innerIter.hasNext) {
currValidBytes += 4 + lastMessageSize
innerIter = null
}
makeNext()
}
} }
} }
override def makeNext(): MessageAndOffset = { override def makeNext(): MessageAndOffset = {
val isInnerDone = innerDone() if(isShallow){
isInnerDone match { makeNextOuter
case true => makeNextOuter }
case false => { else{
val messageAndOffset = innerIter.next val isInnerDone = innerDone()
if (!innerIter.hasNext) debug("makeNext() in internalIterator: innerDone = " + isInnerDone)
currValidBytes += 4 + lastMessageSize isInnerDone match {
new MessageAndOffset(messageAndOffset.message, currValidBytes) case true => makeNextOuter
case false => {
val messageAndOffset = innerIter.next
if (!innerIter.hasNext)
currValidBytes += 4 + lastMessageSize
new MessageAndOffset(messageAndOffset.message, currValidBytes)
}
} }
} }
} }

View File

@ -49,7 +49,7 @@ class GZIPCompression(inputStream: InputStream, outputStream: ByteArrayOutputStr
} }
override def read(a: Array[Byte]): Int = { override def read(a: Array[Byte]): Int = {
gzipIn.read(a) gzipIn.read(a)
} }
} }

View File

@ -20,4 +20,6 @@ package kafka.message
/** /**
* Indicates that a message failed its checksum and is corrupt * Indicates that a message failed its checksum and is corrupt
*/ */
class InvalidMessageException extends RuntimeException class InvalidMessageException(message: String) extends RuntimeException(message) {
def this() = this(null)
}

View File

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

View File

@ -13,11 +13,10 @@
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package kafka.message package kafka.message
/**
* Represents message and offset of the next message. This is used in the MessageSet to iterate over it case class MessageAndOffset(message: Message, offset: Long)
*/
case class MessageAndOffset(val message: Message, val offset: Long)

View File

@ -36,7 +36,7 @@ object ConsoleProducer {
.withRequiredArg .withRequiredArg
.describedAs("connection_string") .describedAs("connection_string")
.ofType(classOf[String]) .ofType(classOf[String])
val asyncOpt = parser.accepts("sync", "If set message send requests to the brokers are synchronously, one at a time as they arrive.") val syncOpt = parser.accepts("sync", "If set message send requests to the brokers are synchronously, one at a time as they arrive.")
val compressOpt = parser.accepts("compress", "If set, messages batches are sent compressed") val compressOpt = parser.accepts("compress", "If set, messages batches are sent compressed")
val batchSizeOpt = parser.accepts("batch-size", "Number of messages to send in a single batch if they are not being sent synchronously.") val batchSizeOpt = parser.accepts("batch-size", "Number of messages to send in a single batch if they are not being sent synchronously.")
.withRequiredArg .withRequiredArg
@ -78,7 +78,7 @@ object ConsoleProducer {
val topic = options.valueOf(topicOpt) val topic = options.valueOf(topicOpt)
val zkConnect = options.valueOf(zkConnectOpt) val zkConnect = options.valueOf(zkConnectOpt)
val async = options.has(asyncOpt) val sync = options.has(syncOpt)
val compress = options.has(compressOpt) val compress = options.has(compressOpt)
val batchSize = options.valueOf(batchSizeOpt) val batchSize = options.valueOf(batchSizeOpt)
val sendTimeout = options.valueOf(sendTimeoutOpt) val sendTimeout = options.valueOf(sendTimeoutOpt)
@ -89,10 +89,10 @@ object ConsoleProducer {
val props = new Properties() val props = new Properties()
props.put("zk.connect", zkConnect) props.put("zk.connect", zkConnect)
props.put("compression.codec", DefaultCompressionCodec.codec.toString) props.put("compression.codec", DefaultCompressionCodec.codec.toString)
props.put("producer.type", if(async) "async" else "sync") props.put("producer.type", if(sync) "sync" else "async")
if(options.has(batchSizeOpt)) if(options.has(batchSizeOpt))
props.put("batch.size", batchSize) props.put("batch.size", batchSize.toString)
props.put("queue.enqueueTimeout.ms", sendTimeout.toString) props.put("queue.time", sendTimeout.toString)
props.put("serializer.class", encoderClass) props.put("serializer.class", encoderClass)
val reader = Class.forName(readerClass).newInstance().asInstanceOf[MessageReader] val reader = Class.forName(readerClass).newInstance().asInstanceOf[MessageReader]

View File

@ -22,9 +22,7 @@ import org.apache.log4j.spi.LoggingEvent
import org.apache.log4j.AppenderSkeleton import org.apache.log4j.AppenderSkeleton
import org.apache.log4j.helpers.LogLog import org.apache.log4j.helpers.LogLog
import kafka.utils.Logging import kafka.utils.Logging
import kafka.serializer.Encoder
import java.util.{Properties, Date} import java.util.{Properties, Date}
import kafka.message.Message
import scala.collection._ import scala.collection._
class KafkaLog4jAppender extends AppenderSkeleton with Logging { class KafkaLog4jAppender extends AppenderSkeleton with Logging {
@ -94,7 +92,3 @@ class KafkaLog4jAppender extends AppenderSkeleton with Logging {
override def requiresLayout: Boolean = false override def requiresLayout: Boolean = false
} }
class DefaultStringEncoder extends Encoder[LoggingEvent] {
override def toMessage(event: LoggingEvent):Message = new Message(event.getMessage.asInstanceOf[String].getBytes)
}

View File

@ -32,10 +32,24 @@ class ProducerConfig(val props: Properties) extends ZKConfig(props)
if(brokerList != null) if(brokerList != null)
throw new InvalidConfigException("broker.list is deprecated. Use zk.connect instead") throw new InvalidConfigException("broker.list is deprecated. Use zk.connect instead")
/**
* If DefaultEventHandler is used, this specifies the number of times to
* retry if an error is encountered during send. Currently, it is only
* appropriate when broker.list points to a VIP. If the zk.connect option
* is used instead, this will not have any effect because with the zk-based
* producer, brokers are not re-selected upon retry. So retries would go to
* the same (potentially still down) broker. (KAFKA-253 will help address
* this.)
*/
val numRetries = Utils.getInt(props, "num.retries", 0)
/** If both broker.list and zk.connect options are specified, throw an exception */ /** If both broker.list and zk.connect options are specified, throw an exception */
if(zkConnect == null) if(zkConnect == null)
throw new InvalidConfigException("zk.connect property is required") throw new InvalidConfigException("zk.connect property is required")
if(!Utils.propertyExists(zkConnect) && !Utils.propertyExists(brokerList))
throw new InvalidConfigException("At least one of zk.connect or broker.list must be specified")
/** the partitioner class for partitioning events amongst sub-topics */ /** the partitioner class for partitioning events amongst sub-topics */
val partitionerClass = Utils.getString(props, "partitioner.class", "kafka.producer.DefaultPartitioner") val partitionerClass = Utils.getString(props, "partitioner.class", "kafka.producer.DefaultPartitioner")

View File

@ -18,10 +18,16 @@
package kafka.producer package kafka.producer
import kafka.api._ import kafka.api._
import kafka.common.MessageSizeTooLargeException
import kafka.message.MessageSet import kafka.message.MessageSet
import kafka.network.{BlockingChannel, BoundedByteBufferSend, Request, Receive} import kafka.network.{BlockingChannel, BoundedByteBufferSend, Request, Receive}
import kafka.utils._ import kafka.utils._
import java.util.Random
import kafka.common.MessageSizeTooLargeException
object SyncProducer {
val RequestKey: Short = 0
val randomGenerator = new Random
}
/* /*
* Send a message set. * Send a message set.
@ -31,14 +37,22 @@ class SyncProducer(val config: SyncProducerConfig) extends Logging {
private val MaxConnectBackoffMs = 60000 private val MaxConnectBackoffMs = 60000
private var sentOnConnection = 0 private var sentOnConnection = 0
/** make time-based reconnect starting at a random time **/
private var lastConnectionTime = System.currentTimeMillis - SyncProducer.randomGenerator.nextDouble() * config.reconnectInterval
private val lock = new Object() private val lock = new Object()
@volatile private var shutdown: Boolean = false @volatile private var shutdown: Boolean = false
private val blockingChannel = new BlockingChannel(config.host, config.port, 0, config.bufferSize, config.socketTimeoutMs) private val blockingChannel = new BlockingChannel(config.host, config.port, 0, config.bufferSize, config.socketTimeoutMs)
debug("Instantiating Scala Sync Producer") trace("Instantiating Scala Sync Producer")
private def verifyRequest(request: Request) = { private def verifyRequest(request: Request) = {
if (logger.isTraceEnabled) { /**
* This seems a little convoluted, but the idea is to turn on verification simply changing log4j settings
* Also, when verification is turned on, care should be taken to see that the logs don't fill up with unnecessary
* data. So, leaving the rest of the logging at TRACE, while errors should be logged at ERROR level
*/
if (logger.isDebugEnabled) {
val buffer = new BoundedByteBufferSend(request).buffer val buffer = new BoundedByteBufferSend(request).buffer
trace("verifying sendbuffer of size " + buffer.limit) trace("verifying sendbuffer of size " + buffer.limit)
val requestTypeId = buffer.getShort() val requestTypeId = buffer.getShort()
@ -71,9 +85,11 @@ class SyncProducer(val config: SyncProducerConfig) extends Logging {
} }
// TODO: do we still need this? // TODO: do we still need this?
sentOnConnection += 1 sentOnConnection += 1
if(sentOnConnection >= config.reconnectInterval) {
if(sentOnConnection >= config.reconnectInterval || (config.reconnectTimeInterval >= 0 && System.currentTimeMillis - lastConnectionTime >= config.reconnectTimeInterval)) {
reconnect() reconnect()
sentOnConnection = 0 sentOnConnection = 0
lastConnectionTime = System.currentTimeMillis
} }
SyncProducerStats.recordProduceRequest(SystemTime.nanoseconds - startTime) SyncProducerStats.recordProduceRequest(SystemTime.nanoseconds - startTime)
response response

View File

@ -40,6 +40,9 @@ trait SyncProducerConfigShared {
val reconnectInterval = Utils.getInt(props, "reconnect.interval", 30000) val reconnectInterval = Utils.getInt(props, "reconnect.interval", 30000)
/** negative reconnect time interval means disabling this time-based reconnect feature */
var reconnectTimeInterval = Utils.getInt(props, "reconnect.time.interval.ms", 1000*1000*10)
val maxMessageSize = Utils.getInt(props, "max.message.size", 1000000) val maxMessageSize = Utils.getInt(props, "max.message.size", 1000000)
/* the client application sending the producer requests */ /* the client application sending the producer requests */

View File

@ -70,10 +70,11 @@ class ProducerSendThread[K,V](val threadName: String,
trace("Dequeued item for topic %s, partition key: %s, data: %s" trace("Dequeued item for topic %s, partition key: %s, data: %s"
.format(currentQueueItem.getTopic, currentQueueItem.getKey.toString, currentQueueItem.getData.toString)) .format(currentQueueItem.getTopic, currentQueueItem.getKey.toString, currentQueueItem.getData.toString))
events += currentQueueItem events += currentQueueItem
// check if the batch size is reached
full = events.size >= batchSize
} }
// check if the batch size is reached
full = events.size >= batchSize
if(full || expired) { if(full || expired) {
if(expired) debug(elapsed + " ms elapsed. Queue time reached. Sending..") if(expired) debug(elapsed + " ms elapsed. Queue time reached. Sending..")
if(full) debug("Batch full. Sending..") if(full) debug("Batch full. Sending..")

View File

@ -96,7 +96,7 @@ class KafkaApis(val requestChannel: RequestChannel, val logManager: LogManager,
// TODO: need to handle ack's here! Will probably move to another method. // TODO: need to handle ack's here! Will probably move to another method.
kafkaZookeeper.ensurePartitionLeaderOnThisBroker(topicData.topic, partitionData.partition) kafkaZookeeper.ensurePartitionLeaderOnThisBroker(topicData.topic, partitionData.partition)
val log = logManager.getOrCreateLog(topicData.topic, partitionData.partition) val log = logManager.getOrCreateLog(topicData.topic, partitionData.partition)
log.append(partitionData.messages) log.append(partitionData.messages.asInstanceOf[ByteBufferMessageSet])
offsets(msgIndex) = log.nextAppendOffset offsets(msgIndex) = log.nextAppendOffset
errors(msgIndex) = ErrorMapping.NoError.toShort errors(msgIndex) = ErrorMapping.NoError.toShort
trace(partitionData.messages.sizeInBytes + " bytes written to logs.") trace(partitionData.messages.sizeInBytes + " bytes written to logs.")

View File

@ -71,7 +71,7 @@ class KafkaConfig(props: Properties) extends ZKConfig(props) {
val logRetentionHours = Utils.getIntInRange(props, "log.retention.hours", 24 * 7, (1, Int.MaxValue)) val logRetentionHours = Utils.getIntInRange(props, "log.retention.hours", 24 * 7, (1, Int.MaxValue))
/* the maximum size of the log before deleting it */ /* the maximum size of the log before deleting it */
val logRetentionSize = Utils.getInt(props, "log.retention.size", -1) val logRetentionSize = Utils.getLong(props, "log.retention.size", -1)
/* the number of hours to keep a log file before deleting it for some specific topic*/ /* the number of hours to keep a log file before deleting it for some specific topic*/
val logRetentionHoursMap = Utils.getTopicRentionHours(Utils.getString(props, "topic.log.retention.hours", "")) val logRetentionHoursMap = Utils.getTopicRentionHours(Utils.getString(props, "topic.log.retention.hours", ""))

View File

@ -17,9 +17,9 @@
package kafka.server package kafka.server
import org.apache.log4j._
import kafka.network._ import kafka.network._
import kafka.utils._ import kafka.utils._
import java.util.concurrent.atomic.AtomicLong
/** /**
* A thread that answers kafka requests. * A thread that answers kafka requests.
@ -60,3 +60,60 @@ class KafkaRequestHandlerPool(val requestChannel: RequestChannel,
} }
} }
trait BrokerTopicStatMBean {
def getMessagesIn: Long
def getBytesIn: Long
def getBytesOut: Long
def getFailedProduceRequest: Long
def getFailedFetchRequest: Long
}
@threadsafe
class BrokerTopicStat extends BrokerTopicStatMBean {
private val numCumulatedMessagesIn = new AtomicLong(0)
private val numCumulatedBytesIn = new AtomicLong(0)
private val numCumulatedBytesOut = new AtomicLong(0)
private val numCumulatedFailedProduceRequests = new AtomicLong(0)
private val numCumulatedFailedFetchRequests = new AtomicLong(0)
def getMessagesIn: Long = numCumulatedMessagesIn.get
def recordMessagesIn(nMessages: Int) = numCumulatedMessagesIn.getAndAdd(nMessages)
def getBytesIn: Long = numCumulatedBytesIn.get
def recordBytesIn(nBytes: Long) = numCumulatedBytesIn.getAndAdd(nBytes)
def getBytesOut: Long = numCumulatedBytesOut.get
def recordBytesOut(nBytes: Long) = numCumulatedBytesOut.getAndAdd(nBytes)
def recordFailedProduceRequest = numCumulatedFailedProduceRequests.getAndIncrement
def getFailedProduceRequest = numCumulatedFailedProduceRequests.get()
def recordFailedFetchRequest = numCumulatedFailedFetchRequests.getAndIncrement
def getFailedFetchRequest = numCumulatedFailedFetchRequests.get()
}
object BrokerTopicStat extends Logging {
private val stats = new Pool[String, BrokerTopicStat]
private val allTopicStat = new BrokerTopicStat
Utils.registerMBean(allTopicStat, "kafka:type=kafka.BrokerAllTopicStat")
def getBrokerAllTopicStat(): BrokerTopicStat = allTopicStat
def getBrokerTopicStat(topic: String): BrokerTopicStat = {
var stat = stats.get(topic)
if (stat == null) {
stat = new BrokerTopicStat
if (stats.putIfNotExists(topic, stat) == null)
Utils.registerMBean(stat, "kafka:type=kafka.BrokerTopicStat." + topic)
else
stat = stats.get(topic)
}
return stat
}
}

View File

@ -17,36 +17,21 @@
package kafka.server package kafka.server
import kafka.utils.{Utils, Logging} import kafka.utils.Logging
import kafka.consumer._
import kafka.producer.{ProducerData, ProducerConfig, Producer}
import kafka.message.Message
import java.util.concurrent.CountDownLatch
import scala.collection.Map
class KafkaServerStartable(val serverConfig: KafkaConfig, class KafkaServerStartable(val serverConfig: KafkaConfig) extends Logging {
val consumerConfig: ConsumerConfig,
val producerConfig: ProducerConfig) extends Logging {
private var server : KafkaServer = null private var server : KafkaServer = null
private var embeddedConsumer : EmbeddedConsumer = null
init init
def this(serverConfig: KafkaConfig) = this(serverConfig, null, null)
private def init() { private def init() {
server = new KafkaServer(serverConfig) server = new KafkaServer(serverConfig)
if (consumerConfig != null)
embeddedConsumer =
new EmbeddedConsumer(consumerConfig, producerConfig, this)
} }
def startup() { def startup() {
try { try {
server.startup() server.startup()
if (embeddedConsumer != null)
embeddedConsumer.startup()
} }
catch { catch {
case e => case e =>
@ -57,8 +42,6 @@ class KafkaServerStartable(val serverConfig: KafkaConfig,
def shutdown() { def shutdown() {
try { try {
if (embeddedConsumer != null)
embeddedConsumer.shutdown()
server.shutdown() server.shutdown()
} }
catch { catch {
@ -73,153 +56,4 @@ class KafkaServerStartable(val serverConfig: KafkaConfig,
} }
class EmbeddedConsumer(private val consumerConfig: ConsumerConfig,
private val producerConfig: ProducerConfig,
private val kafkaServerStartable: KafkaServerStartable) extends TopicEventHandler[String] with Logging {
private val whiteListTopics =
consumerConfig.mirrorTopicsWhitelist.split(",").toList.map(_.trim)
private val blackListTopics =
consumerConfig.mirrorTopicsBlackList.split(",").toList.map(_.trim)
// mirrorTopics should be accessed by handleTopicEvent only
private var mirrorTopics:Seq[String] = List()
private var consumerConnector: ConsumerConnector = null
private var topicEventWatcher:ZookeeperTopicEventWatcher = null
private val producer = new Producer[Null, Message](producerConfig)
var threadList = List[MirroringThread]()
private def isTopicAllowed(topic: String) = {
if (consumerConfig.mirrorTopicsWhitelist.nonEmpty)
whiteListTopics.contains(topic)
else
!blackListTopics.contains(topic)
}
// TopicEventHandler call-back only
@Override
def handleTopicEvent(allTopics: Seq[String]) {
val newMirrorTopics = allTopics.filter(isTopicAllowed)
val addedTopics = newMirrorTopics filterNot (mirrorTopics contains)
if (addedTopics.nonEmpty)
info("topic event: added topics = %s".format(addedTopics))
val deletedTopics = mirrorTopics filterNot (newMirrorTopics contains)
if (deletedTopics.nonEmpty)
info("topic event: deleted topics = %s".format(deletedTopics))
mirrorTopics = newMirrorTopics
if (addedTopics.nonEmpty || deletedTopics.nonEmpty) {
info("mirror topics = %s".format(mirrorTopics))
startNewConsumerThreads(makeTopicMap(mirrorTopics))
}
}
private def makeTopicMap(mirrorTopics: Seq[String]) = {
if (mirrorTopics.nonEmpty)
Utils.getConsumerTopicMap(mirrorTopics.mkString(
"", ":%d,".format(consumerConfig.mirrorConsumerNumThreads),
":%d".format(consumerConfig.mirrorConsumerNumThreads)))
else
Utils.getConsumerTopicMap("")
}
private def startNewConsumerThreads(topicMap: Map[String, Int]) {
if (topicMap.nonEmpty) {
if (consumerConnector != null)
consumerConnector.shutdown()
/**
* Before starting new consumer threads for the updated set of topics,
* shutdown the existing mirroring threads. Since the consumer connector
* is already shutdown, the mirroring threads should finish their task almost
* instantaneously. If they don't, this points to an error that needs to be looked
* into, and further mirroring should stop
*/
threadList.foreach(_.shutdown)
// KAFKA: 212: clear the thread list to remove the older thread references that are already shutdown
threadList = Nil
consumerConnector = Consumer.create(consumerConfig)
val topicMessageStreams = consumerConnector.createMessageStreams(topicMap)
for ((topic, streamList) <- topicMessageStreams)
for (i <- 0 until streamList.length)
threadList ::= new MirroringThread(streamList(i), topic, i)
threadList.foreach(_.start)
}
else
info("Not starting mirroring threads (mirror topic list is empty)")
}
def startup() {
info("staring up embedded consumer")
topicEventWatcher = new ZookeeperTopicEventWatcher(consumerConfig, this, kafkaServerStartable)
/*
* consumer threads are (re-)started upon topic events (which includes an
* initial startup event which lists the current topics)
*/
}
def shutdown() {
// first shutdown the topic watcher to prevent creating new consumer streams
if (topicEventWatcher != null)
topicEventWatcher.shutdown()
info("Stopped the ZK watcher for new topics, now stopping the Kafka consumers")
// stop pulling more data for mirroring
if (consumerConnector != null)
consumerConnector.shutdown()
info("Stopped the kafka consumer threads for existing topics, now stopping the existing mirroring threads")
// wait for all mirroring threads to stop
threadList.foreach(_.shutdown)
info("Stopped all existing mirroring threads, now stopping the producer")
// only then, shutdown the producer
producer.close()
info("Successfully shutdown this Kafka mirror")
}
class MirroringThread(val stream: KafkaMessageStream[Message], val topic: String, val threadId: Int) extends Thread with Logging {
val shutdownComplete = new CountDownLatch(1)
val name = "kafka-embedded-consumer-%s-%d".format(topic, threadId)
this.setDaemon(false)
this.setName(name)
override def run = {
info("Starting mirroring thread %s for topic %s and stream %d".format(name, topic, threadId))
try {
for (message <- stream) {
trace("Mirroring thread received message " + message.checksum)
val pd = new ProducerData[Null, Message](topic, message)
producer.send(pd)
}
}
catch {
case e =>
fatal(topic + " stream " + threadId + " unexpectedly exited", e)
}finally {
shutdownComplete.countDown
info("Stopped mirroring thread %s for topic %s and stream %d".format(name, topic, threadId))
}
}
def shutdown = {
try {
shutdownComplete.await
}catch {
case e: InterruptedException => fatal("Shutdown of thread " + name + " interrupted. " +
"Mirroring thread might leak data!")
}
}
}
}

View File

@ -82,15 +82,15 @@ object ConsumerShell {
} }
} }
class ZKConsumerThread(stream: KafkaMessageStream[String]) extends Thread with Logging { class ZKConsumerThread(stream: KafkaStream[String]) extends Thread with Logging {
val shutdownLatch = new CountDownLatch(1) val shutdownLatch = new CountDownLatch(1)
override def run() { override def run() {
println("Starting consumer thread..") println("Starting consumer thread..")
var count: Int = 0 var count: Int = 0
try { try {
for (message <- stream) { for (messageAndMetadata <- stream) {
println("consumed: " + message) println("consumed: " + messageAndMetadata.message)
count += 1 count += 1
} }
}catch { }catch {

View File

@ -0,0 +1,123 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.tools
import java.io.FileWriter
import joptsimple._
import kafka.utils.{Logging, ZkUtils, ZKStringSerializer,ZKGroupTopicDirs}
import org.I0Itec.zkclient.ZkClient
/**
* A utility that retrieve the offset of broker partitions in ZK and
* prints to an output file in the following format:
*
* /consumers/group1/offsets/topic1/1-0:286894308
* /consumers/group1/offsets/topic1/2-0:284803985
*
* This utility expects 3 arguments:
* 1. Zk host:port string
* 2. group name (all groups implied if omitted)
* 3. output filename
*
* To print debug message, add the following line to log4j.properties:
* log4j.logger.kafka.tools.ExportZkOffsets$=DEBUG
* (for eclipse debugging, copy log4j.properties to the binary directory in "core" such as core/bin)
*/
object ExportZkOffsets extends Logging {
def main(args: Array[String]) {
val parser = new OptionParser
val zkConnectOpt = parser.accepts("zkconnect", "ZooKeeper connect string.")
.withRequiredArg()
.defaultsTo("localhost:2181")
.ofType(classOf[String])
val groupOpt = parser.accepts("group", "Consumer group.")
.withRequiredArg()
.ofType(classOf[String])
val outFileOpt = parser.accepts("output-file", "Output file")
.withRequiredArg()
.ofType(classOf[String])
parser.accepts("help", "Print this message.")
val options = parser.parse(args : _*)
if (options.has("help")) {
parser.printHelpOn(System.out)
System.exit(0)
}
for (opt <- List(zkConnectOpt, outFileOpt)) {
if (!options.has(opt)) {
System.err.println("Missing required argument: %s".format(opt))
parser.printHelpOn(System.err)
System.exit(1)
}
}
val zkConnect = options.valueOf(zkConnectOpt)
val groups = options.valuesOf(groupOpt)
val outfile = options.valueOf(outFileOpt)
var zkClient : ZkClient = null
val fileWriter : FileWriter = new FileWriter(outfile)
try {
zkClient = new ZkClient(zkConnect, 30000, 30000, ZKStringSerializer)
var consumerGroups: Seq[String] = null
if (groups.size == 0) {
consumerGroups = ZkUtils.getChildren(zkClient, ZkUtils.ConsumersPath).toList
}
else {
import scala.collection.JavaConversions._
consumerGroups = groups
}
for (consumerGrp <- consumerGroups) {
val topicsList = getTopicsList(zkClient, consumerGrp)
for (topic <- topicsList) {
val bidPidList = getBrokeridPartition(zkClient, consumerGrp, topic)
for (bidPid <- bidPidList) {
val zkGrpTpDir = new ZKGroupTopicDirs(consumerGrp,topic)
val offsetPath = zkGrpTpDir.consumerOffsetDir + "/" + bidPid
val offsetVal = ZkUtils.readDataMaybeNull(zkClient, offsetPath)
fileWriter.write(offsetPath + ":" + offsetVal + "\n")
debug(offsetPath + " => " + offsetVal)
}
}
}
}
finally {
fileWriter.flush()
fileWriter.close()
}
}
private def getBrokeridPartition(zkClient: ZkClient, consumerGroup: String, topic: String): List[String] = {
return ZkUtils.getChildrenParentMayNotExist(zkClient, "/consumers/%s/offsets/%s".format(consumerGroup, topic)).toList
}
private def getTopicsList(zkClient: ZkClient, consumerGroup: String): List[String] = {
return ZkUtils.getChildren(zkClient, "/consumers/%s/offsets".format(consumerGroup)).toList
}
}

View File

@ -0,0 +1,112 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.tools
import java.io.BufferedReader
import java.io.FileReader
import joptsimple._
import kafka.utils.{Logging, ZkUtils,ZKStringSerializer}
import org.I0Itec.zkclient.ZkClient
/**
* A utility that updates the offset of broker partitions in ZK.
*
* This utility expects 2 input files as arguments:
* 1. consumer properties file
* 2. a file contains partition offsets data such as:
* (This output data file can be obtained by running kafka.tools.ExportZkOffsets)
*
* /consumers/group1/offsets/topic1/3-0:285038193
* /consumers/group1/offsets/topic1/1-0:286894308
*
* To print debug message, add the following line to log4j.properties:
* log4j.logger.kafka.tools.ImportZkOffsets$=DEBUG
* (for eclipse debugging, copy log4j.properties to the binary directory in "core" such as core/bin)
*/
object ImportZkOffsets extends Logging {
def main(args: Array[String]) {
val parser = new OptionParser
val zkConnectOpt = parser.accepts("zkconnect", "ZooKeeper connect string.")
.withRequiredArg()
.defaultsTo("localhost:2181")
.ofType(classOf[String])
val inFileOpt = parser.accepts("input-file", "Input file")
.withRequiredArg()
.ofType(classOf[String])
parser.accepts("help", "Print this message.")
val options = parser.parse(args : _*)
if (options.has("help")) {
parser.printHelpOn(System.out)
System.exit(0)
}
for (opt <- List(inFileOpt)) {
if (!options.has(opt)) {
System.err.println("Missing required argument: %s".format(opt))
parser.printHelpOn(System.err)
System.exit(1)
}
}
val zkConnect = options.valueOf(zkConnectOpt)
val partitionOffsetFile = options.valueOf(inFileOpt)
val zkClient = new ZkClient(zkConnect, 30000, 30000, ZKStringSerializer)
val partitionOffsets: Map[String,String] = getPartitionOffsetsFromFile(partitionOffsetFile)
updateZkOffsets(zkClient, partitionOffsets)
}
private def getPartitionOffsetsFromFile(filename: String):Map[String,String] = {
val fr = new FileReader(filename)
val br = new BufferedReader(fr)
var partOffsetsMap: Map[String,String] = Map()
var s: String = br.readLine()
while ( s != null && s.length() >= 1) {
val tokens = s.split(":")
partOffsetsMap += tokens(0) -> tokens(1)
debug("adding node path [" + s + "]")
s = br.readLine()
}
return partOffsetsMap
}
private def updateZkOffsets(zkClient: ZkClient, partitionOffsets: Map[String,String]): Unit = {
val cluster = ZkUtils.getCluster(zkClient)
var partitions: List[String] = Nil
for ((partition, offset) <- partitionOffsets) {
debug("updating [" + partition + "] with offset [" + offset + "]")
try {
ZkUtils.updatePersistentPath(zkClient, partition, offset.toString)
} catch {
case e => e.printStackTrace()
}
}
}
}

View File

@ -0,0 +1,171 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.tools
import kafka.message.Message
import joptsimple.OptionParser
import kafka.utils.{Utils, Logging}
import kafka.producer.{ProducerData, ProducerConfig, Producer}
import scala.collection.JavaConversions._
import java.util.concurrent.CountDownLatch
import kafka.consumer._
object MirrorMaker extends Logging {
def main(args: Array[String]) {
info ("Starting mirror maker")
val parser = new OptionParser
val consumerConfigOpt = parser.accepts("consumer.config",
"Consumer config to consume from a source cluster. " +
"You may specify multiple of these.")
.withRequiredArg()
.describedAs("config file")
.ofType(classOf[String])
val producerConfigOpt = parser.accepts("producer.config",
"Embedded producer config.")
.withRequiredArg()
.describedAs("config file")
.ofType(classOf[String])
val numProducersOpt = parser.accepts("num.producers",
"Number of producer instances")
.withRequiredArg()
.describedAs("Number of producers")
.ofType(classOf[java.lang.Integer])
.defaultsTo(1)
val numStreamsOpt = parser.accepts("num.streams",
"Number of consumption streams.")
.withRequiredArg()
.describedAs("Number of threads")
.ofType(classOf[java.lang.Integer])
.defaultsTo(1)
val whitelistOpt = parser.accepts("whitelist",
"Whitelist of topics to mirror.")
.withRequiredArg()
.describedAs("Java regex (String)")
.ofType(classOf[String])
val blacklistOpt = parser.accepts("blacklist",
"Blacklist of topics to mirror.")
.withRequiredArg()
.describedAs("Java regex (String)")
.ofType(classOf[String])
val helpOpt = parser.accepts("help", "Print this message.")
val options = parser.parse(args : _*)
if (options.has(helpOpt)) {
parser.printHelpOn(System.out)
System.exit(0)
}
Utils.checkRequiredArgs(
parser, options, consumerConfigOpt, producerConfigOpt)
if (List(whitelistOpt, blacklistOpt).count(options.has) != 1) {
println("Exactly one of whitelist or blacklist is required.")
System.exit(1)
}
val numStreams = options.valueOf(numStreamsOpt)
val producers = (1 to options.valueOf(numProducersOpt).intValue()).map(_ => {
val config = new ProducerConfig(
Utils.loadProps(options.valueOf(producerConfigOpt)))
new Producer[Null, Message](config)
})
val threads = {
val connectors = options.valuesOf(consumerConfigOpt).toList
.map(cfg => new ConsumerConfig(Utils.loadProps(cfg.toString)))
.map(new ZookeeperConsumerConnector(_))
Runtime.getRuntime.addShutdownHook(new Thread() {
override def run() {
connectors.foreach(_.shutdown())
producers.foreach(_.close())
}
})
val filterSpec = if (options.has(whitelistOpt))
new Whitelist(options.valueOf(whitelistOpt))
else
new Blacklist(options.valueOf(blacklistOpt))
val streams =
connectors.map(_.createMessageStreamsByFilter(filterSpec, numStreams.intValue()))
streams.flatten.zipWithIndex.map(streamAndIndex => {
new MirrorMakerThread(streamAndIndex._1, producers, streamAndIndex._2)
})
}
threads.foreach(_.start())
threads.foreach(_.awaitShutdown())
}
class MirrorMakerThread(stream: KafkaStream[Message],
producers: Seq[Producer[Null, Message]],
threadId: Int)
extends Thread with Logging {
private val shutdownLatch = new CountDownLatch(1)
private val threadName = "mirrormaker-" + threadId
private val producerSelector = Utils.circularIterator(producers)
this.setName(threadName)
override def run() {
try {
for (msgAndMetadata <- stream) {
val producer = producerSelector.next()
val pd = new ProducerData[Null, Message](
msgAndMetadata.topic, msgAndMetadata.message)
producer.send(pd)
}
}
catch {
case e =>
fatal("%s stream unexpectedly exited.", e)
}
finally {
shutdownLatch.countDown()
info("Stopped thread %s.".format(threadName))
}
}
def awaitShutdown() {
try {
shutdownLatch.await()
}
catch {
case e: InterruptedException => fatal(
"Shutdown of thread %s interrupted. This might leak data!"
.format(threadName))
}
}
}
}

View File

@ -32,8 +32,6 @@ object ReplayLogProducer extends Logging {
private val GROUPID: String = "replay-log-producer" private val GROUPID: String = "replay-log-producer"
def main(args: Array[String]) { def main(args: Array[String]) {
var isNoPrint = false;
val config = new Config(args) val config = new Config(args)
val executor = Executors.newFixedThreadPool(config.numThreads) val executor = Executors.newFixedThreadPool(config.numThreads)
@ -151,7 +149,7 @@ object ReplayLogProducer extends Logging {
} }
} }
class ZKConsumerThread(config: Config, stream: KafkaMessageStream[Message]) extends Thread with Logging { class ZKConsumerThread(config: Config, stream: KafkaStream[Message]) extends Thread with Logging {
val shutdownLatch = new CountDownLatch(1) val shutdownLatch = new CountDownLatch(1)
val props = new Properties() val props = new Properties()
val brokerInfoList = config.brokerInfo.split("=") val brokerInfoList = config.brokerInfo.split("=")
@ -180,9 +178,9 @@ object ReplayLogProducer extends Logging {
stream.slice(0, config.numMessages) stream.slice(0, config.numMessages)
else else
stream stream
for (message <- iter) { for (messageAndMetadata <- iter) {
try { try {
producer.send(new ProducerData[Message, Message](config.outputTopic, message)) producer.send(new ProducerData[Message, Message](config.outputTopic, messageAndMetadata.message))
if (config.delayedMSBtwSend > 0 && (messageCount + 1) % config.batchSize == 0) if (config.delayedMSBtwSend > 0 && (messageCount + 1) % config.batchSize == 0)
Thread.sleep(config.delayedMSBtwSend) Thread.sleep(config.delayedMSBtwSend)
messageCount += 1 messageCount += 1

View File

@ -57,9 +57,9 @@ object VerifyConsumerRebalance extends Logging {
// check if the rebalancing operation succeeded. // check if the rebalancing operation succeeded.
try { try {
if(validateRebalancingOperation(zkClient, group)) if(validateRebalancingOperation(zkClient, group))
info("Rebalance operation successful !") println("Rebalance operation successful !")
else else
error("Rebalance operation failed !") println("Rebalance operation failed !")
} catch { } catch {
case e2: Throwable => error("Error while verifying current rebalancing operation", e2) case e2: Throwable => error("Error while verifying current rebalancing operation", e2)
} }
@ -132,6 +132,4 @@ object VerifyConsumerRebalance extends Logging {
rebalanceSucceeded rebalanceSucceeded
} }
}
}

View File

@ -23,17 +23,21 @@ trait Logging {
val loggerName = this.getClass.getName val loggerName = this.getClass.getName
lazy val logger = Logger.getLogger(loggerName) lazy val logger = Logger.getLogger(loggerName)
protected var logIdent = ""
private def msgWithLogIdent(msg: String) = "%s%s".format(logIdent, msg)
def trace(msg: => String): Unit = { def trace(msg: => String): Unit = {
if (logger.isTraceEnabled()) if (logger.isTraceEnabled())
logger.trace(msg) logger.trace(msgWithLogIdent(msg))
} }
def trace(e: => Throwable): Any = { def trace(e: => Throwable): Any = {
if (logger.isTraceEnabled()) if (logger.isTraceEnabled())
logger.trace("",e) logger.trace(logIdent,e)
} }
def trace(msg: => String, e: => Throwable) = { def trace(msg: => String, e: => Throwable) = {
if (logger.isTraceEnabled()) if (logger.isTraceEnabled())
logger.trace(msg,e) logger.trace(msgWithLogIdent(msg),e)
} }
def swallowTrace(action: => Unit) { def swallowTrace(action: => Unit) {
Utils.swallow(logger.trace, action) Utils.swallow(logger.trace, action)
@ -41,15 +45,15 @@ trait Logging {
def debug(msg: => String): Unit = { def debug(msg: => String): Unit = {
if (logger.isDebugEnabled()) if (logger.isDebugEnabled())
logger.debug(msg) logger.debug(msgWithLogIdent(msg))
} }
def debug(e: => Throwable): Any = { def debug(e: => Throwable): Any = {
if (logger.isDebugEnabled()) if (logger.isDebugEnabled())
logger.debug("",e) logger.debug(logIdent,e)
} }
def debug(msg: => String, e: => Throwable) = { def debug(msg: => String, e: => Throwable) = {
if (logger.isDebugEnabled()) if (logger.isDebugEnabled())
logger.debug(msg,e) logger.debug(msgWithLogIdent(msg),e)
} }
def swallowDebug(action: => Unit) { def swallowDebug(action: => Unit) {
Utils.swallow(logger.debug, action) Utils.swallow(logger.debug, action)
@ -57,55 +61,54 @@ trait Logging {
def info(msg: => String): Unit = { def info(msg: => String): Unit = {
if (logger.isInfoEnabled()) if (logger.isInfoEnabled())
logger.info(msg) logger.info(msgWithLogIdent(msg))
} }
def info(e: => Throwable): Any = { def info(e: => Throwable): Any = {
if (logger.isInfoEnabled()) if (logger.isInfoEnabled())
logger.info("",e) logger.info(logIdent,e)
} }
def info(msg: => String,e: => Throwable) = { def info(msg: => String,e: => Throwable) = {
if (logger.isInfoEnabled()) if (logger.isInfoEnabled())
logger.info(msg,e) logger.info(msgWithLogIdent(msg),e)
} }
def swallowInfo(action: => Unit) { def swallowInfo(action: => Unit) {
Utils.swallow(logger.info, action) Utils.swallow(logger.info, action)
} }
def warn(msg: => String): Unit = { def warn(msg: => String): Unit = {
logger.warn(msg) logger.warn(msgWithLogIdent(msg))
} }
def warn(e: => Throwable): Any = { def warn(e: => Throwable): Any = {
logger.warn("",e) logger.warn(logIdent,e)
} }
def warn(msg: => String, e: => Throwable) = { def warn(msg: => String, e: => Throwable) = {
logger.warn(msg,e) logger.warn(msgWithLogIdent(msg),e)
} }
def swallowWarn(action: => Unit) { def swallowWarn(action: => Unit) {
Utils.swallow(logger.warn, action) Utils.swallow(logger.warn, action)
} }
def swallow(action: => Unit) = swallowWarn(action) def swallow(action: => Unit) = swallowWarn(action)
def error(msg: => String):Unit = { def error(msg: => String): Unit = {
logger.error(msg) logger.error(msgWithLogIdent(msg))
} }
def error(e: => Throwable): Any = { def error(e: => Throwable): Any = {
logger.error("",e) logger.error(logIdent,e)
} }
def error(msg: => String, e: => Throwable) = { def error(msg: => String, e: => Throwable) = {
logger.error(msg,e) logger.error(msgWithLogIdent(msg),e)
} }
def swallowError(action: => Unit) { def swallowError(action: => Unit) {
Utils.swallow(logger.error, action) Utils.swallow(logger.error, action)
} }
def fatal(msg: => String): Unit = { def fatal(msg: => String): Unit = {
logger.fatal(msg) logger.fatal(msgWithLogIdent(msg))
} }
def fatal(e: => Throwable): Any = { def fatal(e: => Throwable): Any = {
logger.fatal("",e) logger.fatal(logIdent,e)
} }
def fatal(msg: => String, e: => Throwable) = { def fatal(msg: => String, e: => Throwable) = {
logger.fatal(msg,e) logger.fatal(msgWithLogIdent(msg),e)
} }
} }

View File

@ -29,6 +29,8 @@ import scala.collection.mutable
import kafka.message.{NoCompressionCodec, CompressionCodec} import kafka.message.{NoCompressionCodec, CompressionCodec}
import org.I0Itec.zkclient.ZkClient import org.I0Itec.zkclient.ZkClient
import java.util.{Random, Properties} import java.util.{Random, Properties}
import joptsimple.{OptionSpec, OptionSet, OptionParser}
/** /**
* Helper functions! * Helper functions!
@ -250,13 +252,47 @@ object Utils extends Logging {
else value else value
} }
def getLong(props: Properties, name: String, default: Long): Long = def getLongInRange(buffer: ByteBuffer, name: String, range: (Long, Long)): Long = {
val value = buffer.getLong
if(value < range._1 || value > range._2)
throw new IllegalArgumentException(name + " has value " + value + " which is not in the range " + range + ".")
else value
}
/**
* Read a required long property value or throw an exception if no such property is found
*/
def getLong(props: Properties, name: String): Long = {
if(props.containsKey(name))
return getLong(props, name, -1)
else
throw new IllegalArgumentException("Missing required property '" + name + "'")
}
/**
* Read an long from the properties instance
* @param props The properties to read from
* @param name The property name
* @param default The default value to use if the property is not found
* @return the long value
*/
def getLong(props: Properties, name: String, default: Long): Long =
getLongInRange(props, name, default, (Long.MinValue, Long.MaxValue)) getLongInRange(props, name, default, (Long.MinValue, Long.MaxValue))
/**
* Read an long from the properties instance. Throw an exception
* if the value is not in the given range (inclusive)
* @param props The properties to read from
* @param name The property name
* @param default The default value to use if the property is not found
* @param range The range in which the value must fall (inclusive)
* @throws IllegalArgumentException If the value is not in the given range
* @return the long value
*/
def getLongInRange(props: Properties, name: String, default: Long, range: (Long, Long)): Long = { def getLongInRange(props: Properties, name: String, default: Long, range: (Long, Long)): Long = {
val v = val v =
if(props.containsKey(name)) if(props.containsKey(name))
props.getProperty(name).toInt props.getProperty(name).toLong
else else
default default
if(v < range._1 || v > range._2) if(v < range._1 || v > range._2)
@ -265,14 +301,6 @@ object Utils extends Logging {
v v
} }
def getLongInRange(buffer: ByteBuffer, name: String, range: (Long, Long)): Long = {
val value = buffer.getLong
if(value < range._1 || value > range._2)
throw new IllegalArgumentException(name + " has value " + value + " which is not in the range " + range + ".")
else value
}
/** /**
* Read a boolean value from the properties instance * Read a boolean value from the properties instance
* @param props The properties to read from * @param props The properties to read from
@ -720,6 +748,28 @@ object Utils extends Logging {
builder.append(" }") builder.append(" }")
builder.toString builder.toString
} }
def checkRequiredArgs(parser: OptionParser, options: OptionSet, required: OptionSpec[_]*) {
for(arg <- required) {
if(!options.has(arg)) {
error("Missing required argument \"" + arg + "\"")
parser.printHelpOn(System.err)
System.exit(1)
}
}
}
/**
* Create a circular (looping) iterator over a collection.
* @param coll An iterable over the underlying collection.
* @return A circular iterator over the collection.
*/
def circularIterator[T](coll: Iterable[T]) = {
val stream: Stream[T] =
for (forever <- Stream.continually(1); t <- coll) yield t
stream.iterator
}
} }
class SnapshotStats(private val monitorDurationNs: Long = 600L * 1000L * 1000L * 1000L) { class SnapshotStats(private val monitorDurationNs: Long = 600L * 1000L * 1000L * 1000L) {

View File

@ -432,17 +432,11 @@ object ZkUtils extends Logging {
getChildren(zkClient, dirs.consumerRegistryDir) getChildren(zkClient, dirs.consumerRegistryDir)
} }
def getTopicCount(zkClient: ZkClient, group: String, consumerId: String) : TopicCount = {
val dirs = new ZKGroupDirs(group)
val topicCountJson = ZkUtils.readData(zkClient, dirs.consumerRegistryDir + "/" + consumerId)
TopicCount.constructTopicCount(consumerId, topicCountJson)
}
def getConsumerTopicMaps(zkClient: ZkClient, group: String): Map[String, TopicCount] = { def getConsumerTopicMaps(zkClient: ZkClient, group: String): Map[String, TopicCount] = {
val dirs = new ZKGroupDirs(group) val dirs = new ZKGroupDirs(group)
val consumersInGroup = getConsumersInGroup(zkClient, group) val consumersInGroup = getConsumersInGroup(zkClient, group)
val topicCountMaps = consumersInGroup.map(consumerId => TopicCount.constructTopicCount(consumerId, val topicCountMaps = consumersInGroup.map(consumerId => TopicCount.constructTopicCount(consumerId,
ZkUtils.readData(zkClient, dirs.consumerRegistryDir + "/" + consumerId))) ZkUtils.readData(zkClient, dirs.consumerRegistryDir + "/" + consumerId), zkClient))
consumersInGroup.zip(topicCountMaps).toMap consumersInGroup.zip(topicCountMaps).toMap
} }
@ -451,8 +445,8 @@ object ZkUtils extends Logging {
val consumers = getChildrenParentMayNotExist(zkClient, dirs.consumerRegistryDir) val consumers = getChildrenParentMayNotExist(zkClient, dirs.consumerRegistryDir)
val consumersPerTopicMap = new mutable.HashMap[String, List[String]] val consumersPerTopicMap = new mutable.HashMap[String, List[String]]
for (consumer <- consumers) { for (consumer <- consumers) {
val topicCount = getTopicCount(zkClient, group, consumer) val topicCount = TopicCount.constructTopicCount(group, consumer, zkClient)
for ((topic, consumerThreadIdSet) <- topicCount.getConsumerThreadIdsPerTopic()) { for ((topic, consumerThreadIdSet) <- topicCount.getConsumerThreadIdsPerTopic) {
for (consumerThreadId <- consumerThreadIdSet) for (consumerThreadId <- consumerThreadIdSet)
consumersPerTopicMap.get(topic) match { consumersPerTopicMap.get(topic) match {
case Some(curConsumers) => consumersPerTopicMap.put(topic, consumerThreadId :: curConsumers) case Some(curConsumers) => consumersPerTopicMap.put(topic, consumerThreadId :: curConsumers)

View File

@ -52,7 +52,6 @@ object TestEndToEndLatency {
val message = new Message("hello there beautiful".getBytes) val message = new Message("hello there beautiful".getBytes)
var totalTime = 0.0 var totalTime = 0.0
var totalSize = 0L
for(i <- 0 until numMessages) { for(i <- 0 until numMessages) {
var begin = System.nanoTime var begin = System.nanoTime
producer.send(new ProducerData(topic, message)) producer.send(new ProducerData(topic, message))
@ -62,7 +61,6 @@ object TestEndToEndLatency {
if(i % 10000 == 0) if(i % 10000 == 0)
println(i + "\t" + ellapsed / 1000.0 / 1000.0) println(i + "\t" + ellapsed / 1000.0 / 1000.0)
totalTime += ellapsed totalTime += ellapsed
totalSize += received.size
} }
println("Avg latency: " + (totalTime / numMessages / 1000.0 / 1000.0)) + "ms" println("Avg latency: " + (totalTime / numMessages / 1000.0 / 1000.0)) + "ms"
producer.close() producer.close()

View File

@ -56,13 +56,13 @@ object TestZKConsumerOffsets {
} }
} }
private class ConsumerThread(stream: KafkaMessageStream[Message]) extends Thread { private class ConsumerThread(stream: KafkaStream[Message]) extends Thread {
val shutdownLatch = new CountDownLatch(1) val shutdownLatch = new CountDownLatch(1)
override def run() { override def run() {
println("Starting consumer thread..") println("Starting consumer thread..")
for (message <- stream) { for (messageAndMetadata <- stream) {
println("consumed: " + Utils.toString(message.payload, "UTF-8")) println("consumed: " + Utils.toString(messageAndMetadata.message.payload, "UTF-8"))
} }
shutdownLatch.countDown shutdownLatch.countDown
println("thread shutdown !" ) println("thread shutdown !" )

View File

@ -24,7 +24,7 @@ import kafka.cluster.Partition
class TopicCountTest extends JUnitSuite { class TopicCountTest extends JUnitSuite {
/*
@Test @Test
def testBasic() { def testBasic() {
val consumer = "conusmer1" val consumer = "conusmer1"
@ -40,7 +40,7 @@ class TopicCountTest extends JUnitSuite {
val topicCount2 = TopicCount.constructTopicCount(consumer, expectedTopicCount.toJsonString) val topicCount2 = TopicCount.constructTopicCount(consumer, expectedTopicCount.toJsonString)
assertTrue(expectedTopicCount == topicCount2) assertTrue(expectedTopicCount == topicCount2)
} }
*/
@Test @Test
def testPartition() { def testPartition() {
assertTrue(new Partition("foo", 10) == new Partition("foo", 10)) assertTrue(new Partition("foo", 10) == new Partition("foo", 10))

View File

@ -0,0 +1,51 @@
/**
* 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.consumer
import junit.framework.Assert._
import org.scalatest.junit.JUnitSuite
import org.junit.Test
class TopicFilterTest extends JUnitSuite {
@Test
def testWhitelists() {
val topicFilter1 = new Whitelist("white1,white2")
assertFalse(topicFilter1.requiresTopicEventWatcher)
assertTrue(topicFilter1.isTopicAllowed("white2"))
assertFalse(topicFilter1.isTopicAllowed("black1"))
val topicFilter2 = new Whitelist(".+")
assertTrue(topicFilter2.requiresTopicEventWatcher)
assertTrue(topicFilter2.isTopicAllowed("alltopics"))
val topicFilter3 = new Whitelist("white_listed-topic.+")
assertTrue(topicFilter3.requiresTopicEventWatcher)
assertTrue(topicFilter3.isTopicAllowed("white_listed-topic1"))
assertFalse(topicFilter3.isTopicAllowed("black1"))
}
@Test
def testBlacklists() {
val topicFilter1 = new Blacklist("black1")
assertTrue(topicFilter1.requiresTopicEventWatcher)
}
}

View File

@ -338,7 +338,7 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar
val iterator = messageStream.iterator val iterator = messageStream.iterator
for (i <- 0 until nMessages * 2) { for (i <- 0 until nMessages * 2) {
assertTrue(iterator.hasNext()) assertTrue(iterator.hasNext())
val message = iterator.next() val message = iterator.next().message
receivedMessages ::= message receivedMessages ::= message
debug("received message: " + message) debug("received message: " + message)
} }
@ -426,14 +426,14 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar
messages.sortWith((s,t) => s.checksum < t.checksum) messages.sortWith((s,t) => s.checksum < t.checksum)
} }
def getMessages(nMessagesPerThread: Int, topicMessageStreams: Map[String,List[KafkaMessageStream[Message]]]): List[Message]= { def getMessages(nMessagesPerThread: Int, topicMessageStreams: Map[String,List[KafkaStream[Message]]]): List[Message]= {
var messages: List[Message] = Nil var messages: List[Message] = Nil
for ((topic, messageStreams) <- topicMessageStreams) { for ((topic, messageStreams) <- topicMessageStreams) {
for (messageStream <- messageStreams) { for (messageStream <- messageStreams) {
val iterator = messageStream.iterator val iterator = messageStream.iterator
for (i <- 0 until nMessagesPerThread) { for (i <- 0 until nMessagesPerThread) {
assertTrue(iterator.hasNext) assertTrue(iterator.hasNext)
val message = iterator.next val message = iterator.next.message
messages ::= message messages ::= message
debug("received message: " + Utils.toString(message.payload, "UTF-8")) debug("received message: " + Utils.toString(message.payload, "UTF-8"))
} }

View File

@ -24,8 +24,6 @@ import kafka.api.{FetchRequestBuilder, OffsetRequest}
import kafka.consumer.SimpleConsumer import kafka.consumer.SimpleConsumer
import kafka.server.KafkaConfig import kafka.server.KafkaConfig
import kafka.utils.TestUtils import kafka.utils.TestUtils
import org.apache.log4j.Logger
import org.scalatest.junit.JUnit3Suite import org.scalatest.junit.JUnit3Suite
class BackwardsCompatibilityTest extends JUnit3Suite with KafkaServerTestHarness { class BackwardsCompatibilityTest extends JUnit3Suite with KafkaServerTestHarness {
@ -45,8 +43,6 @@ class BackwardsCompatibilityTest extends JUnit3Suite with KafkaServerTestHarness
val configs = List(new KafkaConfig(kafkaProps)) val configs = List(new KafkaConfig(kafkaProps))
var simpleConsumer: SimpleConsumer = null var simpleConsumer: SimpleConsumer = null
private val logger = Logger.getLogger(getClass())
override def setUp() { override def setUp() {
super.setUp() super.setUp()
simpleConsumer = new SimpleConsumer(host, port, 1000000, 64*1024) simpleConsumer = new SimpleConsumer(host, port, 1000000, 64*1024)

View File

@ -56,7 +56,7 @@ class FetcherTest extends JUnit3Suite with KafkaServerTestHarness {
super.setUp super.setUp
fetcher = new Fetcher(new ConsumerConfig(TestUtils.createConsumerProperties("", "", "")), null) fetcher = new Fetcher(new ConsumerConfig(TestUtils.createConsumerProperties("", "", "")), null)
fetcher.stopConnectionsToAllBrokers fetcher.stopConnectionsToAllBrokers
fetcher.startConnections(topicInfos, cluster, null) fetcher.startConnections(topicInfos, cluster)
} }
override def tearDown() { override def tearDown() {

View File

@ -22,14 +22,15 @@ import kafka.integration.KafkaServerTestHarness
import kafka.server._ import kafka.server._
import org.scalatest.junit.JUnit3Suite import org.scalatest.junit.JUnit3Suite
import scala.collection.JavaConversions._ import scala.collection.JavaConversions._
import kafka.consumer.{ConsumerConfig, KafkaMessageStream}
import org.apache.log4j.{Level, Logger} import org.apache.log4j.{Level, Logger}
import kafka.message._ import kafka.message._
import kafka.javaapi.producer.{ProducerData, Producer} import kafka.javaapi.producer.{ProducerData, Producer}
import kafka.utils.TestUtils._ import kafka.utils.TestUtils._
import kafka.utils.{Utils, Logging, TestUtils} import kafka.utils.{Utils, Logging, TestUtils}
import kafka.consumer.{KafkaStream, ConsumerConfig}
import kafka.zk.ZooKeeperTestHarness
class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHarness with Logging { class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHarness with ZooKeeperTestHarness with Logging {
val zookeeperConnect = zkConnect val zookeeperConnect = zkConnect
val numNodes = 2 val numNodes = 2
@ -93,7 +94,7 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar
messages.sortWith((s,t) => s.checksum < t.checksum) messages.sortWith((s,t) => s.checksum < t.checksum)
} }
def getMessages(nMessagesPerThread: Int, jTopicMessageStreams: java.util.Map[String, java.util.List[KafkaMessageStream[Message]]]) def getMessages(nMessagesPerThread: Int, jTopicMessageStreams: java.util.Map[String, java.util.List[KafkaStream[Message]]])
: List[Message]= { : List[Message]= {
var messages: List[Message] = Nil var messages: List[Message] = Nil
val topicMessageStreams = asMap(jTopicMessageStreams) val topicMessageStreams = asMap(jTopicMessageStreams)
@ -102,7 +103,7 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar
val iterator = messageStream.iterator val iterator = messageStream.iterator
for (i <- 0 until nMessagesPerThread) { for (i <- 0 until nMessagesPerThread) {
assertTrue(iterator.hasNext) assertTrue(iterator.hasNext)
val message = iterator.next val message = iterator.next.message
messages ::= message messages ::= message
debug("received message: " + Utils.toString(message.payload, "UTF-8")) debug("received message: " + Utils.toString(message.payload, "UTF-8"))
} }

View File

@ -43,6 +43,7 @@ class LogManagerTest extends JUnit3Suite with ZooKeeperTestHarness {
val props = TestUtils.createBrokerConfig(0, -1) val props = TestUtils.createBrokerConfig(0, -1)
config = new KafkaConfig(props) { config = new KafkaConfig(props) {
override val logFileSize = 1024 override val logFileSize = 1024
override val flushInterval = 100
} }
logManager = new LogManager(config, time, veryLargeLogFlushInterval, maxLogAge, false) logManager = new LogManager(config, time, veryLargeLogFlushInterval, maxLogAge, false)
logManager.startup logManager.startup
@ -86,10 +87,13 @@ class LogManagerTest extends JUnit3Suite with ZooKeeperTestHarness {
offset += set.sizeInBytes offset += set.sizeInBytes
} }
log.flush log.flush
// Why this sleep is required ? File system takes some time to update the last modified time for a file.
// TODO: What is unknown is why 1 second or couple 100 milliseconds didn't work ?
Thread.sleep(2000)
assertTrue("There should be more than one segment now.", log.numberOfSegments > 1) assertTrue("There should be more than one segment now.", log.numberOfSegments > 1)
// update the last modified time of all log segments
val logSegments = log.segments.view
logSegments.foreach(s => s.file.setLastModified(time.currentMs))
time.currentMs += maxLogAge + 3000 time.currentMs += maxLogAge + 3000
logManager.cleanupLogs() logManager.cleanupLogs()
assertEquals("Now there should only be only one segment.", 1, log.numberOfSegments) assertEquals("Now there should only be only one segment.", 1, log.numberOfSegments)
@ -114,8 +118,9 @@ class LogManagerTest extends JUnit3Suite with ZooKeeperTestHarness {
Thread.sleep(100) Thread.sleep(100)
config = new KafkaConfig(props) { config = new KafkaConfig(props) {
override val logFileSize = (10 * (setSize - 1)).asInstanceOf[Int] // each segment will be 10 messages override val logFileSize = (10 * (setSize - 1)).asInstanceOf[Int] // each segment will be 10 messages
override val logRetentionSize = (5 * 10 * setSize + 10).asInstanceOf[Int] // keep exactly 6 segments + 1 roll over override val logRetentionSize = (5 * 10 * setSize + 10).asInstanceOf[Long] // keep exactly 6 segments + 1 roll over
override val logRetentionHours = retentionHours override val logRetentionHours = retentionHours
override val flushInterval = 100
} }
logManager = new LogManager(config, time, veryLargeLogFlushInterval, retentionMs, false) logManager = new LogManager(config, time, veryLargeLogFlushInterval, retentionMs, false)
logManager.startup logManager.startup
@ -182,6 +187,7 @@ class LogManagerTest extends JUnit3Suite with ZooKeeperTestHarness {
config = new KafkaConfig(props) { config = new KafkaConfig(props) {
override val logFileSize = 256 override val logFileSize = 256
override val topicPartitionsMap = Utils.getTopicPartitions("testPartition:2") override val topicPartitionsMap = Utils.getTopicPartitions("testPartition:2")
override val flushInterval = 100
} }
logManager = new LogManager(config, time, veryLargeLogFlushInterval, maxLogAge, false) logManager = new LogManager(config, time, veryLargeLogFlushInterval, maxLogAge, false)

View File

@ -182,7 +182,10 @@ class LogTest extends JUnitSuite {
assertEquals(curOffset, log.nextAppendOffset) assertEquals(curOffset, log.nextAppendOffset)
// time goes by; the log file (which is empty) is deleted again // time goes by; the log file (which is empty) is deleted again
log.markDeletedWhile(_ => true) val deletedSegments = log.markDeletedWhile(_ => true)
// we shouldn't delete the last empty log segment.
assertTrue(deletedSegments.size == 0)
// we now have a new log // we now have a new log
assertEquals(curOffset, log.nextAppendOffset) assertEquals(curOffset, log.nextAppendOffset)

View File

@ -19,19 +19,19 @@ package kafka.log4j
import java.util.Properties import java.util.Properties
import java.io.File import java.io.File
import kafka.consumer.SimpleConsumer
import kafka.server.{KafkaConfig, KafkaServer}
import kafka.utils.{TestUtils, TestZKUtils, Utils, Logging}
import junit.framework.Assert._ import junit.framework.Assert._
import kafka.api.FetchRequestBuilder import kafka.api.FetchRequestBuilder
import kafka.consumer.SimpleConsumer
import kafka.message.Message import kafka.message.Message
import kafka.producer.async.MissingConfigException import kafka.producer.async.MissingConfigException
import kafka.serializer.Encoder import kafka.serializer.Encoder
import kafka.server.{KafkaConfig, KafkaServer}
import kafka.zk.ZooKeeperTestHarness import kafka.zk.ZooKeeperTestHarness
import org.apache.log4j.spi.LoggingEvent import org.apache.log4j.spi.LoggingEvent
import org.apache.log4j.{PropertyConfigurator, Logger} import org.apache.log4j.{PropertyConfigurator, Logger}
import org.junit.{After, Before, Test} import org.junit.{After, Before, Test}
import org.scalatest.junit.JUnit3Suite import org.scalatest.junit.JUnit3Suite
import kafka.utils._
class KafkaLog4jAppenderTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { class KafkaLog4jAppenderTest extends JUnit3Suite with ZooKeeperTestHarness with Logging {

View File

@ -94,6 +94,10 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases {
TestUtils.checkEquals[Message](messageList.iterator, TestUtils.getMessageIterator(messageSet.iterator)) TestUtils.checkEquals[Message](messageList.iterator, TestUtils.getMessageIterator(messageSet.iterator))
//make sure the last offset after iteration is correct //make sure the last offset after iteration is correct
assertEquals("offset of last message not expected", messageSet.last.offset, messageSet.getSerialized().limit) assertEquals("offset of last message not expected", messageSet.last.offset, messageSet.getSerialized().limit)
//make sure shallow iterator is the same as deep iterator
TestUtils.checkEquals[Message](TestUtils.getMessageIterator(messageSet.shallowIterator),
TestUtils.getMessageIterator(messageSet.iterator))
} }
// test for compressed regular messages // test for compressed regular messages
@ -104,6 +108,8 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases {
TestUtils.checkEquals[Message](messageList.iterator, TestUtils.getMessageIterator(messageSet.iterator)) TestUtils.checkEquals[Message](messageList.iterator, TestUtils.getMessageIterator(messageSet.iterator))
//make sure the last offset after iteration is correct //make sure the last offset after iteration is correct
assertEquals("offset of last message not expected", messageSet.last.offset, messageSet.getSerialized().limit) assertEquals("offset of last message not expected", messageSet.last.offset, messageSet.getSerialized().limit)
verifyShallowIterator(messageSet)
} }
// test for mixed empty and non-empty messagesets uncompressed // test for mixed empty and non-empty messagesets uncompressed
@ -121,6 +127,10 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases {
TestUtils.checkEquals[Message](messageList.iterator, TestUtils.getMessageIterator(mixedMessageSet.iterator)) TestUtils.checkEquals[Message](messageList.iterator, TestUtils.getMessageIterator(mixedMessageSet.iterator))
//make sure the last offset after iteration is correct //make sure the last offset after iteration is correct
assertEquals("offset of last message not expected", mixedMessageSet.last.offset, mixedMessageSet.getSerialized().limit) assertEquals("offset of last message not expected", mixedMessageSet.last.offset, mixedMessageSet.getSerialized().limit)
//make sure shallow iterator is the same as deep iterator
TestUtils.checkEquals[Message](TestUtils.getMessageIterator(mixedMessageSet.shallowIterator),
TestUtils.getMessageIterator(mixedMessageSet.iterator))
} }
// test for mixed empty and non-empty messagesets compressed // test for mixed empty and non-empty messagesets compressed
@ -138,7 +148,15 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases {
TestUtils.checkEquals[Message](messageList.iterator, TestUtils.getMessageIterator(mixedMessageSet.iterator)) TestUtils.checkEquals[Message](messageList.iterator, TestUtils.getMessageIterator(mixedMessageSet.iterator))
//make sure the last offset after iteration is correct //make sure the last offset after iteration is correct
assertEquals("offset of last message not expected", mixedMessageSet.last.offset, mixedMessageSet.getSerialized().limit) assertEquals("offset of last message not expected", mixedMessageSet.last.offset, mixedMessageSet.getSerialized().limit)
verifyShallowIterator(mixedMessageSet)
} }
} }
def verifyShallowIterator(messageSet: ByteBufferMessageSet) {
//make sure the offsets returned by a shallow iterator is a subset of that of a deep iterator
val shallowOffsets = messageSet.shallowIterator.map(msgAndOff => msgAndOff.offset).toSet
val deepOffsets = messageSet.iterator.map(msgAndOff => msgAndOff.offset).toSet
assertTrue(shallowOffsets.subsetOf(deepOffsets))
}
} }

View File

@ -20,10 +20,10 @@ package kafka.network;
import java.net._ import java.net._
import java.io._ import java.io._
import org.junit._ import org.junit._
import junit.framework.Assert._
import org.scalatest.junit.JUnitSuite import org.scalatest.junit.JUnitSuite
import kafka.utils.TestUtils import kafka.utils.TestUtils
import java.util.Random import java.util.Random
import junit.framework.Assert._
class SocketServerTest extends JUnitSuite { class SocketServerTest extends JUnitSuite {

View File

@ -23,7 +23,7 @@ import junit.framework.Assert
import kafka.admin.CreateTopicCommand import kafka.admin.CreateTopicCommand
import kafka.common.{ErrorMapping, MessageSizeTooLargeException} import kafka.common.{ErrorMapping, MessageSizeTooLargeException}
import kafka.integration.KafkaServerTestHarness import kafka.integration.KafkaServerTestHarness
import kafka.message.{NoCompressionCodec, Message, ByteBufferMessageSet} import kafka.message.{NoCompressionCodec, DefaultCompressionCodec, Message, ByteBufferMessageSet}
import kafka.server.KafkaConfig import kafka.server.KafkaConfig
import kafka.utils.{TestZKUtils, SystemTime, TestUtils} import kafka.utils.{TestZKUtils, SystemTime, TestUtils}
import org.junit.Test import org.junit.Test
@ -71,7 +71,7 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness {
} }
@Test @Test
def testMessageSizeTooLarge() { def testSingleMessageSizeTooLarge() {
val server = servers.head val server = servers.head
val props = new Properties() val props = new Properties()
props.put("host", "localhost") props.put("host", "localhost")
@ -90,6 +90,26 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness {
} }
} }
@Test
def testCompressedMessageSizeTooLarge() {
val server = servers.head
val props = new Properties()
props.put("host", "localhost")
props.put("port", server.socketServer.port.toString)
props.put("buffer.size", "102400")
props.put("connect.timeout.ms", "300")
props.put("reconnect.interval", "500")
props.put("max.message.size", "100")
val producer = new SyncProducer(new SyncProducerConfig(props))
val bytes = new Array[Byte](101)
try {
producer.send(TestUtils.produceRequest("test", 0, new ByteBufferMessageSet(compressionCodec = DefaultCompressionCodec, messages = new Message(bytes))))
Assert.fail("Message was too large to send, SyncProducer should have thrown exception for DefaultCompressionCodec.")
} catch {
case e: MessageSizeTooLargeException => /* success */
}
}
@Test @Test
def testProduceCorrectlyReceivesResponse() { def testProduceCorrectlyReceivesResponse() {
val server = servers.head val server = servers.head

View File

@ -20,6 +20,7 @@ import java.io.File
import kafka.consumer.SimpleConsumer import kafka.consumer.SimpleConsumer
import java.util.Properties import java.util.Properties
import org.junit.Test import org.junit.Test
import org.scalatest.junit.JUnitSuite
import junit.framework.Assert._ import junit.framework.Assert._
import kafka.message.{Message, ByteBufferMessageSet} import kafka.message.{Message, ByteBufferMessageSet}
import org.scalatest.junit.JUnit3Suite import org.scalatest.junit.JUnit3Suite

View File

@ -30,7 +30,7 @@ import kafka.message._
import org.I0Itec.zkclient.ZkClient import org.I0Itec.zkclient.ZkClient
import kafka.cluster.Broker import kafka.cluster.Broker
import collection.mutable.ListBuffer import collection.mutable.ListBuffer
import kafka.consumer.{KafkaMessageStream, ConsumerConfig} import kafka.consumer.ConsumerConfig
import scala.collection.Map import scala.collection.Map
import kafka.serializer.Encoder import kafka.serializer.Encoder
import kafka.api.{ProducerRequest, TopicData, PartitionData} import kafka.api.{ProducerRequest, TopicData, PartitionData}
@ -318,21 +318,6 @@ object TestUtils extends Logging {
brokers brokers
} }
def getConsumedMessages[T](nMessagesPerThread: Int, topicMessageStreams: Map[String,List[KafkaMessageStream[T]]]): List[T]= {
var messages: List[T] = Nil
for ((topic, messageStreams) <- topicMessageStreams) {
for (messageStream <- messageStreams) {
val iterator = messageStream.iterator
for (i <- 0 until nMessagesPerThread) {
assertTrue(iterator.hasNext)
val message = iterator.next
messages ::= message
}
}
}
messages
}
def getMsgStrings(n: Int): Seq[String] = { def getMsgStrings(n: Int): Seq[String] = {
val buffer = new ListBuffer[String] val buffer = new ListBuffer[String]
for (i <- 0 until n) for (i <- 0 until n)

View File

@ -20,6 +20,8 @@ package kafka.utils
import org.apache.log4j.Logger import org.apache.log4j.Logger
import org.scalatest.junit.JUnitSuite import org.scalatest.junit.JUnitSuite
import org.junit.Test import org.junit.Test
import org.junit.Assert._
class UtilsTest extends JUnitSuite { class UtilsTest extends JUnitSuite {
@ -29,5 +31,24 @@ class UtilsTest extends JUnitSuite {
def testSwallow() { def testSwallow() {
Utils.swallow(logger.info, throw new IllegalStateException("test")) Utils.swallow(logger.info, throw new IllegalStateException("test"))
} }
@Test
def testCircularIterator() {
val l = List(1, 2)
val itl = Utils.circularIterator(l)
assertEquals(1, itl.next())
assertEquals(2, itl.next())
assertEquals(1, itl.next())
assertEquals(2, itl.next())
assertFalse(itl.hasDefiniteSize)
val s = Set(1, 2)
val its = Utils.circularIterator(s)
assertEquals(1, its.next())
assertEquals(2, its.next())
assertEquals(1, its.next())
assertEquals(2, its.next())
assertEquals(1, its.next())
}
} }

View File

@ -16,16 +16,17 @@
*/ */
package kafka.examples; package kafka.examples;
import kafka.consumer.ConsumerConfig;
import kafka.consumer.ConsumerIterator;
import kafka.consumer.KafkaMessageStream;
import kafka.javaapi.consumer.ConsumerConnector;
import kafka.message.Message;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Properties; import java.util.Properties;
import kafka.consumer.ConsumerConfig;
import kafka.consumer.ConsumerIterator;
import kafka.consumer.KafkaStream;
import kafka.javaapi.consumer.ConsumerConnector;
import kafka.message.Message;
public class Consumer extends Thread public class Consumer extends Thread
{ {
@ -55,10 +56,10 @@ public class Consumer extends Thread
public void run() { public void run() {
Map<String, Integer> topicCountMap = new HashMap<String, Integer>(); Map<String, Integer> topicCountMap = new HashMap<String, Integer>();
topicCountMap.put(topic, new Integer(1)); topicCountMap.put(topic, new Integer(1));
Map<String, List<KafkaMessageStream<Message>>> consumerMap = consumer.createMessageStreams(topicCountMap); Map<String, List<KafkaStream<Message>>> consumerMap = consumer.createMessageStreams(topicCountMap);
KafkaMessageStream<Message> stream = consumerMap.get(topic).get(0); KafkaStream<Message> stream = consumerMap.get(topic).get(0);
ConsumerIterator<Message> it = stream.iterator(); ConsumerIterator<Message> it = stream.iterator();
while(it.hasNext()) while(it.hasNext())
System.out.println(ExampleUtils.getMessage(it.next())); System.out.println(ExampleUtils.getMessage(it.next().message()));
} }
} }

View File

@ -16,8 +16,8 @@
*/ */
package kafka.examples; package kafka.examples;
import java.nio.ByteBuffer;
import java.nio.ByteBuffer;
import kafka.message.Message; import kafka.message.Message;
public class ExampleUtils public class ExampleUtils

View File

@ -16,9 +16,10 @@
*/ */
package kafka.examples; package kafka.examples;
import java.util.Properties;
import kafka.javaapi.producer.ProducerData; import kafka.javaapi.producer.ProducerData;
import kafka.producer.ProducerConfig; import kafka.producer.ProducerConfig;
import java.util.Properties;
public class Producer extends Thread public class Producer extends Thread
{ {

View File

@ -19,17 +19,14 @@ package kafka.examples;
import kafka.api.FetchRequest; import kafka.api.FetchRequest;
import kafka.api.FetchRequestBuilder; import kafka.api.FetchRequestBuilder;
import kafka.javaapi.FetchResponse; import kafka.javaapi.FetchResponse;
import java.util.ArrayList;
import java.util.List;
import kafka.javaapi.consumer.SimpleConsumer; import kafka.javaapi.consumer.SimpleConsumer;
import kafka.javaapi.message.ByteBufferMessageSet; import kafka.javaapi.message.ByteBufferMessageSet;
import kafka.javaapi.message.MessageSet;
import kafka.message.MessageAndOffset; import kafka.message.MessageAndOffset;
import java.util.ArrayList;
import java.util.HashMap; import java.util.HashMap;
import java.util.List;
import java.util.Map; import java.util.Map;
public class SimpleConsumerDemo { public class SimpleConsumerDemo {
private static void printMessages(ByteBufferMessageSet messageSet) { private static void printMessages(ByteBufferMessageSet messageSet) {

View File

@ -17,15 +17,12 @@
package kafka.perf package kafka.perf
import java.net.URI
import java.util.concurrent.CountDownLatch import java.util.concurrent.CountDownLatch
import java.util.concurrent.atomic.AtomicLong import java.util.concurrent.atomic.AtomicLong
import java.nio.channels.ClosedByInterruptException import java.nio.channels.ClosedByInterruptException
import joptsimple._
import org.apache.log4j.Logger import org.apache.log4j.Logger
import kafka.message.Message import kafka.message.Message
import org.I0Itec.zkclient.ZkClient import kafka.utils.Utils
import kafka.utils.{ZKStringSerializer, Utils}
import java.util.{Random, Properties} import java.util.{Random, Properties}
import kafka.consumer._ import kafka.consumer._
import java.text.SimpleDateFormat import java.text.SimpleDateFormat
@ -139,7 +136,7 @@ object ConsumerPerformance {
val hideHeader = options.has(hideHeaderOpt) val hideHeader = options.has(hideHeaderOpt)
} }
class ConsumerPerfThread(threadId: Int, name: String, stream: KafkaMessageStream[Message], class ConsumerPerfThread(threadId: Int, name: String, stream: KafkaStream[Message],
config:ConsumerPerfConfig, totalMessagesRead: AtomicLong, totalBytesRead: AtomicLong) config:ConsumerPerfConfig, totalMessagesRead: AtomicLong, totalBytesRead: AtomicLong)
extends Thread(name) { extends Thread(name) {
private val shutdownLatch = new CountDownLatch(1) private val shutdownLatch = new CountDownLatch(1)
@ -157,9 +154,9 @@ object ConsumerPerformance {
var lastMessagesRead = 0L var lastMessagesRead = 0L
try { try {
for (message <- stream if messagesRead < config.numMessages) { for (messageAndMetadata <- stream if messagesRead < config.numMessages) {
messagesRead += 1 messagesRead += 1
bytesRead += message.payloadSize bytesRead += messageAndMetadata.message.payloadSize
if (messagesRead % config.reportingInterval == 0) { if (messagesRead % config.reportingInterval == 0) {
if(config.showDetailedStats) if(config.showDetailedStats)

View File

@ -18,7 +18,7 @@
package kafka.perf package kafka.perf
import joptsimple.OptionParser import joptsimple.OptionParser
import java.text.SimpleDateFormat
class PerfConfig(args: Array[String]) { class PerfConfig(args: Array[String]) {
val parser = new OptionParser val parser = new OptionParser

View File

@ -20,18 +20,16 @@ package kafka.perf
import java.util.concurrent.{CountDownLatch, Executors} import java.util.concurrent.{CountDownLatch, Executors}
import java.util.concurrent.atomic.AtomicLong import java.util.concurrent.atomic.AtomicLong
import kafka.producer._ import kafka.producer._
import async.DefaultEventHandler
import org.apache.log4j.Logger import org.apache.log4j.Logger
import joptsimple.OptionParser
import kafka.message.{CompressionCodec, Message} import kafka.message.{CompressionCodec, Message}
import kafka.serializer.DefaultEncoder
import java.text.SimpleDateFormat import java.text.SimpleDateFormat
import java.util.{Date, Random, Properties} import java.util.{Random, Properties}
import kafka.utils.Logging
/** /**
* Load test for the producer * Load test for the producer
*/ */
object ProducerPerformance { object ProducerPerformance extends Logging {
def main(args: Array[String]) { def main(args: Array[String]) {
@ -141,7 +139,6 @@ object ProducerPerformance {
val totalMessagesSent: AtomicLong, val totalMessagesSent: AtomicLong,
val allDone: CountDownLatch, val allDone: CountDownLatch,
val rand: Random) extends Runnable { val rand: Random) extends Runnable {
val logger = Logger.getLogger(getClass)
val props = new Properties() val props = new Properties()
val brokerInfoList = config.brokerInfo.split("=") val brokerInfoList = config.brokerInfo.split("=")
if (brokerInfoList(0) == "zk.connect") { if (brokerInfoList(0) == "zk.connect") {
@ -171,7 +168,7 @@ object ProducerPerformance {
var lastReportTime = reportTime var lastReportTime = reportTime
val messagesPerThread = if(!config.isAsync) config.numMessages / config.numThreads / config.batchSize val messagesPerThread = if(!config.isAsync) config.numMessages / config.numThreads / config.batchSize
else config.numMessages / config.numThreads else config.numMessages / config.numThreads
if(logger.isDebugEnabled) logger.debug("Messages per thread = " + messagesPerThread) debug("Messages per thread = " + messagesPerThread)
var messageSet: List[Message] = Nil var messageSet: List[Message] = Nil
if(config.isFixSize) { if(config.isFixSize) {
for(k <- 0 until config.batchSize) { for(k <- 0 until config.batchSize) {
@ -203,11 +200,11 @@ object ProducerPerformance {
rand.nextBytes(messageBytes) rand.nextBytes(messageBytes)
val message = new Message(messageBytes) val message = new Message(messageBytes)
producer.send(new ProducerData[Message,Message](config.topic, message)) producer.send(new ProducerData[Message,Message](config.topic, message))
if(logger.isDebugEnabled) println("checksum:" + message.checksum) debug(config.topic + "-checksum:" + message.checksum)
bytesSent += message.payloadSize bytesSent += message.payloadSize
}else { }else {
producer.send(new ProducerData[Message,Message](config.topic, message)) producer.send(new ProducerData[Message,Message](config.topic, message))
if(logger.isDebugEnabled) println("checksum:" + message.checksum) debug(config.topic + "-checksum:" + message.checksum)
bytesSent += message.payloadSize bytesSent += message.payloadSize
} }
nSends += 1 nSends += 1

View File

@ -73,7 +73,9 @@ readonly num_iterations=5
readonly zk_source_port=2181 readonly zk_source_port=2181
readonly zk_mirror_port=2182 readonly zk_mirror_port=2182
readonly topic_1=test01 readonly topic_prefix=test
readonly max_topic_id=2
readonly unbalanced_start_id=2
readonly consumer_grp=group1 readonly consumer_grp=group1
readonly source_console_consumer_grp=source readonly source_console_consumer_grp=source
readonly mirror_console_consumer_grp=mirror readonly mirror_console_consumer_grp=mirror
@ -96,10 +98,16 @@ readonly num_kafka_target_server=3
readonly wait_time_after_killing_broker=0 readonly wait_time_after_killing_broker=0
readonly wait_time_after_restarting_broker=5 readonly wait_time_after_restarting_broker=5
background_producer_pid= readonly producer_4_brokerinfo_str="broker.list=1:localhost:9091,2:localhost:9092,3:localhost:9093,4:localhost:9094"
readonly producer_3_brokerinfo_str="broker.list=1:localhost:9091,2:localhost:9092,3:localhost:9093"
background_producer_pid_1=
background_producer_pid_2=
no_bouncing=$# no_bouncing=$#
iter=1 iter=1
abort_test=false
pid_zk_source= pid_zk_source=
pid_zk_target= pid_zk_target=
@ -177,17 +185,29 @@ get_random_range() {
verify_consumer_rebalancing() { verify_consumer_rebalancing() {
info "Verifying consumer rebalancing operation" info "Verifying consumer rebalancing operation"
$base_dir/bin/kafka-run-class.sh \ CONSUMER_REBALANCING_RESULT=`$base_dir/bin/kafka-run-class.sh \
kafka.tools.VerifyConsumerRebalance \ kafka.tools.VerifyConsumerRebalance \
--zk.connect=localhost:2181 \ --zk.connect=localhost:2181 \
--group $consumer_grp \ --group $consumer_grp`
2>&1 >> $consumer_rebalancing_log echo "$CONSUMER_REBALANCING_RESULT" >> $consumer_rebalancing_log
REBALANCE_STATUS_LINE=`grep "Rebalance operation" $consumer_rebalancing_log | tail -1`
# info "REBALANCE_STATUS_LINE: $REBALANCE_STATUS_LINE"
REBALANCE_STATUS=`echo $REBALANCE_STATUS_LINE | grep "Rebalance operation successful" || echo -n "Rebalance operation failed"`
info "REBALANCE_STATUS: $REBALANCE_STATUS"
if [ "${REBALANCE_STATUS}_x" == "Rebalance operation failed_x" ]; then
info "setting abort_test to true due to Rebalance operation failed"
abort_test="true"
fi
} }
wait_for_zero_consumer_lags() { wait_for_zero_consumer_lags() {
topic_id=$1
# no of times to check for zero lagging # no of times to check for zero lagging
no_of_zero_to_verify=3 no_of_zero_to_verify=3
@ -196,7 +216,7 @@ wait_for_zero_consumer_lags() {
TOTAL_LAG=0 TOTAL_LAG=0
CONSUMER_LAGS=`$base_dir/bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker \ CONSUMER_LAGS=`$base_dir/bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker \
--group $consumer_grp --zkconnect localhost:$zk_source_port \ --group $consumer_grp --zkconnect localhost:$zk_source_port \
--topic $topic_1 | grep "Consumer lag" | tr -d ' ' | cut -f2 -d '='` --topic ${topic_prefix}_${topic_id} | grep "Consumer lag" | tr -d ' ' | cut -f2 -d '='`
for lag in $CONSUMER_LAGS; for lag in $CONSUMER_LAGS;
do do
@ -217,6 +237,8 @@ wait_for_zero_consumer_lags() {
wait_for_zero_source_console_consumer_lags() { wait_for_zero_source_console_consumer_lags() {
topic_id=$1
# no of times to check for zero lagging # no of times to check for zero lagging
no_of_zero_to_verify=3 no_of_zero_to_verify=3
@ -225,7 +247,7 @@ wait_for_zero_source_console_consumer_lags() {
TOTAL_LAG=0 TOTAL_LAG=0
CONSUMER_LAGS=`$base_dir/bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker \ CONSUMER_LAGS=`$base_dir/bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker \
--group $source_console_consumer_grp --zkconnect localhost:$zk_source_port \ --group $source_console_consumer_grp --zkconnect localhost:$zk_source_port \
--topic $topic_1 | grep "Consumer lag" | tr -d ' ' | cut -f2 -d '='` --topic ${topic_prefix}_${topic_id} | grep "Consumer lag" | tr -d ' ' | cut -f2 -d '='`
for lag in $CONSUMER_LAGS; for lag in $CONSUMER_LAGS;
do do
@ -246,6 +268,8 @@ wait_for_zero_source_console_consumer_lags() {
wait_for_zero_mirror_console_consumer_lags() { wait_for_zero_mirror_console_consumer_lags() {
topic_id=$1
# no of times to check for zero lagging # no of times to check for zero lagging
no_of_zero_to_verify=3 no_of_zero_to_verify=3
@ -254,7 +278,7 @@ wait_for_zero_mirror_console_consumer_lags() {
TOTAL_LAG=0 TOTAL_LAG=0
CONSUMER_LAGS=`$base_dir/bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker \ CONSUMER_LAGS=`$base_dir/bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker \
--group $mirror_console_consumer_grp --zkconnect localhost:$zk_mirror_port \ --group $mirror_console_consumer_grp --zkconnect localhost:$zk_mirror_port \
--topic $topic_1 | grep "Consumer lag" | tr -d ' ' | cut -f2 -d '='` --topic ${topic_prefix}_${topic_id} | grep "Consumer lag" | tr -d ' ' | cut -f2 -d '='`
for lag in $CONSUMER_LAGS; for lag in $CONSUMER_LAGS;
do do
@ -321,6 +345,8 @@ cleanup() {
rm -f $console_consumer_source_crc_sorted_log rm -f $console_consumer_source_crc_sorted_log
rm -f $console_consumer_mirror_crc_sorted_uniq_log rm -f $console_consumer_mirror_crc_sorted_uniq_log
rm -f $console_consumer_source_crc_sorted_uniq_log rm -f $console_consumer_source_crc_sorted_uniq_log
rm -f $consumer_rebalancing_log
} }
start_zk() { start_zk() {
@ -380,40 +406,65 @@ start_embedded_consumer_server() {
} }
start_console_consumer_for_source_producer() { start_console_consumer_for_source_producer() {
info "starting console consumers for source producer"
topic_id=$1
info "starting console consumers for source producer on topic id [$topic_id]"
$base_dir/bin/kafka-run-class.sh kafka.consumer.ConsoleConsumer \ $base_dir/bin/kafka-run-class.sh kafka.consumer.ConsoleConsumer \
--zookeeper localhost:$zk_source_port \ --zookeeper localhost:$zk_source_port \
--topic $topic_1 \ --topic ${topic_prefix}_${topic_id} \
--group $source_console_consumer_grp \ --group $source_console_consumer_grp \
--from-beginning \ --from-beginning --consumer-timeout-ms 5000 \
--formatter "kafka.consumer.ConsoleConsumer\$ChecksumMessageFormatter" \ --formatter "kafka.consumer.ConsoleConsumer\$ChecksumMessageFormatter" \
2>&1 > ${console_consumer_source_log} & --property topic=${topic_prefix}_${topic_id} \
console_consumer_source_pid=$! 2>&1 >> ${console_consumer_source_log}
info " -> console consumer source pid: $console_consumer_source_pid"
} }
start_console_consumer_for_mirror_producer() { start_console_consumer_for_mirror_producer() {
info "starting console consumers for mirroring producer"
topic_id=$1
info "starting console consumers for mirroring producer on topic id [$topic_id]"
$base_dir/bin/kafka-run-class.sh kafka.consumer.ConsoleConsumer \ $base_dir/bin/kafka-run-class.sh kafka.consumer.ConsoleConsumer \
--zookeeper localhost:$zk_mirror_port \ --zookeeper localhost:$zk_mirror_port \
--topic $topic_1 \ --topic ${topic_prefix}_${topic_id} \
--group $mirror_console_consumer_grp \ --group $mirror_console_consumer_grp \
--from-beginning \ --from-beginning --consumer-timeout-ms 5000 \
--formatter "kafka.consumer.ConsoleConsumer\$ChecksumMessageFormatter" \ --formatter "kafka.consumer.ConsoleConsumer\$ChecksumMessageFormatter" \
2>&1 > ${console_consumer_mirror_log} & --property topic=${topic_prefix}_${topic_id} \
console_consumer_mirror_pid=$! 2>&1 >> ${console_consumer_mirror_log}
}
info " -> console consumer mirror pid: $console_consumer_mirror_pid" consume_source_producer_messages() {
consumer_counter=1
while [ $consumer_counter -le $max_topic_id ]
do
start_console_consumer_for_source_producer $consumer_counter
consumer_counter=$(( $consumer_counter + 1 ))
done
}
consume_mirror_producer_messages() {
consumer_counter=1
while [ $consumer_counter -le $max_topic_id ]
do
start_console_consumer_for_mirror_producer $consumer_counter
consumer_counter=$(( $consumer_counter + 1 ))
done
} }
shutdown_producer() { shutdown_producer() {
info "shutting down producer" info "shutting down producer"
if [ "x${background_producer_pid}" != "x" ]; then if [ "x${background_producer_pid_1}" != "x" ]; then
# kill_child_processes 0 ${background_producer_pid}; # kill_child_processes 0 ${background_producer_pid_1};
kill -TERM ${background_producer_pid} 2> /dev/null; kill -TERM ${background_producer_pid_1} 2> /dev/null;
fi
if [ "x${background_producer_pid_2}" != "x" ]; then
# kill_child_processes 0 ${background_producer_pid_2};
kill -TERM ${background_producer_pid_2} 2> /dev/null;
fi fi
} }
@ -450,13 +501,15 @@ shutdown_servers() {
} }
start_background_producer() { start_background_producer() {
bkrinfo_str=$1
start_topic_id=$2
end_topic_id=$3
batch_no=0 batch_no=0
curr_iter=0 topic_id=${start_topic_id}
while [ $num_iterations -gt $curr_iter ] while [ 'x' == 'x' ]
do do
topic=$1
sleeptime= sleeptime=
get_random_range $sleep_min $sleep_max get_random_range $sleep_min $sleep_max
@ -464,19 +517,24 @@ start_background_producer() {
batch_no=$(($batch_no + 1)) batch_no=$(($batch_no + 1))
if [ $topic_id -gt $end_topic_id ]; then
topic_id=${start_topic_id}
fi
$base_dir/bin/kafka-run-class.sh \ $base_dir/bin/kafka-run-class.sh \
kafka.perf.ProducerPerformance \ kafka.perf.ProducerPerformance \
--brokerinfo zk.connect=localhost:2181 \ --brokerinfo $bkrinfo_str \
--topic $topic \ --topic ${topic_prefix}_${topic_id} \
--messages $num_msg_per_batch \ --messages $num_msg_per_batch \
--message-size $message_size \ --message-size $message_size \
--batch-size 50 \ --batch-size 50 \
--vary-message-size \ --vary-message-size \
--threads 1 \ --threads 1 \
--reporting-interval $num_msg_per_batch \ --reporting-interval $num_msg_per_batch --async \
--async \
2>&1 >> $base_dir/producer_performance.log # appending all producers' msgs 2>&1 >> $base_dir/producer_performance.log # appending all producers' msgs
topic_id=$(( $topic_id + 1 ))
sleep $sleeptime sleep $sleeptime
done done
} }
@ -485,9 +543,9 @@ cmp_checksum() {
cmp_result=0 cmp_result=0
grep ^checksum $console_consumer_source_log | tr -d ' ' | cut -f2 -d ':' > $console_consumer_source_crc_log grep checksum $console_consumer_source_log | tr -d ' ' | cut -f2 -d ':' > $console_consumer_source_crc_log
grep ^checksum $console_consumer_mirror_log | tr -d ' ' | cut -f2 -d ':' > $console_consumer_mirror_crc_log grep checksum $console_consumer_mirror_log | tr -d ' ' | cut -f2 -d ':' > $console_consumer_mirror_crc_log
grep ^checksum $producer_performance_log | tr -d ' ' | cut -f2 -d ':' > $producer_performance_crc_log grep checksum $producer_performance_log | tr -d ' ' | cut -f4 -d ':' | cut -f1 -d '(' > $producer_performance_crc_log
sort $console_consumer_mirror_crc_log > $console_consumer_mirror_crc_sorted_log sort $console_consumer_mirror_crc_log > $console_consumer_mirror_crc_sorted_log
sort $console_consumer_source_crc_log > $console_consumer_source_crc_sorted_log sort $console_consumer_source_crc_log > $console_consumer_source_crc_sorted_log
@ -555,6 +613,37 @@ cmp_checksum() {
echo "========================================================" >> $checksum_diff_log echo "========================================================" >> $checksum_diff_log
echo "${duplicate_mirror_crc}" >> $checksum_diff_log echo "${duplicate_mirror_crc}" >> $checksum_diff_log
topic_chksum_counter=1
while [ $topic_chksum_counter -le $max_topic_id ]
do
# get producer topic counts
this_chksum_count=`grep -c ${topic_prefix}_${topic_chksum_counter}\- $producer_performance_log`
echo "PRODUCER topic ${topic_prefix}_${topic_chksum_counter} count: ${this_chksum_count}"
topic_chksum_counter=$(($topic_chksum_counter + 1))
done
echo
topic_chksum_counter=1
while [ $topic_chksum_counter -le $max_topic_id ]
do
this_chksum_count=`grep -c ${topic_prefix}_${topic_chksum_counter}\- $console_consumer_source_log`
echo "SOURCE consumer topic ${topic_prefix}_${topic_chksum_counter} count: ${this_chksum_count}"
topic_chksum_counter=$(($topic_chksum_counter + 1))
done
echo
topic_chksum_counter=1
while [ $topic_chksum_counter -le $max_topic_id ]
do
this_chksum_count=`grep -c ${topic_prefix}_${topic_chksum_counter}\- $console_consumer_mirror_log`
echo "MIRROR consumer topic ${topic_prefix}_${topic_chksum_counter} count: ${this_chksum_count}"
topic_chksum_counter=$(($topic_chksum_counter + 1))
done
echo
return $cmp_result return $cmp_result
} }
@ -567,15 +656,32 @@ start_test() {
start_target_servers_cluster start_target_servers_cluster
sleep 2 sleep 2
start_background_producer $topic_1 & start_background_producer $producer_4_brokerinfo_str 1 $(( $unbalanced_start_id - 1 )) &
background_producer_pid=$! background_producer_pid_1=$!
info "==========================================" info "=========================================="
info "Started background producer pid [${background_producer_pid}]" info "Started background producer pid [${background_producer_pid_1}]"
info "==========================================" info "=========================================="
sleep 5 sleep 10
start_background_producer $producer_3_brokerinfo_str $unbalanced_start_id $max_topic_id &
background_producer_pid_2=$!
info "=========================================="
info "Started background producer pid [${background_producer_pid_2}]"
info "=========================================="
sleep 10
verify_consumer_rebalancing
info "abort_test: [${abort_test}]"
if [ "${abort_test}_x" == "true_x" ]; then
info "aborting test"
iter=$((${num_iterations} + 1))
fi
while [ $num_iterations -ge $iter ] while [ $num_iterations -ge $iter ]
do do
echo echo
@ -592,7 +698,6 @@ start_test() {
# even iterations -> bounce target kafka borker # even iterations -> bounce target kafka borker
get_random_range 1 $num_kafka_target_server get_random_range 1 $num_kafka_target_server
idx=$? idx=$?
if [ "x${kafka_target_pids[$idx]}" != "x" ]; then if [ "x${kafka_target_pids[$idx]}" != "x" ]; then
echo echo
info "#### Bouncing kafka TARGET broker ####" info "#### Bouncing kafka TARGET broker ####"
@ -631,7 +736,15 @@ start_test() {
sleep $wait_time_after_restarting_broker sleep $wait_time_after_restarting_broker
fi fi
fi fi
verify_consumer_rebalancing verify_consumer_rebalancing
info "abort_test: [${abort_test}]"
if [ "${abort_test}_x" == "true_x" ]; then
info "aborting test"
iter=$((${num_iterations} + 1))
fi
else else
info "No bouncing performed" info "No bouncing performed"
fi fi
@ -670,8 +783,8 @@ trap "shutdown_producer; shutdown_servers; cmp_checksum; exit 0" INT
start_test start_test
start_console_consumer_for_source_producer consume_source_producer_messages
start_console_consumer_for_mirror_producer consume_mirror_producer_messages
wait_for_zero_source_console_consumer_lags wait_for_zero_source_console_consumer_lags
wait_for_zero_mirror_console_consumer_lags wait_for_zero_mirror_console_consumer_lags

View File

@ -12,30 +12,74 @@
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and # See the License for the specific language governing permissions and
# limitations under the License. # limitations under the License.
log4j.rootLogger=INFO, stdout
log4j.rootLogger=INFO, stdout, kafkaAppender
# ====================================
# messages going to kafkaAppender
# ====================================
log4j.logger.kafka=DEBUG, kafkaAppender
log4j.logger.org.I0Itec.zkclient.ZkClient=INFO, kafkaAppender
log4j.logger.org.apache.zookeeper=INFO, kafkaAppender
# ====================================
# messages going to zookeeperAppender
# ====================================
# (comment out this line to redirect ZK-related messages to kafkaAppender
# to allow reading both Kafka and ZK debugging messages in a single file)
#log4j.logger.org.apache.zookeeper=INFO, zookeeperAppender
# ====================================
# stdout
# ====================================
log4j.appender.stdout=org.apache.log4j.ConsoleAppender log4j.appender.stdout=org.apache.log4j.ConsoleAppender
log4j.appender.stdout.layout=org.apache.log4j.PatternLayout log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c)%n log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c)%n
#log4j.appender.fileAppender=org.apache.log4j.FileAppender # ====================================
#log4j.appender.fileAppender.File=kafka-request.log # fileAppender
#log4j.appender.fileAppender.layout=org.apache.log4j.PatternLayout # ====================================
#log4j.appender.fileAppender.layout.ConversionPattern= %-4r [%t] %-5p %c %x - %m%n log4j.appender.fileAppender=org.apache.log4j.FileAppender
log4j.appender.fileAppender.File=/tmp/kafka_all_request.log
log4j.appender.fileAppender.layout=org.apache.log4j.PatternLayout
log4j.appender.fileAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
# ====================================
# kafkaAppender
# ====================================
log4j.appender.kafkaAppender=org.apache.log4j.DailyRollingFileAppender
log4j.appender.kafkaAppender.File=/tmp/kafka.log
log4j.appender.kafkaAppender.layout=org.apache.log4j.PatternLayout
log4j.appender.kafkaAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
log4j.additivity.kafka=true
# Turn on all our debugging info # ====================================
#log4j.logger.kafka=INFO # zookeeperAppender
log4j.logger.org.I0Itec.zkclient.ZkClient=INFO # ====================================
log4j.logger.org.apache.zookeeper=INFO log4j.appender.zookeeperAppender=org.apache.log4j.DailyRollingFileAppender
log4j.logger.kafka.consumer=DEBUG log4j.appender.zookeeperAppender.File=/tmp/zookeeper.log
log4j.logger.kafka.server.EmbeddedConsumer$MirroringThread=TRACE log4j.appender.zookeeperAppender.layout=org.apache.log4j.PatternLayout
log4j.logger.kafka.server.KafkaRequestHandlers=TRACE log4j.appender.zookeeperAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
log4j.additivity.org.apache.zookeeper=false
# ====================================
# other available debugging info
# ====================================
#log4j.logger.kafka.server.EmbeddedConsumer$MirroringThread=TRACE
#log4j.logger.kafka.server.KafkaRequestHandlers=TRACE
#log4j.logger.kafka.producer.async.AsyncProducer=TRACE #log4j.logger.kafka.producer.async.AsyncProducer=TRACE
#log4j.logger.kafka.producer.async.ProducerSendThread=TRACE #log4j.logger.kafka.producer.async.ProducerSendThread=TRACE
log4j.logger.kafka.producer.async.DefaultEventHandler=TRACE #log4j.logger.kafka.producer.async.DefaultEventHandler=TRACE
log4j.logger.kafka.consumer=DEBUG
log4j.logger.kafka.tools.VerifyConsumerRebalance=DEBUG log4j.logger.kafka.tools.VerifyConsumerRebalance=DEBUG
# to print message checksum from ProducerPerformance # to print message checksum from ProducerPerformance
log4j.logger.kafka.perf.ProducerPerformance$ProducerThread=DEBUG log4j.logger.kafka.perf.ProducerPerformance$ProducerThread=DEBUG
# to print socket buffer size validated by Kafka broker
log4j.logger.kafka.network.Acceptor=DEBUG
# to print socket buffer size validated by SimpleConsumer
log4j.logger.kafka.consumer.SimpleConsumer=TRACE

View File

@ -15,7 +15,7 @@
# zk connection string # zk connection string
# comma separated host:port pairs, each corresponding to a zk # comma separated host:port pairs, each corresponding to a zk
# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" # server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002"
broker.list=0:localhost:9093 broker.list=0:localhost:9081
# timeout in ms for connecting to zookeeper # timeout in ms for connecting to zookeeper
zk.connectiontimeout.ms=1000000 zk.connectiontimeout.ms=1000000

View File

@ -15,7 +15,7 @@
# zk connection string # zk connection string
# comma separated host:port pairs, each corresponding to a zk # comma separated host:port pairs, each corresponding to a zk
# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" # server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002"
broker.list=0:localhost:9094 broker.list=0:localhost:9082
# timeout in ms for connecting to zookeeper # timeout in ms for connecting to zookeeper
zk.connectiontimeout.ms=1000000 zk.connectiontimeout.ms=1000000

View File

@ -15,7 +15,7 @@
# zk connection string # zk connection string
# comma separated host:port pairs, each corresponding to a zk # comma separated host:port pairs, each corresponding to a zk
# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" # server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002"
broker.list=0:localhost:9095 broker.list=0:localhost:9083
# timeout in ms for connecting to zookeeper # timeout in ms for connecting to zookeeper
zk.connectiontimeout.ms=1000000 zk.connectiontimeout.ms=1000000

View File

@ -26,7 +26,7 @@ brokerid=1
num.partitions=1 num.partitions=1
# the port the socket server runs on # the port the socket server runs on
port=9092 port=9091
# the number of processor threads the socket server uses. Defaults to the number of cores on the machine # the number of processor threads the socket server uses. Defaults to the number of cores on the machine
num.threads=8 num.threads=8

View File

@ -26,7 +26,7 @@ brokerid=2
num.partitions=1 num.partitions=1
# the port the socket server runs on # the port the socket server runs on
port=9091 port=9092
# the number of processor threads the socket server uses. Defaults to the number of cores on the machine # the number of processor threads the socket server uses. Defaults to the number of cores on the machine
num.threads=8 num.threads=8

View File

@ -26,7 +26,7 @@ brokerid=3
num.partitions=1 num.partitions=1
# the port the socket server runs on # the port the socket server runs on
port=9090 port=9093
# the number of processor threads the socket server uses. Defaults to the number of cores on the machine # the number of processor threads the socket server uses. Defaults to the number of cores on the machine
num.threads=8 num.threads=8

View File

@ -26,7 +26,7 @@ brokerid=4
num.partitions=1 num.partitions=1
# the port the socket server runs on # the port the socket server runs on
port=9096 port=9094
# the number of processor threads the socket server uses. Defaults to the number of cores on the machine # the number of processor threads the socket server uses. Defaults to the number of cores on the machine
num.threads=8 num.threads=8

View File

@ -26,7 +26,7 @@ brokerid=1
num.partitions=1 num.partitions=1
# the port the socket server runs on # the port the socket server runs on
port=9093 port=9081
# the number of processor threads the socket server uses. Defaults to the number of cores on the machine # the number of processor threads the socket server uses. Defaults to the number of cores on the machine
num.threads=8 num.threads=8

View File

@ -26,7 +26,7 @@ brokerid=2
num.partitions=1 num.partitions=1
# the port the socket server runs on # the port the socket server runs on
port=9094 port=9082
# the number of processor threads the socket server uses. Defaults to the number of cores on the machine # the number of processor threads the socket server uses. Defaults to the number of cores on the machine
num.threads=8 num.threads=8

View File

@ -26,7 +26,7 @@ brokerid=3
num.partitions=1 num.partitions=1
# the port the socket server runs on # the port the socket server runs on
port=9095 port=9083
# the number of processor threads the socket server uses. Defaults to the number of cores on the machine # the number of processor threads the socket server uses. Defaults to the number of cores on the machine
num.threads=8 num.threads=8

View File

@ -25,5 +25,5 @@ zk.connectiontimeout.ms=1000000
#consumer group id #consumer group id
groupid=group1 groupid=group1
mirror.topics.whitelist=test01 mirror.topics.whitelist=test_1,test_2
autooffset.reset=smallest

View File

@ -1,27 +0,0 @@
This test replicates messages from 3 kafka brokers to 2 other kafka brokers
using the embedded consumer. At the end, the messages produced at the source
brokers should match that at the target brokers.
To run this test, do
bin/run-test.sh
The expected output is given in bin/expected.out. There is only 1 thing that's
important.
1. The output should have a line "test passed".
In the event of failure, by default the brokers and zookeepers remain running
to make it easier to debug the issue - hit Ctrl-C to shut them down. You can
change this behavior by setting the action_on_fail flag in the script to "exit"
or "proceed", in which case a snapshot of all the logs and directories is
placed in the test's base directory.
If you are making any changes that may affect the embedded consumer, it is a
good idea to run the test in a loop. E.g.:
:>/tmp/embeddedconsumer_test.log
for i in {1..10}; do echo "run $i"; ./bin/run-test.sh 2>1 >> /tmp/embeddedconsumer_test.log; done
tail -F /tmp/embeddedconsumer_test.log
grep -ic passed /tmp/embeddedconsumer_test.log
grep -ic failed /tmp/embeddedconsumer_test.log

View File

@ -1,328 +0,0 @@
#!/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.
readonly num_messages=400000
readonly message_size=400
readonly action_on_fail="proceed"
readonly test_start_time="$(date +%s)"
readonly base_dir=$(dirname $0)/..
info() {
echo -e "$(date +"%Y-%m-%d %H:%M:%S") $*"
}
kill_child_processes() {
isTopmost=$1
curPid=$2
childPids=$(ps a -o pid= -o ppid= | grep "${curPid}$" | awk '{print $1;}')
for childPid in $childPids
do
kill_child_processes 0 $childPid
done
if [ $isTopmost -eq 0 ]; then
kill -15 $curPid 2> /dev/null
fi
}
cleanup() {
info "cleaning up"
pid_zk_source=
pid_zk_target=
pid_kafka_source1=
pid_kafka_source2=
pid_kafka_source3=
pid_kafka_target1=
pid_kafka_target2=
pid_producer=
rm -rf /tmp/zookeeper_source
rm -rf /tmp/zookeeper_target
rm -rf /tmp/kafka-source{1..3}-logs
# mkdir -p /tmp/kafka-source{1..3}-logs/test0{1..3}-0
# touch /tmp/kafka-source{1..3}-logs/test0{1..3}-0/00000000000000000000.kafka
rm -rf /tmp/kafka-target{1..2}-logs
}
begin_timer() {
t_begin=$(date +%s)
}
end_timer() {
t_end=$(date +%s)
}
start_zk() {
info "starting zookeepers"
$base_dir/../../bin/zookeeper-server-start.sh $base_dir/config/zookeeper_source.properties 2>&1 > $base_dir/zookeeper_source.log &
pid_zk_source=$!
$base_dir/../../bin/zookeeper-server-start.sh $base_dir/config/zookeeper_target.properties 2>&1 > $base_dir/zookeeper_target.log &
pid_zk_target=$!
}
start_source_servers() {
info "starting source cluster"
$base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_source1.properties 2>&1 > $base_dir/kafka_source1.log &
pid_kafka_source1=$!
$base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_source2.properties 2>&1 > $base_dir/kafka_source2.log &
pid_kafka_source2=$!
$base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_source3.properties 2>&1 > $base_dir/kafka_source3.log &
pid_kafka_source3=$!
}
start_target_servers_for_whitelist_test() {
echo "starting mirror cluster"
$base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_target1.properties $base_dir/config/whitelisttest.consumer.properties $base_dir/config/mirror_producer.properties 2>&1 > $base_dir/kafka_target1.log &
pid_kafka_target1=$!
$base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_target2.properties $base_dir/config/whitelisttest.consumer.properties $base_dir/config/mirror_producer.properties 2>&1 > $base_dir/kafka_target2.log &
pid_kafka_target2=$!
}
start_target_servers_for_blacklist_test() {
echo "starting mirror cluster"
$base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_target1.properties $base_dir/config/blacklisttest.consumer.properties $base_dir/config/mirror_producer.properties 2>&1 > $base_dir/kafka_target1.log &
pid_kafka_target1=$!
$base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_target2.properties $base_dir/config/blacklisttest.consumer.properties $base_dir/config/mirror_producer.properties 2>&1 > $base_dir/kafka_target2.log &
pid_kafka_target2=$!
}
shutdown_servers() {
info "stopping producer"
if [ "x${pid_producer}" != "x" ]; then kill_child_processes 0 ${pid_producer}; fi
info "shutting down target servers"
if [ "x${pid_kafka_target1}" != "x" ]; then kill_child_processes 0 ${pid_kafka_target1}; fi
if [ "x${pid_kafka_target2}" != "x" ]; then kill_child_processes 0 ${pid_kafka_target2}; fi
sleep 2
info "shutting down source servers"
if [ "x${pid_kafka_source1}" != "x" ]; then kill_child_processes 0 ${pid_kafka_source1}; fi
if [ "x${pid_kafka_source2}" != "x" ]; then kill_child_processes 0 ${pid_kafka_source2}; fi
if [ "x${pid_kafka_source3}" != "x" ]; then kill_child_processes 0 ${pid_kafka_source3}; fi
info "shutting down zookeeper servers"
if [ "x${pid_zk_target}" != "x" ]; then kill_child_processes 0 ${pid_zk_target}; fi
if [ "x${pid_zk_source}" != "x" ]; then kill_child_processes 0 ${pid_zk_source}; fi
}
start_producer() {
topic=$1
info "start producing messages for topic $topic ..."
$base_dir/../../bin/kafka-run-class.sh kafka.tools.ProducerPerformance --brokerinfo zk.connect=localhost:2181 --topic $topic --messages $num_messages --message-size $message_size --batch-size 200 --vary-message-size --threads 1 --reporting-interval $num_messages --async 2>&1 > $base_dir/producer_performance.log &
pid_producer=$!
}
# In case the consumer does not consume, the test may exit prematurely (i.e.,
# shut down the kafka brokers, and ProducerPerformance will start throwing ugly
# exceptions. So, wait for the producer to finish before shutting down. If it
# takes too long, the user can just hit Ctrl-c which is trapped to kill child
# processes.
# Usage: wait_partition_done ([kafka-server] [topic] [partition-id])+
wait_partition_done() {
n_tuples=$(($# / 3))
i=1
while (($#)); do
kafka_server[i]=$1
topic[i]=$2
partitionid[i]=$3
prev_offset[i]=0
info "\twaiting for partition on server ${kafka_server[i]}, topic ${topic[i]}, partition ${partitionid[i]}"
i=$((i+1))
shift 3
done
all_done=0
# set -x
while [[ $all_done != 1 ]]; do
sleep 4
i=$n_tuples
all_done=1
for ((i=1; i <= $n_tuples; i++)); do
cur_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server ${kafka_server[i]} --topic ${topic[i]} --partition ${partitionid[i]} --time -1 --offsets 1 | tail -1)
if [ "x$cur_size" != "x${prev_offset[i]}" ]; then
all_done=0
prev_offset[i]=$cur_size
fi
done
done
}
cmp_logs() {
topic=$1
info "comparing source and target logs for topic $topic"
source_part0_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server kafka://localhost:9092 --topic $topic --partition 0 --time -1 --offsets 1 | tail -1)
source_part1_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server kafka://localhost:9091 --topic $topic --partition 0 --time -1 --offsets 1 | tail -1)
source_part2_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server kafka://localhost:9090 --topic $topic --partition 0 --time -1 --offsets 1 | tail -1)
target_part0_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server kafka://localhost:9093 --topic $topic --partition 0 --time -1 --offsets 1 | tail -1)
target_part1_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server kafka://localhost:9094 --topic $topic --partition 0 --time -1 --offsets 1 | tail -1)
if [ "x$target_part0_size" == "x" ]; then target_part0_size=0; fi
if [ "x$target_part1_size" == "x" ]; then target_part1_size=0; fi
expected_size=$(($source_part0_size + $source_part1_size + $source_part2_size))
actual_size=$(($target_part0_size + $target_part1_size))
if [ "x$expected_size" != "x$actual_size" ]
then
info "source size: $expected_size target size: $actual_size"
return 1
else
return 0
fi
}
take_fail_snapshot() {
snapshot_dir="$base_dir/failed-${snapshot_prefix}-${test_start_time}"
mkdir $snapshot_dir
for dir in /tmp/zookeeper_source /tmp/zookeeper_target /tmp/kafka-source{1..3}-logs /tmp/kafka-target{1..2}-logs; do
if [ -d $dir ]; then
cp -r $dir $snapshot_dir
fi
done
}
# Usage: process_test_result <result> <action_on_fail>
# result: last test result
# action_on_fail: (exit|wait|proceed)
# ("wait" is useful if you want to troubleshoot using zookeeper)
process_test_result() {
result=$1
if [ $1 -eq 0 ]; then
info "test passed"
else
info "test failed"
case "$2" in
"wait") info "waiting: hit Ctrl-c to quit"
wait
;;
"exit") shutdown_servers
take_fail_snapshot
exit $result
;;
*) shutdown_servers
take_fail_snapshot
info "proceeding"
;;
esac
fi
}
test_whitelists() {
info "### Testing whitelists"
snapshot_prefix="whitelist-test"
cleanup
start_zk
start_source_servers
start_target_servers_for_whitelist_test
sleep 4
begin_timer
start_producer test01
info "waiting for producer to finish producing ..."
wait_partition_done kafka://localhost:9090 test01 0 kafka://localhost:9091 test01 0 kafka://localhost:9092 test01 0
info "waiting for consumer to finish consuming ..."
wait_partition_done kafka://localhost:9093 test01 0 kafka://localhost:9094 test01 0
end_timer
info "embedded consumer took $((t_end - t_begin)) seconds"
sleep 2
cmp_logs test01
result=$?
return $result
}
test_blacklists() {
info "### Testing blacklists"
snapshot_prefix="blacklist-test"
cleanup
start_zk
start_source_servers
start_target_servers_for_blacklist_test
sleep 4
start_producer test02
info "waiting for producer to finish producing test02 ..."
wait_partition_done kafka://localhost:9090 test02 0 kafka://localhost:9091 test02 0 kafka://localhost:9092 test02 0
# start_producer test03
# info "waiting for producer to finish producing test03 ..."
# wait_partition_done kafka://localhost:9090 test03 0 kafka://localhost:9091 test03 0 kafka://localhost:9092 test03 0
begin_timer
start_producer test01
info "waiting for producer to finish producing ..."
wait_partition_done kafka://localhost:9090 test01 0 kafka://localhost:9091 test01 0 kafka://localhost:9092 test01 0
info "waiting for consumer to finish consuming ..."
wait_partition_done kafka://localhost:9093 test01 0 kafka://localhost:9094 test01 0
end_timer
info "embedded consumer took $((t_end - t_begin)) seconds"
sleep 2
cmp_logs test02
result1=$?
# cmp_logs test03
# result2=$?
# if [[ "x$result1" == "x0" || "x$result2" == "x0" ]]; then
if [[ "x$result1" == "x0" ]]; then
result=1
else
cmp_logs test01
result=$?
fi
return $result
}
# main test begins
echo "Test-$test_start_time"
# Ctrl-c trap. Catches INT signal
trap "shutdown_servers; exit 0" INT
test_whitelists
result=$?
process_test_result $result $action_on_fail
shutdown_servers
sleep 2
test_blacklists
result=$?
process_test_result $result $action_on_fail
shutdown_servers
exit $result

View File

@ -1,11 +0,0 @@
start the servers ...
start producing messages ...
Total Num Messages: 10000000 bytes: 1994374785 in 106.076 secs
Messages/sec: 94272.0314
MB/sec: 17.9304
[2011-05-02 11:50:29,022] INFO Disconnecting from localhost:9092 (kafka.producer.SyncProducer)
wait for consumer to finish consuming ...
test passed
bin/../../../bin/kafka-server-start.sh: line 11: 359 Terminated $(dirname $0)/kafka-run-class.sh kafka.Kafka $@
bin/../../../bin/zookeeper-server-start.sh: line 9: 357 Terminated $(dirname $0)/kafka-run-class.sh org.apache.zookeeper.server.quorum.QuorumPeerMain $@
bin/../../../bin/zookeeper-server-start.sh: line 9: 358 Terminated $(dirname $0)/kafka-run-class.sh org.apache.zookeeper.server.quorum.QuorumPeerMain $@

View File

@ -0,0 +1,22 @@
This test replicates messages from two source kafka clusters into one target
kafka cluster using the mirror-maker tool. At the end, the messages produced
at the source brokers should match that at the target brokers.
To run this test, do
bin/run-test.sh
In the event of failure, by default the brokers and zookeepers remain running
to make it easier to debug the issue - hit Ctrl-C to shut them down. You can
change this behavior by setting the action_on_fail flag in the script to "exit"
or "proceed", in which case a snapshot of all the logs and directories is
placed in the test's base directory.
It is a good idea to run the test in a loop. E.g.:
:>/tmp/mirrormaker_test.log
for i in {1..10}; do echo "run $i"; ./bin/run-test.sh 2>1 >> /tmp/mirrormaker_test.log; done
tail -F /tmp/mirrormaker_test.log
grep -ic passed /tmp/mirrormaker_test.log
grep -ic failed /tmp/mirrormaker_test.log

View File

@ -0,0 +1,357 @@
#!/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.
readonly num_messages=10000
readonly message_size=100
readonly action_on_fail="proceed"
# readonly action_on_fail="wait"
readonly test_start_time="$(date +%s)"
readonly base_dir=$(dirname $0)/..
info() {
echo -e "$(date +"%Y-%m-%d %H:%M:%S") $*"
}
kill_child_processes() {
isTopmost=$1
curPid=$2
childPids=$(ps a -o pid= -o ppid= | grep "${curPid}$" | awk '{print $1;}')
for childPid in $childPids
do
kill_child_processes 0 $childPid
done
if [ $isTopmost -eq 0 ]; then
kill -15 $curPid 2> /dev/null
fi
}
cleanup() {
info "cleaning up"
pid_zk_source1=
pid_zk_source2=
pid_zk_target=
pid_kafka_source_1_1=
pid_kafka_source_1_2=
pid_kafka_source_2_1=
pid_kafka_source_2_2=
pid_kafka_target_1_1=
pid_kafka_target_1_2=
pid_producer=
pid_mirrormaker_1=
pid_mirrormaker_2=
rm -rf /tmp/zookeeper*
rm -rf /tmp/kafka*
}
begin_timer() {
t_begin=$(date +%s)
}
end_timer() {
t_end=$(date +%s)
}
start_zk() {
info "starting zookeepers"
$base_dir/../../bin/zookeeper-server-start.sh $base_dir/config/zookeeper_source_1.properties 2>&1 > $base_dir/zookeeper_source-1.log &
pid_zk_source1=$!
$base_dir/../../bin/zookeeper-server-start.sh $base_dir/config/zookeeper_source_2.properties 2>&1 > $base_dir/zookeeper_source-2.log &
pid_zk_source2=$!
$base_dir/../../bin/zookeeper-server-start.sh $base_dir/config/zookeeper_target.properties 2>&1 > $base_dir/zookeeper_target.log &
pid_zk_target=$!
}
start_source_servers() {
info "starting source cluster"
JMX_PORT=1111 $base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_source_1_1.properties 2>&1 > $base_dir/kafka_source-1-1.log &
pid_kafka_source_1_1=$!
JMX_PORT=2222 $base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_source_1_2.properties 2>&1 > $base_dir/kafka_source-1-2.log &
pid_kafka_source_1_2=$!
JMX_PORT=3333 $base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_source_2_1.properties 2>&1 > $base_dir/kafka_source-2-1.log &
pid_kafka_source_2_1=$!
JMX_PORT=4444 $base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_source_2_2.properties 2>&1 > $base_dir/kafka_source-2-2.log &
pid_kafka_source_2_2=$!
}
start_target_servers() {
info "starting mirror cluster"
JMX_PORT=5555 $base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_target_1_1.properties 2>&1 > $base_dir/kafka_target-1-1.log &
pid_kafka_target_1_1=$!
JMX_PORT=6666 $base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_target_1_2.properties 2>&1 > $base_dir/kafka_target-1-2.log &
pid_kafka_target_1_2=$!
}
shutdown_servers() {
info "stopping mirror-maker"
if [ "x${pid_mirrormaker_1}" != "x" ]; then kill_child_processes 0 ${pid_mirrormaker_1}; fi
# sleep to avoid rebalancing during shutdown
sleep 2
if [ "x${pid_mirrormaker_2}" != "x" ]; then kill_child_processes 0 ${pid_mirrormaker_2}; fi
info "stopping producer"
if [ "x${pid_producer}" != "x" ]; then kill_child_processes 0 ${pid_producer}; fi
info "shutting down target servers"
if [ "x${pid_kafka_target_1_1}" != "x" ]; then kill_child_processes 0 ${pid_kafka_target_1_1}; fi
if [ "x${pid_kafka_target_1_2}" != "x" ]; then kill_child_processes 0 ${pid_kafka_target_1_2}; fi
sleep 2
info "shutting down source servers"
if [ "x${pid_kafka_source_1_1}" != "x" ]; then kill_child_processes 0 ${pid_kafka_source_1_1}; fi
if [ "x${pid_kafka_source_1_2}" != "x" ]; then kill_child_processes 0 ${pid_kafka_source_1_2}; fi
if [ "x${pid_kafka_source_2_1}" != "x" ]; then kill_child_processes 0 ${pid_kafka_source_2_1}; fi
if [ "x${pid_kafka_source_2_2}" != "x" ]; then kill_child_processes 0 ${pid_kafka_source_2_2}; fi
info "shutting down zookeeper servers"
if [ "x${pid_zk_target}" != "x" ]; then kill_child_processes 0 ${pid_zk_target}; fi
if [ "x${pid_zk_source1}" != "x" ]; then kill_child_processes 0 ${pid_zk_source1}; fi
if [ "x${pid_zk_source2}" != "x" ]; then kill_child_processes 0 ${pid_zk_source2}; fi
}
start_producer() {
topic=$1
zk=$2
info "start producing messages for topic $topic to zookeeper $zk ..."
$base_dir/../../bin/kafka-run-class.sh kafka.perf.ProducerPerformance --brokerinfo zk.connect=$zk --topic $topic --messages $num_messages --message-size $message_size --batch-size 200 --vary-message-size --threads 1 --reporting-interval $num_messages --async 2>&1 > $base_dir/producer_performance.log &
pid_producer=$!
}
# Usage: wait_partition_done ([kafka-server] [topic] [partition-id])+
wait_partition_done() {
n_tuples=$(($# / 3))
i=1
while (($#)); do
kafka_server[i]=$1
topic[i]=$2
partitionid[i]=$3
prev_offset[i]=0
info "\twaiting for partition on server ${kafka_server[i]}, topic ${topic[i]}, partition ${partitionid[i]}"
i=$((i+1))
shift 3
done
all_done=0
# set -x
while [[ $all_done != 1 ]]; do
sleep 4
i=$n_tuples
all_done=1
for ((i=1; i <= $n_tuples; i++)); do
cur_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server ${kafka_server[i]} --topic ${topic[i]} --partition ${partitionid[i]} --time -1 --offsets 1 | tail -1)
if [ "x$cur_size" != "x${prev_offset[i]}" ]; then
all_done=0
prev_offset[i]=$cur_size
fi
done
done
}
cmp_logs() {
topic=$1
info "comparing source and target logs for topic $topic"
source_part0_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server kafka://localhost:9090 --topic $topic --partition 0 --time -1 --offsets 1 | tail -1)
source_part1_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server kafka://localhost:9091 --topic $topic --partition 0 --time -1 --offsets 1 | tail -1)
source_part2_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server kafka://localhost:9092 --topic $topic --partition 0 --time -1 --offsets 1 | tail -1)
source_part3_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server kafka://localhost:9093 --topic $topic --partition 0 --time -1 --offsets 1 | tail -1)
target_part0_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server kafka://localhost:9094 --topic $topic --partition 0 --time -1 --offsets 1 | tail -1)
target_part1_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server kafka://localhost:9095 --topic $topic --partition 0 --time -1 --offsets 1 | tail -1)
if [ "x$source_part0_size" == "x" ]; then source_part0_size=0; fi
if [ "x$source_part1_size" == "x" ]; then source_part1_size=0; fi
if [ "x$source_part2_size" == "x" ]; then source_part2_size=0; fi
if [ "x$source_part3_size" == "x" ]; then source_part3_size=0; fi
if [ "x$target_part0_size" == "x" ]; then target_part0_size=0; fi
if [ "x$target_part1_size" == "x" ]; then target_part1_size=0; fi
expected_size=$(($source_part0_size + $source_part1_size + $source_part2_size + $source_part3_size))
actual_size=$(($target_part0_size + $target_part1_size))
if [ "x$expected_size" != "x$actual_size" ]
then
info "source size: $expected_size target size: $actual_size"
return 1
else
return 0
fi
}
take_fail_snapshot() {
snapshot_dir="$base_dir/failed-${snapshot_prefix}-${test_start_time}"
mkdir $snapshot_dir
for dir in /tmp/zookeeper_source{1..2} /tmp/zookeeper_target /tmp/kafka-source-{1..2}-{1..2}-logs /tmp/kafka-target{1..2}-logs; do
if [ -d $dir ]; then
cp -r $dir $snapshot_dir
fi
done
}
# Usage: process_test_result <result> <action_on_fail>
# result: last test result
# action_on_fail: (exit|wait|proceed)
# ("wait" is useful if you want to troubleshoot using zookeeper)
process_test_result() {
result=$1
if [ $1 -eq 0 ]; then
info "test passed"
else
info "test failed"
case "$2" in
"wait") info "waiting: hit Ctrl-c to quit"
wait
;;
"exit") shutdown_servers
take_fail_snapshot
exit $result
;;
*) shutdown_servers
take_fail_snapshot
info "proceeding"
;;
esac
fi
}
test_whitelists() {
info "### Testing whitelists"
snapshot_prefix="whitelist-test"
cleanup
start_zk
start_source_servers
start_target_servers
sleep 4
info "starting mirror makers"
JMX_PORT=7777 $base_dir/../../bin/kafka-run-class.sh kafka.tools.MirrorMaker --consumer.config $base_dir/config/whitelisttest_1.consumer.properties --consumer.config $base_dir/config/whitelisttest_2.consumer.properties --producer.config $base_dir/config/mirror_producer.properties --whitelist="white.*" --num.streams 2 2>&1 > $base_dir/kafka_mirrormaker_1.log &
pid_mirrormaker_1=$!
JMX_PORT=8888 $base_dir/../../bin/kafka-run-class.sh kafka.tools.MirrorMaker --consumer.config $base_dir/config/whitelisttest_1.consumer.properties --consumer.config $base_dir/config/whitelisttest_2.consumer.properties --producer.config $base_dir/config/mirror_producer.properties --whitelist="white.*" --num.streams 2 2>&1 > $base_dir/kafka_mirrormaker_2.log &
pid_mirrormaker_2=$!
begin_timer
start_producer whitetopic01 localhost:2181
start_producer whitetopic01 localhost:2182
info "waiting for whitetopic01 producers to finish producing ..."
wait_partition_done kafka://localhost:9090 whitetopic01 0 kafka://localhost:9091 whitetopic01 0 kafka://localhost:9092 whitetopic01 0 kafka://localhost:9093 whitetopic01 0
start_producer whitetopic02 localhost:2181
start_producer whitetopic03 localhost:2181
start_producer whitetopic04 localhost:2182
info "waiting for whitetopic02,whitetopic03,whitetopic04 producers to finish producing ..."
wait_partition_done kafka://localhost:9090 whitetopic02 0 kafka://localhost:9091 whitetopic02 0 kafka://localhost:9090 whitetopic03 0 kafka://localhost:9091 whitetopic03 0 kafka://localhost:9092 whitetopic04 0 kafka://localhost:9093 whitetopic04 0
start_producer blacktopic01 localhost:2182
info "waiting for blacktopic01 producer to finish producing ..."
wait_partition_done kafka://localhost:9092 blacktopic01 0 kafka://localhost:9093 blacktopic01 0
info "waiting for consumer to finish consuming ..."
wait_partition_done kafka://localhost:9094 whitetopic01 0 kafka://localhost:9095 whitetopic01 0 kafka://localhost:9094 whitetopic02 0 kafka://localhost:9095 whitetopic02 0 kafka://localhost:9094 whitetopic03 0 kafka://localhost:9095 whitetopic03 0 kafka://localhost:9094 whitetopic04 0 kafka://localhost:9095 whitetopic04 0
end_timer
info "embedded consumer took $((t_end - t_begin)) seconds"
sleep 2
# if [[ -d /tmp/kafka-target-1-1-logs/blacktopic01 || /tmp/kafka-target-1-2-logs/blacktopic01 ]]; then
# echo "blacktopic01 found on target cluster"
# result=1
# else
# cmp_logs whitetopic01 && cmp_logs whitetopic02 && cmp_logs whitetopic03 && cmp_logs whitetopic04
# result=$?
# fi
cmp_logs blacktopic01
cmp_logs whitetopic01 && cmp_logs whitetopic02 && cmp_logs whitetopic03 && cmp_logs whitetopic04
result=$?
return $result
}
test_blacklists() {
info "### Testing blacklists"
snapshot_prefix="blacklist-test"
cleanup
start_zk
start_source_servers
start_target_servers
sleep 4
info "starting mirror maker"
$base_dir/../../bin/kafka-run-class.sh kafka.tools.MirrorMaker --consumer.config $base_dir/config/blacklisttest.consumer.properties --producer.config $base_dir/config/mirror_producer.properties --blacklist="black.*" --num.streams 2 2>&1 > $base_dir/kafka_mirrormaker_1.log &
pid_mirrormaker_1=$!
start_producer blacktopic01 localhost:2181
start_producer blacktopic02 localhost:2181
info "waiting for producer to finish producing blacktopic01,blacktopic02 ..."
wait_partition_done kafka://localhost:9090 blacktopic01 0 kafka://localhost:9091 blacktopic01 0 kafka://localhost:9090 blacktopic02 0 kafka://localhost:9091 blacktopic02 0
begin_timer
start_producer whitetopic01 localhost:2181
info "waiting for producer to finish producing whitetopic01 ..."
wait_partition_done kafka://localhost:9090 whitetopic01 0 kafka://localhost:9091 whitetopic01 0
info "waiting for consumer to finish consuming ..."
wait_partition_done kafka://localhost:9094 whitetopic01 0 kafka://localhost:9095 whitetopic01 0
end_timer
info "embedded consumer took $((t_end - t_begin)) seconds"
sleep 2
cmp_logs blacktopic01 || cmp_logs blacktopic02
if [ $? -eq 0 ]; then
return 1
fi
cmp_logs whitetopic01
return $?
}
# main test begins
echo "Test-$test_start_time"
# Ctrl-c trap. Catches INT signal
trap "shutdown_servers; exit 0" INT
test_whitelists
result=$?
process_test_result $result $action_on_fail
shutdown_servers
sleep 2
test_blacklists
result=$?
process_test_result $result $action_on_fail
shutdown_servers
exit $result

View File

@ -24,6 +24,5 @@ zk.connectiontimeout.ms=1000000
#consumer group id #consumer group id
groupid=group1 groupid=group1
shallowiterator.enable=true
mirror.topics.blacklist=test02,test03

View File

@ -15,13 +15,16 @@
# zk connection string # zk connection string
# comma separated host:port pairs, each corresponding to a zk # comma separated host:port pairs, each corresponding to a zk
# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" # server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002"
zk.connect=localhost:2182 zk.connect=localhost:2183
# broker.list=1:localhost:9094,2:localhost:9095
# timeout in ms for connecting to zookeeper # timeout in ms for connecting to zookeeper
zk.connectiontimeout.ms=1000000 # zk.connectiontimeout.ms=1000000
producer.type=async producer.type=async
# to avoid dropping events if the queue is full, wait indefinitely # to avoid dropping events if the queue is full, wait indefinitely
queue.enqueueTimeout.ms=-1 queue.enqueueTimeout.ms=-1
num.producers.per.broker=2

View File

@ -26,13 +26,13 @@ brokerid=1
num.partitions=1 num.partitions=1
# the port the socket server runs on # the port the socket server runs on
port=9092 port=9090
# the number of processor threads the socket server uses. Defaults to the number of cores on the machine # the number of processor threads the socket server uses. Defaults to the number of cores on the machine
num.threads=8 num.threads=8
# the directory in which to store log files # the directory in which to store log files
log.dir=/tmp/kafka-source1-logs log.dir=/tmp/kafka-source-1-1-logs
# the send buffer used by the socket server # the send buffer used by the socket server
socket.send.buffer=1048576 socket.send.buffer=1048576

View File

@ -32,7 +32,7 @@ port=9091
num.threads=8 num.threads=8
# the directory in which to store log files # the directory in which to store log files
log.dir=/tmp/kafka-source2-logs log.dir=/tmp/kafka-source-1-2-logs
# the send buffer used by the socket server # the send buffer used by the socket server
socket.send.buffer=1048576 socket.send.buffer=1048576

View File

@ -15,7 +15,7 @@
# see kafka.server.KafkaConfig for additional details and defaults # see kafka.server.KafkaConfig for additional details and defaults
# the id of the broker # the id of the broker
brokerid=3 brokerid=1
# hostname of broker. If not set, will pick up from the value returned # hostname of broker. If not set, will pick up from the value returned
# from getLocalHost. If there are multiple interfaces getLocalHost # from getLocalHost. If there are multiple interfaces getLocalHost
@ -26,13 +26,13 @@ brokerid=3
num.partitions=1 num.partitions=1
# the port the socket server runs on # the port the socket server runs on
port=9090 port=9092
# the number of processor threads the socket server uses. Defaults to the number of cores on the machine # the number of processor threads the socket server uses. Defaults to the number of cores on the machine
num.threads=8 num.threads=8
# the directory in which to store log files # the directory in which to store log files
log.dir=/tmp/kafka-source3-logs log.dir=/tmp/kafka-source-2-1-logs
# the send buffer used by the socket server # the send buffer used by the socket server
socket.send.buffer=1048576 socket.send.buffer=1048576
@ -60,7 +60,7 @@ enable.zookeeper=true
# zk connection string # zk connection string
# comma separated host:port pairs, each corresponding to a zk # comma separated host:port pairs, each corresponding to a zk
# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" # server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002"
zk.connect=localhost:2181 zk.connect=localhost:2182
# timeout in ms for connecting to zookeeper # timeout in ms for connecting to zookeeper
zk.connectiontimeout.ms=1000000 zk.connectiontimeout.ms=1000000

View File

@ -0,0 +1,76 @@
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
# see kafka.server.KafkaConfig for additional details and defaults
# the id of the broker
brokerid=2
# hostname of broker. If not set, will pick up from the value returned
# from getLocalHost. If there are multiple interfaces getLocalHost
# may not be what you want.
# hostname=
# number of logical partitions on this broker
num.partitions=1
# the port the socket server runs on
port=9093
# the number of processor threads the socket server uses. Defaults to the number of cores on the machine
num.threads=8
# the directory in which to store log files
log.dir=/tmp/kafka-source-2-2-logs
# the send buffer used by the socket server
socket.send.buffer=1048576
# the receive buffer used by the socket server
socket.receive.buffer=1048576
# the maximum size of a log segment
log.file.size=536870912
# the interval between running cleanup on the logs
log.cleanup.interval.mins=1
# the minimum age of a log file to eligible for deletion
log.retention.hours=168
#the number of messages to accept without flushing the log to disk
log.flush.interval=600
#set the following properties to use zookeeper
# enable connecting to zookeeper
enable.zookeeper=true
# zk connection string
# comma separated host:port pairs, each corresponding to a zk
# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002"
zk.connect=localhost:2182
# timeout in ms for connecting to zookeeper
zk.connectiontimeout.ms=1000000
# time based topic flush intervals in ms
#topic.flush.intervals.ms=topic:1000
# default time based flush interval in ms
log.default.flush.interval.ms=1000
# time based topic flasher time rate in ms
log.default.flush.scheduler.interval.ms=1000

Some files were not shown because too many files have changed in this diff Show More