[jira] [Work logged] (HDFS-16601) Failed to replace a bad datanode on the existing pipeline due to no more good datanodes being available to try

2022-07-26 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HDFS-16601:
-

Author: ASF GitHub Bot
Created on: 27/Jul/22 04:53
Start Date: 27/Jul/22 04:53
Worklog Time Spent: 10m 
  Work Description: ZanderXu commented on PR #4369:
URL: https://github.com/apache/hadoop/pull/4369#issuecomment-1196264884

   @jojochuang Thanks for you review. We encounter this bug in our prod, 
because the blockā€˜s checksum file of the source DN is corrupted. It caused 
transfer failed. And client tried all DNs and failed.
   
   So Client should sense the status of transfer. But it's difficult to differ 
the exception caused by source Node or target Node. Maybe we can first throw 
the failed exception to Client and let Client try to use the next DN as the 
source to transfer block.
   
   cc @Hexiaoqiao 
   
   




Issue Time Tracking
---

Worklog Id: (was: 795517)
Time Spent: 2h 10m  (was: 2h)

> Failed to replace a bad datanode on the existing pipeline due to no more good 
> datanodes being available to try
> --
>
> Key: HDFS-16601
> URL: https://issues.apache.org/jira/browse/HDFS-16601
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: ZanderXu
>Assignee: ZanderXu
>Priority: Major
>  Labels: pull-request-available
>  Time Spent: 2h 10m
>  Remaining Estimate: 0h
>
> In our production environment, we found a bug and stack like:
> {code:java}
> java.io.IOException: Failed to replace a bad datanode on the existing 
> pipeline due to no more good datanodes being available to try. (Nodes: 
> current=[DatanodeInfoWithStorage[127.0.0.1:59687,DS-b803febc-7b22-4144-9b39-7bf521cdaa8d,DISK],
>  
> DatanodeInfoWithStorage[127.0.0.1:59670,DS-0d652bc2-1784-430d-961f-750f80a290f1,DISK]],
>  
> original=[DatanodeInfoWithStorage[127.0.0.1:59670,DS-0d652bc2-1784-430d-961f-750f80a290f1,DISK],
>  
> DatanodeInfoWithStorage[127.0.0.1:59687,DS-b803febc-7b22-4144-9b39-7bf521cdaa8d,DISK]]).
>  The current failed datanode replacement policy is DEFAULT, and a client may 
> configure this via 
> 'dfs.client.block.write.replace-datanode-on-failure.policy' in its 
> configuration.
>   at 
> org.apache.hadoop.hdfs.DataStreamer.findNewDatanode(DataStreamer.java:1418)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.addDatanode2ExistingPipeline(DataStreamer.java:1478)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.handleDatanodeReplacement(DataStreamer.java:1704)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.setupPipelineInternal(DataStreamer.java:1605)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.setupPipelineForAppendOrRecovery(DataStreamer.java:1587)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.processDatanodeOrExternalError(DataStreamer.java:1371)
>   at org.apache.hadoop.hdfs.DataStreamer.run(DataStreamer.java:674)
> {code}
> And the root cause is that DFSClient cannot  perceive the exception of 
> TransferBlock during PipelineRecovery. If failed during TransferBlock, the 
> DFSClient will retry all datanodes in the cluster and then failed.
> When client is recovering pipeline, the source dn selected to transfer block 
> to new DN may be abnormal,  it cannot successfully transfer the block to the 
> new node. But the failed exception not returned to the client, Client also 
> thought transfer successfully. But there is not block in the new DN, so 
> Client failed to build the pipeline, and marked the new DN is bad. And then 
> Client will add the new DN into exclude list to get a new DN for the new loop 
> pipeline recovery. The new pipeline recovery will still choose the abnormal 
> dn as the source dn to transfer block, and it will fail again..
> So I think that DN should return the failed exception of transfer to Client, 
> so that Client can choose anther existed dn as the source dn to transfer the 
> block to a new DN.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

-
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-16601) Failed to replace a bad datanode on the existing pipeline due to no more good datanodes being available to try

2022-07-26 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HDFS-16601:
-

Author: ASF GitHub Bot
Created on: 26/Jul/22 21:15
Start Date: 26/Jul/22 21:15
Worklog Time Spent: 10m 
  Work Description: jojochuang commented on code in PR #4369:
URL: https://github.com/apache/hadoop/pull/4369#discussion_r930415320


##
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java:
##
@@ -761,6 +762,131 @@ public void failPipeline(ReplicaInPipeline replicaInfo,
 }
   }
 
