[jira] [Commented] (HDFS-15398) hdfs client may hang forever when writing EC file

2020-06-08 Thread Ayush Saxena (Jira)


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

Ayush Saxena commented on HDFS-15398:
-

Thanx [~haiyang Hu] for the report, I too got something similar in HDFS-15211.
Would it be possible to extend a UT for this?

> hdfs client may hang forever when writing EC file
> -
>
> Key: HDFS-15398
> URL: https://issues.apache.org/jira/browse/HDFS-15398
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: ec, hdfs-client
>Affects Versions: 3.2.0
>Reporter: Hongbing Wang
>Priority: Major
> Attachments: HDFS-15398.001.patch
>
>
>  In the operation of writing EC files, when the client calls addBlock() 
> applying for the second block group (or >= the second block group) and it 
> happens to exceed quota at this time, the client program will hang forever. 
>  See the demo below:
> {code:java}
> $ hadoop fs -mkdir -p /user/wanghongbing/quota/ec
> $ hdfs dfsadmin -setSpaceQuota 2g /user/wanghongbing/quota
> $ hdfs ec -setPolicy -path /user/wanghongbing/quota/ec -policy RS-6-3-1024k
> Set RS-6-3-1024k erasure coding policy on /user/wanghongbing/quota/ec
> $ hadoop fs -put 800m /user/wanghongbing/quota/ec
> ^@^@^@^@^@^@^@^@^Z
> {code}
> In the case of blocksize=128M, spaceQuota=2g and EC 6-3 policy, a block group 
> needs to apply for 1152M physical space to write 768M logical data. 
> Therefore, writing 800M data will exceed quota when applying for the second 
> block group. At this point, the client will be hang forever.
> The exception stack of client is as follows:
> {code:java}
> java.lang.Thread.State: TIMED_WAITING (parking)
> at sun.misc.Unsafe.park(Native Method)
> - parking to wait for  <0x8009d5d8> (a 
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
> at 
> java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
> at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
> at 
> java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
> at 
> org.apache.hadoop.hdfs.DFSStripedOutputStream$MultipleBlockingQueue.takeWithTimeout(DFSStripedOutputStream.java:117)
> at 
> org.apache.hadoop.hdfs.DFSStripedOutputStream.waitEndBlocks(DFSStripedOutputStream.java:453)
> at 
> org.apache.hadoop.hdfs.DFSStripedOutputStream.allocateNewBlock(DFSStripedOutputStream.java:477)
> at 
> org.apache.hadoop.hdfs.DFSStripedOutputStream.writeChunk(DFSStripedOutputStream.java:541)
> - locked <0x8009f758> (a 
> org.apache.hadoop.hdfs.DFSStripedOutputStream)
> at 
> org.apache.hadoop.fs.FSOutputSummer.writeChecksumChunks(FSOutputSummer.java:217)
> at 
> org.apache.hadoop.fs.FSOutputSummer.flushBuffer(FSOutputSummer.java:164)
> - locked <0x8009f758> (a 
> org.apache.hadoop.hdfs.DFSStripedOutputStream)
> at 
> org.apache.hadoop.fs.FSOutputSummer.flushBuffer(FSOutputSummer.java:145)
> - locked <0x8009f758> (a 
> org.apache.hadoop.hdfs.DFSStripedOutputStream)
> at 
> org.apache.hadoop.hdfs.DFSStripedOutputStream.closeImpl(DFSStripedOutputStream.java:1182)
> - locked <0x8009f758> (a 
> org.apache.hadoop.hdfs.DFSStripedOutputStream)
> at 
> org.apache.hadoop.hdfs.DFSOutputStream.close(DFSOutputStream.java:847)
> - locked <0x8009f758> (a 
> org.apache.hadoop.hdfs.DFSStripedOutputStream)
> at 
> org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72)
> at 
> org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:101)
> at org.apache.hadoop.io.IOUtils.cleanupWithLogger(IOUtils.java:280)
> at org.apache.hadoop.io.IOUtils.closeStream(IOUtils.java:298)
> at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:77)
> at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:129)
> at 
> org.apache.hadoop.fs.shell.CommandWithDestination$TargetFileSystem.writeStreamToFile(CommandWithDestination.java:485)
> at 
> org.apache.hadoop.fs.shell.CommandWithDestination.copyStreamToTarget(CommandWithDestination.java:407)
> at 
> org.apache.hadoop.fs.shell.CommandWithDestination.copyFileToTarget(CommandWithDestination.java:342)
> at 
> org.apache.hadoop.fs.shell.CommandWithDestination.processPath(CommandWithDestination.java:277)
> at 
> org.apache.hadoop.fs.shell.CommandWithDestination.processPath(CommandWithDestination.java:262)
> {code}
> When an exception occurs in addBlock, the program will call 
> DFSStripedOutputStream.closeImpl() -> flushBuffer() -> writeChunk() -> 
> allocateNewBlock() -> waitEndBlocks(), waitEndBlocks 

[jira] [Updated] (HDFS-15398) hdfs client may hang forever when writing EC file

2020-06-08 Thread Hongbing Wang (Jira)


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

Hongbing Wang updated HDFS-15398:
-
Summary: hdfs client may hang forever when writing EC file  (was: 
DFSStripedOutputStream.waitEndBlocks() may enter an infinite loop)

> hdfs client may hang forever when writing EC file
> -
>
> Key: HDFS-15398
> URL: https://issues.apache.org/jira/browse/HDFS-15398
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: ec, hdfs-client
>Affects Versions: 3.2.0
>Reporter: Hongbing Wang
>Priority: Major
> Attachments: HDFS-15398.001.patch
>
>
>  In the operation of writing EC files, when the client calls addBlock() 
> applying for the second block group (or >= the second block group) and it 
> happens to exceed quota at this time, the client program will hang forever. 
>  See the demo below:
> {code:java}
> $ hadoop fs -mkdir -p /user/wanghongbing/quota/ec
> $ hdfs dfsadmin -setSpaceQuota 2g /user/wanghongbing/quota
> $ hdfs ec -setPolicy -path /user/wanghongbing/quota/ec -policy RS-6-3-1024k
> Set RS-6-3-1024k erasure coding policy on /user/wanghongbing/quota/ec
> $ hadoop fs -put 800m /user/wanghongbing/quota/ec
> ^@^@^@^@^@^@^@^@^Z
> {code}
> In the case of blocksize=128M, spaceQuota=2g and EC 6-3 policy, a block group 
> needs to apply for 1152M physical space to write 768M logical data. 
> Therefore, writing 800M data will exceed quota when applying for the second 
> block group. At this point, the client will be hang forever.
> The exception stack of client is as follows:
> {code:java}
> java.lang.Thread.State: TIMED_WAITING (parking)
> at sun.misc.Unsafe.park(Native Method)
> - parking to wait for  <0x8009d5d8> (a 
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
> at 
> java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
> at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
> at 
> java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
> at 
> org.apache.hadoop.hdfs.DFSStripedOutputStream$MultipleBlockingQueue.takeWithTimeout(DFSStripedOutputStream.java:117)
> at 
> org.apache.hadoop.hdfs.DFSStripedOutputStream.waitEndBlocks(DFSStripedOutputStream.java:453)
> at 
> org.apache.hadoop.hdfs.DFSStripedOutputStream.allocateNewBlock(DFSStripedOutputStream.java:477)
> at 
> org.apache.hadoop.hdfs.DFSStripedOutputStream.writeChunk(DFSStripedOutputStream.java:541)
> - locked <0x8009f758> (a 
> org.apache.hadoop.hdfs.DFSStripedOutputStream)
> at 
> org.apache.hadoop.fs.FSOutputSummer.writeChecksumChunks(FSOutputSummer.java:217)
> at 
> org.apache.hadoop.fs.FSOutputSummer.flushBuffer(FSOutputSummer.java:164)
> - locked <0x8009f758> (a 
> org.apache.hadoop.hdfs.DFSStripedOutputStream)
> at 
> org.apache.hadoop.fs.FSOutputSummer.flushBuffer(FSOutputSummer.java:145)
> - locked <0x8009f758> (a 
> org.apache.hadoop.hdfs.DFSStripedOutputStream)
> at 
> org.apache.hadoop.hdfs.DFSStripedOutputStream.closeImpl(DFSStripedOutputStream.java:1182)
> - locked <0x8009f758> (a 
> org.apache.hadoop.hdfs.DFSStripedOutputStream)
> at 
> org.apache.hadoop.hdfs.DFSOutputStream.close(DFSOutputStream.java:847)
> - locked <0x8009f758> (a 
> org.apache.hadoop.hdfs.DFSStripedOutputStream)
> at 
> org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72)
> at 
> org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:101)
> at org.apache.hadoop.io.IOUtils.cleanupWithLogger(IOUtils.java:280)
> at org.apache.hadoop.io.IOUtils.closeStream(IOUtils.java:298)
> at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:77)
> at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:129)
> at 
> org.apache.hadoop.fs.shell.CommandWithDestination$TargetFileSystem.writeStreamToFile(CommandWithDestination.java:485)
> at 
> org.apache.hadoop.fs.shell.CommandWithDestination.copyStreamToTarget(CommandWithDestination.java:407)
> at 
> org.apache.hadoop.fs.shell.CommandWithDestination.copyFileToTarget(CommandWithDestination.java:342)
> at 
> org.apache.hadoop.fs.shell.CommandWithDestination.processPath(CommandWithDestination.java:277)
> at 
> org.apache.hadoop.fs.shell.CommandWithDestination.processPath(CommandWithDestination.java:262)
> {code}
> When an exception occurs in addBlock, the program will call 
> DFSStripedOutputStream.closeImpl() -> flushBuffer() -> writeChunk() -> 
> allocateNewBlock() -> waitEndBlocks(), waitEndBlocks will enter an infinite 
> 

[jira] [Updated] (HDFS-15398) DFSStripedOutputStream.waitEndBlocks() may enter an infinite loop

2020-06-08 Thread Hongbing Wang (Jira)


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

Hongbing Wang updated HDFS-15398:
-
Attachment: HDFS-15398.001.patch

> DFSStripedOutputStream.waitEndBlocks() may enter an infinite loop
> -
>
> Key: HDFS-15398
> URL: https://issues.apache.org/jira/browse/HDFS-15398
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: ec, hdfs-client
>Affects Versions: 3.2.0
>Reporter: Hongbing Wang
>Priority: Major
> Attachments: HDFS-15398.001.patch
>
>
>  In the operation of writing EC files, when the client calls addBlock() 
> applying for the second block group (or >= the second block group) and it 
> happens to exceed quota at this time, the client program will hang forever. 
>  See the demo below:
> {code:java}
> $ hadoop fs -mkdir -p /user/wanghongbing/quota/ec
> $ hdfs dfsadmin -setSpaceQuota 2g /user/wanghongbing/quota
> $ hdfs ec -setPolicy -path /user/wanghongbing/quota/ec -policy RS-6-3-1024k
> Set RS-6-3-1024k erasure coding policy on /user/wanghongbing/quota/ec
> $ hadoop fs -put 800m /user/wanghongbing/quota/ec
> ^@^@^@^@^@^@^@^@^Z
> {code}
> In the case of blocksize=128M, spaceQuota=2g and EC 6-3 policy, a block group 
> needs to apply for 1152M physical space to write 768M logical data. 
> Therefore, writing 800M data will exceed quota when applying for the second 
> block group. At this point, the client will be hang forever.
> The exception stack of client is as follows:
> {code:java}
> java.lang.Thread.State: TIMED_WAITING (parking)
> at sun.misc.Unsafe.park(Native Method)
> - parking to wait for  <0x8009d5d8> (a 
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
> at 
> java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
> at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
> at 
> java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
> at 
> org.apache.hadoop.hdfs.DFSStripedOutputStream$MultipleBlockingQueue.takeWithTimeout(DFSStripedOutputStream.java:117)
> at 
> org.apache.hadoop.hdfs.DFSStripedOutputStream.waitEndBlocks(DFSStripedOutputStream.java:453)
> at 
> org.apache.hadoop.hdfs.DFSStripedOutputStream.allocateNewBlock(DFSStripedOutputStream.java:477)
> at 
> org.apache.hadoop.hdfs.DFSStripedOutputStream.writeChunk(DFSStripedOutputStream.java:541)
> - locked <0x8009f758> (a 
> org.apache.hadoop.hdfs.DFSStripedOutputStream)
> at 
> org.apache.hadoop.fs.FSOutputSummer.writeChecksumChunks(FSOutputSummer.java:217)
> at 
> org.apache.hadoop.fs.FSOutputSummer.flushBuffer(FSOutputSummer.java:164)
> - locked <0x8009f758> (a 
> org.apache.hadoop.hdfs.DFSStripedOutputStream)
> at 
> org.apache.hadoop.fs.FSOutputSummer.flushBuffer(FSOutputSummer.java:145)
> - locked <0x8009f758> (a 
> org.apache.hadoop.hdfs.DFSStripedOutputStream)
> at 
> org.apache.hadoop.hdfs.DFSStripedOutputStream.closeImpl(DFSStripedOutputStream.java:1182)
> - locked <0x8009f758> (a 
> org.apache.hadoop.hdfs.DFSStripedOutputStream)
> at 
> org.apache.hadoop.hdfs.DFSOutputStream.close(DFSOutputStream.java:847)
> - locked <0x8009f758> (a 
> org.apache.hadoop.hdfs.DFSStripedOutputStream)
> at 
> org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72)
> at 
> org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:101)
> at org.apache.hadoop.io.IOUtils.cleanupWithLogger(IOUtils.java:280)
> at org.apache.hadoop.io.IOUtils.closeStream(IOUtils.java:298)
> at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:77)
> at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:129)
> at 
> org.apache.hadoop.fs.shell.CommandWithDestination$TargetFileSystem.writeStreamToFile(CommandWithDestination.java:485)
> at 
> org.apache.hadoop.fs.shell.CommandWithDestination.copyStreamToTarget(CommandWithDestination.java:407)
> at 
> org.apache.hadoop.fs.shell.CommandWithDestination.copyFileToTarget(CommandWithDestination.java:342)
> at 
> org.apache.hadoop.fs.shell.CommandWithDestination.processPath(CommandWithDestination.java:277)
> at 
> org.apache.hadoop.fs.shell.CommandWithDestination.processPath(CommandWithDestination.java:262)
> {code}
> When an exception occurs in addBlock, the program will call 
> DFSStripedOutputStream.closeImpl() -> flushBuffer() -> writeChunk() -> 
> allocateNewBlock() -> waitEndBlocks(), waitEndBlocks will enter an infinite 
> loop because the queue in endBlocks is empty.
> {code:java}
> private 

[jira] [Updated] (HDFS-15398) DFSStripedOutputStream.waitEndBlocks() may enter an infinite loop

2020-06-08 Thread Hongbing Wang (Jira)


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

Hongbing Wang updated HDFS-15398:
-
Description: 
 In the operation of writing EC files, when the client calls addBlock() 
applying for the second block group (or >= the second block group) and it 
happens to exceed quota at this time, the client program will hang forever. 
 See the demo below:
{code:java}
$ hadoop fs -mkdir -p /user/wanghongbing/quota/ec
$ hdfs dfsadmin -setSpaceQuota 2g /user/wanghongbing/quota
$ hdfs ec -setPolicy -path /user/wanghongbing/quota/ec -policy RS-6-3-1024k
Set RS-6-3-1024k erasure coding policy on /user/wanghongbing/quota/ec
$ hadoop fs -put 800m /user/wanghongbing/quota/ec
^@^@^@^@^@^@^@^@^Z
{code}
In the case of blocksize=128M, spaceQuota=2g and EC 6-3 policy, a block group 
needs to apply for 1152M physical space to write 768M logical data. Therefore, 
writing 800M data will exceed quota when applying for the second block group. 
At this point, the client will be hang forever.

The exception stack of client is as follows:
{code:java}
java.lang.Thread.State: TIMED_WAITING (parking)
at sun.misc.Unsafe.park(Native Method)
- parking to wait for  <0x8009d5d8> (a 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
at 
java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
at 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
at 
java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
at 
org.apache.hadoop.hdfs.DFSStripedOutputStream$MultipleBlockingQueue.takeWithTimeout(DFSStripedOutputStream.java:117)
at 
org.apache.hadoop.hdfs.DFSStripedOutputStream.waitEndBlocks(DFSStripedOutputStream.java:453)
at 
org.apache.hadoop.hdfs.DFSStripedOutputStream.allocateNewBlock(DFSStripedOutputStream.java:477)
at 
org.apache.hadoop.hdfs.DFSStripedOutputStream.writeChunk(DFSStripedOutputStream.java:541)
- locked <0x8009f758> (a 
org.apache.hadoop.hdfs.DFSStripedOutputStream)
at 
org.apache.hadoop.fs.FSOutputSummer.writeChecksumChunks(FSOutputSummer.java:217)
at 
org.apache.hadoop.fs.FSOutputSummer.flushBuffer(FSOutputSummer.java:164)
- locked <0x8009f758> (a 
org.apache.hadoop.hdfs.DFSStripedOutputStream)
at 
org.apache.hadoop.fs.FSOutputSummer.flushBuffer(FSOutputSummer.java:145)
- locked <0x8009f758> (a 
org.apache.hadoop.hdfs.DFSStripedOutputStream)
at 
org.apache.hadoop.hdfs.DFSStripedOutputStream.closeImpl(DFSStripedOutputStream.java:1182)
- locked <0x8009f758> (a 
org.apache.hadoop.hdfs.DFSStripedOutputStream)
at 
org.apache.hadoop.hdfs.DFSOutputStream.close(DFSOutputStream.java:847)
- locked <0x8009f758> (a 
org.apache.hadoop.hdfs.DFSStripedOutputStream)
at 
org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72)
at 
org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:101)
at org.apache.hadoop.io.IOUtils.cleanupWithLogger(IOUtils.java:280)
at org.apache.hadoop.io.IOUtils.closeStream(IOUtils.java:298)
at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:77)
at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:129)
at 
org.apache.hadoop.fs.shell.CommandWithDestination$TargetFileSystem.writeStreamToFile(CommandWithDestination.java:485)
at 
org.apache.hadoop.fs.shell.CommandWithDestination.copyStreamToTarget(CommandWithDestination.java:407)
at 
org.apache.hadoop.fs.shell.CommandWithDestination.copyFileToTarget(CommandWithDestination.java:342)
at 
org.apache.hadoop.fs.shell.CommandWithDestination.processPath(CommandWithDestination.java:277)
at 
org.apache.hadoop.fs.shell.CommandWithDestination.processPath(CommandWithDestination.java:262)
{code}
When an exception occurs in addBlock, the program will call 
DFSStripedOutputStream.closeImpl() -> flushBuffer() -> writeChunk() -> 
allocateNewBlock() -> waitEndBlocks(), waitEndBlocks will enter an infinite 
loop because the queue in endBlocks is empty.
{code:java}
private void waitEndBlocks(int i) throws IOException {
  while (getStripedDataStreamer(i).isHealthy()) {
final ExtendedBlock b = coordinator.endBlocks.takeWithTimeout(i);
if (b != null) {
  StripedBlockUtil.checkBlocks(currentBlockGroup, i, b);
  return;
}
  }
}
{code}
So I close all stripedDataStreamer to fix it When an exception occurs in 
addBlock.

 

  was:
 In the operation of writing EC files, when the client calls addBlock() 
applying for the second block group (or >= the second block group) and it 
happens to exceed quota at this time, the client program will hang forever. 
See the demo below:
{code:java}
$ hadoop fs -mkdir -p 

[jira] [Updated] (HDFS-15398) DFSStripedOutputStream.waitEndBlocks() may enter an infinite loop

2020-06-08 Thread Hongbing Wang (Jira)


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

Hongbing Wang updated HDFS-15398:
-
Description: 
 In the operation of writing EC files, when the client calls addBlock() 
applying for the second block group (or >= the second block group) and it 
happens to exceed quota at this time, the client program will hang forever. 
See the demo below:
{code:java}
$ hadoop fs -mkdir -p /user/wanghongbing/quota/ec
$ hdfs dfsadmin -setSpaceQuota 2g /user/wanghongbing/quota
$ hdfs ec -setPolicy -path /user/wanghongbing/quota/ec -policy RS-6-3-1024k
Set RS-6-3-1024k erasure coding policy on /user/wanghongbing/quota/ec
$ hadoop fs -put 800m /user/wanghongbing/quota/ec
^@^@^@^@^@^@^@^@^Z
{code}
In the case of blocksize=128M, spaceQuota=2g and EC 6-3 policy, a block group 
needs to apply for 1152M physical space to write 768M logical data. Therefore, 
writing 800M data will exceed quota when applying for the second block group. 
At this point, the client will be hang forever.

The exception stack of client is as follows:
{code:java}
java.lang.Thread.State: TIMED_WAITING (parking)
at sun.misc.Unsafe.park(Native Method)
- parking to wait for  <0x8009d5d8> (a 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
at 
java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
at 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
at 
java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
at 
org.apache.hadoop.hdfs.DFSStripedOutputStream$MultipleBlockingQueue.takeWithTimeout(DFSStripedOutputStream.java:117)
at 
org.apache.hadoop.hdfs.DFSStripedOutputStream.waitEndBlocks(DFSStripedOutputStream.java:453)
at 
org.apache.hadoop.hdfs.DFSStripedOutputStream.allocateNewBlock(DFSStripedOutputStream.java:477)
at 
org.apache.hadoop.hdfs.DFSStripedOutputStream.writeChunk(DFSStripedOutputStream.java:541)
- locked <0x8009f758> (a 
org.apache.hadoop.hdfs.DFSStripedOutputStream)
at 
org.apache.hadoop.fs.FSOutputSummer.writeChecksumChunks(FSOutputSummer.java:217)
at 
org.apache.hadoop.fs.FSOutputSummer.flushBuffer(FSOutputSummer.java:164)
- locked <0x8009f758> (a 
org.apache.hadoop.hdfs.DFSStripedOutputStream)
at 
org.apache.hadoop.fs.FSOutputSummer.flushBuffer(FSOutputSummer.java:145)
- locked <0x8009f758> (a 
org.apache.hadoop.hdfs.DFSStripedOutputStream)
at 
org.apache.hadoop.hdfs.DFSStripedOutputStream.closeImpl(DFSStripedOutputStream.java:1182)
- locked <0x8009f758> (a 
org.apache.hadoop.hdfs.DFSStripedOutputStream)
at 
org.apache.hadoop.hdfs.DFSOutputStream.close(DFSOutputStream.java:847)
- locked <0x8009f758> (a 
org.apache.hadoop.hdfs.DFSStripedOutputStream)
at 
org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72)
at 
org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:101)
at org.apache.hadoop.io.IOUtils.cleanupWithLogger(IOUtils.java:280)
at org.apache.hadoop.io.IOUtils.closeStream(IOUtils.java:298)
at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:77)
at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:129)
at 
org.apache.hadoop.fs.shell.CommandWithDestination$TargetFileSystem.writeStreamToFile(CommandWithDestination.java:485)
at 
org.apache.hadoop.fs.shell.CommandWithDestination.copyStreamToTarget(CommandWithDestination.java:407)
at 
org.apache.hadoop.fs.shell.CommandWithDestination.copyFileToTarget(CommandWithDestination.java:342)
at 
org.apache.hadoop.fs.shell.CommandWithDestination.processPath(CommandWithDestination.java:277)
at 
org.apache.hadoop.fs.shell.CommandWithDestination.processPath(CommandWithDestination.java:262)
{code}
When an exception occurs in addBlock, the program will call 
DFSStripedOutputStream.closeImpl() -> flushBuffer() -> writeChunk() -> 
allocateNewBlock() -> waitEndBlocks(), waitEndBlocks will enter an infinite 
loop because the queue in endBlocks is empty.
{code:java}
private void waitEndBlocks(int i) throws IOException {
  while (getStripedDataStreamer(i).isHealthy()) {
final ExtendedBlock b = coordinator.endBlocks.takeWithTimeout(i);
if (b != null) {
  StripedBlockUtil.checkBlocks(currentBlockGroup, i, b);
  return;
}
  }
}
{code}
So I close all stripedDataStreamer to fix it When an exception occurs in 
addBlock.

 

> DFSStripedOutputStream.waitEndBlocks() may enter an infinite loop
> -
>
> Key: HDFS-15398
> URL: https://issues.apache.org/jira/browse/HDFS-15398
> Project: Hadoop HDFS
>  Issue 

[jira] [Created] (HDFS-15399) Support include or exclude datanode by configure file

2020-06-08 Thread maobaolong (Jira)
maobaolong created HDFS-15399:
-

 Summary: Support include or exclude datanode by configure file
 Key: HDFS-15399
 URL: https://issues.apache.org/jira/browse/HDFS-15399
 Project: Hadoop HDFS
  Issue Type: New Feature
  Components: datanode
Reporter: maobaolong
Assignee: maobaolong


When i dislike a datanode, or just want to let specific datanode join to SCM, i 
want to have this feature to limit datanode list.



--
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-15098) Add SM4 encryption method for HDFS

2020-06-08 Thread liusheng (Jira)


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

liusheng edited comment on HDFS-15098 at 6/9/20, 3:40 AM:
--

Hi [~lindongdong],

 As [~zZtai] explained, the SM4 feature is supportted in OpenSSL >=1.1.1 
version, if this requirement is satisfied in environment, we don't need these 2 
steps, if not, the SM4 feature will fallback to use an  alternative 
implementation of SM4 feature with the Bouncy Castle Crypto provider jar 
package. see: [https://www.bouncycastle.org/specifications.html]

The corresponding AES encryption support also have similar implementation. but 
the AES is internally supportted in JDK, while SM4 JCE implementation need the 
external jar package.


was (Author: seanlau):
Hi [~lindongdong],

 As [~zZtai] explained, the SM4 feature is supportted in OpenSSL >=1.1.1 
version, if this requirement is satisfied in environment, we don't need these 2 
steps, if not, the SM4 feature will fallback to use an  alternative 
implementation of SM4 feature with the Bouncy Castle Crypto provider jar 
package. see: [https://www.bouncycastle.org/specifications.html]

The corresponding AES encryption support also have similar implementation.

> Add SM4 encryption method for HDFS
> --
>
> Key: HDFS-15098
> URL: https://issues.apache.org/jira/browse/HDFS-15098
> Project: Hadoop HDFS
>  Issue Type: New Feature
>Affects Versions: 3.4.0
>Reporter: liusheng
>Assignee: zZtai
>Priority: Major
>  Labels: sm4
> Attachments: HDFS-15098.001.patch, HDFS-15098.002.patch, 
> HDFS-15098.003.patch, HDFS-15098.004.patch, HDFS-15098.005.patch
>
>
> SM4 (formerly SMS4)is a block cipher used in the Chinese National Standard 
> for Wireless LAN WAPI (Wired Authentication and Privacy Infrastructure).
>  SM4 was a cipher proposed to for the IEEE 802.11i standard, but has so far 
> been rejected by ISO. One of the reasons for the rejection has been 
> opposition to the WAPI fast-track proposal by the IEEE. please see:
> [https://en.wikipedia.org/wiki/SM4_(cipher)]
>  
> *Use sm4 on hdfs as follows:*
> 1.download Bouncy Castle Crypto APIs from bouncycastle.org
> [https://bouncycastle.org/download/bcprov-ext-jdk15on-165.jar]
> 2.Configure JDK
> Place bcprov-ext-jdk15on-165.jar in $JAVA_HOME/jre/lib/ext directory,
> add "security.provider.10=org.bouncycastle.jce.provider.BouncyCastleProvider" 
> to $JAVA_HOME/jre/lib/security/java.security file
> 3.Configure Hadoop KMS
> 4.test HDFS sm4
> hadoop key create key1 -cipher 'SM4/CTR/NoPadding'
> hdfs dfs -mkdir /benchmarks
> hdfs crypto -createZone -keyName key1 -path /benchmarks
> *requires:*
> 1.openssl version >=1.1.1
> 2.configure Bouncy Castle Crypto on JDK



--
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-15098) Add SM4 encryption method for HDFS

2020-06-08 Thread liusheng (Jira)


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

liusheng edited comment on HDFS-15098 at 6/9/20, 3:39 AM:
--

Hi [~lindongdong],

 As [~zZtai] explained, the SM4 feature is supportted in OpenSSL >=1.1.1 
version, if this requirement is satisfied in environment, we don't need these 2 
steps, if not, the SM4 feature will fallback to use an  alternative 
implementation of SM4 feature with the Bouncy Castle Crypto provider jar 
package. see: [https://www.bouncycastle.org/specifications.html]

The corresponding AES encryption support also have similar implementation.


was (Author: seanlau):
Hi [~lindongdong],

 As [~zZtai] explained, the SM4 feature is supportted in OpenSSL >=1.1.1 
version, if this requirement is satisfied in environment, we don't need these 2 
steps, if not, the SM4 feature will fail back to use an  alternative 
implementation of SM4 feature with the Bouncy Castle Crypto provider jar 
package. see: [https://www.bouncycastle.org/specifications.html]

> Add SM4 encryption method for HDFS
> --
>
> Key: HDFS-15098
> URL: https://issues.apache.org/jira/browse/HDFS-15098
> Project: Hadoop HDFS
>  Issue Type: New Feature
>Affects Versions: 3.4.0
>Reporter: liusheng
>Assignee: zZtai
>Priority: Major
>  Labels: sm4
> Attachments: HDFS-15098.001.patch, HDFS-15098.002.patch, 
> HDFS-15098.003.patch, HDFS-15098.004.patch, HDFS-15098.005.patch
>
>
> SM4 (formerly SMS4)is a block cipher used in the Chinese National Standard 
> for Wireless LAN WAPI (Wired Authentication and Privacy Infrastructure).
>  SM4 was a cipher proposed to for the IEEE 802.11i standard, but has so far 
> been rejected by ISO. One of the reasons for the rejection has been 
> opposition to the WAPI fast-track proposal by the IEEE. please see:
> [https://en.wikipedia.org/wiki/SM4_(cipher)]
>  
> *Use sm4 on hdfs as follows:*
> 1.download Bouncy Castle Crypto APIs from bouncycastle.org
> [https://bouncycastle.org/download/bcprov-ext-jdk15on-165.jar]
> 2.Configure JDK
> Place bcprov-ext-jdk15on-165.jar in $JAVA_HOME/jre/lib/ext directory,
> add "security.provider.10=org.bouncycastle.jce.provider.BouncyCastleProvider" 
> to $JAVA_HOME/jre/lib/security/java.security file
> 3.Configure Hadoop KMS
> 4.test HDFS sm4
> hadoop key create key1 -cipher 'SM4/CTR/NoPadding'
> hdfs dfs -mkdir /benchmarks
> hdfs crypto -createZone -keyName key1 -path /benchmarks
> *requires:*
> 1.openssl version >=1.1.1
> 2.configure Bouncy Castle Crypto on JDK



--
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-15098) Add SM4 encryption method for HDFS

2020-06-08 Thread liusheng (Jira)


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

liusheng edited comment on HDFS-15098 at 6/9/20, 2:53 AM:
--

Hi [~lindongdong],

 As [~zZtai] explained, the SM4 feature is supportted in OpenSSL >=1.1.1 
version, if this requirement is satisfied in environment, we don't need these 2 
steps, if not, the SM4 feature will fail back to use an  alternative 
implementation of SM4 feature with the Bouncy Castle Crypto provider jar 
package. see: [https://www.bouncycastle.org/specifications.html]


was (Author: seanlau):
Hi [~lindongdong],

 As [~zZtai] explained, the SM4 feature is supportted in OpenSSL >=1.1.1 
version, if this requirement is satisfied in environment, we don't need these 2 
steps, if not, the SM4 feature will fail back to use an  alternative 
implementation of SM4 feature with the Bouncy Castle Crypto provider jar 
package.

> Add SM4 encryption method for HDFS
> --
>
> Key: HDFS-15098
> URL: https://issues.apache.org/jira/browse/HDFS-15098
> Project: Hadoop HDFS
>  Issue Type: New Feature
>Affects Versions: 3.4.0
>Reporter: liusheng
>Assignee: zZtai
>Priority: Major
>  Labels: sm4
> Attachments: HDFS-15098.001.patch, HDFS-15098.002.patch, 
> HDFS-15098.003.patch, HDFS-15098.004.patch, HDFS-15098.005.patch
>
>
> SM4 (formerly SMS4)is a block cipher used in the Chinese National Standard 
> for Wireless LAN WAPI (Wired Authentication and Privacy Infrastructure).
>  SM4 was a cipher proposed to for the IEEE 802.11i standard, but has so far 
> been rejected by ISO. One of the reasons for the rejection has been 
> opposition to the WAPI fast-track proposal by the IEEE. please see:
> [https://en.wikipedia.org/wiki/SM4_(cipher)]
>  
> *Use sm4 on hdfs as follows:*
> 1.download Bouncy Castle Crypto APIs from bouncycastle.org
> [https://bouncycastle.org/download/bcprov-ext-jdk15on-165.jar]
> 2.Configure JDK
> Place bcprov-ext-jdk15on-165.jar in $JAVA_HOME/jre/lib/ext directory,
> add "security.provider.10=org.bouncycastle.jce.provider.BouncyCastleProvider" 
> to $JAVA_HOME/jre/lib/security/java.security file
> 3.Configure Hadoop KMS
> 4.test HDFS sm4
> hadoop key create key1 -cipher 'SM4/CTR/NoPadding'
> hdfs dfs -mkdir /benchmarks
> hdfs crypto -createZone -keyName key1 -path /benchmarks
> *requires:*
> 1.openssl version >=1.1.1
> 2.configure Bouncy Castle Crypto on JDK



--
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-15372) Files in snapshots no longer see attribute provider permissions

2020-06-08 Thread Hadoop QA (Jira)


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

Hadoop QA commented on HDFS-15372:
--

| (x) *{color:red}-1 overall{color}* |
\\
\\
|| Vote || Subsystem || Runtime || Comment ||
| {color:blue}0{color} | {color:blue} reexec {color} | {color:blue} 26m 
36s{color} | {color:blue} Docker mode activated. {color} |
|| || || || {color:brown} Prechecks {color} ||
| {color:green}+1{color} | {color:green} dupname {color} | {color:green}  0m  
1s{color} | {color:green} No case conflicting files found. {color} |
| {color:green}+1{color} | {color:green} @author {color} | {color:green}  0m  
0s{color} | {color:green} The patch does not contain any @author tags. {color} |
| {color:green}+1{color} | {color:green} test4tests {color} | {color:green}  0m 
 0s{color} | {color:green} The patch appears to include 1 new or modified test 
files. {color} |
|| || || || {color:brown} trunk Compile Tests {color} ||
| {color:green}+1{color} | {color:green} mvninstall {color} | {color:green} 21m 
51s{color} | {color:green} trunk passed {color} |
| {color:green}+1{color} | {color:green} compile {color} | {color:green}  1m  
8s{color} | {color:green} trunk passed {color} |
| {color:green}+1{color} | {color:green} checkstyle {color} | {color:green}  0m 
47s{color} | {color:green} trunk passed {color} |
| {color:green}+1{color} | {color:green} mvnsite {color} | {color:green}  1m 
12s{color} | {color:green} trunk passed {color} |
| {color:green}+1{color} | {color:green} shadedclient {color} | {color:green} 
17m 32s{color} | {color:green} branch has no errors when building and testing 
our client artifacts. {color} |
| {color:green}+1{color} | {color:green} javadoc {color} | {color:green}  0m 
46s{color} | {color:green} trunk passed {color} |
| {color:blue}0{color} | {color:blue} spotbugs {color} | {color:blue}  3m  
5s{color} | {color:blue} Used deprecated FindBugs config; considering switching 
to SpotBugs. {color} |
| {color:green}+1{color} | {color:green} findbugs {color} | {color:green}  3m  
3s{color} | {color:green} trunk passed {color} |
|| || || || {color:brown} Patch Compile Tests {color} ||
| {color:green}+1{color} | {color:green} mvninstall {color} | {color:green}  1m 
11s{color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} compile {color} | {color:green}  1m  
2s{color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} javac {color} | {color:green}  1m  
2s{color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} checkstyle {color} | {color:green}  0m 
42s{color} | {color:green} hadoop-hdfs-project/hadoop-hdfs: The patch generated 
0 new + 88 unchanged - 1 fixed = 88 total (was 89) {color} |
| {color:green}+1{color} | {color:green} mvnsite {color} | {color:green}  1m  
8s{color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} whitespace {color} | {color:green}  0m 
 0s{color} | {color:green} The patch has no whitespace issues. {color} |
| {color:green}+1{color} | {color:green} shadedclient {color} | {color:green} 
15m 30s{color} | {color:green} patch has no errors when building and testing 
our client artifacts. {color} |
| {color:green}+1{color} | {color:green} javadoc {color} | {color:green}  0m 
39s{color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} findbugs {color} | {color:green}  3m  
7s{color} | {color:green} the patch passed {color} |
|| || || || {color:brown} Other Tests {color} ||
| {color:red}-1{color} | {color:red} unit {color} | {color:red}108m 11s{color} 
| {color:red} hadoop-hdfs in the patch passed. {color} |
| {color:green}+1{color} | {color:green} asflicense {color} | {color:green}  0m 
35s{color} | {color:green} The patch does not generate ASF License warnings. 
{color} |
| {color:black}{color} | {color:black} {color} | {color:black}205m 27s{color} | 
{color:black} {color} |
\\
\\
|| Reason || Tests ||
| Failed junit tests | hadoop.hdfs.TestReconstructStripedFileWithRandomECPolicy 
|
|   | hadoop.hdfs.server.blockmanagement.TestBlockTokenWithDFSStriped |
|   | hadoop.hdfs.web.TestWebHdfsWithMultipleNameNodes |
|   | hadoop.hdfs.TestReconstructStripedFile |
|   | hadoop.hdfs.server.sps.TestExternalStoragePolicySatisfier |
\\
\\
|| Subsystem || Report/Notes ||
| Docker | ClientAPI=1.40 ServerAPI=1.40 base: 
https://builds.apache.org/job/PreCommit-HDFS-Build/29407/artifact/out/Dockerfile
 |
| JIRA Issue | HDFS-15372 |
| JIRA Patch URL | 
https://issues.apache.org/jira/secure/attachment/13005156/HDFS-15372.003.patch |
| Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite 
unit shadedclient findbugs checkstyle |
| uname | Linux c0cf5cf6d4ea 4.15.0-101-generic #102-Ubuntu SMP Mon May 11 
10:07:26 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
| Build tool | maven |
| 

[jira] [Created] (HDFS-15398) DFSStripedOutputStream.waitEndBlocks() may enter an infinite loop

2020-06-08 Thread Hongbing Wang (Jira)
Hongbing Wang created HDFS-15398:


 Summary: DFSStripedOutputStream.waitEndBlocks() may enter an 
infinite loop
 Key: HDFS-15398
 URL: https://issues.apache.org/jira/browse/HDFS-15398
 Project: Hadoop HDFS
  Issue Type: Bug
  Components: ec, hdfs-client
Affects Versions: 3.2.0
Reporter: Hongbing Wang






--
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-15098) Add SM4 encryption method for HDFS

2020-06-08 Thread liusheng (Jira)


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

liusheng edited comment on HDFS-15098 at 6/9/20, 2:22 AM:
--

Hi [~lindongdong],

 As [~zZtai] explained, the SM4 feature is supportted in OpenSSL >=1.1.1 
version, if this requirement is satisfied in environment, we don't need these 2 
steps, if not, the SM4 feature will fail back to use an  alternative 
implementation of SM4 feature with the Bouncy Castle Crypto provider jar 
package.


was (Author: seanlau):
Hi [~lindongdong],

 

> Add SM4 encryption method for HDFS
> --
>
> Key: HDFS-15098
> URL: https://issues.apache.org/jira/browse/HDFS-15098
> Project: Hadoop HDFS
>  Issue Type: New Feature
>Affects Versions: 3.4.0
>Reporter: liusheng
>Assignee: zZtai
>Priority: Major
>  Labels: sm4
> Attachments: HDFS-15098.001.patch, HDFS-15098.002.patch, 
> HDFS-15098.003.patch, HDFS-15098.004.patch, HDFS-15098.005.patch
>
>
> SM4 (formerly SMS4)is a block cipher used in the Chinese National Standard 
> for Wireless LAN WAPI (Wired Authentication and Privacy Infrastructure).
>  SM4 was a cipher proposed to for the IEEE 802.11i standard, but has so far 
> been rejected by ISO. One of the reasons for the rejection has been 
> opposition to the WAPI fast-track proposal by the IEEE. please see:
> [https://en.wikipedia.org/wiki/SM4_(cipher)]
>  
> *Use sm4 on hdfs as follows:*
> 1.download Bouncy Castle Crypto APIs from bouncycastle.org
> [https://bouncycastle.org/download/bcprov-ext-jdk15on-165.jar]
> 2.Configure JDK
> Place bcprov-ext-jdk15on-165.jar in $JAVA_HOME/jre/lib/ext directory,
> add "security.provider.10=org.bouncycastle.jce.provider.BouncyCastleProvider" 
> to $JAVA_HOME/jre/lib/security/java.security file
> 3.Configure Hadoop KMS
> 4.test HDFS sm4
> hadoop key create key1 -cipher 'SM4/CTR/NoPadding'
> hdfs dfs -mkdir /benchmarks
> hdfs crypto -createZone -keyName key1 -path /benchmarks
> *requires:*
> 1.openssl version >=1.1.1
> 2.configure Bouncy Castle Crypto on JDK



--
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-15098) Add SM4 encryption method for HDFS

2020-06-08 Thread liusheng (Jira)


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

liusheng commented on HDFS-15098:
-

Hi [~lindongdong],

 

> Add SM4 encryption method for HDFS
> --
>
> Key: HDFS-15098
> URL: https://issues.apache.org/jira/browse/HDFS-15098
> Project: Hadoop HDFS
>  Issue Type: New Feature
>Affects Versions: 3.4.0
>Reporter: liusheng
>Assignee: zZtai
>Priority: Major
>  Labels: sm4
> Attachments: HDFS-15098.001.patch, HDFS-15098.002.patch, 
> HDFS-15098.003.patch, HDFS-15098.004.patch, HDFS-15098.005.patch
>
>
> SM4 (formerly SMS4)is a block cipher used in the Chinese National Standard 
> for Wireless LAN WAPI (Wired Authentication and Privacy Infrastructure).
>  SM4 was a cipher proposed to for the IEEE 802.11i standard, but has so far 
> been rejected by ISO. One of the reasons for the rejection has been 
> opposition to the WAPI fast-track proposal by the IEEE. please see:
> [https://en.wikipedia.org/wiki/SM4_(cipher)]
>  
> *Use sm4 on hdfs as follows:*
> 1.download Bouncy Castle Crypto APIs from bouncycastle.org
> [https://bouncycastle.org/download/bcprov-ext-jdk15on-165.jar]
> 2.Configure JDK
> Place bcprov-ext-jdk15on-165.jar in $JAVA_HOME/jre/lib/ext directory,
> add "security.provider.10=org.bouncycastle.jce.provider.BouncyCastleProvider" 
> to $JAVA_HOME/jre/lib/security/java.security file
> 3.Configure Hadoop KMS
> 4.test HDFS sm4
> hadoop key create key1 -cipher 'SM4/CTR/NoPadding'
> hdfs dfs -mkdir /benchmarks
> hdfs crypto -createZone -keyName key1 -path /benchmarks
> *requires:*
> 1.openssl version >=1.1.1
> 2.configure Bouncy Castle Crypto on JDK



--
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-15386) ReplicaNotFoundException keeps happening in DN after removing multiple DN's data directories

2020-06-08 Thread Toshihiro Suzuki (Jira)


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

Toshihiro Suzuki commented on HDFS-15386:
-

[~sodonnell] I created a PR for branch-2.10 but it looks like QA is broken.
https://github.com/apache/hadoop/pull/2054

Can you please help me?

> ReplicaNotFoundException keeps happening in DN after removing multiple DN's 
> data directories
> 
>
> Key: HDFS-15386
> URL: https://issues.apache.org/jira/browse/HDFS-15386
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: Toshihiro Suzuki
>Assignee: Toshihiro Suzuki
>Priority: Major
> Fix For: 3.0.4, 3.2.2, 3.3.1, 3.4.0, 3.1.5
>
>
> When removing volumes, we need to invalidate all the blocks in the volumes. 
> In the following code (FsDatasetImpl), we keep the blocks that will be 
> invalidate in *blkToInvalidate* map. However as the key of the map is *bpid* 
> (Block Pool ID), it will be overwritten by other removed volumes. As a 
> result, the map will have only the blocks of the last volume we are removing, 
> and invalidate only them:
> {code:java}
> for (String bpid : volumeMap.getBlockPoolList()) {
>   List blocks = new ArrayList<>();
>   for (Iterator it =
> volumeMap.replicas(bpid).iterator(); it.hasNext();) {
> ReplicaInfo block = it.next();
> final StorageLocation blockStorageLocation =
> block.getVolume().getStorageLocation();
> LOG.trace("checking for block " + block.getBlockId() +
> " with storageLocation " + blockStorageLocation);
> if (blockStorageLocation.equals(sdLocation)) {
>   blocks.add(block);
>   it.remove();
> }
>   }
>   blkToInvalidate.put(bpid, blocks);
> }
> {code}
> [https://github.com/apache/hadoop/blob/704409d53bf7ebf717a3c2e988ede80f623bbad3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java#L580-L595]



--
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-13965) hadoop.security.kerberos.ticket.cache.path setting is not honored when KMS encryption is enabled.

2020-06-08 Thread LOKESKUMAR VIJAYAKUMAR (Jira)


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

LOKESKUMAR VIJAYAKUMAR commented on HDFS-13965:
---

Hello  Kitti Nanasi,
We implemented the workaround you suggested.  (Setting the KRB5CCNAME 
environment variable with hadoop user kerberos cache path )
But this causes the hadoop user ticket cache to be renewed as root user 
automatically when ticket cache is about to expire / already expired.  Which 
changes the ownership of the ticket cache from hadoop user to root user.  As a 
result of all this, further kinit logins fails for hadoop user, as ticket cache 
path is owned by root user now. 

Is there anyway to avoid this issue?  What should we do to not run into this 
issue?





20/05/21 21:01:53 DEBUG security.UserGroupInformation: hadoop login
20/05/21 21:01:53 DEBUG security.UserGroupInformation: hadoop login commit
20/05/21 21:01:53 DEBUG security.UserGroupInformation: using kerberos 
user:hal...@cinfin.com
20/05/21 21:01:53 DEBUG security.UserGroupInformation: Using user: 
"hal...@cinfin.com" with name hal...@cinfin.com
20/05/21 21:01:53 DEBUG security.UserGroupInformation: User entry: 
"hal...@cinfin.com"
20/05/21 21:01:53 DEBUG security.UserGroupInformation: Assuming keytab is 
managed externally since logged in from subject.
20/05/21 21:01:53 DEBUG security.UserGroupInformation: UGI 
loginUser:hal...@cinfin.com (auth:KERBEROS)
20/05/21 21:01:53 DEBUG security.UserGroupInformation: Found tgt Ticket (hex) =
 
Client Principal = hal...@cinfin.com
Server Principal = krbtgt/cinfin@cinfin.com
 
Forwardable Ticket true
Forwarded Ticket false
Proxiable Ticket false
Proxy Ticket false
Postdated Ticket false
Renewable Ticket true
Initial Ticket true
Auth Time = Thu May 21 03:09:53 EDT 2020
Start Time = Thu May 21 11:10:09 EDT 2020
End Time = Thu May 21 21:10:09 EDT 2020
Renew Till = Thu May 28 03:09:53 EDT 2020
Client Addresses  Null
20/05/21 21:01:53 DEBUG security.UserGroupInformation: Current time is 
1590109313240
20/05/21 21:01:53 DEBUG security.UserGroupInformation: Next refresh is 
1590102609000
20/05/21 21:01:53 DEBUG security.UserGroupInformation: renewed ticket
20/05/21 21:01:53 DEBUG security.UserGroupInformation: Initiating logout for 
hal...@cinfin.com
20/05/21 21:01:53 DEBUG security.UserGroupInformation: hadoop logout
20/05/21 21:01:53 DEBUG security.UserGroupInformation: Initiating re-login for 
hal...@cinfin.com
20/05/21 21:01:53 DEBUG security.UserGroupInformation: hadoop login
20/05/21 21:01:53 DEBUG security.UserGroupInformation: hadoop login commit


> hadoop.security.kerberos.ticket.cache.path setting is not honored when KMS 
> encryption is enabled.
> -
>
> Key: HDFS-13965
> URL: https://issues.apache.org/jira/browse/HDFS-13965
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: hdfs-client, kms
>Affects Versions: 2.7.3, 2.7.7
>Reporter: LOKESKUMAR VIJAYAKUMAR
>Assignee: Kitti Nanasi
>Priority: Major
>
> _We use the *+hadoop.security.kerberos.ticket.cache.path+* setting to provide 
> a custom kerberos cache path for all hadoop operations to be run as specified 
> user. But this setting is not honored when KMS encryption is enabled._
> _The below program to read a file works when KMS encryption is not enabled, 
> but it fails when the KMS encryption is enabled._
> _Looks like *hadoop.security.kerberos.ticket.cache.path* setting is not 
> honored by *createConnection on KMSClientProvider.java.*_
>  
> HadoopTest.java (CLASSPATH needs to be set to compile and run)
>  
> import java.io.InputStream;
> import java.net.URI;
> import org.apache.hadoop.conf.Configuration;
> import org.apache.hadoop.fs.FileSystem;
> import org.apache.hadoop.fs.Path;
>  
> public class HadoopTest {
>     public static int runRead(String[] args) throws Exception{
>     if (args.length < 3) {
>     System.err.println("HadoopTest hadoop_file_path 
> hadoop_user kerberos_cache");
>     return 1;
>     }
>     Path inputPath = new Path(args[0]);
>     Configuration conf = new Configuration();
>     URI defaultURI = FileSystem.getDefaultUri(conf);
>     
> conf.set("hadoop.security.kerberos.ticket.cache.path",args[2]);
>     FileSystem fs = 
> FileSystem.newInstance(defaultURI,conf,args[1]);
>     InputStream is = fs.open(inputPath);
>     byte[] buffer = new byte[4096];
>     int nr = is.read(buffer);
>     while (nr != -1)
>     {
>     System.out.write(buffer, 0, nr);
>     nr = is.read(buffer);
>     }
>     

[jira] [Commented] (HDFS-15392) DistrbutedFileSystem#concat api can create large number of small blocks

2020-06-08 Thread Wei-Chiu Chuang (Jira)


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

Wei-Chiu Chuang commented on HDFS-15392:


A file with lots of blocks is known to slow down. Hadoop 2.x allows up to a 
million blocks per file. It is because of this reason we reduced the default 
max limit to 100k.

{quote}

dfs.namenode.fs-limits.max-blocks-per-file
1
Maximum number of blocks per file, enforced by the Namenode on
write. This prevents the creation of extremely large files which can
degrade performance.

{quote}

Some suggestions for supportability:

maybe this can be added to fsck. If the number of blocks of a file exceeds a 
certain number, emit a warning. This is pretty easy to do.

Also note that, the append() API has a flag NEW_BLOCK where it forces to start 
a new block even if the current block is not full. You could end up with lots 
of small blocks with this flag too.

It should also be pretty easy to log a warning in NN log when concate()/ 
append() finds the file has more than a certain number of blocks.

A more thoughtful improvement could add a metrics to maintain the the count of 
such badly behaved files.

> DistrbutedFileSystem#concat api can create large number of small blocks
> ---
>
> Key: HDFS-15392
> URL: https://issues.apache.org/jira/browse/HDFS-15392
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: Lokesh Jain
>Priority: Major
>
> DistrbutedFileSystem#concat moves blocks from source to target. If the api is 
> repeatedly used on small files it can create large number of small blocks in 
> the target file. The Jira aims to optimize the api to avoid the issue of 
> small blocks.



--
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-15393) Review of PendingReconstructionBlocks

2020-06-08 Thread Hadoop QA (Jira)


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

Hadoop QA commented on HDFS-15393:
--

| (x) *{color:red}-1 overall{color}* |
\\
\\
|| Vote || Subsystem || Runtime || Comment ||
| {color:blue}0{color} | {color:blue} reexec {color} | {color:blue}  1m 
11s{color} | {color:blue} Docker mode activated. {color} |
|| || || || {color:brown} Prechecks {color} ||
| {color:green}+1{color} | {color:green} dupname {color} | {color:green}  0m  
0s{color} | {color:green} No case conflicting files found. {color} |
| {color:green}+1{color} | {color:green} @author {color} | {color:green}  0m  
0s{color} | {color:green} The patch does not contain any @author tags. {color} |
| {color:green}+1{color} | {color:green} test4tests {color} | {color:green}  0m 
 0s{color} | {color:green} The patch appears to include 1 new or modified test 
files. {color} |
|| || || || {color:brown} trunk Compile Tests {color} ||
| {color:green}+1{color} | {color:green} mvninstall {color} | {color:green} 21m 
31s{color} | {color:green} trunk passed {color} |
| {color:green}+1{color} | {color:green} compile {color} | {color:green}  1m  
6s{color} | {color:green} trunk passed {color} |
| {color:green}+1{color} | {color:green} checkstyle {color} | {color:green}  0m 
48s{color} | {color:green} trunk passed {color} |
| {color:green}+1{color} | {color:green} mvnsite {color} | {color:green}  1m 
15s{color} | {color:green} trunk passed {color} |
| {color:green}+1{color} | {color:green} shadedclient {color} | {color:green} 
17m 37s{color} | {color:green} branch has no errors when building and testing 
our client artifacts. {color} |
| {color:green}+1{color} | {color:green} javadoc {color} | {color:green}  0m 
41s{color} | {color:green} trunk passed {color} |
| {color:blue}0{color} | {color:blue} spotbugs {color} | {color:blue}  3m  
2s{color} | {color:blue} Used deprecated FindBugs config; considering switching 
to SpotBugs. {color} |
| {color:green}+1{color} | {color:green} findbugs {color} | {color:green}  3m  
0s{color} | {color:green} trunk passed {color} |
|| || || || {color:brown} Patch Compile Tests {color} ||
| {color:green}+1{color} | {color:green} mvninstall {color} | {color:green}  1m 
 7s{color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} compile {color} | {color:green}  1m  
3s{color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} javac {color} | {color:green}  1m  
3s{color} | {color:green} the patch passed {color} |
| {color:orange}-0{color} | {color:orange} checkstyle {color} | {color:orange}  
0m 43s{color} | {color:orange} hadoop-hdfs-project/hadoop-hdfs: The patch 
generated 23 new + 126 unchanged - 3 fixed = 149 total (was 129) {color} |
| {color:green}+1{color} | {color:green} mvnsite {color} | {color:green}  1m  
9s{color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} whitespace {color} | {color:green}  0m 
 0s{color} | {color:green} The patch has no whitespace issues. {color} |
| {color:green}+1{color} | {color:green} shadedclient {color} | {color:green} 
15m 26s{color} | {color:green} patch has no errors when building and testing 
our client artifacts. {color} |
| {color:green}+1{color} | {color:green} javadoc {color} | {color:green}  0m 
39s{color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} findbugs {color} | {color:green}  3m  
6s{color} | {color:green} the patch passed {color} |
|| || || || {color:brown} Other Tests {color} ||
| {color:red}-1{color} | {color:red} unit {color} | {color:red}115m 34s{color} 
| {color:red} hadoop-hdfs in the patch passed. {color} |
| {color:green}+1{color} | {color:green} asflicense {color} | {color:green}  0m 
36s{color} | {color:green} The patch does not generate ASF License warnings. 
{color} |
| {color:black}{color} | {color:black} {color} | {color:black}187m 45s{color} | 
{color:black} {color} |
\\
\\
|| Reason || Tests ||
| Failed junit tests | hadoop.hdfs.server.namenode.TestDiskspaceQuotaUpdate |
|   | hadoop.hdfs.server.sps.TestExternalStoragePolicySatisfier |
|   | hadoop.hdfs.TestGetFileChecksum |
|   | hadoop.hdfs.server.datanode.TestBPOfferService |
|   | hadoop.hdfs.TestStripedFileAppend |
|   | hadoop.hdfs.server.blockmanagement.TestBlockTokenWithDFSStriped |
|   | hadoop.hdfs.server.namenode.ha.TestFailureToReadEdits |
|   | hadoop.hdfs.server.namenode.TestNameNodeRetryCacheMetrics |
|   | hadoop.hdfs.TestReconstructStripedFile |
|   | hadoop.hdfs.TestReconstructStripedFileWithRandomECPolicy |
\\
\\
|| Subsystem || Report/Notes ||
| Docker | ClientAPI=1.40 ServerAPI=1.40 base: 
https://builds.apache.org/job/hadoop-multibranch/job/PR-2055/2/artifact/out/Dockerfile
 |
| GITHUB PR | https://github.com/apache/hadoop/pull/2055 |
| JIRA Issue | HDFS-15393 |
| Optional Tests | dupname asflicense 

[jira] [Resolved] (HDFS-15287) HDFS rollingupgrade prepare never finishes

2020-06-08 Thread Kihwal Lee (Jira)


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

Kihwal Lee resolved HDFS-15287.
---
Resolution: Duplicate

> HDFS rollingupgrade prepare never finishes
> --
>
> Key: HDFS-15287
> URL: https://issues.apache.org/jira/browse/HDFS-15287
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: namenode
>Affects Versions: 2.10.0, 3.3.0
>Reporter: Kihwal Lee
>Priority: Major
>
> After HDFS-12979, the prepare step of rolling upgrade does not work. This is 
> because it added additional check for sufficient time passing since last 
> checkpoint. Since RU rollback image creation and upload can happen any time, 
> uploading of rollback image never succeeds. For a new cluster deployed for 
> testing, it might work since it never checkpointed before.
> It was found that this check is disabled for unit tests, defeating the very 
> purpose of testing.



--
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-15273) CacheReplicationMonitor hold lock for long time and lead to NN out of service

2020-06-08 Thread Hadoop QA (Jira)


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

Hadoop QA commented on HDFS-15273:
--

| (x) *{color:red}-1 overall{color}* |
\\
\\
|| Vote || Subsystem || Runtime || Comment ||
| {color:blue}0{color} | {color:blue} reexec {color} | {color:blue}  0m 
36s{color} | {color:blue} Docker mode activated. {color} |
|| || || || {color:brown} Prechecks {color} ||
| {color:green}+1{color} | {color:green} dupname {color} | {color:green}  0m  
0s{color} | {color:green} No case conflicting files found. {color} |
| {color:green}+1{color} | {color:green} @author {color} | {color:green}  0m  
0s{color} | {color:green} The patch does not contain any @author tags. {color} |
| {color:red}-1{color} | {color:red} test4tests {color} | {color:red}  0m  
0s{color} | {color:red} The patch doesn't appear to include any new or modified 
tests. Please justify why no new tests are needed for this patch. Also please 
list what manual steps were performed to verify this patch. {color} |
|| || || || {color:brown} trunk Compile Tests {color} ||
| {color:green}+1{color} | {color:green} mvninstall {color} | {color:green} 28m 
32s{color} | {color:green} trunk passed {color} |
| {color:green}+1{color} | {color:green} compile {color} | {color:green}  1m 
10s{color} | {color:green} trunk passed {color} |
| {color:green}+1{color} | {color:green} checkstyle {color} | {color:green}  0m 
59s{color} | {color:green} trunk passed {color} |
| {color:green}+1{color} | {color:green} mvnsite {color} | {color:green}  1m 
16s{color} | {color:green} trunk passed {color} |
| {color:green}+1{color} | {color:green} shadedclient {color} | {color:green} 
16m 34s{color} | {color:green} branch has no errors when building and testing 
our client artifacts. {color} |
| {color:green}+1{color} | {color:green} javadoc {color} | {color:green}  0m 
45s{color} | {color:green} trunk passed {color} |
| {color:blue}0{color} | {color:blue} spotbugs {color} | {color:blue}  2m 
57s{color} | {color:blue} Used deprecated FindBugs config; considering 
switching to SpotBugs. {color} |
| {color:green}+1{color} | {color:green} findbugs {color} | {color:green}  2m 
53s{color} | {color:green} trunk passed {color} |
|| || || || {color:brown} Patch Compile Tests {color} ||
| {color:green}+1{color} | {color:green} mvninstall {color} | {color:green}  1m 
12s{color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} compile {color} | {color:green}  1m  
2s{color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} javac {color} | {color:green}  1m  
2s{color} | {color:green} the patch passed {color} |
| {color:orange}-0{color} | {color:orange} checkstyle {color} | {color:orange}  
0m 48s{color} | {color:orange} hadoop-hdfs-project/hadoop-hdfs: The patch 
generated 3 new + 493 unchanged - 0 fixed = 496 total (was 493) {color} |
| {color:green}+1{color} | {color:green} mvnsite {color} | {color:green}  1m  
7s{color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} whitespace {color} | {color:green}  0m 
 0s{color} | {color:green} The patch has no whitespace issues. {color} |
| {color:green}+1{color} | {color:green} shadedclient {color} | {color:green} 
15m 27s{color} | {color:green} patch has no errors when building and testing 
our client artifacts. {color} |
| {color:green}+1{color} | {color:green} javadoc {color} | {color:green}  0m 
44s{color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} findbugs {color} | {color:green}  3m 
52s{color} | {color:green} the patch passed {color} |
|| || || || {color:brown} Other Tests {color} ||
| {color:red}-1{color} | {color:red} unit {color} | {color:red}137m  2s{color} 
| {color:red} hadoop-hdfs in the patch passed. {color} |
| {color:green}+1{color} | {color:green} asflicense {color} | {color:green}  0m 
48s{color} | {color:green} The patch does not generate ASF License warnings. 
{color} |
| {color:black}{color} | {color:black} {color} | {color:black}215m  3s{color} | 
{color:black} {color} |
\\
\\
|| Reason || Tests ||
| Failed junit tests | 
hadoop.hdfs.TestDFSStripedOutputStreamWithFailureWithRandomECPolicy |
|   | hadoop.hdfs.TestFileChecksumCompositeCrc |
|   | hadoop.hdfs.TestDFSStorageStateRecovery |
|   | hadoop.hdfs.TestReconstructStripedFileWithRandomECPolicy |
|   | hadoop.hdfs.tools.TestECAdmin |
|   | hadoop.hdfs.tools.TestViewFSStoragePolicyCommands |
|   | hadoop.hdfs.TestReconstructStripedFile |
|   | hadoop.hdfs.server.namenode.TestNameNodeRetryCacheMetrics |
|   | hadoop.hdfs.TestDecommissionWithStripedBackoffMonitor |
|   | hadoop.hdfs.TestDecommissionWithBackoffMonitor |
|   | hadoop.hdfs.TestFileAppend4 |
|   | hadoop.hdfs.TestErasureCodingExerciseAPIs |
|   | hadoop.hdfs.TestReadStripedFileWithMissingBlocks |
|   | hadoop.hdfs.tools.TestDFSAdmin |
|   | 

[jira] [Updated] (HDFS-15372) Files in snapshots no longer see attribute provider permissions

2020-06-08 Thread Stephen O'Donnell (Jira)


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

Stephen O'Donnell updated HDFS-15372:
-
Attachment: HDFS-15372.003.patch

> Files in snapshots no longer see attribute provider permissions
> ---
>
> Key: HDFS-15372
> URL: https://issues.apache.org/jira/browse/HDFS-15372
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: Stephen O'Donnell
>Assignee: Stephen O'Donnell
>Priority: Major
> Attachments: HDFS-15372.001.patch, HDFS-15372.002.patch, 
> HDFS-15372.003.patch
>
>
> Given a cluster with an authorization provider configured (eg Sentry) and the 
> paths covered by the provider are snapshotable, there was a change in 
> behaviour in how the provider permissions and ACLs are applied to files in 
> snapshots between the 2.x branch and Hadoop 3.0.
> Eg, if we have the snapshotable path /data, which is Sentry managed. The ACLs 
> below are provided by Sentry:
> {code}
> hadoop fs -getfacl -R /data
> # file: /data
> # owner: hive
> # group: hive
> user::rwx
> group::rwx
> other::--x
> # file: /data/tab1
> # owner: hive
> # group: hive
> user::rwx
> group::---
> group:flume:rwx
> user:hive:rwx
> group:hive:rwx
> group:testgroup:rwx
> mask::rwx
> other::--x
> /data/tab1
> {code}
> After taking a snapshot, the files in the snapshot do not see the provider 
> permissions:
> {code}
> hadoop fs -getfacl -R /data/.snapshot
> # file: /data/.snapshot
> # owner: 
> # group: 
> user::rwx
> group::rwx
> other::rwx
> # file: /data/.snapshot/snap1
> # owner: hive
> # group: hive
> user::rwx
> group::rwx
> other::--x
> # file: /data/.snapshot/snap1/tab1
> # owner: hive
> # group: hive
> user::rwx
> group::rwx
> other::--x
> {code}
> However pre-Hadoop 3.0 (when the attribute provider etc was extensively 
> refactored) snapshots did get the provider permissions.
> The reason is this code in FSDirectory.java which ultimately calls the 
> attribute provider and passes the path we want permissions for:
> {code}
>   INodeAttributes getAttributes(INodesInPath iip)
>   throws IOException {
> INode node = FSDirectory.resolveLastINode(iip);
> int snapshot = iip.getPathSnapshotId();
> INodeAttributes nodeAttrs = node.getSnapshotINode(snapshot);
> UserGroupInformation ugi = NameNode.getRemoteUser();
> INodeAttributeProvider ap = this.getUserFilteredAttributeProvider(ugi);
> if (ap != null) {
>   // permission checking sends the full components array including the
>   // first empty component for the root.  however file status
>   // related calls are expected to strip out the root component according
>   // to TestINodeAttributeProvider.
>   byte[][] components = iip.getPathComponents();
>   components = Arrays.copyOfRange(components, 1, components.length);
>   nodeAttrs = ap.getAttributes(components, nodeAttrs);
> }
> return nodeAttrs;
>   }
> {code}
> The line:
> {code}
> INode node = FSDirectory.resolveLastINode(iip);
> {code}
> Picks the last resolved Inode and if you then call node.getPathComponents, 
> for a path like '/data/.snapshot/snap1/tab1' it will return /data/tab1. It 
> resolves the snapshot path to its original location, but its still the 
> snapshot inode.
> However the logic passes 'iip.getPathComponents' which returns 
> "/user/.snapshot/snap1/tab" to the provider.
> The pre Hadoop 3.0 code passes the inode directly to the provider, and hence 
> it only ever sees the path as "/user/data/tab1".
> It is debatable which path should be passed to the provider - 
> /user/.snapshot/snap1/tab or /data/tab1 in the case of snapshots. However as 
> the behaviour has changed I feel we should ensure the old behaviour is 
> retained.
> It would also be fairly easy to provide a config switch so the provider gets 
> the full snapshot path or the resolved path.



--
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-15372) Files in snapshots no longer see attribute provider permissions

2020-06-08 Thread Hadoop QA (Jira)


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

Hadoop QA commented on HDFS-15372:
--

| (x) *{color:red}-1 overall{color}* |
\\
\\
|| Vote || Subsystem || Runtime || Comment ||
| {color:blue}0{color} | {color:blue} reexec {color} | {color:blue}  0m 
50s{color} | {color:blue} Docker mode activated. {color} |
|| || || || {color:brown} Prechecks {color} ||
| {color:green}+1{color} | {color:green} dupname {color} | {color:green}  0m  
1s{color} | {color:green} No case conflicting files found. {color} |
| {color:green}+1{color} | {color:green} @author {color} | {color:green}  0m  
0s{color} | {color:green} The patch does not contain any @author tags. {color} |
| {color:green}+1{color} | {color:green} test4tests {color} | {color:green}  0m 
 0s{color} | {color:green} The patch appears to include 1 new or modified test 
files. {color} |
|| || || || {color:brown} trunk Compile Tests {color} ||
| {color:green}+1{color} | {color:green} mvninstall {color} | {color:green} 20m 
37s{color} | {color:green} trunk passed {color} |
| {color:green}+1{color} | {color:green} compile {color} | {color:green}  1m 
10s{color} | {color:green} trunk passed {color} |
| {color:green}+1{color} | {color:green} checkstyle {color} | {color:green}  0m 
51s{color} | {color:green} trunk passed {color} |
| {color:green}+1{color} | {color:green} mvnsite {color} | {color:green}  1m 
18s{color} | {color:green} trunk passed {color} |
| {color:green}+1{color} | {color:green} shadedclient {color} | {color:green} 
16m  7s{color} | {color:green} branch has no errors when building and testing 
our client artifacts. {color} |
| {color:green}+1{color} | {color:green} javadoc {color} | {color:green}  0m 
46s{color} | {color:green} trunk passed {color} |
| {color:blue}0{color} | {color:blue} spotbugs {color} | {color:blue}  2m 
53s{color} | {color:blue} Used deprecated FindBugs config; considering 
switching to SpotBugs. {color} |
| {color:green}+1{color} | {color:green} findbugs {color} | {color:green}  2m 
51s{color} | {color:green} trunk passed {color} |
|| || || || {color:brown} Patch Compile Tests {color} ||
| {color:green}+1{color} | {color:green} mvninstall {color} | {color:green}  1m 
 7s{color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} compile {color} | {color:green}  1m  
3s{color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} javac {color} | {color:green}  1m  
3s{color} | {color:green} the patch passed {color} |
| {color:orange}-0{color} | {color:orange} checkstyle {color} | {color:orange}  
0m 43s{color} | {color:orange} hadoop-hdfs-project/hadoop-hdfs: The patch 
generated 5 new + 88 unchanged - 1 fixed = 93 total (was 89) {color} |
| {color:green}+1{color} | {color:green} mvnsite {color} | {color:green}  1m  
9s{color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} whitespace {color} | {color:green}  0m 
 0s{color} | {color:green} The patch has no whitespace issues. {color} |
| {color:green}+1{color} | {color:green} shadedclient {color} | {color:green} 
13m 44s{color} | {color:green} patch has no errors when building and testing 
our client artifacts. {color} |
| {color:green}+1{color} | {color:green} javadoc {color} | {color:green}  0m 
43s{color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} findbugs {color} | {color:green}  2m 
53s{color} | {color:green} the patch passed {color} |
|| || || || {color:brown} Other Tests {color} ||
| {color:red}-1{color} | {color:red} unit {color} | {color:red} 98m 55s{color} 
| {color:red} hadoop-hdfs in the patch passed. {color} |
| {color:green}+1{color} | {color:green} asflicense {color} | {color:green}  0m 
43s{color} | {color:green} The patch does not generate ASF License warnings. 
{color} |
| {color:black}{color} | {color:black} {color} | {color:black}165m 43s{color} | 
{color:black} {color} |
\\
\\
|| Reason || Tests ||
| Failed junit tests | 
hadoop.hdfs.server.sps.TestExternalStoragePolicySatisfier |
|   | hadoop.hdfs.TestGetFileChecksum |
|   | hadoop.hdfs.TestReconstructStripedFile |
|   | hadoop.hdfs.TestReconstructStripedFileWithRandomECPolicy |
|   | hadoop.hdfs.server.datanode.TestDataNodeUUID |
|   | hadoop.hdfs.server.blockmanagement.TestBlockTokenWithDFSStriped |
\\
\\
|| Subsystem || Report/Notes ||
| Docker | ClientAPI=1.40 ServerAPI=1.40 base: 
https://builds.apache.org/job/PreCommit-HDFS-Build/29405/artifact/out/Dockerfile
 |
| JIRA Issue | HDFS-15372 |
| JIRA Patch URL | 
https://issues.apache.org/jira/secure/attachment/13005136/HDFS-15372.002.patch |
| Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite 
unit shadedclient findbugs checkstyle |
| uname | Linux 44db74c4268b 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 
11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux 

[jira] [Commented] (HDFS-15290) NPE in HttpServer during NameNode startup

2020-06-08 Thread Konstantin Shvachko (Jira)


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

Konstantin Shvachko commented on HDFS-15290:


During NameNode startup the fsImage is loaded in {{NameNode()}} constructor. 
The constructor calls {{nitialize()}}, which first starts httpServer by 
{{startHttpServer(conf)}}, then loads namesystem. {{loadNamesystem()}} is the 
one which after loading the image will set the {{this.namesystem}} member. 
Until then both {{namesystem}} and {{fsImage}} in {{ServletContext}} are NULLs. 
So {{ImageServlet}} should reject GETs and PUTs while the NameNode is 
initializing.

> NPE in HttpServer during NameNode startup
> -
>
> Key: HDFS-15290
> URL: https://issues.apache.org/jira/browse/HDFS-15290
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: namenode
>Affects Versions: 2.7.8
>Reporter: Konstantin Shvachko
>Priority: Major
>
> When NameNode starts it first starts HttpServer, then starts loading fsImage 
> and edits. While loading the namesystem field in NameNode is null. I saw that 
> a StandbyNode sends a checkpoint request, which fails with NPE because 
> NNStorage is not instantiated yet.
> We should check the NameNode startup status before accepting checkpoint 
> requests.



--
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-15372) Files in snapshots no longer see attribute provider permissions

2020-06-08 Thread hemanthboyina (Jira)


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

hemanthboyina commented on HDFS-15372:
--

thanks for the work [~sodonnell]  , overall the  code looks good 

some comments 

1) AFAIK Only the Snapshot INode will have same Id as of INode's Parent Id , so 
you can use something like iip.getINode(iip.getLength-1).getId() != 
iip.getINode(iip.length()-1).getParent().getId()  instead of checking 
!iip.isDotSnapshotDirPrefix() 

2) In FSPermissionChecker we can get inodes path components by using 
INodesInPath#fromINode , but this method requires rootDir , which you have to 
get when FSDirectory calls FSPermissionChecker#checkTraverse or any other 
better way , upon this changes  you can do same as you have done for 
FSDirectory#getAttributes

kindly correct me if  i am wrong , thanks

> Files in snapshots no longer see attribute provider permissions
> ---
>
> Key: HDFS-15372
> URL: https://issues.apache.org/jira/browse/HDFS-15372
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: Stephen O'Donnell
>Assignee: Stephen O'Donnell
>Priority: Major
> Attachments: HDFS-15372.001.patch, HDFS-15372.002.patch
>
>
> Given a cluster with an authorization provider configured (eg Sentry) and the 
> paths covered by the provider are snapshotable, there was a change in 
> behaviour in how the provider permissions and ACLs are applied to files in 
> snapshots between the 2.x branch and Hadoop 3.0.
> Eg, if we have the snapshotable path /data, which is Sentry managed. The ACLs 
> below are provided by Sentry:
> {code}
> hadoop fs -getfacl -R /data
> # file: /data
> # owner: hive
> # group: hive
> user::rwx
> group::rwx
> other::--x
> # file: /data/tab1
> # owner: hive
> # group: hive
> user::rwx
> group::---
> group:flume:rwx
> user:hive:rwx
> group:hive:rwx
> group:testgroup:rwx
> mask::rwx
> other::--x
> /data/tab1
> {code}
> After taking a snapshot, the files in the snapshot do not see the provider 
> permissions:
> {code}
> hadoop fs -getfacl -R /data/.snapshot
> # file: /data/.snapshot
> # owner: 
> # group: 
> user::rwx
> group::rwx
> other::rwx
> # file: /data/.snapshot/snap1
> # owner: hive
> # group: hive
> user::rwx
> group::rwx
> other::--x
> # file: /data/.snapshot/snap1/tab1
> # owner: hive
> # group: hive
> user::rwx
> group::rwx
> other::--x
> {code}
> However pre-Hadoop 3.0 (when the attribute provider etc was extensively 
> refactored) snapshots did get the provider permissions.
> The reason is this code in FSDirectory.java which ultimately calls the 
> attribute provider and passes the path we want permissions for:
> {code}
>   INodeAttributes getAttributes(INodesInPath iip)
>   throws IOException {
> INode node = FSDirectory.resolveLastINode(iip);
> int snapshot = iip.getPathSnapshotId();
> INodeAttributes nodeAttrs = node.getSnapshotINode(snapshot);
> UserGroupInformation ugi = NameNode.getRemoteUser();
> INodeAttributeProvider ap = this.getUserFilteredAttributeProvider(ugi);
> if (ap != null) {
>   // permission checking sends the full components array including the
>   // first empty component for the root.  however file status
>   // related calls are expected to strip out the root component according
>   // to TestINodeAttributeProvider.
>   byte[][] components = iip.getPathComponents();
>   components = Arrays.copyOfRange(components, 1, components.length);
>   nodeAttrs = ap.getAttributes(components, nodeAttrs);
> }
> return nodeAttrs;
>   }
> {code}
> The line:
> {code}
> INode node = FSDirectory.resolveLastINode(iip);
> {code}
> Picks the last resolved Inode and if you then call node.getPathComponents, 
> for a path like '/data/.snapshot/snap1/tab1' it will return /data/tab1. It 
> resolves the snapshot path to its original location, but its still the 
> snapshot inode.
> However the logic passes 'iip.getPathComponents' which returns 
> "/user/.snapshot/snap1/tab" to the provider.
> The pre Hadoop 3.0 code passes the inode directly to the provider, and hence 
> it only ever sees the path as "/user/data/tab1".
> It is debatable which path should be passed to the provider - 
> /user/.snapshot/snap1/tab or /data/tab1 in the case of snapshots. However as 
> the behaviour has changed I feel we should ensure the old behaviour is 
> retained.
> It would also be fairly easy to provide a config switch so the provider gets 
> the full snapshot path or the resolved path.



--
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-15087) RBF: Balance/Rename across federation namespaces

2020-06-08 Thread Uma Maheswara Rao G (Jira)


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

Uma Maheswara Rao G commented on HDFS-15087:


Hi [~LiJinglun]  and All, great to see this work!

I have not went through the design doc or code yet, So, throwing some questions 
on very high level:  Is that snapshots mandatory for this feature to work?

If it's mandatory, would it be possible to think and make it as optional and 
have alternative thoughts to get the diff?

If it's not mandatory, that's great. Can this be a generic rename function even 
across different hadoop compatible file systems? 

CC: [~weichiu] 

> RBF: Balance/Rename across federation namespaces
> 
>
> Key: HDFS-15087
> URL: https://issues.apache.org/jira/browse/HDFS-15087
> Project: Hadoop HDFS
>  Issue Type: New Feature
>Reporter: Jinglun
>Assignee: Jinglun
>Priority: Major
> Attachments: HDFS-15087.initial.patch, HFR_Rename Across Federation 
> Namespaces.pdf
>
>
> The Xiaomi storage team has developed a new feature called HFR(HDFS 
> Federation Rename) that enables us to do balance/rename across federation 
> namespaces. The idea is to first move the meta to the dst NameNode and then 
> link all the replicas. It has been working in our largest production cluster 
> for 2 months. We use it to balance the namespaces. It turns out HFR is fast 
> and flexible. The detail could be found in the design doc. 
> Looking forward to a lively discussion.



--
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-15397) Blocks recovery could overwhelm KDC when using Kerberos for authentication

2020-06-08 Thread Xiaoqiao He (Jira)
Xiaoqiao He created HDFS-15397:
--

 Summary: Blocks recovery could overwhelm KDC when using Kerberos 
for authentication
 Key: HDFS-15397
 URL: https://issues.apache.org/jira/browse/HDFS-15397
 Project: Hadoop HDFS
  Issue Type: Improvement
  Components: datanode
Reporter: Xiaoqiao He


It could overwhelm KDC if enable Kerberos for authentication when there are 
large number of blocks are triggered to recovery.
Consider killing one running application which opens large number files and 
writing, it will trigger block recovery, If `recoverblocks` commands are not 
under control, there will be huge number pairs between two DataNode need to 
auth when recovery, then it will overwhelm KDC if we enable Kerberos for 
authentication.
I would like to attach more information about the corner case.
Technically, we should control the concurrency when there are large number 
blocks need to recovery IMO.



--
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-15273) CacheReplicationMonitor hold lock for long time and lead to NN out of service

2020-06-08 Thread Xiaoqiao He (Jira)


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

Xiaoqiao He updated HDFS-15273:
---
Attachment: HDFS-15273.001.patch
Status: Patch Available  (was: Open)

submit demo patch and trigger jenkins.

> CacheReplicationMonitor hold lock for long time and lead to NN out of service
> -
>
> Key: HDFS-15273
> URL: https://issues.apache.org/jira/browse/HDFS-15273
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: caching, namenode
>Reporter: Xiaoqiao He
>Assignee: Xiaoqiao He
>Priority: Major
> Attachments: HDFS-15273.001.patch
>
>
> CacheReplicationMonitor scan Cache Directives and Cached BlockMap 
> periodically. If we add more and more cache directives, 
> CacheReplicationMonitor will cost very long time to rescan all of cache 
> directives and cache blocks. Meanwhile, scan operation hold global write 
> lock, during scan period, NameNode could not process other request.
> So I think we should warn this risk to end user who turn on CacheManager 
> feature before improve this implement.
> {code:java}
>   private void rescan() throws InterruptedException {
> scannedDirectives = 0;
> scannedBlocks = 0;
> try {
>   namesystem.writeLock();
>   try {
> lock.lock();
> if (shutdown) {
>   throw new InterruptedException("CacheReplicationMonitor was " +
>   "shut down.");
> }
> curScanCount = completedScanCount + 1;
>   } finally {
> lock.unlock();
>   }
>   resetStatistics();
>   rescanCacheDirectives();
>   rescanCachedBlockMap();
>   blockManager.getDatanodeManager().resetLastCachingDirectiveSentTime();
> } finally {
>   namesystem.writeUnlock();
> }
>   }
> {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-15372) Files in snapshots no longer see attribute provider permissions

2020-06-08 Thread Stephen O'Donnell (Jira)


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

Stephen O'Donnell commented on HDFS-15372:
--

[~hemanthboyina] I have uploaded a new patch. Please have a look and if you 
have any ideas of a better way of handing the change in FSPermissionChecker, I 
would be pleased to hear it. It is not as clean as the change for FSDirectory.

> Files in snapshots no longer see attribute provider permissions
> ---
>
> Key: HDFS-15372
> URL: https://issues.apache.org/jira/browse/HDFS-15372
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: Stephen O'Donnell
>Assignee: Stephen O'Donnell
>Priority: Major
> Attachments: HDFS-15372.001.patch, HDFS-15372.002.patch
>
>
> Given a cluster with an authorization provider configured (eg Sentry) and the 
> paths covered by the provider are snapshotable, there was a change in 
> behaviour in how the provider permissions and ACLs are applied to files in 
> snapshots between the 2.x branch and Hadoop 3.0.
> Eg, if we have the snapshotable path /data, which is Sentry managed. The ACLs 
> below are provided by Sentry:
> {code}
> hadoop fs -getfacl -R /data
> # file: /data
> # owner: hive
> # group: hive
> user::rwx
> group::rwx
> other::--x
> # file: /data/tab1
> # owner: hive
> # group: hive
> user::rwx
> group::---
> group:flume:rwx
> user:hive:rwx
> group:hive:rwx
> group:testgroup:rwx
> mask::rwx
> other::--x
> /data/tab1
> {code}
> After taking a snapshot, the files in the snapshot do not see the provider 
> permissions:
> {code}
> hadoop fs -getfacl -R /data/.snapshot
> # file: /data/.snapshot
> # owner: 
> # group: 
> user::rwx
> group::rwx
> other::rwx
> # file: /data/.snapshot/snap1
> # owner: hive
> # group: hive
> user::rwx
> group::rwx
> other::--x
> # file: /data/.snapshot/snap1/tab1
> # owner: hive
> # group: hive
> user::rwx
> group::rwx
> other::--x
> {code}
> However pre-Hadoop 3.0 (when the attribute provider etc was extensively 
> refactored) snapshots did get the provider permissions.
> The reason is this code in FSDirectory.java which ultimately calls the 
> attribute provider and passes the path we want permissions for:
> {code}
>   INodeAttributes getAttributes(INodesInPath iip)
>   throws IOException {
> INode node = FSDirectory.resolveLastINode(iip);
> int snapshot = iip.getPathSnapshotId();
> INodeAttributes nodeAttrs = node.getSnapshotINode(snapshot);
> UserGroupInformation ugi = NameNode.getRemoteUser();
> INodeAttributeProvider ap = this.getUserFilteredAttributeProvider(ugi);
> if (ap != null) {
>   // permission checking sends the full components array including the
>   // first empty component for the root.  however file status
>   // related calls are expected to strip out the root component according
>   // to TestINodeAttributeProvider.
>   byte[][] components = iip.getPathComponents();
>   components = Arrays.copyOfRange(components, 1, components.length);
>   nodeAttrs = ap.getAttributes(components, nodeAttrs);
> }
> return nodeAttrs;
>   }
> {code}
> The line:
> {code}
> INode node = FSDirectory.resolveLastINode(iip);
> {code}
> Picks the last resolved Inode and if you then call node.getPathComponents, 
> for a path like '/data/.snapshot/snap1/tab1' it will return /data/tab1. It 
> resolves the snapshot path to its original location, but its still the 
> snapshot inode.
> However the logic passes 'iip.getPathComponents' which returns 
> "/user/.snapshot/snap1/tab" to the provider.
> The pre Hadoop 3.0 code passes the inode directly to the provider, and hence 
> it only ever sees the path as "/user/data/tab1".
> It is debatable which path should be passed to the provider - 
> /user/.snapshot/snap1/tab or /data/tab1 in the case of snapshots. However as 
> the behaviour has changed I feel we should ensure the old behaviour is 
> retained.
> It would also be fairly easy to provide a config switch so the provider gets 
> the full snapshot path or the resolved path.



--
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-15372) Files in snapshots no longer see attribute provider permissions

2020-06-08 Thread Stephen O'Donnell (Jira)


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

Stephen O'Donnell updated HDFS-15372:
-
Attachment: HDFS-15372.002.patch

> Files in snapshots no longer see attribute provider permissions
> ---
>
> Key: HDFS-15372
> URL: https://issues.apache.org/jira/browse/HDFS-15372
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: Stephen O'Donnell
>Assignee: Stephen O'Donnell
>Priority: Major
> Attachments: HDFS-15372.001.patch, HDFS-15372.002.patch
>
>
> Given a cluster with an authorization provider configured (eg Sentry) and the 
> paths covered by the provider are snapshotable, there was a change in 
> behaviour in how the provider permissions and ACLs are applied to files in 
> snapshots between the 2.x branch and Hadoop 3.0.
> Eg, if we have the snapshotable path /data, which is Sentry managed. The ACLs 
> below are provided by Sentry:
> {code}
> hadoop fs -getfacl -R /data
> # file: /data
> # owner: hive
> # group: hive
> user::rwx
> group::rwx
> other::--x
> # file: /data/tab1
> # owner: hive
> # group: hive
> user::rwx
> group::---
> group:flume:rwx
> user:hive:rwx
> group:hive:rwx
> group:testgroup:rwx
> mask::rwx
> other::--x
> /data/tab1
> {code}
> After taking a snapshot, the files in the snapshot do not see the provider 
> permissions:
> {code}
> hadoop fs -getfacl -R /data/.snapshot
> # file: /data/.snapshot
> # owner: 
> # group: 
> user::rwx
> group::rwx
> other::rwx
> # file: /data/.snapshot/snap1
> # owner: hive
> # group: hive
> user::rwx
> group::rwx
> other::--x
> # file: /data/.snapshot/snap1/tab1
> # owner: hive
> # group: hive
> user::rwx
> group::rwx
> other::--x
> {code}
> However pre-Hadoop 3.0 (when the attribute provider etc was extensively 
> refactored) snapshots did get the provider permissions.
> The reason is this code in FSDirectory.java which ultimately calls the 
> attribute provider and passes the path we want permissions for:
> {code}
>   INodeAttributes getAttributes(INodesInPath iip)
>   throws IOException {
> INode node = FSDirectory.resolveLastINode(iip);
> int snapshot = iip.getPathSnapshotId();
> INodeAttributes nodeAttrs = node.getSnapshotINode(snapshot);
> UserGroupInformation ugi = NameNode.getRemoteUser();
> INodeAttributeProvider ap = this.getUserFilteredAttributeProvider(ugi);
> if (ap != null) {
>   // permission checking sends the full components array including the
>   // first empty component for the root.  however file status
>   // related calls are expected to strip out the root component according
>   // to TestINodeAttributeProvider.
>   byte[][] components = iip.getPathComponents();
>   components = Arrays.copyOfRange(components, 1, components.length);
>   nodeAttrs = ap.getAttributes(components, nodeAttrs);
> }
> return nodeAttrs;
>   }
> {code}
> The line:
> {code}
> INode node = FSDirectory.resolveLastINode(iip);
> {code}
> Picks the last resolved Inode and if you then call node.getPathComponents, 
> for a path like '/data/.snapshot/snap1/tab1' it will return /data/tab1. It 
> resolves the snapshot path to its original location, but its still the 
> snapshot inode.
> However the logic passes 'iip.getPathComponents' which returns 
> "/user/.snapshot/snap1/tab" to the provider.
> The pre Hadoop 3.0 code passes the inode directly to the provider, and hence 
> it only ever sees the path as "/user/data/tab1".
> It is debatable which path should be passed to the provider - 
> /user/.snapshot/snap1/tab or /data/tab1 in the case of snapshots. However as 
> the behaviour has changed I feel we should ensure the old behaviour is 
> retained.
> It would also be fairly easy to provide a config switch so the provider gets 
> the full snapshot path or the resolved path.



--
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-15346) RBF: DistCpFedBalance implementation

2020-06-08 Thread Jinglun (Jira)


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

Jinglun commented on HDFS-15346:


Hi [~linyiqun], thanks your great comments and valuable suggestions !  I'll 
need some time to shoot all of them. So let me respond to the question first.

 
{quote}Here we reset permission to 0, that means no any operation is allowed? 
Is this expected, why not is 400 (only allow read)? The comment said that 
'cancelling the x permission of the source path.' makes me confused.
{quote}
Yes here we reset the permission to 0. Both read and write in the source path 
and all its sub-paths are denied. As far as I know all the read operations need 
to check its parents' execution permission. So setting to 400 can't make it 
only allowing read. We still can't read its sub-paths. I think the only way to 
make it 'only allowing read' is to recursively reduce each directory's 
permission to 555. Reduce permission means: if the original permission is 777 
then change it to 555. If the original permission is 700 then make it to 500. 
Saving all the directories' permissions is very expensive. A better way may be 
letting the NameNode to support 'readonly-directory'. I think we can first 
using the '0 permission' way to make sure the data is consistent. Then start a 
sub-task to enable the NameNode 'readonly-directory'. Finally change this to 
the NameNode 'readonly-directory'.

> RBF: DistCpFedBalance implementation
> 
>
> Key: HDFS-15346
> URL: https://issues.apache.org/jira/browse/HDFS-15346
> Project: Hadoop HDFS
>  Issue Type: Sub-task
>Reporter: Jinglun
>Assignee: Jinglun
>Priority: Major
> Attachments: HDFS-15346.001.patch, HDFS-15346.002.patch, 
> HDFS-15346.003.patch, HDFS-15346.004.patch, HDFS-15346.005.patch, 
> HDFS-15346.006.patch, HDFS-15346.007.patch
>
>
> Patch in HDFS-15294 is too big to review so we split it into 2 patches. This 
> is the second one. Detail can be found at HDFS-15294.



--
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-15390) client fails forever when namenode ipaddr changed

2020-06-08 Thread hemanthboyina (Jira)


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

hemanthboyina commented on HDFS-15390:
--

[~seanlook] you can click on the More option specified under the Jira title , 
In More you can select MOVE 

> client fails forever when namenode ipaddr changed
> -
>
> Key: HDFS-15390
> URL: https://issues.apache.org/jira/browse/HDFS-15390
> Project: Hadoop HDFS
>  Issue Type: Bug
>Affects Versions: 2.10.0, 2.9.2, 3.2.1
>Reporter: Sean Chow
>Priority: Major
> Attachments: HDFS-15390.01.patch
>
>
> For machine replacement, I replace my standby namenode with a new ipaddr and 
> keep the same hostname. Also update the client's hosts to make it resolve 
> correctly
> When I try to run failover to transite the new namenode(let's say nn2), the 
> client will fail to read or write forever until it's restarted.
> That make yarn nodemanager in sick state. Even the new tasks will encounter 
> this exception  too. Until all nodemanager restart.
>  
> {code:java}
> 20/06/02 15:12:25 WARN ipc.Client: Address change detected. Old: 
> nn2-192-168-1-100/192.168.1.100:9000 New: nn2-192-168-1-100/192.168.1.200:9000
> 20/06/02 15:12:25 DEBUG ipc.Client: closing ipc connection to 
> nn2-192-168-1-100/192.168.1.200:9000: Connection refused
> java.net.ConnectException: Connection refused
> at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
> at 
> sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
> at 
> org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.java:206)
> at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:530)
> at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:494)
> at 
> org.apache.hadoop.ipc.Client$Connection.setupConnection(Client.java:608)
> at 
> org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:707)
> at 
> org.apache.hadoop.ipc.Client$Connection.access$2800(Client.java:368)
> at org.apache.hadoop.ipc.Client.getConnection(Client.java:1517)
> at org.apache.hadoop.ipc.Client.call(Client.java:1440)
> at org.apache.hadoop.ipc.Client.call(Client.java:1401)
> at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:232)
> at com.sun.proxy.$Proxy9.addBlock(Unknown Source)
> at 
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.addBlock(ClientNamenodeProtocolTranslatorPB.java:399)
> at sun.reflect.GeneratedMethodAccessor8.invoke(Unknown Source)
> at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> at java.lang.reflect.Method.invoke(Method.java:498)
> at 
> org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:193)
> at 
> org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102)
> {code}
>  
> We can see the client has {{Address change detected}}, but it still fails. I 
> find out that's because when method {{updateAddress()}} return true,  the 
> {{handleConnectionFailure()}} thow an exception that break the next retry 
> with the right ipaddr.
> Client.java: setupConnection()
> {code:java}
> } catch (ConnectTimeoutException toe) {
>   /* Check for an address change and update the local reference.
>* Reset the failure counter if the address was changed
>*/
>   if (updateAddress()) {
> timeoutFailures = ioFailures = 0;
>   }
>   handleConnectionTimeout(timeoutFailures++,
>   maxRetriesOnSocketTimeouts, toe);
> } catch (IOException ie) {
>   if (updateAddress()) {
> timeoutFailures = ioFailures = 0;
>   }
> // because the namenode ip changed in updateAddress(), the old namenode 
> ipaddress cannot be accessed now
> // handleConnectionFailure will thow an exception, the next retry never have 
> a chance to use the right server updated in updateAddress()
>   handleConnectionFailure(ioFailures++, ie);
> }
> {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] [Comment Edited] (HDFS-15390) client fails forever when namenode ipaddr changed

2020-06-08 Thread Sean Chow (Jira)


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

Sean Chow edited comment on HDFS-15390 at 6/8/20, 12:10 PM:


Hi [~hexiaoqiao] , in fact, I've done some debug and the exception is thrown as 
below:
{code:java}
private void handleConnectionFailure(int curRetries, IOException ioe
) throws IOException {
  closeConnection();

  final RetryAction action;
  try {
action = connectionRetryPolicy.shouldRetry(ioe, curRetries, 0, true);
  } catch(Exception e) {
throw e instanceof IOException? (IOException)e: new IOException(e);
  }
  if (action.action == RetryAction.RetryDecision.FAIL) {
if (action.reason != null) {
  if (LOG.isDebugEnabled()) {
LOG.debug("Failed to connect to server: " + server + ": "
+ action.reason, ioe);
  }
}
// HERE is where the IOException throws
throw ioe;
  }
...{code}
But the strange is  {{Failed to connect to server}}  debug log is not logged.

 

We use hadoop version  hadoop-2.6.0-cdh5.4.11. And I've tested it with the 
trunk version, the same issue.

Though it only affects the hdfs client side, I think you're right. But how to 
move it  common project jira? :(


was (Author: seanlook):
In fact, I've done some debug and the exception is thrown as below:
{code:java}
private void handleConnectionFailure(int curRetries, IOException ioe
) throws IOException {
  closeConnection();

  final RetryAction action;
  try {
action = connectionRetryPolicy.shouldRetry(ioe, curRetries, 0, true);
  } catch(Exception e) {
throw e instanceof IOException? (IOException)e: new IOException(e);
  }
  if (action.action == RetryAction.RetryDecision.FAIL) {
if (action.reason != null) {
  if (LOG.isDebugEnabled()) {
LOG.debug("Failed to connect to server: " + server + ": "
+ action.reason, ioe);
  }
}
// HERE is where the IOException throws
throw ioe;
  }
...{code}
But the strange is  {{Failed to connect to server}}  debug log is not logged.

 

We use hadoop version  hadoop-2.6.0-cdh5.4.11. And I've tested it with the 
trunk version, the same issue.

Though it only affects the hdfs client side, I think you're right. But how to 
move it  common project jira? :(

> client fails forever when namenode ipaddr changed
> -
>
> Key: HDFS-15390
> URL: https://issues.apache.org/jira/browse/HDFS-15390
> Project: Hadoop HDFS
>  Issue Type: Bug
>Affects Versions: 2.10.0, 2.9.2, 3.2.1
>Reporter: Sean Chow
>Priority: Major
> Attachments: HDFS-15390.01.patch
>
>
> For machine replacement, I replace my standby namenode with a new ipaddr and 
> keep the same hostname. Also update the client's hosts to make it resolve 
> correctly
> When I try to run failover to transite the new namenode(let's say nn2), the 
> client will fail to read or write forever until it's restarted.
> That make yarn nodemanager in sick state. Even the new tasks will encounter 
> this exception  too. Until all nodemanager restart.
>  
> {code:java}
> 20/06/02 15:12:25 WARN ipc.Client: Address change detected. Old: 
> nn2-192-168-1-100/192.168.1.100:9000 New: nn2-192-168-1-100/192.168.1.200:9000
> 20/06/02 15:12:25 DEBUG ipc.Client: closing ipc connection to 
> nn2-192-168-1-100/192.168.1.200:9000: Connection refused
> java.net.ConnectException: Connection refused
> at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
> at 
> sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
> at 
> org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.java:206)
> at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:530)
> at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:494)
> at 
> org.apache.hadoop.ipc.Client$Connection.setupConnection(Client.java:608)
> at 
> org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:707)
> at 
> org.apache.hadoop.ipc.Client$Connection.access$2800(Client.java:368)
> at org.apache.hadoop.ipc.Client.getConnection(Client.java:1517)
> at org.apache.hadoop.ipc.Client.call(Client.java:1440)
> at org.apache.hadoop.ipc.Client.call(Client.java:1401)
> at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:232)
> at com.sun.proxy.$Proxy9.addBlock(Unknown Source)
> at 
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.addBlock(ClientNamenodeProtocolTranslatorPB.java:399)
> at sun.reflect.GeneratedMethodAccessor8.invoke(Unknown Source)
> at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> at java.lang.reflect.Method.invoke(Method.java:498)
> at 
> 

