HDFS-14298. Improve log messages of ECTopologyVerifier. Contributed by Kitti Nanasi.

This commit is contained in:
Surendra Singh Lilhore 2019-02-23 01:08:15 +05:30
parent ed13cf8406
commit 7d3b567194
3 changed files with 128 additions and 128 deletions

View File

@ -93,18 +93,19 @@ private static ECTopologyVerifierResult verifyECWithTopology(
final int numOfRacks, final int numOfDataNodes, String readablePolicies) {
String resultMessage;
if (numOfDataNodes < minDN) {
resultMessage = String.format("The number of DataNodes (%d) is less " +
"than the minimum required number of DataNodes (%d) for the " +
"erasure coding policies: %s", numOfDataNodes, minDN,
readablePolicies);
resultMessage = String.format("%d DataNodes are required for " +
"the erasure coding policies: %s. " +
"The number of DataNodes is only %d.",
minDN, readablePolicies, numOfDataNodes);
LOG.debug(resultMessage);
return new ECTopologyVerifierResult(false, resultMessage);
}
if (numOfRacks < minRack) {
resultMessage = String.format("The number of racks (%d) is less than " +
"the minimum required number of racks (%d) for the erasure " +
"coding policies: %s", numOfRacks, minRack, readablePolicies);
resultMessage = String.format("%d racks are required for " +
"the erasure coding policies: %s. " +
"The number of racks is only %d.",
minRack, readablePolicies, numOfRacks);
LOG.debug(resultMessage);
return new ECTopologyVerifierResult(false, resultMessage);
}

View File

@ -1032,8 +1032,8 @@ private void verifyEcClusterSetupVerifyResult(MBeanServer mbs)
assertFalse("Test cluster does not support all enabled " +
"erasure coding policies.", isSupported);
assertTrue(resultMessage.contains("The number of racks"));
assertTrue(resultMessage.contains("is less than the minimum required " +
"number of racks"));
assertTrue(resultMessage.contains("3 racks are required for " +
"the erasure coding policies: RS-6-3-1024k. " +
"The number of racks is only 1."));
}
}

View File

