[
https://issues.apache.org/jira/browse/HADOOP-10278?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13903740#comment-13903740
]
Chris Li commented on HADOOP-10278:
-----------------------------------
[~ikeda] Thanks for taking a look.
bq. Although ReadWriteLock is much more complex than simple locks and is not
prefered to enclose trival logic, I suspect it is not severe overhead compared
to LinkedBlockingQueue.
One of the issues with RWLock I found was that we'd have to use offer(timeout)
instead of put(), or else the readlock may not be released. This makes the
queue swap lock for about a second, and causes a huge backup in the process.
Otherwise the overhead of the lock wasn't much, it's mainly the offer(timeout).
bq. By the way, about the patch, nobody can eliminate the possibility that a
blocked thread at LinkedBlockingQueue.put() cannot wake up in 1 seconds when
another thread drains. At least you should check the reference after put(),
such as
We could increase the timeout period on the queue swap transfer too.
bq. (I also worry about the order of elements is not preserved in spite of the
name "queue".)
Order doesn't matter to the client, since the client cannot expect to send two
consecutive commands without receiving a response from the first, and expect
order that they're executed in order, even today. This is why we can even do
QoS by re-ordering rpc calls in the first place.
As far as the server goes, I guess this will have to be something we're aware
of.
bq. Still, implementation of size() is invalid.
Are you referring to CallQueueManager.size(), due to the possibility that the
queue is being swapped? I'm assuming that size() isn't being used for anything
besides metrics, so a small discrepancy during swapping would be okay.
bq. The atomic reference is not needed for the queue and volatile is enough if
you only set and get. Volatile variables may have much chance to be optimized
by VM because volatile is within the language specification. On the other hand,
increment operation (++) is not atomic for volatile variables, so some of the
test classes should be changed.
Agreed, though I think [~daryn] says he prefers atomicref for clarity. I could
go either way.
Also, in the tests, the atomic members aren't being written by multiple
threads–they're atomic so that I can read them from another thread without
thread caching. Even if they incremented atomically, there's still the issue of
the put happening before the increment, so the tests only check after the
operations complete (using Thread.sleeps)
> Refactor to make CallQueue pluggable
> ------------------------------------
>
> Key: HADOOP-10278
> URL: https://issues.apache.org/jira/browse/HADOOP-10278
> Project: Hadoop Common
> Issue Type: Sub-task
> Components: ipc
> Reporter: Chris Li
> Attachments: HADOOP-10278-atomicref-adapter.patch,
> HADOOP-10278-atomicref-adapter.patch, HADOOP-10278-atomicref-rwlock.patch,
> HADOOP-10278-atomicref.patch, HADOOP-10278-atomicref.patch,
> HADOOP-10278-atomicref.patch, HADOOP-10278-atomicref.patch,
> HADOOP-10278.patch, HADOOP-10278.patch
>
>
> * Refactor CallQueue into an interface, base, and default implementation that
> matches today's behavior
> * Make the call queue impl configurable, keyed on port so that we minimize
> coupling
--
This message was sent by Atlassian JIRA
(v6.1.5#6160)