[jira] [Work logged] (HDFS-15925) The lack of packet-level mirrorError state synchronization in BlockReceiver$PacketResponder can cause the HDFS client to hang

2021-03-26 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/HDFS-15925?focusedWorklogId=572976=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-572976
 ]

ASF GitHub Bot logged work on HDFS-15925:
-

Author: ASF GitHub Bot
Created on: 27/Mar/21 04:13
Start Date: 27/Mar/21 04:13
Worklog Time Spent: 10m 
  Work Description: functioner opened a new pull request #2821:
URL: https://github.com/apache/hadoop/pull/2821


   I propose a fix for 
[HDFS-15925](https://issues.apache.org/jira/browse/HDFS-15925).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
---

Worklog Id: (was: 572976)
Remaining Estimate: 0h
Time Spent: 10m

> The lack of packet-level mirrorError state synchronization in 
> BlockReceiver$PacketResponder can cause the HDFS client to hang
> -
>
> Key: HDFS-15925
> URL: https://issues.apache.org/jira/browse/HDFS-15925
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode
>Affects Versions: 3.2.2
>Reporter: Haoze Wu
>Priority: Critical
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
>     When the datanode is receiving data block packets from a HDFS client and 
> forwarding these packets to a mirror (another datanode) simultaneously, a 
> single IOException in the datanode’s forwarding path can cause the client to 
> get stuck for 1 min, without any logging. After 1 min, the client’s log shows 
> a warning of EOFException and `Slow waitForAckedSeqno took 60106ms 
> (threshold=3ms)`.
>     Normally the datanode will inform the client of this error state 
> immediately, and then the client will resend the packets immediately. The 
> whole process is very fast. After careful analyses, we find the above symptom 
> is due to the lack of packet-level mirrorError state synchronization in 
> BlockReceiver$PacketResponder: in some concurrency condition, the 
> BlockReceiver$PacketResponder will hang for 1 min and then exit, without 
> sending the error state to the client.
> *Root Cause Analysis* 
> {code:java}
> //hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
> class BlockReceiver implements Closeable {
>   // ...
>   private void handleMirrorOutError(IOException ioe) throws IOException {
> // ...
> if (Thread.interrupted()) {
>   throw ioe;
> } else { // encounter an error while writing to mirror
>   // continue to run even if can not write to mirror
>   // notify client of the error
>   // and wait for the client to shut down the pipeline
>   mirrorError = true;// line 
> 461
> }
>   }
>   private int receivePacket() throws IOException {
> // read the next packet
> packetReceiver.receiveNextPacket(in);// line 
> 528
> // ...
> boolean lastPacketInBlock = header.isLastPacketInBlock();// line 
> 551
> //First write the packet to the mirror:
> if (mirrorOut != null && !mirrorError) {
>   try {
> // ...
> packetReceiver.mirrorPacketTo(mirrorOut);// line 
> 588
> // ...
>   } catch (IOException e) {
> handleMirrorOutError(e); // line 
> 604
>   }
> }
> // ...
> return lastPacketInBlock?-1:len; // line 
> 849
>   }
>   void receiveBlock(...) throws IOException {
> // ...
> try {
>   if (isClient && !isTransfer) {
> responder = new Daemon(datanode.threadGroup, 
> new PacketResponder(replyOut, mirrIn, downstreams));
> responder.start();   // line 
> 968
>   }
>   while(receivePacket() >= 0){/*Receive until the last packet*/} // line 
> 971
>   // wait for all outstanding packet responses. And then
>   // indicate responder to gracefully shutdown.
>   // Mark that responder has been closed for future processing
>   if (responder != null) {
> ((PacketResponder)responder.getRunnable()).close();  // line 
> 977
> responderClosed = true;
>   }
>   // ...
> } catch (IOException ioe) {  // line 
> 1003
>   // ...
> } finally {
>   // ...
>   if (!responderClosed) { // Data transfer was not complete.
> if (responder != null) {
>   // ...
>   responder.interrupt(); 

[jira] [Updated] (HDFS-15925) The lack of packet-level mirrorError state synchronization in BlockReceiver$PacketResponder can cause the HDFS client to hang

2021-03-26 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot updated HDFS-15925:
--
Labels: pull-request-available  (was: )

> The lack of packet-level mirrorError state synchronization in 
> BlockReceiver$PacketResponder can cause the HDFS client to hang
> -
>
> Key: HDFS-15925
> URL: https://issues.apache.org/jira/browse/HDFS-15925
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: datanode
>Affects Versions: 3.2.2
>Reporter: Haoze Wu
>Priority: Critical
>  Labels: pull-request-available
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
>     When the datanode is receiving data block packets from a HDFS client and 
> forwarding these packets to a mirror (another datanode) simultaneously, a 
> single IOException in the datanode’s forwarding path can cause the client to 
> get stuck for 1 min, without any logging. After 1 min, the client’s log shows 
> a warning of EOFException and `Slow waitForAckedSeqno took 60106ms 
> (threshold=3ms)`.
>     Normally the datanode will inform the client of this error state 
> immediately, and then the client will resend the packets immediately. The 
> whole process is very fast. After careful analyses, we find the above symptom 
> is due to the lack of packet-level mirrorError state synchronization in 
> BlockReceiver$PacketResponder: in some concurrency condition, the 
> BlockReceiver$PacketResponder will hang for 1 min and then exit, without 
> sending the error state to the client.
> *Root Cause Analysis* 
> {code:java}
> //hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
> class BlockReceiver implements Closeable {
>   // ...
>   private void handleMirrorOutError(IOException ioe) throws IOException {
> // ...
> if (Thread.interrupted()) {
>   throw ioe;
> } else { // encounter an error while writing to mirror
>   // continue to run even if can not write to mirror
>   // notify client of the error
>   // and wait for the client to shut down the pipeline
>   mirrorError = true;// line 
> 461
> }
>   }
>   private int receivePacket() throws IOException {
> // read the next packet
> packetReceiver.receiveNextPacket(in);// line 
> 528
> // ...
> boolean lastPacketInBlock = header.isLastPacketInBlock();// line 
> 551
> //First write the packet to the mirror:
> if (mirrorOut != null && !mirrorError) {
>   try {
> // ...
> packetReceiver.mirrorPacketTo(mirrorOut);// line 
> 588
> // ...
>   } catch (IOException e) {
> handleMirrorOutError(e); // line 
> 604
>   }
> }
> // ...
> return lastPacketInBlock?-1:len; // line 
> 849
>   }
>   void receiveBlock(...) throws IOException {
> // ...
> try {
>   if (isClient && !isTransfer) {
> responder = new Daemon(datanode.threadGroup, 
> new PacketResponder(replyOut, mirrIn, downstreams));
> responder.start();   // line 
> 968
>   }
>   while(receivePacket() >= 0){/*Receive until the last packet*/} // line 
> 971
>   // wait for all outstanding packet responses. And then
>   // indicate responder to gracefully shutdown.
>   // Mark that responder has been closed for future processing
>   if (responder != null) {
> ((PacketResponder)responder.getRunnable()).close();  // line 
> 977
> responderClosed = true;
>   }
>   // ...
> } catch (IOException ioe) {  // line 
> 1003
>   // ...
> } finally {
>   // ...
>   if (!responderClosed) { // Data transfer was not complete.
> if (responder != null) {
>   // ...
>   responder.interrupt(); // line 
> 1046
> }
> // ...
>   }
>   if (responder != null) {
> try {
>   responder.interrupt(); // line 
> 1053
>   // ...
> } catch (InterruptedException e) {
>   responder.interrupt(); // line 
> 1067
>   // ...
> }
> // ...
>   }
> }
>   }
> }
> {code}
>     In the `BlockReceiver.receivePacket` method, if the datanode fails to 
> forward the packet to the mirror ( (line 588) due to an IOException, it is 
> handled by line 604, which sets the mirrorError flag in line 461. According 
> to the comments, the 

[jira] [Updated] (HDFS-15925) The lack of packet-level mirrorError state synchronization in BlockReceiver$PacketResponder can cause the HDFS client to hang

2021-03-26 Thread Haoze Wu (Jira)


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

Haoze Wu updated HDFS-15925:

Description: 
    When the datanode is receiving data block packets from a HDFS client and 
forwarding these packets to a mirror (another datanode) simultaneously, a 
single IOException in the datanode’s forwarding path can cause the client to 
get stuck for 1 min, without any logging. After 1 min, the client’s log shows a 
warning of EOFException and `Slow waitForAckedSeqno took 60106ms 
(threshold=3ms)`.

    Normally the datanode will inform the client of this error state 
immediately, and then the client will resend the packets immediately. The whole 
process is very fast. After careful analyses, we find the above symptom is due 
to the lack of packet-level mirrorError state synchronization in 
BlockReceiver$PacketResponder: in some concurrency condition, the 
BlockReceiver$PacketResponder will hang for 1 min and then exit, without 
sending the error state to the client.

*Root Cause Analysis* 
{code:java}
//hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java

class BlockReceiver implements Closeable {
  // ...

  private void handleMirrorOutError(IOException ioe) throws IOException {
// ...
if (Thread.interrupted()) {
  throw ioe;
} else { // encounter an error while writing to mirror
  // continue to run even if can not write to mirror
  // notify client of the error
  // and wait for the client to shut down the pipeline
  mirrorError = true;// line 461
}
  }

  private int receivePacket() throws IOException {
// read the next packet
packetReceiver.receiveNextPacket(in);// line 528
// ...
boolean lastPacketInBlock = header.isLastPacketInBlock();// line 551
//First write the packet to the mirror:
if (mirrorOut != null && !mirrorError) {
  try {
// ...
packetReceiver.mirrorPacketTo(mirrorOut);// line 588
// ...
  } catch (IOException e) {
handleMirrorOutError(e); // line 604
  }
}
// ...
return lastPacketInBlock?-1:len; // line 849
  }

  void receiveBlock(...) throws IOException {
// ...
try {
  if (isClient && !isTransfer) {
responder = new Daemon(datanode.threadGroup, 
new PacketResponder(replyOut, mirrIn, downstreams));
responder.start();   // line 968
  }

  while(receivePacket() >= 0){/*Receive until the last packet*/} // line 971

  // wait for all outstanding packet responses. And then
  // indicate responder to gracefully shutdown.
  // Mark that responder has been closed for future processing
  if (responder != null) {
((PacketResponder)responder.getRunnable()).close();  // line 977
responderClosed = true;
  }
  // ...
} catch (IOException ioe) {  // line 
1003
  // ...
} finally {
  // ...
  if (!responderClosed) { // Data transfer was not complete.
if (responder != null) {
  // ...
  responder.interrupt(); // line 
1046
}
// ...
  }
  if (responder != null) {
try {
  responder.interrupt(); // line 
1053
  // ...
} catch (InterruptedException e) {
  responder.interrupt(); // line 
1067
  // ...
}
// ...
  }
}
  }
}
{code}
    In the `BlockReceiver.receivePacket` method, if the datanode fails to 
forward the packet to the mirror ( (line 588) due to an IOException, it is 
handled by line 604, which sets the mirrorError flag in line 461. According to 
the comments, the BlockReceiver keeps going with the mirrorError state, and the 
client would be notified of the error.

    However, jstack shows that the datanode gets stuck in the `DataXceiver` 
thread (receiving data block packets from client) and the 
`BlockReceiver$PacketResponder` thread (replying ACK packets to client). In 
particular, the `DataXceiver` thread gets stuck in the loop in line 971, which 
is further caused by blocking in line 528, meaning that the `lastPacketInBlock` 
packet has not arrived, and no more packets are coming in.
{code:java}
//hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java

class BlockReceiver implements Closeable {
  // ...

  class PacketResponder implements Runnable, Closeable {
// ...

public void run() {
  // ...
  while (isRunning() && !lastPacketInBlock) {
// ...
try {
  // ...

[jira] [Updated] (HDFS-15925) The lack of packet-level mirrorError state synchronization in BlockReceiver$PacketResponder can cause the HDFS client to hang

2021-03-26 Thread Haoze Wu (Jira)


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

Haoze Wu updated HDFS-15925:

Description: 
    When the datanode is receiving data block packets from a HDFS client and 
forwarding these packets to a mirror (another datanode) simultaneously, a 
single IOException in the datanode’s forwarding path can cause the client to 
get stuck for 1 min, without any logging. After 1 min, the client’s log shows a 
warning of EOFException and `Slow waitForAckedSeqno took 60106ms 
(threshold=3ms)`.

    Normally the datanode will inform the client of this error state 
immediately, and then the client will resend the packets immediately. The whole 
process is very fast. After careful analyses, we find the above symptom is due 
to the lack of packet-level mirrorError state synchronization in 
BlockReceiver$PacketResponder: in some concurrency condition, the 
BlockReceiver$PacketResponder will hang for 1 min and then exit, without 
sending the error state to the client.

*Root Cause Analysis* 
{code:java}
//hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java

class BlockReceiver implements Closeable {
  // ...

  private void handleMirrorOutError(IOException ioe) throws IOException {
// ...
if (Thread.interrupted()) {
  throw ioe;
} else { // encounter an error while writing to mirror
  // continue to run even if can not write to mirror
  // notify client of the error
  // and wait for the client to shut down the pipeline
  mirrorError = true;// line 461
}
  }

  private int receivePacket() throws IOException {
// read the next packet
packetReceiver.receiveNextPacket(in);// line 528
// ...
boolean lastPacketInBlock = header.isLastPacketInBlock();// line 551
//First write the packet to the mirror:
if (mirrorOut != null && !mirrorError) {
  try {
// ...
packetReceiver.mirrorPacketTo(mirrorOut);// line 588
// ...
  } catch (IOException e) {
handleMirrorOutError(e); // line 604
  }
}
// ...
return lastPacketInBlock?-1:len; // line 849
  }

  void receiveBlock(...) throws IOException {
// ...
try {
  if (isClient && !isTransfer) {
responder = new Daemon(datanode.threadGroup, 
new PacketResponder(replyOut, mirrIn, downstreams));
responder.start();   // line 968
  }

  while(receivePacket() >= 0){/*Receive until the last packet*/} // line 971

  // wait for all outstanding packet responses. And then
  // indicate responder to gracefully shutdown.
  // Mark that responder has been closed for future processing
  if (responder != null) {
((PacketResponder)responder.getRunnable()).close();  // line 977
responderClosed = true;
  }
  // ...
} catch (IOException ioe) {  // line 
1003
  // ...
} finally {
  // ...
  if (!responderClosed) { // Data transfer was not complete.
if (responder != null) {
  // ...
  responder.interrupt(); // line 
1046
}
// ...
  }
  if (responder != null) {
try {
  responder.interrupt(); // line 
1053
  // ...
} catch (InterruptedException e) {
  responder.interrupt(); // line 
1067
  // ...
}
// ...
  }
}
  }
}
{code}
    In the `BlockReceiver.receivePacket` method, if the datanode fails to 
forward the packet to the mirror ( (line 588) due to an IOException, it is 
handled by line 604, which sets the mirrorError flag in line 461. According to 
the comments, the BlockReceiver keeps going with the mirrorError state, and the 
client would be notified of the error.

    However, jstack shows that the datanode gets stuck in the `DataXceiver` 
thread (receiving data block packets from client) and the 
`BlockReceiver$PacketResponder` thread (replying ACK packets to client). In 
particular, the `DataXceiver` thread gets stuck in the loop in line 971, which 
is further caused by blocking in line 528, meaning that the `lastPacketInBlock` 
packet has not arrived, and no more packets are coming in.
{code:java}
//hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java

class BlockReceiver implements Closeable {
  // ...

  class PacketResponder implements Runnable, Closeable {
// ...

public void run() {
  // ...
  while (isRunning() && !lastPacketInBlock) {
// ...
try {
  // ...

[jira] [Updated] (HDFS-15925) The lack of packet-level mirrorError state synchronization in BlockReceiver$PacketResponder can cause the HDFS client to hang

2021-03-26 Thread Haoze Wu (Jira)


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

Haoze Wu updated HDFS-15925:

Description: 
    When the datanode is receiving data block packets from a HDFS client and 
forwarding these packets to a mirror (another datanode) simultaneously, a 
single IOException in the datanode’s forwarding path can cause the client to 
get stuck for 1 min, without any logging. After 1 min, the client’s log shows a 
warning of EOFException and `Slow waitForAckedSeqno took 60106ms 
(threshold=3ms)`.

    Normally the datanode will inform the client of this error state 
immediately, and then the client will resend the packets immediately. The whole 
process is very fast. After careful analyses, we find the above symptom is due 
to the lack of packet-level mirrorError state synchronization in 
BlockReceiver$PacketResponder: in some concurrency condition, the 
BlockReceiver$PacketResponder will hang for 1 min and then exit, without 
sending the error state to the client.

*Root Cause Analysis* 
{code:java}
//hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java

class BlockReceiver implements Closeable {
  // ...

  private void handleMirrorOutError(IOException ioe) throws IOException {
// ...
if (Thread.interrupted()) {
  throw ioe;
} else { // encounter an error while writing to mirror
  // continue to run even if can not write to mirror
  // notify client of the error
  // and wait for the client to shut down the pipeline
  mirrorError = true;// line 461
}
  }

  private int receivePacket() throws IOException {
// read the next packet
packetReceiver.receiveNextPacket(in);// line 528
// ...
boolean lastPacketInBlock = header.isLastPacketInBlock();// line 551
//First write the packet to the mirror:
if (mirrorOut != null && !mirrorError) {
  try {
// ...
packetReceiver.mirrorPacketTo(mirrorOut);// line 588
// ...
  } catch (IOException e) {
handleMirrorOutError(e); // line 604
  }
}
// ...
return lastPacketInBlock?-1:len; // line 849
  }

  void receiveBlock(...) throws IOException {
// ...
try {
  if (isClient && !isTransfer) {
responder = new Daemon(datanode.threadGroup, 
new PacketResponder(replyOut, mirrIn, downstreams));
responder.start();   // line 968
  }

  while(receivePacket() >= 0){/*Receive until the last packet*/} // line 971

  // wait for all outstanding packet responses. And then
  // indicate responder to gracefully shutdown.
  // Mark that responder has been closed for future processing
  if (responder != null) {
((PacketResponder)responder.getRunnable()).close();  // line 977
responderClosed = true;
  }
  // ...
} catch (IOException ioe) {  // line 
1003
  // ...
} finally {
  // ...
  if (!responderClosed) { // Data transfer was not complete.
if (responder != null) {
  // ...
  responder.interrupt(); // line 
1046
}
// ...
  }
  if (responder != null) {
try {
  responder.interrupt(); // line 
1053
  // ...
} catch (InterruptedException e) {
  responder.interrupt(); // line 
1067
  // ...
}
// ...
  }
}
  }
}
{code}
    In the `BlockReceiver.receivePacket` method, if the datanode fails to 
forward the packet to the mirror ( (line 588) due to an IOException, it is 
handled by line 604, which sets the mirrorError flag in line 461. According to 
the comments, the BlockReceiver keeps going with the mirrorError state, and the 
client would be notified of the error.

    However, jstack shows that the datanode gets stuck in the `DataXceiver` 
thread (receiving data block packets from client) and the 
`BlockReceiver$PacketResponder` thread (replying ACK packets to client). In 
particular, the `DataXceiver` thread gets stuck in the loop in line 971, which 
is further caused by blocking in line 528, meaning that the `lastPacketInBlock` 
packet has not arrived, and no more packets are coming in.
{code:java}
//hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java

class BlockReceiver implements Closeable {
  // ...

  class PacketResponder implements Runnable, Closeable {
// ...

public void run() {
  // ...
  while (isRunning() && !lastPacketInBlock) {
// ...
try {
  // ...

[jira] [Created] (HDFS-15925) The lack of packet-level mirrorError state synchronization in BlockReceiver$PacketResponder can cause the HDFS client to hang

2021-03-26 Thread Haoze Wu (Jira)
Haoze Wu created HDFS-15925:
---

 Summary: The lack of packet-level mirrorError state 
synchronization in BlockReceiver$PacketResponder can cause the HDFS client to 
hang
 Key: HDFS-15925
 URL: https://issues.apache.org/jira/browse/HDFS-15925
 Project: Hadoop HDFS
  Issue Type: Bug
  Components: datanode
Affects Versions: 3.2.2
Reporter: Haoze Wu


    When the datanode is receiving data block packets from a HDFS client and 
forwarding these packets to a mirror (another datanode) simultaneously, a 
single IOException in the datanode’s forwarding path can cause the client to 
get stuck for 1 min, without any logging. After 1 min, the client’s log shows a 
warning of EOFException and `Slow waitForAckedSeqno took 60106ms 
(threshold=3ms)`.

    Normally the datanode will inform the client of this error state 
immediately, and then the client will resend the packets immediately. The whole 
process is very fast. After careful analyses, we find the above symptom is due 
to the lack of packet-level mirrorError state synchronization in 
BlockReceiver$PacketResponder: in some concurrency condition, the 
BlockReceiver$PacketResponder will hang for 1 min and then exit, without 
sending the error state to the client.

*Root Cause Analysis*

 
{code:java}
//hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java

class BlockReceiver implements Closeable {
  // ...

  private void handleMirrorOutError(IOException ioe) throws IOException {
// ...
if (Thread.interrupted()) {
  throw ioe;
} else { // encounter an error while writing to mirror
  // continue to run even if can not write to mirror
  // notify client of the error
  // and wait for the client to shut down the pipeline
  mirrorError = true;// line 461
}
  }

  private int receivePacket() throws IOException {
// read the next packet
packetReceiver.receiveNextPacket(in);// line 528
// ...
boolean lastPacketInBlock = header.isLastPacketInBlock();// line 551
//First write the packet to the mirror:
if (mirrorOut != null && !mirrorError) {
  try {
// ...
packetReceiver.mirrorPacketTo(mirrorOut);// line 588
// ...
  } catch (IOException e) {
handleMirrorOutError(e); // line 604
  }
}
// ...
return lastPacketInBlock?-1:len; // line 849
  }

  void receiveBlock(...) throws IOException {
// ...
try {
  if (isClient && !isTransfer) {
responder = new Daemon(datanode.threadGroup, 
new PacketResponder(replyOut, mirrIn, downstreams));
responder.start();   // line 968
  }

  while(receivePacket() >= 0){/*Receive until the last packet*/} // line 971

  // wait for all outstanding packet responses. And then
  // indicate responder to gracefully shutdown.
  // Mark that responder has been closed for future processing
  if (responder != null) {
((PacketResponder)responder.getRunnable()).close();  // line 977
responderClosed = true;
  }
  // ...
} catch (IOException ioe) {  // line 
1003
  // ...
} finally {
  // ...
  if (!responderClosed) { // Data transfer was not complete.
if (responder != null) {
  // ...
  responder.interrupt(); // line 
1046
}
// ...
  }
  if (responder != null) {
try {
  responder.interrupt(); // line 
1053
  // ...
} catch (InterruptedException e) {
  responder.interrupt(); // line 
1067
  // ...
}
// ...
  }
}
  }
}
{code}
    In the `BlockReceiver.receivePacket` method, if the datanode fails to 
forward the packet to the mirror ( (line 588) due to an IOException, it is 
handled by line 604, which sets the mirrorError flag in line 461. According to 
the comments, the BlockReceiver keeps going with the mirrorError state, and the 
client would be notified of the error.

 

    However, jstack shows that the datanode gets stuck in the `DataXceiver` 
thread (receiving data block packets from client) and the 
`BlockReceiver$PacketResponder` thread (replying ACK packets to client). In 
particular, the `DataXceiver` thread gets stuck in the loop in line 971, which 
is further caused by blocking in line 528, meaning that the `lastPacketInBlock` 
packet has not arrived, and no more packets are coming in.

 
{code:java}

[jira] [Work logged] (HDFS-15879) Exclude slow nodes when choose targets for blocks

2021-03-26 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/HDFS-15879?focusedWorklogId=572848=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-572848
 ]

ASF GitHub Bot logged work on HDFS-15879:
-

Author: ASF GitHub Bot
Created on: 26/Mar/21 20:06
Start Date: 26/Mar/21 20:06
Worklog Time Spent: 10m 
  Work Description: hadoop-yetus commented on pull request #2748:
URL: https://github.com/apache/hadoop/pull/2748#issuecomment-808480490


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |::|--:|:|::|:---:|
   | +0 :ok: |  reexec  |   0m 58s |  |  Docker mode activated.  |
    _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files 
found.  |
   | +0 :ok: |  codespell  |   0m  1s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain 
any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to 
include 1 new or modified test files.  |
    _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  36m 14s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   1m 27s |  |  trunk passed with JDK 
Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |   1m 24s |  |  trunk passed with JDK 
Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   1m  9s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   1m 31s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 55s |  |  trunk passed with JDK 
Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   1m 33s |  |  trunk passed with JDK 
Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   3m 40s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  19m  4s |  |  branch has no errors 
when building and testing our client artifacts.  |
    _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   1m 15s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 21s |  |  the patch passed with JDK 
Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |   1m 21s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 18s |  |  the patch passed with JDK 
Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  javac  |   1m 18s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks 
issues.  |
   | +1 :green_heart: |  checkstyle  |   1m  3s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   1m 32s |  |  the patch passed  |
   | +1 :green_heart: |  xml  |   0m  1s |  |  The patch has no ill-formed XML 
file.  |
   | +1 :green_heart: |  javadoc  |   0m 59s |  |  the patch passed with JDK 
Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   1m 26s |  |  the patch passed with JDK 
Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   3m 49s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  21m 53s |  |  patch has no errors 
when building and testing our client artifacts.  |
    _ Other Tests _ |
   | -1 :x: |  unit  | 378m  1s | 
[/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2748/11/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt)
 |  hadoop-hdfs in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 37s |  |  The patch does not 
generate ASF License warnings.  |
   |  |   | 477m 54s |  |  |
   
   
   | Reason | Tests |
   |---:|:--|
   | Failed junit tests | hadoop.hdfs.server.datanode.TestBlockScanner |
   |   | hadoop.hdfs.TestViewDistributedFileSystemWithMountLinks |
   |   | hadoop.hdfs.TestViewDistributedFileSystem |
   |   | hadoop.hdfs.server.namenode.ha.TestBootstrapStandby |
   |   | hadoop.hdfs.TestPersistBlocks |
   |   | hadoop.hdfs.server.datanode.TestBlockRecovery |
   |   | hadoop.hdfs.server.datanode.TestDataNodeUUID |
   |   | hadoop.hdfs.server.namenode.ha.TestEditLogTailer |
   |   | hadoop.hdfs.TestDFSShell |
   |   | hadoop.hdfs.server.namenode.snapshot.TestNestedSnapshots |
   |   | hadoop.hdfs.TestSnapshotCommands |
   |   | hadoop.hdfs.server.datanode.TestIncrementalBrVariations |
   |   | hadoop.hdfs.server.datanode.fsdataset.impl.TestFsVolumeList |
   |   | 
hadoop.hdfs.server.namenode.TestDecommissioningStatusWithBackoffMonitor |
   |   | hadoop.hdfs.server.namenode.TestDecommissioningStatus |
   
   
   | Subsystem | Report/Notes |
   |--:|:-|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: 
https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2748/11/artifact/out/Dockerfile
 |
   | GITHUB PR | 

[jira] [Work logged] (HDFS-15879) Exclude slow nodes when choose targets for blocks

2021-03-26 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/HDFS-15879?focusedWorklogId=572836=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-572836
 ]

ASF GitHub Bot logged work on HDFS-15879:
-

Author: ASF GitHub Bot
Created on: 26/Mar/21 19:42
Start Date: 26/Mar/21 19:42
Worklog Time Spent: 10m 
  Work Description: hadoop-yetus commented on pull request #2748:
URL: https://github.com/apache/hadoop/pull/2748#issuecomment-808468887


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |::|--:|:|::|:---:|
   | +0 :ok: |  reexec  |   0m 50s |  |  Docker mode activated.  |
    _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files 
found.  |
   | +0 :ok: |  codespell  |   0m  1s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain 
any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to 
include 1 new or modified test files.  |
    _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  35m 19s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   1m 29s |  |  trunk passed with JDK 
Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |   1m 24s |  |  trunk passed with JDK 
Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   1m  7s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   1m 34s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 55s |  |  trunk passed with JDK 
Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   1m 26s |  |  trunk passed with JDK 
Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   3m 22s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  19m  0s |  |  branch has no errors 
when building and testing our client artifacts.  |
    _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   1m 24s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 26s |  |  the patch passed with JDK 
Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |   1m 26s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 20s |  |  the patch passed with JDK 
Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  javac  |   1m 20s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks 
issues.  |
   | -0 :warning: |  checkstyle  |   0m 58s | 
[/results-checkstyle-hadoop-hdfs-project_hadoop-hdfs.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2748/10/artifact/out/results-checkstyle-hadoop-hdfs-project_hadoop-hdfs.txt)
 |  hadoop-hdfs-project/hadoop-hdfs: The patch generated 1 new + 535 unchanged 
- 0 fixed = 536 total (was 535)  |
   | +1 :green_heart: |  mvnsite  |   1m 23s |  |  the patch passed  |
   | +1 :green_heart: |  xml  |   0m  1s |  |  The patch has no ill-formed XML 
file.  |
   | +1 :green_heart: |  javadoc  |   0m 52s |  |  the patch passed with JDK 
Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   1m 19s |  |  the patch passed with JDK 
Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   3m 34s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  19m  1s |  |  patch has no errors 
when building and testing our client artifacts.  |
    _ Other Tests _ |
   | -1 :x: |  unit  | 383m 11s | 
[/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2748/10/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt)
 |  hadoop-hdfs in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 42s |  |  The patch does not 
generate ASF License warnings.  |
   |  |   | 478m 50s |  |  |
   
   
   | Reason | Tests |
   |---:|:--|
   | Failed junit tests | hadoop.hdfs.server.balancer.TestBalancer |
   |   | hadoop.hdfs.TestStateAlignmentContextWithHA |
   |   | hadoop.hdfs.server.namenode.ha.TestPipelinesFailover |
   |   | hadoop.hdfs.server.datanode.TestBlockScanner |
   |   | hadoop.hdfs.TestViewDistributedFileSystemWithMountLinks |
   |   | hadoop.hdfs.server.namenode.ha.TestBootstrapStandby |
   |   | hadoop.hdfs.TestPersistBlocks |
   |   | hadoop.hdfs.TestLeaseRecovery |
   |   | hadoop.hdfs.TestLeaseRecovery2 |
   |   | hadoop.hdfs.server.datanode.TestBlockRecovery |
   |   | hadoop.hdfs.server.namenode.ha.TestEditLogTailer |
   |   | hadoop.hdfs.TestDFSShell |
   |   | hadoop.hdfs.server.namenode.snapshot.TestNestedSnapshots |
   |   | hadoop.hdfs.server.datanode.fsdataset.impl.TestFsDatasetImpl |
   |   | hadoop.hdfs.server.datanode.fsdataset.impl.TestFsVolumeList |
 

[jira] [Commented] (HDFS-15922) Use memcpy for copying non-null terminated string in jni_helper.c

2021-03-26 Thread Jira


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

Íñigo Goiri commented on HDFS-15922:


Thanks [~gautham] for the fix, merged the PR.

> Use memcpy for copying non-null terminated string in jni_helper.c
> -
>
> Key: HDFS-15922
> URL: https://issues.apache.org/jira/browse/HDFS-15922
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: libhdfs++
>Affects Versions: 3.4.0
>Reporter: Gautham Banasandra
>Assignee: Gautham Banasandra
>Priority: Major
>  Labels: pull-request-available
> Fix For: 3.4.0
>
>  Time Spent: 40m
>  Remaining Estimate: 0h
>
> We currently get a warning while compiling HDFS native client -
> {code}
> [WARNING] inlined from 'wildcard_expandPath' at 
> /home/jenkins/jenkins-home/workspace/hadoop-multibranch_PR-2792/src/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/jni_helper.c:427:21,
> [WARNING] /usr/include/x86_64-linux-gnu/bits/string_fortified.h:106:10: 
> warning: '__builtin_strncpy' output truncated before terminating nul copying 
> as many bytes from a string as its length [-Wstringop-truncation]
> [WARNING] 
> /home/jenkins/jenkins-home/workspace/hadoop-multibranch_PR-2792/src/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/jni_helper.c:402:43:
>  note: length computed here
> {code}
> The scenario here is such that the copied string is deliberately not null 
> terminated, since we want to insert a PATH_SEPARATOR ourselves. The warning 
> reported by strncpy is valid, but not applicable in this scenario. Thus, we 
> need to use memcpy which doesn't mind if the string is null terminated or not.



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

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



[jira] [Resolved] (HDFS-15922) Use memcpy for copying non-null terminated string in jni_helper.c

2021-03-26 Thread Jira


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

Íñigo Goiri resolved HDFS-15922.

Fix Version/s: 3.4.0
 Hadoop Flags: Reviewed
   Resolution: Fixed

> Use memcpy for copying non-null terminated string in jni_helper.c
> -
>
> Key: HDFS-15922
> URL: https://issues.apache.org/jira/browse/HDFS-15922
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: libhdfs++
>Affects Versions: 3.4.0
>Reporter: Gautham Banasandra
>Assignee: Gautham Banasandra
>Priority: Major
>  Labels: pull-request-available
> Fix For: 3.4.0
>
>  Time Spent: 40m
>  Remaining Estimate: 0h
>
> We currently get a warning while compiling HDFS native client -
> {code}
> [WARNING] inlined from 'wildcard_expandPath' at 
> /home/jenkins/jenkins-home/workspace/hadoop-multibranch_PR-2792/src/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/jni_helper.c:427:21,
> [WARNING] /usr/include/x86_64-linux-gnu/bits/string_fortified.h:106:10: 
> warning: '__builtin_strncpy' output truncated before terminating nul copying 
> as many bytes from a string as its length [-Wstringop-truncation]
> [WARNING] 
> /home/jenkins/jenkins-home/workspace/hadoop-multibranch_PR-2792/src/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/jni_helper.c:402:43:
>  note: length computed here
> {code}
> The scenario here is such that the copied string is deliberately not null 
> terminated, since we want to insert a PATH_SEPARATOR ourselves. The warning 
> reported by strncpy is valid, but not applicable in this scenario. Thus, we 
> need to use memcpy which doesn't mind if the string is null terminated or not.



--
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] [Work logged] (HDFS-15922) Use memcpy for copying non-null terminated string in jni_helper.c

2021-03-26 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/HDFS-15922?focusedWorklogId=572733=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-572733
 ]

ASF GitHub Bot logged work on HDFS-15922:
-

Author: ASF GitHub Bot
Created on: 26/Mar/21 16:26
Start Date: 26/Mar/21 16:26
Worklog Time Spent: 10m 
  Work Description: goiri merged pull request #2818:
URL: https://github.com/apache/hadoop/pull/2818


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
---

Worklog Id: (was: 572733)
Time Spent: 40m  (was: 0.5h)

> Use memcpy for copying non-null terminated string in jni_helper.c
> -
>
> Key: HDFS-15922
> URL: https://issues.apache.org/jira/browse/HDFS-15922
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: libhdfs++
>Affects Versions: 3.4.0
>Reporter: Gautham Banasandra
>Assignee: Gautham Banasandra
>Priority: Major
>  Labels: pull-request-available
>  Time Spent: 40m
>  Remaining Estimate: 0h
>
> We currently get a warning while compiling HDFS native client -
> {code}
> [WARNING] inlined from 'wildcard_expandPath' at 
> /home/jenkins/jenkins-home/workspace/hadoop-multibranch_PR-2792/src/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/jni_helper.c:427:21,
> [WARNING] /usr/include/x86_64-linux-gnu/bits/string_fortified.h:106:10: 
> warning: '__builtin_strncpy' output truncated before terminating nul copying 
> as many bytes from a string as its length [-Wstringop-truncation]
> [WARNING] 
> /home/jenkins/jenkins-home/workspace/hadoop-multibranch_PR-2792/src/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/jni_helper.c:402:43:
>  note: length computed here
> {code}
> The scenario here is such that the copied string is deliberately not null 
> terminated, since we want to insert a PATH_SEPARATOR ourselves. The warning 
> reported by strncpy is valid, but not applicable in this scenario. Thus, we 
> need to use memcpy which doesn't mind if the string is null terminated or not.



--
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-15924) Log4j will cause Server handler blocked when audit log boom.

2021-03-26 Thread Qi Zhu (Jira)


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

Qi Zhu commented on HDFS-15924:
---

[~marvelrock]

The version of this cluster is 2.6.0-cdh5.11.0.

> Log4j will cause Server handler blocked when audit log boom.
> 
>
> Key: HDFS-15924
> URL: https://issues.apache.org/jira/browse/HDFS-15924
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: Qi Zhu
>Priority: Major
> Attachments: image-2021-03-26-16-18-03-341.png, 
> image-2021-03-26-16-19-42-165.png
>
>
> !image-2021-03-26-16-18-03-341.png|width=707,height=234!
> !image-2021-03-26-16-19-42-165.png|width=824,height=198!
> The thread blocked when audit log boom show in above.
> Such as [https://dzone.com/articles/log4j-thread-deadlock-case] , it seems 
> the same case when heavy load, should we update to Log4j2 or other things we 
> can do to improve it in heavy audit log.
>  
> {code:java}
>  /**
>  Call the appenders in the hierrachy starting at
>  this.  If no appenders could be found, emit a
>  warning.
>  This method calls all the appenders inherited from the
>  hierarchy circumventing any evaluation of whether to log or not
>  to log the particular log request.
>  @param event the event to log.  */
> public void callAppenders(LoggingEvent event) {
> int writes = 0;
> for(Category c = this; c != null; c=c.parent) {
>   // Protected against simultaneous call to addAppender, 
> removeAppender,...
>   synchronized(c) {
> if(c.aai != null) {
> writes += c.aai.appendLoopOnAppenders(event);
> }
> if(!c.additive) {
> break;
> }
>   }
> }
> if(writes == 0) {
>   repository.emitNoAppenderWarning(this);
> }
>   }{code}
> The log4j code, use the  global synchronized, it will cause this happened.
> cc [~weichiu] [~hexiaoqiao] [~ayushtkn]  [~shv] [~ferhui]



--
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-15924) Log4j will cause Server handler blocked when audit log boom.

2021-03-26 Thread HuangTao (Jira)


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

HuangTao commented on HDFS-15924:
-

Which Hadoop version used in your cluster?

> Log4j will cause Server handler blocked when audit log boom.
> 
>
> Key: HDFS-15924
> URL: https://issues.apache.org/jira/browse/HDFS-15924
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: Qi Zhu
>Priority: Major
> Attachments: image-2021-03-26-16-18-03-341.png, 
> image-2021-03-26-16-19-42-165.png
>
>
> !image-2021-03-26-16-18-03-341.png|width=707,height=234!
> !image-2021-03-26-16-19-42-165.png|width=824,height=198!
> The thread blocked when audit log boom show in above.
> Such as [https://dzone.com/articles/log4j-thread-deadlock-case] , it seems 
> the same case when heavy load, should we update to Log4j2 or other things we 
> can do to improve it in heavy audit log.
>  
> {code:java}
>  /**
>  Call the appenders in the hierrachy starting at
>  this.  If no appenders could be found, emit a
>  warning.
>  This method calls all the appenders inherited from the
>  hierarchy circumventing any evaluation of whether to log or not
>  to log the particular log request.
>  @param event the event to log.  */
> public void callAppenders(LoggingEvent event) {
> int writes = 0;
> for(Category c = this; c != null; c=c.parent) {
>   // Protected against simultaneous call to addAppender, 
> removeAppender,...
>   synchronized(c) {
> if(c.aai != null) {
> writes += c.aai.appendLoopOnAppenders(event);
> }
> if(!c.additive) {
> break;
> }
>   }
> }
> if(writes == 0) {
>   repository.emitNoAppenderWarning(this);
> }
>   }{code}
> The log4j code, use the  global synchronized, it will cause this happened.
> cc [~weichiu] [~hexiaoqiao] [~ayushtkn]  [~shv] [~ferhui]



--
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-15924) Log4j will cause Server handler blocked when audit log boom.

2021-03-26 Thread Qi Zhu (Jira)


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

Qi Zhu commented on HDFS-15924:
---

Thanks [~sodonnell] for reply.

Actually the async audit logging is already enabled in our cluster.
{code:java}

  dfs.namenode.audit.log.async
  true
{code}
But the thread blocked when audit log boom still happened, and the single 
cluster has thousands of nodes.

> Log4j will cause Server handler blocked when audit log boom.
> 
>
> Key: HDFS-15924
> URL: https://issues.apache.org/jira/browse/HDFS-15924
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: Qi Zhu
>Priority: Major
> Attachments: image-2021-03-26-16-18-03-341.png, 
> image-2021-03-26-16-19-42-165.png
>
>
> !image-2021-03-26-16-18-03-341.png|width=707,height=234!
> !image-2021-03-26-16-19-42-165.png|width=824,height=198!
> The thread blocked when audit log boom show in above.
> Such as [https://dzone.com/articles/log4j-thread-deadlock-case] , it seems 
> the same case when heavy load, should we update to Log4j2 or other things we 
> can do to improve it in heavy audit log.
>  
> {code:java}
>  /**
>  Call the appenders in the hierrachy starting at
>  this.  If no appenders could be found, emit a
>  warning.
>  This method calls all the appenders inherited from the
>  hierarchy circumventing any evaluation of whether to log or not
>  to log the particular log request.
>  @param event the event to log.  */
> public void callAppenders(LoggingEvent event) {
> int writes = 0;
> for(Category c = this; c != null; c=c.parent) {
>   // Protected against simultaneous call to addAppender, 
> removeAppender,...
>   synchronized(c) {
> if(c.aai != null) {
> writes += c.aai.appendLoopOnAppenders(event);
> }
> if(!c.additive) {
> break;
> }
>   }
> }
> if(writes == 0) {
>   repository.emitNoAppenderWarning(this);
> }
>   }{code}
> The log4j code, use the  global synchronized, it will cause this happened.
> cc [~weichiu] [~hexiaoqiao] [~ayushtkn]  [~shv] [~ferhui]



--
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] [Work logged] (HDFS-15879) Exclude slow nodes when choose targets for blocks

2021-03-26 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/HDFS-15879?focusedWorklogId=572619=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-572619
 ]

ASF GitHub Bot logged work on HDFS-15879:
-

Author: ASF GitHub Bot
Created on: 26/Mar/21 11:22
Start Date: 26/Mar/21 11:22
Worklog Time Spent: 10m 
  Work Description: tomscut commented on a change in pull request #2748:
URL: https://github.com/apache/hadoop/pull/2748#discussion_r602202963



##
File path: 
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
##
@@ -201,8 +206,16 @@
*/
   private final boolean useDfsNetworkTopology;
 
+  private static final String IP_PORT_SEPARATOR = ":";
+
   @Nullable
   private final SlowPeerTracker slowPeerTracker;
+  private static Set slowPeers = Sets.newConcurrentHashSet();

Review comment:
   Sorry, I didn't see your reply just now. I will fix it soon.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
---

Worklog Id: (was: 572619)
Time Spent: 3h 40m  (was: 3.5h)

> Exclude slow nodes when choose targets for blocks
> -
>
> Key: HDFS-15879
> URL: https://issues.apache.org/jira/browse/HDFS-15879
> Project: Hadoop HDFS
>  Issue Type: Wish
>Reporter: tomscut
>Assignee: tomscut
>Priority: Major
>  Labels: pull-request-available
>  Time Spent: 3h 40m
>  Remaining Estimate: 0h
>
> Previously, we have monitored the slow nodes, related to 
> [HDFS-11194|https://issues.apache.org/jira/browse/HDFS-11194].
> We can use a thread to periodically collect these slow nodes into a set. Then 
> use the set to filter out slow nodes when choose targets for blocks.
> This feature can be configured to be turned on when needed.



--
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] [Work logged] (HDFS-15879) Exclude slow nodes when choose targets for blocks

2021-03-26 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/HDFS-15879?focusedWorklogId=572615=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-572615
 ]

ASF GitHub Bot logged work on HDFS-15879:
-

Author: ASF GitHub Bot
Created on: 26/Mar/21 11:19
Start Date: 26/Mar/21 11:19
Worklog Time Spent: 10m 
  Work Description: tomscut commented on pull request #2748:
URL: https://github.com/apache/hadoop/pull/2748#issuecomment-808133427


   Hi @tasanuma , those failed unit tests are unrelated to the change, and they 
work fine locally. Please take a look at the new commit, thank you.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
---

Worklog Id: (was: 572615)
Time Spent: 3.5h  (was: 3h 20m)

> Exclude slow nodes when choose targets for blocks
> -
>
> Key: HDFS-15879
> URL: https://issues.apache.org/jira/browse/HDFS-15879
> Project: Hadoop HDFS
>  Issue Type: Wish
>Reporter: tomscut
>Assignee: tomscut
>Priority: Major
>  Labels: pull-request-available
>  Time Spent: 3.5h
>  Remaining Estimate: 0h
>
> Previously, we have monitored the slow nodes, related to 
> [HDFS-11194|https://issues.apache.org/jira/browse/HDFS-11194].
> We can use a thread to periodically collect these slow nodes into a set. Then 
> use the set to filter out slow nodes when choose targets for blocks.
> This feature can be configured to be turned on when needed.



--
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-15745) Make DataNodePeerMetrics#LOW_THRESHOLD_MS and MIN_OUTLIER_DETECTION_NODES configurable

2021-03-26 Thread Haibin Huang (Jira)


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

Haibin Huang commented on HDFS-15745:
-

Thanks [~prasad-acit] for comment, i have update this patch for branch 3.1 & 
3.2 &3.3, [~ayushtkn] would you mind commit them?

> Make DataNodePeerMetrics#LOW_THRESHOLD_MS and MIN_OUTLIER_DETECTION_NODES 
> configurable
> --
>
> Key: HDFS-15745
> URL: https://issues.apache.org/jira/browse/HDFS-15745
> Project: Hadoop HDFS
>  Issue Type: Improvement
>Reporter: Haibin Huang
>Assignee: Haibin Huang
>Priority: Major
> Fix For: 3.4.0
>
> Attachments: HDFS-15745-001.patch, HDFS-15745-002.patch, 
> HDFS-15745-003.patch, HDFS-15745-branch-3.1.001.patch, 
> HDFS-15745-branch-3.2.001.patch, HDFS-15745-branch-3.3.001.patch, 
> image-2020-12-22-17-00-50-796.png
>
>
> When i enable DataNodePeerMetrics to find slow slow peer in cluster, i found 
> there is a lot of slow peer but ReportingNodes's averageDelay is very low, 
> and these slow peer node are normal. I think the reason of why generating so 
> many slow peer is that  the value of DataNodePeerMetrics#LOW_THRESHOLD_MS is 
> too small (only 5ms) and it is not configurable. The default value of slow io 
> warning log threshold is 300ms, i.e. 
> DFSConfigKeys.DFS_DATANODE_SLOW_IO_WARNING_THRESHOLD_DEFAULT = 300, so 
> DataNodePeerMetrics#LOW_THRESHOLD_MS should not be less than 300ms, otherwise 
> namenode will get a lot of invalid slow peer information.
> !image-2020-12-22-17-00-50-796.png!



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

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



[jira] [Updated] (HDFS-15745) Make DataNodePeerMetrics#LOW_THRESHOLD_MS and MIN_OUTLIER_DETECTION_NODES configurable

2021-03-26 Thread Haibin Huang (Jira)


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

Haibin Huang updated HDFS-15745:

Attachment: HDFS-15745-branch-3.3.001.patch
HDFS-15745-branch-3.2.001.patch
HDFS-15745-branch-3.1.001.patch

> Make DataNodePeerMetrics#LOW_THRESHOLD_MS and MIN_OUTLIER_DETECTION_NODES 
> configurable
> --
>
> Key: HDFS-15745
> URL: https://issues.apache.org/jira/browse/HDFS-15745
> Project: Hadoop HDFS
>  Issue Type: Improvement
>Reporter: Haibin Huang
>Assignee: Haibin Huang
>Priority: Major
> Fix For: 3.4.0
>
> Attachments: HDFS-15745-001.patch, HDFS-15745-002.patch, 
> HDFS-15745-003.patch, HDFS-15745-branch-3.1.001.patch, 
> HDFS-15745-branch-3.2.001.patch, HDFS-15745-branch-3.3.001.patch, 
> image-2020-12-22-17-00-50-796.png
>
>
> When i enable DataNodePeerMetrics to find slow slow peer in cluster, i found 
> there is a lot of slow peer but ReportingNodes's averageDelay is very low, 
> and these slow peer node are normal. I think the reason of why generating so 
> many slow peer is that  the value of DataNodePeerMetrics#LOW_THRESHOLD_MS is 
> too small (only 5ms) and it is not configurable. The default value of slow io 
> warning log threshold is 300ms, i.e. 
> DFSConfigKeys.DFS_DATANODE_SLOW_IO_WARNING_THRESHOLD_DEFAULT = 300, so 
> DataNodePeerMetrics#LOW_THRESHOLD_MS should not be less than 300ms, otherwise 
> namenode will get a lot of invalid slow peer information.
> !image-2020-12-22-17-00-50-796.png!



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

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



[jira] [Work logged] (HDFS-15850) Superuser actions should be reported to external enforcers

2021-03-26 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/HDFS-15850?focusedWorklogId=572607=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-572607
 ]

ASF GitHub Bot logged work on HDFS-15850:
-

Author: ASF GitHub Bot
Created on: 26/Mar/21 10:51
Start Date: 26/Mar/21 10:51
Worklog Time Spent: 10m 
  Work Description: hadoop-yetus commented on pull request #2784:
URL: https://github.com/apache/hadoop/pull/2784#issuecomment-808116576


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |::|--:|:|::|:---:|
   | +0 :ok: |  reexec  |  21m 31s |  |  Docker mode activated.  |
    _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  |  No case conflicting files 
found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain 
any @author tags.  |
   | -1 :x: |  test4tests  |   0m  0s |  |  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.  
|
    _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |  14m  7s |  |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  23m 44s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   5m 49s |  |  trunk passed with JDK 
Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |   5m 29s |  |  trunk passed with JDK 
Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   1m 19s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   2m  3s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   1m 32s |  |  trunk passed with JDK 
Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   2m 14s |  |  trunk passed with JDK 
Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   4m 35s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  17m 13s |  |  branch has no errors 
when building and testing our client artifacts.  |
    _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 25s |  |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m 54s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   5m 23s |  |  the patch passed with JDK 
Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |   5m 23s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   4m 52s |  |  the patch passed with JDK 
Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  javac  |   4m 52s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks 
issues.  |
   | +1 :green_heart: |  checkstyle  |   1m 10s |  |  hadoop-hdfs-project: The 
patch generated 0 new + 498 unchanged - 6 fixed = 498 total (was 504)  |
   | +1 :green_heart: |  mvnsite  |   1m 48s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   1m 24s |  |  the patch passed with JDK 
Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   2m 18s |  |  the patch passed with JDK 
Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | -1 :x: |  spotbugs  |   3m 45s | 
[/new-spotbugs-hadoop-hdfs-project_hadoop-hdfs.html](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2784/9/artifact/out/new-spotbugs-hadoop-hdfs-project_hadoop-hdfs.html)
 |  hadoop-hdfs-project/hadoop-hdfs generated 1 new + 0 unchanged - 0 fixed = 1 
total (was 0)  |
   | +1 :green_heart: |  shadedclient  |  19m 28s |  |  patch has no errors 
when building and testing our client artifacts.  |
    _ Other Tests _ |
   | -1 :x: |  unit  | 397m  5s | 
[/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2784/9/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt)
 |  hadoop-hdfs in the patch passed.  |
   | -1 :x: |  unit  |  23m 22s | 
[/patch-unit-hadoop-hdfs-project_hadoop-hdfs-rbf.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2784/9/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs-rbf.txt)
 |  hadoop-hdfs-rbf in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 37s |  |  The patch does not 
generate ASF License warnings.  |
   |  |   | 566m 23s |  |  |
   
   
   | Reason | Tests |
   |---:|:--|
   | SpotBugs | module:hadoop-hdfs-project/hadoop-hdfs |
   |  |  Possible null pointer dereference of r in 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.truncate(String, long, 
String, String, long)  Dereferenced at FSNamesystem.java:r in 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.truncate(String, long, 
String, String, long)  

[jira] [Commented] (HDFS-15924) Log4j will cause Server handler blocked when audit log boom.

2021-03-26 Thread Stephen O'Donnell (Jira)


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

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

Do you have async audit logging enabled, and does the problem still occur?

{code}
  public static final String  DFS_NAMENODE_AUDIT_LOG_ASYNC_KEY = 
"dfs.namenode.audit.log.async";
  public static final boolean DFS_NAMENODE_AUDIT_LOG_ASYNC_DEFAULT = false;
{code}

Aysnc is off by default.

> Log4j will cause Server handler blocked when audit log boom.
> 
>
> Key: HDFS-15924
> URL: https://issues.apache.org/jira/browse/HDFS-15924
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: Qi Zhu
>Priority: Major
> Attachments: image-2021-03-26-16-18-03-341.png, 
> image-2021-03-26-16-19-42-165.png
>
>
> !image-2021-03-26-16-18-03-341.png|width=707,height=234!
> !image-2021-03-26-16-19-42-165.png|width=824,height=198!
> The thread blocked when audit log boom show in above.
> Such as [https://dzone.com/articles/log4j-thread-deadlock-case] , it seems 
> the same case when heavy load, should we update to Log4j2 or other things we 
> can do to improve it in heavy audit log.
>  
> {code:java}
>  /**
>  Call the appenders in the hierrachy starting at
>  this.  If no appenders could be found, emit a
>  warning.
>  This method calls all the appenders inherited from the
>  hierarchy circumventing any evaluation of whether to log or not
>  to log the particular log request.
>  @param event the event to log.  */
> public void callAppenders(LoggingEvent event) {
> int writes = 0;
> for(Category c = this; c != null; c=c.parent) {
>   // Protected against simultaneous call to addAppender, 
> removeAppender,...
>   synchronized(c) {
> if(c.aai != null) {
> writes += c.aai.appendLoopOnAppenders(event);
> }
> if(!c.additive) {
> break;
> }
>   }
> }
> if(writes == 0) {
>   repository.emitNoAppenderWarning(this);
> }
>   }{code}
> The log4j code, use the  global synchronized, it will cause this happened.
> cc [~weichiu] [~hexiaoqiao] [~ayushtkn]  [~shv] [~ferhui]



--
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] [Work logged] (HDFS-15869) Network issue while FSEditLogAsync is executing RpcEdit.logSyncNotify can cause the namenode to hang

2021-03-26 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/HDFS-15869?focusedWorklogId=572606=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-572606
 ]

ASF GitHub Bot logged work on HDFS-15869:
-

Author: ASF GitHub Bot
Created on: 26/Mar/21 10:48
Start Date: 26/Mar/21 10:48
Worklog Time Spent: 10m 
  Work Description: linyiqun commented on a change in pull request #2737:
URL: https://github.com/apache/hadoop/pull/2737#discussion_r602183347



##
File path: 
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogAsync.java
##
@@ -63,6 +68,9 @@
 DFS_NAMENODE_EDITS_ASYNC_LOGGING_PENDING_QUEUE_SIZE_DEFAULT);
 
 editPendingQ = new ArrayBlockingQueue<>(editPendingQSize);
+
+// the thread pool size should be configurable later, and justified with a 
rationale
+logSyncNotifyExecutor = Executors.newFixedThreadPool(10);

Review comment:
   @functioner , we could make 10 as the default pool size.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
---

Worklog Id: (was: 572606)
Time Spent: 1.5h  (was: 1h 20m)

> Network issue while FSEditLogAsync is executing RpcEdit.logSyncNotify can 
> cause the namenode to hang
> 
>
> Key: HDFS-15869
> URL: https://issues.apache.org/jira/browse/HDFS-15869
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: fs async, namenode
>Affects Versions: 3.2.2
>Reporter: Haoze Wu
>Priority: Critical
>  Labels: pull-request-available
>  Time Spent: 1.5h
>  Remaining Estimate: 0h
>
>     We were doing some testing of the latest Hadoop stable release 3.2.2 and 
> found some network issue can cause the namenode to hang even with the async 
> edit logging (FSEditLogAsync).
>     The workflow of the FSEditLogAsync thread is basically:
>  # get EditLog from a queue (line 229)
>  # do the transaction (line 232)
>  # sync the log if doSync (line 243)
>  # do logSyncNotify (line 248)
> {code:java}
> //hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogAsync.java
>   @Override
>   public void run() {
> try {
>   while (true) {
> boolean doSync;
> Edit edit = dequeueEdit(); // 
> line 229
> if (edit != null) {
>   // sync if requested by edit log.
>   doSync = edit.logEdit(); // 
> line 232
>   syncWaitQ.add(edit);
> } else {
>   // sync when editq runs dry, but have edits pending a sync.
>   doSync = !syncWaitQ.isEmpty();
> }
> if (doSync) {
>   // normally edit log exceptions cause the NN to terminate, but tests
>   // relying on ExitUtil.terminate need to see the exception.
>   RuntimeException syncEx = null;
>   try {
> logSync(getLastWrittenTxId()); // 
> line 243
>   } catch (RuntimeException ex) {
> syncEx = ex;
>   }
>   while ((edit = syncWaitQ.poll()) != null) {
> edit.logSyncNotify(syncEx);// 
> line 248
>   }
> }
>   }
> } catch (InterruptedException ie) {
>   LOG.info(Thread.currentThread().getName() + " was interrupted, 
> exiting");
> } catch (Throwable t) {
>   terminate(t);
> }
>   }
> {code}
>     In terms of the step 4, FSEditLogAsync$RpcEdit.logSyncNotify is 
> essentially doing some network write (line 365).
> {code:java}
> //hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogAsync.java
>   private static class RpcEdit extends Edit {
> // ...
> @Override
> public void logSyncNotify(RuntimeException syncEx) {
>   try {
> if (syncEx == null) {
>   call.sendResponse();   // line 
> 365
> } else {
>   call.abortResponse(syncEx);
> }
>   } catch (Exception e) {} // don't care if not sent.
> }
> // ...
>   }{code}
>     If the sendResponse operation in line 365 gets stuck, then the whole 
> FSEditLogAsync thread is not able to proceed. In this case, the critical 
> logSync (line 243) can’t be executed, for the incoming transactions. Then the 
> namenode hangs. This is undesirable because FSEditLogAsync’s key feature is 
> asynchronous edit logging that is supposed to tolerate slow I/O.
> 

[jira] [Work logged] (HDFS-15879) Exclude slow nodes when choose targets for blocks

2021-03-26 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/HDFS-15879?focusedWorklogId=572596=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-572596
 ]

ASF GitHub Bot logged work on HDFS-15879:
-

Author: ASF GitHub Bot
Created on: 26/Mar/21 10:32
Start Date: 26/Mar/21 10:32
Worklog Time Spent: 10m 
  Work Description: tasanuma commented on a change in pull request #2748:
URL: https://github.com/apache/hadoop/pull/2748#discussion_r602173949



##
File path: 
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
##
@@ -201,8 +206,16 @@
*/
   private final boolean useDfsNetworkTopology;
 
+  private static final String IP_PORT_SEPARATOR = ":";
+
   @Nullable
   private final SlowPeerTracker slowPeerTracker;
+  private static Set slowPeers = Sets.newConcurrentHashSet();

Review comment:
   We might as well change this variable name to fix a checkstyle warning.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
---

Worklog Id: (was: 572596)
Time Spent: 3h 20m  (was: 3h 10m)

> Exclude slow nodes when choose targets for blocks
> -
>
> Key: HDFS-15879
> URL: https://issues.apache.org/jira/browse/HDFS-15879
> Project: Hadoop HDFS
>  Issue Type: Wish
>Reporter: tomscut
>Assignee: tomscut
>Priority: Major
>  Labels: pull-request-available
>  Time Spent: 3h 20m
>  Remaining Estimate: 0h
>
> Previously, we have monitored the slow nodes, related to 
> [HDFS-11194|https://issues.apache.org/jira/browse/HDFS-11194].
> We can use a thread to periodically collect these slow nodes into a set. Then 
> use the set to filter out slow nodes when choose targets for blocks.
> This feature can be configured to be turned on when needed.



--
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] [Work logged] (HDFS-15879) Exclude slow nodes when choose targets for blocks

2021-03-26 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/HDFS-15879?focusedWorklogId=572594=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-572594
 ]

ASF GitHub Bot logged work on HDFS-15879:
-

Author: ASF GitHub Bot
Created on: 26/Mar/21 10:27
Start Date: 26/Mar/21 10:27
Worklog Time Spent: 10m 
  Work Description: tasanuma commented on pull request #2748:
URL: https://github.com/apache/hadoop/pull/2748#issuecomment-808102983


   @tomscut Looks good to me, except for the checkstyle issues.
   
https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2748/9/artifact/out/results-checkstyle-hadoop-hdfs-project_hadoop-hdfs.txt
   
   Could you fix them? (I'm sorry that I should have mentioned it before.)
   If you use IntelliJ, I recommend CheckStyle-IDEA plugin. The configuration 
file is `hadoop-build-tools/src/main/resources/checkstyle/checkstyle.xml`.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
---

Worklog Id: (was: 572594)
Time Spent: 3h 10m  (was: 3h)

> Exclude slow nodes when choose targets for blocks
> -
>
> Key: HDFS-15879
> URL: https://issues.apache.org/jira/browse/HDFS-15879
> Project: Hadoop HDFS
>  Issue Type: Wish
>Reporter: tomscut
>Assignee: tomscut
>Priority: Major
>  Labels: pull-request-available
>  Time Spent: 3h 10m
>  Remaining Estimate: 0h
>
> Previously, we have monitored the slow nodes, related to 
> [HDFS-11194|https://issues.apache.org/jira/browse/HDFS-11194].
> We can use a thread to periodically collect these slow nodes into a set. Then 
> use the set to filter out slow nodes when choose targets for blocks.
> This feature can be configured to be turned on when needed.



--
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] [Work logged] (HDFS-15879) Exclude slow nodes when choose targets for blocks

2021-03-26 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/HDFS-15879?focusedWorklogId=572578=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-572578
 ]

ASF GitHub Bot logged work on HDFS-15879:
-

Author: ASF GitHub Bot
Created on: 26/Mar/21 09:59
Start Date: 26/Mar/21 09:59
Worklog Time Spent: 10m 
  Work Description: hadoop-yetus commented on pull request #2748:
URL: https://github.com/apache/hadoop/pull/2748#issuecomment-808086446


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |::|--:|:|::|:---:|
   | +0 :ok: |  reexec  |   2m 17s |  |  Docker mode activated.  |
    _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files 
found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain 
any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to 
include 1 new or modified test files.  |
    _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  36m 42s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   1m 26s |  |  trunk passed with JDK 
Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |   1m 21s |  |  trunk passed with JDK 
Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   1m 13s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   1m 43s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   1m  2s |  |  trunk passed with JDK 
Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   1m 28s |  |  trunk passed with JDK 
Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   4m  2s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  20m  1s |  |  branch has no errors 
when building and testing our client artifacts.  |
    _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   1m 13s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 17s |  |  the patch passed with JDK 
Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |   1m 17s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m  7s |  |  the patch passed with JDK 
Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  javac  |   1m  7s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks 
issues.  |
   | -0 :warning: |  checkstyle  |   0m 57s | 
[/results-checkstyle-hadoop-hdfs-project_hadoop-hdfs.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2748/9/artifact/out/results-checkstyle-hadoop-hdfs-project_hadoop-hdfs.txt)
 |  hadoop-hdfs-project/hadoop-hdfs: The patch generated 12 new + 535 unchanged 
- 0 fixed = 547 total (was 535)  |
   | +1 :green_heart: |  mvnsite  |   1m 15s |  |  the patch passed  |
   | +1 :green_heart: |  xml  |   0m  2s |  |  The patch has no ill-formed XML 
file.  |
   | +1 :green_heart: |  javadoc  |   0m 48s |  |  the patch passed with JDK 
Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   1m 18s |  |  the patch passed with JDK 
Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   3m 24s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  19m 54s |  |  patch has no errors 
when building and testing our client artifacts.  |
    _ Other Tests _ |
   | -1 :x: |  unit  | 395m 50s | 
[/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2748/9/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt)
 |  hadoop-hdfs in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 43s |  |  The patch does not 
generate ASF License warnings.  |
   |  |   | 496m  1s |  |  |
   
   
   | Reason | Tests |
   |---:|:--|
   | Failed junit tests | hadoop.hdfs.TestStateAlignmentContextWithHA |
   |   | hadoop.hdfs.web.TestWebHdfsFileSystemContract |
   |   | hadoop.hdfs.server.datanode.TestBlockScanner |
   |   | hadoop.hdfs.server.datanode.TestDataNodeVolumeFailure |
   |   | hadoop.hdfs.server.namenode.TestFileTruncate |
   |   | hadoop.hdfs.server.namenode.ha.TestBootstrapStandby |
   |   | hadoop.hdfs.TestPersistBlocks |
   |   | hadoop.hdfs.TestViewDistributedFileSystemContract |
   |   | hadoop.hdfs.server.datanode.TestBlockRecovery |
   |   | hadoop.hdfs.server.namenode.ha.TestEditLogTailer |
   |   | hadoop.hdfs.TestDFSShell |
   |   | hadoop.hdfs.server.namenode.snapshot.TestNestedSnapshots |
   |   | hadoop.hdfs.server.datanode.TestIncrementalBrVariations |
   |   | hadoop.hdfs.server.datanode.fsdataset.impl.TestFsVolumeList |
   |   | 

[jira] [Updated] (HDFS-15924) Log4j will cause Server handler blocked when audit log boom.

2021-03-26 Thread Qi Zhu (Jira)


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

Qi Zhu updated HDFS-15924:
--
Description: 
!image-2021-03-26-16-18-03-341.png|width=707,height=234!

!image-2021-03-26-16-19-42-165.png|width=824,height=198!

The thread blocked when audit log boom show in above.

Such as [https://dzone.com/articles/log4j-thread-deadlock-case] , it seems the 
same case when heavy load, should we update to Log4j2 or other things we can do 
to improve it in heavy audit log.

 
{code:java}
 /**
 Call the appenders in the hierrachy starting at
 this.  If no appenders could be found, emit a
 warning.

 This method calls all the appenders inherited from the
 hierarchy circumventing any evaluation of whether to log or not
 to log the particular log request.

 @param event the event to log.  */
public void callAppenders(LoggingEvent event) {
int writes = 0;

for(Category c = this; c != null; c=c.parent) {
  // Protected against simultaneous call to addAppender, removeAppender,...
  synchronized(c) {
if(c.aai != null) {
writes += c.aai.appendLoopOnAppenders(event);
}
if(!c.additive) {
break;
}
  }
}

if(writes == 0) {
  repository.emitNoAppenderWarning(this);
}
  }{code}
The log4j code, use the  global synchronized, it will cause this happened.

cc [~weichiu] [~hexiaoqiao] [~ayushtkn]  [~shv] [~ferhui]

  was:
!image-2021-03-26-16-18-03-341.png|width=707,height=234!

!image-2021-03-26-16-19-42-165.png|width=824,height=198!

The thread blocked when audit log boom show in above.

Such as [https://dzone.com/articles/log4j-thread-deadlock-case] , it seems the 
same case when heavy load, should we update to Log4j2 or other things we can do 
to improve it in heavy audit log.

cc [~weichiu] [~hexiaoqiao] [~ayushtkn]  [~shv] [~ferhui]


> Log4j will cause Server handler blocked when audit log boom.
> 
>
> Key: HDFS-15924
> URL: https://issues.apache.org/jira/browse/HDFS-15924
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: Qi Zhu
>Priority: Major
> Attachments: image-2021-03-26-16-18-03-341.png, 
> image-2021-03-26-16-19-42-165.png
>
>
> !image-2021-03-26-16-18-03-341.png|width=707,height=234!
> !image-2021-03-26-16-19-42-165.png|width=824,height=198!
> The thread blocked when audit log boom show in above.
> Such as [https://dzone.com/articles/log4j-thread-deadlock-case] , it seems 
> the same case when heavy load, should we update to Log4j2 or other things we 
> can do to improve it in heavy audit log.
>  
> {code:java}
>  /**
>  Call the appenders in the hierrachy starting at
>  this.  If no appenders could be found, emit a
>  warning.
>  This method calls all the appenders inherited from the
>  hierarchy circumventing any evaluation of whether to log or not
>  to log the particular log request.
>  @param event the event to log.  */
> public void callAppenders(LoggingEvent event) {
> int writes = 0;
> for(Category c = this; c != null; c=c.parent) {
>   // Protected against simultaneous call to addAppender, 
> removeAppender,...
>   synchronized(c) {
> if(c.aai != null) {
> writes += c.aai.appendLoopOnAppenders(event);
> }
> if(!c.additive) {
> break;
> }
>   }
> }
> if(writes == 0) {
>   repository.emitNoAppenderWarning(this);
> }
>   }{code}
> The log4j code, use the  global synchronized, it will cause this happened.
> cc [~weichiu] [~hexiaoqiao] [~ayushtkn]  [~shv] [~ferhui]



--
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-15924) Log4j will cause Server handler blocked when audit log boom.

2021-03-26 Thread Qi Zhu (Jira)
Qi Zhu created HDFS-15924:
-

 Summary: Log4j will cause Server handler blocked when audit log 
boom.
 Key: HDFS-15924
 URL: https://issues.apache.org/jira/browse/HDFS-15924
 Project: Hadoop HDFS
  Issue Type: Bug
Reporter: Qi Zhu
 Attachments: image-2021-03-26-16-18-03-341.png, 
image-2021-03-26-16-19-42-165.png

!image-2021-03-26-16-18-03-341.png|width=707,height=234!

!image-2021-03-26-16-19-42-165.png|width=824,height=198!

The thread blocked when audit log boom show in above.

Such as [https://dzone.com/articles/log4j-thread-deadlock-case] , it seems the 
same case when heavy load, should we update to Log4j2 or other things we can do 
to improve it in heavy audit log.

cc [~weichiu] [~hexiaoqiao] [~ayushtkn]  [~shv] [~ferhui]



--
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] [Work logged] (HDFS-15923) RBF: Authentication failed when rename accross sub clusters

2021-03-26 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/HDFS-15923?focusedWorklogId=572468=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-572468
 ]

ASF GitHub Bot logged work on HDFS-15923:
-

Author: ASF GitHub Bot
Created on: 26/Mar/21 06:28
Start Date: 26/Mar/21 06:28
Worklog Time Spent: 10m 
  Work Description: zhengzhuobinzzb commented on pull request #2819:
URL: https://github.com/apache/hadoop/pull/2819#issuecomment-807972675


   It is strange that i can pass test TestRouterFederationRename on my local 
computer


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
---

Worklog Id: (was: 572468)
Time Spent: 0.5h  (was: 20m)

> RBF:  Authentication failed when rename accross sub clusters
> 
>
> Key: HDFS-15923
> URL: https://issues.apache.org/jira/browse/HDFS-15923
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: rbf
>Reporter: zhuobin zheng
>Priority: Major
>  Labels: RBF, pull-request-available, rename
>  Time Spent: 0.5h
>  Remaining Estimate: 0h
>
> Rename accross subcluster with RBF and Kerberos environment. Will encounter 
> the following two errors:
>  # Save Object to journal.
>  # Precheck try to get src file status
> So, we need use Proxy UGI doAs create DistcpProcedure and TrashProcedure and 
> submit Job.
> In patch i use proxy ugi doAs above method. It worked.
> But there are another strange thing and this patch not solve:
> Router use ugi itself to submit the Distcp job. But not user ugi or proxy 
> ugi. This may cause excessive distcp permissions.
> First: Save Object to journal.
> {code:java}
> // code placeholder
> 2021-03-23 14:01:16,233 WARN org.apache.hadoop.ipc.Client: Exception 
> encountered while connecting to the server 
> javax.security.sasl.SaslException: GSS initiate failed [Caused by 
> GSSException: No valid credentials provided (Mechanism level: Failed to find 
> any Kerberos tgt)]
> at 
> com.sun.security.sasl.gsskerb.GssKrb5Client.evaluateChallenge(GssKrb5Client.java:211)
> at 
> org.apache.hadoop.security.SaslRpcClient.saslConnect(SaslRpcClient.java:408)
> at 
> org.apache.hadoop.ipc.Client$Connection.setupSaslConnection(Client.java:622)
> at 
> org.apache.hadoop.ipc.Client$Connection.access$2300(Client.java:413)
> at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:822)
> at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:818)
> at java.security.AccessController.doPrivileged(Native Method)
> at javax.security.auth.Subject.doAs(Subject.java:422)
> at 
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1762)
> at 
> org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:818)
> at 
> org.apache.hadoop.ipc.Client$Connection.access$3800(Client.java:413)
> at org.apache.hadoop.ipc.Client.getConnection(Client.java:1636)
> at org.apache.hadoop.ipc.Client.call(Client.java:1452)
> at org.apache.hadoop.ipc.Client.call(Client.java:1405)
> at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:233)
> at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:118)
> at com.sun.proxy.$Proxy11.create(Unknown Source)
> at 
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.create(ClientNamenodeProtocolTranslatorPB.java:376)
> at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
> 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:422)
> at 
> org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invokeMethod(RetryInvocationHandler.java:165)
> at 
> org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invoke(RetryInvocationHandler.java:157)
> at 
> org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invokeOnce(RetryInvocationHandler.java:95)
> at 
> org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:359)
> at com.sun.proxy.$Proxy12.create(Unknown Source)
> at 
> org.apache.hadoop.hdfs.DFSOutputStream.newStreamForCreate(DFSOutputStream.java:277)
> at