Browse Source

Merge trunk into HA branch.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-1623@1291400 13f79535-47bb-0310-9956-ffa450edef68
Aaron Myers 13 years ago
parent
commit
f0db867292
17 changed files with 789 additions and 73 deletions
  1. 14 2
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HardLink.java
  3. 41 22
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSourceAdapter.java
  4. 50 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/AbstractDNSToSwitchMapping.java
  5. 17 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/CachedDNSToSwitchMapping.java
  6. 17 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/ScriptBasedMapping.java
  7. 1 1
      hadoop-common-project/hadoop-common/src/main/packages/hadoop-setup-conf.sh
  8. 26 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/StaticMapping.java
  9. 171 34
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestStaticMapping.java
  10. 69 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestSwitchMapping.java
  11. 4 1
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  12. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
  13. 10 5
      hadoop-mapreduce-project/CHANGES.txt
  14. 123 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/service/BreakableService.java
  15. 49 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/service/ServiceAssert.java
  16. 194 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/service/TestServiceLifecycle.java
  17. 1 1
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml

+ 14 - 2
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -174,7 +174,6 @@ Release 0.23.2 - UNRELEASED
     (sharad, todd via todd)
 
   BUG FIXES
-    HADOOP-8054 NPE with FilterFileSystem (Daryn Sharp via bobby)
 
     HADOOP-8042  When copying a file out of HDFS, modifying it, and uploading
     it back into HDFS, the put fails due to a CRC mismatch
@@ -204,7 +203,18 @@ Release 0.23.2 - UNRELEASED
     HADOOP-8036. TestViewFsTrash assumes the user's home directory is
     2 levels deep. (Colin Patrick McCabe via eli)
 
-Release 0.23.1 - 2012-02-08 
+    HADOOP-8046 Revert StaticMapping semantics to the existing ones, add DNS
+    mapping diagnostics in progress (stevel)
+
+    HADOOP-8057 hadoop-setup-conf.sh not working because of some extra spaces.
+    (Vinayakumar B via stevel)
+
+    HADOOP-7680 TestHardLink fails on Mac OS X, when gnu stat is in path.
+    (Milind Bhandarkar via stevel)
+
+    HADOOP-8050. Deadlock in metrics. (Kihwal Lee via mattf)
+
+Release 0.23.1 - 2012-02-17 
 
   INCOMPATIBLE CHANGES
 
@@ -403,6 +413,8 @@ Release 0.23.1 - 2012-02-08
    HADOOP-8013. ViewFileSystem does not honor setVerifyChecksum
    (Daryn Sharp via bobby)
 
+   HADOOP-8054 NPE with FilterFileSystem (Daryn Sharp via bobby)
+
 Release 0.23.0 - 2011-11-01 
 
   INCOMPATIBLE CHANGES

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HardLink.java

