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

Junping Du commented on YARN-3367:
----------------------------------

Thanks Naga for updating the patch! Quickly go through your patch but haven't 
deep dive there. 
Quickly response on your comments above:
bq. I could reuse/extend Async Dispatcher after YARN-4400 is committed to trunk.
Sounds good. I just commit YARN-4400 to trunk.

bq. I think it can be more organized if i can move the all this related 
code(dispatcher code) to a new class.
Does TimelineEntityAsyncDispatcher can be reused by other classes? If not, 
better to keep it as private class.

bq. will work on other locations(removing the thread pools in the caller side) 
once the approach is finalized.
Make sense. That could make caller code much simpler.

More comments come later.


> Replace starting a separate thread for post entity with event loop in 
> TimelineClient
> ------------------------------------------------------------------------------------
>
>                 Key: YARN-3367
>                 URL: https://issues.apache.org/jira/browse/YARN-3367
>             Project: Hadoop YARN
>          Issue Type: Sub-task
>          Components: timelineserver
>    Affects Versions: YARN-2928
>            Reporter: Junping Du
>            Assignee: Naganarasimha G R
>              Labels: yarn-2928-1st-milestone
>         Attachments: YARN-3367-feature-YARN-2928.003.patch, 
> YARN-3367-feature-YARN-2928.v1.002.patch, 
> YARN-3367-feature-YARN-2928.v1.004.patch, YARN-3367.YARN-2928.001.patch
>
>
> Since YARN-3039, we add loop in TimelineClient to wait for 
> collectorServiceAddress ready before posting any entity. In consumer of  
> TimelineClient (like AM), we are starting a new thread for each call to get 
> rid of potential deadlock in main thread. This way has at least 3 major 
> defects:
> 1. The consumer need some additional code to wrap a thread before calling 
> putEntities() in TimelineClient.
> 2. It cost many thread resources which is unnecessary.
> 3. The sequence of events could be out of order because each posting 
> operation thread get out of waiting loop randomly.
> We should have something like event loop in TimelineClient side, 
> putEntities() only put related entities into a queue of entities and a 
> separated thread handle to deliver entities in queue to collector via REST 
> call.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to