[jira] [Commented] (HDFS-10467) Router-based HDFS federation

2017-09-03 Thread He Tianyi (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-10467?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16152037#comment-16152037
 ] 

He Tianyi commented on HDFS-10467:
--

[~elgoiri], thanks for asking.
our federated cluster has grown to 7000+ nodes this year. I can share some 
lessons learned in production with nnproxy:
* can speed up data rebalance between subclusters with 'fastcopy', or similar 
method, which effectively reduces resource consumption when there is intensive 
rebalance work
* perhaps isolation between subclusters for request forwarding on single router 
is required, otherwise outage of any subcluster could also affect others (from 
client's point of view) due to shared resource, i.e. thread pool (ipc 
handlers), client connection pool (ipc client). we done this by implementing a 
fully nonblocking version of proxy, also use multiple client connections to 
forward requests (as HADOOP-13144 suggests)
* global quota: we've disabled quota on each NameNode. quota is computed by a 
separated service which reads/tails fsimage and editlog from all subclusters, 
while nnproxy plays the part of enforcing quota (rejecting to create file when 
usage exceeds limitation, for example).

> Router-based HDFS federation
> 
>
> Key: HDFS-10467
> URL: https://issues.apache.org/jira/browse/HDFS-10467
> Project: Hadoop HDFS
>  Issue Type: New Feature
>  Components: fs
>Affects Versions: 2.8.1
>Reporter: Íñigo Goiri
>Assignee: Íñigo Goiri
> Fix For: HDFS-10467
>
> Attachments: HDFS-10467.002.patch, HDFS-10467.PoC.001.patch, 
> HDFS-10467.PoC.patch, HDFS Router Federation.pdf, 
> HDFS-Router-Federation-Prototype.patch
>
>
> Add a Router to provide a federated view of multiple HDFS clusters.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Commented] (HDFS-10490) Client may never recovery replica after a timeout during sending packet

2016-06-12 Thread He Tianyi (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-10490?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15326364#comment-15326364
 ] 

He Tianyi commented on HDFS-10490:
--

I believe the whitespace and checkstyle issue is not related to this patch.

> Client may never recovery replica after a timeout during sending packet
> ---
>
> Key: HDFS-10490
> URL: https://issues.apache.org/jira/browse/HDFS-10490
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode
>Affects Versions: 2.6.0
>Reporter: He Tianyi
> Attachments: HDFS-10490.0001.patch, HDFS-10490.patch
>
>
> For newly created replica, a meta file is created in constructor of 
> {{BlockReceiver}} (for {{WRITE_BLOCK}} op). Its header will be written lazily 
> (buffered in memory first by {{BufferedOutputStream}}). 
> If following packets fail to deliver (e.g. in  extreme network condition), 
> the header may never get flush until closed. 
> However, {{BlockReceiver}} will not call close until block receiving is 
> finished or exception(s) encountered. Also in extreme network condition, both 
> RST & FIN may not deliver in time. 
> In this case, if client tries to initiates a {{transferBlock}} to a new 
> datanode (in {{addDatanode2ExistingPipeline}}), existing datanode will see an 
> empty meta if its {{BlockReceiver}} did not close in time. 
> Then, after HDFS-3429, a default {{DataChecksum}} (NULL, 512) will be used 
> during transfer. So when client then tries to recover pipeline after 
> completely transferred, it may encounter the following exception:
> {noformat}
> java.io.IOException: Client requested checksum DataChecksum(type=CRC32C, 
> chunkSize=4096) when appending to an existing block with different chunk 
> size: DataChecksum(type=NULL, chunkSize=512)
> at 
> org.apache.hadoop.hdfs.server.datanode.ReplicaInPipeline.createStreams(ReplicaInPipeline.java:230)
> at 
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.(BlockReceiver.java:226)
> at 
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:798)
> at 
> org.apache.hadoop.hdfs.protocol.datatransfer.Receiver.opWriteBlock(Receiver.java:166)
> at 
> org.apache.hadoop.hdfs.protocol.datatransfer.Receiver.processOp(Receiver.java:76)
> at 
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:243)
> at java.lang.Thread.run(Thread.java:745)
> {noformat}
> This will repeat, until exhausted by datanode replacement policy.
> Also to note that, with bad luck (like I), 20k clients are all doing this. 
> It's to some extend a DDoS attack to NameNode (because of 
> getAdditionalDataNode calls).
> I suggest we flush immediately after header is written, preventing anybody 
> from seeing empty meta file for avoiding the issue.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Updated] (HDFS-10490) Client may never recovery replica after a timeout during sending packet

2016-06-12 Thread He Tianyi (JIRA)

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

He Tianyi updated HDFS-10490:
-
Attachment: HDFS-10490.0001.patch

Added test.

> Client may never recovery replica after a timeout during sending packet
> ---
>
> Key: HDFS-10490
> URL: https://issues.apache.org/jira/browse/HDFS-10490
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode
>Affects Versions: 2.6.0
>Reporter: He Tianyi
> Attachments: HDFS-10490.0001.patch, HDFS-10490.patch
>
>
> For newly created replica, a meta file is created in constructor of 
> {{BlockReceiver}} (for {{WRITE_BLOCK}} op). Its header will be written lazily 
> (buffered in memory first by {{BufferedOutputStream}}). 
> If following packets fail to deliver (e.g. in  extreme network condition), 
> the header may never get flush until closed. 
> However, {{BlockReceiver}} will not call close until block receiving is 
> finished or exception(s) encountered. Also in extreme network condition, both 
> RST & FIN may not deliver in time. 
> In this case, if client tries to initiates a {{transferBlock}} to a new 
> datanode (in {{addDatanode2ExistingPipeline}}), existing datanode will see an 
> empty meta if its {{BlockReceiver}} did not close in time. 
> Then, after HDFS-3429, a default {{DataChecksum}} (NULL, 512) will be used 
> during transfer. So when client then tries to recover pipeline after 
> completely transferred, it may encounter the following exception:
> {noformat}
> java.io.IOException: Client requested checksum DataChecksum(type=CRC32C, 
> chunkSize=4096) when appending to an existing block with different chunk 
> size: DataChecksum(type=NULL, chunkSize=512)
> at 
> org.apache.hadoop.hdfs.server.datanode.ReplicaInPipeline.createStreams(ReplicaInPipeline.java:230)
> at 
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.(BlockReceiver.java:226)
> at 
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:798)
> at 
> org.apache.hadoop.hdfs.protocol.datatransfer.Receiver.opWriteBlock(Receiver.java:166)
> at 
> org.apache.hadoop.hdfs.protocol.datatransfer.Receiver.processOp(Receiver.java:76)
> at 
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:243)
> at java.lang.Thread.run(Thread.java:745)
> {noformat}
> This will repeat, until exhausted by datanode replacement policy.
> Also to note that, with bad luck (like I), 20k clients are all doing this. 
> It's to some extend a DDoS attack to NameNode (because of 
> getAdditionalDataNode calls).
> I suggest we flush immediately after header is written, preventing anybody 
> from seeing empty meta file for avoiding the issue.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Commented] (HDFS-10331) Use java.util.zip.CRC32 for java8 or above in libhadoop

2016-06-12 Thread He Tianyi (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-10331?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15326218#comment-15326218
 ] 

He Tianyi commented on HDFS-10331:
--

Is there any code path (e.g. off heap buffer) still uses CRC32 implementation 
in libhadoop?

> Use java.util.zip.CRC32 for java8 or above in libhadoop
> ---
>
> Key: HDFS-10331
> URL: https://issues.apache.org/jira/browse/HDFS-10331
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: datanode, hdfs, hdfs-client
>Affects Versions: 2.6.0
>Reporter: He Tianyi
>
> In java8, performance of intrinsic CRC32 has been dramatically improved.
> See: https://bugs.openjdk.java.net/browse/JDK-7088419
> I carried an in-memory benchmark of throughput, on a server with two E5-2630 
> v2 cpus, results:
> (single threaded)
> java7  java.util.zip.CRC32: 0.81GB/s
> hdfs DataChecksum, native: 1.46GB/s
> java8  java.util.zip.CRC32: 2.39GB/s
> hdfs DataChecksum, CRC32 on java8: 2.39GB/s
> IMHO I think we could either:
> A) provide a configuration for user to switch CRC32 implementations;
> or B) On java8 or above, always use intrinsic CRC32.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Commented] (HDFS-10467) Router-based HDFS federation

2016-06-06 Thread He Tianyi (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-10467?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15317636#comment-15317636
 ] 

He Tianyi commented on HDFS-10467:
--

+1. This also reduces latency for first request from client (no failover on 
client-side, and router can memorize current active peer).

> Router-based HDFS federation
> 
>
> Key: HDFS-10467
> URL: https://issues.apache.org/jira/browse/HDFS-10467
> Project: Hadoop HDFS
>  Issue Type: New Feature
>  Components: fs
>Affects Versions: 2.7.2
>Reporter: Inigo Goiri
> Attachments: HDFS Router Federation.pdf, 
> HDFS-Router-Federation-Prototype.patch
>
>
> Add a Router to provide a federated view of multiple HDFS clusters.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Commented] (HDFS-10490) Client may never recovery replica after a timeout during sending packet

2016-06-06 Thread He Tianyi (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-10490?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15317587#comment-15317587
 ] 

He Tianyi commented on HDFS-10490:
--

Thanks for pointing that out, [~kihwal].
I think the case is a little different here. The catch is whether DataNode 
flushed metafile header to disk or not. 
If not (as in this case, output stream for metafile is not properly closed yet, 
for there is no way for DataNode to actively detect a network problem), 
{{BlockSender}} will see a 0-length metafile (not even containing header), then 
use (NULL, 512) for checksuming, client will still fail, given that 
configurated bytesPerChecksum != 512.

> Client may never recovery replica after a timeout during sending packet
> ---
>
> Key: HDFS-10490
> URL: https://issues.apache.org/jira/browse/HDFS-10490
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode
>Affects Versions: 2.6.0
>Reporter: He Tianyi
> Attachments: HDFS-10490.patch
>
>
> For newly created replica, a meta file is created in constructor of 
> {{BlockReceiver}} (for {{WRITE_BLOCK}} op). Its header will be written lazily 
> (buffered in memory first by {{BufferedOutputStream}}). 
> If following packets fail to deliver (e.g. in  extreme network condition), 
> the header may never get flush until closed. 
> However, {{BlockReceiver}} will not call close until block receiving is 
> finished or exception(s) encountered. Also in extreme network condition, both 
> RST & FIN may not deliver in time. 
> In this case, if client tries to initiates a {{transferBlock}} to a new 
> datanode (in {{addDatanode2ExistingPipeline}}), existing datanode will see an 
> empty meta if its {{BlockReceiver}} did not close in time. 
> Then, after HDFS-3429, a default {{DataChecksum}} (NULL, 512) will be used 
> during transfer. So when client then tries to recover pipeline after 
> completely transferred, it may encounter the following exception:
> {noformat}
> java.io.IOException: Client requested checksum DataChecksum(type=CRC32C, 
> chunkSize=4096) when appending to an existing block with different chunk 
> size: DataChecksum(type=NULL, chunkSize=512)
> at 
> org.apache.hadoop.hdfs.server.datanode.ReplicaInPipeline.createStreams(ReplicaInPipeline.java:230)
> at 
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.(BlockReceiver.java:226)
> at 
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:798)
> at 
> org.apache.hadoop.hdfs.protocol.datatransfer.Receiver.opWriteBlock(Receiver.java:166)
> at 
> org.apache.hadoop.hdfs.protocol.datatransfer.Receiver.processOp(Receiver.java:76)
> at 
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:243)
> at java.lang.Thread.run(Thread.java:745)
> {noformat}
> This will repeat, until exhausted by datanode replacement policy.
> Also to note that, with bad luck (like I), 20k clients are all doing this. 
> It's to some extend a DDoS attack to NameNode (because of 
> getAdditionalDataNode calls).
> I suggest we flush immediately after header is written, preventing anybody 
> from seeing empty meta file for avoiding the issue.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Updated] (HDFS-10490) Client may never recovery replica after a timeout during sending packet

2016-06-05 Thread He Tianyi (JIRA)

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

He Tianyi updated HDFS-10490:
-
Status: Patch Available  (was: Open)

> Client may never recovery replica after a timeout during sending packet
> ---
>
> Key: HDFS-10490
> URL: https://issues.apache.org/jira/browse/HDFS-10490
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode
>Affects Versions: 2.6.0
>Reporter: He Tianyi
> Attachments: HDFS-10490.patch
>
>
> For newly created replica, a meta file is created in constructor of 
> {{BlockReceiver}} (for {{WRITE_BLOCK}} op). Its header will be written lazily 
> (buffered in memory first by {{BufferedOutputStream}}). 
> If following packets fail to deliver (e.g. in  extreme network condition), 
> the header may never get flush until closed. 
> However, {{BlockReceiver}} will not call close until block receiving is 
> finished or exception(s) encountered. Also in extreme network condition, both 
> RST & FIN may not deliver in time. 
> In this case, if client tries to initiates a {{transferBlock}} to a new 
> datanode (in {{addDatanode2ExistingPipeline}}), existing datanode will see an 
> empty meta if its {{BlockReceiver}} did not close in time. 
> Then, after HDFS-3429, a default {{DataChecksum}} (NULL, 512) will be used 
> during transfer. So when client then tries to recover pipeline after 
> completely transferred, it may encounter the following exception:
> {noformat}
> java.io.IOException: Client requested checksum DataChecksum(type=CRC32C, 
> chunkSize=4096) when appending to an existing block with different chunk 
> size: DataChecksum(type=NULL, chunkSize=512)
> at 
> org.apache.hadoop.hdfs.server.datanode.ReplicaInPipeline.createStreams(ReplicaInPipeline.java:230)
> at 
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.(BlockReceiver.java:226)
> at 
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:798)
> at 
> org.apache.hadoop.hdfs.protocol.datatransfer.Receiver.opWriteBlock(Receiver.java:166)
> at 
> org.apache.hadoop.hdfs.protocol.datatransfer.Receiver.processOp(Receiver.java:76)
> at 
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:243)
> at java.lang.Thread.run(Thread.java:745)
> {noformat}
> This will repeat, until exhausted by datanode replacement policy.
> Also to note that, with bad luck (like I), 20k clients are all doing this. 
> It's to some extend a DDoS attack to NameNode (because of 
> getAdditionalDataNode calls).
> I suggest we flush immediately after header is written, preventing anybody 
> from seeing empty meta file for avoiding the issue.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Updated] (HDFS-10490) Client may never recovery replica after a timeout during sending packet

2016-06-05 Thread He Tianyi (JIRA)

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

He Tianyi updated HDFS-10490:
-
Attachment: HDFS-10490.patch

