mirror of https://github.com/apache/kafka.git
KAFKA-3020; Ensure CheckStyle runs on all Java code
- Adds CheckStyle to core and examples modules - Fixes any existing CheckStyle issues Author: Grant Henke <granthenke@gmail.com> Reviewers: Ewen Cheslack-Postava <ewen@confluent.io> Closes #703 from granthenke/checkstyle-core
This commit is contained in:
parent
a0d21407cb
commit
64b746bd8b
15
build.gradle
15
build.gradle
|
@ -259,6 +259,7 @@ project(':core') {
|
|||
println "Building project 'core' with Scala version $resolvedScalaVersion"
|
||||
|
||||
apply plugin: 'scala'
|
||||
apply plugin: 'checkstyle'
|
||||
archivesBaseName = "kafka_${baseScalaVersion}"
|
||||
|
||||
dependencies {
|
||||
|
@ -334,8 +335,6 @@ project(':core') {
|
|||
into 'site-docs'
|
||||
}
|
||||
|
||||
|
||||
|
||||
tasks.create(name: "releaseTarGz", dependsOn: configurations.archives.artifacts, type: Tar) {
|
||||
into "kafka_${baseScalaVersion}-${version}"
|
||||
compression = Compression.GZIP
|
||||
|
@ -390,15 +389,27 @@ project(':core') {
|
|||
}
|
||||
into "$buildDir/dependant-testlibs"
|
||||
}
|
||||
|
||||
checkstyle {
|
||||
configFile = new File(rootDir, "checkstyle/checkstyle.xml")
|
||||
configProperties = [importControlFile: "$rootDir/checkstyle/import-control-core.xml"]
|
||||
}
|
||||
test.dependsOn('checkstyleMain', 'checkstyleTest')
|
||||
}
|
||||
|
||||
project(':examples') {
|
||||
apply plugin: 'checkstyle'
|
||||
archivesBaseName = "kafka-examples"
|
||||
|
||||
dependencies {
|
||||
compile project(':core')
|
||||
}
|
||||
|
||||
checkstyle {
|
||||
configFile = new File(rootDir, "checkstyle/checkstyle.xml")
|
||||
configProperties = [importControlFile: "$rootDir/checkstyle/import-control-core.xml"]
|
||||
}
|
||||
test.dependsOn('checkstyleMain', 'checkstyleTest')
|
||||
}
|
||||
|
||||
project(':clients') {
|
||||
|
|
|
@ -0,0 +1,69 @@
|
|||
<!DOCTYPE import-control PUBLIC
|
||||
"-//Puppy Crawl//DTD Import Control 1.1//EN"
|
||||
"http://www.puppycrawl.com/dtds/import_control_1_1.dtd">
|
||||
<!--
|
||||
// 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.
|
||||
-->
|
||||
|
||||
<import-control pkg="kafka">
|
||||
|
||||
<!-- THINK HARD ABOUT THE LAYERING OF THE PROJECT BEFORE CHANGING THIS FILE -->
|
||||
|
||||
<!-- common library dependencies -->
|
||||
<allow pkg="java" />
|
||||
<allow pkg="scala" />
|
||||
<allow pkg="javax.management" />
|
||||
<allow pkg="org.slf4j" />
|
||||
<allow pkg="org.junit" />
|
||||
<allow pkg="org.easymock" />
|
||||
<allow pkg="java.security" />
|
||||
<allow pkg="javax.net.ssl" />
|
||||
<allow pkg="javax.security" />
|
||||
|
||||
<allow pkg="kafka.common" />
|
||||
<allow pkg="kafka.utils" />
|
||||
<allow pkg="kafka.serializer" />
|
||||
<allow pkg="org.apache.kafka.common" />
|
||||
|
||||
<subpackage name="javaapi">
|
||||
<subpackage name="consumer">
|
||||
<allow pkg="kafka.consumer" />
|
||||
</subpackage>
|
||||
|
||||
<subpackage name="message">
|
||||
<allow pkg="kafka.message" />
|
||||
</subpackage>
|
||||
|
||||
<subpackage name="producer">
|
||||
<allow pkg="kafka.producer" />
|
||||
</subpackage>
|
||||
</subpackage>
|
||||
|
||||
<subpackage name="tools">
|
||||
<allow pkg="kafka.javaapi" />
|
||||
<allow pkg="kafka.producer" />
|
||||
<allow pkg="kafka.consumer" />
|
||||
<allow pkg="joptsimple" />
|
||||
</subpackage>
|
||||
|
||||
<subpackage name="examples">
|
||||
<allow pkg="org.apache.kafka.clients" />
|
||||
<allow pkg="kafka.api" />
|
||||
<allow pkg="kafka.javaapi" />
|
||||
<allow pkg="kafka.message" />
|
||||
</subpackage>
|
||||
|
||||
</import-control>
|
|
@ -17,16 +17,15 @@
|
|||
|
||||
package kafka.javaapi.consumer;
|
||||
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import kafka.common.OffsetAndMetadata;
|
||||
import kafka.common.TopicAndPartition;
|
||||
import kafka.consumer.KafkaStream;
|
||||
import kafka.consumer.TopicFilter;
|
||||
import kafka.serializer.Decoder;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public interface ConsumerConnector {
|
||||
/**
|
||||
* Create a list of MessageStreams of type T for each topic.
|
||||
|
@ -65,6 +64,7 @@ public interface ConsumerConnector {
|
|||
* Commit the offsets of all broker partitions connected by this connector.
|
||||
*/
|
||||
public void commitOffsets();
|
||||
|
||||
public void commitOffsets(boolean retryOnFailure);
|
||||
|
||||
/**
|
||||
|
|
|
@ -17,10 +17,15 @@
|
|||
|
||||
package kafka.tools;
|
||||
|
||||
import joptsimple.*;
|
||||
import joptsimple.ArgumentAcceptingOptionSpec;
|
||||
import joptsimple.OptionParser;
|
||||
import joptsimple.OptionSet;
|
||||
import joptsimple.OptionSpec;
|
||||
import joptsimple.OptionSpecBuilder;
|
||||
import kafka.javaapi.producer.Producer;
|
||||
import kafka.producer.KeyedMessage;
|
||||
import kafka.producer.ProducerConfig;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
|
@ -38,8 +43,6 @@ import java.util.concurrent.ArrayBlockingQueue;
|
|||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
|
||||
|
||||
/**
|
||||
* This is a kafka 0.7 to 0.8 online migration tool used for migrating data from 0.7 to 0.8 cluster. Internally,
|
||||
|
@ -58,8 +61,7 @@ import org.apache.kafka.common.utils.Utils;
|
|||
* the "serializer.class" config is set to "kafka.serializer.DefaultEncoder" by the code.
|
||||
*/
|
||||
@SuppressWarnings({"unchecked", "rawtypes"})
|
||||
public class KafkaMigrationTool
|
||||
{
|
||||
public class KafkaMigrationTool {
|
||||
private static final org.apache.log4j.Logger log = org.apache.log4j.Logger.getLogger(KafkaMigrationTool.class.getName());
|
||||
private static final String KAFKA_07_STATIC_CONSUMER_CLASS_NAME = "kafka.consumer.Consumer";
|
||||
private static final String KAFKA_07_CONSUMER_CONFIG_CLASS_NAME = "kafka.consumer.ConsumerConfig";
|
||||
|
@ -72,16 +74,16 @@ public class KafkaMigrationTool
|
|||
private static final String KAFKA_07_TOPIC_FILTER_CLASS_NAME = "kafka.consumer.TopicFilter";
|
||||
private static final String KAFKA_07_BLACK_LIST_CLASS_NAME = "kafka.consumer.Blacklist";
|
||||
|
||||
private static Class<?> KafkaStaticConsumer_07 = null;
|
||||
private static Class<?> ConsumerConfig_07 = null;
|
||||
private static Class<?> ConsumerConnector_07 = null;
|
||||
private static Class<?> KafkaStream_07 = null;
|
||||
private static Class<?> TopicFilter_07 = null;
|
||||
private static Class<?> WhiteList_07 = null;
|
||||
private static Class<?> BlackList_07 = null;
|
||||
private static Class<?> KafkaConsumerIteratorClass_07 = null;
|
||||
private static Class<?> KafkaMessageAndMetatDataClass_07 = null;
|
||||
private static Class<?> KafkaMessageClass_07 = null;
|
||||
private static Class<?> kafkaStaticConsumer07 = null;
|
||||
private static Class<?> consumerConfig07 = null;
|
||||
private static Class<?> consumerConnector07 = null;
|
||||
private static Class<?> kafkaStream07 = null;
|
||||
private static Class<?> topicFilter07 = null;
|
||||
private static Class<?> whiteList07 = null;
|
||||
private static Class<?> blackList07 = null;
|
||||
private static Class<?> kafkaConsumerIteratorClass07 = null;
|
||||
private static Class<?> kafkaMessageAndMetaDataClass07 = null;
|
||||
private static Class<?> kafkaMessageClass07 = null;
|
||||
|
||||
public static void main(String[] args) throws InterruptedException, IOException {
|
||||
OptionParser parser = new OptionParser();
|
||||
|
@ -160,65 +162,65 @@ public class KafkaMigrationTool
|
|||
System.exit(1);
|
||||
}
|
||||
|
||||
String kafkaJarFile_07 = options.valueOf(kafka07JarOpt);
|
||||
String kafkaJarFile07 = options.valueOf(kafka07JarOpt);
|
||||
String zkClientJarFile = options.valueOf(zkClient01JarOpt);
|
||||
String consumerConfigFile_07 = options.valueOf(consumerConfigOpt);
|
||||
String consumerConfigFile07 = options.valueOf(consumerConfigOpt);
|
||||
int numConsumers = options.valueOf(numStreamsOpt);
|
||||
String producerConfigFile_08 = options.valueOf(producerConfigOpt);
|
||||
String producerConfigFile08 = options.valueOf(producerConfigOpt);
|
||||
int numProducers = options.valueOf(numProducersOpt);
|
||||
final List<MigrationThread> migrationThreads = new ArrayList<MigrationThread>(numConsumers);
|
||||
final List<ProducerThread> producerThreads = new ArrayList<ProducerThread>(numProducers);
|
||||
|
||||
try {
|
||||
File kafkaJar_07 = new File(kafkaJarFile_07);
|
||||
File kafkaJar07 = new File(kafkaJarFile07);
|
||||
File zkClientJar = new File(zkClientJarFile);
|
||||
ParentLastURLClassLoader c1 = new ParentLastURLClassLoader(new URL[]{
|
||||
kafkaJar_07.toURI().toURL(),
|
||||
kafkaJar07.toURI().toURL(),
|
||||
zkClientJar.toURI().toURL()
|
||||
});
|
||||
|
||||
/** Construct the 07 consumer config **/
|
||||
ConsumerConfig_07 = c1.loadClass(KAFKA_07_CONSUMER_CONFIG_CLASS_NAME);
|
||||
KafkaStaticConsumer_07 = c1.loadClass(KAFKA_07_STATIC_CONSUMER_CLASS_NAME);
|
||||
ConsumerConnector_07 = c1.loadClass(KAFKA_07_CONSUMER_CONNECTOR_CLASS_NAME);
|
||||
KafkaStream_07 = c1.loadClass(KAFKA_07_CONSUMER_STREAM_CLASS_NAME);
|
||||
TopicFilter_07 = c1.loadClass(KAFKA_07_TOPIC_FILTER_CLASS_NAME);
|
||||
WhiteList_07 = c1.loadClass(KAFKA_07_WHITE_LIST_CLASS_NAME);
|
||||
BlackList_07 = c1.loadClass(KAFKA_07_BLACK_LIST_CLASS_NAME);
|
||||
KafkaMessageClass_07 = c1.loadClass(KAFKA_07_MESSAGE_CLASS_NAME);
|
||||
KafkaConsumerIteratorClass_07 = c1.loadClass(KAFKA_07_CONSUMER_ITERATOR_CLASS_NAME);
|
||||
KafkaMessageAndMetatDataClass_07 = c1.loadClass(KAFKA_07_MESSAGE_AND_METADATA_CLASS_NAME);
|
||||
consumerConfig07 = c1.loadClass(KAFKA_07_CONSUMER_CONFIG_CLASS_NAME);
|
||||
kafkaStaticConsumer07 = c1.loadClass(KAFKA_07_STATIC_CONSUMER_CLASS_NAME);
|
||||
consumerConnector07 = c1.loadClass(KAFKA_07_CONSUMER_CONNECTOR_CLASS_NAME);
|
||||
kafkaStream07 = c1.loadClass(KAFKA_07_CONSUMER_STREAM_CLASS_NAME);
|
||||
topicFilter07 = c1.loadClass(KAFKA_07_TOPIC_FILTER_CLASS_NAME);
|
||||
whiteList07 = c1.loadClass(KAFKA_07_WHITE_LIST_CLASS_NAME);
|
||||
blackList07 = c1.loadClass(KAFKA_07_BLACK_LIST_CLASS_NAME);
|
||||
kafkaMessageClass07 = c1.loadClass(KAFKA_07_MESSAGE_CLASS_NAME);
|
||||
kafkaConsumerIteratorClass07 = c1.loadClass(KAFKA_07_CONSUMER_ITERATOR_CLASS_NAME);
|
||||
kafkaMessageAndMetaDataClass07 = c1.loadClass(KAFKA_07_MESSAGE_AND_METADATA_CLASS_NAME);
|
||||
|
||||
Constructor ConsumerConfigConstructor_07 = ConsumerConfig_07.getConstructor(Properties.class);
|
||||
Properties kafkaConsumerProperties_07 = new Properties();
|
||||
kafkaConsumerProperties_07.load(new FileInputStream(consumerConfigFile_07));
|
||||
Constructor consumerConfigConstructor07 = consumerConfig07.getConstructor(Properties.class);
|
||||
Properties kafkaConsumerProperties07 = new Properties();
|
||||
kafkaConsumerProperties07.load(new FileInputStream(consumerConfigFile07));
|
||||
/** Disable shallow iteration because the message format is different between 07 and 08, we have to get each individual message **/
|
||||
if(kafkaConsumerProperties_07.getProperty("shallow.iterator.enable", "").equals("true")) {
|
||||
if (kafkaConsumerProperties07.getProperty("shallow.iterator.enable", "").equals("true")) {
|
||||
log.warn("Shallow iterator should not be used in the migration tool");
|
||||
kafkaConsumerProperties_07.setProperty("shallow.iterator.enable", "false");
|
||||
kafkaConsumerProperties07.setProperty("shallow.iterator.enable", "false");
|
||||
}
|
||||
Object consumerConfig_07 = ConsumerConfigConstructor_07.newInstance(kafkaConsumerProperties_07);
|
||||
Object consumerConfig07 = consumerConfigConstructor07.newInstance(kafkaConsumerProperties07);
|
||||
|
||||
/** Construct the 07 consumer connector **/
|
||||
Method ConsumerConnectorCreationMethod_07 = KafkaStaticConsumer_07.getMethod("createJavaConsumerConnector", ConsumerConfig_07);
|
||||
final Object consumerConnector_07 = ConsumerConnectorCreationMethod_07.invoke(null, consumerConfig_07);
|
||||
Method ConsumerConnectorCreateMessageStreamsMethod_07 = ConsumerConnector_07.getMethod(
|
||||
Method consumerConnectorCreationMethod07 = kafkaStaticConsumer07.getMethod("createJavaConsumerConnector", KafkaMigrationTool.consumerConfig07);
|
||||
final Object consumerConnector07 = consumerConnectorCreationMethod07.invoke(null, consumerConfig07);
|
||||
Method consumerConnectorCreateMessageStreamsMethod07 = KafkaMigrationTool.consumerConnector07.getMethod(
|
||||
"createMessageStreamsByFilter",
|
||||
TopicFilter_07, int.class);
|
||||
final Method ConsumerConnectorShutdownMethod_07 = ConsumerConnector_07.getMethod("shutdown");
|
||||
Constructor WhiteListConstructor_07 = WhiteList_07.getConstructor(String.class);
|
||||
Constructor BlackListConstructor_07 = BlackList_07.getConstructor(String.class);
|
||||
topicFilter07, int.class);
|
||||
final Method consumerConnectorShutdownMethod07 = KafkaMigrationTool.consumerConnector07.getMethod("shutdown");
|
||||
Constructor whiteListConstructor07 = whiteList07.getConstructor(String.class);
|
||||
Constructor blackListConstructor07 = blackList07.getConstructor(String.class);
|
||||
Object filterSpec = null;
|
||||
if (options.has(whitelistOpt))
|
||||
filterSpec = WhiteListConstructor_07.newInstance(options.valueOf(whitelistOpt));
|
||||
filterSpec = whiteListConstructor07.newInstance(options.valueOf(whitelistOpt));
|
||||
else
|
||||
filterSpec = BlackListConstructor_07.newInstance(options.valueOf(blacklistOpt));
|
||||
filterSpec = blackListConstructor07.newInstance(options.valueOf(blacklistOpt));
|
||||
|
||||
Object retKafkaStreams = ConsumerConnectorCreateMessageStreamsMethod_07.invoke(consumerConnector_07, filterSpec, numConsumers);
|
||||
Object retKafkaStreams = consumerConnectorCreateMessageStreamsMethod07.invoke(consumerConnector07, filterSpec, numConsumers);
|
||||
|
||||
Properties kafkaProducerProperties_08 = new Properties();
|
||||
kafkaProducerProperties_08.load(new FileInputStream(producerConfigFile_08));
|
||||
kafkaProducerProperties_08.setProperty("serializer.class", "kafka.serializer.DefaultEncoder");
|
||||
Properties kafkaProducerProperties08 = new Properties();
|
||||
kafkaProducerProperties08.load(new FileInputStream(producerConfigFile08));
|
||||
kafkaProducerProperties08.setProperty("serializer.class", "kafka.serializer.DefaultEncoder");
|
||||
// create a producer channel instead
|
||||
int queueSize = options.valueOf(queueSizeOpt);
|
||||
ProducerDataChannel<KeyedMessage<byte[], byte[]>> producerDataChannel = new ProducerDataChannel<KeyedMessage<byte[], byte[]>>(queueSize);
|
||||
|
@ -228,7 +230,7 @@ public class KafkaMigrationTool
|
|||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
ConsumerConnectorShutdownMethod_07.invoke(consumerConnector_07);
|
||||
consumerConnectorShutdownMethod07.invoke(consumerConnector07);
|
||||
} catch (Exception e) {
|
||||
log.error("Error while shutting down Kafka consumer", e);
|
||||
}
|
||||
|
@ -253,18 +255,17 @@ public class KafkaMigrationTool
|
|||
migrationThreads.add(thread);
|
||||
}
|
||||
|
||||
String clientId = kafkaProducerProperties_08.getProperty("client.id");
|
||||
String clientId = kafkaProducerProperties08.getProperty("client.id");
|
||||
// start producer threads
|
||||
for (int i = 0; i < numProducers; i++) {
|
||||
kafkaProducerProperties_08.put("client.id", clientId + "-" + i);
|
||||
ProducerConfig producerConfig_08 = new ProducerConfig(kafkaProducerProperties_08);
|
||||
Producer producer = new Producer(producerConfig_08);
|
||||
kafkaProducerProperties08.put("client.id", clientId + "-" + i);
|
||||
ProducerConfig producerConfig08 = new ProducerConfig(kafkaProducerProperties08);
|
||||
Producer producer = new Producer(producerConfig08);
|
||||
ProducerThread producerThread = new ProducerThread(producerDataChannel, producer, i);
|
||||
producerThread.start();
|
||||
producerThreads.add(producerThread);
|
||||
}
|
||||
}
|
||||
catch (Throwable e){
|
||||
} catch (Throwable e) {
|
||||
System.out.println("Kafka migration tool failed due to: " + Utils.stackTrace(e));
|
||||
log.error("Kafka migration tool failed: ", e);
|
||||
}
|
||||
|
@ -307,10 +308,10 @@ public class KafkaMigrationTool
|
|||
private CountDownLatch shutdownComplete = new CountDownLatch(1);
|
||||
private final AtomicBoolean isRunning = new AtomicBoolean(true);
|
||||
|
||||
MigrationThread(Object _stream, ProducerDataChannel<KeyedMessage<byte[], byte[]>> _producerDataChannel, int _threadId) {
|
||||
stream = _stream;
|
||||
producerDataChannel = _producerDataChannel;
|
||||
threadId = _threadId;
|
||||
MigrationThread(Object stream, ProducerDataChannel<KeyedMessage<byte[], byte[]>> producerDataChannel, int threadId) {
|
||||
this.stream = stream;
|
||||
this.producerDataChannel = producerDataChannel;
|
||||
this.threadId = threadId;
|
||||
threadName = "MigrationThread-" + threadId;
|
||||
logger = org.apache.log4j.Logger.getLogger(MigrationThread.class.getName());
|
||||
this.setName(threadName);
|
||||
|
@ -318,22 +319,22 @@ public class KafkaMigrationTool
|
|||
|
||||
public void run() {
|
||||
try {
|
||||
Method MessageGetPayloadMethod_07 = KafkaMessageClass_07.getMethod("payload");
|
||||
Method KafkaGetMessageMethod_07 = KafkaMessageAndMetatDataClass_07.getMethod("message");
|
||||
Method KafkaGetTopicMethod_07 = KafkaMessageAndMetatDataClass_07.getMethod("topic");
|
||||
Method ConsumerIteratorMethod = KafkaStream_07.getMethod("iterator");
|
||||
Method KafkaStreamHasNextMethod_07 = KafkaConsumerIteratorClass_07.getMethod("hasNext");
|
||||
Method KafkaStreamNextMethod_07 = KafkaConsumerIteratorClass_07.getMethod("next");
|
||||
Object iterator = ConsumerIteratorMethod.invoke(stream);
|
||||
Method messageGetPayloadMethod07 = kafkaMessageClass07.getMethod("payload");
|
||||
Method kafkaGetMessageMethod07 = kafkaMessageAndMetaDataClass07.getMethod("message");
|
||||
Method kafkaGetTopicMethod07 = kafkaMessageAndMetaDataClass07.getMethod("topic");
|
||||
Method consumerIteratorMethod = kafkaStream07.getMethod("iterator");
|
||||
Method kafkaStreamHasNextMethod07 = kafkaConsumerIteratorClass07.getMethod("hasNext");
|
||||
Method kafkaStreamNextMethod07 = kafkaConsumerIteratorClass07.getMethod("next");
|
||||
Object iterator = consumerIteratorMethod.invoke(stream);
|
||||
|
||||
while (((Boolean) KafkaStreamHasNextMethod_07.invoke(iterator)).booleanValue()) {
|
||||
Object messageAndMetaData_07 = KafkaStreamNextMethod_07.invoke(iterator);
|
||||
Object message_07 = KafkaGetMessageMethod_07.invoke(messageAndMetaData_07);
|
||||
Object topic = KafkaGetTopicMethod_07.invoke(messageAndMetaData_07);
|
||||
Object payload_07 = MessageGetPayloadMethod_07.invoke(message_07);
|
||||
int size = ((ByteBuffer)payload_07).remaining();
|
||||
while (((Boolean) kafkaStreamHasNextMethod07.invoke(iterator)).booleanValue()) {
|
||||
Object messageAndMetaData07 = kafkaStreamNextMethod07.invoke(iterator);
|
||||
Object message07 = kafkaGetMessageMethod07.invoke(messageAndMetaData07);
|
||||
Object topic = kafkaGetTopicMethod07.invoke(messageAndMetaData07);
|
||||
Object payload07 = messageGetPayloadMethod07.invoke(message07);
|
||||
int size = ((ByteBuffer) payload07).remaining();
|
||||
byte[] bytes = new byte[size];
|
||||
((ByteBuffer)payload_07).get(bytes);
|
||||
((ByteBuffer) payload07).get(bytes);
|
||||
if (logger.isDebugEnabled())
|
||||
logger.debug("Migration thread " + threadId + " sending message of size " + bytes.length + " to topic " + topic);
|
||||
KeyedMessage<byte[], byte[]> producerData = new KeyedMessage((String) topic, null, bytes);
|
||||
|
@ -371,12 +372,12 @@ public class KafkaMigrationTool
|
|||
private CountDownLatch shutdownComplete = new CountDownLatch(1);
|
||||
private KeyedMessage<byte[], byte[]> shutdownMessage = new KeyedMessage("shutdown", null, null);
|
||||
|
||||
public ProducerThread(ProducerDataChannel<KeyedMessage<byte[], byte[]>> _producerDataChannel,
|
||||
Producer<byte[], byte[]> _producer,
|
||||
int _threadId) {
|
||||
producerDataChannel = _producerDataChannel;
|
||||
producer = _producer;
|
||||
threadId = _threadId;
|
||||
public ProducerThread(ProducerDataChannel<KeyedMessage<byte[], byte[]>> producerDataChannel,
|
||||
Producer<byte[], byte[]> producer,
|
||||
int threadId) {
|
||||
this.producerDataChannel = producerDataChannel;
|
||||
this.producer = producer;
|
||||
this.threadId = threadId;
|
||||
threadName = "ProducerThread-" + threadId;
|
||||
logger = org.apache.log4j.Logger.getLogger(ProducerThread.class.getName());
|
||||
this.setName(threadName);
|
||||
|
@ -388,9 +389,9 @@ public class KafkaMigrationTool
|
|||
KeyedMessage<byte[], byte[]> data = producerDataChannel.receiveRequest();
|
||||
if (!data.equals(shutdownMessage)) {
|
||||
producer.send(data);
|
||||
if(logger.isDebugEnabled()) logger.debug(String.format("Sending message %s", new String(data.message())));
|
||||
}
|
||||
else
|
||||
if (logger.isDebugEnabled())
|
||||
logger.debug(String.format("Sending message %s", new String(data.message())));
|
||||
} else
|
||||
break;
|
||||
}
|
||||
logger.info("Producer thread " + threadName + " finished running");
|
||||
|
@ -435,6 +436,7 @@ public class KafkaMigrationTool
|
|||
public FindClassClassLoader(ClassLoader parent) {
|
||||
super(parent);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Class<?> findClass(String name) throws ClassNotFoundException {
|
||||
return super.findClass(name);
|
||||
|
@ -447,6 +449,7 @@ public class KafkaMigrationTool
|
|||
*/
|
||||
private static class ChildURLClassLoader extends URLClassLoader {
|
||||
private FindClassClassLoader realParent;
|
||||
|
||||
public ChildURLClassLoader(URL[] urls, FindClassClassLoader realParent) {
|
||||
super(urls, null);
|
||||
this.realParent = realParent;
|
||||
|
@ -457,8 +460,7 @@ public class KafkaMigrationTool
|
|||
try {
|
||||
// first try to use the URLClassLoader findClass
|
||||
return super.findClass(name);
|
||||
}
|
||||
catch( ClassNotFoundException e ) {
|
||||
} catch (ClassNotFoundException e) {
|
||||
// if that fails, we ask our real parent class loader to load the class (we give up)
|
||||
return realParent.loadClass(name);
|
||||
}
|
||||
|
@ -475,8 +477,7 @@ public class KafkaMigrationTool
|
|||
try {
|
||||
// first we try to find a class inside the child class loader
|
||||
return childClassLoader.findClass(name);
|
||||
}
|
||||
catch( ClassNotFoundException e ) {
|
||||
} catch (ClassNotFoundException e) {
|
||||
// didn't find it, try the parent
|
||||
return super.loadClass(name, resolve);
|
||||
}
|
||||
|
|
|
@ -16,22 +16,20 @@
|
|||
*/
|
||||
package kafka.examples;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.Properties;
|
||||
|
||||
import kafka.utils.ShutdownableThread;
|
||||
import org.apache.kafka.clients.consumer.ConsumerConfig;
|
||||
import org.apache.kafka.clients.consumer.ConsumerRecord;
|
||||
import org.apache.kafka.clients.consumer.ConsumerRecords;
|
||||
import org.apache.kafka.clients.consumer.KafkaConsumer;
|
||||
|
||||
public class Consumer extends ShutdownableThread
|
||||
{
|
||||
import java.util.Collections;
|
||||
import java.util.Properties;
|
||||
|
||||
public class Consumer extends ShutdownableThread {
|
||||
private final KafkaConsumer<Integer, String> consumer;
|
||||
private final String topic;
|
||||
|
||||
public Consumer(String topic)
|
||||
{
|
||||
public Consumer(String topic) {
|
||||
super("KafkaConsumerExample", false);
|
||||
Properties props = new Properties();
|
||||
props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
|
||||
|
|
|
@ -16,15 +16,13 @@
|
|||
*/
|
||||
package kafka.examples;
|
||||
|
||||
public class KafkaConsumerProducerDemo implements KafkaProperties
|
||||
{
|
||||
public static void main(String[] args)
|
||||
{
|
||||
public class KafkaConsumerProducerDemo implements KafkaProperties {
|
||||
public static void main(String[] args) {
|
||||
final boolean isAsync = args.length > 0 ? !args[0].trim().toLowerCase().equals("sync") : true;
|
||||
Producer producerThread = new Producer(KafkaProperties.topic, isAsync);
|
||||
Producer producerThread = new Producer(KafkaProperties.TOPIC, isAsync);
|
||||
producerThread.start();
|
||||
|
||||
Consumer consumerThread = new Consumer(KafkaProperties.topic);
|
||||
Consumer consumerThread = new Consumer(KafkaProperties.TOPIC);
|
||||
consumerThread.start();
|
||||
|
||||
}
|
||||
|
|
|
@ -5,9 +5,9 @@
|
|||
* 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
|
||||
*
|
||||
* <p/>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* <p/>
|
||||
* 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.
|
||||
|
@ -16,17 +16,16 @@
|
|||
*/
|
||||
package kafka.examples;
|
||||
|
||||
public interface KafkaProperties
|
||||
{
|
||||
final static String zkConnect = "127.0.0.1:2181";
|
||||
final static String groupId = "group1";
|
||||
final static String topic = "topic1";
|
||||
final static String kafkaServerURL = "localhost";
|
||||
final static int kafkaServerPort = 9092;
|
||||
final static int kafkaProducerBufferSize = 64*1024;
|
||||
final static int connectionTimeOut = 100000;
|
||||
final static int reconnectInterval = 10000;
|
||||
final static String topic2 = "topic2";
|
||||
final static String topic3 = "topic3";
|
||||
final static String clientId = "SimpleConsumerDemoClient";
|
||||
public interface KafkaProperties {
|
||||
String ZK_CONNECT = "127.0.0.1:2181";
|
||||
String GROUP_ID = "group1";
|
||||
String TOPIC = "topic1";
|
||||
String KAFKA_SERVER_URL = "localhost";
|
||||
int KAFKA_SERVER_PORT = 9092;
|
||||
int KAFKA_PRODUCER_BUFFER_SIZE = 64 * 1024;
|
||||
int CONNECTION_TIMEOUT = 100000;
|
||||
int RECONNECT_INTERVAL = 10000;
|
||||
String TOPIC2 = "topic2";
|
||||
String TOPIC3 = "topic3";
|
||||
String CLIENT_ID = "SimpleConsumerDemoClient";
|
||||
}
|
||||
|
|
|
@ -16,23 +16,20 @@
|
|||
*/
|
||||
package kafka.examples;
|
||||
|
||||
|
||||
import java.util.Properties;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
|
||||
import org.apache.kafka.clients.producer.Callback;
|
||||
import org.apache.kafka.clients.producer.KafkaProducer;
|
||||
import org.apache.kafka.clients.producer.ProducerRecord;
|
||||
import org.apache.kafka.clients.producer.RecordMetadata;
|
||||
|
||||
public class Producer extends Thread
|
||||
{
|
||||
import java.util.Properties;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
|
||||
public class Producer extends Thread {
|
||||
private final KafkaProducer<Integer, String> producer;
|
||||
private final String topic;
|
||||
private final Boolean isAsync;
|
||||
|
||||
public Producer(String topic, Boolean isAsync)
|
||||
{
|
||||
public Producer(String topic, Boolean isAsync) {
|
||||
Properties props = new Properties();
|
||||
props.put("bootstrap.servers", "localhost:9092");
|
||||
props.put("client.id", "DemoProducer");
|
||||
|
@ -45,8 +42,7 @@ public class Producer extends Thread
|
|||
|
||||
public void run() {
|
||||
int messageNo = 1;
|
||||
while(true)
|
||||
{
|
||||
while (true) {
|
||||
String messageStr = "Message_" + messageNo;
|
||||
long startTime = System.currentTimeMillis();
|
||||
if (isAsync) { // Send asynchronously
|
||||
|
|
|
@ -19,15 +19,15 @@ package kafka.examples;
|
|||
import kafka.api.FetchRequest;
|
||||
import kafka.api.FetchRequestBuilder;
|
||||
import kafka.javaapi.FetchResponse;
|
||||
import kafka.javaapi.consumer.SimpleConsumer;
|
||||
import kafka.javaapi.message.ByteBufferMessageSet;
|
||||
import kafka.message.MessageAndOffset;
|
||||
|
||||
import java.io.UnsupportedEncodingException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import kafka.javaapi.consumer.SimpleConsumer;
|
||||
import kafka.javaapi.message.ByteBufferMessageSet;
|
||||
import kafka.message.MessageAndOffset;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class SimpleConsumerDemo {
|
||||
|
@ -42,9 +42,9 @@ public class SimpleConsumerDemo {
|
|||
}
|
||||
|
||||
private static void generateData() {
|
||||
Producer producer2 = new Producer(KafkaProperties.topic2, false);
|
||||
Producer producer2 = new Producer(KafkaProperties.TOPIC2, false);
|
||||
producer2.start();
|
||||
Producer producer3 = new Producer(KafkaProperties.topic3, false);
|
||||
Producer producer3 = new Producer(KafkaProperties.TOPIC3, false);
|
||||
producer3.start();
|
||||
try {
|
||||
Thread.sleep(1000);
|
||||
|
@ -56,28 +56,28 @@ public class SimpleConsumerDemo {
|
|||
public static void main(String[] args) throws Exception {
|
||||
generateData();
|
||||
|
||||
SimpleConsumer simpleConsumer = new SimpleConsumer(KafkaProperties.kafkaServerURL,
|
||||
KafkaProperties.kafkaServerPort,
|
||||
KafkaProperties.connectionTimeOut,
|
||||
KafkaProperties.kafkaProducerBufferSize,
|
||||
KafkaProperties.clientId);
|
||||
SimpleConsumer simpleConsumer = new SimpleConsumer(KafkaProperties.KAFKA_SERVER_URL,
|
||||
KafkaProperties.KAFKA_SERVER_PORT,
|
||||
KafkaProperties.CONNECTION_TIMEOUT,
|
||||
KafkaProperties.KAFKA_PRODUCER_BUFFER_SIZE,
|
||||
KafkaProperties.CLIENT_ID);
|
||||
|
||||
System.out.println("Testing single fetch");
|
||||
FetchRequest req = new FetchRequestBuilder()
|
||||
.clientId(KafkaProperties.clientId)
|
||||
.addFetch(KafkaProperties.topic2, 0, 0L, 100)
|
||||
.clientId(KafkaProperties.CLIENT_ID)
|
||||
.addFetch(KafkaProperties.TOPIC2, 0, 0L, 100)
|
||||
.build();
|
||||
FetchResponse fetchResponse = simpleConsumer.fetch(req);
|
||||
printMessages(fetchResponse.messageSet(KafkaProperties.topic2, 0));
|
||||
printMessages(fetchResponse.messageSet(KafkaProperties.TOPIC2, 0));
|
||||
|
||||
System.out.println("Testing single multi-fetch");
|
||||
Map<String, List<Integer>> topicMap = new HashMap<String, List<Integer>>();
|
||||
topicMap.put(KafkaProperties.topic2, Collections.singletonList(0));
|
||||
topicMap.put(KafkaProperties.topic3, Collections.singletonList(0));
|
||||
topicMap.put(KafkaProperties.TOPIC2, Collections.singletonList(0));
|
||||
topicMap.put(KafkaProperties.TOPIC3, Collections.singletonList(0));
|
||||
req = new FetchRequestBuilder()
|
||||
.clientId(KafkaProperties.clientId)
|
||||
.addFetch(KafkaProperties.topic2, 0, 0L, 100)
|
||||
.addFetch(KafkaProperties.topic3, 0, 0L, 100)
|
||||
.clientId(KafkaProperties.CLIENT_ID)
|
||||
.addFetch(KafkaProperties.TOPIC2, 0, 0L, 100)
|
||||
.addFetch(KafkaProperties.TOPIC3, 0, 0L, 100)
|
||||
.build();
|
||||
fetchResponse = simpleConsumer.fetch(req);
|
||||
int fetchReq = 0;
|
||||
|
|
Loading…
Reference in New Issue