[
https://issues.apache.org/jira/browse/HDFS-16568?focusedWorklogId=767000&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-767000
]
ASF GitHub Bot logged work on HDFS-16568:
-----------------------------------------
Author: ASF GitHub Bot
Created on: 06/May/22 03:16
Start Date: 06/May/22 03:16
Worklog Time Spent: 10m
Work Description: virajjasani commented on code in PR #4264:
URL: https://github.com/apache/hadoop/pull/4264#discussion_r866460800
##########
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java:
##########
@@ -1915,12 +1919,57 @@ public int reconfig(String[] argv, int i) throws
IOException {
}
int startReconfiguration(final String nodeThpe, final String address)
- throws IOException {
- return startReconfiguration(nodeThpe, address, System.out, System.err);
+ throws IOException, InterruptedException {
+ return startReconfigurationUtil(nodeThpe, address, System.out, System.err);
+ }
+
+ int startReconfigurationUtil(final String nodeType, final String address,
final PrintStream out,
+ final PrintStream err) throws IOException, InterruptedException {
+ if (!"livenodes".equals(address)) {
+ return startReconfiguration(nodeType, address, out, err);
+ }
+ if (!"datanode".equals(nodeType)) {
+ err.println("Only datanode type supports reconfiguration in bulk.");
+ return 1;
+ }
+ ExecutorService executorService = Executors.newFixedThreadPool(5);
+ DistributedFileSystem dfs = getDFS();
+ DatanodeInfo[] nodes = dfs.getDataNodeStats(DatanodeReportType.LIVE);
+ AtomicInteger successCount = new AtomicInteger();
+ AtomicInteger failCount = new AtomicInteger();
+ if (nodes != null) {
+ for (DatanodeInfo node : nodes) {
+ executorService.submit(() -> {
+ int status = startReconfiguration(nodeType, node.getIpcAddr(false),
out, err);
+ if (status == 0) {
+ successCount.incrementAndGet();
+ } else {
+ failCount.incrementAndGet();
+ }
+ });
+ }
+ while ((successCount.get() + failCount.get()) < nodes.length) {
Review Comment:
Agree, countDownLatch would be better but since we are also keeping count of
both success and failure, and using these counts to return final status (0 or
1), it is difficult to allocate count down number initially. For instance, if
we use two latches - successCountDownLatch and failCountDownLatch, we can't
know what initial count to initiate both latches with. Hence, if we still want
to use CountDownLatch, we will have to use all 3 of them:
```
AtomicInteger successCount = new AtomicInteger();
AtomicInteger failCount = new AtomicInteger();
CountDownLatch countDownLatch = new CountDownLatch(nodes.length); //
total count as nodes array length
```
Hence, rather than using additional CountDownLatch, maybe we are good with
only two AtomicIntegers? WDYT?
Issue Time Tracking
-------------------
Worklog Id: (was: 767000)
Time Spent: 1h 20m (was: 1h 10m)
> dfsadmin -reconfig option to start/query reconfig on all live datanodes
> -----------------------------------------------------------------------
>
> Key: HDFS-16568
> URL: https://issues.apache.org/jira/browse/HDFS-16568
> Project: Hadoop HDFS
> Issue Type: New Feature
> Reporter: Viraj Jasani
> Assignee: Viraj Jasani
> Priority: Major
> Labels: pull-request-available
> Time Spent: 1h 20m
> Remaining Estimate: 0h
>
> DFSAdmin provides option to initiate or query the status of reconfiguration
> operation on only specific host based on host:port provided by user. It would
> be good to provide an ability to initiate such operations in bulk, on all
> live datanodes.
--
This message was sent by Atlassian Jira
(v8.20.7#820007)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]