Bryan Beaudreault created HDFS-16262:
----------------------------------------

             Summary: Async refresh of cached locations in DFSInputStream
                 Key: HDFS-16262
                 URL: https://issues.apache.org/jira/browse/HDFS-16262
             Project: Hadoop HDFS
          Issue Type: Improvement
            Reporter: Bryan Beaudreault
            Assignee: Bryan Beaudreault


HDFS-15119 added the ability to invalidate cached block locations in 
DFSInputStream. As written, the feature will affect all DFSInputStreams 
regardless of whether they need it or not. The invalidation also only applies 
on the next request, so the next request will pay the cost of calling openInfo 
before reading the data.

I'm working on a feature for HBase which enables efficient healing of locality 
through Balancer-style low level block moves (HBASE-26250). I'd like to utilize 
the idea started in HDFS-15119 in order to update DFSInputStreams after blocks 
have been moved to local hosts.

I was considering using the feature as is, but some of our clusters are quite 
large and I'm concerned about the impact on the namenode:
 * We have some clusters with over 350k StoreFiles, so that'd be 350k 
DFSInputStreams. With such a large number and very active usage, having the 
refresh be in-line makes it too hard to ensure we don't DDOS the NameNode.
 * Currently we need to pay the price of openInfo the next time a 
DFSInputStream is invoked. Moving that async would minimize the latency hit. 
Also, some StoreFiles might be far less frequently accessed, so they may live 
on for a long time before ever refreshing. We'd like to be able to know that 
all DFSInputStreams are refreshed by a given time.
 * We may have 350k files, but only a small percentage of them are ever 
non-local at a given time. Refreshing only if necessary will save a lot of work.

In order to make this as painless to end users as possible, I'd like to:
 * Update the implementation to utilize an async thread for managing refreshes. 
This will give more control over rate limiting across all DFSInputStreams in a 
DFSClient, and also ensure that all DFSInputStreams are refreshed.
 * Only refresh files which are lacking a local replica or have known deadNodes 
to be cleaned up

 

 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

---------------------------------------------------------------------
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org

Reply via email to