[jira] [Commented] (HDFS-15235) Transient network failure during NameNode failover kills the NameNode

2020-03-31 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-15235:


I've seen some variant of this issue where NNs kept bouncing back and forth. 
However, I don't think it was caused by network partitioning, rather, the sbnn 
didn't respond because of JVM GC pause (or something that prevented NN from 
sending response back)

> Transient network failure during NameNode failover kills the NameNode
> -
>
> Key: HDFS-15235
> URL: https://issues.apache.org/jira/browse/HDFS-15235
> Project: Hadoop HDFS
>  Issue Type: Bug
>Affects Versions: 3.3.0
>Reporter: YCozy
>Assignee: YCozy
>Priority: Major
> Attachments: HDFS-15235.001.patch
>
>
> We have an HA cluster with two NameNodes: an active NN1 and a standby NN2. At 
> some point, NN1 becomes unhealthy and the admin tries to manually failover to 
> NN2 by running command
> {code:java}
> $ hdfs haadmin -failover NN1 NN2
> {code}
> NN2 receives the request and becomes active:
> {code:java}
> 2020-03-24 00:24:56,412 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem: Stopping services 
> started for standby state
> 2020-03-24 00:24:56,413 WARN 
> org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer: Edit log tailer 
> interrupted: sleep interrupted
> 2020-03-24 00:24:56,415 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem: Starting services 
> required for active state
> 2020-03-24 00:24:56,417 INFO 
> org.apache.hadoop.hdfs.server.namenode.FileJournalManager: Recovering 
> unfinalized segments in /app/ha-name-dir-shared/current
> 2020-03-24 00:24:56,419 INFO 
> org.apache.hadoop.hdfs.server.namenode.FileJournalManager: Recovering 
> unfinalized segments in /app/nn2/name/current
> 2020-03-24 00:24:56,419 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem: Catching up to latest 
> edits from old active before taking over writer role in edits logs
> 2020-03-24 00:24:56,435 INFO org.apache.hadoop.hdfs.server.namenode.FSImage: 
> Reading 
> org.apache.hadoop.hdfs.server.namenode.RedundantEditLogInputStream@7c3095fa 
> expecting start txid #1
> 2020-03-24 00:24:56,436 INFO org.apache.hadoop.hdfs.server.namenode.FSImage: 
> Start loading edits file 
> /app/ha-name-dir-shared/current/edits_001-019 
> maxTxnsToRead = 9223372036854775807
> 2020-03-24 00:24:56,441 INFO 
> org.apache.hadoop.hdfs.server.namenode.RedundantEditLogInputStream: 
> Fast-forwarding stream 
> '/app/ha-name-dir-shared/current/edits_001-019'
>  to transaction ID 1
> 2020-03-24 00:24:56,567 INFO org.apache.hadoop.hdfs.server.namenode.FSImage: 
> Loaded 1 edits file(s) (the last named 
> /app/ha-name-dir-shared/current/edits_001-019)
>  of total size 1305.0, total edits 19.0, total load time 109.0 ms
> 2020-03-24 00:24:56,567 INFO 
> org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager: Marking all 
> datanodes as stale
> 2020-03-24 00:24:56,568 INFO 
> org.apache.hadoop.hdfs.server.blockmanagement.BlockManager: Processing 4 
> messages from DataNodes that were previously queued during standby state
> 2020-03-24 00:24:56,569 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem: Reprocessing replication 
> and invalidation queues
> 2020-03-24 00:24:56,569 INFO 
> org.apache.hadoop.hdfs.server.blockmanagement.BlockManager: initializing 
> replication queues
> 2020-03-24 00:24:56,570 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem: Will take over writing 
> edit logs at txnid 20
> 2020-03-24 00:24:56,571 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSEditLog: Starting log segment at 20
> 2020-03-24 00:24:56,812 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSDirectory: Initializing quota with 4 
> thread(s)
> 2020-03-24 00:24:56,819 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSDirectory: Quota initialization 
> completed in 6 millisecondsname space=3storage space=24690storage 
> types=RAM_DISK=0, SSD=0, DISK=0, ARCHIVE=0, PROVIDED=0
> 2020-03-24 00:24:56,827 INFO 
> org.apache.hadoop.hdfs.server.blockmanagement.CacheReplicationMonitor: 
> Starting CacheReplicationMonitor with interval 3 milliseconds
> {code}
> But NN2 fails to send back the RPC response because of temporary network 
> partitioning.
> {code:java}
> java.io.EOFException: End of File Exception between local host is: 
> "24e7b5a52e85/172.17.0.2"; destination host is: "127.0.0.3":8180; : 
> java.io.EOFException; For more details see:  
> http://wiki.apache.org/hadoop/EOFException
>         at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native 
> Method)
>         at 
> 

[jira] [Commented] (HDFS-15240) Erasure Coding: dirty buffer causes reconstruction block error

2020-03-31 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-15240:


[~marvelrock] could you double check the patch again? there are various 
warnings in the precommit. Thanks.

> Erasure Coding: dirty buffer causes reconstruction block error
> --
>
> Key: HDFS-15240
> URL: https://issues.apache.org/jira/browse/HDFS-15240
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode, erasure-coding
>Reporter: HuangTao
>Assignee: HuangTao
>Priority: Major
> Attachments: HDFS-15240.001.patch
>
>
> When read some lzo files we found some blocks were broken.
> I read back all internal blocks(b0-b8) of the block group(RS-6-3-1024k) from 
> DN directly, and choose 6(b0-b5) blocks to decode other 3(b6', b7', b8') 
> blocks. And find the longest common sequenece(LCS) between b6'(decoded) and 
> b6(read from DN)(b7'/b7 and b8'/b8).
> After selecting 6 blocks of the block group in combinations one time and 
> iterating through all cases, I find one case that the length of LCS is the 
> block length - 64KB, 64KB is just the length of ByteBuffer used by 
> StripedBlockReader. So the corrupt reconstruction block is made by a dirty 
> buffer.
> The following log snippet(only show 2 of 28 cases) is my check program 
> output. In my case, I known the 3th block is corrupt, so need other 5 blocks 
> to decode another 3 blocks, then find the 1th block's LCS substring is block 
> length - 64kb.
> It means (0,1,2,4,5,6)th blocks were used to reconstruct 3th block, and the 
> dirty buffer was used before read the 1th block.
> Must be noted that StripedBlockReader read from the offset 0 of the 1th block 
> after used the dirty buffer.
> {code:java}
> decode from [0, 2, 3, 4, 5, 7] -> [1, 6, 8]
> Check Block(1) first 131072 bytes longest common substring length 4
> Check Block(6) first 131072 bytes longest common substring length 4
> Check Block(8) first 131072 bytes longest common substring length 4
> decode from [0, 2, 3, 4, 5, 6] -> [1, 7, 8]
> Check Block(1) first 131072 bytes longest common substring length 65536
> CHECK AGAIN: Block(1) all 27262976 bytes longest common substring length 
> 27197440  # this one
> Check Block(7) first 131072 bytes longest common substring length 4
> Check Block(8) first 131072 bytes longest common substring length 4{code}
> Now I know the dirty buffer causes reconstruction block error, but how does 
> the dirty buffer come about?
> After digging into the code and DN log, I found this following DN log is the 
> root reason.
> {code:java}
> [INFO] [stripedRead-1017] : Interrupted while waiting for IO on channel 
> java.nio.channels.SocketChannel[connected local=/:52586 
> remote=/:50010]. 18 millis timeout left.
> [WARN] [StripedBlockReconstruction-199] : Failed to reconstruct striped 
> block: BP-714356632--1519726836856:blk_-YY_3472979393
> java.lang.NullPointerException
> at 
> org.apache.hadoop.hdfs.util.StripedBlockUtil.getNextCompletedStripedRead(StripedBlockUtil.java:314)
> at 
> org.apache.hadoop.hdfs.server.datanode.erasurecode.StripedReader.doReadMinimumSources(StripedReader.java:308)
> at 
> org.apache.hadoop.hdfs.server.datanode.erasurecode.StripedReader.readMinimumSources(StripedReader.java:269)
> at 
> org.apache.hadoop.hdfs.server.datanode.erasurecode.StripedBlockReconstructor.reconstruct(StripedBlockReconstructor.java:94)
> at 
> org.apache.hadoop.hdfs.server.datanode.erasurecode.StripedBlockReconstructor.run(StripedBlockReconstructor.java:60)
> at 
> java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
> at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
> at 
> java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
> at 
> java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
> at java.base/java.lang.Thread.run(Thread.java:834) {code}
> Reading from DN may timeout(hold by a future(F)) and output the INFO log, but 
> the futures that contains the future(F)  is cleared, 
> {code:java}
> return new StripingChunkReadResult(futures.remove(future),
> StripingChunkReadResult.CANCELLED); {code}
> futures.remove(future) cause NPE. So the EC reconstruction is failed. In the 
> finally phase, the code snippet in *getStripedReader().close()* 
> {code:java}
> reconstructor.freeBuffer(reader.getReadBuffer());
> reader.freeReadBuffer();
> reader.closeBlockReader(); {code}
> free buffer firstly, but the StripedBlockReader still holds the buffer and 
> write it.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (HDFS-15249) ThrottledAsyncChecker is not thread-safe.

2020-03-30 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-15249:


[~brfrn169] I assume you saw ConcurrentModificationException thrown in the 
DataNode. Can you post the stacktrace for future reference?

> ThrottledAsyncChecker is not thread-safe.
> -
>
> Key: HDFS-15249
> URL: https://issues.apache.org/jira/browse/HDFS-15249
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: federation
>Reporter: Toshihiro Suzuki
>Assignee: Toshihiro Suzuki
>Priority: Major
>
> ThrottledAsyncChecker should be thread-safe because it can be used by 
> multiple threads when we have multiple namespaces.
> *checksInProgress* and *completedChecks* are respectively HashMap and 
> WeakHashMap which are not thread-safe. So we need to put them in synchronized 
> block whenever we access them.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Updated] (HDFS-15249) ThrottledAsyncChecker is not thread-safe.

2020-03-30 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-15249:
---
Component/s: federation

> ThrottledAsyncChecker is not thread-safe.
> -
>
> Key: HDFS-15249
> URL: https://issues.apache.org/jira/browse/HDFS-15249
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: federation
>Reporter: Toshihiro Suzuki
>Assignee: Toshihiro Suzuki
>Priority: Major
>
> ThrottledAsyncChecker should be thread-safe because it can be used by 
> multiple threads when we have multiple namespaces.
> *checksInProgress* and *completedChecks* are respectively HashMap and 
> WeakHashMap which are not thread-safe. So we need to put them in synchronized 
> block whenever we access them.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-15251) Add new zookeeper event type case after zk updated to 3.5.x

2020-03-30 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-15251:


Updated Affected Version to 3.3.0 since zk dependency was only updated in 3.3.0 
(via HADOOP-16579). Let me know if I didn't understand it correctly.

> Add new zookeeper event type case after zk updated to 3.5.x
> ---
>
> Key: HDFS-15251
> URL: https://issues.apache.org/jira/browse/HDFS-15251
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: hdfs
>Affects Versions: 3.3.0
>Reporter: Jianfei Jiang
>Assignee: Jianfei Jiang
>Priority: Major
> Attachments: HDFS-15251.001.patch
>
>
> In zookeeper 3.5.x, KeeperState add a new one named Closed, so should add 
> Close case to the swich as it is not an unexpected Zookeeper watch event 
> state.
> {code:java}
> /** @deprecated */
>  @Deprecated
>  Unknown(-1),
>  Disconnected(0),
>  /** @deprecated */
>  @Deprecated
>  NoSyncConnected(1),
>  SyncConnected(3),
>  AuthFailed(4),
>  ConnectedReadOnly(5),
>  SaslAuthenticated(6),
>  Expired(-112),
>  Closed(7);{code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Updated] (HDFS-15251) Add new zookeeper event type case after zk updated to 3.5.x

2020-03-30 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-15251:
---
Affects Version/s: (was: 3.2.1)
   3.3.0

> Add new zookeeper event type case after zk updated to 3.5.x
> ---
>
> Key: HDFS-15251
> URL: https://issues.apache.org/jira/browse/HDFS-15251
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: hdfs
>Affects Versions: 3.3.0
>Reporter: Jianfei Jiang
>Assignee: Jianfei Jiang
>Priority: Major
> Attachments: HDFS-15251.001.patch
>
>
> In zookeeper 3.5.x, KeeperState add a new one named Closed, so should add 
> Close case to the swich as it is not an unexpected Zookeeper watch event 
> state.
> {code:java}
> /** @deprecated */
>  @Deprecated
>  Unknown(-1),
>  Disconnected(0),
>  /** @deprecated */
>  @Deprecated
>  NoSyncConnected(1),
>  SyncConnected(3),
>  AuthFailed(4),
>  ConnectedReadOnly(5),
>  SaslAuthenticated(6),
>  Expired(-112),
>  Closed(7);{code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-15251) Add new zookeeper event type case after zk updated to 3.5.x

2020-03-30 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-15251:


[~nkalmar] [~symat] fyi can you review? thanks

> Add new zookeeper event type case after zk updated to 3.5.x
> ---
>
> Key: HDFS-15251
> URL: https://issues.apache.org/jira/browse/HDFS-15251
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: hdfs
>Affects Versions: 3.2.1
>Reporter: Jianfei Jiang
>Assignee: Jianfei Jiang
>Priority: Major
> Attachments: HDFS-15251.001.patch
>
>
> In zookeeper 3.5.x, KeeperState add a new one named Closed, so should add 
> Close case to the swich as it is not an unexpected Zookeeper watch event 
> state.
> {code:java}
> /** @deprecated */
>  @Deprecated
>  Unknown(-1),
>  Disconnected(0),
>  /** @deprecated */
>  @Deprecated
>  NoSyncConnected(1),
>  SyncConnected(3),
>  AuthFailed(4),
>  ConnectedReadOnly(5),
>  SaslAuthenticated(6),
>  Expired(-112),
>  Closed(7);{code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-14578) AvailableSpaceBlockPlacementPolicy always prefers local node

2020-03-30 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-14578:


[~ayushtkn] I honestly don't understand what this patch does (as stated in 
hdfs-default.xml) without looking at the code.
{quote}
Only used when the dfs.block.replicator.classname is set to

org.apache.hadoop.hdfs.server.blockmanagement.AvailableSpaceBlockPlacementPolicy.
If true, balances the local node too.
{quote}

Am I correct to assume the local DataNode is chosen if the disk usage is below 
50%?

> AvailableSpaceBlockPlacementPolicy always prefers local node
> 
>
> Key: HDFS-14578
> URL: https://issues.apache.org/jira/browse/HDFS-14578
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: block placement
>Affects Versions: 2.8.0, 2.7.4, 3.0.0-alpha1
>Reporter: Wei-Chiu Chuang
>Assignee: Ayush Saxena
>Priority: Major
> Fix For: 3.3.0, 3.1.4, 3.2.2
>
> Attachments: HDFS-14578-02.patch, HDFS-14578-03.patch, 
> HDFS-14578-04.patch, HDFS-14578-05.patch, HDFS-14578-06.patch, 
> HDFS-14578-07.patch, HDFS-14578-WIP-01.patch, HDFS-14758-01.patch
>
>
> It looks like AvailableSpaceBlockPlacementPolicy prefers local disk just like 
> in the BlockPlacementPolicyDefault
>  
> As Yongjun mentioned in 
> [HDFS-8131|https://issues.apache.org/jira/browse/HDFS-8131?focusedCommentId=16558739=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-16558739],
>  
> {quote}Class AvailableSpaceBlockPlacementPolicy extends 
> BlockPlacementPolicyDefault. But it doesn't change the behavior of choosing 
> the first node in BlockPlacementPolicyDefault, so even with this new feature, 
> the local DN is always chosen as the first DN (of course when it is not 
> excluded), and the new feature only changes the selection of the rest of the 
> two DNs.
> {quote}
> I'm file this Jira as I groom Cloudera's internal Jira and found this 
> unreported issue. We do have a customer hitting this problem. I don't have a 
> fix, but thought it would be beneficial to report it to Apache Jira.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-14957) INodeReference Space Consumed was not same in QuotaUsage and ContentSummary

2020-03-30 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-14957:


nvm... looks like the the isStripe() case is redudant.
{{INode#storagespaceConsumed()}} does the same check already.

> INodeReference Space Consumed was not same in QuotaUsage and ContentSummary
> ---
>
> Key: HDFS-14957
> URL: https://issues.apache.org/jira/browse/HDFS-14957
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: namenode
>Affects Versions: 3.1.4
>Reporter: hemanthboyina
>Assignee: hemanthboyina
>Priority: Major
> Fix For: 3.3.0, 3.1.4, 3.2.2
>
> Attachments: HDFS-14957.001.patch, HDFS-14957.002.patch, 
> HDFS-14957.003.patch, HDFS-14957.JPG
>
>
> for INodeReferences , space consumed was different in QuotaUsage and Content 
> Summary 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-14957) INodeReference Space Consumed was not same in QuotaUsage and ContentSummary

2020-03-30 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-14957:


So I see the fix also treats EC blocks specially. Is there a test covering that 
case?

> INodeReference Space Consumed was not same in QuotaUsage and ContentSummary
> ---
>
> Key: HDFS-14957
> URL: https://issues.apache.org/jira/browse/HDFS-14957
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: namenode
>Affects Versions: 3.1.4
>Reporter: hemanthboyina
>Assignee: hemanthboyina
>Priority: Major
> Fix For: 3.3.0, 3.1.4, 3.2.2
>
> Attachments: HDFS-14957.001.patch, HDFS-14957.002.patch, 
> HDFS-14957.003.patch, HDFS-14957.JPG
>
>
> for INodeReferences , space consumed was different in QuotaUsage and Content 
> Summary 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-14968) Add ability to know datanode staleness

2020-03-30 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-14968:


Why is it hidden behind. a configuration if it's just printing extra logs? Does 
it emit so much log that it slow down NN?
Feels to me this configuration should be enabled all the time (at least that's 
what I would do in Cloudera) 

> Add ability to know datanode staleness
> --
>
> Key: HDFS-14968
> URL: https://issues.apache.org/jira/browse/HDFS-14968
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: datanode, logging, namenode
>Reporter: Ahmed Hussein
>Assignee: Ahmed Hussein
>Priority: Minor
> Fix For: 3.3.0, 3.1.4, 3.2.2, 2.10.1
>
> Attachments: HDFS-14968.001.patch, HDFS-14968.002.patch, 
> HDFS-14968.003.patch
>
>
> There is no way to know whether a DataNode was marked stale or no longer 
> stale by the NameNode.
> It will be good to have the option to enable logging the DataNode staleness 
> to figure out if the  staleness was the reason behind remote reads. 
> Therefore, analyze performance and decision making of the local vs remote 
> reads.
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
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-15248) Make the maximum number of ACLs entries configurable

2020-03-29 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang edited comment on HDFS-15248 at 3/30/20, 1:56 AM:
--

Thanks for offering the patch!

I've had customers asking for extending ACL entry limit before. I'm not sure 
why 32, but here are a few reasons why it's probably not a good idea to extend 
it further

(1) manageability. once you have more than a dozen ACLs per file, it becomes 
hard to manage, error-prone.
(2) NameNode heap size. Especially in a large cluster with hundreds of millions 
of files, each inode occupies more bytes of heap. The memory pressure becomes 
even worse.
(3) serialization cost. We currently serialize the files under a directory to a 
protobuf message, which is limited to 64mb (default), and as the result we 
limit the max number of files per directory to 1 million. Allowing more ACL 
entries per file means more serialized bytes per file, and you may run into the 
protobuf message limit for a large directory well before 1 million files.

For these reasons I usually recommend users to use external authorization 
providers like Sentry or Ranger to delegate the authorization work to a 
separate entity.


was (Author: jojochuang):
Thanks for offering the patch!

I've had customers asking for extending ACL entry limit before. I'm not sure 
why 32, but here are a few reasons why it's probably not a good idea to extend 
it further

(1) manageability. once you have more than a dozen ACLs per file, it becomes 
hard to manage, error-prone.
(2) NameNode heap size. Especially in a large cluster with hundreds of millions 
of files, each inode occupies more bytes of heap. The memory pressure becomes 
even worse.
(3) serialization cost. We currently serialize the files under a directory to a 
protobuf message, which is limited to 64mb (default), and as the result we 
limit the max number of files per directory to 1 million. Allowing more ACL 
entries per file means more serialized bytes per file, and you may run into the 
protobuf message limit for a large directory well before 1 million files.

For these reasons I usually recommend users to use external authorization 
providers like Sentry or Ranger to delete the authorization work to a separate 
entity.

