[
https://issues.apache.org/jira/browse/HDFS-17885?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=18060228#comment-18060228
]
ASF GitHub Bot commented on HDFS-17885:
---------------------------------------
slfan1989 opened a new pull request, #8269:
URL: https://github.com/apache/hadoop/pull/8269
### Description of PR
JIRA: HDFS-17885. Fix TestDFSAdmin.testAllDatanodesReconfig flaky test.
#### Problem
`TestDFSAdmin.testAllDatanodesReconfig` test fails with the following error:
```
Expected size:<3> but was:<1> in:
<["Starting of reconfiguration task successful on 0 nodes, failed on 2
nodes."]>
at
org.apache.hadoop.hdfs.tools.TestDFSAdmin.testAllDatanodesReconfig(TestDFSAdmin.java:1263)
```
#### Root Cause
The test has a "self-conflicting" issue where it starts the reconfiguration
task twice on the same DataNodes:
- First call: `admin.startReconfiguration("datanode", "livenodes")` -
Successfully starts reconfiguration on 2 DataNodes
- Second call: `reconfigurationOutErrFormatter("startReconfiguration", ...)`
- Internally calls `admin.startReconfigurationUtil(...)` again
The problem is that DataNode's `startReconfigurationTask()` does not allow
concurrent reconfiguration. If a reconfiguration task is already running, it
throws `IOException` with message `Another reconfiguration task is running.`
Therefore, the second invocation fails on both DataNodes, resulting in
output containing only the summary line:
```
Starting of reconfiguration task successful on 0 nodes, failed on 2 nodes.
```
This causes the assertion `assertThat(outsForStartReconf).hasSize(3)` to
fail because:
- Expected: 2 "Started reconfiguration task on node" lines + 1 summary line
= 3 lines
- Actual: 0 success lines + 1 summary line = 1 line
#### Solution
Remove the duplicate invocation by:
1. Calling `admin.startReconfigurationUtil()` only once
2. Directly capturing the output to ByteArrayOutputStream
3. Parsing the output for assertions
Additionally, improve the test robustness by:
- Using `NUM_DATANODES` constant instead of hardcoded values
- Using stream filtering to count "Started reconfiguration" lines instead of
relying on fixed positions (which is more resilient to concurrent output
ordering)
- Removing unnecessary `Thread.sleep(1000)` before
`awaitReconfigurationFinished()`
### How was this patch tested?
> ./mvnw -pl hadoop-hdfs-project/hadoop-hdfs
-Dtest=TestDFSAdmin#testAllDatanodesReconfig test
```
[INFO] -------------------------------------------------------
[INFO] T E S T S
[INFO] -------------------------------------------------------
[INFO] Running org.apache.hadoop.hdfs.tools.TestDFSAdmin
OpenJDK 64-Bit Server VM warning: Sharing is only supported for boot loader
classes because bootstrap classpath has been appended
[INFO] Tests run: 1, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 4.421
s
> Fix TestDFSAdmin.testAllDatanodesReconfig flaky test
> ----------------------------------------------------
>
> Key: HDFS-17885
> URL: https://issues.apache.org/jira/browse/HDFS-17885
> Project: Hadoop HDFS
> Issue Type: Bug
> Components: dfsadmin
> Affects Versions: 3.5.0, 3.5.1
> Reporter: Shilun Fan
> Assignee: Shilun Fan
> Priority: Major
>
> {*}Problem{*}{*}{*}
> `TestDFSAdmin.testAllDatanodesReconfig` fails because it invokes
> reconfiguration twice on the same DataNodes, causing the second invocation to
> fail with "Another reconfiguration task is running."
>
> *Solution*
> - Start reconfiguration only once by calling
> `admin.startReconfigurationUtil()` directly
> - Capture output to ByteArrayOutputStream for assertions
> - Improve test robustness: use `NUM_DATANODES` constant, stream-based
> counting instead of fixed positions
> - Remove unnecessary `Thread.sleep(1000)`
>
>
>
--
This message was sent by Atlassian Jira
(v8.20.10#820010)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]