Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

HDFS-17658. HDFS decommissioning does not consider if Under Construction blocks are sufficiently replicated which causes HDFS Data Loss #7179

Open
wants to merge 5 commits into
base: trunk
Choose a base branch
from

Conversation

KevinWikant
Copy link
Contributor

@KevinWikant KevinWikant commented Nov 21, 2024

Problem

Problem background:

  • A datanode should only enter decommissioned state if all the blocks on the datanode are sufficiently replicated to other live datanodes.
  • This expectation is violated for Under Construction blocks which are not considered by the DatanodeAdminMonitor at all.
  • DatanodeAdminMonitor currently only considers blocks in the DatanodeDescriptor StorageInfos. This is because:
    • For a new HDFS block that was just created, it is not be added to the StorageInfos until the HDFS client closes the DFSOutputStream & the block becomes finalized
    • For an existing HDFS block that was opened for append:
      • First, the block version with the previous generation stamp is marked stale & removed from the StorageInfos
      • Next, the block version with the new generation stamp is not be added to the StorageInfos until the HDFS client closes the DFSOutputStream & the block becomes finalized

There is logic in the DatanodeAdminManager/DatanodeAdminMonitor to avoid transitioning datanodes to decommissioned state when they have open (i.e. Under Construction) blocks:

This logic does not work correctly because, as mentioned above, DatanodeAdminMonitor currently only considers blocks in the DatanodeDescriptor StorageInfos which does not include Under Construction blocks for which the DFSOutputStream has not been closed yet.

There is also logic in the HDFS DataStreamer client which will replace bad/dead datanodes in the block write pipeline. Note that:

  • this logic does not work if the replication factor is 1
  • if the replication factor is greater than 1, then this logic does not work if all the datanodes in the block write pipeline are decommissioned/terminated at around the same time

Overall, the Namenode should not be putting datanodes with open blocks into decommissioned state & hope that the DataStreamer client is able to replace them when the decommissioned datanodes are terminated. This will not work depending on the timing & therefore is not a solution which guarantees correctness.

The Namenode needs to honor the rule that "a datanode should only enter decommissioned state if all the blocks on the datanode are sufficiently replicated to other live datanodes", even for blocks which are currently Under Construction.

Potential Solutions

One possible opinion is that if the DFSOutputStream has not been successfuly closed yet, then the client should be able to replay all the data if there is a failure. The client should not have any expectation the data is committed to HDFS until the DFSOutputStream is closed. There are a few reasons I do not think this makes sense:

  • The methods hflush/hsync do not result in the data already appended to the DFSOutputStream being persisted/finalized. This is confusing when compared the standard experience of stream flush/sync methods.
  • This does not handle the case where a block is re-opened by a new DFSOutputStream after having been previously closed (by another different client). In this case, the problem will lead to data loss for data that was previously committed by another client & cannot be replayed.
    • To solve this problem, we could try not removing old block version from StorageInfos when a new block version is created; however, this change is likely to have wider implications on block management.

Another possible option that comes to mind is to add blocks to StorageInfos before they are finalized. However, this change also is likely to have wider implications on block management.

Without modifying any existing block management logic, we can add a new data structure (UnderConstructionBlocks) which temporarily tracks the Under Construction blocks in-memory until they are committed/finalized & added to the StorageInfos.

Solution

Add a new data structure (UnderConstructionBlocks) which temporarily tracks the Under Construction blocks in-memory until they are committed/finalized & added to the StorageInfos.

Pros:

  • works for newly created HDFS block
  • works for re-opened HDFS block (i.e. opened for append)
  • works for block with any replication factor
  • does not change logic in BlockManager, the new UnderConstructionBlocks data structure & associated logic is purely additive

Implementation Details

  • Feature is behind a configuration "dfs.namenode.decommission.track.underconstructionblocks" which is disabled by default.

    • When enabled, feature prevents HDFS data loss & write failures at the cost of potentially slowing down the decommissioning process.
    • Customer who do not use HDFS decommissioning feature can choose to leave the feature disabled as they will not benefit from the additional CPU/memory overhead consumed by UnderConstructionBlocks.
    • Customers who do use HDFS decommissioning feature but who do not care about HDFS data loss & write failures can choose to leave the feature disabled.
    • Main use-case for enabling the feature is for HDFS clusters with many concurrent write operations & datanode decommissioning operations. These clusters will see benefit of reduced HDFS data loss & write failures caused by decommissioning.
  • In the regular case, when a DFSOutputStream is closed it takes 1-2 seconds for the block replicas to be removed from UnderConstructionBlocks & added to the StorageInfos. Therefore, datanode decommissioning is only blocked until the DFSOutputStream is closed & the write operation is finished, after this time there is minimal delay in unblocking decommissioning.

  • In the unhappy case, when an HDFS client fails & the DFSOutputStream is never closed, then it takes dfs.namenode.lease-hard-limit-sec = 20 minutes before the lease expires & the Namenode recovers the block. As part of block recovery, the block replicas are removed from UnderConstructionBlocks & added to the StorageInfos. Therefore, if an HDFS client fails it will (by default) take 20 minutes before decommissioning becomes unblocked.

  • The UnderConstructionBlocks data structure is in-memory only & therefore if the Namenode is restarted then it will lose track of any previously reported Under Construction blocks. This means that datanodes can be decommissioned with Under Construction blocks if the Namenode is restarted (which makes HDFS data loss & write failures possible again).

  • Testing shows that UnderConstructionBlocks should not leak any Under Construction blocks (i.e. which are never removed). However, as a safeguard to monitor for this issue, any block replica open for over 2 hours will have a WARN log printed by the Namenode every 30 minutes which mentions how long the block has been open for.

  • The implementation of UnderConstructionBlocks was borrowed from existing code in PendingDataNodeMessages. PendingDataNodeMessages is already used by the BlockManager to track in-memory the block replicas which have been reported to the standby namenode out-of-order.

How was this patch tested?

Test Methodology

I have used the following HDFS client code to test various scenarios on a Hadoop 3.4.0 cluster with 6 datanodes:

import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.hdfs.DFSOutputStream;
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
import org.apache.hadoop.fs.CreateFlag;

import java.io.IOException;
import java.io.OutputStream;
import java.net.InetAddress;
import java.net.URI;
import java.nio.charset.StandardCharsets;
import java.time.format.DateTimeFormatter;
import java.time.Duration;
import java.time.Instant;
import java.time.ZoneId;
import java.util.EnumSet;
import java.util.UUID;

public class HdfsWrite {
    // Constants
    private static final String HDFS_URI = "hdfs://%s:8020";
    private static final String HDFS_PATH = "/tmp/testfile-" + UUID.randomUUID().toString();
    private static final int DATA_SIZE_KB = 100;
    private static final int BUFFER_SIZE_KB = 25;
    private static final int SLEEP_INTERVAL_SECONDS = 20;
    private static final short REPLICATION = (short) 3;
    private static final long BLOCK_SIZE = 256L * 1024 * 1024;
    private static final DateTimeFormatter TIME_FORMAT = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss").withZone(ZoneId.of("UTC"));

    // Tunable Configs
    private static final Duration INITIAL_WRITE_DURATION = Duration.ofMinutes(2);
    private static final boolean shouldAppend = true;
    private static final Duration SECOND_WRITE_DURATION = Duration.ofMinutes(500);

    private static long totalSize = 0;

    public static void main(String[] args) {
        try {
            URI hdfsURI = new URI(String.format(HDFS_URI, InetAddress.getLocalHost().getHostName()));
            Configuration conf = new Configuration();
            DFSClient client = new DFSClient(hdfsURI, conf);

            // Initial Create/Write
            DFSOutputStream createStream = createFile(client);
            logLine(String.format("Created File %s", HDFS_PATH));
            Instant exitTime = Instant.now().plus(INITIAL_WRITE_DURATION);
            while (Instant.now().isBefore(exitTime)) {
                appendData(createStream);
                Thread.sleep(SLEEP_INTERVAL_SECONDS * 1000);
            }
            createStream.close();
            logLine(String.format("Closed File"));

            // Exit if append disabled
            if (!shouldAppend) {
                return;
            }

            // Second Append/Write
            DFSOutputStream outStream = openFileForAppend(client);
            logLine(String.format("Opened File %s", HDFS_PATH));
            exitTime = Instant.now().plus(SECOND_WRITE_DURATION);
            while (Instant.now().isBefore(exitTime)) {
                appendData(outStream);
                Thread.sleep(SLEEP_INTERVAL_SECONDS * 1000);
            }
            outStream.close();
            logLine(String.format("Closed File"));
        } catch (Exception e) {
            e.printStackTrace();
        }
    }

    private static DFSOutputStream createFile(DFSClient client) throws Exception {
        return (DFSOutputStream) client.create(HDFS_PATH, true, REPLICATION, BLOCK_SIZE);
    }

    private static DFSOutputStream openFileForAppend(DFSClient client) throws Exception {
        return (DFSOutputStream) ((HdfsDataOutputStream) client.append(HDFS_PATH, BUFFER_SIZE_KB * 1024, EnumSet.of(CreateFlag.APPEND), null, null)).getWrappedStream();
    }

    private static void appendData(DFSOutputStream outStream) {
        try {
            String data = generateData(DATA_SIZE_KB * 1024);
            outStream.write(data.getBytes(StandardCharsets.UTF_8));
            outStream.hflush();
            totalSize += DATA_SIZE_KB;
            logLine(String.format("Append Success  %d KB", totalSize));
        } catch (Exception e) {
            logLine(String.format("Append Failed   %d KB", totalSize));
            e.printStackTrace();
        }
    }

    private static String generateData(int size) {
        StringBuilder sb = new StringBuilder();
        while (sb.length() < size) {
            sb.append("A");
        }
        return sb.toString();
    }

    private static void logLine(String line) {
        Instant now = Instant.now();
        System.out.println(TIME_FORMAT.format(now) + " " + line);
    }
}

The code enables testing 2 key scenarios:

  1. Decommissioning datanodes with block replicas that are actively being written by a DFSOutputStream
  2. Decommissioning datanodes with block replicas that were previously written by a DFSOutputStream that has been closed

Both these scenarios can also be tested for:

  • A new HDFS block/file which has just been created with HDFS client "create" method
  • An existing HDFS block/file which was re-opened with HDFS client "append" method

Test Results Summary

Tests which show the behaviour before & after this change:

Test # Test Case Test Script Configurations Behaviour Before this Change Behaviour After this Change
1 Create & write new file, then close file, then decommission datanodes with the block replicas INITIAL_WRITE_DURATION=2mins, shouldAppend=false, SECOND_WRITE_DURATION=n/a DatanodeAdminMonitor moves block replicas to other live datanodes before decommissioning finishes. No HDFS data loss. No HDFS write failures. DatanodeAdminMonitor moves block replicas to other live datanodes before decommissioning finishes. No HDFS data loss. No HDFS write failures.
2 Create & repeatedly write file without closing, at the same time decommission datanodes with the block replicas INITIAL_WRITE_DURATION=60mins, shouldAppend=false, SECOND_WRITE_DURATION=n/a DatanodeAdminMonitor does not consider the under construction block replicas & the datanodes move to decommissioned immediately. When the decommissioned datanodes are terminated this leads to HDFS write failures & HDFS data loss. DatanodeAdminMonitor does not transition datanodes to decommissioned if they have under construction block replicas. No HDFS data loss. No HDFS write failures.
3 Create & write new file, then close file, then open file for append & write file, then close file, then decommission datanodes with the block replicas INITIAL_WRITE_DURATION=2mins, shouldAppend=true, SECOND_WRITE_DURATION=2mins DatanodeAdminMonitor moves block replicas to other live datanodes before decommissioning finishes. No HDFS data loss. No HDFS write failures. DatanodeAdminMonitor moves block replicas to other live datanodes before decommissioning finishes. No HDFS data loss. No HDFS write failures.
4 Create & write new file, then close file, then open file for append & repeatedly write file without closing, at the same time decommission datanodes with the block replicas INITIAL_WRITE_DURATION=2mins, shouldAppend=true, SECOND_WRITE_DURATION=60mins DatanodeAdminMonitor does not consider the under construction block replicas & the datanodes move to decommissioned immediately. When the decommissioned datanodes are terminated this leads to HDFS write failures & HDFS data loss. DatanodeAdminMonitor does not transition datanodes to decommissioned if they have under construction block replicas. No HDFS data loss. No HDFS write failures.

Tests which show how long these changes can possibly block datanode decommissioning:

Test # Test Case Test Script Configurations Behaviour After this Change
5 Create & repeatedly write file without closing, at the same time decommission datanodes with the block replicas, then close the file while datanodes are still decommissioning INITIAL_WRITE_DURATION=10mins, shouldAppend=false, SECOND_WRITE_DURATION=n/a DatanodeAdminMonitor does not transition datanodes to decommissioned while they have under construction block replicas. No HDFS data loss. No HDFS write failures. When the DFSOutputStream is finally close, it takes 1-2 seconds before the replicas are reported to the Namenode as RECEIVED_BLOCK & the decommissioning process can then complete as-usual.
6 Create & repeatedly write file without closing, at the same time decommission datanodes with the block replicas, then kill the client abruptly INITIAL_WRITE_DURATION=60mins, shouldAppend=false, SECOND_WRITE_DURATION=n/a DatanodeAdminMonitor does not transition datanodes to decommissioned while they have under construction block replicas. No HDFS data loss. No HDFS write failures. After dfs.namenode.lease-hard-limit-sec = 20 minutes from the time the client failed, the Namenode transitions the block replicas into recovery & the decommissioning process can then complete as-usual.

In other words:

  • in the happy case, datanode decommissioning is unblocked 1-2 seconds after the DFSOutputStream is closed by the client
  • in the worst case, datanode decommissioning is unblocked "dfs.namenode.lease-hard-limit-sec = 20 minutes" after the HDFS client fails abruptly

The above scenarios where tested with various different configurations:

  • tested with different "dfs.replication" values of 1, 2, & 3
  • tested with the following configurations:
    • "dfs.client.block.write.replace-datanode-on-failure.best-effort": "true"
    • "dfs.client.block.write.replace-datanode-on-failure.enable": "true"
    • "dfs.client.block.write.replace-datanode-on-failure.min-replication": "0"
    • "dfs.client.block.write.replace-datanode-on-failure.policy": "ALWAYS"
    • "dfs.client.block.write.retries": "10000"
    • "dfs.namenode.file.close.num-committed-allowed": "1"
  • tested with "dfs.namenode.decommission.consider.openfiles = false" to confirm behaviour is same as before the change
  • tested with Namenode restart and confirmed that this causes behaviour to be same as before the change
  • tested & confirmed UnderConstructionBlocks is not leaking any entries when there are many concurrent writes, datanode decommissioning operations, and Namenode restarts

Below are the detailed logs for a single test case 4.

Test#4 Detailed Results , Before this Change

The following are the HDFS client logs for the duration of the test:

2024-12-05 15:51:08 Created File /tmp/testfile-03ed9b5c-e1f4-42ad-bca3-8a579ac9d8b2
2024-12-05 15:51:08 Append Success  100 KB
2024-12-05 15:51:28 Append Success  200 KB
2024-12-05 15:51:48 Append Success  300 KB
2024-12-05 15:52:08 Append Success  400 KB
2024-12-05 15:52:28 Append Success  500 KB
2024-12-05 15:52:48 Append Success  600 KB
2024-12-05 15:53:08 Closed File
2024-12-05 15:53:08 Opened File /tmp/testfile-03ed9b5c-e1f4-42ad-bca3-8a579ac9d8b2
2024-12-05 15:53:09 Append Success  700 KB
2024-12-05 15:53:29 Append Success  800 KB
2024-12-05 15:53:49 Append Success  900 KB
2024-12-05 15:54:09 Append Success  1000 KB
2024-12-05 15:54:29 Append Success  1100 KB
2024-12-05 15:54:49 Append Success  1200 KB
2024-12-05 15:55:09 Append Success  1300 KB
2024-12-05 15:55:29 Append Success  1400 KB
2024-12-05 15:55:49 Append Success  1500 KB
2024-12-05 15:56:09 Append Success  1600 KB
2024-12-05 15:56:29 Append Failed   1600 KB
java.net.ConnectException: Connection refused
        at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
        at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:716)
        at org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.java:205)
        at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:635)
        at org.apache.hadoop.hdfs.DataStreamer.createSocketForPipeline(DataStreamer.java:256)
        at org.apache.hadoop.hdfs.DataStreamer$StreamerStreams.<init>(DataStreamer.java:165)
        at org.apache.hadoop.hdfs.DataStreamer.transfer(DataStreamer.java:1593)
        at org.apache.hadoop.hdfs.DataStreamer.addDatanode2ExistingPipeline(DataStreamer.java:1550)
        at org.apache.hadoop.hdfs.DataStreamer.handleDatanodeReplacement(DataStreamer.java:1758)
        at org.apache.hadoop.hdfs.DataStreamer.setupPipelineInternal(DataStreamer.java:1648)
        at org.apache.hadoop.hdfs.DataStreamer.setupPipelineForAppendOrRecovery(DataStreamer.java:1627)
        at org.apache.hadoop.hdfs.DataStreamer.processDatanodeOrExternalError(DataStreamer.java:1408)
        at org.apache.hadoop.hdfs.DataStreamer.run(DataStreamer.java:707)
