从外部重置一个运行中consumer group的消费进度
对于0.10.1以上版本的kafka, 如何从外部重置一个运行中的consumer group的进度呢?比如有一个控制台,可以主动重置任意消费组的消费进度重置到12小时之前, 而用户的程序可以保持运行状态,无需下线或重启。
需要这么几个步骤:
1. 加入这个group
2. 踢掉所有其它group memeber
3. try assign all TopicPartition to this client
4. commit offsets
5. leave group
其中第二步是为了让自己当上leader,当然有可能不需要踢掉其它所有成员就能当上leader(因为谁能当leader实际上是按hashmap的迭代次序来的)。
当上consumer group的leader以后,需要把所有partition assign给自己,这个需要一个特殊的PartitionAssignor。由于这个assignor的协议跟其它consumer group协议不同(但是也可以搞一个表面上协议相同,实际上逻辑不同的assignor),而cooridnator会阻止与当前leader使用的协议不同的成员加入,所以还是需要踢掉其它成员。
public class ExclusiveAssignor extends AbstractPartitionAssignor {
public interface Callback {
void onSuccess();
}
private static Logger LOGGER = LoggerFactory.getLogger(ExclusiveAssignor.class);
public static String NAME = "exclusive";
private String leaderId = null;
private Callback callback = null;
public void setLeaderId(String leaderId) {
this.leaderId = leaderId;
}
public void setCallBack(Callback callBack){this.callback = callBack;}
@Override
public String name() {
return NAME;
}
private Map<String, List<String>> consumersPerTopic(Map<String, List<String>> consumerMetadata) {
Map<String, List<String>> res = new HashMap<>();
for (Map.Entry<String, List<String>> subscriptionEntry : consumerMetadata.entrySet()) {
String consumerId = subscriptionEntry.getKey();
for (String topic : subscriptionEntry.getValue())
put(res, topic, consumerId);
}
return res;
}
@Override
public Map<String, List<TopicPartition>> assign(Map<String, Integer> partitionsPerTopic,
Map<String, List<String>> subscriptions) {
LOGGER.info("perform exclusive assign");
if(leaderId == null)
throw new IllegalArgumentException("leaderId should already been set before assign is called");
if(callback == null)
throw new IllegalArgumentException("callback should already been set before assign is called");
List<TopicPartition> allPartitions = new ArrayList<TopicPartition>();
partitionsPerTopic.forEach((topic, partitionNumber) -> {
for(int i=0; i < partitionNumber; i++)
allPartitions.add(new TopicPartition(topic, i));
});
Map<String, List<TopicPartition>> assignment = new HashMap<>();
for (String memberId : subscriptions.keySet()) {
assignment.put(memberId, new ArrayList<TopicPartition>());
if(memberId.equals(leaderId)){
assignment.get(memberId).addAll(allPartitions);
}
}
callback.onSuccess();
return assignment;
}
}
这个assignor需要知道leaderId是哪个,而leaderId可以在KafkaConsumer的
protected Map<String, ByteBuffer> performAssignment(String leaderId,
String assignmentStrategy,
Map<String, ByteBuffer> allSubscriptions)
中获取,所以还需要修改一下KafkaConsumer的代码,以确保这个KafkaConsumer的poll并不实际拉取消息,而只是执行commit。
驱逐其它member,可以使用AdminClient完成
def forceLeave(coordinator: Node, memberId: String, groupId: String) = {
logger.info(s"forcing group member: $memberId to leave group: $groupId ")
send(coordinator, ApiKeys.LEAVE_GROUP, new LeaveGroupRequest(groupId, memberId))
}
最终的逻辑就是
private def forceCommit(consumer: SimpleKafkaConsumer[_, _], groupId: String, topics: Seq[String], maxRetries: Int, toCommit: Map[TopicPartition, OffsetAndMetadata], coordinatorOpt: Option[Node] = None) = {
consumer.subscribe(JavaConversions.seqAsJavaList(topics))
val assignedAll = new AtomicBoolean(false)
consumer.setExclusiveAssignorCallback(new Callback {
override def onSuccess(): Unit = assignedAll.set(true)
})
var currentRetries = 0
val coordinatorNode = coordinatorOpt.getOrElse(adminClient.findCoordinator(groupId))
while (!assignedAll.get() && currentRetries < maxRetries) {
logger.info(s"trying to reset offset for $groupId, retry count $currentRetries ....")
clearCurrentMembers(coordinatorNode, groupId, Some(ConsumerGroupManager.magicConsumerId))
consumer.poll(5000)
printCurrentAssignment(consumer)
currentRetries = currentRetries + 1
}
if (currentRetries >= maxRetries)
throw new RuntimeException(s"retry exhausted when getting leadership of $groupId")
val javaOffsetToCommit = JavaConversions.mapAsJavaMap(toCommit)
consumer.commitSync(javaOffsetToCommit)
logger.info(s"successfully committed offset for $groupId: $toCommit")
consumer.unsubscribe()
}
def forceReset(offsetLookupActor: ActorRef, groupId: String, ts: Long, maxRetries: Int)(implicit executionContext: ExecutionContext): Boolean = {
logger.info(s"resetting offset for $groupId to $ts")
val groupSummary = adminClient.describeConsumerGroup(groupId)
val topics = groupSummary.subscribedTopics
if (topics.isEmpty)
throw new IllegalStateException(s"group $groupId currently subscribed no topic")
val offsetToCommit = getOffsetsBehindTs(offsetLookupActor, topics, ts, 10000)
val consumer = createConsumer(groupId)
try {
forceCommit(consumer, groupId, topics, maxRetries, offsetToCommit)
true
} finally {
consumer.close()
}
}
具体代码见 https://github.com/iBuddha/kafka-simple-ui/blob/master/app/kafka/authorization/manager/utils/ConsumerGroupManager.scala
需要注意的是,发送LeaveGroupRequest可能会使得某些成员到broker的连接断掉,发生这种情况的原因是:当一个consumer发送JoinGroupRequest以后,外部的client再发送一个LeaveGroupRequest把这个consumer踢掉,会使得它个consumer无法收到JoinGroupResponse,从而使得NetworkClient以为连接挂掉。不过client以后会重新连接。而且,在外部client踢掉其它成员并且重新commit offset的过程中,其它consumer不一定有机会加入到group中,因而可能不受这个问题的影响。
从外部重置一个运行中consumer group的消费进度的更多相关文章
- 在linux下,查看一个运行中的程序, 占用了多少内存
1. 在linux下,查看一个运行中的程序, 占用了多少内存, 一般的命令有 (1). ps aux: 其中 VSZ(或VSS)列 表示,程序占用了多少虚拟内存. RSS列 表示, 程序占用了多少物 ...
- 在linux下,怎么去查看一个运行中的程序, 到底是占用了多少内存
1. 在linux下,查看一个运行中的程序, 占用了多少内存, 一般的命令有 (1). ps aux: 其中 VSZ(或VSS)列 表示,程序占用了多少虚拟内存. RSS列 表示, 程序占用了多少物 ...
- linux下,一个运行中的程序,究竟占用了多少内存
linux下,一个运行中的程序,究竟占用了多少内存 1. 在linux下,查看一个运行中的程序, 占用了多少内存, 一般的命令有 (1). ps aux: 其中 VSZ(或VSS)列 表示,程序占用 ...
- kafka中consumer group 是什么概念?
同样是逻辑上的概念,是Kafka实现单播和广播两种消息模型的手段.同一个topic的数据,会广播给不同的group:同一个group中的worker,只有一个worker能拿到这个数据.换句话说,对于 ...
- WINDOWS中, 如何查看一个运行中的程序是64位还是32位的
转自:https://blog.csdn.net/dayday3923/article/details/78597453?locationNum=7&fps=1 方法一: 任务管理器法任务管理 ...
- linux暂停一个在运行中的进程【转】
转自:https://blog.csdn.net/Tim_phper/article/details/53536621 转载于: http://www.cszhi.com/20120328/linux ...
- Kafka consumer group位移0ffset重设
本文阐述如何使用Kafka自带的kafka-consumer-groups.sh脚本随意设置消费者组(consumer group)的位移.需要特别强调的是, 这是0.11.0.0版本提供的新功能且只 ...
- Kafka设计解析(十九)Kafka consumer group位移重设
转载自 huxihx,原文链接 Kafka consumer group位移重设 本文阐述如何使用Kafka自带的kafka-consumer-groups.sh脚本随意设置消费者组(consumer ...
- Kafka consumer group位移重设
本文阐述如何使用Kafka自带的kafka-consumer-groups.sh脚本随意设置消费者组(consumer group)的位移.需要特别强调的是, 这是0.11.0.0版本提供的新功能且只 ...
随机推荐
- mybatis官网学习
javaType:一个 Java 类的完全限定名,或一个类型别名(参考上面内建类型别名 的列表) .如果你映射到一个 JavaBean,MyBatis 通常可以断定类型. 然而,如果你映射到的是 Ha ...
- selenium c# 的注意事项
http://chromedriver.storage.googleapis.com/index.html chromedriver的下载地址http://selenium-release.stora ...
- saltstack认证报错问题
认证报错 [root@saltstack01 ~]# salt '*' test.ping saltstack01: True saltstack03: Minion did not return. ...
- 《深入理解Android2》读书笔记(七)
接上篇<深入理解Android2>读书笔记(六) 广播接受者 注册 ContextImpl @Override public Intent registerReceiver(Broadca ...
- python 自定义过滤器
文件目录结构: 新建文件并且命名为“templatetags” , 然后复制 __init__.py文件,拷贝到templatetags文件夹里, __pycache__文件夹可以忽略哈,那是程序运行 ...
- Spring的事务传播性
事务是逻辑处理原子性的保证手段,通过使用事务控制,可以极大的避免出现逻辑处理失败导致的脏数据等问题.事务最重要的两个特性,是事务的传播级别和数据隔离级别.传播级别定义的是事务的控制范围,事务隔离级别定 ...
- [Codeforces #192] Tutorial
Link: Codeforces #192 传送门 前两天由于食物中毒现在还要每天挂一天的水 只好晚上回来随便找套题做做找找感觉了o(╯□╰)o A: 看到直接大力模拟了 但有一个更简便的方法,复杂度 ...
- 【递推】【组合数】【容斥原理】UVA - 11806 - Cheerleaders
http://www.cnblogs.com/khbcsu/p/4245943.html 本题如果直接枚举的话难度很大并且会无从下手.那么我们是否可以采取逆向思考的方法来解决问题呢?我们可以用总的情况 ...
- 20162312 2016-2017-2《Java程序设计》课程总结
一.每周作业链接汇总 预备作业01 写的是有关老师和学生的关系: 预备作业02 如何做中学: 预备作业03 实验楼学习linux环境: 第一周作业 java入门,虚拟机等课前准备: 第二周作业 掌握J ...
- 微信小程序-微信自动退款(Java后台)
微信小程序-微信自动退款 1.首先分享 微信自动退款接口: https://pay.weixin.qq.com/wiki/doc/api/jsapi.php?chapter=9_4 微信付款 代码案例 ...