HADOOP-7777 a base class for DNSToSwitchMapping implementations

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1206515 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Steve Loughran 2011-11-26 17:45:21 +00:00
parent 4b65d10cfe
commit 783dbb4125
8 changed files with 510 additions and 91 deletions

View File

@ -0,0 +1,107 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.net;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
/**
* This is a base class for DNS to Switch mappings. <p/> It is not mandatory to
* derive {@link DNSToSwitchMapping} implementations from it, but it is strongly
* recommended, as it makes it easy for the Hadoop developers to add new methods
* to this base class that are automatically picked up by all implementations.
* <p/>
*
* This class does not extend the <code>Configured</code>
* base class, and should not be changed to do so, as it causes problems
* for subclasses. The constructor of the <code>Configured</code> calls
* the {@link #setConf(Configuration)} method, which will call into the
* subclasses before they have been fully constructed.
*
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public abstract class AbstractDNSToSwitchMapping
implements DNSToSwitchMapping, Configurable {
private Configuration conf;
/**
* Create an unconfigured instance
*/
protected AbstractDNSToSwitchMapping() {
}
/**
* Create an instance, caching the configuration file.
* This constructor does not call {@link #setConf(Configuration)}; if
* a subclass extracts information in that method, it must call it explicitly.
* @param conf the configuration
*/
protected AbstractDNSToSwitchMapping(Configuration conf) {
this.conf = conf;
}
@Override
public Configuration getConf() {
return conf;
}
@Override
public void setConf(Configuration conf) {
this.conf = conf;
}
/**
* Predicate that indicates that the switch mapping is known to be
* single-switch. The base class returns false: it assumes all mappings are
* multi-rack. Subclasses may override this with methods that are more aware
* of their topologies.
*
* <p/>
*
* This method is used when parts of Hadoop need know whether to apply
* single rack vs multi-rack policies, such as during block placement.
* Such algorithms behave differently if they are on multi-switch systems.
* </p>
*
* @return true if the mapping thinks that it is on a single switch
*/
public boolean isSingleSwitch() {
return false;
}
/**
* Query for a {@link DNSToSwitchMapping} instance being on a single
* switch.
* <p/>
* This predicate simply assumes that all mappings not derived from
* this class are multi-switch.
* @param mapping the mapping to query
* @return true if the base class says it is single switch, or the mapping
* is not derived from this class.
*/
public static boolean isMappingSingleSwitch(DNSToSwitchMapping mapping) {
return mapping instanceof AbstractDNSToSwitchMapping
&& ((AbstractDNSToSwitchMapping) mapping).isSingleSwitch();
}
}

View File

