Author: stevel
Date: Sat Feb 18 12:57:49 2012
New Revision: 1245911

URL: http://svn.apache.org/viewvc?rev=1245911&view=rev
Log:
HADOOP-8046 Revert StaticMapping semantics to the existing ones, add DNS 
mapping diagnostics in progress

Modified:
    hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt
    
hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/AbstractDNSToSwitchMapping.java
    
hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/CachedDNSToSwitchMapping.java
    
hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/ScriptBasedMapping.java
    
hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/StaticMapping.java
    
hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestStaticMapping.java
    
hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestSwitchMapping.java

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt?rev=1245911&r1=1245910&r2=1245911&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt 
(original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt Sat Feb 
18 12:57:49 2012
@@ -203,6 +203,9 @@ Release 0.23.2 - UNRELEASED 
     HADOOP-8036. TestViewFsTrash assumes the user's home directory is
     2 levels deep. (Colin Patrick McCabe via eli)
 
+    HADOOP-8046 Revert StaticMapping semantics to the existing ones, add DNS
+    mapping diagnostics in progress (stevel)
+
 Release 0.23.1 - 2012-02-17 
 
   INCOMPATIBLE CHANGES

Modified: 
hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/AbstractDNSToSwitchMapping.java
URL: 
http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/AbstractDNSToSwitchMapping.java?rev=1245911&r1=1245910&r2=1245911&view=diff
==============================================================================
--- 
hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/AbstractDNSToSwitchMapping.java
 (original)
+++ 
hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/AbstractDNSToSwitchMapping.java
 Sat Feb 18 12:57:49 2012
@@ -22,6 +22,12 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
 
 /**
  * This is a base class for DNS to Switch mappings. <p/> It is not mandatory to
@@ -90,6 +96,49 @@ public abstract class AbstractDNSToSwitc
   }
 
   /**
+   * Get a copy of the map (for diagnostics)
+   * @return a clone of the map or null for none known
+   */
+  public Map<String, String> getSwitchMap() {
+    return null;
+  }
+
+  /**
+   * Generate a string listing the switch mapping implementation,
+   * the mapping for every known node and the number of nodes and
+   * unique switches known about -each entry to a separate line.
+   * @return a string that can be presented to the ops team or used in
+   * debug messages.
+   */
+  public String dumpTopology() {
+    Map<String, String> rack = getSwitchMap();
+    StringBuilder builder = new StringBuilder();
+    builder.append("Mapping: ").append(toString()).append("\n");
+    if (rack != null) {
+      builder.append("Map:\n");
+      Set<String> switches = new HashSet<String>();
+      for (Map.Entry<String, String> entry : rack.entrySet()) {
+        builder.append("  ")
+            .append(entry.getKey())
+            .append(" -> ")
+            .append(entry.getValue())
+            .append("\n");
+        switches.add(entry.getValue());
+      }
+      builder.append("Nodes: ").append(rack.size()).append("\n");
+      builder.append("Switches: ").append(switches.size()).append("\n");
+    } else {
+      builder.append("No topology information");
+    }
+    return builder.toString();
+  }
+
+  protected boolean isSingleSwitchByScriptPolicy() {
+    return conf != null
+        && conf.get(CommonConfigurationKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY) 
== null;
+  }
+
+  /**
    * Query for a {@link DNSToSwitchMapping} instance being on a single
    * switch.
    * <p/>
@@ -100,7 +149,7 @@ public abstract class AbstractDNSToSwitc
    * is not derived from this class.
    */
   public static boolean isMappingSingleSwitch(DNSToSwitchMapping mapping) {
-    return mapping instanceof AbstractDNSToSwitchMapping
+    return mapping != null && mapping instanceof AbstractDNSToSwitchMapping
         && ((AbstractDNSToSwitchMapping) mapping).isSingleSwitch();
   }
 

Modified: 
hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/CachedDNSToSwitchMapping.java
URL: 
http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/CachedDNSToSwitchMapping.java?rev=1245911&r1=1245910&r2=1245911&view=diff
==============================================================================
--- 
hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/CachedDNSToSwitchMapping.java
 (original)
+++ 
hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/CachedDNSToSwitchMapping.java
 Sat Feb 18 12:57:49 2012
@@ -18,6 +18,7 @@
 package org.apache.hadoop.net;
 
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
@@ -124,6 +125,22 @@ public class CachedDNSToSwitchMapping ex
   }
 
   /**
+   * Get the (host x switch) map.
+   * @return a copy of the cached map of hosts to rack
+   */
+  @Override
+  public Map<String, String> getSwitchMap() {
+    Map<String, String > switchMap = new HashMap<String, String>(cache);
+    return switchMap;
+  }
+
+
+  @Override
+  public String toString() {
+    return "cached switch mapping relaying to " + rawMapping;
+  }
+
+  /**
    * Delegate the switch topology query to the raw mapping, via
    * {@link 
AbstractDNSToSwitchMapping#isMappingSingleSwitch(DNSToSwitchMapping)}
    * @return true iff the raw mapper is considered single-switch.

Modified: 
hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/ScriptBasedMapping.java
URL: 
http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/ScriptBasedMapping.java?rev=1245911&r1=1245910&r2=1245911&view=diff
==============================================================================
--- 
hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/ScriptBasedMapping.java
 (original)
+++ 
hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/ScriptBasedMapping.java
 Sat Feb 18 12:57:49 2012
@@ -66,9 +66,15 @@ public final class ScriptBasedMapping ex
                      CommonConfigurationKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY 
;
   /**
    * key to the argument count that the script supports
+   * {@value}
    */
   static final String SCRIPT_ARG_COUNT_KEY =
                      
