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_r314259665
########## File path: hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java ########## @@ -197,16 +203,66 @@ public static void updateBlocksMap(INodeFile file, BlockManager bm) { private final FSDirectory dir; private final FSNamesystem fsn; private final FSImageFormatProtobuf.Loader parent; + private ReentrantLock cacheNameMapLock; + private ReentrantLock blockMapLock; Loader(FSNamesystem fsn, final FSImageFormatProtobuf.Loader parent) { this.fsn = fsn; this.dir = fsn.dir; this.parent = parent; + cacheNameMapLock = new ReentrantLock(true); + blockMapLock = new ReentrantLock(true); + } + + void loadINodeDirectorySectionInParallel(ExecutorService service, + ArrayList<FileSummary.Section> sections, String compressionCodec) + throws IOException { + LOG.info("Loading the INodeDirectory section in parallel with {} sub-" + + "sections", sections.size()); + CountDownLatch latch = new CountDownLatch(sections.size()); + final CopyOnWriteArrayList<IOException> exceptions = + new CopyOnWriteArrayList<>(); + for (FileSummary.Section s : sections) { + service.submit(new Runnable() { + public void run() { + InputStream ins = null; + try { + ins = parent.getInputStreamForSection(s, + compressionCodec); + loadINodeDirectorySection(ins); + } catch (Exception e) { + LOG.error("An exception occurred loading INodeDirectories in " + + "parallel", e); + exceptions.add(new IOException(e)); + } finally { + latch.countDown(); + try { + ins.close(); + } catch (IOException ioe) { + LOG.warn("Failed to close the input stream, ignoring", ioe); + } + } + } + }); + } + try { + latch.await(); + } catch (InterruptedException e) { + LOG.error("Interrupted waiting for countdown latch", e); + throw new IOException(e); + } + if (exceptions.size() != 0) { + LOG.error("{} exceptions occurred loading INodeDirectories", + exceptions.size()); + throw exceptions.get(0); + } + LOG.info("Completed loading all INodeDirectory sub-sections"); } void loadINodeDirectorySection(InputStream in) throws IOException { final List<INodeReference> refList = parent.getLoaderContext() .getRefList(); + ArrayList<INode> inodeList = new ArrayList<>(); Review comment: I missed this comment earlier. I have changed the 1000 to a constant. For the cache in used, I have no concerns. Its a filename cache where all the file names are loaded to it at startup time, so the same string can be re-used if there are multiple files with the same name. It is not an LRU cache or anything like that, but is a hashmap of filenames used to reduce the overall heap used in the namenode. Loading it in batch like this will not affect the usefulness of it. ---------------------------------------------------------------- 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