> Client may never recovery replica after a timeout during sending packet
> ---
>
> Key: HDFS-10490
> URL: https://issues.apache.org/jira/browse/HDFS-10490
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode
>Affects Versions: 2.6.0
>Reporter: He Tianyi
> Attachments: HDFS-10490.patch
>
>
> For newly created replica, a meta file is created in constructor of 
> {{BlockReceiver}} (for {{WRITE_BLOCK}} op). Its header will be written lazily 
> (buffered in memory first by {{BufferedOutputStream}}). 
> If following packets fail to deliver (e.g. in  extreme network condition), 
> the header may never get flush until closed. 
> However, {{BlockReceiver}} will not call close until block receiving is 
> finished or exception(s) encountered. Also in extreme network condition, both 
> RST & FIN may not deliver in time. 
> In this case, if client tries to initiates a {{transferBlock}} to a new 
> datanode (in {{addDatanode2ExistingPipeline}}), existing datanode will see an 
> empty meta if its {{BlockReceiver}} did not close in time. 
> Then, after HDFS-3429, a default {{DataChecksum}} (NULL, 512) will be used 
> during transfer. So when client then tries to recover pipeline after 
> completely transferred, it may encounter the following exception:
> {noformat}
> java.io.IOException: Client requested checksum DataChecksum(type=CRC32C, 
> chunkSize=4096) when appending to an existing block with different chunk 
> size: DataChecksum(type=NULL, chunkSize=512)
> at 
> org.apache.hadoop.hdfs.server.datanode.ReplicaInPipeline.createStreams(ReplicaInPipeline.java:230)
> at 
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.(BlockReceiver.java:226)
> at 
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:798)
> at 
> org.apache.hadoop.hdfs.protocol.datatransfer.Receiver.opWriteBlock(Receiver.java:166)
> at 
> org.apache.hadoop.hdfs.protocol.datatransfer.Receiver.processOp(Receiver.java:76)
> at 
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:243)
> at java.lang.Thread.run(Thread.java:745)
> {noformat}
> This will repeat, until exhausted by datanode replacement policy.
> Also to note that, with bad luck (like I), 20k clients are all doing this. 
> It's to some extend a DDoS attack to NameNode (because of 
> getAdditionalDataNode calls).
> I suggest we flush immediately after header is written, preventing anybody 
> from seeing empty meta file for avoiding the issue.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Created] (HDFS-10490) Client may never recovery replica after a timeout during sending packet

2016-06-05 Thread He Tianyi (JIRA)
He Tianyi created HDFS-10490:


 Summary: Client may never recovery replica after a timeout during 
sending packet
 Key: HDFS-10490
 URL: https://issues.apache.org/jira/browse/HDFS-10490
 Project: Hadoop HDFS
  Issue Type: Bug
  Components: datanode
Affects Versions: 2.6.0
Reporter: He Tianyi


For newly created replica, a meta file is created in constructor of 
{{BlockReceiver}} (for {{WRITE_BLOCK}} op). Its header will be written lazily 
(buffered in memory first by {{BufferedOutputStream}}). 
If following packets fail to deliver (e.g. in  extreme network condition), the 
header may never get flush until closed. 
However, {{BlockReceiver}} will not call close until block receiving is 
finished or exception(s) encountered. Also in extreme network condition, both 
RST & FIN may not deliver in time. 

In this case, if client tries to initiates a {{transferBlock}} to a new 
datanode (in {{addDatanode2ExistingPipeline}}), existing datanode will see an 
empty meta if its {{BlockReceiver}} did not close in time. 
Then, after HDFS-3429, a default {{DataChecksum}} (NULL, 512) will be used 
during transfer. So when client then tries to recover pipeline after completely 
transferred, it may encounter the following exception:
{noformat}
java.io.IOException: Client requested checksum DataChecksum(type=CRC32C, 
chunkSize=4096) when appending to an existing block with different chunk size: 
DataChecksum(type=NULL, chunkSize=512)
at 
org.apache.hadoop.hdfs.server.datanode.ReplicaInPipeline.createStreams(ReplicaInPipeline.java:230)
at 
org.apache.hadoop.hdfs.server.datanode.BlockReceiver.(BlockReceiver.java:226)
at 
org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:798)
at 
org.apache.hadoop.hdfs.protocol.datatransfer.Receiver.opWriteBlock(Receiver.java:166)
at 
org.apache.hadoop.hdfs.protocol.datatransfer.Receiver.processOp(Receiver.java:76)
at 
org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:243)
at java.lang.Thread.run(Thread.java:745)
{noformat}
This will repeat, until exhausted by datanode replacement policy.

Also to note that, with bad luck (like I), 20k clients are all doing this. It's 
to some extend a DDoS attack to NameNode (because of getAdditionalDataNode 
calls).

I suggest we flush immediately after header is written, preventing anybody from 
seeing empty meta file for avoiding the issue.




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Commented] (HDFS-10467) Router-based HDFS federation

2016-05-28 Thread He Tianyi (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-10467?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15305716#comment-15305716
 ] 

He Tianyi commented on HDFS-10467:
--

Thanks for sharing.

I've implemented similar approach as a separated project, see 
https://github.com/bytedance/nnproxy.
I am currently using it for backing 2 namenodes with a mount table with 20+ 
entries in production and worked well. (about 12K TPS)

Looks like HDFS Router Federation includes more features. Shall we work 
together?

> Router-based HDFS federation
> 
>
> Key: HDFS-10467
> URL: https://issues.apache.org/jira/browse/HDFS-10467
> Project: Hadoop HDFS
>  Issue Type: New Feature
>  Components: fs
>Affects Versions: 2.7.2
>Reporter: Inigo Goiri
> Attachments: HDFS Router Federation.pdf
>
>
> Add a Router to provide a federated view of multiple HDFS clusters.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Updated] (HDFS-10331) Use java.util.zip.CRC32 for java8 or above in libhadoop

2016-04-26 Thread He Tianyi (JIRA)

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

He Tianyi updated HDFS-10331:
-
Description: 
In java8, performance of intrinsic CRC32 has been dramatically improved.
See: https://bugs.openjdk.java.net/browse/JDK-7088419

I carried an in-memory benchmark of throughput, on a server with two E5-2630 v2 
cpus, results:
(single threaded)
java7  java.util.zip.CRC32: 0.81GB/s
hdfs DataChecksum, native: 1.46GB/s
java8  java.util.zip.CRC32: 2.39GB/s
hdfs DataChecksum, CRC32 on java8: 2.39GB/s

IMHO I think we could either:
A) provide a configuration for user to switch CRC32 implementations;
or B) On java8 or above, always use intrinsic CRC32.


  was:
In java8, performance of intrinsic CRC32 has been dramatically improved.
See: https://bugs.openjdk.java.net/browse/JDK-7088419

I carried an in-memory benchmark of throughput, on a server with two E5-2630 v2 
cpus, results:
java7  java.util.zip.CRC32: 0.81GB/s
hdfs DataChecksum, native: 1.46GB/s
java8  java.util.zip.CRC32: 2.39GB/s
hdfs DataChecksum, CRC32 on java8: 2.39GB/s

IMHO I think we could either:
A) provide a configuration for user to switch CRC32 implementations;
or B) On java8 or above, always use intrinsic CRC32.



> Use java.util.zip.CRC32 for java8 or above in libhadoop
> ---
>
> Key: HDFS-10331
> URL: https://issues.apache.org/jira/browse/HDFS-10331
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: datanode, hdfs, hdfs-client
>Affects Versions: 2.6.0
>Reporter: He Tianyi
>
> In java8, performance of intrinsic CRC32 has been dramatically improved.
> See: https://bugs.openjdk.java.net/browse/JDK-7088419
> I carried an in-memory benchmark of throughput, on a server with two E5-2630 
> v2 cpus, results:
> (single threaded)
> java7  java.util.zip.CRC32: 0.81GB/s
> hdfs DataChecksum, native: 1.46GB/s
> java8  java.util.zip.CRC32: 2.39GB/s
> hdfs DataChecksum, CRC32 on java8: 2.39GB/s
> IMHO I think we could either:
> A) provide a configuration for user to switch CRC32 implementations;
> or B) On java8 or above, always use intrinsic CRC32.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HDFS-10331) Use java.util.zip.CRC32 for checksum in java8 or above

2016-04-26 Thread He Tianyi (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-10331?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15259182#comment-15259182
 ] 

He Tianyi commented on HDFS-10331:
--

Yes, okay. Now I see that for array-backed buffers, native implementation in 
libhadoop is not used.

> Use java.util.zip.CRC32 for checksum in java8 or above
> --
>
> Key: HDFS-10331
> URL: https://issues.apache.org/jira/browse/HDFS-10331
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: datanode, hdfs, hdfs-client
>Affects Versions: 2.6.0
>Reporter: He Tianyi
>
> In java8, performance of intrinsic CRC32 has been dramatically improved.
> See: https://bugs.openjdk.java.net/browse/JDK-7088419
> I carried an in-memory benchmark of throughput, on a server with two E5-2630 
> v2 cpus, results:
> java7  java.util.zip.CRC32: 0.81GB/s
> hdfs DataChecksum, native: 1.46GB/s
> java8  java.util.zip.CRC32: 2.39GB/s
> hdfs DataChecksum, CRC32 on java8: 2.39GB/s
> IMHO I think we could either:
> A) provide a configuration for user to switch CRC32 implementations;
> or B) On java8 or above, always use intrinsic CRC32.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HDFS-10326) Disable setting tcp socket send/receive buffers for write pipelines

2016-04-26 Thread He Tianyi (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-10326?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15259173#comment-15259173
 ] 

He Tianyi commented on HDFS-10326:
--

[~cmccabe] [~mingma]
https://www.kernel.org/doc/ols/2009/ols2009-pages-169-184.pdf
This document suggests auto tuning is not introduced in Linux 2.4 before 2.4.27 
or Linux 2.6 before 2.6.7.
That's very old.

So maybe it's appropriate to enable auto tuning by default.

> Disable setting tcp socket send/receive buffers for write pipelines
> ---
>
> Key: HDFS-10326
> URL: https://issues.apache.org/jira/browse/HDFS-10326
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode, hdfs
>Affects Versions: 2.6.0
>Reporter: Daryn Sharp
>Assignee: Daryn Sharp
>
> The DataStreamer and the Datanode use a hardcoded 
> DEFAULT_DATA_SOCKET_SIZE=128K for the send and receive buffers of a write 
> pipeline.  Explicitly setting tcp buffer sizes disables tcp stack 
> auto-tuning.  
> The hardcoded value will saturate a 1Gb with 1ms RTT.  105Mbs at 10ms.  
> Paltry 11Mbs over a 100ms long haul.  10Gb networks are underutilized.
> There should either be a configuration to completely disable setting the 
> buffers, or the the setReceiveBuffer and setSendBuffer should be removed 
> entirely.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HDFS-10331) Use java.util.zip.CRC32 for checksum in java8 or above

2016-04-26 Thread He Tianyi (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-10331?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15258218#comment-15258218
 ] 

He Tianyi commented on HDFS-10331:
--

Forgot to add, on same platform.
The new intrinsic CRC32 also uses CLMUL instruction sets, which does not apply 
on some other platforms.

> Use java.util.zip.CRC32 for checksum in java8 or above
> --
>
> Key: HDFS-10331
> URL: https://issues.apache.org/jira/browse/HDFS-10331
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: datanode, hdfs, hdfs-client
>Affects Versions: 2.6.0
>Reporter: He Tianyi
>
> In java8, performance of intrinsic CRC32 has been dramatically improved.
> See: https://bugs.openjdk.java.net/browse/JDK-7088419
> I carried an in-memory benchmark of throughput, on a server with two E5-2630 
> v2 cpus, results:
> java7  java.util.zip.CRC32: 0.81GB/s
> hdfs DataChecksum, native: 1.46GB/s
> java8  java.util.zip.CRC32: 2.39GB/s
> hdfs DataChecksum, CRC32 on java8: 2.39GB/s
> IMHO I think we could either:
> A) provide a configuration for user to switch CRC32 implementations;
> or B) On java8 or above, always use intrinsic CRC32.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (HDFS-10331) Use java.util.zip.CRC32 for checksum in java8 or above

2016-04-26 Thread He Tianyi (JIRA)
He Tianyi created HDFS-10331:


 Summary: Use java.util.zip.CRC32 for checksum in java8 or above
 Key: HDFS-10331
 URL: https://issues.apache.org/jira/browse/HDFS-10331
 Project: Hadoop HDFS
  Issue Type: Improvement
  Components: datanode, hdfs, hdfs-client
Affects Versions: 2.6.0
Reporter: He Tianyi


In java8, performance of intrinsic CRC32 has been dramatically improved.
See: https://bugs.openjdk.java.net/browse/JDK-7088419

I carried an in-memory benchmark of throughput, on a server with two E5-2630 v2 
cpus, results:
java7  java.util.zip.CRC32: 0.81GB/s
hdfs DataChecksum, native: 1.46GB/s
java8  java.util.zip.CRC32: 2.39GB/s
hdfs DataChecksum, CRC32 on java8: 2.39GB/s

IMHO I think we could either:
A) provide a configuration for user to switch CRC32 implementations;
or B) On java8 or above, always use intrinsic CRC32.




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HDFS-10326) Disable setting tcp socket send/receive buffers for write pipelines

2016-04-25 Thread He Tianyi (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-10326?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15257422#comment-15257422
 ] 

He Tianyi commented on HDFS-10326:
--

Some system may not support auto tuning, defaulting to a small window size (say 
64k? which may make the scenario worse).

I'd suggest we keep the configuration.  Or maybe add another one, say 
{{dfs.socket.detect-auto-turning}}. When this is set to {{true}} (maybe turned 
on by default), socket buffer behavior depends on whether OS supports 
auto-tuning. If auto-tuning is not supported, use configured value 
automatically.

> Disable setting tcp socket send/receive buffers for write pipelines
> ---
>
> Key: HDFS-10326
> URL: https://issues.apache.org/jira/browse/HDFS-10326
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode, hdfs
>Affects Versions: 2.6.0
>Reporter: Daryn Sharp
>Assignee: Daryn Sharp
>
> The DataStreamer and the Datanode use a hardcoded 
> DEFAULT_DATA_SOCKET_SIZE=128K for the send and receive buffers of a write 
> pipeline.  Explicitly setting tcp buffer sizes disables tcp stack 
> auto-tuning.  
> The hardcoded value will saturate a 1Gb with 1ms RTT.  105Mbs at 10ms.  
> Paltry 11Mbs over a 100ms long haul.  10Gb networks are underutilized.
> There should either be a configuration to completely disable setting the 
> buffers, or the the setReceiveBuffer and setSendBuffer should be removed 
> entirely.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HDFS-9412) getBlocks occupies FSLock and takes too long to complete

2016-04-17 Thread He Tianyi (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-9412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15244578#comment-15244578
 ] 

He Tianyi commented on HDFS-9412:
-

Hi, [~walter.k.su]. Would you commit this patch when available?