+
+  @Test
+  public void testPipelineRecoveryWithFailedTransferBlock() throws Exception {
+final int chunkSize = 512;
+final int oneWriteSize = 5000;
+final int totalSize = 1024 * 1024;
+final int errorInjectionPos = 512;
+Configuration conf = new HdfsConfiguration();
+// Need 5 datanodes to verify the replaceDatanode during pipeline recovery
+final MiniDFSCluster cluster =
+new MiniDFSCluster.Builder(conf).numDataNodes(5).build();
+DataNodeFaultInjector old = DataNodeFaultInjector.get();
+
+try {
+  DistributedFileSystem fs = cluster.getFileSystem();
+  Path fileName = new Path("/f");
+  FSDataOutputStream o = fs.create(fileName);
+  int count = 0;
+  // Flush to get the pipeline created.
+  o.writeBytes("hello");
+  o.hflush();
+  DFSOutputStream dfsO = (DFSOutputStream) o.getWrappedStream();
+  final DatanodeInfo[] pipeline = dfsO.getStreamer().getNodes();
+  final String firstDn = pipeline[0].getXferAddr(false);
+  final String secondDn = pipeline[1].getXferAddr(false);
+  final AtomicBoolean pipelineFailed = new AtomicBoolean(false);
+  final AtomicBoolean transferFailed = new AtomicBoolean(false);
+
+  DataNodeFaultInjector.set(new DataNodeFaultInjector() {
+@Override
+public void failPipeline(ReplicaInPipeline replicaInfo,
+String mirror) throws IOException {
+  if (!secondDn.equals(mirror)) {
+// Only fail for first DN
+return;
+  }
+  if (!pipelineFailed.get() &&
+  (replicaInfo.getBytesAcked() > errorInjectionPos) &&
+  (replicaInfo.getBytesAcked() % chunkSize != 0)) {
+int count = 0;
+while (count < 10) {
+  // Fail the pipeline (Throw exception) when:
+  //   1. bytsAcked is not at chunk boundary (checked in the if
+  //  statement above)
+  //   2. bytesOnDisk is bigger than bytesAcked and at least
+  //  reaches (or go beyond) the end of the chunk that
+  //  bytesAcked is in (checked in the if statement below).
+  // At this condition, transferBlock that happens during
+  // pipeline recovery would transfer extra bytes to make up to the
+  // end of the chunk. And this is when the block corruption
+  // described in HDFS-4660 would occur.

Review Comment:
   Oh HDFS-4660 brought back my worst nightmare when I spent a month chasing 
this bug.





Issue Time Tracking
---

Worklog Id: (was: 795437)
Time Spent: 2h  (was: 1h 50m)

> Failed to replace a bad datanode on the existing pipeline due to no more good 
> datanodes being available to try
> --
>
> Key: HDFS-16601
> URL: https://issues.apache.org/jira/browse/HDFS-16601
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: ZanderXu
>Assignee: ZanderXu
>Priority: Major
>  Labels: pull-request-available
>  Time Spent: 2h
>  Remaining Estimate: 0h
>
> In our production environment, we found a bug and stack like:
> {code:java}
> java.io.IOException: Failed to replace a bad datanode on the existing 
> pipeline due to no more good datanodes being available to try. (Nodes: 
> current=[DatanodeInfoWithStorage[127.0.0.1:59687,DS-b803febc-7b22-4144-9b39-7bf521cdaa8d,DISK],
>  
> DatanodeInfoWithStorage[127.0.0.1:59670,DS-0d652bc2-1784-430d-961f-750f80a290f1,DISK]],
>  
> original=[DatanodeInfoWithStorage[127.0.0.1:59670,DS-0d652bc2-1784-430d-961f-750f80a290f1,DISK],
>  
> DatanodeInfoWithStorage[127.0.0.1:59687,DS-b803febc-7b22-4144-9b39-7bf521cdaa8d,DISK]]).
>  The current failed datanode replacement policy is DEFAULT, and a client may 
> configure this via 
> 'dfs.client.block.write.replace-datanode-on-failure.policy' in its 
> configuration.
>   at 
> org.apache.hadoop.hdfs.DataStreamer.findNewDatanode(DataStreamer.java:1418)
>   at 
> 

[jira] [Work logged] (HDFS-16601) Failed to replace a bad datanode on the existing pipeline due to no more good datanodes being available to try

2022-06-10 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HDFS-16601:
-

Author: ASF GitHub Bot
Created on: 10/Jun/22 10:32
Start Date: 10/Jun/22 10:32
Worklog Time Spent: 10m 
  Work Description: ZanderXu commented on PR #4369:
URL: https://github.com/apache/hadoop/pull/4369#issuecomment-1152220339

   Got, thanks @Hexiaoqiao .
   
   > Actually, I mean, it will be helpful for client to do many fault-tolerant 
improvement later if we could differ the exception about transfer
   
   I will try to work for it.




Issue Time Tracking
---

Worklog Id: (was: 780278)
Time Spent: 1h 50m  (was: 1h 40m)

> Failed to replace a bad datanode on the existing pipeline due to no more good 
> datanodes being available to try
> --
>
> Key: HDFS-16601
> URL: https://issues.apache.org/jira/browse/HDFS-16601
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: ZanderXu
>Assignee: ZanderXu
>Priority: Major
>  Labels: pull-request-available
>  Time Spent: 1h 50m
>  Remaining Estimate: 0h
>
> In our production environment, we found a bug and stack like:
> {code:java}
> java.io.IOException: Failed to replace a bad datanode on the existing 
> pipeline due to no more good datanodes being available to try. (Nodes: 
> current=[DatanodeInfoWithStorage[127.0.0.1:59687,DS-b803febc-7b22-4144-9b39-7bf521cdaa8d,DISK],
>  
> DatanodeInfoWithStorage[127.0.0.1:59670,DS-0d652bc2-1784-430d-961f-750f80a290f1,DISK]],
>  
> original=[DatanodeInfoWithStorage[127.0.0.1:59670,DS-0d652bc2-1784-430d-961f-750f80a290f1,DISK],
>  
> DatanodeInfoWithStorage[127.0.0.1:59687,DS-b803febc-7b22-4144-9b39-7bf521cdaa8d,DISK]]).
>  The current failed datanode replacement policy is DEFAULT, and a client may 
> configure this via 
> 'dfs.client.block.write.replace-datanode-on-failure.policy' in its 
> configuration.
>   at 
> org.apache.hadoop.hdfs.DataStreamer.findNewDatanode(DataStreamer.java:1418)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.addDatanode2ExistingPipeline(DataStreamer.java:1478)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.handleDatanodeReplacement(DataStreamer.java:1704)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.setupPipelineInternal(DataStreamer.java:1605)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.setupPipelineForAppendOrRecovery(DataStreamer.java:1587)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.processDatanodeOrExternalError(DataStreamer.java:1371)
>   at org.apache.hadoop.hdfs.DataStreamer.run(DataStreamer.java:674)
> {code}
> And the root cause is that DFSClient cannot  perceive the exception of 
> TransferBlock during PipelineRecovery. If failed during TransferBlock, the 
> DFSClient will retry all datanodes in the cluster and then failed.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)

-
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-16601) Failed to replace a bad datanode on the existing pipeline due to no more good datanodes being available to try

2022-06-10 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HDFS-16601:
-

Author: ASF GitHub Bot
Created on: 10/Jun/22 10:29
Start Date: 10/Jun/22 10:29
Worklog Time Spent: 10m 
  Work Description: Hexiaoqiao commented on PR #4369:
URL: https://github.com/apache/hadoop/pull/4369#issuecomment-1152217871

   Sorry for not very clear comment. I know not it's round-robin way to pick 
the source node, and at third round it will pick the original node again (no 
matter if it is bad/slow node.), of course it will be a tiny probability. 
Actually, I mean, it will be helpful for client to do many fault-tolerant 
improvement later if we could differ the exception about transfer. Once more, 
this is not blocker comment. Thanks again.




Issue Time Tracking
---

Worklog Id: (was: 780277)
Time Spent: 1h 40m  (was: 1.5h)

> Failed to replace a bad datanode on the existing pipeline due to no more good 
> datanodes being available to try
> --
>
> Key: HDFS-16601
> URL: https://issues.apache.org/jira/browse/HDFS-16601
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: ZanderXu
>Assignee: ZanderXu
>Priority: Major
>  Labels: pull-request-available
>  Time Spent: 1h 40m
>  Remaining Estimate: 0h
>
> In our production environment, we found a bug and stack like:
> {code:java}
> java.io.IOException: Failed to replace a bad datanode on the existing 
> pipeline due to no more good datanodes being available to try. (Nodes: 
> current=[DatanodeInfoWithStorage[127.0.0.1:59687,DS-b803febc-7b22-4144-9b39-7bf521cdaa8d,DISK],
>  
> DatanodeInfoWithStorage[127.0.0.1:59670,DS-0d652bc2-1784-430d-961f-750f80a290f1,DISK]],
>  
> original=[DatanodeInfoWithStorage[127.0.0.1:59670,DS-0d652bc2-1784-430d-961f-750f80a290f1,DISK],
>  
> DatanodeInfoWithStorage[127.0.0.1:59687,DS-b803febc-7b22-4144-9b39-7bf521cdaa8d,DISK]]).
>  The current failed datanode replacement policy is DEFAULT, and a client may 
> configure this via 
> 'dfs.client.block.write.replace-datanode-on-failure.policy' in its 
> configuration.
>   at 
> org.apache.hadoop.hdfs.DataStreamer.findNewDatanode(DataStreamer.java:1418)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.addDatanode2ExistingPipeline(DataStreamer.java:1478)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.handleDatanodeReplacement(DataStreamer.java:1704)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.setupPipelineInternal(DataStreamer.java:1605)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.setupPipelineForAppendOrRecovery(DataStreamer.java:1587)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.processDatanodeOrExternalError(DataStreamer.java:1371)
>   at org.apache.hadoop.hdfs.DataStreamer.run(DataStreamer.java:674)
> {code}
> And the root cause is that DFSClient cannot  perceive the exception of 
> TransferBlock during PipelineRecovery. If failed during TransferBlock, the 
> DFSClient will retry all datanodes in the cluster and then failed.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)

-
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-16601) Failed to replace a bad datanode on the existing pipeline due to no more good datanodes being available to try

2022-06-10 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HDFS-16601:
-

Author: ASF GitHub Bot
Created on: 10/Jun/22 10:09
Start Date: 10/Jun/22 10:09
Worklog Time Spent: 10m 
  Work Description: ZanderXu commented on PR #4369:
URL: https://github.com/apache/hadoop/pull/4369#issuecomment-1152200912

   > the source datanode meets issue and choose the same one when retry
   
   It will chose the next datanode as source datanode when retry.
   
   Code like blew, and tried will +1 when retry.
   ```
 final DatanodeInfo src = original[tried % original.length];
 final DatanodeInfo[] targets = {nodes[d]};
 final StorageType[] targetStorageTypes = {storageTypes[d]};
   
 try {
   transfer(src, targets, targetStorageTypes, lb.getBlockToken());
 } catch (IOException ioe) {
   DFSClient.LOG.warn("Error transferring data from " + src + " to " +
   nodes[d] + ": " + ioe.getMessage());
   caughtException = ioe;
   // add the allocated node to the exclude list.
   exclude.add(nodes[d]);
   setPipeline(original, originalTypes, originalIDs);
   tried++;
   continue;
 }
   ```




Issue Time Tracking
---

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

> Failed to replace a bad datanode on the existing pipeline due to no more good 
> datanodes being available to try
> --
>
> Key: HDFS-16601
> URL: https://issues.apache.org/jira/browse/HDFS-16601
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: ZanderXu
>Assignee: ZanderXu
>Priority: Major
>  Labels: pull-request-available
>  Time Spent: 1.5h
>  Remaining Estimate: 0h
>
> In our production environment, we found a bug and stack like:
> {code:java}
> java.io.IOException: Failed to replace a bad datanode on the existing 
> pipeline due to no more good datanodes being available to try. (Nodes: 
> current=[DatanodeInfoWithStorage[127.0.0.1:59687,DS-b803febc-7b22-4144-9b39-7bf521cdaa8d,DISK],
>  
> DatanodeInfoWithStorage[127.0.0.1:59670,DS-0d652bc2-1784-430d-961f-750f80a290f1,DISK]],
>  
> original=[DatanodeInfoWithStorage[127.0.0.1:59670,DS-0d652bc2-1784-430d-961f-750f80a290f1,DISK],
>  
> DatanodeInfoWithStorage[127.0.0.1:59687,DS-b803febc-7b22-4144-9b39-7bf521cdaa8d,DISK]]).
>  The current failed datanode replacement policy is DEFAULT, and a client may 
> configure this via 
> 'dfs.client.block.write.replace-datanode-on-failure.policy' in its 
> configuration.
>   at 
> org.apache.hadoop.hdfs.DataStreamer.findNewDatanode(DataStreamer.java:1418)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.addDatanode2ExistingPipeline(DataStreamer.java:1478)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.handleDatanodeReplacement(DataStreamer.java:1704)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.setupPipelineInternal(DataStreamer.java:1605)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.setupPipelineForAppendOrRecovery(DataStreamer.java:1587)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.processDatanodeOrExternalError(DataStreamer.java:1371)
>   at org.apache.hadoop.hdfs.DataStreamer.run(DataStreamer.java:674)
> {code}
> And the root cause is that DFSClient cannot  perceive the exception of 
> TransferBlock during PipelineRecovery. If failed during TransferBlock, the 
> DFSClient will retry all datanodes in the cluster and then failed.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)

-
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-16601) Failed to replace a bad datanode on the existing pipeline due to no more good datanodes being available to try

2022-06-10 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HDFS-16601:
-

Author: ASF GitHub Bot
Created on: 10/Jun/22 10:01
Start Date: 10/Jun/22 10:01
Worklog Time Spent: 10m 
  Work Description: Hexiaoqiao commented on PR #4369:
URL: https://github.com/apache/hadoop/pull/4369#issuecomment-1152193661

   > Fortunately, at present, as long as failed exception throw to client, the 
client defaults to thinking that the new dn is abnormal, and will exclude it 
and retry transfer. During retrying transfer, Client will chose new source dn 
and new target dn. 
   
   Thanks for furthermore comment here. Agree that it will improve 
fault-tolerant for transfer, however, we have to accept the truth that the 
source datanode meets issue and choose the same one when retry, thus we could 
not avoid to fail.  I am not sure if any way to expose exceptions to differ 
source Node or target Node exception?  If it is true, it will be helpful for 
the following fault-tolerant improvement at client side. 




Issue Time Tracking
---

Worklog Id: (was: 780266)
Time Spent: 1h 20m  (was: 1h 10m)

> Failed to replace a bad datanode on the existing pipeline due to no more good 
> datanodes being available to try
> --
>
> Key: HDFS-16601
> URL: https://issues.apache.org/jira/browse/HDFS-16601
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: ZanderXu
>Assignee: ZanderXu
>Priority: Major
>  Labels: pull-request-available
>  Time Spent: 1h 20m
>  Remaining Estimate: 0h
>
> In our production environment, we found a bug and stack like:
> {code:java}
> java.io.IOException: Failed to replace a bad datanode on the existing 
> pipeline due to no more good datanodes being available to try. (Nodes: 
> current=[DatanodeInfoWithStorage[127.0.0.1:59687,DS-b803febc-7b22-4144-9b39-7bf521cdaa8d,DISK],
>  
> DatanodeInfoWithStorage[127.0.0.1:59670,DS-0d652bc2-1784-430d-961f-750f80a290f1,DISK]],
>  
> original=[DatanodeInfoWithStorage[127.0.0.1:59670,DS-0d652bc2-1784-430d-961f-750f80a290f1,DISK],
>  
> DatanodeInfoWithStorage[127.0.0.1:59687,DS-b803febc-7b22-4144-9b39-7bf521cdaa8d,DISK]]).
>  The current failed datanode replacement policy is DEFAULT, and a client may 
> configure this via 
> 'dfs.client.block.write.replace-datanode-on-failure.policy' in its 
> configuration.
>   at 
> org.apache.hadoop.hdfs.DataStreamer.findNewDatanode(DataStreamer.java:1418)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.addDatanode2ExistingPipeline(DataStreamer.java:1478)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.handleDatanodeReplacement(DataStreamer.java:1704)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.setupPipelineInternal(DataStreamer.java:1605)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.setupPipelineForAppendOrRecovery(DataStreamer.java:1587)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.processDatanodeOrExternalError(DataStreamer.java:1371)
>   at org.apache.hadoop.hdfs.DataStreamer.run(DataStreamer.java:674)
> {code}
> And the root cause is that DFSClient cannot  perceive the exception of 
> TransferBlock during PipelineRecovery. If failed during TransferBlock, the 
> DFSClient will retry all datanodes in the cluster and then failed.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)