2024-12-05 15:56:49 Append Failed   1600 KB
java.net.ConnectException: Connection refused
        ...
2024-12-05 15:57:09 Append Failed   1600 KB
java.net.ConnectException: Connection refused
        ...
2024-12-05 15:57:29 Append Failed   1600 KB
java.net.ConnectException: Connection refused
        ...
2024-12-05 15:57:49 Append Failed   1600 KB
java.net.ConnectException: Connection refused
        ...
2024-12-05 15:58:09 Append Failed   1600 KB
java.net.ConnectException: Connection refused
        ...
2024-12-05 15:58:29 Append Failed   1600 KB
java.net.ConnectException: Connection refused
        ...
2024-12-05 15:58:49 Append Failed   1600 KB
java.net.ConnectException: Connection refused
        ...
2024-12-05 15:59:09 Append Failed   1600 KB
java.net.ConnectException: Connection refused
        ...
2024-12-05 15:59:29 Append Failed   1600 KB
java.net.ConnectException: Connection refused
        ...
2024-12-05 15:59:49 Append Failed   1600 KB
java.net.ConnectException: Connection refused
        ...

First step is to identify the block locations:

> cat /var/log/hadoop-hdfs/hadoop-hdfs-namenode-ip-*.out | grep testfile | grep blk
...
2024-12-05 15:51:08,342 INFO hdfs.StateChange: BLOCK* allocate blk_1073741825_1001, replicas=172.31.86.141:9866, 172.31.93.89:9866, 172.31.87.40:9866 for /tmp/testfile-03ed9b5c-e1f4-42ad-bca3-8a579ac9d8b2
...

Then we wait for the block to be closed & re-opened. Once this is completed, we can start decommissioning the 3 datanodes with under construction block replicas:

> cat dfs.hosts.exclude
ip-172-31-86-141.ec2.internal
ip-172-31-93-89.ec2.internal
ip-172-31-87-40.ec2.internal

> export HADOOP_USER_NAME=hdfs

> hdfs dfsadmin -refreshNodes
...
2024-12-05 15:54:28,269 DEBUG ipc.ProtobufRpcEngine2: Call: refreshNodes took 100ms
Refresh nodes successful
...

Check the decommissioning status of the datanodes to confirm they are decommissioned:

> cat /var/log/hadoop-hdfs/hadoop-hdfs-namenode-ip-*.out | grep DatanodeAdmin
...
2024-12-05 15:53:32,804 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: DatanodeAdminMonitor is running.
2024-12-05 15:54:02,804 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: DatanodeAdminMonitor is running.
2024-12-05 15:54:28,264 INFO blockmanagement.DatanodeAdminManager: Starting decommission of 172.31.86.141:9866 [DISK]DS-723c51a0-a6ba-4f89-b394-66b750fc8506:NORMAL:172.31.86.141:9866 with 0 blocks
2024-12-05 15:54:28,264 INFO blockmanagement.DatanodeAdminManager: Starting decommission of 172.31.86.141:9866 [DISK]DS-29dcdc8e-7114-4eb4-adc0-4c64f9976afc:NORMAL:172.31.86.141:9866 with 0 blocks
2024-12-05 15:54:28,264 INFO blockmanagement.DatanodeAdminManager: Starting decommission of 172.31.93.89:9866 [DISK]DS-69dfe6d3-da97-4079-ab38-bdf48da523d0:NORMAL:172.31.93.89:9866 with 0 blocks
2024-12-05 15:54:28,264 INFO blockmanagement.DatanodeAdminManager: Starting decommission of 172.31.93.89:9866 [DISK]DS-1e28d508-5693-44e5-ada0-13f403130695:NORMAL:172.31.93.89:9866 with 0 blocks
2024-12-05 15:54:28,264 INFO blockmanagement.DatanodeAdminManager: Starting decommission of 172.31.87.40:9866 [DISK]DS-d7d188b2-1408-4a61-80ee-43dbb5982bff:NORMAL:172.31.87.40:9866 with 0 blocks
2024-12-05 15:54:28,264 INFO blockmanagement.DatanodeAdminManager: Starting decommission of 172.31.87.40:9866 [DISK]DS-2827b751-40f9-48fe-bb1e-f15db0c3f721:NORMAL:172.31.87.40:9866 with 0 blocks
2024-12-05 15:54:32,804 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: DatanodeAdminMonitor is running.
2024-12-05 15:54:32,806 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: Newly-added node 172.31.86.141:9866, doing full scan to find insufficiently-replicated blocks.
2024-12-05 15:54:32,806 INFO blockmanagement.DatanodeAdminManager: Decommissioning complete for node 172.31.86.141:9866
2024-12-05 15:54:32,806 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: Node 172.31.86.141:9866 is sufficiently replicated and healthy, marked as Decommissioned.
2024-12-05 15:54:32,806 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: Newly-added node 172.31.87.40:9866, doing full scan to find insufficiently-replicated blocks.
2024-12-05 15:54:32,806 INFO blockmanagement.DatanodeAdminManager: Decommissioning complete for node 172.31.87.40:9866
2024-12-05 15:54:32,806 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: Node 172.31.87.40:9866 is sufficiently replicated and healthy, marked as Decommissioned.
2024-12-05 15:54:32,806 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: Newly-added node 172.31.93.89:9866, doing full scan to find insufficiently-replicated blocks.
2024-12-05 15:54:32,806 INFO blockmanagement.DatanodeAdminManager: Decommissioning complete for node 172.31.93.89:9866
2024-12-05 15:54:32,806 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: Node 172.31.93.89:9866 is sufficiently replicated and healthy, marked as Decommissioned.
2024-12-05 15:54:32,806 INFO blockmanagement.DatanodeAdminDefaultMonitor: Checked 0 blocks and 3 nodes this tick. 0 nodes are now in maintenance or transitioning state. 0 nodes pending.
2024-12-05 15:55:02,806 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: DatanodeAdminMonitor is running.
2024-12-05 15:55:32,807 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: DatanodeAdminMonitor is running.
...
> hdfs dfsadmin -report | grep -i decommissioned -B 3
...
2024-12-05 15:54:52,548 DEBUG ipc.ProtobufRpcEngine2: Call: getDatanodeReport took 3ms

Name: 172.31.86.141:9866 (ip-172-31-86-141.ec2.internal)
Hostname: ip-172-31-86-141.ec2.internal
Decommission Status : Decommissioned
--

Name: 172.31.87.40:9866 (ip-172-31-87-40.ec2.internal)
Hostname: ip-172-31-87-40.ec2.internal
Decommission Status : Decommissioned
--

Name: 172.31.93.89:9866 (ip-172-31-93-89.ec2.internal)
Hostname: ip-172-31-93-89.ec2.internal
Decommission Status : Decommissioned

Then we stop/terminate all 3 decommissioned datanodes at the same time, this is expected to be safe since they are in decommissioned state:

> date
Thu Dec  5 15:56:17 UTC 2024

After this occurs, we see HDFS write failures from the client:

2024-12-05 15:55:49 Append Success  1500 KB
2024-12-05 15:56:09 Append Success  1600 KB
2024-12-05 15:56:29 Append Failed   1600 KB
java.net.ConnectException: Connection refused
        ...
2024-12-05 15:56:49 Append Failed   1600 KB
java.net.ConnectException: Connection refused
        ...

And we also see the underlying HDFS data is lost:

> hdfs dfs -get /tmp/testfile-03ed9b5c-e1f4-42ad-bca3-8a579ac9d8b2 /var/testfile
...
2024-12-05 15:58:07,912 DEBUG ipc.Client: Failed to connect to server: ip-172-31-87-40.ec2.internal/172.31.87.40:9867: retries get failed due to exceeded maximum allowed retries number: 10
java.net.NoRouteToHostException: No route to host
...
2024-12-05 15:58:42,232 DEBUG ipc.Client: Failed to connect to server: ip-172-31-93-89.ec2.internal/172.31.93.89:9867: retries get failed due to exceeded maximum allowed retries number: 10
java.net.NoRouteToHostException: No route to host
...
2024-12-05 15:59:16,562 DEBUG ipc.Client: Failed to connect to server: ip-172-31-86-141.ec2.internal/172.31.86.141:9867: retries get failed due to exceeded maximum allowed retries number: 10
java.net.NoRouteToHostException: No route to host
...
get: Cannot obtain block length for LocatedBlock{BP-571135626-172.31.88.138-1733413798130:blk_1073741825_1002; getBlockSize()=716800; corrupt=false; offset=0; locs=[DatanodeInfoWithStorage[172.31.87.40:9866,DS-2827b751-40f9-48fe-bb1e-f15db0c3f721,DISK], DatanodeInfoWithStorage[172.31.93.89:9866,DS-69dfe6d3-da97-4079-ab38-bdf48da523d0,DISK], DatanodeInfoWithStorage[172.31.86.141:9866,DS-29dcdc8e-7114-4eb4-adc0-4c64f9976afc,DISK]]; cachedLocs=[]} of /tmp/testfile-03ed9b5c-e1f4-42ad-bca3-8a579ac9d8b2

Introspecting further into the logs, we see that when the block was re-opened:

  • the BlockInfo with previous GenerationStamp was removed from the StorageInfos due to being "stale"
  • the BlockInfo with new GenerationStamp does not get added to the StorageInfos until the DFSOutputStream is closed and it transitions from RBW to FINALIZED
> cat /var/log/hadoop-hdfs/hadoop-hdfs-namenode-ip-*.out | grep blk_1073741825

