关于有状态Flink作业改变Kafka Consumer UID后位点跳变到EARLIEST这档事

前言

这是很多Flink用户不太注意的一个隐藏得比较深的坑(严格来讲不算bug),近期组内同学频繁踩坑,故十分有必要快速记录一下,以提请注意。

复现问题

  • 用户意图:将有状态Flink流作业的部分或全部Source Topic重置消费位点,实现数据补录。

  • 操作步骤:

  1. 停止作业,在消息队列管理平台设定新位点;
  2. 改变新位点对应的Kafka Consumer的UID,使状态记录的旧位点失效;
  3. 从最新Checkpoint恢复作业。
  • 预期现象:作业成功恢复,并从预定的位点开始消费。

  • 实际现象:作业成功恢复,但是从对应topic的EARLIEST位点开始消费,数据大量积压。

下面图表中橙色折线为消费者的积压量,蓝色箭头为用户手动重置位点的时间,红色箭头为改动UID的任务恢复并完成第一次Checkpoint的时间,此时积压量几乎等于Topic消息总量。

另外,在TaskManager中也可发现类似如下的日志,说明实际恢复的位点是EARLIEST,而非用户指定的位点。

2025-02-12 18:49:32.643 INFO  [Source:: waybill_rou:r (7/20)#0] o.a.flink.streaming.connectors.kafka.FlinkKafkaConsumerBase - Consumer subtask 6 restored state: {}.

2025-02-13 18:49:34.714 INFO  [Legacy Source Thread: (7/20)#0] o.a.flink.streaming.connectors.kafka.FlinkKafkaConsumerBase - Consumer subtask 6 creating fetcher with offsets {KafkaTopicPartition{topic='my_topic', partition=38}=-915623761775}.

分析原因

我们知道,FlinkKafkaConsumer将位点信息储存在内部的OperatorState中,当FlinkKafkaConsumer实例初始化时,会一并初始化状态,并填充KafkaTopicPartition及其对应的Offset。代码如下。

// org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumerBase
    @Override
    public final void initializeState(FunctionInitializationContext context) throws Exception {

        OperatorStateStore stateStore = context.getOperatorStateStore();

        this.unionOffsetStates =
                stateStore.getUnionListState(
                        new ListStateDescriptor<>(
                                OFFSETS_STATE_NAME,
                                createStateSerializer(getRuntimeContext().getExecutionConfig())));

        if (context.isRestored()) {
            restoredState = new TreeMap<>(new KafkaTopicPartition.Comparator());

            // populate actual holder for restored state
            for (Tuple2<KafkaTopicPartition, Long> kafkaOffset : unionOffsetStates.get()) {
                restoredState.put(kafkaOffset.f0, kafkaOffset.f1);
            }

            LOG.info(
                    "Consumer subtask {} restored state: {}.",
                    getRuntimeContext().getIndexOfThisSubtask(),
                    restoredState);
        } else {
            LOG.info(
                    "Consumer subtask {} has no restore state.",
                    getRuntimeContext().getIndexOfThisSubtask());
        }
    }

当用户修改了FlinkKafkaConsumer算子的UID后,再从状态恢复作业,此时无法再取得unionOffsetStates,故恢复的状态容器restoredState是一个空的Map。这样又会导致Kafka Consumer启动时无法从restoredState取得任何Partition Offset信息,故将所有分区都填充为EARLIEST_OFFSET(见如下代码中标注感叹号的部分),引发问题。

    @Override
    public void open(Configuration configuration) throws Exception {
        // determine the offset commit mode
        this.offsetCommitMode =
                OffsetCommitModes.fromConfiguration(
                        getIsAutoCommitEnabled(),
                        enableCommitOnCheckpoints,
                        ((StreamingRuntimeContext) getRuntimeContext()).isCheckpointingEnabled());

        // create the partition discoverer
        this.partitionDiscoverer =
                createPartitionDiscoverer(
                        topicsDescriptor,
                        getRuntimeContext().getIndexOfThisSubtask(),
                        getRuntimeContext().getNumberOfParallelSubtasks());
        this.partitionDiscoverer.open();

        subscribedPartitionsToStartOffsets = new HashMap<>();
        final List<KafkaTopicPartition> allPartitions = partitionDiscoverer.discoverPartitions();
        if (restoredState != null) {
            for (KafkaTopicPartition partition : allPartitions) {
                // [!]
                if (!restoredState.containsKey(partition)) {
                    restoredState.put(partition, KafkaTopicPartitionStateSentinel.EARLIEST_OFFSET);
                }
            }

            for (Map.Entry<KafkaTopicPartition, Long> restoredStateEntry :
                    restoredState.entrySet()) {
                // seed the partition discoverer with the union state while filtering out
                // restored partitions that should not be subscribed by this subtask
                if (KafkaTopicPartitionAssigner.assign(
                                restoredStateEntry.getKey(),
                                getRuntimeContext().getNumberOfParallelSubtasks())
                        == getRuntimeContext().getIndexOfThisSubtask()) {
                    subscribedPartitionsToStartOffsets.put(
                            restoredStateEntry.getKey(), restoredStateEntry.getValue());
                }
            }

            if (filterRestoredPartitionsWithCurrentTopicsDescriptor) {
                // 移除此Sub-Task不需要再订阅的Partition,代码略
            }

            LOG.info(
                    "Consumer subtask {} will start reading {} partitions with offsets in restored state: {}",
                    getRuntimeContext().getIndexOfThisSubtask(),
                    subscribedPartitionsToStartOffsets.size(),
                    subscribedPartitionsToStartOffsets);
        } else {
            // use the partition discoverer to fetch the initial seed partitions,
            // and set their initial offsets depending on the startup mode.
            // for SPECIFIC_OFFSETS and TIMESTAMP modes, we set the specific offsets now;
            // for other modes (EARLIEST, LATEST, and GROUP_OFFSETS), the offset is lazily
            // determined
            // when the partition is actually read.
            // 以下根据不同的startupMode参数处理,代码略去
        }

        this.deserializer.open(
                RuntimeContextInitializationContextAdapters.deserializationAdapter(
                        getRuntimeContext(), metricGroup -> metricGroup.addGroup("user")));
    }

处理方法

最稳妥的方法当然是不要更改任何有状态作业的Source UID。

如果有强需求的话,我们可以新增一个Kafka Property参数(例如properties.restore-from-group-offsets),并修改FlinkKafkaConsumerBase#open()方法的逻辑,在UID改变且参数为true时,使用GROUP_OFFSET标记填充restoredState,而非默认的EARLIEST_OFFSET。简单方便。

The End

©著作权归作者所有,转载或内容合作请联系作者
平台声明:文章内容(如有图片或视频亦包括在内)由作者上传并发布,文章内容仅代表作者本人观点,简书系信息发布平台,仅提供信息存储服务。
  • 序言:七十年代末,一起剥皮案震惊了整个滨河市,随后出现的几起案子,更是在滨河造成了极大的恐慌,老刑警刘岩,带你破解...
    沈念sama阅读 228,119评论 6 531
  • 序言:滨河连续发生了三起死亡事件,死亡现场离奇诡异,居然都是意外死亡,警方通过查阅死者的电脑和手机,发现死者居然都...
    沈念sama阅读 98,382评论 3 415
  • 文/潘晓璐 我一进店门,熙熙楼的掌柜王于贵愁眉苦脸地迎上来,“玉大人,你说我怎么就摊上这事。” “怎么了?”我有些...
    开封第一讲书人阅读 176,038评论 0 373
  • 文/不坏的土叔 我叫张陵,是天一观的道长。 经常有香客问我,道长,这世上最难降的妖魔是什么? 我笑而不...
    开封第一讲书人阅读 62,853评论 1 309
  • 正文 为了忘掉前任,我火速办了婚礼,结果婚礼上,老公的妹妹穿的比我还像新娘。我一直安慰自己,他们只是感情好,可当我...
    茶点故事阅读 71,616评论 6 408
  • 文/花漫 我一把揭开白布。 她就那样静静地躺着,像睡着了一般。 火红的嫁衣衬着肌肤如雪。 梳的纹丝不乱的头发上,一...
    开封第一讲书人阅读 55,112评论 1 323
  • 那天,我揣着相机与录音,去河边找鬼。 笑死,一个胖子当着我的面吹牛,可吹牛的内容都是我干的。 我是一名探鬼主播,决...
    沈念sama阅读 43,192评论 3 441
  • 文/苍兰香墨 我猛地睁开眼,长吁一口气:“原来是场噩梦啊……” “哼!你这毒妇竟也来了?” 一声冷哼从身侧响起,我...
    开封第一讲书人阅读 42,355评论 0 288
  • 序言:老挝万荣一对情侣失踪,失踪者是张志新(化名)和其女友刘颖,没想到半个月后,有当地人在树林里发现了一具尸体,经...
    沈念sama阅读 48,869评论 1 334
  • 正文 独居荒郊野岭守林人离奇死亡,尸身上长有42处带血的脓包…… 初始之章·张勋 以下内容为张勋视角 年9月15日...
    茶点故事阅读 40,727评论 3 354
  • 正文 我和宋清朗相恋三年,在试婚纱的时候发现自己被绿了。 大学时的朋友给我发了我未婚夫和他白月光在一起吃饭的照片。...
    茶点故事阅读 42,928评论 1 369
  • 序言:一个原本活蹦乱跳的男人离奇死亡,死状恐怖,灵堂内的尸体忽然破棺而出,到底是诈尸还是另有隐情,我是刑警宁泽,带...
    沈念sama阅读 38,467评论 5 358
  • 正文 年R本政府宣布,位于F岛的核电站,受9级特大地震影响,放射性物质发生泄漏。R本人自食恶果不足惜,却给世界环境...
    茶点故事阅读 44,165评论 3 347
  • 文/蒙蒙 一、第九天 我趴在偏房一处隐蔽的房顶上张望。 院中可真热闹,春花似锦、人声如沸。这庄子的主人今日做“春日...
    开封第一讲书人阅读 34,570评论 0 26
  • 文/苍兰香墨 我抬头看了看天上的太阳。三九已至,却和暖如春,着一层夹袄步出监牢的瞬间,已是汗流浃背。 一阵脚步声响...
    开封第一讲书人阅读 35,813评论 1 282
  • 我被黑心中介骗来泰国打工, 没想到刚下飞机就差点儿被人妖公主榨干…… 1. 我叫王不留,地道东北人。 一个月前我还...
    沈念sama阅读 51,585评论 3 390
  • 正文 我出身青楼,却偏偏与公主长得像,于是被迫代替她去往敌国和亲。 传闻我的和亲对象是个残疾皇子,可洞房花烛夜当晚...
    茶点故事阅读 47,892评论 2 372

推荐阅读更多精彩内容