Re: Issue with Flink UI for Flink 1.14.0

2022-03-18 Thread Chesnay Schepler

That issues tracked under https://issues.apache.org/jira/browse/FLINK-25904.
We don't yet know the cause.

On 18/03/2022 13:37, Peter Westermann wrote:


Just started testing Flink 1.14.4 since that fixes FLINK-25732 … and I 
am now running into another UI issue. On the jobmanager that is not 
currently the active jobmanager, the checkpoints tab is not working. 
It just displays “No Data“, when I use the corresponding API 
/jobs/{id}/checkpoints, I get the following error:


{"errors":["Internal server error.","side:\norg.apache.commons.math3.exception.NullArgumentException: input 
array\n\tat 
org.apache.commons.math3.util.MathArrays.verifyValues(MathArrays.java:1650)\n\tat 
org.apache.commons.math3.stat.descriptive.AbstractUnivariateStatistic.test(AbstractUnivariateStatistic.java:158)\n\tat 
org.apache.commons.math3.stat.descriptive.rank.Percentile.evaluate(Percentile.java:272)\n\tat 
org.apache.commons.math3.stat.descriptive.rank.Percentile.evaluate(Percentile.java:241)\n\tat 
org.apache.flink.runtime.metrics.DescriptiveStatisticsHistogramStatistics$CommonMetricsSnapshot.getPercentile(DescriptiveStatisticsHistogramStatistics.java:158)\n\tat 
org.apache.flink.runtime.metrics.DescriptiveStatisticsHistogramStatistics.getQuantile(DescriptiveStatisticsHistogramStatistics.java:52)\n\tat 
org.apache.flink.runtime.checkpoint.StatsSummarySnapshot.getQuantile(StatsSummarySnapshot.java:108)\n\tat 
org.apache.flink.runtime.rest.messages.checkpoints.StatsSummaryDto.valueOf(StatsSummaryDto.java:81)\n\tat 
org.apache.flink.runtime.rest.handler.job.checkpoints.CheckpointingStatisticsHandler.createCheckpointingStatistics(CheckpointingStatisticsHandler.java:129)\n\tat 
org.apache.flink.runtime.rest.handler.job.checkpoints.CheckpointingStatisticsHandler.handleRequest(CheckpointingStatisticsHandler.java:84)\n\tat 
org.apache.flink.runtime.rest.handler.job.checkpoints.CheckpointingStatisticsHandler.handleRequest(CheckpointingStatisticsHandler.java:58)\n\tat 
org.apache.flink.runtime.rest.handler.job.AbstractAccessExecutionGraphHandler.handleRequest(AbstractAccessExecutionGraphHandler.java:68)\n\tat 
org.apache.flink.runtime.rest.handler.job.AbstractExecutionGraphHandler.lambda$handleRequest$0(AbstractExecutionGraphHandler.java:87)\n\tat 
java.base/java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:642)\n\tat 
java.base/java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:478)\n\tat 
util.TokenAwareRunnable.run(TokenAwareRunnable.java:28)\n\tat 
java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)\n\tat 
java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)\n\tat 
java.base/java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:304)\n\tat 
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)\n\tat 
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)\n\tat 
java.base/java.lang.Thread.run(Thread.java:834)\n\nEnd of exception on 
server side>"]}


Peter Westermann

Analytics Software Architect

cidimage001.jpg@01D78D4C.C00AC080

peter.westerm...@genesys.com <mailto:peter.westerm...@genesys.com>

cidimage001.jpg@01D78D4C.C00AC080

cidimage002.jpg@01D78D4C.C00AC080 <http://www.genesys.com/>

*From: *Chesnay Schepler 
*Date: *Friday, January 21, 2022 at 3:28 AM
*To: *Peter Westermann , Dawid Wysakowicz 
, user@flink.apache.org 

*Subject: *Re: Issue with Flink UI for Flink 1.14.0