-
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-16601) Failed to replace a bad datanode on the existing pipeline due to no more good datanodes being available to try

2022-06-09 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HDFS-16601:
-

Author: ASF GitHub Bot
Created on: 09/Jun/22 14:56
Start Date: 09/Jun/22 14:56
Worklog Time Spent: 10m 
  Work Description: ZanderXu commented on PR #4369:
URL: https://github.com/apache/hadoop/pull/4369#issuecomment-1151228050

   Thanks @Hexiaoqiao for your suggestion. Yeah, your are right, we need more 
failed information for client, like transfer source failed or transfer target 
failed.  If client have more information about failed transfer, It can 
accurately and efficiently remove abnormal nodes. But this would be a big 
feature.
   
   Fortunately, at present, as long as failed exception throw to client, the 
client defaults to thinking that the new dn is abnormal, and will exclude it 
and retry transfer. During retrying transfer, Client will chose new source dn 
and new target dn. Therefor, the source and target dn in the previous failed 
transfer round will be replaced. 
   If it is target dn caused failed, excluded the target dn will be ok.
   If it is source dn caused failed,  it will be removed when building the new 
pipeline.
   
   So I think simple process is just throw failed exception to client, and 
client can find and remove the real abnormal datanode. 




Issue Time Tracking
---

Worklog Id: (was: 779997)
Time Spent: 1h 10m  (was: 1h)

