mirror of https://github.com/apache/kafka.git
KAFKA-829 Mirror maker needs to share the migration tool request channel; reviewed by Jun Rao
This commit is contained in:
parent
be3ce14721
commit
66b1038957
|
|
@ -31,7 +31,7 @@ producer.type=sync
|
|||
compression.codec=none
|
||||
|
||||
# message encoder
|
||||
serializer.class=kafka.serializer.StringEncoder
|
||||
serializer.class=kafka.serializer.DefaultEncoder
|
||||
|
||||
# allow topic level compression
|
||||
#compressed.topics=
|
||||
|
|
|
|||
|
|
@ -26,7 +26,7 @@ class Producer[K,V](private val underlying: kafka.producer.Producer[K,V]) // for
|
|||
/**
|
||||
* Sends the data to a single topic, partitioned by key, using either the
|
||||
* synchronous or the asynchronous producer
|
||||
* @param producerData the producer data object that encapsulates the topic, key and message data
|
||||
* @param message the producer data object that encapsulates the topic, key and message data
|
||||
*/
|
||||
def send(message: KeyedMessage[K,V]) {
|
||||
underlying.send(message)
|
||||
|
|
@ -34,7 +34,7 @@ class Producer[K,V](private val underlying: kafka.producer.Producer[K,V]) // for
|
|||
|
||||
/**
|
||||
* Use this API to send data to multiple topics
|
||||
* @param producerData list of producer data objects that encapsulate the topic, key and message data
|
||||
* @param messages list of producer data objects that encapsulate the topic, key and message data
|
||||
*/
|
||||
def send(messages: java.util.List[KeyedMessage[K,V]]) {
|
||||
import collection.JavaConversions._
|
||||
|
|
|
|||
|
|
@ -220,7 +220,7 @@ public class KafkaMigrationTool {
|
|||
kafkaProducerProperties_08.setProperty("serializer.class", "kafka.serializer.DefaultEncoder");
|
||||
// create a producer channel instead
|
||||
int queueSize = options.valueOf(queueSizeOpt);
|
||||
ProducerDataChannel<KeyedMessage<String, byte[]>> producerDataChannel = new ProducerDataChannel<KeyedMessage<String, byte[]>>(queueSize);
|
||||
ProducerDataChannel<KeyedMessage<byte[], byte[]>> producerDataChannel = new ProducerDataChannel<KeyedMessage<byte[], byte[]>>(queueSize);
|
||||
int threadId = 0;
|
||||
|
||||
Runtime.getRuntime().addShutdownHook(new Thread() {
|
||||
|
|
@ -279,8 +279,7 @@ public class KafkaMigrationTool {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
private static class ProducerDataChannel<T> {
|
||||
static class ProducerDataChannel<T> {
|
||||
private final int producerQueueSize;
|
||||
private final BlockingQueue<T> producerRequestQueue;
|
||||
|
||||
|
|
@ -300,14 +299,14 @@ public class KafkaMigrationTool {
|
|||
|
||||
private static class MigrationThread extends Thread {
|
||||
private final Object stream;
|
||||
private final ProducerDataChannel<KeyedMessage<String, byte[]>> producerDataChannel;
|
||||
private final ProducerDataChannel<KeyedMessage<byte[], byte[]>> producerDataChannel;
|
||||
private final int threadId;
|
||||
private final String threadName;
|
||||
private final org.apache.log4j.Logger logger;
|
||||
private CountDownLatch shutdownComplete = new CountDownLatch(1);
|
||||
private final AtomicBoolean isRunning = new AtomicBoolean(true);
|
||||
|
||||
MigrationThread(Object _stream, ProducerDataChannel<KeyedMessage<String, byte[]>> _producerDataChannel, int _threadId) {
|
||||
MigrationThread(Object _stream, ProducerDataChannel<KeyedMessage<byte[], byte[]>> _producerDataChannel, int _threadId) {
|
||||
stream = _stream;
|
||||
producerDataChannel = _producerDataChannel;
|
||||
threadId = _threadId;
|
||||
|
|
@ -336,7 +335,7 @@ public class KafkaMigrationTool {
|
|||
((ByteBuffer)payload_07).get(bytes);
|
||||
if(logger.isDebugEnabled())
|
||||
logger.debug("Migration thread " + threadId + " sending message of size " + bytes.length + " to topic "+ topic);
|
||||
KeyedMessage<String, byte[]> producerData = new KeyedMessage((String)topic, null, bytes);
|
||||
KeyedMessage<byte[], byte[]> producerData = new KeyedMessage((String)topic, null, bytes);
|
||||
producerDataChannel.sendRequest(producerData);
|
||||
}
|
||||
logger.info("Migration thread " + threadName + " finished running");
|
||||
|
|
@ -362,17 +361,17 @@ public class KafkaMigrationTool {
|
|||
}
|
||||
}
|
||||
|
||||
private static class ProducerThread extends Thread {
|
||||
private final ProducerDataChannel<KeyedMessage<String, byte[]>> producerDataChannel;
|
||||
private final Producer<String, byte[]> producer;
|
||||
static class ProducerThread extends Thread {
|
||||
private final ProducerDataChannel<KeyedMessage<byte[], byte[]>> producerDataChannel;
|
||||
private final Producer<byte[], byte[]> producer;
|
||||
private final int threadId;
|
||||
private String threadName;
|
||||
private org.apache.log4j.Logger logger;
|
||||
private CountDownLatch shutdownComplete = new CountDownLatch(1);
|
||||
private KeyedMessage<String, byte[]> shutdownMessage = new KeyedMessage("shutdown", null, null);
|
||||
private KeyedMessage<byte[], byte[]> shutdownMessage = new KeyedMessage("shutdown", null, null);
|
||||
|
||||
public ProducerThread(ProducerDataChannel<KeyedMessage<String, byte[]>> _producerDataChannel,
|
||||
Producer<String, byte[]> _producer,
|
||||
public ProducerThread(ProducerDataChannel<KeyedMessage<byte[], byte[]>> _producerDataChannel,
|
||||
Producer<byte[], byte[]> _producer,
|
||||
int _threadId) {
|
||||
producerDataChannel = _producerDataChannel;
|
||||
producer = _producer;
|
||||
|
|
@ -385,7 +384,7 @@ public class KafkaMigrationTool {
|
|||
public void run() {
|
||||
try{
|
||||
while(true) {
|
||||
KeyedMessage<String, byte[]> data = producerDataChannel.receiveRequest();
|
||||
KeyedMessage<byte[], byte[]> data = producerDataChannel.receiveRequest();
|
||||
if(!data.equals(shutdownMessage))
|
||||
producer.send(data);
|
||||
else
|
||||
|
|
|
|||
|
|
@ -24,6 +24,9 @@ import scala.collection.JavaConversions._
|
|||
import java.util.concurrent.CountDownLatch
|
||||
import kafka.consumer._
|
||||
import kafka.serializer._
|
||||
import collection.mutable.ListBuffer
|
||||
import kafka.tools.KafkaMigrationTool.{ProducerThread, ProducerDataChannel}
|
||||
import kafka.javaapi
|
||||
|
||||
|
||||
object MirrorMaker extends Logging {
|
||||
|
|
@ -60,6 +63,12 @@ object MirrorMaker extends Logging {
|
|||
.ofType(classOf[java.lang.Integer])
|
||||
.defaultsTo(1)
|
||||
|
||||
val bufferSizeOpt = parser.accepts("queue.size", "Number of messages that are buffered between the consumer and producer")
|
||||
.withRequiredArg()
|
||||
.describedAs("Queue size in terms of number of messages")
|
||||
.ofType(classOf[java.lang.Integer])
|
||||
.defaultsTo(10000);
|
||||
|
||||
val whitelistOpt = parser.accepts("whitelist",
|
||||
"Whitelist of topics to mirror.")
|
||||
.withRequiredArg()
|
||||
|
|
@ -88,6 +97,7 @@ object MirrorMaker extends Logging {
|
|||
}
|
||||
|
||||
val numStreams = options.valueOf(numStreamsOpt)
|
||||
val bufferSize = options.valueOf(bufferSizeOpt).intValue()
|
||||
|
||||
val producers = (1 to options.valueOf(numProducersOpt).intValue()).map(_ => {
|
||||
val config = new ProducerConfig(
|
||||
|
|
@ -95,18 +105,10 @@ object MirrorMaker extends Logging {
|
|||
new Producer[Array[Byte], Array[Byte]](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
|
||||
|
|
@ -115,32 +117,51 @@ object MirrorMaker extends Logging {
|
|||
val streams =
|
||||
connectors.map(_.createMessageStreamsByFilter(filterSpec, numStreams.intValue(), new DefaultDecoder(), new DefaultDecoder()))
|
||||
|
||||
streams.flatten.zipWithIndex.map(streamAndIndex => new MirrorMakerThread(streamAndIndex._1, producers, streamAndIndex._2))
|
||||
val producerDataChannel = new ProducerDataChannel[KeyedMessage[Array[Byte], Array[Byte]]](bufferSize);
|
||||
|
||||
val consumerThreads =
|
||||
streams.flatten.zipWithIndex.map(streamAndIndex => new MirrorMakerThread(streamAndIndex._1, producerDataChannel, streamAndIndex._2))
|
||||
|
||||
val producerThreads = new ListBuffer[ProducerThread]()
|
||||
|
||||
Runtime.getRuntime.addShutdownHook(new Thread() {
|
||||
override def run() {
|
||||
connectors.foreach(_.shutdown)
|
||||
consumerThreads.foreach(_.awaitShutdown)
|
||||
producerThreads.foreach(_.shutdown)
|
||||
producerThreads.foreach(_.awaitShutdown)
|
||||
logger.info("Kafka migration tool shutdown successfully");
|
||||
}
|
||||
})
|
||||
|
||||
// create producer threads
|
||||
var i: Int = 1
|
||||
for(producer <- producers) {
|
||||
val producerThread: KafkaMigrationTool.ProducerThread = new KafkaMigrationTool.ProducerThread(producerDataChannel,
|
||||
new javaapi.producer.Producer[Array[Byte], Array[Byte]](producer), i)
|
||||
producerThreads += producerThread
|
||||
i += 1
|
||||
}
|
||||
|
||||
threads.foreach(_.start())
|
||||
|
||||
threads.foreach(_.awaitShutdown())
|
||||
consumerThreads.foreach(_.start)
|
||||
producerThreads.foreach(_.start)
|
||||
}
|
||||
|
||||
class MirrorMakerThread(stream: KafkaStream[Array[Byte], Array[Byte]],
|
||||
producers: Seq[Producer[Array[Byte], Array[Byte]]],
|
||||
producerDataChannel: ProducerDataChannel[KeyedMessage[Array[Byte], Array[Byte]]],
|
||||
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 KeyedMessage[Array[Byte], Array[Byte]](
|
||||
msgAndMetadata.topic, msgAndMetadata.message)
|
||||
producer.send(pd)
|
||||
val pd = new KeyedMessage[Array[Byte], Array[Byte]](msgAndMetadata.topic, msgAndMetadata.message)
|
||||
producerDataChannel.sendRequest(pd)
|
||||
}
|
||||
} catch {
|
||||
case e =>
|
||||
|
|
@ -155,9 +176,7 @@ object MirrorMaker extends Logging {
|
|||
try {
|
||||
shutdownLatch.await()
|
||||
} catch {
|
||||
case e: InterruptedException => fatal(
|
||||
"Shutdown of thread %s interrupted. This might leak data!"
|
||||
.format(threadName))
|
||||
case e: InterruptedException => fatal("Shutdown of thread %s interrupted. This might leak data!".format(threadName))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
Loading…
Reference in New Issue