[
https://issues.apache.org/jira/browse/HBASE-16393?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15444387#comment-15444387
]
binlijin commented on HBASE-16393:
----------------------------------
After dig into it, i find for a a storefile
DistributedFileSystem#listLocatedStatus(final Path p, final PathFilter filter)
actually have two rpc call.
{code}
@Override
protected RemoteIterator<LocatedFileStatus> listLocatedStatus(final Path p,
final PathFilter filter)
throws IOException {
{ // initializer
// Fully resolve symlinks in path first to avoid additional resolution
// round-trips as we fetch more batches of listings
src = getPathName(resolvePath(absF));
// fetch the first batch of entries in the directory
thisListing = dfs.listPaths(src, HdfsFileStatus.EMPTY_NAME, true);
statistics.incrementReadOps(1);
if (thisListing == null) { // the directory does not exist
throw new FileNotFoundException("File " + p + " does not exist.");
}
}
}
/**
* Return the fully-qualified path of path f resolving the path
* through any symlinks or mount point
* @param p path to be resolved
* @return fully qualified path
* @throws FileNotFoundException
*/
public Path resolvePath(final Path p) throws IOException {
checkPath(p);
return getFileStatus(p).getPath();
}
/**
* Returns the stat information about the file.
* @throws FileNotFoundException if the file does not exist.
*/
@Override
public FileStatus getFileStatus(Path f) throws IOException {
statistics.incrementReadOps(1);
Path absF = fixRelativePart(f);
return new FileSystemLinkResolver<FileStatus>() {
@Override
public FileStatus doCall(final Path p) throws IOException,
UnresolvedLinkException {
HdfsFileStatus fi = dfs.getFileInfo(getPathName(p));
if (fi != null) {
return fi.makeQualified(getUri(), p);
} else {
throw new FileNotFoundException("File does not exist: " + p);
}
}
@Override
public FileStatus next(final FileSystem fs, final Path p)
throws IOException {
return fs.getFileStatus(p);
}
}.resolve(this, absF);
}
{code}
> Improve computeHDFSBlocksDistribution
> -------------------------------------
>
> Key: HBASE-16393
> URL: https://issues.apache.org/jira/browse/HBASE-16393
> Project: HBase
> Issue Type: Improvement
> Reporter: binlijin
> Assignee: binlijin
> Attachments: HBASE-16393.patch
>
>
> With our cluster is big, i can see the balancer is slow from time to time.
> And the balancer will be called on master startup, so we can see the startup
> is slow also.
> The first thing i think whether if we can parallel compute different region's
> HDFSBlocksDistribution.
> The second i think we can improve compute single region's
> HDFSBlocksDistribution.
> When to compute a storefile's HDFSBlocksDistribution first we call
> FileSystem#getFileStatus(path) and then
> FileSystem#getFileBlockLocations(status, start, length), so two namenode rpc
> call for every storefile. Instead we can use FileSystem#listLocatedStatus to
> get a LocatedFileStatus for the information we need, so reduce the namenode
> rpc call to one. This can speed the computeHDFSBlocksDistribution, but also
> send out less rpc call to namenode.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)