> getBlocks occupies FSLock and takes too long to complete
> 
>
> Key: HDFS-9412
> URL: https://issues.apache.org/jira/browse/HDFS-9412
> Project: Hadoop HDFS
>  Issue Type: Improvement
>Reporter: He Tianyi
>Assignee: He Tianyi
> Attachments: HDFS-9412..patch, HDFS-9412.0001.patch, 
> HDFS-9412.0002.patch
>
>
> {{getBlocks}} in {{NameNodeRpcServer}} acquires a read lock then may take a 
> long time to complete (probably several seconds, if number of blocks are too 
> much). 
> During this period, other threads attempting to acquire write lock will wait. 
> In an extreme case, RPC handlers are occupied by one reader thread calling 
> {{getBlocks}} and all other threads waiting for write lock, rpc server acts 
> like hung. Unfortunately, this tends to happen in heavy loaded cluster, since 
> read operations come and go fast (they do not need to wait), leaving write 
> operations waiting.
> Looks like we can optimize this thing like DN block report did in past, by 
> splitting the operation into smaller sub operations, and let other threads do 
> their work between each sub operation. The whole result is returned at once, 
> though (one thing different from DN block report). 
> I am not sure whether this will work. Any better idea?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (HDFS-10290) Move getBlocks calls to DataNode in Balancer

2016-04-14 Thread He Tianyi (JIRA)
He Tianyi created HDFS-10290:


 Summary: Move getBlocks calls to DataNode in Balancer
 Key: HDFS-10290
 URL: https://issues.apache.org/jira/browse/HDFS-10290
 Project: Hadoop HDFS
  Issue Type: New Feature
  Components: balancer & mover
Affects Versions: 2.6.0
Reporter: He Tianyi


In current implementation, Balancer asks NameNode for a list of blocks on 
specific DataNode. This made workload of NameNode heavier, and actually it 
caused NameNode flappy when average # of blocks on each DataNode reaches 
1,000,000 (NameNode heap size is 192GB, cpu: Xeon E5-2630 * 2).

Recently I investigated whether {{getBlocks}} invocation from Balancer can be 
handled by DataNodes, turned out to be practical. 
The only pitfall is: since DataNode has no information about other locations of 
each block it possesses, some block move may fail (since target node may 
already has a replica of that particular block).

I think this may be beneficial for large clusters.

Any suggestions or comments?
Thanks.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HDFS-9412) getBlocks occupies FSLock and takes too long to complete

2016-04-13 Thread He Tianyi (JIRA)

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

He Tianyi updated HDFS-9412:

Attachment: HDFS-9412.0002.patch

Fix codestyle, whitespace and unit test.

> getBlocks occupies FSLock and takes too long to complete
> 
>
> Key: HDFS-9412
> URL: https://issues.apache.org/jira/browse/HDFS-9412
> Project: Hadoop HDFS
>  Issue Type: Improvement
>Reporter: He Tianyi
>Assignee: He Tianyi
> Attachments: HDFS-9412..patch, HDFS-9412.0001.patch, 
> HDFS-9412.0002.patch
>
>
> {{getBlocks}} in {{NameNodeRpcServer}} acquires a read lock then may take a 
> long time to complete (probably several seconds, if number of blocks are too 
> much). 
> During this period, other threads attempting to acquire write lock will wait. 
> In an extreme case, RPC handlers are occupied by one reader thread calling 
> {{getBlocks}} and all other threads waiting for write lock, rpc server acts 
> like hung. Unfortunately, this tends to happen in heavy loaded cluster, since 
> read operations come and go fast (they do not need to wait), leaving write 
> operations waiting.
> Looks like we can optimize this thing like DN block report did in past, by 
> splitting the operation into smaller sub operations, and let other threads do 
> their work between each sub operation. The whole result is returned at once, 
> though (one thing different from DN block report). 
> I am not sure whether this will work. Any better idea?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HDFS-9412) getBlocks occupies FSLock and takes too long to complete

2016-04-13 Thread He Tianyi (JIRA)

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

He Tianyi updated HDFS-9412:

Attachment: HDFS-9412.0001.patch

Rebase against current trunk.

> getBlocks occupies FSLock and takes too long to complete
> 
>
> Key: HDFS-9412
> URL: https://issues.apache.org/jira/browse/HDFS-9412
> Project: Hadoop HDFS
>  Issue Type: Improvement
>Reporter: He Tianyi
>Assignee: He Tianyi
> Attachments: HDFS-9412..patch, HDFS-9412.0001.patch
>
>
> {{getBlocks}} in {{NameNodeRpcServer}} acquires a read lock then may take a 
> long time to complete (probably several seconds, if number of blocks are too 
> much). 
> During this period, other threads attempting to acquire write lock will wait. 
> In an extreme case, RPC handlers are occupied by one reader thread calling 
> {{getBlocks}} and all other threads waiting for write lock, rpc server acts 
> like hung. Unfortunately, this tends to happen in heavy loaded cluster, since 
> read operations come and go fast (they do not need to wait), leaving write 
> operations waiting.
> Looks like we can optimize this thing like DN block report did in past, by 
> splitting the operation into smaller sub operations, and let other threads do 
> their work between each sub operation. The whole result is returned at once, 
> though (one thing different from DN block report). 
> I am not sure whether this will work. Any better idea?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (HDFS-9725) Make capacity of centralized cache dynamic

2016-01-30 Thread He Tianyi (JIRA)
He Tianyi created HDFS-9725:
---

 Summary: Make capacity of centralized cache dynamic
 Key: HDFS-9725
 URL: https://issues.apache.org/jira/browse/HDFS-9725
 Project: Hadoop HDFS
  Issue Type: Wish
  Components: caching, datanode, namenode
Affects Versions: 2.6.0
Reporter: He Tianyi
Assignee: He Tianyi
Priority: Minor


Currently in centralized cache, Datanode uses {{mlock}} to keep blocks in 
memory, with limit of maximum amount of bytes specified by 
{{dnConf.maxLockedMemory}}.
In general deployment, each machine run both Datanode and Nodemanager. In this 
case, statically specified memory capacity either potentially causes OOM, or 
hurts memory utilization. 
That is, if one specify a large capacity for caching (permitted by {{ulimit}} 
as prerequisite), Datanode may have gone too far to reserve any memory for new 
container process to launch from NodeManager. On the other hand, specifying a 
small value may leave memory insufficiently used.

A simple idea is: perhaps it is better to make cache capacity dynamic. 
Adjusting its capacity corresponding to current (or future, ideally) memory 
usage to avoid problems above.

Any suggestions or comments?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HDFS-8829) Make SO_RCVBUF and SO_SNDBUF size configurable for DataTransferProtocol sockets and allow configuring auto-tuning

2016-01-30 Thread He Tianyi (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-8829?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15124825#comment-15124825
 ] 

He Tianyi commented on HDFS-8829:
-

Sorry for late response.
There is one corner cases: Old linux kernels do not support auto tuning. I am 
not sure what would happen when setting these values to 0.

> Make SO_RCVBUF and SO_SNDBUF size configurable for DataTransferProtocol 
> sockets and allow configuring auto-tuning
> -
>
> Key: HDFS-8829
> URL: https://issues.apache.org/jira/browse/HDFS-8829
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: datanode
>Affects Versions: 2.3.0, 2.6.0
>Reporter: He Tianyi
>Assignee: He Tianyi
> Fix For: 2.8.0
>
> Attachments: HDFS-8829.0001.patch, HDFS-8829.0002.patch, 
> HDFS-8829.0003.patch, HDFS-8829.0004.patch, HDFS-8829.0005.patch, 
> HDFS-8829.0006.patch
>
>
> {code:java}
>   private void initDataXceiver(Configuration conf) throws IOException {
> // find free port or use privileged port provided
> TcpPeerServer tcpPeerServer;
> if (secureResources != null) {
>   tcpPeerServer = new TcpPeerServer(secureResources);
> } else {
>   tcpPeerServer = new TcpPeerServer(dnConf.socketWriteTimeout,
>   DataNode.getStreamingAddr(conf));
> }
> 
> tcpPeerServer.setReceiveBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
> {code}
> The last line sets SO_RCVBUF explicitly, thus disabling tcp auto-tuning on 
> some system.
> Shall we make this behavior configurable?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HDFS-9412) getBlocks occupies FSLock and takes too long to complete

2016-01-30 Thread He Tianyi (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-9412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15124822#comment-15124822
 ] 

He Tianyi commented on HDFS-9412:
-

Added ~200 nodes to HDFS cluster recently. As balancing become radical, 
NameNode still have spikes in latency, but better than before (usually peak 
queue time reaches 10~20s, now is 5~6s).

> getBlocks occupies FSLock and takes too long to complete
> 
>
> Key: HDFS-9412
> URL: https://issues.apache.org/jira/browse/HDFS-9412
> Project: Hadoop HDFS
>  Issue Type: Improvement
>Reporter: He Tianyi
>Assignee: He Tianyi
> Attachments: HDFS-9412..patch
>
>
> {{getBlocks}} in {{NameNodeRpcServer}} acquires a read lock then may take a 
> long time to complete (probably several seconds, if number of blocks are too 
> much). 
> During this period, other threads attempting to acquire write lock will wait. 
> In an extreme case, RPC handlers are occupied by one reader thread calling 
> {{getBlocks}} and all other threads waiting for write lock, rpc server acts 
> like hung. Unfortunately, this tends to happen in heavy loaded cluster, since 
> read operations come and go fast (they do not need to wait), leaving write 
> operations waiting.
> Looks like we can optimize this thing like DN block report did in past, by 
> splitting the operation into smaller sub operations, and let other threads do 
> their work between each sub operation. The whole result is returned at once, 
> though (one thing different from DN block report). 
> I am not sure whether this will work. Any better idea?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HDFS-9665) Cilents are easily affected by standby namenode

2016-01-19 Thread He Tianyi (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-9665?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15107854#comment-15107854
 ] 

He Tianyi commented on HDFS-9665:
-

Also, I observed that {{doTailEdits}} may acquire write lock in standby 
namenode and last several seconds in my cluster.
This causes first request from client to block either.

> Cilents are easily affected by standby namenode
> ---
>
> Key: HDFS-9665
> URL: https://issues.apache.org/jira/browse/HDFS-9665
> Project: Hadoop HDFS
>  Issue Type: Wish
>  Components: hdfs-client, namenode
>Affects Versions: 2.6.0
>Reporter: He Tianyi
>Assignee: He Tianyi
>Priority: Minor
>
> My case is during restarting of standby NameNode, there is chances that 
> {{hadoop fs}} command get hung until either IPC timeout reached or 
> {{StandbyException}} received, and then failover to active NameNode.
> Normally, duration of the 'hung stage' depends on 
> {{min(timeout_configuration, rpc_queue_time)}}. However, RPC queue in standby 
> NameNode is usually filled with block reports at this period, client requests 
> can't get processed quickly.
> I wish to get rid of this, by one of the following manners:
> a) we distinguish priority in RPC queue (chances causing starvation)
> b) we speculate first request, send it to both NameNodes, and take one valid 
> response.
> c) make client aware of HA state (by accessing ZK probably, chances causing 
> performance issue)
> Any suggestions or comments?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (HDFS-9665) Cilents are easily affected by standby namenode

2016-01-19 Thread He Tianyi (JIRA)
He Tianyi created HDFS-9665:
---

 Summary: Cilents are easily affected by standby namenode
 Key: HDFS-9665
 URL: https://issues.apache.org/jira/browse/HDFS-9665
 Project: Hadoop HDFS
  Issue Type: Wish
  Components: hdfs-client, namenode
Affects Versions: 2.6.0
Reporter: He Tianyi
Assignee: He Tianyi
Priority: Minor


My case is during restarting of standby NameNode, there is chances that 
{{hadoop fs}} command get hung until either IPC timeout reached or 
{{StandbyException}} received, and then failover to active NameNode.

Normally, duration of the 'hung stage' depends on {{min(timeout_configuration, 
rpc_queue_time)}}. However, RPC queue in standby NameNode is usually filled 
with block reports at this period, client requests can't get processed quickly.

I wish to get rid of this, by one of the following manners:
a) we distinguish priority in RPC queue (chances causing starvation)
b) we speculate first request, send it to both NameNodes, and take one valid 
response.
c) make client aware of HA state (by accessing ZK probably, chances causing 
performance issue)

Any suggestions or comments?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HDFS-9665) Cilents are easily affected by standby namenode

2016-01-19 Thread He Tianyi (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-9665?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15107896#comment-15107896
 ] 

He Tianyi commented on HDFS-9665:
-

Thanks. That patch worked well.

Closing issue.

> Cilents are easily affected by standby namenode
> ---
>
> Key: HDFS-9665
> URL: https://issues.apache.org/jira/browse/HDFS-9665
> Project: Hadoop HDFS
>  Issue Type: Wish
>  Components: hdfs-client, namenode
>Affects Versions: 2.6.0
>Reporter: He Tianyi
>Assignee: He Tianyi
>Priority: Minor
>
> My case is during restarting of standby NameNode, there is chances that 
> {{hadoop fs}} command get hung until either IPC timeout reached or 
> {{StandbyException}} received, and then failover to active NameNode.
> Normally, duration of the 'hung stage' depends on 
> {{min(timeout_configuration, rpc_queue_time)}}. However, RPC queue in standby 
> NameNode is usually filled with block reports at this period, client requests 
> can't get processed quickly.
> I wish to get rid of this, by one of the following manners:
> a) we distinguish priority in RPC queue (chances causing starvation)
> b) we speculate first request, send it to both NameNodes, and take one valid 
> response.
> c) make client aware of HA state (by accessing ZK probably, chances causing 
> performance issue)
> Any suggestions or comments?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Resolved] (HDFS-9665) Cilents are easily affected by standby namenode

2016-01-19 Thread He Tianyi (JIRA)

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

He Tianyi resolved HDFS-9665.
-
Resolution: Duplicate

> Cilents are easily affected by standby namenode
> ---
>
> Key: HDFS-9665
> URL: https://issues.apache.org/jira/browse/HDFS-9665
> Project: Hadoop HDFS
>  Issue Type: Wish
>  Components: hdfs-client, namenode
>Affects Versions: 2.6.0
>Reporter: He Tianyi
>Assignee: He Tianyi
>Priority: Minor
>
> My case is during restarting of standby NameNode, there is chances that 
> {{hadoop fs}} command get hung until either IPC timeout reached or 
> {{StandbyException}} received, and then failover to active NameNode.
> Normally, duration of the 'hung stage' depends on 
> {{min(timeout_configuration, rpc_queue_time)}}. However, RPC queue in standby 
> NameNode is usually filled with block reports at this period, client requests 
> can't get processed quickly.
> I wish to get rid of this, by one of the following manners:
> a) we distinguish priority in RPC queue (chances causing starvation)
> b) we speculate first request, send it to both NameNodes, and take one valid 
> response.
> c) make client aware of HA state (by accessing ZK probably, chances causing 
> performance issue)
> Any suggestions or comments?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HDFS-9412) getBlocks occupies FSLock and takes too long to complete

2015-12-20 Thread He Tianyi (JIRA)

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

He Tianyi updated HDFS-9412:

Status: Patch Available  (was: Open)

> getBlocks occupies FSLock and takes too long to complete
> 
>
> Key: HDFS-9412
> URL: https://issues.apache.org/jira/browse/HDFS-9412
> Project: Hadoop HDFS
>  Issue Type: Improvement
>Reporter: He Tianyi
>Assignee: He Tianyi
> Attachments: HDFS-9412..patch
>
>
> {{getBlocks}} in {{NameNodeRpcServer}} acquires a read lock then may take a 
> long time to complete (probably several seconds, if number of blocks are too 
> much). 
> During this period, other threads attempting to acquire write lock will wait. 
> In an extreme case, RPC handlers are occupied by one reader thread calling 
> {{getBlocks}} and all other threads waiting for write lock, rpc server acts 
> like hung. Unfortunately, this tends to happen in heavy loaded cluster, since 
> read operations come and go fast (they do not need to wait), leaving write 
> operations waiting.
> Looks like we can optimize this thing like DN block report did in past, by 
> splitting the operation into smaller sub operations, and let other threads do 
> their work between each sub operation. The whole result is returned at once, 
> though (one thing different from DN block report). 
> I am not sure whether this will work. Any better idea?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HDFS-9412) getBlocks occupies FSLock and takes too long to complete

2015-12-08 Thread He Tianyi (JIRA)

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

He Tianyi updated HDFS-9412:

Attachment: HDFS-9412..patch

A patch that skip small blocks in {{getBlocks}}. These are unused anyway.
This reduces average queue time of NameNode nearly 40% shorter during burst 
{{getBlocks}} requests with 160 dispatcher threads (in my cluster).

> getBlocks occupies FSLock and takes too long to complete
> 
>
> Key: HDFS-9412
> URL: https://issues.apache.org/jira/browse/HDFS-9412
> Project: Hadoop HDFS
>  Issue Type: Improvement
>Reporter: He Tianyi
>Assignee: He Tianyi
> Attachments: HDFS-9412..patch
>
>
> {{getBlocks}} in {{NameNodeRpcServer}} acquires a read lock then may take a 
> long time to complete (probably several seconds, if number of blocks are too 
> much). 
> During this period, other threads attempting to acquire write lock will wait. 
> In an extreme case, RPC handlers are occupied by one reader thread calling 
> {{getBlocks}} and all other threads waiting for write lock, rpc server acts 
> like hung. Unfortunately, this tends to happen in heavy loaded cluster, since 
> read operations come and go fast (they do not need to wait), leaving write 
> operations waiting.
> Looks like we can optimize this thing like DN block report did in past, by 
> splitting the operation into smaller sub operations, and let other threads do 
> their work between each sub operation. The whole result is returned at once, 
> though (one thing different from DN block report). 
> I am not sure whether this will work. Any better idea?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HDFS-9412) getBlocks occupies FSLock and takes too long to complete

2015-12-07 Thread He Tianyi (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-9412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15046417#comment-15046417
 ] 

He Tianyi commented on HDFS-9412:
-

[~andrew.wang] Perhaps switching to unfair RWLock may cause other issues, since 
machine running NameNode does not necessarily have SMP architecture. 

I think this is due to having many small blocks in cluster, {{getBlocks}} is 
called by Balancer and will not return until exhausted or total size satisfies, 
and there are actually many threads doing the same thing 
({{dfs.balancer.dispatcherThreads}}). 
Besides decreasing number of threads, maybe we can make this faster either.

> getBlocks occupies FSLock and takes too long to complete
> 
>
> Key: HDFS-9412
> URL: https://issues.apache.org/jira/browse/HDFS-9412
> Project: Hadoop HDFS
>  Issue Type: Improvement
>Reporter: He Tianyi
>Assignee: He Tianyi
>
> {{getBlocks}} in {{NameNodeRpcServer}} acquires a read lock then may take a 
> long time to complete (probably several seconds, if number of blocks are too 
> much). 
> During this period, other threads attempting to acquire write lock will wait. 
> In an extreme case, RPC handlers are occupied by one reader thread calling 
> {{getBlocks}} and all other threads waiting for write lock, rpc server acts 
> like hung. Unfortunately, this tends to happen in heavy loaded cluster, since 
> read operations come and go fast (they do not need to wait), leaving write 
> operations waiting.
> Looks like we can optimize this thing like DN block report did in past, by 
> splitting the operation into smaller sub operations, and let other threads do 
> their work between each sub operation. The whole result is returned at once, 
> though (one thing different from DN block report). 
> I am not sure whether this will work. Any better idea?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (HDFS-9412) getBlocks occupies FSLock and takes too long to complete

2015-11-11 Thread He Tianyi (JIRA)
He Tianyi created HDFS-9412:
---

 Summary: getBlocks occupies FSLock and takes too long to complete
 Key: HDFS-9412
 URL: https://issues.apache.org/jira/browse/HDFS-9412
 Project: Hadoop HDFS
  Issue Type: Improvement
Reporter: He Tianyi
Assignee: He Tianyi


{{getBlocks}} in {{NameNodeRpcServer}} acquires a read lock then may take a 
long time to complete (probably several seconds, if number of blocks are too 
much). 
During this period, other threads attempting to acquire write lock will wait. 
In an extreme case, RPC handlers are occupied by one reader thread calling 
{{getBlocks}} and all other threads waiting for write lock, rpc server acts 
like hung. Unfortunately, this tends to happen in heavy loaded cluster, since 
read operations come and go fast (they do not need to wait), leaving write 
operations waiting.

Looks like we can optimize this thing like DN block report did in past, by 
splitting the operation into smaller sub operations, and let other threads do 
their work between each sub operation. The whole result is returned at once, 
though (one thing different from DN block report). But there will be no more 
starvation.
I am not sure whether this will work. Any better idea?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HDFS-9412) getBlocks occupies FSLock and takes too long to complete

2015-11-11 Thread He Tianyi (JIRA)

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

He Tianyi updated HDFS-9412:

Description: 
{{getBlocks}} in {{NameNodeRpcServer}} acquires a read lock then may take a 
long time to complete (probably several seconds, if number of blocks are too 
much). 
During this period, other threads attempting to acquire write lock will wait. 
In an extreme case, RPC handlers are occupied by one reader thread calling 
{{getBlocks}} and all other threads waiting for write lock, rpc server acts 
like hung. Unfortunately, this tends to happen in heavy loaded cluster, since 
read operations come and go fast (they do not need to wait), leaving write 
operations waiting.

Looks like we can optimize this thing like DN block report did in past, by 
splitting the operation into smaller sub operations, and let other threads do 
their work between each sub operation. The whole result is returned at once, 
though (one thing different from DN block report). 
I am not sure whether this will work. Any better idea?

  was:
{{getBlocks}} in {{NameNodeRpcServer}} acquires a read lock then may take a 
long time to complete (probably several seconds, if number of blocks are too 
much). 
During this period, other threads attempting to acquire write lock will wait. 
In an extreme case, RPC handlers are occupied by one reader thread calling 
{{getBlocks}} and all other threads waiting for write lock, rpc server acts 
like hung. Unfortunately, this tends to happen in heavy loaded cluster, since 
read operations come and go fast (they do not need to wait), leaving write 
operations waiting.

Looks like we can optimize this thing like DN block report did in past, by 
splitting the operation into smaller sub operations, and let other threads do 
their work between each sub operation. The whole result is returned at once, 
though (one thing different from DN block report). But there will be no more 
starvation.
I am not sure whether this will work. Any better idea?


> getBlocks occupies FSLock and takes too long to complete
> 
>
> Key: HDFS-9412
> URL: https://issues.apache.org/jira/browse/HDFS-9412
> Project: Hadoop HDFS
>  Issue Type: Improvement
>Reporter: He Tianyi
>Assignee: He Tianyi
>
> {{getBlocks}} in {{NameNodeRpcServer}} acquires a read lock then may take a 
> long time to complete (probably several seconds, if number of blocks are too 
> much). 
> During this period, other threads attempting to acquire write lock will wait. 
> In an extreme case, RPC handlers are occupied by one reader thread calling 
> {{getBlocks}} and all other threads waiting for write lock, rpc server acts 
> like hung. Unfortunately, this tends to happen in heavy loaded cluster, since 
> read operations come and go fast (they do not need to wait), leaving write 
> operations waiting.
> Looks like we can optimize this thing like DN block report did in past, by 
> splitting the operation into smaller sub operations, and let other threads do 
> their work between each sub operation. The whole result is returned at once, 
> though (one thing different from DN block report). 
> I am not sure whether this will work. Any better idea?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HDFS-9412) getBlocks occupies FSLock and takes too long to complete

2015-11-11 Thread He Tianyi (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-9412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15000242#comment-15000242
 ] 

He Tianyi commented on HDFS-9412:
-

Sorry the misuse of word "starvation". It's not starvation exactly, just hung 
for a while.

> getBlocks occupies FSLock and takes too long to complete
> 
>
> Key: HDFS-9412
> URL: https://issues.apache.org/jira/browse/HDFS-9412
> Project: Hadoop HDFS
>  Issue Type: Improvement
>Reporter: He Tianyi
>Assignee: He Tianyi
>
> {{getBlocks}} in {{NameNodeRpcServer}} acquires a read lock then may take a 
> long time to complete (probably several seconds, if number of blocks are too 
> much). 
> During this period, other threads attempting to acquire write lock will wait. 
> In an extreme case, RPC handlers are occupied by one reader thread calling 
> {{getBlocks}} and all other threads waiting for write lock, rpc server acts 
> like hung. Unfortunately, this tends to happen in heavy loaded cluster, since 
> read operations come and go fast (they do not need to wait), leaving write 
> operations waiting.
> Looks like we can optimize this thing like DN block report did in past, by 
> splitting the operation into smaller sub operations, and let other threads do 
> their work between each sub operation. The whole result is returned at once, 
> though (one thing different from DN block report). 
> I am not sure whether this will work. Any better idea?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HDFS-9412) getBlocks occupies FSLock and takes too long to complete

2015-11-11 Thread He Tianyi (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-9412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15000269#comment-15000269
 ] 

He Tianyi commented on HDFS-9412:
-

A simpler approach would be ignore small blocks during iteration (just as 
Balancer did in HDFS-8824, if HDFS-8824 is implemented, Balancer don't need 
these blocks anyway), this will reduce the cost of gathering enough amount of 
size of data (I did not do the profiling but perhaps there are enough overhead 
in {{addBlock}}, lots of objects are created).

> getBlocks occupies FSLock and takes too long to complete
> 
>
> Key: HDFS-9412
> URL: https://issues.apache.org/jira/browse/HDFS-9412
> Project: Hadoop HDFS
>  Issue Type: Improvement
>Reporter: He Tianyi
>Assignee: He Tianyi
>
> {{getBlocks}} in {{NameNodeRpcServer}} acquires a read lock then may take a 
> long time to complete (probably several seconds, if number of blocks are too 
> much). 
> During this period, other threads attempting to acquire write lock will wait. 
> In an extreme case, RPC handlers are occupied by one reader thread calling 
> {{getBlocks}} and all other threads waiting for write lock, rpc server acts 
> like hung. Unfortunately, this tends to happen in heavy loaded cluster, since 
> read operations come and go fast (they do not need to wait), leaving write 
> operations waiting.
> Looks like we can optimize this thing like DN block report did in past, by 
> splitting the operation into smaller sub operations, and let other threads do 
> their work between each sub operation. The whole result is returned at once, 
> though (one thing different from DN block report). 
> I am not sure whether this will work. Any better idea?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HDFS-9071) chooseTargets in ReplicationWork may pass incomplete srcPath

2015-10-08 Thread He Tianyi (JIRA)

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

He Tianyi updated HDFS-9071:

Attachment: HDFS-9071.0004.patch

Fix jenkins

> chooseTargets in ReplicationWork may pass incomplete srcPath
> 
>
> Key: HDFS-9071
> URL: https://issues.apache.org/jira/browse/HDFS-9071
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: namenode
>Affects Versions: 2.6.0
>Reporter: He Tianyi
>Assignee: He Tianyi
> Attachments: HDFS-9071.0001.patch, HDFS-9071.0001.patch, 
> HDFS-9071.0003.patch, HDFS-9071.0004.patch
>
>
> I've observed that chooseTargets in ReplicationWork may pass incomplete 
> srcPath (not starting with '/') to block placement policy.
> It is possible that srcPath is extensively used in custom placement policy. 
> In this case, the incomplete srcPath may further cause AssertionError if try 
> to get INode with it inside placement policy.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HDFS-9071) chooseTargets in ReplicationWork may pass incomplete srcPath

2015-10-07 Thread He Tianyi (JIRA)

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

He Tianyi updated HDFS-9071:

Attachment: HDFS-9071.0003.patch

Rebase.

> chooseTargets in ReplicationWork may pass incomplete srcPath
> 
>
> Key: HDFS-9071
> URL: https://issues.apache.org/jira/browse/HDFS-9071
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: namenode
>Affects Versions: 2.6.0
>Reporter: He Tianyi
>Assignee: He Tianyi
> Attachments: HDFS-9071.0001.patch, HDFS-9071.0001.patch, 
> HDFS-9071.0003.patch
>
>
> I've observed that chooseTargets in ReplicationWork may pass incomplete 
> srcPath (not starting with '/') to block placement policy.
> It is possible that srcPath is extensively used in custom placement policy. 
> In this case, the incomplete srcPath may further cause AssertionError if try 
> to get INode with it inside placement policy.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HDFS-9149) Consider multi datacenter when sortByDistance

2015-10-02 Thread He Tianyi (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-9149?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14941100#comment-14941100
 ] 

He Tianyi commented on HDFS-9149:
-

I think that's a good point [~hexiaoqiao].

One simple idea is generalizes {{getWeight}} into a function that calculates 
distance between two locations (more like {{getDistance}}), regardless of the 
meaning of each hierarchy. 

The only thing is that, I'm not aware why did {{getWeight}} designed to be like 
this in the first place, i.e. whether there is some particular concern. 
Does someone know the idea behind this design choice?

> Consider multi datacenter when sortByDistance
> -
>
> Key: HDFS-9149
> URL: https://issues.apache.org/jira/browse/HDFS-9149
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: namenode
>Reporter: He Xiaoqiao
>Assignee: He Tianyi
>
> {{sortByDistance}} doesn't consider multi-datacenter when read data, so there 
> my be reading data via other datacenter when hadoop deployment with multi-IDC.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Assigned] (HDFS-9149) Consider multi datacenter when sortByDistance

2015-10-02 Thread He Tianyi (JIRA)

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

He Tianyi reassigned HDFS-9149:
---

Assignee: He Tianyi

> Consider multi datacenter when sortByDistance
> -
>
> Key: HDFS-9149
> URL: https://issues.apache.org/jira/browse/HDFS-9149
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: namenode
>Reporter: He Xiaoqiao
>Assignee: He Tianyi
>
> {{sortByDistance}} doesn't consider multi-datacenter when read data, so there 
> my be reading data via other datacenter when hadoop deployment with multi-IDC.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HDFS-9149) Consider multi datacenter when sortByDistance

2015-10-02 Thread He Tianyi (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-9149?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14941264#comment-14941264
 ] 

He Tianyi commented on HDFS-9149:
-

Thanks, [~hexiaoqiao].

The simpler idea sounds good! But I'm not quite sure adding one if statement 
could cover all cases.
We'll need to assume that grandparent represents a IDC node if we go with it, 
which does not always hold (since {{NetworkTopology}} did not imply that). e.g. 
I have a real scenario that location are configured like 
{{/DC/BUILDING/RACK/NODE}}. In this case, it is true that locality will happen 
to be better, but perhaps not better enough.


> Consider multi datacenter when sortByDistance
> -
>
> Key: HDFS-9149
> URL: https://issues.apache.org/jira/browse/HDFS-9149
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: namenode
>Reporter: He Xiaoqiao
>Assignee: He Tianyi
>
> {{sortByDistance}} doesn't consider multi-datacenter when read data, so there 
> my be reading data via other datacenter when hadoop deployment with multi-IDC.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HDFS-9081) False-positive ACK slow log in DFSClient

2015-10-02 Thread He Tianyi (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-9081?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14942008#comment-14942008
 ] 

He Tianyi commented on HDFS-9081:
-

I think code below in {{waitForAckedSeqno}} is already a good indicator of slow 
acks:
{noformat}
  long duration = Time.monotonicNow() - begin;
  if (duration > dfsclientSlowLogThresholdMs) {
LOG.warn("Slow waitForAckedSeqno took " + duration
+ "ms (threshold=" + dfsclientSlowLogThresholdMs + "ms)");
  }
{noformat}

While slow log in {{ResponseProcessor}} indicates either slow data producer or 
slow ack. I'd suggest more informative slow log.

> False-positive ACK slow log in DFSClient
> 
>
> Key: HDFS-9081
> URL: https://issues.apache.org/jira/browse/HDFS-9081
> Project: Hadoop HDFS
>  Issue Type: Bug
>Affects Versions: 2.6.0
>Reporter: He Tianyi
>Assignee: He Tianyi
>Priority: Minor
>
> This issue is related with code below:
> {noformat}
> if (duration > dfsclientSlowLogThresholdMs
> && ack.getSeqno() != Packet.HEART_BEAT_SEQNO) {
>   DFSClient.LOG
>   .warn("Slow ReadProcessor read fields took " + duration
>   + "ms (threshold=" + dfsclientSlowLogThresholdMs + "ms); ack: "
>   + ack + ", targets: " + Arrays.asList(targets));
> } else if (DFSClient.LOG.isDebugEnabled()) {
>   DFSClient.LOG.debug("DFSClient " + ack);
> }
> {noformat}
> DFSClient prints slow log when awaited after unexpected amount of time 
> (usually 3 ms). This is a good indicator for network or I/O performance 
> issue.
> However, there is scenario that this slow log is false-positive, i.e. a 
> reducer, (StageA) iterates over records with identical key, this takes 
> arbitrary amount of time, but generates no output. (StageB) Then, it output 
> arbitrary number of records when meet a different key.
> If one StageA lasts more than 3 ms (as the example above), there will be 
> one or more slow log generated, which is not related to any HDFS performance 
> issue. 
> In general cases, user should not expect this, as they could be misguided.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HDFS-9071) chooseTargets in ReplicationWork may pass incomplete srcPath

2015-10-02 Thread He Tianyi (JIRA)

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

He Tianyi updated HDFS-9071:

Attachment: HDFS-9071.0001.patch

Kick jenkins again.

> chooseTargets in ReplicationWork may pass incomplete srcPath
> 
>
> Key: HDFS-9071
> URL: https://issues.apache.org/jira/browse/HDFS-9071
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: namenode
>Affects Versions: 2.6.0
>Reporter: He Tianyi
>Assignee: He Tianyi
> Attachments: HDFS-9071.0001.patch, HDFS-9071.0001.patch
>
>
> I've observed that chooseTargets in ReplicationWork may pass incomplete 
> srcPath (not starting with '/') to block placement policy.
> It is possible that srcPath is extensively used in custom placement policy. 
> In this case, the incomplete srcPath may further cause AssertionError if try 
> to get INode with it inside placement policy.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HDFS-9149) Consider multi datacenter when sortByDistance

2015-10-02 Thread He Tianyi (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-9149?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14942031#comment-14942031
 ] 

He Tianyi commented on HDFS-9149:
-

Perhaps we can either go explicit approach or implicit one.

Explicit approach suggests we make the notion if IDC explicit, make it a part 
of network location just like {{NodeGroup}}. But that may require to extend 
{{NetworkTopology}} since the default one did not imply anything about IDC, 
thus behavior may become different.
If we go this way, other components may benefit from DC awareness. For example, 
one can add {{IsDCAware}} and {{getDataCenterOfNode}} to {{NetworkTopology}} 
and implement {{NetworkTopologyWithMultiDC}}, then further consider DC to 
achieve better locality in {{BlockPlacementPolicy}}.

On the other hand, implicit approach suggests we better off substituting 
{{getWeight}} with distance function or more complicated weight function. 
Perhaps calculate the number of common ancestors. This does not imply anything 
about IDC, and can be extend to whatever hierarchy necessary.

In cases which reader is not a part of the cluster, we still need to maintain 
the correct network location for potential readers (e.g. maintain a rack table 
for all hosts in every DC, and give default location with only DC judged by ip 
address when dealing with unknown), and the only issue is when location of 
reader is something like {{/DC1}}, however, this won't be an issue since it 
would have same number of ancestors to all datanodes within {{DC1}}, and no 
ancestor to datanodes within other DC, the outcome is still correct.

> Consider multi datacenter when sortByDistance
> -
>
> Key: HDFS-9149
> URL: https://issues.apache.org/jira/browse/HDFS-9149
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: namenode
>Reporter: He Xiaoqiao
>Assignee: He Tianyi
>
> {{sortByDistance}} doesn't consider multi-datacenter when read data, so there 
> my be reading data via other datacenter when hadoop deployment with multi-IDC.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HDFS-9149) Consider multi datacenter when sortByDistance

2015-09-27 Thread He Tianyi (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-9149?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14909740#comment-14909740
 ] 

He Tianyi commented on HDFS-9149:
-

The default {{NetworkTopology}} is not dc aware. Perhaps implement a custom one 
with datacenter location awareness and override {{sortByDistance}}?

> Consider multi datacenter when sortByDistance
> -
>
> Key: HDFS-9149
> URL: https://issues.apache.org/jira/browse/HDFS-9149
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: namenode
>Reporter: He Xiaoqiao
>
> {{sortByDistance}} doesn't consider multi-datacenter when read data, so there 
> my be reading data via other datacenter when hadoop deployment with multi-IDC.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HDFS-9071) chooseTargets in ReplicationWork may pass incomplete srcPath

2015-09-23 Thread He Tianyi (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-9071?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14905613#comment-14905613
 ] 

He Tianyi commented on HDFS-9071:
-

Thanks for your time, [~andreina].

I think there may be compatibility issue if {{chooseTarget}} get skipped if 
srcPath is invalid, e.g. there are tests that would fail. 


> chooseTargets in ReplicationWork may pass incomplete srcPath
> 
>
> Key: HDFS-9071
> URL: https://issues.apache.org/jira/browse/HDFS-9071
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: namenode
>Affects Versions: 2.6.0
>Reporter: He Tianyi
>Assignee: He Tianyi
> Attachments: HDFS-9071.0001.patch
>
>
> I've observed that chooseTargets in ReplicationWork may pass incomplete 
> srcPath (not starting with '/') to block placement policy.
> It is possible that srcPath is extensively used in custom placement policy. 
> In this case, the incomplete srcPath may further cause AssertionError if try 
> to get INode with it inside placement policy.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HDFS-9071) chooseTargets in ReplicationWork may pass incomplete srcPath

2015-09-21 Thread He Tianyi (JIRA)

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

He Tianyi updated HDFS-9071:

Attachment: HDFS-9071.0001.patch

Adds precondition to make sure {{chooseTargets}} receives either complete path 
or null.

> chooseTargets in ReplicationWork may pass incomplete srcPath
> 
>
> Key: HDFS-9071
> URL: https://issues.apache.org/jira/browse/HDFS-9071
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: namenode
>Affects Versions: 2.6.0
>Reporter: He Tianyi
>Assignee: He Tianyi
> Attachments: HDFS-9071.0001.patch
>
>
> I've observed that chooseTargets in ReplicationWork may pass incomplete 
> srcPath (not starting with '/') to block placement policy.
> It is possible that srcPath is extensively used in custom placement policy. 
> In this case, the incomplete srcPath may further cause AssertionError if try 
> to get INode with it inside placement policy.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HDFS-9071) chooseTargets in ReplicationWork may pass incomplete srcPath

2015-09-21 Thread He Tianyi (JIRA)

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

He Tianyi updated HDFS-9071:

Status: Patch Available  (was: Open)

> chooseTargets in ReplicationWork may pass incomplete srcPath
> 
>
> Key: HDFS-9071
> URL: https://issues.apache.org/jira/browse/HDFS-9071
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: namenode
>Affects Versions: 2.6.0
>Reporter: He Tianyi
>Assignee: He Tianyi
> Attachments: HDFS-9071.0001.patch
>
>
> I've observed that chooseTargets in ReplicationWork may pass incomplete 
> srcPath (not starting with '/') to block placement policy.
> It is possible that srcPath is extensively used in custom placement policy. 
> In this case, the incomplete srcPath may further cause AssertionError if try 
> to get INode with it inside placement policy.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HDFS-9090) Write hot data on few nodes may cause performance issue

2015-09-20 Thread He Tianyi (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-9090?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14900148#comment-14900148
 ] 

He Tianyi commented on HDFS-9090:
-

The combination theory makes sense. Thanks guys.

Since multiple placement policy is not supported yet, I took the approach that 
DFSClient add nodes in local rack to {{excludeNodes}} during calls of 
{{getAdditionalBlock}}. This is ugly but solved the problem right now. 
I'll further wait for either HDFS-4894 or HDFS-7068 implemented, then use the 
custom policy without write locality only for these data.

> Write hot data on few nodes may cause performance issue
> ---
>
> Key: HDFS-9090
> URL: https://issues.apache.org/jira/browse/HDFS-9090
> Project: Hadoop HDFS
>  Issue Type: Improvement
>Affects Versions: 2.3.0
>Reporter: He Tianyi
>Assignee: He Tianyi
>
> (I am not sure whether this should be reported as BUG, feel free to modify 
> this)
> Current block placement policy makes best effort to guarantee first replica 
> on local node whenever possible.
> Consider the following scenario:
> 1. There are 500 datanodes across plenty of racks,
> 2. Raw user action log (just an example) are being written only on 10 nodes, 
> which also have datanode deployed locally,
> 3. Then, before any balance, all these logs will have at least one replica in 
> 10 nodes, implying one thirds data read on these log will be served by these 
> 10 nodes if repl factor is 3, performance suffers.
> I propose to solve this scenario by introducing a configuration entry for 
> client to disable arbitrary level of write locality.
> Then we can either (A) add local nodes to excludedNodes, or (B) tell NameNode 
> the locality we prefer.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (HDFS-9090) Write hot data on few nodes may cause performance issue

2015-09-16 Thread He Tianyi (JIRA)
He Tianyi created HDFS-9090:
---

 Summary: Write hot data on few nodes may cause performance issue
 Key: HDFS-9090
 URL: https://issues.apache.org/jira/browse/HDFS-9090
 Project: Hadoop HDFS
  Issue Type: Bug
Affects Versions: 2.3.0
Reporter: He Tianyi
Assignee: He Tianyi


(I am not sure whether this should be reported as BUG, feel free to modify this)

Current block placement policy makes best effort to guarantee first replica on 
local node whenever possible.

Consider the following scenario:
1. There are 500 datanodes across plenty of racks,
2. Raw user action log (just an example) are being written only on 10 nodes, 
which also have datanode deployed locally,
3. Then, before any balance, all these logs will have at least one replica in 
10 nodes, implying one third data read will be served by these 10 nodes if repl 
factor is 3, performance suffer.

I propose to solve this scenario by introducing a configuration entry for 
client to disable arbitrary level of write locality.
Then we can either (A) add local nodes to excludedNodes, or (B) tell NameNode 
the locality we prefer.




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HDFS-9090) Write hot data on few nodes may cause performance issue

2015-09-16 Thread He Tianyi (JIRA)

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

He Tianyi updated HDFS-9090:

Description: 
(I am not sure whether this should be reported as BUG, feel free to modify this)

Current block placement policy makes best effort to guarantee first replica on 
local node whenever possible.

Consider the following scenario:
1. There are 500 datanodes across plenty of racks,
2. Raw user action log (just an example) are being written only on 10 nodes, 
which also have datanode deployed locally,
3. Then, before any balance, all these logs will have at least one replica in 
10 nodes, implying one thirds data read on these log will be served by these 10 
nodes if repl factor is 3, performance suffers.

I propose to solve this scenario by introducing a configuration entry for 
client to disable arbitrary level of write locality.
Then we can either (A) add local nodes to excludedNodes, or (B) tell NameNode 
the locality we prefer.


  was:
(I am not sure whether this should be reported as BUG, feel free to modify this)

Current block placement policy makes best effort to guarantee first replica on 
local node whenever possible.

Consider the following scenario:
1. There are 500 datanodes across plenty of racks,
2. Raw user action log (just an example) are being written only on 10 nodes, 
which also have datanode deployed locally,
3. Then, before any balance, all these logs will have at least one replica in 
10 nodes, implying one third data read will be served by these 10 nodes if repl 
factor is 3, performance suffer.

I propose to solve this scenario by introducing a configuration entry for 
client to disable arbitrary level of write locality.
Then we can either (A) add local nodes to excludedNodes, or (B) tell NameNode 
the locality we prefer.



> Write hot data on few nodes may cause performance issue
> ---
>
> Key: HDFS-9090
> URL: https://issues.apache.org/jira/browse/HDFS-9090
> Project: Hadoop HDFS
>  Issue Type: Bug
>Affects Versions: 2.3.0
>Reporter: He Tianyi
>Assignee: He Tianyi
>
> (I am not sure whether this should be reported as BUG, feel free to modify 
> this)
> Current block placement policy makes best effort to guarantee first replica 
> on local node whenever possible.
> Consider the following scenario:
> 1. There are 500 datanodes across plenty of racks,
> 2. Raw user action log (just an example) are being written only on 10 nodes, 
> which also have datanode deployed locally,
> 3. Then, before any balance, all these logs will have at least one replica in 
> 10 nodes, implying one thirds data read on these log will be served by these 
> 10 nodes if repl factor is 3, performance suffers.
> I propose to solve this scenario by introducing a configuration entry for 
> client to disable arbitrary level of write locality.
> Then we can either (A) add local nodes to excludedNodes, or (B) tell NameNode 
> the locality we prefer.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HDFS-9090) Write hot data on few nodes may cause performance issue

2015-09-16 Thread He Tianyi (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-9090?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14768973#comment-14768973
 ] 

He Tianyi commented on HDFS-9090:
-

