HDFS-17361. DiskBalancer: Query command support with multiple nodes (#6508)

Signed-off-by: Takanobu Asanuma <tasanuma@apache.org>
This commit is contained in:
huhaiyang 2024-02-19 08:34:59 +08:00 committed by GitHub
parent 9751b6e41a
commit 03d9acaa86
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
4 changed files with 95 additions and 39 deletions

View File

@ -19,6 +19,8 @@
package org.apache.hadoop.hdfs.server.diskbalancer.command; package org.apache.hadoop.hdfs.server.diskbalancer.command;
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.text.TextStringBuilder;
import org.apache.hadoop.util.Preconditions; import org.apache.hadoop.util.Preconditions;
import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.HelpFormatter; import org.apache.commons.cli.HelpFormatter;
@ -30,6 +32,11 @@
import org.apache.hadoop.hdfs.tools.DiskBalancerCLI; import org.apache.hadoop.hdfs.tools.DiskBalancerCLI;
import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.NetUtils;
import java.io.PrintStream;
import java.util.Collections;
import java.util.Set;
import java.util.TreeSet;
/** /**
* Gets the current status of disk balancer command. * Gets the current status of disk balancer command.
*/ */
@ -41,9 +48,13 @@ public class QueryCommand extends Command {
* @param conf - Configuration. * @param conf - Configuration.
*/ */
public QueryCommand(Configuration conf) { public QueryCommand(Configuration conf) {
super(conf); this(conf, System.out);
}
public QueryCommand(Configuration conf, final PrintStream ps) {
super(conf, ps);
addValidCommandParameters(DiskBalancerCLI.QUERY, addValidCommandParameters(DiskBalancerCLI.QUERY,
"Queries the status of disk plan running on a given datanode."); "Queries the status of disk plan running on given datanode(s).");
addValidCommandParameters(DiskBalancerCLI.VERBOSE, addValidCommandParameters(DiskBalancerCLI.VERBOSE,
"Prints verbose results."); "Prints verbose results.");
} }
@ -56,37 +67,55 @@ public QueryCommand(Configuration conf) {
@Override @Override
public void execute(CommandLine cmd) throws Exception { public void execute(CommandLine cmd) throws Exception {
LOG.info("Executing \"query plan\" command."); LOG.info("Executing \"query plan\" command.");
TextStringBuilder result = new TextStringBuilder();
Preconditions.checkState(cmd.hasOption(DiskBalancerCLI.QUERY)); Preconditions.checkState(cmd.hasOption(DiskBalancerCLI.QUERY));
verifyCommandOptions(DiskBalancerCLI.QUERY, cmd); verifyCommandOptions(DiskBalancerCLI.QUERY, cmd);
String nodeName = cmd.getOptionValue(DiskBalancerCLI.QUERY); String nodeVal = cmd.getOptionValue(DiskBalancerCLI.QUERY);
Preconditions.checkNotNull(nodeName); if (StringUtils.isBlank(nodeVal)) {
nodeName = nodeName.trim(); String warnMsg = "The number of input nodes is 0. "
String nodeAddress = nodeName; + "Please input the valid nodes.";
throw new DiskBalancerException(warnMsg,
// if the string is not name:port format use the default port. DiskBalancerException.Result.INVALID_NODE);
if (!nodeName.matches("[^\\:]+:[0-9]{2,5}")) {
int defaultIPC = NetUtils.createSocketAddr(
getConf().getTrimmed(DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_KEY,
DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_DEFAULT)).getPort();
nodeAddress = nodeName + ":" + defaultIPC;
LOG.debug("Using default data node port : {}", nodeAddress);
} }
nodeVal = nodeVal.trim();
ClientDatanodeProtocol dataNode = getDataNodeProxy(nodeAddress); Set<String> resultSet = new TreeSet<>();
try { String[] nodes = nodeVal.split(",");
DiskBalancerWorkStatus workStatus = dataNode.queryDiskBalancerPlan(); Collections.addAll(resultSet, nodes);
System.out.printf("Plan File: %s%nPlan ID: %s%nResult: %s%n", String outputLine = String.format(
workStatus.getPlanFile(), "Get current status of the diskbalancer for DataNode(s). "
workStatus.getPlanID(), + "These DataNode(s) are parsed from '%s'.", nodeVal);
workStatus.getResult().toString()); recordOutput(result, outputLine);
for (String nodeName : resultSet) {
if (cmd.hasOption(DiskBalancerCLI.VERBOSE)) { // if the string is not name:port format use the default port.
System.out.printf("%s", workStatus.currentStateString()); String nodeAddress = nodeName;
if (!nodeName.matches("[^\\:]+:[0-9]{2,5}")) {
int defaultIPC = NetUtils.createSocketAddr(
getConf().getTrimmed(DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_KEY,
DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_DEFAULT)).getPort();
nodeAddress = nodeName + ":" + defaultIPC;
LOG.debug("Using default data node port : {}", nodeAddress);
}
ClientDatanodeProtocol dataNode = getDataNodeProxy(nodeAddress);
try {
DiskBalancerWorkStatus workStatus = dataNode.queryDiskBalancerPlan();
outputLine = String.format("DataNode: %s%nPlan File: %s%nPlan ID: %s%nResult: %s%n",
nodeAddress,
workStatus.getPlanFile(),
workStatus.getPlanID(),
workStatus.getResult().toString());
result.append(outputLine);
if (cmd.hasOption(DiskBalancerCLI.VERBOSE)) {
outputLine = String.format("%s", workStatus.currentStateString());
result.append(outputLine);
}
result.append(System.lineSeparator());
} catch (DiskBalancerException ex) {
LOG.error("Query plan failed by {}", nodeAddress, ex);
throw ex;
} }
} catch (DiskBalancerException ex) {
LOG.error("Query plan failed.", ex);
throw ex;
} }
getPrintStream().println(result);
} }
/** /**
@ -94,14 +123,14 @@ public void execute(CommandLine cmd) throws Exception {
*/ */
@Override @Override
public void printHelp() { public void printHelp() {
String header = "Query Plan queries a given data node about the " + String header = "Query Plan queries given datanode(s) about the " +
"current state of disk balancer execution.\n\n"; "current state of disk balancer execution.\n\n";
String footer = "\nQuery command retrievs the plan ID and the current " + String footer = "\nQuery command retrievs the plan ID and the current " +
"running state. "; "running state. ";
HelpFormatter helpFormatter = new HelpFormatter(); HelpFormatter helpFormatter = new HelpFormatter();
helpFormatter.printHelp("hdfs diskbalancer -query <hostname> [options]", helpFormatter.printHelp("hdfs diskbalancer -query <hostname,hostname,...> " +
" [options]",
header, DiskBalancerCLI.getQueryOptions(), footer); header, DiskBalancerCLI.getQueryOptions(), footer);
} }
} }

View File

@ -378,7 +378,7 @@ private void addQueryCommands(Options opt) {
Option query = Option.builder().longOpt(QUERY) Option query = Option.builder().longOpt(QUERY)
.hasArg() .hasArg()
.desc("Queries the disk balancer " + .desc("Queries the disk balancer " +
"status of a given datanode.") "status of given datanode(s).")
.build(); .build();
getQueryOptions().addOption(query); getQueryOptions().addOption(query);
opt.addOption(query); opt.addOption(query);
@ -387,7 +387,7 @@ private void addQueryCommands(Options opt) {
// added to global table. // added to global table.
Option verbose = Option.builder().longOpt(VERBOSE) Option verbose = Option.builder().longOpt(VERBOSE)
.desc("Prints details of the plan that is being executed " + .desc("Prints details of the plan that is being executed " +
"on the node.") "on the datanode(s).")
.build(); .build();
getQueryOptions().addOption(verbose); getQueryOptions().addOption(verbose);
} }
@ -482,7 +482,7 @@ private int dispatch(CommandLine cmd)
} }
if (cmd.hasOption(DiskBalancerCLI.QUERY)) { if (cmd.hasOption(DiskBalancerCLI.QUERY)) {
dbCmd = new QueryCommand(getConf()); dbCmd = new QueryCommand(getConf(), this.printStream);
} }
if (cmd.hasOption(DiskBalancerCLI.CANCEL)) { if (cmd.hasOption(DiskBalancerCLI.CANCEL)) {

View File

@ -86,9 +86,9 @@ So, query command can help to get the current status of execute command.
### Query ### Query
Query command gets the current status of the diskbalancer from a datanode. Query command gets the current status of the diskbalancer from specified node(s).
`hdfs diskbalancer -query nodename.mycluster.com` `hdfs diskbalancer -query nodename1.mycluster.com,nodename2.mycluster.com,...`
| COMMAND\_OPTION | Description | | COMMAND\_OPTION | Description |
|:---- |:---- | |:---- |:---- |

View File

@ -814,17 +814,44 @@ private List<String> runCommand(
return runCommandInternal(cmdLine, clusterConf); return runCommandInternal(cmdLine, clusterConf);
} }
/**
* Making sure that we can query the multiple nodes without having done a submit.
* @throws Exception
*/
@Test
public void testDiskBalancerQueryWithoutSubmitAndMultipleNodes() throws Exception {
Configuration hdfsConf = new HdfsConfiguration();
hdfsConf.setBoolean(DFSConfigKeys.DFS_DISK_BALANCER_ENABLED, true);
final int numDatanodes = 2;
File basedir = new File(GenericTestUtils.getRandomizedTempPath());
MiniDFSCluster miniDFSCluster = new MiniDFSCluster.Builder(hdfsConf, basedir)
.numDataNodes(numDatanodes).build();
try {
miniDFSCluster.waitActive();
DataNode dataNode1 = miniDFSCluster.getDataNodes().get(0);
DataNode dataNode2 = miniDFSCluster.getDataNodes().get(1);
final String queryArg = String.format("-query localhost:%d,localhost:%d", dataNode1
.getIpcPort(), dataNode2.getIpcPort());
final String cmdLine = String.format("hdfs diskbalancer %s", queryArg);
List<String> outputs = runCommand(cmdLine);
assertThat(outputs.get(1), containsString("localhost:" + dataNode1.getIpcPort()));
assertThat(outputs.get(6), containsString("localhost:" + dataNode2.getIpcPort()));
} finally {
miniDFSCluster.shutdown();
}
}
/** /**
* Making sure that we can query the node without having done a submit. * Making sure that we can query the node without having done a submit.
* @throws Exception * @throws Exception
*/ */
@Test @Test
public void testDiskBalancerQueryWithoutSubmit() throws Exception { public void testDiskBalancerQueryWithoutSubmit() throws Exception {
Configuration conf = new HdfsConfiguration(); Configuration hdfsConf = new HdfsConfiguration();
conf.setBoolean(DFSConfigKeys.DFS_DISK_BALANCER_ENABLED, true); hdfsConf.setBoolean(DFSConfigKeys.DFS_DISK_BALANCER_ENABLED, true);
final int numDatanodes = 2; final int numDatanodes = 2;
File basedir = new File(GenericTestUtils.getRandomizedTempPath()); File basedir = new File(GenericTestUtils.getRandomizedTempPath());
MiniDFSCluster miniDFSCluster = new MiniDFSCluster.Builder(conf, basedir) MiniDFSCluster miniDFSCluster = new MiniDFSCluster.Builder(hdfsConf, basedir)
.numDataNodes(numDatanodes).build(); .numDataNodes(numDatanodes).build();
try { try {
miniDFSCluster.waitActive(); miniDFSCluster.waitActive();