HDFS-11137. Ozone: SCM: Add negative tests cases for datanodeStatemachine. Contributed by Weiwei Yang.

This commit is contained in:
Anu Engineer 2017-03-03 10:29:05 -08:00 committed by Owen O'Malley
parent d55564143d
commit 753979865b
4 changed files with 193 additions and 21 deletions

View File

@ -29,7 +29,9 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
import java.util.Collection;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet;
import java.util.Map; import java.util.Map;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
@ -82,6 +84,44 @@ public final class OzoneClientUtils {
// Never constructed // Never constructed
} }
/**
* Retrieve the socket addresses of all storage container managers.
*
* @param conf
* @return A collection of SCM addresses
* @throws IllegalArgumentException If the configuration is invalid
*/
public static Collection<InetSocketAddress> getSCMAddresses(
Configuration conf) throws IllegalArgumentException {
Collection<InetSocketAddress> addresses =
new HashSet<InetSocketAddress>();
Collection<String> names =
conf.getTrimmedStringCollection(OzoneConfigKeys.OZONE_SCM_NAMES);
if (names == null || names.isEmpty()) {
throw new IllegalArgumentException(OzoneConfigKeys.OZONE_SCM_NAMES
+ " need to be a set of valid DNS names or IP addresses."
+ " Null or empty address list found.");
}
final com.google.common.base.Optional<Integer>
defaultPort = com.google.common.base.Optional.of(OzoneConfigKeys
.OZONE_SCM_DEFAULT_PORT);
for (String address : names) {
com.google.common.base.Optional<String> hostname =
OzoneClientUtils.getHostName(address);
if (!hostname.isPresent()) {
throw new IllegalArgumentException("Invalid hostname for SCM: "
+ hostname);
}
com.google.common.base.Optional<Integer> port =
OzoneClientUtils.getHostPort(address);
InetSocketAddress addr = NetUtils.createSocketAddr(hostname.get(),
port.or(defaultPort.get()));
addresses.add(addr);
}
return addresses;
}
/** /**
* Retrieve the socket address that should be used by clients to connect * Retrieve the socket address that should be used by clients to connect
* to the SCM. * to the SCM.

View File

@ -16,11 +16,9 @@
*/ */
package org.apache.hadoop.ozone.container.common.states.datanode; package org.apache.hadoop.ozone.container.common.states.datanode;
import com.google.common.base.Optional; import com.google.common.base.Strings;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.ozone.OzoneClientUtils; import org.apache.hadoop.ozone.OzoneClientUtils;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine; import org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine;
import org.apache.hadoop.ozone.container.common.statemachine.SCMConnectionManager; import org.apache.hadoop.ozone.container.common.statemachine.SCMConnectionManager;
import org.apache.hadoop.ozone.container.common.statemachine.StateContext; import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
@ -29,6 +27,7 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
import java.util.Collection;
import java.util.concurrent.Callable; import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService; import java.util.concurrent.ExecutorService;
@ -70,27 +69,24 @@ public class InitDatanodeState implements DatanodeState,
*/ */
@Override @Override
public DatanodeStateMachine.DatanodeStates call() throws Exception { public DatanodeStateMachine.DatanodeStates call() throws Exception {
String[] addresses = conf.getStrings(OzoneConfigKeys.OZONE_SCM_NAMES); Collection<InetSocketAddress> addresses = null;
final Optional<Integer> defaultPort = Optional.of(OzoneConfigKeys try {
.OZONE_SCM_DEFAULT_PORT); addresses = OzoneClientUtils.getSCMAddresses(conf);
} catch (IllegalArgumentException e) {
if(!Strings.isNullOrEmpty(e.getMessage())) {
LOG.error("Failed to get SCM addresses: " + e.getMessage());
}
return DatanodeStateMachine.DatanodeStates.SHUTDOWN;
}
if (addresses == null || addresses.length <= 0) { if (addresses == null || addresses.isEmpty()) {
LOG.error("SCM addresses need to be a set of valid DNS names " + LOG.error("Null or empty SCM address list found.");
"or IP addresses. Null or empty address list found. Aborting " +
"containers.");
return DatanodeStateMachine.DatanodeStates.SHUTDOWN; return DatanodeStateMachine.DatanodeStates.SHUTDOWN;
} } else {
for (String address : addresses) { for (InetSocketAddress addr : addresses) {
Optional<String> hostname = OzoneClientUtils.getHostName(address);
if (!hostname.isPresent()) {
LOG.error("Invalid hostname for SCM.");
return DatanodeStateMachine.DatanodeStates.SHUTDOWN;
}
Optional<Integer> port = OzoneClientUtils.getHostPort(address);
InetSocketAddress addr = NetUtils.createSocketAddr(hostname.get(),
port.or(defaultPort.get()));
connectionManager.addSCMServer(addr); connectionManager.addSCMServer(addr);
} }
}
return this.context.getState().getNextState(); return this.context.getState().getNextState();
} }

View File