CommonConfigurationKeys.NET_TOPOLOGY_SCRIPT_NUMBER_ARGS_KEY ;
+  /**
+   * Text used in the {@link #toString()} method if there is no string
+   * {@value}
+   */
+  public static final String NO_SCRIPT = "no script";
 
   /**
    * Create an instance with the default configuration.
@@ -104,6 +110,11 @@ public final class ScriptBasedMapping ex
     return getRawMapping().getConf();
   }
 
+  @Override
+  public String toString() {
+    return "script-based mapping with " + getRawMapping().toString();
+  }
+
   /**
    * {@inheritDoc}
    * <p/>
@@ -231,7 +242,7 @@ public final class ScriptBasedMapping ex
           s.execute();
           allOutput.append(s.getOutput()).append(" ");
         } catch (Exception e) {
-          LOG.warn("Exception: ", e);
+          LOG.warn("Exception running " + s, e);
           return null;
         }
         loopCount++;
@@ -248,5 +259,10 @@ public final class ScriptBasedMapping ex
     public boolean isSingleSwitch() {
       return scriptName == null;
     }
+
+    @Override
+    public String toString() {
+      return scriptName != null ? ("script " + scriptName) : NO_SCRIPT;
+    }
   }
 }

Modified: 
hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/StaticMapping.java
URL: 
http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/StaticMapping.java?rev=1245911&r1=1245910&r2=1245911&view=diff
==============================================================================
--- 
hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/StaticMapping.java
 (original)
+++ 
hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/StaticMapping.java
 Sat Feb 18 12:57:49 2012
@@ -21,8 +21,10 @@ import org.apache.hadoop.conf.Configurat
 
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 /**
  * Implements the {@link DNSToSwitchMapping} via static mappings. Used
@@ -34,6 +36,10 @@ import java.util.Map;
  * 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.
+ *
+ * The switch mapping policy of this class is the same as for the
+ * {@link ScriptBasedMapping} -the presence of a non-empty topology script.
+ * The script itself is not used.
  */
 public class StaticMapping extends AbstractDNSToSwitchMapping  {
 
@@ -109,12 +115,30 @@ public class StaticMapping extends Abstr
   }
 
   /**
-   * Declare that this mapping is always multi-switch
+   * The switch policy of this mapping is driven by the same policy
+   * as the Scripted mapping: the presence of the script name in
+   * the configuration file
    * @return false, always
    */
   @Override
   public boolean isSingleSwitch() {
-    return false;
+    return isSingleSwitchByScriptPolicy();
+  }
+
+  /**
+   * Get a copy of the map (for diagnostics)
+   * @return a clone of the map or null for none known
+   */
+  @Override
+  public Map<String, String> getSwitchMap() {
+    synchronized (nameToRackMap) {
+      return new HashMap<String, String>(nameToRackMap);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "static mapping with single switch = " + isSingleSwitch();
   }
 
   /**

Modified: 
hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestStaticMapping.java
URL: 
http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestStaticMapping.java?rev=1245911&r1=1245910&r2=1245911&view=diff
==============================================================================
--- 
hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestStaticMapping.java
 (original)
+++ 
hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestStaticMapping.java
 Sat Feb 18 12:57:49 2012
@@ -18,22 +18,27 @@
 
 package org.apache.hadoop.net;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.junit.Assert;
 import org.junit.Test;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 
 /**
  * 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 {
+  private static final Log LOG = LogFactory.getLog(TestStaticMapping.class);
 
   /**
    * Reset the map then create a new instance of the {@link StaticMapping}
-   * class
+   * class with a null configuration
    * @return a new instance
    */
   private StaticMapping newInstance() {
@@ -41,63 +46,195 @@ public class TestStaticMapping extends A
     return new StaticMapping();
   }
 
-  @Test
-  public void testStaticIsSingleSwitch() throws Throwable {
+
+  /**
+   * Reset the map then create a new instance of the {@link StaticMapping}
+   * class with the topology script in the configuration set to
+   * the parameter
+   * @param script a (never executed) script, can be null
+   * @return a new instance
+   */
+  private StaticMapping newInstance(String script) {
     StaticMapping mapping = newInstance();
-    assertFalse("Empty maps should not be not single switch",
-                mapping.isSingleSwitch());
+    mapping.setConf(createConf(script));
+    return mapping;
   }
 
+  /**
+   * Create a configuration with a specific topology script
+   * @param script a (never executed) script, can be null
+   * @return a configuration
+   */
+  private Configuration createConf(String script) {
+    Configuration conf = new Configuration();
+    if (script != null) {
+      conf.set(CommonConfigurationKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY,
+               script);
+    } else {
+      conf.unset(CommonConfigurationKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY);
+    }
+    return conf;
+  }
+
+  private void assertSingleSwitch(DNSToSwitchMapping mapping) {
+    assertEquals("Expected a single switch mapping "
+                     + mapping,
+                 true,
+                 AbstractDNSToSwitchMapping.isMappingSingleSwitch(mapping));
+  }
+
+  private void assertMultiSwitch(DNSToSwitchMapping mapping) {
+    assertEquals("Expected a multi switch mapping "
+                     + mapping,
+                 false,
+                 AbstractDNSToSwitchMapping.isMappingSingleSwitch(mapping));
+  }
+
+  protected void assertMapSize(AbstractDNSToSwitchMapping switchMapping, int 
expectedSize) {
+    assertEquals(
+        "Expected two entries in the map " + switchMapping.dumpTopology(),
+        expectedSize, switchMapping.getSwitchMap().size());
+  }
+
+  private List<String> createQueryList() {
+    List<String> l1 = new ArrayList<String>(2);
+    l1.add("n1");
+    l1.add("unknown");
+    return l1;
+  }
 
   @Test
-  public void testCachingRelaysQueries() throws Throwable {
-    StaticMapping staticMapping = newInstance();
-    CachedDNSToSwitchMapping mapping =
-        new CachedDNSToSwitchMapping(staticMapping);
-    StaticMapping.addNodeToRack("n1", "r1");
-    assertFalse("Expected multi switch", mapping.isSingleSwitch());
+  public void testStaticIsSingleSwitchOnNullScript() throws Throwable {
+    StaticMapping mapping = newInstance(null);
+    mapping.setConf(createConf(null));
+    assertSingleSwitch(mapping);
+  }
+
+  @Test
+  public void testStaticIsMultiSwitchOnScript() throws Throwable {
+    StaticMapping mapping = newInstance("ls");
+    assertMultiSwitch(mapping);
   }
 
   @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());
+    StaticMapping.addNodeToRack("n1", "/r1");
+    List<String> queryList = createQueryList();
+    List<String> resolved = mapping.resolve(queryList);
+    assertEquals(2, resolved.size());
+    assertEquals("/r1", resolved.get(0));
+    assertEquals(NetworkTopology.DEFAULT_RACK, resolved.get(1));
+    // get the switch map and examine it
+    Map<String, String> switchMap = mapping.getSwitchMap();
+    String topology = mapping.dumpTopology();
+    LOG.info(topology);
+    assertEquals(topology, 1, switchMap.size());
+    assertEquals(topology, "/r1", switchMap.get("n1"));
   }
 
+  /**
+   * Verify that a configuration string builds a topology
+   */
   @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");
+    conf.set(StaticMapping.KEY_HADOOP_CONFIGURED_NODE_MAPPING, 
"n1=/r1,n2=/r2");
     mapping.setConf(conf);
+    //even though we have inserted elements into the list, because 
+    //it is driven by the script key in the configuration, it still
+    //thinks that it is single rack
+    assertSingleSwitch(mapping);
     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));
+    List<String> resolved = mapping.resolve(l1);
+    assertEquals(3, resolved.size());
+    assertEquals("/r1", resolved.get(0));
+    assertEquals(NetworkTopology.DEFAULT_RACK, resolved.get(1));
+    assertEquals("/r2", resolved.get(2));
+
+    Map<String, String> switchMap = mapping.getSwitchMap();
+    String topology = mapping.dumpTopology();
+    LOG.info(topology);
+    assertEquals(topology, 2, switchMap.size());
+    assertEquals(topology, "/r1", switchMap.get("n1"));
+    assertNull(topology, switchMap.get("unknown"));
+  }
+
+
+  /**
+   * Verify that if the inner mapping is single-switch, so is the cached one
+   * @throws Throwable on any problem
+   */
+  @Test
+  public void testCachingRelaysSingleSwitchQueries() throws Throwable {
+    //create a single switch map
+    StaticMapping staticMapping = newInstance(null);
+    assertSingleSwitch(staticMapping);
+    CachedDNSToSwitchMapping cachedMap =
+        new CachedDNSToSwitchMapping(staticMapping);
+    LOG.info("Mapping: " + cachedMap + "\n" + cachedMap.dumpTopology());
+    assertSingleSwitch(cachedMap);
+  }
+
+  /**
+   * Verify that if the inner mapping is multi-switch, so is the cached one
+   * @throws Throwable on any problem
+   */
+  @Test
+  public void testCachingRelaysMultiSwitchQueries() throws Throwable {
+    StaticMapping staticMapping = newInstance("top");
+    assertMultiSwitch(staticMapping);
+    CachedDNSToSwitchMapping cachedMap =
+        new CachedDNSToSwitchMapping(staticMapping);
+    LOG.info("Mapping: " + cachedMap + "\n" + cachedMap.dumpTopology());
+    assertMultiSwitch(cachedMap);
   }
 
