前往小程序,Get更优阅读体验!
立即前往
首页
学习
活动
专区
工具
TVP
发布
社区首页 >专栏 >Kafka 消费者旧版低级 API

Kafka 消费者旧版低级 API

作者头像
CoderJed
发布2018-09-13 10:30:40
1.4K0
发布2018-09-13 10:30:40
举报
文章被收录于专栏:Jed的技术阶梯Jed的技术阶梯

Kafka 消费者总共有 3 种 API,新版 API、旧版高级 API、旧版低级 API,新版 API 是在 kafka 0.9 版本后增加的,推荐使用新版 API,但由于旧版低级 API 可以对消息进行更加灵活的控制,所有在实际开发中使用的也较多,本文讨论消费者旧版低级 API 的基本使用。

旧版低级 API 处理以下场景更为方便:

  • 消息重复消费
  • 添加事务管理机制,保证 Exactly Once
  • 消费指定分区或者指定分区的某些片段

使用旧版低级 API的步骤:

  • 获取你要读取的topic的partition的元数据信息
  • 找到这个partition的leader节点,然后通过这个leader节点找到存有这个partition副本的节点
  • 构造消费请求,获取数据并处理
  • 手动管理偏移量
  • 识别并处理分区leader节点的改变

以下示例代码实现的功能是,指定主题和分区,从该分区的第一条记录开始读取数据,打印到控制台:

代码语言:javascript
复制
package com.bonc.rdpe.kafka110.consumer;

import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;

import kafka.api.FetchRequest;
import kafka.api.FetchRequestBuilder;
import kafka.api.PartitionOffsetRequestInfo;
import kafka.common.ErrorMapping;
import kafka.common.TopicAndPartition;
import kafka.javaapi.FetchResponse;
import kafka.javaapi.TopicMetadataRequest;
import kafka.javaapi.TopicMetadataResponse;
import kafka.javaapi.OffsetRequest;
import kafka.javaapi.OffsetResponse;
import kafka.javaapi.PartitionMetadata;
import kafka.javaapi.TopicMetadata;
import kafka.javaapi.consumer.SimpleConsumer;
import kafka.message.MessageAndOffset;

/**
 * @Title SimpleConsumerTest.java 
 * @Description Kafka旧版消费者API——低级API
 * @Author YangYunhe
 * @Date 2018-06-26 13:16:29
 */
public class SimpleConsumerTest {
    
    // broker list
    private static final String BROKER_LIST = "192.168.42.101,192.168.42.102,192.168.42.103";
    // 连接超时时间:1min
    private static final int TIME_OUT = 60 * 1000;
    // 读取消息缓存区大小:1M
    private static final int BUFFER_SIZE = 1024 * 1024;
    // 每次获取消息的条数
    private static final int FETCH_SIZE = 100000;
    // 发生错误时重试的次数
    private static final int RETRIES_TIME = 3;
    // 允许发生错误的最大次数
    private static final int MAX_ERROR_NUM = 3;
    
    /**
     * 获取指定主题指定分区的元数据
     */
    private PartitionMetadata fetchPartitionMetadata(List<String> brokerList, int port, String topic, int partitionId) {
        SimpleConsumer consumer = null;
        TopicMetadataRequest metadataRequest = null;
        TopicMetadataResponse metadataResponse = null;
        List<TopicMetadata> topicMetadatas = null;
        
        try{
            /*
             * 循环是因为不确定传入的partition的leader节点是哪个
             */
            for(String host : brokerList) {
                // 1. 构建一个消费者,它是获取元数据的执行者
                consumer = new SimpleConsumer(host, port, TIME_OUT, BUFFER_SIZE, "dev3-client-001");  // 最后一个参数是 clientId
                // 2. 构造请求主题元数据的 request
                metadataRequest = new TopicMetadataRequest(Arrays.asList(topic));
                // 3. 发送请求获取元数据
                try {
                    metadataResponse = consumer.send(metadataRequest);
                } catch (Exception e) {
                    System.out.println("get TopicMetadataResponse failed!");
                    e.printStackTrace();
                    continue;
                }
                // 4. 获取主题元数据列表
                topicMetadatas = metadataResponse.topicsMetadata();
                // 5. 提取主题元数据列表中指定分区的元数据信息
                for(TopicMetadata topicMetadata : topicMetadatas) {
                    for(PartitionMetadata partitionMetadata : topicMetadata.partitionsMetadata()) {
                        if(partitionMetadata.partitionId() != partitionId) {
                            continue;
                        } else {
                            return partitionMetadata;
                        }
                    }
                }
            }
        } catch (Exception e) {
            System.out.println("Fetch PartitionMetadata failed!");
            e.printStackTrace();
        } finally {
            if(consumer != null) {
                consumer.close();
            }
        }
        
        return null;
        
    }
    
    /**
     * 根据分区的元数据信息获取它的leader节点
     */
    private String getLeader(PartitionMetadata metadata) {
        if(metadata.leader() == null) {
            System.out.println("can not find partition" + metadata.partitionId() + "'s leader!");
            return null;
        }
        return metadata.leader().host();
    }
    
    /**
     * 重新寻找partition的leader节点的方法
     */
    private String findNewLeader(List<String> brokerList, String oldLeader, String topic, int partition, int port) throws Exception {
        for (int i = 0; i < 3; i++) {
            boolean goToSleep = false;
            PartitionMetadata metadata = fetchPartitionMetadata(brokerList, port, topic, partition);
            if (metadata == null) {
                goToSleep = true;
            } else if (metadata.leader() == null) {
                goToSleep = true;
            } else if (oldLeader.equalsIgnoreCase(metadata.leader().host()) && i == 0) {
                // 这里考虑到 zookeeper 还没有来得及重新选举 leader 或者在故障转移之前挂掉的 leader 又重新连接的情况
                goToSleep = true;
            } else {
                return metadata.leader().host();
            }
            if (goToSleep) {
                Thread.sleep(1000);
            }
        }
        System.out.println("Unable to find new leader after Broker failure!");
        throw new Exception("Unable to find new leader after Broker failure!");
    }
    
    /**
     * 获取指定主题指定分区的消费偏移量
     */
    private long getOffset(SimpleConsumer consumer, String topic, int partition, long beginTime, String clientName) {
        TopicAndPartition topicAndPartition = new TopicAndPartition(topic, partition);
        Map<TopicAndPartition, PartitionOffsetRequestInfo> requestInfo = new HashMap<>();
        /*
         * PartitionOffsetRequestInfo(beginTime, 1)用于配置获取offset的策略
         * beginTime有两个值可以取
         *     kafka.api.OffsetRequest.EarliestTime(),获取最开始的消费偏移量,不一定是0,因为segment会删除
         *     kafka.api.OffsetRequest.LatestTime(),获取最新的消费偏移量
         * 另一个参数 1 暂不清楚有什么意义
         */
        requestInfo.put(topicAndPartition, new PartitionOffsetRequestInfo(beginTime, 1));
        // 构造获取offset的请求
        OffsetRequest request = new OffsetRequest(requestInfo, kafka.api.OffsetRequest.CurrentVersion(), clientName);
        OffsetResponse response = consumer.getOffsetsBefore(request);
        if(response.hasError()) {
            System.out.println("get offset failed!" + response.errorCode(topic, partition));
            return -1;
        }
        long[] offsets = response.offsets(topic, partition);
        if(offsets == null || offsets.length == 0) {
            System.out.println("get offset failed! offsets is null");
            return -1;
        }
        return offsets[0];
    }
    
