sodonnel commented on a change in pull request #1028: HDFS-14617 - Improve fsimage load time by writing sub-sections to the fsimage index URL: https://github.com/apache/hadoop/pull/1028#discussion_r306737322
########## File path: hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java ########## @@ -217,33 +273,151 @@ void loadINodeDirectorySection(InputStream in) throws IOException { INodeDirectory p = dir.getInode(e.getParent()).asDirectory(); for (long id : e.getChildrenList()) { INode child = dir.getInode(id); - addToParent(p, child); + if (addToParent(p, child)) { + if (child.isFile()) { + inodeList.add(child); + } + if (inodeList.size() >= 1000) { + addToCacheAndBlockMap(inodeList); + inodeList.clear(); + } + } + } + for (int refId : e.getRefChildrenList()) { INodeReference ref = refList.get(refId); - addToParent(p, ref); + if (addToParent(p, ref)) { + if (ref.isFile()) { + inodeList.add(ref); + } + if (inodeList.size() >= 1000) { + addToCacheAndBlockMap(inodeList); + inodeList.clear(); + } + } } } + addToCacheAndBlockMap(inodeList); + } + + private void addToCacheAndBlockMap(ArrayList<INode> inodeList) { + try { + cacheNameMapLock.lock(); + for (INode i : inodeList) { + dir.cacheName(i); + } + } finally { + cacheNameMapLock.unlock(); + } + + try { + blockMapLock.lock(); + for (INode i : inodeList) { + updateBlocksMap(i.asFile(), fsn.getBlockManager()); + } + } finally { + blockMapLock.unlock(); + } } void loadINodeSection(InputStream in, StartupProgress prog, Step currentStep) throws IOException { - INodeSection s = INodeSection.parseDelimitedFrom(in); - fsn.dir.resetLastInodeId(s.getLastInodeId()); - long numInodes = s.getNumInodes(); - LOG.info("Loading " + numInodes + " INodes."); - prog.setTotal(Phase.LOADING_FSIMAGE, currentStep, numInodes); + loadINodeSectionHeader(in, prog, currentStep); Counter counter = prog.getCounter(Phase.LOADING_FSIMAGE, currentStep); - for (int i = 0; i < numInodes; ++i) { + int totalLoaded = loadINodesInSection(in, counter); + LOG.info("Successfully loaded {} inodes", totalLoaded); + } + + private int loadINodesInSection(InputStream in, Counter counter) + throws IOException { + // As the input stream is a LimitInputStream, the reading will stop when + // EOF is encountered at the end of the stream. + int cntr = 0; + while (true) { INodeSection.INode p = INodeSection.INode.parseDelimitedFrom(in); + if (p == null) { + break; + } if (p.getId() == INodeId.ROOT_INODE_ID) { - loadRootINode(p); + synchronized(this) { Review comment: The synchronized call does not seem to add any significant overhead in the single threaded case, so it would make the code cleaner to just leave it in place, rather than having a branch for parallel and serial. What do you think? ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services --------------------------------------------------------------------- To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org