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

Liang Xie commented on HDFS-5776:
---------------------------------

[~arpitagarwal], thanks for your nice review!
bq.  The concern is too many thread pools created by multiple clients on the 
same node
take it easy, the default configuration: pool=0, that means no extra new 
threads be created by default. if a end user/application enable hedged read, 
they should know about this

bq. what do you think of not exposing the 
DFS_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE setting at all
IMHO, i personally prefer the current style, it's less risky, we had a bound 
queue and once reach the queue limit, we force to exec it in current thread. 
about the "internal upper bound", how much?  5000? 500000? or sth else? i think 
if enabling this feature explicitly, the end user/application should know a 
little backgroud at least, right? just like lots of hadoop timeout config 
parameter, i never see any internal upper bound impl at all...   but if you 
strongly insist on it, i can add.

bq. DFSClient#allowHedgedReads seems unnecessary
let's keep it there, it's more easier to understand for developer or end user.

bq. For DEFAULT_DFSCLIENT_HEDGED_READ_THRESHOLD_MILLIS - can we add an inbuilt 
minimum delay to defeat applications that set it too low or even zero
my opinion is same as the above one. since we don't have any knowledge about 
end-user's storage configuration, just image if they have a fast flash(with 
HDFS-2832 enabled), say fusionio, probably one real disk read only cost tens of 
microseconds, how should we decide a good minimum defeat setting? so i don't 
like to add it, i totally get your kindly concern:)

bq. DFSInputStream#chooseDataNode - can the call to getBestNodeErrorString go 
inside the "if (failures >=..." clause?
another log statement also use it, see "DFSClient.LOG.info("Could not obtain " 
+ block.getBlock...", so it's impossible here.

bq. #fetchBlockByteRange - can we rename retVal to something like addressPair?
good. let me rename it

bq. Do we still need the while loop still there in actualGetFromOneDataNode?
yes, but the loop is very very light,  only when some exceptions like 
AccessControlException/InvalidEncryptionKeyException/InvalidBlockTokenException 
happened, will do extra loop, and all those have a fast quit mechanism, like 
refetchToken/refetchEncryptionKey or disableLegacyBlockReaderLocal, so this 
loop will only be executed just a very few times:)

bq. There is already a while loop in fetchBlockByteRange enclosing the call to 
actualGetFromOneDataNode. Now we have a nested loop.
In the loop inside fetchBlockByteRange, the responsibily is picking another dn 
if there's IOException thrown from actualGetFromOneDataNode,  so not a fearful 
nested loop at all, take it easy:)

bq. Maybe I misunderstood the code flow but it looks like the way the while 
loops are nested it defeats the usage of refetchToken and refetchEncryptionKey. 
It looks like the intention was to limit the refetch to 1 across all retries, 
now we can refetch multiple times.
yes, you had a misunderstanding here. that's why i catch IOException fbae 
around fetchBlockAt. If we don't catch here, there will be always new refetch 
from outside loop and will have a spin loop

bq. Related to the previous, #actualGetFromOneDataNode, line 1026, - sorry I 
did not understand why the try-catch was added around the call to fetchBlockAt.
hope the above answer could make you clear?  hope my poor english doesn't make 
everything worse, haha:)

bq. #actualGetFromOneDataNode, line 1033 - the call to DFSClient.LOG.warn is 
deleted. Assume that was unintentional?
Gooood catch!

bq. Nitpick - some lines have whitespace-only changes.
i found several unnessessiry whitespaces existing, i just removed them to make 
more clear.

Really thanks all for review!!!

> Support 'hedged' reads in DFSClient
> -----------------------------------
>
>                 Key: HDFS-5776
>                 URL: https://issues.apache.org/jira/browse/HDFS-5776
>             Project: Hadoop HDFS
>          Issue Type: Improvement
>          Components: hdfs-client
>    Affects Versions: 3.0.0
>            Reporter: Liang Xie
>            Assignee: Liang Xie
>         Attachments: HDFS-5776-v2.txt, HDFS-5776-v3.txt, HDFS-5776-v4.txt, 
> HDFS-5776-v5.txt, HDFS-5776.txt
>
>
> This is a placeholder of hdfs related stuff backport from 
> https://issues.apache.org/jira/browse/HBASE-7509
> The quorum read ability should be helpful especially to optimize read outliers
> we can utilize "dfs.dfsclient.quorum.read.threshold.millis" & 
> "dfs.dfsclient.quorum.read.threadpool.size" to enable/disable the hedged read 
> ability from client side(e.g. HBase), and by using DFSQuorumReadMetrics, we 
> could export the interested metric valus into client system(e.g. HBase's 
> regionserver metric).
> The core logic is in pread code path, we decide to goto the original 
> fetchBlockByteRange or the new introduced fetchBlockByteRangeSpeculative per 
> the above config items.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)

Reply via email to