    /**
     * 处理数据的方法
     */
    public void consume(List<String> brokerList, int port, String topic, int partition) {
        SimpleConsumer consumer = null;
        try {
            // 1. 获取分区元数据信息
            PartitionMetadata metadata = fetchPartitionMetadata(brokerList, port, topic, partition);
            if(metadata == null) {
                System.out.println("can not find metadata!");
                return;
            }
            // 2. 找到分区的leader节点
            String leaderBroker = getLeader(metadata);
            String clientId = topic + "-" + partition + "-" + "client";
            
            // 3. 创建一个消费者用于消费消息
            consumer = new SimpleConsumer(leaderBroker, port, TIME_OUT, BUFFER_SIZE, clientId);
            
            // 4. 配置获取offset的策略为,获取分区最开始的消费偏移量
            long offset = getOffset(consumer, topic, partition, kafka.api.OffsetRequest.EarliestTime(), clientId);
            
            int errorCount = 0;
            FetchRequest request = null;
            FetchResponse response = null;
            
            while(offset > -1) {
                // 运行过程中,可能因为处理错误,把consumer置为 null,所以这里需要再实例化
                if(consumer == null) {
                    consumer = new SimpleConsumer(leaderBroker, port, TIME_OUT, BUFFER_SIZE, clientId);
                }
                // 5. 构建获取消息的request
                request = new FetchRequestBuilder().clientId(clientId).addFetch(topic, partition, offset, FETCH_SIZE).build();
                // 6. 获取响应并处理
                response = consumer.fetch(request);
                if(response.hasError()) {
                    errorCount ++;
                    if(errorCount > MAX_ERROR_NUM) {
                        break;
                    }
                    short errorCode = response.errorCode(topic, partition);
                    
                    if(ErrorMapping.OffsetOutOfRangeCode() == errorCode) {
                        // 如果是因为获取到的偏移量无效,那么应该重新获取
                        // 这里简单处理,改为获取最新的消费偏移量
                        offset = getOffset(consumer, topic, partition, kafka.api.OffsetRequest.LatestTime(), clientId);
                        continue;
                    } else if (ErrorMapping.OffsetsLoadInProgressCode() == errorCode) {
                        Thread.sleep(300000);
                        continue;
                    } else {
                        consumer.close();
                        consumer = null;
                        // 更新leader broker
                        leaderBroker = findNewLeader(brokerList, leaderBroker, topic, partition, port); 
                        continue;
                    }
                // 如果没有错误
                } else {
                    errorCount = 0;  // 清空错误记录
                    long fetchCount = 0;
                    // 处理消息
                    for(MessageAndOffset messageAndOffset : response.messageSet(topic, partition)) {
                        long currentOffset = messageAndOffset.offset();
                        if(currentOffset < offset) {
                            System.out.println("get an old offset[" + currentOffset + "], excepted offset is offset[" + offset + "]");
                            continue;
                        }
                        offset = messageAndOffset.nextOffset();
                        ByteBuffer payload = messageAndOffset.message().payload();
                        byte[] bytes = new byte[payload.limit()];
                        payload.get(bytes);
                        
                        // 把消息打印到控制台
                        System.out.println("message: " + new String(bytes, "UTF-8") + ", offset: " + messageAndOffset.offset());
                        
                        fetchCount++;
                    }
                    
                    if (fetchCount == 0) {
                        Thread.sleep(1000);
                    }
                    
                }
            }
            
        } catch (Exception e) {
            System.out.println("exception occurs when consume message");
            e.printStackTrace();
        } finally {
            if (consumer != null) {
                consumer.close();
            }
        }
    }
    
    public static void main(String[] args) {
        SimpleConsumerTest sct = new SimpleConsumerTest();
        sct.consume(Arrays.asList(BROKER_LIST.split(",")), 9092, "dev3-yangyunhe-topic001", 0);
    }
    
}

运行结果为:

代码语言:javascript
复制
message: t13229543255|13229543255|2017-10-04 05:04:19|2017-10-04 05:04:19|01001|x13229543255|75165|UaiOKGnr|wx|1003|0, offset: 0
message: t15554236866|15554236866|2017-10-04 20:50:36|2017-10-04 20:50:36|01001|x15554236866|66815|cLhDPEfl|wx|1001|0, offset: 1
message: t13053448010|13053448010|2018-04-01 18:22:27|2018-04-01 18:22:27|01001|x13053448010|67860|UTpzF05R|wx|1005|1, offset: 2
message: t13016064334|13016064334|2017-07-01 15:10:31|2017-07-01 15:10:31|01001|x13016064334|78549|nzbMDJXs|wx|1005|0, offset: 3
message: t13025257802|13025257802|2018-01-21 18:49:55|2018-01-21 18:49:55|01001|x13025257802|39067|zOHLBrje|wx|1003|0, offset: 4

......
本文参与 腾讯云自媒体分享计划,分享自作者个人站点/博客。
原始发表:2018.06.29 ,如有侵权请联系 cloudcommunity@tencent.com 删除

本文分享自 作者个人站点/博客 前往查看

如有侵权,请联系 cloudcommunity@tencent.com 删除。

本文参与 腾讯云自媒体分享计划  ,欢迎热爱写作的你一起参与!

评论
登录后参与评论
0 条评论
热度
最新
推荐阅读
领券
问题归档专栏文章快讯文章归档关键词归档开发者手册归档开发者手册 Section 归档