2024-12-05 15:51:08,342 DEBUG hdfs.StateChange: DIR* FSDirectory.addBlock: /tmp/testfile-03ed9b5c-e1f4-42ad-bca3-8a579ac9d8b2 with blk_1073741825_1001 block is added to the in-memory file system
2024-12-05 15:51:08,342 INFO hdfs.StateChange: BLOCK* allocate blk_1073741825_1001, replicas=172.31.86.141:9866, 172.31.93.89:9866, 172.31.87.40:9866 for /tmp/testfile-03ed9b5c-e1f4-42ad-bca3-8a579ac9d8b2
2024-12-05 15:51:08,342 DEBUG namenode.FSEditLog: logEdit [RpcEdit op:AddBlockOp [path=/tmp/testfile-03ed9b5c-e1f4-42ad-bca3-8a579ac9d8b2, penultimateBlock=NULL, lastBlock=blk_1073741825_1001, RpcClientId=, RpcCallId=-2] call:Call#1 Retry#0 org.apache.hadoop.hdfs.protocol.ClientProtocol.addBlock from ip-172-31-88-138.ec2.internal:58352 / 172.31.88.138:58352]
2024-12-05 15:51:08,342 DEBUG hdfs.StateChange: persistNewBlock: /tmp/testfile-03ed9b5c-e1f4-42ad-bca3-8a579ac9d8b2 with new block blk_1073741825_1001, current total block count is 1
2024-12-05 15:51:08,343 DEBUG namenode.FSEditLog: doEditTx() op=AddBlockOp [path=/tmp/testfile-03ed9b5c-e1f4-42ad-bca3-8a579ac9d8b2, penultimateBlock=NULL, lastBlock=blk_1073741825_1001, RpcClientId=, RpcCallId=-2] txid=45
2024-12-05 15:51:08,879 DEBUG namenode.FSEditLog: logEdit [RpcEdit op:UpdateBlocksOp [path=/tmp/testfile-03ed9b5c-e1f4-42ad-bca3-8a579ac9d8b2, blocks=[blk_1073741825_1001], RpcClientId=, RpcCallId=-2] call:Call#3 Retry#0 org.apache.hadoop.hdfs.protocol.ClientProtocol.fsync from ip-172-31-88-138.ec2.internal:58352 / 172.31.88.138:58352]
2024-12-05 15:51:08,879 DEBUG namenode.FSEditLog: doEditTx() op=UpdateBlocksOp [path=/tmp/testfile-03ed9b5c-e1f4-42ad-bca3-8a579ac9d8b2, blocks=[blk_1073741825_1001], RpcClientId=, RpcCallId=-2] txid=46
2024-12-05 15:51:08,879 DEBUG namenode.FSEditLog: logSync [RpcEdit op:UpdateBlocksOp [path=/tmp/testfile-03ed9b5c-e1f4-42ad-bca3-8a579ac9d8b2, blocks=[blk_1073741825_1001], RpcClientId=, RpcCallId=-2] call:Call#3 Retry#0 org.apache.hadoop.hdfs.protocol.ClientProtocol.fsync from ip-172-31-88-138.ec2.internal:58352 / 172.31.88.138:58352]
2024-12-05 15:51:09,399 DEBUG blockmanagement.BlockManager: Reported block blk_1073741825_1001 on 172.31.93.89:9866 size 268435456 replicaState = RBW
2024-12-05 15:51:09,399 DEBUG blockmanagement.BlockManager: Reported block blk_1073741825_1001 on 172.31.93.89:9866 size 268435456 replicaState = RBW
2024-12-05 15:51:09,399 DEBUG BlockStateChange: BLOCK* block RECEIVING_BLOCK: blk_1073741825_1001 is received from 172.31.93.89:9866
2024-12-05 15:51:09,889 DEBUG blockmanagement.BlockManager: Reported block blk_1073741825_1001 on 172.31.86.141:9866 size 268435456 replicaState = RBW
2024-12-05 15:51:09,889 DEBUG blockmanagement.BlockManager: Reported block blk_1073741825_1001 on 172.31.86.141:9866 size 268435456 replicaState = RBW
2024-12-05 15:51:09,889 DEBUG BlockStateChange: BLOCK* block RECEIVING_BLOCK: blk_1073741825_1001 is received from 172.31.86.141:9866
2024-12-05 15:51:09,952 DEBUG blockmanagement.BlockManager: Reported block blk_1073741825_1001 on 172.31.87.40:9866 size 268435456 replicaState = RBW
2024-12-05 15:51:09,952 DEBUG blockmanagement.BlockManager: Reported block blk_1073741825_1001 on 172.31.87.40:9866 size 268435456 replicaState = RBW
2024-12-05 15:51:09,952 DEBUG BlockStateChange: BLOCK* block RECEIVING_BLOCK: blk_1073741825_1001 is received from 172.31.87.40:9866
2024-12-05 15:53:08,926 DEBUG blockmanagement.BlockManager: Reported block blk_1073741825_1001 on 172.31.87.40:9866 size 614400 replicaState = FINALIZED
2024-12-05 15:53:08,927 DEBUG blockmanagement.BlockManager: Reported block blk_1073741825_1001 on 172.31.87.40:9866 size 614400 replicaState = FINALIZED
2024-12-05 15:53:08,927 INFO BlockStateChange: BLOCK* addStoredBlock: 172.31.87.40:9866 is added to blk_1073741825_1001 (size=614400)
2024-12-05 15:53:08,928 DEBUG BlockStateChange: BLOCK* block RECEIVED_BLOCK: blk_1073741825_1001 is received from 172.31.87.40:9866
2024-12-05 15:53:08,929 DEBUG blockmanagement.BlockManager: Reported block blk_1073741825_1001 on 172.31.93.89:9866 size 614400 replicaState = FINALIZED
2024-12-05 15:53:08,929 DEBUG blockmanagement.BlockManager: Reported block blk_1073741825_1001 on 172.31.93.89:9866 size 614400 replicaState = FINALIZED
2024-12-05 15:53:08,929 INFO BlockStateChange: BLOCK* addStoredBlock: 172.31.93.89:9866 is added to blk_1073741825_1001 (size=614400)
2024-12-05 15:53:08,929 DEBUG BlockStateChange: BLOCK* block RECEIVED_BLOCK: blk_1073741825_1001 is received from 172.31.93.89:9866
2024-12-05 15:53:08,933 DEBUG blockmanagement.BlockManager: Reported block blk_1073741825_1001 on 172.31.86.141:9866 size 614400 replicaState = FINALIZED
2024-12-05 15:53:08,933 DEBUG blockmanagement.BlockManager: Reported block blk_1073741825_1001 on 172.31.86.141:9866 size 614400 replicaState = FINALIZED
2024-12-05 15:53:08,933 INFO BlockStateChange: BLOCK* addStoredBlock: 172.31.86.141:9866 is added to blk_1073741825_1001 (size=614400)
2024-12-05 15:53:08,933 DEBUG BlockStateChange: BLOCK* block RECEIVED_BLOCK: blk_1073741825_1001 is received from 172.31.86.141:9866
2024-12-05 15:53:08,937 DEBUG namenode.FSEditLog: logEdit [RpcEdit op:CloseOp [length=0, inodeId=0, path=/tmp/testfile-03ed9b5c-e1f4-42ad-bca3-8a579ac9d8b2, replication=3, mtime=1733413988937, atime=1733413868246, blockSize=268435456, blocks=[blk_1073741825_1001], permissions=root:hdfsadmingroup:rw-r--r--, aclEntries=null, clientName=null, clientMachine=null, overwrite=false, storagePolicyId=0, erasureCodingPolicyId=0, opCode=OP_CLOSE, txid=-12345] call:Call#8 Retry#0 org.apache.hadoop.hdfs.protocol.ClientProtocol.complete from ip-172-31-88-138.ec2.internal:33782 / 172.31.88.138:33782]
2024-12-05 15:53:08,937 DEBUG namenode.FSEditLog: doEditTx() op=CloseOp [length=0, inodeId=0, path=/tmp/testfile-03ed9b5c-e1f4-42ad-bca3-8a579ac9d8b2, replication=3, mtime=1733413988937, atime=1733413868246, blockSize=268435456, blocks=[blk_1073741825_1001], permissions=root:hdfsadmingroup:rw-r--r--, aclEntries=null, clientName=null, clientMachine=null, overwrite=false, storagePolicyId=0, erasureCodingPolicyId=0, opCode=OP_CLOSE, txid=47] txid=47
2024-12-05 15:53:08,937 DEBUG namenode.FSEditLog: logSync [RpcEdit op:CloseOp [length=0, inodeId=0, path=/tmp/testfile-03ed9b5c-e1f4-42ad-bca3-8a579ac9d8b2, replication=3, mtime=1733413988937, atime=1733413868246, blockSize=268435456, blocks=[blk_1073741825_1001], permissions=root:hdfsadmingroup:rw-r--r--, aclEntries=null, clientName=null, clientMachine=null, overwrite=false, storagePolicyId=0, erasureCodingPolicyId=0, opCode=OP_CLOSE, txid=47] call:Call#8 Retry#0 org.apache.hadoop.hdfs.protocol.ClientProtocol.complete from ip-172-31-88-138.ec2.internal:33782 / 172.31.88.138:33782]
2024-12-05 15:53:08,949 DEBUG hdfs.StateChange: DIR* NameSystem.appendFile: file /tmp/testfile-03ed9b5c-e1f4-42ad-bca3-8a579ac9d8b2 for DFSClient_NONMAPREDUCE_-958881537_1 at 172.31.88.138 block BP-571135626-172.31.88.138-1733413798130:blk_1073741825_1001 block size 614400
2024-12-05 15:53:08,960 INFO namenode.FSNamesystem: bumpBlockGenerationStamp(BP-571135626-172.31.88.138-1733413798130:blk_1073741825_1002, client=DFSClient_NONMAPREDUCE_-958881537_1) success
2024-12-05 15:53:09,020 INFO namenode.FSNamesystem: updatePipeline(blk_1073741825_1001, newGS=1002, newLength=614400, newNodes=[172.31.87.40:9866, 172.31.93.89:9866, 172.31.86.141:9866], client=DFSClient_NONMAPREDUCE_-958881537_1)
2024-12-05 15:53:09,020 DEBUG BlockStateChange: BLOCK* removeStoredBlock: blk_1073741825_1002 from 172.31.87.40:9866
2024-12-05 15:53:09,021 DEBUG BlockStateChange: BLOCK* Removing stale replica ReplicaUC[[DISK]DS-2827b751-40f9-48fe-bb1e-f15db0c3f721:NORMAL:172.31.87.40:9866|RBW] of blk_1073741825_1001
2024-12-05 15:53:09,021 DEBUG BlockStateChange: BLOCK* removeStoredBlock: blk_1073741825_1002 from 172.31.93.89:9866
2024-12-05 15:53:09,021 DEBUG BlockStateChange: BLOCK* Removing stale replica ReplicaUC[[DISK]DS-69dfe6d3-da97-4079-ab38-bdf48da523d0:NORMAL:172.31.93.89:9866|RBW] of blk_1073741825_1001
2024-12-05 15:53:09,021 DEBUG BlockStateChange: BLOCK* removeStoredBlock: blk_1073741825_1002 from 172.31.86.141:9866
2024-12-05 15:53:09,021 DEBUG BlockStateChange: BLOCK* Removing stale replica ReplicaUC[[DISK]DS-29dcdc8e-7114-4eb4-adc0-4c64f9976afc:NORMAL:172.31.86.141:9866|RBW] of blk_1073741825_1001
2024-12-05 15:53:09,021 DEBUG namenode.FSEditLog: logEdit [RpcEdit op:UpdateBlocksOp [path=/tmp/testfile-03ed9b5c-e1f4-42ad-bca3-8a579ac9d8b2, blocks=[blk_1073741825_1002], RpcClientId=cd00dc8b-bb34-48b4-993b-326ef1e57052, RpcCallId=11] call:Call#11 Retry#0 org.apache.hadoop.hdfs.protocol.ClientProtocol.updatePipeline from ip-172-31-88-138.ec2.internal:33782 / 172.31.88.138:33782]
2024-12-05 15:53:09,021 DEBUG namenode.FSEditLog: logSync [RpcEdit op:UpdateBlocksOp [path=/tmp/testfile-03ed9b5c-e1f4-42ad-bca3-8a579ac9d8b2, blocks=[blk_1073741825_1002], RpcClientId=cd00dc8b-bb34-48b4-993b-326ef1e57052, RpcCallId=11] call:Call#11 Retry#0 org.apache.hadoop.hdfs.protocol.ClientProtocol.updatePipeline from ip-172-31-88-138.ec2.internal:33782 / 172.31.88.138:33782]
2024-12-05 15:53:09,021 INFO namenode.FSNamesystem: updatePipeline(blk_1073741825_1001 => blk_1073741825_1002) success
2024-12-05 15:53:09,021 DEBUG namenode.FSEditLog: doEditTx() op=UpdateBlocksOp [path=/tmp/testfile-03ed9b5c-e1f4-42ad-bca3-8a579ac9d8b2, blocks=[blk_1073741825_1002], RpcClientId=cd00dc8b-bb34-48b4-993b-326ef1e57052, RpcCallId=11] txid=50
2024-12-05 15:53:09,031 DEBUG namenode.FSEditLog: logEdit [RpcEdit op:UpdateBlocksOp [path=/tmp/testfile-03ed9b5c-e1f4-42ad-bca3-8a579ac9d8b2, blocks=[blk_1073741825_1002], RpcClientId=, RpcCallId=-2] call:Call#12 Retry#0 org.apache.hadoop.hdfs.protocol.ClientProtocol.fsync from ip-172-31-88-138.ec2.internal:33782 / 172.31.88.138:33782]
2024-12-05 15:53:09,031 DEBUG namenode.FSEditLog: logSync [RpcEdit op:UpdateBlocksOp [path=/tmp/testfile-03ed9b5c-e1f4-42ad-bca3-8a579ac9d8b2, blocks=[blk_1073741825_1002], RpcClientId=, RpcCallId=-2] call:Call#12 Retry#0 org.apache.hadoop.hdfs.protocol.ClientProtocol.fsync from ip-172-31-88-138.ec2.internal:33782 / 172.31.88.138:33782]
2024-12-05 15:53:09,031 DEBUG namenode.FSEditLog: doEditTx() op=UpdateBlocksOp [path=/tmp/testfile-03ed9b5c-e1f4-42ad-bca3-8a579ac9d8b2, blocks=[blk_1073741825_1002], RpcClientId=, RpcCallId=-2] txid=51
2024-12-05 15:53:09,385 DEBUG blockmanagement.BlockManager: Reported block blk_1073741825_1002 on 172.31.93.89:9866 size 268435456 replicaState = RBW
2024-12-05 15:53:09,385 DEBUG blockmanagement.BlockManager: Reported block blk_1073741825_1002 on 172.31.93.89:9866 size 268435456 replicaState = RBW
2024-12-05 15:53:09,385 DEBUG BlockStateChange: BLOCK* block RECEIVING_BLOCK: blk_1073741825_1002 is received from 172.31.93.89:9866
2024-12-05 15:53:09,888 DEBUG blockmanagement.BlockManager: Reported block blk_1073741825_1002 on 172.31.86.141:9866 size 268435456 replicaState = RBW
2024-12-05 15:53:09,888 DEBUG blockmanagement.BlockManager: Reported block blk_1073741825_1002 on 172.31.86.141:9866 size 268435456 replicaState = RBW
2024-12-05 15:53:09,888 DEBUG BlockStateChange: BLOCK* block RECEIVING_BLOCK: blk_1073741825_1002 is received from 172.31.86.141:9866
2024-12-05 15:53:09,951 DEBUG blockmanagement.BlockManager: Reported block blk_1073741825_1002 on 172.31.87.40:9866 size 268435456 replicaState = RBW
2024-12-05 15:53:09,951 DEBUG blockmanagement.BlockManager: Reported block blk_1073741825_1002 on 172.31.87.40:9866 size 268435456 replicaState = RBW
2024-12-05 15:53:09,951 DEBUG BlockStateChange: BLOCK* block RECEIVING_BLOCK: blk_1073741825_1002 is received from 172.31.87.40:9866
2024-12-05 15:56:13,956 DEBUG namenode.NameNode: getAdditionalDatanode: src=/tmp/testfile-03ed9b5c-e1f4-42ad-bca3-8a579ac9d8b2, fileId=16403, blk=BP-571135626-172.31.88.138-1733413798130:blk_1073741825_1002, existings=[172.31.93.89:9866, 172.31.86.141:9866], excludes=[172.31.87.40:9866], numAdditionalNodes=1, clientName=DFSClient_NONMAPREDUCE_-958881537_1
2024-12-05 15:56:13,964 DEBUG namenode.NameNode: getAdditionalDatanode: src=/tmp/testfile-03ed9b5c-e1f4-42ad-bca3-8a579ac9d8b2, fileId=16403, blk=BP-571135626-172.31.88.138-1733413798130:blk_1073741825_1002, existings=[172.31.93.89:9866, 172.31.86.141:9866], excludes=[172.31.87.40:9866, 172.31.92.246:9866], numAdditionalNodes=1, clientName=DFSClient_NONMAPREDUCE_-958881537_1
2024-12-05 15:56:13,967 DEBUG namenode.NameNode: getAdditionalDatanode: src=/tmp/testfile-03ed9b5c-e1f4-42ad-bca3-8a579ac9d8b2, fileId=16403, blk=BP-571135626-172.31.88.138-1733413798130:blk_1073741825_1002, existings=[172.31.93.89:9866, 172.31.86.141:9866], excludes=[172.31.87.40:9866, 172.31.92.246:9866, 172.31.95.50:9866], numAdditionalNodes=1, clientName=DFSClient_NONMAPREDUCE_-958881537_1
2024-12-05 15:57:33,493 DEBUG blockmanagement.BlockManager: blocks = [blk_1073741825_1002]

Test#5 Detailed Results , After this Change

The following are the HDFS client logs for the duration of the test:

2024-12-05 16:25:41 Created File /tmp/testfile-081431ba-2feb-4e0a-b23f-a3d3a73ed00e
2024-12-05 16:25:42 Append Success  100 KB
2024-12-05 16:26:02 Append Success  200 KB
2024-12-05 16:26:22 Append Success  300 KB
2024-12-05 16:26:42 Append Success  400 KB
2024-12-05 16:27:02 Append Success  500 KB
2024-12-05 16:27:22 Append Success  600 KB
2024-12-05 16:27:42 Closed File
2024-12-05 16:27:42 Opened File /tmp/testfile-081431ba-2feb-4e0a-b23f-a3d3a73ed00e
2024-12-05 16:27:42 Append Success  700 KB
2024-12-05 16:28:02 Append Success  800 KB
2024-12-05 16:28:22 Append Success  900 KB
2024-12-05 16:28:42 Append Success  1000 KB
2024-12-05 16:29:02 Append Success  1100 KB
2024-12-05 16:29:22 Append Success  1200 KB
2024-12-05 16:29:42 Append Success  1300 KB
2024-12-05 16:30:02 Append Success  1400 KB
2024-12-05 16:30:22 Append Success  1500 KB
2024-12-05 16:30:42 Append Success  1600 KB
2024-12-05 16:31:02 Append Success  1700 KB
2024-12-05 16:31:22 Append Success  1800 KB
2024-12-05 16:31:42 Append Success  1900 KB
2024-12-05 16:32:02 Append Success  2000 KB
2024-12-05 16:32:22 Append Success  2100 KB
2024-12-05 16:32:42 Append Success  2200 KB
2024-12-05 16:33:02 Append Success  2300 KB
2024-12-05 16:33:22 Append Success  2400 KB
2024-12-05 16:33:42 Append Success  2500 KB
2024-12-05 16:34:02 Append Success  2600 KB
2024-12-05 16:34:22 Append Success  2700 KB
2024-12-05 16:34:42 Append Success  2800 KB
2024-12-05 16:35:02 Append Success  2900 KB
2024-12-05 16:35:22 Append Success  3000 KB
2024-12-05 16:35:42 Append Success  3100 KB
2024-12-05 16:36:02 Append Success  3200 KB
2024-12-05 16:36:22 Append Success  3300 KB
2024-12-05 16:36:42 Append Success  3400 KB
2024-12-05 16:37:02 Append Success  3500 KB
2024-12-05 16:37:22 Append Success  3600 KB
2024-12-05 16:37:42 Append Success  3700 KB
2024-12-05 16:38:02 Append Success  3800 KB
2024-12-05 16:38:22 Append Success  3900 KB
2024-12-05 16:38:42 Append Success  4000 KB
2024-12-05 16:39:02 Append Success  4100 KB
2024-12-05 16:39:22 Append Success  4200 KB
2024-12-05 16:39:42 Append Success  4300 KB
2024-12-05 16:40:02 Append Success  4400 KB
2024-12-05 16:40:22 Append Success  4500 KB
2024-12-05 16:40:42 Append Success  4600 KB
2024-12-05 16:41:02 Append Success  4700 KB
2024-12-05 16:41:22 Append Success  4800 KB
2024-12-05 16:41:42 Append Success  4900 KB
2024-12-05 16:42:02 Append Success  5000 KB
2024-12-05 16:42:22 Append Success  5100 KB
2024-12-05 16:42:43 Closed File

First step is to identify the block locations:

> cat /var/log/hadoop-hdfs/hadoop-hdfs-namenode-ip-*.out | grep testfile | grep blk
...
2024-12-05 16:25:41,999 INFO hdfs.StateChange: BLOCK* allocate blk_1073741825_1001, replicas=172.31.81.166:9866, 172.31.84.85:9866, 172.31.82.31:9866 for /tmp/testfile-081431ba-2feb-4e0a-b23f-a3d3a73ed00e
...

Then we wait for the block to be closed & re-opened. Once this is completed, we can start decommissioning the 3 datanodes with under construction block replicas:

> cat dfs.hosts.exclude
ip-172-31-82-31.ec2.internal
ip-172-31-84-85.ec2.internal
ip-172-31-81-166.ec2.internal

> export HADOOP_USER_NAME=hdfs

> hdfs dfsadmin -refreshNodes
...
2024-12-05 16:28:21,849 DEBUG ipc.ProtobufRpcEngine2: Call: refreshNodes took 98ms
Refresh nodes successful
...

Check the decommissioning status of the datanodes to confirm they are not decommissioned. Wait for the DFSOutputStream to be closed & then confirm decommissioning finishes after this time:

> cat /var/log/hadoop-hdfs/hadoop-hdfs-namenode-ip-*.out | grep DatanodeAdmin
...
2024-12-05 16:26:55,256 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: DatanodeAdminMonitor is running.
2024-12-05 16:27:25,257 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: DatanodeAdminMonitor is running.
2024-12-05 16:27:55,257 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: DatanodeAdminMonitor is running.
2024-12-05 16:28:21,844 INFO blockmanagement.DatanodeAdminManager: Starting decommission of 172.31.82.31:9866 [DISK]DS-9dd3db74-f327-4b44-bc81-c611dbe144ec:NORMAL:172.31.82.31:9866 with 0 blocks
2024-12-05 16:28:21,844 INFO blockmanagement.DatanodeAdminManager: Starting decommission of 172.31.82.31:9866 [DISK]DS-38366739-8eef-4345-bd6a-184eb3ab8b3c:NORMAL:172.31.82.31:9866 with 0 blocks
2024-12-05 16:28:21,844 INFO blockmanagement.DatanodeAdminManager: Starting decommission of 172.31.84.85:9866 [DISK]DS-8bc5aaf9-db5d-4049-be89-67f35bb038d5:NORMAL:172.31.84.85:9866 with 0 blocks
2024-12-05 16:28:21,844 INFO blockmanagement.DatanodeAdminManager: Starting decommission of 172.31.84.85:9866 [DISK]DS-7be3f1ef-23f5-4d6c-9a60-7f8a1aa299c6:NORMAL:172.31.84.85:9866 with 0 blocks
2024-12-05 16:28:21,844 INFO blockmanagement.DatanodeAdminManager: Starting decommission of 172.31.81.166:9866 [DISK]DS-f5533115-a135-4db0-9086-c4e9243c719e:NORMAL:172.31.81.166:9866 with 0 blocks
2024-12-05 16:28:21,844 INFO blockmanagement.DatanodeAdminManager: Starting decommission of 172.31.81.166:9866 [DISK]DS-57c2a4cc-b235-4508-8ac0-f836f1485e5d:NORMAL:172.31.81.166:9866 with 0 blocks
2024-12-05 16:28:25,258 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: DatanodeAdminMonitor is running.
2024-12-05 16:28:25,259 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: Newly-added node 172.31.81.166:9866, doing full scan to find insufficiently-replicated blocks.
2024-12-05 16:28:25,260 INFO blockmanagement.DatanodeAdminDefaultMonitor: Cannot decommission datanode 172.31.81.166:9866 with 1 UC blocks: [blk_1073741825_1002]
2024-12-05 16:28:25,260 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: Newly-added node 172.31.82.31:9866, doing full scan to find insufficiently-replicated blocks.
2024-12-05 16:28:25,261 INFO blockmanagement.DatanodeAdminDefaultMonitor: Cannot decommission datanode 172.31.82.31:9866 with 1 UC blocks: [blk_1073741825_1002]
2024-12-05 16:28:25,261 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: Newly-added node 172.31.84.85:9866, doing full scan to find insufficiently-replicated blocks.
2024-12-05 16:28:25,261 INFO blockmanagement.DatanodeAdminDefaultMonitor: Cannot decommission datanode 172.31.84.85:9866 with 1 UC blocks: [blk_1073741825_1002]
2024-12-05 16:28:25,261 INFO blockmanagement.DatanodeAdminDefaultMonitor: Checked 0 blocks and 3 nodes this tick. 3 nodes are now in maintenance or transitioning state. 0 nodes pending.
2024-12-05 16:28:55,261 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: DatanodeAdminMonitor is running.
2024-12-05 16:28:55,261 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: Processing Decommission In Progress node 172.31.81.166:9866
2024-12-05 16:28:55,269 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: Node 172.31.81.166:9866 has finished replicating current set of blocks, checking with the full block map.
2024-12-05 16:28:55,269 INFO blockmanagement.DatanodeAdminDefaultMonitor: Cannot decommission datanode 172.31.81.166:9866 with 1 UC blocks: [blk_1073741825_1002]
2024-12-05 16:28:55,269 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: Processing Decommission In Progress node 172.31.82.31:9866
2024-12-05 16:28:55,269 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: Node 172.31.82.31:9866 has finished replicating current set of blocks, checking with the full block map.
2024-12-05 16:28:55,269 INFO blockmanagement.DatanodeAdminDefaultMonitor: Cannot decommission datanode 172.31.82.31:9866 with 1 UC blocks: [blk_1073741825_1002]
2024-12-05 16:28:55,269 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: Processing Decommission In Progress node 172.31.84.85:9866
2024-12-05 16:28:55,270 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: Node 172.31.84.85:9866 has finished replicating current set of blocks, checking with the full block map.
2024-12-05 16:28:55,270 INFO blockmanagement.DatanodeAdminDefaultMonitor: Cannot decommission datanode 172.31.84.85:9866 with 1 UC blocks: [blk_1073741825_1002]
2024-12-05 16:28:55,270 INFO blockmanagement.DatanodeAdminDefaultMonitor: Checked 0 blocks and 3 nodes this tick. 3 nodes are now in maintenance or transitioning state. 0 nodes pending.
2024-12-05 16:29:25,270 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: DatanodeAdminMonitor is running.
2024-12-05 16:29:25,270 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: Processing Decommission In Progress node 172.31.81.166:9866
2024-12-05 16:29:25,270 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: Node 172.31.81.166:9866 has finished replicating current set of blocks, checking with the full block map.
2024-12-05 16:29:25,270 INFO blockmanagement.DatanodeAdminDefaultMonitor: Cannot decommission datanode 172.31.81.166:9866 with 1 UC blocks: [blk_1073741825_1002]
2024-12-05 16:29:25,270 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: Processing Decommission In Progress node 172.31.82.31:9866
2024-12-05 16:29:25,270 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: Node 172.31.82.31:9866 has finished replicating current set of blocks, checking with the full block map.
2024-12-05 16:29:25,270 INFO blockmanagement.DatanodeAdminDefaultMonitor: Cannot decommission datanode 172.31.82.31:9866 with 1 UC blocks: [blk_1073741825_1002]
2024-12-05 16:29:25,270 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: Processing Decommission In Progress node 172.31.84.85:9866
2024-12-05 16:29:25,270 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: Node 172.31.84.85:9866 has finished replicating current set of blocks, checking with the full block map.
2024-12-05 16:29:25,270 INFO blockmanagement.DatanodeAdminDefaultMonitor: Cannot decommission datanode 172.31.84.85:9866 with 1 UC blocks: [blk_1073741825_1002]
2024-12-05 16:29:25,270 INFO blockmanagement.DatanodeAdminDefaultMonitor: Checked 0 blocks and 3 nodes this tick. 3 nodes are now in maintenance or transitioning state. 0 nodes pending.
...
2024-12-05 16:42:25,289 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: DatanodeAdminMonitor is running.
2024-12-05 16:42:25,289 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: Processing Decommission In Progress node 172.31.81.166:9866
2024-12-05 16:42:25,289 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: Node 172.31.81.166:9866 has finished replicating current set of blocks, checking with the full block map.
2024-12-05 16:42:25,289 INFO blockmanagement.DatanodeAdminDefaultMonitor: Cannot decommission datanode 172.31.81.166:9866 with 1 UC blocks: [blk_1073741825_1002]
2024-12-05 16:42:25,289 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: Processing Decommission In Progress node 172.31.82.31:9866
2024-12-05 16:42:25,289 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: Node 172.31.82.31:9866 has finished replicating current set of blocks, checking with the full block map.
2024-12-05 16:42:25,289 INFO blockmanagement.DatanodeAdminDefaultMonitor: Cannot decommission datanode 172.31.82.31:9866 with 1 UC blocks: [blk_1073741825_1002]
2024-12-05 16:42:25,289 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: Processing Decommission In Progress node 172.31.84.85:9866
2024-12-05 16:42:25,289 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: Node 172.31.84.85:9866 has finished replicating current set of blocks, checking with the full block map.
2024-12-05 16:42:25,289 INFO blockmanagement.DatanodeAdminDefaultMonitor: Cannot decommission datanode 172.31.84.85:9866 with 1 UC blocks: [blk_1073741825_1002]
2024-12-05 16:42:25,289 INFO blockmanagement.DatanodeAdminDefaultMonitor: Checked 0 blocks and 3 nodes this tick. 3 nodes are now in maintenance or transitioning state. 0 nodes pending.
2024-12-05 16:42:55,289 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: DatanodeAdminMonitor is running.
2024-12-05 16:42:55,290 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: Processing Decommission In Progress node 172.31.81.166:9866
2024-12-05 16:42:55,290 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: Node 172.31.81.166:9866 has finished replicating current set of blocks, checking with the full block map.
2024-12-05 16:42:55,290 INFO blockmanagement.DatanodeAdminManager: Decommissioning complete for node 172.31.81.166:9866
2024-12-05 16:42:55,290 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: Node 172.31.81.166:9866 is sufficiently replicated and healthy, marked as Decommissioned.
2024-12-05 16:42:55,290 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: Processing Decommission In Progress node 172.31.82.31:9866
2024-12-05 16:42:55,290 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: Node 172.31.82.31:9866 has finished replicating current set of blocks, checking with the full block map.
2024-12-05 16:42:55,291 INFO blockmanagement.DatanodeAdminManager: Decommissioning complete for node 172.31.82.31:9866
2024-12-05 16:42:55,291 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: Node 172.31.82.31:9866 is sufficiently replicated and healthy, marked as Decommissioned.
2024-12-05 16:42:55,291 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: Processing Decommission In Progress node 172.31.84.85:9866
2024-12-05 16:42:55,291 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: Node 172.31.84.85:9866 has finished replicating current set of blocks, checking with the full block map.
2024-12-05 16:42:55,291 INFO blockmanagement.DatanodeAdminManager: Decommissioning complete for node 172.31.84.85:9866
2024-12-05 16:42:55,291 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: Node 172.31.84.85:9866 is sufficiently replicated and healthy, marked as Decommissioned.
2024-12-05 16:42:55,291 INFO blockmanagement.DatanodeAdminDefaultMonitor: Checked 3 blocks and 3 nodes this tick. 0 nodes are now in maintenance or transitioning state. 0 nodes pending.
2024-12-05 16:43:25,291 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: DatanodeAdminMonitor is running.
2024-12-05 16:43:55,291 DEBUG blockmanagement.DatanodeAdminDefaultMonitor: DatanodeAdminMonitor is running.
...
> hdfs dfsadmin -report | grep -i decommissioned -B 3
...
2024-12-05 16:45:07,815 DEBUG ipc.ProtobufRpcEngine2: Call: getDatanodeReport took 3ms

Name: 172.31.81.166:9866 (ip-172-31-81-166.ec2.internal)
Hostname: ip-172-31-81-166.ec2.internal
Decommission Status : Decommissioned
--

Name: 172.31.82.31:9866 (ip-172-31-82-31.ec2.internal)
Hostname: ip-172-31-82-31.ec2.internal
Decommission Status : Decommissioned
--

Name: 172.31.84.85:9866 (ip-172-31-84-85.ec2.internal)
Hostname: ip-172-31-84-85.ec2.internal
Decommission Status : Decommissioned

Then we stop/terminate all 3 decommissioned datanodes at the same time, this is expected to be safe since they are in decommissioned state:

> date
Thu Dec  5 16:46:01 UTC 2024

There is no HDFS data loss because the block replicas are moved to other live datanodes before decommissioning finishes:

> hdfs dfs -get /tmp/testfile-081431ba-2feb-4e0a-b23f-a3d3a73ed00e /var/testfile
...
2024-12-05 16:47:25,377 DEBUG ipc.Client: Stopping client
> ll -h /var/testfile
-rw-r--r--. 1 root root 5.0M Dec  5 16:47 /var/testfile

Introspecting further into the logs, we see that:

  • when the block is opened in RBW state, it gets added to UnderConstructionBlocks
  • when the block is transitioned to FINALIZED state, it gets removed from UnderConstructionBlocks (and added to StorageInfos)
> cat /var/log/hadoop-hdfs/hadoop-hdfs-namenode-ip-*.out | grep 'UnderConstructionBlocks\|Under Construction'

2024-12-05 16:25:24,504 INFO blockmanagement.UnderConstructionBlocks: Tracking Under Construction blocks for DatanodeAdminManager
2024-12-05 16:25:43,602 DEBUG blockmanagement.UnderConstructionBlocks: Add UC block blk_1073741825_1001 to 172.31.84.85:9866, new total is [replicas=1 / blocks=1]
2024-12-05 16:25:43,642 DEBUG blockmanagement.UnderConstructionBlocks: Add UC block blk_1073741825_1001 to 172.31.81.166:9866, new total is [replicas=2 / blocks=1]
2024-12-05 16:25:44,514 DEBUG blockmanagement.UnderConstructionBlocks: Add UC block blk_1073741825_1001 to 172.31.82.31:9866, new total is [replicas=3 / blocks=1]
2024-12-05 16:25:55,256 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{172.31.82.31:9866=1,172.31.84.85:9866=1,172.31.81.166:9866=1}]
2024-12-05 16:26:25,256 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{172.31.82.31:9866=1,172.31.84.85:9866=1,172.31.81.166:9866=1}]
2024-12-05 16:26:55,257 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{172.31.82.31:9866=1,172.31.84.85:9866=1,172.31.81.166:9866=1}]
2024-12-05 16:27:25,257 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{172.31.82.31:9866=1,172.31.84.85:9866=1,172.31.81.166:9866=1}]
2024-12-05 16:27:42,659 DEBUG blockmanagement.UnderConstructionBlocks: Removed UC block [blk_1073741825_1001->blk_1073741825_1001] from 172.31.82.31:9866, new total is [replicas=2 / blocks=1]
2024-12-05 16:27:42,661 DEBUG blockmanagement.UnderConstructionBlocks: Removed UC block [blk_1073741825_1001->blk_1073741825_1001] from 172.31.84.85:9866, new total is [replicas=1 / blocks=1]
2024-12-05 16:27:42,665 DEBUG blockmanagement.UnderConstructionBlocks: Removed UC block [blk_1073741825_1001->blk_1073741825_1001] from 172.31.81.166:9866, new total is [replicas=0 / blocks=0]
2024-12-05 16:27:42,753 DEBUG blockmanagement.UnderConstructionBlocks: UC block blk_1073741825_1002 not found on 172.31.82.31:9866, total is [replicas=0 / blocks=0]2024-12-05 16:27:42,754 DEBUG blockmanagement.UnderConstructionBlocks: UC block blk_1073741825_1002 not found on 172.31.84.85:9866, total is [replicas=0 / blocks=0]
2024-12-05 16:27:42,754 DEBUG blockmanagement.UnderConstructionBlocks: UC block blk_1073741825_1002 not found on 172.31.81.166:9866, total is [replicas=0 / blocks=0]
2024-12-05 16:27:43,586 DEBUG blockmanagement.UnderConstructionBlocks: Add UC block blk_1073741825_1002 to 172.31.84.85:9866, new total is [replicas=1 / blocks=1]
2024-12-05 16:27:43,641 DEBUG blockmanagement.UnderConstructionBlocks: Add UC block blk_1073741825_1002 to 172.31.81.166:9866, new total is [replicas=2 / blocks=1]
2024-12-05 16:27:44,515 DEBUG blockmanagement.UnderConstructionBlocks: Add UC block blk_1073741825_1002 to 172.31.82.31:9866, new total is [replicas=3 / blocks=1]2024-12-05 16:27:55,257 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{172.31.82.31:9866=1,172.31.84.85:9866=1,172.31.81.166:9866=1}]
2024-12-05 16:28:25,259 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{172.31.82.31:9866=1,172.31.84.85:9866=1,172.31.81.166:9866=1}]
2024-12-05 16:28:55,261 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{172.31.82.31:9866=1,172.31.84.85:9866=1,172.31.81.166:9866=1}]
2024-12-05 16:29:25,270 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{172.31.82.31:9866=1,172.31.84.85:9866=1,172.31.81.166:9866=1}]
2024-12-05 16:29:55,271 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{172.31.82.31:9866=1,172.31.84.85:9866=1,172.31.81.166:9866=1}]
2024-12-05 16:30:25,272 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{172.31.82.31:9866=1,172.31.84.85:9866=1,172.31.81.166:9866=1}]
2024-12-05 16:30:55,273 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{172.31.82.31:9866=1,172.31.84.85:9866=1,172.31.81.166:9866=1}]
2024-12-05 16:31:25,274 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{172.31.82.31:9866=1,172.31.84.85:9866=1,172.31.81.166:9866=1}]
2024-12-05 16:31:55,274 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{172.31.82.31:9866=1,172.31.84.85:9866=1,172.31.81.166:9866=1}]
2024-12-05 16:32:25,275 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{172.31.82.31:9866=1,172.31.84.85:9866=1,172.31.81.166:9866=1}]
2024-12-05 16:32:55,276 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{172.31.82.31:9866=1,172.31.84.85:9866=1,172.31.81.166:9866=1}]
2024-12-05 16:33:25,276 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{172.31.82.31:9866=1,172.31.84.85:9866=1,172.31.81.166:9866=1}]
2024-12-05 16:33:55,277 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{172.31.82.31:9866=1,172.31.84.85:9866=1,172.31.81.166:9866=1}]2024-12-05 16:34:25,278 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{172.31.82.31:9866=1,172.31.84.85:9866=1,172.31.81.166:9866=1}]
2024-12-05 16:34:55,279 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{172.31.82.31:9866=1,172.31.84.85:9866=1,172.31.81.166:9866=1}]
2024-12-05 16:35:25,280 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{172.31.82.31:9866=1,172.31.84.85:9866=1,172.31.81.166:9866=1}]
2024-12-05 16:35:55,280 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{172.31.82.31:9866=1,172.31.84.85:9866=1,172.31.81.166:9866=1}]
2024-12-05 16:36:25,281 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{172.31.82.31:9866=1,172.31.84.85:9866=1,172.31.81.166:9866=1}]2024-12-05 16:36:55,282 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{172.31.82.31:9866=1,172.31.84.85:9866=1,172.31.81.166:9866=1}]2024-12-05 16:37:25,282 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{172.31.82.31:9866=1,172.31.84.85:9866=1,172.31.81.166:9866=1}]2024-12-05 16:37:55,283 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{172.31.82.31:9866=1,172.31.84.85:9866=1,172.31.81.166:9866=1}]2024-12-05 16:38:25,284 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{172.31.82.31:9866=1,172.31.84.85:9866=1,172.31.81.166:9866=1}]2024-12-05 16:38:55,284 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{172.31.82.31:9866=1,172.31.84.85:9866=1,172.31.81.166:9866=1}]
2024-12-05 16:39:25,285 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{172.31.82.31:9866=1,172.31.84.85:9866=1,172.31.81.166:9866=1}]
2024-12-05 16:39:55,286 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{172.31.82.31:9866=1,172.31.84.85:9866=1,172.31.81.166:9866=1}]2024-12-05 16:40:25,286 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{172.31.82.31:9866=1,172.31.84.85:9866=1,172.31.81.166:9866=1}]2024-12-05 16:40:55,287 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{172.31.82.31:9866=1,172.31.84.85:9866=1,172.31.81.166:9866=1}]2024-12-05 16:41:25,288 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{172.31.82.31:9866=1,172.31.84.85:9866=1,172.31.81.166:9866=1}]2024-12-05 16:41:55,288 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{172.31.82.31:9866=1,172.31.84.85:9866=1,172.31.81.166:9866=1}]2024-12-05 16:42:25,289 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{172.31.82.31:9866=1,172.31.84.85:9866=1,172.31.81.166:9866=1}]
2024-12-05 16:42:42,995 DEBUG blockmanagement.UnderConstructionBlocks: Removed UC block [blk_1073741825_1002->blk_1073741825_1002] from 172.31.81.166:9866, new total is [replicas=2 / blocks=1]
2024-12-05 16:42:42,996 DEBUG blockmanagement.UnderConstructionBlocks: Removed UC block [blk_1073741825_1002->blk_1073741825_1002] from 172.31.84.85:9866, new total is [replicas=1 / blocks=1]
2024-12-05 16:42:42,997 DEBUG blockmanagement.UnderConstructionBlocks: Removed UC block [blk_1073741825_1002->blk_1073741825_1002] from 172.31.82.31:9866, new total is [replicas=0 / blocks=0]2024-12-05 16:42:44,879 DEBUG blockmanagement.UnderConstructionBlocks: UC block blk_1073741825_1002 not found on 172.31.87.161:9866, total is [replicas=0 / blocks=0]
2024-12-05 16:42:44,935 DEBUG blockmanagement.UnderConstructionBlocks: UC block blk_1073741825_1002 not found on 172.31.84.145:9866, total is [replicas=0 / blocks=0]
2024-12-05 16:42:44,935 DEBUG blockmanagement.UnderConstructionBlocks: UC block blk_1073741825_1002 not found on 172.31.93.40:9866, total is [replicas=0 / blocks=0]
2024-12-05 16:42:55,290 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{}]
2024-12-05 16:43:25,291 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{}]
2024-12-05 16:43:55,291 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{}]
2024-12-05 16:44:25,291 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{}]
2024-12-05 16:44:55,292 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{}]
2024-12-05 16:45:25,292 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{}]
2024-12-05 16:45:55,292 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{}]
2024-12-05 16:46:25,292 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{}]
2024-12-05 16:46:55,293 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{}]
2024-12-05 16:47:25,293 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{}]
2024-12-05 16:47:55,293 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{}]
2024-12-05 16:48:25,293 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{}]
2024-12-05 16:48:55,294 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{}]
2024-12-05 16:49:25,294 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{}]
2024-12-05 16:49:55,294 DEBUG blockmanagement.BlockManager: Under Construction block counts: [{}]

For code changes:

  • Does the title or this PR starts with the corresponding JIRA issue id (e.g. 'HADOOP-17799. Your PR title ...')?
  • Object storage: have the integration tests been executed and the endpoint declared according to the connector-specific documentation?
  • [n/a] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under ASF 2.0?
  • [n/a] If applicable, have you updated the LICENSE, LICENSE-binary, NOTICE-binary files?

…ion blocks are sufficiently replicated which causes HDFS Data Loss
@hadoop-yetus
Copy link

💔 -1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 17m 35s Docker mode activated.
_ Prechecks _
+1 💚 dupname 0m 0s No case conflicting files found.
+0 🆗 codespell 0m 1s codespell was not available.
+0 🆗 detsecrets 0m 1s detect-secrets was not available.
+1 💚 @author 0m 0s The patch does not contain any @author tags.
+1 💚 test4tests 0m 0s The patch appears to include 2 new or modified test files.
_ trunk Compile Tests _
+1 💚 mvninstall 49m 41s trunk passed
+1 💚 compile 1m 28s trunk passed with JDK Ubuntu-11.0.25+9-post-Ubuntu-1ubuntu120.04
+1 💚 compile 1m 16s trunk passed with JDK Private Build-1.8.0_432-8u432-gaus1-0ubuntu220.04-ga
+1 💚 checkstyle 1m 17s trunk passed
+1 💚 mvnsite 1m 27s trunk passed
+1 💚 javadoc 1m 17s trunk passed with JDK Ubuntu-11.0.25+9-post-Ubuntu-1ubuntu120.04
+1 💚 javadoc 1m 47s trunk passed with JDK Private Build-1.8.0_432-8u432-gaus1-0ubuntu220.04-ga
+1 💚 spotbugs 3m 24s trunk passed
+1 💚 shadedclient 41m 59s branch has no errors when building and testing our client artifacts.
_ Patch Compile Tests _
+1 💚 mvninstall 1m 14s the patch passed
+1 💚 compile 1m 18s the patch passed with JDK Ubuntu-11.0.25+9-post-Ubuntu-1ubuntu120.04
+1 💚 javac 1m 18s the patch passed
+1 💚 compile 1m 9s the patch passed with JDK Private Build-1.8.0_432-8u432-gaus1-0ubuntu220.04-ga
+1 💚 javac 1m 9s the patch passed
+1 💚 blanks 0m 0s The patch has no blanks issues.
+1 💚 checkstyle 1m 5s the patch passed
+1 💚 mvnsite 1m 17s the patch passed
-1 ❌ javadoc 1m 4s /patch-javadoc-hadoop-hdfs-project_hadoop-hdfs-jdkUbuntu-11.0.25+9-post-Ubuntu-1ubuntu120.04.txt hadoop-hdfs in the patch failed with JDK Ubuntu-11.0.25+9-post-Ubuntu-1ubuntu120.04.
+1 💚 javadoc 1m 41s the patch passed with JDK Private Build-1.8.0_432-8u432-gaus1-0ubuntu220.04-ga
-1 ❌ spotbugs 3m 22s /new-spotbugs-hadoop-hdfs-project_hadoop-hdfs.html hadoop-hdfs-project/hadoop-hdfs generated 3 new + 0 unchanged - 0 fixed = 3 total (was 0)
+1 💚 shadedclient 42m 26s patch has no errors when building and testing our client artifacts.
_ Other Tests _
-1 ❌ unit 255m 49s /patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt hadoop-hdfs in the patch passed.
+1 💚 asflicense 0m 51s The patch does not generate ASF License warnings.
430m 34s
Reason Tests
SpotBugs module:hadoop-hdfs-project/hadoop-hdfs
Load of known null value in org.apache.hadoop.hdfs.server.blockmanagement.UnderConstructionBlocks.removeAllUcBlocksForDatanode(DatanodeDescriptor) At UnderConstructionBlocks.java:in org.apache.hadoop.hdfs.server.blockmanagement.UnderConstructionBlocks.removeAllUcBlocksForDatanode(DatanodeDescriptor) At UnderConstructionBlocks.java:[line 198]
Possible null pointer dereference of reportedBlock in org.apache.hadoop.hdfs.server.blockmanagement.UnderConstructionBlocks.addUcBlock(DatanodeDescriptor, Block) Dereferenced at UnderConstructionBlocks.java:reportedBlock in org.apache.hadoop.hdfs.server.blockmanagement.UnderConstructionBlocks.addUcBlock(DatanodeDescriptor, Block) Dereferenced at UnderConstructionBlocks.java:[line 242]
Possible null pointer dereference of reportedBlock in org.apache.hadoop.hdfs.server.blockmanagement.UnderConstructionBlocks.removeUcBlock(DatanodeDescriptor, Block) Dereferenced at UnderConstructionBlocks.java:reportedBlock in org.apache.hadoop.hdfs.server.blockmanagement.UnderConstructionBlocks.removeUcBlock(DatanodeDescriptor, Block) Dereferenced at UnderConstructionBlocks.java:[line 137]
Failed junit tests hadoop.tools.TestHdfsConfigFields
Subsystem Report/Notes
Docker ClientAPI=1.47 ServerAPI=1.47 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-7179/1/artifact/out/Dockerfile
GITHUB PR #7179
Optional Tests dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell detsecrets
uname Linux b4b19a546e9f 5.15.0-124-generic #134-Ubuntu SMP Fri Sep 27 20:20:17 UTC 2024 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision trunk / e0193a2
Default Java Private Build-1.8.0_432-8u432-gaus1-0ubuntu220.04-ga
Multi-JDK versions /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.25+9-post-Ubuntu-1ubuntu120.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_432-8u432-gaus1-0ubuntu220.04-ga
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-7179/1/testReport/
Max. process+thread count 3051 (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-7179/1/console
versions git=2.25.1 maven=3.6.3 spotbugs=4.2.2
Powered by Apache Yetus 0.14.0 https://yetus.apache.org

This message was automatically generated.

@KevinWikant
Copy link
Contributor Author

KevinWikant commented Dec 4, 2024

Javadoc Failures are due to usage of &:

[ERROR] /home/jenkins/jenkins-home/workspace/hadoop-multibranch_PR-7179/ubuntu-focal/src/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderConstructionBlocks.java:40: error: bad HTML entity
[ERROR]  * the block is fully committed & finalized.
[ERROR]                                 ^

[ERROR] /home/jenkins/jenkins-home/workspace/hadoop-multibranch_PR-7179/ubuntu-focal/src/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java:1755: error: bad HTML entity
[ERROR]    * to other live datanodes. This can lead to HDFS write failures & data loss,
[ERROR]                                                                    ^

[ERROR] /home/jenkins/jenkins-home/workspace/hadoop-multibranch_PR-7179/ubuntu-focal/src/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java:1757: error: bad HTML entity
[ERROR]    * & terminated at around the same time.
[ERROR]      ^

[ERROR] /home/jenkins/jenkins-home/workspace/hadoop-multibranch_PR-7179/ubuntu-focal/src/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java:1759: error: bad HTML entity
[ERROR]    * Enable the following setting to have the Namenode track & consider

SpotBugs Failures:

Bug type NP_NULL_ON_SOME_PATH (click for details)
In class org.apache.hadoop.hdfs.server.blockmanagement.UnderConstructionBlocks
In method org.apache.hadoop.hdfs.server.blockmanagement.UnderConstructionBlocks.addUcBlock(DatanodeDescriptor, Block)
Value loaded from reportedBlock
Dereferenced at UnderConstructionBlocks.java:[line 242]
Known null at UnderConstructionBlocks.java:[line 237]
Bug type NP_NULL_ON_SOME_PATH (click for details)
In class org.apache.hadoop.hdfs.server.blockmanagement.UnderConstructionBlocks
In method org.apache.hadoop.hdfs.server.blockmanagement.UnderConstructionBlocks.removeUcBlock(DatanodeDescriptor, Block)
Value loaded from reportedBlock
Dereferenced at UnderConstructionBlocks.java:[line 137]
Known null at UnderConstructionBlocks.java:[line 132]
Bug type NP_LOAD_OF_KNOWN_NULL_VALUE (click for details)
In class org.apache.hadoop.hdfs.server.blockmanagement.UnderConstructionBlocks
In method org.apache.hadoop.hdfs.server.blockmanagement.UnderConstructionBlocks.removeAllUcBlocksForDatanode(DatanodeDescriptor)
Value loaded from reportingNode
At UnderConstructionBlocks.java:[line 198]

Unit Test Report:

[ERROR] Tests run: 7139, Failures: 1, Errors: 0, Skipped: 26, Flakes: 4
[INFO] 
[ERROR] There are test failures.
[INFO] Results:
[INFO] 
[ERROR] Failures: 
[ERROR] org.apache.hadoop.tools.TestHdfsConfigFields.testCompareConfigurationClassAgainstXml(org.apache.hadoop.tools.TestHdfsConfigFields)
[ERROR]   Run 1: TestHdfsConfigFields>TestConfigurationFieldsBase.testCompareConfigurationClassAgainstXml:480 interface org.apache.hadoop.hdfs.client.HdfsClientConfigKeys interface org.apache.hadoop.hdfs.client.HdfsClientConfigKeys$Failover interface org.apache.hadoop.hdfs.client.HdfsClientConfigKeys$StripedRead class org.apache.hadoop.hdfs.DFSConfigKeys interface org.apache.hadoop.hdfs.client.HdfsClientConfigKeys$BlockWrite interface org.apache.hadoop.hdfs.client.HdfsClientConfigKeys$Write interface org.apache.hadoop.hdfs.client.HdfsClientConfigKeys$Read interface org.apache.hadoop.hdfs.client.HdfsClientConfigKeys$HedgedRead interface org.apache.hadoop.hdfs.client.HdfsClientConfigKeys$ShortCircuit interface org.apache.hadoop.hdfs.client.HdfsClientConfigKeys$Retry interface org.apache.hadoop.hdfs.client.HdfsClientConfigKeys$Mmap interface org.apache.hadoop.hdfs.client.HdfsClientConfigKeys$BlockWrite$ReplaceDatanodeOnFailure has 1 variables missing in hdfs-default.xml Entries:   dfs.namenode.decommission.track.underconstructionblocks expected:<0> but was:<1>
[ERROR]   Run 2: TestHdfsConfigFields>TestConfigurationFieldsBase.testCompareConfigurationClassAgainstXml:480 interface org.apache.hadoop.hdfs.client.HdfsClientConfigKeys interface org.apache.hadoop.hdfs.client.HdfsClientConfigKeys$Failover interface org.apache.hadoop.hdfs.client.HdfsClientConfigKeys$StripedRead class org.apache.hadoop.hdfs.DFSConfigKeys interface org.apache.hadoop.hdfs.client.HdfsClientConfigKeys$BlockWrite interface org.apache.hadoop.hdfs.client.HdfsClientConfigKeys$Write interface org.apache.hadoop.hdfs.client.HdfsClientConfigKeys$Read interface org.apache.hadoop.hdfs.client.HdfsClientConfigKeys$HedgedRead interface org.apache.hadoop.hdfs.client.HdfsClientConfigKeys$ShortCircuit interface org.apache.hadoop.hdfs.client.HdfsClientConfigKeys$Retry interface org.apache.hadoop.hdfs.client.HdfsClientConfigKeys$Mmap interface org.apache.hadoop.hdfs.client.HdfsClientConfigKeys$BlockWrite$ReplaceDatanodeOnFailure has 1 variables missing in hdfs-default.xml Entries:   dfs.namenode.decommission.track.underconstructionblocks expected:<0> but was:<1>
[ERROR]   Run 3: TestHdfsConfigFields>TestConfigurationFieldsBase.testCompareConfigurationClassAgainstXml:480 interface org.apache.hadoop.hdfs.client.HdfsClientConfigKeys interface org.apache.hadoop.hdfs.client.HdfsClientConfigKeys$Failover interface org.apache.hadoop.hdfs.client.HdfsClientConfigKeys$StripedRead class org.apache.hadoop.hdfs.DFSConfigKeys interface org.apache.hadoop.hdfs.client.HdfsClientConfigKeys$BlockWrite interface org.apache.hadoop.hdfs.client.HdfsClientConfigKeys$Write interface org.apache.hadoop.hdfs.client.HdfsClientConfigKeys$Read interface org.apache.hadoop.hdfs.client.HdfsClientConfigKeys$HedgedRead interface org.apache.hadoop.hdfs.client.HdfsClientConfigKeys$ShortCircuit interface org.apache.hadoop.hdfs.client.HdfsClientConfigKeys$Retry interface org.apache.hadoop.hdfs.client.HdfsClientConfigKeys$Mmap interface org.apache.hadoop.hdfs.client.HdfsClientConfigKeys$BlockWrite$ReplaceDatanodeOnFailure has 1 variables missing in hdfs-default.xml Entries:   dfs.namenode.decommission.track.underconstructionblocks expected:<0> but was:<1>
[INFO] 
[WARNING] Flakes: 
[WARNING] org.apache.hadoop.hdfs.TestReconstructStripedFileWithValidator.testValidatorWithBadDecoding(org.apache.hadoop.hdfs.TestReconstructStripedFileWithValidator)
[ERROR]   Run 1: TestReconstructStripedFileWithValidator.testValidatorWithBadDecoding:96 expected:<1> but was:<0>
[INFO]   Run 2: PASS
[INFO] 
[WARNING] org.apache.hadoop.hdfs.server.blockmanagement.TestUnderReplicatedBlocks.testSetRepIncWithUnderReplicatedBlocks(org.apache.hadoop.hdfs.server.blockmanagement.TestUnderReplicatedBlocks)
[ERROR]   Run 1: TestUnderReplicatedBlocks.testSetRepIncWithUnderReplicatedBlocks:80 » TestTimedOut
[INFO]   Run 2: PASS
[INFO] 
[WARNING] org.apache.hadoop.hdfs.server.namenode.TestEditLogRace.testDeadlock[0](org.apache.hadoop.hdfs.server.namenode.TestEditLogRace)
[ERROR]   Run 1: TestEditLogRace.testDeadlock:685 » Timeout Timed out waiting for condition. 
T...
[INFO]   Run 2: PASS
[INFO] 
[WARNING] org.apache.hadoop.hdfs.server.namenode.snapshot.TestFsShellMoveToTrashWithSnapshots.test100tasks20files(org.apache.hadoop.hdfs.server.namenode.snapshot.TestFsShellMoveToTrashWithSnapshots)
[ERROR]   Run 1: TestFsShellMoveToTrashWithSnapshots.test100tasks20files:259->runMultipleTasks:289->assertExists:388 Path not found: /user/jenkins/.Trash/Current/warehouse/sub/db24
runTestMoveToTrashWithShell(173)) -/sub2/sub1/dir0/dir1/dir2/dir3/bucket_0000
[INFO]   Run 2: PASS
[INFO] 
[INFO] 
[ERROR] Tests run: 7139, Failures: 1, Errors: 0, Skipped: 26, Flakes: 4