While FLINK-24550 was indeed fixed unfortunately a similar bug was 
also introduced (https://issues.apache.org/jira/browse/FLINK-25732).


On 20/01/2022 21:18, Peter Westermann wrote:

Just tried this again with Flink 1.14.3 since
https://issues.apache.org/jira/browse/FLINK-24550 is listed as
fixed. I am running into similar errors when calling the
/v1/jobs/overview endpoint (without any running jobs):

{"errors":["Internal server error.",""]}

Peter Westermann

Team Lead – Realtime Analytics

peter.westerm...@genesys.com <mailto:peter.westerm...@genesys.com>

<http://www.genesys.com/>

*From: *Dawid Wysakowicz 
<mailto:dwysakow...@apache.org>
*Date: *Thursday, October 14, 2021 at 10:00 AM
*To: *Peter Westermann 
<mailto:no.westerm...@genesys.com>, user@flink.apache.org
 <mailto:user@flink.apache.org>
*Subject: *Re: Issue with Flink UI for Flink 1.14.0

I am afraid it is a bug in flink 1.14. I created a ticket for it
FLINK-24550[1]. I believe we should pick it up soonish. Thanks for
reporting the issue!

Best,

Dawid

[1] https://issues.apache.org/jira/browse/FLINK-24550

On 13/10/2021 20:32, Peter Westermann wrote:

Hello,

I just started testing Flink 1.14.0 and noticed some weird
behavior. This i

Re: Issue with Flink UI for Flink 1.14.0

2022-03-18 Thread Peter Westermann
Just started testing Flink 1.14.4 since that fixes FLINK-25732 … and I am now 
running into another UI issue. On the jobmanager that is not currently the 
active jobmanager, the checkpoints tab is not working. It just displays “No 
Data“, when I use the corresponding API /jobs/{id}/checkpoints, I get the 
following error:
{"errors":["Internal server error.",""]}


Peter Westermann
Analytics Software Architect
[cidimage001.jpg@01D78D4C.C00AC080]
peter.westerm...@genesys.com<mailto:peter.westerm...@genesys.com>
[cidimage001.jpg@01D78D4C.C00AC080]
[cidimage002.jpg@01D78D4C.C00AC080]<http://www.genesys.com/>


From: Chesnay Schepler 
Date: Friday, January 21, 2022 at 3:28 AM
To: Peter Westermann , Dawid Wysakowicz 
, user@flink.apache.org 
Subject: Re: Issue with Flink UI for Flink 1.14.0

While FLINK-24550 was indeed fixed unfortunately a similar bug was also 
introduced 
(https://issues.apache.org/jira/browse/FLINK-25732<https://issues.apache.org/jira/browse/FLINK-25732>).
On 20/01/2022 21:18, Peter Westermann wrote:
Just tried this again with Flink 1.14.3 since 
https://issues.apache.org/jira/browse/FLINK-24550<https://issues.apache.org/jira/browse/FLINK-24550>
 is listed as fixed. I am running into similar errors when calling the 
/v1/jobs/overview endpoint (without any running jobs):
{"errors":["Internal server error.",""]}



Peter Westermann
Team Lead – Realtime Analytics
[cid:part1.0DsTQX9K.Qdi1tLpR@apache.org]
peter.westerm...@genesys.com<mailto:peter.westerm...@genesys.com>
[cid:part1.0DsTQX9K.Qdi1tLpR@apache.org]
[cid:part2.iLee7F9y.tfm6J3nj@apache.org]<http://www.genesys.com/>


From: Dawid Wysakowicz <mailto:dwysakow...@apache.org>
Date: Thursday, October 14, 2021 at 10:00 AM
To: Peter Westermann 
<mailto:no.westerm...@genesys.com>, 
user@flink.apache.org<mailto:user@flink.apache.org> 
<mailto:user@flink.apache.org>
Subject: Re: Issue with Flink UI for Flink 1.14.0

I am afraid it is a bug in flink 1.14. I created a ticket for it 
FLINK-24550[1]. I believe we should pick it up soonish. Thanks for reporting 
the issue!

Best,

Dawid

[1] 
https://issues.apache.org/jira/browse/FLINK-24550<https://issues.apache.org/jira/browse/FLINK-24550>
On 13/10/2021 20:32, Peter Westermann wrote:
Hello,

I just started testing Flink 1.14.0 and noticed some weird behavior. This is 
for a Flink cluster with zookeeper for HA and two job managers (one leader, one 
backup). The UI on the leader works fine. The UI on the other job manager does 
not load any job-specific data. Same applies to the REST interface. If I 
requests job data from /v1/jobs/{jobId}, I get the expected response on the 
leader but on the other job manager, I only get an exception stack trace:

{"errors":["Internal server error.",""]}


Peter Westermann
Team Lead – Realtime Analytics
[cidimage001.jpg@01D78D4C.C00AC080]
peter.westerm...@genesys.com<mailto:peter.westerm...@genesys.com>
[cidimage001.jpg@01D78D4C.C00AC080]
[cidimage002.jpg@01D78D4C.C00AC080]<http://www.genesys.com/>





Re: Issue with Flink UI for Flink 1.14.0

2022-02-10 Thread Guillaume Vauvert

Hi,

This issue is impacting all deployments with 2 JobManagers or more (HA 
mode), because in this case serialization is used (well, depending on 
the JobManager who is responding, the Leader or a Follower).


It prevents:

* usage of Flink UI

* usage of Flink command "flink.sh list"

* usage of Flink REST API "/jobs/overview"

There are some workaround for all impacts, but that is additional work, 
so impact is important.


Should it be possible to release sooner than "planned" ?

Thanks !

--

Guillaume

On 10/02/2022 11.35, Roman Khachatryan wrote:

Hi,

AFAIK there are no plans currently to release 1.14.4.
The previous one (1.14.3) was released on Jan 20, so I'd 
1.14.4 preparation to start in the next several weeks.


Regards,
Roman


On Tue, Feb 8, 2022 at 7:31 PM Sweta Kalakuntla 
 wrote:


I am facing the same issue, do we know when 1.14.4 will be released?

Thanks.

On Fri, Jan 21, 2022 at 3:28 AM Chesnay Schepler
 wrote:

While FLINK-24550 was indeed fixed unfortunately a similar bug
was also introduced
(https://issues.apache.org/jira/browse/FLINK-25732).

On 20/01/2022 21:18, Peter Westermann wrote:


Just tried this again with Flink 1.14.3 since
https://issues.apache.org/jira/browse/FLINK-24550 is listed
as fixed. I am running into similar errors when calling the
/v1/jobs/overview endpoint (without any running jobs):

{"errors":["Internal server error.",""]}

Peter Westermann

Team Lead – Realtime Analytics

cidimage001.jpg@01D78D4C.C00AC080

peter.westerm...@genesys.com
<mailto:peter.westerm...@genesys.com>

cidimage001.jpg@01D78D4C.C00AC080

cidimage002.jpg@01D78D4C.C00AC080 <http://www.genesys.com/>

*From: *Dawid Wysakowicz 
<mailto:dwysakow...@apache.org>
*Date: *Thursday, October 14, 2021 at 10:00 AM
*To: *Peter Westermann 
<mailto:no.westerm...@genesys.com>, user@flink.apache.org
     <mailto:user@flink.apache.org>
*Subject: *Re: Issue with Flink UI for Flink 1.14.0

I am afraid it is a bug in flink 1.14. I created a ticket for
it FLINK-24550[1]. I believe we should pick it up soonish.
Thanks for reporting the issue!

Best,

Dawid

[1] https://issues.apache.org/jira/browse/FLINK-24550

On 13/10/2021 20:32, Peter Westermann wrote:

Hello,

I just started testing Flink 1.14.0 and noticed some
weird behavior. This is for a Flink cluster with
zookeeper for HA and two job managers (one leader, one
backup). The UI on the leader works fine. The UI on the
other job manager does not load any job-specific data.
Same applies to the REST interface. If I requests job
data from /v1/jobs/{jobId}, I get the expected response
on the leader but on the other job manager, I only get an
exception stack trace:

{"errors":["Internal server error.",""]}

Peter Westermann

Team Lead – Realtime Analytics

cidimage001.jpg@01D78D4C.C00AC080

peter.westerm...@genesys.com
<mailto:peter.westerm...@genesys.com>

cidimage001.jpg@01D78D4C.C00AC080

cidimage002.jpg@01D78D4C.C00AC080 <http://www.genesys.com/>



Re: Issue with Flink UI for Flink 1.14.0

2022-02-10 Thread Roman Khachatryan
Westermann
>>
>> Team Lead – Realtime Analytics
>>
>> [image: cidimage001.jpg@01D78D4C.C00AC080]
>>
>> peter.westerm...@genesys.com
>>
>> [image: cidimage001.jpg@01D78D4C.C00AC080]
>>
>> [image: cidimage002.jpg@01D78D4C.C00AC080] <http://www.genesys.com/>
>>
>>
>>
>>
>>
>> *From: *Dawid Wysakowicz 
>> 
>> *Date: *Thursday, October 14, 2021 at 10:00 AM
>> *To: *Peter Westermann 
>> , user@flink.apache.org
>>  
>> *Subject: *Re: Issue with Flink UI for Flink 1.14.0
>>
>> I am afraid it is a bug in flink 1.14. I created a ticket for it
>> FLINK-24550[1]. I believe we should pick it up soonish. Thanks for
>> reporting the issue!
>>
>> Best,
>>
>> Dawid
>>
>> [1] https://issues.apache.org/jira/browse/FLINK-24550
>>
>> On 13/10/2021 20:32, Peter Westermann wrote:
>>
>> Hello,
>>
>>
>>
>> I just started testing Flink 1.14.0 and noticed some weird behavior. This
>> is for a Flink cluster with zookeeper for HA and two job managers (one
>> leader, one backup). The UI on the leader works fine. The UI on the other
>> job manager does not load any job-specific data. Same applies to the REST
>> interface. If I requests job data from /v1/jobs/{jobId}, I get the expected
>> response on the leader but on the other job manager, I only get an
>> exception stack trace:
>>
>>
>>
>> {"errors":["Internal server error.","> side:\njava.util.concurrent.CancellationException\n\tat
>> java.util.concurrent.CompletableFuture.cancel(CompletableFuture.java:2276)\n\tat
>> org.apache.flink.runtime.rest.handler.legacy.DefaultExecutionGraphCache.getExecutionGraphInternal(DefaultExecutionGraphCache.java:98)\n\tat
>> org.apache.flink.runtime.rest.handler.legacy.DefaultExecutionGraphCache.getExecutionGraphInfo(DefaultExecutionGraphCache.java:67)\n\tat
>> org.apache.flink.runtime.rest.handler.job.AbstractExecutionGraphHandler.handleRequest(AbstractExecutionGraphHandler.java:81)\n\tat
>> org.apache.flink.runtime.rest.handler.AbstractRestHandler.respondToRequest(AbstractRestHandler.java:83)\n\tat
>> org.apache.flink.runtime.rest.handler.AbstractHandler.respondAsLeader(AbstractHandler.java:195)\n\tat
>> org.apache.flink.runtime.rest.handler.LeaderRetrievalHandler.lambda$channelRead0$0(LeaderRetrievalHandler.java:83)\n\tat
>> java.util.Optional.ifPresent(Optional.java:159)\n\tat
>> org.apache.flink.util.OptionalConsumer.ifPresent(OptionalConsumer.java:45)\n\tat
>> org.apache.flink.runtime.rest.handler.LeaderRetrievalHandler.channelRead0(LeaderRetrievalHandler.java:80)\n\tat
>> org.apache.flink.runtime.rest.handler.LeaderRetrievalHandler.channelRead0(LeaderRetrievalHandler.java:49)\n\tat
>> org.apache.flink.shaded.netty4.io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:99)\n\tat
>> org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)\n\tat
>> org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)\n\tat
>> org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357)\n\tat
>> org.apache.flink.runtime.rest.handler.router.RouterHandler.routed(RouterHandler.java:115)\n\tat
>> org.apache.flink.runtime.rest.handler.router.RouterHandler.channelRead0(RouterHandler.java:94)\n\tat
>> org.apache.flink.runtime.rest.handler.router.RouterHandler.channelRead0(RouterHandler.java:55)\n\tat
>> org.apache.flink.shaded.netty4.io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:99)\n\tat
>> org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)\n\tat
>> org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)\n\tat
>> org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357)\n\tat
>> org.apache.flink.shaded.netty4.io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103)\n\tat
>> org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)\n\tat
>> org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)\n\tat
>> org.apache.flink

Re: Issue with Flink UI for Flink 1.14.0

2022-02-08 Thread Sweta Kalakuntla
I am facing the same issue, do we know when 1.14.4 will be released?

Thanks.

On Fri, Jan 21, 2022 at 3:28 AM Chesnay Schepler  wrote:

> While FLINK-24550 was indeed fixed unfortunately a similar bug was also
> introduced (https://issues.apache.org/jira/browse/FLINK-25732).
> On 20/01/2022 21:18, Peter Westermann wrote:
>
> Just tried this again with Flink 1.14.3 since
> https://issues.apache.org/jira/browse/FLINK-24550 is listed as fixed. I
> am running into similar errors when calling the /v1/jobs/overview endpoint
> (without any running jobs):
>
> {"errors":["Internal server error."," side:\norg.apache.flink.runtime.rpc.akka.exceptions.AkkaRpcException:
> Failed to serialize the result for RPC call :
> requestMultipleJobDetails.\n\tat
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.serializeRemoteResultAndVerifySize(AkkaRpcActor.java:417)\n\tat
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.lambda$sendAsyncResponse$2(AkkaRpcActor.java:373)\n\tat
> java.base/java.util.concurrent.CompletableFuture.uniHandle(CompletableFuture.java:930)\n\tat
> java.base/java.util.concurrent.CompletableFuture.uniHandleStage(CompletableFuture.java:946)\n\tat
> java.base/java.util.concurrent.CompletableFuture.handle(CompletableFuture.java:2266)\n\tat
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.sendAsyncResponse(AkkaRpcActor.java:365)\n\tat
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcInvocation(AkkaRpcActor.java:332)\n\tat
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:217)\n\tat
> org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:78)\n\tat
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:163)\n\tat
> akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:24)\n\tat
> akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:20)\n\tat
> scala.PartialFunction.applyOrElse(PartialFunction.scala:123)\n\tat
> scala.PartialFunction.applyOrElse$(PartialFunction.scala:122)\n\tat
> akka.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:20)\n\tat
> scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171)\n\tat
> scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:172)\n\tat
> scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:172)\n\tat
> akka.actor.Actor.aroundReceive(Actor.scala:537)\n\tat
> akka.actor.Actor.aroundReceive$(Actor.scala:535)\n\tat
> akka.actor.AbstractActor.aroundReceive(AbstractActor.scala:220)\n\tat
> akka.actor.ActorCell.receiveMessage(ActorCell.scala:580)\n\tat
> akka.actor.ActorCell.invoke(ActorCell.scala:548)\n\tat
> akka.dispatch.Mailbox.processMailbox(Mailbox.scala:270)\n\tat
> akka.dispatch.Mailbox.run(Mailbox.scala:231)\n\tat
> akka.dispatch.Mailbox.exec(Mailbox.scala:243)\n\tat
> java.base/java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:290)\n\tat
> java.base/java.util.concurrent.ForkJoinPool$WorkQueue.topLevelExec(ForkJoinPool.java:1020)\n\tat
> java.base/java.util.concurrent.ForkJoinPool.scan(ForkJoinPool.java:1656)\n\tat
> java.base/java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1594)\n\tat
> java.base/java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:183)\nCaused
> by: java.io.NotSerializableException: java.util.HashMap$Values\n\tat
> java.base/java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1185)\n\tat
> java.base/java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1553)\n\tat
> java.base/java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1510)\n\tat
> java.base/java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1433)\n\tat
> java.base/java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1179)\n\tat
> java.base/java.io.ObjectOutputStream.writeObject(ObjectOutputStream.java:349)\n\tat
> org.apache.flink.util.InstantiationUtil.serializeObject(InstantiationUtil.java:632)\n\tat
> org.apache.flink.runtime.rpc.akka.AkkaRpcSerializedValue.valueOf(AkkaRpcSerializedValue.java:66)\n\tat
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.serializeRemoteResultAndVerifySize(AkkaRpcActor.java:400)\n\t...
> 30 more\n\nEnd of exception on server side>"]}
>
>
>
>
>
>
>
> Peter Westermann
>
> Team Lead – Realtime Analytics
>
> [image: cidimage001.jpg@01D78D4C.C00AC080]
>
> peter.westerm...@genesys.com
>
> [image: cidimage001.jpg@01D78D4C.C00AC080]
>
> [image: cidimage002.jpg@01D78D4C.C00AC080] <http://www.genesys.com/>
>
>
>
>
>
> *From: *Dawid Wysakowicz  
> *Date: *Thursday, October 14, 2021 at 10:00 AM
> *To: *Peter Westermann 
> , user@flink.apache.org 
> 
> *Subject: *Re: Issue with Flink UI for Flink 1.14.0
>
> I am afrai