@@ -64,7 +64,7 @@ public class HardLink {
       //override getLinkCountCommand for the particular Unix variant
       //Linux is already set as the default - {"stat","-c%h", null}
       if (osType == OSType.OS_TYPE_MAC) {
-        String[] linkCountCmdTemplate = {"stat","-f%l", null};
+        String[] linkCountCmdTemplate = {"/usr/bin/stat","-f%l", null};
         HardLinkCGUnix.setLinkCountCmdTemplate(linkCountCmdTemplate);
       } else if (osType == OSType.OS_TYPE_SOLARIS) {
         String[] linkCountCmdTemplate = {"ls","-l", null};

+ 41 - 22
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSourceAdapter.java

@@ -94,17 +94,19 @@ class MetricsSourceAdapter implements DynamicMBean {
   }
 
   @Override
-  public synchronized Object getAttribute(String attribute)
+  public Object getAttribute(String attribute)
       throws AttributeNotFoundException, MBeanException, ReflectionException {
     updateJmxCache();
-    Attribute a = attrCache.get(attribute);
-    if (a == null) {
-      throw new AttributeNotFoundException(attribute +" not found");
-    }
-    if (LOG.isDebugEnabled()) {
-      LOG.debug(attribute +": "+ a);
+    synchronized(this) {
+      Attribute a = attrCache.get(attribute);
+      if (a == null) {
+        throw new AttributeNotFoundException(attribute +" not found");
+      }
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(attribute +": "+ a);
+      }
+      return a.getValue();
     }
-    return a.getValue();
   }
 
   @Override
@@ -115,17 +117,19 @@ class MetricsSourceAdapter implements DynamicMBean {
   }
 
   @Override
-  public synchronized AttributeList getAttributes(String[] attributes) {
+  public AttributeList getAttributes(String[] attributes) {
     updateJmxCache();
-    AttributeList ret = new AttributeList();
-    for (String key : attributes) {
-      Attribute attr = attrCache.get(key);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(key +": "+ attr);
+    synchronized(this) {
+      AttributeList ret = new AttributeList();
+      for (String key : attributes) {
+        Attribute attr = attrCache.get(key);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(key +": "+ attr);
+        }
+        ret.add(attr);
       }
-      ret.add(attr);
+      return ret;
     }
-    return ret;
   }
 
   @Override
@@ -140,17 +144,32 @@ class MetricsSourceAdapter implements DynamicMBean {
   }
 
   @Override
-  public synchronized MBeanInfo getMBeanInfo() {
+  public MBeanInfo getMBeanInfo() {
     updateJmxCache();
     return infoCache;
   }
 
-  private synchronized void updateJmxCache() {
-    if (System.currentTimeMillis() - jmxCacheTS >= jmxCacheTTL) {
-      if (lastRecs == null) {
-        MetricsCollectorImpl builder = new MetricsCollectorImpl();
-        getMetrics(builder, true);
+  private void updateJmxCache() {
+    boolean getAllMetrics = false;
+    synchronized(this) {
+      if (System.currentTimeMillis() - jmxCacheTS >= jmxCacheTTL) {
+        // temporarilly advance the expiry while updating the cache
+        jmxCacheTS = System.currentTimeMillis() + jmxCacheTTL;
+        if (lastRecs == null) {
+          getAllMetrics = true;
+        }
+      }
+      else {
+        return;
       }
+    }
+
+    if (getAllMetrics) {
+      MetricsCollectorImpl builder = new MetricsCollectorImpl();
+      getMetrics(builder, true);
+    }
+
+    synchronized(this) {
       int oldCacheSize = attrCache.size();
       int newCacheSize = updateAttrCache();
       if (oldCacheSize < newCacheSize) {

+ 50 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/AbstractDNSToSwitchMapping.java

@@ -22,6 +22,12 @@ import org.apache.hadoop.classification.InterfaceAudience;
 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
@@ -89,6 +95,49 @@ public abstract class AbstractDNSToSwitchMapping
     return false;
   }
 
+  /**
+   * 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.
@@ -100,7 +149,7 @@ public abstract class AbstractDNSToSwitchMapping
    * 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();
   }
 

+ 17 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/CachedDNSToSwitchMapping.java

@@ -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;
@@ -123,6 +124,22 @@ public class CachedDNSToSwitchMapping extends AbstractDNSToSwitchMapping {
 
   }
 
+  /**
+   * 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)}

+ 17 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/ScriptBasedMapping.java

@@ -66,9 +66,15 @@ public final class ScriptBasedMapping extends CachedDNSToSwitchMapping {
                      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 extends CachedDNSToSwitchMapping {
     return getRawMapping().getConf();
   }
 
+  @Override
+  public String toString() {
+    return "script-based mapping with " + getRawMapping().toString();
+  }
+
   /**
    * {@inheritDoc}
    * <p/>
@@ -231,7 +242,7 @@ public final class ScriptBasedMapping extends CachedDNSToSwitchMapping {
           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 extends CachedDNSToSwitchMapping {
     public boolean isSingleSwitch() {
       return scriptName == null;
     }
+
+    @Override
+    public String toString() {
+      return scriptName != null ? ("script " + scriptName) : NO_SCRIPT;
+    }
   }
 }

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/packages/hadoop-setup-conf.sh

@@ -246,7 +246,7 @@ OPTS=$(getopt \
   -l 'dfs-datanode-dir-perm:' \
   -l 'dfs-block-local-path-access-user:' \
   -l 'dfs-client-read-shortcircuit:' \
-  -l 'dfs-client-read-shortcircuit-skip-checksum:' \   
+  -l 'dfs-client-read-shortcircuit-skip-checksum:' \
   -o 'h' \
   -- "$@") 
   

+ 26 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/StaticMapping.java

@@ -21,8 +21,10 @@ import org.apache.hadoop.conf.Configuration;
 
 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 AbstractDNSToSwitchMapping  {
   }
 
   /**
-   * 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();
   }
 
   /**

+ 171 - 34
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestStaticMapping.java

@@ -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 Assert {
     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 testNullConfiguration() throws Throwable {
+  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 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);
   }
+
+
 }

+ 69 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestSwitchMapping.java

@@ -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) {

+ 4 - 1
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -235,6 +235,9 @@ Release 0.23.2 - UNRELEASED
     HDFS-2931. Switch DataNode's BlockVolumeChoosingPolicy to private-audience.
     (harsh via szetszwo)
 
+    HDFS-2725. hdfs script usage information is missing the information 
+    about "dfs" command (Prashant Sharma via stevel)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -262,7 +265,7 @@ Release 0.23.2 - UNRELEASED
 
     HDFS-2969. ExtendedBlock.equals is incorrectly implemented (todd)
 
-Release 0.23.1 - 2012-02-08 
+Release 0.23.1 - 2012-02-17 
 
   INCOMPATIBLE CHANGES
 

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs

@@ -26,6 +26,7 @@ HADOOP_LIBEXEC_DIR=${HADOOP_LIBEXEC_DIR:-$DEFAULT_LIBEXEC_DIR}
 function print_usage(){
   echo "Usage: hdfs [--config confdir] COMMAND"
   echo "       where COMMAND is one of:"
+  echo "  dfs                  run a filesystem command on the file systems supported in Hadoop."
   echo "  namenode -format     format the DFS filesystem"
   echo "  secondarynamenode    run the DFS secondary namenode"
   echo "  namenode             run the DFS namenode"

+ 10 - 5
hadoop-mapreduce-project/CHANGES.txt

@@ -102,6 +102,9 @@ Release 0.23.2 - UNRELEASED
     MAPREDUCE-3854. Fixed and reenabled tests related to MR child JVM's
     environmental variables in TestMiniMRChildTask. (Tom White via vinodkv)
 
+    MAPREDUCE-3877 Add a test to formalise the current state transitions
+    of the yarn lifecycle. (stevel)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -120,14 +123,10 @@ Release 0.23.2 - UNRELEASED
     MAPREDUCE-3864. Fix cluster setup docs for correct SecondaryNameNode
     HTTPS parameters. (todd)
 
-    MAPREDUCE-3856. Instances of RunningJob class givs incorrect job tracking
-    urls when mutiple jobs are submitted from same client jvm. (Eric Payne via
-    sseth)
-
     MAPREDUCE-3583. Change pid to String and stime to BigInteger in order to
     avoid NumberFormatException caused by overflow.  (Zhihong Yu via szetszwo)
  
-Release 0.23.1 - 2012-02-08 
+Release 0.23.1 - 2012-02-17 
 
   INCOMPATIBLE CHANGES
 
@@ -838,6 +837,12 @@ Release 0.23.1 - 2012-02-08
     MAPREDUCE-3858. Task attempt failure during commit results in task never completing.
     (Tom White via mahadev)
 
+    MAPREDUCE-3856. Instances of RunningJob class givs incorrect job tracking
+    urls when mutiple jobs are submitted from same client jvm. (Eric Payne via
+    sseth)
+
+    MAPREDUCE-3880. Changed LCE binary to be 32-bit. (acmurthy)
+
 Release 0.23.0 - 2011-11-01 
 
   INCOMPATIBLE CHANGES

+ 123 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/service/BreakableService.java

@@ -0,0 +1,123 @@
+/**
+ * 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.yarn.service;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * This is a service that can be configured to break on any of the lifecycle
+ * events, so test the failure handling of other parts of the service
+ * infrastructure.
+ *
+ * It retains a counter to the number of times each entry point is called -
+ * these counters are incremented before the exceptions are raised and
+ * before the superclass state methods are invoked.
+ *
+ */
+
+
+public class BreakableService extends AbstractService {
+
+  private boolean failOnInit;
+  private boolean failOnStart;
+  private boolean failOnStop;
+  private int[] counts = new int[4];
+
+  public BreakableService() {
+    this(false, false, false);
+  }
+
+  public BreakableService(boolean failOnInit,
+                          boolean failOnStart,
+                          boolean failOnStop) {
+    super("BreakableService");
+    this.failOnInit = failOnInit;
+    this.failOnStart = failOnStart;
+    this.failOnStop = failOnStop;
+    inc(STATE.NOTINITED);
+  }
+
+  private int convert(STATE state) {
+    switch (state) {
+      case NOTINITED: return 0;
+      case INITED:    return 1;
+      case STARTED:   return 2;
+      case STOPPED:   return 3;
+      default:        return 0;
+    }
+  }
+
+  private void inc(STATE state) {
+    int index = convert(state);
+    counts[index] ++;
+  }
+
+  public int getCount(STATE state) {
+    return counts[convert(state)];
+  }
+
+  private void maybeFail(boolean fail, String action) {
+    if (fail) {
+      throw new BrokenLifecycleEvent(action);
+    }
+  }
+
+  @Override
+  public void init(Configuration conf) {
+    inc(STATE.INITED);
+    maybeFail(failOnInit, "init");
+    super.init(conf);
+  }
+
+  @Override
+  public void start() {
+    inc(STATE.STARTED);
+    maybeFail(failOnStart, "start");
+    super.start();
+  }
+
+  @Override
+  public void stop() {
+    inc(STATE.STOPPED);
+    maybeFail(failOnStop, "stop");
+    super.stop();
+  }
+
+  public void setFailOnInit(boolean failOnInit) {
+    this.failOnInit = failOnInit;
+  }
+
+  public void setFailOnStart(boolean failOnStart) {
+    this.failOnStart = failOnStart;
+  }
+
+  public void setFailOnStop(boolean failOnStop) {
+    this.failOnStop = failOnStop;
+  }
+
+  /**
+   * The exception explicitly raised on a failure
+   */
+  public static class BrokenLifecycleEvent extends RuntimeException {
+    BrokenLifecycleEvent(String action) {
+      super("Lifecycle Failure during " + action);
+    }
+  }
+}

+ 49 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/service/ServiceAssert.java

@@ -0,0 +1,49 @@
+/**
+ * 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.yarn.service;
+
+import org.junit.Assert;
+
+/**
+ * A set of assertions about the state of any service
+ */
+public class ServiceAssert extends Assert {
+
+  public static void assertServiceStateCreated(Service service) {
+    assertServiceInState(service, Service.STATE.NOTINITED);
+  }
+
+  public static void assertServiceStateInited(Service service) {
+    assertServiceInState(service, Service.STATE.INITED);
+  }
+
+  public static void assertServiceStateStarted(Service service) {
+    assertServiceInState(service, Service.STATE.STARTED);
+  }
+
+  public static void assertServiceStateStopped(Service service) {
+    assertServiceInState(service, Service.STATE.STOPPED);
+  }
+
+  public static void assertServiceInState(Service service, Service.STATE state) {
+    assertNotNull("Null service", service);
+    assertEquals("Service in wrong state: " + service, state,
+                 service.getServiceState());
+  }
+}

+ 194 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/service/TestServiceLifecycle.java

@@ -0,0 +1,194 @@
+/**
+ * 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.yarn.service;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Test;
+
+public class TestServiceLifecycle extends ServiceAssert {
+
+  void assertStateCount(BreakableService service,
+                        Service.STATE state,
+                        int expected) {
+    int actual = service.getCount(state);
+    if (expected != actual) {
+      fail("Expected entry count for state [" + state +"] of " + service
+               + " to be " + expected + " but was " + actual);
+    }
+  }
+
+
+  @Test
+  public void testWalkthrough() throws Throwable {
+
+    BreakableService svc = new BreakableService();
+    assertServiceStateCreated(svc);
+    assertStateCount(svc, Service.STATE.NOTINITED, 1);
+    assertStateCount(svc, Service.STATE.INITED, 0);
+    assertStateCount(svc, Service.STATE.STARTED, 0);
+    assertStateCount(svc, Service.STATE.STOPPED, 0);
+    svc.init(new Configuration());
+    assertServiceStateInited(svc);
+    assertStateCount(svc, Service.STATE.INITED, 1);
+    svc.start();
+    assertServiceStateStarted(svc);
+    assertStateCount(svc, Service.STATE.STARTED, 1);
+    svc.stop();
+    assertServiceStateStopped(svc);
+    assertStateCount(svc, Service.STATE.STOPPED, 1);
+  }
+
+  /**
+   * call init twice
+   * @throws Throwable
+   */
+  @Test
+  public void testInitTwice() throws Throwable {
+    BreakableService svc = new BreakableService();
+    svc.init(new Configuration());
+    try {
+      svc.init(new Configuration());
+      fail("Expected a failure, got " + svc);
+    } catch (IllegalStateException e) {
+      //expected
+    }
+    assertStateCount(svc, Service.STATE.INITED, 2);
+  }
+
+  /**
+   * call start twice
+   * @throws Throwable
+   */
+  @Test
+  public void testStartTwice() throws Throwable {
+    BreakableService svc = new BreakableService();
+    svc.init(new Configuration());
+    svc.start();
+    try {
+      svc.start();
+      fail("Expected a failure, got " + svc);
+    } catch (IllegalStateException e) {
+      //expected
+    }
+    assertStateCount(svc, Service.STATE.STARTED, 2);
+  }
+
+
+  /**
+   * verify that when a service is stopped more than once, no exception
+   * is thrown, and the counter is incremented
+   * this is because the state change operations happen after the counter in
+   * the subclass is incremented, even though stop is meant to be a no-op
+   * @throws Throwable
+   */
+  @Test
+  public void testStopTwice() throws Throwable {
+    BreakableService svc = new BreakableService();
+    svc.init(new Configuration());
+    svc.start();
+    svc.stop();
+    assertStateCount(svc, Service.STATE.STOPPED, 1);
+    svc.stop();
+    assertStateCount(svc, Service.STATE.STOPPED, 2);
+  }
+
+
+  /**
+   * Show that if the service failed during an init
+   * operation, it stays in the created state, even after stopping it
+   * @throws Throwable
+   */
+
+  @Test
+  public void testStopFailedInit() throws Throwable {
+    BreakableService svc = new BreakableService(true, false, false);
+    assertServiceStateCreated(svc);
+    try {
+      svc.init(new Configuration());
+      fail("Expected a failure, got " + svc);
+    } catch (BreakableService.BrokenLifecycleEvent e) {
+      //expected
+    }
+    //the service state wasn't passed
+    assertServiceStateCreated(svc);
+    assertStateCount(svc, Service.STATE.INITED, 1);
+    //now try to stop
+    svc.stop();
+    //even after the stop operation, we haven't entered the state
+    assertServiceStateCreated(svc);
+  }
+
+
+  /**
+   * Show that if the service failed during an init
+   * operation, it stays in the created state, even after stopping it
+   * @throws Throwable
+   */
+
+  @Test
+  public void testStopFailedStart() throws Throwable {
+    BreakableService svc = new BreakableService(false, true, false);
+    svc.init(new Configuration());
+    assertServiceStateInited(svc);
+    try {
+      svc.start();
+      fail("Expected a failure, got " + svc);
+    } catch (BreakableService.BrokenLifecycleEvent e) {
+      //expected
+    }
+    //the service state wasn't passed
+    assertServiceStateInited(svc);
+    assertStateCount(svc, Service.STATE.INITED, 1);
+    //now try to stop
+    svc.stop();
+    //even after the stop operation, we haven't entered the state
+    assertServiceStateInited(svc);
+  }
+
+  /**
+   * verify that when a service is stopped more than once, no exception
+   * is thrown, and the counter is incremented
+   * this is because the state change operations happen after the counter in
+   * the subclass is incremented, even though stop is meant to be a no-op
+   * @throws Throwable
+   */
+  @Test
+  public void testFailingStop() throws Throwable {
+    BreakableService svc = new BreakableService(false, false, true);
+    svc.init(new Configuration());
+    svc.start();
+    try {
+      svc.stop();
+      fail("Expected a failure, got " + svc);
+    } catch (BreakableService.BrokenLifecycleEvent e) {
+      //expected
+    }
+    assertStateCount(svc, Service.STATE.STOPPED, 1);
+    //now try again, and expect it to happen again
+    try {
+      svc.stop();
+      fail("Expected a failure, got " + svc);
+    } catch (BreakableService.BrokenLifecycleEvent e) {
+      //expected
+    }
+    assertStateCount(svc, Service.STATE.STOPPED, 2);
+  }
+
+}

+ 1 - 1
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml

@@ -74,7 +74,7 @@
               <configureEnvironment>
                 <property>
                   <name>CFLAGS</name>
-                  <value>-DHADOOP_CONF_DIR=${container-executor.conf.dir}</value>
+                  <value>-DHADOOP_CONF_DIR=${container-executor.conf.dir} -m32</value>
                 </property>
               </configureEnvironment>
               <configureWorkDir>${project.build.directory}/native/container-executor</configureWorkDir>