> Failed to replace a bad datanode on the existing pipeline due to no more good 
> datanodes being available to try
> --
>
> Key: HDFS-16601
> URL: https://issues.apache.org/jira/browse/HDFS-16601
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: ZanderXu
>Assignee: ZanderXu
>Priority: Major
>  Labels: pull-request-available
>  Time Spent: 1h 10m
>  Remaining Estimate: 0h
>
> In our production environment, we found a bug and stack like:
> {code:java}
> java.io.IOException: Failed to replace a bad datanode on the existing 
> pipeline due to no more good datanodes being available to try. (Nodes: 
> current=[DatanodeInfoWithStorage[127.0.0.1:59687,DS-b803febc-7b22-4144-9b39-7bf521cdaa8d,DISK],
>  
> DatanodeInfoWithStorage[127.0.0.1:59670,DS-0d652bc2-1784-430d-961f-750f80a290f1,DISK]],
>  
> original=[DatanodeInfoWithStorage[127.0.0.1:59670,DS-0d652bc2-1784-430d-961f-750f80a290f1,DISK],
>  
> DatanodeInfoWithStorage[127.0.0.1:59687,DS-b803febc-7b22-4144-9b39-7bf521cdaa8d,DISK]]).
>  The current failed datanode replacement policy is DEFAULT, and a client may 
> configure this via 
> 'dfs.client.block.write.replace-datanode-on-failure.policy' in its 
> configuration.
>   at 
> org.apache.hadoop.hdfs.DataStreamer.findNewDatanode(DataStreamer.java:1418)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.addDatanode2ExistingPipeline(DataStreamer.java:1478)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.handleDatanodeReplacement(DataStreamer.java:1704)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.setupPipelineInternal(DataStreamer.java:1605)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.setupPipelineForAppendOrRecovery(DataStreamer.java:1587)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.processDatanodeOrExternalError(DataStreamer.java:1371)
>   at org.apache.hadoop.hdfs.DataStreamer.run(DataStreamer.java:674)
> {code}
> And the root cause is that DFSClient cannot  perceive the exception of 
> TransferBlock during PipelineRecovery. If failed during TransferBlock, the 
> DFSClient will retry all datanodes in the cluster and then failed.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)

