Hey Steve,

My research project (Cake, published at SoCC '12) was trying to provide
SLAs for mixed workloads of latency-sensitive and throughput-bound
applications, e.g. HBase running alongside MR. This was challenging because
seeks are a real killer. Basically, we had to strongly limit MR I/O to keep
worst-case seek latency down, and did so by putting schedulers on the RPC
queues in HBase and HDFS to restrict queuing in the OS and disk where we
lacked preemption.

Regarding citations of note, most academics consider throughput-sharing to
be a solved problem. It's not dissimilar from normal time slicing, you try
to ensure fairness over some coarse timescale. I think cgroups [1] and
ioprio_set [2] essentially provide this.

Mixing throughput and latency though is difficult, and my conclusion is
that there isn't a really great solution for spinning disks besides
physical isolation. As we all know, you can get either IOPS or bandwidth,
but not both, and it's not a linear tradeoff between the two. If you're
interested in this though, I can dig up some related work from my Cake
paper.

However, since it seems that we're more concerned with throughput-bound
apps, we might be okay just using cgroups and ioprio_set to do
time-slicing. I actually hacked up some code a while ago which passed a
client-provided priority byte to the DN, which used it to set the I/O
priority of the handling DataXceiver accordingly. This isn't the most
outlandish idea, since we've put QoS fields in our RPC protocol for
instance; this would just be another byte. Short-circuit reads are outside
this paradigm, but then you can use cgroup controls instead.

My casual conversations with Googlers indicate that there isn't any special
Borg/Omega sauce either, just that they heavily prioritize DFS I/O over
non-DFS. Maybe that's another approach: if we can separate block management
in HDFS, MR tasks could just write their output to a raw HDFS block, thus
bringing a lot of I/O back into the fold of "datanode as I/O manager" for a
machine.

Overall, I strongly agree with you that it's important to first define what
our goals are regarding I/O QoS. The general case is a tarpit, so it'd be
good to carve off useful things that can be done now (like Lohit's
direction of per-stream/FS throughput throttling with trusted clients) and
then carefully grow the scope as we find more usecases we can confidently
solve.

Best,
Andrew

[1] cgroups blkio controller
https://www.kernel.org/doc/Documentation/cgroups/blkio-controller.txt
[2] ioprio_set http://man7.org/linux/man-pages/man2/ioprio_set.2.html


On Tue, Nov 12, 2013 at 1:38 AM, Steve Loughran <ste...@hortonworks.com>wrote:

> I've looked at it a bit within the context of YARN.
>
> YARN containers are where this would be ideal, as then you'd be able to
> request IO capacity as well as CPU and RAM. For that to work, the
> throttling would have to be outside the App, as you are trying to limit
> code whether or not it wants to be, and because you probably (*) want to
> give it more bandwidth if the system is otherwise idle. Self-throttling
> doesn't pick up spare IO
>
>
>    1. you can use cgroups in YARN to throttle local disk IO through the
>    file:// URLs or the java filesystem APIs -such as for MR temp data
>    2. you can't c-group throttle HDFS per YARN container, which would be
>    the ideal use case for it. The IO is taking place in the DN, and cgroups
>    only limits IO in the throttled process group.
>    3. implementing it in the DN would require a lot more complex code there
>    to prioritise work based on block ID (sole identifier that goes around
>    everywhere) or input source (local sockets for HBase IO vs TCP stack)
>    4. One you go to a heterogenous filesystem you need to think about IO
>    load per storage layer as well as/alongside per-volume
>    5. There's also generic RPC request throttle to prevent DoS against the
>    NN and other HDFS services. That would need to be server side, but once
>    implemented in the RPC code be universal.
>
> You also need to define what is the load you are trying to throttle, pure
> RPCs/second, read bandwidth, write bandwidth, seeks or IOPs. Once a file is
> lined up for sequential reading, you'd almost want it to stream through the
> next blocks until a high priority request came through, but operations like
> a seek which would involve a disk head movement backwards would be
> something to throttle (hence you need to be storage type aware as SSD seeks
> costs less). You also need to consider that although the cost of writes is
> high, it's usually being done with the goal of preserving data -and you
> don't want to impact durability.
>
> (*) probably, because that's one of the issues that causes debates in other
> datacentre platforms, such as Google Omega: do you want max cluster
> utilisation vs max determinism of workload.
>
> If someone were to do IOP throttling in the 3.x+ timeline,
>
>    1. It needs clear use cases, YARN containers being #1 for me
>    2. We'd have to look at all the research done on this in the past to see
>    what works, doesn't
>
> Andrew, what citations of relevance do you have?
>
> -steve
>
>
> On 12 November 2013 04:24, lohit <lohit.vijayar...@gmail.com> wrote:
>
> > 2013/11/11 Andrew Wang <andrew.w...@cloudera.com>
> >
> > > Hey Lohit,
> > >
> > > This is an interesting topic, and something I actually worked on in
> grad
> > > school before coming to Cloudera. It'd help if you could outline some
> of
> > > your usecases and how per-FileSystem throttling would help. For what I
> > was
> > > doing, it made more sense to throttle on the DN side since you have a
> > > better view over all the I/O happening on the system, and you have
> > > knowledge of different volumes so you can set limits per-disk. This
> still
> > > isn't 100% reliable though since normally a portion of each disk is
> used
> > > for MR scratch space, which the DN doesn't have control over. I tried
> > > playing with thread I/O priorities here, but didn't see much
> improvement.
> > > Maybe the newer cgroups stuff can help out.
> > >
> >
> > Thanks. Yes, we also thought about having something on DataNode. This
> would
> > also mean one could easily throttle client who access from outside the
> > cluster, for example distcp or hftp copies. Clients need not worry about
> > throttle configs and each cluster can control how much much throughput
> can
> > be achieved. We do want to have something like this.
> >
> > >
> > > I'm sure per-FileSystem throttling will have some benefits (and
> probably
> > be
> > > easier than some DN-side implementation) but again, it'd help to better
> > > understand the problem you are trying to solve.
> > >
> >
> > One idea was flexibility for client to override and have value they can
> > set. For on trusted cluster we could allow clients to go beyond default
> > value for some usecases. Alternatively we also thought about having
> default
> > value and max value where clients could change default, but not go beyond
> > default. Another problem with DN side config is having different values
> for
> > different clients and easily changing those for selective clients.
> >
> > As, Haosong also suggested we could wrap FSDataOutputStream/FSDataInput
> > stream with ThrottleInputStream. But we might have to be careful of any
> > code which uses FileSystem APIs and accidentally throttling itself. (like
> > reducer copy,  distributed cache and such...)
> >
> >
> >
> > > Best,
> > > Andrew
> > >
> > >
> > > On Mon, Nov 11, 2013 at 6:16 PM, Haosong Huang <haosd...@gmail.com>
> > wrote:
> > >
> > > > Hi, lohit. There is a Class named
> > > > ThrottledInputStream<
> > > >
> > >
> >
> http://svn.apache.org/repos/asf/hadoop/common/trunk/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/ThrottledInputStream.java
> > > > >
> > > >  in hadoop-distcp, you could check it out and find more details.
> > > >
> > > > In addition to this, I am working on this and try to achieve
> resources
> > > > control(include CPU, Network, Disk IO) in JVM. But my implementation
> is
> > > > depends on cgroup, which only could run in Linux. I would push my
> > > > library(java-cgroup) to github in the next several months. If you are
> > > > interested at it, give my any advices and help me improve it please.
> > :-)
> > > >
> > > >
> > > > On Tue, Nov 12, 2013 at 3:47 AM, lohit <lohit.vijayar...@gmail.com>
> > > wrote:
> > > >
> > > > > Hi Adam,
> > > > >
> > > > > Thanks for the reply. The changes I was referring was in
> > > FileSystem.java
> > > > > layer which should not affect HDFS Replication/NameNode operations.
> > > > > To give better idea this would affect clients something like this
> > > > >
> > > > > Configuration conf = new Configuration();
> > > > > conf.setInt("read.bandwitdh.mbpersec", 20); // 20MB/s
> > > > > FileSystem fs = FileSystem.get(conf);
> > > > >
> > > > > FSDataInputStream fis = fs.open("/path/to/file.xt");
> > > > > fis.read(); // <-- This would be max of 20MB/s
> > > > >
> > > > >
> > > > >
> > > > >
> > > > > 2013/11/11 Adam Muise <amu...@hortonworks.com>
> > > > >
> > > > > > See https://issues.apache.org/jira/browse/HDFS-3475
> > > > > >
> > > > > > Please note that this has met with many unexpected impacts on
> > > workload.
> > > > > Be
> > > > > > careful and be mindful of your Datanode memory and network
> > capacity.
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > On Mon, Nov 11, 2013 at 1:59 PM, lohit <
> lohit.vijayar...@gmail.com
> > >
> > > > > wrote:
> > > > > >
> > > > > > > Hello Devs,
> > > > > > >
> > > > > > > Wanted to reach out and see if anyone has thought about ability
> > to
> > > > > > throttle
> > > > > > > data transfer within HDFS. One option we have been thinking is
> to
> > > > > > throttle
> > > > > > > on a per FileSystem basis, similar to Statistics in FileSystem.
> > > This
> > > > > > would
> > > > > > > mean anyone with handle to HDFS/Hftp will be throttled globally
> > > > within
> > > > > > JVM.
> > > > > > > Right value to come up for this would be based on type of
> > hardware
> > > we
> > > > > use
> > > > > > > and how many tasks/clients we allow.
> > > > > > >
> > > > > > > On the other hand doing something like this at FileSystem layer
> > > would
> > > > > > mean
> > > > > > > many other tasks such as Job jar copy, DistributedCache copy
> and
> > > any
> > > > > > hidden
> > > > > > > data movement would also be throttled. We wanted to know if
> > anyone
> > > > has
> > > > > > had
> > > > > > > such requirement on their clusters in the past and what was the
> > > > > thinking
> > > > > > > around it. Appreciate your inputs/comments
> > > > > > >
> > > > > > > --
> > > > > > > Have a Nice Day!
> > > > > > > Lohit
> > > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > --
> > > > > >    * Adam Muise *       Solutions Engineer
> > > > > > ------------------------------
> > > > > >
> > > > > >     Phone:        416-417-4037
> > > > > >   Email:      amu...@hortonworks.com
> > > > > >   Website:   http://www.hortonworks.com/
> > > > > >
> > > > > >       * Follow Us: *
> > > > > > <
> > > > > >
> > > > >
> > > >
> > >
> >
> http://facebook.com/hortonworks/?utm_source=WiseStamp&utm_medium=email&utm_term=&utm_content=&utm_campaign=signature
> > > > > > >
> > > > > > <
> > > > > >
> > > > >
> > > >
> > >
> >
> http://twitter.com/hortonworks?utm_source=WiseStamp&utm_medium=email&utm_term=&utm_content=&utm_campaign=signature
> > > > > > >
> > > > > > <
> > > > > >
> > > > >
> > > >
> > >
> >
> http://www.linkedin.com/company/hortonworks?utm_source=WiseStamp&utm_medium=email&utm_term=&utm_content=&utm_campaign=signature
> > > > > > >
> > > > > >
> > > > > >  [image: photo]
> > > > > >
> > > > > >   Latest From Our Blog:  How to use R and other non-Java
> languages
> > in
> > > > > > MapReduce and Hive
> > > > > > <
> > > > > >
> > > > >
> > > >
> > >
> >
> http://hortonworks.com/blog/using-r-and-other-non-java-languages-in-mapreduce-and-hive/?utm_source=WiseStamp&utm_medium=email&utm_term=&utm_content=&utm_campaign=signature
> > > > > > >
> > > > > >
> > > > > > --
> > > > > > CONFIDENTIALITY NOTICE
> > > > > > NOTICE: This message is intended for the use of the individual or
> > > > entity
> > > > > to
> > > > > > which it is addressed and may contain information that is
> > > confidential,
> > > > > > privileged and exempt from disclosure under applicable law. If
> the
> > > > reader
> > > > > > of this message is not the intended recipient, you are hereby
> > > notified
> > > > > that
> > > > > > any printing, copying, dissemination, distribution, disclosure or
> > > > > > forwarding of this communication is strictly prohibited. If you
> > have
> > > > > > received this communication in error, please contact the sender
> > > > > immediately
> > > > > > and delete it from your system. Thank You.
> > > > > >
> > > > >
> > > > >
> > > > >
> > > > > --
> > > > > Have a Nice Day!
> > > > > Lohit
> > > > >
> > > >
> > > >
> > > >
> > > > --
> > > > Best Regards,
> > > > Haosdent Huang
> > > >
> > >
> >
> >
> >
> > --
> > Have a Nice Day!
> > Lohit
> >
>
> --
> CONFIDENTIALITY NOTICE
> NOTICE: This message is intended for the use of the individual or entity to
> which it is addressed and may contain information that is confidential,
> privileged and exempt from disclosure under applicable law. If the reader
> of this message is not the intended recipient, you are hereby notified that
> any printing, copying, dissemination, distribution, disclosure or
> forwarding of this communication is strictly prohibited. If you have
> received this communication in error, please contact the sender immediately
> and delete it from your system. Thank You.
>

Reply via email to