[
https://issues.apache.org/jira/browse/FLINK-7240?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16118017#comment-16118017
]
ASF GitHub Bot commented on FLINK-7240:
---------------------------------------
GitHub user tillrohrmann opened a pull request:
https://github.com/apache/flink/pull/4497
[FLINK-7240] [tests] Stabilize ExternalizedCheckpointITCase
## What is the purpose of the change
Stabilize `ExternalizedCheckpointITCase`.
## Brief change log
The problem was that the TestingCluster did not wait properly after
canceling the
job that the job was also completely removed from the cluster before
submitting
the next job. This could lead to a NoResourceAvailableException which
ultimately
made the job fail.
## Verifying this change
This change is a trivial rework / code cleanup without any test coverage.
## Does this pull request potentially affect one of the following parts:
- Dependencies (does it add or upgrade a dependency): (no)
- The public API, i.e., is any changed class annotated with
`@Public(Evolving)`: (no)
- The serializers: (no)
- The runtime per-record code paths (performance sensitive): (no)
- Anything that affects deployment or recovery: JobManager (and its
components), Checkpointing, Yarn/Mesos, ZooKeeper: (no)
## Documentation
- Does this pull request introduce a new feature? (no)
- If yes, how is the feature documented? (not applicable)
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/tillrohrmann/flink
fixExternalizedCheckpointITCase
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/flink/pull/4497.patch
To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:
This closes #4497
----
commit 2a8adef594c965182324d3a02c1979e118ebf850
Author: Till Rohrmann <[email protected]>
Date: 2017-08-08T08:04:34Z
[FLINK-7240] [tests] Stabilize ExternalizedCheckpointITCase
The problem was that the TestingCluster did not wait properly after
canceling the
job that the job was also completely removed from the cluster before
submitting
the next job. This could lead to a NoResourceAvailableException which
ultimately
made the job fail.
----
> Externalized RocksDB can fail with stackoverflow
> ------------------------------------------------
>
> Key: FLINK-7240
> URL: https://issues.apache.org/jira/browse/FLINK-7240
> Project: Flink
> Issue Type: Bug
> Components: State Backends, Checkpointing, Tests
> Affects Versions: 1.3.1, 1.4.0
> Environment: https://travis-ci.org/zentol/flink/jobs/255760513
> Reporter: Chesnay Schepler
> Assignee: Till Rohrmann
> Priority: Critical
> Labels: test-stability
>
> {code}
> testExternalizedFullRocksDBCheckpointsStandalone(org.apache.flink.test.checkpointing.ExternalizedCheckpointITCase)
> Time elapsed: 146.894 sec <<< ERROR!
> java.lang.StackOverflowError: null
> at java.util.Hashtable.get(Hashtable.java:363)
> at java.util.Properties.getProperty(Properties.java:969)
> at java.lang.System.getProperty(System.java:720)
> at sun.security.action.GetPropertyAction.run(GetPropertyAction.java:84)
> at sun.security.action.GetPropertyAction.run(GetPropertyAction.java:49)
> at java.security.AccessController.doPrivileged(Native Method)
> at java.io.PrintWriter.<init>(PrintWriter.java:116)
> at java.io.PrintWriter.<init>(PrintWriter.java:100)
> at
> org.apache.log4j.DefaultThrowableRenderer.render(DefaultThrowableRenderer.java:58)
> at
> org.apache.log4j.spi.ThrowableInformation.getThrowableStrRep(ThrowableInformation.java:87)
> at
> org.apache.log4j.spi.LoggingEvent.getThrowableStrRep(LoggingEvent.java:413)
> at org.apache.log4j.WriterAppender.subAppend(WriterAppender.java:313)
> at org.apache.log4j.WriterAppender.append(WriterAppender.java:162)
> at org.apache.log4j.AppenderSkeleton.doAppend(AppenderSkeleton.java:251)
> at
> org.apache.log4j.helpers.AppenderAttachableImpl.appendLoopOnAppenders(AppenderAttachableImpl.java:66)
> at org.apache.log4j.Category.callAppenders(Category.java:206)
> at org.apache.log4j.Category.forcedLog(Category.java:391)
> at org.apache.log4j.Category.log(Category.java:856)
> at org.slf4j.impl.Log4jLoggerAdapter.info(Log4jLoggerAdapter.java:381)
> at
> org.apache.flink.runtime.testingUtils.TestingCluster.requestCheckpoint(TestingCluster.scala:392)
> at
> org.apache.flink.runtime.testingUtils.TestingCluster.requestCheckpoint(TestingCluster.scala:394)
> at
> org.apache.flink.runtime.testingUtils.TestingCluster.requestCheckpoint(TestingCluster.scala:394)
> at
> org.apache.flink.runtime.testingUtils.TestingCluster.requestCheckpoint(TestingCluster.scala:394)
> at
> org.apache.flink.runtime.testingUtils.TestingCluster.requestCheckpoint(TestingCluster.scala:394)
> at
> org.apache.flink.runtime.testingUtils.TestingCluster.requestCheckpoint(TestingCluster.scala:394)
> at
> org.apache.flink.runtime.testingUtils.TestingCluster.requestCheckpoint(TestingCluster.scala:394)
> at
> org.apache.flink.runtime.testingUtils.TestingCluster.requestCheckpoint(TestingCluster.scala:394)
> ...
> {code}
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)