Make it configurable how long the KafkaSpout sleeps if there is no message ready.

This required changes in the config classes.

Also, make it not a fixed sleep, but a wait that can get notified if a message does arrive.
This commit is contained in:
Craig Bryant 2014-06-25 12:02:36 -06:00
parent b59b2f3508
commit e77566f4f3
7 changed files with 63 additions and 35 deletions

View File

@ -2,6 +2,7 @@ metricSpoutThreads: 2
metricSpoutTasks: 2 metricSpoutTasks: 2
metricSpoutConfig: metricSpoutConfig:
maxWaitTime: 500
#Kafka settings. #Kafka settings.
kafkaConsumerConfiguration: kafkaConsumerConfiguration:
@ -33,6 +34,8 @@ eventSpoutThreads: 2
eventSpoutTasks: 2 eventSpoutTasks: 2
eventSpoutConfig: eventSpoutConfig:
maxWaitTime: 500
#Kafka settings. #Kafka settings.
kafkaConsumerConfiguration: kafkaConsumerConfiguration:
# See http://kafka.apache.org/documentation.html#api for semantics and defaults. # See http://kafka.apache.org/documentation.html#api for semantics and defaults.

View File

@ -16,14 +16,7 @@
*/ */
package com.hpcloud.mon; package com.hpcloud.mon;
import java.io.Serializable; public class EventSpoutConfig extends KafkaSpoutConfig {
import com.hpcloud.configuration.KafkaConsumerConfiguration;
public class EventSpoutConfig implements Serializable {
private static final long serialVersionUID = -8129774848323598123L; private static final long serialVersionUID = -8129774848323598123L;
public KafkaConsumerConfiguration kafkaConsumerConfiguration;
} }

View File

@ -0,0 +1,16 @@
package com.hpcloud.mon;
import java.io.Serializable;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.hpcloud.configuration.KafkaConsumerConfiguration;
public class KafkaSpoutConfig implements Serializable {
private static final long serialVersionUID = -6477042435089264571L;
@JsonProperty
public Integer maxWaitTime = 100;
public KafkaConsumerConfiguration kafkaConsumerConfiguration;
}

View File

@ -16,13 +16,7 @@
*/ */
package com.hpcloud.mon; package com.hpcloud.mon;
import java.io.Serializable; public class MetricSpoutConfig extends KafkaSpoutConfig {
import com.hpcloud.configuration.*;
public class MetricSpoutConfig implements Serializable {
private static final long serialVersionUID = -4285448019855024921L; private static final long serialVersionUID = -4285448019855024921L;
public KafkaConsumerConfiguration kafkaConsumerConfiguration;
} }

View File