Re: Issue with Flink UI for Flink 1.14.0

2022-01-21 Thread Chesnay Schepler
While FLINK-24550 was indeed fixed unfortunately a similar bug was also 
introduced (https://issues.apache.org/jira/browse/FLINK-25732).


On 20/01/2022 21:18, Peter Westermann wrote:


Just tried this again with Flink 1.14.3 since 
https://issues.apache.org/jira/browse/FLINK-24550 is listed as fixed. 
I am running into similar errors when calling the /v1/jobs/overview 
endpoint (without any running jobs):


{"errors":["Internal server error.","side:\norg.apache.flink.runtime.rpc.akka.exceptions.AkkaRpcException: 
Failed to serialize the result for RPC call : 
requestMultipleJobDetails.\n\tat 
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.serializeRemoteResultAndVerifySize(AkkaRpcActor.java:417)\n\tat 
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.lambda$sendAsyncResponse$2(AkkaRpcActor.java:373)\n\tat 
java.base/java.util.concurrent.CompletableFuture.uniHandle(CompletableFuture.java:930)\n\tat 
java.base/java.util.concurrent.CompletableFuture.uniHandleStage(CompletableFuture.java:946)\n\tat 
java.base/java.util.concurrent.CompletableFuture.handle(CompletableFuture.java:2266)\n\tat 
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.sendAsyncResponse(AkkaRpcActor.java:365)\n\tat 
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcInvocation(AkkaRpcActor.java:332)\n\tat 
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:217)\n\tat 
org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:78)\n\tat 
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:163)\n\tat 
akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:24)\n\tat 
akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:20)\n\tat 
scala.PartialFunction.applyOrElse(PartialFunction.scala:123)\n\tat 
scala.PartialFunction.applyOrElse$(PartialFunction.scala:122)\n\tat 
akka.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:20)\n\tat 
scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171)\n\tat 
scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:172)\n\tat 
scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:172)\n\tat 
akka.actor.Actor.aroundReceive(Actor.scala:537)\n\tat 
akka.actor.Actor.aroundReceive$(Actor.scala:535)\n\tat 
akka.actor.AbstractActor.aroundReceive(AbstractActor.scala:220)\n\tat 
akka.actor.ActorCell.receiveMessage(ActorCell.scala:580)\n\tat 
akka.actor.ActorCell.invoke(ActorCell.scala:548)\n\tat 
akka.dispatch.Mailbox.processMailbox(Mailbox.scala:270)\n\tat 
akka.dispatch.Mailbox.run(Mailbox.scala:231)\n\tat 
akka.dispatch.Mailbox.exec(Mailbox.scala:243)\n\tat 
java.base/java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:290)\n\tat 
java.base/java.util.concurrent.ForkJoinPool$WorkQueue.topLevelExec(ForkJoinPool.java:1020)\n\tat 
java.base/java.util.concurrent.ForkJoinPool.scan(ForkJoinPool.java:1656)\n\tat 
java.base/java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1594)\n\tat 
java.base/java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:183)\nCaused 
by: java.io.NotSerializableException: java.util.HashMap$Values\n\tat 
java.base/java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1185)\n\tat 
java.base/java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1553)\n\tat 
java.base/java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1510)\n\tat 
java.base/java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1433)\n\tat 
java.base/java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1179)\n\tat 
java.base/java.io.ObjectOutputStream.writeObject(ObjectOutputStream.java:349)\n\tat 
org.apache.flink.util.InstantiationUtil.serializeObject(InstantiationUtil.java:632)\n\tat 
org.apache.flink.runtime.rpc.akka.AkkaRpcSerializedValue.valueOf(AkkaRpcSerializedValue.java:66)\n\tat 
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.serializeRemoteResultAndVerifySize(AkkaRpcActor.java:400)\n\t... 
30 more\n\nEnd of exception on server side>"]}


Peter Westermann

Team Lead – Realtime Analytics

cidimage001.jpg@01D78D4C.C00AC080

peter.westerm...@genesys.com <mailto:peter.westerm...@genesys.com>

cidimage001.jpg@01D78D4C.C00AC080

cidimage002.jpg@01D78D4C.C00AC080 <http://www.genesys.com/>

*From: *Dawid Wysakowicz 
*Date: *Thursday, October 14, 2021 at 10:00 AM
*To: *Peter Westermann , 
user@flink.apache.org 

*Subject: *Re: Issue with Flink UI for Flink 1.14.0

I am afraid it is a bug in flink 1.14. I created a ticket for it 
FLINK-24550[1]. I believe we should pick it up soonish. Thanks for 
reporting the issue!


Best,

Dawid

[1] https://issues.apache.org/jira/browse/FLINK-24550

On 13/10/2021 20:32, Peter Westermann wrote:

Hello,

I just started testing Flink 1.14.0 and noticed some weird
behavior. This is for a Flink cluster with zookeeper for HA and
two job managers (one leader, one backup). The UI on the leader
wor

Re: Issue with Flink UI for Flink 1.14.0

2022-01-20 Thread Peter Westermann
Just tried this again with Flink 1.14.3 since 
https://issues.apache.org/jira/browse/FLINK-24550 is listed as fixed. I am 
running into similar errors when calling the /v1/jobs/overview endpoint 
(without any running jobs):
{"errors":["Internal server error.",""]}



Peter Westermann
Team Lead – Realtime Analytics
[cidimage001.jpg@01D78D4C.C00AC080]
peter.westerm...@genesys.com<mailto:peter.westerm...@genesys.com>
[cidimage001.jpg@01D78D4C.C00AC080]
[cidimage002.jpg@01D78D4C.C00AC080]<http://www.genesys.com/>


From: Dawid Wysakowicz 
Date: Thursday, October 14, 2021 at 10:00 AM
To: Peter Westermann , user@flink.apache.org 

Subject: Re: Issue with Flink UI for Flink 1.14.0

I am afraid it is a bug in flink 1.14. I created a ticket for it 
FLINK-24550[1]. I believe we should pick it up soonish. Thanks for reporting 
the issue!

Best,

Dawid

[1] https://issues.apache.org/jira/browse/FLINK-24550
On 13/10/2021 20:32, Peter Westermann wrote:
Hello,

I just started testing Flink 1.14.0 and noticed some weird behavior. This is 
for a Flink cluster with zookeeper for HA and two job managers (one leader, one 
backup). The UI on the leader works fine. The UI on the other job manager does 
not load any job-specific data. Same applies to the REST interface. If I 
requests job data from /v1/jobs/{jobId}, I get the expected response on the 
leader but on the other job manager, I only get an exception stack trace:

{"errors":["Internal server error.",""]}


Peter Westermann
Team Lead – Realtime Analytics
[cidimage001.jpg@01D78D4C.C00AC080]
peter.westerm...@genesys.com<mailto:peter.westerm...@genesys.com>
[cidimage001.jpg@01D78D4C.C00AC080]
[cidimage002.jpg@01D78D4C.C00AC080]<http://www.genesys.com/>



Re: Issue with Flink UI for Flink 1.14.0

2021-10-14 Thread Dawid Wysakowicz
I am afraid it is a bug in flink 1.14. I created a ticket for it
FLINK-24550[1]. I believe we should pick it up soonish. Thanks for
reporting the issue!

Best,

Dawid

[1] https://issues.apache.org/jira/browse/FLINK-24550

