You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by te...@apache.org on 2013/10/24 19:07:55 UTC

svn commit: r1535442 - in /hbase/branches/0.94/src: main/java/org/apache/hadoop/hbase/regionserver/CompoundConfiguration.java test/java/org/apache/hadoop/hbase/regionserver/TestCompoundConfiguration.java

Author: tedyu
Date: Thu Oct 24 17:07:55 2013
New Revision: 1535442

URL: http://svn.apache.org/r1535442
Log:
HBASE-9819 Backport HBASE-8372 'Provide mutability to CompoundConfiguration' to 0.94 (Ted Yu)


Modified:
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/CompoundConfiguration.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundConfiguration.java

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/CompoundConfiguration.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/CompoundConfiguration.java?rev=1535442&r1=1535441&r2=1535442&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/CompoundConfiguration.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/CompoundConfiguration.java Thu Oct 24 17:07:55 2013
@@ -19,12 +19,10 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
-import java.io.DataInput;
 import java.io.DataOutput;
-import java.io.IOException;
+import java.io.IOException; 
 import java.io.OutputStream;
 import java.util.ArrayList;
-import java.util.Collection;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
@@ -36,7 +34,6 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.util.StringUtils;
 
 /**
  * Do a shallow merge of multiple KV configuration pools. This is a very useful
@@ -53,9 +50,15 @@ import org.apache.hadoop.util.StringUtil
  * This class is package private because we expect significant refactoring here
  * on the HBase side when certain HDFS changes are added & ubiquitous. Will
  * revisit expanding access at that point.
+ *
+ * WARNING: The values set in the CompoundConfiguration are do not handle Property variable
+ * substitution.  However, if they are set in the underlying configuration substitutions are
+ * done.
  */
 @InterfaceAudience.Private
 class CompoundConfiguration extends Configuration {
+	private Configuration mutableConf = null;
+
   /**
    * Default Constructor. Initializes empty configuration
    */
@@ -78,51 +81,73 @@ class CompoundConfiguration extends Conf
    * These initial APIs actually required original thought
    ***************************************************************************/
 
+  static class ImmutableConfWrapper implements  ImmutableConfigMap {
+    Configuration c;
+    
+    ImmutableConfWrapper(Configuration conf) {
+      c = conf;
+    }
+
+    @Override
+    public Iterator<Map.Entry<String,String>> iterator() {
+      return c.iterator();
+    }
+    
+    @Override
+    public String get(String key) {
+      return c.get(key);
+    }
+
+    @Override
+    public String getRaw(String key) {
+      return c.getRaw(key);
+    }
+
+    @Override
+    public Class<?> getClassByName(String name)
+        throws ClassNotFoundException {
+      return c.getClassByName(name);
+    }
+
+    @Override
+    public int size() {
+      return c.size();
+    }
+
+    @Override
+    public String toString() {
+      return c.toString();
+    }
+  }
+
+  /**
+   * If set has been called, it will create a mutableConf.  This converts the mutableConf to an
+   * immutable one and resets it to allow a new mutable conf.  This is used when a new map or
+   * conf is added to the compound configuration to preserve proper override semantics.
+   */
+  void freezeMutableConf() {
+    if (mutableConf == null) {
+      // do nothing if there is no current mutableConf
+      return;
+    }
+
+    this.configs.add(0, new ImmutableConfWrapper(mutableConf));
+    mutableConf = null;
+  }
+
   /**
    * Add Hadoop Configuration object to config list
    * @param conf configuration object
    * @return this, for builder pattern
    */
   public CompoundConfiguration add(final Configuration conf) {
+    freezeMutableConf();
     if (conf instanceof CompoundConfiguration) {
       this.configs.addAll(0, ((CompoundConfiguration) conf).configs);
       return this;
     }
     // put new config at the front of the list (top priority)
-    this.configs.add(0, new ImmutableConfigMap() {
-      Configuration c = conf;
-
-      @Override
-      public String get(String key) {
-        return c.get(key);
-      }
-
-      @Override
-      public String getRaw(String key) {
-        return c.getRaw(key);
-      }
-
-      @Override
-      public Class<?> getClassByName(String name)
-          throws ClassNotFoundException {
-        return c.getClassByName(name);
-      }
-
-      @Override
-      public int size() {
-        return c.size();
-      }
-
-      @Override
-      public String toString() {
-        return c.toString();
-      }
-
-      @Override
-      public Iterator<Entry<String, String>> iterator() {
-        return c.iterator();
-      }
-    });
+    this.configs.add(0, new ImmutableConfWrapper(conf));
     return this;
   }
 
@@ -137,6 +162,8 @@ class CompoundConfiguration extends Conf
    */
   public CompoundConfiguration add(
       final Map<ImmutableBytesWritable, ImmutableBytesWritable> map) {
+    freezeMutableConf();
+
     // put new map at the front of the list (top priority)
     this.configs.add(0, new ImmutableConfigMap() {
       Map<ImmutableBytesWritable, ImmutableBytesWritable> m = map;
@@ -227,6 +254,54 @@ class CompoundConfiguration extends Conf
     return this;
   }
 
+  /**
+   * Add String map to config list. This map is generally created by HTableDescriptor
+   * or HColumnDescriptor, but can be abstractly used. The added configuration
+   * overrides the previous ones if there are name collisions.
+   *
+   * @return this, for builder pattern
+   */
+  public CompoundConfiguration addStringMap(final Map<String, String> map) {
+    freezeMutableConf();
+
+    // put new map at the front of the list (top priority)
+    this.configs.add(0, new ImmutableConfigMap() {
+      Map<String, String> m = map;
+
+      @Override
+      public Iterator<Map.Entry<String,String>> iterator() {
+        return map.entrySet().iterator();
+      }
+
+      @Override
+      public String get(String key) {
+        return m.get(key);
+      }
+
+      @Override
+      public String getRaw(String key) {
+        return get(key);
+      }
+
+      @Override
+      public Class<?> getClassByName(String name)
+      throws ClassNotFoundException {
+        return null;
+      }
+
+      @Override
+      public int size() {
+        return m.size();
+      }
+
+      @Override
+      public String toString() {
+        return m.toString();
+      }
+    });
+    return this;
+  }
+
   @Override
   public String toString() {
     StringBuffer sb = new StringBuffer();
@@ -239,6 +314,13 @@ class CompoundConfiguration extends Conf
 
   @Override
   public String get(String key) {
+    if (mutableConf != null) {
+      String value = mutableConf.get(key);
+      if (value != null) {
+        return value;
+      }
+    }
+
     for (ImmutableConfigMap m : this.configs) {
       String value = m.get(key);
       if (value != null) {
@@ -250,6 +332,13 @@ class CompoundConfiguration extends Conf
 
   @Override
   public String getRaw(String key) {
+    if (mutableConf != null) {
+      String value = mutableConf.getRaw(key);
+      if (value != null) {
+        return value;
+      }
+    }
+
     for (ImmutableConfigMap m : this.configs) {
       String value = m.getRaw(key);
       if (value != null) {
@@ -261,6 +350,13 @@ class CompoundConfiguration extends Conf
 
   @Override
   public Class<?> getClassByName(String name) throws ClassNotFoundException {
+    if (mutableConf != null) {
+      Class<?> value = mutableConf.getClassByName(name);
+      if (value != null) {
+        return value;
+      }
+    }
+
     for (ImmutableConfigMap m : this.configs) {
       try {
         Class<?> value = m.getClassByName(name);
@@ -275,9 +371,14 @@ class CompoundConfiguration extends Conf
     throw new ClassNotFoundException();
   }
 
+  // TODO: This method overestimates the number of configuration settings -- if a value is masked
+  // by an overriding config or map, it will be counted multiple times.
   @Override
   public int size() {
     int ret = 0;
+    if (mutableConf != null) {
+      ret += mutableConf.size();
+    }
     for (ImmutableConfigMap m : this.configs) {
       ret += m.size();
     }
@@ -299,28 +400,36 @@ class CompoundConfiguration extends Conf
         }
       }
     }
+    // add mutations to this CompoundConfiguration last.
+    if (mutableConf != null) {
+      Iterator<Map.Entry<String, String>> miter = mutableConf.iterator();
+      while (miter.hasNext()) {
+        Map.Entry<String, String> entry = miter.next();
+        ret.put(entry.getKey(), entry.getValue());
+      }
+    }
 
     return UnmodifiableIterator.decorate(ret.entrySet().iterator());
   }
 
-  /***************************************************************************
-   * You should just ignore everything below this line unless there's a bug in
-   * Configuration.java...
+  /**
+   * Get the value of the <code>name</code>. If the key is deprecated,
+   * it returns the value of the first key which replaces the deprecated key
+   * and is not null.
+   * If no such property exists,
+   * then <code>defaultValue</code> is returned.
+
+   * The CompooundConfiguration does not do property substitution.  To do so we need
+   * Configuration.getProps to be protected or package visible.  Though in hadoop2 it is
+   * protected, in hadoop1 the method is private and not accessible.
    *
-   * Below get APIs are directly copied from Configuration.java Oh, how I wish
-   * this wasn't so! A tragically-sad example of why you use interfaces instead
-   * of inheritance.
+   * All of the get* methods call this overridden get method.
    *
-   * Why the duplication? We basically need to override Configuration.getProps
-   * or we'd need protected access to Configuration.properties so we can modify
-   * that pointer. There are a bunch of functions in the base Configuration that
-   * call getProps() and we need to use our derived version instead of the base
-   * version. We need to make a generic implementation that works across all
-   * HDFS versions. We should modify Configuration.properties in HDFS 1.0 to be
-   * protected, but we still need to have this code until that patch makes it to
-   * all the HDFS versions we support.
-   ***************************************************************************/
-
+   * @param name property name.
+   * @param defaultValue default value.
+   * @return property value, or <code>defaultValue</code> if the property
+   *         doesn't exist.
+   **/
   @Override
   public String get(String name, String defaultValue) {
     String ret = get(name);
@@ -328,200 +437,20 @@ class CompoundConfiguration extends Conf
   }
 
   @Override
-  public int getInt(String name, int defaultValue) {
-    String valueString = get(name);
-    if (valueString == null)
-      return defaultValue;
-    try {
-      String hexString = getHexDigits(valueString);
-      if (hexString != null) {
-        return Integer.parseInt(hexString, 16);
-      }
-      return Integer.parseInt(valueString);
-    } catch (NumberFormatException e) {
-      return defaultValue;
-    }
-  }
-
-  @Override
-  public long getLong(String name, long defaultValue) {
-    String valueString = get(name);
-    if (valueString == null)
-      return defaultValue;
-    try {
-      String hexString = getHexDigits(valueString);
-      if (hexString != null) {
-        return Long.parseLong(hexString, 16);
-      }
-      return Long.parseLong(valueString);
-    } catch (NumberFormatException e) {
-      return defaultValue;
-    }
-  }
-
-  protected String getHexDigits(String value) {
-    boolean negative = false;
-    String str = value;
-    String hexString = null;
-    if (value.startsWith("-")) {
-      negative = true;
-      str = value.substring(1);
-    }
-    if (str.startsWith("0x") || str.startsWith("0X")) {
-      hexString = str.substring(2);
-      if (negative) {
-        hexString = "-" + hexString;
-      }
-      return hexString;
-    }
-    return null;
-  }
-
-  @Override
-  public float getFloat(String name, float defaultValue) {
-    String valueString = get(name);
-    if (valueString == null)
-      return defaultValue;
-    try {
-      return Float.parseFloat(valueString);
-    } catch (NumberFormatException e) {
-      return defaultValue;
-    }
-  }
-
-  @Override
-  public boolean getBoolean(String name, boolean defaultValue) {
-    String valueString = get(name);
-    if ("true".equals(valueString))
-      return true;
-    else if ("false".equals(valueString))
-      return false;
-    else return defaultValue;
-  }
-
-  @Override
-  public IntegerRanges getRange(String name, String defaultValue) {
-    return new IntegerRanges(get(name, defaultValue));
-  }
-
-  @Override
-  public Collection<String> getStringCollection(String name) {
-    String valueString = get(name);
-    return StringUtils.getStringCollection(valueString);
-  }
-
-  @Override
-  public String[] getStrings(String name) {
-    String valueString = get(name);
-    return StringUtils.getStrings(valueString);
-  }
-
-  @Override
-  public String[] getStrings(String name, String... defaultValue) {
-    String valueString = get(name);
-    if (valueString == null) {
-      return defaultValue;
-    } else {
-      return StringUtils.getStrings(valueString);
-    }
-  }
-
-  @Override
-  public Class<?>[] getClasses(String name, Class<?>... defaultValue) {
-    String[] classnames = getStrings(name);
-    if (classnames == null)
-      return defaultValue;
-    try {
-      Class<?>[] classes = new Class<?>[classnames.length];
-      for (int i = 0; i < classnames.length; i++) {
-        classes[i] = getClassByName(classnames[i]);
-      }
-      return classes;
-    } catch (ClassNotFoundException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  @Override
-  public Class<?> getClass(String name, Class<?> defaultValue) {
-    String valueString = get(name);
-    if (valueString == null)
-      return defaultValue;
-    try {
-      return getClassByName(valueString);
-    } catch (ClassNotFoundException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  @Override
-  public <U> Class<? extends U> getClass(String name,
-      Class<? extends U> defaultValue, Class<U> xface) {
-    try {
-      Class<?> theClass = getClass(name, defaultValue);
-      if (theClass != null && !xface.isAssignableFrom(theClass))
-        throw new RuntimeException(theClass + " not " + xface.getName());
-      else if (theClass != null)
-        return theClass.asSubclass(xface);
-      else
-        return null;
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  /*******************************************************************
-   * This class is immutable. Quickly abort any attempts to alter it *
-   *******************************************************************/
-
-  @Override
-  public void clear() {
-    throw new UnsupportedOperationException("Immutable Configuration");
-  }
-
-  @Override
   public void set(String name, String value) {
-    throw new UnsupportedOperationException("Immutable Configuration");
-  }
-  @Override
-  public void setIfUnset(String name, String value) {
-    throw new UnsupportedOperationException("Immutable Configuration");
-  }
-  @Override
-  public void setInt(String name, int value) {
-    throw new UnsupportedOperationException("Immutable Configuration");
-  }
-  @Override
-  public void setLong(String name, long value) {
-    throw new UnsupportedOperationException("Immutable Configuration");
-  }
-  @Override
-  public void setFloat(String name, float value) {
-    throw new UnsupportedOperationException("Immutable Configuration");
-  }
-  @Override
-  public void setBoolean(String name, boolean value) {
-    throw new UnsupportedOperationException("Immutable Configuration");
-  }
-  @Override
-  public void setBooleanIfUnset(String name, boolean value) {
-    throw new UnsupportedOperationException("Immutable Configuration");
-  }
-  @Override
-  public void setStrings(String name, String... values) {
-    throw new UnsupportedOperationException("Immutable Configuration");
-  }
-  @Override
-  public void setClass(String name, Class<?> theClass, Class<?> xface) {
-    throw new UnsupportedOperationException("Immutable Configuration");
-  }
-  @Override
-  public void setClassLoader(ClassLoader classLoader) {
-    throw new UnsupportedOperationException("Immutable Configuration");
-  }
-
+    if (mutableConf == null) {
+      // not thread safe
+      mutableConf = new Configuration(false); // an empty configuration
+    }
+    mutableConf.set(name,  value);
+  }
+  
+  /***********************************************************************************************
+   * These methods are unsupported, and no code using CompoundConfiguration depend upon them.
+   * Quickly abort upon any attempts to use them.
+   **********************************************************************************************/
   @Override
-  public void readFields(DataInput in) throws IOException {
+  public void clear() {
     throw new UnsupportedOperationException("Immutable Configuration");
   }
 

Modified: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundConfiguration.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundConfiguration.java?rev=1535442&r1=1535441&r2=1535442&view=diff
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundConfiguration.java (original)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundConfiguration.java Thu Oct 24 17:07:55 2013
@@ -50,7 +50,7 @@ public class TestCompoundConfiguration e
   @Test
   public void testBasicFunctionality() throws ClassNotFoundException {
     CompoundConfiguration compoundConf = new CompoundConfiguration()
-        .add(baseConf);
+        .add(baseConf); 
     assertEquals("1", compoundConf.get("A"));
     assertEquals(2, compoundConf.getInt("B", 0));
     assertEquals(3, compoundConf.getInt("C", 0));
@@ -67,6 +67,29 @@ public class TestCompoundConfiguration e
   }
 
   @Test
+  public void testPut() {
+    CompoundConfiguration compoundConf = new CompoundConfiguration()
+      .add(baseConf);
+    assertEquals("1", compoundConf.get("A"));
+    assertEquals(2, compoundConf.getInt("B", 0));
+    assertEquals(3, compoundConf.getInt("C", 0));
+    assertEquals(0, compoundConf.getInt("D", 0));
+
+    compoundConf.set("A", "1337");
+    compoundConf.set("string", "stringvalue");
+    assertEquals(1337, compoundConf.getInt("A", 0));
+    assertEquals("stringvalue", compoundConf.get("string"));
+
+    // we didn't modify the base conf
+    assertEquals("1", baseConf.get("A"));
+    assertNull(baseConf.get("string"));
+
+    // adding to the base shows up in the compound
+    baseConf.set("setInParent", "fromParent");
+    assertEquals("fromParent", compoundConf.get("setInParent"));
+  }
+
+  @Test
   public void testWithConfig() {
     Configuration conf = new Configuration();
     conf.set("B", "2b");
@@ -128,6 +151,52 @@ public class TestCompoundConfiguration e
     }
     // verify that entries from ImmutableConfigMap's are merged in the iterator's view
     assertEquals(baseConfSize + 2, cnt);
+
+    // Verify that adding map after compound configuration is modified overrides properly
+    CompoundConfiguration conf2 = new CompoundConfiguration();
+    conf2.set("X", "modification");
+    conf2.set("D", "not4");
+    assertEquals("modification", conf2.get("X"));
+    assertEquals("not4", conf2.get("D"));
+    conf2.add(map);
+    assertEquals("4", conf2.get("D")); // map overrides
+  }
+
+  @Test
+  public void testWithStringMap() {
+    Map<String, String> map = new HashMap<String, String>();
+    map.put("B", "2b");
+    map.put("C", "33");
+    map.put("D", "4");
+    // unlike config, note that IBW Maps can accept null values
+    map.put("G", null);
+
+    CompoundConfiguration compoundConf = new CompoundConfiguration().addStringMap(map);
+    assertEquals("2b", compoundConf.get("B"));
+    assertEquals(33, compoundConf.getInt("C", 0));
+    assertEquals("4", compoundConf.get("D"));
+    assertEquals(4, compoundConf.getInt("D", 0));
+    assertNull(compoundConf.get("E"));
+    assertEquals(6, compoundConf.getInt("F", 6));
+    assertNull(compoundConf.get("G"));
+
+    int cnt = 0;
+    for (Map.Entry<String,String> entry : compoundConf) {
+      cnt++;
+      if (entry.getKey().equals("B")) assertEquals("2b", entry.getValue());
+      else if (entry.getKey().equals("G")) assertEquals(null, entry.getValue());
+    }
+    // verify that entries from ImmutableConfigMap's are merged in the iterator's view
+    assertEquals(4, cnt);
+    
+    // Verify that adding map after compound configuration is modified overrides properly
+    CompoundConfiguration conf2 = new CompoundConfiguration();
+    conf2.set("X", "modification");
+    conf2.set("D", "not4");
+    assertEquals("modification", conf2.get("X"));
+    assertEquals("not4", conf2.get("D"));
+    conf2.addStringMap(map);
+    assertEquals("4", conf2.get("D")); // map overrides
   }
 
   @Test