You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user-zh@flink.apache.org by 史 正超 <sh...@outlook.com> on 2020/11/13 01:59:48 UTC

回复: flink-1.11.2 执行checkpoint失败

从上面看是的。

public void handleJobLevelCheckpointException(CheckpointException exception, long checkpointId) {
   checkFailureCounter(exception, checkpointId);
   if (continuousFailureCounter.get() > tolerableCpFailureNumber) {
      clearCount();
      failureCallback.failJob(new FlinkRuntimeException("Exceeded checkpoint tolerable failure threshold."));
   }
}

大于阈值就报那个错了。
________________________________
发件人: 赵一旦 <hi...@gmail.com>
发送时间: 2020年11月13日 1:56
收件人: user-zh@flink.apache.org <us...@flink.apache.org>
主题: Re: flink-1.11.2 执行checkpoint失败

org.apache.flink.util.FlinkRuntimeException: Exceeded checkpoint tolerable
failure threshold.
顺着这个问个问题。 检查点“超时”是否计算进入checkpoint failure呢?

史 正超 <sh...@outlook.com> 于2020年11月12日周四 下午9:23写道:

> 执行checkpoint失败,报下面的错。
> 2020-11-12 21:04:56
> org.apache.flink.util.FlinkRuntimeException: Exceeded checkpoint tolerable
> failure threshold.
> at
> org.apache.flink.runtime.checkpoint.CheckpointFailureManager.handleJobLevelCheckpointException(CheckpointFailureManager.java:66)
> at
> org.apache.flink.runtime.checkpoint.CheckpointCoordinator.abortPendingCheckpoint(CheckpointCoordinator.java:1673)
> at
> org.apache.flink.runtime.checkpoint.CheckpointCoordinator.abortPendingCheckpoint(CheckpointCoordinator.java:1650)
> at
> org.apache.flink.runtime.checkpoint.CheckpointCoordinator.access$600(CheckpointCoordinator.java:91)
> at
> org.apache.flink.runtime.checkpoint.CheckpointCoordinator$CheckpointCanceller.run(CheckpointCoordinator.java:1783)
> at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
> at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> at java.lang.Thread.run(Thread.java:745)
>

Re: 回复: Re:回复: flink-1.11.2 执行checkpoint失败

Posted by smallwong <sm...@163.com>.
任务会一直重启吗?



--
Sent from: http://apache-flink.147419.n8.nabble.com/

回复: Re:回复: flink-1.11.2 执行checkpoint失败

Posted by 史 正超 <sh...@outlook.com>.
谢谢回复,我看了下我的任务,是背压导致的checkpoint超时,超时是没有异常日志打印的,每超时一次 就打印 org.apache.flink.util.FlinkRuntimeException: Exceeded checkpoint tolerable failure threshold.
这里的failure counter是 checkpoint的count。
任务是没有挂的,只是一直背压了。

________________________________
发件人: hailongwang <18...@163.com>
发送时间: 2020年11月13日 6:13
收件人: user-zh@flink.apache.org <us...@flink.apache.org>
主题: Re:回复: flink-1.11.2 执行checkpoint失败

据我所知,“超时”并不会导致 failure counter 加 1,也就是说“超时”并不是“错误”,或者说 Exception。
我觉得是否可以看下 checkpoint 抛了什么 exception 导致超过了最大可容能的数量(默认应该是有异常就会重启)
如果这个 Exception 是期望的或者因为 HDFS 等原因无法避免的话,那么可以适当加大 tolerableCpFailureNumber。