> Make the maximum number of ACLs entries configurable
> 
>
> Key: HDFS-15248
> URL: https://issues.apache.org/jira/browse/HDFS-15248
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: namenode
>Reporter: Yang Yun
>Assignee: Yang Yun
>Priority: Minor
> Attachments: HDFS-15248.001.patch, HDFS-15248.patch
>
>
> For big cluster, the hardcode 32 of ACLs maximum number is not enough, make 
> it configurable.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-15248) Make the maximum number of ACLs entries configurable

2020-03-29 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-15248:


Thanks for offering the patch!

I've had customers asking for extending ACL entry limit before. I'm not sure 
why 32, but here are a few reasons why it's probably not a good idea to extend 
it further

(1) manageability. once you have more than a dozen ACLs per file, it becomes 
hard to manage, error-prone.
(2) NameNode heap size. Especially in a large cluster with hundreds of millions 
of files, each inode occupies more bytes of heap. The memory pressure becomes 
even worse.
(3) serialization cost. We currently serialize the files under a directory to a 
protobuf message, which is limited to 64mb (default), and as the result we 
limit the max number of files per directory to 1 million. Allowing more ACL 
entries per file means more serialized bytes per file, and you may run into the 
protobuf message limit for a large directory well before 1 million files.

For these reasons I usually recommend users to use external authorization 
providers like Sentry or Ranger to delete the authorization work to a separate 
entity.

> Make the maximum number of ACLs entries configurable
> 
>
> Key: HDFS-15248
> URL: https://issues.apache.org/jira/browse/HDFS-15248
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: namenode
>Reporter: Yang Yun
>Assignee: Yang Yun
>Priority: Minor
> Attachments: HDFS-15248.001.patch, HDFS-15248.patch
>
>
> For big cluster, the hardcode 32 of ACLs maximum number is not enough, make 
> it configurable.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-15245) Improve JournalNode web UI

2020-03-27 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-15245:


Big +1 to the proposal! This is long overdue. I wish it also shows additional 
metrics like RpcDetailedActivity, but I am happy with the current one.

> Improve JournalNode web UI
> --
>
> Key: HDFS-15245
> URL: https://issues.apache.org/jira/browse/HDFS-15245
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: journal-node, ui
>Affects Versions: 3.2.1
>Reporter: Jianfei Jiang
>Assignee: Jianfei Jiang
>Priority: Major
> Attachments: HDFS-15245.002.patch, jn web 1.PNG, jn web 2.PNG
>
>
> At present, journalnode web UI is a almost blank page. It almost has no 
> useful infomation.
>  # Add some infomation of journalnode at the main page.
>  # Add a dropdown menu Utilities like DN and JN, contains Logs, Log Level, 
> Metrics, Configuration & Process Thread Dump
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Updated] (HDFS-15240) Erasure Coding: dirty buffer causes reconstruction block error

2020-03-26 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-15240:
---
Status: Patch Available  (was: Open)

> Erasure Coding: dirty buffer causes reconstruction block error
> --
>
> Key: HDFS-15240
> URL: https://issues.apache.org/jira/browse/HDFS-15240
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode, erasure-coding
>Reporter: HuangTao
>Assignee: HuangTao
>Priority: Major
> Attachments: HDFS-15240.001.patch
>
>
> When read some lzo files we found some blocks were broken.
> I read back all internal blocks(b0-b8) of the block group(RS-6-3-1024k) from 
> DN directly, and choose 6(b0-b5) blocks to decode other 3(b6', b7', b8') 
> blocks. And find the longest common sequenece(LCS) between b6'(decoded) and 
> b6(read from DN)(b7'/b7 and b8'/b8).
> After selecting 6 blocks of the block group in combinations one time and 
> iterating through all cases, I find one case that the length of LCS is the 
> block length - 64KB, 64KB is just the length of ByteBuffer used by 
> StripedBlockReader. So the corrupt reconstruction block is made by a dirty 
> buffer.
> The following log snippet(only show 2 of 28 cases) is my check program 
> output. In my case, I known the 3th block is corrupt, so need other 5 blocks 
> to decode another 3 blocks, then find the 1th block's LCS substring is block 
> length - 64kb.
> It means (0,1,2,4,5,6)th blocks were used to reconstruct 3th block, and the 
> dirty buffer was used before read the 1th block.
> Must be noted that StripedBlockReader read from the offset 0 of the 1th block 
> after used the dirty buffer.
> {code:java}
> decode from [0, 2, 3, 4, 5, 7] -> [1, 6, 8]
> Check Block(1) first 131072 bytes longest common substring length 4
> Check Block(6) first 131072 bytes longest common substring length 4
> Check Block(8) first 131072 bytes longest common substring length 4
> decode from [0, 2, 3, 4, 5, 6] -> [1, 7, 8]
> Check Block(1) first 131072 bytes longest common substring length 65536
> CHECK AGAIN: Block(1) all 27262976 bytes longest common substring length 
> 27197440  # this one
> Check Block(7) first 131072 bytes longest common substring length 4
> Check Block(8) first 131072 bytes longest common substring length 4{code}
> Now I know the dirty buffer causes reconstruction block error, but how does 
> the dirty buffer come about?
> After digging into the code and DN log, I found this following DN log is the 
> root reason.
> {code:java}
> [INFO] [stripedRead-1017] : Interrupted while waiting for IO on channel 
> java.nio.channels.SocketChannel[connected local=/:52586 
> remote=/:50010]. 18 millis timeout left.
> [WARN] [StripedBlockReconstruction-199] : Failed to reconstruct striped 
> block: BP-714356632--1519726836856:blk_-YY_3472979393
> java.lang.NullPointerException
> at 
> org.apache.hadoop.hdfs.util.StripedBlockUtil.getNextCompletedStripedRead(StripedBlockUtil.java:314)
> at 
> org.apache.hadoop.hdfs.server.datanode.erasurecode.StripedReader.doReadMinimumSources(StripedReader.java:308)
> at 
> org.apache.hadoop.hdfs.server.datanode.erasurecode.StripedReader.readMinimumSources(StripedReader.java:269)
> at 
> org.apache.hadoop.hdfs.server.datanode.erasurecode.StripedBlockReconstructor.reconstruct(StripedBlockReconstructor.java:94)
> at 
> org.apache.hadoop.hdfs.server.datanode.erasurecode.StripedBlockReconstructor.run(StripedBlockReconstructor.java:60)
> at 
> java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
> at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
> at 
> java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
> at 
> java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
> at java.base/java.lang.Thread.run(Thread.java:834) {code}
> Reading from DN may timeout(hold by a future(F)) and output the INFO log, but 
> the futures that contains the future(F)  is cleared, 
> {code:java}
> return new StripingChunkReadResult(futures.remove(future),
> StripingChunkReadResult.CANCELLED); {code}
> futures.remove(future) cause NPE. So the EC reconstruction is failed. In the 
> finally phase, the code snippet in *getStripedReader().close()* 
> {code:java}
> reconstructor.freeBuffer(reader.getReadBuffer());
> reader.freeReadBuffer();
> reader.closeBlockReader(); {code}
> free buffer firstly, but the StripedBlockReader still holds the buffer and 
> write it.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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

[jira] [Updated] (HDFS-14434) webhdfs that connect secure hdfs should not use user.name parameter

2020-03-25 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-14434:
---
Fix Version/s: 3.2.2
   3.1.4

> webhdfs that connect secure hdfs should not use user.name parameter
> ---
>
> Key: HDFS-14434
> URL: https://issues.apache.org/jira/browse/HDFS-14434
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: webhdfs
>Affects Versions: 3.1.2
>Reporter: KWON BYUNGCHANG
>Assignee: KWON BYUNGCHANG
>Priority: Minor
> Fix For: 3.3.0, 3.1.4, 3.2.2
>
> Attachments: HDFS-14434.001.patch, HDFS-14434.002.patch, 
> HDFS-14434.003.patch, HDFS-14434.004.patch, HDFS-14434.005.patch, 
> HDFS-14434.006.patch, HDFS-14434.007.patch, HDFS-14434.008.patch
>
>
> I have two secure hadoop cluster.  Both cluster use cross-realm 
> authentication. 
> [use...@a.com|mailto:use...@a.com] can access to HDFS of B.COM realm
> by the way, hadoop username of use...@a.com  in B.COM realm is  
> cross_realm_a_com_user_a.
> hdfs dfs command of use...@a.com using B.COM webhdfs failed.
> root cause is  webhdfs that connect secure hdfs use user.name parameter.
> according to webhdfs spec,  insecure webhdfs use user.name,  secure webhdfs 
> use SPNEGO for authentication.
> I think webhdfs that connect secure hdfs  should not use user.name parameter.
> I will attach patch.
> below is error log
>  
> {noformat}
> $ hdfs dfs -ls  webhdfs://b.com:50070/
> ls: Usernames not matched: name=user_a != expected=cross_realm_a_com_user_a
>  
> # user.name in cross realm webhdfs
> $ curl -u : --negotiate 
> 'http://b.com:50070/webhdfs/v1/?op=GETDELEGATIONTOKEN=user_a' 
> {"RemoteException":{"exception":"SecurityException","javaClassName":"java.lang.SecurityException","message":"Failed
>  to obtain user group information: java.io.IOException: Usernames not 
> matched: name=user_a != expected=cross_realm_a_com_user_a"}}
> # USE SPNEGO
> $ curl -u : --negotiate 'http://b.com:50070/webhdfs/v1/?op=GETDELEGATIONTOKEN'
> {"Token"{"urlString":"XgA."}}
>  
> {noformat}
>  
>  
>  
>  
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Updated] (HDFS-14006) Refactor name node to allow different token verification implementations

2020-03-25 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-14006:
---
Fix Version/s: 3.2.2
   3.1.4

> Refactor name node to allow different token verification implementations
> 
>
> Key: HDFS-14006
> URL: https://issues.apache.org/jira/browse/HDFS-14006
> Project: Hadoop HDFS
>  Issue Type: Improvement
>Reporter: CR Hota
>Assignee: CR Hota
>Priority: Major
> Fix For: 3.3.0, 3.1.4, 3.2.2
>
> Attachments: HDFS-14006.001.patch, HDFS-14006.002.patch, 
> HDFS-14006.003.patch
>
>
> Router currently uses Namenode web resources to read and verify delegation 
> tokens. This model doesn't work when router will be deployed in secured mode. 
> This change will introduce router's own UserProvider resource and 
> dependencies.
> In the current deployment one can see this exception.
> {"RemoteException":\{"exception":"ClassCastException","javaClassName":"java.lang.ClassCastException","message":"org.apache.hadoop.hdfs.server.federation.router.Router
>  cannot be cast to org.apache.hadoop.hdfs.server.namenode.NameNode"}}
> In the proposed change, router will maintain its own web resource, that will 
> be similar to current namenode, but modified to get back a router instance 
> instead of namenode.
>  
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Updated] (HDFS-14647) NPE during secure namenode startup

2020-03-25 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-14647:
---
Fix Version/s: 3.2.2
   3.1.4

> NPE during secure namenode startup
> --
>
> Key: HDFS-14647
> URL: https://issues.apache.org/jira/browse/HDFS-14647
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: hdfs
>Affects Versions: 2.8.2
>Reporter: Fengnan Li
>Assignee: Fengnan Li
>Priority: Minor
> Fix For: 3.3.0, 3.1.4, 3.2.2
>
> Attachments: HDFS-14647-2.002.patch, HDFS-14647-trunk.001.patch, 
> HDFS-14647-trunk.002.patch, HDFS-14647-trunk.003.patch, 
> HDFS-14647-trunk.004.patch, HDFS-14647.001.patch
>
>
> In secure HDFS, during Namenode loading fsimage, when hitting Namenode 
> through the REST API, below exception would be thrown out. (This is in 
> version 2.8.2)
> {quote}org.apache.hadoop.hdfs.web.resources.ExceptionHandler: 
> INTERNAL_SERVER_ERROR
>  java.lang.NullPointerException
>  at 
> org.apache.hadoop.hdfs.server.common.JspHelper.getTokenUGI(JspHelper.java:283)
>  at org.apache.hadoop.hdfs.server.common.JspHelper.getUGI(JspHelper.java:226)
>  at 
> org.apache.hadoop.hdfs.web.resources.UserProvider.getValue(UserProvider.java:54)
>  at 
> org.apache.hadoop.hdfs.web.resources.UserProvider.getValue(UserProvider.java:42)
>  at 
> com.sun.jersey.server.impl.inject.InjectableValuesProvider.getInjectableValues(InjectableValuesProvider.java:46)
>  at 
> com.sun.jersey.server.impl.model.method.dispatch.AbstractResourceMethodDispatchProvider$EntityParamInInvoker.getParams(AbstractResourceMethodDispatchProvider.java:153)
>  at 
> com.sun.jersey.server.impl.model.method.dispatch.AbstractResourceMethodDispatchProvider$ResponseOutInvoker._dispatch(AbstractResourceMethodDispatchProvider.java:203)
>  at 
> com.sun.jersey.server.impl.model.method.dispatch.ResourceJavaMethodDispatcher.dispatch(ResourceJavaMethodDispatcher.java:75)
>  at 
> com.sun.jersey.server.impl.uri.rules.HttpMethodRule.accept(HttpMethodRule.java:288)
>  at 
> com.sun.jersey.server.impl.uri.rules.RightHandPathRule.accept(RightHandPathRule.java:147)
>  at 
> com.sun.jersey.server.impl.uri.rules.ResourceClassRule.accept(ResourceClassRule.java:108)
>  at 
> com.sun.jersey.server.impl.uri.rules.RightHandPathRule.accept(RightHandPathRule.java:147)
>  at 
> com.sun.jersey.server.impl.uri.rules.RootResourceClassesRule.accept(RootResourceClassesRule.java:84)
>  at 
> com.sun.jersey.server.impl.application.WebApplicationImpl._handleRequest(WebApplicationImpl.java:1469)
>  at 
> com.sun.jersey.server.impl.application.WebApplicationImpl._handleRequest(WebApplicationImpl.java:1400)
>  at 
> com.sun.jersey.server.impl.application.WebApplicationImpl.handleRequest(WebApplicationImpl.java:1349)
>  at 
> com.sun.jersey.server.impl.application.WebApplicationImpl.handleRequest(WebApplicationImpl.java:1339)
>  at 
> com.sun.jersey.spi.container.servlet.WebComponent.service(WebComponent.java:416)
>  at 
> com.sun.jersey.spi.container.servlet.ServletContainer.service(ServletContainer.java:537)
>  at 
> com.sun.jersey.spi.container.servlet.ServletContainer.service(ServletContainer.java:699)
>  at javax.servlet.http.HttpServlet.service(HttpServlet.java:820)
>  at org.mortbay.jetty.servlet.ServletHolder.handle(ServletHolder.java:511)
>  at 
> org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1221)
>  at org.apache.hadoop.hdfs.web.AuthFilter.doFilter(AuthFilter.java:87)
>  at 
> org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1212)
>  at 
> org.apache.hadoop.http.HttpServer2$QuotingInputFilter.doFilter(HttpServer2.java:1353)
>  at 
> org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1212)
>  at org.apache.hadoop.http.NoCacheFilter.doFilter(NoCacheFilter.java:45)
>  at 
> org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1212)
>  at org.apache.hadoop.http.NoCacheFilter.doFilter(NoCacheFilter.java:45)
>  at 
> org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1212)
>  at org.mortbay.jetty.servlet.ServletHandler.handle(ServletHandler.java:399)
>  at 
> org.mortbay.jetty.security.SecurityHandler.handle(SecurityHandler.java:216)
>  at org.mortbay.jetty.servlet.SessionHandler.handle(SessionHandler.java:182)
>  at org.mortbay.jetty.handler.ContextHandler.handle(ContextHandler.java:766)
>  at org.mortbay.jetty.webapp.WebAppContext.handle(WebAppContext.java:450)
>  at 
> org.mortbay.jetty.handler.ContextHandlerCollection.handle(ContextHandlerCollection.java:230)
>  at org.mortbay.jetty.handler.HandlerWrapper.handle(HandlerWrapper.java:152)
>  at org.mortbay.jetty.Server.handle(Server.java:326)
>  at 

[jira] [Updated] (HDFS-15223) FSCK fails if one namenode is not available

2020-03-25 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-15223:
---
Fix Version/s: 3.2.2

> FSCK fails if one namenode is not available
> ---
>
> Key: HDFS-15223
> URL: https://issues.apache.org/jira/browse/HDFS-15223
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: Ayush Saxena
>Assignee: Ayush Saxena
>Priority: Major
> Fix For: 3.3.0, 3.2.2
>
> Attachments: HDFS-15223-01.patch, HDFS-15223-02.patch
>
>
> If one namenode is not available FSCK should try on other namenode, ignoring 
> the namenode not available



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Updated] (HDFS-15158) The number of failed volumes mismatch with volumeFailures of Datanode metrics

2020-03-25 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-15158:
---
Fix Version/s: 3.2.2
   3.1.4

> The number of failed volumes mismatch  with volumeFailures of Datanode 
> metrics 
> ---
>
> Key: HDFS-15158
> URL: https://issues.apache.org/jira/browse/HDFS-15158
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode
>Reporter: Yang Yun
>Assignee: Yang Yun
>Priority: Minor
> Fix For: 3.3.0, 3.1.4, 3.2.2
>
> Attachments: HDFS-15158.patch, HDFS-15158.patch, HDFS-15158.patch
>
>
> The metrics of Datanode only increment 1, even If more than one volume fails 
> during a disk check.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Updated] (HDFS-14986) ReplicaCachingGetSpaceUsed throws ConcurrentModificationException

2020-03-25 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-14986:
---
Fix Version/s: 3.2.2
   3.1.4

> ReplicaCachingGetSpaceUsed throws  ConcurrentModificationException
> --
>
> Key: HDFS-14986
> URL: https://issues.apache.org/jira/browse/HDFS-14986
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode, performance
>Affects Versions: 2.10.0
>Reporter: Ryan Wu
>Assignee: Aiphago
>Priority: Major
> Fix For: 3.3.0, 3.1.4, 3.2.2, 2.10.1
>
> Attachments: HDFS-14986.001.patch, HDFS-14986.002.patch, 
> HDFS-14986.003.patch, HDFS-14986.004.patch, HDFS-14986.005.patch, 
> HDFS-14986.006.patch
>
>
> Running DU across lots of disks is very expensive . We applied the patch 
> HDFS-14313 to get  used space from ReplicaInfo in memory.However, new du 
> threads throw the exception
> {code:java}
> // 2019-11-08 18:07:13,858 ERROR 
> [refreshUsed-/home/vipshop/hard_disk/7/dfs/dn/current/BP-1203969992--1450855658517]
>  
> org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.ReplicaCachingGetSpaceUsed:
>  ReplicaCachingGetSpaceUsed refresh error
> java.util.ConcurrentModificationException: Tree has been modified outside of 
> iterator
> at 
> org.apache.hadoop.hdfs.util.FoldedTreeSet$TreeSetIterator.checkForModification(FoldedTreeSet.java:311)
> 
> at 
> org.apache.hadoop.hdfs.util.FoldedTreeSet$TreeSetIterator.hasNext(FoldedTreeSet.java:256)
> 
> at java.util.AbstractCollection.addAll(AbstractCollection.java:343)
> at java.util.HashSet.(HashSet.java:120)
> at 
> org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetImpl.deepCopyReplica(FsDatasetImpl.java:1052)
> 
> at 
> org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.ReplicaCachingGetSpaceUsed.refresh(ReplicaCachingGetSpaceUsed.java:73)
> 
> at 
> org.apache.hadoop.fs.CachingGetSpaceUsed$RefreshThread.run(CachingGetSpaceUsed.java:178)
>    
> at java.lang.Thread.run(Thread.java:748)
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Updated] (HDFS-15219) DFS Client will stuck when ResponseProcessor.run throw Error

2020-03-25 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-15219:
---
Fix Version/s: 3.2.2
   3.1.4

> DFS Client will stuck when ResponseProcessor.run throw Error
> 
>
> Key: HDFS-15219
> URL: https://issues.apache.org/jira/browse/HDFS-15219
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: hdfs-client
>Affects Versions: 2.7.3
>Reporter: zhengchenyu
>Assignee: zhengchenyu
>Priority: Major
> Fix For: 3.3.0, 3.1.4, 3.2.2
>
>   Original Estimate: 672h
>  Remaining Estimate: 672h
>
> In my case, a Tez application stucked more than 2 hours util we kill this 
> applicaiton. The Reason is a task attempt stucked, becuase speculative 
> execution is disable. 
> Then Exception like this:
> {code:java}
> 2020-03-11 01:23:59,141 [INFO] [TezChild] |exec.MapOperator|: MAP[4]: records 
> read - 10
> 2020-03-11 01:24:50,294 [INFO] [TezChild] |exec.FileSinkOperator|: FS[3]: 
> records written - 100
> 2020-03-11 01:24:50,294 [INFO] [TezChild] |exec.MapOperator|: MAP[4]: records 
> read - 100
> 2020-03-11 01:29:02,967 [FATAL] [ResponseProcessor for block 
> BP-1856561198-172.16.6.67-1421842461517:blk_15177828027_14109212073] 
> |yarn.YarnUncaughtExceptionHandler|: Thread Thread[ResponseProcessor for 
> block 
> BP-1856561198-172.16.6.67-1421842461517:blk_15177828027_14109212073,5,main] 
> threw an Error. Shutting down now...
> java.lang.NoClassDefFoundError: com/google/protobuf/TextFormat
>  at 
> org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck.toString(PipelineAck.java:253)
>  at java.lang.String.valueOf(String.java:2847)
>  at java.lang.StringBuilder.append(StringBuilder.java:128)
>  at 
> org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer$ResponseProcessor.run(DFSOutputStream.java:737)
> Caused by: java.lang.ClassNotFoundException: com.google.protobuf.TextFormat
>  at java.net.URLClassLoader$1.run(URLClassLoader.java:363)
>  at java.net.URLClassLoader$1.run(URLClassLoader.java:355)
>  at java.security.AccessController.doPrivileged(Native Method)
>  at java.net.URLClassLoader.findClass(URLClassLoader.java:354)
>  at java.lang.ClassLoader.loadClass(ClassLoader.java:425)
>  at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:308)
>  at java.lang.ClassLoader.loadClass(ClassLoader.java:358)
>  ... 4 more
> Caused by: java.util.zip.ZipException: error reading zip file
>  at java.util.zip.ZipFile.read(Native Method)
>  at java.util.zip.ZipFile.access$1400(ZipFile.java:56)
>  at java.util.zip.ZipFile$ZipFileInputStream.read(ZipFile.java:679)
>  at java.util.zip.ZipFile$ZipFileInflaterInputStream.fill(ZipFile.java:415)
>  at java.util.zip.InflaterInputStream.read(InflaterInputStream.java:158)
>  at sun.misc.Resource.getBytes(Resource.java:124)
>  at java.net.URLClassLoader.defineClass(URLClassLoader.java:444)
>  at java.net.URLClassLoader.access$100(URLClassLoader.java:71)
>  at java.net.URLClassLoader$1.run(URLClassLoader.java:361)
>  ... 10 more
> 2020-03-11 01:29:02,970 [INFO] [ResponseProcessor for block 
> BP-1856561198-172.16.6.67-1421842461517:blk_15177828027_14109212073] 
> |util.ExitUtil|: Exiting with status -1
> 2020-03-11 03:27:26,833 [INFO] [TaskHeartbeatThread] |task.TaskReporter|: 
> Received should die response from AM
> 2020-03-11 03:27:26,834 [INFO] [TaskHeartbeatThread] |task.TaskReporter|: 
> Asked to die via task heartbeat
> 2020-03-11 03:27:26,839 [INFO] [TaskHeartbeatThread] |task.TezTaskRunner2|: 
> Attempting to abort attempt_1583335296048_917815_3_01_000704_0 due to an 
> invocation of shutdownRequested
> {code}
> Reason is UncaughtException. When time is 01:29, a disk was error, so throw 
> NoClassDefFoundError. ResponseProcessor.run only catch Exception, can't catch 
> NoClassDefFoundError. So the ReponseProcessor didn't set errorState. Then 
> DataStream didn't know ReponseProcessor was dead, and can't trigger 
> closeResponder, so stucked in DataStream.run.
>  I tested in unit-test TestDataStream.testDfsClient. When I throw 
> NoClassDefFoundError in ResponseProcessor.run, the 
> TestDataStream.testDfsClient will failed bacause of timeout.
> I think we should catch Throwable but not Exception in ReponseProcessor.run.
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Updated] (HDFS-15234) Add a default method body for the INodeAttributeProvider#checkPermissionWithContext API

2020-03-25 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-15234:
---
Fix Version/s: 3.3.0
   Resolution: Fixed
   Status: Resolved  (was: Patch Available)

> Add a default method body for the 
> INodeAttributeProvider#checkPermissionWithContext API
> ---
>
> Key: HDFS-15234
> URL: https://issues.apache.org/jira/browse/HDFS-15234
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: namenode
>Affects Versions: 3.3.0
>Reporter: Wei-Chiu Chuang
>Assignee: Wei-Chiu Chuang
>Priority: Blocker
> Fix For: 3.3.0
>
>
> The new API INodeAttributeProvider#checkPermissionWithContext() needs a 
> default method body. Otherwise old implementations fail to compile.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-15242) Add metrics for operations hold lock times of FsDatasetImpl

2020-03-25 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-15242:


[~inigoiri] bless the +1 on the former version of the patch (HDFS-15075).

> Add metrics for operations hold lock times of FsDatasetImpl
> ---
>
> Key: HDFS-15242
> URL: https://issues.apache.org/jira/browse/HDFS-15242
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: datanode
>Reporter: Xiaoqiao He
>Assignee: Xiaoqiao He
>Priority: Major
> Attachments: HDFS-15242.001.patch
>
>
> Some operations of FsDatasetImpl need to hold Lock, and sometimes it costs 
> long time to execute since it include IO operation in Lock. I propose to add 
> metrics for this operations then it could be more convenient for monitor and 
> dig bottleneck.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-15075) Remove process command timing from BPServiceActor

2020-03-25 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-15075:


+1

> Remove process command timing from BPServiceActor
> -
>
> Key: HDFS-15075
> URL: https://issues.apache.org/jira/browse/HDFS-15075
> Project: Hadoop HDFS
>  Issue Type: Improvement
>Reporter: Íñigo Goiri
>Assignee: Xiaoqiao He
>Priority: Major
> Attachments: HDFS-15075.001.patch, HDFS-15075.002.patch, 
> HDFS-15075.003.patch, HDFS-15075.004.patch, HDFS-15075.005.patch, 
> HDFS-15075.006.patch, HDFS-15075.007.patch, HDFS-15075.008.patch, 
> HDFS-15075.009.patch
>
>
> HDFS-14997 moved the command processing into async.
> Right now, we are checking the time to add to a queue.
> We should remove this one and maybe move the timing within the thread.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-14385) RBF: Optimize MiniRouterDFSCluster with optional light weight MiniDFSCluster

2020-03-24 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-14385:


Triggering a rebuild: https://builds.apache.org/job/PreCommit-HDFS-Build/29014/

> RBF: Optimize MiniRouterDFSCluster with optional light weight MiniDFSCluster
> 
>
> Key: HDFS-14385
> URL: https://issues.apache.org/jira/browse/HDFS-14385
> Project: Hadoop HDFS
>  Issue Type: Sub-task
>  Components: rbf
>Reporter: Xiaoqiao He
>Assignee: Xiaoqiao He
>Priority: Major
> Attachments: HDFS-14385-HDFS-13891.001.patch
>
>
> MiniRouterDFSCluster mimic federated HDFS cluster with routers to support RBF 
> test, In MiniRouterDFSCluster, it starts MiniDFSCluster with complete roles 
> of HDFS which have significant time cost. As HDFS-14351 discussed, it is 
> better to provide mock MiniDFSCluster/Namenodes as one option to support some 
> test case and reduce time cost.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Resolved] (HDFS-14587) Support fail fast when client wait ACK by pipeline over threshold

2020-03-23 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang resolved HDFS-14587.

Resolution: Duplicate

I believe this is a dup of HDFS-8311, so I"ll resolve this one. Feel free to 
reopen if I am wrong.

> Support fail fast when client wait ACK by pipeline over threshold
> -
>
> Key: HDFS-14587
> URL: https://issues.apache.org/jira/browse/HDFS-14587
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: hdfs-client
>Reporter: Xiaoqiao He
>Assignee: Xiaoqiao He
>Priority: Major
>
> Recently, I meet corner case that client wait for data to be acknowledged by 
> pipeline over 9 hours. After check branch trunk, I think this issue still 
> exist. So I propose to add threshold about wait timeout then fail fast.
> {code:java}
> 2019-06-18 12:53:46,217 WARN [Thread-127] org.apache.hadoop.hdfs.DFSClient: 
> Slow waitForAckedSeqno took 35560718ms (threshold=3ms)
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-15075) Remove process command timing from BPServiceActor

2020-03-23 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-15075:


Great work. I wish the extra metrics are a separate jira. Looking at the jira 
summary I wouldn't know the bulk of the change is for adding extra metrics.

Also, the added metrics are a little similar/overlap with the per volume file 
IO metrics added in HDFS-10959 (which is not enabled by default).

> Remove process command timing from BPServiceActor
> -
>
> Key: HDFS-15075
> URL: https://issues.apache.org/jira/browse/HDFS-15075
> Project: Hadoop HDFS
>  Issue Type: Improvement
>Reporter: Íñigo Goiri
>Assignee: Xiaoqiao He
>Priority: Major
> Attachments: HDFS-15075.001.patch, HDFS-15075.002.patch, 
> HDFS-15075.003.patch, HDFS-15075.004.patch, HDFS-15075.005.patch, 
> HDFS-15075.006.patch, HDFS-15075.007.patch
>
>
> HDFS-14997 moved the command processing into async.
> Right now, we are checking the time to add to a queue.
> We should remove this one and maybe move the timing within the thread.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Updated] (HDFS-15088) RBF: Correct annotation typo of RouterPermissionChecker#checkPermission

2020-03-23 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-15088:
---
Fix Version/s: 3.3.0
   Resolution: Fixed
   Status: Resolved  (was: Patch Available)

> RBF: Correct annotation typo of RouterPermissionChecker#checkPermission
> ---
>
> Key: HDFS-15088
> URL: https://issues.apache.org/jira/browse/HDFS-15088
> Project: Hadoop HDFS
>  Issue Type: Sub-task
>  Components: rbf
>Reporter: Xiaoqiao He
>Assignee: Xiaoqiao He
>Priority: Trivial
> Fix For: 3.3.0
>
> Attachments: HDFS-15088.patch
>
>
> Correct annotation typo of RouterPermissionChecker#checkPermission.
> {code:java}
>   /**
>* Whether a mount table entry can be accessed by the current context.
>*
>* @param mountTable
>*  MountTable being accessed
>* @param access
>*  type of action being performed on the cache pool
>* @throws AccessControlException
>*   if mount table cannot be accessed
>*/
>   public void checkPermission(MountTable mountTable, FsAction access)
>   throws AccessControlException {
> }
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-15088) RBF: Correct annotation typo of RouterPermissionChecker#checkPermission

2020-03-23 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-15088:


+1

> RBF: Correct annotation typo of RouterPermissionChecker#checkPermission
> ---
>
> Key: HDFS-15088
> URL: https://issues.apache.org/jira/browse/HDFS-15088
> Project: Hadoop HDFS
>  Issue Type: Sub-task
>  Components: rbf
>Reporter: Xiaoqiao He
>Assignee: Xiaoqiao He
>Priority: Trivial
> Attachments: HDFS-15088.patch
>
>
> Correct annotation typo of RouterPermissionChecker#checkPermission.
> {code:java}
>   /**
>* Whether a mount table entry can be accessed by the current context.
>*
>* @param mountTable
>*  MountTable being accessed
>* @param access
>*  type of action being performed on the cache pool
>* @throws AccessControlException
>*   if mount table cannot be accessed
>*/
>   public void checkPermission(MountTable mountTable, FsAction access)
>   throws AccessControlException {
> }
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-15160) ReplicaMap, Disk Balancer, Directory Scanner and various FsDatasetImpl methods should use datanode readlock

2020-03-23 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-15160:


Looks correct to me.

nit:
{code}
/**
   * Acquire the lock of the data set.
   */
  AutoCloseableLock acquireDatasetLock();

  /***
   * Acquire the read lock of the data set.
   * @return The AutoClosable read lock instance.
   */
  AutoCloseableLock acquireDatasetReadLock();
{code}
It would be great to make the javadoc more clear of the expected behavior. With 
the read lock, it is expected that the block map does it change; however, the 
Block data structure may be updated (update gen stamp, etc). Am I correct?



> ReplicaMap, Disk Balancer, Directory Scanner and various FsDatasetImpl 
> methods should use datanode readlock
> ---
>
> Key: HDFS-15160
> URL: https://issues.apache.org/jira/browse/HDFS-15160
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: datanode
>Affects Versions: 3.3.0
>Reporter: Stephen O'Donnell
>Assignee: Stephen O'Donnell
>Priority: Major
> Attachments: HDFS-15160.001.patch, HDFS-15160.002.patch, 
> HDFS-15160.003.patch
>
>
> Now we have HDFS-15150, we can start to move some DN operations to use the 
> read lock rather than the write lock to improve concurrence. The first step 
> is to make the changes to ReplicaMap, as many other methods make calls to it.
> This Jira switches read operations against the volume map to use the readLock 
> rather than the write lock.
> Additionally, some methods make a call to replicaMap.replicas() (eg 
> getBlockReports, getFinalizedBlocks, deepCopyReplica) and only use the result 
> in a read only fashion, so they can also be switched to using a readLock.
> Next is the directory scanner and disk balancer, which only require a read 
> lock.
> Finally (for this Jira) are various "low hanging fruit" items in BlockSender 
> and fsdatasetImpl where is it fairly obvious they only need a read lock.
> For now, I have avoided changing anything which looks too risky, as I think 
> its better to do any larger refactoring or risky changes each in their own 
> Jira.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Updated] (HDFS-14743) Enhance INodeAttributeProvider/ AccessControlEnforcer Interface in HDFS to support Authorization of mkdir, rm, rmdir, copy, move etc...

2020-03-23 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-14743:
---
Release Note: A new INodeAttributeProvider API 
checkPermissionWithContext(AuthorizationContext) is added. Authorization 
provider implementations may implement this API to get additional context 
(operation name and caller context) of an authorization request.

> Enhance INodeAttributeProvider/ AccessControlEnforcer Interface in HDFS to 
> support Authorization of mkdir, rm, rmdir, copy, move etc...
> ---
>
> Key: HDFS-14743
> URL: https://issues.apache.org/jira/browse/HDFS-14743
> Project: Hadoop HDFS
>  Issue Type: New Feature
>  Components: hdfs
>Affects Versions: 3.1.0
>Reporter: Ramesh Mani
>Assignee: Wei-Chiu Chuang
>Priority: Critical
> Fix For: 3.3.0
>
> Attachments: HDFS-14743 Enhance INodeAttributeProvider_ 
> AccessControlEnforcer Interface.pdf
>
>
> Enhance INodeAttributeProvider / AccessControlEnforcer Interface in HDFS to 
> support Authorization of mkdir, rm, rmdir, copy, move etc..., this should 
> help the implementors of the interface like Apache Ranger's HDFS 
> Authorization plugin to authorize and audit those command sets.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Updated] (HDFS-14743) Enhance INodeAttributeProvider/ AccessControlEnforcer Interface in HDFS to support Authorization of mkdir, rm, rmdir, copy, move etc...

2020-03-23 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-14743:
---
Issue Type: New Feature  (was: Improvement)

> Enhance INodeAttributeProvider/ AccessControlEnforcer Interface in HDFS to 
> support Authorization of mkdir, rm, rmdir, copy, move etc...
> ---
>
> Key: HDFS-14743
> URL: https://issues.apache.org/jira/browse/HDFS-14743
> Project: Hadoop HDFS
>  Issue Type: New Feature
>  Components: hdfs
>Affects Versions: 3.1.0
>Reporter: Ramesh Mani
>Assignee: Wei-Chiu Chuang
>Priority: Critical
> Fix For: 3.3.0
>
> Attachments: HDFS-14743 Enhance INodeAttributeProvider_ 
> AccessControlEnforcer Interface.pdf
>
>
> Enhance INodeAttributeProvider / AccessControlEnforcer Interface in HDFS to 
> support Authorization of mkdir, rm, rmdir, copy, move etc..., this should 
> help the implementors of the interface like Apache Ranger's HDFS 
> Authorization plugin to authorize and audit those command sets.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Updated] (HDFS-14743) Enhance INodeAttributeProvider/ AccessControlEnforcer Interface in HDFS to support Authorization of mkdir, rm, rmdir, copy, move etc...

2020-03-23 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-14743:
---
Issue Type: Improvement  (was: Bug)

> Enhance INodeAttributeProvider/ AccessControlEnforcer Interface in HDFS to 
> support Authorization of mkdir, rm, rmdir, copy, move etc...
> ---
>
> Key: HDFS-14743
> URL: https://issues.apache.org/jira/browse/HDFS-14743
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: hdfs
>Affects Versions: 3.1.0
>Reporter: Ramesh Mani
>Assignee: Wei-Chiu Chuang
>Priority: Critical
> Fix For: 3.3.0
>
> Attachments: HDFS-14743 Enhance INodeAttributeProvider_ 
> AccessControlEnforcer Interface.pdf
>
>
> Enhance INodeAttributeProvider / AccessControlEnforcer Interface in HDFS to 
> support Authorization of mkdir, rm, rmdir, copy, move etc..., this should 
> help the implementors of the interface like Apache Ranger's HDFS 
> Authorization plugin to authorize and audit those command sets.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Updated] (HDFS-15234) Add a default method body for the INodeAttributeProvider#checkPermissionWithContext API

2020-03-23 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-15234:
---
Description: The new API 
INodeAttributeProvider#checkPermissionWithContext() needs a default method 
body. Otherwise old implementations fail to compile.  (was: The new API 
INodeAttributeProvider#checkPermissionWithContext() needs a default method 
body. Otherwise old implementations fails to compile.)

> Add a default method body for the 
> INodeAttributeProvider#checkPermissionWithContext API
> ---
>
> Key: HDFS-15234
> URL: https://issues.apache.org/jira/browse/HDFS-15234
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: namenode
>Affects Versions: 3.3.0
>Reporter: Wei-Chiu Chuang
>Assignee: Wei-Chiu Chuang
>Priority: Blocker
>
> The new API INodeAttributeProvider#checkPermissionWithContext() needs a 
> default method body. Otherwise old implementations fail to compile.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Created] (HDFS-15234) Add a default method body for the INodeAttributeProvider#checkPermissionWithContext API

2020-03-23 Thread Wei-Chiu Chuang (Jira)
Wei-Chiu Chuang created HDFS-15234:
--

 Summary: Add a default method body for the 
INodeAttributeProvider#checkPermissionWithContext API
 Key: HDFS-15234
 URL: https://issues.apache.org/jira/browse/HDFS-15234
 Project: Hadoop HDFS
  Issue Type: Bug
  Components: namenode
Affects Versions: 3.3.0
Reporter: Wei-Chiu Chuang
Assignee: Wei-Chiu Chuang


The new API INodeAttributeProvider#checkPermissionWithContext() needs a default 
method body. Otherwise old implementations fails to compile.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-15075) Remove process command timing from BPServiceActor

2020-03-23 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-15075:


Retriggered jenkins: https://builds.apache.org/job/PreCommit-HDFS-Build/29009/

> Remove process command timing from BPServiceActor
> -
>
> Key: HDFS-15075
> URL: https://issues.apache.org/jira/browse/HDFS-15075
> Project: Hadoop HDFS
>  Issue Type: Improvement
>Reporter: Íñigo Goiri
>Assignee: Xiaoqiao He
>Priority: Major
> Attachments: HDFS-15075.001.patch, HDFS-15075.002.patch, 
> HDFS-15075.003.patch, HDFS-15075.004.patch, HDFS-15075.005.patch, 
> HDFS-15075.006.patch, HDFS-15075.007.patch
>
>
> HDFS-14997 moved the command processing into async.
> Right now, we are checking the time to add to a queue.
> We should remove this one and maybe move the timing within the thread.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Updated] (HDFS-15113) Missing IBR when NameNode restart if open processCommand async feature

2020-03-23 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-15113:
---
Resolution: Fixed
Status: Resolved  (was: Patch Available)

Committed the addendum patch. Thanks again [~hexiaoqiao]!

> Missing IBR when NameNode restart if open processCommand async feature
> --
>
> Key: HDFS-15113
> URL: https://issues.apache.org/jira/browse/HDFS-15113
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode
>Reporter: Xiaoqiao He
>Assignee: Xiaoqiao He
>Priority: Blocker
> Fix For: 3.3.0
>
> Attachments: HDFS-15113.001.patch, HDFS-15113.002.patch, 
> HDFS-15113.003.patch, HDFS-15113.004.patch, HDFS-15113.005.patch, 
> HDFS-15113.addendum.patch
>
>
> Recently, I meet one case that NameNode missing block after restart which is 
> related with HDFS-14997.
> a. during NameNode restart, it will return command `DNA_REGISTER` to DataNode 
> when receive some RPC request from DataNode.
> b. when DataNode receive `DNA_REGISTER` command, it will run #reRegister 
> async.
> {code:java}
>   void reRegister() throws IOException {
> if (shouldRun()) {
>   // re-retrieve namespace info to make sure that, if the NN
>   // was restarted, we still match its version (HDFS-2120)
>   NamespaceInfo nsInfo = retrieveNamespaceInfo();
>   // and re-register
>   register(nsInfo);
>   scheduler.scheduleHeartbeat();
>   // HDFS-9917,Standby NN IBR can be very huge if standby namenode is down
>   // for sometime.
>   if (state == HAServiceState.STANDBY || state == 
> HAServiceState.OBSERVER) {
> ibrManager.clearIBRs();
>   }
> }
>   }
> {code}
> c. As we know, #register will trigger BR immediately.
> d. because #reRegister run async, so we could not make sure which one run 
> first between send FBR and clear IBR. If clean IBR run first, it will be OK. 
> But if send FBR first then clear IBR, it will missing some blocks received 
> between these two time point until next FBR.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-15113) Missing IBR when NameNode restart if open processCommand async feature

2020-03-22 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-15113:


+1

> Missing IBR when NameNode restart if open processCommand async feature
> --
>
> Key: HDFS-15113
> URL: https://issues.apache.org/jira/browse/HDFS-15113
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode
>Reporter: Xiaoqiao He
>Assignee: Xiaoqiao He
>Priority: Blocker
> Fix For: 3.3.0
>
> Attachments: HDFS-15113.001.patch, HDFS-15113.002.patch, 
> HDFS-15113.003.patch, HDFS-15113.004.patch, HDFS-15113.005.patch, 
> HDFS-15113.addendum.patch
>
>
> Recently, I meet one case that NameNode missing block after restart which is 
> related with HDFS-14997.
> a. during NameNode restart, it will return command `DNA_REGISTER` to DataNode 
> when receive some RPC request from DataNode.
> b. when DataNode receive `DNA_REGISTER` command, it will run #reRegister 
> async.
> {code:java}
>   void reRegister() throws IOException {
> if (shouldRun()) {
>   // re-retrieve namespace info to make sure that, if the NN
>   // was restarted, we still match its version (HDFS-2120)
>   NamespaceInfo nsInfo = retrieveNamespaceInfo();
>   // and re-register
>   register(nsInfo);
>   scheduler.scheduleHeartbeat();
>   // HDFS-9917,Standby NN IBR can be very huge if standby namenode is down
>   // for sometime.
>   if (state == HAServiceState.STANDBY || state == 
> HAServiceState.OBSERVER) {
> ibrManager.clearIBRs();
>   }
> }
>   }
> {code}
> c. As we know, #register will trigger BR immediately.
> d. because #reRegister run async, so we could not make sure which one run 
> first between send FBR and clear IBR. If clean IBR run first, it will be OK. 
> But if send FBR first then clear IBR, it will missing some blocks received 
> between these two time point until next FBR.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Updated] (HDFS-15113) Missing IBR when NameNode restart if open processCommand async feature

2020-03-22 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-15113:
---
Status: Patch Available  (was: Reopened)

> Missing IBR when NameNode restart if open processCommand async feature
> --
>
> Key: HDFS-15113
> URL: https://issues.apache.org/jira/browse/HDFS-15113
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode
>Reporter: Xiaoqiao He
>Assignee: Xiaoqiao He
>Priority: Blocker
> Fix For: 3.3.0
>
> Attachments: HDFS-15113.001.patch, HDFS-15113.002.patch, 
> HDFS-15113.003.patch, HDFS-15113.004.patch, HDFS-15113.005.patch, 
> HDFS-15113.addendum.patch
>
>
> Recently, I meet one case that NameNode missing block after restart which is 
> related with HDFS-14997.
> a. during NameNode restart, it will return command `DNA_REGISTER` to DataNode 
> when receive some RPC request from DataNode.
> b. when DataNode receive `DNA_REGISTER` command, it will run #reRegister 
> async.
> {code:java}
>   void reRegister() throws IOException {
> if (shouldRun()) {
>   // re-retrieve namespace info to make sure that, if the NN
>   // was restarted, we still match its version (HDFS-2120)
>   NamespaceInfo nsInfo = retrieveNamespaceInfo();
>   // and re-register
>   register(nsInfo);
>   scheduler.scheduleHeartbeat();
>   // HDFS-9917,Standby NN IBR can be very huge if standby namenode is down
>   // for sometime.
>   if (state == HAServiceState.STANDBY || state == 
> HAServiceState.OBSERVER) {
> ibrManager.clearIBRs();
>   }
> }
>   }
> {code}
> c. As we know, #register will trigger BR immediately.
> d. because #reRegister run async, so we could not make sure which one run 
> first between send FBR and clear IBR. If clean IBR run first, it will be OK. 
> But if send FBR first then clear IBR, it will missing some blocks received 
> between these two time point until next FBR.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Reopened] (HDFS-15113) Missing IBR when NameNode restart if open processCommand async feature

2020-03-22 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang reopened HDFS-15113:


Reopen to have the addendum tested.

> Missing IBR when NameNode restart if open processCommand async feature
> --
>
> Key: HDFS-15113
> URL: https://issues.apache.org/jira/browse/HDFS-15113
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode
>Reporter: Xiaoqiao He
>Assignee: Xiaoqiao He
>Priority: Blocker
> Fix For: 3.3.0
>
> Attachments: HDFS-15113.001.patch, HDFS-15113.002.patch, 
> HDFS-15113.003.patch, HDFS-15113.004.patch, HDFS-15113.005.patch, 
> HDFS-15113.addendum.patch
>
>
> Recently, I meet one case that NameNode missing block after restart which is 
> related with HDFS-14997.
> a. during NameNode restart, it will return command `DNA_REGISTER` to DataNode 
> when receive some RPC request from DataNode.
> b. when DataNode receive `DNA_REGISTER` command, it will run #reRegister 
> async.
> {code:java}
>   void reRegister() throws IOException {
> if (shouldRun()) {
>   // re-retrieve namespace info to make sure that, if the NN
>   // was restarted, we still match its version (HDFS-2120)
>   NamespaceInfo nsInfo = retrieveNamespaceInfo();
>   // and re-register
>   register(nsInfo);
>   scheduler.scheduleHeartbeat();
>   // HDFS-9917,Standby NN IBR can be very huge if standby namenode is down
>   // for sometime.
>   if (state == HAServiceState.STANDBY || state == 
> HAServiceState.OBSERVER) {
> ibrManager.clearIBRs();
>   }
> }
>   }
> {code}
> c. As we know, #register will trigger BR immediately.
> d. because #reRegister run async, so we could not make sure which one run 
> first between send FBR and clear IBR. If clean IBR run first, it will be OK. 
> But if send FBR first then clear IBR, it will missing some blocks received 
> between these two time point until next FBR.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-15227) NPE if the last block changes from COMMITTED to COMPLETE during FSCK

2020-03-21 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-15227:


LGTM. Thanks Ayush for the great work here. Feel free to commit the patch.

> NPE if the last block changes from COMMITTED to COMPLETE during FSCK
> 
>
> Key: HDFS-15227
> URL: https://issues.apache.org/jira/browse/HDFS-15227
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: krishna reddy
>Assignee: Ayush Saxena
>Priority: Major
> Attachments: HDFS-15227-01.patch, TestToRepro.patch
>
>
> FSCK -upgradedomains is failing for upgradedomains when more than 2 million 
> blocks present in hdfs and write in progress of some blocks
> "hdfs fsck / -files -blocks -upgradedomains"



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-15230) Sanity check should not assume key base name can be derived from version name

2020-03-20 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-15230:


[~msingh] fyi

> Sanity check should not assume key base name can be derived from version name
> -
>
> Key: HDFS-15230
> URL: https://issues.apache.org/jira/browse/HDFS-15230
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: Wei-Chiu Chuang
>Priority: Major
>
> HDFS-14884 checks if the encryption info of a file matches the encryption 
> zone key.
> {code}
> if (!KeyProviderCryptoExtension.
> getBaseName(keyVersionName).equals(zoneKeyName)) {
>   throw new IllegalArgumentException(String.format(
>   "KeyVersion '%s' does not belong to the key '%s'",
>   keyVersionName, zoneKeyName));
> }
> {code}
> Here it assumes the "base name" can be derived from key version name, and 
> that the base name should be the same as zone key.
> However, there is no published definition of what a key version name should 
> be. 
> While the code works for the builtin JKS key provider, it may not work for 
> other kind of key providers. (Specifically, it breaks Cloudera's KeyTrustee 
> KMS KeyProvider)



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Created] (HDFS-15230) Sanity check should not assume key base name can be derived from version name

2020-03-20 Thread Wei-Chiu Chuang (Jira)
Wei-Chiu Chuang created HDFS-15230:
--

 Summary: Sanity check should not assume key base name can be 
derived from version name
 Key: HDFS-15230
 URL: https://issues.apache.org/jira/browse/HDFS-15230
 Project: Hadoop HDFS
  Issue Type: Bug
Reporter: Wei-Chiu Chuang


HDFS-14884 checks if the encryption info of a file matches the encryption zone 
key.

{code}
if (!KeyProviderCryptoExtension.
getBaseName(keyVersionName).equals(zoneKeyName)) {
  throw new IllegalArgumentException(String.format(
  "KeyVersion '%s' does not belong to the key '%s'",
  keyVersionName, zoneKeyName));
}
{code}
Here it assumes the "base name" can be derived from key version name, and that 
the base name should be the same as zone key.

However, there is no published definition of what a key version name should be. 

While the code works for the builtin JKS key provider, it may not work for 
other kind of key providers. (Specifically, it breaks Cloudera's KeyTrustee KMS 
KeyProvider)



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Resolved] (HDFS-15208) Suppress bogus AbstractWadlGeneratorGrammarGenerator in KMS stderr in hdfs

2020-03-18 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang resolved HDFS-15208.

Fix Version/s: 3.2.2
   3.1.4
   Resolution: Fixed

> Suppress bogus AbstractWadlGeneratorGrammarGenerator in KMS stderr in hdfs
> --
>
> Key: HDFS-15208
> URL: https://issues.apache.org/jira/browse/HDFS-15208
> Project: Hadoop HDFS
>  Issue Type: Bug
>Affects Versions: 3.0.0
>Reporter: Wei-Chiu Chuang
>Assignee: Wei-Chiu Chuang
>Priority: Trivial
> Fix For: 3.3.0, 3.1.4, 3.2.2
>
>
> Continuation of HADOOP-15686
> Add the same log4j property to disable error log in hadoop-hdfs.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Updated] (HDFS-14820) The default 8KB buffer of BlockReaderRemote#newBlockReader#BufferedOutputStream is too big

2020-03-13 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-14820:
---
Release Note: Reduce the output stream buffer size of a DFSClient remote 
read from 8KB to 512 bytes.  (was: Update the output stream buffer size of a 
DFSClient remote read.)

>  The default 8KB buffer of 
> BlockReaderRemote#newBlockReader#BufferedOutputStream is too big
> ---
>
> Key: HDFS-14820
> URL: https://issues.apache.org/jira/browse/HDFS-14820
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: Lisheng Sun
>Assignee: Lisheng Sun
>Priority: Major
> Attachments: HDFS-14820.001.patch, HDFS-14820.002.patch, 
> HDFS-14820.003.patch
>
>
> this issue is similar to HDFS-14535.
> {code:java}
> public static BlockReader newBlockReader(String file,
> ExtendedBlock block,
> Token blockToken,
> long startOffset, long len,
> boolean verifyChecksum,
> String clientName,
> Peer peer, DatanodeID datanodeID,
> PeerCache peerCache,
> CachingStrategy cachingStrategy,
> int networkDistance) throws IOException {
>   // in and out will be closed when sock is closed (by the caller)
>   final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
>   peer.getOutputStream()));
>   new Sender(out).readBlock(block, blockToken, clientName, startOffset, len,
>   verifyChecksum, cachingStrategy);
> }
> public BufferedOutputStream(OutputStream out) {
> this(out, 8192);
> }
> {code}
> Sender#readBlock parameter( block,blockToken, clientName, startOffset, len, 
> verifyChecksum, cachingStrategy) could not use such a big buffer.
> So i think it should reduce BufferedOutputStream buffer.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Updated] (HDFS-14820) The default 8KB buffer of BlockReaderRemote#newBlockReader#BufferedOutputStream is too big

2020-03-13 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-14820:
---
Fix Version/s: 3.3.0
   Resolution: Fixed
   Status: Resolved  (was: Patch Available)

>  The default 8KB buffer of 
> BlockReaderRemote#newBlockReader#BufferedOutputStream is too big
> ---
>
> Key: HDFS-14820
> URL: https://issues.apache.org/jira/browse/HDFS-14820
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: Lisheng Sun
>Assignee: Lisheng Sun
>Priority: Major
> Fix For: 3.3.0
>
> Attachments: HDFS-14820.001.patch, HDFS-14820.002.patch, 
> HDFS-14820.003.patch
>
>
> this issue is similar to HDFS-14535.
> {code:java}
> public static BlockReader newBlockReader(String file,
> ExtendedBlock block,
> Token blockToken,
> long startOffset, long len,
> boolean verifyChecksum,
> String clientName,
> Peer peer, DatanodeID datanodeID,
> PeerCache peerCache,
> CachingStrategy cachingStrategy,
> int networkDistance) throws IOException {
>   // in and out will be closed when sock is closed (by the caller)
>   final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
>   peer.getOutputStream()));
>   new Sender(out).readBlock(block, blockToken, clientName, startOffset, len,
>   verifyChecksum, cachingStrategy);
> }
> public BufferedOutputStream(OutputStream out) {
> this(out, 8192);
> }
> {code}
> Sender#readBlock parameter( block,blockToken, clientName, startOffset, len, 
> verifyChecksum, cachingStrategy) could not use such a big buffer.
> So i think it should reduce BufferedOutputStream buffer.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Updated] (HDFS-15113) Missing IBR when NameNode restart if open processCommand async feature

2020-03-13 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-15113:
---
Fix Version/s: 3.3.0
   Resolution: Fixed
   Status: Resolved  (was: Patch Available)

Thanks [~hexiaoqiao] for the patch and [~brahmareddy] and [~inigoiri] for the 
reviews!

> Missing IBR when NameNode restart if open processCommand async feature
> --
>
> Key: HDFS-15113
> URL: https://issues.apache.org/jira/browse/HDFS-15113
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode
>Reporter: Xiaoqiao He
>Assignee: Xiaoqiao He
>Priority: Blocker
> Fix For: 3.3.0
>
> Attachments: HDFS-15113.001.patch, HDFS-15113.002.patch, 
> HDFS-15113.003.patch, HDFS-15113.004.patch, HDFS-15113.005.patch
>
>
> Recently, I meet one case that NameNode missing block after restart which is 
> related with HDFS-14997.
> a. during NameNode restart, it will return command `DNA_REGISTER` to DataNode 
> when receive some RPC request from DataNode.
> b. when DataNode receive `DNA_REGISTER` command, it will run #reRegister 
> async.
> {code:java}
>   void reRegister() throws IOException {
> if (shouldRun()) {
>   // re-retrieve namespace info to make sure that, if the NN
>   // was restarted, we still match its version (HDFS-2120)
>   NamespaceInfo nsInfo = retrieveNamespaceInfo();
>   // and re-register
>   register(nsInfo);
>   scheduler.scheduleHeartbeat();
>   // HDFS-9917,Standby NN IBR can be very huge if standby namenode is down
>   // for sometime.
>   if (state == HAServiceState.STANDBY || state == 
> HAServiceState.OBSERVER) {
> ibrManager.clearIBRs();
>   }
> }
>   }
> {code}
> c. As we know, #register will trigger BR immediately.
> d. because #reRegister run async, so we could not make sure which one run 
> first between send FBR and clear IBR. If clean IBR run first, it will be OK. 
> But if send FBR first then clear IBR, it will missing some blocks received 
> between these two time point until next FBR.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Resolved] (HDFS-14743) Enhance INodeAttributeProvider/ AccessControlEnforcer Interface in HDFS to support Authorization of mkdir, rm, rmdir, copy, move etc...

2020-03-13 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang resolved HDFS-14743.

Fix Version/s: 3.3.0
   Resolution: Fixed

Thanks [~xyao] and [~ste...@apache.org] for the throughout reviews!

> Enhance INodeAttributeProvider/ AccessControlEnforcer Interface in HDFS to 
> support Authorization of mkdir, rm, rmdir, copy, move etc...
> ---
>
> Key: HDFS-14743
> URL: https://issues.apache.org/jira/browse/HDFS-14743
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: hdfs
>Affects Versions: 3.1.0
>Reporter: Ramesh Mani
>Assignee: Wei-Chiu Chuang
>Priority: Critical
> Fix For: 3.3.0
>
> Attachments: HDFS-14743 Enhance INodeAttributeProvider_ 
> AccessControlEnforcer Interface.pdf
>
>
> Enhance INodeAttributeProvider / AccessControlEnforcer Interface in HDFS to 
> support Authorization of mkdir, rm, rmdir, copy, move etc..., this should 
> help the implementors of the interface like Apache Ranger's HDFS 
> Authorization plugin to authorize and audit those command sets.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-15113) Missing IBR when NameNode restart if open processCommand async feature

2020-03-13 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-15113:


Thanks [~hexiaoqiao] for the nicely written test!

{code}
  DataNodeFaultInjector.set(new DataNodeFaultInjector() {
public void blockUtilSendFullBlockReport() {
  try {
Thread.sleep(200);
  } catch (InterruptedException e) {
e.printStackTrace();
  }
}
  });
{code}
Pausing the thread for a short duration and expect the invariants to hold true, 
is likely going to generate lots of flaky failures.

{code}
  // Make sure that generate blocks for DataNode and IBR not empty now.
  Thread.sleep(200);
{code}
Ideally you should use something like a Semaphore to ensure this.


{code}
addNewBlockThread.start();
{code}
As a good habit, join() the thread at the end of test.

{code}
Mockito.doAnswer((Answer) invocation -> {
  Object[] arguments = invocation.getArguments();
  StorageReceivedDeletedBlocks[] list =
  (StorageReceivedDeletedBlocks[])arguments[2];
  setIncreaseBlockReportCount(list[0].getBlocks().length);
  return null;
}).when(mockNN).blockReceivedAndDeleted(
{code}
Here we assume the method is always used to add blocks and will not be used to 
test deleting blocks. This can be confusing for future test writers. Suggest to 
add a comment.

On the condition that we will address these in the future I am +1. Don't want 
to hold off a release because of my picky comments in the tests. Will commit 
later.

> Missing IBR when NameNode restart if open processCommand async feature
> --
>
> Key: HDFS-15113
> URL: https://issues.apache.org/jira/browse/HDFS-15113
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode
>Reporter: Xiaoqiao He
>Assignee: Xiaoqiao He
>Priority: Blocker
> Attachments: HDFS-15113.001.patch, HDFS-15113.002.patch, 
> HDFS-15113.003.patch, HDFS-15113.004.patch, HDFS-15113.005.patch
>
>
> Recently, I meet one case that NameNode missing block after restart which is 
> related with HDFS-14997.
> a. during NameNode restart, it will return command `DNA_REGISTER` to DataNode 
> when receive some RPC request from DataNode.
> b. when DataNode receive `DNA_REGISTER` command, it will run #reRegister 
> async.
> {code:java}
>   void reRegister() throws IOException {
> if (shouldRun()) {
>   // re-retrieve namespace info to make sure that, if the NN
>   // was restarted, we still match its version (HDFS-2120)
>   NamespaceInfo nsInfo = retrieveNamespaceInfo();
>   // and re-register
>   register(nsInfo);
>   scheduler.scheduleHeartbeat();
>   // HDFS-9917,Standby NN IBR can be very huge if standby namenode is down
>   // for sometime.
>   if (state == HAServiceState.STANDBY || state == 
> HAServiceState.OBSERVER) {
> ibrManager.clearIBRs();
>   }
> }
>   }
> {code}
> c. As we know, #register will trigger BR immediately.
> d. because #reRegister run async, so we could not make sure which one run 
> first between send FBR and clear IBR. If clean IBR run first, it will be OK. 
> But if send FBR first then clear IBR, it will missing some blocks received 
> between these two time point until next FBR.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-15220) FSCK calls are redirecting to Active NN

2020-03-12 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-15220:


Why? Other than -move, all of fsck operations are read-only and observer should 
be able to handle them.

> FSCK calls are redirecting to Active NN
> ---
>
> Key: HDFS-15220
> URL: https://issues.apache.org/jira/browse/HDFS-15220
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: krishna reddy
>Assignee: Ravuri Sushma sree
>Priority: Major
> Attachments: screenshot-1.png
>
>
> Run any fsck except (-delete & - move) should go to ONN as it is read 
> operation
> In below image spikes indicates when it ran fsck / -storagepolicies
>  !screenshot-1.png! 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Updated] (HDFS-12136) BlockSender performance regression due to volume scanner edge case

2020-03-11 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-12136:
---
Resolution: Won't Fix
Status: Resolved  (was: Patch Available)

Resolved by HDFS-11187 

> BlockSender performance regression due to volume scanner edge case
> --
>
> Key: HDFS-12136
> URL: https://issues.apache.org/jira/browse/HDFS-12136
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: datanode
>Affects Versions: 2.8.0
>Reporter: Daryn Sharp
>Assignee: Daryn Sharp
>Priority: Critical
> Attachments: HDFS-12136.branch-2.patch, HDFS-12136.trunk.patch
>
>
> HDFS-11160 attempted to fix a volume scan race for a file appended mid-scan 
> by reading the last checksum of finalized blocks within the {{BlockSender}} 
> ctor.  Unfortunately it's holding the exclusive dataset lock to open and read 
> the metafile multiple times  Block sender instantiation becomes serialized.
> Performance completely collapses under heavy disk i/o utilization or high 
> xceiver activity.  Ex. lost node replication, balancing, or decommissioning.  
> The xceiver threads congest creating block senders and impair the heartbeat 
> processing that is contending for the same lock.  Combined with other lock 
> contention issues, pipelines break and nodes sporadically go dead.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Updated] (HDFS-14338) TestPread timeouts in branch-2.8

2020-03-11 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-14338:
---
Resolution: Won't Fix
Status: Resolved  (was: Patch Available)

Branch-2.8 is EOL. Resolve as Won't Fix.

> TestPread timeouts in branch-2.8
> 
>
> Key: HDFS-14338
> URL: https://issues.apache.org/jira/browse/HDFS-14338
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: test
>Reporter: Akira Ajisaka
>Assignee: Akira Ajisaka
>Priority: Major
> Attachments: HDFS-14338-001.patch, 
> HDFS-14338-branch-2.8-001-testing.patch, HDFS-14338-branch-2.8-001.patch
>
>
> TestPread timeouts in branch-2.8.
> {noformat}
> ---
>  T E S T S
> ---
> OpenJDK 64-Bit Server VM warning: ignoring option MaxPermSize=768m; support 
> was removed in 8.0
> Running org.apache.hadoop.hdfs.TestPread
> Results :
> Tests run: 0, Failures: 0, Errors: 0, Skipped: 0
> {noformat}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-15039) Cache meta file length of FinalizedReplica to reduce call File.length()

2020-03-11 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-15039:


The patch doesn't apply any more. Updated the patch to resolve conflicts.

> Cache meta file length of FinalizedReplica to reduce call File.length()
> ---
>
> Key: HDFS-15039
> URL: https://issues.apache.org/jira/browse/HDFS-15039
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: datanode
>Reporter: Yang Yun
>Assignee: Yang Yun
>Priority: Minor
> Attachments: HDFS-15039.006.patch, HDFS-15039.patch, 
> HDFS-15039.patch, HDFS-15039.patch, HDFS-15039.patch, HDFS-15039.patch
>
>
> When use ReplicaCachingGetSpaceUsed to get the volume space used.  It will 
> call File.length() for every meta file of replica. That add more disk IO, we 
> found the slow log as below. For finalized replica, the size of meta file is 
> not changed, i think we can cache the value.
> {code:java}
> org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.ReplicaCachingGetSpaceUsed:
>  Refresh dfs used, bpid: BP-898717543-10.75.1.240-1519386995727 replicas 
> size: 1166 dfsUsed: 72227113183 on volume: 
> DS-3add8d62-d69a-4f5a-a29f-b7bbb400af2e duration: 17206ms{code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Updated] (HDFS-15039) Cache meta file length of FinalizedReplica to reduce call File.length()

2020-03-11 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-15039:
---
Attachment: HDFS-15039.006.patch

> Cache meta file length of FinalizedReplica to reduce call File.length()
> ---
>
> Key: HDFS-15039
> URL: https://issues.apache.org/jira/browse/HDFS-15039
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: datanode
>Reporter: Yang Yun
>Assignee: Yang Yun
>Priority: Minor
> Attachments: HDFS-15039.006.patch, HDFS-15039.patch, 
> HDFS-15039.patch, HDFS-15039.patch, HDFS-15039.patch, HDFS-15039.patch
>
>
> When use ReplicaCachingGetSpaceUsed to get the volume space used.  It will 
> call File.length() for every meta file of replica. That add more disk IO, we 
> found the slow log as below. For finalized replica, the size of meta file is 
> not changed, i think we can cache the value.
> {code:java}
> org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.ReplicaCachingGetSpaceUsed:
>  Refresh dfs used, bpid: BP-898717543-10.75.1.240-1519386995727 replicas 
> size: 1166 dfsUsed: 72227113183 on volume: 
> DS-3add8d62-d69a-4f5a-a29f-b7bbb400af2e duration: 17206ms{code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Updated] (HDFS-13351) Revert HDFS-11156 from branch-2/branch-2.8

2020-03-11 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-13351:
---
Target Version/s: 2.10.1
  Labels: release-blocker  (was: )

> Revert HDFS-11156 from branch-2/branch-2.8
> --
>
> Key: HDFS-13351
> URL: https://issues.apache.org/jira/browse/HDFS-13351
> Project: Hadoop HDFS
>  Issue Type: Task
>  Components: webhdfs
>Reporter: Weiwei Yang
>Assignee: Weiwei Yang
>Priority: Major
>  Labels: release-blocker
> Attachments: HDFS-13351-branch-2.001.patch, 
> HDFS-13351-branch-2.002.patch, HDFS-13351-branch-2.003.patch
>
>
> Per discussion in HDFS-11156, lets revert the change from branch-2 and 
> branch-2.8. New patch can be tracked in HDFS-12459 .



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-15039) Cache meta file length of FinalizedReplica to reduce call File.length()

2020-03-11 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-15039:


+1

> Cache meta file length of FinalizedReplica to reduce call File.length()
> ---
>
> Key: HDFS-15039
> URL: https://issues.apache.org/jira/browse/HDFS-15039
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: datanode
>Reporter: Yang Yun
>Assignee: Yang Yun
>Priority: Minor
> Attachments: HDFS-15039.patch, HDFS-15039.patch, HDFS-15039.patch, 
> HDFS-15039.patch, HDFS-15039.patch
>
>
> When use ReplicaCachingGetSpaceUsed to get the volume space used.  It will 
> call File.length() for every meta file of replica. That add more disk IO, we 
> found the slow log as below. For finalized replica, the size of meta file is 
> not changed, i think we can cache the value.
> {code:java}
> org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.ReplicaCachingGetSpaceUsed:
>  Refresh dfs used, bpid: BP-898717543-10.75.1.240-1519386995727 replicas 
> size: 1166 dfsUsed: 72227113183 on volume: 
> DS-3add8d62-d69a-4f5a-a29f-b7bbb400af2e duration: 17206ms{code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-14820) The default 8KB buffer of BlockReaderRemote#newBlockReader#BufferedOutputStream is too big

2020-03-11 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-14820:


I am +1 and will commit by end of week unless there's objection to my 
explanation above. Thanks.

>  The default 8KB buffer of 
> BlockReaderRemote#newBlockReader#BufferedOutputStream is too big
> ---
>
> Key: HDFS-14820
> URL: https://issues.apache.org/jira/browse/HDFS-14820
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: Lisheng Sun
>Assignee: Lisheng Sun
>Priority: Major
> Attachments: HDFS-14820.001.patch, HDFS-14820.002.patch, 
> HDFS-14820.003.patch
>
>
> this issue is similar to HDFS-14535.
> {code:java}
> public static BlockReader newBlockReader(String file,
> ExtendedBlock block,
> Token blockToken,
> long startOffset, long len,
> boolean verifyChecksum,
> String clientName,
> Peer peer, DatanodeID datanodeID,
> PeerCache peerCache,
> CachingStrategy cachingStrategy,
> int networkDistance) throws IOException {
>   // in and out will be closed when sock is closed (by the caller)
>   final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
>   peer.getOutputStream()));
>   new Sender(out).readBlock(block, blockToken, clientName, startOffset, len,
>   verifyChecksum, cachingStrategy);
> }
> public BufferedOutputStream(OutputStream out) {
> this(out, 8192);
> }
> {code}
> Sender#readBlock parameter( block,blockToken, clientName, startOffset, len, 
> verifyChecksum, cachingStrategy) could not use such a big buffer.
> So i think it should reduce BufferedOutputStream buffer.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Updated] (HDFS-15208) Supress bogus AbstractWadlGeneratorGrammarGenerator in KMS stderr in hdfs

2020-03-05 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-15208:
---
Priority: Trivial  (was: Major)

> Supress bogus AbstractWadlGeneratorGrammarGenerator in KMS stderr in hdfs
> -
>
> Key: HDFS-15208
> URL: https://issues.apache.org/jira/browse/HDFS-15208
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: Wei-Chiu Chuang
>Assignee: Wei-Chiu Chuang
>Priority: Trivial
>
> Continuation of HADOOP-15686
> Add the same log4j property to disable error log in hadoop-hdfs.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Assigned] (HDFS-15208) Supress bogus AbstractWadlGeneratorGrammarGenerator in KMS stderr in hdfs

2020-03-05 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang reassigned HDFS-15208:
--

Assignee: Wei-Chiu Chuang

> Supress bogus AbstractWadlGeneratorGrammarGenerator in KMS stderr in hdfs
> -
>
> Key: HDFS-15208
> URL: https://issues.apache.org/jira/browse/HDFS-15208
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: Wei-Chiu Chuang
>Assignee: Wei-Chiu Chuang
>Priority: Major
>
> Continuation of HADOOP-15686
> Add the same log4j property to disable error log in hadoop-hdfs.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Created] (HDFS-15208) Supress bogus AbstractWadlGeneratorGrammarGenerator in KMS stderr in hdfs

2020-03-05 Thread Wei-Chiu Chuang (Jira)
Wei-Chiu Chuang created HDFS-15208:
--

 Summary: Supress bogus AbstractWadlGeneratorGrammarGenerator in 
KMS stderr in hdfs
 Key: HDFS-15208
 URL: https://issues.apache.org/jira/browse/HDFS-15208
 Project: Hadoop HDFS
  Issue Type: Bug
Reporter: Wei-Chiu Chuang


Continuation of HADOOP-15686

Add the same log4j property to disable error log in hadoop-hdfs.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-15113) Missing IBR when NameNode restart if open processCommand async feature

2020-03-05 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-15113:


i think TestEncryptionZones and TestEncryptionZonesWithKMS are broken by 
HADOOP-16885

> Missing IBR when NameNode restart if open processCommand async feature
> --
>
> Key: HDFS-15113
> URL: https://issues.apache.org/jira/browse/HDFS-15113
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode
>Reporter: Xiaoqiao He
>Assignee: Xiaoqiao He
>Priority: Blocker
> Attachments: HDFS-15113.001.patch, HDFS-15113.002.patch, 
> HDFS-15113.003.patch, HDFS-15113.004.patch, HDFS-15113.005.patch
>
>
> Recently, I meet one case that NameNode missing block after restart which is 
> related with HDFS-14997.
> a. during NameNode restart, it will return command `DNA_REGISTER` to DataNode 
> when receive some RPC request from DataNode.
> b. when DataNode receive `DNA_REGISTER` command, it will run #reRegister 
> async.
> {code:java}
>   void reRegister() throws IOException {
> if (shouldRun()) {
>   // re-retrieve namespace info to make sure that, if the NN
>   // was restarted, we still match its version (HDFS-2120)
>   NamespaceInfo nsInfo = retrieveNamespaceInfo();
>   // and re-register
>   register(nsInfo);
>   scheduler.scheduleHeartbeat();
>   // HDFS-9917,Standby NN IBR can be very huge if standby namenode is down
>   // for sometime.
>   if (state == HAServiceState.STANDBY || state == 
> HAServiceState.OBSERVER) {
> ibrManager.clearIBRs();
>   }
> }
>   }
> {code}
> c. As we know, #register will trigger BR immediately.
> d. because #reRegister run async, so we could not make sure which one run 
> first between send FBR and clear IBR. If clean IBR run first, it will be OK. 
> But if send FBR first then clear IBR, it will missing some blocks received 
> between these two time point until next FBR.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-15205) FSImage sort section logic is wrong

2020-03-03 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-15205:


Upgrade to blocker to raise awareness. The code in question was written back in 
2014 (HDFS-5698). Unsure how this is only exposed now.

> FSImage sort section logic is wrong
> ---
>
> Key: HDFS-15205
> URL: https://issues.apache.org/jira/browse/HDFS-15205
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: angerszhu
>Priority: Blocker
> Attachments: HDFS-15205.001.patch
>
>
> When load FSImage, it will sort sections in FileSummary and load Section's in 
> SectionName enum sequence. But the sort method is wrong , when I use 
> branch-2.6.0 to load fsimage write by branch-2 with patch  
> https://issues.apache.org/jira/browse/HDFS-14771, it will throw NPE because 
> it load INODE first 
> {code:java}
> 2020-03-03 14:33:26,618 ERROR 
> org.apache.hadoop.hdfs.server.namenode.NameNode: Failed to start namenode.
> java.lang.NullPointerException
> at 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode$Loader.loadPermission(FSImageFormatPBINode.java:101)
>   at 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode$Loader.loadINodeDirectory(FSImageFormatPBINode.java:148)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode$Loader.loadRootINode(FSImageFormatPBINode.java:332)
>   at 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode$Loader.loadINodeSection(FSImageFormatPBINode.java:218)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf$Loader.loadInternal(FSImageFormatProtobuf.java:254)
>   at 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf$Loader.load(FSImageFormatProtobuf.java:180)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormat$LoaderDelegator.load(FSImageFormat.java:226)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSImage.loadFSImage(FSImage.java:1036)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSImage.loadFSImage(FSImage.java:1020)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSImage.loadFSImageFile(FSImage.java:741)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSImage.loadFSImage(FSImage.java:677)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSImage.recoverTransitionRead(FSImage.java:290)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.loadFSImage(FSNamesystem.java:1092)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.loadFromDisk(FSNamesystem.java:780)
> at 
> org.apache.hadoop.hdfs.server.namenode.NameNode.loadNamesystem(NameNode.java:609)
> at 
> org.apache.hadoop.hdfs.server.namenode.NameNode.initialize(NameNode.java:666)
> at 
> org.apache.hadoop.hdfs.server.namenode.NameNode.(NameNode.java:838)
> at 
> org.apache.hadoop.hdfs.server.namenode.NameNode.(NameNode.java:817)
> at 
> org.apache.hadoop.hdfs.server.namenode.NameNode.createNameNode(NameNode.java:1538)
> at 
> org.apache.hadoop.hdfs.server.namenode.NameNode.main(NameNode.java:1606)
> {code}
> I print the load  order:
> {code:java}
> 2020-03-03 15:49:36,424 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf: [name = INODE,  
> offset = 37, length = 11790829 ]
> 2020-03-03 15:49:36,424 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf: [name = 
> INODE_SUB,  offset = 37, length = 826591 ]
> 2020-03-03 15:49:36,424 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf: [name = 
> INODE_SUB,  offset = 826628, length = 828192 ]
> 2020-03-03 15:49:36,424 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf: [name = 
> INODE_SUB,  offset = 1654820, length = 835240 ]
> 2020-03-03 15:49:36,424 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf: [name = 
> INODE_SUB,  offset = 2490060, length = 833630 ]
> 2020-03-03 15:49:36,424 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf: [name = 
> INODE_SUB,  offset = 3323690, length = 909445 ]
> 2020-03-03 15:49:36,424 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf: [name = 
> INODE_SUB,  offset = 4233135, length = 866147 ]
> 2020-03-03 15:49:36,424 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf: [name = 
> INODE_SUB,  offset = 5099282, length = 1272751 ]
> 2020-03-03 15:49:36,424 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf: [name = 
> INODE_SUB,  offset = 6372033, length = 1311876 ]
> 2020-03-03 15:49:36,424 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf: [name = 
> INODE_SUB,  offset = 7683909, length = 1251510 ]
> 2020-03-03 15:49:36,424 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf: [name = 
> INODE_SUB,  offset = 8935419, length = 

[jira] [Commented] (HDFS-15205) FSImage sort section logic is wrong

2020-03-03 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-15205:


[~hexiaoqiao] [~sodonnell] mind to take a look?

> FSImage sort section logic is wrong
> ---
>
> Key: HDFS-15205
> URL: https://issues.apache.org/jira/browse/HDFS-15205
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: angerszhu
>Priority: Major
> Attachments: HDFS-15205.001.patch
>
>
> When load FSImage, it will sort sections in FileSummary and load Section's in 
> SectionName enum sequence. But the sort method is wrong , when I use 
> branch-2.6.0 to load fsimage write by branch-2 with patch  
> https://issues.apache.org/jira/browse/HDFS-14771, it will throw NPE because 
> it load INODE first 
> {code:java}
> 2020-03-03 14:33:26,618 ERROR 
> org.apache.hadoop.hdfs.server.namenode.NameNode: Failed to start namenode.
> java.lang.NullPointerException
> at 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode$Loader.loadPermission(FSImageFormatPBINode.java:101)
>   at 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode$Loader.loadINodeDirectory(FSImageFormatPBINode.java:148)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode$Loader.loadRootINode(FSImageFormatPBINode.java:332)
>   at 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode$Loader.loadINodeSection(FSImageFormatPBINode.java:218)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf$Loader.loadInternal(FSImageFormatProtobuf.java:254)
>   at 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf$Loader.load(FSImageFormatProtobuf.java:180)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormat$LoaderDelegator.load(FSImageFormat.java:226)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSImage.loadFSImage(FSImage.java:1036)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSImage.loadFSImage(FSImage.java:1020)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSImage.loadFSImageFile(FSImage.java:741)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSImage.loadFSImage(FSImage.java:677)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSImage.recoverTransitionRead(FSImage.java:290)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.loadFSImage(FSNamesystem.java:1092)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.loadFromDisk(FSNamesystem.java:780)
> at 
> org.apache.hadoop.hdfs.server.namenode.NameNode.loadNamesystem(NameNode.java:609)
> at 
> org.apache.hadoop.hdfs.server.namenode.NameNode.initialize(NameNode.java:666)
> at 
> org.apache.hadoop.hdfs.server.namenode.NameNode.(NameNode.java:838)
> at 
> org.apache.hadoop.hdfs.server.namenode.NameNode.(NameNode.java:817)
> at 
> org.apache.hadoop.hdfs.server.namenode.NameNode.createNameNode(NameNode.java:1538)
> at 
> org.apache.hadoop.hdfs.server.namenode.NameNode.main(NameNode.java:1606)
> {code}
> I print the load  order:
> {code:java}
> 2020-03-03 15:49:36,424 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf: [name = INODE,  
> offset = 37, length = 11790829 ]
> 2020-03-03 15:49:36,424 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf: [name = 
> INODE_SUB,  offset = 37, length = 826591 ]
> 2020-03-03 15:49:36,424 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf: [name = 
> INODE_SUB,  offset = 826628, length = 828192 ]
> 2020-03-03 15:49:36,424 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf: [name = 
> INODE_SUB,  offset = 1654820, length = 835240 ]
> 2020-03-03 15:49:36,424 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf: [name = 
> INODE_SUB,  offset = 2490060, length = 833630 ]
> 2020-03-03 15:49:36,424 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf: [name = 
> INODE_SUB,  offset = 3323690, length = 909445 ]
> 2020-03-03 15:49:36,424 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf: [name = 
> INODE_SUB,  offset = 4233135, length = 866147 ]
> 2020-03-03 15:49:36,424 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf: [name = 
> INODE_SUB,  offset = 5099282, length = 1272751 ]
> 2020-03-03 15:49:36,424 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf: [name = 
> INODE_SUB,  offset = 6372033, length = 1311876 ]
> 2020-03-03 15:49:36,424 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf: [name = 
> INODE_SUB,  offset = 7683909, length = 1251510 ]
> 2020-03-03 15:49:36,424 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf: [name = 
> INODE_SUB,  offset = 8935419, length = 1296120 ]
> 2020-03-03 15:49:36,424 INFO 
> 

[jira] [Updated] (HDFS-15205) FSImage sort section logic is wrong

2020-03-03 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-15205:
---
Priority: Blocker  (was: Major)

> FSImage sort section logic is wrong
> ---
>
> Key: HDFS-15205
> URL: https://issues.apache.org/jira/browse/HDFS-15205
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: angerszhu
>Priority: Blocker
> Attachments: HDFS-15205.001.patch
>
>
> When load FSImage, it will sort sections in FileSummary and load Section's in 
> SectionName enum sequence. But the sort method is wrong , when I use 
> branch-2.6.0 to load fsimage write by branch-2 with patch  
> https://issues.apache.org/jira/browse/HDFS-14771, it will throw NPE because 
> it load INODE first 
> {code:java}
> 2020-03-03 14:33:26,618 ERROR 
> org.apache.hadoop.hdfs.server.namenode.NameNode: Failed to start namenode.
> java.lang.NullPointerException
> at 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode$Loader.loadPermission(FSImageFormatPBINode.java:101)
>   at 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode$Loader.loadINodeDirectory(FSImageFormatPBINode.java:148)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode$Loader.loadRootINode(FSImageFormatPBINode.java:332)
>   at 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode$Loader.loadINodeSection(FSImageFormatPBINode.java:218)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf$Loader.loadInternal(FSImageFormatProtobuf.java:254)
>   at 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf$Loader.load(FSImageFormatProtobuf.java:180)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormat$LoaderDelegator.load(FSImageFormat.java:226)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSImage.loadFSImage(FSImage.java:1036)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSImage.loadFSImage(FSImage.java:1020)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSImage.loadFSImageFile(FSImage.java:741)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSImage.loadFSImage(FSImage.java:677)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSImage.recoverTransitionRead(FSImage.java:290)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.loadFSImage(FSNamesystem.java:1092)
> at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.loadFromDisk(FSNamesystem.java:780)
> at 
> org.apache.hadoop.hdfs.server.namenode.NameNode.loadNamesystem(NameNode.java:609)
> at 
> org.apache.hadoop.hdfs.server.namenode.NameNode.initialize(NameNode.java:666)
> at 
> org.apache.hadoop.hdfs.server.namenode.NameNode.(NameNode.java:838)
> at 
> org.apache.hadoop.hdfs.server.namenode.NameNode.(NameNode.java:817)
> at 
> org.apache.hadoop.hdfs.server.namenode.NameNode.createNameNode(NameNode.java:1538)
> at 
> org.apache.hadoop.hdfs.server.namenode.NameNode.main(NameNode.java:1606)
> {code}
> I print the load  order:
> {code:java}
> 2020-03-03 15:49:36,424 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf: [name = INODE,  
> offset = 37, length = 11790829 ]
> 2020-03-03 15:49:36,424 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf: [name = 
> INODE_SUB,  offset = 37, length = 826591 ]
> 2020-03-03 15:49:36,424 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf: [name = 
> INODE_SUB,  offset = 826628, length = 828192 ]
> 2020-03-03 15:49:36,424 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf: [name = 
> INODE_SUB,  offset = 1654820, length = 835240 ]
> 2020-03-03 15:49:36,424 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf: [name = 
> INODE_SUB,  offset = 2490060, length = 833630 ]
> 2020-03-03 15:49:36,424 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf: [name = 
> INODE_SUB,  offset = 3323690, length = 909445 ]
> 2020-03-03 15:49:36,424 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf: [name = 
> INODE_SUB,  offset = 4233135, length = 866147 ]
> 2020-03-03 15:49:36,424 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf: [name = 
> INODE_SUB,  offset = 5099282, length = 1272751 ]
> 2020-03-03 15:49:36,424 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf: [name = 
> INODE_SUB,  offset = 6372033, length = 1311876 ]
> 2020-03-03 15:49:36,424 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf: [name = 
> INODE_SUB,  offset = 7683909, length = 1251510 ]
> 2020-03-03 15:49:36,424 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf: [name = 
> INODE_SUB,  offset = 8935419, length = 1296120 ]
> 2020-03-03 15:49:36,424 INFO 
> org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf: [name = 
> INODE_SUB,  offset = 

[jira] [Updated] (HDFS-15202) HDFS-client: boost ShortCircuit Cache

2020-03-03 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-15202:
---
Component/s: dfsclient

> HDFS-client: boost ShortCircuit Cache
> -
>
> Key: HDFS-15202
> URL: https://issues.apache.org/jira/browse/HDFS-15202
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: dfsclient
> Environment: 4 nodes E5-2698 v4 @ 2.20GHz, 700 Gb Mem.
> 8 RegionServers (2 by host)
> 8 tables by 64 regions by 1.88 Gb data in each = 900 Gb total
> Random read in 800 threads via YCSB and a little bit updates (10% of reads)
>Reporter: Danil Lipovoy
>Assignee: Danil Lipovoy
>Priority: Minor
> Attachments: hdfs_cpu.png, hdfs_reads.png
>
>
> I want to propose how to improve reading performance HDFS-client. The idea: 
> create few instances ShortCircuit caches instead of one. 
> The key points:
> 1. Create array of caches (set by 
> clientShortCircuitNum=*dfs.client.short.circuit.num*, see in the pull 
> requests below):
> {code:java}
> private ClientContext(String name, DfsClientConf conf, Configuration config) {
> ...
> shortCircuitCache = new ShortCircuitCache[this.clientShortCircuitNum];
> for (int i = 0; i < this.clientShortCircuitNum; i++) {
>   this.shortCircuitCache[i] = ShortCircuitCache.fromConf(scConf);
> }
> {code}
> 2 Then divide blocks by caches:
> {code:java}
>   public ShortCircuitCache getShortCircuitCache(long idx) {
> return shortCircuitCache[(int) (idx % clientShortCircuitNum)];
>   }
> {code}
> 3. And how to call it:
> {code:java}
> ShortCircuitCache cache = 
> clientContext.getShortCircuitCache(block.getBlockId());
> {code}
> The last number of offset evenly distributed from 0 to 9 - that's why all 
> caches will full approximately the same.
> It is good for performance. Below the attachment, it is load test reading 
> HDFS via HBase where clientShortCircuitNum = 1 vs 3. We can see that 
> performance grows ~30%, CPU usage about +15%. 
> Hope it is interesting for someone.
> Ready to explain some unobvious things.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Assigned] (HDFS-14809) Reduce BlockReaderLocal RPC calls

2020-03-03 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang reassigned HDFS-14809:
--

Assignee: KenCao  (was: kencao)

> Reduce BlockReaderLocal RPC calls
> -
>
> Key: HDFS-14809
> URL: https://issues.apache.org/jira/browse/HDFS-14809
> Project: Hadoop HDFS
>  Issue Type: New Feature
>Affects Versions: 2.6.0
>Reporter: KenCao
>Assignee: KenCao
>Priority: Major
> Attachments: HADOOP-14809
>
>
> as we known, the hdfs client java lib uses BlockReaderLocal for short circuit 
> read by default, which allocate shared memory first, and make a slot within 
> it. After all these steps, it will request the fds from the DataNode. 
> However, the slot and shared memory sturcture is only used by DataNode when 
> uncaching replicas, the client process can work well just with the fds asked 
> later and it is nearly impossible to cache replicas in product environment. 
> The api to release fds is called by client only with the slot given, the fds 
> is close in the client process finally.  
> so i think we can make a new BlockReader implementation which just requests 
> the fds, and it will reduce the rpc calls from 3(allocate shm, request fds, 
> release fds) to 1(request fds).



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
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-14820) The default 8KB buffer of BlockReaderRemote#newBlockReader#BufferedOutputStream is too big

2020-03-03 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang edited comment on HDFS-14820 at 3/3/20 8:16 PM:


The current implementation is, DFS client sends a request (which is short) to 
DataNode asking for a block using an output stream. After that, client receives 
block data DataNode  (which can be several MBs long) using an input stream.

This patch changes the buffer size of the former, the output stream. There is 
absolutely no reason to use a 8kb buffer size for this stream. For the input 
stream, yes what [~eyang] says makes sense.

The following is the code snippet for the data sent in each output stream)

{code}
OpReadBlockProto proto = OpReadBlockProto.newBuilder()
.setHeader(DataTransferProtoUtil.buildClientHeader(blk, clientName,
blockToken))
.setOffset(blockOffset)
.setLen(length)
.setSendChecksums(sendChecksum)
.setCachingStrategy(getCachingStrategy(cachingStrategy))
.build();
{code}

Also note that the stream objects are not recycled. One block is one 
output/input stream object.


was (Author: jojochuang):
The current implementation is, DFS client send a request (which is short) to 
DataNode asking for a block using an output stream. After that, client receives 
block data DataNode  (which can be several MBs long) using an input stream.

This patch changes the buffer size of the former, output stream. There is 
absolutely no reason to use a 8kb buffer size for this stream. The input 
stream, yes what [~eyang] says makes sense.

{code}
OpReadBlockProto proto = OpReadBlockProto.newBuilder()
.setHeader(DataTransferProtoUtil.buildClientHeader(blk, clientName,
blockToken))
.setOffset(blockOffset)
.setLen(length)
.setSendChecksums(sendChecksum)
.setCachingStrategy(getCachingStrategy(cachingStrategy))
.build();
{code}

Also note that the stream objects are not recycled. One block is one 
output/input stream object.

>  The default 8KB buffer of 
> BlockReaderRemote#newBlockReader#BufferedOutputStream is too big
> ---
>
> Key: HDFS-14820
> URL: https://issues.apache.org/jira/browse/HDFS-14820
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: Lisheng Sun
>Assignee: Lisheng Sun
>Priority: Major
> Attachments: HDFS-14820.001.patch, HDFS-14820.002.patch, 
> HDFS-14820.003.patch
>
>
> this issue is similar to HDFS-14535.
> {code:java}
> public static BlockReader newBlockReader(String file,
> ExtendedBlock block,
> Token blockToken,
> long startOffset, long len,
> boolean verifyChecksum,
> String clientName,
> Peer peer, DatanodeID datanodeID,
> PeerCache peerCache,
> CachingStrategy cachingStrategy,
> int networkDistance) throws IOException {
>   // in and out will be closed when sock is closed (by the caller)
>   final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
>   peer.getOutputStream()));
>   new Sender(out).readBlock(block, blockToken, clientName, startOffset, len,
>   verifyChecksum, cachingStrategy);
> }
> public BufferedOutputStream(OutputStream out) {
> this(out, 8192);
> }
> {code}
> Sender#readBlock parameter( block,blockToken, clientName, startOffset, len, 
> verifyChecksum, cachingStrategy) could not use such a big buffer.
> So i think it should reduce BufferedOutputStream buffer.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Updated] (HDFS-14809) Reduce BlockReaderLocal RPC calls

2020-03-03 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-14809:
---
Summary: Reduce BlockReaderLocal RPC calls  (was: Make a new BlockReader  
for hdfs client lib)

> Reduce BlockReaderLocal RPC calls
> -
>
> Key: HDFS-14809
> URL: https://issues.apache.org/jira/browse/HDFS-14809
> Project: Hadoop HDFS
>  Issue Type: New Feature
>Affects Versions: 2.6.0
>Reporter: KenCao
>Assignee: kencao
>Priority: Major
> Attachments: HADOOP-14809
>
>
> as we known, the hdfs client java lib uses BlockReaderLocal for short circuit 
> read by default, which allocate shared memory first, and make a slot within 
> it. After all these steps, it will request the fds from the DataNode. 
> However, the slot and shared memory sturcture is only used by DataNode when 
> uncaching replicas, the client process can work well just with the fds asked 
> later and it is nearly impossible to cache replicas in product environment. 
> The api to release fds is called by client only with the slot given, the fds 
> is close in the client process finally.  
> so i think we can make a new BlockReader implementation which just requests 
> the fds, and it will reduce the rpc calls from 3(allocate shm, request fds, 
> release fds) to 1(request fds).



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-14809) Reduce BlockReaderLocal RPC calls

2020-03-03 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-14809:


Updated the summary based on the suggestion.

I'm extremely sorry for missing out this one. [~leosun08] [~openinx] does this 
make sense to you?

> Reduce BlockReaderLocal RPC calls
> -
>
> Key: HDFS-14809
> URL: https://issues.apache.org/jira/browse/HDFS-14809
> Project: Hadoop HDFS
>  Issue Type: New Feature
>Affects Versions: 2.6.0
>Reporter: KenCao
>Assignee: kencao
>Priority: Major
> Attachments: HADOOP-14809
>
>
> as we known, the hdfs client java lib uses BlockReaderLocal for short circuit 
> read by default, which allocate shared memory first, and make a slot within 
> it. After all these steps, it will request the fds from the DataNode. 
> However, the slot and shared memory sturcture is only used by DataNode when 
> uncaching replicas, the client process can work well just with the fds asked 
> later and it is nearly impossible to cache replicas in product environment. 
> The api to release fds is called by client only with the slot given, the fds 
> is close in the client process finally.  
> so i think we can make a new BlockReader implementation which just requests 
> the fds, and it will reduce the rpc calls from 3(allocate shm, request fds, 
> release fds) to 1(request fds).



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Updated] (HDFS-14809) Reduce BlockReaderLocal RPC calls

2020-03-03 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-14809:
---
Status: Patch Available  (was: Open)

> Reduce BlockReaderLocal RPC calls
> -
>
> Key: HDFS-14809
> URL: https://issues.apache.org/jira/browse/HDFS-14809
> Project: Hadoop HDFS
>  Issue Type: New Feature
>Affects Versions: 2.6.0
>Reporter: KenCao
>Assignee: kencao
>Priority: Major
> Attachments: HADOOP-14809
>
>
> as we known, the hdfs client java lib uses BlockReaderLocal for short circuit 
> read by default, which allocate shared memory first, and make a slot within 
> it. After all these steps, it will request the fds from the DataNode. 
> However, the slot and shared memory sturcture is only used by DataNode when 
> uncaching replicas, the client process can work well just with the fds asked 
> later and it is nearly impossible to cache replicas in product environment. 
> The api to release fds is called by client only with the slot given, the fds 
> is close in the client process finally.  
> so i think we can make a new BlockReader implementation which just requests 
> the fds, and it will reduce the rpc calls from 3(allocate shm, request fds, 
> release fds) to 1(request fds).



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Assigned] (HDFS-14809) Make a new BlockReader for hdfs client lib

2020-03-03 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang reassigned HDFS-14809:
--

Assignee: kencao

> Make a new BlockReader  for hdfs client lib
> ---
>
> Key: HDFS-14809
> URL: https://issues.apache.org/jira/browse/HDFS-14809
> Project: Hadoop HDFS
>  Issue Type: New Feature
>Affects Versions: 2.6.0
>Reporter: KenCao
>Assignee: kencao
>Priority: Major
> Attachments: HADOOP-14809
>
>
> as we known, the hdfs client java lib uses BlockReaderLocal for short circuit 
> read by default, which allocate shared memory first, and make a slot within 
> it. After all these steps, it will request the fds from the DataNode. 
> However, the slot and shared memory sturcture is only used by DataNode when 
> uncaching replicas, the client process can work well just with the fds asked 
> later and it is nearly impossible to cache replicas in product environment. 
> The api to release fds is called by client only with the slot given, the fds 
> is close in the client process finally.  
> so i think we can make a new BlockReader implementation which just requests 
> the fds, and it will reduce the rpc calls from 3(allocate shm, request fds, 
> release fds) to 1(request fds).



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-15202) HDFS-client: boost ShortCircuit Cache

2020-03-03 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-15202:


[~pustota] could you raise a PR against https://github.com/apache/hadoop (trunk 
branch)
I'm sorry it must be confusing to you.

> HDFS-client: boost ShortCircuit Cache
> -
>
> Key: HDFS-15202
> URL: https://issues.apache.org/jira/browse/HDFS-15202
> Project: Hadoop HDFS
>  Issue Type: Improvement
> Environment: 4 nodes E5-2698 v4 @ 2.20GHz, 700 Gb Mem.
> 8 RegionServers (2 by host)
> 8 tables by 64 regions by 1.88 Gb data in each = 900 Gb total
> Random read in 800 threads via YCSB and a little bit updates (10% of reads)
>Reporter: Danil Lipovoy
>Assignee: Danil Lipovoy
>Priority: Minor
> Attachments: hdfs_cpu.png, hdfs_reads.png
>
>
> I want to propose how to improve reading performance HDFS-client. The idea: 
> create few instances ShortCircuit caches instead of one. 
> The key points:
> 1. Create array of caches (set by 
> clientShortCircuitNum=*dfs.client.short.circuit.num*, see in the pull 
> requests below):
> {code:java}
> private ClientContext(String name, DfsClientConf conf, Configuration config) {
> ...
> shortCircuitCache = new ShortCircuitCache[this.clientShortCircuitNum];
> for (int i = 0; i < this.clientShortCircuitNum; i++) {
>   this.shortCircuitCache[i] = ShortCircuitCache.fromConf(scConf);
> }
> {code}
> 2 Then divide blocks by caches:
> {code:java}
>   public ShortCircuitCache getShortCircuitCache(long idx) {
> return shortCircuitCache[(int) (idx % clientShortCircuitNum)];
>   }
> {code}
> 3. And how to call it:
> {code:java}
> ShortCircuitCache cache = 
> clientContext.getShortCircuitCache(block.getBlockId());
> {code}
> The last number of offset evenly distributed from 0 to 9 - that's why all 
> caches will full approximately the same.
> It is good for performance. Below the attachment, it is load test reading 
> HDFS via HBase where clientShortCircuitNum = 1 vs 3. We can see that 
> performance grows ~30%, CPU usage about +15%. 
> Hope it is interesting for someone.
> Ready to explain some unobvious things.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-15202) HDFS-client: boost ShortCircuit Cache

2020-03-03 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-15202:


[~openinx] [~leosun08] are you guys interested in reviewing this improvement 
since this concerns SCR performance?

> HDFS-client: boost ShortCircuit Cache
> -
>
> Key: HDFS-15202
> URL: https://issues.apache.org/jira/browse/HDFS-15202
> Project: Hadoop HDFS
>  Issue Type: Improvement
> Environment: 4 nodes E5-2698 v4 @ 2.20GHz, 700 Gb Mem.
> 8 RegionServers (2 by host)
> 8 tables by 64 regions by 1.88 Gb data in each = 900 Gb total
> Random read in 800 threads via YCSB and a little bit updates (10% of reads)
>Reporter: Danil Lipovoy
>Assignee: Danil Lipovoy
>Priority: Minor
> Attachments: hdfs_cpu.png, hdfs_reads.png
>
>
> I want to propose how to improve reading performance HDFS-client. The idea: 
> create few instances ShortCircuit caches instead of one. 
> The key points:
> 1. Create array of caches (set by 
> clientShortCircuitNum=*dfs.client.short.circuit.num*, see in the pull 
> requests below):
> {code:java}
> private ClientContext(String name, DfsClientConf conf, Configuration config) {
> ...
> shortCircuitCache = new ShortCircuitCache[this.clientShortCircuitNum];
> for (int i = 0; i < this.clientShortCircuitNum; i++) {
>   this.shortCircuitCache[i] = ShortCircuitCache.fromConf(scConf);
> }
> {code}
> 2 Then divide blocks by caches:
> {code:java}
>   public ShortCircuitCache getShortCircuitCache(long idx) {
> return shortCircuitCache[(int) (idx % clientShortCircuitNum)];
>   }
> {code}
> 3. And how to call it:
> {code:java}
> ShortCircuitCache cache = 
> clientContext.getShortCircuitCache(block.getBlockId());
> {code}
> The last number of offset evenly distributed from 0 to 9 - that's why all 
> caches will full approximately the same.
> It is good for performance. Below the attachment, it is load test reading 
> HDFS via HBase where clientShortCircuitNum = 1 vs 3. We can see that 
> performance grows ~30%, CPU usage about +15%. 
> Hope it is interesting for someone.
> Ready to explain some unobvious things.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-14820) The default 8KB buffer of BlockReaderRemote#newBlockReader#BufferedOutputStream is too big

2020-03-03 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-14820:


The current implementation is, DFS client send a request (which is short) to 
DataNode asking for a block using an output stream. After that, client receives 
block data DataNode  (which can be several MBs long) using an input stream.

This patch changes the buffer size of the former, output stream. There is 
absolutely no reason to use a 8kb buffer size for this stream. The input 
stream, yes what [~eyang] says makes sense.

{code}
OpReadBlockProto proto = OpReadBlockProto.newBuilder()
.setHeader(DataTransferProtoUtil.buildClientHeader(blk, clientName,
blockToken))
.setOffset(blockOffset)
.setLen(length)
.setSendChecksums(sendChecksum)
.setCachingStrategy(getCachingStrategy(cachingStrategy))
.build();
{code}

Also note that the stream objects are not recycled. One block is one 
output/input stream object.

