[
https://issues.apache.org/jira/browse/YARN-3367?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Naganarasimha G R updated YARN-3367:
------------------------------------
Attachment: YARN-3367-YARN-2928.v1.005.patch
Thanks [~sjlee0],
Have uploaded a patch with fix for your comments and few others which were
discussed in the meeting. Apart from it Patch has following modifications :
* TimelineEntities should have List of Entities similar to V1, problem i was
facing was Entities were getting replaced when the EntityIdentifier was same
and additional effort needs to be put to merge them which i felt was not
necessary.
* Removed the thread pools in Distributed shell's ApplicationMaster and MR's
JobHistoryEventHandler
* AMRMClientAsync should update the timelineClient's address first when it gets
the HB response so that the initial timeline events are not lost after trying
for 30 times.
Open Question :
* refer
{{distributedshell.ApplicationMaster.publishApplicationAttemptEventOnTimelineServiceV2(DSEvent)}}
{code}
appSubmitterUgi.doAs(new PrivilegedExceptionAction<Object>() {
@Override
public TimelinePutResponse run() throws Exception {
timelineClient.putEntities(entity);
return null;
}
});
{code}
the thread which is doing the rest call is different from the above thread
which is just adding to the dispatcher. Hence i felt we need to take UGI with
which the REST calls needs to be set either as part of each PUT or much better
per TimelineClient . thoughts?
> 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-YARN-2928.v1.005.patch,
> 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)