[jira] [Created] (HDFS-5503) Datanode#checkDiskError also should check for ClosedChannelException

2013-11-12 Thread Vinay (JIRA)
Vinay created HDFS-5503:
---

 Summary: Datanode#checkDiskError also should check for 
ClosedChannelException
 Key: HDFS-5503
 URL: https://issues.apache.org/jira/browse/HDFS-5503
 Project: Hadoop HDFS
  Issue Type: Bug
Reporter: Vinay
Assignee: Vinay


out file
==
{noformat}
Exception in thread "PacketResponder: 
BP-52063768-x-1383447451718:blk_1073755206_1099511661730, 
type=LAST_IN_PIPELINE, downstreams=0:[]" java.lang.NullPointerException
at 
org.apache.hadoop.hdfs.server.datanode.DataNode.checkDiskError(DataNode.java:1363)
at 
org.apache.hadoop.hdfs.server.datanode.BlockReceiver$PacketResponder.run(BlockReceiver.java:1233)
at java.lang.Thread.run(Thread.java:662){noformat}

log file
===
{noformat}2013-11-08 21:23:36,622 WARN 
org.apache.hadoop.hdfs.server.datanode.DataNode: IOException in 
BlockReceiver.run():
java.nio.channels.ClosedChannelException
at 
sun.nio.ch.SocketChannelImpl.ensureWriteOpen(SocketChannelImpl.java:133)
at sun.nio.ch.SocketChannelImpl.write(SocketChannelImpl.java:324)
at 
org.apache.hadoop.net.SocketOutputStream$Writer.performIO(SocketOutputStream.java:63)
at 
org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:142)
at 
org.apache.hadoop.net.SocketOutputStream.write(SocketOutputStream.java:159)
at 
org.apache.hadoop.net.SocketOutputStream.write(SocketOutputStream.java:117)
at 
java.io.BufferedOutputStream.flushBuffer(BufferedOutputStream.java:65)
at java.io.BufferedOutputStream.flush(BufferedOutputStream.java:123)
at java.io.DataOutputStream.flush(DataOutputStream.java:106)
at 
org.apache.hadoop.hdfs.server.datanode.BlockReceiver$PacketResponder.run(BlockReceiver.java:1212)
at java.lang.Thread.run(Thread.java:662)
2013-11-08 21:23:36,622 WARN org.apache.hadoop.hdfs.server.datanode.DataNode: 
checkDiskError: exception:
java.nio.channels.ClosedChannelException
at 
sun.nio.ch.SocketChannelImpl.ensureWriteOpen(SocketChannelImpl.java:133)
at sun.nio.ch.SocketChannelImpl.write(SocketChannelImpl.java:324)
at 
org.apache.hadoop.net.SocketOutputStream$Writer.performIO(SocketOutputStream.java:63)
at 
org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:142)
at 
org.apache.hadoop.net.SocketOutputStream.write(SocketOutputStream.java:159)
at 
org.apache.hadoop.net.SocketOutputStream.write(SocketOutputStream.java:117)
at 
java.io.BufferedOutputStream.flushBuffer(BufferedOutputStream.java:65)
at java.io.BufferedOutputStream.flush(BufferedOutputStream.java:123)
at java.io.DataOutputStream.flush(DataOutputStream.java:106)
at 
org.apache.hadoop.hdfs.server.datanode.BlockReceiver$PacketResponder.run(BlockReceiver.java:1212)
at java.lang.Thread.run(Thread.java:662){noformat}



--
This message was sent by Atlassian JIRA
(v6.1#6144)


Re: HDFS read/write data throttling

2013-11-12 Thread Steve Loughran
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  wrote:

> 2013/11/11 Andrew Wang 
>
> > 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 
> 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 

[jira] [Created] (HDFS-5504) In HA mode, OP_DELETE_SNAPSHOT is not decrementing the safemode threshold, leads to NN safemode.

2013-11-12 Thread Vinay (JIRA)
Vinay created HDFS-5504:
---

 Summary: In HA mode, OP_DELETE_SNAPSHOT is not decrementing the 
safemode threshold, leads to NN safemode.
 Key: HDFS-5504
 URL: https://issues.apache.org/jira/browse/HDFS-5504
 Project: Hadoop HDFS
  Issue Type: Bug
  Components: snapshots
Reporter: Vinay
Assignee: Vinay


1. HA installation, standby NN is down.
2. delete snapshot is called and it has deleted the blocks from blocksmap and 
all datanodes. log sync also happened.
3. before next log roll NN crashed
4. When the namenode restartes then it will fsimage and finalized edits from 
shared storage and set the safemode threshold. which includes blocks from 
deleted snapshot also. (because this edits is not yet read as namenode is 
restarted before the last edits segment is not finalized)
5. When it becomes active, it finalizes the edits and read the delete snapshot 
edits_op. but at this time, it was not reducing the safemode count. and it will 
continuing in safemode.
6. On next restart, as the edits is already finalized, on startup only it will 
read and set the safemode threshold correctly.

But one more restart will bring NN out of safemode.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Created] (HDFS-5505) DataStorage.format() should ignore NamespaceID

2013-11-12 Thread Eric Sirianni (JIRA)
Eric Sirianni created HDFS-5505:
---

 Summary: DataStorage.format() should ignore NamespaceID
 Key: HDFS-5505
 URL: https://issues.apache.org/jira/browse/HDFS-5505
 Project: Hadoop HDFS
  Issue Type: Bug
  Components: datanode, federation
Affects Versions: 2.2.0
Reporter: Eric Sirianni
Priority: Minor


I'm observing inconsistencies in the {{DataStorage.namespaceID}} value passed 
in to my {{FsDatasetSpi}} plugin:
* Upon _initial_ startup (when the DataNode is formatted), the storage 
{{DataStorage.namespaceID}} value is populated with the namespace ID from the 
NameNode (or an arbitrary namenode in a federated setup).
* Upon subsequent startups the {{DataStorage.namespaceID}} value is {{NULL}}.

The issue appears to be different handling of the {{DataStorage.namespaceID}} 
value in {{DataStorage.format()}} vs. {{DataStorage.setFieldsFromProperties()}}:

{code:title=DataStorage.java}
  void format(StorageDirectory sd, NamespaceInfo nsInfo, String datanodeUuid) 
throws IOException {
sd.clearDirectory(); // create directory
this.layoutVersion = HdfsConstants.LAYOUT_VERSION;
this.clusterID = nsInfo.getClusterID();
this.namespaceID = nsInfo.getNamespaceID();
  }


  protected void setFieldsFromProperties(Properties props, StorageDirectory sd)
// Read NamespaceID in version before federation
if (!LayoutVersion.supports(Feature.FEDERATION, layoutVersion)) {
  setNamespaceID(props, sd);
}
{code}

If I understand correctly, the proper fix is to leave namespaceID *unset* in 
{{DataStorage.format()}}.  The namespaceID here is invalid anyway since it is 
susceptible to race conditions in federated NameNode setups -- it will happen 
to match the namespaceID of the first {{BPOfferService}} thread that 
successfully handshakes.




--
This message was sent by Atlassian JIRA
(v6.1#6144)


Re: Next releases

2013-11-12 Thread Todd Lipcon
On Mon, Nov 11, 2013 at 2:57 PM, Colin McCabe wrote:

> To be honest, I'm not aware of anything in 2.2.1 that shouldn't be
> there.  However, I have only been following the HDFS and common side
> of things so I may not have the full picture.  Arun, can you give a
> specific example of something you'd like to "blow away"?
>

I agree with Colin. If we've been backporting things into a patch release
(third version component) which don't belong, we should explicitly call out
those patches, so we can learn from our mistakes and have a discussion
about what belongs. Otherwise we'll just end up doing it again. Saying
"there were a few mistakes, so let's reset back a bunch of backport work"
seems like a baby-with-the-bathwater situation.

Todd


[jira] [Created] (HDFS-5506) Use URLConnectionFactory in DelegationTokenFetcher

2013-11-12 Thread Haohui Mai (JIRA)
Haohui Mai created HDFS-5506:


 Summary: Use URLConnectionFactory in DelegationTokenFetcher
 Key: HDFS-5506
 URL: https://issues.apache.org/jira/browse/HDFS-5506
 Project: Hadoop HDFS
  Issue Type: Sub-task
Reporter: Haohui Mai
Assignee: Haohui Mai
 Attachments: HDFS-5506.000.patch

HftpFileSystem uses DelegationTokenFetcher to get delegation token from the 
server. DelegationTokenFetcher should use the same URLConnectionFactory to open 
all HTTP / HTTPS connections so that things like SSL certificates, timeouts are 
respected.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Created] (HDFS-5507) Consider maximum DN memory, stale status when scheduling recaching

2013-11-12 Thread Colin Patrick McCabe (JIRA)
Colin Patrick McCabe created HDFS-5507:
--

 Summary: Consider maximum DN memory, stale status when scheduling 
recaching
 Key: HDFS-5507
 URL: https://issues.apache.org/jira/browse/HDFS-5507
 Project: Hadoop HDFS
  Issue Type: Sub-task
  Components: namenode
Affects Versions: 3.0.0
Reporter: Colin Patrick McCabe
Assignee: Colin Patrick McCabe


We should consider the maximum available DN memory and the stale status of the 
DN when scheduling recaching work.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Created] (HDFS-5508) Fix compilation error after merge

2013-11-12 Thread Tsz Wo (Nicholas), SZE (JIRA)
Tsz Wo (Nicholas), SZE created HDFS-5508:


 Summary: Fix compilation error after merge
 Key: HDFS-5508
 URL: https://issues.apache.org/jira/browse/HDFS-5508
 Project: Hadoop HDFS
  Issue Type: Sub-task
Reporter: Tsz Wo (Nicholas), SZE
Assignee: Tsz Wo (Nicholas), SZE
Priority: Minor


[ERROR] COMPILATION ERROR : 
[INFO] -
[ERROR] 
/Users/szetszwo/hadoop/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java:[416,38]
 cannot find symbol
symbol  : constructor 
LocatedBlock(org.apache.hadoop.hdfs.protocol.ExtendedBlock,org.apache.hadoop.hdfs.protocol.DatanodeInfo[],long,boolean,org.apache.hadoop.hdfs.protocol.DatanodeInfo[])
location: class org.apache.hadoop.hdfs.protocol.LocatedBlock
[INFO] 1 error




--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Created] (HDFS-5509) TestPathBasedCacheRequests#testReplicationFactor is flaky

2013-11-12 Thread Andrew Wang (JIRA)
Andrew Wang created HDFS-5509:
-

 Summary: TestPathBasedCacheRequests#testReplicationFactor is flaky
 Key: HDFS-5509
 URL: https://issues.apache.org/jira/browse/HDFS-5509
 Project: Hadoop HDFS
  Issue Type: Sub-task
Affects Versions: 3.0.0
Reporter: Andrew Wang
Assignee: Andrew Wang


Test flakes out sometimes with the following:

{code}
Running org.apache.hadoop.hdfs.server.namenode.TestPathBasedCacheRequests
Tests run: 9, Failures: 1, Errors: 0, Skipped: 0, Time elapsed: 72.191 sec <<< 
FAILURE! - in org.apache.hadoop.hdfs.server.namenode.TestPathBasedCacheRequests
testReplicationFactor(org.apache.hadoop.hdfs.server.namenode.TestPathBasedCacheRequests)
  Time elapsed: 7.449 sec  <<< FAILURE!
java.lang.AssertionError: Unexpected number of cached blocks expected:<4> but 
was:<3>
at org.junit.Assert.fail(Assert.java:93)
at org.junit.Assert.failNotEquals(Assert.java:647)
at org.junit.Assert.assertEquals(Assert.java:128)
at org.junit.Assert.assertEquals(Assert.java:472)
at 
org.apache.hadoop.hdfs.server.namenode.TestPathBasedCacheRequests.checkNumCachedReplicas(TestPathBasedCacheRequests.java:619)
at 
org.apache.hadoop.hdfs.server.namenode.TestPathBasedCacheRequests.testReplicationFactor(TestPathBasedCacheRequests.java:843)
{code}



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Resolved] (HDFS-5508) Fix compilation error after merge

2013-11-12 Thread Arpit Agarwal (JIRA)

 [ 
https://issues.apache.org/jira/browse/HDFS-5508?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Arpit Agarwal resolved HDFS-5508.
-

   Resolution: Fixed
Fix Version/s: Heterogeneous Storage (HDFS-2832)
 Hadoop Flags: Reviewed

Thanks Nicholas! I committed it to branch HDFS-2832.

> Fix compilation error after merge
> -
>
> Key: HDFS-5508
> URL: https://issues.apache.org/jira/browse/HDFS-5508
> Project: Hadoop HDFS
>  Issue Type: Sub-task
>Reporter: Tsz Wo (Nicholas), SZE
>Assignee: Tsz Wo (Nicholas), SZE
>Priority: Minor
> Fix For: Heterogeneous Storage (HDFS-2832)
>
> Attachments: h5508_20131112.patch
>
>
> [ERROR] COMPILATION ERROR : 
> [INFO] -
> [ERROR] 
> /Users/szetszwo/hadoop/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java:[416,38]
>  cannot find symbol
> symbol  : constructor 
> LocatedBlock(org.apache.hadoop.hdfs.protocol.ExtendedBlock,org.apache.hadoop.hdfs.protocol.DatanodeInfo[],long,boolean,org.apache.hadoop.hdfs.protocol.DatanodeInfo[])
> location: class org.apache.hadoop.hdfs.protocol.LocatedBlock
> [INFO] 1 error



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Resolved] (HDFS-5501) Fix pendingReceivedRequests tracking in BPServiceActor

2013-11-12 Thread Arpit Agarwal (JIRA)

 [ 
https://issues.apache.org/jira/browse/HDFS-5501?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Arpit Agarwal resolved HDFS-5501.
-

   Resolution: Fixed
Fix Version/s: Heterogeneous Storage (HDFS-2832)

Thanks Nicholas, I committed this to branch HDFS-2832.

> Fix pendingReceivedRequests tracking in BPServiceActor
> --
>
> Key: HDFS-5501
> URL: https://issues.apache.org/jira/browse/HDFS-5501
> Project: Hadoop HDFS
>  Issue Type: Sub-task
>  Components: datanode
>Affects Versions: Heterogeneous Storage (HDFS-2832)
>Reporter: Arpit Agarwal
>Assignee: Arpit Agarwal
> Fix For: Heterogeneous Storage (HDFS-2832)
>
> Attachments: h5501.01.patch, h5501.02.patch, h5501.03.patch
>
>
> {{BPServiceActor#pendingReceivedRequests}} is not tracked correctly for 
> multiple storages. Previously this counter was treated more like a flag, i.e. 
> we only cared whether it was zero or not. With multiple storages we need to 
> be more precise in tracking this value to correctly determine when to 
> generate incremental block reports.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


block access token

2013-11-12 Thread lei liu
When client read block from DataNode, the block access token is used for
authorization on DataNode. But if the block access token is stolen by
impostor, the  impostor can read the block,
I think this is one security hole.

I think we can use the replay cache mechanism in Kerberos to resolve the
question, example below explaining:

The possibility exists for an impostor to simultaneously steal both the
ticket and the authenticator and use them during the 2 minutes the
authenticator is valid. This is very difficult but not impossible. To solve
this problem with Kerberos 5, Replay Cache has been introduced. In
application servers (but also in TGS), there exists the capacity to
remember authenticators which have arrived within the last 2 minutes, and
to reject them if they are replicas. With this the problem is resolved as
long as the impostor is not smart enough to copy the ticket and
authenticator and make them arrive at the application server before the
legitimate request arrives. This really would be a hoax, since the
authentic user would be rejected while the impostor would have access to
the service.


Thanks,

LiuLei


Re: HDFS read/write data throttling

2013-11-12 Thread Andrew Wang
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 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