+
+  /**
+   * This test verifies that resultion queries get relayed to the inner rack
+   * @throws Throwable on any problem
+   */
   @Test
-  public void testNullConfiguration() throws Throwable {
+  public void testCachingRelaysResolveQueries() throws Throwable {
     StaticMapping mapping = newInstance();
-    mapping.setConf(null);
-    assertFalse("Null maps are expected to be multi switch",
-                mapping.isSingleSwitch());
-    assertFalse("Expected to be multi switch",
-               AbstractDNSToSwitchMapping.isMappingSingleSwitch(mapping));
+    mapping.setConf(createConf("top"));
+    StaticMapping staticMapping = mapping;
+    CachedDNSToSwitchMapping cachedMap =
+        new CachedDNSToSwitchMapping(staticMapping);
+    assertMapSize(cachedMap, 0);
+    //add a node to the static map
+    StaticMapping.addNodeToRack("n1", "/r1");
+    //verify it is there
+    assertMapSize(staticMapping, 1);
+    //verify that the cache hasn't picked it up yet
+    assertMapSize(cachedMap, 0);
+    //now relay the query
+    cachedMap.resolve(createQueryList());
+    //and verify the cache is no longer empty
+    assertMapSize(cachedMap, 2);
+  }
+
+  /**
+   * This test verifies that resultion queries get relayed to the inner rack
+   * @throws Throwable on any problem
+   */
+  @Test
+  public void testCachingCachesNegativeEntries() throws Throwable {
+    StaticMapping staticMapping = newInstance();
+    CachedDNSToSwitchMapping cachedMap =
+        new CachedDNSToSwitchMapping(staticMapping);
+    assertMapSize(cachedMap, 0);
+    assertMapSize(staticMapping, 0);
+    List<String> resolved = cachedMap.resolve(createQueryList());
+    //and verify the cache is no longer empty while the static map is
+    assertMapSize(staticMapping, 0);
+    assertMapSize(cachedMap, 2);
   }
+
+
 }

