Hadoop Common
  1. Hadoop Common
  2. HADOOP-4802

RPC Server send buffer retains size of largest response ever sent

    Details

    • Type: Bug Bug
    • Status: Resolved
    • Priority: Major Major
    • Resolution: Duplicate
    • Affects Version/s: 0.18.2, 0.19.0
    • Fix Version/s: 0.20.2
    • Component/s: ipc
    • Labels:
      None

      Description

      The stack-based ByteArrayOutputStream in Server.Hander is reset each time through the run loop. This will set the BAOS 'size' back to zero but the allocated backing buffer is unaltered. If during an Handlers' lifecycle, any particular RPC response was fat – Megabytes, even – the buffer expands during the write to accommodate the particular response but then never shrinks subsequently. If a hosting Server has had more than one 'fat payload' occurrence, the resultant occupied heap can provoke memory woes (See https://issues.apache.org/jira/browse/HBASE-900?focusedCommentId=12654009#action_12654009 for an extreme example; occasional payloads of 20-50MB with 30 handlers robbed the heap of 700MB).

        Issue Links

          Activity

          Hide
          stack added a comment -

          Suggested fix: reset BAOS if larger than N * initial buffer size.

          Show
          stack added a comment - Suggested fix: reset BAOS if larger than N * initial buffer size.
          Hide
          Doug Cutting added a comment -

          I'd rather not see new configuration options for this. We should rather try hard to pick values that work well in all cases, if possible.

          I wonder if the simpler change of not re-using the buffer might work just as well. If we simply allocate a new buffer per request whose initial size is 10k. I would be surprised if that allocation would be a significant RPC bottleneck, but we should of course test that with a benchmark.

          Show
          Doug Cutting added a comment - I'd rather not see new configuration options for this. We should rather try hard to pick values that work well in all cases, if possible. I wonder if the simpler change of not re-using the buffer might work just as well. If we simply allocate a new buffer per request whose initial size is 10k. I would be surprised if that allocation would be a significant RPC bottleneck, but we should of course test that with a benchmark.
          Hide
          Raghu Angadi added a comment -

          I was thinking on the same lines. Allocation per request would not hurt (mostly because various other extra costs). Coule of improvements by subclassing BAOS:

          1. Keep a fixed buffer 10kb. inside reset() replace the previous buffer with this. This avoids allocation for most RPCs.
          2. extra benefit : this would allow us to avoid a copy if the response could be written to socket in-line (common case). Right now server always copies (through asByteArray()).

          Note that this jira will increase the buffer copies for large (multi-MB) responses because of copy each time buffer expands. (Partly negated by copies reduced by HADOOP-4797, since it takes multiple write() invocations to write all the data. in jdk, each write copies all the data!). A future improvement could be to write our own stream that uses a list of buffers.

          Show
          Raghu Angadi added a comment - I was thinking on the same lines. Allocation per request would not hurt (mostly because various other extra costs). Coule of improvements by subclassing BAOS: Keep a fixed buffer 10kb. inside reset() replace the previous buffer with this. This avoids allocation for most RPCs. extra benefit : this would allow us to avoid a copy if the response could be written to socket in-line (common case). Right now server always copies (through asByteArray()). Note that this jira will increase the buffer copies for large (multi-MB) responses because of copy each time buffer expands. (Partly negated by copies reduced by HADOOP-4797 , since it takes multiple write() invocations to write all the data. in jdk, each write copies all the data!). A future improvement could be to write our own stream that uses a list of buffers.
          Hide
          Raghu Angadi added a comment -

          > extra benefit : this would allow us to avoid a copy if the response could be written to socket in-line (common case).

          Once we have this (here or in a future jira), the fixed buffer could be a direct buffer and that will remove yet another copy.

          Show
          Raghu Angadi added a comment - > extra benefit : this would allow us to avoid a copy if the response could be written to socket in-line (common case). Once we have this (here or in a future jira), the fixed buffer could be a direct buffer and that will remove yet another copy.
          Hide
          stack added a comment -

          Thanks for the feedback lads.

          I like your suggestions Raghu. This patch removes configuration and makes an attempt at #1. #2 and the third suggestion of direct buffer to socket would take more work (though we should do them, yeah).

          Show
          stack added a comment - Thanks for the feedback lads. I like your suggestions Raghu. This patch removes configuration and makes an attempt at #1. #2 and the third suggestion of direct buffer to socket would take more work (though we should do them, yeah).
          Hide
          stack added a comment -

          Patch applies to 0.19 and to TRUNK.

          Show
          stack added a comment - Patch applies to 0.19 and to TRUNK.
          Hide
          Doug Cutting added a comment -

          I'm still not convinced we should do more than replace the buf.reset() with buf = new ByteArrayOutputStream() and remove the initialization of buf altogether.

          Show
          Doug Cutting added a comment - I'm still not convinced we should do more than replace the buf.reset() with buf = new ByteArrayOutputStream() and remove the initialization of buf altogether.
          Hide
          stack added a comment -

          > I'm still not convinced we should do more than replace the buf.reset() with buf = new ByteArrayOutputStream() and remove the initialization of buf altogether.

          Above is predicated on our running a 'benchmark'. What would you suggest I run?

          Why create new objects when it can be avoided (when response < 10k)?

          Show
          stack added a comment - > I'm still not convinced we should do more than replace the buf.reset() with buf = new ByteArrayOutputStream() and remove the initialization of buf altogether. Above is predicated on our running a 'benchmark'. What would you suggest I run? Why create new objects when it can be avoided (when response < 10k)?
          Hide
          Raghu Angadi added a comment -

          yes, other buffering improvements are more work and better suited in a new jira. Btw, we don't need the check 'buf != initialBuffer'.

          Couple of advantages to current patch :

          1. we can clearly say this won't change performance for common case. Though I agree that creating new stream (i.e. two more allocations) each time won't be noticeable (except ay be more frequent partial G/C on benchmarks).
          2. Subclassing BAOS (or a new output stream) will be required for other buffering improvements any way.
          Show
          Raghu Angadi added a comment - yes, other buffering improvements are more work and better suited in a new jira. Btw, we don't need the check ' buf != initialBuffer '. Couple of advantages to current patch : we can clearly say this won't change performance for common case. Though I agree that creating new stream (i.e. two more allocations) each time won't be noticeable (except ay be more frequent partial G/C on benchmarks). Subclassing BAOS (or a new output stream) will be required for other buffering improvements any way.
          Hide
          stack added a comment -

          v3 removes the unnecessary equality test (Thanks Raghu)

          Show
          stack added a comment - v3 removes the unnecessary equality test (Thanks Raghu)
          Hide
          Raghu Angadi added a comment -

          Also, the new class needs to be static.

          Subjective : I don't think we should have to use jira numbers in comments. I think this is a specific enough problem that comment or javadoc could explain it in enough detail. Readers don't need to go to jira.

          Show
          Raghu Angadi added a comment - Also, the new class needs to be static. Subjective : I don't think we should have to use jira numbers in comments. I think this is a specific enough problem that comment or javadoc could explain it in enough detail. Readers don't need to go to jira.
          Hide
          stack added a comment -

          Made HBAOS static (Had to move it out of Server#Handler to do so). Removed pointer to this issue from comments (Can go to svn log if really needed).

          Show
          stack added a comment - Made HBAOS static (Had to move it out of Server#Handler to do so). Removed pointer to this issue from comments (Can go to svn log if really needed).
          Hide
          Raghu Angadi added a comment -

          +1 for the patch.

          Show
          Raghu Angadi added a comment - +1 for the patch.
          Hide
          Doug Cutting added a comment -

          > Why create new objects when it can be avoided (when response < 10k)?

          It's probably a premature optimization. It adds code. Less is more.

          Show
          Doug Cutting added a comment - > Why create new objects when it can be avoided (when response < 10k)? It's probably a premature optimization. It adds code. Less is more.
          Hide
          stack added a comment -

          > It's probably a premature optimization. It adds code. Less is more.

          Do the above add up to a -1 on v4 of the patch?

          Show
          stack added a comment - > It's probably a premature optimization. It adds code. Less is more. Do the above add up to a -1 on v4 of the patch?
          Hide
          Doug Cutting added a comment -

          > Do the above add up to a -1 on v4 of the patch?

          No, more like a +0. Without benchmarking its safest to not change things much. Do we have a good pure RPC benchmark?

          Show
          Doug Cutting added a comment - > Do the above add up to a -1 on v4 of the patch? No, more like a +0. Without benchmarking its safest to not change things much. Do we have a good pure RPC benchmark?
          Hide
          Edward J. Yoon added a comment -

          > stack - 09/Dec/08 12:32 PM
          > Patch applies to 0.19 and to TRUNK.

          I hope it'll be fixed at the 0.18.3 release.

          Show
          Edward J. Yoon added a comment - > stack - 09/Dec/08 12:32 PM > Patch applies to 0.19 and to TRUNK. I hope it'll be fixed at the 0.18.3 release.
          Hide
          Raghu Angadi added a comment -

          TestRpcCpu micro-benchmark attached to HADOOP-4797 can be used to asses the CPU penalty of this patch for larger buffers. May not matter much since we do expect extra copies with this patch for larger responses.

          Show
          Raghu Angadi added a comment - TestRpcCpu micro-benchmark attached to HADOOP-4797 can be used to asses the CPU penalty of this patch for larger buffers. May not matter much since we do expect extra copies with this patch for larger responses.
          Hide
          stack added a comment -

          Removed my patches. Write me offline if want to know why.

          Show
          stack added a comment - Removed my patches. Write me offline if want to know why.
          Hide
          Suresh Srinivas added a comment -

          This is duplicate of HADOOP-6460, which has been fixed. Marking this as resolved as well.

          Show
          Suresh Srinivas added a comment - This is duplicate of HADOOP-6460 , which has been fixed. Marking this as resolved as well.

            People

            • Assignee:
              Unassigned
              Reporter:
              stack
            • Votes:
              0 Vote for this issue
              Watchers:
              7 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development