Thanks, [~ste...@apache.org].

My case may be a little rare. Actually these writer nodes have Storm deployed 
and it is storm jobs that feed HDFS with logs.
And due to cost control and budget cycle, it is natural to deploy DataNode on 
every machine that has enough hardware resource.
(Otherwise it would be a waste to keep hard disks of 'ingest nodes' almost 
empty)

IMHO perhaps this could be a common scenario for medium-sized startups.

> Write hot data on few nodes may cause performance issue
> ---
>
> Key: HDFS-9090
> URL: https://issues.apache.org/jira/browse/HDFS-9090
> Project: Hadoop HDFS
>  Issue Type: Improvement
>Affects Versions: 2.3.0
>Reporter: He Tianyi
>Assignee: He Tianyi
>
> (I am not sure whether this should be reported as BUG, feel free to modify 
> this)
> Current block placement policy makes best effort to guarantee first replica 
> on local node whenever possible.
> Consider the following scenario:
> 1. There are 500 datanodes across plenty of racks,
> 2. Raw user action log (just an example) are being written only on 10 nodes, 
> which also have datanode deployed locally,
> 3. Then, before any balance, all these logs will have at least one replica in 
> 10 nodes, implying one thirds data read on these log will be served by these 
> 10 nodes if repl factor is 3, performance suffers.
> I propose to solve this scenario by introducing a configuration entry for 
> client to disable arbitrary level of write locality.
> Then we can either (A) add local nodes to excludedNodes, or (B) tell NameNode 
> the locality we prefer.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HDFS-9090) Write hot data on few nodes may cause performance issue

2015-09-16 Thread He Tianyi (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-9090?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14769041#comment-14769041
 ] 

He Tianyi commented on HDFS-9090:
-

Alternatively, from a different perspective, perhaps we can consider add a 
{{considerLoad}} for getBlockLocations either, which affects sort weight of 
overloaded data nodes. This is similar to {{considerLoad}} during write.

> Write hot data on few nodes may cause performance issue
> ---
>
> Key: HDFS-9090
> URL: https://issues.apache.org/jira/browse/HDFS-9090
> Project: Hadoop HDFS
>  Issue Type: Improvement
>Affects Versions: 2.3.0
>Reporter: He Tianyi
>Assignee: He Tianyi
>
> (I am not sure whether this should be reported as BUG, feel free to modify 
> this)
> Current block placement policy makes best effort to guarantee first replica 
> on local node whenever possible.
> Consider the following scenario:
> 1. There are 500 datanodes across plenty of racks,
> 2. Raw user action log (just an example) are being written only on 10 nodes, 
> which also have datanode deployed locally,
> 3. Then, before any balance, all these logs will have at least one replica in 
> 10 nodes, implying one thirds data read on these log will be served by these 
> 10 nodes if repl factor is 3, performance suffers.
> I propose to solve this scenario by introducing a configuration entry for 
> client to disable arbitrary level of write locality.
> Then we can either (A) add local nodes to excludedNodes, or (B) tell NameNode 
> the locality we prefer.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HDFS-9090) Write hot data on few nodes may cause performance issue

2015-09-16 Thread He Tianyi (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-9090?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14791457#comment-14791457
 ] 

He Tianyi commented on HDFS-9090:
-

Not quite sure but I do think perhaps this is perpendicular with 
{{BlockPlacementPolicy}}.

Assume that HDFS-7068 is implemented. In this case, one can configure 
{{BlockPlacementPolicy}} for specified INode. It is certain that write 
operation under particular directory can be enforced to scatter data across the 
cluster. 
But, given that {{BlockPlacementPolicy}} focuses on where replica should be 
located, each identical policy may differentiate to two different versions 
(with locality, and without).
That is, we have {{BlockPlacementPolicyDefault}}, then perhaps we need a 
{{BlockPlacementPolicyDefaultWithoutWriteLocality}}.
And for a real case, we have {{BlockPlacementPolicyWithMultiDC}}, then perhaps 
we also need a {{BlockPlacementPolicyWithMultiDCWithoutWriteLocality}}.
Let alone the latter one could be implemented by just overriding several 
methods.

Based on that, how about add one parameter, perhaps named "localityLevel" to 
{{chooseTarget}}, then each policy can have their own consideration without 
having the burden of implement two versions?

This could also work when multiple policy is not supported.

> Write hot data on few nodes may cause performance issue
> ---
>
> Key: HDFS-9090
> URL: https://issues.apache.org/jira/browse/HDFS-9090
> Project: Hadoop HDFS
>  Issue Type: Improvement
>Affects Versions: 2.3.0
>Reporter: He Tianyi
>Assignee: He Tianyi
>
> (I am not sure whether this should be reported as BUG, feel free to modify 
> this)
> Current block placement policy makes best effort to guarantee first replica 
> on local node whenever possible.
> Consider the following scenario:
> 1. There are 500 datanodes across plenty of racks,
> 2. Raw user action log (just an example) are being written only on 10 nodes, 
> which also have datanode deployed locally,
> 3. Then, before any balance, all these logs will have at least one replica in 
> 10 nodes, implying one thirds data read on these log will be served by these 
> 10 nodes if repl factor is 3, performance suffers.
> I propose to solve this scenario by introducing a configuration entry for 
> client to disable arbitrary level of write locality.
> Then we can either (A) add local nodes to excludedNodes, or (B) tell NameNode 
> the locality we prefer.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HDFS-9075) Multiple datacenter replication inside one HDFS cluster

2015-09-15 Thread He Tianyi (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-9075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14746675#comment-14746675
 ] 

He Tianyi commented on HDFS-9075:
-

Thanks for point that out, [~cnauroth].

Prior discusses mentioned global namespace model, which i think is the most 
valuable direction to work on.

There are consistency choices about namespace:
1. strong consistent namespace, perhaps either requires a global quorum to 
ensure consistency or namespace segmentation (bit like federation, with only 
local block pool)
2. eventual consistent namespace, can be achieved via snapshots. (What happens

Besides, there are choices about data replication fashion:
1. sync replication, add remote nodes to pipeline during write,
2. async replication.

IMHO strong consistent namespace is a must, otherwise global operations tend to 
be hard to become transparent.
i.e. What happens if append operation on different file (or same file) in same 
directory take place simultaneously in two datacenters?
(Of course a global lease manager would do the trick, but that requires remote 
communication)
If we go the strong consistent way, performance suffers anyway (R/W needs 
global communication anyway). It's no harm simply use one central active 
NameNode, but with JournalNode and standby NameNode deployed globally.

As for replication, I think performance will not be an issue when given latency 
is tolerable and bandwidth is sufficient (See HDFS-8829). We can certainly let 
user decide.

We have a real scenario that communication between two datacenters have a 
latency of nearly 3ms, while bandwidth is sufficient.
In this case, we see no performance drop so far.

But with high latency, I think that will not hold. Perhaps we need some fresh 
idea.

> Multiple datacenter replication inside one HDFS cluster
> ---
>
> Key: HDFS-9075
> URL: https://issues.apache.org/jira/browse/HDFS-9075
> Project: Hadoop HDFS
>  Issue Type: New Feature
>  Components: datanode, namenode
>Reporter: He Tianyi
>Assignee: He Tianyi
>
> It is common scenario of deploying multiple datacenter for scaling and 
> disaster tolerant. 
> In this case we certainly want that data can be shared transparently (to 
> user) across datacenters.
> For example, say we have a raw user action log stored daily, different 
> computations may take place with the log as input. As scale grows, we may 
> want to schedule various kind of computations in more than one datacenter.
> As far as i know, current solution is to deploy multiple independent clusters 
> corresponding to datacenters, using {{distcp}} to sync data files between 
> them.
> But in this case, user needs to know exactly where data is stored, and 
> mistakes may be made during human-intervened operations. After all, it is 
> basically a computer job.
> Based on these facts, it is obvious that a multiple datacenter replication 
> solution may solve the scenario.
> I am working one prototype that works with 2 datacenters, the goal is to 
> provide data replication between datacenters transparently and minimize the 
> inter-dc bandwidth usage. Basic idea is replicate blocks to both DC and 
> determine number of replications by historical statistics of access behaviors 
> of that part of namespace.
> I will post a design document soon.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (HDFS-9081) False-positive of ACK slow log in DFSClient

2015-09-15 Thread He Tianyi (JIRA)
He Tianyi created HDFS-9081:
---

 Summary: False-positive of ACK slow log in DFSClient
 Key: HDFS-9081
 URL: https://issues.apache.org/jira/browse/HDFS-9081
 Project: Hadoop HDFS
  Issue Type: Bug
Affects Versions: 2.6.0
Reporter: He Tianyi
Assignee: He Tianyi
Priority: Minor


This issue is related with code below:
{noformat}
if (duration > dfsclientSlowLogThresholdMs
&& ack.getSeqno() != Packet.HEART_BEAT_SEQNO) {
  DFSClient.LOG
  .warn("Slow ReadProcessor read fields took " + duration
  + "ms (threshold=" + dfsclientSlowLogThresholdMs + "ms); ack: "
  + ack + ", targets: " + Arrays.asList(targets));
} else if (DFSClient.LOG.isDebugEnabled()) {
  DFSClient.LOG.debug("DFSClient " + ack);
}
{noformat}

DFSClient prints slow log when awaited after unexpected amount of time (usually 
3 ms). This is a good indicator for network or I/O performance issue.

However, there is scenario that this slow log is false-positive, i.e. a 
reducer, (StageA) iterates over records with identical key, this takes 
arbitrary amount of time, but generates no output. (StageB) Then, it output 
arbitrary number of records when meet a different key.

If one StageA lasts more than 3 ms (as the example above), there will be 
one or more slow log generated, which is not related to any HDFS performance 
issue. 
In general cases, user should not expect this, as they could be misguided.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HDFS-9081) False-positive ACK slow log in DFSClient

2015-09-15 Thread He Tianyi (JIRA)

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

He Tianyi updated HDFS-9081:

Summary: False-positive ACK slow log in DFSClient  (was: False-positive of 
ACK slow log in DFSClient)

> False-positive ACK slow log in DFSClient
> 
>
> Key: HDFS-9081
> URL: https://issues.apache.org/jira/browse/HDFS-9081
> Project: Hadoop HDFS
>  Issue Type: Bug
>Affects Versions: 2.6.0
>Reporter: He Tianyi
>Assignee: He Tianyi
>Priority: Minor
>
> This issue is related with code below:
> {noformat}
> if (duration > dfsclientSlowLogThresholdMs
> && ack.getSeqno() != Packet.HEART_BEAT_SEQNO) {
>   DFSClient.LOG
>   .warn("Slow ReadProcessor read fields took " + duration
>   + "ms (threshold=" + dfsclientSlowLogThresholdMs + "ms); ack: "
>   + ack + ", targets: " + Arrays.asList(targets));
> } else if (DFSClient.LOG.isDebugEnabled()) {
>   DFSClient.LOG.debug("DFSClient " + ack);
> }
> {noformat}
> DFSClient prints slow log when awaited after unexpected amount of time 
> (usually 3 ms). This is a good indicator for network or I/O performance 
> issue.
> However, there is scenario that this slow log is false-positive, i.e. a 
> reducer, (StageA) iterates over records with identical key, this takes 
> arbitrary amount of time, but generates no output. (StageB) Then, it output 
> arbitrary number of records when meet a different key.
> If one StageA lasts more than 3 ms (as the example above), there will be 
> one or more slow log generated, which is not related to any HDFS performance 
> issue. 
> In general cases, user should not expect this, as they could be misguided.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HDFS-8829) DataNode sets SO_RCVBUF explicitly is disabling tcp auto-tuning

2015-09-14 Thread He Tianyi (JIRA)

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

He Tianyi updated HDFS-8829:

Attachment: HDFS-8829.0006.patch

Rebased again current trunk.

> DataNode sets SO_RCVBUF explicitly is disabling tcp auto-tuning
> ---
>
> Key: HDFS-8829
> URL: https://issues.apache.org/jira/browse/HDFS-8829
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: datanode
>Affects Versions: 2.3.0, 2.6.0
>Reporter: He Tianyi
>Assignee: He Tianyi
> Attachments: HDFS-8829.0001.patch, HDFS-8829.0002.patch, 
> HDFS-8829.0003.patch, HDFS-8829.0004.patch, HDFS-8829.0005.patch, 
> HDFS-8829.0006.patch
>
>
> {code:java}
>   private void initDataXceiver(Configuration conf) throws IOException {
> // find free port or use privileged port provided
> TcpPeerServer tcpPeerServer;
> if (secureResources != null) {
>   tcpPeerServer = new TcpPeerServer(secureResources);
> } else {
>   tcpPeerServer = new TcpPeerServer(dnConf.socketWriteTimeout,
>   DataNode.getStreamingAddr(conf));
> }
> 
> tcpPeerServer.setReceiveBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
> {code}
> The last line sets SO_RCVBUF explicitly, thus disabling tcp auto-tuning on 
> some system.
> Shall we make this behavior configurable?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (HDFS-9075) Multiple datacenter replication inside one HDFS cluster

2015-09-14 Thread He Tianyi (JIRA)
He Tianyi created HDFS-9075:
---

 Summary: Multiple datacenter replication inside one HDFS cluster
 Key: HDFS-9075
 URL: https://issues.apache.org/jira/browse/HDFS-9075
 Project: Hadoop HDFS
  Issue Type: New Feature
  Components: datanode, namenode
Reporter: He Tianyi
Assignee: He Tianyi


It is common scenario for deploying multiple datacenter for scaling and 
disaster tolerant. 
In this case we certainly want that data can be shared transparently (to user) 
across datacenters.

For example, say we have a raw user action log stored daily, different 
computations may take place with the log as input. As scale grows, we may want 
to schedule various kind of computations in more than one datacenter.

As far as i know, current solution is to deploy multiple clusters corresponding 
to datacenters, using {{distcp}} to sync data between them.
But in this case, user needs to know exactly where data is stored, and mistakes 
may be made during human-intervened operations. After all, it is basically a 
computer job.

Based on these facts, it is obvious that a multiple datacenter replication 
solution may solve the scenario.

I am working one prototype that works with 2 datacenters, the goal is to 
provide data replication between datacenters transparently and minimize the 
inter-dc bandwidth usage. Basic idea is replicate blocks to both DC and 
determine number of replications by historical statistics of access behaviors 
of that part of namespace.

I will post a design document soon.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HDFS-9075) Multiple datacenter replication inside one HDFS cluster

2015-09-14 Thread He Tianyi (JIRA)

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

He Tianyi updated HDFS-9075:

Description: 
It is common scenario of deploying multiple datacenter for scaling and disaster 
tolerant. 
In this case we certainly want that data can be shared transparently (to user) 
across datacenters.

For example, say we have a raw user action log stored daily, different 
computations may take place with the log as input. As scale grows, we may want 
to schedule various kind of computations in more than one datacenter.

As far as i know, current solution is to deploy multiple independent clusters 
corresponding to datacenters, using {{distcp}} to sync data files between them.
But in this case, user needs to know exactly where data is stored, and mistakes 
may be made during human-intervened operations. After all, it is basically a 
computer job.

