[ 
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  <0x000000008009d5d8> (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 <0x000000008009f758> (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 <0x000000008009f758> (a 
org.apache.hadoop.hdfs.DFSStripedOutputStream)
        at 
org.apache.hadoop.fs.FSOutputSummer.flushBuffer(FSOutputSummer.java:145)
        - locked <0x000000008009f758> (a 
org.apache.hadoop.hdfs.DFSStripedOutputStream)
        at 
org.apache.hadoop.hdfs.DFSStripedOutputStream.closeImpl(DFSStripedOutputStream.java:1182)
        - locked <0x000000008009f758> (a 
org.apache.hadoop.hdfs.DFSStripedOutputStream)
        at 
org.apache.hadoop.hdfs.DFSOutputStream.close(DFSOutputStream.java:847)
        - locked <0x000000008009f758> (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 /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  <0x000000008009d5d8> (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 <0x000000008009f758> (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 <0x000000008009f758> (a 
org.apache.hadoop.hdfs.DFSStripedOutputStream)
        at 
org.apache.hadoop.fs.FSOutputSummer.flushBuffer(FSOutputSummer.java:145)
        - locked <0x000000008009f758> (a 
org.apache.hadoop.hdfs.DFSStripedOutputStream)
        at 
org.apache.hadoop.hdfs.DFSStripedOutputStream.closeImpl(DFSStripedOutputStream.java:1182)
        - locked <0x000000008009f758> (a 
org.apache.hadoop.hdfs.DFSStripedOutputStream)
        at 
org.apache.hadoop.hdfs.DFSOutputStream.close(DFSOutputStream.java:847)
        - locked <0x000000008009f758> (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 Type: Bug
>          Components: ec, hdfs-client
>    Affects Versions: 3.2.0
>            Reporter: Hongbing Wang
>            Priority: Major
>
>  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  <0x000000008009d5d8> (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 <0x000000008009f758> (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 <0x000000008009f758> (a 
> org.apache.hadoop.hdfs.DFSStripedOutputStream)
>         at 
> org.apache.hadoop.fs.FSOutputSummer.flushBuffer(FSOutputSummer.java:145)
>         - locked <0x000000008009f758> (a 
> org.apache.hadoop.hdfs.DFSStripedOutputStream)
>         at 
> org.apache.hadoop.hdfs.DFSStripedOutputStream.closeImpl(DFSStripedOutputStream.java:1182)
>         - locked <0x000000008009f758> (a 
> org.apache.hadoop.hdfs.DFSStripedOutputStream)
>         at 
> org.apache.hadoop.hdfs.DFSOutputStream.close(DFSOutputStream.java:847)
>         - locked <0x000000008009f758> (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.
>  



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

Reply via email to