[GitHub] [hadoop] sodonnel commented on a change in pull request #1028: HDFS-14617 - Improve fsimage load time by writing sub-sections to the fsimage index

2019-08-15 Thread GitBox
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 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 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 refList = parent.getLoaderContext()
   .getRefList();
+  ArrayList 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



[GitHub] [hadoop] sodonnel commented on a change in pull request #1028: HDFS-14617 - Improve fsimage load time by writing sub-sections to the fsimage index

2019-08-14 Thread GitBox
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_r313993233
 
 

 ##
 File path: 
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
 ##
 @@ -255,14 +345,28 @@ public int compare(FileSummary.Section s1, 
FileSummary.Section s2) {
 case INODE: {
   currentStep = new Step(StepType.INODES);
   prog.beginStep(Phase.LOADING_FSIMAGE, currentStep);
-  inodeLoader.loadINodeSection(in, prog, currentStep);
+  stageSubSections = getSubSectionsOfName(
+  subSections, SectionName.INODE_SUB);
+  if (loadInParallel && (stageSubSections.size() > 0)) {
+inodeLoader.loadINodeSectionInParallel(executorService,
+stageSubSections, summary.getCodec(), prog, currentStep);
+  } else {
+inodeLoader.loadINodeSection(in, prog, currentStep);
+  }
 }
   break;
 case INODE_REFERENCE:
   snapshotLoader.loadINodeReferenceSection(in);
   break;
 case INODE_DIR:
-  inodeLoader.loadINodeDirectorySection(in);
+  stageSubSections = getSubSectionsOfName(
+  subSections, SectionName.INODE_DIR_SUB);
+  if (loadInParallel && stageSubSections.size() > 0) {
 
 Review comment:
   I have a unit test which ensures a parallel image can be created and loaded. 
It would be fairly easy to create another test which generates a non-parallel 
image, validates it is non-parallel and then attempt to load it with parallel 
enabled. Do you think that would cover what we need?
   I also have a test that enables parallel and compression and then it 
verifies the parallel part is not used as compression disables 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



[GitHub] [hadoop] sodonnel commented on a change in pull request #1028: HDFS-14617 - Improve fsimage load time by writing sub-sections to the fsimage index

2019-08-14 Thread GitBox
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_r313991605
 
 

 ##
 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 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:
   @Hexiaoqiao Are you happy if we leave the synchronized blocks in place for 
the single threaded case? I don't believe it will cause any performance issues 
and it makes the code much cleaner than having two different paths for parallel 
and serial.


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



[GitHub] [hadoop] sodonnel commented on a change in pull request #1028: HDFS-14617 - Improve fsimage load time by writing sub-sections to the fsimage index

2019-08-14 Thread GitBox
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_r313941659
 
 

 ##
 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 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) {
+loadRootINode(p);
+  }
 } else {
   INode n = loadINode(p);
-  dir.addToInodeMap(n);
+  synchronized(this) {
+dir.addToInodeMap(n);
+  }
+}
+cntr ++;
+if (counter != null) {
+  counter.increment();
 }
-counter.increment();
   }
