public void setWaitingToEmit(ConsumerRecords<K, V> consumerRecords) { List<ConsumerRecord<K, V>> waitingToEmitList = new LinkedList<>(); for (TopicPartition tp : consumerRecords.partitions()) { waitingToEmitList.addAll(consumerRecords.records(tp)); } waitingToEmit = waitingToEmitList.iterator(); }
@Override public ConsumerRecords<K, V> onConsume(ConsumerRecords<K, V> records) { onConsumeCount++; if (throwExceptionOnConsume) throw new KafkaException("Injected exception in FilterConsumerInterceptor.onConsume."); // filters out topic/partitions with partition == FILTER_PARTITION Map<TopicPartition, List<ConsumerRecord<K, V>>> recordMap = new HashMap<>(); for (TopicPartition tp : records.partitions()) { if (tp.partition() != filterPartition) recordMap.put(tp, records.records(tp)); } return new ConsumerRecords<K, V>(recordMap); }
@Test public void testOnConsumeChain() { List<ConsumerInterceptor<Integer, Integer>> interceptorList = new ArrayList<>(); // we are testing two different interceptors by configuring the same interceptor differently, // which is not // how it would be done in KafkaConsumer, but ok for testing interceptor callbacks FilterConsumerInterceptor<Integer, Integer> interceptor1 = new FilterConsumerInterceptor<>(filterPartition1); FilterConsumerInterceptor<Integer, Integer> interceptor2 = new FilterConsumerInterceptor<>(filterPartition2); interceptorList.add(interceptor1); interceptorList.add(interceptor2); ConsumerInterceptors<Integer, Integer> interceptors = new ConsumerInterceptors<>(interceptorList); // verify that onConsumer modifies ConsumerRecords Map<TopicPartition, List<ConsumerRecord<Integer, Integer>>> records = new HashMap<>(); List<ConsumerRecord<Integer, Integer>> list1 = new ArrayList<>(); list1.add(consumerRecord); List<ConsumerRecord<Integer, Integer>> list2 = new ArrayList<>(); list2.add( new ConsumerRecord<>( filterTopicPart1.topic(), filterTopicPart1.partition(), 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, 1, 1)); List<ConsumerRecord<Integer, Integer>> list3 = new ArrayList<>(); list3.add( new ConsumerRecord<>( filterTopicPart2.topic(), filterTopicPart2.partition(), 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, 1, 1)); records.put(tp, list1); records.put(filterTopicPart1, list2); records.put(filterTopicPart2, list3); ConsumerRecords<Integer, Integer> consumerRecords = new ConsumerRecords<>(records); ConsumerRecords<Integer, Integer> interceptedRecords = interceptors.onConsume(consumerRecords); assertEquals(1, interceptedRecords.count()); assertTrue(interceptedRecords.partitions().contains(tp)); assertFalse(interceptedRecords.partitions().contains(filterTopicPart1)); assertFalse(interceptedRecords.partitions().contains(filterTopicPart2)); assertEquals(2, onConsumeCount); // verify that even if one of the intermediate interceptors throws an exception, all // interceptors' onConsume are called interceptor1.injectOnConsumeError(true); ConsumerRecords<Integer, Integer> partInterceptedRecs = interceptors.onConsume(consumerRecords); assertEquals(2, partInterceptedRecs.count()); assertTrue( partInterceptedRecs .partitions() .contains(filterTopicPart1)); // since interceptor1 threw exception assertFalse( partInterceptedRecs .partitions() .contains(filterTopicPart2)); // interceptor2 should still be called assertEquals(4, onConsumeCount); // if all interceptors throw an exception, records should be unmodified interceptor2.injectOnConsumeError(true); ConsumerRecords<Integer, Integer> noneInterceptedRecs = interceptors.onConsume(consumerRecords); assertEquals(noneInterceptedRecs, consumerRecords); assertEquals(3, noneInterceptedRecs.count()); assertEquals(6, onConsumeCount); interceptors.close(); }