Modified: 
hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestSwitchMapping.java
URL: 
http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestSwitchMapping.java?rev=1245911&r1=1245910&r2=1245911&view=diff
==============================================================================
--- 
hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestSwitchMapping.java
 (original)
+++ 
hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestSwitchMapping.java
 Sat Feb 18 12:57:49 2012
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.net;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -28,22 +30,87 @@ import java.util.List;
  */
 public class TestSwitchMapping extends Assert {
 
+
+  /**
+   * Verify the switch mapping query handles arbitrary DNSToSwitchMapping
+   * implementations
+   *
+   * @throws Throwable on any problem
+   */
   @Test
   public void testStandaloneClassesAssumedMultiswitch() throws Throwable {
     DNSToSwitchMapping mapping = new StandaloneSwitchMapping();
-    assertFalse("Expected to be multi switch",
+    assertFalse("Expected to be multi switch " + mapping,
                 AbstractDNSToSwitchMapping.isMappingSingleSwitch(mapping));
   }
 
 
+  /**
+   * Verify the cached mapper delegates the switch mapping query to the inner
+   * mapping, which again handles arbitrary DNSToSwitchMapping implementations
+   *
+   * @throws Throwable on any problem
+   */
   @Test
   public void testCachingRelays() throws Throwable {
     CachedDNSToSwitchMapping mapping =
         new CachedDNSToSwitchMapping(new StandaloneSwitchMapping());
-    assertFalse("Expected to be multi switch",
+    assertFalse("Expected to be multi switch " + mapping,
                 mapping.isSingleSwitch());
   }
 
+
+  /**
+   * Verify the cached mapper delegates the switch mapping query to the inner
+   * mapping, which again handles arbitrary DNSToSwitchMapping implementations
+   *
+   * @throws Throwable on any problem
+   */
+  @Test
+  public void testCachingRelaysStringOperations() throws Throwable {
+    Configuration conf = new Configuration();
+    String scriptname = "mappingscript.sh";
+    conf.set(CommonConfigurationKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY,
+             scriptname);
+    ScriptBasedMapping scriptMapping = new ScriptBasedMapping(conf);
+    assertTrue("Did not find " + scriptname + " in " + scriptMapping,
+               scriptMapping.toString().contains(scriptname));
+    CachedDNSToSwitchMapping mapping =
+        new CachedDNSToSwitchMapping(scriptMapping);
+    assertTrue("Did not find " + scriptname + " in " + mapping,
+               mapping.toString().contains(scriptname));
+  }
+
+  /**
+   * Verify the cached mapper delegates the switch mapping query to the inner
+   * mapping, which again handles arbitrary DNSToSwitchMapping implementations
+   *
+   * @throws Throwable on any problem
+   */
+  @Test
+  public void testCachingRelaysStringOperationsToNullScript() throws Throwable 
{
+    Configuration conf = new Configuration();
+    ScriptBasedMapping scriptMapping = new ScriptBasedMapping(conf);
+    assertTrue("Did not find " + ScriptBasedMapping.NO_SCRIPT
+                   + " in " + scriptMapping,
+               
scriptMapping.toString().contains(ScriptBasedMapping.NO_SCRIPT));
+    CachedDNSToSwitchMapping mapping =
+        new CachedDNSToSwitchMapping(scriptMapping);
+    assertTrue("Did not find " + ScriptBasedMapping.NO_SCRIPT
+                   + " in " + mapping,
+               mapping.toString().contains(ScriptBasedMapping.NO_SCRIPT));
+  }
+
+  @Test
+  public void testNullMapping() {
+    assertFalse(AbstractDNSToSwitchMapping.isMappingSingleSwitch(null));
+  }
+
+  /**
+   * This class does not extend the abstract switch mapping, and verifies that
+   * the switch mapping logic assumes that this is multi switch
+   */
+
   private static class StandaloneSwitchMapping implements DNSToSwitchMapping {
     @Override
     public List<String> resolve(List<String> names) {


Reply via email to