You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2018/03/23 15:25:00 UTC

[jira] [Commented] (FLINK-5411) LocalStreamEnvironmentITCase#testRunIsolatedJob failed on travis

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

ASF GitHub Bot commented on FLINK-5411:
---------------------------------------

GitHub user tillrohrmann opened a pull request:

    https://github.com/apache/flink/pull/5757

    [FLINK-5411] [flip6] Fix JobLeaderIdService shut down in ResourceManager

    ## What is the purpose of the change
    
    The JobLeaderIdService was formerly closed at two different locations. Once in the
    ResourceManager and once in the ResourceManagerRuntimeServices. Since the JobLeaderIdService
    is a RM specific component. It should also be closed in the scope of the RM.
    
    ## 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)
      - The S3 file system connector: (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 fixJobLeaderIdServiceShutDown

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/5757.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 #5757
    
----
commit 2bd3d6d1474b85dfaa341b829d7b0e9816bc9353
Author: Till Rohrmann <tr...@...>
Date:   2018-03-23T15:21:55Z

    [FLINK-5411] [flip6] Fix JobLeaderIdService shut down in ResourceManager
    
    The JobLeaderIdService was formerly closed at two different locations. Once in the
    ResourceManager and once in the ResourceManagerRuntimeServices. Since the JobLeaderIdService
    is a RM specific component. It should also be closed in the scope of the RM.

----


> LocalStreamEnvironmentITCase#testRunIsolatedJob failed on travis
> ----------------------------------------------------------------
>
>                 Key: FLINK-5411
>                 URL: https://issues.apache.org/jira/browse/FLINK-5411
>             Project: Flink
>          Issue Type: Bug
>          Components: ResourceManager, Tests
>    Affects Versions: 1.3.0
>            Reporter: Chesnay Schepler
>            Assignee: Till Rohrmann
>            Priority: Blocker
>              Labels: test-stability
>             Fix For: 1.5.0
>
>
> https://s3.amazonaws.com/archive.travis-ci.org/jobs/189148248/log.txt
> Running org.apache.flink.streaming.api.environment.LocalStreamEnvironmentITCase
> Tests run: 2, Failures: 0, Errors: 1, Skipped: 0, Time elapsed: 2.011 sec <<< FAILURE! - in org.apache.flink.streaming.api.environment.LocalStreamEnvironmentITCase
> testRunIsolatedJob(org.apache.flink.streaming.api.environment.LocalStreamEnvironmentITCase)  Time elapsed: 1.604 sec  <<< ERROR!
> java.util.ConcurrentModificationException: null
> 	at java.util.HashMap$HashIterator.nextNode(HashMap.java:1429)
> 	at java.util.HashMap$ValueIterator.next(HashMap.java:1458)
> 	at org.apache.flink.runtime.resourcemanager.JobLeaderIdService.clear(JobLeaderIdService.java:114)
> 	at org.apache.flink.runtime.resourcemanager.JobLeaderIdService.stop(JobLeaderIdService.java:92)
> 	at org.apache.flink.runtime.resourcemanager.ResourceManager.shutDown(ResourceManager.java:182)
> 	at org.apache.flink.runtime.resourcemanager.ResourceManagerRunner.shutDownInternally(ResourceManagerRunner.java:83)
> 	at org.apache.flink.runtime.resourcemanager.ResourceManagerRunner.shutDown(ResourceManagerRunner.java:78)
> 	at org.apache.flink.runtime.minicluster.MiniCluster.shutdownInternally(MiniCluster.java:313)
> 	at org.apache.flink.runtime.minicluster.MiniCluster.shutdown(MiniCluster.java:281)
> 	at org.apache.flink.streaming.api.environment.Flip6LocalStreamEnvironment.execute(Flip6LocalStreamEnvironment.java:124)
> 	at org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.execute(StreamExecutionEnvironment.java:1486)
> 	at org.apache.flink.streaming.api.environment.LocalStreamEnvironmentITCase.testRunIsolatedJob(LocalStreamEnvironmentITCase.java:41)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)