@ -53,6 +53,19 @@ public class TestECAdmin {
private static final PrintStream OLD_OUT = System.out;
private static final PrintStream OLD_ERR = System.err;
private final static String RS_3_2 =
SystemErasureCodingPolicies.getByID(
SystemErasureCodingPolicies.RS_3_2_POLICY_ID).getName();
private final static String RS_6_3 =
SystemErasureCodingPolicies.getByID(
SystemErasureCodingPolicies.RS_6_3_POLICY_ID).getName();
private final static String RS_10_4 =
SystemErasureCodingPolicies.getByID(
SystemErasureCodingPolicies.RS_10_4_POLICY_ID).getName();
private final static String XOR_2_1 =
SystemErasureCodingPolicies.getByID(
SystemErasureCodingPolicies.XOR_2_1_POLICY_ID).getName();
@Rule
public Timeout globalTimeout =
new Timeout(300000, TimeUnit.MILLISECONDS);
@ -82,99 +95,77 @@ public void tearDown() throws Exception {
@Test
public void testRS63MinDN() throws Exception {
cluster = DFSTestUtil.setupCluster(conf, 6, 3, 0);
String[] args = {"-verifyClusterSetup"};
final int ret = admin.run(args);
LOG.info("Command stdout: {}", out.toString());
LOG.info("Command stderr: {}", err.toString());
final int numDataNodes = 6;
final int numRacks = 3;
final int expectedNumDataNodes = 9;
cluster = DFSTestUtil.setupCluster(conf, numDataNodes, numRacks, 0);
int ret = runCommandWithParams("-verifyClusterSetup");
assertEquals("Return value of the command is not successful", 2, ret);
assertTrue("Result of cluster topology verify " +
"should be logged correctly", out.toString()
.contains("less than the minimum required number of DataNodes"));
assertTrue("Error output should be empty", err.toString().isEmpty());
assertNotEnoughDataNodesMessage(RS_6_3, numDataNodes, expectedNumDataNodes);
}
@Test
public void testRS104MinRacks() throws Exception {
cluster = DFSTestUtil.setupCluster(conf, 15, 3, 0);
cluster.getFileSystem().enableErasureCodingPolicy(
SystemErasureCodingPolicies
.getByID(SystemErasureCodingPolicies.RS_10_4_POLICY_ID).getName());
String[] args = {"-verifyClusterSetup"};
final int ret = admin.run(args);
LOG.info("Command stdout: {}", out.toString());
LOG.info("Command stderr: {}", err.toString());
final String testPolicy = RS_10_4;
final int numDataNodes = 15;
final int numRacks = 3;
final int expectedNumRacks = 4;
cluster = DFSTestUtil.setupCluster(conf, numDataNodes, numRacks, 0);
cluster.getFileSystem().disableErasureCodingPolicy(RS_6_3);
cluster.getFileSystem().enableErasureCodingPolicy(testPolicy);
int ret = runCommandWithParams("-verifyClusterSetup");
assertEquals("Return value of the command is not successful", 2, ret);
assertTrue("Result of cluster topology verify " +
"should be logged correctly", out.toString()
.contains("less than the minimum required number of racks"));
assertTrue("Error output should be empty", err.toString().isEmpty());
assertNotEnoughRacksMessage(testPolicy, numRacks, expectedNumRacks);
}
@Test
public void testXOR21MinRacks() throws Exception {
cluster = DFSTestUtil.setupCluster(conf, 5, 2, 0);
cluster.getFileSystem().disableErasureCodingPolicy(
SystemErasureCodingPolicies
.getByID(SystemErasureCodingPolicies.RS_6_3_POLICY_ID).getName());
cluster.getFileSystem().enableErasureCodingPolicy(
SystemErasureCodingPolicies
.getByID(SystemErasureCodingPolicies.XOR_2_1_POLICY_ID).getName());
String[] args = {"-verifyClusterSetup"};
final int ret = admin.run(args);
LOG.info("Command stdout: {}", out.toString());
LOG.info("Command stderr: {}", err.toString());
final String testPolicy = XOR_2_1;
final int numDataNodes = 5;
final int numRacks = 2;
final int expectedNumRacks = 3;
cluster = DFSTestUtil.setupCluster(conf, numDataNodes, numRacks, 0);
cluster.getFileSystem().disableErasureCodingPolicy(RS_6_3);
cluster.getFileSystem().enableErasureCodingPolicy(testPolicy);
int ret = runCommandWithParams("-verifyClusterSetup");
assertEquals("Return value of the command is not successful", 2, ret);
assertTrue("Result of cluster topology verify " +
"should be logged correctly", out.toString()
.contains("less than the minimum required number of racks"));
assertTrue("Error output should be empty", err.toString().isEmpty());
assertNotEnoughRacksMessage(testPolicy, numRacks, expectedNumRacks);
}
@Test
public void testRS32MinRacks() throws Exception {
cluster = DFSTestUtil.setupCluster(conf, 5, 2, 0);
cluster.getFileSystem().disableErasureCodingPolicy(
SystemErasureCodingPolicies
.getByID(SystemErasureCodingPolicies.RS_6_3_POLICY_ID).getName());
cluster.getFileSystem().enableErasureCodingPolicy(
SystemErasureCodingPolicies
.getByID(SystemErasureCodingPolicies.RS_3_2_POLICY_ID).getName());
String[] args = {"-verifyClusterSetup"};
final int ret = admin.run(args);
LOG.info("Command stdout: {}", out.toString());
LOG.info("Command stderr: {}", err.toString());
final String testPolicy = RS_3_2;
final int numDataNodes = 5;
final int numRacks = 2;
final int expectedNumRacks = 3;
cluster = DFSTestUtil.setupCluster(conf, numDataNodes, numRacks, 0);
cluster.getFileSystem().disableErasureCodingPolicy(RS_6_3);
cluster.getFileSystem().enableErasureCodingPolicy(testPolicy);
int ret = runCommandWithParams("-verifyClusterSetup");
assertEquals("Return value of the command is not successful", 2, ret);
assertTrue("Result of cluster topology verify " +
"should be logged correctly", out.toString()
.contains("less than the minimum required number of racks"));
assertTrue("Error output should be empty", err.toString().isEmpty());
assertNotEnoughRacksMessage(testPolicy, numRacks, expectedNumRacks);
}
@Test
public void testRS63Good() throws Exception {
cluster = DFSTestUtil.setupCluster(conf, 9, 3, 0);
String[] args = {"-verifyClusterSetup"};
final int ret = admin.run(args);
LOG.info("Command stdout: {}", out.toString());
LOG.info("Command stderr: {}", err.toString());
int ret = runCommandWithParams("-verifyClusterSetup");
assertEquals("Return value of the command is successful", 0, ret);
assertTrue("Result of cluster topology verify " +
"should be logged correctly", out.toString().contains(
"The cluster setup can support EC policies: RS-6-3-1024k"));
"The cluster setup can support EC policies: " + RS_6_3));
assertTrue("Error output should be empty", err.toString().isEmpty());
}
@Test
public void testNoECEnabled() throws Exception {
cluster = DFSTestUtil.setupCluster(conf, 9, 3, 0);
cluster.getFileSystem().disableErasureCodingPolicy(
SystemErasureCodingPolicies
.getByID(SystemErasureCodingPolicies.RS_6_3_POLICY_ID).getName());
String[] args = {"-verifyClusterSetup"};
final int ret = admin.run(args);
LOG.info("Command stdout: {}", out.toString());
LOG.info("Command stderr: {}", err.toString());
cluster.getFileSystem().disableErasureCodingPolicy(RS_6_3);
int ret = runCommandWithParams("-verifyClusterSetup");
assertEquals("Return value of the command is successful", 0, ret);
assertTrue("Result of cluster topology verify " +
"should be logged correctly",
@ -184,40 +175,38 @@ public void testNoECEnabled() throws Exception {
@Test
public void testUnsuccessfulEnablePolicyMessage() throws Exception {
cluster = DFSTestUtil.setupCluster(conf, 5, 2, 0);
cluster.getFileSystem().disableErasureCodingPolicy(
SystemErasureCodingPolicies
.getByID(SystemErasureCodingPolicies.RS_6_3_POLICY_ID).getName());
String[] args = {"-enablePolicy", "-policy", "RS-3-2-1024k"};
final String testPolicy = RS_3_2;
final int numDataNodes = 5;
final int numRacks = 2;
cluster = DFSTestUtil.setupCluster(conf, numDataNodes, numRacks, 0);
cluster.getFileSystem().disableErasureCodingPolicy(RS_6_3);
final int ret = runCommandWithParams("-enablePolicy", "-policy",
testPolicy);
final int ret = admin.run(args);
LOG.info("Command stdout: {}", out.toString());
LOG.info("Command stderr: {}", err.toString());
assertEquals("Return value of the command is successful", 0, ret);
assertTrue("Enabling policy should be logged", out.toString()
.contains("Erasure coding policy RS-3-2-1024k is enabled"));
.contains("Erasure coding policy " + testPolicy + " is enabled"));
assertTrue("Warning about cluster topology should be printed",
err.toString().contains("Warning: The cluster setup does not support " +
"EC policy RS-3-2-1024k. Reason:"));
"EC policy " + testPolicy + ". Reason:"));
assertTrue("Warning about cluster topology should be printed",
err.toString()
.contains("less than the minimum required number of racks"));
.contains(" racks are required for the erasure coding policies: " +
testPolicy));
}
@Test
public void testSuccessfulEnablePolicyMessage() throws Exception {
final String testPolicy = RS_3_2;
cluster = DFSTestUtil.setupCluster(conf, 5, 3, 0);
cluster.getFileSystem().disableErasureCodingPolicy(
SystemErasureCodingPolicies
.getByID(SystemErasureCodingPolicies.RS_6_3_POLICY_ID).getName());
String[] args = {"-enablePolicy", "-policy", "RS-3-2-1024k"};
cluster.getFileSystem().disableErasureCodingPolicy(RS_6_3);
final int ret = runCommandWithParams("-enablePolicy", "-policy",
testPolicy);
final int ret = admin.run(args);
LOG.info("Command stdout: {}", out.toString());
LOG.info("Command stderr: {}", err.toString());
assertEquals("Return value of the command is successful", 0, ret);
assertTrue("Enabling policy should be logged", out.toString()
.contains("Erasure coding policy RS-3-2-1024k is enabled"));
.contains("Erasure coding policy " + testPolicy + " is enabled"));
assertFalse("Warning about cluster topology should not be printed",
out.toString().contains("Warning: The cluster setup does not support"));
assertTrue("Error output should be empty", err.toString().isEmpty());
@ -226,14 +215,10 @@ public void testSuccessfulEnablePolicyMessage() throws Exception {
@Test
public void testEnableNonExistentPolicyMessage() throws Exception {
cluster = DFSTestUtil.setupCluster(conf, 5, 3, 0);
cluster.getFileSystem().disableErasureCodingPolicy(
SystemErasureCodingPolicies
.getByID(SystemErasureCodingPolicies.RS_6_3_POLICY_ID).getName());
String[] args = {"-enablePolicy", "-policy", "NonExistentPolicy"};
cluster.getFileSystem().disableErasureCodingPolicy(RS_6_3);
final int ret = runCommandWithParams("-enablePolicy", "-policy",
"NonExistentPolicy");
final int ret = admin.run(args);
LOG.info("Command stdout: {}", out.toString());
LOG.info("Command stderr: {}", err.toString());
assertEquals("Return value of the command is unsuccessful", 2, ret);
assertFalse("Enabling policy should not be logged when " +
"it was unsuccessful", out.toString().contains("is enabled"));
@ -244,48 +229,31 @@ public void testEnableNonExistentPolicyMessage() throws Exception {
@Test
public void testVerifyClusterSetupWithGivenPolicies() throws Exception {
cluster = DFSTestUtil.setupCluster(conf, 5, 2, 0);
final int numDataNodes = 5;
final int numRacks = 2;
cluster = DFSTestUtil.setupCluster(conf, numDataNodes, numRacks, 0);
String[] args = new String[]{"-verifyClusterSetup", "-policy",
"RS-3-2-1024k"};
int ret = admin.run(args);
LOG.info("Command stdout: {}", out.toString());
LOG.info("Command stderr: {}", err.toString());
int ret = runCommandWithParams("-verifyClusterSetup", "-policy", RS_3_2);
assertEquals("Return value of the command is not successful", 2, ret);
assertTrue("Result of cluster topology verify " +
"should be logged correctly", out.toString()
.contains("less than the minimum required number of racks (3) " +
"for the erasure coding policies: RS-3-2-1024k"));
assertTrue("Error output should be empty", err.toString().isEmpty());
assertNotEnoughRacksMessage(RS_3_2, numRacks, 3);
resetOutputs();
args = new String[]{"-verifyClusterSetup", "-policy",
"RS-10-4-1024k", "RS-3-2-1024k"};
ret = admin.run(args);
LOG.info("Command stdout: {}", out.toString());
LOG.info("Command stderr: {}", err.toString());
ret = runCommandWithParams("-verifyClusterSetup", "-policy",
RS_10_4, RS_3_2);
assertEquals("Return value of the command is not successful", 2, ret);
assertTrue("Result of cluster topology verify " +
"should be logged correctly", out.toString()
.contains(
"for the erasure coding policies: RS-10-4-1024k, RS-3-2-1024k"));
assertTrue("Error output should be empty", err.toString().isEmpty());
assertNotEnoughDataNodesMessage(RS_10_4 + ", " + RS_3_2,
numDataNodes, 14);
resetOutputs();
args = new String[]{"-verifyClusterSetup", "-policy", "invalidPolicy"};
ret = admin.run(args);
LOG.info("Command stdout: {}", out.toString());
LOG.info("Command stderr: {}", err.toString());
ret = runCommandWithParams("-verifyClusterSetup", "-policy",
"invalidPolicy");
assertEquals("Return value of the command is not successful", -1, ret);
assertTrue("Error message should be logged", err.toString()
.contains("The given erasure coding policy invalidPolicy " +
"does not exist."));
resetOutputs();
args = new String[]{"-verifyClusterSetup", "-policy"};
ret = admin.run(args);
LOG.info("Command stdout: {}", out.toString());
LOG.info("Command stderr: {}", err.toString());
ret = runCommandWithParams("-verifyClusterSetup", "-policy");
assertEquals("Return value of the command is not successful", -1, ret);
assertTrue("Error message should be logged", err.toString()
.contains("NotEnoughArgumentsException: Not enough arguments: " +
@ -296,4 +264,35 @@ private void resetOutputs() {
out.reset();
err.reset();
}
private void assertNotEnoughDataNodesMessage(String policy,
int numDataNodes,
int expectedNumDataNodes) {
assertTrue("Result of cluster topology verify " +
"should be logged correctly", out.toString()
.contains(expectedNumDataNodes + " DataNodes are required " +
"for the erasure coding policies: " +
policy + ". The number of DataNodes is only " + numDataNodes));
assertTrue("Error output should be empty",
err.toString().isEmpty());
}
private void assertNotEnoughRacksMessage(String policy,
int numRacks,
int expectedNumRacks) {
assertTrue("Result of cluster topology verify " +
"should be logged correctly", out.toString()
.contains(expectedNumRacks + " racks are required for " +
"the erasure coding policies: " +
policy + ". The number of racks is only " + numRacks));
assertTrue("Error output should be empty",
err.toString().isEmpty());
}
private int runCommandWithParams(String... args) throws Exception{
final int ret = admin.run(args);
LOG.info("Command stdout: {}", out.toString());
LOG.info("Command stderr: {}", err.toString());
return ret;
}
}