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

Reducer's Maximum Shuffle Buffer Size should be enlarged for 64bit JVM

    XMLWordPrintableJSON

Details

    • Improvement
    • Status: Resolved
    • Major
    • Resolution: Duplicate
    • 0.20.2, 1.0.3
    • None
    • None
    • Especially for 64bit JVM

    • ReduceTask Heap JVM Shuffle

    Description

      In hadoop-0.20.2, hadoop-1.0.3 or other versions, reducer's shuffle buffer size cannot exceed 2048MB (i.e., Integer.MAX_VALUE). This is reasonable for 32bit JVM.
      But for 64bit JVM, although reducer's JVM size can be set more than 2048MB (e.g., mapred.child.java.opts=-Xmx4000m), the heap size used for shuffle buffer is at most "2048MB * maxInMemCopyUse (default 0.7)" not "4000MB * maxInMemCopyUse".

      So the pointed piece of code in ReduceTask.java needs modification for 64bit JVM.
      ---------------------------------------------------------------------------------------
      private final long maxSize;
      private final long maxSingleShuffleLimit;

      private long size = 0;

      private Object dataAvailable = new Object();
      private long fullSize = 0;
      private int numPendingRequests = 0;
      private int numRequiredMapOutputs = 0;
      private int numClosed = 0;
      private boolean closed = false;

      public ShuffleRamManager(Configuration conf) throws IOException {
      final float maxInMemCopyUse =
      conf.getFloat("mapred.job.shuffle.input.buffer.percent", 0.70f);
      if (maxInMemCopyUse > 1.0 || maxInMemCopyUse < 0.0)

      { throw new IOException("mapred.job.shuffle.input.buffer.percent" + maxInMemCopyUse); }

      // Allow unit tests to fix Runtime memory
      --> maxSize = (int)(conf.getInt("mapred.job.reduce.total.mem.bytes",
      --> (int)Math.min(Runtime.getRuntime().maxMemory(), Integer.MAX_VALUE))
      --> * maxInMemCopyUse);
      maxSingleShuffleLimit = (long)(maxSize * MAX_SINGLE_SHUFFLE_SEGMENT_FRACTION);
      LOG.info("ShuffleRamManager: MemoryLimit=" + maxSize +
      ", MaxSingleShuffleLimit=" + maxSingleShuffleLimit);
      }

      Attachments

        1. MAPREDUCE-4883.patch
          4 kB
          Haifeng Chen

        Issue Links

          Activity

            People

              jerrychenhf Haifeng Chen
              jerrylead Lijie Xu
              Votes:
              0 Vote for this issue
              Watchers:
              13 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved:

                Time Tracking

                  Estimated:
                  Original Estimate - 12h
                  12h
                  Remaining:
                  Remaining Estimate - 12h
                  12h
                  Logged:
                  Time Spent - Not Specified
                  Not Specified