-
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-16601) Failed to replace a bad datanode on the existing pipeline due to no more good datanodes being available to try

2022-06-09 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HDFS-16601:
-

Author: ASF GitHub Bot
Created on: 09/Jun/22 14:01
Start Date: 09/Jun/22 14:01
Worklog Time Spent: 10m 
  Work Description: Hexiaoqiao commented on PR #4369:
URL: https://github.com/apache/hadoop/pull/4369#issuecomment-1151160510

   Thanks for starting this proposal. I think there are still many issues for 
data transfer for pipeline recovery from my practice, which includes both basic 
function and performance. IIRC, there are only timeout exception and one no 
explicit meaning exception, thus client has no helpful information (such src 
node or target node meet issue, or other exceptions) to make decision. 
   Back to this PR, I totally agree to throw exception from datanode to client 
first(but I am not sure if it is enough at this PR, maybe we need more 
information) then add more fault-tolerant logic at client side.
   IMO, we should file one new JIRA to design/refactor fault-tolerant for data 
transfer of pipeline recovery. Just my own suggestion, not blocker.




Issue Time Tracking
---

Worklog Id: (was: 779967)
Time Spent: 1h  (was: 50m)

> Failed to replace a bad datanode on the existing pipeline due to no more good 
> datanodes being available to try
> --
>
> Key: HDFS-16601
> URL: https://issues.apache.org/jira/browse/HDFS-16601
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: ZanderXu
>Assignee: ZanderXu
>Priority: Major
>  Labels: pull-request-available
>  Time Spent: 1h
>  Remaining Estimate: 0h
>
> In our production environment, we found a bug and stack like:
> {code:java}
> java.io.IOException: Failed to replace a bad datanode on the existing 
> pipeline due to no more good datanodes being available to try. (Nodes: 
> current=[DatanodeInfoWithStorage[127.0.0.1:59687,DS-b803febc-7b22-4144-9b39-7bf521cdaa8d,DISK],
>  
> DatanodeInfoWithStorage[127.0.0.1:59670,DS-0d652bc2-1784-430d-961f-750f80a290f1,DISK]],
>  
> original=[DatanodeInfoWithStorage[127.0.0.1:59670,DS-0d652bc2-1784-430d-961f-750f80a290f1,DISK],
>  
> DatanodeInfoWithStorage[127.0.0.1:59687,DS-b803febc-7b22-4144-9b39-7bf521cdaa8d,DISK]]).
>  The current failed datanode replacement policy is DEFAULT, and a client may 
> configure this via 
> 'dfs.client.block.write.replace-datanode-on-failure.policy' in its 
> configuration.
>   at 
> org.apache.hadoop.hdfs.DataStreamer.findNewDatanode(DataStreamer.java:1418)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.addDatanode2ExistingPipeline(DataStreamer.java:1478)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.handleDatanodeReplacement(DataStreamer.java:1704)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.setupPipelineInternal(DataStreamer.java:1605)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.setupPipelineForAppendOrRecovery(DataStreamer.java:1587)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.processDatanodeOrExternalError(DataStreamer.java:1371)
>   at org.apache.hadoop.hdfs.DataStreamer.run(DataStreamer.java:674)
> {code}
> And the root cause is that DFSClient cannot  perceive the exception of 
> TransferBlock during PipelineRecovery. If failed during TransferBlock, the 
> DFSClient will retry all datanodes in the cluster and then failed.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)

-
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-16601) Failed to replace a bad datanode on the existing pipeline due to no more good datanodes being available to try

2022-06-07 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HDFS-16601:
-

Author: ASF GitHub Bot
Created on: 07/Jun/22 14:09
Start Date: 07/Jun/22 14:09
Worklog Time Spent: 10m 
  Work Description: ZanderXu commented on PR #4369:
URL: https://github.com/apache/hadoop/pull/4369#issuecomment-1148728940

   Thanks @Hexiaoqiao .
   When client is recovering pipeline, the source dn of selected to transfer 
block to new DN may be abnormal, so that the source dn cannot transfer the 
block to the new node normally, but the failed exception not returned to the 
client,  caused the client to think that the transfer is completed 
successfully. Because new DN not contains the block, so client will fail to 
build the pipeline and mark the new DN is bad. And then Client will add the new 
DN into exclude list to get a new DN for the new loop pipeline recovery. 
   The new pipeline recovery will still choose the abnormal dn as source dn to 
transfer block, and will failed again..
   
   So Dn should return the failed transfer exception to client, so that client 
can choose anther existed dn as source dn to transfer the block to new DN.
   
   




Issue Time Tracking
---

Worklog Id: (was: 779109)
Time Spent: 50m  (was: 40m)

> Failed to replace a bad datanode on the existing pipeline due to no more good 
> datanodes being available to try
> --
>
> Key: HDFS-16601
> URL: https://issues.apache.org/jira/browse/HDFS-16601
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: ZanderXu
>Assignee: ZanderXu
>Priority: Major
>  Labels: pull-request-available
>  Time Spent: 50m
>  Remaining Estimate: 0h
>
> In our production environment, we found a bug and stack like:
> {code:java}
> java.io.IOException: Failed to replace a bad datanode on the existing 
> pipeline due to no more good datanodes being available to try. (Nodes: 
> current=[DatanodeInfoWithStorage[127.0.0.1:59687,DS-b803febc-7b22-4144-9b39-7bf521cdaa8d,DISK],
>  
> DatanodeInfoWithStorage[127.0.0.1:59670,DS-0d652bc2-1784-430d-961f-750f80a290f1,DISK]],
>  
> original=[DatanodeInfoWithStorage[127.0.0.1:59670,DS-0d652bc2-1784-430d-961f-750f80a290f1,DISK],
>  
> DatanodeInfoWithStorage[127.0.0.1:59687,DS-b803febc-7b22-4144-9b39-7bf521cdaa8d,DISK]]).
>  The current failed datanode replacement policy is DEFAULT, and a client may 
> configure this via 
> 'dfs.client.block.write.replace-datanode-on-failure.policy' in its 
> configuration.
>   at 
> org.apache.hadoop.hdfs.DataStreamer.findNewDatanode(DataStreamer.java:1418)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.addDatanode2ExistingPipeline(DataStreamer.java:1478)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.handleDatanodeReplacement(DataStreamer.java:1704)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.setupPipelineInternal(DataStreamer.java:1605)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.setupPipelineForAppendOrRecovery(DataStreamer.java:1587)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.processDatanodeOrExternalError(DataStreamer.java:1371)
>   at org.apache.hadoop.hdfs.DataStreamer.run(DataStreamer.java:674)
> {code}
> And the root cause is that DFSClient cannot  perceive the exception of 
> TransferBlock during PipelineRecovery. If failed during TransferBlock, the 
> DFSClient will retry all datanodes in the cluster and then failed.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)

-
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-16601) Failed to replace a bad datanode on the existing pipeline due to no more good datanodes being available to try

2022-06-07 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HDFS-16601:
-

Author: ASF GitHub Bot
Created on: 07/Jun/22 13:27
Start Date: 07/Jun/22 13:27
Worklog Time Spent: 10m 
  Work Description: Hexiaoqiao commented on PR #4369:
URL: https://github.com/apache/hadoop/pull/4369#issuecomment-1148675874

   @ZanderXu Thanks for report and contribution. Sorry I don't get what 
scenario lead this issue. Would you like to offer more information. Thanks.




Issue Time Tracking
---

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

> Failed to replace a bad datanode on the existing pipeline due to no more good 
> datanodes being available to try
> --
>
> Key: HDFS-16601
> URL: https://issues.apache.org/jira/browse/HDFS-16601
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: ZanderXu
>Assignee: ZanderXu
>Priority: Major
>  Labels: pull-request-available
>  Time Spent: 40m
>  Remaining Estimate: 0h
>
> In our production environment, we found a bug and stack like:
> {code:java}
> java.io.IOException: Failed to replace a bad datanode on the existing 
> pipeline due to no more good datanodes being available to try. (Nodes: 
> current=[DatanodeInfoWithStorage[127.0.0.1:59687,DS-b803febc-7b22-4144-9b39-7bf521cdaa8d,DISK],
>  
> DatanodeInfoWithStorage[127.0.0.1:59670,DS-0d652bc2-1784-430d-961f-750f80a290f1,DISK]],
>  
> original=[DatanodeInfoWithStorage[127.0.0.1:59670,DS-0d652bc2-1784-430d-961f-750f80a290f1,DISK],
>  
> DatanodeInfoWithStorage[127.0.0.1:59687,DS-b803febc-7b22-4144-9b39-7bf521cdaa8d,DISK]]).
>  The current failed datanode replacement policy is DEFAULT, and a client may 
> configure this via 
> 'dfs.client.block.write.replace-datanode-on-failure.policy' in its 
> configuration.
>   at 
> org.apache.hadoop.hdfs.DataStreamer.findNewDatanode(DataStreamer.java:1418)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.addDatanode2ExistingPipeline(DataStreamer.java:1478)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.handleDatanodeReplacement(DataStreamer.java:1704)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.setupPipelineInternal(DataStreamer.java:1605)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.setupPipelineForAppendOrRecovery(DataStreamer.java:1587)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.processDatanodeOrExternalError(DataStreamer.java:1371)
>   at org.apache.hadoop.hdfs.DataStreamer.run(DataStreamer.java:674)
> {code}
> And the root cause is that DFSClient cannot  perceive the exception of 
> TransferBlock during PipelineRecovery. If failed during TransferBlock, the 
> DFSClient will retry all datanodes in the cluster and then failed.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)