@ -34,9 +34,13 @@ import org.apache.hadoop.classification.InterfaceStability;
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class CachedDNSToSwitchMapping implements DNSToSwitchMapping {
public class CachedDNSToSwitchMapping extends AbstractDNSToSwitchMapping {
private Map<String, String> cache = new ConcurrentHashMap<String, String>();
protected DNSToSwitchMapping rawMapping;
/**
* The uncached mapping
*/
protected final DNSToSwitchMapping rawMapping;
/**
* cache a raw DNS mapping
@ -118,4 +122,14 @@ public class CachedDNSToSwitchMapping implements DNSToSwitchMapping {
return getCachedHosts(names);
}
/**
* Delegate the switch topology query to the raw mapping, via
* {@link AbstractDNSToSwitchMapping#isMappingSingleSwitch(DNSToSwitchMapping)}
* @return true iff the raw mapper is considered single-switch.
*/
@Override
public boolean isSingleSwitch() {
return isMappingSingleSwitch(rawMapping);
}
}

View File

@ -40,6 +40,12 @@ public interface DNSToSwitchMapping {
* Note the hostname/ip-address is not part of the returned path.
* The network topology of the cluster would determine the number of
* components in the network path.
* <p/>
*
* If a name cannot be resolved to a rack, the implementation
* should return {@link NetworkTopology#DEFAULT_RACK}. This
* is what the bundled implementations do, though it is not a formal requirement
*
* @param names the list of hosts to resolve (can be empty)
* @return list of resolved network paths.
* If <i>names</i> is empty, the returned list is also empty

View File

@ -32,16 +32,21 @@ import org.apache.hadoop.fs.CommonConfigurationKeys;
/**
* This class implements the {@link DNSToSwitchMapping} interface using a
* script configured via the {@link CommonConfigurationKeys#NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY}
* script configured via the
* {@link CommonConfigurationKeys#NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY} option.
* <p/>
* It contains a static class <code>RawScriptBasedMapping</code> that performs
* the work: reading the configuration parameters, executing any defined
* script, handling errors and such like. The outer
* class extends {@link CachedDNSToSwitchMapping} to cache the delegated
* queries.
* <p/>
* This DNS mapper's {@link #isSingleSwitch()} predicate returns
* true if and only if a script is defined.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public final class ScriptBasedMapping extends CachedDNSToSwitchMapping
implements Configurable
{
public ScriptBasedMapping() {
super(new RawScriptBasedMapping());
}
public final class ScriptBasedMapping extends CachedDNSToSwitchMapping {
/**
* Minimum number of arguments: {@value}
@ -65,6 +70,18 @@ implements Configurable
static final String SCRIPT_ARG_COUNT_KEY =
CommonConfigurationKeys.NET_TOPOLOGY_SCRIPT_NUMBER_ARGS_KEY ;
/**
* Create an instance with the default configuration.
* </p>
* Calling {@link #setConf(Configuration)} will trigger a
* re-evaluation of the configuration settings and so be used to
* set up the mapping script.
*
*/
public ScriptBasedMapping() {
super(new RawScriptBasedMapping());
}
/**
* Create an instance from the given configuration
* @param conf configuration
@ -74,14 +91,31 @@ implements Configurable
setConf(conf);
}
@Override
public Configuration getConf() {
return ((RawScriptBasedMapping)rawMapping).getConf();
/**
* Get the cached mapping and convert it to its real type
* @return the inner raw script mapping.
*/
private RawScriptBasedMapping getRawMapping() {
return (RawScriptBasedMapping)rawMapping;
}
@Override
public Configuration getConf() {
return getRawMapping().getConf();
}
/**
* {@inheritDoc}
* <p/>
* This will get called in the superclass constructor, so a check is needed
* to ensure that the raw mapping is defined before trying to relaying a null
* configuration.
* @param conf
*/
@Override
public void setConf(Configuration conf) {
((RawScriptBasedMapping)rawMapping).setConf(conf);
super.setConf(conf);
getRawMapping().setConf(conf);
}
/**
@ -89,29 +123,26 @@ implements Configurable
* by the superclass {@link CachedDNSToSwitchMapping}
*/
private static final class RawScriptBasedMapping
implements DNSToSwitchMapping {
extends AbstractDNSToSwitchMapping {
private String scriptName;
private Configuration conf;
private int maxArgs; //max hostnames per call of the script
private static Log LOG =
private static final Log LOG =
LogFactory.getLog(ScriptBasedMapping.class);
/**
* Set the configuration and
* @param conf extract the configuration parameters of interest
* Set the configuration and extract the configuration parameters of interest
* @param conf the new configuration
*/
@Override
public void setConf (Configuration conf) {
this.scriptName = conf.get(SCRIPT_FILENAME_KEY);
this.maxArgs = conf.getInt(SCRIPT_ARG_COUNT_KEY, DEFAULT_ARG_COUNT);
this.conf = conf;
}
/**
* Get the configuration
* @return the configuration
*/
public Configuration getConf () {
return conf;
super.setConf(conf);
if (conf != null) {
scriptName = conf.get(SCRIPT_FILENAME_KEY);
maxArgs = conf.getInt(SCRIPT_ARG_COUNT_KEY, DEFAULT_ARG_COUNT);
} else {
scriptName = null;
maxArgs = 0;
}
}
/**
@ -122,42 +153,42 @@ implements Configurable
@Override
public List<String> resolve(List<String> names) {
List <String> m = new ArrayList<String>(names.size());
if (names.isEmpty()) {
return m;
}
List<String> m = new ArrayList<String>(names.size());
if (scriptName == null) {
for (int i = 0; i < names.size(); i++) {
m.add(NetworkTopology.DEFAULT_RACK);
if (names.isEmpty()) {
return m;
}
return m;
}
String output = runResolveCommand(names);
if (output != null) {
StringTokenizer allSwitchInfo = new StringTokenizer(output);
while (allSwitchInfo.hasMoreTokens()) {
String switchInfo = allSwitchInfo.nextToken();
m.add(switchInfo);
if (scriptName == null) {
for (String name : names) {
m.add(NetworkTopology.DEFAULT_RACK);
}
return m;
}
if (m.size() != names.size()) {
// invalid number of entries returned by the script
LOG.error("Script " + scriptName + " returned "
+ Integer.toString(m.size()) + " values when "
+ Integer.toString(names.size()) + " were expected.");
String output = runResolveCommand(names);
if (output != null) {
StringTokenizer allSwitchInfo = new StringTokenizer(output);
while (allSwitchInfo.hasMoreTokens()) {
String switchInfo = allSwitchInfo.nextToken();
m.add(switchInfo);
}
if (m.size() != names.size()) {
// invalid number of entries returned by the script
LOG.error("Script " + scriptName + " returned "
+ Integer.toString(m.size()) + " values when "
+ Integer.toString(names.size()) + " were expected.");
return null;
}
} else {
// an error occurred. return null to signify this.
// (exn was already logged in runResolveCommand)
return null;
}
} else {
// an error occurred. return null to signify this.
// (exn was already logged in runResolveCommand)
return null;
return m;
}
return m;
}
/**
* Build and execute the resolution command. The command is
@ -195,10 +226,10 @@ implements Configurable
dir = new File(userDir);
}
ShellCommandExecutor s = new ShellCommandExecutor(
cmdList.toArray(new String[0]), dir);
cmdList.toArray(new String[cmdList.size()]), dir);
try {
s.execute();
allOutput.append(s.getOutput() + " ");
allOutput.append(s.getOutput()).append(" ");
} catch (Exception e) {
LOG.warn("Exception: ", e);
return null;
@ -207,5 +238,15 @@ implements Configurable
}
return allOutput.toString();
}
/**
* Declare that the mapper is single-switched if a script was not named
* in the configuration.
* @return true iff there is no script
*/
@Override
public boolean isSingleSwitch() {
return scriptName == null;
}
}
}

View File

@ -17,34 +17,80 @@
*/
package org.apache.hadoop.net;
import java.util.*;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
/**
* Implements the {@link DNSToSwitchMapping} via static mappings. Used
* in testcases that simulate racks.
* in testcases that simulate racks, and in the
* {@link org.apache.hadoop.hdfs.MiniDFSCluster}
*
* A shared, static mapping is used; to reset it call {@link #resetMap()}.
*
* When an instance of the class has its {@link #setConf(Configuration)}
* method called, nodes listed in the configuration will be added to the map.
* These do not get removed when the instance is garbage collected.
*/
public class StaticMapping extends Configured implements DNSToSwitchMapping {
public void setconf(Configuration conf) {
String[] mappings = conf.getStrings("hadoop.configured.node.mapping");
if (mappings != null) {
for (int i = 0; i < mappings.length; i++) {
String str = mappings[i];
String host = str.substring(0, str.indexOf('='));
String rack = str.substring(str.indexOf('=') + 1);
addNodeToRack(host, rack);
public class StaticMapping extends AbstractDNSToSwitchMapping {
/**
* key to define the node mapping as a comma-delimited list of host=rack
* mappings, e.g. <code>host1=r1,host2=r1,host3=r2</code>.
* </p>
* <b>Important: </b>spaces not trimmed and are considered significant.
*/
public static final String KEY_HADOOP_CONFIGURED_NODE_MAPPING =
"hadoop.configured.node.mapping";
/**
* Configure the mapping by extracting any mappings defined in the
* {@link #KEY_HADOOP_CONFIGURED_NODE_MAPPING} field
* @param conf new configuration
*/
@Override
public void setConf(Configuration conf) {
super.setConf(conf);
if (conf != null) {
String[] mappings = conf.getStrings(KEY_HADOOP_CONFIGURED_NODE_MAPPING);
if (mappings != null) {
for (String str : mappings) {
String host = str.substring(0, str.indexOf('='));
String rack = str.substring(str.indexOf('=') + 1);
addNodeToRack(host, rack);
}
}
}
}
/* Only one instance per JVM */
private static Map<String, String> nameToRackMap = new HashMap<String, String>();
static synchronized public void addNodeToRack(String name, String rackId) {
nameToRackMap.put(name, rackId);
/**
* retained lower case setter for compatibility reasons; relays to
* {@link #setConf(Configuration)}
* @param conf new configuration
*/
public void setconf(Configuration conf) {
setConf(conf);
}
/* Only one instance per JVM */
private static final Map<String, String> nameToRackMap = new HashMap<String, String>();
/**
* Add a node to the static map. The moment any entry is added to the map,
* the map goes multi-rack.
* @param name node name
* @param rackId rack ID
*/
public static void addNodeToRack(String name, String rackId) {
synchronized (nameToRackMap) {
nameToRackMap.put(name, rackId);
}
}
@Override
public List<String> resolve(List<String> names) {
List<String> m = new ArrayList<String>();
synchronized (nameToRackMap) {
@ -59,4 +105,24 @@ public class StaticMapping extends Configured implements DNSToSwitchMapping {
return m;
}
}
/**
* This mapping is only single switch if the map is empty
* @return the current switching status
*/
@Override
public boolean isSingleSwitch() {
synchronized (nameToRackMap) {
return nameToRackMap.isEmpty();
}
}
/**
* Clear the map and revert to being a single switch
*/
public static void resetMap() {
synchronized (nameToRackMap) {
nameToRackMap.clear();
}
}
}

View File

@ -23,30 +23,59 @@ import java.util.List;
import org.apache.hadoop.conf.Configuration;
import junit.framework.TestCase;
import org.junit.Test;
public class TestScriptBasedMapping extends TestCase {
private ScriptBasedMapping mapping;
private Configuration conf;
private List<String> names;
public TestScriptBasedMapping() {
mapping = new ScriptBasedMapping();
conf = new Configuration();
conf.setInt(ScriptBasedMapping.SCRIPT_ARG_COUNT_KEY,
ScriptBasedMapping.MIN_ALLOWABLE_ARGS - 1);
conf.set(ScriptBasedMapping.SCRIPT_FILENAME_KEY, "any-filename");
mapping.setConf(conf);
}
@Test
public void testNoArgsMeansNoResult() {
names = new ArrayList<String>();
Configuration conf = new Configuration();
conf.setInt(ScriptBasedMapping.SCRIPT_ARG_COUNT_KEY,
ScriptBasedMapping.MIN_ALLOWABLE_ARGS - 1);
conf.set(ScriptBasedMapping.SCRIPT_FILENAME_KEY, "any-filename");
conf.set(ScriptBasedMapping.SCRIPT_FILENAME_KEY, "any-filename");
ScriptBasedMapping mapping = createMapping(conf);
List<String> names = new ArrayList<String>();
names.add("some.machine.name");
names.add("other.machine.name");
List<String> result = mapping.resolve(names);
assertNull(result);
assertNull("Expected an empty list", result);
}
@Test
public void testNoFilenameMeansSingleSwitch() throws Throwable {
Configuration conf = new Configuration();
ScriptBasedMapping mapping = createMapping(conf);
assertTrue("Expected to be single switch", mapping.isSingleSwitch());
assertTrue("Expected to be single switch",
AbstractDNSToSwitchMapping.isMappingSingleSwitch(mapping));
}
@Test
public void testFilenameMeansMultiSwitch() throws Throwable {
Configuration conf = new Configuration();
conf.set(ScriptBasedMapping.SCRIPT_FILENAME_KEY, "any-filename");
ScriptBasedMapping mapping = createMapping(conf);
assertFalse("Expected to be multi switch", mapping.isSingleSwitch());
mapping.setConf(new Configuration());
assertTrue("Expected to be single switch", mapping.isSingleSwitch());
}
@Test
public void testNullConfig() throws Throwable {
ScriptBasedMapping mapping = createMapping(null);
assertTrue("Expected to be single switch", mapping.isSingleSwitch());
}
private ScriptBasedMapping createMapping(Configuration conf) {
ScriptBasedMapping mapping = new ScriptBasedMapping();
mapping.setConf(conf);
return mapping;
}
}

View File

@ -0,0 +1,103 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.net;
import org.apache.hadoop.conf.Configuration;
import org.junit.Assert;
import org.junit.Test;
import java.util.ArrayList;
import java.util.List;
/**
* Test the static mapping class.
* Because the map is actually static, this map needs to be reset for every test
*/
public class TestStaticMapping extends Assert {
/**
* Reset the map then create a new instance of the {@link StaticMapping}
* class
* @return a new instance
*/
private StaticMapping newInstance() {
StaticMapping.resetMap();
return new StaticMapping();
}
@Test
public void testStaticIsSingleSwitch() throws Throwable {
StaticMapping mapping = newInstance();
assertTrue("Empty maps are not single switch", mapping.isSingleSwitch());
}
@Test
public void testCachingRelaysQueries() throws Throwable {
StaticMapping staticMapping = newInstance();
CachedDNSToSwitchMapping mapping =
new CachedDNSToSwitchMapping(staticMapping);
assertTrue("Expected single switch", mapping.isSingleSwitch());
StaticMapping.addNodeToRack("n1", "r1");
assertFalse("Expected to be multi switch",
mapping.isSingleSwitch());
}
@Test
public void testAddResolveNodes() throws Throwable {
StaticMapping mapping = newInstance();
StaticMapping.addNodeToRack("n1", "r1");
List<String> l1 = new ArrayList<String>(2);
l1.add("n1");
l1.add("unknown");
List<String> mappings = mapping.resolve(l1);
assertEquals(2, mappings.size());
assertEquals("r1", mappings.get(0));
assertEquals(NetworkTopology.DEFAULT_RACK, mappings.get(1));
assertFalse("Mapping is still single switch", mapping.isSingleSwitch());
}
@Test
public void testReadNodesFromConfig() throws Throwable {
StaticMapping mapping = newInstance();
Configuration conf = new Configuration();
conf.set(StaticMapping.KEY_HADOOP_CONFIGURED_NODE_MAPPING, "n1=r1,n2=r2");
mapping.setConf(conf);
List<String> l1 = new ArrayList<String>(3);
l1.add("n1");
l1.add("unknown");
l1.add("n2");
List<String> mappings = mapping.resolve(l1);
assertEquals(3, mappings.size());
assertEquals("r1", mappings.get(0));
assertEquals(NetworkTopology.DEFAULT_RACK, mappings.get(1));
assertEquals("r2", mappings.get(2));
assertFalse("Expected to be multi switch",
AbstractDNSToSwitchMapping.isMappingSingleSwitch(mapping));
}
@Test
public void testNullConfiguration() throws Throwable {
StaticMapping mapping = newInstance();
mapping.setConf(null);
assertTrue("Null maps is not single switch", mapping.isSingleSwitch());
assertTrue("Expected to be single switch",
AbstractDNSToSwitchMapping.isMappingSingleSwitch(mapping));
}
}

View File

@ -0,0 +1,53 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.net;
import org.junit.Assert;
import org.junit.Test;
import java.util.List;
/**
* Test some other details of the switch mapping
*/
public class TestSwitchMapping extends Assert {
@Test
public void testStandaloneClassesAssumedMultiswitch() throws Throwable {
DNSToSwitchMapping mapping = new StandaloneSwitchMapping();
assertFalse("Expected to be multi switch",
AbstractDNSToSwitchMapping.isMappingSingleSwitch(mapping));
}
@Test
public void testCachingRelays() throws Throwable {
CachedDNSToSwitchMapping mapping =
new CachedDNSToSwitchMapping(new StandaloneSwitchMapping());
assertFalse("Expected to be multi switch",
mapping.isSingleSwitch());
}
private static class StandaloneSwitchMapping implements DNSToSwitchMapping {
@Override
public List<String> resolve(List<String> names) {
return names;
}
}
}