参考资料:
SPARK-27281
KAFKA-7703
Kafka2.2.0多线程导致Offset获取错误问题复现
KAFKA-7703官方修复
KafkaConsumer的poll分析2之获取分区偏移量
前言
在之前的一篇踩坑文章SparkStreaming读取Kafka报OffsetOutOfRangeException中,还留下了一个未解之谜
查看spark日志发现报错如下:
java.lang.IllegalArgumentException: requirement failed: numRecords must not be negative
在后续的查找中,在Kafka的issue中发现一个可能是这个问题的原因KAFKA-7703,在此记录一下这个issue的复现和研究。
复现
首先,我们需要编译Kafka2.2.0的源码,这个我就不在此展开了,网上也有很多资料。我们在IDEA中打开Kafka2.2.0的源码,并在其中一些关键地方增加日志和线程代码。具体可以查看源码修改
本地启动Zookeeper,然后执行我们的测试方法repo
@Test
def repo(): Unit = {
val producer = createProducer()
for (_ <- 0 until 10) {
val record =
new ProducerRecord(tp.topic, tp.partition, null, "key".getBytes, "value".getBytes)
producer.send(record)
}
for (_ <- 0 until 10) {
val record =
new ProducerRecord(tp2.topic, tp2.partition, null, "key".getBytes, "value".getBytes)
producer.send(record)
}
producer.flush()
producer.close()
// The offset of each partition should be 10
val consumer = createConsumer()
consumer.subscribe(List(tp.topic()).asJava)
consumer.poll(0)
consumer.seekToEnd(List(tp, tp2).asJava)
val offset1 = consumer.position(tp)
val offset2 = consumer.position(tp2)
println(tp + ": " + offset1)
println(tp2 + ": " + offset2)
assert(offset1 == 10)
// This would fail because the "earliest" reset response triggered by `poll(0)` set it to 0.
assert(offset2 == 10)
consumer.close()
}
结果如下,可以明显看到topic-1的返回offset是错误的,明明topic-1最大offset应该是10,执行了seekToEnd
方法后,position()应该返回10才对,结果却返回了0。
ListOffsetResult: {topic-0=offset: 0}
ListOffsetResult: {topic-1=offset: 0}
start to reset
ListOffsetResult: {topic-0=offset: 10}
start sleep 5000 ms
ListOffsetResult: {topic-1=offset: 10}
topic-0: 10
topic-1: 0
问题分析
增加打印日志
前面的日志有点少,我们在源码中一些关键地方再增加一些日志,方便我们理解和观察
Fetcher.java
private void resetOffsetIfNeeded(TopicPartition partition, Long requestedResetTimestamp, OffsetData offsetData) {
// we might lose the assignment while fetching the offset, or the user might seek to a different offset,
// so verify it is still assigned and still in need of the requested reset
if (!subscriptions.isAssigned(partition)) {
log.debug("Skipping reset of partition {} since it is no longer assigned", partition);
} else if (!subscriptions.isOffsetResetNeeded(partition)) {
log.debug("Skipping reset of partition {} since reset is no longer needed", partition);
} else if (!requestedResetTimestamp.equals(offsetResetStrategyTimestamp(partition))) {
log.debug("Skipping reset of partition {} since an alternative reset has been requested", partition);
} else {
System.out.println("Moving " + partition + " to: " + offsetData);
if (partition.partition() == 1) {
try {
System.out.println("resetOffsetIfNeeded ===> "+Thread.currentThread().getName());
Thread.sleep(5000);
} catch (InterruptedException e) {
e.printStackTrace();
}
if (offsetData.offset > 0) {
// Make sure `position` returns before we change the offset to 10.
//System.out.println("Make sure `position` returns before we change the offset to 10.");
try {
Thread.sleep(5000);
} catch (InterruptedException e) {
e.printStackTrace();
}
}
}
log.info("Resetting offset for partition {} to offset {}.", partition, offsetData.offset);
System.out.println("Resetting offset for " + partition + " to: " + offsetData);
offsetData.leaderEpoch.ifPresent(epoch -> metadata.updateLastSeenEpochIfNewer(partition, epoch));
subscriptions.seek(partition, offsetData.offset);
System.out.println("Moved " + partition + " to: " + offsetData);
System.out.println("after seek:: subscriptions.isOffsetResetNeeded("+ partition+") is "+ subscriptions.isOffsetResetNeeded(partition));
System.out.println("after seek ===> "+Thread.currentThread().getName());
}
}
private void resetOffsetsAsync(Map<TopicPartition, Long> partitionResetTimestamps) {
System.out.println("resetOffsetsAsync===>"+Thread.currentThread().getName());
System.out.println("resetOffsetsAsync: " + partitionResetTimestamps);
// Add the topics to the metadata to do a single metadata fetch.
for (TopicPartition tp : partitionResetTimestamps.keySet())
metadata.add(tp.topic());
Map<Node, Map<TopicPartition, ListOffsetRequest.PartitionData>> timestampsToSearchByNode =
groupListOffsetRequests(partitionResetTimestamps, new HashSet<>());
for (Map.Entry<Node, Map<TopicPartition, ListOffsetRequest.PartitionData>> entry : timestampsToSearchByNode.entrySet()) {
Node node = entry.getKey();
final Map<TopicPartition, ListOffsetRequest.PartitionData> resetTimestamps = entry.getValue();
subscriptions.setResetPending(resetTimestamps.keySet(), time.milliseconds() + requestTimeoutMs);
RequestFuture<ListOffsetResult> future = sendListOffsetRequest(node, resetTimestamps, false);
future.addListener(new RequestFutureListener<ListOffsetResult>() {
@Override
public void onSuccess(ListOffsetResult result) {
System.out.println("ListOffsetResult: " + result.fetchedOffsets);
System.out.println("onSuccess===>"+Thread.currentThread().getName());
if (!result.partitionsToRetry.isEmpty()) {
subscriptions.resetFailed(result.partitionsToRetry, time.milliseconds() + retryBackoffMs);
metadata.requestUpdate();
}
for (Map.Entry<TopicPartition, OffsetData> fetchedOffset : result.fetchedOffsets.entrySet()) {
TopicPartition partition = fetchedOffset.getKey();
OffsetData offsetData = fetchedOffset.getValue();
ListOffsetRequest.PartitionData requestedReset = resetTimestamps.get(partition);
System.out.println("before resetOffsetIfNeeded:: subscriptions.isOffsetResetNeeded("+ partition+") is "+ subscriptions.isOffsetResetNeeded(partition));
resetOffsetIfNeeded(partition, requestedReset.timestamp, offsetData);
}
}
});
}
}
KafkaConsumer.java
private ConsumerRecords<K, V> poll(final Timer timer, final boolean includeMetadataInTimeout) {
acquireAndEnsureOpen();
try {
...
if (includeMetadataInTimeout) {
System.out.println("poll1");
if (!updateAssignmentMetadataIfNeeded(timer)) {
return ConsumerRecords.empty();
}
} else {
System.out.println("poll2");
while (!updateAssignmentMetadataIfNeeded(time.timer(Long.MAX_VALUE))) {
log.warn("Still waiting for metadata");
}
}
...
}
@Override
public void seekToEnd(Collection<TopicPartition> partitions) {
try {
Thread.sleep(2000);
System.out.println("start to reset");
} catch (InterruptedException e) {
e.printStackTrace();
}
if (partitions == null)
throw new IllegalArgumentException("Partitions collection cannot be null");
acquireAndEnsureOpen();
try {
Collection<TopicPartition> parts = partitions.size() == 0 ? this.subscriptions.assignedPartitions() : partitions;
for (TopicPartition tp : parts) {
log.debug("Seeking to end of partition {}", tp);
System.out.println("Seeking to end of partition "+ tp);
subscriptions.requestOffsetReset(tp, OffsetResetStrategy.LATEST);
System.out.println("after seekToEnd:: subscriptions.isOffsetResetNeeded("+ tp+") is "+ subscriptions.isOffsetResetNeeded(tp));
}
} finally {
release();
}
}
@Override
public long position(TopicPartition partition, final Duration timeout) {
acquireAndEnsureOpen();
try {
if (!this.subscriptions.isAssigned(partition))
throw new IllegalStateException("You can only check the position for partitions assigned to this consumer.");
System.out.println("before position sleep, and partition.partition() is " + partition.partition());
if (partition.partition() == 1) {
// Wait to make sure the background offset request for earliest has finished and `offset` is not null.
try {
System.out.println("start sleep 5000 ms");
System.out.println("position===>"+Thread.currentThread().getName());
Thread.sleep(5000);
} catch (InterruptedException e) {
e.printStackTrace();
}
}
System.out.println("after position sleep");
Timer timer = time.timer(timeout);
do {
Long offset = this.subscriptions.position(partition);
System.out.println("position2====>"+Thread.currentThread().getName());
System.out.println("return offset :"+offset+", partition :"+ partition);
if (offset != null)
return offset;
System.out.println("start position updateFetchPositions of partition:"+partition);
updateFetchPositions(timer);
client.poll(timer);
} while (timer.notExpired());
throw new TimeoutException("Timeout of " + timeout.toMillis() + "ms expired before the position " +
"for partition " + partition + " could be determined");
} finally {
release();
}
}
增加日志打印后,我们再执行一次repo测试方法:
poll2
start poll updateFetchPositions
resetOffsetsAsync===>main
resetOffsetsAsync: {topic-0=-2, topic-1=-2}
ListOffsetResult: {topic-0=offset: 0}
onSuccess===>kafka-coordinator-heartbeat-thread | my-test
before resetOffsetIfNeeded:: subscriptions.isOffsetResetNeeded(topic-0) is true
Moving topic-0 to: offset: 0
Resetting offset for topic-0 to: offset: 0
Moved topic-0 to: offset: 0
after seek:: subscriptions.isOffsetResetNeeded(topic-0) is false
after seek ===> kafka-coordinator-heartbeat-thread | my-test
KafkaApis===>data-plane-kafka-request-handler-3
returning for topic-1: 0
ListOffsetResult: {topic-1=offset: 0}
onSuccess===>kafka-coordinator-heartbeat-thread | my-test
before resetOffsetIfNeeded:: subscriptions.isOffsetResetNeeded(topic-1) is true
Moving topic-1 to: offset: 0
resetOffsetIfNeeded ===> kafka-coordinator-heartbeat-thread | my-test
start to reset
Seeking to end of partition topic-0
after seekToEnd:: subscriptions.isOffsetResetNeeded(topic-0) is true
Seeking to end of partition topic-1
after seekToEnd:: subscriptions.isOffsetResetNeeded(topic-1) is true
before position sleep, and partition.partition() is 0
after position sleep
position2====>main
return offset :null, partition :topic-0
start position updateFetchPositions of partition:topic-0
resetOffsetsAsync===>main
resetOffsetsAsync: {topic-0=-1, topic-1=-1}
position2====>main
return offset :null, partition :topic-0
start position updateFetchPositions of partition:topic-0
position2====>main
return offset :null, partition :topic-0
start position updateFetchPositions of partition:topic-0
ListOffsetResult: {topic-0=offset: 10}
onSuccess===>main
before resetOffsetIfNeeded:: subscriptions.isOffsetResetNeeded(topic-0) is true
Moving topic-0 to: offset: 10
Resetting offset for topic-0 to: offset: 10
Moved topic-0 to: offset: 10
after seek:: subscriptions.isOffsetResetNeeded(topic-0) is false
after seek ===> main
position2====>main
return offset :10, partition :topic-0
before position sleep, and partition.partition() is 1
start sleep 5000 ms
position===>main
KafkaApis===>data-plane-kafka-request-handler-2
returning for topic-1: 10
Resetting offset for topic-1 to: offset: 0
Moved topic-1 to: offset: 0
after seek:: subscriptions.isOffsetResetNeeded(topic-1) is false
after seek ===> kafka-coordinator-heartbeat-thread | my-test
ListOffsetResult: {topic-1=offset: 10}
onSuccess===>kafka-coordinator-heartbeat-thread | my-test
before resetOffsetIfNeeded:: subscriptions.isOffsetResetNeeded(topic-1) is false
after position sleep
position2====>main
return offset :0, partition :topic-1
topic-0: 10
topic-1: 0
理顺执行流程
好的,在分析日志之前,我们要理顺一下测试类方法所调用的poll()
,seekToEnd()
,position()
三个方法的执行流程。seekToEnd的方法比较简单,就是修改了OffsetResetStrategy为LATEST subscriptions.requestOffsetReset(tp, OffsetResetStrategy.LATEST);
。下面是position()
和poll()
方法的流程,可以看出两个方法在后续执行上都是一致的。
KafkaConsumer.position() ---->
updateFetchPositions() ---->
Fetch.resetOffsetsIfNeeded() ---->
resetOffsetsAsync() ----->
future.onSuccess() ---->
Fetch.resetOffsetIfNeeded(partition,Timestamp,offsetData)
-----
KafkaConsumer.poll --->
KafkaConsumer.updateAssignmentMetadataIfNeeded ---->
updateFetchPositions() ---->
Fetch.resetOffsetsIfNeeded() ---->
resetOffsetsAsync() ----->
future.onSuccess() ---->
Fetch.resetOffsetIfNeeded(partition,Timestamp,offsetData)
我们顺着这个逻辑顺序理一遍,可以发现在最后设置position的位置是在Fetcher#resetOffsetIfNeeded()
方法中subscriptions.seek(partition, offsetData.offset);
。我们在这个位置的后面增加了“Moved”的日志,可以在最终日志中发现,只打印了3次“Moved”日志,分别是:
Moved topic-0 to: offset: 0
Moved topic-0 to: offset: 10
Moved topic-1 to: offset: 0
可以看出,我们最后应当"Moved topic-1 to offset 10"的日志并没有打印,也就是没有调用到。那这个到底是onSuccess()
请求未返回呢?还是在Fetcher.resetOffsetIfNeeded(partition,Timestamp,offsetData)
方法中判断失败呢?我们可以看到“ListOffsetResult”的日志是出现了4次,这说明onSuccess()请求返回了4次,那也就是说是resetOffsetIfNeeded
方法中判断为不需要重新seek offset。
Debug
于是我们通过debug模式运行repo测试方法。可以发现理应“Moved topic-1 to 10”的这一次onSuccess()请求,是在Fetcher.resetOffsetIfNeeded(partition,Timestamp,offsetData)
方法的以下判断中,判断为false,从而没有执行seek方法。
else if (!subscriptions.isOffsetResetNeeded(partition)) {
log.debug("Skipping reset of partition {} since reset is no longer needed", partition);
}
这里有个小技巧,因为是多线程debug,IDEA的断点默认是ALL模式,如果想要每个线程都能触发断点,需要把IDEA的断点设置为Thread模式。
我们跟进这个方法subscriptions.isOffsetResetNeeded(partition)
private boolean awaitingReset() {
return resetStrategy != null;
}
private OffsetResetStrategy resetStrategy;
public enum OffsetResetStrategy {
LATEST, EARLIEST, NONE
}
可以发现该方法是判断resetStrategy
属性是否为null,resetStrategy
是一个OffsetResetStrategy枚举类,枚举值就只有三种。这个resetStrategy
属性会在执行seek()方法的时候重置为null,在执行需要offset变更的时候重置为LATEST或者EARLIEST,比如我们执行的seekToEnd()方法,就会将该属性设置为LATEST。
于是,我们在一些关键调用的前后增加日志,去打印执行情况时这个判断是否成功。我们注意到以下这段日志:
Resetting offset for topic-1 to: offset: 0
Moved topic-1 to: offset: 0
after seek:: subscriptions.isOffsetResetNeeded(topic-1) is false
ListOffsetResult: {topic-1=offset: 10}
before resetOffsetIfNeeded:: subscriptions.isOffsetResetNeeded(topic-1) is false
在上一次执行“Moved topic-1 to 0”的seek()之后,subscriptions.isOffsetResetNeeded(topic-1)
的结果是false的。然后接下来执行“Moved topic-1 to 10”时,在执行seek()之前会判断subscriptions.isOffsetResetNeeded(topic-1)
的结果为false。从而就导致了“Moved topic-1 to 10”的不执行。
这个时候可能会有个疑问,为什么?
触发这个bug主要是因为Kafka去执行seek()方法或者requestOffsetReset()也就是修改resetStrategy属性时,没有上锁,从而导致了多线程对同一个partition的resetStrategy修改不可控。 我们增加打印线程的日志再执行一次测试类repo方法验证一下,以下是关键日志:
resetOffsetsAsync: {topic-0=-2, topic-1=-2}
onSuccess===>kafka-coordinator-heartbeat-thread | my-test
before resetOffsetIfNeeded:: subscriptions.isOffsetResetNeeded(topic-1) is true
Moving topic-1 to: offset: 0
resetOffsetIfNeeded ===> kafka-coordinator-heartbeat-thread | my-test
start to reset
Seeking to end of partition topic-0
after seekToEnd:: subscriptions.isOffsetResetNeeded(topic-0) is true
Seeking to end of partition topic-1
after seekToEnd:: subscriptions.isOffsetResetNeeded(topic-1) is true
position2====>main
resetOffsetsAsync===>main
resetOffsetsAsync: {topic-0=-1, topic-1=-1}
...
position2====>main
before position sleep, and partition.partition() is 1
start sleep 5000 ms
position===>main
Resetting offset for topic-1 to: offset: 0
Moved topic-1 to: offset: 0
after seek:: subscriptions.isOffsetResetNeeded(topic-1) is false
after seek ===> kafka-coordinator-heartbeat-thread | my-test
ListOffsetResult: {topic-1=offset: 10}
onSuccess===>kafka-coordinator-heartbeat-thread | my-test
before resetOffsetIfNeeded:: subscriptions.isOffsetResetNeeded(topic-1) is false
after position sleep
position2====>main
可以发现:
- 第一次 “Moved topic-1 to 0” 执行线程是
kafka-coordinator-heartbeat-thread | my-test
,然后在即将执行seek()方法之前,执行了sleep(5000)。 - 此时主线程
main
开始执行seekToEnd(),sleep(2000)并将resetStrategy
属性改为LATEST。接着执行postition()方法,然后sleep(5000)。 - 接着
kafka-coordinator-heartbeat-thread | my-test
会比main
线程更早苏醒,它将继续执行 “Moved topic-1 to 0” 的seek()方法,此时会将resetStrategy
属性改为null,然后结束。 main
线程醒来,执行 “Moved topic-1 to 10” 的onSuccess()方法,去判断resetStrategy
属性是否为null。发现为null,就不再执行后续的seek()方法了,也就是没有将offset设置为10。
如何修复
这个问题修复记录可以看KAFKA-7703官方修复,简单来说就是加锁,防止多个线程在修改TopicPartitionState类时,造成的不可控。
再简单一点,将Kafka-Client版本更新为2.3.0。