>  The default 8KB buffer of 
> BlockReaderRemote#newBlockReader#BufferedOutputStream is too big
> ---
>
> Key: HDFS-14820
> URL: https://issues.apache.org/jira/browse/HDFS-14820
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: Lisheng Sun
>Assignee: Lisheng Sun
>Priority: Major
> Attachments: HDFS-14820.001.patch, HDFS-14820.002.patch, 
> HDFS-14820.003.patch
>
>
> this issue is similar to HDFS-14535.
> {code:java}
> public static BlockReader newBlockReader(String file,
> ExtendedBlock block,
> Token blockToken,
> long startOffset, long len,
> boolean verifyChecksum,
> String clientName,
> Peer peer, DatanodeID datanodeID,
> PeerCache peerCache,
> CachingStrategy cachingStrategy,
> int networkDistance) throws IOException {
>   // in and out will be closed when sock is closed (by the caller)
>   final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
>   peer.getOutputStream()));
>   new Sender(out).readBlock(block, blockToken, clientName, startOffset, len,
>   verifyChecksum, cachingStrategy);
> }
> public BufferedOutputStream(OutputStream out) {
> this(out, 8192);
> }
> {code}
> Sender#readBlock parameter( block,blockToken, clientName, startOffset, len, 
> verifyChecksum, cachingStrategy) could not use such a big buffer.
> So i think it should reduce BufferedOutputStream buffer.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Moved] (HDFS-15202) HDFS-client: boost ShortCircuit Cache

2020-03-03 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang moved HADOOP-16901 to HDFS-15202:
-

   Key: HDFS-15202  (was: HADOOP-16901)
Issue Type: Improvement  (was: New Feature)
   Project: Hadoop HDFS  (was: Hadoop Common)

> HDFS-client: boost ShortCircuit Cache
> -
>
> Key: HDFS-15202
> URL: https://issues.apache.org/jira/browse/HDFS-15202
> Project: Hadoop HDFS
>  Issue Type: Improvement
> Environment: 4 nodes E5-2698 v4 @ 2.20GHz, 700 Gb Mem.
> 8 RegionServers (2 by host)
> 8 tables by 64 regions by 1.88 Gb data in each = 1200 Gb total
> Random read in 800 threads via YCSB and a little bit updates (10% of reads)
>Reporter: Danil Lipovoy
>Assignee: Danil Lipovoy
>Priority: Minor
> Attachments: hdfs_cpu.png, hdfs_reads.png
>
>
> I want to propose how to improve reading performance HDFS-client. The idea: 
> create few instances SchortCircuit caches instead of one. 
> The key points:
> 1. Create array of caches (see *dfs.client.short.circuit.num* in the pull 
> requests below):
> {code:java}
> private ClientContext(String name, DfsClientConf conf, Configuration config) {
> ...
> shortCircuitCache = new ShortCircuitCache[this.clientShortCircuitNum];
> for (int i = 0; i < this.clientShortCircuitNum; i++) {
>   this.shortCircuitCache[i] = ShortCircuitCache.fromConf(scConf);
> }
> {code}
> 2 Then divide blocks by caches:
> {code:java}
>   public ShortCircuitCache getShortCircuitCache(long idx) {
> return shortCircuitCache[(int) (idx % clientShortCircuitNum)];
>   }
> {code}
> 3. And how to call it:
> {code:java}
> ShortCircuitCache cache = 
> clientContext.getShortCircuitCache(block.getBlockId());
> {code}
> The last number of offset evenly distributed from 0 to 9 - that's why all 
> caches will full approximately the same.
> It is good for performance. Below the attachment, it is load test reading 
> HDFS via HBase where clientShortCircuitNum = 1 vs 3. We can see that 
> performance grows ~30%, CPU usage about +15%. 
> Hope it is interesting for someone.
> Ready to explain some unobvious things.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-15111) stopStandbyServices() should log which service state it is transitioning from.

2020-03-02 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-15111:


I'm sorry i pushed it into branch-2.10 and broke the build. This is now 
reverted.

> stopStandbyServices() should log which service state it is transitioning from.
> --
>
> Key: HDFS-15111
> URL: https://issues.apache.org/jira/browse/HDFS-15111
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: hdfs, logging
>Affects Versions: 2.10.0
>Reporter: Konstantin Shvachko
>Assignee: Xieming Li
>Priority: Major
>  Labels: newbie++
> Fix For: 3.3.0, 3.1.4, 3.2.2
>
> Attachments: HDFS-15111.001.patch, HDFS-15111.002.patch, 
> HDFS-15111.003.patch
>
>
> Trying to transition Observer to Standby state. {{stopStandbyServices()}} 
> logs that it is "Stopping services started for standby state". It should be 
> "Stopping services started for observer state"



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-15200) Delete Corrupt Replica Immediately Irrespective of Replicas On Stale Storage

2020-03-02 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-15200:


Interesting case ... thanks for digging into this.

bq. The standby Namenode will mark all the storages as stale.
can you explain why sbnn marks all storages stale? is it because sbnn took too 
much time to transition to active and therefore all block reports were lost?


> Delete Corrupt Replica Immediately Irrespective of Replicas On Stale Storage 
> -
>
> Key: HDFS-15200
> URL: https://issues.apache.org/jira/browse/HDFS-15200
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: Ayush Saxena
>Assignee: Ayush Saxena
>Priority: Critical
>
> Presently {{invalidateBlock(..)}} before adding a replica into invalidates, 
> checks whether any  block replica is on stale storage, if any replica is on 
> stale storage, it postpones deletion of the replica.
> Here :
> {code:java}
>// Check how many copies we have of the block
> if (nr.replicasOnStaleNodes() > 0) {
>   blockLog.debug("BLOCK* invalidateBlocks: postponing " +
>   "invalidation of {} on {} because {} replica(s) are located on " +
>   "nodes with potentially out-of-date block reports", b, dn,
>   nr.replicasOnStaleNodes());
>   postponeBlock(b.getCorrupted());
>   return false;
> {code}
>  
> In case of corrupt replica, we can skip this logic and delete the corrupt 
> replica immediately, as a corrupt replica can't get corrected.
> One outcome of this behavior presently is namenodes showing different block 
> states post failover, as:
> If a replica is marked corrupt, the Active NN, will mark it as corrupt, and 
> mark it for deletion and remove it from corruptReplica's and  
> excessRedundancyMap.
> If before the deletion of replica, Failover happens.
> The standby Namenode will mark all the storages as stale.
> Then will start processing IBR's, Now since the replica's would be on stale 
> storage, it will skip deletion, and removal from corruptReplica's
> Hence both the namenode will show different numbers and different corrupt 
> replicas.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Updated] (HDFS-15113) Missing IBR when NameNode restart if open processCommand async feature

2020-02-27 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-15113:
---
Target Version/s: 3.3.0

> Missing IBR when NameNode restart if open processCommand async feature
> --
>
> Key: HDFS-15113
> URL: https://issues.apache.org/jira/browse/HDFS-15113
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode
>Reporter: Xiaoqiao He
>Assignee: Xiaoqiao He
>Priority: Blocker
> Attachments: HDFS-15113.001.patch, HDFS-15113.002.patch, 
> HDFS-15113.003.patch
>
>
> Recently, I meet one case that NameNode missing block after restart which is 
> related with HDFS-14997.
> a. during NameNode restart, it will return command `DNA_REGISTER` to DataNode 
> when receive some RPC request from DataNode.
> b. when DataNode receive `DNA_REGISTER` command, it will run #reRegister 
> async.
> {code:java}
>   void reRegister() throws IOException {
> if (shouldRun()) {
>   // re-retrieve namespace info to make sure that, if the NN
>   // was restarted, we still match its version (HDFS-2120)
>   NamespaceInfo nsInfo = retrieveNamespaceInfo();
>   // and re-register
>   register(nsInfo);
>   scheduler.scheduleHeartbeat();
>   // HDFS-9917,Standby NN IBR can be very huge if standby namenode is down
>   // for sometime.
>   if (state == HAServiceState.STANDBY || state == 
> HAServiceState.OBSERVER) {
> ibrManager.clearIBRs();
>   }
> }
>   }
> {code}
> c. As we know, #register will trigger BR immediately.
> d. because #reRegister run async, so we could not make sure which one run 
> first between send FBR and clear IBR. If clean IBR run first, it will be OK. 
> But if send FBR first then clear IBR, it will missing some blocks received 
> between these two time point until next FBR.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Updated] (HDFS-15113) Missing IBR when NameNode restart if open processCommand async feature

2020-02-27 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-15113:
---
Priority: Blocker  (was: Major)

> Missing IBR when NameNode restart if open processCommand async feature
> --
>
> Key: HDFS-15113
> URL: https://issues.apache.org/jira/browse/HDFS-15113
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode
>Reporter: Xiaoqiao He
>Assignee: Xiaoqiao He
>Priority: Blocker
> Attachments: HDFS-15113.001.patch, HDFS-15113.002.patch, 
> HDFS-15113.003.patch
>
>
> Recently, I meet one case that NameNode missing block after restart which is 
> related with HDFS-14997.
> a. during NameNode restart, it will return command `DNA_REGISTER` to DataNode 
> when receive some RPC request from DataNode.
> b. when DataNode receive `DNA_REGISTER` command, it will run #reRegister 
> async.
> {code:java}
>   void reRegister() throws IOException {
> if (shouldRun()) {
>   // re-retrieve namespace info to make sure that, if the NN
>   // was restarted, we still match its version (HDFS-2120)
>   NamespaceInfo nsInfo = retrieveNamespaceInfo();
>   // and re-register
>   register(nsInfo);
>   scheduler.scheduleHeartbeat();
>   // HDFS-9917,Standby NN IBR can be very huge if standby namenode is down
>   // for sometime.
>   if (state == HAServiceState.STANDBY || state == 
> HAServiceState.OBSERVER) {
> ibrManager.clearIBRs();
>   }
> }
>   }
> {code}
> c. As we know, #register will trigger BR immediately.
> d. because #reRegister run async, so we could not make sure which one run 
> first between send FBR and clear IBR. If clean IBR run first, it will be OK. 
> But if send FBR first then clear IBR, it will missing some blocks received 
> between these two time point until next FBR.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-15113) Missing IBR when NameNode restart if open processCommand async feature

2020-02-27 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-15113:


This has to be a blocker for 3.3.0. Updated jira to reflect the reality.

> Missing IBR when NameNode restart if open processCommand async feature
> --
>
> Key: HDFS-15113
> URL: https://issues.apache.org/jira/browse/HDFS-15113
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode
>Reporter: Xiaoqiao He
>Assignee: Xiaoqiao He
>Priority: Blocker
> Attachments: HDFS-15113.001.patch, HDFS-15113.002.patch, 
> HDFS-15113.003.patch
>
>
> Recently, I meet one case that NameNode missing block after restart which is 
> related with HDFS-14997.
> a. during NameNode restart, it will return command `DNA_REGISTER` to DataNode 
> when receive some RPC request from DataNode.
> b. when DataNode receive `DNA_REGISTER` command, it will run #reRegister 
> async.
> {code:java}
>   void reRegister() throws IOException {
> if (shouldRun()) {
>   // re-retrieve namespace info to make sure that, if the NN
>   // was restarted, we still match its version (HDFS-2120)
>   NamespaceInfo nsInfo = retrieveNamespaceInfo();
>   // and re-register
>   register(nsInfo);
>   scheduler.scheduleHeartbeat();
>   // HDFS-9917,Standby NN IBR can be very huge if standby namenode is down
>   // for sometime.
>   if (state == HAServiceState.STANDBY || state == 
> HAServiceState.OBSERVER) {
> ibrManager.clearIBRs();
>   }
> }
>   }
> {code}
> c. As we know, #register will trigger BR immediately.
> d. because #reRegister run async, so we could not make sure which one run 
> first between send FBR and clear IBR. If clean IBR run first, it will be OK. 
> But if send FBR first then clear IBR, it will missing some blocks received 
> between these two time point until next FBR.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Updated] (HDFS-13660) DistCp job fails when new data is appended in the file while the distCp copy job is running

2020-02-27 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-13660:
---
Fix Version/s: 3.2.2
   3.1.4

> DistCp job fails when new data is appended in the file while the distCp copy 
> job is running
> ---
>
> Key: HDFS-13660
> URL: https://issues.apache.org/jira/browse/HDFS-13660
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: distcp
>Reporter: Mukund Thakur
>Assignee: Mukund Thakur
>Priority: Critical
> Fix For: 3.3.0, 3.1.4, 3.2.2
>
> Attachments: distcp_failure_when_file_append.log
>
>
> Steps to reproduce: 
> Suppose distcp MR job is copying the file /tmp/web_returns_merged/data-m-002 
> and 
> we append some more data to this file using command 
> hadoop fs -appendToFile xaa  /tmp/web_returns_merged/data-m-002
> the job fails with exception 
>  Mismatch in length of 
> source:hdfs://mycluster0/tmp/web_returns_merged/data-m-002 and target.
> Attached the logs.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Updated] (HDFS-12999) When reach the end of the block group, it may not need to flush all the data packets(flushAllInternals) twice.

2020-02-27 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-12999:
---
Fix Version/s: 3.2.2
   3.1.4

> When reach the end of the block group, it may not need to flush all the data 
> packets(flushAllInternals) twice. 
> ---
>
> Key: HDFS-12999
> URL: https://issues.apache.org/jira/browse/HDFS-12999
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: erasure-coding, hdfs-client
>Affects Versions: 3.0.0-beta1, 3.1.0
>Reporter: lufei
>Assignee: lufei
>Priority: Major
> Fix For: 3.3.0, 3.1.4, 3.2.2
>
> Attachments: HDFS-12999.001.patch, HDFS-12999.002.patch, 
> HDFS-12999.003.patch
>
>
> In order to make the process simplification. It's no need to flush all the 
> data packets(flushAllInternals) twice,when reach the end of the block group.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Updated] (HDFS-15068) DataNode could meet deadlock if invoke refreshVolumes when register

2020-02-27 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-15068:
---
Fix Version/s: 3.2.2
   3.1.4

> DataNode could meet deadlock if invoke refreshVolumes when register
> ---
>
> Key: HDFS-15068
> URL: https://issues.apache.org/jira/browse/HDFS-15068
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode
>Reporter: Xiaoqiao He
>Assignee: Aiphago
>Priority: Major
> Fix For: 3.3.0, 3.1.4, 3.2.2
>
> Attachments: HDFS-15068.001.patch, HDFS-15068.002.patch, 
> HDFS-15068.003.patch, HDFS-15068.004.patch, HDFS-15068.005.patch
>
>
> DataNode could meet deadlock when invoke `dfsadmin -reconfig datanode ip:host 
> start` to trigger #refreshVolumes.
> 1. DataNod#refreshVolumes hold datanode instance ownable {{synchronizer}} 
> when enter this method first, then try to hold BPOfferService {{readlock}} 
> when `bpos.getNamespaceInfo()` in following code segment. 
> {code:java}
> for (BPOfferService bpos : blockPoolManager.getAllNamenodeThreads()) {
>   nsInfos.add(bpos.getNamespaceInfo());
> }
> {code}
> 2. BPOfferService#registrationSucceeded (which is invoked by #register when 
> DataNode start or #reregister when processCommandFromActor) hold 
> BPOfferService {{writelock}} first, then try to hold datanode instance 
> ownable {{synchronizer}} in following method.
> {code:java}
>   synchronized void bpRegistrationSucceeded(DatanodeRegistration 
> bpRegistration,
>   String blockPoolId) throws IOException {
> id = bpRegistration;
> if(!storage.getDatanodeUuid().equals(bpRegistration.getDatanodeUuid())) {
>   throw new IOException("Inconsistent Datanode IDs. Name-node returned "
>   + bpRegistration.getDatanodeUuid()
>   + ". Expecting " + storage.getDatanodeUuid());
> }
> 
> registerBlockPoolWithSecretManager(bpRegistration, blockPoolId);
>   }
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Updated] (HDFS-15111) stopStandbyServices() should log which service state it is transitioning from.

2020-02-27 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-15111:
---
Resolution: Fixed
Status: Resolved  (was: Patch Available)

Cherrypicked to branch-3.1

> stopStandbyServices() should log which service state it is transitioning from.
> --
>
> Key: HDFS-15111
> URL: https://issues.apache.org/jira/browse/HDFS-15111
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: hdfs, logging
>Affects Versions: 2.10.0
>Reporter: Konstantin Shvachko
>Assignee: Xieming Li
>Priority: Major
>  Labels: newbie++
> Fix For: 3.3.0, 3.1.4, 3.2.2
>
> Attachments: HDFS-15111.001.patch, HDFS-15111.002.patch, 
> HDFS-15111.003.patch
>
>
> Trying to transition Observer to Standby state. {{stopStandbyServices()}} 
> logs that it is "Stopping services started for standby state". It should be 
> "Stopping services started for observer state"



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Updated] (HDFS-15111) stopStandbyServices() should log which service state it is transitioning from.

2020-02-27 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-15111:
---
Fix Version/s: 3.1.4

> stopStandbyServices() should log which service state it is transitioning from.
> --
>
> Key: HDFS-15111
> URL: https://issues.apache.org/jira/browse/HDFS-15111
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: hdfs, logging
>Affects Versions: 2.10.0
>Reporter: Konstantin Shvachko
>Assignee: Xieming Li
>Priority: Major
>  Labels: newbie++
> Fix For: 3.3.0, 3.1.4, 3.2.2
>
> Attachments: HDFS-15111.001.patch, HDFS-15111.002.patch, 
> HDFS-15111.003.patch
>
>
> Trying to transition Observer to Standby state. {{stopStandbyServices()}} 
> logs that it is "Stopping services started for standby state". It should be 
> "Stopping services started for observer state"



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Resolved] (HDFS-14668) Support Fuse with Users from multiple Security Realms

2020-02-27 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang resolved HDFS-14668.

Fix Version/s: 3.2.2
   3.1.4
   3.3.0
   Resolution: Fixed

Thanks [~pifta]!

> Support Fuse with Users from multiple Security Realms
> -
>
> Key: HDFS-14668
> URL: https://issues.apache.org/jira/browse/HDFS-14668
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: fuse-dfs
>Affects Versions: 3.1.0, 3.0.3
>Reporter: Sailesh Patel
>Assignee: Istvan Fajth
>Priority: Critical
>  Labels: regression
> Fix For: 3.3.0, 3.1.4, 3.2.2
>
>
> UPDATE:
> See 
> [this|https://issues.apache.org/jira/browse/HDFS-14668?focusedCommentId=16979466=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-16979466]
>  comment for the complete description of what is happening here.
> Users from non-default  krb5 domain can't use hadoop-fuse.
> There are 2 Realms with kdc. 
> -one realm is for human users  (USERS.COM.US) 
> -the other is for service principals.   (SERVICE.COM.US) 
> Cross realm trust is setup.
> In krb5.conf  the default domain  is set to SERVICE.COM.US
> Users within USERS.COM.US Realm are not able to put any files to Fuse mounted 
> location
> The client shows:
>   cp: cannot create regular file ‘/hdfs_mount/tmp/hello_from_fuse.txt’: 
> Input/output error



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-15186) Erasure Coding: Decommission may generate the parity block's content with all 0 in some case

2020-02-26 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-15186:


Thank you. The patch makes much more sense now.
Now, I start to wonder if EC works well with maintenance mode ... but that's a 
separate topic.

> Erasure Coding: Decommission may generate the parity block's content with all 
> 0 in some case
> 
>
> Key: HDFS-15186
> URL: https://issues.apache.org/jira/browse/HDFS-15186
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode, erasure-coding
>Affects Versions: 3.0.3, 3.2.1, 3.1.3
>Reporter: Yao Guangdong
>Assignee: Yao Guangdong
>Priority: Critical
> Attachments: HDFS-15186.001.patch, HDFS-15186.002.patch, 
> HDFS-15186.003.patch, HDFS-15186.004.patch
>
>
> I can find some parity block's content with all 0 when i decommission some 
> DataNode(more than 1) from a cluster. And the probability is very big(parts 
> per thousand).This is a big problem.You can think that if we read data from 
> the zero parity block or use the zero parity block to recover a block which 
> can make us use the error data even we don't know it.
> There is some case in the below:
> B: Busy DataNode, 
> D:Decommissioning DataNode,
> Others is normal.
> 1.Group indices is [0, 1, 2, 3, 4, 5, 6(B,D), 7, 8(D)].
> 2.Group indices is [0(B,D), 1, 2, 3, 4, 5, 6(B,D), 7, 8(D)].
> 
> In the first case when the block group indices is [0, 1, 2, 3, 4, 5, 6(B,D), 
> 7, 8(D)], the DN may received reconstruct block command and the 
> liveIndices=[0, 1, 2, 3, 4, 5, 7, 8] and the targets's(the field which  in 
> the class StripedReconstructionInfo) length is 2. 
> The targets's length is 2 which mean that the DataNode need recover 2 
> internal block in current code.But from the liveIndices we only can find 1 
> missing block, so the method StripedWriter#initTargetIndices will use 0 as 
> the default recover block and don't care the indices 0 is in the sources 
> indices or not.
> When they use sources indices [0, 1, 2, 3, 4, 5] to recover indices [6, 0] 
> use the ec algorithm.We can find that the indices [0] is in the both the 
> sources indices and the targets indices in this case. The returned target 
> buffer in the indices [6] is always 0 from the ec  algorithm.So I think this 
> is the ec algorithm's problem. Because it should more fault tolerance.I try 
> to fixed it .But it is too hard. Because the case is too more. The second is 
> another case in the example above(use sources indices [1, 2, 3, 4, 5, 7] to 
> recover indices [0, 6, 0]). So I changed my mind.Invoke the ec  algorithm 
> with a correct parameters. Which mean that remove the duplicate target 
> indices 0 in this case.Finally, I fixed it in this way.
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Updated] (HDFS-14861) Reset LowRedundancyBlocks Iterator periodically

