Uploaded image for project: 'Hadoop Map/Reduce'
  1. Hadoop Map/Reduce
  2. MAPREDUCE-5450

Unnecessary Configuration instantiation in IFileInputStream slows down merge - Port to branch-1

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Blocker
    • Resolution: Fixed
    • 1.1.0
    • 1.2.2
    • mrv1
    • None
    • Reviewed
    • Fixes blank Configuration object creation overhead by reusing the Job configuration in InMemoryReader.

    Description

      We are using hadoop-2.0.0+1357-1.cdh4.3.0.p0.21 with MRv1. After upgrade from 4.1.2 to 4.3.0, I have noticed some performance deterioration in our MR job in the Reduce phase. The MR job has usually 10 000 map tasks (10 000 files on input each about 100MB) and 6 000 reducers (one reducer per table region). I was trying to figure out what at which phase the slow down appears (firstly I suspected that the slow gathering of the 10000 map output files is the culprit) and found out that the problem is not reading the map output (the shuffle) but the sort/merge phase that follows - the last and actual reduce phase is fast. I have tried to up the io.sort.factor because I thought the lots of small files are being merged on disk, but again upping that to 1000 didnt do any difference. I have then printed the stack trace and found out that the problem is initialization of the org.apache.hadoop.mapred.IFileInputStream namely the creation of the Configuration object which is not propagated along from earlier context, see the stack trace:

      Thread 13332: (state = IN_NATIVE)

      • java.io.UnixFileSystem.getBooleanAttributes0(java.io.File) @bci=0 (Compiled frame; information may be imprecise)
      • java.io.UnixFileSystem.getBooleanAttributes(java.io.File) @bci=2, line=228 (Compiled frame)
      • java.io.File.exists() @bci=20, line=733 (Compiled frame)
      • sun.misc.URLClassPath$FileLoader.getResource(java.lang.String, boolean) @bci=136, line=999 (Compiled frame)
      • sun.misc.URLClassPath$FileLoader.findResource(java.lang.String, boolean) @bci=3, line=966 (Compiled frame)
      • sun.misc.URLClassPath.findResource(java.lang.String, boolean) @bci=17, line=146 (Compiled frame)
      • java.net.URLClassLoader$2.run() @bci=12, line=385 (Compiled frame)
      • java.security.AccessController.doPrivileged(java.security.PrivilegedAction, java.security.AccessControlContext) @bci=0 (Compiled frame)
      • java.net.URLClassLoader.findResource(java.lang.String) @bci=13, line=382 (Compiled frame)
      • java.lang.ClassLoader.getResource(java.lang.String) @bci=30, line=1002 (Compiled frame)
      • java.lang.ClassLoader.getResourceAsStream(java.lang.String) @bci=2, line=1192 (Compiled frame)
      • javax.xml.parsers.SecuritySupport$4.run() @bci=26, line=96 (Compiled frame)
      • java.security.AccessController.doPrivileged(java.security.PrivilegedAction) @bci=0 (Compiled frame)
      • javax.xml.parsers.SecuritySupport.getResourceAsStream(java.lang.ClassLoader, java.lang.String) @bci=10, line=89 (Compiled frame)
      • javax.xml.parsers.FactoryFinder.findJarServiceProvider(java.lang.String) @bci=38, line=250 (Interpreted frame)
      • javax.xml.parsers.FactoryFinder.find(java.lang.String, java.lang.String) @bci=273, line=223 (Interpreted frame)
      • javax.xml.parsers.DocumentBuilderFactory.newInstance() @bci=4, line=123 (Compiled frame)
      • org.apache.hadoop.conf.Configuration.loadResource(java.util.Properties, org.apache.hadoop.conf.Configuration$Resource, boolean) @bci=16, line=1890 (Compiled frame)
      • org.apache.hadoop.conf.Configuration.loadResources(java.util.Properties, java.util.ArrayList, boolean) @bci=49, line=1867 (Compiled frame)
      • org.apache.hadoop.conf.Configuration.getProps() @bci=43, line=1785 (Compiled frame)
      • org.apache.hadoop.conf.Configuration.get(java.lang.String) @bci=35, line=712 (Compiled frame)
      • org.apache.hadoop.conf.Configuration.getTrimmed(java.lang.String) @bci=2, line=731 (Compiled frame)
      • org.apache.hadoop.conf.Configuration.getBoolean(java.lang.String, boolean) @bci=2, line=1047 (Interpreted frame)
      • org.apache.hadoop.mapred.IFileInputStream.<init>(java.io.InputStream, long, org.apache.hadoop.conf.Configuration) @bci=111, line=93 (Interpreted frame)
      • org.apache.hadoop.mapred.IFile$Reader.<init>(org.apache.hadoop.conf.Configuration, org.apache.hadoop.fs.FSDataInputStream, long, org.apache.hadoop.io.compress.CompressionCodec, org.apache.hadoop.mapred.Counters$Counter) @bci=60, line=303 (Interpreted frame)
      • org.apache.hadoop.mapred.IFile$InMemoryReader.<init>(org.apache.hadoop.mapred.RamManager, org.apache.hadoop.mapred.TaskAttemptID, byte[], int, int) @bci=11, line=480 (Interpreted frame)
      • org.apache.hadoop.mapred.ReduceTask$ReduceCopier.createInMemorySegments(java.util.List, long) @bci=133, line=2416 (Interpreted frame)
      • org.apache.hadoop.mapred.ReduceTask$ReduceCopier.createKVIterator() @bci=669, line=2530 (Interpreted frame)
      • org.apache.hadoop.mapred.ReduceTask.run(org.apache.hadoop.mapred.JobConf, org.apache.hadoop.mapred.TaskUmbilicalProtocol) @bci=513, line=425 (Interpreted frame)
      • org.apache.hadoop.mapred.Child$4.run() @bci=29, line=268 (Interpreted frame)
      • java.security.AccessController.doPrivileged(java.security.PrivilegedExceptionAction, java.security.AccessControlContext) @bci=0 (Interpreted frame)
      • javax.security.auth.Subject.doAs(javax.security.auth.Subject, java.security.PrivilegedExceptionAction) @bci=42, line=396 (Interpreted frame)
      • org.apache.hadoop.security.UserGroupInformation.doAs(java.security.PrivilegedExceptionAction) @bci=14, line=1408 (Interpreted frame)
      • org.apache.hadoop.mapred.Child.main(java.lang.String[]) @bci=776, line=262 (Interpreted frame)

      A blank configuration object is created at IFileInputStream. I have made a test and found out that this operation costs about 10-15ms depending on the load on the system, because it goes to the local FS to load the properties!!! This is to my opinion a bug since in the context the configuration (of the job) is known and could be reused at that point. My problem (and every others who has big number of reducer and mapper tasks) is that for 10K map taks it does 10000 x 15 = 150 seconds just to find out that there is nothing to sort. The overhead should be normally zero.

      At this moment, the 10-15ms problem is amplified by 6 000 reducers so the bottom line is that my reduce phase is at least 1.6 hours longer than it should be.

      Attachments

        1. mapreduce-5450.txt
          2 kB
          Ted Yu
        2. MAPREDUCE-5450-1.1.0.txt
          1 kB
          Stanislav Barton

        Issue Links

          Activity

            People

              stanislav.barton Stanislav Barton
              stanislav.barton Stanislav Barton
              Votes:
              0 Vote for this issue
              Watchers:
              4 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: