[ https://issues.apache.org/jira/browse/HDFS-17299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17814535#comment-17814535 ]
ASF GitHub Bot commented on HDFS-17299: --------------------------------------- shahrs87 commented on code in PR #6513: URL: https://github.com/apache/hadoop/pull/6513#discussion_r1479041865 ########## hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java: ########## @@ -1607,8 +1607,11 @@ private void transfer(final DatanodeInfo src, final DatanodeInfo[] targets, * it can be written to. * This happens when a file is appended or data streaming fails * It keeps on trying until a pipeline is setup + * + * Returns boolean whether pipeline was setup successfully or not. + * This boolean is used upstream on whether to continue creating pipeline or throw exception */ - private void setupPipelineForAppendOrRecovery() throws IOException { + private boolean setupPipelineForAppendOrRecovery() throws IOException { Review Comment: We are changing the return type of `setupPipelineForAppendOrRecovery` and `setupPipelineInternal` methods. IIRC this is the reason: `handleBadDatanode` can silently fail to handle bad datanode [here](https://github.com/apache/hadoop/blob/trunk/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java#L1700-L1706) and `setupPipelineInternal` will silently return [here](https://github.com/apache/hadoop/blob/trunk/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java#L1637-L1638) without bubbling up the exception. ########## hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java: ########## @@ -1618,24 +1621,33 @@ private void setupPipelineForAppendOrRecovery() throws IOException { LOG.warn(msg); lastException.set(new IOException(msg)); streamerClosed = true; - return; + return false; } - setupPipelineInternal(nodes, storageTypes, storageIDs); + return setupPipelineInternal(nodes, storageTypes, storageIDs); } - protected void setupPipelineInternal(DatanodeInfo[] datanodes, + protected boolean setupPipelineInternal(DatanodeInfo[] datanodes, StorageType[] nodeStorageTypes, String[] nodeStorageIDs) throws IOException { boolean success = false; long newGS = 0L; + boolean isCreateStage = BlockConstructionStage.PIPELINE_SETUP_CREATE == stage; while (!success && !streamerClosed && dfsClient.clientRunning) { if (!handleRestartingDatanode()) { - return; + return false; + } + + final boolean isRecovery = errorState.hasInternalError() && !isCreateStage; + + // During create stage, if we remove a node (nodes.length - 1) + // min replication should still be satisfied. + if (isCreateStage && !(dfsClient.dtpReplaceDatanodeOnFailureReplication > 0 && Review Comment: Reason behind adding this check here: We are already doing this check in catch block of `addDatanode2ExistingPipeline` method [here](https://github.com/apache/hadoop/blob/trunk/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java#L1528-L1539). But when `isAppend` flag is set to `false` and we are in `PIPELINE_SETUP_CREATE` phase, we exit early from `addDatanode2ExistingPipeline` method [here](https://github.com/apache/hadoop/blob/trunk/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java#L1489-L1492) Lets say the replication factor is 3 and we have set the config property `dfs.client.block.write.replace-datanode-on-failure.min-replication` to 3 and there is one bad node in the pipeline. Even if we have set the config property to `ReplaceDatanodeOnFailure.CONDITION_TRUE`, the code will exit the addDatanode2ExistingPipeline method early since `isAppend` is set to false and stage is `PIPELINE_SETUP_CREATE`. Assuming that there are NO available nodes in the rack, the pipeline will succeed with 2 nodes in the pipeline which will violate the config property: `dfs.client.block.write.replace-datanode-on-failure.min-replication` Having written all of these, I realized that even if there are some good nodes available in the rack, we will exit early after this patch. Should we move this check after `handleDatanodeReplacement` method? @ritegarg > HDFS is not rack failure tolerant while creating a new file. > ------------------------------------------------------------ > > Key: HDFS-17299 > URL: https://issues.apache.org/jira/browse/HDFS-17299 > Project: Hadoop HDFS > Issue Type: Bug > Affects Versions: 2.10.1 > Reporter: Rushabh Shah > Assignee: Ritesh > Priority: Critical > Labels: pull-request-available > Attachments: repro.patch > > > Recently we saw an HBase cluster outage when we mistakenly brought down 1 AZ. > Our configuration: > 1. We use 3 Availability Zones (AZs) for fault tolerance. > 2. We use BlockPlacementPolicyRackFaultTolerant as the block placement policy. > 3. We use the following configuration parameters: > dfs.namenode.heartbeat.recheck-interval: 600000 > dfs.heartbeat.interval: 3 > So it will take 1230000 ms (20.5mins) to detect that datanode is dead. > > Steps to reproduce: > # Bring down 1 AZ. > # HBase (HDFS client) tries to create a file (WAL file) and then calls > hflush on the newly created file. > # DataStreamer is not able to find blocks locations that satisfies the rack > placement policy (one copy in each rack which essentially means one copy in > each AZ) > # Since all the datanodes in that AZ are down but still alive to namenode, > the client gets different datanodes but still all of them are in the same AZ. > See logs below. > # HBase is not able to create a WAL file and it aborts the region server. > > Relevant logs from hdfs client and namenode > > {noformat} > 2023-12-16 17:17:43,818 INFO [on default port 9000] FSNamesystem.audit - > allowed=true ugi=hbase/<rs-name> (auth:KERBEROS) ip=<rs-IP> > cmd=create src=/hbase/WALs/<WAL-file> dst=null > 2023-12-16 17:17:43,978 INFO [on default port 9000] hdfs.StateChange - > BLOCK* allocate blk_1214652565_140946716, replicas=<AZ-1-dn-1>:50010, > <AZ-2-dn-1>:50010, <AZ-3-dn-1>:50010 for /hbase/WALs/<WAL-file> > 2023-12-16 17:17:44,061 INFO [Thread-39087] hdfs.DataStreamer - Exception in > createBlockOutputStream > java.io.IOException: Got error, status=ERROR, status message , ack with > firstBadLink as <AZ-2-dn-1>:50010 > at > org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil.checkBlockOpStatus(DataTransferProtoUtil.java:113) > at > org.apache.hadoop.hdfs.DataStreamer.createBlockOutputStream(DataStreamer.java:1747) > at > org.apache.hadoop.hdfs.DataStreamer.nextBlockOutputStream(DataStreamer.java:1651) > at org.apache.hadoop.hdfs.DataStreamer.run(DataStreamer.java:715) > 2023-12-16 17:17:44,061 WARN [Thread-39087] hdfs.DataStreamer - Abandoning > BP-179318874-<NN-IP>-1594838129323:blk_1214652565_140946716 > 2023-12-16 17:17:44,179 WARN [Thread-39087] hdfs.DataStreamer - Excluding > datanode > DatanodeInfoWithStorage[<AZ-2-dn-1>:50010,DS-a493abdb-3ac3-49b1-9bfb-848baf5c1c2c,DISK] > 2023-12-16 17:17:44,339 INFO [on default port 9000] hdfs.StateChange - > BLOCK* allocate blk_1214652580_140946764, replicas=<AZ-1-dn-2>:50010, > <AZ-3-dn-2>:50010, <AZ-2-dn-2>:50010 for /hbase/WALs/<WAL-file> > 2023-12-16 17:17:44,369 INFO [Thread-39087] hdfs.DataStreamer - Exception in > createBlockOutputStream > java.io.IOException: Got error, status=ERROR, status message , ack with > firstBadLink as <AZ-2-dn-2>:50010 > at > org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil.checkBlockOpStatus(DataTransferProtoUtil.java:113) > at > org.apache.hadoop.hdfs.DataStreamer.createBlockOutputStream(DataStreamer.java:1747) > at > org.apache.hadoop.hdfs.DataStreamer.nextBlockOutputStream(DataStreamer.java:1651) > at org.apache.hadoop.hdfs.DataStreamer.run(DataStreamer.java:715) > 2023-12-16 17:17:44,369 WARN [Thread-39087] hdfs.DataStreamer - Abandoning > BP-179318874-NN-IP-1594838129323:blk_1214652580_140946764 > 2023-12-16 17:17:44,454 WARN [Thread-39087] hdfs.DataStreamer - Excluding > datanode > DatanodeInfoWithStorage[AZ-2-dn-2:50010,DS-46bb45cc-af89-46f3-9f9d-24e4fdc35b6d,DISK] > 2023-12-16 17:17:44,522 INFO [on default port 9000] hdfs.StateChange - > BLOCK* allocate blk_1214652594_140946796, replicas=<AZ-1-dn-2>:50010, > <AZ-2-dn-3>:50010, <AZ-3-dn-3>:50010 for /hbase/WALs/<WAL-file> > 2023-12-16 17:17:44,712 INFO [Thread-39087] hdfs.DataStreamer - Exception in > createBlockOutputStream > java.io.IOException: Got error, status=ERROR, status message , ack with > firstBadLink as <AZ-2-dn-3>:50010 > at > org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil.checkBlockOpStatus(DataTransferProtoUtil.java:113) > at > org.apache.hadoop.hdfs.DataStreamer.createBlockOutputStream(DataStreamer.java:1747) > at > org.apache.hadoop.hdfs.DataStreamer.nextBlockOutputStream(DataStreamer.java:1651) > at org.apache.hadoop.hdfs.DataStreamer.run(DataStreamer.java:715) > 2023-12-16 17:17:44,712 WARN [Thread-39087] hdfs.DataStreamer - Abandoning > BP-179318874-NN-IP-1594838129323:blk_1214652594_140946796 > 2023-12-16 17:17:44,732 WARN [Thread-39087] hdfs.DataStreamer - Excluding > datanode > DatanodeInfoWithStorage[<AZ-2-dn-3>:50010,DS-86b77463-a26f-4f42-ae1b-21b75c407203,DISK] > 2023-12-16 17:17:44,855 INFO [on default port 9000] hdfs.StateChange - > BLOCK* allocate blk_1214652607_140946850, replicas=<AZ-1-dn-4>:50010, > <AZ-2-dn-4>:50010, <AZ-3-dn-4>:50010 for /hbase/WALs/<WAL-file> > 2023-12-16 17:17:44,867 INFO [Thread-39087] hdfs.DataStreamer - Exception in > createBlockOutputStream > java.io.IOException: Got error, status=ERROR, status message , ack with > firstBadLink as <AZ-2-dn-4>:50010 > at > org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil.checkBlockOpStatus(DataTransferProtoUtil.java:113) > at > org.apache.hadoop.hdfs.DataStreamer.createBlockOutputStream(DataStreamer.java:1747) > at > org.apache.hadoop.hdfs.DataStreamer.nextBlockOutputStream(DataStreamer.java:1651) > at org.apache.hadoop.hdfs.DataStreamer.run(DataStreamer.java:715) > 2023-12-16 17:17:44,988 WARN [Thread-39087] hdfs.DataStreamer - DataStreamer > Exception > java.io.IOException: Unable to create new block. > at > org.apache.hadoop.hdfs.DataStreamer.nextBlockOutputStream(DataStreamer.java:1665) > at org.apache.hadoop.hdfs.DataStreamer.run(DataStreamer.java:715) > 2023-12-16 17:17:44,988 WARN [Thread-39087] hdfs.DataStreamer - Could not > get block locations. Source file "/hbase/WALs/<WAL-file>" - Aborting... > {noformat} > > *Proposed fix:* > Client always correctly identifies the bad datanode in the pipeline. > The number of retries dfs client makes is controlled by > dfs.client.block.write.retries (defaults to 3). So in total it tries 4 times > to create the pipeline. > So on the 3rd or 4th attempt, if we see all the excluded nodes in the > pipeline belongs to the same rack, we can pass the hint to namenode to > exclude that rack for the next attempt. > Once that rack is back online, Replication monitor will handle to replicate > that block to that rack. > -- 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