2020-02-25 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-14861:
---
Fix Version/s: 3.2.2
   3.1.4
   3.3.0
   Resolution: Fixed
   Status: Resolved  (was: Patch Available)

Thanks [~sodonnell]
pushed to trunk branch-3.2 and branch-3.1

> Reset LowRedundancyBlocks Iterator periodically
> ---
>
> Key: HDFS-14861
> URL: https://issues.apache.org/jira/browse/HDFS-14861
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: namenode
>Affects Versions: 3.3.0
>Reporter: Stephen O'Donnell
>Assignee: Stephen O'Donnell
>Priority: Major
>  Labels: decommission
> Fix For: 3.3.0, 3.1.4, 3.2.2
>
> Attachments: HDFS-14861.001.patch, HDFS-14861.002.patch
>
>
> When the namenode needs to schedule blocks for reconstruction, the blocks are 
> placed into the neededReconstruction object in the BlockManager. This is an 
> instance of LowRedundancyBlocks, which maintains a list of priority queues 
> where the blocks are held until they are scheduled for reconstruction / 
> replication.
> Every 3 seconds, by default, a number of blocks are retrieved from 
> LowRedundancyBlocks. The method 
> LowRedundancyBlocks.chooseLowRedundancyBlocks() is used to retrieve the next 
> set of blocks using a bookmarked iterator. Each call to this method moves the 
> iterator forward. The number of blocks retrieved is governed by the formula:
> number_of_live_nodes * dfs.namenode.replication.work.multiplier.per.iteration 
> (default 2)
> Then the namenode attempts to schedule those blocks on datanodes, but each 
> datanode has a limit of how many blocks can be queued against it (controlled 
> by dfs.namenode.replication.max-streams) so all of the retrieved blocks may 
> not be scheduled. There may be other block availability reasons the blocks 
> are not scheduled too.
> As the iterator in chooseLowRedundancyBlocks() always moves forward, the 
> blocks which were not scheduled are not retried until the end of the queue is 
> reached and the iterator is reset.
> If the replication queue is very large (eg several nodes are being 
> decommissioned) or if blocks are being continuously added to the replication 
> queue (eg nodes decommission using the proposal in HDFS-14854) it may take a 
> very long time for the iterator to be reset to the start.
> The result of this, could be a few blocks for a decommissioning or entering 
> maintenance mode node getting left behind and it taking many hours or even 
> days for them to be retried, and this could stop decommission completing.
> With this Jira, I would like to suggest we reset the iterator after a 
> configurable number of calls to chooseLowRedundancyBlocks() so any left 
> behind blocks are retried.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-15039) Cache meta file length of FinalizedReplica to reduce call File.length()

2020-02-25 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-15039:


I'm sorry I didn't catch this. Why do we make metaLength an int rather than 
long? Is it because you want to save memory size? Thanks

> Cache meta file length of FinalizedReplica to reduce call File.length()
> ---
>
> Key: HDFS-15039
> URL: https://issues.apache.org/jira/browse/HDFS-15039
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: datanode
>Reporter: Yang Yun
>Assignee: Yang Yun
>Priority: Minor
> Attachments: HDFS-15039.patch, HDFS-15039.patch, HDFS-15039.patch, 
> HDFS-15039.patch, HDFS-15039.patch
>
>
> When use ReplicaCachingGetSpaceUsed to get the volume space used.  It will 
> call File.length() for every meta file of replica. That add more disk IO, we 
> found the slow log as below. For finalized replica, the size of meta file is 
> not changed, i think we can cache the value.
> {code:java}
> org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.ReplicaCachingGetSpaceUsed:
>  Refresh dfs used, bpid: BP-898717543-10.75.1.240-1519386995727 replicas 
> size: 1166 dfsUsed: 72227113183 on volume: 
> DS-3add8d62-d69a-4f5a-a29f-b7bbb400af2e duration: 17206ms{code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-15039) Cache meta file length of FinalizedReplica to reduce call File.length()

2020-02-24 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-15039:


Patch has a conflict so submit a new patch and let it run through the precommit.

> Cache meta file length of FinalizedReplica to reduce call File.length()
> ---
>
> Key: HDFS-15039
> URL: https://issues.apache.org/jira/browse/HDFS-15039
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: datanode
>Reporter: Yang Yun
>Assignee: Yang Yun
>Priority: Minor
> Attachments: HDFS-15039.patch
>
>
> When use ReplicaCachingGetSpaceUsed to get the volume space used.  It will 
> call File.length() for every meta file of replica. That add more disk IO, we 
> found the slow log as below. For finalized replica, the size of meta file is 
> not changed, i think we can cache the value.
> {code:java}
> org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.ReplicaCachingGetSpaceUsed:
>  Refresh dfs used, bpid: BP-898717543-10.75.1.240-1519386995727 replicas 
> size: 1166 dfsUsed: 72227113183 on volume: 
> DS-3add8d62-d69a-4f5a-a29f-b7bbb400af2e duration: 17206ms{code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Updated] (HDFS-15174) Optimize ReplicaCachingGetSpaceUsed by reducing unnecessary io operations

2020-02-24 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-15174:
---
Resolution: Fixed
Status: Resolved  (was: Patch Available)

Patch applies cleanly in trunk branch-3.2 and branch-3.1.
Branch-2.10 will require an update.

> Optimize ReplicaCachingGetSpaceUsed by reducing unnecessary io operations
> -
>
> Key: HDFS-15174
> URL: https://issues.apache.org/jira/browse/HDFS-15174
> Project: Hadoop HDFS
>  Issue Type: Improvement
>Reporter: Lisheng Sun
>Assignee: Lisheng Sun
>Priority: Major
> Fix For: 3.3.0, 3.1.4, 3.2.2
>
> Attachments: HDFS-15174-001.patch
>
>
> Calculating the size of each block and the size of the meta file requires io 
> operation In ReplicaCachingGetSpaceUsed#refresh(). Pressure on disk 
> performance when there are many block. HDFS-14313 is intended to reduce io 
> operation. So get block size by ReplicaInfo and meta size by 
> DataChecksum#getChecksumSize().
> {code:java}
> @Override
>   protected void refresh() {
>   if (CollectionUtils.isNotEmpty(replicaInfos)) {
> for (ReplicaInfo replicaInfo : replicaInfos) {
>   if (Objects.equals(replicaInfo.getVolume().getStorageID(),
>   volume.getStorageID())) {
> dfsUsed += replicaInfo.getBlockDataLength();
> dfsUsed += replicaInfo.getMetadataLength();
> count++;
>   }
> }
>   }
>   }
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Updated] (HDFS-15174) Optimize ReplicaCachingGetSpaceUsed by reducing unnecessary io operations

2020-02-24 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-15174:
---
Fix Version/s: 3.2.2
   3.1.4
   3.3.0

> Optimize ReplicaCachingGetSpaceUsed by reducing unnecessary io operations
> -
>
> Key: HDFS-15174
> URL: https://issues.apache.org/jira/browse/HDFS-15174
> Project: Hadoop HDFS
>  Issue Type: Improvement
>Reporter: Lisheng Sun
>Assignee: Lisheng Sun
>Priority: Major
> Fix For: 3.3.0, 3.1.4, 3.2.2
>
> Attachments: HDFS-15174-001.patch
>
>
> Calculating the size of each block and the size of the meta file requires io 
> operation In ReplicaCachingGetSpaceUsed#refresh(). Pressure on disk 
> performance when there are many block. HDFS-14313 is intended to reduce io 
> operation. So get block size by ReplicaInfo and meta size by 
> DataChecksum#getChecksumSize().
> {code:java}
> @Override
>   protected void refresh() {
>   if (CollectionUtils.isNotEmpty(replicaInfos)) {
> for (ReplicaInfo replicaInfo : replicaInfos) {
>   if (Objects.equals(replicaInfo.getVolume().getStorageID(),
>   volume.getStorageID())) {
> dfsUsed += replicaInfo.getBlockDataLength();
> dfsUsed += replicaInfo.getMetadataLength();
> count++;
>   }
> }
>   }
>   }
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-15174) Optimize ReplicaCachingGetSpaceUsed by reducing unnecessary io operations

2020-02-21 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-15174:


+1 looks like this is useful along with HDFS-15039

> Optimize ReplicaCachingGetSpaceUsed by reducing unnecessary io operations
> -
>
> Key: HDFS-15174
> URL: https://issues.apache.org/jira/browse/HDFS-15174
> Project: Hadoop HDFS
>  Issue Type: Improvement
>Reporter: Lisheng Sun
>Assignee: Lisheng Sun
>Priority: Major
> Attachments: HDFS-15174-001.patch
>
>
> Calculating the size of each block and the size of the meta file requires io 
> operation In ReplicaCachingGetSpaceUsed#refresh(). Pressure on disk 
> performance when there are many block. HDFS-14313 is intended to reduce io 
> operation. So get block size by ReplicaInfo and meta size by 
> DataChecksum#getChecksumSize().
> {code:java}
> @Override
>   protected void refresh() {
>   if (CollectionUtils.isNotEmpty(replicaInfos)) {
> for (ReplicaInfo replicaInfo : replicaInfos) {
>   if (Objects.equals(replicaInfo.getVolume().getStorageID(),
>   volume.getStorageID())) {
> dfsUsed += replicaInfo.getBlockDataLength();
> dfsUsed += replicaInfo.getMetadataLength();
> count++;
>   }
> }
>   }
>   }
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-15039) Cache meta file length of FinalizedReplica to reduce call File.length()

2020-02-20 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-15039:


Looks good to me. [~leosun08] fyi

> Cache meta file length of FinalizedReplica to reduce call File.length()
> ---
>
> Key: HDFS-15039
> URL: https://issues.apache.org/jira/browse/HDFS-15039
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: datanode
>Reporter: Yang Yun
>Assignee: Yang Yun
>Priority: Minor
> Attachments: HDFS-15039.patch
>
>
> When use ReplicaCachingGetSpaceUsed to get the volume space used.  It will 
> call File.length() for every meta file of replica. That add more disk IO, we 
> found the slow log as below. For finalized replica, the size of meta file is 
> not changed, i think we can cache the value.
> {code:java}
> org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.ReplicaCachingGetSpaceUsed:
>  Refresh dfs used, bpid: BP-898717543-10.75.1.240-1519386995727 replicas 
> size: 1166 dfsUsed: 72227113183 on volume: 
> DS-3add8d62-d69a-4f5a-a29f-b7bbb400af2e duration: 17206ms{code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-15171) Add a thread to call saveDfsUsed periodically, to prevent datanode too long restart time.

2020-02-20 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-15171:


The first thing coming to my mind is increasing 
dfs.datanode.cached-dfsused.check.interval.ms like Stephen said.
Otherwise, HDFS-14313 may be useful too.

> Add a thread to call saveDfsUsed periodically, to prevent datanode too long 
> restart time.  
> ---
>
> Key: HDFS-15171
> URL: https://issues.apache.org/jira/browse/HDFS-15171
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: datanode
>Affects Versions: 3.2.0
>Reporter: zhuqi
>Assignee: zhuqi
>Priority: Major
>
> There are 30 storage dirs per datanode in our production cluster , it will 
> take too many time to restart, because sometimes the datanode didn't shutdown 
> gracefully. Now only the datanode graceful shut down hook and the 
> blockpoolslice shutdown will cause the saveDfsUsed function, that cause the 
> restart of datanode can't reuse the dfsuse cache sometimes. I think if we can 
> add a thread to periodically call the saveDfsUsed function.
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-15176) Enable GcTimePercentage Metric in NameNode's JvmMetrics.

2020-02-20 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-15176:


Thanks i didn't even realize there's GcTimeMonitor in our code. Shouldn't we 
enable it by default? it doesn't look like a heavy weight operation...

> Enable GcTimePercentage Metric in NameNode's JvmMetrics.
> 
>
> Key: HDFS-15176
> URL: https://issues.apache.org/jira/browse/HDFS-15176
> Project: Hadoop HDFS
>  Issue Type: Improvement
>Reporter: Jinglun
>Assignee: Jinglun
>Priority: Minor
> Attachments: HDFS-15176.001.patch, HDFS-15176.002.patch, 
> HDFS-15176.003.patch, HDFS-15176.004.patch
>
>
> The GcTimePercentage(computed by GcTimeMonitor) could be used as a dimension 
> to analyze the NameNode GC.  We should add a switch config to enable the 
> GcTimePercentage metric in HDFS.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-15186) Erasure Coding: Decommission may generate the parity block's content with all 0 in some case

2020-02-20 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-15186:


[~ferhui] would you be interested in reviewing this patch? You've fixed a 
number of critical EC decomm bugs in the past. Thanks.

> Erasure Coding: Decommission may generate the parity block's content with all 
> 0 in some case
> 
>
> Key: HDFS-15186
> URL: https://issues.apache.org/jira/browse/HDFS-15186
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode, erasure-coding
>Affects Versions: 3.0.3, 3.2.1, 3.1.3
>Reporter: Yao Guangdong
>Assignee: Yao Guangdong
>Priority: Critical
> Attachments: HDFS-15186.001.patch
>
>
> I can find some parity block's content with all 0 when i decommission some 
> DataNode(more than 1) from a cluster. And the probability is very big(parts 
> per thousand).This is a big problem.You can think that if we read data from 
> the zero parity block or use the zero parity block to recover a block which 
> can make us use the error data even we don't know it.
> There is some case in the below:
> B: Busy DataNode, 
> D:Decommissioning DataNode,
> Others is normal.
> 1.Group indices is [0, 1, 2, 3, 4, 5, 6(B,D), 7, 8(D)].
> 2.Group indices is [0(B,D), 1, 2, 3, 4, 5, 6(B,D), 7, 8(D)].
> 
> In the first case when the block group indices is [0, 1, 2, 3, 4, 5, 6(B,D), 
> 7, 8(D)], the DN may received reconstruct block command and the 
> liveIndices=[0, 1, 2, 3, 4, 5, 7, 8] and the targets's(the field which  in 
> the class StripedReconstructionInfo) length is 2. 
> The targets's length is 2 which mean that the DataNode need recover 2 
> internal block in current code.But from the liveIndices we only can find 1 
> missing block, so the method StripedWriter#initTargetIndices will use 0 as 
> the default recover block and don't care the indices 0 is in the sources 
> indices or not.
> When they use sources indices [0, 1, 2, 3, 4, 5] to recover indices [6, 0] 
> use the ec algorithm.We can find that the indices [0] is in the both the 
> sources indices and the targets indices in this case. The returned target 
> buffer in the indices [6] is always 0 from the ec  algorithm.So I think this 
> is the ec algorithm's problem. Because it should more fault tolerance.I try 
> to fixed it .But it is too hard. Because the case is too more. The second is 
> another case in the example above(use sources indices [1, 2, 3, 4, 5, 7] to 
> recover indices [0, 6, 0]). So I changed my mind.Invoke the ec  algorithm 
> with a correct parameters. Which mean that remove the duplicate target 
> indices 0 in this case.Finally, I fixed it in this way.
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Commented] (HDFS-15186) Erasure Coding: Decommission may generate the parity block's content with all 0 in some case

2020-02-20 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-15186:


[~avijayan] could you take a look at the patch? Or [~ayushtkn] please

> Erasure Coding: Decommission may generate the parity block's content with all 
> 0 in some case
> 
>
> Key: HDFS-15186
> URL: https://issues.apache.org/jira/browse/HDFS-15186
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode, erasure-coding
>Affects Versions: 3.0.3, 3.2.1, 3.1.3
>Reporter: Yao Guangdong
>Assignee: Yao Guangdong
>Priority: Critical
> Attachments: HDFS-15186.001.patch
>
>
> I can find some parity block's content with all 0 when i decommission some 
> DataNode(more than 1) from a cluster. And the probability is very big(parts 
> per thousand).This is a big problem.You can think that if we read data from 
> the zero parity block or use the zero parity block to recover a block which 
> can make us use the error data even we don't know it.
> There is some case in the below:
> B: Busy DataNode, 
> D:Decommissioning DataNode,
> Others is normal.
> 1.Group indices is [0, 1, 2, 3, 4, 5, 6(B,D), 7, 8(D)].
> 2.Group indices is [0(B,D), 1, 2, 3, 4, 5, 6(B,D), 7, 8(D)].
> 
> In the first case when the block group indices is [0, 1, 2, 3, 4, 5, 6(B,D), 
> 7, 8(D)], the DN may received reconstruct block command and the 
> liveIndices=[0, 1, 2, 3, 4, 5, 7, 8] and the targets's(the field which  in 
> the class StripedReconstructionInfo) length is 2. 
> The targets's length is 2 which mean that the DataNode need recover 2 
> internal block in current code.But from the liveIndices we only can find 1 
> missing block, so the method StripedWriter#initTargetIndices will use 0 as 
> the default recover block and don't care the indices 0 is in the sources 
> indices or not.
> When they use sources indices [0, 1, 2, 3, 4, 5] to recover indices [6, 0] 
> use the ec algorithm.We can find that the indices [0] is in the both the 
> sources indices and the targets indices in this case. The returned target 
> buffer in the indices [6] is always 0 from the ec  algorithm.So I think this 
> is the ec algorithm's problem. Because it should more fault tolerance.I try 
> to fixed it .But it is too hard. Because the case is too more. The second is 
> another case in the example above(use sources indices [1, 2, 3, 4, 5, 7] to 
> recover indices [0, 6, 0]). So I changed my mind.Invoke the ec  algorithm 
> with a correct parameters. Which mean that remove the duplicate target 
> indices 0 in this case.Finally, I fixed it in this way.
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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



[jira] [Updated] (HDFS-15186) Erasure Coding: Decommission may generate the parity block's content with all 0 in some case

2020-02-20 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang updated HDFS-15186:
---
Target Version/s: 3.3.0, 3.1.4, 3.2.2  (was: 3.3.0)

> Erasure Coding: Decommission may generate the parity block's content with all 
> 0 in some case
> 
>
> Key: HDFS-15186
> URL: https://issues.apache.org/jira/browse/HDFS-15186
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode, erasure-coding
>Affects Versions: 3.0.3, 3.2.1, 3.1.3
>Reporter: Yao Guangdong
>Assignee: Yao Guangdong
>Priority: Critical
> Attachments: HDFS-15186.001.patch
>
>
> I can find some parity block's content with all 0 when i decommission some 
> DataNode(more than 1) from a cluster. And the probability is very big(parts 
> per thousand).This is a big problem.You can think that if we read data from 
> the zero parity block or use the zero parity block to recover a block which 
> can make us use the error data even we don't know it.
> There is some case in the below:
> B: Busy DataNode, 
> D:Decommissioning DataNode,
> Others is normal.
> 1.Group indices is [0, 1, 2, 3, 4, 5, 6(B,D), 7, 8(D)].
> 2.Group indices is [0(B,D), 1, 2, 3, 4, 5, 6(B,D), 7, 8(D)].
> 
> In the first case when the block group indices is [0, 1, 2, 3, 4, 5, 6(B,D), 
> 7, 8(D)], the DN may received reconstruct block command and the 
> liveIndices=[0, 1, 2, 3, 4, 5, 7, 8] and the targets's(the field which  in 
> the class StripedReconstructionInfo) length is 2. 
> The targets's length is 2 which mean that the DataNode need recover 2 
> internal block in current code.But from the liveIndices we only can find 1 
> missing block, so the method StripedWriter#initTargetIndices will use 0 as 
> the default recover block and don't care the indices 0 is in the sources 
> indices or not.
> When they use sources indices [0, 1, 2, 3, 4, 5] to recover indices [6, 0] 
> use the ec algorithm.We can find that the indices [0] is in the both the 
> sources indices and the targets indices in this case. The returned target 
> buffer in the indices [6] is always 0 from the ec  algorithm.So I think this 
> is the ec algorithm's problem. Because it should more fault tolerance.I try 
> to fixed it .But it is too hard. Because the case is too more. The second is 
> another case in the example above(use sources indices [1, 2, 3, 4, 5, 7] to 
> recover indices [0, 6, 0]). So I changed my mind.Invoke the ec  algorithm 
> with a correct parameters. Which mean that remove the duplicate target 
> indices 0 in this case.Finally, I fixed it in this way.
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
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   10   11   >