On 13/10/2021 20:32, Peter Westermann wrote:
>
> Hello,
>
>  
>
> I just started testing Flink 1.14.0 and noticed some weird behavior.
> This is for a Flink cluster with zookeeper for HA and two job managers
> (one leader, one backup). The UI on the leader works fine. The UI on
> the other job manager does not load any job-specific data. Same
> applies to the REST interface. If I requests job data from
> /v1/jobs/{jobId}, I get the expected response on the leader but on the
> other job manager, I only get an exception stack trace:
>
>  
>
> {"errors":["Internal server error."," side:\njava.util.concurrent.CancellationException\n\tat
> java.util.concurrent.CompletableFuture.cancel(CompletableFuture.java:2276)\n\tat
> org.apache.flink.runtime.rest.handler.legacy.DefaultExecutionGraphCache.getExecutionGraphInternal(DefaultExecutionGraphCache.java:98)\n\tat
> org.apache.flink.runtime.rest.handler.legacy.DefaultExecutionGraphCache.getExecutionGraphInfo(DefaultExecutionGraphCache.java:67)\n\tat
> org.apache.flink.runtime.rest.handler.job.AbstractExecutionGraphHandler.handleRequest(AbstractExecutionGraphHandler.java:81)\n\tat
> org.apache.flink.runtime.rest.handler.AbstractRestHandler.respondToRequest(AbstractRestHandler.java:83)\n\tat
> org.apache.flink.runtime.rest.handler.AbstractHandler.respondAsLeader(AbstractHandler.java:195)\n\tat
> org.apache.flink.runtime.rest.handler.LeaderRetrievalHandler.lambda$channelRead0$0(LeaderRetrievalHandler.java:83)\n\tat
> java.util.Optional.ifPresent(Optional.java:159)\n\tat
> org.apache.flink.util.OptionalConsumer.ifPresent(OptionalConsumer.java:45)\n\tat
> org.apache.flink.runtime.rest.handler.LeaderRetrievalHandler.channelRead0(LeaderRetrievalHandler.java:80)\n\tat
> org.apache.flink.runtime.rest.handler.LeaderRetrievalHandler.channelRead0(LeaderRetrievalHandler.java:49)\n\tat
> org.apache.flink.shaded.netty4.io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:99)\n\tat
> org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)\n\tat
> org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)\n\tat
> org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357)\n\tat
> org.apache.flink.runtime.rest.handler.router.RouterHandler.routed(RouterHandler.java:115)\n\tat
> org.apache.flink.runtime.rest.handler.router.RouterHandler.channelRead0(RouterHandler.java:94)\n\tat
> org.apache.flink.runtime.rest.handler.router.RouterHandler.channelRead0(RouterHandler.java:55)\n\tat
> org.apache.flink.shaded.netty4.io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:99)\n\tat
> org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)\n\tat
> org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)\n\tat
> org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357)\n\tat
> org.apache.flink.shaded.netty4.io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103)\n\tat
> org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)\n\tat
> org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)\n\tat
> org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357)\n\tat
> org.apache.flink.runtime.rest.FileUploadHandler.channelRead0(FileUploadHandler.java:238)\n\tat
> org.apache.flink.runtime.rest.FileUploadHandler.channelRead0(FileUploadHandler.java:71)\n\tat
> org.apache.flink.shaded.netty4.io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:99)\n\tat
> org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)\n\tat
> org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)\n\tat
> org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357)\n\tat
> org.apache.flink.shaded.netty4.io.netty.channel.CombinedChannelDuplexHandler$DelegatingChannelHandlerContext.fireChannelRead(CombinedChannelDuplexHandler.java:436)\n\tat
> 

Issue with Flink UI for Flink 1.14.0

2021-10-13 Thread Peter Westermann
Hello,

I just started testing Flink 1.14.0 and noticed some weird behavior. This is 
for a Flink cluster with zookeeper for HA and two job managers (one leader, one 
backup). The UI on the leader works fine. The UI on the other job manager does 
not load any job-specific data. Same applies to the REST interface. If I 
requests job data from /v1/jobs/{jobId}, I get the expected response on the 
leader but on the other job manager, I only get an exception stack trace:

{"errors":["Internal server error.",""]}


Peter Westermann
Team Lead – Realtime Analytics
[cidimage001.jpg@01D78D4C.C00AC080]
peter.westerm...@genesys.com
[cidimage001.jpg@01D78D4C.C00AC080]
[cidimage002.jpg@01D78D4C.C00AC080]



Re: Root Exception can not be shown on Web UI in Flink 1.13.0

2021-05-19 Thread Matthias Pohl
Hi Gary,
Not sure whether you've seen my question in the Jira issue: May you be able
to share the overall JobManager/TaskManager logs with us? That would help
us understand the context a bit more on why no TaskManagerLocation was set.
Let's move any further correspondence into FLINK-22688 [1]

Best,
Matthias

[1] https://issues.apache.org/jira/browse/FLINK-22688

On Wed, May 19, 2021 at 5:45 AM Gary Wu  wrote:

> Thanks! I have updated the detail and task manager log in
> https://issues.apache.org/jira/browse/FLINK-22688.
>
> Regards,
> -Gary
>
> On Tue, 18 May 2021 at 16:22, Matthias Pohl 
> wrote:
>
>> Sorry, for not getting back earlier. I missed that thread. It looks like
>> some wrong assumption on our end. Hence, Yangze and Guowei are right. I'm
>> gonna look into the issue.
>>
>> Matthias
>>
>> On Fri, May 14, 2021 at 4:21 AM Guowei Ma  wrote:
>>
>>> Hi, Gary
>>>
>>> I think it might be a bug. So would you like to open a jira for this.
>>> And could you share the exception ,which the TaskManagerLocation is
>>> null? It might be very helpful to verify the cause.
>>>
>>> Best,
>>> Guowei
>>>
>>>
>>> On Thu, May 13, 2021 at 10:36 AM Yangze Guo  wrote:
>>>
 Hi, it seems to be related to FLINK-22276. Thus, I'd involve Matthias
 to take a look.

 @Matthias My gut feeling is that not all execution who has failureInfo
 has been deployed?

 Best,
 Yangze Guo

 On Wed, May 12, 2021 at 10:12 PM Gary Wu  wrote:
 >
 > Hi,
 >
 > We have upgraded our Flink applications to 1.13.0 but we found that
 Root Exception can not be shown on Web UI with an internal server error
 message. After opening browser development console and trace the message,
 we found that there is a exception in jobmanager:
 >
 > 2021-05-12 13:30:45,589 ERROR
 org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler [] -
 Unhandled exception.
 > java.lang.IllegalArgumentException: The location must not be null for
 a non-global failure.
 > at
 org.apache.flink.util.Preconditions.checkArgument(Preconditions.java:138)
 ~[flink-dist_2.12-1.13.0.jar:1.13.0]
 > at
 org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler.assertLocalExceptionInfo(JobExceptionsHandler.java:218)
 ~[flink-dist_2.12-1.13.0.jar:1.13.0]
 > at
 org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler.createRootExceptionInfo(JobExceptionsHandler.java:191)
 ~[flink-dist_2.12-1.13.0.jar:1.13.0]
 > at
 java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:195)
 ~[?:?]
 > at java.util.stream.SliceOps$1$1.accept(SliceOps.java:199) ~[?:?]
 > at
 java.util.ArrayList$ArrayListSpliterator.tryAdvance(ArrayList.java:1632)
 ~[?:?]
 > at
 java.util.stream.ReferencePipeline.forEachWithCancel(ReferencePipeline.java:127)
 ~[?:?]
 > at
 java.util.stream.AbstractPipeline.copyIntoWithCancel(AbstractPipeline.java:502)
 ~[?:?]
 > at
 java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:488) 
 ~[?:?]
 > at
 java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:474)
 ~[?:?]
 > at
 java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:913)
 ~[?:?]
 > at
 java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234) 
 ~[?:?]
 > at
 java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:578)
 ~[?:?]
 > at
 org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler.createJobExceptionHistory(JobExceptionsHandler.java:169)
 ~[flink-dist_2.12-1.13.0.jar:1.13.0]
 > at
 org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler.createJobExceptionsInfo(JobExceptionsHandler.java:154)
 ~[flink-dist_2.12-1.13.0.jar:1.13.0]
 > at
 org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler.handleRequest(JobExceptionsHandler.java:101)
 ~[flink-dist_2.12-1.13.0.jar:1.13.0]
 > at
 org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler.handleRequest(JobExceptionsHandler.java:63)
 ~[flink-dist_2.12-1.13.0.jar:1.13.0]
 > at
 org.apache.flink.runtime.rest.handler.job.AbstractExecutionGraphHandler.lambda$handleRequest$0(AbstractExecutionGraphHandler.java:87)
 ~[flink-dist_2.12-1.13.0.jar:1.13.0]
 > at
 java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:642)
 [?:?]
 > at
 java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:478)
 [?:?]
 > at
 java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
 [?:?]
 > at java.util.concurrent.FutureTask.run(FutureTask.java:264) [?:?]
 > at
 java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:304)
 [?:?]
 >

Re: Root Exception can not be shown on Web UI in Flink 1.13.0

2021-05-18 Thread Gary Wu
Thanks! I have updated the detail and task manager log in
https://issues.apache.org/jira/browse/FLINK-22688.

Regards,
-Gary

On Tue, 18 May 2021 at 16:22, Matthias Pohl  wrote:

> Sorry, for not getting back earlier. I missed that thread. It looks like
> some wrong assumption on our end. Hence, Yangze and Guowei are right. I'm
> gonna look into the issue.
>
> Matthias
>
> On Fri, May 14, 2021 at 4:21 AM Guowei Ma  wrote:
>
>> Hi, Gary
>>
>> I think it might be a bug. So would you like to open a jira for this.
>> And could you share the exception ,which the TaskManagerLocation is null?
>> It might be very helpful to verify the cause.
>>
>> Best,
>> Guowei
>>
>>
>> On Thu, May 13, 2021 at 10:36 AM Yangze Guo  wrote:
>>
>>> Hi, it seems to be related to FLINK-22276. Thus, I'd involve Matthias
>>> to take a look.
>>>
>>> @Matthias My gut feeling is that not all execution who has failureInfo
>>> has been deployed?
>>>
>>> Best,
>>> Yangze Guo
>>>
>>> On Wed, May 12, 2021 at 10:12 PM Gary Wu  wrote:
>>> >
>>> > Hi,
>>> >
>>> > We have upgraded our Flink applications to 1.13.0 but we found that
>>> Root Exception can not be shown on Web UI with an internal server error
>>> message. After opening browser development console and trace the message,
>>> we found that there is a exception in jobmanager:
>>> >
>>> > 2021-05-12 13:30:45,589 ERROR
>>> org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler [] -
>>> Unhandled exception.
>>> > java.lang.IllegalArgumentException: The location must not be null for
>>> a non-global failure.
>>> > at
>>> org.apache.flink.util.Preconditions.checkArgument(Preconditions.java:138)
>>> ~[flink-dist_2.12-1.13.0.jar:1.13.0]
>>> > at
>>> org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler.assertLocalExceptionInfo(JobExceptionsHandler.java:218)
>>> ~[flink-dist_2.12-1.13.0.jar:1.13.0]
>>> > at
>>> org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler.createRootExceptionInfo(JobExceptionsHandler.java:191)
>>> ~[flink-dist_2.12-1.13.0.jar:1.13.0]
>>> > at
>>> java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:195)
>>> ~[?:?]
>>> > at java.util.stream.SliceOps$1$1.accept(SliceOps.java:199) ~[?:?]
>>> > at
>>> java.util.ArrayList$ArrayListSpliterator.tryAdvance(ArrayList.java:1632)
>>> ~[?:?]
>>> > at
>>> java.util.stream.ReferencePipeline.forEachWithCancel(ReferencePipeline.java:127)
>>> ~[?:?]
>>> > at
>>> java.util.stream.AbstractPipeline.copyIntoWithCancel(AbstractPipeline.java:502)
>>> ~[?:?]
>>> > at
>>> java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:488) ~[?:?]
>>> > at
>>> java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:474)
>>> ~[?:?]
>>> > at
>>> java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:913)
>>> ~[?:?]
>>> > at
>>> java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234) ~[?:?]
>>> > at
>>> java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:578)
>>> ~[?:?]
>>> > at
>>> org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler.createJobExceptionHistory(JobExceptionsHandler.java:169)
>>> ~[flink-dist_2.12-1.13.0.jar:1.13.0]
>>> > at
>>> org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler.createJobExceptionsInfo(JobExceptionsHandler.java:154)
>>> ~[flink-dist_2.12-1.13.0.jar:1.13.0]
>>> > at
>>> org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler.handleRequest(JobExceptionsHandler.java:101)
>>> ~[flink-dist_2.12-1.13.0.jar:1.13.0]
>>> > at
>>> org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler.handleRequest(JobExceptionsHandler.java:63)
>>> ~[flink-dist_2.12-1.13.0.jar:1.13.0]
>>> > at
>>> org.apache.flink.runtime.rest.handler.job.AbstractExecutionGraphHandler.lambda$handleRequest$0(AbstractExecutionGraphHandler.java:87)
>>> ~[flink-dist_2.12-1.13.0.jar:1.13.0]
>>> > at
>>> java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:642)
>>> [?:?]
>>> > at
>>> java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:478)
>>> [?:?]
>>> > at
>>> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
>>> [?:?]
>>> > at java.util.concurrent.FutureTask.run(FutureTask.java:264) [?:?]
>>> > at
>>> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:304)
>>> [?:?]
>>> > at
>>> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
>>> [?:?]
>>> > at
>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
>>> [?:?]
>>> > at java.lang.Thread.run(Thread.java:834) [?:?]
>>> >
>>> > We would like to check Is there any configuration change should be
>>> done for the application? Thanks!
>>> >
>>> > Regards,
>>> > -Gary
>>> >
>>> >
>>> >
>>> > APPIER EMAIL NOTICE
>>> >
>>> > The contents of this email message and any attachments from Appier
>>> Group 

Re: Root Exception can not be shown on Web UI in Flink 1.13.0

2021-05-18 Thread Matthias Pohl
Sorry, for not getting back earlier. I missed that thread. It looks like
some wrong assumption on our end. Hence, Yangze and Guowei are right. I'm
gonna look into the issue.

Matthias

On Fri, May 14, 2021 at 4:21 AM Guowei Ma  wrote:

> Hi, Gary
>
> I think it might be a bug. So would you like to open a jira for this.
> And could you share the exception ,which the TaskManagerLocation is null?
> It might be very helpful to verify the cause.
>
> Best,
> Guowei
>
>
> On Thu, May 13, 2021 at 10:36 AM Yangze Guo  wrote:
>
>> Hi, it seems to be related to FLINK-22276. Thus, I'd involve Matthias
>> to take a look.
>>
>> @Matthias My gut feeling is that not all execution who has failureInfo
>> has been deployed?
>>
>> Best,
>> Yangze Guo
>>
>> On Wed, May 12, 2021 at 10:12 PM Gary Wu  wrote:
>> >
>> > Hi,
>> >
>> > We have upgraded our Flink applications to 1.13.0 but we found that
>> Root Exception can not be shown on Web UI with an internal server error
>> message. After opening browser development console and trace the message,
>> we found that there is a exception in jobmanager:
>> >
>> > 2021-05-12 13:30:45,589 ERROR
>> org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler [] -
>> Unhandled exception.
>> > java.lang.IllegalArgumentException: The location must not be null for a
>> non-global failure.
>> > at
>> org.apache.flink.util.Preconditions.checkArgument(Preconditions.java:138)
>> ~[flink-dist_2.12-1.13.0.jar:1.13.0]
>> > at
>> org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler.assertLocalExceptionInfo(JobExceptionsHandler.java:218)
>> ~[flink-dist_2.12-1.13.0.jar:1.13.0]
>> > at
>> org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler.createRootExceptionInfo(JobExceptionsHandler.java:191)
>> ~[flink-dist_2.12-1.13.0.jar:1.13.0]
>> > at
>> java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:195)
>> ~[?:?]
>> > at java.util.stream.SliceOps$1$1.accept(SliceOps.java:199) ~[?:?]
>> > at
>> java.util.ArrayList$ArrayListSpliterator.tryAdvance(ArrayList.java:1632)
>> ~[?:?]
>> > at
>> java.util.stream.ReferencePipeline.forEachWithCancel(ReferencePipeline.java:127)
>> ~[?:?]
>> > at
>> java.util.stream.AbstractPipeline.copyIntoWithCancel(AbstractPipeline.java:502)
>> ~[?:?]
>> > at
>> java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:488) ~[?:?]
>> > at
>> java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:474)
>> ~[?:?]
>> > at
>> java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:913)
>> ~[?:?]
>> > at
>> java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234) ~[?:?]
>> > at
>> java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:578)
>> ~[?:?]
>> > at
>> org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler.createJobExceptionHistory(JobExceptionsHandler.java:169)
>> ~[flink-dist_2.12-1.13.0.jar:1.13.0]
>> > at
>> org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler.createJobExceptionsInfo(JobExceptionsHandler.java:154)
>> ~[flink-dist_2.12-1.13.0.jar:1.13.0]
>> > at
>> org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler.handleRequest(JobExceptionsHandler.java:101)
>> ~[flink-dist_2.12-1.13.0.jar:1.13.0]
>> > at
>> org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler.handleRequest(JobExceptionsHandler.java:63)
>> ~[flink-dist_2.12-1.13.0.jar:1.13.0]
>> > at
>> org.apache.flink.runtime.rest.handler.job.AbstractExecutionGraphHandler.lambda$handleRequest$0(AbstractExecutionGraphHandler.java:87)
>> ~[flink-dist_2.12-1.13.0.jar:1.13.0]
>> > at
>> java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:642)
>> [?:?]
>> > at
>> java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:478)
>> [?:?]
>> > at
>> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
>> [?:?]
>> > at java.util.concurrent.FutureTask.run(FutureTask.java:264) [?:?]
>> > at
>> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:304)
>> [?:?]
>> > at
>> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
>> [?:?]
>> > at
>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
>> [?:?]
>> > at java.lang.Thread.run(Thread.java:834) [?:?]
>> >
>> > We would like to check Is there any configuration change should be done
>> for the application? Thanks!
>> >
>> > Regards,
>> > -Gary
>> >
>> >
>> >
>> > APPIER EMAIL NOTICE
>> >
>> > The contents of this email message and any attachments from Appier
>> Group Inc. and/or its affiliates may be privileged and confidential. If you
>> are not the intended recipient of this email, please note that any
>> disclosure, copying, distribution, or use of this message or its
>> attachments is prohibited. If you have received this email in error, please
>> contact us immediately 

Re: Root Exception can not be shown on Web UI in Flink 1.13.0

2021-05-13 Thread Guowei Ma
Hi, Gary

I think it might be a bug. So would you like to open a jira for this.
And could you share the exception ,which the TaskManagerLocation is null?
It might be very helpful to verify the cause.

Best,
Guowei


On Thu, May 13, 2021 at 10:36 AM Yangze Guo  wrote:

> Hi, it seems to be related to FLINK-22276. Thus, I'd involve Matthias
> to take a look.
>
> @Matthias My gut feeling is that not all execution who has failureInfo
> has been deployed?
>
> Best,
> Yangze Guo
>
> On Wed, May 12, 2021 at 10:12 PM Gary Wu  wrote:
> >
> > Hi,
> >
> > We have upgraded our Flink applications to 1.13.0 but we found that Root
> Exception can not be shown on Web UI with an internal server error message.
> After opening browser development console and trace the message, we found
> that there is a exception in jobmanager:
> >
> > 2021-05-12 13:30:45,589 ERROR
> org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler [] -
> Unhandled exception.
> > java.lang.IllegalArgumentException: The location must not be null for a
> non-global failure.
> > at
> org.apache.flink.util.Preconditions.checkArgument(Preconditions.java:138)
> ~[flink-dist_2.12-1.13.0.jar:1.13.0]
> > at
> org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler.assertLocalExceptionInfo(JobExceptionsHandler.java:218)
> ~[flink-dist_2.12-1.13.0.jar:1.13.0]
> > at
> org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler.createRootExceptionInfo(JobExceptionsHandler.java:191)
> ~[flink-dist_2.12-1.13.0.jar:1.13.0]
> > at
> java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:195)
> ~[?:?]
> > at java.util.stream.SliceOps$1$1.accept(SliceOps.java:199) ~[?:?]
> > at
> java.util.ArrayList$ArrayListSpliterator.tryAdvance(ArrayList.java:1632)
> ~[?:?]
> > at
> java.util.stream.ReferencePipeline.forEachWithCancel(ReferencePipeline.java:127)
> ~[?:?]
> > at
> java.util.stream.AbstractPipeline.copyIntoWithCancel(AbstractPipeline.java:502)
> ~[?:?]
> > at
> java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:488) ~[?:?]
> > at
> java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:474)
> ~[?:?]
> > at
> java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:913)
> ~[?:?]
> > at
> java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234) ~[?:?]
> > at
> java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:578)
> ~[?:?]
> > at
> org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler.createJobExceptionHistory(JobExceptionsHandler.java:169)
> ~[flink-dist_2.12-1.13.0.jar:1.13.0]
> > at
> org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler.createJobExceptionsInfo(JobExceptionsHandler.java:154)
> ~[flink-dist_2.12-1.13.0.jar:1.13.0]
> > at
> org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler.handleRequest(JobExceptionsHandler.java:101)
> ~[flink-dist_2.12-1.13.0.jar:1.13.0]
> > at
> org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler.handleRequest(JobExceptionsHandler.java:63)
> ~[flink-dist_2.12-1.13.0.jar:1.13.0]
> > at
> org.apache.flink.runtime.rest.handler.job.AbstractExecutionGraphHandler.lambda$handleRequest$0(AbstractExecutionGraphHandler.java:87)
> ~[flink-dist_2.12-1.13.0.jar:1.13.0]
> > at
> java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:642)
> [?:?]
> > at
> java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:478)
> [?:?]
> > at
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
> [?:?]
> > at java.util.concurrent.FutureTask.run(FutureTask.java:264) [?:?]
> > at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:304)
> [?:?]
> > at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
> [?:?]
> > at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
> [?:?]
> > at java.lang.Thread.run(Thread.java:834) [?:?]
> >
> > We would like to check Is there any configuration change should be done
> for the application? Thanks!
> >
> > Regards,
> > -Gary
> >
> >
> >
> > APPIER EMAIL NOTICE
> >
> > The contents of this email message and any attachments from Appier Group
> Inc. and/or its affiliates may be privileged and confidential. If you are
> not the intended recipient of this email, please note that any disclosure,
> copying, distribution, or use of this message or its attachments is
> prohibited. If you have received this email in error, please contact us
> immediately and delete this message and any attachments.
>


Re: Root Exception can not be shown on Web UI in Flink 1.13.0

2021-05-12 Thread Yangze Guo
Hi, it seems to be related to FLINK-22276. Thus, I'd involve Matthias
to take a look.

@Matthias My gut feeling is that not all execution who has failureInfo
has been deployed?

Best,
Yangze Guo

On Wed, May 12, 2021 at 10:12 PM Gary Wu  wrote:
>
> Hi,
>
> We have upgraded our Flink applications to 1.13.0 but we found that Root 
> Exception can not be shown on Web UI with an internal server error message. 
> After opening browser development console and trace the message, we found 
> that there is a exception in jobmanager:
>
> 2021-05-12 13:30:45,589 ERROR 
> org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler [] - Unhandled 
> exception.
> java.lang.IllegalArgumentException: The location must not be null for a 
> non-global failure.
> at 
> org.apache.flink.util.Preconditions.checkArgument(Preconditions.java:138) 
> ~[flink-dist_2.12-1.13.0.jar:1.13.0]
> at 
> org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler.assertLocalExceptionInfo(JobExceptionsHandler.java:218)
>  ~[flink-dist_2.12-1.13.0.jar:1.13.0]
> at 
> org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler.createRootExceptionInfo(JobExceptionsHandler.java:191)
>  ~[flink-dist_2.12-1.13.0.jar:1.13.0]
> at 
> java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:195) 
> ~[?:?]
> at java.util.stream.SliceOps$1$1.accept(SliceOps.java:199) ~[?:?]
> at 
> java.util.ArrayList$ArrayListSpliterator.tryAdvance(ArrayList.java:1632) 
> ~[?:?]
> at 
> java.util.stream.ReferencePipeline.forEachWithCancel(ReferencePipeline.java:127)
>  ~[?:?]
> at 
> java.util.stream.AbstractPipeline.copyIntoWithCancel(AbstractPipeline.java:502)
>  ~[?:?]
> at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:488) 
> ~[?:?]
> at 
> java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:474) 
> ~[?:?]
> at 
> java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:913) 
> ~[?:?]
> at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234) 
> ~[?:?]
> at java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:578) 
> ~[?:?]
> at 
> org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler.createJobExceptionHistory(JobExceptionsHandler.java:169)
>  ~[flink-dist_2.12-1.13.0.jar:1.13.0]
> at 
> org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler.createJobExceptionsInfo(JobExceptionsHandler.java:154)
>  ~[flink-dist_2.12-1.13.0.jar:1.13.0]
> at 
> org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler.handleRequest(JobExceptionsHandler.java:101)
>  ~[flink-dist_2.12-1.13.0.jar:1.13.0]
> at 
> org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler.handleRequest(JobExceptionsHandler.java:63)
>  ~[flink-dist_2.12-1.13.0.jar:1.13.0]
> at 
> org.apache.flink.runtime.rest.handler.job.AbstractExecutionGraphHandler.lambda$handleRequest$0(AbstractExecutionGraphHandler.java:87)
>  ~[flink-dist_2.12-1.13.0.jar:1.13.0]
> at 
> java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:642)
>  [?:?]
> at 
> java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:478)
>  [?:?]
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515) [?:?]
> at java.util.concurrent.FutureTask.run(FutureTask.java:264) [?:?]
> at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:304)
>  [?:?]
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
>  [?:?]
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
>  [?:?]
> at java.lang.Thread.run(Thread.java:834) [?:?]
>
> We would like to check Is there any configuration change should be done for 
> the application? Thanks!
>
> Regards,
> -Gary
>
>
>
> APPIER EMAIL NOTICE
>
> The contents of this email message and any attachments from Appier Group Inc. 
> and/or its affiliates may be privileged and confidential. If you are not the 
> intended recipient of this email, please note that any disclosure, copying, 
> distribution, or use of this message or its attachments is prohibited. If you 
> have received this email in error, please contact us immediately and delete 
> this message and any attachments.


Root Exception can not be shown on Web UI in Flink 1.13.0

2021-05-12 Thread Gary Wu
Hi,

We have upgraded our Flink applications to 1.13.0 but we found that Root
Exception can not be shown on Web UI with an internal server error message.
After opening browser development console and trace the message, we found
that there is a exception in jobmanager:

*2021-05-12 13:30:45,589 ERROR
org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler [] -
Unhandled exception.*
*java.lang.IllegalArgumentException: The location must not be null for a
non-global failure.*
*at
org.apache.flink.util.Preconditions.checkArgument(Preconditions.java:138)
~[flink-dist_2.12-1.13.0.jar:1.13.0]*
*at
org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler.assertLocalExceptionInfo(JobExceptionsHandler.java:218)
~[flink-dist_2.12-1.13.0.jar:1.13.0]*
*at
org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler.createRootExceptionInfo(JobExceptionsHandler.java:191)
~[flink-dist_2.12-1.13.0.jar:1.13.0]*
*at
java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:195)
~[?:?]*
*at java.util.stream.SliceOps$1$1.accept(SliceOps.java:199) ~[?:?]*
*at
java.util.ArrayList$ArrayListSpliterator.tryAdvance(ArrayList.java:1632)
~[?:?]*
*at
java.util.stream.ReferencePipeline.forEachWithCancel(ReferencePipeline.java:127)
~[?:?]*
*at
java.util.stream.AbstractPipeline.copyIntoWithCancel(AbstractPipeline.java:502)
~[?:?]*
*at
java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:488)
~[?:?]*
*at
java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:474)
~[?:?]*
*at
java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:913)
~[?:?]*
*at
java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
~[?:?]*
*at
java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:578)
~[?:?]*
*at
org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler.createJobExceptionHistory(JobExceptionsHandler.java:169)
~[flink-dist_2.12-1.13.0.jar:1.13.0]*
*at
org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler.createJobExceptionsInfo(JobExceptionsHandler.java:154)
~[flink-dist_2.12-1.13.0.jar:1.13.0]*
*at
org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler.handleRequest(JobExceptionsHandler.java:101)
~[flink-dist_2.12-1.13.0.jar:1.13.0]*
*at
org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler.handleRequest(JobExceptionsHandler.java:63)
~[flink-dist_2.12-1.13.0.jar:1.13.0]*
*at
org.apache.flink.runtime.rest.handler.job.AbstractExecutionGraphHandler.lambda$handleRequest$0(AbstractExecutionGraphHandler.java:87)
~[flink-dist_2.12-1.13.0.jar:1.13.0]*
*at
java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:642)
[?:?]*
*at
java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:478)
[?:?]*
*at
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
[?:?]*
*at java.util.concurrent.FutureTask.run(FutureTask.java:264) [?:?]*
*at
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:304)
[?:?]*
*at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
[?:?]*
*at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
[?:?]*
*at java.lang.Thread.run(Thread.java:834) [?:?]*

We would like to check Is there any configuration change should be done for
the application? Thanks!

Regards,
-Gary

-- 
*APPIER EMAIL NOTICE*
*
*
The contents of this email message and any 
attachments from Appier Group Inc. and/or its affiliates may be privileged 
and confidential. If you are not the intended recipient of this email, 
please note that any disclosure, copying, distribution, or use of this 
message or its attachments is prohibited. If you have received this email 
in error, please contact us immediately and delete this message and any 
attachments.


Re: Uploading job jar via web UI in flink HA mode

2020-12-02 Thread sidhant gupta
Hi Till,

Thanks for the clarification and suggestions

Regards
Sidhant Gupta

