[ 
https://issues.apache.org/jira/browse/FLINK-21659?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17325723#comment-17325723
 ] 

Chesnay Schepler commented on FLINK-21659:
------------------------------------------

There is some regex parsing being applied to the result of a REST request, but 
we don't check that the request was actually successful. We end up trying to 
parse a checkpoint id from a stacktrace:
{code:java}
/home/vsts/work/1/s/flink-end-to-end-tests/test-scripts/common_ha.sh: line 151: 
[: 58)\n\tat 
org.apache.flink.runtime.rest.handler.job.AbstractAccessExecutionGraphHandler.handleRequest(AbstractAccessExecutionGraphHandler.java:
 integer expression expected {code}
The reason this request fails is because, if a job is in the INITIALIZING 
state, then the ArchivedExecutionGraph is created with a nulled 
CheckpointStatsSnapshot, which is mistakenly interpreted by the handler as 
checkpointing not being enabled.

This is not logged, because for this specific case the logging was disabled in 
https://github.com/apache/flink/pull/14129.

> Running HA per-job cluster (rocks, incremental) end-to-end test fails
> ---------------------------------------------------------------------
>
>                 Key: FLINK-21659
>                 URL: https://issues.apache.org/jira/browse/FLINK-21659
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Coordination
>    Affects Versions: 1.13.0
>            Reporter: Guowei Ma
>            Assignee: Chesnay Schepler
>            Priority: Critical
>              Labels: test-stability
>             Fix For: 1.13.0
>
>         Attachments: screenshot-1.png
>
>
> https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=14232&view=logs&j=4dd4dbdd-1802-5eb7-a518-6acd9d24d0fc&t=8d6b4dd3-4ca1-5611-1743-57a7d76b395a
> “Completed checkpoint” is more than two times(42 times in the 
> "flink-vsts-standalonejob-2-fv-az83-563.log") but the test still fail. 
>  !screenshot-1.png! 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to