-
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-16601) Failed to replace a bad datanode on the existing pipeline due to no more good datanodes being available to try

2022-06-05 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HDFS-16601:
-

Author: ASF GitHub Bot
Created on: 05/Jun/22 09:58
Start Date: 05/Jun/22 09:58
Worklog Time Spent: 10m 
  Work Description: ZanderXu commented on PR #4369:
URL: https://github.com/apache/hadoop/pull/4369#issuecomment-1146776005

   @Hexiaoqiao @MingXiangLi can you help me review this patch? thanks~




Issue Time Tracking
---

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

> Failed to replace a bad datanode on the existing pipeline due to no more good 
> datanodes being available to try
> --
>
> Key: HDFS-16601
> URL: https://issues.apache.org/jira/browse/HDFS-16601
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: ZanderXu
>Assignee: ZanderXu
>Priority: Major
>  Labels: pull-request-available
>  Time Spent: 0.5h
>  Remaining Estimate: 0h
>
> In our production environment, we found a bug and stack like:
> {code:java}
> java.io.IOException: Failed to replace a bad datanode on the existing 
> pipeline due to no more good datanodes being available to try. (Nodes: 
> current=[DatanodeInfoWithStorage[127.0.0.1:59687,DS-b803febc-7b22-4144-9b39-7bf521cdaa8d,DISK],
>  
> DatanodeInfoWithStorage[127.0.0.1:59670,DS-0d652bc2-1784-430d-961f-750f80a290f1,DISK]],
>  
> original=[DatanodeInfoWithStorage[127.0.0.1:59670,DS-0d652bc2-1784-430d-961f-750f80a290f1,DISK],
>  
> DatanodeInfoWithStorage[127.0.0.1:59687,DS-b803febc-7b22-4144-9b39-7bf521cdaa8d,DISK]]).
>  The current failed datanode replacement policy is DEFAULT, and a client may 
> configure this via 
> 'dfs.client.block.write.replace-datanode-on-failure.policy' in its 
> configuration.
>   at 
> org.apache.hadoop.hdfs.DataStreamer.findNewDatanode(DataStreamer.java:1418)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.addDatanode2ExistingPipeline(DataStreamer.java:1478)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.handleDatanodeReplacement(DataStreamer.java:1704)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.setupPipelineInternal(DataStreamer.java:1605)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.setupPipelineForAppendOrRecovery(DataStreamer.java:1587)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.processDatanodeOrExternalError(DataStreamer.java:1371)
>   at org.apache.hadoop.hdfs.DataStreamer.run(DataStreamer.java:674)
> {code}
> And the root cause is that DFSClient cannot  perceive the exception of 
> TransferBlock during PipelineRecovery. If failed during TransferBlock, the 
> DFSClient will retry all datanodes in the cluster and then failed.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)

-
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-16601) Failed to replace a bad datanode on the existing pipeline due to no more good datanodes being available to try

2022-05-28 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HDFS-16601:
-

Author: ASF GitHub Bot
Created on: 28/May/22 10:43
Start Date: 28/May/22 10:43
Worklog Time Spent: 10m 
  Work Description: hadoop-yetus commented on PR #4369:
URL: https://github.com/apache/hadoop/pull/4369#issuecomment-1140234577

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |::|--:|:|::|:---:|
   | +0 :ok: |  reexec  |   0m 39s |  |  Docker mode activated.  |
    _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files 
found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +0 :ok: |  detsecrets  |   0m  0s |  |  detect-secrets 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  |  54m 30s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   1m 46s |  |  trunk passed with JDK 
Private Build-11.0.15+10-Ubuntu-0ubuntu0.20.04.1  |
   | +1 :green_heart: |  compile  |   1m 37s |  |  trunk passed with JDK 
Private Build-1.8.0_312-8u312-b07-0ubuntu1~20.04-b07  |
   | +1 :green_heart: |  checkstyle  |   1m 25s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   1m 48s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   1m 26s |  |  trunk passed with JDK 
Private Build-11.0.15+10-Ubuntu-0ubuntu0.20.04.1  |
   | +1 :green_heart: |  javadoc  |   1m 49s |  |  trunk passed with JDK 
Private Build-1.8.0_312-8u312-b07-0ubuntu1~20.04-b07  |
   | +1 :green_heart: |  spotbugs  |   3m 39s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  22m 53s |  |  branch has no errors 
when building and testing our client artifacts.  |
    _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   1m 25s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 26s |  |  the patch passed with JDK 
Private Build-11.0.15+10-Ubuntu-0ubuntu0.20.04.1  |
   | +1 :green_heart: |  javac  |   1m 26s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 21s |  |  the patch passed with JDK 
Private Build-1.8.0_312-8u312-b07-0ubuntu1~20.04-b07  |
   | +1 :green_heart: |  javac  |   1m 21s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks 
issues.  |
   | +1 :green_heart: |  checkstyle  |   1m  2s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   1m 26s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 57s |  |  the patch passed with JDK 
Private Build-11.0.15+10-Ubuntu-0ubuntu0.20.04.1  |
   | +1 :green_heart: |  javadoc  |   1m 32s |  |  the patch passed with JDK 
Private Build-1.8.0_312-8u312-b07-0ubuntu1~20.04-b07  |
   | +1 :green_heart: |  spotbugs  |   3m 22s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  22m 32s |  |  patch has no errors 
when building and testing our client artifacts.  |
    _ Other Tests _ |
   | +1 :green_heart: |  unit  | 243m  7s |  |  hadoop-hdfs in the patch 
passed.  |
   | +1 :green_heart: |  asflicense  |   1m 15s |  |  The patch does not 