@ -26,11 +26,15 @@ import org.junit.Rule;
import org.junit.rules.Timeout; import org.junit.rules.Timeout;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
import java.util.Collection;
import java.util.HashMap;
import java.util.Iterator;
import static org.apache.hadoop.ozone.OzoneConfigKeys.*; import static org.apache.hadoop.ozone.OzoneConfigKeys.*;
import static org.hamcrest.core.Is.is; import static org.hamcrest.core.Is.is;
import static org.junit.Assert.assertThat; import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
/** /**
* This test class verifies the parsing of SCM endpoint config settings. * This test class verifies the parsing of SCM endpoint config settings.
@ -217,4 +221,104 @@ public class TestOzoneClientUtils {
assertThat(addr.getHostString(), is("5.6.7.8")); assertThat(addr.getHostString(), is("5.6.7.8"));
assertThat(addr.getPort(), is(200)); assertThat(addr.getPort(), is(200));
} }
@Test
public void testGetSCMAddresses() {
final Configuration conf = new OzoneConfiguration();
Collection<InetSocketAddress> addresses = null;
InetSocketAddress addr = null;
Iterator<InetSocketAddress> it = null;
// Verify valid IP address setup
conf.setStrings(OZONE_SCM_NAMES, "1.2.3.4");
addresses = OzoneClientUtils.getSCMAddresses(conf);
assertThat(addresses.size(), is(1));
addr = addresses.iterator().next();
assertThat(addr.getHostName(), is("1.2.3.4"));
assertThat(addr.getPort(), is(OZONE_SCM_DEFAULT_PORT));
// Verify valid hostname setup
conf.setStrings(OZONE_SCM_NAMES, "scm1");
addresses = OzoneClientUtils.getSCMAddresses(conf);
assertThat(addresses.size(), is(1));
addr = addresses.iterator().next();
assertThat(addr.getHostName(), is("scm1"));
assertThat(addr.getPort(), is(OZONE_SCM_DEFAULT_PORT));
// Verify valid hostname and port
conf.setStrings(OZONE_SCM_NAMES, "scm1:1234");
addresses = OzoneClientUtils.getSCMAddresses(conf);
assertThat(addresses.size(), is(1));
addr = addresses.iterator().next();
assertThat(addr.getHostName(), is("scm1"));
assertThat(addr.getPort(), is(1234));
final HashMap<String, Integer> hostsAndPorts =
new HashMap<String, Integer>();
hostsAndPorts.put("scm1", 1234);
hostsAndPorts.put("scm2", 2345);
hostsAndPorts.put("scm3", 3456);
// Verify multiple hosts and port
conf.setStrings(OZONE_SCM_NAMES, "scm1:1234,scm2:2345,scm3:3456");
addresses = OzoneClientUtils.getSCMAddresses(conf);
assertThat(addresses.size(), is(3));
it = addresses.iterator();
HashMap<String, Integer> expected1 = new HashMap<>(hostsAndPorts);
while(it.hasNext()) {
InetSocketAddress current = it.next();
assertTrue(expected1.remove(current.getHostName(),
current.getPort()));
}
assertTrue(expected1.isEmpty());
// Verify names with spaces
conf.setStrings(OZONE_SCM_NAMES, " scm1:1234, scm2:2345 , scm3:3456 ");
addresses = OzoneClientUtils.getSCMAddresses(conf);
assertThat(addresses.size(), is(3));
it = addresses.iterator();
HashMap<String, Integer> expected2 = new HashMap<>(hostsAndPorts);
while(it.hasNext()) {
InetSocketAddress current = it.next();
assertTrue(expected2.remove(current.getHostName(),
current.getPort()));
}
assertTrue(expected2.isEmpty());
// Verify empty value
conf.setStrings(OZONE_SCM_NAMES, "");
try {
addresses = OzoneClientUtils.getSCMAddresses(conf);
fail("Empty value should cause an IllegalArgumentException");
} catch (Exception e) {
assertTrue(e instanceof IllegalArgumentException);
}
// Verify invalid hostname
conf.setStrings(OZONE_SCM_NAMES, "s..x..:1234");
try {
addresses = OzoneClientUtils.getSCMAddresses(conf);
fail("An invalid hostname should cause an IllegalArgumentException");
} catch (Exception e) {
assertTrue(e instanceof IllegalArgumentException);
}
// Verify invalid port
conf.setStrings(OZONE_SCM_NAMES, "scm:xyz");
try {
addresses = OzoneClientUtils.getSCMAddresses(conf);
fail("An invalid port should cause an IllegalArgumentException");
} catch (Exception e) {
assertTrue(e instanceof IllegalArgumentException);
}
// Verify a mixed case (valid and invalid value both appears)
conf.setStrings(OZONE_SCM_NAMES, "scm1:1234, scm:xyz");
try {
addresses = OzoneClientUtils.getSCMAddresses(conf);
fail("An invalid value should cause an IllegalArgumentException");
} catch (Exception e) {
assertTrue(e instanceof IllegalArgumentException);
}
}
} }

View File

@ -261,4 +261,36 @@ public class TestDatanodeStateMachine {
Assert.assertEquals(1, mock.getHeartbeatCount()); Assert.assertEquals(1, mock.getHeartbeatCount());
} }
} }
/**
* Test state transition with a list of invalid SCM names,
* and verify the state transits to SHUTDOWN each time.
*/
@Test
public void testDatanodeStateMachineWithInvalidSCMNames()
throws Exception {
for (String name : new String[] {
"", // Empty
"x..y", // Invalid schema
"scm:", // Missing port
"scm:xyz", // Invalid port
"scm:123456" // Port out of range
}) {
conf.setStrings(OzoneConfigKeys.OZONE_SCM_NAMES, name);
final DatanodeStateMachine stateMachine =
new DatanodeStateMachine(conf);
DatanodeStateMachine.DatanodeStates currentState =
stateMachine.getContext().getState();
Assert.assertEquals(DatanodeStateMachine.DatanodeStates.INIT,
currentState);
DatanodeState<DatanodeStateMachine.DatanodeStates> task =
stateMachine.getContext().getTask();
task.execute(executorService);
DatanodeStateMachine.DatanodeStates newState =
task.await(2, TimeUnit.SECONDS);
Assert.assertEquals(DatanodeStateMachine.DatanodeStates.SHUTDOWN,
newState);
}
}
} }