Based on these facts, it is obvious that a multiple datacenter replication 
solution may solve the scenario.

I am working one prototype that works with 2 datacenters, the goal is to 
provide data replication between datacenters transparently and minimize the 
inter-dc bandwidth usage. Basic idea is replicate blocks to both DC and 
determine number of replications by historical statistics of access behaviors 
of that part of namespace.

I will post a design document soon.

  was:
It is common scenario for deploying multiple datacenter for scaling and 
disaster tolerant. 
In this case we certainly want that data can be shared transparently (to user) 
across datacenters.

For example, say we have a raw user action log stored daily, different 
computations may take place with the log as input. As scale grows, we may want 
to schedule various kind of computations in more than one datacenter.

As far as i know, current solution is to deploy multiple clusters corresponding 
to datacenters, using {{distcp}} to sync data between them.
But in this case, user needs to know exactly where data is stored, and mistakes 
may be made during human-intervened operations. After all, it is basically a 
computer job.

Based on these facts, it is obvious that a multiple datacenter replication 
solution may solve the scenario.

I am working one prototype that works with 2 datacenters, the goal is to 
provide data replication between datacenters transparently and minimize the 
inter-dc bandwidth usage. Basic idea is replicate blocks to both DC and 
determine number of replications by historical statistics of access behaviors 
of that part of namespace.

I will post a design document soon.


> Multiple datacenter replication inside one HDFS cluster
> ---
>
> Key: HDFS-9075
> URL: https://issues.apache.org/jira/browse/HDFS-9075
> Project: Hadoop HDFS
>  Issue Type: New Feature
>  Components: datanode, namenode
>Reporter: He Tianyi
>Assignee: He Tianyi
>
> It is common scenario of deploying multiple datacenter for scaling and 
> disaster tolerant. 
> In this case we certainly want that data can be shared transparently (to 
> user) across datacenters.
> For example, say we have a raw user action log stored daily, different 
> computations may take place with the log as input. As scale grows, we may 
> want to schedule various kind of computations in more than one datacenter.
> As far as i know, current solution is to deploy multiple independent clusters 
> corresponding to datacenters, using {{distcp}} to sync data files between 
> them.
> But in this case, user needs to know exactly where data is stored, and 
> mistakes may be made during human-intervened operations. After all, it is 
> basically a computer job.
> Based on these facts, it is obvious that a multiple datacenter replication 
> solution may solve the scenario.
> I am working one prototype that works with 2 datacenters, the goal is to 
> provide data replication between datacenters transparently and minimize the 
> inter-dc bandwidth usage. Basic idea is replicate blocks to both DC and 
> determine number of replications by historical statistics of access behaviors 
> of that part of namespace.
> I will post a design document soon.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (HDFS-9071) chooseTargets in ReplicationWork may pass incomplete srcPath

2015-09-13 Thread He Tianyi (JIRA)
He Tianyi created HDFS-9071:
---

 Summary: chooseTargets in ReplicationWork may pass incomplete 
srcPath
 Key: HDFS-9071
 URL: https://issues.apache.org/jira/browse/HDFS-9071
 Project: Hadoop HDFS
  Issue Type: Bug
  Components: namenode
Affects Versions: 2.6.0
Reporter: He Tianyi
Assignee: He Tianyi


I've observed that chooseTargets in ReplicationWork may pass incomplete srcPath 
(not starting with '/') to block placement policy.

It is possible that srcPath is extensively used in custom placement policy. In 
this case, the incomplete srcPath may further cause AssertionError if try to 
get INode with it inside placement policy.




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HDFS-8829) DataNode sets SO_RCVBUF explicitly is disabling tcp auto-tuning

2015-09-13 Thread He Tianyi (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-8829?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14742846#comment-14742846
 ] 

He Tianyi commented on HDFS-8829:
-

* "I've" -> "I'll"

> DataNode sets SO_RCVBUF explicitly is disabling tcp auto-tuning
> ---
>
> Key: HDFS-8829
> URL: https://issues.apache.org/jira/browse/HDFS-8829
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: datanode
>Affects Versions: 2.3.0, 2.6.0
>Reporter: He Tianyi
>Assignee: He Tianyi
> Attachments: HDFS-8829.0001.patch, HDFS-8829.0002.patch, 
> HDFS-8829.0003.patch, HDFS-8829.0004.patch, HDFS-8829.0005.patch
>
>
> {code:java}
>   private void initDataXceiver(Configuration conf) throws IOException {
> // find free port or use privileged port provided
> TcpPeerServer tcpPeerServer;
> if (secureResources != null) {
>   tcpPeerServer = new TcpPeerServer(secureResources);
> } else {
>   tcpPeerServer = new TcpPeerServer(dnConf.socketWriteTimeout,
>   DataNode.getStreamingAddr(conf));
> }
> 
> tcpPeerServer.setReceiveBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
> {code}
> The last line sets SO_RCVBUF explicitly, thus disabling tcp auto-tuning on 
> some system.
> Shall we make this behavior configurable?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HDFS-8829) DataNode sets SO_RCVBUF explicitly is disabling tcp auto-tuning

2015-09-13 Thread He Tianyi (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-8829?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14742844#comment-14742844
 ] 

He Tianyi commented on HDFS-8829:
-

Hi, thanks for the comments.

I've update the patch against current trunk in few days.

> DataNode sets SO_RCVBUF explicitly is disabling tcp auto-tuning
> ---
>
> Key: HDFS-8829
> URL: https://issues.apache.org/jira/browse/HDFS-8829
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: datanode
>Affects Versions: 2.3.0, 2.6.0
>Reporter: He Tianyi
>Assignee: He Tianyi
> Attachments: HDFS-8829.0001.patch, HDFS-8829.0002.patch, 
> HDFS-8829.0003.patch, HDFS-8829.0004.patch, HDFS-8829.0005.patch
>
>
> {code:java}
>   private void initDataXceiver(Configuration conf) throws IOException {
> // find free port or use privileged port provided
> TcpPeerServer tcpPeerServer;
> if (secureResources != null) {
>   tcpPeerServer = new TcpPeerServer(secureResources);
> } else {
>   tcpPeerServer = new TcpPeerServer(dnConf.socketWriteTimeout,
>   DataNode.getStreamingAddr(conf));
> }
> 
> tcpPeerServer.setReceiveBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
> {code}
> The last line sets SO_RCVBUF explicitly, thus disabling tcp auto-tuning on 
> some system.
> Shall we make this behavior configurable?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HDFS-8829) DataNode sets SO_RCVBUF explicitly is disabling tcp auto-tuning

2015-09-11 Thread He Tianyi (JIRA)

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

He Tianyi updated HDFS-8829:

Attachment: HDFS-8829.0005.patch

Fix codestyle.

> DataNode sets SO_RCVBUF explicitly is disabling tcp auto-tuning
> ---
>
> Key: HDFS-8829
> URL: https://issues.apache.org/jira/browse/HDFS-8829
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: datanode
>Affects Versions: 2.3.0, 2.6.0
>Reporter: He Tianyi
>Assignee: He Tianyi
> Attachments: HDFS-8829.0001.patch, HDFS-8829.0002.patch, 
> HDFS-8829.0003.patch, HDFS-8829.0004.patch, HDFS-8829.0005.patch
>
>
> {code:java}
>   private void initDataXceiver(Configuration conf) throws IOException {
> // find free port or use privileged port provided
> TcpPeerServer tcpPeerServer;
> if (secureResources != null) {
>   tcpPeerServer = new TcpPeerServer(secureResources);
> } else {
>   tcpPeerServer = new TcpPeerServer(dnConf.socketWriteTimeout,
>   DataNode.getStreamingAddr(conf));
> }
> 
> tcpPeerServer.setReceiveBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
> {code}
> The last line sets SO_RCVBUF explicitly, thus disabling tcp auto-tuning on 
> some system.
> Shall we make this behavior configurable?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HDFS-8829) DataNode sets SO_RCVBUF explicitly is disabling tcp auto-tuning

2015-09-05 Thread He Tianyi (JIRA)

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

He Tianyi updated HDFS-8829:

Attachment: HDFS-8829.0004.patch

Hi, sorry for late response.

Agree with you, [~cmccabe].
I've updated patch.

> DataNode sets SO_RCVBUF explicitly is disabling tcp auto-tuning
> ---
>
> Key: HDFS-8829
> URL: https://issues.apache.org/jira/browse/HDFS-8829
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: datanode
>Affects Versions: 2.3.0, 2.6.0
>Reporter: He Tianyi
>Assignee: He Tianyi
> Attachments: HDFS-8829.0001.patch, HDFS-8829.0002.patch, 
> HDFS-8829.0003.patch, HDFS-8829.0004.patch
>
>
> {code:java}
>   private void initDataXceiver(Configuration conf) throws IOException {
> // find free port or use privileged port provided
> TcpPeerServer tcpPeerServer;
> if (secureResources != null) {
>   tcpPeerServer = new TcpPeerServer(secureResources);
> } else {
>   tcpPeerServer = new TcpPeerServer(dnConf.socketWriteTimeout,
>   DataNode.getStreamingAddr(conf));
> }
> 
> tcpPeerServer.setReceiveBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
> {code}
> The last line sets SO_RCVBUF explicitly, thus disabling tcp auto-tuning on 
> some system.
> Shall we make this behavior configurable?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HDFS-8829) DataNode sets SO_RCVBUF explicitly is disabling tcp auto-tuning

2015-08-27 Thread He Tianyi (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-8829?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14716471#comment-14716471
 ] 

He Tianyi commented on HDFS-8829:
-

Is there anything wrong with CI?
I've run {{test-patch.sh}} on my machine and it succeed.

 DataNode sets SO_RCVBUF explicitly is disabling tcp auto-tuning
 ---

 Key: HDFS-8829
 URL: https://issues.apache.org/jira/browse/HDFS-8829
 Project: Hadoop HDFS
  Issue Type: Improvement
  Components: datanode
Affects Versions: 2.3.0, 2.6.0
Reporter: He Tianyi
Assignee: He Tianyi
 Attachments: HDFS-8829.0001.patch, HDFS-8829.0002.patch, 
 HDFS-8829.0003.patch


 {code:java}
   private void initDataXceiver(Configuration conf) throws IOException {
 // find free port or use privileged port provided
 TcpPeerServer tcpPeerServer;
 if (secureResources != null) {
   tcpPeerServer = new TcpPeerServer(secureResources);
 } else {
   tcpPeerServer = new TcpPeerServer(dnConf.socketWriteTimeout,
   DataNode.getStreamingAddr(conf));
 }
 
 tcpPeerServer.setReceiveBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
 {code}
 The last line sets SO_RCVBUF explicitly, thus disabling tcp auto-tuning on 
 some system.
 Shall we make this behavior configurable?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HDFS-8829) DataNode sets SO_RCVBUF explicitly is disabling tcp auto-tuning

2015-08-26 Thread He Tianyi (JIRA)

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

He Tianyi updated HDFS-8829:

Attachment: HDFS-8829.0003.patch

Yes, right.
Thanks for comments.

I've updated the patch.

 DataNode sets SO_RCVBUF explicitly is disabling tcp auto-tuning
 ---

 Key: HDFS-8829
 URL: https://issues.apache.org/jira/browse/HDFS-8829
 Project: Hadoop HDFS
  Issue Type: Improvement
  Components: datanode
Affects Versions: 2.3.0, 2.6.0
Reporter: He Tianyi
Assignee: He Tianyi
 Attachments: HDFS-8829.0001.patch, HDFS-8829.0002.patch, 
 HDFS-8829.0003.patch


 {code:java}
   private void initDataXceiver(Configuration conf) throws IOException {
 // find free port or use privileged port provided
 TcpPeerServer tcpPeerServer;
 if (secureResources != null) {
   tcpPeerServer = new TcpPeerServer(secureResources);
 } else {
   tcpPeerServer = new TcpPeerServer(dnConf.socketWriteTimeout,
   DataNode.getStreamingAddr(conf));
 }
 
 tcpPeerServer.setReceiveBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
 {code}
 The last line sets SO_RCVBUF explicitly, thus disabling tcp auto-tuning on 
 some system.
 Shall we make this behavior configurable?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HDFS-8829) DataNode sets SO_RCVBUF explicitly is disabling tcp auto-tuning

2015-08-24 Thread He Tianyi (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-8829?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14710434#comment-14710434
 ] 

He Tianyi commented on HDFS-8829:
-

Hi [~cmccabe],

About the second comment:
Actually {{DEFAULT_DATA_SOCKET_SIZE}} used to act as both receive buffer size 
and send buffer size. Therefore it is not exactly send buffer size.
Besides, I observed that it is receive buffer size that has tremendous impact 
on performance in this case.



 DataNode sets SO_RCVBUF explicitly is disabling tcp auto-tuning
 ---

 Key: HDFS-8829
 URL: https://issues.apache.org/jira/browse/HDFS-8829
 Project: Hadoop HDFS
  Issue Type: Improvement
  Components: datanode
Affects Versions: 2.3.0, 2.6.0
Reporter: He Tianyi
Assignee: He Tianyi
 Attachments: HDFS-8829.0001.patch, HDFS-8829.0002.patch


 {code:java}
   private void initDataXceiver(Configuration conf) throws IOException {
 // find free port or use privileged port provided
 TcpPeerServer tcpPeerServer;
 if (secureResources != null) {
   tcpPeerServer = new TcpPeerServer(secureResources);
 } else {
   tcpPeerServer = new TcpPeerServer(dnConf.socketWriteTimeout,
   DataNode.getStreamingAddr(conf));
 }
 
 tcpPeerServer.setReceiveBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
 {code}
 The last line sets SO_RCVBUF explicitly, thus disabling tcp auto-tuning on 
 some system.
 Shall we make this behavior configurable?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HDFS-8829) DataNode sets SO_RCVBUF explicitly is disabling tcp auto-tuning

2015-08-22 Thread He Tianyi (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-8829?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14708270#comment-14708270
 ] 

He Tianyi commented on HDFS-8829:
-

I am confused with this checkstyle warning:
{{./hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java:74:13:
 Variable 'dataSocketSize' must be private and have accessor methods.}}

Should I follow contemporary convention (since most other fields are also 
declared package-private and has no accessor method) or style guideline?

 DataNode sets SO_RCVBUF explicitly is disabling tcp auto-tuning
 ---

 Key: HDFS-8829
 URL: https://issues.apache.org/jira/browse/HDFS-8829
 Project: Hadoop HDFS
  Issue Type: Improvement
  Components: datanode