+  return cntr;
+}
+
+
+private void loadINodeSectionHeader(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);
+}
+
+void loadINodeSectionInParallel(ExecutorService service,
+ArrayList sections,
+String compressionCodec, StartupProgress prog,
+Step currentStep) throws IOException {
+  LOG.info("Loading the INode section in parallel with {} sub-sections",
+  sections.size());
+  CountDownLatch latch = new CountDownLatch(sections.size());
+  AtomicInteger totalLoaded = new AtomicInteger(0);
+  final CopyOnWriteArrayList exceptions =
+  new CopyOnWriteArrayList<>();
+
+  for (int i=0; i < sections.size(); i++) {
+FileSummary.Section s = sections.get(i);
+InputStream ins = parent.getInputStreamForSection(s, compressionCodec);
+if (i == 0) {
+  // The first inode section has a header which must be processed first
+  loadINodeSectionHeader(ins, prog, currentStep);
+}
+
+service.submit(new Runnable() {
+   public void run() {
+try {
+   totalLoaded.addAndGet(loadINodesInSection(ins, null));
+   

[GitHub] [hadoop] sodonnel commented on a change in pull request #1028: HDFS-14617 - Improve fsimage load time by writing sub-sections to the fsimage index

2019-08-14 Thread GitBox
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_r313806350
 
 

 ##
 File path: 
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
 ##
 @@ -525,6 +689,59 @@ long save(File file, FSImageCompression compression) 
throws IOException {
   }
 }
 
+private void enableSubSectionsIfRequired() {
+  boolean parallelEnabled = conf.getBoolean(
+  DFSConfigKeys.DFS_IMAGE_PARALLEL_LOAD_KEY,
+  DFSConfigKeys.DFS_IMAGE_PARALLEL_LOAD_DEFAULT);
+  int inodeThreshold = conf.getInt(
+  DFSConfigKeys.DFS_IMAGE_PARALLEL_INODE_THRESHOLD_KEY,
+  DFSConfigKeys.DFS_IMAGE_PARALLEL_INODE_THRESHOLD_DEFAULT);
+  int targetSections = conf.getInt(
+  DFSConfigKeys.DFS_IMAGE_PARALLEL_TARGET_SECTIONS_KEY,
+  DFSConfigKeys.DFS_IMAGE_PARALLEL_TARGET_SECTIONS_DEFAULT);
+  boolean compressionEnabled = conf.getBoolean(
+  DFSConfigKeys.DFS_IMAGE_COMPRESS_KEY,
+  DFSConfigKeys.DFS_IMAGE_COMPRESS_DEFAULT);
+
+  if (parallelEnabled) {
+if (compressionEnabled) {
+  LOG.warn("Parallel Image loading is not supported when {} is set to" 
+
+  " true. Parallel loading will be disabled.",
+  DFSConfigKeys.DFS_IMAGE_COMPRESS_KEY);
+  writeSubSections = false;
+  return;
+}
+if (targetSections <= 0) {
+  LOG.warn("{} is set to {}. It must be greater than zero. Setting to" 
+
+  "default of {}",
+  DFSConfigKeys.DFS_IMAGE_PARALLEL_TARGET_SECTIONS_KEY,
+  targetSections,
+  DFSConfigKeys.DFS_IMAGE_PARALLEL_TARGET_SECTIONS_DEFAULT);
+  targetSections =
+  DFSConfigKeys.DFS_IMAGE_PARALLEL_TARGET_SECTIONS_DEFAULT;
+}
+if (inodeThreshold <= 0) {
+  LOG.warn("{} is set to {}. It must be greater than zero. Setting to" 
+
+  "default of {}",
 
 Review comment:
   Fixed.


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



[GitHub] [hadoop] sodonnel commented on a change in pull request #1028: HDFS-14617 - Improve fsimage load time by writing sub-sections to the fsimage index

2019-08-14 Thread GitBox
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_r313806054
 
 

 ##
 File path: 
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
 ##
 @@ -294,6 +368,19 @@ public int compare(FileSummary.Section s1, 
FileSummary.Section s2) {
* a particular step to be started for once.
*/
   Step currentStep = null;
+  boolean loadInParallel =
+  conf.getBoolean(DFSConfigKeys.DFS_IMAGE_PARALLEL_LOAD_KEY,
+  DFSConfigKeys.DFS_IMAGE_PARALLEL_LOAD_DEFAULT);
+  // TODO - check for compression and if enabled disable parallel
+
+  ExecutorService executorService = null;
+  ArrayList subSections =
+  getAndRemoveSubSections(sections);
+  if (loadInParallel) {
+executorService = Executors.newFixedThreadPool(
+conf.getInt(DFSConfigKeys.DFS_IMAGE_PARALLEL_THREADS_KEY,
+DFSConfigKeys.DFS_IMAGE_PARALLEL_THREADS_DEFAULT));
 
 Review comment:
   I added a log message here, and validated the thread count setting is not 
less than 1, otherwise I reset it to the default (4). I also pulled the code to 
do this check and to create the executor into a private method to reduce the 
noise in the already too long loadInternal method.


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



[GitHub] [hadoop] sodonnel commented on a change in pull request #1028: HDFS-14617 - Improve fsimage load time by writing sub-sections to the fsimage index

2019-08-14 Thread GitBox
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_r313799031
 
 

 ##
 File path: 
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
 ##
 @@ -217,33 +272,147 @@ 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();
+}
+  }
 
 Review comment:
   I have added a message like this to both adding the inode and inode 
references to the directory:
   ```
   LOG.warn("Failed to add the inode reference {} to the directory {}",
   ref.getId(), p.getId());
   ```
   I opted to log only the inode and directory "inode id" as I am not sure if 
the system will be able to resolve the full path of an inode or directory at 
this stage, as it is still loading the image. Also this "should never happen" 
so hopefully we will not see these messages in practice, but if we do, it will 
likely require manually investigating an image corruption, so the ID numbers 
should be enough to start with that.


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



[GitHub] [hadoop] sodonnel commented on a change in pull request #1028: HDFS-14617 - Improve fsimage load time by writing sub-sections to the fsimage index

2019-08-14 Thread GitBox
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_r313794899
 
 

 ##
 File path: 
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
 ##
 @@ -217,33 +272,147 @@ 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) {
 
 Review comment:
   Done.


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



[GitHub] [hadoop] sodonnel commented on a change in pull request #1028: HDFS-14617 - Improve fsimage load time by writing sub-sections to the fsimage index

2019-08-13 Thread GitBox
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_r313493210
 
 

 ##
 File path: 
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
 ##
 @@ -250,6 +257,73 @@ void load(File file) throws IOException {
   }
 }
 
+/**
+ * Given a FSImage FileSummary.section, return a LimitInput stream set to
+ * the starting position of the section and limited to the section length.
+ * @param section The FileSummary.Section containing the offset and length
+ * @param compressionCodec The compression codec in use, if any
+ * @return An InputStream for the given section
+ * @throws IOException
+ */
+public InputStream getInputStreamForSection(FileSummary.Section section,
+String compressionCodec)
+throws IOException {
+  FileInputStream fin = new FileInputStream(filename);
+  FileChannel channel = fin.getChannel();
+  channel.position(section.getOffset());
+  InputStream in = new BufferedInputStream(new LimitInputStream(fin,
+  section.getLength()));
+
+  in = FSImageUtil.wrapInputStreamForCompression(conf,
+  compressionCodec, in);
+  return in;
+}
+
+/**
+ * Takes an ArrayList of Section's and removes all Section's whose
+ * name ends in _SUB, indicating they are sub-sections. The original
+ * array list is modified and a new list of the removed Section's is
+ * returned.
+ * @param sections Array List containing all Sections and Sub Sections
+ * in the image.
+ * @return ArrayList of the sections removed, or an empty list if none are
+ * removed.
+ */
+private ArrayList getAndRemoveSubSections(
+ArrayList sections) {
+  ArrayList subSections = new ArrayList<>();
+  Iterator iter = sections.iterator();
+  while (iter.hasNext()) {
+FileSummary.Section s = iter.next();
+String name = s.getName();
+if (name.matches(".*_SUB$")) {
+  subSections.add(s);
+  iter.remove();
+}
+  }
+  return subSections;
+}
+
+/**
+ * Given an ArrayList of Section's, return all Section's with the given
+ * name, or an empty list if none are found.
+ * @param sections ArrayList of the Section's to search though
+ * @param name The name of the Sections to search for
+ * @return ArrayList of the sections matching the given name
+ */
+private ArrayList getSubSectionsOfName(
+ArrayList sections, SectionName name) {
+  ArrayList subSec = new ArrayList<>();
+  for (FileSummary.Section s : sections) {
+String n = s.getName();
+SectionName sectionName = SectionName.fromString(n);
+if (sectionName == name) {
+  subSec.add(s);
+}
+  }
+  return subSec;
+}
+
 
 Review comment:
   It was a long method already, and this has made it longer. I will see if I 
can find a way to refactor it without introducing too much change.


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



[GitHub] [hadoop] sodonnel commented on a change in pull request #1028: HDFS-14617 - Improve fsimage load time by writing sub-sections to the fsimage index

2019-08-13 Thread GitBox
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_r313491763
 
 

 ##
 File path: 
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
 ##
 @@ -294,6 +368,19 @@ public int compare(FileSummary.Section s1, 
FileSummary.Section s2) {
* a particular step to be started for once.
*/
   Step currentStep = null;
+  boolean loadInParallel =
+  conf.getBoolean(DFSConfigKeys.DFS_IMAGE_PARALLEL_LOAD_KEY,
+  DFSConfigKeys.DFS_IMAGE_PARALLEL_LOAD_DEFAULT);
+  // TODO - check for compression and if enabled disable parallel
 
 Review comment:
   I have addressed this TODO and also refactored the code a little to allow 
reuse between the saving and loading sections.


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



[GitHub] [hadoop] sodonnel commented on a change in pull request #1028: HDFS-14617 - Improve fsimage load time by writing sub-sections to the fsimage index

2019-08-13 Thread GitBox
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_r313482053
 
 

 ##
 File path: 
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
 ##
 @@ -525,6 +689,59 @@ long save(File file, FSImageCompression compression) 
throws IOException {
   }
 }
 
+private void enableSubSectionsIfRequired() {
+  boolean parallelEnabled = conf.getBoolean(
+  DFSConfigKeys.DFS_IMAGE_PARALLEL_LOAD_KEY,
+  DFSConfigKeys.DFS_IMAGE_PARALLEL_LOAD_DEFAULT);
+  int inodeThreshold = conf.getInt(
+  DFSConfigKeys.DFS_IMAGE_PARALLEL_INODE_THRESHOLD_KEY,
+  DFSConfigKeys.DFS_IMAGE_PARALLEL_INODE_THRESHOLD_DEFAULT);
+  int targetSections = conf.getInt(
+  DFSConfigKeys.DFS_IMAGE_PARALLEL_TARGET_SECTIONS_KEY,
+  DFSConfigKeys.DFS_IMAGE_PARALLEL_TARGET_SECTIONS_DEFAULT);
+  boolean compressionEnabled = conf.getBoolean(
+  DFSConfigKeys.DFS_IMAGE_COMPRESS_KEY,
+  DFSConfigKeys.DFS_IMAGE_COMPRESS_DEFAULT);
+
+  if (parallelEnabled) {
+if (compressionEnabled) {
+  LOG.warn("Parallel Image loading is not supported when {} is set to" 
+
+  " true. Parallel loading will be disabled.",
+  DFSConfigKeys.DFS_IMAGE_COMPRESS_KEY);
+  writeSubSections = false;
+  return;
+}
+if (targetSections <= 0) {
+  LOG.warn("{} is set to {}. It must be greater than zero. Setting to" 
+
 
 Review comment:
   Fixed.


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



[GitHub] [hadoop] sodonnel commented on a change in pull request #1028: HDFS-14617 - Improve fsimage load time by writing sub-sections to the fsimage index

2019-08-13 Thread GitBox
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_r313481427
 
 

 ##
 File path: 
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
 ##
 @@ -525,6 +689,59 @@ long save(File file, FSImageCompression compression) 
throws IOException {
   }
 }
 
+private void enableSubSectionsIfRequired() {
+  boolean parallelEnabled = conf.getBoolean(
+  DFSConfigKeys.DFS_IMAGE_PARALLEL_LOAD_KEY,
+  DFSConfigKeys.DFS_IMAGE_PARALLEL_LOAD_DEFAULT);
+  int inodeThreshold = conf.getInt(
+  DFSConfigKeys.DFS_IMAGE_PARALLEL_INODE_THRESHOLD_KEY,
+  DFSConfigKeys.DFS_IMAGE_PARALLEL_INODE_THRESHOLD_DEFAULT);
+  int targetSections = conf.getInt(
+  DFSConfigKeys.DFS_IMAGE_PARALLEL_TARGET_SECTIONS_KEY,
+  DFSConfigKeys.DFS_IMAGE_PARALLEL_TARGET_SECTIONS_DEFAULT);
+  boolean compressionEnabled = conf.getBoolean(
+  DFSConfigKeys.DFS_IMAGE_COMPRESS_KEY,
+  DFSConfigKeys.DFS_IMAGE_COMPRESS_DEFAULT);
+
+  if (parallelEnabled) {
+if (compressionEnabled) {
+  LOG.warn("Parallel Image loading is not supported when {} is set to" 
+
+  " true. Parallel loading will be disabled.",
 
 Review comment:
   I have added a note to hdfs-default.xml for this.


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



[GitHub] [hadoop] sodonnel commented on a change in pull request #1028: HDFS-14617 - Improve fsimage load time by writing sub-sections to the fsimage index

2019-07-24 Thread GitBox
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_r306740049
 
 

 ##
 File path: 
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
 ##
 @@ -187,6 +195,73 @@ void load(File file) throws IOException {
   }
 }
 
+/**
+ * Given a FSImage FileSummary.section, return a LimitInput stream set to
+ * the starting position of the section and limited to the section length
+ * @param section The FileSummary.Section containing the offset and length
+ * @param compressionCodec The compression codec in use, if any
 
 Review comment:
   Looks like I missed the 'return' description. I have added it now.


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



[GitHub] [hadoop] sodonnel commented on a change in pull request #1028: HDFS-14617 - Improve fsimage load time by writing sub-sections to the fsimage index

2019-07-24 Thread GitBox
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_r306739479
 
 

 ##
 File path: 
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
 ##
 @@ -255,14 +345,28 @@ public int compare(FileSummary.Section s1, 
FileSummary.Section s2) {
 case INODE: {
   currentStep = new Step(StepType.INODES);
   prog.beginStep(Phase.LOADING_FSIMAGE, currentStep);
-  inodeLoader.loadINodeSection(in, prog, currentStep);
+  stageSubSections = getSubSectionsOfName(
+  subSections, SectionName.INODE_SUB);
+  if (loadInParallel && (stageSubSections.size() > 0)) {
+inodeLoader.loadINodeSectionInParallel(executorService,
+stageSubSections, summary.getCodec(), prog, currentStep);
+  } else {
+inodeLoader.loadINodeSection(in, prog, currentStep);
+  }
 }
   break;
 case INODE_REFERENCE:
   snapshotLoader.loadINodeReferenceSection(in);
   break;
 case INODE_DIR:
-  inodeLoader.loadINodeDirectorySection(in);
+  stageSubSections = getSubSectionsOfName(
+  subSections, SectionName.INODE_DIR_SUB);
+  if (loadInParallel && stageSubSections.size() > 0) {
 
 Review comment:
   Yes, I need to figure out how to add some tests for this overall change.


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



[GitHub] [hadoop] sodonnel commented on a change in pull request #1028: HDFS-14617 - Improve fsimage load time by writing sub-sections to the fsimage index

2019-07-24 Thread GitBox
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_r306739148
 
 

 ##
 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 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) {
+loadRootINode(p);
+  }
 } else {
   INode n = loadINode(p);
-  dir.addToInodeMap(n);
+  synchronized(this) {
+dir.addToInodeMap(n);
+  }
+}
+cntr ++;
+if (counter != null) {
+  counter.increment();
 }
-counter.increment();
   }
+  return cntr;
+}
+
+
+private void loadINodeSectionHeader(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);
+}
+
+void loadINodeSectionInParallel(ExecutorService service,
+ArrayList sections,
+String compressionCodec, StartupProgress prog,
+Step currentStep) throws IOException {
+  LOG.info("Loading the INode section in parallel with {} sub-sections",
+  sections.size());
+  CountDownLatch latch = new CountDownLatch(sections.size());
+  AtomicInteger totalLoaded = new AtomicInteger(0);
+  final CopyOnWriteArrayList exceptions =
+  new CopyOnWriteArrayList<>();
+
+  for (int i=0; i < sections.size(); i++) {
+FileSummary.Section s = sections.get(i);
+InputStream ins = parent.getInputStreamForSection(s, compressionCodec);
+if (i == 0) {
+  // The first inode section has a header which must be processed first
+  loadINodeSectionHeader(ins, prog, currentStep);
+}
+
+service.submit(new Runnable() {
+   public void run() {
+try {
+   totalLoaded.addAndGet(loadINodesInSection(ins, null));
+   

[GitHub] [hadoop] sodonnel commented on a change in pull request #1028: HDFS-14617 - Improve fsimage load time by writing sub-sections to the fsimage index

2019-07-24 Thread GitBox
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 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



[GitHub] [hadoop] sodonnel commented on a change in pull request #1028: HDFS-14617 - Improve fsimage load time by writing sub-sections to the fsimage index

2019-07-24 Thread GitBox
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_r306736876
 
 

 ##
 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();
+}
+  }
+
 
 Review comment:
   Fixed.


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



[GitHub] [hadoop] sodonnel commented on a change in pull request #1028: HDFS-14617 - Improve fsimage load time by writing sub-sections to the fsimage index

2019-07-24 Thread GitBox
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_r306736414
 
 

 ##
 File path: 
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
 ##
 @@ -389,21 +496,26 @@ private void loadErasureCodingSection(InputStream in)
 
   public static final class Saver {
 public static final int CHECK_CANCEL_INTERVAL = 4096;
+public static boolean WRITE_SUB_SECTIONS = false;
 
 Review comment:
   I agree we should make this change.  I was originally copying the pattern 
set by CHECK_CANCEL_INTERVAL, but best to avoid the constants I think. I have 
made this change.


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



[GitHub] [hadoop] sodonnel commented on a change in pull request #1028: HDFS-14617 - Improve fsimage load time by writing sub-sections to the fsimage index

2019-07-24 Thread GitBox
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_r306709830
 
 

 ##
 File path: 
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
 ##
 @@ -462,6 +619,60 @@ long save(File file, FSImageCompression compression) 
throws IOException {
   }
 }
 
+private void enableSubSectionsIfRequired() {
+  boolean parallelEnabled = conf.getBoolean(
+  DFSConfigKeys.DFS_IMAGE_PARALLEL_LOAD_KEY,
+  DFSConfigKeys.DFS_IMAGE_PARALLEL_LOAD_DEFAULT);
+  int inodeThreshold = conf.getInt(
+  DFSConfigKeys.DFS_IMAGE_PARALLEL_INODE_THRESHOLD_KEY,
+  DFSConfigKeys.DFS_IMAGE_PARALLEL_INODE_THRESHOLD_DEFAULT);
+  int targetSections = conf.getInt(
+  DFSConfigKeys.DFS_IMAGE_PARALLEL_TARGET_SECTIONS_KEY,
+  DFSConfigKeys.DFS_IMAGE_PARALLEL_TARGET_SECTIONS_DEFAULT);
+  boolean compressionEnabled = conf.getBoolean(
+  DFSConfigKeys.DFS_IMAGE_COMPRESS_KEY,
+  DFSConfigKeys.DFS_IMAGE_COMPRESS_DEFAULT);
+
 
 Review comment:
   Fixed.


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



[GitHub] [hadoop] sodonnel commented on a change in pull request #1028: HDFS-14617 - Improve fsimage load time by writing sub-sections to the fsimage index

2019-07-24 Thread GitBox
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_r306708881
 
 

 ##
 File path: 
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
 ##
 @@ -462,6 +619,60 @@ long save(File file, FSImageCompression compression) 
throws IOException {
   }
 }
 
+private void enableSubSectionsIfRequired() {
+  boolean parallelEnabled = conf.getBoolean(
+  DFSConfigKeys.DFS_IMAGE_PARALLEL_LOAD_KEY,
+  DFSConfigKeys.DFS_IMAGE_PARALLEL_LOAD_DEFAULT);
+  int inodeThreshold = conf.getInt(
+  DFSConfigKeys.DFS_IMAGE_PARALLEL_INODE_THRESHOLD_KEY,
+  DFSConfigKeys.DFS_IMAGE_PARALLEL_INODE_THRESHOLD_DEFAULT);
+  int targetSections = conf.getInt(
+  DFSConfigKeys.DFS_IMAGE_PARALLEL_TARGET_SECTIONS_KEY,
+  DFSConfigKeys.DFS_IMAGE_PARALLEL_TARGET_SECTIONS_DEFAULT);
+  boolean compressionEnabled = conf.getBoolean(
+  DFSConfigKeys.DFS_IMAGE_COMPRESS_KEY,
+  DFSConfigKeys.DFS_IMAGE_COMPRESS_DEFAULT);
+
+
+  if (parallelEnabled) {
+if (compressionEnabled) {
+  LOG.warn("Parallel Image loading is not supported when {} is set to" 
+
+  " true. Parallel loading will be disabled.",
+  DFSConfigKeys.DFS_IMAGE_COMPRESS_KEY);
+  WRITE_SUB_SECTIONS = false;
+  return;
+}
+if (targetSections <= 0) {
+  LOG.warn("{} is set to {}. It must be greater than zero. Setting to" 
+
+  "default of {}",
+  DFSConfigKeys.DFS_IMAGE_PARALLEL_TARGET_SECTIONS_KEY,
+  targetSections,
+  DFSConfigKeys.DFS_IMAGE_PARALLEL_TARGET_SECTIONS_DEFAULT);
+  targetSections =
+  DFSConfigKeys.DFS_IMAGE_PARALLEL_TARGET_SECTIONS_DEFAULT;
+}
+if (inodeThreshold <= 0) {
+  LOG.warn("{} is set to {}. It must be greater than zero. Setting to" 
+
+  "default of {}",
+  DFSConfigKeys.DFS_IMAGE_PARALLEL_INODE_THRESHOLD_KEY,
+  targetSections,
 
 Review comment:
   Correct - well spotted! I have fixed this.


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