[jira] [Commented] (HDFS-14201) Ability to disallow safemode NN to become active

2019-01-13 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao commented on HDFS-14201:


[~surmountian] is it possible to check the namenode if in safemode when invoke 
#transitionToActive? if that I consider it can cover both auto-failover by ZKFC 
and manual transition to active. FYI.

> Ability to disallow safemode NN to become active
> 
>
> Key: HDFS-14201
> URL: https://issues.apache.org/jira/browse/HDFS-14201
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: auto-failover
>Affects Versions: 3.1.1, 2.9.2
>Reporter: Xiao Liang
>Assignee: Xiao Liang
>Priority: Major
>
> Currently with HA, Namenode in safemode can be possibly selected as active, 
> for availability of both read and write, Namenodes not in safemode are better 
> choices to become active though.
> It can take tens of minutes for a cold started Namenode to get out of 
> safemode, especially when there are large number of files and blocks in HDFS, 
> that means if a Namenode in safemode become active, the cluster will be not 
> fully functioning for quite a while, even if it can while there is some 
> Namenode not in safemode.
> The proposal here is to add an option, to allow Namenode to report itself as 
> UNHEALTHY to ZKFC, if it's in safemode, so as to only allow fully functioning 
> Namenode to become active, improving the general availability of the cluster.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (HDFS-14186) blockreport storm slow down namenode restart seriously in large cluster

2019-01-13 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao commented on HDFS-14186:


as mentioned above, the core issue: namenode uses blocks number as condition to 
leave safe mode rather than ALL blocks replication number, then heartbeat 
checker go back into working order and DN could be set DEAD if its heartbeat 
does not send to namenode for long time over than 630s by default, it is not 
related to lifeline overall i think.

> blockreport storm slow down namenode restart seriously in large cluster
> ---
>
> Key: HDFS-14186
> URL: https://issues.apache.org/jira/browse/HDFS-14186
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: namenode
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-14186.001.patch
>
>
> In the current implementation, the datanode sends blockreport immediately 
> after register to namenode successfully when restart, and the blockreport 
> storm will make namenode high load to process them. One result is some 
> received RPC have to skip because queue time is timeout. If some datanodes' 
> heartbeat RPC are continually skipped for long times (default is 
> heartbeatExpireInterval=630s) it will be set DEAD, then datanode has to 
> re-register and send blockreport again, aggravate blockreport storm and trap 
> in a vicious circle, and slow down (more than one hour and even more) 
> namenode startup seriously in a large (several thousands of datanodes) and 
> busy cluster especially. Although there are many work to optimize namenode 
> startup, the issue still exists. 
> I propose to postpone dead datanode check when namenode have finished startup.
> Any comments and suggestions are welcome.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Comment Edited] (HDFS-14186) blockreport storm slow down namenode restart seriously in large cluster

2019-01-12 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao edited comment on HDFS-14186 at 1/12/19 3:31 PM:
-

Hi [~elgoiri],
{quote}I'm guessing that the lifeline doesn't help because the DN is not 
registered at all?{quote}
Yep, you are correct, lifeline do not help absolutely. Actually I have open the 
feature in my production cluster. In this case, DN has registered and reported 
blocks but it is lost and set DEAD after NN just leave safe mode since its load 
is very high so DN has to register and block report again.
{quote}I'm also curious about the numbers, 300M blocks takes 8 hours? We see 
around 30 minutes with 60M block.{quote}
I found the case to different degrees with above 300M blocks.  The worst case 
which takes more than 8 hours I met once: more than 15K nodes with more than 
500M blocks.
Generally, I think number of slave nodes and block numbers both can impact the 
startup times under the same fsimage+editlogs. blocks number is more affected 
factor of course. 30min can finish to restart with 60M blocks, I see the same 
result as [~elgoiri] mentioned. Actually, with 100M blocks or less we can 
control  startup time within ~40min. It appears with more large-scale. more 
comments are welcome.


was (Author: hexiaoqiao):
Hi [~elgoiri],
{quote}I'm guessing that the lifeline doesn't help because the DN is not 
registered at all?{quote}
Yep, you are correct, lifeline do not help absolutely. Actually I have open the 
feature in my production cluster.
{quote}I'm also curious about the numbers, 300M blocks takes 8 hours? We see 
around 30 minutes with 60M block.{quote}
I found the case to different degrees with above 300M blocks.  The worst case 
which takes more than 8 hours I met: more than 15K nodes with more than 500M 
blocks.
Generally, I think number of slave nodes and block numbers both can impact the 
startup times under the same fsimage+editlogs. blocks number is more affected 
factor of course. 30min can finish to restart with 60M blocks, I see the same 
result as [~elgoiri] mentioned. Actually, with 100M blocks or less we can 
control  startup time within ~40min. It appears with more large-scale. more 
comments are welcome.

> blockreport storm slow down namenode restart seriously in large cluster
> ---
>
> Key: HDFS-14186
> URL: https://issues.apache.org/jira/browse/HDFS-14186
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: namenode
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-14186.001.patch
>
>
> In the current implementation, the datanode sends blockreport immediately 
> after register to namenode successfully when restart, and the blockreport 
> storm will make namenode high load to process them. One result is some 
> received RPC have to skip because queue time is timeout. If some datanodes' 
> heartbeat RPC are continually skipped for long times (default is 
> heartbeatExpireInterval=630s) it will be set DEAD, then datanode has to 
> re-register and send blockreport again, aggravate blockreport storm and trap 
> in a vicious circle, and slow down (more than one hour and even more) 
> namenode startup seriously in a large (several thousands of datanodes) and 
> busy cluster especially. Although there are many work to optimize namenode 
> startup, the issue still exists. 
> I propose to postpone dead datanode check when namenode have finished startup.
> Any comments and suggestions are welcome.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (HDFS-14186) blockreport storm slow down namenode restart seriously in large cluster

2019-01-12 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao commented on HDFS-14186:


Hi [~elgoiri],
{quote}I'm guessing that the lifeline doesn't help because the DN is not 
registered at all?{quote}
Yep, you are correct, lifeline do not help absolutely. Actually I have open the 
feature in my production cluster.
{quote}I'm also curious about the numbers, 300M blocks takes 8 hours? We see 
around 30 minutes with 60M block.{quote}
I found the case to different degrees with above 300M blocks.  The worst case 
which takes more than 8 hours I met: more than 15K nodes with more than 500M 
blocks.
Generally, I think number of slave nodes and block numbers both can impact the 
startup times under the same fsimage+editlogs. blocks number is more affected 
factor of course. 30min can finish to restart with 60M blocks, I see the same 
result as [~elgoiri] mentioned. Actually, with 100M blocks or less we can 
control  startup time within ~40min. It appears with more large-scale. more 
comments are welcome.

> blockreport storm slow down namenode restart seriously in large cluster
> ---
>
> Key: HDFS-14186
> URL: https://issues.apache.org/jira/browse/HDFS-14186
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: namenode
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-14186.001.patch
>
>
> In the current implementation, the datanode sends blockreport immediately 
> after register to namenode successfully when restart, and the blockreport 
> storm will make namenode high load to process them. One result is some 
> received RPC have to skip because queue time is timeout. If some datanodes' 
> heartbeat RPC are continually skipped for long times (default is 
> heartbeatExpireInterval=630s) it will be set DEAD, then datanode has to 
> re-register and send blockreport again, aggravate blockreport storm and trap 
> in a vicious circle, and slow down (more than one hour and even more) 
> namenode startup seriously in a large (several thousands of datanodes) and 
> busy cluster especially. Although there are many work to optimize namenode 
> startup, the issue still exists. 
> I propose to postpone dead datanode check when namenode have finished startup.
> Any comments and suggestions are welcome.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (HDFS-14186) blockreport storm slow down namenode restart seriously in large cluster

2019-01-10 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao commented on HDFS-14186:


I think safe mode extension could cover the issue in small or medium scale 
cluster (include 300M blocks or less) using default extension time 30s, but 
when the block size is too large, default extension time could not absorb all 
block report from datanode. As result, in our production cluster, we have to 
monitor load of service port 8040 after namenode leave safe mode, then manual 
execute master slave switching util load recovery, waiting about 30min after 
namenode leave safe mode under normal conditions.
I try to add replication total checking when namenode startup with 
configuration item for open this feature or not in the demonstration patch 
[^HDFS-14186.001.patch]. FYI. If the idea could pass, I will add some unittest 
next couple days.
[~kihwal] Thanks again, and look forward to your comments.

> blockreport storm slow down namenode restart seriously in large cluster
> ---
>
> Key: HDFS-14186
> URL: https://issues.apache.org/jira/browse/HDFS-14186
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: namenode
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-14186.001.patch
>
>
> In the current implementation, the datanode sends blockreport immediately 
> after register to namenode successfully when restart, and the blockreport 
> storm will make namenode high load to process them. One result is some 
> received RPC have to skip because queue time is timeout. If some datanodes' 
> heartbeat RPC are continually skipped for long times (default is 
> heartbeatExpireInterval=630s) it will be set DEAD, then datanode has to 
> re-register and send blockreport again, aggravate blockreport storm and trap 
> in a vicious circle, and slow down (more than one hour and even more) 
> namenode startup seriously in a large (several thousands of datanodes) and 
> busy cluster especially. Although there are many work to optimize namenode 
> startup, the issue still exists. 
> I propose to postpone dead datanode check when namenode have finished startup.
> Any comments and suggestions are welcome.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (HDFS-13473) DataNode update BlockKeys using mode PULL rather than PUSH from NameNode

2019-01-09 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao commented on HDFS-13473:


ping [~daryn], do you mind to recheck this feature and feedback some more 
suggestion if you are available.

> DataNode update BlockKeys using mode PULL rather than PUSH from NameNode
> 
>
> Key: HDFS-13473
> URL: https://issues.apache.org/jira/browse/HDFS-13473
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: datanode
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-13473-trunk.001.patch, HDFS-13473-trunk.002.patch, 
> HDFS-13473-trunk.003.patch, HDFS-13473-trunk.004.patch, 
> HDFS-13473-trunk.005.patch
>
>
> It is passive behavior about updating Block keys for DataNode currently, and 
> it depends on if NameNode return #KeyUpdateCommand for heartbeat response.
> There are several problems of this Block keys synchronization mode:
> a. NameNode can't be sensed about if Block Keys reach DataNode successfully,
> b. It is also not sensed for DataNode who meets some exception while receive 
> or process heartbeat response which include BlockKeyCommand,
> such as HDFS-13441 and HDFS-12749 mentioned.
> So I propose improve Push Block Keys from NameNode for DataNode to DataNode 
> Pull Block Keys.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (HDFS-14186) blockreport storm slow down namenode restart seriously in large cluster

2019-01-09 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao commented on HDFS-14186:


after checking trunk branch, I believe this issue still exists.
ping watcher guys, any comments and suggestions? 

> blockreport storm slow down namenode restart seriously in large cluster
> ---
>
> Key: HDFS-14186
> URL: https://issues.apache.org/jira/browse/HDFS-14186
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: namenode
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-14186.001.patch
>
>
> In the current implementation, the datanode sends blockreport immediately 
> after register to namenode successfully when restart, and the blockreport 
> storm will make namenode high load to process them. One result is some 
> received RPC have to skip because queue time is timeout. If some datanodes' 
> heartbeat RPC are continually skipped for long times (default is 
> heartbeatExpireInterval=630s) it will be set DEAD, then datanode has to 
> re-register and send blockreport again, aggravate blockreport storm and trap 
> in a vicious circle, and slow down (more than one hour and even more) 
> namenode startup seriously in a large (several thousands of datanodes) and 
> busy cluster especially. Although there are many work to optimize namenode 
> startup, the issue still exists. 
> I propose to postpone dead datanode check when namenode have finished startup.
> Any comments and suggestions are welcome.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (HDFS-14186) blockreport storm slow down namenode restart seriously in large cluster

2019-01-08 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao commented on HDFS-14186:


attached a quick-and-dirty demonstration patch ([^HDFS-14186.001.patch]) 
showing how solve this issue with replication safepoint checking just when 
namenode restart.

> blockreport storm slow down namenode restart seriously in large cluster
> ---
>
> Key: HDFS-14186
> URL: https://issues.apache.org/jira/browse/HDFS-14186
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: namenode
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-14186.001.patch
>
>
> In the current implementation, the datanode sends blockreport immediately 
> after register to namenode successfully when restart, and the blockreport 
> storm will make namenode high load to process them. One result is some 
> received RPC have to skip because queue time is timeout. If some datanodes' 
> heartbeat RPC are continually skipped for long times (default is 
> heartbeatExpireInterval=630s) it will be set DEAD, then datanode has to 
> re-register and send blockreport again, aggravate blockreport storm and trap 
> in a vicious circle, and slow down (more than one hour and even more) 
> namenode startup seriously in a large (several thousands of datanodes) and 
> busy cluster especially. Although there are many work to optimize namenode 
> startup, the issue still exists. 
> I propose to postpone dead datanode check when namenode have finished startup.
> Any comments and suggestions are welcome.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Updated] (HDFS-14186) blockreport storm slow down namenode restart seriously in large cluster

2019-01-08 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao updated HDFS-14186:
---
Attachment: HDFS-14186.001.patch

> blockreport storm slow down namenode restart seriously in large cluster
> ---
>
> Key: HDFS-14186
> URL: https://issues.apache.org/jira/browse/HDFS-14186
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: namenode
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-14186.001.patch
>
>
> In the current implementation, the datanode sends blockreport immediately 
> after register to namenode successfully when restart, and the blockreport 
> storm will make namenode high load to process them. One result is some 
> received RPC have to skip because queue time is timeout. If some datanodes' 
> heartbeat RPC are continually skipped for long times (default is 
> heartbeatExpireInterval=630s) it will be set DEAD, then datanode has to 
> re-register and send blockreport again, aggravate blockreport storm and trap 
> in a vicious circle, and slow down (more than one hour and even more) 
> namenode startup seriously in a large (several thousands of datanodes) and 
> busy cluster especially. Although there are many work to optimize namenode 
> startup, the issue still exists. 
> I propose to postpone dead datanode check when namenode have finished startup.
> Any comments and suggestions are welcome.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (HDFS-14186) blockreport storm slow down namenode restart seriously in large cluster

2019-01-08 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao commented on HDFS-14186:


Note logs about namenode and one datanode which is marked dead when namenode 
restart.
namenode log:
{code:java}
2019-01-03 02:13:16,197 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* 
removeDeadDatanode: lost heartbeat from datanode:50010
2019-01-03 02:13:16,198 INFO 
org.apache.hadoop.hdfs.server.blockmanagement.NodeStat: remove child: 
/ROOT/RACK/datanode:50010
2019-01-03 02:13:16,200 INFO org.apache.hadoop.net.NetworkTopology: Removing a 
node: /ROOT/RACK/datanode:50010
2019-01-03 02:13:43,207 INFO org.apache.hadoop.ipc.Server: IPC Server handler 
25 on 8040, call 
org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol.blockReport from 
datanode:53518 Call#134261749 Retry#0
java.io.IOException: ProcessReport from dead or unregistered node: 
DatanodeRegistration(datanode:50010, 
datanodeUuid=8a54acea-fec9-4267-bd9b-e21cdc821787, infoPort=50075, 
infoSecurePort=0, ipcPort=50020, 
storageInfo=lv=-57;cid=CID-13cec691-e813-4241-a752-5bbfc4342f2f;nsid=138673305;c=1417744182994)
at 
org.apache.hadoop.hdfs.server.blockmanagement.BlockManager.processReport(BlockManager.java:2457)
at 
org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.blockReport(NameNodeRpcServer.java:1525)
at 
org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolServerSideTranslatorPB.blockReport(DatanodeProtocolServerSideTranslatorPB.java:176)
at 
org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos$DatanodeProtocolService$2.callBlockingMethod(DatanodeProtocolProtos.java:33713)
at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:616)
at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:969)
at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2049)
at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2045)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:422)
at 
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1689)
at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2045)
{code} 

datanode log:
{code:java}
2019-01-03 02:12:07,202 INFO org.apache.hadoop.ipc.Client: Retrying connect to 
server: namenode/ip:8040. Already tried 3 time(s); maxRetries=45
2019-01-03 02:12:27,223 INFO org.apache.hadoop.ipc.Client: Retrying connect to 
server: namenode/ip:8040. Already tried 4 time(s); maxRetries=45
2019-01-03 02:12:47,242 INFO org.apache.hadoop.ipc.Client: Retrying connect to 
server: namenode/ip:8040. Already tried 5 time(s); maxRetries=45
2019-01-03 02:15:18,951 INFO org.apache.hadoop.ipc.Client: Retrying connect to 
server: namenode/ip:8040. Already tried 0 time(s); maxRetries=45
2019-01-03 02:15:38,953 INFO org.apache.hadoop.ipc.Client: Retrying connect to 
server: namenode/ip:8040. Already tried 1 time(s); maxRetries=45
{code}


> blockreport storm slow down namenode restart seriously in large cluster
> ---
>
> Key: HDFS-14186
> URL: https://issues.apache.org/jira/browse/HDFS-14186
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: namenode
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
>
> In the current implementation, the datanode sends blockreport immediately 
> after register to namenode successfully when restart, and the blockreport 
> storm will make namenode high load to process them. One result is some 
> received RPC have to skip because queue time is timeout. If some datanodes' 
> heartbeat RPC are continually skipped for long times (default is 
> heartbeatExpireInterval=630s) it will be set DEAD, then datanode has to 
> re-register and send blockreport again, aggravate blockreport storm and trap 
> in a vicious circle, and slow down (more than one hour and even more) 
> namenode startup seriously in a large (several thousands of datanodes) and 
> busy cluster especially. Although there are many work to optimize namenode 
> startup, the issue still exists. 
> I propose to postpone dead datanode check when namenode have finished startup.
> Any comments and suggestions are welcome.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (HDFS-14186) blockreport storm slow down namenode restart seriously in large cluster

2019-01-07 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao commented on HDFS-14186:


[~kihwal] Thanks for your comments.
{quote}I think nodes are already not marked "dead" in the startup safe mode
{quote}
As mentioned above, nodes are marked "dead" after NameNode leave safe mode.
{quote}Is your datanodes configured to break up the reports per storage and 
send one by one?
{quote}
I do not split block report per storage, but I do not think it is the key point 
for this issue. Because when I trace the log of NameNode, the process time 
about block report is almost less than 30ms, due to the optimization of the 
first block report. Another way, blocks num per DataNode for the separate 
namespace is less than 100K.
{quote}how much was the GC overhead? Was it replaying edits during the starup?
{quote}
do not found any GC/EditLogs exception action of the worst case I met, 2 times 
CMS GC, during ~100s, and no FGC found. YGC count is normal and STW is all less 
than 200ms. During startup, replaying editlogs about each 2 mins. and the most 
lock times is about ~50s(>2000K txn), other less than 10s.

As the worst case the beginning mentioned, it costs about 1hour to load fsimage 
+ replay editlogs + block report, and consider startup is done then auto leave 
safe mode, but 8040 port (separate 8040=service RPC port and 8020=RPC port) is 
continued FULL last ~7hours. During that time, ~1K different datanodes are 
marked dead and re-register/block report again and again.
 So I think leave safe mode if wait for the majority replication reported 
rather than the majority block reported may be resolve this case. FYI.

Add Environment Info: Hadoop-2.7.1, HA using QJM.

> blockreport storm slow down namenode restart seriously in large cluster
> ---
>
> Key: HDFS-14186
> URL: https://issues.apache.org/jira/browse/HDFS-14186
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: namenode
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
>
> In the current implementation, the datanode sends blockreport immediately 
> after register to namenode successfully when restart, and the blockreport 
> storm will make namenode high load to process them. One result is some 
> received RPC have to skip because queue time is timeout. If some datanodes' 
> heartbeat RPC are continually skipped for long times (default is 
> heartbeatExpireInterval=630s) it will be set DEAD, then datanode has to 
> re-register and send blockreport again, aggravate blockreport storm and trap 
> in a vicious circle, and slow down (more than one hour and even more) 
> namenode startup seriously in a large (several thousands of datanodes) and 
> busy cluster especially. Although there are many work to optimize namenode 
> startup, the issue still exists. 
> I propose to postpone dead datanode check when namenode have finished startup.
> Any comments and suggestions are welcome.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (HDFS-14186) blockreport storm slow down namenode restart seriously in large cluster

2019-01-06 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao commented on HDFS-14186:


Would like to offer more details:
1. when namenode restart, all datanode has to re-register and send blockreport 
for a while.
2. NameNode will leave safe mode if reported blocks (NOT replications) num 
reached configure block threshold (default=99.9%) * blockTotals, and num live 
datanodes (which is equal to registered datanodes) reached datanode 
threshold(default = 0).
3. when NameNode leave safe mode at startup progress, heartbeat checker will 
back into working order. BUT block report storm will be continue at this moment.
4. because NameNode load is very high, some heartbeat RPC from datanode will be 
discard as mentioned above. then some datanode may be stale even dead and 
re-register and send block report again.
5. it slows down restart time of NameNode.(more than 8 hours the worst case I 
met, 20K slaves.)

The core issue: use blocks number as condition to leave safe mode rather than 
ALL blocks replication number.
for instance, there are 12K datanodes in cluster, if ~4K=1/3*12K has reported 
by default configuration, NameNode may leave safe mode, because ~4K datanodes 
may include all blocks theoretically. However, there are 8K datanodes still 
reporting and namenode load is still very high currently.

One solution is configuration datanode threshold (not 0 by default) but it is 
not stable value and limited in use.
I think we can use replicationTotal rather than blockTotal as condition to 
leave safe mode and postpone heartbeat checker to work order.

> blockreport storm slow down namenode restart seriously in large cluster
> ---
>
> Key: HDFS-14186
> URL: https://issues.apache.org/jira/browse/HDFS-14186
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: namenode
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
>
> In the current implementation, the datanode sends blockreport immediately 
> after register to namenode successfully when restart, and the blockreport 
> storm will make namenode high load to process them. One result is some 
> received RPC have to skip because queue time is timeout. If some datanodes' 
> heartbeat RPC are continually skipped for long times (default is 
> heartbeatExpireInterval=630s) it will be set DEAD, then datanode has to 
> re-register and send blockreport again, aggravate blockreport storm and trap 
> in a vicious circle, and slow down (more than one hour and even more) 
> namenode startup seriously in a large (several thousands of datanodes) and 
> busy cluster especially. Although there are many work to optimize namenode 
> startup, the issue still exists. 
> I propose to postpone dead datanode check when namenode have finished startup.
> Any comments and suggestions are welcome.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Created] (HDFS-14186) blockreport storm slow down namenode restart seriously in large cluster

2019-01-05 Thread He Xiaoqiao (JIRA)
He Xiaoqiao created HDFS-14186:
--

 Summary: blockreport storm slow down namenode restart seriously in 
large cluster
 Key: HDFS-14186
 URL: https://issues.apache.org/jira/browse/HDFS-14186
 Project: Hadoop HDFS
  Issue Type: Improvement
  Components: namenode
Reporter: He Xiaoqiao
Assignee: He Xiaoqiao


In the current implementation, the datanode sends blockreport immediately after 
register to namenode successfully when restart, and the blockreport storm will 
make namenode high load to process them. One result is some received RPC have 
to skip because queue time is timeout. If some datanodes' heartbeat RPC are 
continually skipped for long times (default is heartbeatExpireInterval=630s) it 
will be set DEAD, then datanode has to re-register and send blockreport again, 
aggravate blockreport storm and trap in a vicious circle, and slow down (more 
than one hour and even more) namenode startup seriously in a large (several 
thousands of datanodes) and busy cluster especially. Although there are many 
work to optimize namenode startup, the issue still exists. 
I propose to postpone dead datanode check when namenode have finished startup.
Any comments and suggestions are welcome.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (HDFS-14181) Suspect there is a bug in NetworkTopology.java chooseRandom function.

2019-01-02 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao commented on HDFS-14181:


[~sihai], Thanks for your work, as you mentioned before, read lock is 
necessary, It can protect both {{clusterMap}} and {{excludedNodes}} changed 
between double {{countNumOfAvailableNodes}} calc, another, {{netlock}} is 
reentrant lock, so it is ok for me. ping [~elgoiri],[~ayushtkn] please double 
check.
To [~sihai], rename new unit test method (#testChooseRandom1) to more 
comprehensible may be better. FYI.
+1 LGTM after update on [^HDFS-14181.05.patch].

> Suspect there is a bug in NetworkTopology.java chooseRandom function.
> -
>
> Key: HDFS-14181
> URL: https://issues.apache.org/jira/browse/HDFS-14181
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: hdfs, namenode
>Affects Versions: 2.9.2
>Reporter: Sihai Ke
>Assignee: Sihai Ke
>Priority: Major
> Attachments: 0001-add-UT-for-NetworkTopology.patch, 
> 0001-fix-NetworkTopology.java-chooseRandom-bug.patch, HDFS-14181.01.patch, 
> HDFS-14181.02.patch, HDFS-14181.03.patch, HDFS-14181.04.patch, 
> HDFS-14181.05.patch, image-2018-12-29-15-02-19-415.png
>
>
> During reading the hadoop NetworkTopology.java, I suspect there is a bug in 
> function 
> chooseRandom (line 498, hadoop version 2.9.2-RC0), 
>  I think there is a bug in{color:#f79232} code, ~excludedScope doesn't mean 
> availableNodes under Scope node, and I also add unit test for this and get an 
> exception.{color}
> bug code in the else.
> {code:java}
> // code placeholder
>  if (excludedScope == null) {
> availableNodes = countNumOfAvailableNodes(scope, excludedNodes);
>   } else {
> availableNodes =
> countNumOfAvailableNodes("~" + excludedScope, excludedNodes);
>   }{code}
> Source code:
> {code:java}
> // code placeholder
> protected Node chooseRandom(final String scope, String excludedScope,
> final Collection excludedNodes) {
>   if (excludedScope != null) {
> if (scope.startsWith(excludedScope)) {
>   return null;
> }
> if (!excludedScope.startsWith(scope)) {
>   excludedScope = null;
> }
>   }
>   Node node = getNode(scope);
>   if (!(node instanceof InnerNode)) {
> return excludedNodes != null && excludedNodes.contains(node) ?
> null : node;
>   }
>   InnerNode innerNode = (InnerNode)node;
>   int numOfDatanodes = innerNode.getNumOfLeaves();
>   if (excludedScope == null) {
> node = null;
>   } else {
> node = getNode(excludedScope);
> if (!(node instanceof InnerNode)) {
>   numOfDatanodes -= 1;
> } else {
>   numOfDatanodes -= ((InnerNode)node).getNumOfLeaves();
> }
>   }
>   if (numOfDatanodes <= 0) {
> LOG.debug("Failed to find datanode (scope=\"{}\" excludedScope=\"{}\")."
> + " numOfDatanodes={}",
> scope, excludedScope, numOfDatanodes);
> return null;
>   }
>   final int availableNodes;
>   if (excludedScope == null) {
> availableNodes = countNumOfAvailableNodes(scope, excludedNodes);
>   } else {
> availableNodes =
> countNumOfAvailableNodes("~" + excludedScope, excludedNodes);
>   }
>   LOG.debug("Choosing random from {} available nodes on node {},"
>   + " scope={}, excludedScope={}, excludeNodes={}. numOfDatanodes={}.",
>   availableNodes, innerNode, scope, excludedScope, excludedNodes,
>   numOfDatanodes);
>   Node ret = null;
>   if (availableNodes > 0) {
> ret = chooseRandom(innerNode, node, excludedNodes, numOfDatanodes,
> availableNodes);
>   }
>   LOG.debug("chooseRandom returning {}", ret);
>   return ret;
> }
> {code}
>  
>  
> Add Unit Test in TestClusterTopology.java, but get exception.
>  
> {code:java}
> // code placeholder
> @Test
> public void testChooseRandom1() {
>   // create the topology
>   NetworkTopology cluster = NetworkTopology.getInstance(new Configuration());
>   NodeElement node1 = getNewNode("node1", "/a1/b1/c1");
>   cluster.add(node1);
>   NodeElement node2 = getNewNode("node2", "/a1/b1/c1");
>   cluster.add(node2);
>   NodeElement node3 = getNewNode("node3", "/a1/b1/c2");
>   cluster.add(node3);
>   NodeElement node4 = getNewNode("node4", "/a1/b2/c3");
>   cluster.add(node4);
>   Node node = cluster.chooseRandom("/a1/b1", "/a1/b1/c1", null);
>   assertSame(node.getName(), "node3");
> }
> {code}
>  
> Exception:
> {code:java}
> // code placeholder
> java.lang.IllegalArgumentException: 1 should >= 2, and both should be 
> positive. 
> at com.google.common.base.Preconditions.checkArgument(Preconditions.java:88) 
> at 
> org.apache.hadoop.net.NetworkTopology.chooseRandom(NetworkTopology.java:567) 
> at 
> org.apache.hadoop.net.NetworkTopology.chooseRandom(NetworkTopology.java:544) 
> 

[jira] [Commented] (HDFS-14181) Suspect there is a bug in NetworkTopology.java chooseRandom function.

2019-01-01 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao commented on HDFS-14181:


[~elgoiri][~sihai] Thanks for your corrected comments.
[^HDFS-14181.03.patch] almost looks good to me. some minor comments from me:
1. it may be not necessary to add lock since already does it. +1
2. to [~sihai], could you submit patch rebase trunk branch, trunk flawed as 
above.

> Suspect there is a bug in NetworkTopology.java chooseRandom function.
> -
>
> Key: HDFS-14181
> URL: https://issues.apache.org/jira/browse/HDFS-14181
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: hdfs, namenode
>Affects Versions: 2.9.2
>Reporter: Sihai Ke
>Assignee: Sihai Ke
>Priority: Major
> Attachments: 0001-add-UT-for-NetworkTopology.patch, 
> 0001-fix-NetworkTopology.java-chooseRandom-bug.patch, HDFS-14181.01.patch, 
> HDFS-14181.02.patch, HDFS-14181.03.patch, image-2018-12-29-15-02-19-415.png
>
>
> During reading the hadoop NetworkTopology.java, I suspect there is a bug in 
> function 
> chooseRandom (line 498, hadoop version 2.9.2-RC0), 
>  I think there is a bug in{color:#f79232} code, ~excludedScope doesn't mean 
> availableNodes under Scope node, and I also add unit test for this and get an 
> exception.{color}
> bug code in the else.
> {code:java}
> // code placeholder
>  if (excludedScope == null) {
> availableNodes = countNumOfAvailableNodes(scope, excludedNodes);
>   } else {
> availableNodes =
> countNumOfAvailableNodes("~" + excludedScope, excludedNodes);
>   }{code}
> Source code:
> {code:java}
> // code placeholder
> protected Node chooseRandom(final String scope, String excludedScope,
> final Collection excludedNodes) {
>   if (excludedScope != null) {
> if (scope.startsWith(excludedScope)) {
>   return null;
> }
> if (!excludedScope.startsWith(scope)) {
>   excludedScope = null;
> }
>   }
>   Node node = getNode(scope);
>   if (!(node instanceof InnerNode)) {
> return excludedNodes != null && excludedNodes.contains(node) ?
> null : node;
>   }
>   InnerNode innerNode = (InnerNode)node;
>   int numOfDatanodes = innerNode.getNumOfLeaves();
>   if (excludedScope == null) {
> node = null;
>   } else {
> node = getNode(excludedScope);
> if (!(node instanceof InnerNode)) {
>   numOfDatanodes -= 1;
> } else {
>   numOfDatanodes -= ((InnerNode)node).getNumOfLeaves();
> }
>   }
>   if (numOfDatanodes <= 0) {
> LOG.debug("Failed to find datanode (scope=\"{}\" excludedScope=\"{}\")."
> + " numOfDatanodes={}",
> scope, excludedScope, numOfDatanodes);
> return null;
>   }
>   final int availableNodes;
>   if (excludedScope == null) {
> availableNodes = countNumOfAvailableNodes(scope, excludedNodes);
>   } else {
> availableNodes =
> countNumOfAvailableNodes("~" + excludedScope, excludedNodes);
>   }
>   LOG.debug("Choosing random from {} available nodes on node {},"
>   + " scope={}, excludedScope={}, excludeNodes={}. numOfDatanodes={}.",
>   availableNodes, innerNode, scope, excludedScope, excludedNodes,
>   numOfDatanodes);
>   Node ret = null;
>   if (availableNodes > 0) {
> ret = chooseRandom(innerNode, node, excludedNodes, numOfDatanodes,
> availableNodes);
>   }
>   LOG.debug("chooseRandom returning {}", ret);
>   return ret;
> }
> {code}
>  
>  
> Add Unit Test in TestClusterTopology.java, but get exception.
>  
> {code:java}
> // code placeholder
> @Test
> public void testChooseRandom1() {
>   // create the topology
>   NetworkTopology cluster = NetworkTopology.getInstance(new Configuration());
>   NodeElement node1 = getNewNode("node1", "/a1/b1/c1");
>   cluster.add(node1);
>   NodeElement node2 = getNewNode("node2", "/a1/b1/c1");
>   cluster.add(node2);
>   NodeElement node3 = getNewNode("node3", "/a1/b1/c2");
>   cluster.add(node3);
>   NodeElement node4 = getNewNode("node4", "/a1/b2/c3");
>   cluster.add(node4);
>   Node node = cluster.chooseRandom("/a1/b1", "/a1/b1/c1", null);
>   assertSame(node.getName(), "node3");
> }
> {code}
>  
> Exception:
> {code:java}
> // code placeholder
> java.lang.IllegalArgumentException: 1 should >= 2, and both should be 
> positive. 
> at com.google.common.base.Preconditions.checkArgument(Preconditions.java:88) 
> at 
> org.apache.hadoop.net.NetworkTopology.chooseRandom(NetworkTopology.java:567) 
> at 
> org.apache.hadoop.net.NetworkTopology.chooseRandom(NetworkTopology.java:544) 
> atorg.apache.hadoop.net.TestClusterTopology.testChooseRandom1(TestClusterTopology.java:198)
> {code}
>  
> {color:#f79232}!image-2018-12-29-15-02-19-415.png!{color}
>  
>  
> [~vagarychen] this change is imported in PR HDFS-11577, 

[jira] [Commented] (HDFS-14181) Suspect there is a bug in NetworkTopology.java chooseRandom function.

2018-12-30 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao commented on HDFS-14181:


[~sihai] Thanks for your reporting, I think it may be not bug, since 
{{NetworkTopology#chooseRandom(final String scope, String excludedScope, final 
Collection excludedNodes)}} is protect method, and invoke by 
{{NetworkTopology#chooseRandom(final String scope, final Collection 
excludedNodes)}} only, which code as following, and the unit test you attached 
could not appear.
{code:java}
  public Node chooseRandom(final String scope,
  final Collection excludedNodes) {
netlock.readLock().lock();
try {
  if (scope.startsWith("~")) {
return chooseRandom(NodeBase.ROOT, scope.substring(1), excludedNodes);
  } else {
return chooseRandom(scope, null, excludedNodes);
  }
} finally {
  netlock.readLock().unlock();
}
  }
{code}
I think it may be a fix type to change the method 
{{NetworkTopology#chooseRandom(final String scope, String excludedScope, final 
Collection excludedNodes)}} access scope from {{protected}} to 
{{private}}.

> Suspect there is a bug in NetworkTopology.java chooseRandom function.
> -
>
> Key: HDFS-14181
> URL: https://issues.apache.org/jira/browse/HDFS-14181
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: hdfs, namenode
>Affects Versions: 2.9.2
>Reporter: Sihai Ke
>Priority: Major
> Attachments: 0001-add-UT-for-NetworkTopology.patch, 
> image-2018-12-29-15-02-19-415.png
>
>
> During reading the hadoop NetworkTopology.java, I suspect there is a bug in 
> function 
> chooseRandom (line 498, hadoop version 2.9.2-RC0), 
>  I think there is a bug in{color:#f79232} code, ~excludedScope doesn't mean 
> availableNodes under Scope node, and I also add unit test for this and get an 
> exception.{color}
> bug code in the else.
> {code:java}
> // code placeholder
>  if (excludedScope == null) {
> availableNodes = countNumOfAvailableNodes(scope, excludedNodes);
>   } else {
> availableNodes =
> countNumOfAvailableNodes("~" + excludedScope, excludedNodes);
>   }{code}
> Source code:
> {code:java}
> // code placeholder
> protected Node chooseRandom(final String scope, String excludedScope,
> final Collection excludedNodes) {
>   if (excludedScope != null) {
> if (scope.startsWith(excludedScope)) {
>   return null;
> }
> if (!excludedScope.startsWith(scope)) {
>   excludedScope = null;
> }
>   }
>   Node node = getNode(scope);
>   if (!(node instanceof InnerNode)) {
> return excludedNodes != null && excludedNodes.contains(node) ?
> null : node;
>   }
>   InnerNode innerNode = (InnerNode)node;
>   int numOfDatanodes = innerNode.getNumOfLeaves();
>   if (excludedScope == null) {
> node = null;
>   } else {
> node = getNode(excludedScope);
> if (!(node instanceof InnerNode)) {
>   numOfDatanodes -= 1;
> } else {
>   numOfDatanodes -= ((InnerNode)node).getNumOfLeaves();
> }
>   }
>   if (numOfDatanodes <= 0) {
> LOG.debug("Failed to find datanode (scope=\"{}\" excludedScope=\"{}\")."
> + " numOfDatanodes={}",
> scope, excludedScope, numOfDatanodes);
> return null;
>   }
>   final int availableNodes;
>   if (excludedScope == null) {
> availableNodes = countNumOfAvailableNodes(scope, excludedNodes);
>   } else {
> availableNodes =
> countNumOfAvailableNodes("~" + excludedScope, excludedNodes);
>   }
>   LOG.debug("Choosing random from {} available nodes on node {},"
>   + " scope={}, excludedScope={}, excludeNodes={}. numOfDatanodes={}.",
>   availableNodes, innerNode, scope, excludedScope, excludedNodes,
>   numOfDatanodes);
>   Node ret = null;
>   if (availableNodes > 0) {
> ret = chooseRandom(innerNode, node, excludedNodes, numOfDatanodes,
> availableNodes);
>   }
>   LOG.debug("chooseRandom returning {}", ret);
>   return ret;
> }
> {code}
>  
>  
> Add Unit Test in TestClusterTopology.java, but get exception.
>  
> {code:java}
> // code placeholder
> @Test
> public void testChooseRandom1() {
>   // create the topology
>   NetworkTopology cluster = NetworkTopology.getInstance(new Configuration());
>   NodeElement node1 = getNewNode("node1", "/a1/b1/c1");
>   cluster.add(node1);
>   NodeElement node2 = getNewNode("node2", "/a1/b1/c1");
>   cluster.add(node2);
>   NodeElement node3 = getNewNode("node3", "/a1/b1/c2");
>   cluster.add(node3);
>   NodeElement node4 = getNewNode("node4", "/a1/b2/c3");
>   cluster.add(node4);
>   Node node = cluster.chooseRandom("/a1/b1", "/a1/b1/c1", null);
>   assertSame(node.getName(), "node3");
> }
> {code}
>  
> Exception:
> {code:java}
> // code placeholder
> 

[jira] [Commented] (HDFS-9198) Coalesce IBR processing in the NN

2018-12-21 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao commented on HDFS-9198:
---

Thanks [~daryn] for the nice work, and it is very useful for me. I am confused 
about the following code, LOG means block report queue is full now, however the 
IF statement is about how long from lastFull to now. Why not use queue size to 
estimate if queue is full or not directly? 
{code:java}
void enqueue(Runnable action) throws InterruptedException {
  if (!queue.offer(action)) {
if (!isAlive() && namesystem.isRunning()) {
  ExitUtil.terminate(1, getName() + " is not running");
}
long now = Time.monotonicNow();
if (now - lastFull > 4000) {
  lastFull = now;
  LOG.info("Block report queue is full");
}
queue.put(action);
  }
}
{code}
If I missed something, please correct me. Thanks again.

> Coalesce IBR processing in the NN
> -
>
> Key: HDFS-9198
> URL: https://issues.apache.org/jira/browse/HDFS-9198
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: namenode
>Affects Versions: 2.0.0-alpha
>Reporter: Daryn Sharp
>Assignee: Daryn Sharp
>Priority: Major
> Fix For: 2.8.0, 2.7.3, 3.0.0-alpha1
>
> Attachments: HDFS-9198-Branch-2-withamend.diff, 
> HDFS-9198-Branch-2.8-withamend.diff, HDFS-9198-branch-2.7.patch, 
> HDFS-9198-branch2.patch, HDFS-9198-trunk.patch, HDFS-9198-trunk.patch, 
> HDFS-9198-trunk.patch, HDFS-9198-trunk.patch, HDFS-9198-trunk.patch
>
>
> IBRs from thousands of DNs under load will degrade NN performance due to 
> excessive write-lock contention from multiple IPC handler threads.  The IBR 
> processing is quick, so the lock contention may be reduced by coalescing 
> multiple IBRs into a single write-lock transaction.  The handlers will also 
> be freed up faster for other operations.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (HDFS-14109) Improve hdfs auditlog format and support federation friendly

2018-12-06 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao commented on HDFS-14109:


Thanks [~xkrogen],[~kihwal] for discussing this issue.
{quote}I think as with most recent additions to the audit log, it should be 
protected by a config which defaults to off. In particular, in an environment 
using only a single namespace, we definitely don't want this information.{quote}
+1, only for federation with multi-namespace, and switch off default by a 
config.
{quote}People deal with logs from multiple systems today without having to 
insert the source identity in every single log line. {quote}
Actually, there are multiple system can deal with mass logs data. my opinion is:
1) the lowest-cost method to deal with logs. e.g. 10B audit-log records may 
cost our amount computing resource if relay with other system.
2) another point, I consider this is scope of hdfs rather than push to other 
system.
Maybe I missing some information, please give your feedback if there are 
something wrong.
Thanks [~xkrogen],  [~kihwal] again.

> Improve hdfs auditlog format and support federation friendly
> 
>
> Key: HDFS-14109
> URL: https://issues.apache.org/jira/browse/HDFS-14109
> Project: Hadoop HDFS
>  Issue Type: Improvement
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-14109.patch
>
>
> The following auditlog format does not well meet requirement for federation 
> arch currently. Since some case we need to aggregate all namespace audit log, 
> if there are some common path request(e.g. /tmp, /user/ etc. some path may 
> not appear in mountTable, but the path is very real), we will have no idea to 
> split them that which namespace it request to. So I propose add column 
> {{nsid}} to support federation more friendly.  
> {quote}2018-11-27 13:20:30,028 INFO FSNamesystem.audit: allowed=true   
> ugi=hdfs/hostn...@realm.com (auth:KERBEROS)  ip=/10.1.1.2 cmd=getfileinfo 
> src=/path   dst=null        perm=null       proto=rpc       clientName=null
> {quote}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (HDFS-14109) Improve hdfs auditlog format and support federation friendly

2018-11-28 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao commented on HDFS-14109:


[~kihwal] Thanks for your comments firstly. 
{quote}NN audit log is already huge and making it even bigger by adding 
redundant bytes is far from ideal.{quote}
1. I agree with you one hundred percent.
2. NN audit log is incomplete information now if we stand unified filesystem 
perspective for Federation.
Please correct me if there are something wrong. Thanks again.

> Improve hdfs auditlog format and support federation friendly
> 
>
> Key: HDFS-14109
> URL: https://issues.apache.org/jira/browse/HDFS-14109
> Project: Hadoop HDFS
>  Issue Type: Improvement
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-14109.patch
>
>
> The following auditlog format does not well meet requirement for federation 
> arch currently. Since some case we need to aggregate all namespace audit log, 
> if there are some common path request(e.g. /tmp, /user/ etc. some path may 
> not appear in mountTable, but the path is very real), we will have no idea to 
> split them that which namespace it request to. So I propose add column 
> {{nsid}} to support federation more friendly.  
> {quote}2018-11-27 13:20:30,028 INFO FSNamesystem.audit: allowed=true   
> ugi=hdfs/hostn...@realm.com (auth:KERBEROS)  ip=/10.1.1.2 cmd=getfileinfo 
> src=/path   dst=null        perm=null       proto=rpc       clientName=null
> {quote}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Updated] (HDFS-14109) Improve hdfs auditlog format and support federation friendly

2018-11-28 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao updated HDFS-14109:
---
Status: Patch Available  (was: Open)

submit v0 patch and trigger jenkins.

> Improve hdfs auditlog format and support federation friendly
> 
>
> Key: HDFS-14109
> URL: https://issues.apache.org/jira/browse/HDFS-14109
> Project: Hadoop HDFS
>  Issue Type: Improvement
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-14109.patch
>
>
> The following auditlog format does not well meet requirement for federation 
> arch currently. Since some case we need to aggregate all namespace audit log, 
> if there are some common path request(e.g. /tmp, /user/ etc. some path may 
> not appear in mountTable, but the path is very real), we will have no idea to 
> split them that which namespace it request to. So I propose add column 
> {{nsid}} to support federation more friendly.  
> {quote}2018-11-27 13:20:30,028 INFO FSNamesystem.audit: allowed=true   
> ugi=hdfs/hostn...@realm.com (auth:KERBEROS)  ip=/10.1.1.2 cmd=getfileinfo 
> src=/path   dst=null        perm=null       proto=rpc       clientName=null
> {quote}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Updated] (HDFS-14109) Improve hdfs auditlog format and support federation friendly

2018-11-28 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao updated HDFS-14109:
---
Attachment: HDFS-14109.patch

> Improve hdfs auditlog format and support federation friendly
> 
>
> Key: HDFS-14109
> URL: https://issues.apache.org/jira/browse/HDFS-14109
> Project: Hadoop HDFS
>  Issue Type: Improvement
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-14109.patch
>
>
> The following auditlog format does not well meet requirement for federation 
> arch currently. Since some case we need to aggregate all namespace audit log, 
> if there are some common path request(e.g. /tmp, /user/ etc. some path may 
> not appear in mountTable, but the path is very real), we will have no idea to 
> split them that which namespace it request to. So I propose add column 
> {{nsid}} to support federation more friendly.  
> {quote}2018-11-27 13:20:30,028 INFO FSNamesystem.audit: allowed=true   
> ugi=hdfs/hostn...@realm.com (auth:KERBEROS)  ip=/10.1.1.2 cmd=getfileinfo 
> src=/path   dst=null        perm=null       proto=rpc       clientName=null
> {quote}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Created] (HDFS-14109) Improve hdfs auditlog format and support federation friendly

2018-11-28 Thread He Xiaoqiao (JIRA)
He Xiaoqiao created HDFS-14109:
--

 Summary: Improve hdfs auditlog format and support federation 
friendly
 Key: HDFS-14109
 URL: https://issues.apache.org/jira/browse/HDFS-14109
 Project: Hadoop HDFS
  Issue Type: Improvement
Reporter: He Xiaoqiao
Assignee: He Xiaoqiao


The following auditlog format does not well meet requirement for federation 
arch currently. Since some case we need to aggregate all namespace audit log, 
if there are some common path request(e.g. /tmp, /user/ etc. some path may not 
appear in mountTable, but the path is very real), we will have no idea to split 
them that which namespace it request to. So I propose add column {{nsid}} to 
support federation more friendly.  
{quote}2018-11-27 13:20:30,028 INFO FSNamesystem.audit: allowed=true   
ugi=hdfs/hostn...@realm.com (auth:KERBEROS)  ip=/10.1.1.2 cmd=getfileinfo 
src=/path   dst=null        perm=null       proto=rpc       clientName=null
{quote}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (HDFS-10943) rollEditLog expects empty EditsDoubleBuffer.bufCurrent which is not guaranteed

2018-11-19 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao commented on HDFS-10943:


[~daryn],[~kihwal],[~zhz],[~yzhangal]
Unfortunately, I meet this issue again recently. another interesting note was 
that ioutil of NN keeps on 100 for 2~5min before NN crash (the same observation 
when I first meet this issue), I am not sure it is related with this issue. 
After digging, I do not find that {{FileJournalManager}} will block 
{{JournalSetOutputStream}} or lead to NN crash, and FileJournal is not required 
by default if HA using QJM, in other word, even filejournal do not write/sync 
successfully, NN process will not terminate.
This is just additional information for reference and DO NOT resolve it util 
now.

> rollEditLog expects empty EditsDoubleBuffer.bufCurrent which is not guaranteed
> --
>
> Key: HDFS-10943
> URL: https://issues.apache.org/jira/browse/HDFS-10943
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: Yongjun Zhang
>Priority: Major
>
> Per the following trace stack:
> {code}
> FATAL org.apache.hadoop.hdfs.server.namenode.FSEditLog: Error: finalize log 
> segment 10562075963, 10562174157 failed for required journal 
> (JournalAndStream(mgr=QJM to [0.0.0.1:8485, 0.0.0.2:8485, 0.0.0.3:8485, 
> 0.0.0.4:8485, 0.0.0.5:8485], stream=QuorumOutputStream starting at txid 
> 10562075963))
> java.io.IOException: FSEditStream has 49708 bytes still to be flushed and 
> cannot be closed.
> at 
> org.apache.hadoop.hdfs.server.namenode.EditsDoubleBuffer.close(EditsDoubleBuffer.java:66)
> at 
> org.apache.hadoop.hdfs.qjournal.client.QuorumOutputStream.close(QuorumOutputStream.java:65)
> at 
> org.apache.hadoop.hdfs.server.namenode.JournalSet$JournalAndStream.closeStream(JournalSet.java:115)
> at 
> org.apache.hadoop.hdfs.server.namenode.JournalSet$4.apply(JournalSet.java:235)
> at 
> org.apache.hadoop.hdfs.server.namenode.JournalSet.mapJournalsAndReportErrors(JournalSet.java:393)
> at 
> org.apache.hadoop.hdfs.server.namenode.JournalSet.finalizeLogSegment(JournalSet.java:231)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSEditLog.endCurrentLogSegment(FSEditLog.java:1243)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSEditLog.rollEditLog(FSEditLog.java:1172)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSImage.rollEditLog(FSImage.java:1243)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.rollEditLog(FSNamesystem.java:6437)
> at 
> org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.rollEditLog(NameNodeRpcServer.java:1002)
> at 
> org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolServerSideTranslatorPB.rollEditLog(NamenodeProtocolServerSideTranslatorPB.java:142)
> at 
> org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos$NamenodeProtocolService$2.callBlockingMethod(NamenodeProtocolProtos.java:12025)
> at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:617)
> at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1060)
> at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2086)
> at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2082)
> at java.security.AccessController.doPrivileged(Native Method)
> at javax.security.auth.Subject.doAs(Subject.java:422)
> at 
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1671)
> at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2080)
> 2016-09-23 21:40:59,618 WARN 
> org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager: Aborting 
> QuorumOutputStream starting at txid 10562075963
> {code}
> The exception is from  EditsDoubleBuffer
> {code}
>  public void close() throws IOException {
> Preconditions.checkNotNull(bufCurrent);
> Preconditions.checkNotNull(bufReady);
> int bufSize = bufCurrent.size();
> if (bufSize != 0) {
>   throw new IOException("FSEditStream has " + bufSize
>   + " bytes still to be flushed and cannot be closed.");
> }
> IOUtils.cleanup(null, bufCurrent, bufReady);
> bufCurrent = bufReady = null;
>   }
> {code}
> We can see that FSNamesystem.rollEditLog expects  
> EditsDoubleBuffer.bufCurrent to be empty.
> Edits are recorded via FSEditLog$logSync, which does:
> {code}
>* The data is double-buffered within each edit log implementation so that
>* in-memory writing can occur in parallel with the on-disk writing.
>*
>* Each sync occurs in three steps:
>*   1. synchronized, it swaps the double buffer and sets the isSyncRunning
>*  flag.
>*   2. unsynchronized, it flushes the data to storage
>* 

[jira] [Commented] (HDFS-12749) DN may not send block report to NN after NN restart

2018-11-19 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao commented on HDFS-12749:


[~kihwal],[~xkrogen] do you mind anymore review to push this issue forward.

> DN may not send block report to NN after NN restart
> ---
>
> Key: HDFS-12749
> URL: https://issues.apache.org/jira/browse/HDFS-12749
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode
>Affects Versions: 2.7.1, 2.8.3, 2.7.5, 3.0.0, 2.9.1
>Reporter: TanYuxin
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-12749-branch-2.7.002.patch, 
> HDFS-12749-trunk.003.patch, HDFS-12749-trunk.004.patch, 
> HDFS-12749-trunk.005.patch, HDFS-12749.001.patch
>
>
> Now our cluster have thousands of DN, millions of files and blocks. When NN 
> restart, NN's load is very high.
> After NN restart,DN will call BPServiceActor#reRegister method to register. 
> But register RPC will get a IOException since NN is busy dealing with Block 
> Report.  The exception is caught at BPServiceActor#processCommand.
> Next is the caught IOException:
> {code:java}
> WARN org.apache.hadoop.hdfs.server.datanode.DataNode: Error processing 
> datanode Command
> java.io.IOException: Failed on local exception: java.io.IOException: 
> java.net.SocketTimeoutException: 6 millis timeout while waiting for 
> channel to be ready for read. ch : java.nio.channels.SocketChannel[connected 
> local=/DataNode_IP:Port remote=NameNode_Host/IP:Port]; Host Details : local 
> host is: "DataNode_Host/Datanode_IP"; destination host is: 
> "NameNode_Host":Port;
> at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:773)
> at org.apache.hadoop.ipc.Client.call(Client.java:1474)
> at org.apache.hadoop.ipc.Client.call(Client.java:1407)
> at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:229)
> at com.sun.proxy.$Proxy13.registerDatanode(Unknown Source)
> at 
> org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB.registerDatanode(DatanodeProtocolClientSideTranslatorPB.java:126)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.register(BPServiceActor.java:793)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.reRegister(BPServiceActor.java:926)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPOfferService.processCommandFromActor(BPOfferService.java:604)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.processCommand(BPServiceActor.java:898)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.offerService(BPServiceActor.java:711)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.run(BPServiceActor.java:864)
> at java.lang.Thread.run(Thread.java:745)
> {code}
> The un-catched IOException breaks BPServiceActor#register, and the Block 
> Report can not be sent immediately. 
> {code}
>   /**
>* Register one bp with the corresponding NameNode
>* 
>* The bpDatanode needs to register with the namenode on startup in order
>* 1) to report which storage it is serving now and 
>* 2) to receive a registrationID
>*  
>* issued by the namenode to recognize registered datanodes.
>* 
>* @param nsInfo current NamespaceInfo
>* @see FSNamesystem#registerDatanode(DatanodeRegistration)
>* @throws IOException
>*/
>   void register(NamespaceInfo nsInfo) throws IOException {
> // The handshake() phase loaded the block pool storage
> // off disk - so update the bpRegistration object from that info
> DatanodeRegistration newBpRegistration = bpos.createRegistration();
> LOG.info(this + " beginning handshake with NN");
> while (shouldRun()) {
>   try {
> // Use returned registration from namenode with updated fields
> newBpRegistration = bpNamenode.registerDatanode(newBpRegistration);
> newBpRegistration.setNamespaceInfo(nsInfo);
> bpRegistration = newBpRegistration;
> break;
>   } catch(EOFException e) {  // namenode might have just restarted
> LOG.info("Problem connecting to server: " + nnAddr + " :"
> + e.getLocalizedMessage());
> sleepAndLogInterrupts(1000, "connecting to server");
>   } catch(SocketTimeoutException e) {  // namenode is busy
> LOG.info("Problem connecting to server: " + nnAddr);
> sleepAndLogInterrupts(1000, "connecting to server");
>   }
> }
> 
> LOG.info("Block pool " + this + " successfully registered with NN");
> bpos.registrationSucceeded(this, bpRegistration);
> // random short delay - helps scatter the BR from all DNs
> 

[jira] [Commented] (HDFS-12862) CacheDirective may invalidata,when NN restart or make a transition to Active.

2018-11-19 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao commented on HDFS-12862:


LGTM for [^HDFS-12862-trunk.003.patch] , ping [~daryn],[~jojochuang] Do you 
mind another review?

> CacheDirective may invalidata,when NN restart or make a transition to Active.
> -
>
> Key: HDFS-12862
> URL: https://issues.apache.org/jira/browse/HDFS-12862
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: caching, hdfs
>Affects Versions: 2.7.1
> Environment: 
>Reporter: Wang XL
>Priority: Major
>  Labels: patch
> Attachments: HDFS-12862-branch-2.7.1.001.patch, 
> HDFS-12862-trunk.002.patch, HDFS-12862-trunk.003.patch
>
>
> The logic in FSNDNCacheOp#modifyCacheDirective is not correct.  when modify 
> cacheDirective,the expiration in directive may be a relative expiryTime, and 
> EditLog will serial a relative expiry time.
> {code:java}
> // Some comments here
> static void modifyCacheDirective(
>   FSNamesystem fsn, CacheManager cacheManager, CacheDirectiveInfo 
> directive,
>   EnumSet flags, boolean logRetryCache) throws IOException {
> final FSPermissionChecker pc = getFsPermissionChecker(fsn);
> cacheManager.modifyDirective(directive, pc, flags);
> fsn.getEditLog().logModifyCacheDirectiveInfo(directive, logRetryCache);
>   }
> {code}
> But when SBN replay the log ,it will invoke 
> FSImageSerialization#readCacheDirectiveInfo  as a absolute expiryTime.It will 
> result in the inconsistency .
> {code:java}
>   public static CacheDirectiveInfo readCacheDirectiveInfo(DataInput in)
>   throws IOException {
> CacheDirectiveInfo.Builder builder =
> new CacheDirectiveInfo.Builder();
> builder.setId(readLong(in));
> int flags = in.readInt();
> if ((flags & 0x1) != 0) {
>   builder.setPath(new Path(readString(in)));
> }
> if ((flags & 0x2) != 0) {
>   builder.setReplication(readShort(in));
> }
> if ((flags & 0x4) != 0) {
>   builder.setPool(readString(in));
> }
> if ((flags & 0x8) != 0) {
>   builder.setExpiration(
>   CacheDirectiveInfo.Expiration.newAbsolute(readLong(in)));
> }
> if ((flags & ~0xF) != 0) {
>   throw new IOException("unknown flags set in " +
>   "ModifyCacheDirectiveInfoOp: " + flags);
> }
> return builder.build();
>   }
> {code}
> In other words, fsn.getEditLog().logModifyCacheDirectiveInfo(directive, 
> logRetryCache)  may serial a relative expiry time,But  
> builder.setExpiration(CacheDirectiveInfo.Expiration.newAbsolute(readLong(in)))
>read it as a absolute expiryTime.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (HDFS-13915) replace datanode failed because of NameNodeRpcServer#getAdditionalDatanode returning excessive datanodeInfo

2018-09-14 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao commented on HDFS-13915:


hi [~yangjiandan], I wonder which version do you encounter this issue.

> replace datanode failed because of  NameNodeRpcServer#getAdditionalDatanode 
> returning excessive datanodeInfo
> 
>
> Key: HDFS-13915
> URL: https://issues.apache.org/jira/browse/HDFS-13915
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: hdfs
> Environment: 
>Reporter: Jiandan Yang 
>Priority: Major
>
> Consider following situation:
> 1. create a file with ALLSSD policy
> 2. return [SSD,SSD,DISK] due to lack of SSD space
> 3. client call NameNodeRpcServer#getAdditionalDatanode when recovering write 
> pipeline and replacing bad datanode
> 4. BlockPlacementPolicyDefault#chooseTarget will call 
> StoragePolicy#chooseStorageTypes(3, [SSD,DISK], none, false), but 
> chooseStorageTypes return [SSD,SSD]
> {code:java}
>   @Test
>   public void testAllSSDFallbackAndNonNewBlock() {
> final BlockStoragePolicy allSSD = POLICY_SUITE.getPolicy(ALLSSD);
> List storageTypes = allSSD.chooseStorageTypes((short) 3,
> Arrays.asList(StorageType.DISK, StorageType.SSD),
> EnumSet.noneOf(StorageType.class), false);
> assertEquals(2, storageTypes.size());
> assertEquals(StorageType.SSD, storageTypes.get(0));
> assertEquals(StorageType.SSD, storageTypes.get(1));
>   }
> {code}
> 5. do numOfReplicas = requiredStorageTypes.size() and numOfReplicas is set to 
> 2 and choose additional two datanodes
> 6. BlockPlacementPolicyDefault#chooseTarget return four datanodes to client
> 7. DataStreamer#findNewDatanode find nodes.length != original.length + 1  and 
> throw IOException, and finally lead to write failed
> {code:java}
> private int findNewDatanode(final DatanodeInfo[] original
>   ) throws IOException {
> if (nodes.length != original.length + 1) {
>   throw new IOException(
>   "Failed to replace a bad datanode on the existing pipeline "
>   + "due to no more good datanodes being available to try. "
>   + "(Nodes: current=" + Arrays.asList(nodes)
>   + ", original=" + Arrays.asList(original) + "). "
>   + "The current failed datanode replacement policy is "
>   + dfsClient.dtpReplaceDatanodeOnFailure
>   + ", and a client may configure this via '"
>   + BlockWrite.ReplaceDatanodeOnFailure.POLICY_KEY
>   + "' in its configuration.");
> }
> for(int i = 0; i < nodes.length; i++) {
>   int j = 0;
>   for(; j < original.length && !nodes[i].equals(original[j]); j++);
>   if (j == original.length) {
> return i;
>   }
> }
> throw new IOException("Failed: new datanode not found: nodes="
> + Arrays.asList(nodes) + ", original=" + Arrays.asList(original));
>   }
> {code}
> client warn logs is:
>  {code:java}
> WARN [DataStreamer for file 
> /home/yarn/opensearch/in/data/120141286/0_65535/table/ucs_process/MANIFEST-093545
>  block BP-1742758844-11.138.8.184-1483707043031:blk_7086344902_6012765313] 
> org.apache.hadoop.hdfs.DFSClient: DataStreamer Exception
> java.io.IOException: Failed to replace a bad datanode on the existing 
> pipeline due to no more good datanodes being available to try. (Nodes: 
> current=[DatanodeInfoWithStorage[11.138.5.4:50010,DS-04826cfc-1885-4213-a58b-8606845c5c42,SSD],
>  
> DatanodeInfoWithStorage[11.138.5.9:50010,DS-f6d8eb8b-2550-474b-a692-c991d7a6f6b3,SSD],
>  
> DatanodeInfoWithStorage[11.138.5.153:50010,DS-f5d77ca0-6fe3-4523-8ca8-5af975f845b6,SSD],
>  
> DatanodeInfoWithStorage[11.138.9.156:50010,DS-0d15ea12-1bad--84f7-1a4917a1e194,DISK]],
>  
> original=[DatanodeInfoWithStorage[11.138.5.4:50010,DS-04826cfc-1885-4213-a58b-8606845c5c42,SSD],
>  
> DatanodeInfoWithStorage[11.138.9.156:50010,DS-0d15ea12-1bad--84f7-1a4917a1e194,DISK]]).
>  The current failed datanode replacement policy is DEFAULT, and a client may 
> configure this via 
> 'dfs.client.block.write.replace-datanode-on-failure.policy' in its 
> configuration.
> {code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (HDFS-13833) Failed to choose from local rack (location = /default); the second replica is not found, retry choosing ramdomly

2018-08-17 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao commented on HDFS-13833:


IIUC I think inconsistent {{stats}} is the main reason about this corner case:
a. {{chooseTarget}} at {{BlockPlacementPolicyDefault}} is not under global 
lock, and it can been invoke with {{sendHeartbeat}} at same time.
b. when {{chooseTarget}} has choose the target node, it's stat (of course 
include {{load}}) is immutable, because {{DatanodeStorageInfo}} instances is 
created by {{DatanodeDescriptor#getStorageInfos}}.
c. On the other hand, when compare load about target node to average load (* 
factor) of the whole cluster, it get the real time value.
d. if update heartbeat between step b and step c, and the latest load of the 
only node is zero unfortunately, chooseTarget will fail, and exception stack 
show as depict above. 
e. the good news is the average load will update when next heartbeat at the 
latest, and chooseTarget will work as expect. 

{code:java}
// check the communication traffic of the target machine
if (considerLoad) {
  final double maxLoad = 2.0 * stats.getInServiceXceiverAverage();
  final int nodeLoad = node.getXceiverCount();
  if (nodeLoad > maxLoad) {
logNodeIsNotChosen(storage, "the node is too busy (load: " + nodeLoad
+ " > " + maxLoad + ") ");
return false;
  }
}
{code}
my suggestion is try to add more datanode (for instance: 3) in cluster.
[~jojochuang],[~xiaochen] branch trunk also has this problem.

> Failed to choose from local rack (location = /default); the second replica is 
> not found, retry choosing ramdomly
> 
>
> Key: HDFS-13833
> URL: https://issues.apache.org/jira/browse/HDFS-13833
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: Henrique Barros
>Priority: Critical
>
> I'm having a random problem with blocks replication with Hadoop 
> 2.6.0-cdh5.15.0
> With Cloudera CDH-5.15.0-1.cdh5.15.0.p0.21
>  
> In my case we are getting this error very randomly (after some hours) and 
> with only one Datanode (for now, we are trying this cloudera cluster for a 
> POC)
> Here is the Log.
> {code:java}
> Choosing random from 1 available nodes on node /default, scope=/default, 
> excludedScope=null, excludeNodes=[]
> 2:38:20.527 PMDEBUG   NetworkTopology 
> Choosing random from 0 available nodes on node /default, scope=/default, 
> excludedScope=null, excludeNodes=[192.168.220.53:50010]
> 2:38:20.527 PMDEBUG   NetworkTopology 
> chooseRandom returning null
> 2:38:20.527 PMDEBUG   BlockPlacementPolicy
> [
> Node /default/192.168.220.53:50010 [
>   Datanode 192.168.220.53:50010 is not chosen since the node is too busy 
> (load: 8 > 0.0).
> 2:38:20.527 PMDEBUG   NetworkTopology 
> chooseRandom returning 192.168.220.53:50010
> 2:38:20.527 PMINFOBlockPlacementPolicy
> Not enough replicas was chosen. Reason:{NODE_TOO_BUSY=1}
> 2:38:20.527 PMDEBUG   StateChange 
> closeFile: 
> /mobi.me/development/apps/flink/checkpoints/a5a6806866c1640660924ea1453cbe34/chk-2118/eef8bff6-75a9-43c1-ae93-4b1a9ca31ad9
>  with 1 blocks is persisted to the file system
> 2:38:20.527 PMDEBUG   StateChange 
> *BLOCK* NameNode.addBlock: file 
> /mobi.me/development/apps/flink/checkpoints/a5a6806866c1640660924ea1453cbe34/chk-2118/1cfe900d-6f45-4b55-baaa-73c02ace2660
>  fileId=129628869 for DFSClient_NONMAPREDUCE_467616914_65
> 2:38:20.527 PMDEBUG   BlockPlacementPolicy
> Failed to choose from local rack (location = /default); the second replica is 
> not found, retry choosing ramdomly
> org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy$NotEnoughReplicasException:
>  
>   at 
> org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault.chooseRandom(BlockPlacementPolicyDefault.java:784)
>   at 
> org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault.chooseRandom(BlockPlacementPolicyDefault.java:694)
>   at 
> org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault.chooseLocalRack(BlockPlacementPolicyDefault.java:601)
>   at 
> org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault.chooseLocalStorage(BlockPlacementPolicyDefault.java:561)
>   at 
> org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault.chooseTargetInOrder(BlockPlacementPolicyDefault.java:464)
>   at 
> org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault.chooseTarget(BlockPlacementPolicyDefault.java:395)
>   at 
> 

[jira] [Commented] (HDFS-12862) CacheDirective may invalidata,when NN restart or make a transition to Active.

2018-08-14 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao commented on HDFS-12862:


Thanks [~Wang XL], for reporting and working on this, some minor comments:
1. jenknis compile failed, maybe not rebase on the correct branch;
2. some whitespace issues reported, please fix them
ping [~drankye] [~jojochuang], would you mind giving a look? thanks.

> CacheDirective may invalidata,when NN restart or make a transition to Active.
> -
>
> Key: HDFS-12862
> URL: https://issues.apache.org/jira/browse/HDFS-12862
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: caching, hdfs
>Affects Versions: 2.7.1
> Environment: 
>Reporter: Wang XL
>Priority: Major
>  Labels: patch
> Attachments: HDFS-12862-branch-2.7.1.001.patch
>
>
> The logic in FSNDNCacheOp#modifyCacheDirective is not correct.  when modify 
> cacheDirective,the expiration in directive may be a relative expiryTime, and 
> EditLog will serial a relative expiry time.
> {code:java}
> // Some comments here
> static void modifyCacheDirective(
>   FSNamesystem fsn, CacheManager cacheManager, CacheDirectiveInfo 
> directive,
>   EnumSet flags, boolean logRetryCache) throws IOException {
> final FSPermissionChecker pc = getFsPermissionChecker(fsn);
> cacheManager.modifyDirective(directive, pc, flags);
> fsn.getEditLog().logModifyCacheDirectiveInfo(directive, logRetryCache);
>   }
> {code}
> But when SBN replay the log ,it will invoke 
> FSImageSerialization#readCacheDirectiveInfo  as a absolute expiryTime.It will 
> result in the inconsistency .
> {code:java}
>   public static CacheDirectiveInfo readCacheDirectiveInfo(DataInput in)
>   throws IOException {
> CacheDirectiveInfo.Builder builder =
> new CacheDirectiveInfo.Builder();
> builder.setId(readLong(in));
> int flags = in.readInt();
> if ((flags & 0x1) != 0) {
>   builder.setPath(new Path(readString(in)));
> }
> if ((flags & 0x2) != 0) {
>   builder.setReplication(readShort(in));
> }
> if ((flags & 0x4) != 0) {
>   builder.setPool(readString(in));
> }
> if ((flags & 0x8) != 0) {
>   builder.setExpiration(
>   CacheDirectiveInfo.Expiration.newAbsolute(readLong(in)));
> }
> if ((flags & ~0xF) != 0) {
>   throw new IOException("unknown flags set in " +
>   "ModifyCacheDirectiveInfoOp: " + flags);
> }
> return builder.build();
>   }
> {code}
> In other words, fsn.getEditLog().logModifyCacheDirectiveInfo(directive, 
> logRetryCache)  may serial a relative expiry time,But  
> builder.setExpiration(CacheDirectiveInfo.Expiration.newAbsolute(readLong(in)))
>read it as a absolute expiryTime.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Comment Edited] (HDFS-13668) FSPermissionChecker may throws AIOOE when check if inode has permission

2018-08-10 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao edited comment on HDFS-13668 at 8/10/18 11:00 AM:
--

{quote}We usually call miniDFS.getFileSystem() instead. But this is entirely 
valid.
{quote}
v003 update about getting FileSystem instance in {{TestINodeAttributeProvider}} 
following [~jojochuang]'s suggestion, and sorry for missing this ungraceful 
usage yesterday.


was (Author: hexiaoqiao):
{quote}We usually call miniDFS.getFileSystem() instead. But this is entirely 
valid.
{quote}
update about getting FileSystem instance in{{TestINodeAttributeProvider}} 
following [~jojochuang]'s suggestion, and sorry for missing this ungraceful 
usage yesterday.

> FSPermissionChecker may throws AIOOE when check if inode has permission
> ---
>
> Key: HDFS-13668
> URL: https://issues.apache.org/jira/browse/HDFS-13668
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: namenode
>Affects Versions: 3.1.0, 2.10.0, 2.7.7
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-13668-trunk.001.patch, HDFS-13668-trunk.002.patch, 
> HDFS-13668-trunk.003.patch
>
>
> {{FSPermissionChecker}} may throw {{ArrayIndexOutOfBoundsException:0}} when 
> check if has permission, since it only check inode's {{aclFeature}} if null 
> or not but not check it's entry size. When it meets {{aclFeature}} not null 
> but it's entry size equal to 0, it will throw AIOOE.
> {code:java}
> private boolean hasPermission(INodeAttributes inode, FsAction access) {
>   ..
>   final AclFeature aclFeature = inode.getAclFeature();
>   if (aclFeature != null) {
> // It's possible that the inode has a default ACL but no access ACL.
> int firstEntry = aclFeature.getEntryAt(0);
> if (AclEntryStatusFormat.getScope(firstEntry) == AclEntryScope.ACCESS) {
>   return hasAclPermission(inode, access, mode, aclFeature);
> }
>   }
>   ..
> }
> {code}
> Actually if use default {{INodeAttributeProvider}}, it can ensure that when 
> {{inode}}'s aclFeature is not null and it's entry size also will be greater 
> than 0, but {{INodeAttributeProvider}} is a public interface, we could not 
> ensure external implement (e.g. Apache Sentry, Apache Ranger) also has the 
> similar constraint. 



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (HDFS-13668) FSPermissionChecker may throws AIOOE when check if inode has permission

2018-08-10 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao commented on HDFS-13668:


{quote}We usually call miniDFS.getFileSystem() instead. But this is entirely 
valid.
{quote}
update about getting FileSystem instance in{{TestINodeAttributeProvider}} 
following [~jojochuang]'s suggestion, and sorry for missing this ungraceful 
usage yesterday.

> FSPermissionChecker may throws AIOOE when check if inode has permission
> ---
>
> Key: HDFS-13668
> URL: https://issues.apache.org/jira/browse/HDFS-13668
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: namenode
>Affects Versions: 3.1.0, 2.10.0, 2.7.7
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-13668-trunk.001.patch, HDFS-13668-trunk.002.patch, 
> HDFS-13668-trunk.003.patch
>
>
> {{FSPermissionChecker}} may throw {{ArrayIndexOutOfBoundsException:0}} when 
> check if has permission, since it only check inode's {{aclFeature}} if null 
> or not but not check it's entry size. When it meets {{aclFeature}} not null 
> but it's entry size equal to 0, it will throw AIOOE.
> {code:java}
> private boolean hasPermission(INodeAttributes inode, FsAction access) {
>   ..
>   final AclFeature aclFeature = inode.getAclFeature();
>   if (aclFeature != null) {
> // It's possible that the inode has a default ACL but no access ACL.
> int firstEntry = aclFeature.getEntryAt(0);
> if (AclEntryStatusFormat.getScope(firstEntry) == AclEntryScope.ACCESS) {
>   return hasAclPermission(inode, access, mode, aclFeature);
> }
>   }
>   ..
> }
> {code}
> Actually if use default {{INodeAttributeProvider}}, it can ensure that when 
> {{inode}}'s aclFeature is not null and it's entry size also will be greater 
> than 0, but {{INodeAttributeProvider}} is a public interface, we could not 
> ensure external implement (e.g. Apache Sentry, Apache Ranger) also has the 
> similar constraint. 



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Updated] (HDFS-13668) FSPermissionChecker may throws AIOOE when check if inode has permission

2018-08-10 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao updated HDFS-13668:
---
Attachment: HDFS-13668-trunk.003.patch

> FSPermissionChecker may throws AIOOE when check if inode has permission
> ---
>
> Key: HDFS-13668
> URL: https://issues.apache.org/jira/browse/HDFS-13668
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: namenode
>Affects Versions: 3.1.0, 2.10.0, 2.7.7
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-13668-trunk.001.patch, HDFS-13668-trunk.002.patch, 
> HDFS-13668-trunk.003.patch
>
>
> {{FSPermissionChecker}} may throw {{ArrayIndexOutOfBoundsException:0}} when 
> check if has permission, since it only check inode's {{aclFeature}} if null 
> or not but not check it's entry size. When it meets {{aclFeature}} not null 
> but it's entry size equal to 0, it will throw AIOOE.
> {code:java}
> private boolean hasPermission(INodeAttributes inode, FsAction access) {
>   ..
>   final AclFeature aclFeature = inode.getAclFeature();
>   if (aclFeature != null) {
> // It's possible that the inode has a default ACL but no access ACL.
> int firstEntry = aclFeature.getEntryAt(0);
> if (AclEntryStatusFormat.getScope(firstEntry) == AclEntryScope.ACCESS) {
>   return hasAclPermission(inode, access, mode, aclFeature);
> }
>   }
>   ..
> }
> {code}
> Actually if use default {{INodeAttributeProvider}}, it can ensure that when 
> {{inode}}'s aclFeature is not null and it's entry size also will be greater 
> than 0, but {{INodeAttributeProvider}} is a public interface, we could not 
> ensure external implement (e.g. Apache Sentry, Apache Ranger) also has the 
> similar constraint. 



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (HDFS-13668) FSPermissionChecker may throws AIOOE when check if inode has permission

2018-08-10 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao commented on HDFS-13668:


ping [~drankye],[~jojochuang],[~shashikant], any further more suggestions?

> FSPermissionChecker may throws AIOOE when check if inode has permission
> ---
>
> Key: HDFS-13668
> URL: https://issues.apache.org/jira/browse/HDFS-13668
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: namenode
>Affects Versions: 3.1.0, 2.10.0, 2.7.7
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-13668-trunk.001.patch, HDFS-13668-trunk.002.patch
>
>
> {{FSPermissionChecker}} may throw {{ArrayIndexOutOfBoundsException:0}} when 
> check if has permission, since it only check inode's {{aclFeature}} if null 
> or not but not check it's entry size. When it meets {{aclFeature}} not null 
> but it's entry size equal to 0, it will throw AIOOE.
> {code:java}
> private boolean hasPermission(INodeAttributes inode, FsAction access) {
>   ..
>   final AclFeature aclFeature = inode.getAclFeature();
>   if (aclFeature != null) {
> // It's possible that the inode has a default ACL but no access ACL.
> int firstEntry = aclFeature.getEntryAt(0);
> if (AclEntryStatusFormat.getScope(firstEntry) == AclEntryScope.ACCESS) {
>   return hasAclPermission(inode, access, mode, aclFeature);
> }
>   }
>   ..
> }
> {code}
> Actually if use default {{INodeAttributeProvider}}, it can ensure that when 
> {{inode}}'s aclFeature is not null and it's entry size also will be greater 
> than 0, but {{INodeAttributeProvider}} is a public interface, we could not 
> ensure external implement (e.g. Apache Sentry, Apache Ranger) also has the 
> similar constraint. 



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (HDFS-13668) FSPermissionChecker may throws AIOOE when check if inode has permission

2018-08-09 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao commented on HDFS-13668:


Thanks [~jojochuang]
{quote}We should probably revisit ACL checks in NameNode at some point and make 
them more defensive.{quote}
Yes, that's a good point, actually I have checked {{AclStorage}},{{INode}} and 
its subclass, also {{FSPermissionChecker}} and I do not find any probably AIOOE.

{quote}We usually call miniDFS.getFileSystem() instead. But this is entirely 
valid.
{quote}
Sorry don't get the point.

> FSPermissionChecker may throws AIOOE when check if inode has permission
> ---
>
> Key: HDFS-13668
> URL: https://issues.apache.org/jira/browse/HDFS-13668
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: namenode
>Affects Versions: 3.1.0, 2.10.0, 2.7.7
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-13668-trunk.001.patch, HDFS-13668-trunk.002.patch
>
>
> {{FSPermissionChecker}} may throw {{ArrayIndexOutOfBoundsException:0}} when 
> check if has permission, since it only check inode's {{aclFeature}} if null 
> or not but not check it's entry size. When it meets {{aclFeature}} not null 
> but it's entry size equal to 0, it will throw AIOOE.
> {code:java}
> private boolean hasPermission(INodeAttributes inode, FsAction access) {
>   ..
>   final AclFeature aclFeature = inode.getAclFeature();
>   if (aclFeature != null) {
> // It's possible that the inode has a default ACL but no access ACL.
> int firstEntry = aclFeature.getEntryAt(0);
> if (AclEntryStatusFormat.getScope(firstEntry) == AclEntryScope.ACCESS) {
>   return hasAclPermission(inode, access, mode, aclFeature);
> }
>   }
>   ..
> }
> {code}
> Actually if use default {{INodeAttributeProvider}}, it can ensure that when 
> {{inode}}'s aclFeature is not null and it's entry size also will be greater 
> than 0, but {{INodeAttributeProvider}} is a public interface, we could not 
> ensure external implement (e.g. Apache Sentry, Apache Ranger) also has the 
> similar constraint. 



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (HDFS-13668) FSPermissionChecker may throws AIOOE when check if inode has permission

2018-08-09 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao commented on HDFS-13668:


[~drankye],[~shashikant],
check failed UT(TestSnapshotFileLength, 
TestDFSStripedOutputStreamWithFailureWithRandomECPolicy) and test locally, it 
seems to work fine and I am sure it is not relate to this patch.

> FSPermissionChecker may throws AIOOE when check if inode has permission
> ---
>
> Key: HDFS-13668
> URL: https://issues.apache.org/jira/browse/HDFS-13668
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: namenode
>Affects Versions: 3.1.0, 2.10.0, 2.7.7
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-13668-trunk.001.patch, HDFS-13668-trunk.002.patch
>
>
> {{FSPermissionChecker}} may throw {{ArrayIndexOutOfBoundsException:0}} when 
> check if has permission, since it only check inode's {{aclFeature}} if null 
> or not but not check it's entry size. When it meets {{aclFeature}} not null 
> but it's entry size equal to 0, it will throw AIOOE.
> {code:java}
> private boolean hasPermission(INodeAttributes inode, FsAction access) {
>   ..
>   final AclFeature aclFeature = inode.getAclFeature();
>   if (aclFeature != null) {
> // It's possible that the inode has a default ACL but no access ACL.
> int firstEntry = aclFeature.getEntryAt(0);
> if (AclEntryStatusFormat.getScope(firstEntry) == AclEntryScope.ACCESS) {
>   return hasAclPermission(inode, access, mode, aclFeature);
> }
>   }
>   ..
> }
> {code}
> Actually if use default {{INodeAttributeProvider}}, it can ensure that when 
> {{inode}}'s aclFeature is not null and it's entry size also will be greater 
> than 0, but {{INodeAttributeProvider}} is a public interface, we could not 
> ensure external implement (e.g. Apache Sentry, Apache Ranger) also has the 
> similar constraint. 



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (HDFS-13668) FSPermissionChecker may throws AIOOE when check if inode has permission

2018-08-08 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao commented on HDFS-13668:


Thanks [~drankye],[~shashikant] for your suggestions, submit v002 following 
your advice and trigger jenkins.

> FSPermissionChecker may throws AIOOE when check if inode has permission
> ---
>
> Key: HDFS-13668
> URL: https://issues.apache.org/jira/browse/HDFS-13668
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: namenode
>Affects Versions: 3.1.0, 2.10.0, 2.7.7
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-13668-trunk.001.patch, HDFS-13668-trunk.002.patch
>
>
> {{FSPermissionChecker}} may throw {{ArrayIndexOutOfBoundsException:0}} when 
> check if has permission, since it only check inode's {{aclFeature}} if null 
> or not but not check it's entry size. When it meets {{aclFeature}} not null 
> but it's entry size equal to 0, it will throw AIOOE.
> {code:java}
> private boolean hasPermission(INodeAttributes inode, FsAction access) {
>   ..
>   final AclFeature aclFeature = inode.getAclFeature();
>   if (aclFeature != null) {
> // It's possible that the inode has a default ACL but no access ACL.
> int firstEntry = aclFeature.getEntryAt(0);
> if (AclEntryStatusFormat.getScope(firstEntry) == AclEntryScope.ACCESS) {
>   return hasAclPermission(inode, access, mode, aclFeature);
> }
>   }
>   ..
> }
> {code}
> Actually if use default {{INodeAttributeProvider}}, it can ensure that when 
> {{inode}}'s aclFeature is not null and it's entry size also will be greater 
> than 0, but {{INodeAttributeProvider}} is a public interface, we could not 
> ensure external implement (e.g. Apache Sentry, Apache Ranger) also has the 
> similar constraint. 



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Updated] (HDFS-13668) FSPermissionChecker may throws AIOOE when check if inode has permission

2018-08-08 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao updated HDFS-13668:
---
Attachment: HDFS-13668-trunk.002.patch

> FSPermissionChecker may throws AIOOE when check if inode has permission
> ---
>
> Key: HDFS-13668
> URL: https://issues.apache.org/jira/browse/HDFS-13668
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: namenode
>Affects Versions: 3.1.0, 2.10.0, 2.7.7
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-13668-trunk.001.patch, HDFS-13668-trunk.002.patch
>
>
> {{FSPermissionChecker}} may throw {{ArrayIndexOutOfBoundsException:0}} when 
> check if has permission, since it only check inode's {{aclFeature}} if null 
> or not but not check it's entry size. When it meets {{aclFeature}} not null 
> but it's entry size equal to 0, it will throw AIOOE.
> {code:java}
> private boolean hasPermission(INodeAttributes inode, FsAction access) {
>   ..
>   final AclFeature aclFeature = inode.getAclFeature();
>   if (aclFeature != null) {
> // It's possible that the inode has a default ACL but no access ACL.
> int firstEntry = aclFeature.getEntryAt(0);
> if (AclEntryStatusFormat.getScope(firstEntry) == AclEntryScope.ACCESS) {
>   return hasAclPermission(inode, access, mode, aclFeature);
> }
>   }
>   ..
> }
> {code}
> Actually if use default {{INodeAttributeProvider}}, it can ensure that when 
> {{inode}}'s aclFeature is not null and it's entry size also will be greater 
> than 0, but {{INodeAttributeProvider}} is a public interface, we could not 
> ensure external implement (e.g. Apache Sentry, Apache Ranger) also has the 
> similar constraint. 



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Created] (HDFS-13760) improve ZKFC fencing action when network of ZKFC interrupt

2018-07-23 Thread He Xiaoqiao (JIRA)
He Xiaoqiao created HDFS-13760:
--

 Summary: improve ZKFC fencing action when network of ZKFC interrupt
 Key: HDFS-13760
 URL: https://issues.apache.org/jira/browse/HDFS-13760
 Project: Hadoop HDFS
  Issue Type: Improvement
  Components: ha
Reporter: He Xiaoqiao


when host of Active NameNode & ZKFC meet network fault for quite a time, HDFS 
will be not available since ZKFC located on Standby NameNode will never ssh 
fence success due to it could not ssh to Active NameNode. In such situation, 
for Client, it could not connect to Active NameNode, then failover to Standby 
but it could not provide READ/WRITE.
{code:xml}
2018-07-23 15:57:10,836 INFO org.apache.hadoop.ipc.Client: Retrying connect to 
server: rz-data-hdp-nn14.rz.sankuai.com/10.16.70.34:8060. Already tried 40 
time(s); maxRetries=45
2018-07-23 15:57:30,856 INFO org.apache.hadoop.ipc.Client: Retrying connect to 
server: rz-data-hdp-nn14.rz.sankuai.com/10.16.70.34:8060. Already tried 41 
time(s); maxRetries=45
2018-07-23 15:57:50,872 INFO org.apache.hadoop.ipc.Client: Retrying connect to 
server: rz-data-hdp-nn14.rz.sankuai.com/10.16.70.34:8060. Already tried 42 
time(s); maxRetries=45
2018-07-23 15:58:10,892 INFO org.apache.hadoop.ipc.Client: Retrying connect to 
server: rz-data-hdp-nn14.rz.sankuai.com/10.16.70.34:8060. Already tried 43 
time(s); maxRetries=45
2018-07-23 15:58:30,912 INFO org.apache.hadoop.ipc.Client: Retrying connect to 
server: rz-data-hdp-nn14.rz.sankuai.com/10.16.70.34:8060. Already tried 44 
time(s); maxRetries=45
2018-07-23 15:58:50,933 INFO org.apache.hadoop.ha.ZKFailoverController: get old 
active state exception: org.apache.hadoop.net.ConnectTimeoutException: 2 
millis timeout while waiting for channel to be 
ready for connect. ch : java.nio.channels.SocketChannel[connection-pending 
local=/ip:port remote=hostname]
2018-07-23 15:58:50,933 INFO org.apache.hadoop.ha.ActiveStandbyElector: old 
active is not healthy. need to create znode
2018-07-23 15:58:50,933 INFO org.apache.hadoop.ha.ActiveStandbyElector: Elector 
callbacks for NameNode at standbynn start create node, now time: 
45179010079342817
2018-07-23 15:58:50,936 INFO org.apache.hadoop.ha.ActiveStandbyElector: 
CreateNode result: 0 code:OK for path: /hadoop-ha/ns/ActiveStandbyElectorLock 
connectionState: CONNECTED  for elector id=469098346 
appData=0a07727a2d6e6e313312046e6e31331a1f727a2d646174612d6864702d6e6e31332e727a2e73616e6b7561692e636f6d20e83e28d33e
 cb=Elector callbacks for NameNode at standbynamenode
2018-07-23 15:58:50,936 INFO org.apache.hadoop.ha.ActiveStandbyElector: 
Checking for any old active which needs to be fenced...
2018-07-23 15:58:50,938 INFO org.apache.hadoop.ha.ActiveStandbyElector: Old 
node exists: 
0a07727a2d6e6e313312046e6e31341a1f727a2d646174612d6864702d6e6e31342e727a2e73616e6b7561692e636f6d20e83e28d33e
2018-07-23 15:58:50,939 INFO org.apache.hadoop.ha.ZKFailoverController: Should 
fence: NameNode at activenamenode
2018-07-23 15:59:10,960 INFO org.apache.hadoop.ipc.Client: Retrying connect to 
server: activenamenode. Already tried 0 time(s); maxRetries=1
2018-07-23 15:59:30,980 WARN org.apache.hadoop.ha.FailoverController: Unable to 
gracefully make NameNode at activenamenode standby (unable to connect)
org.apache.hadoop.net.ConnectTimeoutException: Call From standbynamenode to 
activenamenode failed on socket timeout exception: 
org.apache.hadoop.net.ConnectTimeoutException: 2 millis timeout while 
waiting for channel to be ready for connect. ch : 
java.nio.channels.SocketChannel[connection-pending local=ip:port 
remote=activenamenode]; For more details see:  
http://wiki.apache.org/hadoop/SocketTimeout
{code}

I propose that when Active NameNode meet network fault, ZKFC force this 
NameNode to become Standby, and another ZKFC could hold the ZNode for election 
and transition other NameNode to Active even when ssh fence fail.

There is no available patch now, and I am very welcome to hear some suggestion.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (HDFS-13473) DataNode update BlockKeys using mode PULL rather than PUSH from NameNode

2018-06-18 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao commented on HDFS-13473:


upload v005 patch to fix bug about auto-update block token keys which trigger 
by HeartbeatManager#Monitor, and failed ut 
{{hadoop.hdfs.server.namenode.ha.TestFailoverWithBlockTokensEnabled
}} is related to this bug.
I checked another failed ut {{hadoop.hdfs.TestEncryptedTransfer}} and tested 
locally, it seems to work fine and might not relate to this patch.
could anyone help to double check at convenience?

> DataNode update BlockKeys using mode PULL rather than PUSH from NameNode
> 
>
> Key: HDFS-13473
> URL: https://issues.apache.org/jira/browse/HDFS-13473
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: datanode
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-13473-trunk.001.patch, HDFS-13473-trunk.002.patch, 
> HDFS-13473-trunk.003.patch, HDFS-13473-trunk.004.patch, 
> HDFS-13473-trunk.005.patch
>
>
> It is passive behavior about updating Block keys for DataNode currently, and 
> it depends on if NameNode return #KeyUpdateCommand for heartbeat response.
> There are several problems of this Block keys synchronization mode:
> a. NameNode can't be sensed about if Block Keys reach DataNode successfully,
> b. It is also not sensed for DataNode who meets some exception while receive 
> or process heartbeat response which include BlockKeyCommand,
> such as HDFS-13441 and HDFS-12749 mentioned.
> So I propose improve Push Block Keys from NameNode for DataNode to DataNode 
> Pull Block Keys.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Comment Edited] (HDFS-13473) DataNode update BlockKeys using mode PULL rather than PUSH from NameNode

2018-06-18 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao edited comment on HDFS-13473 at 6/18/18 1:24 PM:
-

upload v005 patch to fix bug about auto-update block token keys which trigger 
by HeartbeatManager#Monitor, and failed ut 
{{hadoop.hdfs.server.namenode.ha.TestFailoverWithBlockTokensEnabled}} is 
related to this bug.
I checked another failed ut {{hadoop.hdfs.TestEncryptedTransfer}} and tested 
locally, it seems to work fine and might not relate to this patch.
could anyone help to double check at convenience?


was (Author: hexiaoqiao):
upload v005 patch to fix bug about auto-update block token keys which trigger 
by HeartbeatManager#Monitor, and failed ut 
{{hadoop.hdfs.server.namenode.ha.TestFailoverWithBlockTokensEnabled
}} is related to this bug.
I checked another failed ut {{hadoop.hdfs.TestEncryptedTransfer}} and tested 
locally, it seems to work fine and might not relate to this patch.
could anyone help to double check at convenience?

> DataNode update BlockKeys using mode PULL rather than PUSH from NameNode
> 
>
> Key: HDFS-13473
> URL: https://issues.apache.org/jira/browse/HDFS-13473
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: datanode
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-13473-trunk.001.patch, HDFS-13473-trunk.002.patch, 
> HDFS-13473-trunk.003.patch, HDFS-13473-trunk.004.patch, 
> HDFS-13473-trunk.005.patch
>
>
> It is passive behavior about updating Block keys for DataNode currently, and 
> it depends on if NameNode return #KeyUpdateCommand for heartbeat response.
> There are several problems of this Block keys synchronization mode:
> a. NameNode can't be sensed about if Block Keys reach DataNode successfully,
> b. It is also not sensed for DataNode who meets some exception while receive 
> or process heartbeat response which include BlockKeyCommand,
> such as HDFS-13441 and HDFS-12749 mentioned.
> So I propose improve Push Block Keys from NameNode for DataNode to DataNode 
> Pull Block Keys.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Updated] (HDFS-13473) DataNode update BlockKeys using mode PULL rather than PUSH from NameNode

2018-06-18 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao updated HDFS-13473:
---
Attachment: HDFS-13473-trunk.005.patch

> DataNode update BlockKeys using mode PULL rather than PUSH from NameNode
> 
>
> Key: HDFS-13473
> URL: https://issues.apache.org/jira/browse/HDFS-13473
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: datanode
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-13473-trunk.001.patch, HDFS-13473-trunk.002.patch, 
> HDFS-13473-trunk.003.patch, HDFS-13473-trunk.004.patch, 
> HDFS-13473-trunk.005.patch
>
>
> It is passive behavior about updating Block keys for DataNode currently, and 
> it depends on if NameNode return #KeyUpdateCommand for heartbeat response.
> There are several problems of this Block keys synchronization mode:
> a. NameNode can't be sensed about if Block Keys reach DataNode successfully,
> b. It is also not sensed for DataNode who meets some exception while receive 
> or process heartbeat response which include BlockKeyCommand,
> such as HDFS-13441 and HDFS-12749 mentioned.
> So I propose improve Push Block Keys from NameNode for DataNode to DataNode 
> Pull Block Keys.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (HDFS-13473) DataNode update BlockKeys using mode PULL rather than PUSH from NameNode

2018-06-16 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao commented on HDFS-13473:


re-upload v004 patch, fix compile fail and kick on jenknis again.

> DataNode update BlockKeys using mode PULL rather than PUSH from NameNode
> 
>
> Key: HDFS-13473
> URL: https://issues.apache.org/jira/browse/HDFS-13473
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: datanode
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-13473-trunk.001.patch, HDFS-13473-trunk.002.patch, 
> HDFS-13473-trunk.003.patch, HDFS-13473-trunk.004.patch
>
>
> It is passive behavior about updating Block keys for DataNode currently, and 
> it depends on if NameNode return #KeyUpdateCommand for heartbeat response.
> There are several problems of this Block keys synchronization mode:
> a. NameNode can't be sensed about if Block Keys reach DataNode successfully,
> b. It is also not sensed for DataNode who meets some exception while receive 
> or process heartbeat response which include BlockKeyCommand,
> such as HDFS-13441 and HDFS-12749 mentioned.
> So I propose improve Push Block Keys from NameNode for DataNode to DataNode 
> Pull Block Keys.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Updated] (HDFS-13473) DataNode update BlockKeys using mode PULL rather than PUSH from NameNode

2018-06-16 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao updated HDFS-13473:
---
Attachment: HDFS-13473-trunk.004.patch

> DataNode update BlockKeys using mode PULL rather than PUSH from NameNode
> 
>
> Key: HDFS-13473
> URL: https://issues.apache.org/jira/browse/HDFS-13473
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: datanode
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-13473-trunk.001.patch, HDFS-13473-trunk.002.patch, 
> HDFS-13473-trunk.003.patch, HDFS-13473-trunk.004.patch
>
>
> It is passive behavior about updating Block keys for DataNode currently, and 
> it depends on if NameNode return #KeyUpdateCommand for heartbeat response.
> There are several problems of this Block keys synchronization mode:
> a. NameNode can't be sensed about if Block Keys reach DataNode successfully,
> b. It is also not sensed for DataNode who meets some exception while receive 
> or process heartbeat response which include BlockKeyCommand,
> such as HDFS-13441 and HDFS-12749 mentioned.
> So I propose improve Push Block Keys from NameNode for DataNode to DataNode 
> Pull Block Keys.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Updated] (HDFS-13473) DataNode update BlockKeys using mode PULL rather than PUSH from NameNode

2018-06-16 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao updated HDFS-13473:
---
Attachment: (was: HDFS-13473-trunk.004.patch)

> DataNode update BlockKeys using mode PULL rather than PUSH from NameNode
> 
>
> Key: HDFS-13473
> URL: https://issues.apache.org/jira/browse/HDFS-13473
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: datanode
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-13473-trunk.001.patch, HDFS-13473-trunk.002.patch, 
> HDFS-13473-trunk.003.patch
>
>
> It is passive behavior about updating Block keys for DataNode currently, and 
> it depends on if NameNode return #KeyUpdateCommand for heartbeat response.
> There are several problems of this Block keys synchronization mode:
> a. NameNode can't be sensed about if Block Keys reach DataNode successfully,
> b. It is also not sensed for DataNode who meets some exception while receive 
> or process heartbeat response which include BlockKeyCommand,
> such as HDFS-13441 and HDFS-12749 mentioned.
> So I propose improve Push Block Keys from NameNode for DataNode to DataNode 
> Pull Block Keys.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Updated] (HDFS-13473) DataNode update BlockKeys using mode PULL rather than PUSH from NameNode

2018-06-16 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao updated HDFS-13473:
---
Attachment: HDFS-13473-trunk.004.patch

> DataNode update BlockKeys using mode PULL rather than PUSH from NameNode
> 
>
> Key: HDFS-13473
> URL: https://issues.apache.org/jira/browse/HDFS-13473
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: datanode
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-13473-trunk.001.patch, HDFS-13473-trunk.002.patch, 
> HDFS-13473-trunk.003.patch, HDFS-13473-trunk.004.patch
>
>
> It is passive behavior about updating Block keys for DataNode currently, and 
> it depends on if NameNode return #KeyUpdateCommand for heartbeat response.
> There are several problems of this Block keys synchronization mode:
> a. NameNode can't be sensed about if Block Keys reach DataNode successfully,
> b. It is also not sensed for DataNode who meets some exception while receive 
> or process heartbeat response which include BlockKeyCommand,
> such as HDFS-13441 and HDFS-12749 mentioned.
> So I propose improve Push Block Keys from NameNode for DataNode to DataNode 
> Pull Block Keys.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (HDFS-13473) DataNode update BlockKeys using mode PULL rather than PUSH from NameNode

2018-06-16 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao commented on HDFS-13473:


[~daryn], Thanks for your review.
In v003, update logic about checking access key update and make sure that NN 
sends the keys to every DataNode, also I add unittest for this case.
{quote}Why is the BPServiceActor swallowing IllegalArgumentException?{quote}
It seems to slip up and I also remove this unnecessary exception catching.
Thanks agian [~daryn], and I look forward to your feedback. 

> DataNode update BlockKeys using mode PULL rather than PUSH from NameNode
> 
>
> Key: HDFS-13473
> URL: https://issues.apache.org/jira/browse/HDFS-13473
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: datanode
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-13473-trunk.001.patch, HDFS-13473-trunk.002.patch, 
> HDFS-13473-trunk.003.patch
>
>
> It is passive behavior about updating Block keys for DataNode currently, and 
> it depends on if NameNode return #KeyUpdateCommand for heartbeat response.
> There are several problems of this Block keys synchronization mode:
> a. NameNode can't be sensed about if Block Keys reach DataNode successfully,
> b. It is also not sensed for DataNode who meets some exception while receive 
> or process heartbeat response which include BlockKeyCommand,
> such as HDFS-13441 and HDFS-12749 mentioned.
> So I propose improve Push Block Keys from NameNode for DataNode to DataNode 
> Pull Block Keys.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Updated] (HDFS-13473) DataNode update BlockKeys using mode PULL rather than PUSH from NameNode

2018-06-16 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao updated HDFS-13473:
---
Attachment: HDFS-13473-trunk.003.patch

> DataNode update BlockKeys using mode PULL rather than PUSH from NameNode
> 
>
> Key: HDFS-13473
> URL: https://issues.apache.org/jira/browse/HDFS-13473
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: datanode
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-13473-trunk.001.patch, HDFS-13473-trunk.002.patch, 
> HDFS-13473-trunk.003.patch
>
>
> It is passive behavior about updating Block keys for DataNode currently, and 
> it depends on if NameNode return #KeyUpdateCommand for heartbeat response.
> There are several problems of this Block keys synchronization mode:
> a. NameNode can't be sensed about if Block Keys reach DataNode successfully,
> b. It is also not sensed for DataNode who meets some exception while receive 
> or process heartbeat response which include BlockKeyCommand,
> such as HDFS-13441 and HDFS-12749 mentioned.
> So I propose improve Push Block Keys from NameNode for DataNode to DataNode 
> Pull Block Keys.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Updated] (HDFS-13668) FSPermissionChecker may throws AIOOE when check if inode has permission

2018-06-10 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao updated HDFS-13668:
---
Status: Patch Available  (was: Open)

submit v0 patch and trigger jenkins.

> FSPermissionChecker may throws AIOOE when check if inode has permission
> ---
>
> Key: HDFS-13668
> URL: https://issues.apache.org/jira/browse/HDFS-13668
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: namenode
>Affects Versions: 3.1.0, 2.10.0, 2.7.7
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-13668-trunk.001.patch
>
>
> {{FSPermissionChecker}} may throw {{ArrayIndexOutOfBoundsException:0}} when 
> check if has permission, since it only check inode's {{aclFeature}} if null 
> or not but not check it's entry size. When it meets {{aclFeature}} not null 
> but it's entry size equal to 0, it will throw AIOOE.
> {code:java}
> private boolean hasPermission(INodeAttributes inode, FsAction access) {
>   ..
>   final AclFeature aclFeature = inode.getAclFeature();
>   if (aclFeature != null) {
> // It's possible that the inode has a default ACL but no access ACL.
> int firstEntry = aclFeature.getEntryAt(0);
> if (AclEntryStatusFormat.getScope(firstEntry) == AclEntryScope.ACCESS) {
>   return hasAclPermission(inode, access, mode, aclFeature);
> }
>   }
>   ..
> }
> {code}
> Actually if use default {{INodeAttributeProvider}}, it can ensure that when 
> {{inode}}'s aclFeature is not null and it's entry size also will be greater 
> than 0, but {{INodeAttributeProvider}} is a public interface, we could not 
> ensure external implement (e.g. Apache Sentry, Apache Ranger) also has the 
> similar constraint. 



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Updated] (HDFS-13668) FSPermissionChecker may throws AIOOE when check if inode has permission

2018-06-10 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao updated HDFS-13668:
---
Attachment: HDFS-13668-trunk.001.patch

> FSPermissionChecker may throws AIOOE when check if inode has permission
> ---
>
> Key: HDFS-13668
> URL: https://issues.apache.org/jira/browse/HDFS-13668
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: namenode
>Affects Versions: 3.1.0, 2.10.0, 2.7.7
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-13668-trunk.001.patch
>
>
> {{FSPermissionChecker}} may throw {{ArrayIndexOutOfBoundsException:0}} when 
> check if has permission, since it only check inode's {{aclFeature}} if null 
> or not but not check it's entry size. When it meets {{aclFeature}} not null 
> but it's entry size equal to 0, it will throw AIOOE.
> {code:java}
> private boolean hasPermission(INodeAttributes inode, FsAction access) {
>   ..
>   final AclFeature aclFeature = inode.getAclFeature();
>   if (aclFeature != null) {
> // It's possible that the inode has a default ACL but no access ACL.
> int firstEntry = aclFeature.getEntryAt(0);
> if (AclEntryStatusFormat.getScope(firstEntry) == AclEntryScope.ACCESS) {
>   return hasAclPermission(inode, access, mode, aclFeature);
> }
>   }
>   ..
> }
> {code}
> Actually if use default {{INodeAttributeProvider}}, it can ensure that when 
> {{inode}}'s aclFeature is not null and it's entry size also will be greater 
> than 0, but {{INodeAttributeProvider}} is a public interface, we could not 
> ensure external implement (e.g. Apache Sentry, Apache Ranger) also has the 
> similar constraint. 



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Created] (HDFS-13668) FSPermissionChecker may throws AIOOE when check if inode has permission

2018-06-10 Thread He Xiaoqiao (JIRA)
He Xiaoqiao created HDFS-13668:
--

 Summary: FSPermissionChecker may throws AIOOE when check if inode 
has permission
 Key: HDFS-13668
 URL: https://issues.apache.org/jira/browse/HDFS-13668
 Project: Hadoop HDFS
  Issue Type: Bug
  Components: namenode
Affects Versions: 3.1.0, 2.10.0, 2.7.7
Reporter: He Xiaoqiao
Assignee: He Xiaoqiao


{{FSPermissionChecker}} may throw {{ArrayIndexOutOfBoundsException:0}} when 
check if has permission, since it only check inode's {{aclFeature}} if null or 
not but not check it's entry size. When it meets {{aclFeature}} not null but 
it's entry size equal to 0, it will throw AIOOE.
{code:java}
private boolean hasPermission(INodeAttributes inode, FsAction access) {
  ..
  final AclFeature aclFeature = inode.getAclFeature();
  if (aclFeature != null) {
// It's possible that the inode has a default ACL but no access ACL.
int firstEntry = aclFeature.getEntryAt(0);
if (AclEntryStatusFormat.getScope(firstEntry) == AclEntryScope.ACCESS) {
  return hasAclPermission(inode, access, mode, aclFeature);
}
  }
  ..
}
{code}

Actually if use default {{INodeAttributeProvider}}, it can ensure that when 
{{inode}}'s aclFeature is not null and it's entry size also will be greater 
than 0, but {{INodeAttributeProvider}} is a public interface, we could not 
ensure external implement (e.g. Apache Sentry, Apache Ranger) also has the 
similar constraint. 



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (HDFS-13473) DataNode update BlockKeys using mode PULL rather than PUSH from NameNode

2018-06-07 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao commented on HDFS-13473:


[~daryn] any further comments and suggestions?

> DataNode update BlockKeys using mode PULL rather than PUSH from NameNode
> 
>
> Key: HDFS-13473
> URL: https://issues.apache.org/jira/browse/HDFS-13473
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: datanode
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-13473-trunk.001.patch, HDFS-13473-trunk.002.patch
>
>
> It is passive behavior about updating Block keys for DataNode currently, and 
> it depends on if NameNode return #KeyUpdateCommand for heartbeat response.
> There are several problems of this Block keys synchronization mode:
> a. NameNode can't be sensed about if Block Keys reach DataNode successfully,
> b. It is also not sensed for DataNode who meets some exception while receive 
> or process heartbeat response which include BlockKeyCommand,
> such as HDFS-13441 and HDFS-12749 mentioned.
> So I propose improve Push Block Keys from NameNode for DataNode to DataNode 
> Pull Block Keys.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (HDFS-13183) Standby NameNode process getBlocks request to reduce Active load

2018-06-06 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao commented on HDFS-13183:


Thanks [~elgoiri] and [~shv] for your comments.
{quote}
If we have that mechanism, we could make this generic and the 
ActiveDenyOfServiceException would only need to implement this failover 
exception.
{quote}
ActiveDenyOfServiceException can trigger client failover now actually. but even 
that it also has some problems, for instance, Balancer would not work well if 
SBN shutdown. Of course, these cases can be resolved, if someone interested, 
please go on work, I am sorry that I have no time to fix it recently.

{quote}
I see that you are coming to the same problems as we do, but in a more general 
case. getBlocks() was actually one of our initial use cases for the feature.
{quote}
HDFS-12943 is a very compelling feature indeed. However, there are many users 
who used branch-2.7 or earlier are eager for resolving ANN overhead of 
#getBlocks from Balancer as far as I know. This patch just puts forward one 
solution to this problem for reference.

> Standby NameNode process getBlocks request to reduce Active load
> 
>
> Key: HDFS-13183
> URL: https://issues.apache.org/jira/browse/HDFS-13183
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: balancer  mover, namenode
>Affects Versions: 2.7.5, 3.1.0, 2.9.1, 2.8.4, 3.0.2
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-13183-trunk.001.patch, HDFS-13183-trunk.002.patch, 
> HDFS-13183-trunk.003.patch
>
>
> The performance of Active NameNode could be impact when {{Balancer}} requests 
> #getBlocks, since query blocks of overly full DNs performance is extremely 
> inefficient currently. The main reason is {{NameNodeRpcServer#getBlocks}} 
> hold read lock for long time. In extreme case, all handlers of Active 
> NameNode RPC server are occupied by one reader 
> {{NameNodeRpcServer#getBlocks}} and other write operation calls, thus Active 
> NameNode enter a state of false death for number of seconds even for minutes.
> The similar performance concerns of Balancer have reported by HDFS-9412, 
> HDFS-7967, etc.
> If Standby NameNode can shoulder #getBlocks heavy burden, it could speed up 
> the progress of balancing and reduce performance impact to Active NameNode.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (HDFS-13638) DataNode Can't replicate block because NameNode thinks the length is 9223372036854775807

2018-05-29 Thread He Xiaoqiao (JIRA)


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

He Xiaoqiao commented on HDFS-13638:


[~jojochuang], I met this problem in apache-2.7.1, IIUC, it may be relevant to 
HDFS-10453, FYI.

> DataNode Can't replicate block because NameNode thinks the length is 
> 9223372036854775807
> 
>
> Key: HDFS-13638
> URL: https://issues.apache.org/jira/browse/HDFS-13638
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode
>Reporter: Wei-Chiu Chuang
>Priority: Major
>
> I occasionally find the following warning in CDH clusters, but haven't 
> figured out why. Thought I should better raise the issue anyway.
> {quote}
> 2018-05-29 09:15:58,092 WARN org.apache.hadoop.hdfs.server.datanode.DataNode: 
> Can't replicate block 
> BP-725378529-10.0.0.8-1410027444173:blk_13276745777_1112363330268 because 
> on-disk length 175085 is shorter than NameNode recorded length 
> 9223372036854775807
> {quote}
> Infact, 9223372036854775807 = Long.MAX_VALUE.
> Chasing in the HDFS codebase but didn't find where this length could come from



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (HDFS-12749) DN may not send block report to NN after NN restart

2018-05-26 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao commented on HDFS-12749:


thanks [~kihwal], v005 patch is with test case following your suggestions, FYI. 
I looks forward to your feedback.

> DN may not send block report to NN after NN restart
> ---
>
> Key: HDFS-12749
> URL: https://issues.apache.org/jira/browse/HDFS-12749
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode
>Affects Versions: 2.7.1, 2.8.3, 2.7.5, 3.0.0, 2.9.1
>Reporter: TanYuxin
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-12749-branch-2.7.002.patch, 
> HDFS-12749-trunk.003.patch, HDFS-12749-trunk.004.patch, 
> HDFS-12749-trunk.005.patch, HDFS-12749.001.patch
>
>
> Now our cluster have thousands of DN, millions of files and blocks. When NN 
> restart, NN's load is very high.
> After NN restart,DN will call BPServiceActor#reRegister method to register. 
> But register RPC will get a IOException since NN is busy dealing with Block 
> Report.  The exception is caught at BPServiceActor#processCommand.
> Next is the caught IOException:
> {code:java}
> WARN org.apache.hadoop.hdfs.server.datanode.DataNode: Error processing 
> datanode Command
> java.io.IOException: Failed on local exception: java.io.IOException: 
> java.net.SocketTimeoutException: 6 millis timeout while waiting for 
> channel to be ready for read. ch : java.nio.channels.SocketChannel[connected 
> local=/DataNode_IP:Port remote=NameNode_Host/IP:Port]; Host Details : local 
> host is: "DataNode_Host/Datanode_IP"; destination host is: 
> "NameNode_Host":Port;
> at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:773)
> at org.apache.hadoop.ipc.Client.call(Client.java:1474)
> at org.apache.hadoop.ipc.Client.call(Client.java:1407)
> at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:229)
> at com.sun.proxy.$Proxy13.registerDatanode(Unknown Source)
> at 
> org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB.registerDatanode(DatanodeProtocolClientSideTranslatorPB.java:126)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.register(BPServiceActor.java:793)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.reRegister(BPServiceActor.java:926)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPOfferService.processCommandFromActor(BPOfferService.java:604)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.processCommand(BPServiceActor.java:898)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.offerService(BPServiceActor.java:711)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.run(BPServiceActor.java:864)
> at java.lang.Thread.run(Thread.java:745)
> {code}
> The un-catched IOException breaks BPServiceActor#register, and the Block 
> Report can not be sent immediately. 
> {code}
>   /**
>* Register one bp with the corresponding NameNode
>* 
>* The bpDatanode needs to register with the namenode on startup in order
>* 1) to report which storage it is serving now and 
>* 2) to receive a registrationID
>*  
>* issued by the namenode to recognize registered datanodes.
>* 
>* @param nsInfo current NamespaceInfo
>* @see FSNamesystem#registerDatanode(DatanodeRegistration)
>* @throws IOException
>*/
>   void register(NamespaceInfo nsInfo) throws IOException {
> // The handshake() phase loaded the block pool storage
> // off disk - so update the bpRegistration object from that info
> DatanodeRegistration newBpRegistration = bpos.createRegistration();
> LOG.info(this + " beginning handshake with NN");
> while (shouldRun()) {
>   try {
> // Use returned registration from namenode with updated fields
> newBpRegistration = bpNamenode.registerDatanode(newBpRegistration);
> newBpRegistration.setNamespaceInfo(nsInfo);
> bpRegistration = newBpRegistration;
> break;
>   } catch(EOFException e) {  // namenode might have just restarted
> LOG.info("Problem connecting to server: " + nnAddr + " :"
> + e.getLocalizedMessage());
> sleepAndLogInterrupts(1000, "connecting to server");
>   } catch(SocketTimeoutException e) {  // namenode is busy
> LOG.info("Problem connecting to server: " + nnAddr);
> sleepAndLogInterrupts(1000, "connecting to server");
>   }
> }
> 
> LOG.info("Block pool " + this + " successfully registered with NN");
> bpos.registrationSucceeded(this, bpRegistration);
> // random short delay - helps scatter the BR from all DNs
> 

[jira] [Updated] (HDFS-12749) DN may not send block report to NN after NN restart

2018-05-26 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao updated HDFS-12749:
---
Attachment: HDFS-12749-trunk.005.patch

> DN may not send block report to NN after NN restart
> ---
>
> Key: HDFS-12749
> URL: https://issues.apache.org/jira/browse/HDFS-12749
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode
>Affects Versions: 2.7.1, 2.8.3, 2.7.5, 3.0.0, 2.9.1
>Reporter: TanYuxin
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-12749-branch-2.7.002.patch, 
> HDFS-12749-trunk.003.patch, HDFS-12749-trunk.004.patch, 
> HDFS-12749-trunk.005.patch, HDFS-12749.001.patch
>
>
> Now our cluster have thousands of DN, millions of files and blocks. When NN 
> restart, NN's load is very high.
> After NN restart,DN will call BPServiceActor#reRegister method to register. 
> But register RPC will get a IOException since NN is busy dealing with Block 
> Report.  The exception is caught at BPServiceActor#processCommand.
> Next is the caught IOException:
> {code:java}
> WARN org.apache.hadoop.hdfs.server.datanode.DataNode: Error processing 
> datanode Command
> java.io.IOException: Failed on local exception: java.io.IOException: 
> java.net.SocketTimeoutException: 6 millis timeout while waiting for 
> channel to be ready for read. ch : java.nio.channels.SocketChannel[connected 
> local=/DataNode_IP:Port remote=NameNode_Host/IP:Port]; Host Details : local 
> host is: "DataNode_Host/Datanode_IP"; destination host is: 
> "NameNode_Host":Port;
> at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:773)
> at org.apache.hadoop.ipc.Client.call(Client.java:1474)
> at org.apache.hadoop.ipc.Client.call(Client.java:1407)
> at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:229)
> at com.sun.proxy.$Proxy13.registerDatanode(Unknown Source)
> at 
> org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB.registerDatanode(DatanodeProtocolClientSideTranslatorPB.java:126)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.register(BPServiceActor.java:793)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.reRegister(BPServiceActor.java:926)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPOfferService.processCommandFromActor(BPOfferService.java:604)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.processCommand(BPServiceActor.java:898)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.offerService(BPServiceActor.java:711)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.run(BPServiceActor.java:864)
> at java.lang.Thread.run(Thread.java:745)
> {code}
> The un-catched IOException breaks BPServiceActor#register, and the Block 
> Report can not be sent immediately. 
> {code}
>   /**
>* Register one bp with the corresponding NameNode
>* 
>* The bpDatanode needs to register with the namenode on startup in order
>* 1) to report which storage it is serving now and 
>* 2) to receive a registrationID
>*  
>* issued by the namenode to recognize registered datanodes.
>* 
>* @param nsInfo current NamespaceInfo
>* @see FSNamesystem#registerDatanode(DatanodeRegistration)
>* @throws IOException
>*/
>   void register(NamespaceInfo nsInfo) throws IOException {
> // The handshake() phase loaded the block pool storage
> // off disk - so update the bpRegistration object from that info
> DatanodeRegistration newBpRegistration = bpos.createRegistration();
> LOG.info(this + " beginning handshake with NN");
> while (shouldRun()) {
>   try {
> // Use returned registration from namenode with updated fields
> newBpRegistration = bpNamenode.registerDatanode(newBpRegistration);
> newBpRegistration.setNamespaceInfo(nsInfo);
> bpRegistration = newBpRegistration;
> break;
>   } catch(EOFException e) {  // namenode might have just restarted
> LOG.info("Problem connecting to server: " + nnAddr + " :"
> + e.getLocalizedMessage());
> sleepAndLogInterrupts(1000, "connecting to server");
>   } catch(SocketTimeoutException e) {  // namenode is busy
> LOG.info("Problem connecting to server: " + nnAddr);
> sleepAndLogInterrupts(1000, "connecting to server");
>   }
> }
> 
> LOG.info("Block pool " + this + " successfully registered with NN");
> bpos.registrationSucceeded(this, bpRegistration);
> // random short delay - helps scatter the BR from all DNs
> scheduler.scheduleBlockReport(dnConf.initialBlockReportDelay);
>   }
> {code}
> But NameNode has processed registerDatanode 

[jira] [Commented] (HDFS-13529) Fix default trash policy emptier trigger time correctly

2018-05-06 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao commented on HDFS-13529:


submit v001 patch and trigger jenkins.

> Fix default trash policy emptier trigger time correctly
> ---
>
> Key: HDFS-13529
> URL: https://issues.apache.org/jira/browse/HDFS-13529
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: namenode
>Affects Versions: 3.1.0, 2.7.6, 3.2.0, 2.9.2, 2.8.5
>Reporter: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-13529-trunk.001.patch
>
>
> Trash emptier is designed to auto trigger at UTC 00:00, however I am confused 
> all the time that it usually triggers at a few minutes even half a hour after 
> UTC 00:00 actually in our production cluster.
> The main reason is default policy emptier thread sleep more time than as 
> expect, since it does not consider the delete operation time cost itself. 
> especially for a large cluster, auto trash cleaner may cost dozens of minutes.
> The right way is that gets current time {{now}} before calculate {{end}} time.
> {code:java}
>   long now = Time.now();
>   while (true) {
> end = ceiling(now, emptierInterval);
> try { // sleep for interval
>   Thread.sleep(end - now);
> } catch (InterruptedException e) {
>   break;  // exit on interrupt
> }
> try {
>   now = Time.now();
>   .. // delete trash checkpoint
> } catch (Exception e) {
>   LOG.warn("RuntimeException during Trash.Emptier.run(): ", e); 
> }
>   }
> {code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Updated] (HDFS-13529) Fix default trash policy emptier trigger time correctly

2018-05-06 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao updated HDFS-13529:
---
Status: Patch Available  (was: Open)

> Fix default trash policy emptier trigger time correctly
> ---
>
> Key: HDFS-13529
> URL: https://issues.apache.org/jira/browse/HDFS-13529
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: namenode
>Affects Versions: 2.7.6, 3.1.0, 3.2.0, 2.9.2, 2.8.5
>Reporter: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-13529-trunk.001.patch
>
>
> Trash emptier is designed to auto trigger at UTC 00:00, however I am confused 
> all the time that it usually triggers at a few minutes even half a hour after 
> UTC 00:00 actually in our production cluster.
> The main reason is default policy emptier thread sleep more time than as 
> expect, since it does not consider the delete operation time cost itself. 
> especially for a large cluster, auto trash cleaner may cost dozens of minutes.
> The right way is that gets current time {{now}} before calculate {{end}} time.
> {code:java}
>   long now = Time.now();
>   while (true) {
> end = ceiling(now, emptierInterval);
> try { // sleep for interval
>   Thread.sleep(end - now);
> } catch (InterruptedException e) {
>   break;  // exit on interrupt
> }
> try {
>   now = Time.now();
>   .. // delete trash checkpoint
> } catch (Exception e) {
>   LOG.warn("RuntimeException during Trash.Emptier.run(): ", e); 
> }
>   }
> {code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Updated] (HDFS-13529) Fix default trash policy emptier trigger time correctly

2018-05-06 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao updated HDFS-13529:
---
Attachment: HDFS-13529-trunk.001.patch

> Fix default trash policy emptier trigger time correctly
> ---
>
> Key: HDFS-13529
> URL: https://issues.apache.org/jira/browse/HDFS-13529
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: namenode
>Affects Versions: 3.1.0, 2.7.6, 3.2.0, 2.9.2, 2.8.5
>Reporter: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-13529-trunk.001.patch
>
>
> Trash emptier is designed to auto trigger at UTC 00:00, however I am confused 
> all the time that it usually triggers at a few minutes even half a hour after 
> UTC 00:00 actually in our production cluster.
> The main reason is default policy emptier thread sleep more time than as 
> expect, since it does not consider the delete operation time cost itself. 
> especially for a large cluster, auto trash cleaner may cost dozens of minutes.
> The right way is that gets current time {{now}} before calculate {{end}} time.
> {code:java}
>   long now = Time.now();
>   while (true) {
> end = ceiling(now, emptierInterval);
> try { // sleep for interval
>   Thread.sleep(end - now);
> } catch (InterruptedException e) {
>   break;  // exit on interrupt
> }
> try {
>   now = Time.now();
>   .. // delete trash checkpoint
> } catch (Exception e) {
>   LOG.warn("RuntimeException during Trash.Emptier.run(): ", e); 
> }
>   }
> {code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Created] (HDFS-13529) Fix default trash policy emptier trigger time correctly

2018-05-06 Thread He Xiaoqiao (JIRA)
He Xiaoqiao created HDFS-13529:
--

 Summary: Fix default trash policy emptier trigger time correctly
 Key: HDFS-13529
 URL: https://issues.apache.org/jira/browse/HDFS-13529
 Project: Hadoop HDFS
  Issue Type: Bug
  Components: namenode
Affects Versions: 2.7.6, 3.1.0, 3.2.0, 2.9.2, 2.8.5
Reporter: He Xiaoqiao


Trash emptier is designed to auto trigger at UTC 00:00, however I am confused 
all the time that it usually triggers at a few minutes even half a hour after 
UTC 00:00 actually in our production cluster.
The main reason is default policy emptier thread sleep more time than as 
expect, since it does not consider the delete operation time cost itself. 
especially for a large cluster, auto trash cleaner may cost dozens of minutes.
The right way is that gets current time {{now}} before calculate {{end}} time.
{code:java}
  long now = Time.now();
  while (true) {
end = ceiling(now, emptierInterval);
try { // sleep for interval
  Thread.sleep(end - now);
} catch (InterruptedException e) {
  break;  // exit on interrupt
}

try {
  now = Time.now();
  .. // delete trash checkpoint
} catch (Exception e) {
  LOG.warn("RuntimeException during Trash.Emptier.run(): ", e); 
}
  }
{code}




--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (HDFS-13473) DataNode update BlockKeys using mode PULL rather than PUSH from NameNode

2018-04-22 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao commented on HDFS-13473:


Patch V2 is following [~daryn]'s suggestion, and DataNode's heartbeat contains 
current key version, and NameNode handle heartbeat to compare DataNode's 
current key version with NameNode's current key version, return 
{{KeyUpdateCommand}} if different.

> DataNode update BlockKeys using mode PULL rather than PUSH from NameNode
> 
>
> Key: HDFS-13473
> URL: https://issues.apache.org/jira/browse/HDFS-13473
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: datanode
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-13473-trunk.001.patch, HDFS-13473-trunk.002.patch
>
>
> It is passive behavior about updating Block keys for DataNode currently, and 
> it depends on if NameNode return #KeyUpdateCommand for heartbeat response.
> There are several problems of this Block keys synchronization mode:
> a. NameNode can't be sensed about if Block Keys reach DataNode successfully,
> b. It is also not sensed for DataNode who meets some exception while receive 
> or process heartbeat response which include BlockKeyCommand,
> such as HDFS-13441 and HDFS-12749 mentioned.
> So I propose improve Push Block Keys from NameNode for DataNode to DataNode 
> Pull Block Keys.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Updated] (HDFS-13473) DataNode update BlockKeys using mode PULL rather than PUSH from NameNode

2018-04-22 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao updated HDFS-13473:
---
Attachment: HDFS-13473-trunk.002.patch

> DataNode update BlockKeys using mode PULL rather than PUSH from NameNode
> 
>
> Key: HDFS-13473
> URL: https://issues.apache.org/jira/browse/HDFS-13473
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: datanode
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-13473-trunk.001.patch, HDFS-13473-trunk.002.patch
>
>
> It is passive behavior about updating Block keys for DataNode currently, and 
> it depends on if NameNode return #KeyUpdateCommand for heartbeat response.
> There are several problems of this Block keys synchronization mode:
> a. NameNode can't be sensed about if Block Keys reach DataNode successfully,
> b. It is also not sensed for DataNode who meets some exception while receive 
> or process heartbeat response which include BlockKeyCommand,
> such as HDFS-13441 and HDFS-12749 mentioned.
> So I propose improve Push Block Keys from NameNode for DataNode to DataNode 
> Pull Block Keys.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (HDFS-10943) rollEditLog expects empty EditsDoubleBuffer.bufCurrent which is not guaranteed

2018-04-20 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao commented on HDFS-10943:


hi [~daryn], I wonder is there any condition to make sure that 
{{EditsDoubleBuffer.bufReady}} is empty when swap buffer if disable assertions. 
Thanks for your help.

> rollEditLog expects empty EditsDoubleBuffer.bufCurrent which is not guaranteed
> --
>
> Key: HDFS-10943
> URL: https://issues.apache.org/jira/browse/HDFS-10943
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: Yongjun Zhang
>Priority: Major
>
> Per the following trace stack:
> {code}
> FATAL org.apache.hadoop.hdfs.server.namenode.FSEditLog: Error: finalize log 
> segment 10562075963, 10562174157 failed for required journal 
> (JournalAndStream(mgr=QJM to [0.0.0.1:8485, 0.0.0.2:8485, 0.0.0.3:8485, 
> 0.0.0.4:8485, 0.0.0.5:8485], stream=QuorumOutputStream starting at txid 
> 10562075963))
> java.io.IOException: FSEditStream has 49708 bytes still to be flushed and 
> cannot be closed.
> at 
> org.apache.hadoop.hdfs.server.namenode.EditsDoubleBuffer.close(EditsDoubleBuffer.java:66)
> at 
> org.apache.hadoop.hdfs.qjournal.client.QuorumOutputStream.close(QuorumOutputStream.java:65)
> at 
> org.apache.hadoop.hdfs.server.namenode.JournalSet$JournalAndStream.closeStream(JournalSet.java:115)
> at 
> org.apache.hadoop.hdfs.server.namenode.JournalSet$4.apply(JournalSet.java:235)
> at 
> org.apache.hadoop.hdfs.server.namenode.JournalSet.mapJournalsAndReportErrors(JournalSet.java:393)
> at 
> org.apache.hadoop.hdfs.server.namenode.JournalSet.finalizeLogSegment(JournalSet.java:231)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSEditLog.endCurrentLogSegment(FSEditLog.java:1243)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSEditLog.rollEditLog(FSEditLog.java:1172)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSImage.rollEditLog(FSImage.java:1243)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.rollEditLog(FSNamesystem.java:6437)
> at 
> org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.rollEditLog(NameNodeRpcServer.java:1002)
> at 
> org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolServerSideTranslatorPB.rollEditLog(NamenodeProtocolServerSideTranslatorPB.java:142)
> at 
> org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos$NamenodeProtocolService$2.callBlockingMethod(NamenodeProtocolProtos.java:12025)
> at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:617)
> at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1060)
> at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2086)
> at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2082)
> at java.security.AccessController.doPrivileged(Native Method)
> at javax.security.auth.Subject.doAs(Subject.java:422)
> at 
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1671)
> at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2080)
> 2016-09-23 21:40:59,618 WARN 
> org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager: Aborting 
> QuorumOutputStream starting at txid 10562075963
> {code}
> The exception is from  EditsDoubleBuffer
> {code}
>  public void close() throws IOException {
> Preconditions.checkNotNull(bufCurrent);
> Preconditions.checkNotNull(bufReady);
> int bufSize = bufCurrent.size();
> if (bufSize != 0) {
>   throw new IOException("FSEditStream has " + bufSize
>   + " bytes still to be flushed and cannot be closed.");
> }
> IOUtils.cleanup(null, bufCurrent, bufReady);
> bufCurrent = bufReady = null;
>   }
> {code}
> We can see that FSNamesystem.rollEditLog expects  
> EditsDoubleBuffer.bufCurrent to be empty.
> Edits are recorded via FSEditLog$logSync, which does:
> {code}
>* The data is double-buffered within each edit log implementation so that
>* in-memory writing can occur in parallel with the on-disk writing.
>*
>* Each sync occurs in three steps:
>*   1. synchronized, it swaps the double buffer and sets the isSyncRunning
>*  flag.
>*   2. unsynchronized, it flushes the data to storage
>*   3. synchronized, it resets the flag and notifies anyone waiting on the
>*  sync.
>*
>* The lack of synchronization on step 2 allows other threads to continue
>* to write into the memory buffer while the sync is in progress.
>* Because this step is unsynchronized, actions that need to avoid
>* concurrency with sync() should be synchronized and also call
>* waitForSyncToFinish() before assuming they are running alone.
>*/
> 

[jira] [Commented] (HDFS-10943) rollEditLog expects empty EditsDoubleBuffer.bufCurrent which is not guaranteed

2018-04-20 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao commented on HDFS-10943:


Correct:
{quote}1. only 2/3 JournalNodes active at that time since logs show 
SyncTimes(ms): 42337 29016{quote}
this conclusion is not correct, and please ignore.

> rollEditLog expects empty EditsDoubleBuffer.bufCurrent which is not guaranteed
> --
>
> Key: HDFS-10943
> URL: https://issues.apache.org/jira/browse/HDFS-10943
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: Yongjun Zhang
>Priority: Major
>
> Per the following trace stack:
> {code}
> FATAL org.apache.hadoop.hdfs.server.namenode.FSEditLog: Error: finalize log 
> segment 10562075963, 10562174157 failed for required journal 
> (JournalAndStream(mgr=QJM to [0.0.0.1:8485, 0.0.0.2:8485, 0.0.0.3:8485, 
> 0.0.0.4:8485, 0.0.0.5:8485], stream=QuorumOutputStream starting at txid 
> 10562075963))
> java.io.IOException: FSEditStream has 49708 bytes still to be flushed and 
> cannot be closed.
> at 
> org.apache.hadoop.hdfs.server.namenode.EditsDoubleBuffer.close(EditsDoubleBuffer.java:66)
> at 
> org.apache.hadoop.hdfs.qjournal.client.QuorumOutputStream.close(QuorumOutputStream.java:65)
> at 
> org.apache.hadoop.hdfs.server.namenode.JournalSet$JournalAndStream.closeStream(JournalSet.java:115)
> at 
> org.apache.hadoop.hdfs.server.namenode.JournalSet$4.apply(JournalSet.java:235)
> at 
> org.apache.hadoop.hdfs.server.namenode.JournalSet.mapJournalsAndReportErrors(JournalSet.java:393)
> at 
> org.apache.hadoop.hdfs.server.namenode.JournalSet.finalizeLogSegment(JournalSet.java:231)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSEditLog.endCurrentLogSegment(FSEditLog.java:1243)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSEditLog.rollEditLog(FSEditLog.java:1172)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSImage.rollEditLog(FSImage.java:1243)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.rollEditLog(FSNamesystem.java:6437)
> at 
> org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.rollEditLog(NameNodeRpcServer.java:1002)
> at 
> org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolServerSideTranslatorPB.rollEditLog(NamenodeProtocolServerSideTranslatorPB.java:142)
> at 
> org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos$NamenodeProtocolService$2.callBlockingMethod(NamenodeProtocolProtos.java:12025)
> at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:617)
> at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1060)
> at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2086)
> at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2082)
> at java.security.AccessController.doPrivileged(Native Method)
> at javax.security.auth.Subject.doAs(Subject.java:422)
> at 
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1671)
> at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2080)
> 2016-09-23 21:40:59,618 WARN 
> org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager: Aborting 
> QuorumOutputStream starting at txid 10562075963
> {code}
> The exception is from  EditsDoubleBuffer
> {code}
>  public void close() throws IOException {
> Preconditions.checkNotNull(bufCurrent);
> Preconditions.checkNotNull(bufReady);
> int bufSize = bufCurrent.size();
> if (bufSize != 0) {
>   throw new IOException("FSEditStream has " + bufSize
>   + " bytes still to be flushed and cannot be closed.");
> }
> IOUtils.cleanup(null, bufCurrent, bufReady);
> bufCurrent = bufReady = null;
>   }
> {code}
> We can see that FSNamesystem.rollEditLog expects  
> EditsDoubleBuffer.bufCurrent to be empty.
> Edits are recorded via FSEditLog$logSync, which does:
> {code}
>* The data is double-buffered within each edit log implementation so that
>* in-memory writing can occur in parallel with the on-disk writing.
>*
>* Each sync occurs in three steps:
>*   1. synchronized, it swaps the double buffer and sets the isSyncRunning
>*  flag.
>*   2. unsynchronized, it flushes the data to storage
>*   3. synchronized, it resets the flag and notifies anyone waiting on the
>*  sync.
>*
>* The lack of synchronization on step 2 allows other threads to continue
>* to write into the memory buffer while the sync is in progress.
>* Because this step is unsynchronized, actions that need to avoid
>* concurrency with sync() should be synchronized and also call
>* waitForSyncToFinish() before assuming they are running alone.
>*/
> 

[jira] [Comment Edited] (HDFS-10943) rollEditLog expects empty EditsDoubleBuffer.bufCurrent which is not guaranteed

2018-04-20 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao edited comment on HDFS-10943 at 4/20/18 6:23 AM:
-

I would like to provide more detail information:
 A. NameNode logs before fatal which is same as [~yzhangal] mentioned above:
{code:java}
2018-04-19 13:10:24,222 INFO org.apache.hadoop.hdfs.server.namenode.FSEditLog: 
Rolling edit logs
2018-04-19 13:10:24,222 INFO org.apache.hadoop.hdfs.server.namenode.FSEditLog: 
Ending log segment 15221708217
2018-04-19 13:10:24,595 INFO org.apache.hadoop.hdfs.server.namenode.FSEditLog: 
Number of transactions: 206713 Total time for transactions(ms): 2575 Number of 
transactions batched in Syncs: 8218 Number of syncs: 138767 SyncTimes(ms): 
42337 29016
2018-04-19 13:10:25,264 FATAL org.apache.hadoop.hdfs.server.namenode.FSEditLog: 
Error: finalize log segment 15221708217, 15221914929 failed for required 
journal (JournalAndStream(mgr=QJM to [0.0.0.0:8485, 0.0.0.0:8485, 
0.0.0.0:8485], stream=QuorumOutputStream starting at txid 15221708217))
java.io.IOException: FSEditStream has 160370 bytes still to be flushed and 
cannot be closed.
at 
org.apache.hadoop.hdfs.server.namenode.EditsDoubleBuffer.close(EditsDoubleBuffer.java:66)
at 
org.apache.hadoop.hdfs.qjournal.client.QuorumOutputStream.close(QuorumOutputStream.java:65)
at 
org.apache.hadoop.hdfs.server.namenode.JournalSet$JournalAndStream.closeStream(JournalSet.java:115)
at 
org.apache.hadoop.hdfs.server.namenode.JournalSet$4.apply(JournalSet.java:235)
at 
org.apache.hadoop.hdfs.server.namenode.JournalSet.mapJournalsAndReportErrors(JournalSet.java:393)
at 
org.apache.hadoop.hdfs.server.namenode.JournalSet.finalizeLogSegment(JournalSet.java:231)
at 
org.apache.hadoop.hdfs.server.namenode.FSEditLog.endCurrentLogSegment(FSEditLog.java:1274)
at 
org.apache.hadoop.hdfs.server.namenode.FSEditLog.rollEditLog(FSEditLog.java:1203)
at 
org.apache.hadoop.hdfs.server.namenode.FSImage.rollEditLog(FSImage.java:1293)
at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.rollEditLog(FSNamesystem.java:6084)
at 
org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.rollEditLog(NameNodeRpcServer.java:1295)
at 
org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolServerSideTranslatorPB.rollEditLog(NamenodeProtocolServerSideTranslatorPB.java:142)
at 
org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos$NamenodeProtocolService$2.callBlockingMethod(NamenodeProtocolProtos.java:12025)
at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:616)
at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:969)
at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2049)
at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2045)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:415)
at 
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1690)
at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2043)
2018-04-19 13:10:25,390 WARN 
org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager: Aborting 
QuorumOutputStream starting at txid 15221708217
{code}
B. JournalNode logs when Standby NameNode transition To Active and recovery 
EditLog:
{code:java}
2018-04-19 13:10:47,137 INFO org.apache.hadoop.hdfs.qjournal.server.Journal: 
Accepted recovery for segment 15221708217: segmentState { startTxId: 
15221708217 endTxId: 15221913880 isInProgress: true } acceptedInEpoch: 24
{code}
C. CallQueue of NameNode is full and lasted no more than three seconds.
{code:java}
Date   Time RpcProcessingTime  CallQueueLength RpcQueueTime
04-19 13:10:22 0.120   0  0.031 
04-19 13:10:24 0.5053200 10.619 
04-19 13:10:25  1038.2863200265.548
{code}
Based on logs, we can come to conclusions:
 1. only 2/3 JournalNodes active at that time since logs show {{SyncTimes(ms): 
42337 29016}}
 2. 1049 {{FSEditLogOp}} transations (finalize TxId of NameNode before fatal: 
15221914929 - endTxId of JournalNode after recovery: 15221913880) are still in 
{{bufCurrent}} of EditsDoubleBuffer and do not flushed out.
 3. All write of NameNode from DFSClient request still not return at that time, 
and FsEditLogOps of DFSClient's write op. still continue to put into 
{{bufCurrent}}.


was (Author: hexiaoqiao):
I would like to provide more detail information:
 A. NameNode logs before fatal which is same as [~yzhangal] mentioned above:
{code:java}
2018-04-19 13:10:24,222 INFO org.apache.hadoop.hdfs.server.namenode.FSEditLog: 
Rolling edit logs
2018-04-19 13:10:24,222 INFO 

[jira] [Comment Edited] (HDFS-10943) rollEditLog expects empty EditsDoubleBuffer.bufCurrent which is not guaranteed

2018-04-20 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao edited comment on HDFS-10943 at 4/20/18 6:22 AM:
-

I would like to provide more detail information:
 A. NameNode logs before fatal which is same as [~yzhangal] mentioned above:
{code:java}
2018-04-19 13:10:24,222 INFO org.apache.hadoop.hdfs.server.namenode.FSEditLog: 
Rolling edit logs
2018-04-19 13:10:24,222 INFO org.apache.hadoop.hdfs.server.namenode.FSEditLog: 
Ending log segment 15221708217
2018-04-19 13:10:24,595 INFO org.apache.hadoop.hdfs.server.namenode.FSEditLog: 
Number of transactions: 206713 Total time for transactions(ms): 2575 Number of 
transactions batched in Syncs: 8218 Number of syncs: 138767 SyncTimes(ms): 
42337 29016
2018-04-19 13:10:25,264 FATAL org.apache.hadoop.hdfs.server.namenode.FSEditLog: 
Error: finalize log segment 15221708217, 15221914929 failed for required 
journal (JournalAndStream(mgr=QJM to [0.0.0.0:8485, 0.0.0.0:8485, 
0.0.0.0:8485], stream=QuorumOutputStream starting at txid 15221708217))
java.io.IOException: FSEditStream has 160370 bytes still to be flushed and 
cannot be closed.
at 
org.apache.hadoop.hdfs.server.namenode.EditsDoubleBuffer.close(EditsDoubleBuffer.java:66)
at 
org.apache.hadoop.hdfs.qjournal.client.QuorumOutputStream.close(QuorumOutputStream.java:65)
at 
org.apache.hadoop.hdfs.server.namenode.JournalSet$JournalAndStream.closeStream(JournalSet.java:115)
at 
org.apache.hadoop.hdfs.server.namenode.JournalSet$4.apply(JournalSet.java:235)
at 
org.apache.hadoop.hdfs.server.namenode.JournalSet.mapJournalsAndReportErrors(JournalSet.java:393)
at 
org.apache.hadoop.hdfs.server.namenode.JournalSet.finalizeLogSegment(JournalSet.java:231)
at 
org.apache.hadoop.hdfs.server.namenode.FSEditLog.endCurrentLogSegment(FSEditLog.java:1274)
at 
org.apache.hadoop.hdfs.server.namenode.FSEditLog.rollEditLog(FSEditLog.java:1203)
at 
org.apache.hadoop.hdfs.server.namenode.FSImage.rollEditLog(FSImage.java:1293)
at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.rollEditLog(FSNamesystem.java:6084)
at 
org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.rollEditLog(NameNodeRpcServer.java:1295)
at 
org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolServerSideTranslatorPB.rollEditLog(NamenodeProtocolServerSideTranslatorPB.java:142)
at 
org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos$NamenodeProtocolService$2.callBlockingMethod(NamenodeProtocolProtos.java:12025)
at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:616)
at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:969)
at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2049)
at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2045)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:415)
at 
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1690)
at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2043)
2018-04-19 13:10:25,390 WARN 
org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager: Aborting 
QuorumOutputStream starting at txid 15221708217
{code}
B. JournalNode logs when Standby NameNode transition To Active and recovery 
EditLog:
{code:java}
2018-04-19 13:10:47,137 INFO org.apache.hadoop.hdfs.qjournal.server.Journal: 
Accepted recovery for segment 15221708217: segmentState { startTxId: 
15221708217 endTxId: 15221913880 isInProgress: true } acceptedInEpoch: 24
{code}
C. CallQueue of NameNode is full and lasted no more than three seconds.
{code:java}
Date   Time RpcProcessingTime  CallQueueLength RpcQueueTime
04-19 13:10:22 0.120   0  0.031 
04-19 13:10:24 0.5053200 10.619 
04-19 13:10:25  1038.2863200265.548
{code}
Based on logs, we can come to conclusions:
 1. only two JournalNodes active at that time since logs show {{SyncTimes(ms): 
42337 29016}}
 2. 1049 {{FSEditLogOp}} transations (finalize TxId of NameNode before fatal: 
15221914929 - endTxId of JournalNode after recovery: 15221913880) are still in 
{{bufCurrent}} of EditsDoubleBuffer and do not flushed out.
 3. All write of NameNode from DFSClient request still not return at that time, 
and FsEditLogOps of DFSClient's write op. still continue to put into 
{{bufCurrent}}.


was (Author: hexiaoqiao):
I would like to provide more detail information:
A. NameNode logs before fatal which is same as [~yzhangal] mentioned above:
{code:java}
2018-04-19 13:10:24,222 INFO org.apache.hadoop.hdfs.server.namenode.FSEditLog: 
Rolling edit logs
2018-04-19 13:10:24,222 INFO 

[jira] [Commented] (HDFS-10943) rollEditLog expects empty EditsDoubleBuffer.bufCurrent which is not guaranteed

2018-04-20 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao commented on HDFS-10943:


I would like to provide more detail information:
A. NameNode logs before fatal which is same as [~yzhangal] mentioned above:
{code:java}
2018-04-19 13:10:24,222 INFO org.apache.hadoop.hdfs.server.namenode.FSEditLog: 
Rolling edit logs
2018-04-19 13:10:24,222 INFO org.apache.hadoop.hdfs.server.namenode.FSEditLog: 
Ending log segment 15221708217
2018-04-19 13:10:24,595 INFO org.apache.hadoop.hdfs.server.namenode.FSEditLog: 
Number of transactions: 206713 Total time for transactions(ms): 2575 Number of 
transactions batched in Syncs: 8218 Number of syncs: 138767 SyncTimes(ms): 
42337 29016
2018-04-19 13:10:25,264 FATAL org.apache.hadoop.hdfs.server.namenode.FSEditLog: 
Error: finalize log segment 15221708217, 15221914929 failed for required 
journal (JournalAndStream(mgr=QJM to [0.0.0.0:8485, 0.0.0.0:8485, 
0.0.0.0:8485], stream=QuorumOutputStream starting at txid 15221708217))
java.io.IOException: FSEditStream has 160370 bytes still to be flushed and 
cannot be closed.
at 
org.apache.hadoop.hdfs.server.namenode.EditsDoubleBuffer.close(EditsDoubleBuffer.java:66)
at 
org.apache.hadoop.hdfs.qjournal.client.QuorumOutputStream.close(QuorumOutputStream.java:65)
at 
org.apache.hadoop.hdfs.server.namenode.JournalSet$JournalAndStream.closeStream(JournalSet.java:115)
at 
org.apache.hadoop.hdfs.server.namenode.JournalSet$4.apply(JournalSet.java:235)
at 
org.apache.hadoop.hdfs.server.namenode.JournalSet.mapJournalsAndReportErrors(JournalSet.java:393)
at 
org.apache.hadoop.hdfs.server.namenode.JournalSet.finalizeLogSegment(JournalSet.java:231)
at 
org.apache.hadoop.hdfs.server.namenode.FSEditLog.endCurrentLogSegment(FSEditLog.java:1274)
at 
org.apache.hadoop.hdfs.server.namenode.FSEditLog.rollEditLog(FSEditLog.java:1203)
at 
org.apache.hadoop.hdfs.server.namenode.FSImage.rollEditLog(FSImage.java:1293)
at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.rollEditLog(FSNamesystem.java:6084)
at 
org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.rollEditLog(NameNodeRpcServer.java:1295)
at 
org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolServerSideTranslatorPB.rollEditLog(NamenodeProtocolServerSideTranslatorPB.java:142)
at 
org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos$NamenodeProtocolService$2.callBlockingMethod(NamenodeProtocolProtos.java:12025)
at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:616)
at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:969)
at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2049)
at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2045)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:415)
at 
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1690)
at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2043)
2018-04-19 13:10:25,390 WARN 
org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager: Aborting 
QuorumOutputStream starting at txid 15221708217
{code}
B. JournalNode logs when Standby NameNode transition To Active and recovery 
EditLog:
{code:java}
2018-04-19 13:10:47,137 INFO org.apache.hadoop.hdfs.qjournal.server.Journal: 
Accepted recovery for segment 15221708217: segmentState { startTxId: 
15221708217 endTxId: 15221913880 isInProgress: true } acceptedInEpoch: 24
{code}
C. CallQueue of NameNode is full and lasted no more than three seconds.
{code:java}
Date   Time RpcProcessingTimeAvgTime  CallQueueLength 
RpcQueueTimeAvgTime
04-19 13:10:22  0.12
0  0.031 
04-19 13:10:240.505 
 320010.619 
04-19 13:10:25  1038.286  
3200  265.548
{code}

Based on logs, we can come to conclusions:
1. only two JournalNodes active at that time since logs show {{SyncTimes(ms): 
42337 29016}}
2. 1049 {{FSEditLogOp}} transations (finalize TxId of NameNode before fatal: 
15221914929 - endTxId of JournalNode after recovery: 15221913880)  are still in 
{{bufCurrent}} of EditsDoubleBuffer and do not flushed out.
3. All write of NameNode from DFSClient request still not return at that time, 
and FsEditLogOps of DFSClient's write op. still continue to put into 
{{bufCurrent}}.

> rollEditLog expects empty EditsDoubleBuffer.bufCurrent which is not guaranteed
> --
>
> Key: HDFS-10943
> 

[jira] [Issue Comment Deleted] (HDFS-10943) rollEditLog expects empty EditsDoubleBuffer.bufCurrent which is not guaranteed

2018-04-19 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao updated HDFS-10943:
---
Comment: was deleted

(was: Plz ignore my latest comments. HDFS-11292 is related for this issue and I 
am sorry that I don't read HDFS-11292 carefully.)

> rollEditLog expects empty EditsDoubleBuffer.bufCurrent which is not guaranteed
> --
>
> Key: HDFS-10943
> URL: https://issues.apache.org/jira/browse/HDFS-10943
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: Yongjun Zhang
>Priority: Major
>
> Per the following trace stack:
> {code}
> FATAL org.apache.hadoop.hdfs.server.namenode.FSEditLog: Error: finalize log 
> segment 10562075963, 10562174157 failed for required journal 
> (JournalAndStream(mgr=QJM to [0.0.0.1:8485, 0.0.0.2:8485, 0.0.0.3:8485, 
> 0.0.0.4:8485, 0.0.0.5:8485], stream=QuorumOutputStream starting at txid 
> 10562075963))
> java.io.IOException: FSEditStream has 49708 bytes still to be flushed and 
> cannot be closed.
> at 
> org.apache.hadoop.hdfs.server.namenode.EditsDoubleBuffer.close(EditsDoubleBuffer.java:66)
> at 
> org.apache.hadoop.hdfs.qjournal.client.QuorumOutputStream.close(QuorumOutputStream.java:65)
> at 
> org.apache.hadoop.hdfs.server.namenode.JournalSet$JournalAndStream.closeStream(JournalSet.java:115)
> at 
> org.apache.hadoop.hdfs.server.namenode.JournalSet$4.apply(JournalSet.java:235)
> at 
> org.apache.hadoop.hdfs.server.namenode.JournalSet.mapJournalsAndReportErrors(JournalSet.java:393)
> at 
> org.apache.hadoop.hdfs.server.namenode.JournalSet.finalizeLogSegment(JournalSet.java:231)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSEditLog.endCurrentLogSegment(FSEditLog.java:1243)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSEditLog.rollEditLog(FSEditLog.java:1172)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSImage.rollEditLog(FSImage.java:1243)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.rollEditLog(FSNamesystem.java:6437)
> at 
> org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.rollEditLog(NameNodeRpcServer.java:1002)
> at 
> org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolServerSideTranslatorPB.rollEditLog(NamenodeProtocolServerSideTranslatorPB.java:142)
> at 
> org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos$NamenodeProtocolService$2.callBlockingMethod(NamenodeProtocolProtos.java:12025)
> at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:617)
> at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1060)
> at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2086)
> at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2082)
> at java.security.AccessController.doPrivileged(Native Method)
> at javax.security.auth.Subject.doAs(Subject.java:422)
> at 
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1671)
> at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2080)
> 2016-09-23 21:40:59,618 WARN 
> org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager: Aborting 
> QuorumOutputStream starting at txid 10562075963
> {code}
> The exception is from  EditsDoubleBuffer
> {code}
>  public void close() throws IOException {
> Preconditions.checkNotNull(bufCurrent);
> Preconditions.checkNotNull(bufReady);
> int bufSize = bufCurrent.size();
> if (bufSize != 0) {
>   throw new IOException("FSEditStream has " + bufSize
>   + " bytes still to be flushed and cannot be closed.");
> }
> IOUtils.cleanup(null, bufCurrent, bufReady);
> bufCurrent = bufReady = null;
>   }
> {code}
> We can see that FSNamesystem.rollEditLog expects  
> EditsDoubleBuffer.bufCurrent to be empty.
> Edits are recorded via FSEditLog$logSync, which does:
> {code}
>* The data is double-buffered within each edit log implementation so that
>* in-memory writing can occur in parallel with the on-disk writing.
>*
>* Each sync occurs in three steps:
>*   1. synchronized, it swaps the double buffer and sets the isSyncRunning
>*  flag.
>*   2. unsynchronized, it flushes the data to storage
>*   3. synchronized, it resets the flag and notifies anyone waiting on the
>*  sync.
>*
>* The lack of synchronization on step 2 allows other threads to continue
>* to write into the memory buffer while the sync is in progress.
>* Because this step is unsynchronized, actions that need to avoid
>* concurrency with sync() should be synchronized and also call
>* waitForSyncToFinish() before assuming they are running alone.
>*/
> {code}
> We can see that step 2 is on-purposely not 

[jira] [Commented] (HDFS-10943) rollEditLog expects empty EditsDoubleBuffer.bufCurrent which is not guaranteed

2018-04-19 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao commented on HDFS-10943:


Plz ignore my latest comments. HDFS-11292 is related for this issue and I am 
sorry that I don't read HDFS-11292 carefully.

> rollEditLog expects empty EditsDoubleBuffer.bufCurrent which is not guaranteed
> --
>
> Key: HDFS-10943
> URL: https://issues.apache.org/jira/browse/HDFS-10943
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: Yongjun Zhang
>Priority: Major
>
> Per the following trace stack:
> {code}
> FATAL org.apache.hadoop.hdfs.server.namenode.FSEditLog: Error: finalize log 
> segment 10562075963, 10562174157 failed for required journal 
> (JournalAndStream(mgr=QJM to [0.0.0.1:8485, 0.0.0.2:8485, 0.0.0.3:8485, 
> 0.0.0.4:8485, 0.0.0.5:8485], stream=QuorumOutputStream starting at txid 
> 10562075963))
> java.io.IOException: FSEditStream has 49708 bytes still to be flushed and 
> cannot be closed.
> at 
> org.apache.hadoop.hdfs.server.namenode.EditsDoubleBuffer.close(EditsDoubleBuffer.java:66)
> at 
> org.apache.hadoop.hdfs.qjournal.client.QuorumOutputStream.close(QuorumOutputStream.java:65)
> at 
> org.apache.hadoop.hdfs.server.namenode.JournalSet$JournalAndStream.closeStream(JournalSet.java:115)
> at 
> org.apache.hadoop.hdfs.server.namenode.JournalSet$4.apply(JournalSet.java:235)
> at 
> org.apache.hadoop.hdfs.server.namenode.JournalSet.mapJournalsAndReportErrors(JournalSet.java:393)
> at 
> org.apache.hadoop.hdfs.server.namenode.JournalSet.finalizeLogSegment(JournalSet.java:231)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSEditLog.endCurrentLogSegment(FSEditLog.java:1243)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSEditLog.rollEditLog(FSEditLog.java:1172)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSImage.rollEditLog(FSImage.java:1243)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.rollEditLog(FSNamesystem.java:6437)
> at 
> org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.rollEditLog(NameNodeRpcServer.java:1002)
> at 
> org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolServerSideTranslatorPB.rollEditLog(NamenodeProtocolServerSideTranslatorPB.java:142)
> at 
> org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos$NamenodeProtocolService$2.callBlockingMethod(NamenodeProtocolProtos.java:12025)
> at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:617)
> at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1060)
> at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2086)
> at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2082)
> at java.security.AccessController.doPrivileged(Native Method)
> at javax.security.auth.Subject.doAs(Subject.java:422)
> at 
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1671)
> at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2080)
> 2016-09-23 21:40:59,618 WARN 
> org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager: Aborting 
> QuorumOutputStream starting at txid 10562075963
> {code}
> The exception is from  EditsDoubleBuffer
> {code}
>  public void close() throws IOException {
> Preconditions.checkNotNull(bufCurrent);
> Preconditions.checkNotNull(bufReady);
> int bufSize = bufCurrent.size();
> if (bufSize != 0) {
>   throw new IOException("FSEditStream has " + bufSize
>   + " bytes still to be flushed and cannot be closed.");
> }
> IOUtils.cleanup(null, bufCurrent, bufReady);
> bufCurrent = bufReady = null;
>   }
> {code}
> We can see that FSNamesystem.rollEditLog expects  
> EditsDoubleBuffer.bufCurrent to be empty.
> Edits are recorded via FSEditLog$logSync, which does:
> {code}
>* The data is double-buffered within each edit log implementation so that
>* in-memory writing can occur in parallel with the on-disk writing.
>*
>* Each sync occurs in three steps:
>*   1. synchronized, it swaps the double buffer and sets the isSyncRunning
>*  flag.
>*   2. unsynchronized, it flushes the data to storage
>*   3. synchronized, it resets the flag and notifies anyone waiting on the
>*  sync.
>*
>* The lack of synchronization on step 2 allows other threads to continue
>* to write into the memory buffer while the sync is in progress.
>* Because this step is unsynchronized, actions that need to avoid
>* concurrency with sync() should be synchronized and also call
>* waitForSyncToFinish() before assuming they are running alone.
>*/
> {code}
> We can see that step 2 is 

[jira] [Commented] (HDFS-10943) rollEditLog expects empty EditsDoubleBuffer.bufCurrent which is not guaranteed

2018-04-19 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao commented on HDFS-10943:


hi [~zhz],[~daryn],[~yzhangal],[~kihwal] is this issue still going on?
I meet the same problem in my production cluster, and my hadoop version is 
apache hadoop-2.7.1.
I agree that some editlog in doublebuffer is not flush. maybe it is not related 
to {{logSyncAll}} since compare {{logSyncAll}} to {{logSync}}, it is only 
different about set {{txid}} but not different for doublebuffer.

> rollEditLog expects empty EditsDoubleBuffer.bufCurrent which is not guaranteed
> --
>
> Key: HDFS-10943
> URL: https://issues.apache.org/jira/browse/HDFS-10943
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: Yongjun Zhang
>Priority: Major
>
> Per the following trace stack:
> {code}
> FATAL org.apache.hadoop.hdfs.server.namenode.FSEditLog: Error: finalize log 
> segment 10562075963, 10562174157 failed for required journal 
> (JournalAndStream(mgr=QJM to [0.0.0.1:8485, 0.0.0.2:8485, 0.0.0.3:8485, 
> 0.0.0.4:8485, 0.0.0.5:8485], stream=QuorumOutputStream starting at txid 
> 10562075963))
> java.io.IOException: FSEditStream has 49708 bytes still to be flushed and 
> cannot be closed.
> at 
> org.apache.hadoop.hdfs.server.namenode.EditsDoubleBuffer.close(EditsDoubleBuffer.java:66)
> at 
> org.apache.hadoop.hdfs.qjournal.client.QuorumOutputStream.close(QuorumOutputStream.java:65)
> at 
> org.apache.hadoop.hdfs.server.namenode.JournalSet$JournalAndStream.closeStream(JournalSet.java:115)
> at 
> org.apache.hadoop.hdfs.server.namenode.JournalSet$4.apply(JournalSet.java:235)
> at 
> org.apache.hadoop.hdfs.server.namenode.JournalSet.mapJournalsAndReportErrors(JournalSet.java:393)
> at 
> org.apache.hadoop.hdfs.server.namenode.JournalSet.finalizeLogSegment(JournalSet.java:231)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSEditLog.endCurrentLogSegment(FSEditLog.java:1243)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSEditLog.rollEditLog(FSEditLog.java:1172)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSImage.rollEditLog(FSImage.java:1243)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.rollEditLog(FSNamesystem.java:6437)
> at 
> org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.rollEditLog(NameNodeRpcServer.java:1002)
> at 
> org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolServerSideTranslatorPB.rollEditLog(NamenodeProtocolServerSideTranslatorPB.java:142)
> at 
> org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos$NamenodeProtocolService$2.callBlockingMethod(NamenodeProtocolProtos.java:12025)
> at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:617)
> at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1060)
> at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2086)
> at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2082)
> at java.security.AccessController.doPrivileged(Native Method)
> at javax.security.auth.Subject.doAs(Subject.java:422)
> at 
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1671)
> at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2080)
> 2016-09-23 21:40:59,618 WARN 
> org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager: Aborting 
> QuorumOutputStream starting at txid 10562075963
> {code}
> The exception is from  EditsDoubleBuffer
> {code}
>  public void close() throws IOException {
> Preconditions.checkNotNull(bufCurrent);
> Preconditions.checkNotNull(bufReady);
> int bufSize = bufCurrent.size();
> if (bufSize != 0) {
>   throw new IOException("FSEditStream has " + bufSize
>   + " bytes still to be flushed and cannot be closed.");
> }
> IOUtils.cleanup(null, bufCurrent, bufReady);
> bufCurrent = bufReady = null;
>   }
> {code}
> We can see that FSNamesystem.rollEditLog expects  
> EditsDoubleBuffer.bufCurrent to be empty.
> Edits are recorded via FSEditLog$logSync, which does:
> {code}
>* The data is double-buffered within each edit log implementation so that
>* in-memory writing can occur in parallel with the on-disk writing.
>*
>* Each sync occurs in three steps:
>*   1. synchronized, it swaps the double buffer and sets the isSyncRunning
>*  flag.
>*   2. unsynchronized, it flushes the data to storage
>*   3. synchronized, it resets the flag and notifies anyone waiting on the
>*  sync.
>*
>* The lack of synchronization on step 2 allows other threads to continue
>* to write into the memory buffer while the sync is in 

[jira] [Commented] (HDFS-12749) DN may not send block report to NN after NN restart

2018-04-18 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao commented on HDFS-12749:


ping [~kihwal],[~daryn],[~arpitagarwal],[~ajayydv] do you mind having a look?

> DN may not send block report to NN after NN restart
> ---
>
> Key: HDFS-12749
> URL: https://issues.apache.org/jira/browse/HDFS-12749
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode
>Affects Versions: 2.7.1, 2.8.3, 2.7.5, 3.0.0, 2.9.1
>Reporter: TanYuxin
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-12749-branch-2.7.002.patch, 
> HDFS-12749-trunk.003.patch, HDFS-12749-trunk.004.patch, HDFS-12749.001.patch
>
>
> Now our cluster have thousands of DN, millions of files and blocks. When NN 
> restart, NN's load is very high.
> After NN restart,DN will call BPServiceActor#reRegister method to register. 
> But register RPC will get a IOException since NN is busy dealing with Block 
> Report.  The exception is caught at BPServiceActor#processCommand.
> Next is the caught IOException:
> {code:java}
> WARN org.apache.hadoop.hdfs.server.datanode.DataNode: Error processing 
> datanode Command
> java.io.IOException: Failed on local exception: java.io.IOException: 
> java.net.SocketTimeoutException: 6 millis timeout while waiting for 
> channel to be ready for read. ch : java.nio.channels.SocketChannel[connected 
> local=/DataNode_IP:Port remote=NameNode_Host/IP:Port]; Host Details : local 
> host is: "DataNode_Host/Datanode_IP"; destination host is: 
> "NameNode_Host":Port;
> at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:773)
> at org.apache.hadoop.ipc.Client.call(Client.java:1474)
> at org.apache.hadoop.ipc.Client.call(Client.java:1407)
> at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:229)
> at com.sun.proxy.$Proxy13.registerDatanode(Unknown Source)
> at 
> org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB.registerDatanode(DatanodeProtocolClientSideTranslatorPB.java:126)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.register(BPServiceActor.java:793)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.reRegister(BPServiceActor.java:926)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPOfferService.processCommandFromActor(BPOfferService.java:604)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.processCommand(BPServiceActor.java:898)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.offerService(BPServiceActor.java:711)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.run(BPServiceActor.java:864)
> at java.lang.Thread.run(Thread.java:745)
> {code}
> The un-catched IOException breaks BPServiceActor#register, and the Block 
> Report can not be sent immediately. 
> {code}
>   /**
>* Register one bp with the corresponding NameNode
>* 
>* The bpDatanode needs to register with the namenode on startup in order
>* 1) to report which storage it is serving now and 
>* 2) to receive a registrationID
>*  
>* issued by the namenode to recognize registered datanodes.
>* 
>* @param nsInfo current NamespaceInfo
>* @see FSNamesystem#registerDatanode(DatanodeRegistration)
>* @throws IOException
>*/
>   void register(NamespaceInfo nsInfo) throws IOException {
> // The handshake() phase loaded the block pool storage
> // off disk - so update the bpRegistration object from that info
> DatanodeRegistration newBpRegistration = bpos.createRegistration();
> LOG.info(this + " beginning handshake with NN");
> while (shouldRun()) {
>   try {
> // Use returned registration from namenode with updated fields
> newBpRegistration = bpNamenode.registerDatanode(newBpRegistration);
> newBpRegistration.setNamespaceInfo(nsInfo);
> bpRegistration = newBpRegistration;
> break;
>   } catch(EOFException e) {  // namenode might have just restarted
> LOG.info("Problem connecting to server: " + nnAddr + " :"
> + e.getLocalizedMessage());
> sleepAndLogInterrupts(1000, "connecting to server");
>   } catch(SocketTimeoutException e) {  // namenode is busy
> LOG.info("Problem connecting to server: " + nnAddr);
> sleepAndLogInterrupts(1000, "connecting to server");
>   }
> }
> 
> LOG.info("Block pool " + this + " successfully registered with NN");
> bpos.registrationSucceeded(this, bpRegistration);
> // random short delay - helps scatter the BR from all DNs
> scheduler.scheduleBlockReport(dnConf.initialBlockReportDelay);
>   }
> {code}
> But 

[jira] [Commented] (HDFS-13473) DataNode update BlockKeys using mode PULL rather than PUSH from NameNode

2018-04-18 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao commented on HDFS-13473:


Thanks [~daryn] for your comments.
{quote}What about something like the DN's heartbeat contains the current key 
version it has? The NN's handleHeartbeat compares with its current key, calls 
setNeedKeyUpdate if different.{quote}
It is good suggestion to update Block Keys for DataNode. But there may be more 
code changes since we need to update {{DatanodeProtocol#sendHeartbeat}} and add 
new parameter about version for BlockKeys?

> DataNode update BlockKeys using mode PULL rather than PUSH from NameNode
> 
>
> Key: HDFS-13473
> URL: https://issues.apache.org/jira/browse/HDFS-13473
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: datanode
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-13473-trunk.001.patch
>
>
> It is passive behavior about updating Block keys for DataNode currently, and 
> it depends on if NameNode return #KeyUpdateCommand for heartbeat response.
> There are several problems of this Block keys synchronization mode:
> a. NameNode can't be sensed about if Block Keys reach DataNode successfully,
> b. It is also not sensed for DataNode who meets some exception while receive 
> or process heartbeat response which include BlockKeyCommand,
> such as HDFS-13441 and HDFS-12749 mentioned.
> So I propose improve Push Block Keys from NameNode for DataNode to DataNode 
> Pull Block Keys.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Assigned] (HDFS-13473) DataNode update BlockKeys using mode PULL rather than PUSH from NameNode

2018-04-18 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao reassigned HDFS-13473:
--

Assignee: He Xiaoqiao

> DataNode update BlockKeys using mode PULL rather than PUSH from NameNode
> 
>
> Key: HDFS-13473
> URL: https://issues.apache.org/jira/browse/HDFS-13473
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: datanode
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-13473-trunk.001.patch
>
>
> It is passive behavior about updating Block keys for DataNode currently, and 
> it depends on if NameNode return #KeyUpdateCommand for heartbeat response.
> There are several problems of this Block keys synchronization mode:
> a. NameNode can't be sensed about if Block Keys reach DataNode successfully,
> b. It is also not sensed for DataNode who meets some exception while receive 
> or process heartbeat response which include BlockKeyCommand,
> such as HDFS-13441 and HDFS-12749 mentioned.
> So I propose improve Push Block Keys from NameNode for DataNode to DataNode 
> Pull Block Keys.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (HDFS-13473) DataNode update BlockKeys using mode PULL rather than PUSH from NameNode

2018-04-18 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao commented on HDFS-13473:


submit initial patch and use {{NamenodeProtocol#getBlockKeys}} interface to 
update Block Keys periodically by DataNode.
also add configuration items to support if switch on this feature.

> DataNode update BlockKeys using mode PULL rather than PUSH from NameNode
> 
>
> Key: HDFS-13473
> URL: https://issues.apache.org/jira/browse/HDFS-13473
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: datanode
>Reporter: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-13473-trunk.001.patch
>
>
> It is passive behavior about updating Block keys for DataNode currently, and 
> it depends on if NameNode return #KeyUpdateCommand for heartbeat response.
> There are several problems of this Block keys synchronization mode:
> a. NameNode can't be sensed about if Block Keys reach DataNode successfully,
> b. It is also not sensed for DataNode who meets some exception while receive 
> or process heartbeat response which include BlockKeyCommand,
> such as HDFS-13441 and HDFS-12749 mentioned.
> So I propose improve Push Block Keys from NameNode for DataNode to DataNode 
> Pull Block Keys.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Updated] (HDFS-13473) DataNode update BlockKeys using mode PULL rather than PUSH from NameNode

2018-04-18 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao updated HDFS-13473:
---
Status: Patch Available  (was: Open)

> DataNode update BlockKeys using mode PULL rather than PUSH from NameNode
> 
>
> Key: HDFS-13473
> URL: https://issues.apache.org/jira/browse/HDFS-13473
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: datanode
>Reporter: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-13473-trunk.001.patch
>
>
> It is passive behavior about updating Block keys for DataNode currently, and 
> it depends on if NameNode return #KeyUpdateCommand for heartbeat response.
> There are several problems of this Block keys synchronization mode:
> a. NameNode can't be sensed about if Block Keys reach DataNode successfully,
> b. It is also not sensed for DataNode who meets some exception while receive 
> or process heartbeat response which include BlockKeyCommand,
> such as HDFS-13441 and HDFS-12749 mentioned.
> So I propose improve Push Block Keys from NameNode for DataNode to DataNode 
> Pull Block Keys.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Updated] (HDFS-13473) DataNode update BlockKeys using mode PULL rather than PUSH from NameNode

2018-04-18 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao updated HDFS-13473:
---
Attachment: HDFS-13473-trunk.001.patch

> DataNode update BlockKeys using mode PULL rather than PUSH from NameNode
> 
>
> Key: HDFS-13473
> URL: https://issues.apache.org/jira/browse/HDFS-13473
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: datanode
>Reporter: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-13473-trunk.001.patch
>
>
> It is passive behavior about updating Block keys for DataNode currently, and 
> it depends on if NameNode return #KeyUpdateCommand for heartbeat response.
> There are several problems of this Block keys synchronization mode:
> a. NameNode can't be sensed about if Block Keys reach DataNode successfully,
> b. It is also not sensed for DataNode who meets some exception while receive 
> or process heartbeat response which include BlockKeyCommand,
> such as HDFS-13441 and HDFS-12749 mentioned.
> So I propose improve Push Block Keys from NameNode for DataNode to DataNode 
> Pull Block Keys.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Created] (HDFS-13473) DataNode update BlockKeys using mode PULL rather than PUSH from NameNode

2018-04-18 Thread He Xiaoqiao (JIRA)
He Xiaoqiao created HDFS-13473:
--

 Summary: DataNode update BlockKeys using mode PULL rather than 
PUSH from NameNode
 Key: HDFS-13473
 URL: https://issues.apache.org/jira/browse/HDFS-13473
 Project: Hadoop HDFS
  Issue Type: Improvement
  Components: datanode
Reporter: He Xiaoqiao


It is passive behavior about updating Block keys for DataNode currently, and it 
depends on if NameNode return #KeyUpdateCommand for heartbeat response.
There are several problems of this Block keys synchronization mode:
a. NameNode can't be sensed about if Block Keys reach DataNode successfully,
b. It is also not sensed for DataNode who meets some exception while receive or 
process heartbeat response which include BlockKeyCommand,
such as HDFS-13441 and HDFS-12749 mentioned.
So I propose improve Push Block Keys from NameNode for DataNode to DataNode 
Pull Block Keys.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Comment Edited] (HDFS-13441) DataNode missed BlockKey update from NameNode due to HeartbeatResponse was dropped

2018-04-17 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao edited comment on HDFS-13441 at 4/17/18 12:17 PM:
--

{quote}Let DataNode pull Block Key from NameNode is one choice, since it need 
change protocol, if I didn't understand wrong, it will need go to Hadoop 
4.{quote}
[~zhaoyunjiong] 
The interface about get BlockKeys is ready now, reference: 
NamenodeProtocol#getBlockKeys.
if DataNode get Block Keys from NameNode using this interface, then it is more 
convenient to process kind of exceptions. if possible, is it necessary to 
create new issue to push this feature?


was (Author: hexiaoqiao):
{quote}Let DataNode pull Block Key from NameNode is one choice, since it need 
change protocol, if I didn't understand wrong, it will need go to Hadoop 
4.{quote}
[~zhaoyunjiong] 
The interface about get BlockKeys is ready now, reference: 
NamenodeProtocol#getBlockKeys.
if DataNode get Block Keys from NameNode using this interface, then it is more 
convenient to process kind of exceptions.

> DataNode missed BlockKey update from NameNode due to HeartbeatResponse was 
> dropped
> --
>
> Key: HDFS-13441
> URL: https://issues.apache.org/jira/browse/HDFS-13441
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode, namenode
>Affects Versions: 2.7.1
>Reporter: yunjiong zhao
>Assignee: yunjiong zhao
>Priority: Major
> Attachments: HDFS-13441.002.patch, HDFS-13441.patch
>
>
> After NameNode failover, lots of application failed due to some DataNodes 
> can't re-compute password from block token.
> {code:java}
> 2018-04-11 20:10:52,448 ERROR 
> org.apache.hadoop.hdfs.server.datanode.DataNode: 
> hdc3-lvs01-400-1701-048.stratus.lvs.ebay.com:50010:DataXceiver error 
> processing unknown operation  src: /10.142.74.116:57404 dst: 
> /10.142.77.45:50010
> javax.security.sasl.SaslException: DIGEST-MD5: IO error acquiring password 
> [Caused by org.apache.hadoop.security.token.SecretManager$InvalidToken: Can't 
> re-compute password for block_token_identifier (expiryDate=1523538652448, 
> keyId=1762737944, userId=hadoop, 
> blockPoolId=BP-36315570-10.103.108.13-1423055488042, blockId=12142862700, 
> access modes=[WRITE]), since the required block key (keyID=1762737944) 
> doesn't exist.]
>         at 
> com.sun.security.sasl.digest.DigestMD5Server.validateClientResponse(DigestMD5Server.java:598)
>         at 
> com.sun.security.sasl.digest.DigestMD5Server.evaluateResponse(DigestMD5Server.java:244)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslParticipant.evaluateChallengeOrResponse(SaslParticipant.java:115)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer.doSaslHandshake(SaslDataTransferServer.java:376)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer.getSaslStreams(SaslDataTransferServer.java:300)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer.receive(SaslDataTransferServer.java:127)
>         at 
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:194)
>         at java.lang.Thread.run(Thread.java:745)
> Caused by: org.apache.hadoop.security.token.SecretManager$InvalidToken: Can't 
> re-compute password for block_token_identifier (expiryDate=1523538652448, 
> keyId=1762737944, userId=hadoop, 
> blockPoolId=BP-36315570-10.103.108.13-1423055488042, blockId=12142862700, 
> access modes=[WRITE]), since the required block key (keyID=1762737944) 
> doesn't exist.
>         at 
> org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.retrievePassword(BlockTokenSecretManager.java:382)
>         at 
> org.apache.hadoop.hdfs.security.token.block.BlockPoolTokenSecretManager.retrievePassword(BlockPoolTokenSecretManager.java:79)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer.buildServerPassword(SaslDataTransferServer.java:318)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer.access$100(SaslDataTransferServer.java:73)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer$2.apply(SaslDataTransferServer.java:297)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer$SaslServerCallbackHandler.handle(SaslDataTransferServer.java:241)
>         at 
> com.sun.security.sasl.digest.DigestMD5Server.validateClientResponse(DigestMD5Server.java:589)
>         ... 7 more
> {code}
>  
> In the DataNode log, we didn't see DataNode update block keys around 
> 2018-04-11 09:55:00 and around 2018-04-11 19:55:00.
> {code:java}
> 2018-04-10 14:51:36,424 INFO 
> 

[jira] [Commented] (HDFS-13441) DataNode missed BlockKey update from NameNode due to HeartbeatResponse was dropped

2018-04-17 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao commented on HDFS-13441:


{quote}Let DataNode pull Block Key from NameNode is one choice, since it need 
change protocol, if I didn't understand wrong, it will need go to Hadoop 
4.{quote}
[~zhaoyunjiong] 
The interface about get BlockKeys is ready now, reference: 
NamenodeProtocol#getBlockKeys.
if DataNode get Block Keys from NameNode using this interface, then it is more 
convenient to process kind of exceptions.

> DataNode missed BlockKey update from NameNode due to HeartbeatResponse was 
> dropped
> --
>
> Key: HDFS-13441
> URL: https://issues.apache.org/jira/browse/HDFS-13441
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode, namenode
>Affects Versions: 2.7.1
>Reporter: yunjiong zhao
>Assignee: yunjiong zhao
>Priority: Major
> Attachments: HDFS-13441.002.patch, HDFS-13441.patch
>
>
> After NameNode failover, lots of application failed due to some DataNodes 
> can't re-compute password from block token.
> {code:java}
> 2018-04-11 20:10:52,448 ERROR 
> org.apache.hadoop.hdfs.server.datanode.DataNode: 
> hdc3-lvs01-400-1701-048.stratus.lvs.ebay.com:50010:DataXceiver error 
> processing unknown operation  src: /10.142.74.116:57404 dst: 
> /10.142.77.45:50010
> javax.security.sasl.SaslException: DIGEST-MD5: IO error acquiring password 
> [Caused by org.apache.hadoop.security.token.SecretManager$InvalidToken: Can't 
> re-compute password for block_token_identifier (expiryDate=1523538652448, 
> keyId=1762737944, userId=hadoop, 
> blockPoolId=BP-36315570-10.103.108.13-1423055488042, blockId=12142862700, 
> access modes=[WRITE]), since the required block key (keyID=1762737944) 
> doesn't exist.]
>         at 
> com.sun.security.sasl.digest.DigestMD5Server.validateClientResponse(DigestMD5Server.java:598)
>         at 
> com.sun.security.sasl.digest.DigestMD5Server.evaluateResponse(DigestMD5Server.java:244)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslParticipant.evaluateChallengeOrResponse(SaslParticipant.java:115)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer.doSaslHandshake(SaslDataTransferServer.java:376)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer.getSaslStreams(SaslDataTransferServer.java:300)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer.receive(SaslDataTransferServer.java:127)
>         at 
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:194)
>         at java.lang.Thread.run(Thread.java:745)
> Caused by: org.apache.hadoop.security.token.SecretManager$InvalidToken: Can't 
> re-compute password for block_token_identifier (expiryDate=1523538652448, 
> keyId=1762737944, userId=hadoop, 
> blockPoolId=BP-36315570-10.103.108.13-1423055488042, blockId=12142862700, 
> access modes=[WRITE]), since the required block key (keyID=1762737944) 
> doesn't exist.
>         at 
> org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.retrievePassword(BlockTokenSecretManager.java:382)
>         at 
> org.apache.hadoop.hdfs.security.token.block.BlockPoolTokenSecretManager.retrievePassword(BlockPoolTokenSecretManager.java:79)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer.buildServerPassword(SaslDataTransferServer.java:318)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer.access$100(SaslDataTransferServer.java:73)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer$2.apply(SaslDataTransferServer.java:297)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer$SaslServerCallbackHandler.handle(SaslDataTransferServer.java:241)
>         at 
> com.sun.security.sasl.digest.DigestMD5Server.validateClientResponse(DigestMD5Server.java:589)
>         ... 7 more
> {code}
>  
> In the DataNode log, we didn't see DataNode update block keys around 
> 2018-04-11 09:55:00 and around 2018-04-11 19:55:00.
> {code:java}
> 2018-04-10 14:51:36,424 INFO 
> org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager: Setting 
> block keys
> 2018-04-10 23:55:38,420 INFO 
> org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager: Setting 
> block keys
> 2018-04-11 00:51:34,792 INFO 
> org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager: Setting 
> block keys
> 2018-04-11 10:51:39,403 INFO 
> org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager: Setting 
> block keys
> 2018-04-11 20:51:44,422 INFO 
> 

[jira] [Commented] (HDFS-13441) DataNode missed BlockKey update from NameNode due to HeartbeatResponse was dropped

2018-04-15 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao commented on HDFS-13441:


[~zhaoyunjiong] it is my misunderstand above
One minor suggestion for patch-v1, if catch IOException in {{DataXceiver}} and 
re-register DataNode to NameNode, it also could be failure, so I thinks this 
solution can reduce the possibility of failure only but not solute it 
completely.
Is there possibility that change the mode of getting BlockKey from *Push* by 
NameNode to *Pull* periodically by DataNode, just like scheduler 
{{BlockReport}} of DataNode.

> DataNode missed BlockKey update from NameNode due to HeartbeatResponse was 
> dropped
> --
>
> Key: HDFS-13441
> URL: https://issues.apache.org/jira/browse/HDFS-13441
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode, namenode
>Affects Versions: 2.7.1
>Reporter: yunjiong zhao
>Assignee: yunjiong zhao
>Priority: Major
> Attachments: HDFS-13441.patch
>
>
> After NameNode failover, lots of application failed due to some DataNodes 
> can't re-compute password from block token.
> {code:java}
> 2018-04-11 20:10:52,448 ERROR 
> org.apache.hadoop.hdfs.server.datanode.DataNode: 
> hdc3-lvs01-400-1701-048.stratus.lvs.ebay.com:50010:DataXceiver error 
> processing unknown operation  src: /10.142.74.116:57404 dst: 
> /10.142.77.45:50010
> javax.security.sasl.SaslException: DIGEST-MD5: IO error acquiring password 
> [Caused by org.apache.hadoop.security.token.SecretManager$InvalidToken: Can't 
> re-compute password for block_token_identifier (expiryDate=1523538652448, 
> keyId=1762737944, userId=hadoop, 
> blockPoolId=BP-36315570-10.103.108.13-1423055488042, blockId=12142862700, 
> access modes=[WRITE]), since the required block key (keyID=1762737944) 
> doesn't exist.]
>         at 
> com.sun.security.sasl.digest.DigestMD5Server.validateClientResponse(DigestMD5Server.java:598)
>         at 
> com.sun.security.sasl.digest.DigestMD5Server.evaluateResponse(DigestMD5Server.java:244)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslParticipant.evaluateChallengeOrResponse(SaslParticipant.java:115)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer.doSaslHandshake(SaslDataTransferServer.java:376)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer.getSaslStreams(SaslDataTransferServer.java:300)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer.receive(SaslDataTransferServer.java:127)
>         at 
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:194)
>         at java.lang.Thread.run(Thread.java:745)
> Caused by: org.apache.hadoop.security.token.SecretManager$InvalidToken: Can't 
> re-compute password for block_token_identifier (expiryDate=1523538652448, 
> keyId=1762737944, userId=hadoop, 
> blockPoolId=BP-36315570-10.103.108.13-1423055488042, blockId=12142862700, 
> access modes=[WRITE]), since the required block key (keyID=1762737944) 
> doesn't exist.
>         at 
> org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.retrievePassword(BlockTokenSecretManager.java:382)
>         at 
> org.apache.hadoop.hdfs.security.token.block.BlockPoolTokenSecretManager.retrievePassword(BlockPoolTokenSecretManager.java:79)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer.buildServerPassword(SaslDataTransferServer.java:318)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer.access$100(SaslDataTransferServer.java:73)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer$2.apply(SaslDataTransferServer.java:297)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer$SaslServerCallbackHandler.handle(SaslDataTransferServer.java:241)
>         at 
> com.sun.security.sasl.digest.DigestMD5Server.validateClientResponse(DigestMD5Server.java:589)
>         ... 7 more
> {code}
>  
> In the DataNode log, we didn't see DataNode update block keys around 
> 2018-04-11 09:55:00 and around 2018-04-11 19:55:00.
> {code:java}
> 2018-04-10 14:51:36,424 INFO 
> org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager: Setting 
> block keys
> 2018-04-10 23:55:38,420 INFO 
> org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager: Setting 
> block keys
> 2018-04-11 00:51:34,792 INFO 
> org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager: Setting 
> block keys
> 2018-04-11 10:51:39,403 INFO 
> org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager: Setting 
> block keys
> 2018-04-11 20:51:44,422 INFO 
> 

[jira] [Comment Edited] (HDFS-13441) DataNode missed BlockKey update from NameNode due to HeartbeatResponse was dropped

2018-04-14 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao edited comment on HDFS-13441 at 4/14/18 6:09 PM:
-

[~zhaoyunjiong], If I understand correctly, this issue was caused by 
StandbyNameNode restart.
NameNode's load is very high when restart in a large cluster, especially when 
NameNode starts to process block report, if there are some DataNode to 
reregister in this period, it may be timeout very likely, since 
BPServiceActor#register can not catch IOException which wrap over 
SocketTimeoutException. In one word, NameNode correctly processed the 
registration, but the DataNode timeout before receiving the response. so 
#updateBlockKeysWhenStartup could not be invoke.
{code:java}
  void register(NamespaceInfo nsInfo) throws IOException {
// The handshake() phase loaded the block pool storage
// off disk - so update the bpRegistration object from that info
DatanodeRegistration newBpRegistration = bpos.createRegistration();

LOG.info(this + " beginning handshake with NN");

while (shouldRun()) {
  try {
// Use returned registration from namenode with updated fields
newBpRegistration = bpNamenode.registerDatanode(newBpRegistration);
newBpRegistration.setNamespaceInfo(nsInfo);
bpRegistration = newBpRegistration;
break;
  } catch(EOFException e) {  // namenode might have just restarted
LOG.info("Problem connecting to server: " + nnAddr + " :"
+ e.getLocalizedMessage());
sleepAndLogInterrupts(1000, "connecting to server");
  } catch(SocketTimeoutException e) {  // namenode is busy
LOG.info("Problem connecting to server: " + nnAddr);
sleepAndLogInterrupts(1000, "connecting to server");
  }
}

LOG.info("Block pool " + this + " successfully registered with NN");
bpos.registrationSucceeded(this, bpRegistration);

// random short delay - helps scatter the BR from all DNs
scheduler.scheduleBlockReport(dnConf.initialBlockReportDelay);
updateBlockKeysWhenStartup();
  }
{code}

In this case, the following Read/Write from client to this DataNode would be 
certain to thrown {{SaslException}}. HDFS-12749 is trying to resolve this 
matter once for all. FYI.


was (Author: hexiaoqiao):
[~zhaoyunjiong], If I understand correctly, this issue was caused by 
StandbyNameNode restart.
NameNode's load is very high when restart in a large cluster, especially when 
NameNode starts to process block report, if there are some DataNode to 
reregister in this period, it may be timeout very likely, since 
BPServiceActor#register can not catch SocketTimeoutException. In one word, 
NameNode correctly processed the registration, but the DataNode timeout before 
receiving the response. so #updateBlockKeysWhenStartup could not be invoke.
{code:java}
  void register(NamespaceInfo nsInfo) throws IOException {
// The handshake() phase loaded the block pool storage
// off disk - so update the bpRegistration object from that info
DatanodeRegistration newBpRegistration = bpos.createRegistration();

LOG.info(this + " beginning handshake with NN");

while (shouldRun()) {
  try {
// Use returned registration from namenode with updated fields
newBpRegistration = bpNamenode.registerDatanode(newBpRegistration);
newBpRegistration.setNamespaceInfo(nsInfo);
bpRegistration = newBpRegistration;
break;
  } catch(EOFException e) {  // namenode might have just restarted
LOG.info("Problem connecting to server: " + nnAddr + " :"
+ e.getLocalizedMessage());
sleepAndLogInterrupts(1000, "connecting to server");
  } catch(SocketTimeoutException e) {  // namenode is busy
LOG.info("Problem connecting to server: " + nnAddr);
sleepAndLogInterrupts(1000, "connecting to server");
  }
}

LOG.info("Block pool " + this + " successfully registered with NN");
bpos.registrationSucceeded(this, bpRegistration);

// random short delay - helps scatter the BR from all DNs
scheduler.scheduleBlockReport(dnConf.initialBlockReportDelay);
updateBlockKeysWhenStartup();
  }
{code}

In this case, the following Read/Write from client to this DataNode would be 
certain to thrown {{SaslException}}. HDFS-12749 is trying to resolve this 
matter once for all. FYI.

> DataNode missed BlockKey update from NameNode due to HeartbeatResponse was 
> dropped
> --
>
> Key: HDFS-13441
> URL: https://issues.apache.org/jira/browse/HDFS-13441
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode, namenode
>Affects Versions: 2.7.1
>Reporter: yunjiong zhao
>Assignee: 

[jira] [Commented] (HDFS-13441) DataNode missed BlockKey update from NameNode due to HeartbeatResponse was dropped

2018-04-14 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao commented on HDFS-13441:


[~zhaoyunjiong], If I understand correctly, this issue was caused by 
StandbyNameNode restart.
NameNode's load is very high when restart in a large cluster, especially when 
NameNode starts to process block report, if there are some DataNode to 
reregister in this period, it may be timeout very likely, since 
BPServiceActor#register can not catch SocketTimeoutException. In one word, 
NameNode correctly processed the registration, but the DataNode timeout before 
receiving the response. so #updateBlockKeysWhenStartup could not be invoke.
{code:java}
  void register(NamespaceInfo nsInfo) throws IOException {
// The handshake() phase loaded the block pool storage
// off disk - so update the bpRegistration object from that info
DatanodeRegistration newBpRegistration = bpos.createRegistration();

LOG.info(this + " beginning handshake with NN");

while (shouldRun()) {
  try {
// Use returned registration from namenode with updated fields
newBpRegistration = bpNamenode.registerDatanode(newBpRegistration);
newBpRegistration.setNamespaceInfo(nsInfo);
bpRegistration = newBpRegistration;
break;
  } catch(EOFException e) {  // namenode might have just restarted
LOG.info("Problem connecting to server: " + nnAddr + " :"
+ e.getLocalizedMessage());
sleepAndLogInterrupts(1000, "connecting to server");
  } catch(SocketTimeoutException e) {  // namenode is busy
LOG.info("Problem connecting to server: " + nnAddr);
sleepAndLogInterrupts(1000, "connecting to server");
  }
}

LOG.info("Block pool " + this + " successfully registered with NN");
bpos.registrationSucceeded(this, bpRegistration);

// random short delay - helps scatter the BR from all DNs
scheduler.scheduleBlockReport(dnConf.initialBlockReportDelay);
updateBlockKeysWhenStartup();
  }
{code}

In this case, the following Read/Write from client to this DataNode would be 
certain to thrown {{SaslException}}. HDFS-12749 is trying to resolve this 
matter once for all. FYI.

> DataNode missed BlockKey update from NameNode due to HeartbeatResponse was 
> dropped
> --
>
> Key: HDFS-13441
> URL: https://issues.apache.org/jira/browse/HDFS-13441
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode, namenode
>Affects Versions: 2.7.1
>Reporter: yunjiong zhao
>Assignee: yunjiong zhao
>Priority: Major
> Attachments: HDFS-13441.patch
>
>
> After NameNode failover, lots of application failed due to some DataNodes 
> can't re-compute password from block token.
> {code:java}
> 2018-04-11 20:10:52,448 ERROR 
> org.apache.hadoop.hdfs.server.datanode.DataNode: 
> hdc3-lvs01-400-1701-048.stratus.lvs.ebay.com:50010:DataXceiver error 
> processing unknown operation  src: /10.142.74.116:57404 dst: 
> /10.142.77.45:50010
> javax.security.sasl.SaslException: DIGEST-MD5: IO error acquiring password 
> [Caused by org.apache.hadoop.security.token.SecretManager$InvalidToken: Can't 
> re-compute password for block_token_identifier (expiryDate=1523538652448, 
> keyId=1762737944, userId=hadoop, 
> blockPoolId=BP-36315570-10.103.108.13-1423055488042, blockId=12142862700, 
> access modes=[WRITE]), since the required block key (keyID=1762737944) 
> doesn't exist.]
>         at 
> com.sun.security.sasl.digest.DigestMD5Server.validateClientResponse(DigestMD5Server.java:598)
>         at 
> com.sun.security.sasl.digest.DigestMD5Server.evaluateResponse(DigestMD5Server.java:244)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslParticipant.evaluateChallengeOrResponse(SaslParticipant.java:115)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer.doSaslHandshake(SaslDataTransferServer.java:376)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer.getSaslStreams(SaslDataTransferServer.java:300)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer.receive(SaslDataTransferServer.java:127)
>         at 
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:194)
>         at java.lang.Thread.run(Thread.java:745)
> Caused by: org.apache.hadoop.security.token.SecretManager$InvalidToken: Can't 
> re-compute password for block_token_identifier (expiryDate=1523538652448, 
> keyId=1762737944, userId=hadoop, 
> blockPoolId=BP-36315570-10.103.108.13-1423055488042, blockId=12142862700, 
> access modes=[WRITE]), since the required block key (keyID=1762737944) 
> doesn't exist.
>         at 
> 

[jira] [Commented] (HDFS-13441) DataNode missed BlockKey update from NameNode due to HeartbeatResponse was dropped

2018-04-12 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao commented on HDFS-13441:


the detailed analysis ref: 
https://issues.apache.org/jira/browse/HDFS-12749?focusedCommentId=16364280=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-16364280

> DataNode missed BlockKey update from NameNode due to HeartbeatResponse was 
> dropped
> --
>
> Key: HDFS-13441
> URL: https://issues.apache.org/jira/browse/HDFS-13441
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode, namenode
>Affects Versions: 2.7.1
>Reporter: yunjiong zhao
>Assignee: yunjiong zhao
>Priority: Major
> Attachments: HDFS-13441.patch
>
>
> After NameNode failover, lots of application failed due to some DataNodes 
> can't re-compute password from block token.
> {code:java}
> 2018-04-11 20:10:52,448 ERROR 
> org.apache.hadoop.hdfs.server.datanode.DataNode: 
> hdc3-lvs01-400-1701-048.stratus.lvs.ebay.com:50010:DataXceiver error 
> processing unknown operation  src: /10.142.74.116:57404 dst: 
> /10.142.77.45:50010
> javax.security.sasl.SaslException: DIGEST-MD5: IO error acquiring password 
> [Caused by org.apache.hadoop.security.token.SecretManager$InvalidToken: Can't 
> re-compute password for block_token_identifier (expiryDate=1523538652448, 
> keyId=1762737944, userId=hadoop, 
> blockPoolId=BP-36315570-10.103.108.13-1423055488042, blockId=12142862700, 
> access modes=[WRITE]), since the required block key (keyID=1762737944) 
> doesn't exist.]
>         at 
> com.sun.security.sasl.digest.DigestMD5Server.validateClientResponse(DigestMD5Server.java:598)
>         at 
> com.sun.security.sasl.digest.DigestMD5Server.evaluateResponse(DigestMD5Server.java:244)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslParticipant.evaluateChallengeOrResponse(SaslParticipant.java:115)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer.doSaslHandshake(SaslDataTransferServer.java:376)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer.getSaslStreams(SaslDataTransferServer.java:300)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer.receive(SaslDataTransferServer.java:127)
>         at 
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:194)
>         at java.lang.Thread.run(Thread.java:745)
> Caused by: org.apache.hadoop.security.token.SecretManager$InvalidToken: Can't 
> re-compute password for block_token_identifier (expiryDate=1523538652448, 
> keyId=1762737944, userId=hadoop, 
> blockPoolId=BP-36315570-10.103.108.13-1423055488042, blockId=12142862700, 
> access modes=[WRITE]), since the required block key (keyID=1762737944) 
> doesn't exist.
>         at 
> org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.retrievePassword(BlockTokenSecretManager.java:382)
>         at 
> org.apache.hadoop.hdfs.security.token.block.BlockPoolTokenSecretManager.retrievePassword(BlockPoolTokenSecretManager.java:79)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer.buildServerPassword(SaslDataTransferServer.java:318)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer.access$100(SaslDataTransferServer.java:73)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer$2.apply(SaslDataTransferServer.java:297)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer$SaslServerCallbackHandler.handle(SaslDataTransferServer.java:241)
>         at 
> com.sun.security.sasl.digest.DigestMD5Server.validateClientResponse(DigestMD5Server.java:589)
>         ... 7 more
> {code}
>  
> In the DataNode log, we didn't see DataNode update block keys around 
> 2018-04-11 09:55.
> {code:java}
> 2018-04-10 14:51:36,424 INFO 
> org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager: Setting 
> block keys
> 2018-04-10 23:55:38,420 INFO 
> org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager: Setting 
> block keys
> 2018-04-11 00:51:34,792 INFO 
> org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager: Setting 
> block keys
> 2018-04-11 10:51:39,403 INFO 
> org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager: Setting 
> block keys
> 2018-04-11 20:51:44,422 INFO 
> org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager: Setting 
> block keys
> 2018-04-12 02:54:47,855 INFO 
> org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager: Setting 
> block keys
> 2018-04-12 05:55:44,456 INFO 
> 

[jira] [Commented] (HDFS-13441) DataNode missed BlockKey update from NameNode due to HeartbeatResponse was dropped

2018-04-12 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao commented on HDFS-13441:


hi [~zhaoyunjiong],[~jojochuang]
we meet the similar problem in our production cluster, and when dig deeply, we 
found that the root case is about reregister. HDFS-12749 is following this 
case. FYI.

> DataNode missed BlockKey update from NameNode due to HeartbeatResponse was 
> dropped
> --
>
> Key: HDFS-13441
> URL: https://issues.apache.org/jira/browse/HDFS-13441
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode, namenode
>Affects Versions: 2.7.1
>Reporter: yunjiong zhao
>Assignee: yunjiong zhao
>Priority: Major
> Attachments: HDFS-13441.patch
>
>
> After NameNode failover, lots of application failed due to some DataNodes 
> can't re-compute password from block token.
> {code:java}
> 2018-04-11 20:10:52,448 ERROR 
> org.apache.hadoop.hdfs.server.datanode.DataNode: 
> hdc3-lvs01-400-1701-048.stratus.lvs.ebay.com:50010:DataXceiver error 
> processing unknown operation  src: /10.142.74.116:57404 dst: 
> /10.142.77.45:50010
> javax.security.sasl.SaslException: DIGEST-MD5: IO error acquiring password 
> [Caused by org.apache.hadoop.security.token.SecretManager$InvalidToken: Can't 
> re-compute password for block_token_identifier (expiryDate=1523538652448, 
> keyId=1762737944, userId=hadoop, 
> blockPoolId=BP-36315570-10.103.108.13-1423055488042, blockId=12142862700, 
> access modes=[WRITE]), since the required block key (keyID=1762737944) 
> doesn't exist.]
>         at 
> com.sun.security.sasl.digest.DigestMD5Server.validateClientResponse(DigestMD5Server.java:598)
>         at 
> com.sun.security.sasl.digest.DigestMD5Server.evaluateResponse(DigestMD5Server.java:244)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslParticipant.evaluateChallengeOrResponse(SaslParticipant.java:115)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer.doSaslHandshake(SaslDataTransferServer.java:376)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer.getSaslStreams(SaslDataTransferServer.java:300)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer.receive(SaslDataTransferServer.java:127)
>         at 
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:194)
>         at java.lang.Thread.run(Thread.java:745)
> Caused by: org.apache.hadoop.security.token.SecretManager$InvalidToken: Can't 
> re-compute password for block_token_identifier (expiryDate=1523538652448, 
> keyId=1762737944, userId=hadoop, 
> blockPoolId=BP-36315570-10.103.108.13-1423055488042, blockId=12142862700, 
> access modes=[WRITE]), since the required block key (keyID=1762737944) 
> doesn't exist.
>         at 
> org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.retrievePassword(BlockTokenSecretManager.java:382)
>         at 
> org.apache.hadoop.hdfs.security.token.block.BlockPoolTokenSecretManager.retrievePassword(BlockPoolTokenSecretManager.java:79)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer.buildServerPassword(SaslDataTransferServer.java:318)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer.access$100(SaslDataTransferServer.java:73)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer$2.apply(SaslDataTransferServer.java:297)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer$SaslServerCallbackHandler.handle(SaslDataTransferServer.java:241)
>         at 
> com.sun.security.sasl.digest.DigestMD5Server.validateClientResponse(DigestMD5Server.java:589)
>         ... 7 more
> {code}
>  
> In the DataNode log, we didn't see DataNode update block keys around 
> 2018-04-11 09:55.
> {code:java}
> 2018-04-10 14:51:36,424 INFO 
> org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager: Setting 
> block keys
> 2018-04-10 23:55:38,420 INFO 
> org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager: Setting 
> block keys
> 2018-04-11 00:51:34,792 INFO 
> org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager: Setting 
> block keys
> 2018-04-11 10:51:39,403 INFO 
> org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager: Setting 
> block keys
> 2018-04-11 20:51:44,422 INFO 
> org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager: Setting 
> block keys
> 2018-04-12 02:54:47,855 INFO 
> org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager: Setting 
> block keys
> 2018-04-12 05:55:44,456 INFO 
> 

[jira] [Commented] (HDFS-12976) Introduce StandbyReadProxyProvider

2018-04-07 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao commented on HDFS-12976:


[~csun] is this issue still going on?

> Introduce StandbyReadProxyProvider
> --
>
> Key: HDFS-12976
> URL: https://issues.apache.org/jira/browse/HDFS-12976
> Project: Hadoop HDFS
>  Issue Type: Sub-task
>  Components: hdfs-client
>Reporter: Konstantin Shvachko
>Assignee: Chao Sun
>Priority: Major
>
> {{StandbyReadProxyProvider}} should implement {{FailoverProxyProvider}} 
> interface and be able to submit read requests to ANN and SBN(s).



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (HDFS-12749) DN may not send block report to NN after NN restart

2018-04-03 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao commented on HDFS-12749:


update patch v4 and fix bug based comments and without testcase.
[~kihwal], I don't find a grace way to test the scenario that NN correctly 
processed the registration, but the DN timed out before receiving the response, 
do you have a good idea?
Thanks again.

> DN may not send block report to NN after NN restart
> ---
>
> Key: HDFS-12749
> URL: https://issues.apache.org/jira/browse/HDFS-12749
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode
>Affects Versions: 2.7.1, 2.8.3, 2.7.5, 3.0.0, 2.9.1
>Reporter: TanYuxin
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-12749-branch-2.7.002.patch, 
> HDFS-12749-trunk.003.patch, HDFS-12749-trunk.004.patch, HDFS-12749.001.patch
>
>
> Now our cluster have thousands of DN, millions of files and blocks. When NN 
> restart, NN's load is very high.
> After NN restart,DN will call BPServiceActor#reRegister method to register. 
> But register RPC will get a IOException since NN is busy dealing with Block 
> Report.  The exception is caught at BPServiceActor#processCommand.
> Next is the caught IOException:
> {code:java}
> WARN org.apache.hadoop.hdfs.server.datanode.DataNode: Error processing 
> datanode Command
> java.io.IOException: Failed on local exception: java.io.IOException: 
> java.net.SocketTimeoutException: 6 millis timeout while waiting for 
> channel to be ready for read. ch : java.nio.channels.SocketChannel[connected 
> local=/DataNode_IP:Port remote=NameNode_Host/IP:Port]; Host Details : local 
> host is: "DataNode_Host/Datanode_IP"; destination host is: 
> "NameNode_Host":Port;
> at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:773)
> at org.apache.hadoop.ipc.Client.call(Client.java:1474)
> at org.apache.hadoop.ipc.Client.call(Client.java:1407)
> at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:229)
> at com.sun.proxy.$Proxy13.registerDatanode(Unknown Source)
> at 
> org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB.registerDatanode(DatanodeProtocolClientSideTranslatorPB.java:126)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.register(BPServiceActor.java:793)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.reRegister(BPServiceActor.java:926)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPOfferService.processCommandFromActor(BPOfferService.java:604)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.processCommand(BPServiceActor.java:898)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.offerService(BPServiceActor.java:711)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.run(BPServiceActor.java:864)
> at java.lang.Thread.run(Thread.java:745)
> {code}
> The un-catched IOException breaks BPServiceActor#register, and the Block 
> Report can not be sent immediately. 
> {code}
>   /**
>* Register one bp with the corresponding NameNode
>* 
>* The bpDatanode needs to register with the namenode on startup in order
>* 1) to report which storage it is serving now and 
>* 2) to receive a registrationID
>*  
>* issued by the namenode to recognize registered datanodes.
>* 
>* @param nsInfo current NamespaceInfo
>* @see FSNamesystem#registerDatanode(DatanodeRegistration)
>* @throws IOException
>*/
>   void register(NamespaceInfo nsInfo) throws IOException {
> // The handshake() phase loaded the block pool storage
> // off disk - so update the bpRegistration object from that info
> DatanodeRegistration newBpRegistration = bpos.createRegistration();
> LOG.info(this + " beginning handshake with NN");
> while (shouldRun()) {
>   try {
> // Use returned registration from namenode with updated fields
> newBpRegistration = bpNamenode.registerDatanode(newBpRegistration);
> newBpRegistration.setNamespaceInfo(nsInfo);
> bpRegistration = newBpRegistration;
> break;
>   } catch(EOFException e) {  // namenode might have just restarted
> LOG.info("Problem connecting to server: " + nnAddr + " :"
> + e.getLocalizedMessage());
> sleepAndLogInterrupts(1000, "connecting to server");
>   } catch(SocketTimeoutException e) {  // namenode is busy
> LOG.info("Problem connecting to server: " + nnAddr);
> sleepAndLogInterrupts(1000, "connecting to server");
>   }
> }
> 
> LOG.info("Block pool " + this + " successfully registered with NN");
> 

[jira] [Updated] (HDFS-12749) DN may not send block report to NN after NN restart

2018-04-03 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao updated HDFS-12749:
---
Attachment: HDFS-12749-trunk.004.patch

> DN may not send block report to NN after NN restart
> ---
>
> Key: HDFS-12749
> URL: https://issues.apache.org/jira/browse/HDFS-12749
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode
>Affects Versions: 2.7.1, 2.8.3, 2.7.5, 3.0.0, 2.9.1
>Reporter: TanYuxin
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-12749-branch-2.7.002.patch, 
> HDFS-12749-trunk.003.patch, HDFS-12749-trunk.004.patch, HDFS-12749.001.patch
>
>
> Now our cluster have thousands of DN, millions of files and blocks. When NN 
> restart, NN's load is very high.
> After NN restart,DN will call BPServiceActor#reRegister method to register. 
> But register RPC will get a IOException since NN is busy dealing with Block 
> Report.  The exception is caught at BPServiceActor#processCommand.
> Next is the caught IOException:
> {code:java}
> WARN org.apache.hadoop.hdfs.server.datanode.DataNode: Error processing 
> datanode Command
> java.io.IOException: Failed on local exception: java.io.IOException: 
> java.net.SocketTimeoutException: 6 millis timeout while waiting for 
> channel to be ready for read. ch : java.nio.channels.SocketChannel[connected 
> local=/DataNode_IP:Port remote=NameNode_Host/IP:Port]; Host Details : local 
> host is: "DataNode_Host/Datanode_IP"; destination host is: 
> "NameNode_Host":Port;
> at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:773)
> at org.apache.hadoop.ipc.Client.call(Client.java:1474)
> at org.apache.hadoop.ipc.Client.call(Client.java:1407)
> at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:229)
> at com.sun.proxy.$Proxy13.registerDatanode(Unknown Source)
> at 
> org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB.registerDatanode(DatanodeProtocolClientSideTranslatorPB.java:126)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.register(BPServiceActor.java:793)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.reRegister(BPServiceActor.java:926)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPOfferService.processCommandFromActor(BPOfferService.java:604)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.processCommand(BPServiceActor.java:898)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.offerService(BPServiceActor.java:711)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.run(BPServiceActor.java:864)
> at java.lang.Thread.run(Thread.java:745)
> {code}
> The un-catched IOException breaks BPServiceActor#register, and the Block 
> Report can not be sent immediately. 
> {code}
>   /**
>* Register one bp with the corresponding NameNode
>* 
>* The bpDatanode needs to register with the namenode on startup in order
>* 1) to report which storage it is serving now and 
>* 2) to receive a registrationID
>*  
>* issued by the namenode to recognize registered datanodes.
>* 
>* @param nsInfo current NamespaceInfo
>* @see FSNamesystem#registerDatanode(DatanodeRegistration)
>* @throws IOException
>*/
>   void register(NamespaceInfo nsInfo) throws IOException {
> // The handshake() phase loaded the block pool storage
> // off disk - so update the bpRegistration object from that info
> DatanodeRegistration newBpRegistration = bpos.createRegistration();
> LOG.info(this + " beginning handshake with NN");
> while (shouldRun()) {
>   try {
> // Use returned registration from namenode with updated fields
> newBpRegistration = bpNamenode.registerDatanode(newBpRegistration);
> newBpRegistration.setNamespaceInfo(nsInfo);
> bpRegistration = newBpRegistration;
> break;
>   } catch(EOFException e) {  // namenode might have just restarted
> LOG.info("Problem connecting to server: " + nnAddr + " :"
> + e.getLocalizedMessage());
> sleepAndLogInterrupts(1000, "connecting to server");
>   } catch(SocketTimeoutException e) {  // namenode is busy
> LOG.info("Problem connecting to server: " + nnAddr);
> sleepAndLogInterrupts(1000, "connecting to server");
>   }
> }
> 
> LOG.info("Block pool " + this + " successfully registered with NN");
> bpos.registrationSucceeded(this, bpRegistration);
> // random short delay - helps scatter the BR from all DNs
> scheduler.scheduleBlockReport(dnConf.initialBlockReportDelay);
>   }
> {code}
> But NameNode has processed registerDatanode successfully, so it won't ask DN 
> to 

[jira] [Commented] (HDFS-12749) DN may not send block report to NN after NN restart

2018-04-02 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao commented on HDFS-12749:


[~kihwal] Thanks for your feedback. I misunderstood your idea above. It is 
indeed to fix in the catch block of {{processCommand()}},  I will solve this 
issue in couple of days.

> DN may not send block report to NN after NN restart
> ---
>
> Key: HDFS-12749
> URL: https://issues.apache.org/jira/browse/HDFS-12749
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode
>Affects Versions: 2.7.1, 2.8.3, 2.7.5, 3.0.0, 2.9.1
>Reporter: TanYuxin
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-12749-branch-2.7.002.patch, 
> HDFS-12749-trunk.003.patch, HDFS-12749.001.patch
>
>
> Now our cluster have thousands of DN, millions of files and blocks. When NN 
> restart, NN's load is very high.
> After NN restart,DN will call BPServiceActor#reRegister method to register. 
> But register RPC will get a IOException since NN is busy dealing with Block 
> Report.  The exception is caught at BPServiceActor#processCommand.
> Next is the caught IOException:
> {code:java}
> WARN org.apache.hadoop.hdfs.server.datanode.DataNode: Error processing 
> datanode Command
> java.io.IOException: Failed on local exception: java.io.IOException: 
> java.net.SocketTimeoutException: 6 millis timeout while waiting for 
> channel to be ready for read. ch : java.nio.channels.SocketChannel[connected 
> local=/DataNode_IP:Port remote=NameNode_Host/IP:Port]; Host Details : local 
> host is: "DataNode_Host/Datanode_IP"; destination host is: 
> "NameNode_Host":Port;
> at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:773)
> at org.apache.hadoop.ipc.Client.call(Client.java:1474)
> at org.apache.hadoop.ipc.Client.call(Client.java:1407)
> at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:229)
> at com.sun.proxy.$Proxy13.registerDatanode(Unknown Source)
> at 
> org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB.registerDatanode(DatanodeProtocolClientSideTranslatorPB.java:126)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.register(BPServiceActor.java:793)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.reRegister(BPServiceActor.java:926)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPOfferService.processCommandFromActor(BPOfferService.java:604)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.processCommand(BPServiceActor.java:898)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.offerService(BPServiceActor.java:711)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.run(BPServiceActor.java:864)
> at java.lang.Thread.run(Thread.java:745)
> {code}
> The un-catched IOException breaks BPServiceActor#register, and the Block 
> Report can not be sent immediately. 
> {code}
>   /**
>* Register one bp with the corresponding NameNode
>* 
>* The bpDatanode needs to register with the namenode on startup in order
>* 1) to report which storage it is serving now and 
>* 2) to receive a registrationID
>*  
>* issued by the namenode to recognize registered datanodes.
>* 
>* @param nsInfo current NamespaceInfo
>* @see FSNamesystem#registerDatanode(DatanodeRegistration)
>* @throws IOException
>*/
>   void register(NamespaceInfo nsInfo) throws IOException {
> // The handshake() phase loaded the block pool storage
> // off disk - so update the bpRegistration object from that info
> DatanodeRegistration newBpRegistration = bpos.createRegistration();
> LOG.info(this + " beginning handshake with NN");
> while (shouldRun()) {
>   try {
> // Use returned registration from namenode with updated fields
> newBpRegistration = bpNamenode.registerDatanode(newBpRegistration);
> newBpRegistration.setNamespaceInfo(nsInfo);
> bpRegistration = newBpRegistration;
> break;
>   } catch(EOFException e) {  // namenode might have just restarted
> LOG.info("Problem connecting to server: " + nnAddr + " :"
> + e.getLocalizedMessage());
> sleepAndLogInterrupts(1000, "connecting to server");
>   } catch(SocketTimeoutException e) {  // namenode is busy
> LOG.info("Problem connecting to server: " + nnAddr);
> sleepAndLogInterrupts(1000, "connecting to server");
>   }
> }
> 
> LOG.info("Block pool " + this + " successfully registered with NN");
> bpos.registrationSucceeded(this, bpRegistration);
> // random short delay - helps scatter the BR from all DNs
> 

[jira] [Commented] (HDFS-12749) DN may not send block report to NN after NN restart

2018-03-28 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao commented on HDFS-12749:


Thanks [~kihwal] for your detailed comment.
{quote}You can keep your change in register() and simply add the same logic to 
the processCommand()'s catch block. I.e. crack open the RemoteException and 
stop the actor thread if it is one of the terminal exceptions.{quote}
I think it may not be able to resolve this issue when catch and crack 
{{RemoteException}} for #processCommand, since #processCommand throws 
{{IOException}} which wrap {{SocketTimeoutException}} as [~tanyuxin] mentioned 
in description
{quote}java.io.IOException: Failed on local exception: java.io.IOException: 
java.net.SocketTimeoutException: 6 millis timeout while waiting for channel 
to be ready for read. ch : java.nio.channels.SocketChannel{quote}
According to your suggestions, is it better to create new issue to push that 
stop the actor thread if it meet some fatal or terminal exceptions? 

> DN may not send block report to NN after NN restart
> ---
>
> Key: HDFS-12749
> URL: https://issues.apache.org/jira/browse/HDFS-12749
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode
>Affects Versions: 2.7.1, 2.8.3, 2.7.5, 3.0.0, 2.9.1
>Reporter: TanYuxin
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-12749-branch-2.7.002.patch, 
> HDFS-12749-trunk.003.patch, HDFS-12749.001.patch
>
>
> Now our cluster have thousands of DN, millions of files and blocks. When NN 
> restart, NN's load is very high.
> After NN restart,DN will call BPServiceActor#reRegister method to register. 
> But register RPC will get a IOException since NN is busy dealing with Block 
> Report.  The exception is caught at BPServiceActor#processCommand.
> Next is the caught IOException:
> {code:java}
> WARN org.apache.hadoop.hdfs.server.datanode.DataNode: Error processing 
> datanode Command
> java.io.IOException: Failed on local exception: java.io.IOException: 
> java.net.SocketTimeoutException: 6 millis timeout while waiting for 
> channel to be ready for read. ch : java.nio.channels.SocketChannel[connected 
> local=/DataNode_IP:Port remote=NameNode_Host/IP:Port]; Host Details : local 
> host is: "DataNode_Host/Datanode_IP"; destination host is: 
> "NameNode_Host":Port;
> at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:773)
> at org.apache.hadoop.ipc.Client.call(Client.java:1474)
> at org.apache.hadoop.ipc.Client.call(Client.java:1407)
> at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:229)
> at com.sun.proxy.$Proxy13.registerDatanode(Unknown Source)
> at 
> org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB.registerDatanode(DatanodeProtocolClientSideTranslatorPB.java:126)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.register(BPServiceActor.java:793)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.reRegister(BPServiceActor.java:926)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPOfferService.processCommandFromActor(BPOfferService.java:604)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.processCommand(BPServiceActor.java:898)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.offerService(BPServiceActor.java:711)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.run(BPServiceActor.java:864)
> at java.lang.Thread.run(Thread.java:745)
> {code}
> The un-catched IOException breaks BPServiceActor#register, and the Block 
> Report can not be sent immediately. 
> {code}
>   /**
>* Register one bp with the corresponding NameNode
>* 
>* The bpDatanode needs to register with the namenode on startup in order
>* 1) to report which storage it is serving now and 
>* 2) to receive a registrationID
>*  
>* issued by the namenode to recognize registered datanodes.
>* 
>* @param nsInfo current NamespaceInfo
>* @see FSNamesystem#registerDatanode(DatanodeRegistration)
>* @throws IOException
>*/
>   void register(NamespaceInfo nsInfo) throws IOException {
> // The handshake() phase loaded the block pool storage
> // off disk - so update the bpRegistration object from that info
> DatanodeRegistration newBpRegistration = bpos.createRegistration();
> LOG.info(this + " beginning handshake with NN");
> while (shouldRun()) {
>   try {
> // Use returned registration from namenode with updated fields
> newBpRegistration = bpNamenode.registerDatanode(newBpRegistration);
> newBpRegistration.setNamespaceInfo(nsInfo);
> bpRegistration = 

[jira] [Commented] (HDFS-13234) Remove renew configuration instance in ConfiguredFailoverProxyProvider and reduce memory footprint for client

2018-03-11 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao commented on HDFS-13234:


Thanks [~kihwal] for your detailed comments.
It is interesting issues (HADOOP-11223 and HADOOP-9570) for resolving 
duplicated Configuration instances. But I am not sure if these issue are 
complete solution for huge memory footprint waste of the case mentioned above. 
Beside HADOOP-11223 and HADOOP-9570, I think it is necessary to maintain 
incremental change for Configuration, thus Configuration::getDefault() + 
Incremental Change could form the complete configuration and no unintended conf 
update propagation, meaning while it could reduce memory footprint. If I am 
wrong please correct me.
Thanks again.

> Remove renew configuration instance in ConfiguredFailoverProxyProvider and 
> reduce memory footprint for client
> -
>
> Key: HDFS-13234
> URL: https://issues.apache.org/jira/browse/HDFS-13234
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: fs, ha, hdfs-client
>Reporter: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-13234.001.patch
>
>
> The memory footprint of #DFSClient is very considerable in some special 
> scenario since there are many #Configuration instances and occupy much memory 
> resource (In an extreme case, org.apache.hadoop.conf.Configuration occupies 
> over 600MB we meet under HDFS Federation an HA with QJM and there are dozens 
> of NameNodes). I think some new Configuration instance is not necessary. Such 
> as  #ConfiguredFailoverProxyProvider initialization.
> {code:java}
>   public ConfiguredFailoverProxyProvider(Configuration conf, URI uri,
>   Class xface, HAProxyFactory factory) {
> this.xface = xface;
> this.conf = new Configuration(conf);
> ..
>   }
> {code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (HDFS-13096) HDFS group quota

2018-03-08 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao commented on HDFS-13096:


[~Tagar], is this issue going on?

> HDFS group quota
> 
>
> Key: HDFS-13096
> URL: https://issues.apache.org/jira/browse/HDFS-13096
> Project: Hadoop HDFS
>  Issue Type: New Feature
>  Components: datanode, fs, hdfs, nn
>Affects Versions: 2.8.3, 2.7.5, 3.0.0
>Reporter: Ruslan Dautkhanov
>Priority: Major
>
> We have groups of people that have their own set of HDFS directories. 
> For example, they have HDFS staging place for new files:
> /datascience
> /analysts 
> ... 
> but at the same time they have Hive warehouse directory 
> /hivewarehouse/datascience
> /hivewarehouse/analysts 
> ... 
> on top of that they also have some files stored under /user/${username}/ 
> It's always been a challenge to maintain a combined quota on all HDFS 
> locations a particular group of people owns. As we're currently forced to put 
> a particular quota for each directory independently.
> It would be great if HDFS would have a quota tied either
> - to a set of HDFS locations ;
> - or to a group of people (where `group`is defined as which HDFS group a 
> particular file/directory belongs to).
> Linux allows to define quotas at group level, i.e. `edquota -g devel` etc.. 
> would be great to have the same at HDFS level.
> Other thoughts and ideas?



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (HDFS-13234) Remove renew configuration instance in ConfiguredFailoverProxyProvider and reduce memory footprint for client

2018-03-07 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao commented on HDFS-13234:


Thanks [~kihwal],[~elgoiri] for your comments.
{quote}How big is a single instance in your use case? Bloated conf in dfs 
client is obviously a serious issue, but it can create bigger issues in 
apps/jobs.{quote}
Actually this is yarn logs upload service, and the size of single 
{{Configuration}} instance which located at NodeManager is about 120KB, but it 
is bloated to 600MB over all {{Configuration}} instances since two factors:
a. HDFS Federation + HA with QJM and there are dozens of nameservices (~20), 
and it create {{ConfiguredFailoverProxyProvider}} instance for each name 
service at client,  while num of {{Configuration}} instances will *2;
b. there are 150 single threads at most to execute upload yarn logs to HDFS;
so, in the extreme case, memory footprint of {{Configuration}} instances will 
occupy ~20 * 2 * 150 * 120KB;

{quote}New conf objects are created to prevent unintended conf update 
propagation. {quote}
it is true to prevent unintended conf update propagation, I think there are 
other ways to avoid clone the whole conf for only two parameters of 
{{ConfiguredFailoverProxyProvider}} and {{IPFailoverProxyProvider}} and waste 
huge memory resource probably as you mentioned, is there some suggestions? 
[~kihwal]

Thanks again.

> Remove renew configuration instance in ConfiguredFailoverProxyProvider and 
> reduce memory footprint for client
> -
>
> Key: HDFS-13234
> URL: https://issues.apache.org/jira/browse/HDFS-13234
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: fs, ha, hdfs-client
>Reporter: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-13234.001.patch
>
>
> The memory footprint of #DFSClient is very considerable in some special 
> scenario since there are many #Configuration instances and occupy much memory 
> resource (In an extreme case, org.apache.hadoop.conf.Configuration occupies 
> over 600MB we meet under HDFS Federation an HA with QJM and there are dozens 
> of NameNodes). I think some new Configuration instance is not necessary. Such 
> as  #ConfiguredFailoverProxyProvider initialization.
> {code:java}
>   public ConfiguredFailoverProxyProvider(Configuration conf, URI uri,
>   Class xface, HAProxyFactory factory) {
> this.xface = xface;
> this.conf = new Configuration(conf);
> ..
>   }
> {code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (HDFS-13234) Remove renew configuration instance in ConfiguredFailoverProxyProvider and reduce memory footprint for client

2018-03-07 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao commented on HDFS-13234:


upload patch v1 for trunk and pending jenkins.

> Remove renew configuration instance in ConfiguredFailoverProxyProvider and 
> reduce memory footprint for client
> -
>
> Key: HDFS-13234
> URL: https://issues.apache.org/jira/browse/HDFS-13234
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: fs, ha, hdfs-client
>Reporter: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-13234.001.patch
>
>
> The memory footprint of #DFSClient is very considerable in some special 
> scenario since there are many #Configuration instances and occupy much memory 
> resource (In an extreme case, org.apache.hadoop.conf.Configuration occupies 
> over 600MB we meet under HDFS Federation an HA with QJM and there are dozens 
> of NameNodes). I think some new Configuration instance is not necessary. Such 
> as  #ConfiguredFailoverProxyProvider initialization.
> {code:java}
>   public ConfiguredFailoverProxyProvider(Configuration conf, URI uri,
>   Class xface, HAProxyFactory factory) {
> this.xface = xface;
> this.conf = new Configuration(conf);
> ..
>   }
> {code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Updated] (HDFS-13234) Remove renew configuration instance in ConfiguredFailoverProxyProvider and reduce memory footprint for client

2018-03-07 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao updated HDFS-13234:
---
Attachment: HDFS-13234.001.patch

> Remove renew configuration instance in ConfiguredFailoverProxyProvider and 
> reduce memory footprint for client
> -
>
> Key: HDFS-13234
> URL: https://issues.apache.org/jira/browse/HDFS-13234
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: fs, ha, hdfs-client
>Reporter: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-13234.001.patch
>
>
> The memory footprint of #DFSClient is very considerable in some special 
> scenario since there are many #Configuration instances and occupy much memory 
> resource (In an extreme case, org.apache.hadoop.conf.Configuration occupies 
> over 600MB we meet under HDFS Federation an HA with QJM and there are dozens 
> of NameNodes). I think some new Configuration instance is not necessary. Such 
> as  #ConfiguredFailoverProxyProvider initialization.
> {code:java}
>   public ConfiguredFailoverProxyProvider(Configuration conf, URI uri,
>   Class xface, HAProxyFactory factory) {
> this.xface = xface;
> this.conf = new Configuration(conf);
> ..
>   }
> {code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Created] (HDFS-13234) Remove renew configuration instance in ConfiguredFailoverProxyProvider and reduce memory footprint for client

2018-03-05 Thread He Xiaoqiao (JIRA)
He Xiaoqiao created HDFS-13234:
--

 Summary: Remove renew configuration instance in 
ConfiguredFailoverProxyProvider and reduce memory footprint for client
 Key: HDFS-13234
 URL: https://issues.apache.org/jira/browse/HDFS-13234
 Project: Hadoop HDFS
  Issue Type: Improvement
  Components: fs, ha, hdfs-client
Reporter: He Xiaoqiao


The memory footprint of #DFSClient is very considerable in some special 
scenario since there are many #Configuration instances and occupy much memory 
resource (In an extreme case, org.apache.hadoop.conf.Configuration occupies 
over 600MB we meet under HDFS Federation an HA with QJM and there are dozens of 
NameNodes). I think some new Configuration instance is not necessary. Such as  
#ConfiguredFailoverProxyProvider initialization.

{code:java}
  public ConfiguredFailoverProxyProvider(Configuration conf, URI uri,
  Class xface, HAProxyFactory factory) {
this.xface = xface;
this.conf = new Configuration(conf);
..
  }
{code}




--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (HDFS-12749) DN may not send block report to NN after NN restart

2018-03-05 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao commented on HDFS-12749:


[~kihwal] could you help to review or give some suggestions?

> DN may not send block report to NN after NN restart
> ---
>
> Key: HDFS-12749
> URL: https://issues.apache.org/jira/browse/HDFS-12749
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode
>Affects Versions: 2.7.1, 2.8.3, 2.7.5, 3.0.0, 2.9.1
>Reporter: TanYuxin
>Priority: Major
> Attachments: HDFS-12749-branch-2.7.002.patch, 
> HDFS-12749-trunk.003.patch, HDFS-12749.001.patch
>
>
> Now our cluster have thousands of DN, millions of files and blocks. When NN 
> restart, NN's load is very high.
> After NN restart,DN will call BPServiceActor#reRegister method to register. 
> But register RPC will get a IOException since NN is busy dealing with Block 
> Report.  The exception is caught at BPServiceActor#processCommand.
> Next is the caught IOException:
> {code:java}
> WARN org.apache.hadoop.hdfs.server.datanode.DataNode: Error processing 
> datanode Command
> java.io.IOException: Failed on local exception: java.io.IOException: 
> java.net.SocketTimeoutException: 6 millis timeout while waiting for 
> channel to be ready for read. ch : java.nio.channels.SocketChannel[connected 
> local=/DataNode_IP:Port remote=NameNode_Host/IP:Port]; Host Details : local 
> host is: "DataNode_Host/Datanode_IP"; destination host is: 
> "NameNode_Host":Port;
> at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:773)
> at org.apache.hadoop.ipc.Client.call(Client.java:1474)
> at org.apache.hadoop.ipc.Client.call(Client.java:1407)
> at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:229)
> at com.sun.proxy.$Proxy13.registerDatanode(Unknown Source)
> at 
> org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB.registerDatanode(DatanodeProtocolClientSideTranslatorPB.java:126)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.register(BPServiceActor.java:793)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.reRegister(BPServiceActor.java:926)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPOfferService.processCommandFromActor(BPOfferService.java:604)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.processCommand(BPServiceActor.java:898)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.offerService(BPServiceActor.java:711)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.run(BPServiceActor.java:864)
> at java.lang.Thread.run(Thread.java:745)
> {code}
> The un-catched IOException breaks BPServiceActor#register, and the Block 
> Report can not be sent immediately. 
> {code}
>   /**
>* Register one bp with the corresponding NameNode
>* 
>* The bpDatanode needs to register with the namenode on startup in order
>* 1) to report which storage it is serving now and 
>* 2) to receive a registrationID
>*  
>* issued by the namenode to recognize registered datanodes.
>* 
>* @param nsInfo current NamespaceInfo
>* @see FSNamesystem#registerDatanode(DatanodeRegistration)
>* @throws IOException
>*/
>   void register(NamespaceInfo nsInfo) throws IOException {
> // The handshake() phase loaded the block pool storage
> // off disk - so update the bpRegistration object from that info
> DatanodeRegistration newBpRegistration = bpos.createRegistration();
> LOG.info(this + " beginning handshake with NN");
> while (shouldRun()) {
>   try {
> // Use returned registration from namenode with updated fields
> newBpRegistration = bpNamenode.registerDatanode(newBpRegistration);
> newBpRegistration.setNamespaceInfo(nsInfo);
> bpRegistration = newBpRegistration;
> break;
>   } catch(EOFException e) {  // namenode might have just restarted
> LOG.info("Problem connecting to server: " + nnAddr + " :"
> + e.getLocalizedMessage());
> sleepAndLogInterrupts(1000, "connecting to server");
>   } catch(SocketTimeoutException e) {  // namenode is busy
> LOG.info("Problem connecting to server: " + nnAddr);
> sleepAndLogInterrupts(1000, "connecting to server");
>   }
> }
> 
> LOG.info("Block pool " + this + " successfully registered with NN");
> bpos.registrationSucceeded(this, bpRegistration);
> // random short delay - helps scatter the BR from all DNs
> scheduler.scheduleBlockReport(dnConf.initialBlockReportDelay);
>   }
> {code}
> But NameNode has processed registerDatanode successfully, so it won't ask DN 
> to 

[jira] [Commented] (HDFS-13183) Standby NameNode process getBlocks request to reduce Active load

2018-03-01 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao commented on HDFS-13183:


[~xkrogen]
{quote}if the SbNN goes down, the ANN is not aware of this, but the balancer 
should start to read from the ANN instead of SbNN.{quote}
v003 can not process this situation indeed, and i think it is better if client 
is able to make decision to request the proper namenode which may need to 
refactor {{NameNodeConnector}}, and I review the target of HDFS-12976, maybe we 
need wait for finishing.
Thanks again for your detailed code reviewed. [~xkrogen]

> Standby NameNode process getBlocks request to reduce Active load
> 
>
> Key: HDFS-13183
> URL: https://issues.apache.org/jira/browse/HDFS-13183
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: balancer  mover, namenode
>Affects Versions: 2.7.5, 3.1.0, 2.9.1, 2.8.4, 3.0.2
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-13183-trunk.001.patch, HDFS-13183-trunk.002.patch, 
> HDFS-13183-trunk.003.patch
>
>
> The performance of Active NameNode could be impact when {{Balancer}} requests 
> #getBlocks, since query blocks of overly full DNs performance is extremely 
> inefficient currently. The main reason is {{NameNodeRpcServer#getBlocks}} 
> hold read lock for long time. In extreme case, all handlers of Active 
> NameNode RPC server are occupied by one reader 
> {{NameNodeRpcServer#getBlocks}} and other write operation calls, thus Active 
> NameNode enter a state of false death for number of seconds even for minutes.
> The similar performance concerns of Balancer have reported by HDFS-9412, 
> HDFS-7967, etc.
> If Standby NameNode can shoulder #getBlocks heavy burden, it could speed up 
> the progress of balancing and reduce performance impact to Active NameNode.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (HDFS-12749) DN may not send block report to NN after NN restart

2018-03-01 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao commented on HDFS-12749:


Thanks [~xkrogen] for your review all the same.
{quote}I am not satisfied that we fully understand the over-wrapped 
exception{quote}
[~xkrogen] do you have any different opinions? look forwards to your share. 
thanks again.

> DN may not send block report to NN after NN restart
> ---
>
> Key: HDFS-12749
> URL: https://issues.apache.org/jira/browse/HDFS-12749
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode
>Affects Versions: 2.7.1, 2.8.3, 2.7.5, 3.0.0, 2.9.1
>Reporter: TanYuxin
>Priority: Major
> Attachments: HDFS-12749-branch-2.7.002.patch, 
> HDFS-12749-trunk.003.patch, HDFS-12749.001.patch
>
>
> Now our cluster have thousands of DN, millions of files and blocks. When NN 
> restart, NN's load is very high.
> After NN restart,DN will call BPServiceActor#reRegister method to register. 
> But register RPC will get a IOException since NN is busy dealing with Block 
> Report.  The exception is caught at BPServiceActor#processCommand.
> Next is the caught IOException:
> {code:java}
> WARN org.apache.hadoop.hdfs.server.datanode.DataNode: Error processing 
> datanode Command
> java.io.IOException: Failed on local exception: java.io.IOException: 
> java.net.SocketTimeoutException: 6 millis timeout while waiting for 
> channel to be ready for read. ch : java.nio.channels.SocketChannel[connected 
> local=/DataNode_IP:Port remote=NameNode_Host/IP:Port]; Host Details : local 
> host is: "DataNode_Host/Datanode_IP"; destination host is: 
> "NameNode_Host":Port;
> at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:773)
> at org.apache.hadoop.ipc.Client.call(Client.java:1474)
> at org.apache.hadoop.ipc.Client.call(Client.java:1407)
> at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:229)
> at com.sun.proxy.$Proxy13.registerDatanode(Unknown Source)
> at 
> org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB.registerDatanode(DatanodeProtocolClientSideTranslatorPB.java:126)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.register(BPServiceActor.java:793)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.reRegister(BPServiceActor.java:926)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPOfferService.processCommandFromActor(BPOfferService.java:604)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.processCommand(BPServiceActor.java:898)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.offerService(BPServiceActor.java:711)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.run(BPServiceActor.java:864)
> at java.lang.Thread.run(Thread.java:745)
> {code}
> The un-catched IOException breaks BPServiceActor#register, and the Block 
> Report can not be sent immediately. 
> {code}
>   /**
>* Register one bp with the corresponding NameNode
>* 
>* The bpDatanode needs to register with the namenode on startup in order
>* 1) to report which storage it is serving now and 
>* 2) to receive a registrationID
>*  
>* issued by the namenode to recognize registered datanodes.
>* 
>* @param nsInfo current NamespaceInfo
>* @see FSNamesystem#registerDatanode(DatanodeRegistration)
>* @throws IOException
>*/
>   void register(NamespaceInfo nsInfo) throws IOException {
> // The handshake() phase loaded the block pool storage
> // off disk - so update the bpRegistration object from that info
> DatanodeRegistration newBpRegistration = bpos.createRegistration();
> LOG.info(this + " beginning handshake with NN");
> while (shouldRun()) {
>   try {
> // Use returned registration from namenode with updated fields
> newBpRegistration = bpNamenode.registerDatanode(newBpRegistration);
> newBpRegistration.setNamespaceInfo(nsInfo);
> bpRegistration = newBpRegistration;
> break;
>   } catch(EOFException e) {  // namenode might have just restarted
> LOG.info("Problem connecting to server: " + nnAddr + " :"
> + e.getLocalizedMessage());
> sleepAndLogInterrupts(1000, "connecting to server");
>   } catch(SocketTimeoutException e) {  // namenode is busy
> LOG.info("Problem connecting to server: " + nnAddr);
> sleepAndLogInterrupts(1000, "connecting to server");
>   }
> }
> 
> LOG.info("Block pool " + this + " successfully registered with NN");
> bpos.registrationSucceeded(this, bpRegistration);
> // random short delay - helps scatter the BR from all DNs
> 

[jira] [Commented] (HDFS-13183) Standby NameNode process getBlocks request to reduce Active load

2018-03-01 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao commented on HDFS-13183:


[~xkrogen] Thanks for your comments and correct me.
In patch v003, I open {{NameNode#getBlock}} and uncheck operation at Standby 
NameNode, in order to trigger failover at once when request getBlocks to Active 
NameNode I add new {{Exception}} named {{ActiveDenyOfServiceException}}. Add a 
simple unit test about failover for {{ActiveDenyOfServiceException}} also.

> Standby NameNode process getBlocks request to reduce Active load
> 
>
> Key: HDFS-13183
> URL: https://issues.apache.org/jira/browse/HDFS-13183
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: balancer  mover, namenode
>Affects Versions: 2.7.5, 3.1.0, 2.9.1, 2.8.4, 3.0.2
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-13183-trunk.001.patch, HDFS-13183-trunk.002.patch, 
> HDFS-13183-trunk.003.patch
>
>
> The performance of Active NameNode could be impact when {{Balancer}} requests 
> #getBlocks, since query blocks of overly full DNs performance is extremely 
> inefficient currently. The main reason is {{NameNodeRpcServer#getBlocks}} 
> hold read lock for long time. In extreme case, all handlers of Active 
> NameNode RPC server are occupied by one reader 
> {{NameNodeRpcServer#getBlocks}} and other write operation calls, thus Active 
> NameNode enter a state of false death for number of seconds even for minutes.
> The similar performance concerns of Balancer have reported by HDFS-9412, 
> HDFS-7967, etc.
> If Standby NameNode can shoulder #getBlocks heavy burden, it could speed up 
> the progress of balancing and reduce performance impact to Active NameNode.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Updated] (HDFS-13183) Standby NameNode process getBlocks request to reduce Active load

2018-03-01 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao updated HDFS-13183:
---
Attachment: HDFS-13183-trunk.003.patch

> Standby NameNode process getBlocks request to reduce Active load
> 
>
> Key: HDFS-13183
> URL: https://issues.apache.org/jira/browse/HDFS-13183
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: balancer  mover, namenode
>Affects Versions: 2.7.5, 3.1.0, 2.9.1, 2.8.4, 3.0.2
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-13183-trunk.001.patch, HDFS-13183-trunk.002.patch, 
> HDFS-13183-trunk.003.patch
>
>
> The performance of Active NameNode could be impact when {{Balancer}} requests 
> #getBlocks, since query blocks of overly full DNs performance is extremely 
> inefficient currently. The main reason is {{NameNodeRpcServer#getBlocks}} 
> hold read lock for long time. In extreme case, all handlers of Active 
> NameNode RPC server are occupied by one reader 
> {{NameNodeRpcServer#getBlocks}} and other write operation calls, thus Active 
> NameNode enter a state of false death for number of seconds even for minutes.
> The similar performance concerns of Balancer have reported by HDFS-9412, 
> HDFS-7967, etc.
> If Standby NameNode can shoulder #getBlocks heavy burden, it could speed up 
> the progress of balancing and reduce performance impact to Active NameNode.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (HDFS-12749) DN may not send block report to NN after NN restart

2018-02-28 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao commented on HDFS-12749:


ping [~xkrogen],[~kihwal] any suggestions or feedback for this issue?

> DN may not send block report to NN after NN restart
> ---
>
> Key: HDFS-12749
> URL: https://issues.apache.org/jira/browse/HDFS-12749
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode
>Affects Versions: 2.7.1, 2.8.3, 2.7.5, 3.0.0, 2.9.1
>Reporter: TanYuxin
>Priority: Major
> Attachments: HDFS-12749-branch-2.7.002.patch, 
> HDFS-12749-trunk.003.patch, HDFS-12749.001.patch
>
>
> Now our cluster have thousands of DN, millions of files and blocks. When NN 
> restart, NN's load is very high.
> After NN restart,DN will call BPServiceActor#reRegister method to register. 
> But register RPC will get a IOException since NN is busy dealing with Block 
> Report.  The exception is caught at BPServiceActor#processCommand.
> Next is the caught IOException:
> {code:java}
> WARN org.apache.hadoop.hdfs.server.datanode.DataNode: Error processing 
> datanode Command
> java.io.IOException: Failed on local exception: java.io.IOException: 
> java.net.SocketTimeoutException: 6 millis timeout while waiting for 
> channel to be ready for read. ch : java.nio.channels.SocketChannel[connected 
> local=/DataNode_IP:Port remote=NameNode_Host/IP:Port]; Host Details : local 
> host is: "DataNode_Host/Datanode_IP"; destination host is: 
> "NameNode_Host":Port;
> at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:773)
> at org.apache.hadoop.ipc.Client.call(Client.java:1474)
> at org.apache.hadoop.ipc.Client.call(Client.java:1407)
> at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:229)
> at com.sun.proxy.$Proxy13.registerDatanode(Unknown Source)
> at 
> org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB.registerDatanode(DatanodeProtocolClientSideTranslatorPB.java:126)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.register(BPServiceActor.java:793)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.reRegister(BPServiceActor.java:926)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPOfferService.processCommandFromActor(BPOfferService.java:604)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.processCommand(BPServiceActor.java:898)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.offerService(BPServiceActor.java:711)
> at 
> org.apache.hadoop.hdfs.server.datanode.BPServiceActor.run(BPServiceActor.java:864)
> at java.lang.Thread.run(Thread.java:745)
> {code}
> The un-catched IOException breaks BPServiceActor#register, and the Block 
> Report can not be sent immediately. 
> {code}
>   /**
>* Register one bp with the corresponding NameNode
>* 
>* The bpDatanode needs to register with the namenode on startup in order
>* 1) to report which storage it is serving now and 
>* 2) to receive a registrationID
>*  
>* issued by the namenode to recognize registered datanodes.
>* 
>* @param nsInfo current NamespaceInfo
>* @see FSNamesystem#registerDatanode(DatanodeRegistration)
>* @throws IOException
>*/
>   void register(NamespaceInfo nsInfo) throws IOException {
> // The handshake() phase loaded the block pool storage
> // off disk - so update the bpRegistration object from that info
> DatanodeRegistration newBpRegistration = bpos.createRegistration();
> LOG.info(this + " beginning handshake with NN");
> while (shouldRun()) {
>   try {
> // Use returned registration from namenode with updated fields
> newBpRegistration = bpNamenode.registerDatanode(newBpRegistration);
> newBpRegistration.setNamespaceInfo(nsInfo);
> bpRegistration = newBpRegistration;
> break;
>   } catch(EOFException e) {  // namenode might have just restarted
> LOG.info("Problem connecting to server: " + nnAddr + " :"
> + e.getLocalizedMessage());
> sleepAndLogInterrupts(1000, "connecting to server");
>   } catch(SocketTimeoutException e) {  // namenode is busy
> LOG.info("Problem connecting to server: " + nnAddr);
> sleepAndLogInterrupts(1000, "connecting to server");
>   }
> }
> 
> LOG.info("Block pool " + this + " successfully registered with NN");
> bpos.registrationSucceeded(this, bpRegistration);
> // random short delay - helps scatter the BR from all DNs
> scheduler.scheduleBlockReport(dnConf.initialBlockReportDelay);
>   }
> {code}
> But NameNode has processed registerDatanode successfully, so it won't ask DN 

[jira] [Commented] (HDFS-13183) Standby NameNode process getBlocks request to reduce Active load

2018-02-28 Thread He Xiaoqiao (JIRA)

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

He Xiaoqiao commented on HDFS-13183:


[~xkrogen]
Thanks for your detailed comments, and sorry for slow response.
{quote}we should not remove checkOperation altogether, but rather it should be 
OperationCategory.UNCHECKED. We should have this feature be opt-in.{quote}
It is good suggestions, and I will update this issue following your advice.
{quote}Additionally the exception that should be thrown to purposefully trigger 
failover for a client is currently a StandbyException, not a generic 
IOException. {quote}
will client not failover immediately when it meet IOException even if 
{{NamenodeProtocol#getBlocks}} is idempotent? maybe i don't understand 
correctly, if that please correct me.
Thanks again for [~xkrogen] pushing this.

> Standby NameNode process getBlocks request to reduce Active load
> 
>
> Key: HDFS-13183
> URL: https://issues.apache.org/jira/browse/HDFS-13183
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: balancer  mover, namenode
>Affects Versions: 2.7.5, 3.1.0, 2.9.1, 2.8.4, 3.0.2
>Reporter: He Xiaoqiao
>Assignee: He Xiaoqiao
>Priority: Major
> Attachments: HDFS-13183-trunk.001.patch, HDFS-13183-trunk.002.patch
>
>
> The performance of Active NameNode could be impact when {{Balancer}} requests 
> #getBlocks, since query blocks of overly full DNs performance is extremely 
> inefficient currently. The main reason is {{NameNodeRpcServer#getBlocks}} 
> hold read lock for long time. In extreme case, all handlers of Active 
> NameNode RPC server are occupied by one reader 
> {{NameNodeRpcServer#getBlocks}} and other write operation calls, thus Active 
> NameNode enter a state of false death for number of seconds even for minutes.
> The similar performance concerns of Balancer have reported by HDFS-9412, 
> HDFS-7967, etc.
> If Standby NameNode can shoulder #getBlocks heavy burden, it could speed up 
> the progress of balancing and reduce performance impact to Active NameNode.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



<    2   3   4   5   6   7   8   9   >