Affects Versions: 2.3.0, 2.6.0
Reporter: He Tianyi
Assignee: He Tianyi
 Attachments: HDFS-8829.0001.patch, HDFS-8829.0002.patch


 {code:java}
   private void initDataXceiver(Configuration conf) throws IOException {
 // find free port or use privileged port provided
 TcpPeerServer tcpPeerServer;
 if (secureResources != null) {
   tcpPeerServer = new TcpPeerServer(secureResources);
 } else {
   tcpPeerServer = new TcpPeerServer(dnConf.socketWriteTimeout,
   DataNode.getStreamingAddr(conf));
 }
 
 tcpPeerServer.setReceiveBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
 {code}
 The last line sets SO_RCVBUF explicitly, thus disabling tcp auto-tuning on 
 some system.
 Shall we make this behavior configurable?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HDFS-8829) DataNode sets SO_RCVBUF explicitly is disabling tcp auto-tuning

2015-08-21 Thread He Tianyi (JIRA)

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

He Tianyi updated HDFS-8829:

Attachment: HDFS-8829.0002.patch

Added configuration to hdfs-default.xml

 DataNode sets SO_RCVBUF explicitly is disabling tcp auto-tuning
 ---

 Key: HDFS-8829
 URL: https://issues.apache.org/jira/browse/HDFS-8829
 Project: Hadoop HDFS
  Issue Type: Improvement
  Components: datanode
Affects Versions: 2.3.0, 2.6.0
Reporter: He Tianyi
Assignee: He Tianyi
 Attachments: HDFS-8829.0001.patch, HDFS-8829.0002.patch


 {code:java}
   private void initDataXceiver(Configuration conf) throws IOException {
 // find free port or use privileged port provided
 TcpPeerServer tcpPeerServer;
 if (secureResources != null) {
   tcpPeerServer = new TcpPeerServer(secureResources);
 } else {
   tcpPeerServer = new TcpPeerServer(dnConf.socketWriteTimeout,
   DataNode.getStreamingAddr(conf));
 }
 
 tcpPeerServer.setReceiveBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
 {code}
 The last line sets SO_RCVBUF explicitly, thus disabling tcp auto-tuning on 
 some system.
 Shall we make this behavior configurable?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Assigned] (HDFS-8829) DataNode sets SO_RCVBUF explicitly is disabling tcp auto-tuning

2015-08-20 Thread He Tianyi (JIRA)

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

He Tianyi reassigned HDFS-8829:
---

Assignee: He Tianyi  (was: kanaka kumar avvaru)

 DataNode sets SO_RCVBUF explicitly is disabling tcp auto-tuning
 ---

 Key: HDFS-8829
 URL: https://issues.apache.org/jira/browse/HDFS-8829
 Project: Hadoop HDFS
  Issue Type: Improvement
  Components: datanode
Affects Versions: 2.3.0, 2.6.0
Reporter: He Tianyi
Assignee: He Tianyi

 {code:java}
   private void initDataXceiver(Configuration conf) throws IOException {
 // find free port or use privileged port provided
 TcpPeerServer tcpPeerServer;
 if (secureResources != null) {
   tcpPeerServer = new TcpPeerServer(secureResources);
 } else {
   tcpPeerServer = new TcpPeerServer(dnConf.socketWriteTimeout,
   DataNode.getStreamingAddr(conf));
 }
 
 tcpPeerServer.setReceiveBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
 {code}
 The last line sets SO_RCVBUF explicitly, thus disabling tcp auto-tuning on 
 some system.
 Shall we make this behavior configurable?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HDFS-8829) DataNode sets SO_RCVBUF explicitly is disabling tcp auto-tuning

2015-08-20 Thread He Tianyi (JIRA)

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

He Tianyi updated HDFS-8829:

Attachment: HDFS-8829.0001.patch

Added *dfs.data.socket.size*, with default value 128KB.
When set to 0 or negative value, enable auto-tuning.

 DataNode sets SO_RCVBUF explicitly is disabling tcp auto-tuning
 ---

 Key: HDFS-8829
 URL: https://issues.apache.org/jira/browse/HDFS-8829
 Project: Hadoop HDFS
  Issue Type: Improvement
  Components: datanode
Affects Versions: 2.3.0, 2.6.0
Reporter: He Tianyi
Assignee: He Tianyi
 Attachments: HDFS-8829.0001.patch


 {code:java}
   private void initDataXceiver(Configuration conf) throws IOException {
 // find free port or use privileged port provided
 TcpPeerServer tcpPeerServer;
 if (secureResources != null) {
   tcpPeerServer = new TcpPeerServer(secureResources);
 } else {
   tcpPeerServer = new TcpPeerServer(dnConf.socketWriteTimeout,
   DataNode.getStreamingAddr(conf));
 }
 
 tcpPeerServer.setReceiveBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
 {code}
 The last line sets SO_RCVBUF explicitly, thus disabling tcp auto-tuning on 
 some system.
 Shall we make this behavior configurable?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HDFS-8829) DataNode sets SO_RCVBUF explicitly is disabling tcp auto-tuning

2015-08-20 Thread He Tianyi (JIRA)

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

He Tianyi updated HDFS-8829:

Status: Patch Available  (was: Open)

Added *dfs.data.socket.size*, with default value 128KB.
When set to 0 or negative value, enable auto-tuning.

 DataNode sets SO_RCVBUF explicitly is disabling tcp auto-tuning
 ---

 Key: HDFS-8829
 URL: https://issues.apache.org/jira/browse/HDFS-8829
 Project: Hadoop HDFS
  Issue Type: Improvement
  Components: datanode
Affects Versions: 2.6.0, 2.3.0
Reporter: He Tianyi
Assignee: He Tianyi
 Attachments: HDFS-8829.0001.patch


 {code:java}
   private void initDataXceiver(Configuration conf) throws IOException {
 // find free port or use privileged port provided
 TcpPeerServer tcpPeerServer;
 if (secureResources != null) {
   tcpPeerServer = new TcpPeerServer(secureResources);
 } else {
   tcpPeerServer = new TcpPeerServer(dnConf.socketWriteTimeout,
   DataNode.getStreamingAddr(conf));
 }
 
 tcpPeerServer.setReceiveBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
 {code}
 The last line sets SO_RCVBUF explicitly, thus disabling tcp auto-tuning on 
 some system.
 Shall we make this behavior configurable?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HDFS-8829) DataNode sets SO_RCVBUF explicitly is disabling tcp auto-tuning

2015-08-19 Thread He Tianyi (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-8829?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14704266#comment-14704266
 ] 

He Tianyi commented on HDFS-8829:
-

Completely agree with you, [~cmccabe].

 DataNode sets SO_RCVBUF explicitly is disabling tcp auto-tuning
 ---

 Key: HDFS-8829
 URL: https://issues.apache.org/jira/browse/HDFS-8829
 Project: Hadoop HDFS
  Issue Type: Improvement
  Components: datanode
Affects Versions: 2.3.0, 2.6.0
Reporter: He Tianyi
Assignee: kanaka kumar avvaru

 {code:java}
   private void initDataXceiver(Configuration conf) throws IOException {
 // find free port or use privileged port provided
 TcpPeerServer tcpPeerServer;
 if (secureResources != null) {
   tcpPeerServer = new TcpPeerServer(secureResources);
 } else {
   tcpPeerServer = new TcpPeerServer(dnConf.socketWriteTimeout,
   DataNode.getStreamingAddr(conf));
 }
 
 tcpPeerServer.setReceiveBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
 {code}
 The last line sets SO_RCVBUF explicitly, thus disabling tcp auto-tuning on 
 some system.
 Shall we make this behavior configurable?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HDFS-8829) DataNode sets SO_RCVBUF explicitly is disabling tcp auto-tuning

2015-08-18 Thread He Tianyi (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-8829?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14702365#comment-14702365
 ] 

He Tianyi commented on HDFS-8829:
-

I think auto-tuning does the magic when network latency is relatively high 
(1ms), and has no effect when latency is low ( 0.1ms). But in standard 
cluster, latency won't be that high. 
So enabling auto-tuning by default would probably brings no performance 
gaining. While setting default value to 128 * 1024 will be compatible with 
current behavior.


 DataNode sets SO_RCVBUF explicitly is disabling tcp auto-tuning
 ---

 Key: HDFS-8829
 URL: https://issues.apache.org/jira/browse/HDFS-8829
 Project: Hadoop HDFS
  Issue Type: Improvement
  Components: datanode
Affects Versions: 2.3.0, 2.6.0
Reporter: He Tianyi
Assignee: kanaka kumar avvaru

 {code:java}
   private void initDataXceiver(Configuration conf) throws IOException {
 // find free port or use privileged port provided
 TcpPeerServer tcpPeerServer;
 if (secureResources != null) {
   tcpPeerServer = new TcpPeerServer(secureResources);
 } else {
   tcpPeerServer = new TcpPeerServer(dnConf.socketWriteTimeout,
   DataNode.getStreamingAddr(conf));
 }
 
 tcpPeerServer.setReceiveBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
 {code}
 The last line sets SO_RCVBUF explicitly, thus disabling tcp auto-tuning on 
 some system.
 Shall we make this behavior configurable?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HDFS-8829) DataNode sets SO_RCVBUF explicitly is disabling tcp auto-tuning

2015-07-31 Thread He Tianyi (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-8829?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14650141#comment-14650141
 ] 

He Tianyi commented on HDFS-8829:
-

Hi kanaka kumar avvaru,

I've applied the improvement to my cluster, and should be able to manage to 
produce a patch in next few days.
Can I work on this?

 DataNode sets SO_RCVBUF explicitly is disabling tcp auto-tuning
 ---

 Key: HDFS-8829
 URL: https://issues.apache.org/jira/browse/HDFS-8829
 Project: Hadoop HDFS
  Issue Type: Improvement
  Components: datanode
Affects Versions: 2.3.0, 2.6.0
Reporter: He Tianyi
Assignee: kanaka kumar avvaru

 {code:java}
   private void initDataXceiver(Configuration conf) throws IOException {
 // find free port or use privileged port provided
 TcpPeerServer tcpPeerServer;
 if (secureResources != null) {
   tcpPeerServer = new TcpPeerServer(secureResources);
 } else {
   tcpPeerServer = new TcpPeerServer(dnConf.socketWriteTimeout,
   DataNode.getStreamingAddr(conf));
 }
 
 tcpPeerServer.setReceiveBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
 {code}
 The last line sets SO_RCVBUF explicitly, thus disabling tcp auto-tuning on 
 some system.
 Shall we make this behavior configurable?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (HDFS-8829) DataNode sets SO_RCVBUF explicitly is disabling tcp auto-tuning

2015-07-28 Thread He Tianyi (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-8829?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14645305#comment-14645305
 ] 

He Tianyi commented on HDFS-8829:
-

This affects pipeline throughput, and I've observed a 30% performance gain with 
tcp auto-tuning enabled. 
(since 128KB window size is not always optimal)

Could be particularly useful with SSD drives (where disk throughput may be 
greater than net throughput).


 DataNode sets SO_RCVBUF explicitly is disabling tcp auto-tuning
 ---

 Key: HDFS-8829
 URL: https://issues.apache.org/jira/browse/HDFS-8829
 Project: Hadoop HDFS
  Issue Type: Improvement
  Components: datanode
Affects Versions: 2.3.0, 2.6.0
Reporter: He Tianyi
Assignee: kanaka kumar avvaru

 {code:java}
   private void initDataXceiver(Configuration conf) throws IOException {
 // find free port or use privileged port provided
 TcpPeerServer tcpPeerServer;
 if (secureResources != null) {
   tcpPeerServer = new TcpPeerServer(secureResources);
 } else {
   tcpPeerServer = new TcpPeerServer(dnConf.socketWriteTimeout,
   DataNode.getStreamingAddr(conf));
 }
 
 tcpPeerServer.setReceiveBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
 {code}
 The last line sets SO_RCVBUF explicitly, thus disabling tcp auto-tuning on 
 some system.
 Shall we make this behavior configurable?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (HDFS-8829) DataNode sets SO_RCVBUF explicitly is disabling tcp auto-tuning

2015-07-28 Thread He Tianyi (JIRA)

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

He Tianyi updated HDFS-8829:

Description: 
{code:java}

  private void initDataXceiver(Configuration conf) throws IOException {
// find free port or use privileged port provided
TcpPeerServer tcpPeerServer;
if (secureResources != null) {
  tcpPeerServer = new TcpPeerServer(secureResources);
} else {
  tcpPeerServer = new TcpPeerServer(dnConf.socketWriteTimeout,
  DataNode.getStreamingAddr(conf));
}
tcpPeerServer.setReceiveBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
{code}

The last line sets SO_RCVBUF explicitly, thus disabling tcp auto-tuning on some 
system.

Shall we make this behavior configurable?

  was:
{quote}

  private void initDataXceiver(Configuration conf) throws IOException {
// find free port or use privileged port provided
TcpPeerServer tcpPeerServer;
if (secureResources != null) {
  tcpPeerServer = new TcpPeerServer(secureResources);
} else {
  tcpPeerServer = new TcpPeerServer(dnConf.socketWriteTimeout,
  DataNode.getStreamingAddr(conf));
}
tcpPeerServer.setReceiveBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
{quote}

The last line sets SO_RCVBUF explicitly, thus disabling tcp auto-tuning on some 
system.

Shall we make this behavior configurable?


 DataNode sets SO_RCVBUF explicitly is disabling tcp auto-tuning
 ---

 Key: HDFS-8829
 URL: https://issues.apache.org/jira/browse/HDFS-8829
 Project: Hadoop HDFS
  Issue Type: Improvement
  Components: datanode
Affects Versions: 2.3.0, 2.6.0
Reporter: He Tianyi

 {code:java}
   private void initDataXceiver(Configuration conf) throws IOException {
 // find free port or use privileged port provided
 TcpPeerServer tcpPeerServer;
 if (secureResources != null) {
   tcpPeerServer = new TcpPeerServer(secureResources);
 } else {
   tcpPeerServer = new TcpPeerServer(dnConf.socketWriteTimeout,
   DataNode.getStreamingAddr(conf));
 }
 
 tcpPeerServer.setReceiveBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
 {code}
 The last line sets SO_RCVBUF explicitly, thus disabling tcp auto-tuning on 
 some system.
 Shall we make this behavior configurable?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (HDFS-8829) DataNode sets SO_RCVBUF explicitly is disabling tcp auto-tuning

2015-07-28 Thread He Tianyi (JIRA)
He Tianyi created HDFS-8829:
---

 Summary: DataNode sets SO_RCVBUF explicitly is disabling tcp 
auto-tuning
 Key: HDFS-8829
 URL: https://issues.apache.org/jira/browse/HDFS-8829
 Project: Hadoop HDFS
  Issue Type: Improvement
  Components: datanode
Affects Versions: 2.6.0, 2.3.0
Reporter: He Tianyi


{quote}

  private void initDataXceiver(Configuration conf) throws IOException {
// find free port or use privileged port provided
TcpPeerServer tcpPeerServer;
if (secureResources != null) {
  tcpPeerServer = new TcpPeerServer(secureResources);
} else {
  tcpPeerServer = new TcpPeerServer(dnConf.socketWriteTimeout,
  DataNode.getStreamingAddr(conf));
}
tcpPeerServer.setReceiveBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
{quote}

The last line sets SO_RCVBUF explicitly, thus disabling tcp auto-tuning on some 
system.

Shall we make this behavior configurable?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)