Mirror of Apache Kafka
You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
 
 
 
 

158 lines
7.4 KiB

<!--
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.
-->
Apache Kafka includes new java clients (in the org.apache.kafka.clients package). These are meant to supplant the older Scala clients, but for compatability they will co-exist for some time. These clients are available in a seperate jar with minimal dependencies, while the old Scala clients remain packaged with the server.
<h3><a id="producerapi" href="#producerapi">2.1 Producer API</a></h3>
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):
<pre>
&lt;dependency&gt;
&lt;groupId&gt;org.apache.kafka&lt;/groupId&gt;
&lt;artifactId&gt;kafka-clients&lt;/artifactId&gt;
&lt;version&gt;0.9.0.0&lt;/version&gt;
&lt;/dependency&gt;
</pre>
Examples showing how to use the producer are given in the
<a href="http://kafka.apache.org/090/javadoc/index.html?org/apache/kafka/clients/producer/KafkaProducer.html" title="Kafka 0.9.0 Javadoc">javadocs</a>.
<p>
For those interested in the legacy Scala producer api, information can be found <a href="http://kafka.apache.org/081/documentation.html#producerapi">
here</a>.
</p>
<h3><a id="highlevelconsumerapi" href="#highlevelconsumerapi">2.2 High Level Consumer API</a></h3>
<pre>
class Consumer {
/**
* Create a ConsumerConnector
*
* @param config at the minimum, need to specify the groupid of the consumer and the zookeeper
* connection string zookeeper.connect.
*/
public static kafka.javaapi.consumer.ConsumerConnector createJavaConsumerConnector(ConsumerConfig config);
}
/**
* V: type of the message
* K: type of the optional key assciated with the message
*/
public interface kafka.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 &lt;K,V&gt; Map&lt;String, List&lt;KafkaStream&lt;K,V&gt;&gt;&gt;
createMessageStreams(Map&lt;String, Integer&gt; topicCountMap, Decoder&lt;K&gt; keyDecoder, Decoder&lt;V&gt; valueDecoder);
/**
* Create a list of message streams of type T for each topic, using the default decoder.
*/
public Map&lt;String, List&lt;KafkaStream&lt;byte[], byte[]&gt;&gt;&gt; createMessageStreams(Map&lt;String, Integer&gt; 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 &lt;K,V&gt; List&lt;KafkaStream&lt;K,V&gt;&gt;
createMessageStreamsByFilter(TopicFilter topicFilter, int numStreams, Decoder&lt;K&gt; keyDecoder, Decoder&lt;V&gt; valueDecoder);
/**
* Create a list of message streams for topics matching a wildcard, using the default decoder.
*/
public List&lt;KafkaStream&lt;byte[], byte[]&gt;&gt; createMessageStreamsByFilter(TopicFilter topicFilter, int numStreams);
/**
* Create a list of message streams for topics matching a wildcard, using the default decoder, with one stream.
*/
public List&lt;KafkaStream&lt;byte[], byte[]&gt;&gt; createMessageStreamsByFilter(TopicFilter topicFilter);
/**
* Commit the offsets of all topic/partitions connected by this connector.
*/
public void commitOffsets();
/**
* Shut down the connector
*/
public void shutdown();
}
</pre>
You can follow
<a href="https://cwiki.apache.org/confluence/display/KAFKA/Consumer+Group+Example" title="Kafka 0.8 consumer example">this example</a> to learn how to use the high level consumer api.
<h3><a id="simpleconsumerapi" href="#simpleconsumerapi">2.3 Simple Consumer API</a></h3>
<pre>
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();
}
</pre>
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
<a href="https://cwiki.apache.org/confluence/display/KAFKA/0.8.0+SimpleConsumer+Example" title="Kafka 0.8 SimpleConsumer example">here</a>.
<h3><a id="newconsumerapi" href="#newconsumerapi">2.4 New Consumer API</a></h3>
As of the 0.9.0 release we have added a replacement for our existing simple and high-level consumers. This client is considered beta quality. 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):
<pre>
&lt;dependency&gt;
&lt;groupId&gt;org.apache.kafka&lt;/groupId&gt;
&lt;artifactId&gt;kafka-clients&lt;/artifactId&gt;
&lt;version&gt;0.9.0.0&lt;/version&gt;
&lt;/dependency&gt;
</pre>
Examples showing how to use the consumer are given in the
<a href="http://kafka.apache.org/090/javadoc/index.html?org/apache/kafka/clients/consumer/KafkaConsumer.html" title="Kafka 0.9.0 Javadoc">javadocs</a>.