在 2020-11-13 09:13:34,"史 正超" <sh...@outlook.com> 写道:
>这是个思路,谢谢回复,我先试下。
>________________________________
>发件人: 赵一旦 <hi...@gmail.com>
>发送时间: 2020年11月13日 2:05
>收件人: user-zh@flink.apache.org <us...@flink.apache.org>
>主题: Re: flink-1.11.2 执行checkpoint失败
>
>如果超时也是错误的话,推进你设置容忍数量为INT最大值。因为有时候检查点失败或超时并不真正代表反压,至少我的生产中是这样。
>有部分情况,压力高,但刚刚好的情况下,会出现部分检查点失败的case,但实际压力刚刚能顶住。没必要因此导致任务失败。
>
>史 正超 <sh...@outlook.com> 于2020年11月13日周五 上午10:01写道:
>
>> 从上面看是的。
>>
>> public void handleJobLevelCheckpointException(CheckpointException
>> exception, long checkpointId) {
>>    checkFailureCounter(exception, checkpointId);
>>    if (continuousFailureCounter.get() > tolerableCpFailureNumber) {
>>       clearCount();
>>       failureCallback.failJob(new FlinkRuntimeException("Exceeded
>> checkpoint tolerable failure threshold."));
>>    }
>> }
>>
>> 大于阈值就报那个错了。
>> ________________________________
>> 发件人: 赵一旦 <hi...@gmail.com>
>> 发送时间: 2020年11月13日 1:56
>> 收件人: user-zh@flink.apache.org <us...@flink.apache.org>
>> 主题: Re: flink-1.11.2 执行checkpoint失败
>>
>> org.apache.flink.util.FlinkRuntimeException: Exceeded checkpoint tolerable
>> failure threshold.
>> 顺着这个问个问题。 检查点“超时”是否计算进入checkpoint failure呢?
>>
>> 史 正超 <sh...@outlook.com> 于2020年11月12日周四 下午9:23写道:
>>
>> > 执行checkpoint失败,报下面的错。
>> > 2020-11-12 21:04:56
>> > org.apache.flink.util.FlinkRuntimeException: Exceeded checkpoint
>> tolerable
>> > failure threshold.
>> > at
>> >
>> org.apache.flink.runtime.checkpoint.CheckpointFailureManager.handleJobLevelCheckpointException(CheckpointFailureManager.java:66)
>> > at
>> >
>> org.apache.flink.runtime.checkpoint.CheckpointCoordinator.abortPendingCheckpoint(CheckpointCoordinator.java:1673)
>> > at
>> >
>> org.apache.flink.runtime.checkpoint.CheckpointCoordinator.abortPendingCheckpoint(CheckpointCoordinator.java:1650)
>> > at
>> >
>> org.apache.flink.runtime.checkpoint.CheckpointCoordinator.access$600(CheckpointCoordinator.java:91)
>> > at
>> >
>> org.apache.flink.runtime.checkpoint.CheckpointCoordinator$CheckpointCanceller.run(CheckpointCoordinator.java:1783)
>> > at
>> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>> > at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>> > at
>> >
>> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
>> > at
>> >
>> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
>> > at
>> >
>> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>> > at
>> >
>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>> > at java.lang.Thread.run(Thread.java:745)
>> >
>>

Re:回复: flink-1.11.2 执行checkpoint失败

Posted by hailongwang <18...@163.com>.
据我所知,“超时”并不会导致 failure counter 加 1,也就是说“超时”并不是“错误”,或者说 Exception。
我觉得是否可以看下 checkpoint 抛了什么 exception 导致超过了最大可容能的数量(默认应该是有异常就会重启)
如果这个 Exception 是期望的或者因为 HDFS 等原因无法避免的话,那么可以适当加大 tolerableCpFailureNumber。

在 2020-11-13 09:13:34,"史 正超" <sh...@outlook.com> 写道:
>这是个思路,谢谢回复,我先试下。
>________________________________
>发件人: 赵一旦 <hi...@gmail.com>
>发送时间: 2020年11月13日 2:05
>收件人: user-zh@flink.apache.org <us...@flink.apache.org>
>主题: Re: flink-1.11.2 执行checkpoint失败
>
>如果超时也是错误的话,推进你设置容忍数量为INT最大值。因为有时候检查点失败或超时并不真正代表反压,至少我的生产中是这样。
>有部分情况,压力高,但刚刚好的情况下,会出现部分检查点失败的case,但实际压力刚刚能顶住。没必要因此导致任务失败。
>
>史 正超 <sh...@outlook.com> 于2020年11月13日周五 上午10:01写道:
>
>> 从上面看是的。
>>
>> public void handleJobLevelCheckpointException(CheckpointException
>> exception, long checkpointId) {
>>    checkFailureCounter(exception, checkpointId);
>>    if (continuousFailureCounter.get() > tolerableCpFailureNumber) {
>>       clearCount();
>>       failureCallback.failJob(new FlinkRuntimeException("Exceeded
>> checkpoint tolerable failure threshold."));
>>    }
>> }
>>
>> 大于阈值就报那个错了。
>> ________________________________
>> 发件人: 赵一旦 <hi...@gmail.com>
>> 发送时间: 2020年11月13日 1:56
>> 收件人: user-zh@flink.apache.org <us...@flink.apache.org>
>> 主题: Re: flink-1.11.2 执行checkpoint失败
>>
>> org.apache.flink.util.FlinkRuntimeException: Exceeded checkpoint tolerable
>> failure threshold.
>> 顺着这个问个问题。 检查点“超时”是否计算进入checkpoint failure呢?
>>
>> 史 正超 <sh...@outlook.com> 于2020年11月12日周四 下午9:23写道:
>>
>> > 执行checkpoint失败,报下面的错。
>> > 2020-11-12 21:04:56
>> > org.apache.flink.util.FlinkRuntimeException: Exceeded checkpoint
>> tolerable
>> > failure threshold.
>> > at
>> >
>> org.apache.flink.runtime.checkpoint.CheckpointFailureManager.handleJobLevelCheckpointException(CheckpointFailureManager.java:66)
>> > at
>> >
>> org.apache.flink.runtime.checkpoint.CheckpointCoordinator.abortPendingCheckpoint(CheckpointCoordinator.java:1673)
>> > at
>> >
>> org.apache.flink.runtime.checkpoint.CheckpointCoordinator.abortPendingCheckpoint(CheckpointCoordinator.java:1650)
>> > at
>> >
>> org.apache.flink.runtime.checkpoint.CheckpointCoordinator.access$600(CheckpointCoordinator.java:91)
>> > at
>> >
>> org.apache.flink.runtime.checkpoint.CheckpointCoordinator$CheckpointCanceller.run(CheckpointCoordinator.java:1783)
>> > at
>> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>> > at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>> > at
>> >
>> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
>> > at
>> >
>> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
>> > at
>> >
>> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>> > at
>> >
>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>> > at java.lang.Thread.run(Thread.java:745)
>> >
>>

回复: flink-1.11.2 执行checkpoint失败

Posted by 史 正超 <sh...@outlook.com>.
这是个思路,谢谢回复,我先试下。
________________________________
发件人: 赵一旦 <hi...@gmail.com>
发送时间: 2020年11月13日 2:05
收件人: user-zh@flink.apache.org <us...@flink.apache.org>
主题: Re: flink-1.11.2 执行checkpoint失败

如果超时也是错误的话,推进你设置容忍数量为INT最大值。因为有时候检查点失败或超时并不真正代表反压,至少我的生产中是这样。
有部分情况,压力高,但刚刚好的情况下,会出现部分检查点失败的case,但实际压力刚刚能顶住。没必要因此导致任务失败。

史 正超 <sh...@outlook.com> 于2020年11月13日周五 上午10:01写道:

> 从上面看是的。
>
> public void handleJobLevelCheckpointException(CheckpointException
> exception, long checkpointId) {
>    checkFailureCounter(exception, checkpointId);
>    if (continuousFailureCounter.get() > tolerableCpFailureNumber) {
>       clearCount();
>       failureCallback.failJob(new FlinkRuntimeException("Exceeded
> checkpoint tolerable failure threshold."));
>    }
> }
>
> 大于阈值就报那个错了。
> ________________________________
> 发件人: 赵一旦 <hi...@gmail.com>
> 发送时间: 2020年11月13日 1:56
> 收件人: user-zh@flink.apache.org <us...@flink.apache.org>
> 主题: Re: flink-1.11.2 执行checkpoint失败
>
> org.apache.flink.util.FlinkRuntimeException: Exceeded checkpoint tolerable
> failure threshold.
> 顺着这个问个问题。 检查点“超时”是否计算进入checkpoint failure呢?
>
> 史 正超 <sh...@outlook.com> 于2020年11月12日周四 下午9:23写道:
>
> > 执行checkpoint失败,报下面的错。
> > 2020-11-12 21:04:56
> > org.apache.flink.util.FlinkRuntimeException: Exceeded checkpoint
> tolerable
> > failure threshold.
> > at
> >
> org.apache.flink.runtime.checkpoint.CheckpointFailureManager.handleJobLevelCheckpointException(CheckpointFailureManager.java:66)
> > at
> >
> org.apache.flink.runtime.checkpoint.CheckpointCoordinator.abortPendingCheckpoint(CheckpointCoordinator.java:1673)
> > at
> >
> org.apache.flink.runtime.checkpoint.CheckpointCoordinator.abortPendingCheckpoint(CheckpointCoordinator.java:1650)
> > at
> >
> org.apache.flink.runtime.checkpoint.CheckpointCoordinator.access$600(CheckpointCoordinator.java:91)
> > at
> >
> org.apache.flink.runtime.checkpoint.CheckpointCoordinator$CheckpointCanceller.run(CheckpointCoordinator.java:1783)
> > at
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> > at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> > at
> >
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
> > at
> >
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
> > at
> >
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> > at
> >
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> > at java.lang.Thread.run(Thread.java:745)
> >
>

Re: flink-1.11.2 执行checkpoint失败

Posted by 赵一旦 <hi...@gmail.com>.
如果超时也是错误的话,推进你设置容忍数量为INT最大值。因为有时候检查点失败或超时并不真正代表反压,至少我的生产中是这样。
有部分情况,压力高,但刚刚好的情况下,会出现部分检查点失败的case,但实际压力刚刚能顶住。没必要因此导致任务失败。

史 正超 <sh...@outlook.com> 于2020年11月13日周五 上午10:01写道:

> 从上面看是的。
>
> public void handleJobLevelCheckpointException(CheckpointException
> exception, long checkpointId) {
>    checkFailureCounter(exception, checkpointId);
>    if (continuousFailureCounter.get() > tolerableCpFailureNumber) {
>       clearCount();
>       failureCallback.failJob(new FlinkRuntimeException("Exceeded
> checkpoint tolerable failure threshold."));
>    }
> }
>
> 大于阈值就报那个错了。
> ________________________________
> 发件人: 赵一旦 <hi...@gmail.com>
> 发送时间: 2020年11月13日 1:56
> 收件人: user-zh@flink.apache.org <us...@flink.apache.org>
> 主题: Re: flink-1.11.2 执行checkpoint失败
>
> org.apache.flink.util.FlinkRuntimeException: Exceeded checkpoint tolerable
> failure threshold.
> 顺着这个问个问题。 检查点“超时”是否计算进入checkpoint failure呢?
>
> 史 正超 <sh...@outlook.com> 于2020年11月12日周四 下午9:23写道:
>
> > 执行checkpoint失败,报下面的错。
> > 2020-11-12 21:04:56
> > org.apache.flink.util.FlinkRuntimeException: Exceeded checkpoint
> tolerable
> > failure threshold.
> > at
> >
> org.apache.flink.runtime.checkpoint.CheckpointFailureManager.handleJobLevelCheckpointException(CheckpointFailureManager.java:66)
> > at
> >
> org.apache.flink.runtime.checkpoint.CheckpointCoordinator.abortPendingCheckpoint(CheckpointCoordinator.java:1673)
> > at
> >
> org.apache.flink.runtime.checkpoint.CheckpointCoordinator.abortPendingCheckpoint(CheckpointCoordinator.java:1650)
> > at
> >
> org.apache.flink.runtime.checkpoint.CheckpointCoordinator.access$600(CheckpointCoordinator.java:91)
> > at
> >
> org.apache.flink.runtime.checkpoint.CheckpointCoordinator$CheckpointCanceller.run(CheckpointCoordinator.java:1783)
> > at
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> > at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> > at
> >
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
> > at
> >
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
> > at
> >
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> > at
> >
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> > at java.lang.Thread.run(Thread.java:745)
> >
>