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/01/24 16:36:00 UTC

[jira] [Commented] (FLINK-8503) Port TaskManagerLogHandler to new REST endpoint

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

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

GitHub user tillrohrmann opened a pull request:

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

    [FLINK-8503] [flip6] Display TaskExecutor logs and stdout files in web ui

    ## What is the purpose of the change
    
    Introduce the AbstractHandler which takes a typed request and returns an untyped
    response. The AbstractRestHandler extends the AbstractHandler to add typed reponses.
    
    Introduce AbstractTaskManagerFileHandler which encapsulates the file loading logic.
    Upon request of a TaskManager file, the handler will trigger the file upload via
    the ResourceManager. The returned TransientBlobKey is then downloaded via the
    TransientBlobService. Once downloaded, the file is served to the client. Each
    transient blob key is cached for maximum duration after which it is purged and has
    to be reuploaded by the TaskExecutor.
    
    This PR is based on #5341 
    
    ## Brief change log
    
    - Introduced untyped response handler `AbstractHandler`
    - Added `AbstractTaskManagerFileHandler` which is responsible for serving files from the `TaskExecutor`
    - The `AbstractTaskManagerFileHandler` triggers the file upload via the `ResourceManager` which knows the `TaskExecutors`, additionally it caches the `TransientBlobKeys` in order to not always trigger a file upload
    - Added `TaskManagerLogFileHandler` to serve the log file
    - Added `TaskManagerStdoutFileHandler` to serve the stdout file
    
    ## Verifying this change
    
    - Added `AbstractTaskManagerFileHandlerTest`
    - Tested functionality manually
    
    ## 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 enableTaskManagerLogRetrieval

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

    https://github.com/apache/flink/pull/5353.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 #5353
    
----
commit b9db2ab22c346ec64363b446ed23692af1365239
Author: Till Rohrmann <tr...@...>
Date:   2018-01-23T18:12:27Z

    [FLINK-8501] [flip6] Use single BlobCacheService per TaskExecutor
    
    Instead of creating for each new JobManagerConnection a dedicated BlobCacheService
    the TaskExecutor uses a single BlobCacheService which it shares between the
    different JobManagerConnections. The initial BlobServer address is passed by the
    ResourceManager when the TaskExecutor registers at it. In order to avoid the re-
    creation of BlobCacheServices, this commit changes the behaviour such that one can
    update the BlobServer address.

commit 5bd4db619ff0e984477ead323345f5f7fa740626
Author: Till Rohrmann <tr...@...>
Date:   2018-01-24T12:41:53Z

    [hotfix] [tests] Remove JobManagerRunnerMockTest
    
    The JobManagerRunnerMockTest is completely ignored. Moreover, it tests things with
    heavy usage of Mockito which is hard to maintain.

commit c38adb65dd49150a03a2f5f7ea8d421b2ef34616
Author: Till Rohrmann <tr...@...>
Date:   2018-01-24T12:42:41Z

    [FLINK-8502] [flip6] Remove LibraryCacheManager from JobMaster
    
    This commit removes the LibraryCacheManager from the JobMaster since it is
    no longer needed. The JobMaster is started with the correct user code class
    loader and, thus, does not need the LibraryCacheManager.
    
    This commit also corrects that the BlobServer is not closed by the
    JobManagerServices#shutdown method.

commit eb3fed4ce96b4ce27f01d72f2dc76b151dbdd1ae
Author: Till Rohrmann <tr...@...>
Date:   2018-01-23T14:17:16Z

    [FLINK-8495] [flip6] Enable main cluster component's log and stdout file retrieval
    
    This commit enables the log and stdout file retrieval of the cluster's main component
    via the web ui. This happens via the StaticFileServerHandler which serves the log
    and stdout file.

commit 23496241962c69ceb3ada9bd0d67c0344554cf99
Author: Till Rohrmann <tr...@...>
Date:   2018-01-23T17:27:28Z

    [FLINK-8503] [flip6] Display TaskExecutor logs and stdout files in web ui
    
    Introduce the AbstractHandler which takes a typed request and returns an untyped
    response. The AbstractRestHandler extends the AbstractHandler to add typed reponses.
    
    Introduce AbstractTaskManagerFileHandler which encapsulates the file loading logic.
    Upon request of a TaskManager file, the handler will trigger the file upload via
    the ResourceManager. The returned TransientBlobKey is then downloaded via the
    TransientBlobService. Once downloaded, the file is served to the client. Each
    transient blob key is cached for maximum duration after which it is purged and has
    to be reuploaded by the TaskExecutor.

----


> Port TaskManagerLogHandler to new REST endpoint
> -----------------------------------------------
>
>                 Key: FLINK-8503
>                 URL: https://issues.apache.org/jira/browse/FLINK-8503
>             Project: Flink
>          Issue Type: Sub-task
>          Components: REST
>    Affects Versions: 1.5.0
>            Reporter: Till Rohrmann
>            Assignee: Till Rohrmann
>            Priority: Major
>              Labels: flip-6
>             Fix For: 1.5.0
>
>
> In order to serve {{TaskExecutor}} log stdout files, we have to port the {{TaskManagerLogHandler}} to the new REST endpoint.
> In order to properly support serving of files, I propose to introduce an {{AbstractHandler}} which takes a typed request but has not typed response. That way we can easily output the file contents.



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