Apache Kafka includes new java clients (in the org.apache.kafka.clients package). These are meant to supplant the older Scala clients, but for compatibility they will co-exist for some time. These clients are available in a separate jar with minimal dependencies, while the old Scala clients remain packaged with the server.
We encourage all new development to use the new Java producer. This client is production tested and generally both faster and more fully featured than the previous Scala client. You can use this client by adding a dependency on the client jar using the following example maven co-ordinates (you can change the version numbers with new releases):
As of the 0.9.0 release we have added a new Java consumer to replace our existing high-level ZooKeeper-based consumer and low-level consumer APIs. This client is considered beta quality. To ensure a smooth upgrade path for users, we still maintain the old 0.8 consumer clients that continue to work on an 0.9 Kafka cluster. In the following sections we introduce both the old 0.8 consumer APIs (both high-level ConsumerConnector and low-level SimpleConsumer) and the new Java consumer API respectively.
classConsumer { /** * Create a ConsumerConnector * * @param config at the minimum, need to specify the groupid of the consumer and the zookeeper * connection string zookeeper.connect. */publicstatickafka.javaapi.consumer.ConsumerConnectorcreateJavaConsumerConnector(ConsumerConfig config);}/** * V: type of the message * K: type of the optional key associated with the message */publicinterfacekafka.javaapi.consumer.ConsumerConnector { /** * Create a list of message streams of type T for each topic. * * @param topicCountMap a map of (topic, #streams) pair * @param decoder a decoder that converts from Message to T * @return a map of (topic, list of KafkaStream) pairs. * The number of items in the list is #streams. Each stream supports * an iterator over message/metadata pairs. */public <K,V> Map<String,List<KafkaStream<K,V>>>createMessageStreams(Map<String,Integer> topicCountMap,Decoder<K> keyDecoder,Decoder<V> valueDecoder); /** * Create a list of message streams of type T for each topic, using the default decoder. */publicMap<String,List<KafkaStream<byte[],byte[]>>> createMessageStreams(Map<String,Integer> topicCountMap); /** * Create a list of message streams for topics matching a wildcard. * * @param topicFilter a TopicFilter that specifies which topics to * subscribe to (encapsulates a whitelist or a blacklist). * @param numStreams the number of message streams to return. * @param keyDecoder a decoder that decodes the message key * @param valueDecoder a decoder that decodes the message itself * @return a list of KafkaStream. Each stream supports an * iterator over its MessageAndMetadata elements. */public <K,V> List<KafkaStream<K,V>>createMessageStreamsByFilter(TopicFilter topicFilter,int numStreams,Decoder<K> keyDecoder,Decoder<V> valueDecoder); /** * Create a list of message streams for topics matching a wildcard, using the default decoder. */publicList<KafkaStream<byte[],byte[]>> createMessageStreamsByFilter(TopicFilter topicFilter,int numStreams); /** * Create a list of message streams for topics matching a wildcard, using the default decoder, with one stream. */publicList<KafkaStream<byte[],byte[]>> createMessageStreamsByFilter(TopicFilter topicFilter); /** * Commit the offsets of all topic/partitions connected by this connector. */publicvoidcommitOffsets(); /** * Shut down the connector */publicvoidshutdown();}
You can follow this example to learn how to use the high level consumer api.
class kafka.javaapi.consumer.SimpleConsumer {
/**
* Fetch a set of messages from a topic.
*
* @param request specifies the topic name, topic partition, starting byte offset, maximum bytes to be fetched.
* @return a set of fetched messages
*/
public FetchResponse fetch(kafka.javaapi.FetchRequest request);
/**
* Fetch metadata for a sequence of topics.
*
* @param request specifies the versionId, clientId, sequence of topics.
* @return metadata for each topic in the request.
*/
public kafka.javaapi.TopicMetadataResponse send(kafka.javaapi.TopicMetadataRequest request);
/**
* Get a list of valid offsets (up to maxSize) before the given time.
*
* @param request a [[kafka.javaapi.OffsetRequest]] object.
* @return a [[kafka.javaapi.OffsetResponse]] object.
*/
public kafka.javaapi.OffsetResponse getOffsetsBefore(OffsetRequest request);
/**
* Close the SimpleConsumer.
*/
public void close();
}
For most applications, the high level consumer Api is good enough. Some applications want features not exposed to the high level consumer yet (e.g., set initial offset when restarting the consumer). They can instead use our low level SimpleConsumer Api. The logic will be a bit more complicated and you can follow the example in here.
This new unified consumer API removes the distinction between the 0.8 high-level and low-level consumer APIs. You can use this client by adding a dependency on the client jar using the following example maven co-ordinates (you can change the version numbers with new releases):
As of the 0.10.0 release we have added a new client library named Kafka Streams to let users implement their stream processing applications with data stored in Kafka topics. Kafka Streams is considered alpha quality and its public APIs are likely to change in future releases. You can use Kafka Streams by adding a dependency on the streams jar using the following example maven co-ordinates (you can change the version numbers with new releases):
Examples showing how to use this library are given in the javadocs (note those classes annotated with@InterfaceStability.Unstable, indicating their public APIs may change without backward-compatibility in future releases).