hadoop-yarn-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Sangjin Lee (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (YARN-3367) Replace starting a separate thread for post entity with event loop in TimelineClient
Date Thu, 07 Jan 2016 18:19:39 GMT

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

Sangjin Lee commented on YARN-3367:
-----------------------------------

[~Naganarasimha] and I spoke offline about this, but wanted to recapture some of them.

I generally agree with the direction and the scope of the patch. The things that are being
considered here are related enough that they should be handled as a whole.

Regarding clubbing asynchronous puts together in a single REST call, I wanted to be clear
that its purpose is to reduce the overhead of the REST call latency. That said, we should
be careful to keep buffering to a minimum as it would certainly complicate the picture in
terms of handling put calls that arrive very late (that won't happen with the current patch).

Also, we should make sure the combined put REST call doesn't result in too big a payload.
So in addition to combining calls, we should have a limitation on the size of the resulting
PUT call. I don't think we need to be absolutely accurate here. We can use a simple measure
(e.g. the number of entities + events + metrics, or even simply number of entities) to make
sure things do not get out of control when put calls are made rapidly. I don't think this
is an oft-occurring situation, but it would be good to have that safety.

Regarding the implementation of the async thread and the interaction with the workload:
ExecutorService is thread management plus work queue management, but at minimum we can use
the thread management portion of things. That should help eliminate the need for the wait-notify
done for shutting down the thread, coordinating with the thread shutdown, etc.

Also, instead of wait-nofity for the work completion, I would encourage using things like
CountDownLatch or Future. Those will simplify code tremendously and also minimize room for
errors. As a rule, I would advocate using higher level abstractions provided by java.util.concurrent
over primitives, unless the concurrency utilities are not able to provide the right feature
(which should be uncommon).

Another item: it would be good to have a self-contained "work item" instead of using a lookup
on a map. For example, timeline entities + async flag can be a self-contained work item. Then
it would be much simpler to deal with.

> 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)

Mime
View raw message