1. 我在jobmanager日志中并没有找到相关的日志信息;
2. 用的是zk,看当时的日志有如下报错:
2020-08-22 05:38:30,974 INFO
org.apache.zookeeper.server.PrepRequestProcessor: Got user-level
KeeperException when processing sessionid:0x26ea6955ea90581 type:create
cxid:0x32311f1 zxid:0x2079a5014 txntype:-1 reqpath:n/a Error
Path:/flink-bdp/application_1549925837808_10088957/leaderlatch/c05c969720b830770ffe0395aa37bf42/job_manager_lock
Error:KeeperErrorCode = NoNode for
/flink-bdp/application_1549925837808_10088957/leaderlatch/c05c969720b830770ffe0395aa37bf42/job_manager_lock
不过当时的zk服务是正常的,应该是这个jobmanager有问题。

3. 重新起一个是没有问题的,不过有问题的这个yarn-session就恢复不了了。

Xintong Song <tonysong...@gmail.com> 于2020年8月26日周三 上午9:39写道:

> >
> > 1. 怎么可以确认是leader丢失呢?
> >
> 看下是否能找到类似 "ResourceManager xxx was revoked leadership" 的日志
>
>
> > 2. 通常是什么原因造成的呢?比如网络延迟?或者机器负载过高?
>
> 网络原因是一种可能。另外也可能是 HA service 有问题,要看你集群用的是什么 HA(比如ZooKeeper),排查下 HA
> 的服务状态是否正常。
>
>
> > 3. 有什么办法可以恢复吗?
>
> 要根据具体原因才能知道如何恢复。
> 你现在是 yarn-session 上还有正在运行的作业吗?有可能试下停掉再重启 yarn-session 吗?
> 或者集群上是否有资源可以不停当前 session 再新起一个下看是否能正常工作?资源应该够的吧,这么长时间的话之前起的 TM 应该都释放了。
>
>
> Thank you~
>
> Xintong Song
>
>
>
> On Tue, Aug 25, 2020 at 7:05 PM song wang <sdlcwangson...@gmail.com>
> wrote:
>
> > 你好,
> > 现在yarn-session上还是可以提交新作业的。只是运行时无法分配slot,报错无法解析 resourcemanager 地址。
> >
> > 如果是RM leadership丢失的话,
> > 1. 怎么可以确认是leader丢失呢?
> > 2. 通常是什么原因造成的呢?比如网络延迟?或者机器负载过高?
> > 3. 有什么办法可以恢复吗?
> >
> > Xintong Song <tonysong...@gmail.com> 于2020年8月25日周二 下午5:26写道:
> >
> > > >
> > > > 出现这个报错后就提交不了任务了
> > > >
> > > 我确认一下,你之前这句话的意思,是出现这个报错之后,新的作业不能提交了,还是新的作业能提交但是提交之后页报这个找不到 RM 的错?
> > >
> > > 从 RM 心跳超时但是整个进程还在运行这个现象来看,比较符合 RM leadership 丢失的情况,这种情况下 RM 会停止服务。
> > > 如果是新的作业干脆就无法提交了,也符合 rest server leadership 丢失的情况。
> > >
> > > 我目前怀疑是 HA 出现问题,导致 RM 和 rest server 都认为自己不再是 leader,但是又迟迟没有新的 leader
> > 产生。所以对于
> > > JobMaster,由于没有发现有新的 RM leader,就会一直尝试重连原来的 RM,而对于 rest server 的表现则是找不到新的
> > > leader 无法提交新的作业。
> > >
> > > Thank you~
> > >
> > > Xintong Song
> > >
> > >
> > >
> > > On Tue, Aug 25, 2020 at 4:50 PM song wang <sdlcwangson...@gmail.com>
> > > wrote:
> > >
> > > > hi, Xintong:
> > > >
> > > > 我仔细查看了下日志,发现在报错"Could not resolve ResourceManager address"之前有如下日志:
> > > >
> > > > 2020-08-22 05:39:24,473 INFO
> > > org.apache.flink.runtime.jobmaster.JobMaster
> > > >                  - The heartbeat of ResourceManager with id
> > > > 6724e1ef8ee1c5fe5212eec6182319b6 timed out.
> > > > 2020-08-22 05:39:24,473 INFO
> > > org.apache.flink.runtime.jobmaster.JobMaster
> > > >                  - Close ResourceManager connection
> > > > 6724e1ef8ee1c5fe5212eec6182319b6: The heartbeat of ResourceManager
> with
> > > id
> > > > 6724e1ef8ee1c5fe5212eec6182319b6 timed out..
> > > >
> > > > 之后就一直报错 "Could not resolve ResourceManager address" 了,
> > > > 看了下flink 1.9.0 版本的代码,是在rpcService.connect() 时报的错,
> > > > 可是之后就没有日志输出了,单从报错信息来看只是说无法解析地址,
> > > > 可是resourcemanager地址是没有问题的。
> > > >
> > > > 请问有没有办法可以查看resourcemanager的健康状况呢?
> > > >
> > > >
> > > > // 代码:
> > > > package org.apache.flink.runtime.registration;
> > > > public abstract class RetryingRegistration<F extends Serializable, G
> > > > extends RpcGateway, S extends RegistrationResponse.Success> {
> > > >    public void startRegistration() {
> > > >          ...
> > > >          if (FencedRpcGateway.class.isAssignableFrom(targetType)) {
> > > >             rpcGatewayFuture = (CompletableFuture<G>)
> > rpcService.connect(
> > > >                targetAddress,
> > > >                fencingToken,
> > > >                targetType.asSubclass(FencedRpcGateway.class));
> > > >          } else {
> > > >             // 连接resourcemanager
> > > >             rpcGatewayFuture = rpcService.connect(targetAddress,
> > > > targetType);
> > > >          }
> > > >          ...
> > > >          rpcGatewayAcceptFuture.whenCompleteAsync(
> > > >             (Void v, Throwable failure) -> {
> > > >                if (failure != null && !canceled) {
> > > >                   final Throwable strippedFailure =
> > > > ExceptionUtils.stripCompletionException(failure);
> > > >                   if (log.isDebugEnabled()) {
> > > >                      ...
> > > >                   } else {
> > > >                      // 报错
> > > >                      log.info(
> > > >                         "Could not resolve {} address {}, retrying in
> > {}
> > > > ms:
> > > >
> > > >
> > >
> >
> {}.",targetName,targetAddress,retryingRegistrationConfiguration.getErrorDelayMillis(),strippedFailure.getMessage());
> > > >                   }
> > > >                   // 重连
> > > >
> > > >
> > > >
> > >
> >
> startRegistrationLater(retryingRegistrationConfiguration.getErrorDelayMillis());
> > > >                }
> > > >             },
> > > >             rpcService.getExecutor());
> > > >    }
> > > > }
> > > >
> > > >
> > > >
> > > >
> > >
> >
> ==================================================================================================================
> > > > 以下是一次提交job的完成报错日志
> > > >
> > > >
> > >
> >
> ==================================================================================================================
> > > > 2020-08-25 16:02:55,737 INFO
> > > >  org.apache.flink.runtime.dispatcher.StandaloneDispatcher      -
> > Received
> > > > JobGraph submission 4a324bc1e1eeb964116686e568cea8ad (Streaming
> > > WordCount).
> > > > 2020-08-25 16:02:55,738 INFO
> > > >  org.apache.flink.runtime.dispatcher.StandaloneDispatcher      -
> > > Submitting
> > > > job 4a324bc1e1eeb964116686e568cea8ad (Streaming WordCount).
> > > > 2020-08-25 16:02:56,552 INFO
> > > >
> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  -
> > > > Added SubmittedJobGraph(4a324bc1e1eeb964116686e568cea8ad) to
> ZooKeeper.
> > > > 2020-08-25 16:02:56,554 INFO
> > > >  org.apache.flink.runtime.rpc.akka.AkkaRpcService              -
> > Starting
> > > > RPC endpoint for org.apache.flink.runtime.jobmaster.JobMaster at
> > > > akka://flink/user/jobmanager_42 .
> > > > 2020-08-25 16:02:56,554 INFO
> > > org.apache.flink.runtime.jobmaster.JobMaster
> > > >                  - Initializing job Streaming WordCount
> > > > (4a324bc1e1eeb964116686e568cea8ad).
> > > > 2020-08-25 16:02:56,555 INFO
> > > org.apache.flink.runtime.jobmaster.JobMaster
> > > >                  - Using restart strategy
> > > > FailureRateRestartStrategy(failuresInterval=300000
> > msdelayInterval=10000
> > > > msmaxFailuresPerInterval=10) for Streaming WordCount
> > > > (4a324bc1e1eeb964116686e568cea8ad).
> > > > 2020-08-25 16:02:56,555 INFO
> > > >  org.apache.flink.runtime.executiongraph.ExecutionGraph        - Job
> > > > recovers via failover strategy: New Pipelined Region Failover
> > > > 2020-08-25 16:02:56,555 INFO
> > > org.apache.flink.runtime.jobmaster.JobMaster
> > > >                  - Running initialization on master for job Streaming
> > > > WordCount (4a324bc1e1eeb964116686e568cea8ad).
> > > > 2020-08-25 16:02:56,555 INFO
> > > org.apache.flink.runtime.jobmaster.JobMaster
> > > >                  - Successfully ran initialization on master in 0 ms.
> > > > 2020-08-25 16:02:56,556 INFO
> > > >
> > > >
> > >
> >
> org.apache.flink.runtime.executiongraph.failover.flip1.RestartPipelinedRegionStrategy
> > > >  - Start building failover regions.
> > > > 2020-08-25 16:02:56,556 INFO
> > > >
> > > >
> > >
> >
> org.apache.flink.runtime.executiongraph.failover.flip1.RestartPipelinedRegionStrategy
> > > >  - Created 1 failover regions.
> > > > 2020-08-25 16:02:56,559 INFO
> > > org.apache.flink.runtime.util.ZooKeeperUtils
> > > >                  - Initialized ZooKeeperCompletedCheckpointStore in
> > > > '/checkpoints/4a324bc1e1eeb964116686e568cea8ad'.
> > > > 2020-08-25 16:02:56,560 INFO
> > > org.apache.flink.runtime.jobmaster.JobMaster
> > > >                  - Loading state backend via factory
> > > > org.apache.flink.contrib.streaming.state.RocksDBStateBackendFactory
> > > > 2020-08-25 16:02:56,560 INFO
> > > >  org.apache.flink.contrib.streaming.state.RocksDBStateBackend  -
> Using
> > > > predefined options: DEFAULT.
> > > > 2020-08-25 16:02:56,560 INFO
> > > >  org.apache.flink.contrib.streaming.state.RocksDBStateBackend  -
> Using
> > > > default options factory:
> > > > DefaultConfigurableOptionsFactory{configuredOptions={}}.
> > > > 2020-08-25 16:02:56,609 INFO
> > > >
> > org.apache.flink.runtime.checkpoint.ZooKeeperCompletedCheckpointStore  -
> > > > Recovering checkpoints from ZooKeeper.
> > > > 2020-08-25 16:02:56,611 INFO
> > > >
> > org.apache.flink.runtime.checkpoint.ZooKeeperCompletedCheckpointStore  -
> > > > Found 0 checkpoints in ZooKeeper.
> > > > 2020-08-25 16:02:56,611 INFO
> > > >
> > org.apache.flink.runtime.checkpoint.ZooKeeperCompletedCheckpointStore  -
> > > > Trying to fetch 0 checkpoints from storage.
> > > > 2020-08-25 16:02:56,611 INFO
> > > >
> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService
> > > -
> > > > Starting ZooKeeperLeaderElectionService
> > > >
> > > >
> > >
> >
> ZooKeeperLeaderElectionService{leaderPath='/leader/4a324bc1e1eeb964116686e568cea8ad/job_manager_lock'}.
> > > > 2020-08-25 16:02:56,620 INFO
> > > >  org.apache.flink.runtime.jobmaster.JobManagerRunner           -
> > > JobManager
> > > > runner for job Streaming WordCount (4a324bc1e1eeb964116686e568cea8ad)
> > was
> > > > granted leadership with session id
> 5843c4ec-82d7-441f-befb-e4757c34f3f2
> > > at
> > > > akka.tcp://flink@hostname:16098/user/jobmanager_42.
> > > > 2020-08-25 16:02:56,623 INFO
> > > >
> > org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService
> > > >  - Starting ZooKeeperLeaderRetrievalService
> > > /leader/resource_manager_lock.
> > > > 2020-08-25 16:02:56,623 INFO
> > > org.apache.flink.runtime.jobmaster.JobMaster
> > > >                  - Starting execution of job Streaming WordCount
> > > > (4a324bc1e1eeb964116686e568cea8ad) under job master id
> > > > befbe4757c34f3f25843c4ec82d7441f.
> > > > 2020-08-25 16:02:56,623 INFO
> > > >  org.apache.flink.runtime.executiongraph.ExecutionGraph        - Job
> > > > Streaming WordCount (4a324bc1e1eeb964116686e568cea8ad) switched from
> > > state
> > > > CREATED to RUNNING.
> > > > 2020-08-25 16:02:56,623 INFO
> > > >  org.apache.flink.runtime.executiongraph.ExecutionGraph        -
> > Source:
> > > > Collection Source (1/1) (2b6a86725fe87a56e8706654205ac74c) switched
> > from
> > > > CREATED to SCHEDULED.
> > > > 2020-08-25 16:02:56,623 INFO
> > > >  org.apache.flink.runtime.jobmaster.slotpool.SlotPoolImpl      -
> Cannot
> > > > serve slot request, no ResourceManager connected. Adding as pending
> > > request
> > > > [SlotRequestId{a32583aa56cf8fdfd6c9b42fb2b60c6f}]
> > > > 2020-08-25 16:02:56,623 INFO
> > > >  org.apache.flink.runtime.executiongraph.ExecutionGraph        - Flat
> > Map
> > > > (1/4) (896ecc39967bd803dca4656262f0d499) switched from CREATED to
> > > > SCHEDULED.
> > > > 2020-08-25 16:02:56,623 INFO
> > > >  org.apache.flink.runtime.executiongraph.ExecutionGraph        - Flat
> > Map
> > > > (2/4) (081e8d6d45fc1d8b0c0311341039f03e) switched from CREATED to
> > > > SCHEDULED.
> > > > 2020-08-25 16:02:56,623 INFO
> > > >  org.apache.flink.runtime.executiongraph.ExecutionGraph        - Flat
> > Map
> > > > (3/4) (b02a3b8db2b07b4e54d6129d43189e17) switched from CREATED to
> > > > SCHEDULED.
> > > > 2020-08-25 16:02:56,623 INFO
> > > >  org.apache.flink.runtime.executiongraph.ExecutionGraph        - Flat
> > Map
> > > > (4/4) (c011b9d3544b434cf56dbd454a309644) switched from CREATED to
> > > > SCHEDULED.
> > > > 2020-08-25 16:02:56,623 INFO
> > > >  org.apache.flink.runtime.executiongraph.ExecutionGraph        -
> Keyed
> > > > Aggregation -> Sink: Print to Std. Out (1/4)
> > > > (b515b0f4a11f7e906f7258211492d8d5) switched from CREATED to
> SCHEDULED.
> > > > 2020-08-25 16:02:56,623 INFO
> > > >  org.apache.flink.runtime.executiongraph.ExecutionGraph        -
> Keyed
> > > > Aggregation -> Sink: Print to Std. Out (2/4)
> > > > (78b55fa685e6f9ac0db41627e95789a7) switched from CREATED to
> SCHEDULED.
> > > > 2020-08-25 16:02:56,623 INFO
> > > >  org.apache.flink.runtime.executiongraph.ExecutionGraph        -
> Keyed
> > > > Aggregation -> Sink: Print to Std. Out (3/4)
> > > > (cafa28075572c7cea6dce65fef14e110) switched from CREATED to
> SCHEDULED.
> > > > 2020-08-25 16:02:56,623 INFO
> > > >  org.apache.flink.runtime.executiongraph.ExecutionGraph        -
> Keyed
> > > > Aggregation -> Sink: Print to Std. Out (4/4)
> > > > (6914cb284564f53914aa2a14b9a9f667) switched from CREATED to
> SCHEDULED.
> > > > 2020-08-25 16:02:56,625 INFO
> > > org.apache.flink.runtime.jobmaster.JobMaster
> > > >                  - Connecting to ResourceManager
> > > akka.tcp://flink@hostname
> > > > :16098/user/resourcemanager(869831904fde985ece020bb52023471f)
> > > > 2020-08-25 16:02:57,483 ERROR
> > > >
> org.apache.flink.runtime.rest.handler.cluster.ClusterOverviewHandler  -
> > > > Unhandled exception.
> > > > akka.pattern.AskTimeoutException: Ask timed out on
> > > > [Actor[akka://flink/user/dispatcher#-1681919171]] after [10000 ms].
> > > Message
> > > > of type [org.apache.flink.runtime.rpc.messages.LocalFencedMessage]. A
> > > > typical reason for `AskTimeoutException` is that the recipient actor
> > > didn't
> > > > send a reply.
> > > >    at
> > akka.pattern.PromiseActorRef$$anonfun$2.apply(AskSupport.scala:635)
> > > >    at
> > akka.pattern.PromiseActorRef$$anonfun$2.apply(AskSupport.scala:635)
> > > >    at
> > > >
> > >
> >
> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:648)
> > > >    at akka.actor.Scheduler$$anon$4.run(Scheduler.scala:205)
> > > >    at
> > > >
> > > >
> > >
> >
> scala.concurrent.Future$InternalCallbackExecutor$.unbatchedExecute(Future.scala:601)
> > > >    at
> > > >
> > >
> >
> scala.concurrent.BatchingExecutor$class.execute(BatchingExecutor.scala:109)
> > > >    at
> > > >
> > >
> >
> scala.concurrent.Future$InternalCallbackExecutor$.execute(Future.scala:599)
> > > >    at
> > > >
> > > >
> > >
> >
> akka.actor.LightArrayRevolverScheduler$TaskHolder.executeTask(LightArrayRevolverScheduler.scala:328)
> > > >    at
> > > >
> > > >
> > >
> >
> akka.actor.LightArrayRevolverScheduler$$anon$4.executeBucket$1(LightArrayRevolverScheduler.scala:279)
> > > >    at
> > > >
> > > >
> > >
> >
> akka.actor.LightArrayRevolverScheduler$$anon$4.nextTick(LightArrayRevolverScheduler.scala:283)
> > > >    at
> > > >
> > > >
> > >
> >
> akka.actor.LightArrayRevolverScheduler$$anon$4.run(LightArrayRevolverScheduler.scala:235)
> > > >    at java.lang.Thread.run(Thread.java:748)
> > > > 2020-08-25 16:03:03,723 ERROR
> > > >
> org.apache.flink.runtime.rest.handler.cluster.ClusterOverviewHandler  -
> > > > Unhandled exception.
> > > > akka.pattern.AskTimeoutException: Ask timed out on
> > > > [Actor[akka://flink/user/dispatcher#-1681919171]] after [10000 ms].
> > > Message
> > > > of type [org.apache.flink.runtime.rpc.messages.LocalFencedMessage]. A
> > > > typical reason for `AskTimeoutException` is that the recipient actor
> > > didn't
> > > > send a reply.
> > > >    at
> > akka.pattern.PromiseActorRef$$anonfun$2.apply(AskSupport.scala:635)
> > > >    at
> > akka.pattern.PromiseActorRef$$anonfun$2.apply(AskSupport.scala:635)
> > > >    at
> > > >
> > >
> >
> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:648)
> > > >    at akka.actor.Scheduler$$anon$4.run(Scheduler.scala:205)
> > > >    at
> > > >
> > > >
> > >
> >
> scala.concurrent.Future$InternalCallbackExecutor$.unbatchedExecute(Future.scala:601)
> > > >    at
> > > >
> > >
> >
> scala.concurrent.BatchingExecutor$class.execute(BatchingExecutor.scala:109)
> > > >    at
> > > >
> > >
> >
> scala.concurrent.Future$InternalCallbackExecutor$.execute(Future.scala:599)
> > > >    at
> > > >
> > > >
> > >
> >
> akka.actor.LightArrayRevolverScheduler$TaskHolder.executeTask(LightArrayRevolverScheduler.scala:328)
> > > >    at
> > > >
> > > >
> > >
> >
> akka.actor.LightArrayRevolverScheduler$$anon$4.executeBucket$1(LightArrayRevolverScheduler.scala:279)
> > > >    at
> > > >
> > > >
> > >
> >
> akka.actor.LightArrayRevolverScheduler$$anon$4.nextTick(LightArrayRevolverScheduler.scala:283)
> > > >    at
> > > >
> > > >
> > >
> >
> akka.actor.LightArrayRevolverScheduler$$anon$4.run(LightArrayRevolverScheduler.scala:235)
> > > >    at java.lang.Thread.run(Thread.java:748)
> > > > 2020-08-25 16:03:06,646 INFO
> > > org.apache.flink.runtime.jobmaster.JobMaster
> > > >                  - Could not resolve ResourceManager address
> > > > akka.tcp://flink@hostname:16098/user/resourcemanager, retrying in
> > 10000
> > > > ms:
> > > > Ask timed out on [ActorSelection[Anchor(akka://flink/),
> > > > Path(/user/resourcemanager)]] after [10000 ms]. Message of type
> > > > [akka.actor.Identify]. A typical reason for `AskTimeoutException` is
> > that
> > > > the recipient actor didn't send a reply..
> > > >
> > > >
> > >
> >
> ==================================================================================================================
> > > >
> > > > Xintong Song <tonysong...@gmail.com> 于2020年8月25日周二 上午10:55写道:
> > > >
> > > > > 按你的描述,我理解是在 yarn 上起了一个 flink session,然后往这个 session
> > > > > 里提交作业,一开始能正常提交运行,后来再作业开始出现问题?
> > > > >
> > > > > 具体问题的原因还是要看日志才能确定,这个有可能是 ResourceManager,也有可能是 HA 或者 Akka
> > > 的问题,仅凭现象描述很难判断。
> > > > >
> > > > >
> > > > > Thank you~
> > > > >
> > > > > Xintong Song
> > > > >
> > > > >
> > > > >
> > > > > On Tue, Aug 25, 2020 at 10:23 AM song wang <
> sdlcwangson...@gmail.com
> > >
> > > > > wrote:
> > > > >
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> 你好,这个报错是任务启动了很长一段时间后才发生的,之前一直正常运行,出现这个报错后就提交不了任务了,感觉是ResourceManager出现了问题,ResourceManager有可能自己死掉吗?
> > > > > > 另外,这个现场还没有关掉,有什么办法排查下jobmaster和ResourceManager是否正常吗?
> > > > > >
> > > > > > Xintong Song <tonysong...@gmail.com> 于2020年8月25日周二 上午9:46写道:
> > > > > >
> > > > > > > 从日志上看是 JobMaster 连不上 ResourceManager。这两个组件应该是在同一个进程内的,通过 akka
> > > 进行本地通信。
> > > > > > > 需要看下完整的日志,RM 是否成功启动并注册到了 akka  的 actor system,以及注册的路径是否和 JM
> > > > 尝试连接的路径一致。
> > > > > > >
> > > > > > >
> > > > > > > Thank you~
> > > > > > >
> > > > > > > Xintong Song
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > On Mon, Aug 24, 2020 at 3:41 PM song wang <
> > > sdlcwangson...@gmail.com>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > 各位老哥, flink
> > > > > > > >
> > > > 运行在yarn上,偶尔报错无法解析ResourceManager地址,可是从对应的host上查找是有flink进程的,请问是什么原因呢?
> > > > > > > > flink 版本1.9.0
> > > > > > > >
> > > > > > > > 部分日志如下:
> > > > > > > > ```
> > > > > > > > 2020-08-24 15:11:31,566 INFO
> > > > > > > org.apache.flink.runtime.jobmaster.JobMaster
> > > > > > > >                  - Could not resolve ResourceManager address
> > > > > > > > akka.tcp://flink@hostname:16098/user/resourcemanager,
> retrying
> > > in
> > > > > > 10000
> > > > > > > > ms:
> > > > > > > > Ask timed out on [ActorSelection[Anchor(akka://flink/),
> > > > > > > > Path(/user/resourcemanager)]] after [10000 ms]. Message of
> type
> > > > > > > > [akka.actor.Identify]. A typical reason for
> > `AskTimeoutException`
> > > > is
> > > > > > that
> > > > > > > > the recipient actor didn't send a reply..
> > > > > > > > 2020-08-24 15:11:51,606 INFO
> > > > > > > org.apache.flink.runtime.jobmaster.JobMaster
> > > > > > > >                  - Could not resolve ResourceManager address
> > > > > > > > akka.tcp://flink@hostname:16098/user/resourcemanager,
> retrying
> > > in
> > > > > > 10000
> > > > > > > > ms:
> > > > > > > > Ask timed out on [ActorSelection[Anchor(akka://flink/),
> > > > > > > > Path(/user/resourcemanager)]] after [10000 ms]. Message of
> type
> > > > > > > > [akka.actor.Identify]. A typical reason for
> > `AskTimeoutException`
> > > > is
> > > > > > that
> > > > > > > > the recipient actor didn't send a reply..
> > > > > > > > 2020-08-24 15:12:11,645 INFO
> > > > > > > org.apache.flink.runtime.jobmaster.JobMaster
> > > > > > > >                  - Could not resolve ResourceManager address
> > > > > > > > akka.tcp://flink@hostname:16098/user/resourcemanager,
> retrying
> > > in
> > > > > > 10000
> > > > > > > > ms:
> > > > > > > > Ask timed out on [ActorSelection[Anchor(akka://flink/),
> > > > > > > > Path(/user/resourcemanager)]] after [10000 ms]. Message of
> type
> > > > > > > > [akka.actor.Identify]. A typical reason for
> > `AskTimeoutException`
> > > > is
> > > > > > that
> > > > > > > > the recipient actor didn't send a reply..
> > > > > > > > 2020-08-24 15:12:31,687 INFO
> > > > > > > org.apache.flink.runtime.jobmaster.JobMaster
> > > > > > > >                  - Could not resolve ResourceManager address
> > > > > > > > akka.tcp://flink@hostname:16098/user/resourcemanager,
> retrying
> > > in
> > > > > > 10000
> > > > > > > > ms:
> > > > > > > > Ask timed out on [ActorSelection[Anchor(akka://flink/),
> > > > > > > > Path(/user/resourcemanager)]] after [10000 ms]. Message of
> type
> > > > > > > > [akka.actor.Identify]. A typical reason for
> > `AskTimeoutException`
> > > > is
> > > > > > that
> > > > > > > > the recipient actor didn't send a reply..
> > > > > > > > 2020-08-24 15:12:51,727 INFO
> > > > > > > org.apache.flink.runtime.jobmaster.JobMaster
> > > > > > > >                  - Could not resolve ResourceManager address
> > > > > > > > akka.tcp://flink@hostname:16098/user/resourcemanager,
> retrying
> > > in
> > > > > > 10000
> > > > > > > > ms:
> > > > > > > > Ask timed out on [ActorSelection[Anchor(akka://flink/),
> > > > > > > > Path(/user/resourcemanager)]] after [10000 ms]. Message of
> type
> > > > > > > > [akka.actor.Identify]. A typical reason for
> > `AskTimeoutException`
> > > > is
> > > > > > that
> > > > > > > > the recipient actor didn't send a reply..
> > > > > > > > 2020-08-24 15:13:08,198 INFO
> > > > > > > >  org.apache.flink.runtime.executiongraph.ExecutionGraph
> > -
> > > > Job
> > > > > > > > Streaming WordCount (ff0ab7ec3e577a8e0c69e1c8454e5b72)
> switched
> > > > from
> > > > > > > state
> > > > > > > > RUNNING to FAILING.
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException:
> > > > > > > > Could not allocate all requires slots within timeout of
> 300000
> > > ms.
> > > > > > Slots
> > > > > > > > required: 9, slots allocated: 0, previous allocation IDs: [],
> > > > > execution
> > > > > > > > status: completed exceptionally:
> > > > > > > java.util.concurrent.CompletionException:
> > > > > > > > java.util.concurrent.CompletionException:
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> java.util.concurrent.TimeoutException/java.util.concurrent.CompletableFuture@28d7ad5d
> > > > > > > > [Completed
> > > > > > > > exceptionally], incomplete:
> > > > > > > java.util.concurrent.CompletableFuture@663cdf7e
> > > > > > > > [Not
> > > > > > > > completed, 1 dependents], incomplete:
> > > > > > > > java.util.concurrent.CompletableFuture@2058a7e9[Not
> > completed, 1
> > > > > > > > dependents], incomplete:
> > > > > > java.util.concurrent.CompletableFuture@5c1121c8
> > > > > > > > [Not
> > > > > > > > completed, 1 dependents], incomplete:
> > > > > > > > java.util.concurrent.CompletableFuture@49b9c252[Not
> > completed, 1
> > > > > > > > dependents], incomplete:
> > > > > > java.util.concurrent.CompletableFuture@497e3334
> > > > > > > > [Not
> > > > > > > > completed, 1 dependents], incomplete:
> > > > > > > > java.util.concurrent.CompletableFuture@2c7ca21d[Not
> > completed, 1
> > > > > > > > dependents], incomplete:
> > > > > > java.util.concurrent.CompletableFuture@7936c93b
> > > > > > > > [Not
> > > > > > > > completed, 1 dependents], incomplete:
> > > > > > > > java.util.concurrent.CompletableFuture@7e9a2f1d[Not
> > completed, 1
> > > > > > > > dependents]
> > > > > > > >    at
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.flink.runtime.executiongraph.SchedulingUtils.lambda$scheduleEager$1(SchedulingUtils.java:194)
> > > > > > > >    at
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> java.util.concurrent.CompletableFuture.uniExceptionally(CompletableFuture.java:870)
> > > > > > > >    at
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> java.util.concurrent.CompletableFuture$UniExceptionally.tryFire(CompletableFuture.java:852)
> > > > > > > >    at
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
> > > > > > > >    at
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)
> > > > > > > >    at
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.flink.runtime.concurrent.FutureUtils$ResultConjunctFuture.handleCompletedFuture(FutureUtils.java:633)
> > > > > > > >    at
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.flink.runtime.concurrent.FutureUtils$ResultConjunctFuture.lambda$new$0(FutureUtils.java:656)
> > > > > > > >    at
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:760)
> > > > > > > >    at
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:736)
> > > > > > > >    at
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
> > > > > > > >    at
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)
> > > > > > > >    at
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.flink.runtime.jobmaster.slotpool.SchedulerImpl.lambda$internalAllocateSlot$0(SchedulerImpl.java:190)
> > > > > > > >    at
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:760)
> > > > > > > >    at
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:736)
> > > > > > > >    at
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
> > > > > > > >    at
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)
> > > > > > > >    at
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.flink.runtime.jobmaster.slotpool.SlotSharingManager$SingleTaskSlot.release(SlotSharingManager.java:700)
> > > > > > > >    at
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.flink.runtime.jobmaster.slotpool.SlotSharingManager$MultiTaskSlot.release(SlotSharingManager.java:484)
> > > > > > > >    at
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.flink.runtime.jobmaster.slotpool.SlotSharingManager$MultiTaskSlot.lambda$new$0(SlotSharingManager.java:380)
> > > > > > > >    at
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> java.util.concurrent.CompletableFuture.uniHandle(CompletableFuture.java:822)
> > > > > > > >    at
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> java.util.concurrent.CompletableFuture$UniHandle.tryFire(CompletableFuture.java:797)
> > > > > > > >    at
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
> > > > > > > >    at
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)
> > > > > > > >    at
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.flink.runtime.concurrent.FutureUtils$Timeout.run(FutureUtils.java:998)
> > > > > > > >    at
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:397)
> > > > > > > >    at
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:190)
> > > > > > > >    at
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:74)
> > > > > > > >    at
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:152)
> > > > > > > >    at akka.japi.pf
> > > > .UnitCaseStatement.apply(CaseStatements.scala:26)
> > > > > > > >    at akka.japi.pf
> > > > .UnitCaseStatement.apply(CaseStatements.scala:21)
> > > > > > > >    at
> > > > > >
> scala.PartialFunction$class.applyOrElse(PartialFunction.scala:123)
> > > > > > > >    at akka.japi.pf
> > > > > > > .UnitCaseStatement.applyOrElse(CaseStatements.scala:21)
> > > > > > > >    at
> > > > > >
> scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:170)
> > > > > > > >    at
> > > > > >
> scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171)
> > > > > > > >    at
> > > > > >
> scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171)
> > > > > > > >    at akka.actor.Actor$class.aroundReceive(Actor.scala:517)
> > > > > > > >    at
> > > > akka.actor.AbstractActor.aroundReceive(AbstractActor.scala:225)
> > > > > > > >    at
> akka.actor.ActorCell.receiveMessage(ActorCell.scala:592)
> > > > > > > >    at akka.actor.ActorCell.invoke(ActorCell.scala:561)
> > > > > > > >    at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:258)
> > > > > > > >    at akka.dispatch.Mailbox.run(Mailbox.scala:225)
> > > > > > > >    at akka.dispatch.Mailbox.exec(Mailbox.scala:235)
> > > > > > > >    at
> > > > > akka.dispatch.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
> > > > > > > >    at
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> akka.dispatch.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
> > > > > > > >    at
> > > > > > >
> > > akka.dispatch.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
> > > > > > > >    at
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> akka.dispatch.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
> > > > > > > > ```
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

回复