On Wed, Dec 2, 2020, 10:10 PM Till Rohrmann  wrote:

> Hi Sidhant,
>
> Have you seen this discussion [1]? If you want to use S3, then you need to
> make sure that you start your Flink processes with the appropriate
> FileSystemProvider for S3 [2]. So the problem you are seeing is most likely
> caused by the JVM not knowing a S3 file system implementation.
>
> Be aware that the filesystem used by the FileUploadHandler
> is java.nio.file.FileSystem and not
> Flink's org.apache.flink.core.fs.FileSystem for which we provide different
> FileSystem implementations.
>
> My best recommendation would be to not use the web submission for Flink
> jobs when running Flink behind a load balancer. Instead use the CLI which
> will retrieve the leader address from ZooKeeper.
>
> [1]
> https://lists.apache.org/thread.html/1dbaf68bc86ad3a1acc87e4bbc2dd5fa8c7560b3f59eb256f81bc855%40%3Cuser.flink.apache.org%3E
> [2]
> https://stackoverflow.com/questions/41113119/java-nio-file-implementation-for-aws
>
> Cheers,
> Till
>
> On Wed, Dec 2, 2020 at 11:31 AM sidhant gupta  wrote:
>
>> Hi All,
>>
>> I have 2 job managers in flink HA mode cluster setup. I have a load
>> balancer forwarding request to both (leader and stand by) the job managers
>> in default round-robin fashion. While uploading the job jar the Web UI is
>> fluctuating between the leader and standby page. Its difficult to upload
>> the job jar and also getting errors after submitting the job as below:
>>
>> Caused by: org.apache.flink.runtime.rest.handler.RestHandlerException:
>> Jar file /tmp/flink-web-4be8d72f-6a7d-4ec9-b71e-516e58
>> c629d5/flink-web-upload/ce291201-31e4-4e55
>> -a68b-eae1a770518d_flinkfhirmapperjob-1.2.jar does not exist
>>
>>
>> I tried using *web.upload.dir* property to use a folder location in S3
>> but getting below error:
>>
>> 2020-12-02 08:26:06,242 WARN
>> org.apache.flink.runtime.rest.FileUploadHandler  [] - File
>> upload failed.
>>
>> java.lang.IllegalArgumentException: UploadDirectory is not absolute.
>>
>> at
>> org.apache.flink.util.Preconditions.checkArgument(Preconditions.java:139)
>> ~[flink-dist_2.11-1.11.2.jar:1.11.2]
>>
>> at
>> org.apache.flink.runtime.rest.handler.FileUploads.(FileUploads.java:58)
>> ~[flink-dist_2.11-1.11.2.jar:1.11.2]
>>
>> at
>> org.apache.flink.runtime.rest.FileUploadHandler.channelRead0(FileUploadHandler.java:159)
>> [flink-dist_2.11-1.11.2.jar:1.11.2]
>>
>> at
>> org.apache.flink.runtime.rest.FileUploadHandler.channelRead0(FileUploadHandler.java:68)
>> [flink-dist_2.11-1.11.2.jar:1.11.2]
>>
>> at
>> org.apache.flink.shaded.netty4.io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105)
>> [flink-dist_2.11-1.11.2.jar:1.11.2]
>>
>>
>> Looks like web.upload.dir only supports local path.
>>
>>
>> Any suggestions on how to upload and submit job jar in flink HA cluster
>> setup via web UI and also it from the CLI ?
>>
>>
>> Thanks and regards
>>
>> Sidhant Gupta
>>
>>
>>
>>


Re: Uploading job jar via web UI in flink HA mode

2020-12-02 Thread Till Rohrmann
Hi Sidhant,

Have you seen this discussion [1]? If you want to use S3, then you need to
make sure that you start your Flink processes with the appropriate
FileSystemProvider for S3 [2]. So the problem you are seeing is most likely
caused by the JVM not knowing a S3 file system implementation.

Be aware that the filesystem used by the FileUploadHandler
is java.nio.file.FileSystem and not
Flink's org.apache.flink.core.fs.FileSystem for which we provide different
FileSystem implementations.

My best recommendation would be to not use the web submission for Flink
jobs when running Flink behind a load balancer. Instead use the CLI which
will retrieve the leader address from ZooKeeper.

[1]
https://lists.apache.org/thread.html/1dbaf68bc86ad3a1acc87e4bbc2dd5fa8c7560b3f59eb256f81bc855%40%3Cuser.flink.apache.org%3E
[2]
https://stackoverflow.com/questions/41113119/java-nio-file-implementation-for-aws

Cheers,
Till

On Wed, Dec 2, 2020 at 11:31 AM sidhant gupta  wrote:

> Hi All,
>
> I have 2 job managers in flink HA mode cluster setup. I have a load
> balancer forwarding request to both (leader and stand by) the job managers
> in default round-robin fashion. While uploading the job jar the Web UI is
> fluctuating between the leader and standby page. Its difficult to upload
> the job jar and also getting errors after submitting the job as below:
>
> Caused by: org.apache.flink.runtime.rest.handler.RestHandlerException: Jar
> file /tmp/flink-web-4be8d72f-6a7d-4ec9-b71e-516e58
> c629d5/flink-web-upload/ce291201-31e4-4e55
> -a68b-eae1a770518d_flinkfhirmapperjob-1.2.jar does not exist
>
>
> I tried using *web.upload.dir* property to use a folder location in S3
> but getting below error:
>
> 2020-12-02 08:26:06,242 WARN
> org.apache.flink.runtime.rest.FileUploadHandler  [] - File
> upload failed.
>
> java.lang.IllegalArgumentException: UploadDirectory is not absolute.
>
> at
> org.apache.flink.util.Preconditions.checkArgument(Preconditions.java:139)
> ~[flink-dist_2.11-1.11.2.jar:1.11.2]
>
> at
> org.apache.flink.runtime.rest.handler.FileUploads.(FileUploads.java:58)
> ~[flink-dist_2.11-1.11.2.jar:1.11.2]
>
> at
> org.apache.flink.runtime.rest.FileUploadHandler.channelRead0(FileUploadHandler.java:159)
> [flink-dist_2.11-1.11.2.jar:1.11.2]
>
> at
> org.apache.flink.runtime.rest.FileUploadHandler.channelRead0(FileUploadHandler.java:68)
> [flink-dist_2.11-1.11.2.jar:1.11.2]
>
> at
> org.apache.flink.shaded.netty4.io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105)
> [flink-dist_2.11-1.11.2.jar:1.11.2]
>
>
> Looks like web.upload.dir only supports local path.
>
>
> Any suggestions on how to upload and submit job jar in flink HA cluster
> setup via web UI and also it from the CLI ?
>
>
> Thanks and regards
>
> Sidhant Gupta
>
>
>
>


Uploading job jar via web UI in flink HA mode

2020-12-02 Thread sidhant gupta
Hi All,

I have 2 job managers in flink HA mode cluster setup. I have a load
balancer forwarding request to both (leader and stand by) the job managers
in default round-robin fashion. While uploading the job jar the Web UI is
fluctuating between the leader and standby page. Its difficult to upload
the job jar and also getting errors after submitting the job as below:

Caused by: org.apache.flink.runtime.rest.handler.RestHandlerException: Jar
file /tmp/flink-web-4be8d72f-6a7d-4ec9-b71e-516e58
c629d5/flink-web-upload/ce291201-31e4-4e55
-a68b-eae1a770518d_flinkfhirmapperjob-1.2.jar does not exist


I tried using *web.upload.dir* property to use a folder location in S3 but
getting below error:

2020-12-02 08:26:06,242 WARN
org.apache.flink.runtime.rest.FileUploadHandler  [] - File
upload failed.

java.lang.IllegalArgumentException: UploadDirectory is not absolute.

at
org.apache.flink.util.Preconditions.checkArgument(Preconditions.java:139)
~[flink-dist_2.11-1.11.2.jar:1.11.2]

at
org.apache.flink.runtime.rest.handler.FileUploads.(FileUploads.java:58)
~[flink-dist_2.11-1.11.2.jar:1.11.2]

at
org.apache.flink.runtime.rest.FileUploadHandler.channelRead0(FileUploadHandler.java:159)
[flink-dist_2.11-1.11.2.jar:1.11.2]

at
org.apache.flink.runtime.rest.FileUploadHandler.channelRead0(FileUploadHandler.java:68)
[flink-dist_2.11-1.11.2.jar:1.11.2]

at
org.apache.flink.shaded.netty4.io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105)
[flink-dist_2.11-1.11.2.jar:1.11.2]


Looks like web.upload.dir only supports local path.


Any suggestions on how to upload and submit job jar in flink HA cluster
setup via web UI and also it from the CLI ?


Thanks and regards

Sidhant Gupta


Re: Unable to submit job from the web UI in flink 4.0

2018-02-13 Thread Puneet Kinra
yep working from cli.

On Wed, Feb 14, 2018 at 1:41 AM, Vishal Santoshi 
wrote:

> This I think is a known issue with 1.4.  Good submissions work on UI (
> one's that do not have errors to report )  and  CLI works always ( in our
> case )
>
> On Tue, Feb 13, 2018 at 1:02 PM, Gary Yao  wrote:
>
>> cc'ing user mailing list
>>
>> On Mon, Feb 12, 2018 at 12:40 PM, Puneet Kinra <
>> puneet.ki...@customercentria.com> wrote:
>>
>>> Hi Gary
>>>
>>> Thanks for the response i am able to upload the jar but i when am
>>> pressing the submit button
>>> It keep on running but jar is not getting deployed. & even not getting
>>> any logs
>>> for the submission. is there any specific port need to be opened for this
>>> Somebody also raise the same concern on stackoverflow
>>>
>>> *https://stackoverflow.com/questions/48158617/flink-cannot-submit-new-job
>>> *
>>>
>>>
>>> On Mon, Feb 12, 2018 at 12:23 PM, Gary Yao 
>>> wrote:
>>>
 Hi Puneet,

 Are you not able to upload the jars? If the jar upload already fails,
 can you
 try to upload from the command line using curl, and post the output?

 curl -v -X POST -H "Expect:" -F "jarfile=@/path/to/jarfile.jar"
 http://host:port/jars/upload

 Also, in the JobManager logs there might be some hints.

 Best,
 Gary

 On Sat, Feb 10, 2018 at 12:51 PM, Puneet Kinra <
 puneet.ki...@customercentria.com> wrote:

> Hi All
>
> I am unable to deposit jobs from frontend, no errors
> are also getting generated.
>
> --
> *Cheers *
>
> *Puneet Kinra*
>
> *Mobile:+918800167808 <+91%2088001%2067808> | Skype :
> puneet.ki...@customercentria.com *
>
> *e-mail :puneet.ki...@customercentria.com
> *
>
>
>

>>>
>>>
>>> --
>>> *Cheers *
>>>
>>> *Puneet Kinra*
>>>
>>> *Mobile:+918800167808 <+91%2088001%2067808> | Skype :
>>> puneet.ki...@customercentria.com *
>>>
>>> *e-mail :puneet.ki...@customercentria.com
>>> *
>>>
>>>
>>>
>>
>


-- 
*Cheers *

*Puneet Kinra*

*Mobile:+918800167808 | Skype : puneet.ki...@customercentria.com
*

*e-mail :puneet.ki...@customercentria.com
*


Re: Unable to submit job from the web UI in flink 4.0

2018-02-13 Thread Vishal Santoshi
This I think is a known issue with 1.4.  Good submissions work on UI (
one's that do not have errors to report )  and  CLI works always ( in our
case )

On Tue, Feb 13, 2018 at 1:02 PM, Gary Yao  wrote:

> cc'ing user mailing list
>
> On Mon, Feb 12, 2018 at 12:40 PM, Puneet Kinra <
> puneet.ki...@customercentria.com> wrote:
>
>> Hi Gary
>>
>> Thanks for the response i am able to upload the jar but i when am
>> pressing the submit button
>> It keep on running but jar is not getting deployed. & even not getting
>> any logs
>> for the submission. is there any specific port need to be opened for this
>> Somebody also raise the same concern on stackoverflow
>>
>> *https://stackoverflow.com/questions/48158617/flink-cannot-submit-new-job
>> *
>>
>>
>> On Mon, Feb 12, 2018 at 12:23 PM, Gary Yao 
>> wrote:
>>
>>> Hi Puneet,
>>>
>>> Are you not able to upload the jars? If the jar upload already fails,
>>> can you
>>> try to upload from the command line using curl, and post the output?
>>>
>>> curl -v -X POST -H "Expect:" -F "jarfile=@/path/to/jarfile.jar"
>>> http://host:port/jars/upload
>>>
>>> Also, in the JobManager logs there might be some hints.
>>>
>>> Best,
>>> Gary
>>>
>>> On Sat, Feb 10, 2018 at 12:51 PM, Puneet Kinra <
>>> puneet.ki...@customercentria.com> wrote:
>>>
 Hi All

 I am unable to deposit jobs from frontend, no errors
 are also getting generated.

 --
 *Cheers *

 *Puneet Kinra*

 *Mobile:+918800167808 <+91%2088001%2067808> | Skype :
 puneet.ki...@customercentria.com *

 *e-mail :puneet.ki...@customercentria.com
 *



>>>
>>
>>
>> --
>> *Cheers *
>>
>> *Puneet Kinra*
>>
>> *Mobile:+918800167808 <+91%2088001%2067808> | Skype :
>> puneet.ki...@customercentria.com *
>>
>> *e-mail :puneet.ki...@customercentria.com
>> *
>>
>>
>>
>


Re: Unable to submit job from the web UI in flink 4.0

2018-02-13 Thread Gary Yao
cc'ing user mailing list

On Mon, Feb 12, 2018 at 12:40 PM, Puneet Kinra <
puneet.ki...@customercentria.com> wrote:

> Hi Gary
>
> Thanks for the response i am able to upload the jar but i when am pressing
> the submit button
> It keep on running but jar is not getting deployed. & even not getting any
> logs
> for the submission. is there any specific port need to be opened for this
> Somebody also raise the same concern on stackoverflow
>
> *https://stackoverflow.com/questions/48158617/flink-cannot-submit-new-job
> *
>
>
> On Mon, Feb 12, 2018 at 12:23 PM, Gary Yao  wrote:
>
>> Hi Puneet,
>>
>> Are you not able to upload the jars? If the jar upload already fails, can
>> you
>> try to upload from the command line using curl, and post the output?
>>
>> curl -v -X POST -H "Expect:" -F "jarfile=@/path/to/jarfile.jar"
>> http://host:port/jars/upload
>>
>> Also, in the JobManager logs there might be some hints.
>>
>> Best,
>> Gary
>>
>> On Sat, Feb 10, 2018 at 12:51 PM, Puneet Kinra <
>> puneet.ki...@customercentria.com> wrote:
>>
>>> Hi All
>>>
>>> I am unable to deposit jobs from frontend, no errors
>>> are also getting generated.
>>>
>>> --
>>> *Cheers *
>>>
>>> *Puneet Kinra*
>>>
>>> *Mobile:+918800167808 <+91%2088001%2067808> | Skype :
>>> puneet.ki...@customercentria.com *
>>>
>>> *e-mail :puneet.ki...@customercentria.com
>>> *
>>>
>>>
>>>
>>
>
>
> --
> *Cheers *
>
> *Puneet Kinra*
>
> *Mobile:+918800167808 <+91%2088001%2067808> | Skype :
> puneet.ki...@customercentria.com *
>
> *e-mail :puneet.ki...@customercentria.com
> *
>
>
>


Re: Unable to submit job from the web UI in flink 4.0

2018-02-12 Thread Timo Walther

Hi Puneet,

in Flink 1.4.0 there was a bug such that error were not printed on the 
web UI. What happens if you submit the job via the command line?


Regards,
Timo


Am 2/10/18 um 12:51 PM schrieb Puneet Kinra:

Hi All

I am unable to deposit jobs from frontend, no errors
are also getting generated.

--
*Cheers *
*
*
*Puneet Kinra*
*
*

*Mobile:+918800167808 | Skype : puneet.ki...@customercentria.com 
*


*e-mail :puneet.ki...@customercentria.com 
*







Unable to submit job from the web UI in flink 4.0

2018-02-10 Thread Puneet Kinra
Hi All

I am unable to deposit jobs from frontend, no errors
are also getting generated.

-- 
*Cheers *

*Puneet Kinra*

*Mobile:+918800167808 | Skype : puneet.ki...@customercentria.com
*

*e-mail :puneet.ki...@customercentria.com
*


Re: UI for flink

2015-07-13 Thread Chiwan Park
If you would search a graphical interface for data analytics like Jupyter, you 
should look Apache Zeppelin [1].
Apache Zeppelin is a web-based notebook. It supports Scala, Spark and Flink.

Regards,
Chiwan Park

[1] https://zeppelin.incubator.apache.org

 On Jul 13, 2015, at 9:23 PM, Till Rohrmann trohrm...@apache.org wrote:
 
 Hi Hermann,
 
 when you start a Flink cluster, then also the web interface is started. It is 
 reachable under http://jobManagerURL:8081. The web interface tells you a 
 lot about the current state of your cluster and the currently executed Flink 
 jobs.
 
 Additionally, you can start the web client via ./start-webclient.sh, which 
 you can find in the bin directory. The web client, which is reachable under 
 port 8080, allows you to submit Flink jobs to your cluster via a browser.
 
 Cheers,
 Till
 
 
 On Mon, Jul 13, 2015 at 2:07 PM, Hermann Azong hermann.az...@gmail.com 
 wrote:
 Hello Flinkers,
 
 I'm wondering if a UI Solution for Flink already exist when starting
 
 Sincerly,
 
 Hermann 
 







Re: UI for flink

2015-07-13 Thread Hermann Azong

Hey Till,

thank you for the answer.

Cheers

Am 13.07.2015 um 14:23 schrieb Till Rohrmann:


Hi Hermann,

when you start a Flink cluster, then also the web interface is 
started. It is reachable under |http://jobManagerURL:8081|. The web 
interface tells you a lot about the current state of your cluster and 
the currently executed Flink jobs.


Additionally, you can start the web client via |./start-webclient.sh|, 
which you can find in the |bin| directory. The web client, which is 
reachable under port |8080|, allows you to submit Flink jobs to your 
cluster via a browser.


Cheers,
Till

​

On Mon, Jul 13, 2015 at 2:07 PM, Hermann Azong 
hermann.az...@gmail.com mailto:hermann.az...@gmail.com wrote:


Hello Flinkers,

I'm wondering if a UI Solution for Flink already exist when starting

Sincerly,

Hermann






UI for flink

2015-07-13 Thread Hermann Azong
Hello Flinkers,

I'm wondering if a UI Solution for Flink already exist when starting

Sincerly,

Hermann


Re: UI for flink

2015-07-13 Thread Till Rohrmann
Hi Hermann,

when you start a Flink cluster, then also the web interface is started. It
is reachable under http://jobManagerURL:8081. The web interface tells you
a lot about the current state of your cluster and the currently executed
Flink jobs.

Additionally, you can start the web client via ./start-webclient.sh, which
you can find in the bin directory. The web client, which is reachable under
port 8080, allows you to submit Flink jobs to your cluster via a browser.

Cheers,
Till
​

On Mon, Jul 13, 2015 at 2:07 PM, Hermann Azong hermann.az...@gmail.com
wrote:

 Hello Flinkers,

 I'm wondering if a UI Solution for Flink already exist when starting

 Sincerly,

 Hermann