generate ASF License warnings.  |
   |  |   | 369m 21s |  |  |
   
   
   | Subsystem | Report/Notes |
   |--:|:-|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: 
https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4369/1/artifact/out/Dockerfile
 |
   | GITHUB PR | https://github.com/apache/hadoop/pull/4369 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall 
mvnsite unit shadedclient spotbugs checkstyle codespell detsecrets |
   | uname | Linux 2a881ab226da 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 
11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / ea78ff70fe4e1e527ebca5486eba4fd67203fa37 |
   | Default Java | Private Build-1.8.0_312-8u312-b07-0ubuntu1~20.04-b07 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Private 
Build-11.0.15+10-Ubuntu-0ubuntu0.20.04.1 
/usr/lib/jvm/java-8-openjdk-amd64:Private 
Build-1.8.0_312-8u312-b07-0ubuntu1~20.04-b07 |
   |  Test Results | 
https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4369/1/testReport/ |
   | Max. process+thread count | 3466 (vs. ulimit of 5500) |
   | modules | C: hadoop-hdfs-project/hadoop-hdfs U: 
hadoop-hdfs-project/hadoop-hdfs |
   | Console output | 
https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4369/1/console |
   | versions | 

[jira] [Work logged] (HDFS-16601) Failed to replace a bad datanode on the existing pipeline due to no more good datanodes being available to try

2022-05-27 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HDFS-16601:
-

Author: ASF GitHub Bot
Created on: 28/May/22 04:32
Start Date: 28/May/22 04:32
Worklog Time Spent: 10m 
  Work Description: ZanderXu opened a new pull request, #4369:
URL: https://github.com/apache/hadoop/pull/4369

   Detail info please refer to 
[HDFS-16601](https://issues.apache.org/jira/browse/HDFS-16601). 
   
   Bug stack like:
   ```
   java.io.IOException: Failed to replace a bad datanode on the existing 
pipeline due to no more good datanodes being available to try. (Nodes: 
current=[DatanodeInfoWithStorage[127.0.0.1:59687,DS-b803febc-7b22-4144-9b39-7bf521cdaa8d,DISK],
 
DatanodeInfoWithStorage[127.0.0.1:59670,DS-0d652bc2-1784-430d-961f-750f80a290f1,DISK]],
 
original=[DatanodeInfoWithStorage[127.0.0.1:59670,DS-0d652bc2-1784-430d-961f-750f80a290f1,DISK],
 
DatanodeInfoWithStorage[127.0.0.1:59687,DS-b803febc-7b22-4144-9b39-7bf521cdaa8d,DISK]]).
 The current failed datanode replacement policy is DEFAULT, and a client may 
configure this via 'dfs.client.block.write.replace-datanode-on-failure.policy' 
in its configuration.
at 
org.apache.hadoop.hdfs.DataStreamer.findNewDatanode(DataStreamer.java:1418)
at 
org.apache.hadoop.hdfs.DataStreamer.addDatanode2ExistingPipeline(DataStreamer.java:1478)
at 
org.apache.hadoop.hdfs.DataStreamer.handleDatanodeReplacement(DataStreamer.java:1704)
at 
org.apache.hadoop.hdfs.DataStreamer.setupPipelineInternal(DataStreamer.java:1605)
at 
org.apache.hadoop.hdfs.DataStreamer.setupPipelineForAppendOrRecovery(DataStreamer.java:1587)
at 
org.apache.hadoop.hdfs.DataStreamer.processDatanodeOrExternalError(DataStreamer.java:1371)
at org.apache.hadoop.hdfs.DataStreamer.run(DataStreamer.java:674)
   ```




Issue Time Tracking
---

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

> Failed to replace a bad datanode on the existing pipeline due to no more good 
> datanodes being available to try
> --
>
> Key: HDFS-16601
> URL: https://issues.apache.org/jira/browse/HDFS-16601
> Project: Hadoop HDFS
>  Issue Type: Bug
>Reporter: ZanderXu
>Assignee: ZanderXu
>Priority: Major
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> In our production environment, we found a bug and stack like:
> {code:java}
> java.io.IOException: Failed to replace a bad datanode on the existing 
> pipeline due to no more good datanodes being available to try. (Nodes: 
> current=[DatanodeInfoWithStorage[127.0.0.1:59687,DS-b803febc-7b22-4144-9b39-7bf521cdaa8d,DISK],
>  
> DatanodeInfoWithStorage[127.0.0.1:59670,DS-0d652bc2-1784-430d-961f-750f80a290f1,DISK]],
>  
> original=[DatanodeInfoWithStorage[127.0.0.1:59670,DS-0d652bc2-1784-430d-961f-750f80a290f1,DISK],
>  
> DatanodeInfoWithStorage[127.0.0.1:59687,DS-b803febc-7b22-4144-9b39-7bf521cdaa8d,DISK]]).
>  The current failed datanode replacement policy is DEFAULT, and a client may 
> configure this via 
> 'dfs.client.block.write.replace-datanode-on-failure.policy' in its 
> configuration.
>   at 
> org.apache.hadoop.hdfs.DataStreamer.findNewDatanode(DataStreamer.java:1418)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.addDatanode2ExistingPipeline(DataStreamer.java:1478)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.handleDatanodeReplacement(DataStreamer.java:1704)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.setupPipelineInternal(DataStreamer.java:1605)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.setupPipelineForAppendOrRecovery(DataStreamer.java:1587)
>   at 
> org.apache.hadoop.hdfs.DataStreamer.processDatanodeOrExternalError(DataStreamer.java:1371)
>   at org.apache.hadoop.hdfs.DataStreamer.run(DataStreamer.java:674)
> {code}
> And the root cause is that DFSClient cannot  perceive the exception of 
> TransferBlock during PipelineRecovery. If failed during TransferBlock, the 
> DFSClient will retry all datanodes in the cluster and then failed.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)

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