@ -12,19 +12,6 @@
@@ -12,19 +12,6 @@
* /
package org.apache.kafka.clients.consumer ;
import java.net.InetSocketAddress ;
import java.util.Collection ;
import java.util.Collections ;
import java.util.HashMap ;
import java.util.HashSet ;
import java.util.List ;
import java.util.Map ;
import java.util.Properties ;
import java.util.Set ;
import java.util.Map.Entry ;
import java.util.concurrent.Future ;
import org.apache.kafka.clients.producer.RecordMetadata ;
import org.apache.kafka.common.Metric ;
import org.apache.kafka.common.TopicPartition ;
import org.apache.kafka.common.metrics.JmxReporter ;
@ -36,6 +23,9 @@ import org.apache.kafka.common.utils.SystemTime;
@@ -36,6 +23,9 @@ import org.apache.kafka.common.utils.SystemTime;
import org.slf4j.Logger ;
import org.slf4j.LoggerFactory ;
import java.net.InetSocketAddress ;
import java.util.* ;
/ * *
* A Kafka client that consumes records from a Kafka cluster .
* < P >
@ -50,12 +40,12 @@ import org.slf4j.LoggerFactory;
@@ -50,12 +40,12 @@ import org.slf4j.LoggerFactory;
* a convenience method to demonstrate the different use cases of the consumer APIs . Here is a sample implementation of such a process ( ) method .
* < pre >
* { @code
* private Map < TopicPartition , Long > process ( Map < String , ConsumerRecords > records ) {
* private Map < TopicPartition , Long > process ( Map < String , ConsumerRecord < byte [ ] , byte [ ] > records ) {
* Map < TopicPartition , Long > processedOffsets = new HashMap < TopicPartition , Long > ( ) ;
* for ( Entry < String , ConsumerRecords > recordMetadata : records . entrySet ( ) ) {
* List < ConsumerRecord > recordsPerTopic = recordMetadata . getValue ( ) . records ( ) ;
* for ( Entry < String , ConsumerRecords < byte [ ] , byte [ ] > > recordMetadata : records . entrySet ( ) ) {
* List < ConsumerRecord < byte [ ] , byte [ ] > > recordsPerTopic = recordMetadata . getValue ( ) . records ( ) ;
* for ( int i = 0 ; i < recordsPerTopic . size ( ) ; i + + ) {
* ConsumerRecord record = recordsPerTopic . get ( i ) ;
* ConsumerRecord < byte [ ] , byte [ ] > record = recordsPerTopic . get ( i ) ;
* // process record
* try {
* processedOffsets . put ( record . topicAndpartition ( ) , record . offset ( ) ) ;
@ -80,11 +70,11 @@ import org.slf4j.LoggerFactory;
@@ -80,11 +70,11 @@ import org.slf4j.LoggerFactory;
* props . put ( "session.timeout.ms" , "1000" ) ;
* props . put ( "enable.auto.commit" , "true" ) ;
* props . put ( "auto.commit.interval.ms" , "10000" ) ;
* KafkaConsumer consumer = new KafkaConsumer ( props ) ;
* KafkaConsumer < byte [ ] , byte [ ] > consumer = new KafkaConsumer < byte [ ] , byte [ ] > ( props ) ;
* consumer . subscribe ( "foo" , "bar" ) ;
* boolean isRunning = true ;
* while ( isRunning ) {
* Map < String , ConsumerRecords > records = consumer . poll ( 100 ) ;
* Map < String , ConsumerRecords < byte [ ] , byte [ ] > > records = consumer . poll ( 100 ) ;
* process ( records ) ;
* }
* consumer . close ( ) ;
@ -102,14 +92,14 @@ import org.slf4j.LoggerFactory;
@@ -102,14 +92,14 @@ import org.slf4j.LoggerFactory;
* props . put ( "group.id" , "test" ) ;
* props . put ( "session.timeout.ms" , "1000" ) ;
* props . put ( "enable.auto.commit" , "false" ) ;
* KafkaConsumer consumer = new KafkaConsumer ( props ) ;
* KafkaConsumer < byte [ ] , byte [ ] > consumer = new KafkaConsumer < byte [ ] , byte [ ] > ( props ) ;
* consumer . subscribe ( "foo" , "bar" ) ;
* int commitInterval = 100 ;
* int numRecords = 0 ;
* boolean isRunning = true ;
* Map < TopicPartition , Long > consumedOffsets = new HashMap < TopicPartition , Long > ( ) ;
* while ( isRunning ) {
* Map < String , ConsumerRecords > records = consumer . poll ( 100 ) ;
* Map < String , ConsumerRecords < byte [ ] , byte [ ] > > records = consumer . poll ( 100 ) ;
* try {
* Map < TopicPartition , Long > lastConsumedOffsets = process ( records ) ;
* consumedOffsets . putAll ( lastConsumedOffsets ) ;
@ -156,16 +146,17 @@ import org.slf4j.LoggerFactory;
@@ -156,16 +146,17 @@ import org.slf4j.LoggerFactory;
* props . put ( "group.id" , "test" ) ;
* props . put ( "session.timeout.ms" , "1000" ) ;
* props . put ( "enable.auto.commit" , "false" ) ;
* KafkaConsumer consumer = new KafkaConsumer ( props ,
* KafkaConsumer < byte [ ] , byte [ ] > consumer = new KafkaConsumer < byte [ ] , byte [ ] > (
* props ,
* new ConsumerRebalanceCallback ( ) {
* boolean rewindOffsets = true ; // should be retrieved from external application config
* public void onPartitionsAssigned ( Consumer consumer , Collection < TopicPartition > partitions ) {
* public void onPartitionsAssigned ( Consumer < ? , ? > consumer , Collection < TopicPartition > partitions ) {
* Map < TopicPartition , Long > latestCommittedOffsets = consumer . committed ( partitions ) ;
* if ( rewindOffsets )
* Map < TopicPartition , Long > newOffsets = rewindOffsets ( latestCommittedOffsets , 100 ) ;
* consumer . seek ( newOffsets ) ;
* }
* public void onPartitionsRevoked ( Consumer consumer , Collection < TopicPartition > partitions ) {
* public void onPartitionsRevoked ( Consumer < ? , ? > consumer , Collection < TopicPartition > partitions ) {
* consumer . commit ( true ) ;
* }
* // this API rewinds every partition back by numberOfMessagesToRewindBackTo messages
@ -183,7 +174,7 @@ import org.slf4j.LoggerFactory;
@@ -183,7 +174,7 @@ import org.slf4j.LoggerFactory;
* boolean isRunning = true ;
* Map < TopicPartition , Long > consumedOffsets = new HashMap < TopicPartition , Long > ( ) ;
* while ( isRunning ) {
* Map < String , ConsumerRecords > records = consumer . poll ( 100 ) ;
* Map < String , ConsumerRecords < byte [ ] , byte [ ] > > records = consumer . poll ( 100 ) ;
* Map < TopicPartition , Long > lastConsumedOffsets = process ( records ) ;
* consumedOffsets . putAll ( lastConsumedOffsets ) ;
* numRecords + = records . size ( ) ;
@ -211,13 +202,14 @@ import org.slf4j.LoggerFactory;
@@ -211,13 +202,14 @@ import org.slf4j.LoggerFactory;
* props . put ( "group.id" , "test" ) ;
* props . put ( "session.timeout.ms" , "1000" ) ;
* props . put ( "enable.auto.commit" , "false" ) ; // since enable.auto.commit only applies to Kafka based offset storage
* KafkaConsumer consumer = new KafkaConsumer ( props ,
* KafkaConsumer < byte [ ] , byte [ ] > consumer = new KafkaConsumer < byte [ ] , byte [ ] > (
* props ,
* new ConsumerRebalanceCallback ( ) {
* public void onPartitionsAssigned ( Consumer consumer , Collection < TopicPartition > partitions ) {
* public void onPartitionsAssigned ( Consumer < ? , ? > consumer , Collection < TopicPartition > partitions ) {
* Map < TopicPartition , Long > lastCommittedOffsets = getLastCommittedOffsetsFromCustomStore ( partitions ) ;
* consumer . seek ( lastCommittedOffsets ) ;
* }
* public void onPartitionsRevoked ( Consumer consumer , Collection < TopicPartition > partitions ) {
* public void onPartitionsRevoked ( Consumer < ? , ? > consumer , Collection < TopicPartition > partitions ) {
* Map < TopicPartition , Long > offsets = getLastConsumedOffsets ( partitions ) ;
* commitOffsetsToCustomStore ( offsets ) ;
* }
@ -234,7 +226,7 @@ import org.slf4j.LoggerFactory;
@@ -234,7 +226,7 @@ import org.slf4j.LoggerFactory;
* boolean isRunning = true ;
* Map < TopicPartition , Long > consumedOffsets = new HashMap < TopicPartition , Long > ( ) ;
* while ( isRunning ) {
* Map < String , ConsumerRecords > records = consumer . poll ( 100 ) ;
* Map < String , ConsumerRecords < byte [ ] , byte [ ] > > records = consumer . poll ( 100 ) ;
* Map < TopicPartition , Long > lastConsumedOffsets = process ( records ) ;
* consumedOffsets . putAll ( lastConsumedOffsets ) ;
* numRecords + = records . size ( ) ;
@ -259,7 +251,7 @@ import org.slf4j.LoggerFactory;
@@ -259,7 +251,7 @@ import org.slf4j.LoggerFactory;
* props . put ( "group.id" , "test" ) ;
* props . put ( "enable.auto.commit" , "true" ) ;
* props . put ( "auto.commit.interval.ms" , "10000" ) ;
* KafkaConsumer consumer = new KafkaConsumer ( props ) ;
* KafkaConsumer < byte [ ] , byte [ ] > consumer = new KafkaConsumer < byte [ ] , byte [ ] > ( props ) ;
* // subscribe to some partitions of topic foo
* TopicPartition partition0 = new TopicPartition ( "foo" , 0 ) ;
* TopicPartition partition1 = new TopicPartition ( "foo" , 1 ) ;
@ -276,7 +268,7 @@ import org.slf4j.LoggerFactory;
@@ -276,7 +268,7 @@ import org.slf4j.LoggerFactory;
* boolean isRunning = true ;
* Map < TopicPartition , Long > consumedOffsets = new HashMap < TopicPartition , Long > ( ) ;
* while ( isRunning ) {
* Map < String , ConsumerRecords > records = consumer . poll ( 100 ) ;
* Map < String , ConsumerRecords < byte [ ] , byte [ ] > > records = consumer . poll ( 100 ) ;
* Map < TopicPartition , Long > lastConsumedOffsets = process ( records ) ;
* consumedOffsets . putAll ( lastConsumedOffsets ) ;
* for ( TopicPartition partition : partitions ) {
@ -298,7 +290,7 @@ import org.slf4j.LoggerFactory;
@@ -298,7 +290,7 @@ import org.slf4j.LoggerFactory;
* { @code
* Properties props = new Properties ( ) ;
* props . put ( "metadata.broker.list" , "localhost:9092" ) ;
* KafkaConsumer consumer = new KafkaConsumer ( props ) ;
* KafkaConsumer < byte [ ] , byte [ ] > consumer = new KafkaConsumer < byte [ ] , byte [ ] > ( props ) ;
* // subscribe to some partitions of topic foo
* TopicPartition partition0 = new TopicPartition ( "foo" , 0 ) ;
* TopicPartition partition1 = new TopicPartition ( "foo" , 1 ) ;
@ -314,7 +306,7 @@ import org.slf4j.LoggerFactory;
@@ -314,7 +306,7 @@ import org.slf4j.LoggerFactory;
* boolean isRunning = true ;
* Map < TopicPartition , Long > consumedOffsets = new HashMap < TopicPartition , Long > ( ) ;
* while ( isRunning ) {
* Map < String , ConsumerRecords > records = consumer . poll ( 100 ) ;
* Map < String , ConsumerRecords < byte [ ] , byte [ ] > > records = consumer . poll ( 100 ) ;
* Map < TopicPartition , Long > lastConsumedOffsets = process ( records ) ;
* consumedOffsets . putAll ( lastConsumedOffsets ) ;
* // commit offsets for partitions 0,1 for topic foo to custom store
@ -331,7 +323,7 @@ import org.slf4j.LoggerFactory;
@@ -331,7 +323,7 @@ import org.slf4j.LoggerFactory;
* }
* < / pre >
* /
public class KafkaConsumer implements Consumer {
public class KafkaConsumer < K , V > implements Consumer < K , V > {
private static final Logger log = LoggerFactory . getLogger ( KafkaConsumer . class ) ;
@ -340,7 +332,9 @@ public class KafkaConsumer implements Consumer {
@@ -340,7 +332,9 @@ public class KafkaConsumer implements Consumer {
private final Metrics metrics ;
private final Set < String > subscribedTopics ;
private final Set < TopicPartition > subscribedPartitions ;
private final Deserializer < K > keyDeserializer ;
private final Deserializer < V > valueDeserializer ;
/ * *
* A consumer is instantiated by providing a set of key - value pairs as configuration . Valid configuration strings
* are documented < a href = "http://kafka.apache.org/documentation.html#consumerconfigs" > here < / a > . Values can be
@ -351,7 +345,7 @@ public class KafkaConsumer implements Consumer {
@@ -351,7 +345,7 @@ public class KafkaConsumer implements Consumer {
* @param configs The consumer configs
* /
public KafkaConsumer ( Map < String , Object > configs ) {
this ( new ConsumerConfig ( configs ) , null ) ;
this ( new ConsumerConfig ( configs ) , null , null , null ) ;
}
/ * *
@ -364,7 +358,24 @@ public class KafkaConsumer implements Consumer {
@@ -364,7 +358,24 @@ public class KafkaConsumer implements Consumer {
* every rebalance operation .
* /
public KafkaConsumer ( Map < String , Object > configs , ConsumerRebalanceCallback callback ) {
this ( new ConsumerConfig ( configs ) , callback ) ;
this ( new ConsumerConfig ( configs ) , callback , null , null ) ;
}
/ * *
* A consumer is instantiated by providing a set of key - value pairs as configuration , a { @link ConsumerRebalanceCallback }
* implementation , a key and a value { @link Deserializer } .
* < p >
* Valid configuration strings are documented at { @link ConsumerConfig }
* @param configs The consumer configs
* @param callback A callback interface that the user can implement to manage customized offsets on the start and end of
* every rebalance operation .
* @param keyDeserializer The deserializer for key that implements { @link Deserializer } . The configure ( ) method won ' t
* be called in the consumer when the deserializer is passed in directly .
* @param valueDeserializer The deserializer for value that implements { @link Deserializer } . The configure ( ) method
* won ' t be called in the consumer when the deserializer is passed in directly .
* /
public KafkaConsumer ( Map < String , Object > configs , ConsumerRebalanceCallback callback , Deserializer < K > keyDeserializer , Deserializer < V > valueDeserializer ) {
this ( new ConsumerConfig ( configs ) , callback , keyDeserializer , valueDeserializer ) ;
}
/ * *
@ -372,12 +383,12 @@ public class KafkaConsumer implements Consumer {
@@ -372,12 +383,12 @@ public class KafkaConsumer implements Consumer {
* Valid configuration strings are documented at { @link ConsumerConfig }
* /
public KafkaConsumer ( Properties properties ) {
this ( new ConsumerConfig ( properties ) , null ) ;
this ( new ConsumerConfig ( properties ) , null , null , null ) ;
}
/ * *
* A consumer is instantiated by providing a { @link java . util . Properties } object as configuration and a
* { @link ConsumerRebalanceCallback } implementation .
* { @link ConsumerRebalanceCallback } implementation .
* < p >
* Valid configuration strings are documented at { @link ConsumerConfig }
* @param properties The consumer configuration properties
@ -385,14 +396,27 @@ public class KafkaConsumer implements Consumer {
@@ -385,14 +396,27 @@ public class KafkaConsumer implements Consumer {
* every rebalance operation .
* /
public KafkaConsumer ( Properties properties , ConsumerRebalanceCallback callback ) {
this ( new ConsumerConfig ( properties ) , callback ) ;
this ( new ConsumerConfig ( properties ) , callback , null , null ) ;
}
private KafkaConsumer ( ConsumerConfig config ) {
this ( config , null ) ;
/ * *
* A consumer is instantiated by providing a { @link java . util . Properties } object as configuration and a
* { @link ConsumerRebalanceCallback } implementation , a key and a value { @link Deserializer } .
* < p >
* Valid configuration strings are documented at { @link ConsumerConfig }
* @param properties The consumer configuration properties
* @param callback A callback interface that the user can implement to manage customized offsets on the start and end of
* every rebalance operation .
* @param keyDeserializer The deserializer for key that implements { @link Deserializer } . The configure ( ) method won ' t
* be called in the consumer when the deserializer is passed in directly .
* @param valueDeserializer The deserializer for value that implements { @link Deserializer } . The configure ( ) method
* won ' t be called in the consumer when the deserializer is passed in directly .
* /
public KafkaConsumer ( Properties properties , ConsumerRebalanceCallback callback , Deserializer < K > keyDeserializer , Deserializer < V > valueDeserializer ) {
this ( new ConsumerConfig ( properties ) , callback , keyDeserializer , valueDeserializer ) ;
}
private KafkaConsumer ( ConsumerConfig config , ConsumerRebalanceCallback callback ) {
private KafkaConsumer ( ConsumerConfig config , ConsumerRebalanceCallback callback , Deserializer < K > keyDeserializer , Deserializer < V > valueDeserializer ) {
log . trace ( "Starting the Kafka consumer" ) ;
subscribedTopics = new HashSet < String > ( ) ;
subscribedPartitions = new HashSet < TopicPartition > ( ) ;
@ -402,6 +426,18 @@ public class KafkaConsumer implements Consumer {
@@ -402,6 +426,18 @@ public class KafkaConsumer implements Consumer {
this . metadataFetchTimeoutMs = config . getLong ( ConsumerConfig . METADATA_FETCH_TIMEOUT_CONFIG ) ;
this . totalMemorySize = config . getLong ( ConsumerConfig . TOTAL_BUFFER_MEMORY_CONFIG ) ;
List < InetSocketAddress > addresses = ClientUtils . parseAndValidateAddresses ( config . getList ( ConsumerConfig . BOOTSTRAP_SERVERS_CONFIG ) ) ;
if ( keyDeserializer = = null )
this . keyDeserializer = config . getConfiguredInstance ( ConsumerConfig . KEY_DESERIALIZER_CLASS_CONFIG ,
Deserializer . class ) ;
else
this . keyDeserializer = keyDeserializer ;
if ( valueDeserializer = = null )
this . valueDeserializer = config . getConfiguredInstance ( ConsumerConfig . VALUE_DESERIALIZER_CLASS_CONFIG ,
Deserializer . class ) ;
else
this . valueDeserializer = valueDeserializer ;
config . logUnused ( ) ;
log . debug ( "Kafka consumer started" ) ;
}
@ -488,7 +524,7 @@ public class KafkaConsumer implements Consumer {
@@ -488,7 +524,7 @@ public class KafkaConsumer implements Consumer {
* @return map of topic to records since the last fetch for the subscribed list of topics and partitions
* /
@Override
public Map < String , ConsumerRecords > poll ( long timeout ) {
public Map < String , ConsumerRecords < K , V > > poll ( long timeout ) {
// TODO Auto-generated method stub
return null ;
}