Skip to content
Closed
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -56,23 +56,36 @@ public class KafkaBolt<K, V> extends BaseRichBolt {

private static final Logger LOG = LoggerFactory.getLogger(KafkaBolt.class);

public static final String TOPIC = "topic";
private static final String TOPIC = "topic";

private static final String DEFAULT_KEY_SERIALIZER = "org.apache.kafka.common.serialization.StringSerializer";
private static final String DEFAULT_VALUE_SERIALIZER = "org.apache.kafka.common.serialization.StringSerializer";
private static final String DEFAULT_ACKS = "1";

private KafkaProducer<K, V> producer;
private OutputCollector collector;
private TupleToKafkaMapper<K,V> mapper;
private TupleToKafkaMapper<K,V> mapper = new FieldNameBasedTupleToKafkaMapper<>();
private KafkaTopicSelector topicSelector;
private Properties boltSpecfiedProperties = new Properties();
/**
* With default setting for fireAndForget and async, the callback is called when the sending succeeds.
* By setting fireAndForget true, the send will not wait at all for kafka to ack.
* "acks" setting in 0.8.2 Producer API config doesn't matter if fireAndForget is set.
* By setting async false, synchronous sending is used.
* By setting async false, synchronous sending is used.
*/
private boolean fireAndForget = false;
private boolean async = true;

public KafkaBolt() {}
public KafkaBolt() {
boltSpecfiedProperties.setProperty("acks", DEFAULT_ACKS);
boltSpecfiedProperties.setProperty("key.serializer", DEFAULT_KEY_SERIALIZER);
boltSpecfiedProperties.setProperty("value.serializer", DEFAULT_VALUE_SERIALIZER);
}

public KafkaBolt(String hosts) {
this();
boltSpecfiedProperties.setProperty("bootstrap.servers", hosts);
}

public KafkaBolt<K,V> withTupleToKafkaMapper(TupleToKafkaMapper<K,V> mapper) {
this.mapper = mapper;
Expand Down Expand Up @@ -108,8 +121,8 @@ public void prepare(Map stormConf, TopologyContext context, OutputCollector coll
@Override
public void execute(final Tuple input) {
if (TupleUtils.isTick(input)) {
collector.ack(input);
return; // Do not try to send ticks to Kafka
collector.ack(input);
return; // Do not try to send ticks to Kafka
}
K key = null;
V message = null;
Expand Down