Details

    • Type: New Feature
    • Status: Open
    • Priority: Major
    • Resolution: Unresolved
    • Affects Version/s: None
    • Fix Version/s: None
    • Component/s: None
    • Labels:
      None

      Description

      When an applications upload/download files from/to HDFS clusters, it would be nice if the IO could be throttled so that they won't go beyond the specified maximum bandwidth.

      Two options to implement this IO throttling:

      #1. IO Throttling happens at the FSDataInputStream and FSDataOutputStream level.

      Add an IO Throttler to FSDataInputStream/FSDataOutputStram, and whenever an read/write happens, throttle it first(if throttler is set), then do the actual read/write.

      We may need to add new FileSystem apis to take an IO throttler as input parameter.

      #2. IO Throttling happens at the application level.

      Instead of changing the FSDataInputStream/FSDataOutputStream, all IO throttling is done at the application level.

      In this approach, FileSystem api remains unchanged.

      Either case, an IO throttler interface is needed, which has a:
      public void throttle(long numOfBytes);

      The current DataTransferThrottler could be an implementation of this IO throttler interface.

        Issue Links

          Activity

          Hide
          zhenxiao Zhenxiao Luo added a comment -

          Any comments are welcome. Which approach is better?

          Show
          zhenxiao Zhenxiao Luo added a comment - Any comments are welcome. Which approach is better?
          Hide
          tucu00 Alejandro Abdelnur added a comment -

          Is the objective to be able to enforce IO throttling for the cluster or for certain applications to be 'nice'? If the former this must be enforced on the cluster side, not on the client side. If the later apps wanting to be 'nice' could wrap the IO streams with throttling aware ones.

          Show
          tucu00 Alejandro Abdelnur added a comment - Is the objective to be able to enforce IO throttling for the cluster or for certain applications to be 'nice'? If the former this must be enforced on the cluster side, not on the client side. If the later apps wanting to be 'nice' could wrap the IO streams with throttling aware ones.
          Hide
          daryn Daryn Sharp added a comment -

          A simple app side change might be to extend IOUtils.copyBytes.

          Show
          daryn Daryn Sharp added a comment - A simple app side change might be to extend IOUtils.copyBytes.
          Hide
          zhenxiao Zhenxiao Luo added a comment -

          Thanks Alejandro and Daryn.

          I am thinking of providing an IOThrottler interface, which DataTransferThrottler implements.

          An app side change is extending IOUtils.copyBytes with an additional IOThrottler parameter, which does the throttling when users doing DFSShell -put or -get.

          Cluster side throttling might need to add additional api in FileSystem open() and create(), also pass an additional IOThrottler parameter, and put IOThrottler in FSDataInputStream/FSDataOutputStream.

          As Alejandro said, if enforce throttling for cluster, we will go to Cluster Side, and if only enforce application throttling, we could go app side. Or, maybe in general, we could support both?

          Comments and suggestions are welcome.

          Show
          zhenxiao Zhenxiao Luo added a comment - Thanks Alejandro and Daryn. I am thinking of providing an IOThrottler interface, which DataTransferThrottler implements. An app side change is extending IOUtils.copyBytes with an additional IOThrottler parameter, which does the throttling when users doing DFSShell -put or -get. Cluster side throttling might need to add additional api in FileSystem open() and create(), also pass an additional IOThrottler parameter, and put IOThrottler in FSDataInputStream/FSDataOutputStream. As Alejandro said, if enforce throttling for cluster, we will go to Cluster Side, and if only enforce application throttling, we could go app side. Or, maybe in general, we could support both? Comments and suggestions are welcome.
          Hide
          cmccabe Colin P. McCabe added a comment -

          I think it makes sense to create some kind of throttler class that wraps a generic OutputStream. That doesn't need to be in HDFS-- in fact, that code should probably go in common.

          Show
          cmccabe Colin P. McCabe added a comment - I think it makes sense to create some kind of throttler class that wraps a generic OutputStream . That doesn't need to be in HDFS-- in fact, that code should probably go in common.
          Hide
          denistmp Denis Petrov added a comment -

          It would be nice to throttle on per-datanode basis, liming not the bandwidth of the current stream, but the IO bandwidth on the bottleneck datanode.

          If few throttled writes go to the same datanode, the throttling threshold should be adjusted.

          Show
          denistmp Denis Petrov added a comment - It would be nice to throttle on per-datanode basis, liming not the bandwidth of the current stream, but the IO bandwidth on the bottleneck datanode. If few throttled writes go to the same datanode, the throttling threshold should be adjusted.
          Hide
          cmccabe Colin P. McCabe added a comment -

          I think you need to decide what problem you're trying to solve. Are you trying to implement cluster-wide QoS (quality of service)? Are you trying to avoid a problem with "hot spots" in the network? (And if so, have you quantified how big a problem that really is?)

          Show
          cmccabe Colin P. McCabe added a comment - I think you need to decide what problem you're trying to solve. Are you trying to implement cluster-wide QoS (quality of service)? Are you trying to avoid a problem with "hot spots" in the network? (And if so, have you quantified how big a problem that really is?)
          Hide
          denistmp Denis Petrov added a comment -

          I am trying to avoid a problem with "hot spots" in the disk IO.
          In particular, during major compaction of Accumulo tablets (https://issues.apache.org/jira/browse/ACCUMULO-1128).

          This problem is difficult to solve at the application level, because actual disk IO can be (and often is) performed on another server, not on the server which runs the Accumulo tablet server doing the compaction.
          Two of three compactions running on different servers can result in heavy disk IO on the same HDFS datanode resulting in degradation of query performance and latency.

          Show
          denistmp Denis Petrov added a comment - I am trying to avoid a problem with "hot spots" in the disk IO. In particular, during major compaction of Accumulo tablets ( https://issues.apache.org/jira/browse/ACCUMULO-1128 ). This problem is difficult to solve at the application level, because actual disk IO can be (and often is) performed on another server, not on the server which runs the Accumulo tablet server doing the compaction. Two of three compactions running on different servers can result in heavy disk IO on the same HDFS datanode resulting in degradation of query performance and latency.
          Hide
          vicaya Luke Lu added a comment -

          #1 probably won't work very well as requests (especially writes) will be queued up at RPC layer due throttling at lower layer (FS*Stream).

          #2 would work (a la distcp and balancer) but under utilize cluster resource and tedious to use.

          The long term solution for such issue is QoS support (starting from HADOOP-9194).

          Show
          vicaya Luke Lu added a comment - #1 probably won't work very well as requests (especially writes) will be queued up at RPC layer due throttling at lower layer (FS*Stream). #2 would work (a la distcp and balancer) but under utilize cluster resource and tedious to use. The long term solution for such issue is QoS support (starting from HADOOP-9194 ).
          Hide
          zhangyongxyz Yong Zhang added a comment -

          Why not try IO throttling in a fire mode, like HADOOP-9640?

          Show
          zhangyongxyz Yong Zhang added a comment - Why not try IO throttling in a fire mode, like HADOOP-9640 ?
          Hide
          zhangyongxyz Yong Zhang added a comment -

          sorry, fair mode

          Show
          zhangyongxyz Yong Zhang added a comment - sorry, fair mode
          Hide
          zhangyongxyz Yong Zhang added a comment -

          sorry, fair mode

          Show
          zhangyongxyz Yong Zhang added a comment - sorry, fair mode
          Hide
          zhangyongxyz Yong Zhang added a comment -

          sorry, fair mode

          Show
          zhangyongxyz Yong Zhang added a comment - sorry, fair mode

            People

            • Assignee:
              Unassigned
              Reporter:
              zhenxiao Zhenxiao Luo
            • Votes:
              2 Vote for this issue
              Watchers:
              29 Start watching this issue

              Dates

              • Created:
                Updated:

                Development