[jira] [Comment Edited] (HDFS-15390) client fails forever when namenode ipaddr changed

2020-06-08 Thread Sean Chow (Jira)


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

Sean Chow edited comment on HDFS-15390 at 6/8/20, 12:09 PM:


In fact, I've done some debug and the exception is thrown as below:
{code:java}
private void handleConnectionFailure(int curRetries, IOException ioe
) throws IOException {
  closeConnection();

  final RetryAction action;
  try {
action = connectionRetryPolicy.shouldRetry(ioe, curRetries, 0, true);
  } catch(Exception e) {
throw e instanceof IOException? (IOException)e: new IOException(e);
  }
  if (action.action == RetryAction.RetryDecision.FAIL) {
if (action.reason != null) {
  if (LOG.isDebugEnabled()) {
LOG.debug("Failed to connect to server: " + server + ": "
+ action.reason, ioe);
  }
}
// HERE is where the IOException throws
throw ioe;
  }
...{code}
But the strange is  {{Failed to connect to server}}  debug log is not logged.

 

We use hadoop version  hadoop-2.6.0-cdh5.4.11. And I've tested it with the 
trunk version, the same issue.

Though it only affects the hdfs client side, I think you're right. But how to 
move it  common project jira? :(


was (Author: seanlook):
In fact, I've done some debug and the exception is thrown as below:
{code:java}
private void handleConnectionFailure(int curRetries, IOException ioe
) throws IOException {
  closeConnection();

  final RetryAction action;
  try {
action = connectionRetryPolicy.shouldRetry(ioe, curRetries, 0, true);
  } catch(Exception e) {
throw e instanceof IOException? (IOException)e: new IOException(e);
  }
  if (action.action == RetryAction.RetryDecision.FAIL) {
if (action.reason != null) {
  if (LOG.isDebugEnabled()) {
LOG.debug("Failed to connect to server: " + server + ": "
+ action.reason, ioe);
  }
}
// HERE is where the IOException throws
throw ioe;
  }
...{code}
But the strange is  {{Failed to connect to server}}  debug log is not logged.

 

We use hadoop version  hadoop-2.6.0-cdh5.4.11. And I've tested it with the 
trunk version, the same issue.

Though it only affects the client side, I move this ticket to hadoop-common.

> client fails forever when namenode ipaddr changed
> -
>
> Key: HDFS-15390
> URL: https://issues.apache.org/jira/browse/HDFS-15390
> Project: Hadoop HDFS
>  Issue Type: Bug
>Affects Versions: 2.10.0, 2.9.2, 3.2.1
>Reporter: Sean Chow
>Priority: Major
> Attachments: HDFS-15390.01.patch
>
>
> For machine replacement, I replace my standby namenode with a new ipaddr and 
> keep the same hostname. Also update the client's hosts to make it resolve 
> correctly
> When I try to run failover to transite the new namenode(let's say nn2), the 
> client will fail to read or write forever until it's restarted.
> That make yarn nodemanager in sick state. Even the new tasks will encounter 
> this exception  too. Until all nodemanager restart.
>  
> {code:java}
> 20/06/02 15:12:25 WARN ipc.Client: Address change detected. Old: 
> nn2-192-168-1-100/192.168.1.100:9000 New: nn2-192-168-1-100/192.168.1.200:9000
> 20/06/02 15:12:25 DEBUG ipc.Client: closing ipc connection to 
> nn2-192-168-1-100/192.168.1.200:9000: Connection refused
> java.net.ConnectException: Connection refused
> at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
> at 
> sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
> at 
> org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.java:206)
> at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:530)
> at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:494)
> at 
> org.apache.hadoop.ipc.Client$Connection.setupConnection(Client.java:608)
> at 
> org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:707)
> at 
> org.apache.hadoop.ipc.Client$Connection.access$2800(Client.java:368)
> at org.apache.hadoop.ipc.Client.getConnection(Client.java:1517)
> at org.apache.hadoop.ipc.Client.call(Client.java:1440)
> at org.apache.hadoop.ipc.Client.call(Client.java:1401)
> at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:232)
> at com.sun.proxy.$Proxy9.addBlock(Unknown Source)
> at 
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.addBlock(ClientNamenodeProtocolTranslatorPB.java:399)
> at sun.reflect.GeneratedMethodAccessor8.invoke(Unknown Source)
> at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> at java.lang.reflect.Method.invoke(Method.java:498)
> at 
> 

