/* # Licensed Materials - Property of IBM # Copyright IBM Corp. 2015 */ package com.ibm.streamsx.topology.messaging.kafka; import java.util.Collections; import java.util.HashMap; import java.util.Map; import com.ibm.streams.operator.Tuple; import com.ibm.streamsx.topology.TStream; import com.ibm.streamsx.topology.Topology; import com.ibm.streamsx.topology.TopologyElement; import com.ibm.streamsx.topology.builder.BOperatorInvocation; import com.ibm.streamsx.topology.builder.BOutputPort; import com.ibm.streamsx.topology.function.Function; import com.ibm.streamsx.topology.function.Supplier; import com.ibm.streamsx.topology.logic.Value; import com.ibm.streamsx.topology.spl.SPL; import com.ibm.streamsx.topology.spl.SPLStream; import com.ibm.streamsx.topology.tuple.Message; import com.ibm.streamsx.topology.tuple.SimpleMessage; /** * A simple connector to an Apache Kafka cluster for consuming Kafka messages * -- subscribing to Kafka topics and creating a {@code TStream<Message>}. * <p> * A single connector is for a specific Kafka cluster as specified in * the consumer configuration. * <p> * A connector can create any number of consumers in the topology. * A consumer can subscribe to one or more topics. * <p> * Sample use: * <pre> * Topology top = ... * Properties consumerConfig = ... * KafkaConsumer cc = new KafkaConsumer(top, consumerConfig); * // with Java8 Lambda expressions... * TStream<Message> rcvdMsgs = cc.consumer(()->"myTopic"); * // without Java8... * TStream<Message> rcvdMsgs = cc.consumer(new Value<String>("myTopic")); * </pre> * * @see <a href="http://kafka.apache.org">http://kafka.apache.org</a> * @see <a * href="http://ibmstreams.github.io/streamsx.messaging/">com.ibm.streamsx.messaging</a> */ public class KafkaConsumer { private static final String PROP_FILE_PARAM = "etc/kafkaStreams/emptyConsumerProperties"; private final TopologyElement te; private final Map<String,Object> config; private boolean addedFileDependency; void addPropertiesFile() { if (!addedFileDependency) { addedFileDependency = true; Util.addPropertiesFile(te, PROP_FILE_PARAM); } } /** * Create a consumer connector for subscribing to topics. * <p> * See the Apache Kafka documentation for {@code KafkaConsumer} * configuration properties at <a href="http://kafka.apache.org">http://kafka.apache.org</a>. * Configuration property values are strings. * <p> * Minimal configuration typically includes: * <ul> * <li><code>zookeeper.connect</code></li> * <li><code>group.id</code></li> * <li><code>zookeeper.session.timeout.ms</code></li> * <li><code>zookeeper.sync.time.ms</code></li> * <li><code>auto.commit.interval.ms</code></li> * </ul> * @param te {@link TopologyElement} * @param config KafkaConsumer configuration information. */ public KafkaConsumer(TopologyElement te, Map<String, Object> config) { this.te = te; this.config = new HashMap<>(); this.config.putAll(config); } /** * Get the connector's {@code KafkaConsumer} configuration information. * @return the unmodifiable configuration */ public Map<String,Object> getConfig() { return Collections.unmodifiableMap(config); } /** * Subscribe to a topic and create a stream of messages. * <p> * Same as {@code subscribe(new Value<Integer>(1), topic)} * @param topic the topic to subscribe to. May be a submission parameter. * @return TStream<Message> * @see Value * @see Topology#createSubmissionParameter(String, Class) */ public TStream<Message> subscribe(Supplier<String> topic) { return subscribe(new Value<Integer>(1), topic); } /** * Subscribe to a topic and create a stream of messages. * <p> * N.B., A topology that includes this will not support * {@code StreamsContext.Type.EMBEDDED}. * <p> * N.B. due to com.ibm.streamsx.messaging * <a href="https://github.com/IBMStreams/streamsx.messaging/issues/118">issue#118</a>, * multiple consumers will have issues in * {@code StreamsContext.Type.STANDALONE}. * <p> * N.B. due to com.ibm.streamsx.messaging * <a href="https://github.com/IBMStreams/streamsx.messaging/issues/117">issue#117</a>, * a consumer in {@code StreamsContext.Type.STANDALONE} subsequently results * in an orphaned @{code standalone} processes that continues as the lead * group/topic consumer thereby preventing subsequent instances of the * group/topic consumer from receiving messages. * <p> * N.B. due to com.ibm.streamsx.messaging * <a href="https://github.com/IBMStreams/streamsx.messaging/issues/114">issue#114</a>, * a consumer essentially ignores messages generated by producers where the * optional {@code key} is {@code null}. * e.g., Kafka's {@code kafka-console-producer.sh} tool generates * {@code key==null} messages. * * @param threadsPerTopic number of threads to allocate to processing each * topic. May be a submission parameter. * @param topic the topic to subscribe to. May be a submission parameter. * @return TStream<Message> * The generated {@code Message} tuples have a non-null {@code topic}. * The tuple's {@code key} will be null if the Kafka message * lacked a key or it's key was the empty string. * @throws IllegalArgumentException if topic is null. * @throws IllegalArgumentException if threadsPerTopic is null. * @see Value * @see Topology#createSubmissionParameter(String, Class) */ public TStream<Message> subscribe(Supplier<Integer> threadsPerTopic, Supplier<String> topic) { if (topic == null) throw new IllegalArgumentException("topic"); if (threadsPerTopic == null || (threadsPerTopic.get() != null && threadsPerTopic.get() <= 0)) throw new IllegalArgumentException("threadsPerTopic"); Map<String, Object> params = new HashMap<>(); params.put("topic", topic); if (threadsPerTopic instanceof Value && threadsPerTopic.get() == 1) ; // The default is one. else params.put("threadsPerTopic", threadsPerTopic); if (!config.isEmpty()) params.put("kafkaProperty", Util.toKafkaProperty(config)); // workaround streamsx.messaging issue #107 params.put("propertiesFile", PROP_FILE_PARAM); addPropertiesFile(); // Use SPL.invoke to avoid adding a compile time dependency // to com.ibm.streamsx.messaging since JavaPrimitive.invoke*() // lack "kind" based variants. String kind = "com.ibm.streamsx.messaging.kafka::KafkaConsumer"; String className = "com.ibm.streamsx.messaging.kafka.KafkaSource"; SPLStream rawKafka = SPL.invokeSource( te, kind, params, KafkaSchemas.KAFKA); SPL.tagOpAsJavaPrimitive(toOp(rawKafka), kind, className); TStream<Message> rcvdMsgs = toMessageStream(rawKafka); rcvdMsgs.colocate(rawKafka); // workaround streamsx.messaging issue#118 w/java8 // isolate even in the single consumer case since we don't // know if others may be subsequently created. rcvdMsgs = rcvdMsgs.isolate(); return rcvdMsgs; } private BOperatorInvocation toOp(SPLStream splStream) { BOutputPort oport = (BOutputPort) splStream.output(); return (BOperatorInvocation) oport.operator(); } /** * Convert an {@link SPLStream} with schema {@link KafkaSchemas.KAFKA} * to a TStream<{@link Message}>. * The returned stream will contain a {@code Message} tuple for * each tuple on {@code stream}. * A runtime error will occur if the schema of {@code stream} doesn't * have the attributes defined by {@code KafkaSchemas.KAFKA}. * @param stream Stream to be converted to a TStream<Message>. * @return Stream of {@code Message} tuples from {@code stream}. */ private static TStream<Message> toMessageStream(SPLStream stream) { return stream.convert(new Function<Tuple, Message>() { private static final long serialVersionUID = 1L; @Override public Message apply(Tuple tuple) { return new SimpleMessage(tuple.getString("message"), fromSplValue(tuple.getString("key")), tuple.getString("topic")); } private String fromSplValue(String s) { // SPL doesn't allow null value. For our use, // assume an empty string meant null. return (s==null || s.isEmpty()) ? null : s; } }); } }