The testCompareConfigurationClassAgainstXml test failure is related, I suspect the test needs to be updated to account for the new DFSConfigKeys

The 4 flaky tests are unrelated to this change.

@hadoop-yetus
Copy link

💔 -1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 0m 51s Docker mode activated.
_ Prechecks _
+1 💚 dupname 0m 1s No case conflicting files found.
+0 🆗 codespell 0m 0s codespell was not available.
+0 🆗 detsecrets 0m 0s detect-secrets was not available.
+0 🆗 xmllint 0m 0s xmllint was not available.
+1 💚 @author 0m 0s The patch does not contain any @author tags.
+1 💚 test4tests 0m 0s The patch appears to include 2 new or modified test files.
_ trunk Compile Tests _
+1 💚 mvninstall 47m 49s trunk passed
+1 💚 compile 1m 22s trunk passed with JDK Ubuntu-11.0.25+9-post-Ubuntu-1ubuntu120.04
+1 💚 compile 1m 11s trunk passed with JDK Private Build-1.8.0_432-8u432-gaus1-0ubuntu220.04-ga
+1 💚 checkstyle 1m 14s trunk passed
+1 💚 mvnsite 1m 20s trunk passed
+1 💚 javadoc 1m 14s trunk passed with JDK Ubuntu-11.0.25+9-post-Ubuntu-1ubuntu120.04
+1 💚 javadoc 1m 41s trunk passed with JDK Private Build-1.8.0_432-8u432-gaus1-0ubuntu220.04-ga
+1 💚 spotbugs 3m 12s trunk passed
+1 💚 shadedclient 39m 29s branch has no errors when building and testing our client artifacts.
_ Patch Compile Tests _
+1 💚 mvninstall 1m 7s the patch passed
+1 💚 compile 1m 13s the patch passed with JDK Ubuntu-11.0.25+9-post-Ubuntu-1ubuntu120.04
+1 💚 javac 1m 13s the patch passed
+1 💚 compile 1m 4s the patch passed with JDK Private Build-1.8.0_432-8u432-gaus1-0ubuntu220.04-ga
+1 💚 javac 1m 4s the patch passed
+1 💚 blanks 0m 0s The patch has no blanks issues.
+1 💚 checkstyle 1m 4s the patch passed
+1 💚 mvnsite 1m 10s the patch passed
-1 ❌ javadoc 1m 2s /results-javadoc-javadoc-hadoop-hdfs-project_hadoop-hdfs-jdkUbuntu-11.0.25+9-post-Ubuntu-1ubuntu120.04.txt hadoop-hdfs-project_hadoop-hdfs-jdkUbuntu-11.0.25+9-post-Ubuntu-1ubuntu120.04 with JDK Ubuntu-11.0.25+9-post-Ubuntu-1ubuntu120.04 generated 2 new + 3117 unchanged - 0 fixed = 3119 total (was 3117)
+1 💚 javadoc 1m 34s the patch passed with JDK Private Build-1.8.0_432-8u432-gaus1-0ubuntu220.04-ga
-1 ❌ spotbugs 3m 14s /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 💚 shadedclient 39m 32s patch has no errors when building and testing our client artifacts.
_ Other Tests _
-1 ❌ unit 263m 25s /patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt hadoop-hdfs in the patch passed.
+1 💚 asflicense 0m 44s The patch does not generate ASF License warnings.
412m 40s
Reason Tests
SpotBugs module:hadoop-hdfs-project/hadoop-hdfs
Load of known null value in org.apache.hadoop.hdfs.server.blockmanagement.UnderConstructionBlocks.removeAllUcBlocksForDatanode(DatanodeDescriptor) At UnderConstructionBlocks.java:in org.apache.hadoop.hdfs.server.blockmanagement.UnderConstructionBlocks.removeAllUcBlocksForDatanode(DatanodeDescriptor) At UnderConstructionBlocks.java:[line 199]
Failed junit tests hadoop.hdfs.TestRollingUpgrade
Subsystem Report/Notes
Docker ClientAPI=1.47 ServerAPI=1.47 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-7179/2/artifact/out/Dockerfile
GITHUB PR #7179
Optional Tests dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell detsecrets xmllint
uname Linux 8f2ebee4a0af 5.15.0-124-generic #134-Ubuntu SMP Fri Sep 27 20:20:17 UTC 2024 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision trunk / e07907a
Default Java Private Build-1.8.0_432-8u432-gaus1-0ubuntu220.04-ga
Multi-JDK versions /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.25+9-post-Ubuntu-1ubuntu120.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_432-8u432-gaus1-0ubuntu220.04-ga
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-7179/2/testReport/
Max. process+thread count 2652 (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-7179/2/console
versions git=2.25.1 maven=3.6.3 spotbugs=4.2.2
Powered by Apache Yetus 0.14.0 https://yetus.apache.org

This message was automatically generated.

ucBlocksByDatanode.getOrDefault(dn, Collections.emptyList());
final String ucBlocksString =
ucBlocks.stream().map(Object::toString).collect(Collectors.joining(","));
LOG.info("Cannot decommission datanode {} with {} UC blocks: [{}]",
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Will this info log become noisy if the client takes time to close the stream ?

Copy link
Contributor Author

@KevinWikant KevinWikant Dec 19, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This log will be printed at most once per datanode per DatanodeAdminDefaultMonitor cycle. This means in a 1k datanode HDFS cluster, there could be up to 1k log lines printed every 30 seconds (if all the 1k datanodes have under construction blocks).

This behaviour matches existing behaviour where if a decommissioning datanode has under replicated blocks, then 1 log line will be printed every single DatanodeAdminDefaultMonitor cycle (for that datanode):

return;
}
if (reportingNode == null) {
LOG.warn("Unexpected null input {}", reportingNode);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can we have better msg here ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This code should never occur in practice & so I don't know what additional details/guidance we can add here. Please let me know if you have any specific thoughts/suggestions on this

Since there is a different "Unexpected null input" log printed for various different methods in UnderConstructionBlocks, I have gone ahead & added the method name to the log so that they can be distinguished

@Hexiaoqiao
Copy link
Contributor

@KevinWikant Thanks for your report and contribution. Just wonder which version do you deployed.
BTW, I am excited to see the complete and clear description here. Thanks again.

Copy link

@vnhive vnhive left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  • Please consider adding javadoc in the correct format to all the methods and detailed comments.
  • I have tried to comment two methods, please consider following a similar pattern for the other methods.

@@ -190,6 +190,7 @@ public class BlockManager implements BlockStatsMXBean {

private final PendingDataNodeMessages pendingDNMessages =
new PendingDataNodeMessages();
private final UnderConstructionBlocks ucBlocks;
Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is this UnderConstruction or UnderReplication ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good question. I am definitely open to changing the terminology here.

I don't know if UnderReplication is the correct term though, I think this may be more applicable to a block which needs to be replicated asynchronously by the Namenode to meet replication factor (i.e. due to datanode decommissioning or datanode failures).

From the Namenode logs we can see the block replica state is reported as RBW (replica being written) \ RECEIVING_BLOCK:

2024-12-05 15:51:09,399 DEBUG blockmanagement.BlockManager: Reported block blk_1073741825_1001 on 172.31.93.89:9866 size 268435456 replicaState = RBW
2024-12-05 15:51:09,399 DEBUG BlockStateChange: BLOCK* block RECEIVING_BLOCK: blk_1073741825_1001 is received from 172.31.93.89:9866

I took the term UnderConstruction from here in the code:

However, upon further inspection it seems this term is related to a BlockCollection (as opposed to a block replica):

Let me know your thoughts. I can refactor UnderConstructionBlocks to something like RbwBlocks or ReplicaBeingWrittenBlocks if this is a more accurate term

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am seeing that "UnderConstruction" terminology is already used in the code for both blocks & block replicas:

Therefore, I do think that the name UnderConstructionBlocks aligns with existing terminology already used in the code base

@KevinWikant
Copy link
Contributor Author

Thank you @vnhive @Hexiaoqiao & @shameersss1 for the feedback on this change. Apologies for the delayed response, I was out travelling past several days.

I will address the remaining Javadoc & Spotbugs warnings. I will also address all of the above conversations/comments.

Note that the unit test failures are unrelated:

[INFO] Results:
[INFO] 
[ERROR] Failures: 
[ERROR] org.apache.hadoop.hdfs.TestRollingUpgrade.testDFSAdminRollingUpgradeCommands(org.apache.hadoop.hdfs.TestRollingUpgrade)
[ERROR]   Run 1: TestRollingUpgrade.testDFSAdminRollingUpgradeCommands:115->checkMxBeanIsNull:308 expected null, but was:<javax.management.openmbean.CompositeDataSupport(compositeType=javax.management.openmbean.CompositeType(name=org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo$Bean,items=((itemName=blockPoolId,itemType=javax.management.openmbean.SimpleType(name=java.lang.String)),(itemName=createdRollbackImages,itemType=javax.management.openmbean.SimpleType(name=java.lang.Boolean)),(itemName=finalizeTime,itemType=javax.management.openmbean.SimpleType(name=java.lang.Long)),(itemName=startTime,itemType=javax.management.openmbean.SimpleType(name=java.lang.Long)))),contents={blockPoolId=BP-1510961105-172.17.0.2-1733436045661, createdRollbackImages=true, finalizeTime=0, startTime=1733436049973})>
[ERROR]   Run 2: TestRollingUpgrade.testDFSAdminRollingUpgradeCommands:115->checkMxBeanIsNull:308 expected null, but was:<javax.management.openmbean.CompositeDataSupport(compositeType=javax.management.openmbean.CompositeType(name=org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo$Bean,items=((itemName=blockPoolId,itemType=javax.management.openmbean.SimpleType(name=java.lang.String)),(itemName=createdRollbackImages,itemType=javax.management.openmbean.SimpleType(name=java.lang.Boolean)),(itemName=finalizeTime,itemType=javax.management.openmbean.SimpleType(name=java.lang.Long)),(itemName=startTime,itemType=javax.management.openmbean.SimpleType(name=java.lang.Long)))),contents={blockPoolId=BP-1510961105-172.17.0.2-1733436045661, createdRollbackImages=true, finalizeTime=0, startTime=1733436049973})>
[ERROR]   Run 3: TestRollingUpgrade.testDFSAdminRollingUpgradeCommands:115->checkMxBeanIsNull:308 expected null, but was:<javax.management.openmbean.CompositeDataSupport(compositeType=javax.management.openmbean.CompositeType(name=org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo$Bean,items=((itemName=blockPoolId,itemType=javax.management.openmbean.SimpleType(name=java.lang.String)),(itemName=createdRollbackImages,itemType=javax.management.openmbean.SimpleType(name=java.lang.Boolean)),(itemName=finalizeTime,itemType=javax.management.openmbean.SimpleType(name=java.lang.Long)),(itemName=startTime,itemType=javax.management.openmbean.SimpleType(name=java.lang.Long)))),contents={blockPoolId=BP-1510961105-172.17.0.2-1733436045661, createdRollbackImages=true, finalizeTime=0, startTime=1733436049973})>
[INFO] 
[ERROR] Errors: 
[ERROR] org.apache.hadoop.hdfs.TestRollingUpgrade.testRollback(org.apache.hadoop.hdfs.TestRollingUpgrade)
[ERROR]   Run 1: TestRollingUpgrade.testRollback:337->waitForNullMxBean:367 » Timeout Timed out...
[ERROR]   Run 2: TestRollingUpgrade.testRollback:337->waitForNullMxBean:367 » Timeout Timed out...
[ERROR]   Run 3: TestRollingUpgrade.testRollback:337->waitForNullMxBean:367 » Timeout Timed out...
[INFO] 
[WARNING] Flakes: 
[WARNING] org.apache.hadoop.hdfs.TestRollingUpgrade.testRollBackImage(org.apache.hadoop.hdfs.TestRollingUpgrade)
[ERROR]   Run 1: TestRollingUpgrade.testRollBackImage:645 Test resulted in an unexpected exit: 1: java.io.IOException: All the storage failed while writing properties to VERSION file
[INFO]   Run 2: PASS
[INFO] 
[WARNING] org.apache.hadoop.hdfs.server.balancer.TestBalancerRPCDelay.testBalancerRPCDelayQps3(org.apache.hadoop.hdfs.server.balancer.TestBalancerRPCDelay)
[ERROR]   Run 1: TestBalancerRPCDelay.testBalancerRPCDelayQps3:52 » UnfinishedStubbing 
Unfinis...
[ERROR]   Run 2: TestBalancerRPCDelay.teardown:46 Test resulted in an unexpected exit: 1: Block report processor encountered fatal exception: java.lang.ClassCastException: java.lang.Boolean cannot be cast to org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations
[INFO]   Run 3: PASS
[INFO] 
[WARNING] org.apache.hadoop.hdfs.server.datanode.TestBlockReplacement.testBlockReplacementWithPinnedBlocks(org.apache.hadoop.hdfs.server.datanode.TestBlockReplacement)
[ERROR]   Run 1: TestBlockReplacement.testBlockReplacementWithPinnedBlocks:260 Status code mismatches!
[INFO]   Run 2: PASS
[INFO] 
[WARNING] org.apache.hadoop.hdfs.server.namenode.ha.TestPipelinesFailover.testFailoverRightBeforeCommitSynchronization(org.apache.hadoop.hdfs.server.namenode.ha.TestPipelinesFailover)
[ERROR]   Run 1: TestPipelinesFailover.testFailoverRightBeforeCommitSynchronization:380 » TestTimedOut
[INFO]   Run 2: PASS
[INFO] 
[INFO] 
[ERROR] Tests run: 7072, Failures: 1, Errors: 1, Skipped: 25, Flakes: 4
[INFO] 
[ERROR] There are test failures.

@KevinWikant
Copy link
Contributor Author

@KevinWikant Thanks for your report and contribution. Just wonder which version do you deployed. BTW, I am excited to see the complete and clear description here. Thanks again.

@Hexiaoqiao the testing was conducted against Hadoop version 3.4.0

Please let me know if you have any questions or comments regarding the details/description in the Pull Request and/or the JIRA

@hadoop-yetus
Copy link

💔 -1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 6m 41s Docker mode activated.
_ Prechecks _
+1 💚 dupname 0m 0s No case conflicting files found.
+0 🆗 codespell 0m 0s codespell was not available.
+0 🆗 detsecrets 0m 0s detect-secrets was not available.
+0 🆗 xmllint 0m 0s xmllint was not available.
+1 💚 @author 0m 0s The patch does not contain any @author tags.
+1 💚 test4tests 0m 0s The patch appears to include 2 new or modified test files.
_ trunk Compile Tests _
+1 💚 mvninstall 22m 49s trunk passed
+1 💚 compile 0m 41s trunk passed with JDK Ubuntu-11.0.25+9-post-Ubuntu-1ubuntu120.04
+1 💚 compile 0m 39s trunk passed with JDK Private Build-1.8.0_432-8u432-gaus1-0ubuntu220.04-ga
+1 💚 checkstyle 0m 39s trunk passed
+1 💚 mvnsite 0m 45s trunk passed
+1 💚 javadoc 0m 44s trunk passed with JDK Ubuntu-11.0.25+9-post-Ubuntu-1ubuntu120.04
+1 💚 javadoc 1m 7s trunk passed with JDK Private Build-1.8.0_432-8u432-gaus1-0ubuntu220.04-ga
+1 💚 spotbugs 1m 39s trunk passed
+1 💚 shadedclient 20m 19s branch has no errors when building and testing our client artifacts.
_ Patch Compile Tests _
+1 💚 mvninstall 0m 35s the patch passed
+1 💚 compile 0m 37s the patch passed with JDK Ubuntu-11.0.25+9-post-Ubuntu-1ubuntu120.04
+1 💚 javac 0m 37s the patch passed
+1 💚 compile 0m 33s the patch passed with JDK Private Build-1.8.0_432-8u432-gaus1-0ubuntu220.04-ga
+1 💚 javac 0m 33s the patch passed
+1 💚 blanks 0m 0s The patch has no blanks issues.
+1 💚 checkstyle 0m 31s the patch passed
+1 💚 mvnsite 0m 35s the patch passed
-1 ❌ javadoc 0m 32s /patch-javadoc-hadoop-hdfs-project_hadoop-hdfs-jdkUbuntu-11.0.25+9-post-Ubuntu-1ubuntu120.04.txt hadoop-hdfs in the patch failed with JDK Ubuntu-11.0.25+9-post-Ubuntu-1ubuntu120.04.
+1 💚 javadoc 0m 58s the patch passed with JDK Private Build-1.8.0_432-8u432-gaus1-0ubuntu220.04-ga
+1 💚 spotbugs 1m 38s the patch passed
+1 💚 shadedclient 20m 10s patch has no errors when building and testing our client artifacts.
_ Other Tests _
-1 ❌ unit 199m 0s /patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt hadoop-hdfs in the patch passed.
+1 💚 asflicense 0m 31s The patch does not generate ASF License warnings.
281m 6s
Reason Tests
Failed junit tests hadoop.hdfs.TestDecommissionWithBackoffMonitor
Subsystem Report/Notes
Docker ClientAPI=1.47 ServerAPI=1.47 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-7179/4/artifact/out/Dockerfile
GITHUB PR #7179
Optional Tests dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell detsecrets xmllint
uname Linux 588d7265c933 5.15.0-124-generic #134-Ubuntu SMP Fri Sep 27 20:20:17 UTC 2024 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision trunk / 290ff09
Default Java Private Build-1.8.0_432-8u432-gaus1-0ubuntu220.04-ga
Multi-JDK versions /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.25+9-post-Ubuntu-1ubuntu120.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_432-8u432-gaus1-0ubuntu220.04-ga
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-7179/4/testReport/
Max. process+thread count 4114 (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-7179/4/console
versions git=2.25.1 maven=3.6.3 spotbugs=4.2.2
Powered by Apache Yetus 0.14.0 https://yetus.apache.org

This message was automatically generated.

@hadoop-yetus
Copy link

💔 -1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 17m 45s Docker mode activated.
_ Prechecks _
+1 💚 dupname 0m 0s No case conflicting files found.
+0 🆗 codespell 0m 1s codespell was not available.
+0 🆗 detsecrets 0m 1s detect-secrets was not available.
+0 🆗 xmllint 0m 1s xmllint was not available.
+1 💚 @author 0m 0s The patch does not contain any @author tags.
+1 💚 test4tests 0m 0s The patch appears to include 2 new or modified test files.
_ trunk Compile Tests _
+1 💚 mvninstall 40m 27s trunk passed
+1 💚 compile 1m 25s trunk passed with JDK Ubuntu-11.0.25+9-post-Ubuntu-1ubuntu120.04
+1 💚 compile 1m 16s trunk passed with JDK Private Build-1.8.0_432-8u432-gaus1-0ubuntu220.04-ga
+1 💚 checkstyle 1m 16s trunk passed
+1 💚 mvnsite 1m 25s trunk passed
+1 💚 javadoc 1m 14s trunk passed with JDK Ubuntu-11.0.25+9-post-Ubuntu-1ubuntu120.04
+1 💚 javadoc 1m 44s trunk passed with JDK Private Build-1.8.0_432-8u432-gaus1-0ubuntu220.04-ga
+1 💚 spotbugs 3m 22s trunk passed
+1 💚 shadedclient 42m 45s branch has no errors when building and testing our client artifacts.
_ Patch Compile Tests _
+1 💚 mvninstall 1m 12s the patch passed
+1 💚 compile 1m 16s the patch passed with JDK Ubuntu-11.0.25+9-post-Ubuntu-1ubuntu120.04
+1 💚 javac 1m 16s the patch passed
+1 💚 compile 1m 7s the patch passed with JDK Private Build-1.8.0_432-8u432-gaus1-0ubuntu220.04-ga
+1 💚 javac 1m 7s the patch passed
+1 💚 blanks 0m 0s The patch has no blanks issues.
+1 💚 checkstyle 1m 7s the patch passed
+1 💚 mvnsite 1m 16s the patch passed
-1 ❌ javadoc 1m 5s /patch-javadoc-hadoop-hdfs-project_hadoop-hdfs-jdkUbuntu-11.0.25+9-post-Ubuntu-1ubuntu120.04.txt hadoop-hdfs in the patch failed with JDK Ubuntu-11.0.25+9-post-Ubuntu-1ubuntu120.04.
+1 💚 javadoc 1m 38s the patch passed with JDK Private Build-1.8.0_432-8u432-gaus1-0ubuntu220.04-ga
+1 💚 spotbugs 3m 22s the patch passed
+1 💚 shadedclient 42m 10s patch has no errors when building and testing our client artifacts.
_ Other Tests _
-1 ❌ unit 256m 4s /patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt hadoop-hdfs in the patch passed.
+1 💚 asflicense 0m 48s The patch does not generate ASF License warnings.
421m 42s
Reason Tests
Failed junit tests hadoop.hdfs.TestRollingUpgrade
hadoop.hdfs.TestDecommission
hadoop.hdfs.server.balancer.TestBalancerLongRunningTasks
Subsystem Report/Notes
Docker ClientAPI=1.47 ServerAPI=1.47 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-7179/3/artifact/out/Dockerfile
GITHUB PR #7179
Optional Tests dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell detsecrets xmllint
uname Linux ff0293bb107e 5.15.0-124-generic #134-Ubuntu SMP Fri Sep 27 20:20:17 UTC 2024 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision trunk / b450fee
Default Java Private Build-1.8.0_432-8u432-gaus1-0ubuntu220.04-ga
Multi-JDK versions /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.25+9-post-Ubuntu-1ubuntu120.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_432-8u432-gaus1-0ubuntu220.04-ga
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-7179/3/testReport/
Max. process+thread count 3056 (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-7179/3/console
versions git=2.25.1 maven=3.6.3 spotbugs=4.2.2
Powered by Apache Yetus 0.14.0 https://yetus.apache.org

This message was automatically generated.

@KevinWikant
Copy link
Contributor Author

KevinWikant commented Dec 20, 2024

One more Javadoc issue to fix, interestingly when I run mvn javadoc:javadoc locally I do not see this error:

[ERROR] /home/jenkins/jenkins-home/workspace/hadoop-multibranch_PR-7179/ubuntu-focal/src/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java:1775: error: bad HTML entity
[ERROR]    * reduction in HDFS write failures & HDFS data loss.

One of the unit test failures is potentially related for TestDecommission.testDecommissionWithUCBlocksFeatureDisabledAndDefaultMonitor:

[INFO] Results:
[INFO] 
[ERROR] Failures: 
[ERROR] org.apache.hadoop.hdfs.TestDecommission.testDecommissionWithUCBlocksFeatureDisabledAndDefaultMonitor(org.apache.hadoop.hdfs.TestDecommission)
[ERROR]   Run 1: TestDecommission.testDecommissionWithUCBlocksFeatureDisabledAndDefaultMonitor:2192->testDecommissionWithUnderConstructionBlocksFeatureDisabled:2328 Write expected to fail if all nodes in write pipeline are terminated
[ERROR]   Run 2: TestDecommission.testDecommissionWithUCBlocksFeatureDisabledAndDefaultMonitor:2192->testDecommissionWithUnderConstructionBlocksFeatureDisabled:2328 Write expected to fail if all nodes in write pipeline are terminated
[ERROR]   Run 3: TestDecommission.testDecommissionWithUCBlocksFeatureDisabledAndDefaultMonitor:2192->testDecommissionWithUnderConstructionBlocksFeatureDisabled:2328 Write expected to fail if all nodes in write pipeline are terminated
[INFO] 
[ERROR] Errors: 
[ERROR] org.apache.hadoop.hdfs.TestRollingUpgrade.testRollback(org.apache.hadoop.hdfs.TestRollingUpgrade)
[ERROR]   Run 1: TestRollingUpgrade.testRollback:337->waitForNullMxBean:367 » Timeout Timed out...
[ERROR]   Run 2: TestRollingUpgrade.testRollback:337->waitForNullMxBean:367 » Timeout Timed out...
[ERROR]   Run 3: TestRollingUpgrade.testRollback:337->waitForNullMxBean:367 » Timeout Timed out...
[INFO] 
[ERROR] org.apache.hadoop.hdfs.server.balancer.TestBalancerLongRunningTasks.testBalancerMetricsDuplicate(org.apache.hadoop.hdfs.server.balancer.TestBalancerLongRunningTasks)
[ERROR]   Run 1: TestBalancerLongRunningTasks.testBalancerMetricsDuplicate:822 expected:<0> but was:<-3>
[ERROR]   Run 2: TestBalancerLongRunningTasks.testBalancerMetricsDuplicate:784 » Metrics Metric...
[ERROR]   Run 3: TestBalancerLongRunningTasks.testBalancerMetricsDuplicate:784 » Metrics Metric...
[INFO] 
[ERROR] org.apache.hadoop.hdfs.server.balancer.TestBalancerLongRunningTasks.testBalancerWithLimitOverUtilizedNum(org.apache.hadoop.hdfs.server.balancer.TestBalancerLongRunningTasks)
[ERROR]   Run 1: TestBalancerLongRunningTasks.testBalancerWithLimitOverUtilizedNum:701 » Metrics
[ERROR]   Run 2: TestBalancerLongRunningTasks.testBalancerWithLimitOverUtilizedNum:701 » Metrics
[ERROR]   Run 3: TestBalancerLongRunningTasks.testBalancerWithLimitOverUtilizedNum:701 » Metrics
[INFO] 
[ERROR] org.apache.hadoop.hdfs.server.balancer.TestBalancerLongRunningTasks.testBalancerWithSortTopNodes(org.apache.hadoop.hdfs.server.balancer.TestBalancerLongRunningTasks)
[ERROR]   Run 1: TestBalancerLongRunningTasks.testBalancerWithSortTopNodes:584 » Metrics Metric...
[ERROR]   Run 2: TestBalancerLongRunningTasks.testBalancerWithSortTopNodes:584 » Metrics Metric...
[ERROR]   Run 3: TestBalancerLongRunningTasks.testBalancerWithSortTopNodes:584 » Metrics Metric...
[INFO] 
[ERROR] org.apache.hadoop.hdfs.server.balancer.TestBalancerLongRunningTasks.testMaxIterationTime(org.apache.hadoop.hdfs.server.balancer.TestBalancerLongRunningTasks)
[ERROR]   Run 1: TestBalancerLongRunningTasks.testMaxIterationTime:851 » Metrics Metrics source...
[ERROR]   Run 2: TestBalancerLongRunningTasks.testMaxIterationTime:851 » Metrics Metrics source...
[ERROR]   Run 3: TestBalancerLongRunningTasks.testMaxIterationTime:851 » Metrics Metrics source...
[INFO] 
[ERROR] org.apache.hadoop.hdfs.server.balancer.TestBalancerLongRunningTasks.testRackPolicyAfterBalance(org.apache.hadoop.hdfs.server.balancer.TestBalancerLongRunningTasks)
[ERROR]   Run 1: TestBalancerLongRunningTasks.testRackPolicyAfterBalance:435->runBalancerAndVerifyBlockPlacmentPolicy:446 » Metrics
[ERROR]   Run 2: TestBalancerLongRunningTasks.testRackPolicyAfterBalance:435->runBalancerAndVerifyBlockPlacmentPolicy:446 » Metrics
[ERROR]   Run 3: TestBalancerLongRunningTasks.testRackPolicyAfterBalance:435->runBalancerAndVerifyBlockPlacmentPolicy:446 » Metrics
[INFO] 
[ERROR] org.apache.hadoop.hdfs.server.balancer.TestBalancerLongRunningTasks.testUpgradeDomainPolicyAfterBalance(org.apache.hadoop.hdfs.server.balancer.TestBalancerLongRunningTasks)
[ERROR]   Run 1: TestBalancerLongRunningTasks.testUpgradeDomainPolicyAfterBalance:419->runBalancerAndVerifyBlockPlacmentPolicy:446 » Metrics
[ERROR]   Run 2: TestBalancerLongRunningTasks.testUpgradeDomainPolicyAfterBalance:419->runBalancerAndVerifyBlockPlacmentPolicy:446 » Metrics
[ERROR]   Run 3: TestBalancerLongRunningTasks.testUpgradeDomainPolicyAfterBalance:419->runBalancerAndVerifyBlockPlacmentPolicy:446 » Metrics
[INFO] 
[WARNING] Flakes: 
[WARNING] org.apache.hadoop.hdfs.TestRollingUpgrade.testCheckpoint(org.apache.hadoop.hdfs.TestRollingUpgrade)
[ERROR]   Run 1: TestRollingUpgrade.testCheckpoint:596->testCheckpoint:681 Test resulted in an unexpected exit: 1: java.io.IOException: All the storage failed while writing properties to VERSION file
[INFO]   Run 2: PASS
[INFO] 
[WARNING] org.apache.hadoop.hdfs.TestRollingUpgrade.testFinalize(org.apache.hadoop.hdfs.TestRollingUpgrade)
[ERROR]   Run 1: TestRollingUpgrade.testFinalize:445->testFinalize:464->testFinalize:520 Test resulted in an unexpected exit: 1: java.io.IOException: All the storage failed while writing properties to VERSION file
[INFO]   Run 2: PASS
[INFO] 
[WARNING] org.apache.hadoop.hdfs.server.balancer.TestBalancer.testBalancerExcludeTargetNodesNoMoveBlock(org.apache.hadoop.hdfs.server.balancer.TestBalancer)
[ERROR]   Run 1: TestBalancer.testBalancerExcludeTargetNodesNoMoveBlock:2084 unexpected exception type thrown; expected:<java.lang.Exception> but was:<java.lang.AssertionError>
[INFO]   Run 2: PASS
[INFO] 
[WARNING] org.apache.hadoop.hdfs.server.blockmanagement.TestBlockManager.testBlockReportSetNoAckBlockToInvalidate(org.apache.hadoop.hdfs.server.blockmanagement.TestBlockManager)
[ERROR]   Run 1: TestBlockManager.testBlockReportSetNoAckBlockToInvalidate:2165 expected null, but was:<blk_1073741825_1001>
[INFO]   Run 2: PASS
[INFO] 
[WARNING] org.apache.hadoop.hdfs.server.blockmanagement.TestUnderReplicatedBlocks.testSetRepIncWithUnderReplicatedBlocks(org.apache.hadoop.hdfs.server.blockmanagement.TestUnderReplicatedBlocks)
[ERROR]   Run 1: TestUnderReplicatedBlocks.testSetRepIncWithUnderReplicatedBlocks:80 » TestTimedOut
[INFO]   Run 2: PASS
[INFO] 
[WARNING] org.apache.hadoop.hdfs.server.datanode.TestIncrementalBrVariations.testDataNodeDoesNotSplitReports(org.apache.hadoop.hdfs.server.datanode.TestIncrementalBrVariations)
[ERROR]   Run 1: TestIncrementalBrVariations.testDataNodeDoesNotSplitReports:230 Bad value for metric BlockReceivedAndDeletedOps expected:<12> but was:<11>
[INFO]   Run 2: PASS
[INFO] 
[WARNING] org.apache.hadoop.hdfs.server.namenode.TestFileTruncate.testTruncateWithRollingUpgrade(org.apache.hadoop.hdfs.server.namenode.TestFileTruncate)
[ERROR]   Run 1: TestFileTruncate.testTruncateWithRollingUpgrade:1274 block num should 0 expected:<0> but was:<1>
[ERROR]   Run 2: TestFileTruncate.testTruncateWithRollingUpgrade:1274 block num should 0 expected:<0> but was:<1>
[INFO]   Run 3: PASS
[INFO] 
[INFO] 
[ERROR] Tests run: 7140, Failures: 1, Errors: 7, Skipped: 25, Flakes: 7

Will wait to see if there are any more follow-up comments before pushing a new revision

@KevinWikant
Copy link
Contributor Author

Javadoc Failure

I have fixed the last javadoc warning:

[ERROR] /home/jenkins/jenkins-home/workspace/hadoop-multibranch_PR-7179/ubuntu-focal/src/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java:1775: error: bad HTML entity
[ERROR]    * reduction in HDFS write failures & HDFS data loss.

I am probably missing something here, but when I previously ran javadoc on my local I did not see this error:

> pwd
.../hadoop-hdfs-project/hadoop-hdfs

> mvn javadoc:javadoc
...
[INFO] --- javadoc:3.0.1:javadoc (default-cli) @ hadoop-hdfs ---
[INFO] 
ExcludePrivateAnnotationsStandardDoclet
1 warning
[WARNING] Javadoc Warnings
[WARNING] .../hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeAttributeProvider.java:386: warning - Tag @link: can't find checkPermissionWithContext(AuthorizationContext) in org.apache.hadoop.hdfs.server.namenode.INodeAttributeProvider.AccessControlEnforcer
[INFO] ------------------------------------------------------------------------
[INFO] BUILD SUCCESS
[INFO] ------------------------------------------------------------------------
[INFO] Total time:  22.182 s
[INFO] Finished at: 2024-12-21T17:05:06-05:00
[INFO] ------------------------------------------------------------------------

Unit Test Failure - testDecommissionWithUCBlocksFeatureDisabledAndDefaultMonitor

I ran the following command 100 times in a loop:

mvn -Dtest=TestDecommission#testDecommissionWithUCBlocksFeatureDisabledAndDefaultMonitor test -o

I then ran the following command 50 times in a loop:

mvn -Dtest=TestDecommission test -o

I did not see any failures for testDecommissionWithUCBlocksFeatureDisabledAndDefaultMonitor

I suspect the testDecommissionWithUCBlocksFeatureDisabledAndDefaultMonitor failure (when executed by Yetus) may be due to timing condition which only reproduces on the Yetus test runner (such as JVM pause, thread scheduling, or other).

Proposing that we do not block this change on this potentially flaky test.

  • The testDecommissionWithUCBlocksFeatureDisabledAndDefaultMonitor test shows that when dfs.namenode.decommission.track.underconstructionblocks = false, there will be HDFS write failures & HDFS data loss in the majority of cases.
  • If the test flakiness is not a test issue (i.e. a MiniDFSCluster issue for example), then it would only mean that sporadically there might not be HDFS write failures & HDFS data loss (which is technically a good thing).

Also, about 1 in 5 test runs I was seeing failures in testDecommissionWithUCBlocksFeatureEnabledAndBackoffMonitor. I have root caused this & made some minor changes in latest commit.

@KevinWikant
Copy link
Contributor Author

Unit Test Failure - testDecommissionWithUCBlocksFeatureEnabledAndBackoffMonitor

When I run mvn -Dtest=TestDecommission test -o locally, I am seeing about 1 in 5 cases where testDecommissionWithUCBlocksFeatureEnabledAndBackoffMonitor fails.

The root cause is as follows:

  • one of the datanodes gets blocked in decommissioning state due to an Under Construction block replica (on the affected datanode) being tracked in UnderConstructionBlocks
  • logs show that the relevant Under Construction block replica was removed from UnderConstructionBlocks, but then was re-added very shortly after
  • the block replica was reported 2 times in FINALIZED state in short succession (i.e. a duplicate 2nd block report was generated for some reason with the same replica in the same state on the same datanode).
  • when the block replica is reported a 2nd time in FINALIZED state, the "block.findStorageInfo(storageInfo) < 0"" part of this condition evaluates to false which causes the else condition to be executed

Relevant logs:

[ERROR] testDecommissionWithUCBlocksFeatureEnabledAndBackoffMonitor(org.apache.hadoop.hdfs.TestDecommission)  Time elapsed: 43.735 s  <<< FAILURE!
java.lang.AssertionError: Datanodes did not enter DECOMMISSIONED as expected
        at org.junit.Assert.fail(Assert.java:89)
        at org.apache.hadoop.hdfs.TestDecommission.testDecommissionWithUnderConstructionBlocksFeatureEnabled(TestDecommission.java:2487)
        at org.apache.hadoop.hdfs.TestDecommission.testDecommissionWithUCBlocksFeatureEnabledAndBackoffMonitor(TestDecommission.java:2346)
        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.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59)
        at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
        at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56)
        at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
        at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:299)
        at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:293)
        at java.util.concurrent.FutureTask.run(FutureTask.java:266)
        at java.lang.Thread.run(Thread.java:750)
2024-12-23 19:11:18,180 [Time-limited test] DEBUG blockmanagement.UnderConstructionBlocks (UnderConstructionBlocks.java:getUnderConstructionBlocksByDatanode(448)) - Under Construction block counts: [{}]
2024-12-23 19:11:18,682 [Time-limited test] DEBUG blockmanagement.UnderConstructionBlocks (UnderConstructionBlocks.java:getUnderConstructionBlocksByDatanode(448)) - Under Construction block counts: [{}]
...
2024-12-23 19:11:19,184 [Time-limited test] INFO  hdfs.TestDecommission (TestDecommission.java:testDecommissionWithUnderConstructionBlocksFeatureEnabled(2423)) - Decommission node 127.0.0.1:59384 with the UC replica
2024-12-23 19:11:19,259 [Time-limited test] INFO  hdfs.TestDecommission (TestDecommission.java:testDecommissionWithUnderConstructionBlocksFeatureEnabled(2423)) - Decommission node 127.0.0.1:59371 with the UC replica
...
2024-12-23 19:11:22,710 [DatanodeAdminMonitor-0] DEBUG blockmanagement.UnderConstructionBlocks (UnderConstructionBlocks.java:getUnderConstructionBlocksByDatanode(448)) - Under Construction block counts: [{127.0.0.1:59384=1,127.0.0.1:59371=1}]
2024-12-23 19:11:22,823 [DatanodeAdminMonitor-0] DEBUG blockmanagement.UnderConstructionBlocks (UnderConstructionBlocks.java:getUnderConstructionBlocksByDatanode(448)) - Under Construction block counts: [{127.0.0.1:59384=1,127.0.0.1:59371=1}]
...
2024-12-23 19:11:22,711 [DatanodeAdminMonitor-0] INFO  blockmanagement.DatanodeAdminBackoffMonitor (DatanodeAdminBackoffMonitor.java:processCompletedNodes(385)) - Cannot decommission datanode 127.0.0.1:59384 with 1 UC blocks: [blk_1073741825_1001]
2024-12-23 19:11:22,711 [DatanodeAdminMonitor-0] INFO  blockmanagement.DatanodeAdminBackoffMonitor (DatanodeAdminBackoffMonitor.java:processCompletedNodes(385)) - Cannot decommission datanode 127.0.0.1:59371 with 1 UC blocks: [blk_1073741825_1001]
2024-12-23 19:11:22,711 [DatanodeAdminMonitor-0] INFO  blockmanagement.DatanodeAdminBackoffMonitor (DatanodeAdminBackoffMonitor.java:run(229)) - Checked 0 blocks this tick. 2 nodes are now in maintenance or transitioning state. 0 nodes pending. 0 nodes waiting to be cancelled.
...
2024-12-23 19:11:29,384 [DatanodeAdminMonitor-0] DEBUG blockmanagement.UnderConstructionBlocks (UnderConstructionBlocks.java:getUnderConstructionBlocksByDatanode(448)) - Under Construction block counts: [{127.0.0.1:59384=1,127.0.0.1:59371=1}]
2024-12-23 19:11:29,888 [DatanodeAdminMonitor-0] DEBUG blockmanagement.UnderConstructionBlocks (UnderConstructionBlocks.java:getUnderConstructionBlocksByDatanode(448)) - Under Construction block counts: [{127.0.0.1:59384=1,127.0.0.1:59371=1}]
...
2024-12-23 19:11:30,912 [Block report processor] DEBUG blockmanagement.BlockManager (BlockManager.java:processReportedBlock(3449)) - Reported block blk_1073741825_1001 on 127.0.0.1:59371 size 1536 replicaState = FINALIZED
2024-12-23 19:11:30,912 [Block report processor] DEBUG blockmanagement.UnderConstructionBlocks (UnderConstructionBlocks.java:removeUcBlockFromSet(265)) - Removed UC block [blk_1073741825_1001->blk_1073741825_1001] from 127.0.0.1:59371, new total is [replicas=1 / blocks=1]
2024-12-23 19:11:30,912 [Block report processor] INFO  BlockStateChange (BlockManager.java:addStoredBlock(3828)) - BLOCK* addStoredBlock: 127.0.0.1:59371 is added to blk_1073741825_1001 (size=1536)
...
2024-12-23 19:11:30,921 [Block report processor] DEBUG blockmanagement.BlockManager (BlockManager.java:processReportedBlock(3449)) - Reported block blk_1073741825_1001 on 127.0.0.1:59384 size 1536 replicaState = FINALIZED
2024-12-23 19:11:30,922 [Block report processor] DEBUG blockmanagement.UnderConstructionBlocks (UnderConstructionBlocks.java:removeUcBlockFromSet(265)) - Removed UC block [blk_1073741825_1001->blk_1073741825_1001] from 127.0.0.1:59384, new total is [replicas=0 / blocks=0]
2024-12-23 19:11:30,922 [Block report processor] INFO  BlockStateChange (BlockManager.java:addStoredBlock(3828)) - BLOCK* addStoredBlock: 127.0.0.1:59384 is added to blk_1073741825_1001 (size=1536)
...
2024-12-23 19:11:30,922 [Block report processor] DEBUG blockmanagement.BlockManager (BlockManager.java:processReportedBlock(3449)) - Reported block blk_1073741825_1001 on 127.0.0.1:59371 size 1536 replicaState = FINALIZED
2024-12-23 19:11:30,923 [Block report processor] DEBUG blockmanagement.UnderConstructionBlocks (UnderConstructionBlocks.java:addUcBlockToSet(407)) - Add UC block blk_1073741825_1001 to 127.0.0.1:59371, new total is [replicas=1 / blocks=1]
...
2024-12-23 19:11:32,718 [DatanodeAdminMonitor-0] DEBUG blockmanagement.UnderConstructionBlocks (UnderConstructionBlocks.java:getUnderConstructionBlocksByDatanode(448)) - Under Construction block counts: [{127.0.0.1:59371=1}]
...
2024-12-23 19:11:32,718 [DatanodeAdminMonitor-0] INFO  blockmanagement.DatanodeAdminBackoffMonitor (DatanodeAdminBackoffMonitor.java:processCompletedNodes(414)) - Node 127.0.0.1:59384 is sufficiently replicated and healthy, marked as Decommissioned.
2024-12-23 19:11:32,718 [DatanodeAdminMonitor-0] INFO  blockmanagement.DatanodeAdminBackoffMonitor (DatanodeAdminBackoffMonitor.java:processCompletedNodes(385)) - Cannot decommission datanode 127.0.0.1:59371 with 1 UC blocks: [blk_1073741825_1001]
...
2024-12-23 19:11:37,723 [DatanodeAdminMonitor-0] DEBUG blockmanagement.UnderConstructionBlocks (UnderConstructionBlocks.java:getUnderConstructionBlocksByDatanode(448)) - Under Construction block counts: [{127.0.0.1:59371=1}]
...

Note how on datanode 127.0.0.1:59384 the replica is reported in FINALIZED state once & there are no issues.

However, on 127.0.0.1:59371 the replica is reported in FINALIZED state twice & on the 2nd time it gets re-added to UnderConstructionBlocks erroneously.

The solution I have implemented is to only add the replica to UnderConstructionBlocks if its reported in state ReplicaState.RBW. This way:

  • if the replica is reported in ReplicaState.RBW multiple times, then addUcBlock will be invoked multiple times with subsequent attempts being a no-op.
  • if the replica is reported in ReplicaState.FINALIZED (or other) multiple times, then addUcBlock will not be invoked erroneously on subsequent reports.
  • if there is an out-of-order state reporting for a replica (i.e. reported in ReplicaState.FINALIZED then erroneously reported in ReplicaState.RBW afterwards), then this scenario will lead to issues. But this should not be possible in practice per my understanding.

@hadoop-yetus
Copy link

🎊 +1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 0m 50s Docker mode activated.
_ Prechecks _
+1 💚 dupname 0m 0s No case conflicting files found.
+0 🆗 codespell 0m 1s codespell was not available.
+0 🆗 detsecrets 0m 1s detect-secrets was not available.
+0 🆗 xmllint 0m 1s xmllint was not available.
+1 💚 @author 0m 0s The patch does not contain any @author tags.
+1 💚 test4tests 0m 0s The patch appears to include 2 new or modified test files.
_ trunk Compile Tests _
+1 💚 mvninstall 40m 13s trunk passed
+1 💚 compile 1m 25s trunk passed with JDK Ubuntu-11.0.25+9-post-Ubuntu-1ubuntu120.04
+1 💚 compile 1m 17s trunk passed with JDK Private Build-1.8.0_432-8u432-gaus1-0ubuntu220.04-ga
+1 💚 checkstyle 1m 17s trunk passed
+1 💚 mvnsite 1m 26s trunk passed
+1 💚 javadoc 1m 19s trunk passed with JDK Ubuntu-11.0.25+9-post-Ubuntu-1ubuntu120.04
+1 💚 javadoc 1m 45s trunk passed with JDK Private Build-1.8.0_432-8u432-gaus1-0ubuntu220.04-ga
+1 💚 spotbugs 3m 23s trunk passed
+1 💚 shadedclient 41m 58s branch has no errors when building and testing our client artifacts.
_ Patch Compile Tests _
+1 💚 mvninstall 1m 13s the patch passed
+1 💚 compile 1m 17s the patch passed with JDK Ubuntu-11.0.25+9-post-Ubuntu-1ubuntu120.04
+1 💚 javac 1m 17s the patch passed
+1 💚 compile 1m 10s the patch passed with JDK Private Build-1.8.0_432-8u432-gaus1-0ubuntu220.04-ga
+1 💚 javac 1m 10s the patch passed
+1 💚 blanks 0m 0s The patch has no blanks issues.
+1 💚 checkstyle 1m 5s the patch passed
+1 💚 mvnsite 1m 18s the patch passed
+1 💚 javadoc 1m 5s the patch passed with JDK Ubuntu-11.0.25+9-post-Ubuntu-1ubuntu120.04
+1 💚 javadoc 1m 38s the patch passed with JDK Private Build-1.8.0_432-8u432-gaus1-0ubuntu220.04-ga
+1 💚 spotbugs 3m 23s the patch passed
+1 💚 shadedclient 42m 25s patch has no errors when building and testing our client artifacts.
_ Other Tests _
+1 💚 unit 257m 6s hadoop-hdfs in the patch passed.
+1 💚 asflicense 0m 50s The patch does not generate ASF License warnings.
405m 19s
Subsystem Report/Notes
Docker ClientAPI=1.47 ServerAPI=1.47 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-7179/5/artifact/out/Dockerfile
GITHUB PR #7179
Optional Tests dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell detsecrets xmllint
uname Linux 7a1cc9053ffb 5.15.0-124-generic #134-Ubuntu SMP Fri Sep 27 20:20:17 UTC 2024 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision trunk / 731900e
Default Java Private Build-1.8.0_432-8u432-gaus1-0ubuntu220.04-ga
Multi-JDK versions /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.25+9-post-Ubuntu-1ubuntu120.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_432-8u432-gaus1-0ubuntu220.04-ga
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-7179/5/testReport/
Max. process+thread count 3048 (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-7179/5/console
versions git=2.25.1 maven=3.6.3 spotbugs=4.2.2
Powered by Apache Yetus 0.14.0 https://yetus.apache.org

This message was automatically generated.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

5 participants