@ -37,7 +37,7 @@ public class EventSpout extends KafkaSpout {
private final EventDeserializer deserializer; private final EventDeserializer deserializer;
public EventSpout(EventSpoutConfig configuration, EventDeserializer deserializer) { public EventSpout(EventSpoutConfig configuration, EventDeserializer deserializer) {
super(configuration.kafkaConsumerConfiguration); super(configuration);
this.deserializer = deserializer; this.deserializer = deserializer;
LOG.info("EventSpout created"); LOG.info("EventSpout created");
} }

View File

@ -19,13 +19,16 @@ package com.hpcloud.mon.infrastructure.thresholding;
import backtype.storm.spout.SpoutOutputCollector; import backtype.storm.spout.SpoutOutputCollector;
import backtype.storm.task.TopologyContext; import backtype.storm.task.TopologyContext;
import backtype.storm.topology.base.BaseRichSpout; import backtype.storm.topology.base.BaseRichSpout;
import com.hpcloud.configuration.KafkaConsumerConfiguration;
import com.hpcloud.configuration.KafkaConsumerProperties; import com.hpcloud.configuration.KafkaConsumerProperties;
import com.hpcloud.mon.KafkaSpoutConfig;
import kafka.consumer.Consumer; import kafka.consumer.Consumer;
import kafka.consumer.ConsumerConfig; import kafka.consumer.ConsumerConfig;
import kafka.consumer.ConsumerIterator; import kafka.consumer.ConsumerIterator;
import kafka.consumer.KafkaStream; import kafka.consumer.KafkaStream;
import kafka.javaapi.consumer.ConsumerConnector; import kafka.javaapi.consumer.ConsumerConnector;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -39,7 +42,7 @@ public abstract class KafkaSpout extends BaseRichSpout implements Runnable {
private static final long serialVersionUID = 744004533863562119L; private static final long serialVersionUID = 744004533863562119L;
private final KafkaConsumerConfiguration kafkaConsumerConfig; private final KafkaSpoutConfig kafkaSpoutConfig;
private transient ConsumerConnector consumerConnector; private transient ConsumerConnector consumerConnector;
@ -55,8 +58,10 @@ public abstract class KafkaSpout extends BaseRichSpout implements Runnable {
private String spoutName; private String spoutName;
protected KafkaSpout(KafkaConsumerConfiguration kafkaConsumerConfig) { private boolean waiting = false;
this.kafkaConsumerConfig = kafkaConsumerConfig;
protected KafkaSpout(KafkaSpoutConfig kafkaSpoutConfig) {
this.kafkaSpoutConfig = kafkaSpoutConfig;
} }
@Override @Override
@ -64,9 +69,9 @@ public abstract class KafkaSpout extends BaseRichSpout implements Runnable {
LOG.info("Activated"); LOG.info("Activated");
if (streams == null) { if (streams == null) {
Map<String, Integer> topicCountMap = new HashMap<>(); Map<String, Integer> topicCountMap = new HashMap<>();
topicCountMap.put(kafkaConsumerConfig.getTopic(), new Integer(1)); topicCountMap.put(kafkaSpoutConfig.kafkaConsumerConfiguration.getTopic(), new Integer(1));
Map<String, List<KafkaStream<byte[], byte[]>>> consumerMap = consumerConnector.createMessageStreams(topicCountMap); Map<String, List<KafkaStream<byte[], byte[]>>> consumerMap = consumerConnector.createMessageStreams(topicCountMap);
streams = consumerMap.get(kafkaConsumerConfig.getTopic()); streams = consumerMap.get(kafkaSpoutConfig.kafkaConsumerConfiguration.getTopic());
} }
} }
@ -74,10 +79,11 @@ public abstract class KafkaSpout extends BaseRichSpout implements Runnable {
public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) { public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
LOG.info("Opened"); LOG.info("Opened");
this.collector = collector; this.collector = collector;
LOG.info(" topic = " + kafkaConsumerConfig.getTopic()); LOG.info(" topic = " + kafkaSpoutConfig.kafkaConsumerConfiguration.getTopic());
this.spoutName = String.format("%s-%d", context.getThisComponentId(), context.getThisTaskId()); this.spoutName = String.format("%s-%d", context.getThisComponentId(), context.getThisTaskId());
Properties kafkaProperties = KafkaConsumerProperties.createKafkaProperties(kafkaConsumerConfig); Properties kafkaProperties = KafkaConsumerProperties.createKafkaProperties(
kafkaSpoutConfig.kafkaConsumerConfiguration);
// Have to use a different consumer.id for each spout so use the storm taskId. Otherwise, // Have to use a different consumer.id for each spout so use the storm taskId. Otherwise,
// zookeeper complains about a conflicted ephemeral node when there is more than one spout // zookeeper complains about a conflicted ephemeral node when there is more than one spout
// reading from a topic // reading from a topic
@ -100,10 +106,12 @@ public abstract class KafkaSpout extends BaseRichSpout implements Runnable {
while (this.shouldContinue) { while (this.shouldContinue) {
final ConsumerIterator<byte[], byte[]> it = streams.get(0).iterator(); final ConsumerIterator<byte[], byte[]> it = streams.get(0).iterator();
if (it.hasNext()) { if (it.hasNext()) {
LOG.debug("streams iterator has next");
final byte[] message = it.next().message(); final byte[] message = it.next().message();
synchronized (this) { synchronized (this) {
this.message = message; this.message = message;
// Wake up getMessage() if it is waiting
if (this.waiting)
notify();
while (this.message != null && this.shouldContinue) while (this.message != null && this.shouldContinue)
try { try {
wait(); wait();
@ -138,22 +146,36 @@ public abstract class KafkaSpout extends BaseRichSpout implements Runnable {
} }
} }
private synchronized byte[] getMessage() { /**
* Must only be called from a synchronized method
*
* @return
*/
private byte[] tryToGetMessage() {
final byte[] result = this.message; final byte[] result = this.message;
if (result != null) { if (result != null) {
this.message = null; this.message = null;
notify(); notify();
} }
else {
// Storm docs recommend a short sleep
try {
Thread.sleep(10);
} catch (InterruptedException e) {
LOG.info("Sleep interrupted", e);
}
}
return result; return result;
} }
private synchronized byte[] getMessage() {
final byte[] result = tryToGetMessage();
if (result != null) {
return result;
}
// Storm docs recommend a short sleep but make the sleep time
// configurable so we can lessen the load on dev systems
this.waiting = true;
try {
wait(kafkaSpoutConfig.maxWaitTime);
} catch (InterruptedException e) {
LOG.info("Sleep interrupted", e);
}
this.waiting = false;
return tryToGetMessage(); // We might have been woken up because there was a message
}
protected abstract void processMessage(byte[] message, SpoutOutputCollector collector2); protected abstract void processMessage(byte[] message, SpoutOutputCollector collector2);
} }

View File

@ -38,7 +38,7 @@ public class MetricSpout extends KafkaSpout {
public static final String DEFAULT_TENANT_ID = "TENANT_ID_NOT_SET"; public static final String DEFAULT_TENANT_ID = "TENANT_ID_NOT_SET";
public MetricSpout(MetricSpoutConfig metricSpoutConfig) { public MetricSpout(MetricSpoutConfig metricSpoutConfig) {
super(metricSpoutConfig.kafkaConsumerConfiguration); super(metricSpoutConfig);
LOG.info("Created"); LOG.info("Created");
} }