[jira] [Commented] (HDFS-15390) client fails forever when namenode ipaddr changed

2020-06-08 Thread Sean Chow (Jira)


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

Sean Chow commented on HDFS-15390:
--

In fact, I've done some debug and the exception is thrown as below:
{code:java}
private void handleConnectionFailure(int curRetries, IOException ioe
) throws IOException {
  closeConnection();

  final RetryAction action;
  try {
action = connectionRetryPolicy.shouldRetry(ioe, curRetries, 0, true);
  } catch(Exception e) {
throw e instanceof IOException? (IOException)e: new IOException(e);
  }
  if (action.action == RetryAction.RetryDecision.FAIL) {
if (action.reason != null) {
  if (LOG.isDebugEnabled()) {
LOG.debug("Failed to connect to server: " + server + ": "
+ action.reason, ioe);
  }
}
// HERE is where the IOException throws
throw ioe;
  }
...{code}
But the strange is  {{Failed to connect to server}}  debug log is not logged.

 

We use hadoop version  hadoop-2.6.0-cdh5.4.11. And I've tested it with the 
trunk version, the same issue.

Though it only affects the client side, I move this ticket to hadoop-common.

> client fails forever when namenode ipaddr changed
> -
>
> Key: HDFS-15390
> URL: https://issues.apache.org/jira/browse/HDFS-15390
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: dfsclient
>Affects Versions: 2.10.0, 2.9.2, 3.2.1
>Reporter: Sean Chow
>Priority: Major
> Attachments: HDFS-15390.01.patch
>
>
> For machine replacement, I replace my standby namenode with a new ipaddr and 
> keep the same hostname. Also update the client's hosts to make it resolve 
> correctly
> When I try to run failover to transite the new namenode(let's say nn2), the 
> client will fail to read or write forever until it's restarted.
> That make yarn nodemanager in sick state. Even the new tasks will encounter 
> this exception  too. Until all nodemanager restart.
>  
> {code:java}
> 20/06/02 15:12:25 WARN ipc.Client: Address change detected. Old: 
> nn2-192-168-1-100/192.168.1.100:9000 New: nn2-192-168-1-100/192.168.1.200:9000
> 20/06/02 15:12:25 DEBUG ipc.Client: closing ipc connection to 
> nn2-192-168-1-100/192.168.1.200:9000: Connection refused
> java.net.ConnectException: Connection refused
> at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
> at 
> sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
> at 
> org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.java:206)
> at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:530)
> at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:494)
> at 
> org.apache.hadoop.ipc.Client$Connection.setupConnection(Client.java:608)
> at 
> org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:707)
> at 
> org.apache.hadoop.ipc.Client$Connection.access$2800(Client.java:368)
> at org.apache.hadoop.ipc.Client.getConnection(Client.java:1517)
> at org.apache.hadoop.ipc.Client.call(Client.java:1440)
> at org.apache.hadoop.ipc.Client.call(Client.java:1401)
> at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:232)
> at com.sun.proxy.$Proxy9.addBlock(Unknown Source)
> at 
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.addBlock(ClientNamenodeProtocolTranslatorPB.java:399)
> at sun.reflect.GeneratedMethodAccessor8.invoke(Unknown Source)
> at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> at java.lang.reflect.Method.invoke(Method.java:498)
> at 
> org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:193)
> at 
> org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102)
> {code}
>  
> We can see the client has {{Address change detected}}, but it still fails. I 
> find out that's because when method {{updateAddress()}} return true,  the 
> {{handleConnectionFailure()}} thow an exception that break the next retry 
> with the right ipaddr.
> Client.java: setupConnection()
> {code:java}
> } catch (ConnectTimeoutException toe) {
>   /* Check for an address change and update the local reference.
>* Reset the failure counter if the address was changed
>*/
>   if (updateAddress()) {
> timeoutFailures = ioFailures = 0;
>   }
>   handleConnectionTimeout(timeoutFailures++,
>   maxRetriesOnSocketTimeouts, toe);
> } catch (IOException ie) {
>   if (updateAddress()) {
> timeoutFailures = ioFailures = 0;
>   }
> // because the namenode ip changed in 

[jira] [Updated] (HDFS-15390) client fails forever when namenode ipaddr changed

2020-06-08 Thread Sean Chow (Jira)


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

Sean Chow updated HDFS-15390:
-
Component/s: (was: dfsclient)

> client fails forever when namenode ipaddr changed
> -
>
> Key: HDFS-15390
> URL: https://issues.apache.org/jira/browse/HDFS-15390
> Project: Hadoop HDFS
>  Issue Type: Bug
>Affects Versions: 2.10.0, 2.9.2, 3.2.1
>Reporter: Sean Chow
>Priority: Major
> Attachments: HDFS-15390.01.patch
>
>
> For machine replacement, I replace my standby namenode with a new ipaddr and 
> keep the same hostname. Also update the client's hosts to make it resolve 
> correctly
> When I try to run failover to transite the new namenode(let's say nn2), the 
> client will fail to read or write forever until it's restarted.
> That make yarn nodemanager in sick state. Even the new tasks will encounter 
> this exception  too. Until all nodemanager restart.
>  
> {code:java}
> 20/06/02 15:12:25 WARN ipc.Client: Address change detected. Old: 
> nn2-192-168-1-100/192.168.1.100:9000 New: nn2-192-168-1-100/192.168.1.200:9000
> 20/06/02 15:12:25 DEBUG ipc.Client: closing ipc connection to 
> nn2-192-168-1-100/192.168.1.200:9000: Connection refused
> java.net.ConnectException: Connection refused
> at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
> at 
> sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
> at 
> org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.java:206)
> at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:530)
> at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:494)
> at 
> org.apache.hadoop.ipc.Client$Connection.setupConnection(Client.java:608)
> at 
> org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:707)
> at 
> org.apache.hadoop.ipc.Client$Connection.access$2800(Client.java:368)
> at org.apache.hadoop.ipc.Client.getConnection(Client.java:1517)
> at org.apache.hadoop.ipc.Client.call(Client.java:1440)
> at org.apache.hadoop.ipc.Client.call(Client.java:1401)
> at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:232)
> at com.sun.proxy.$Proxy9.addBlock(Unknown Source)
> at 
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.addBlock(ClientNamenodeProtocolTranslatorPB.java:399)
> at sun.reflect.GeneratedMethodAccessor8.invoke(Unknown Source)
> at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> at java.lang.reflect.Method.invoke(Method.java:498)
> at 
> org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:193)
> at 
> org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102)
> {code}
>  
> We can see the client has {{Address change detected}}, but it still fails. I 
> find out that's because when method {{updateAddress()}} return true,  the 
> {{handleConnectionFailure()}} thow an exception that break the next retry 
> with the right ipaddr.
> Client.java: setupConnection()
> {code:java}
> } catch (ConnectTimeoutException toe) {
>   /* Check for an address change and update the local reference.
>* Reset the failure counter if the address was changed
>*/
>   if (updateAddress()) {
> timeoutFailures = ioFailures = 0;
>   }
>   handleConnectionTimeout(timeoutFailures++,
>   maxRetriesOnSocketTimeouts, toe);
> } catch (IOException ie) {
>   if (updateAddress()) {
> timeoutFailures = ioFailures = 0;
>   }
> // because the namenode ip changed in updateAddress(), the old namenode 
> ipaddress cannot be accessed now
> // handleConnectionFailure will thow an exception, the next retry never have 
> a chance to use the right server updated in updateAddress()
>   handleConnectionFailure(ioFailures++, ie);
> }
> {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