001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018
019package org.apache.hadoop.conf;
020
021import com.google.common.annotations.VisibleForTesting;
022
023import java.io.BufferedInputStream;
024import java.io.DataInput;
025import java.io.DataOutput;
026import java.io.File;
027import java.io.FileInputStream;
028import java.io.IOException;
029import java.io.InputStream;
030import java.io.InputStreamReader;
031import java.io.OutputStream;
032import java.io.OutputStreamWriter;
033import java.io.Reader;
034import java.io.Writer;
035import java.lang.ref.WeakReference;
036import java.net.InetSocketAddress;
037import java.net.URL;
038import java.util.ArrayList;
039import java.util.Arrays;
040import java.util.Collection;
041import java.util.Collections;
042import java.util.Enumeration;
043import java.util.HashMap;
044import java.util.HashSet;
045import java.util.Iterator;
046import java.util.LinkedList;
047import java.util.List;
048import java.util.ListIterator;
049import java.util.Map;
050import java.util.Map.Entry;
051import java.util.Properties;
052import java.util.Set;
053import java.util.StringTokenizer;
054import java.util.WeakHashMap;
055import java.util.concurrent.ConcurrentHashMap;
056import java.util.concurrent.CopyOnWriteArrayList;
057import java.util.regex.Matcher;
058import java.util.regex.Pattern;
059import java.util.regex.PatternSyntaxException;
060import java.util.concurrent.TimeUnit;
061import java.util.concurrent.atomic.AtomicBoolean;
062import java.util.concurrent.atomic.AtomicReference;
063
064import javax.xml.parsers.DocumentBuilder;
065import javax.xml.parsers.DocumentBuilderFactory;
066import javax.xml.parsers.ParserConfigurationException;
067import javax.xml.transform.Transformer;
068import javax.xml.transform.TransformerException;
069import javax.xml.transform.TransformerFactory;
070import javax.xml.transform.dom.DOMSource;
071import javax.xml.transform.stream.StreamResult;
072
073import com.google.common.base.Charsets;
074import org.apache.commons.collections.map.UnmodifiableMap;
075import org.apache.commons.logging.Log;
076import org.apache.commons.logging.LogFactory;
077import org.apache.hadoop.classification.InterfaceAudience;
078import org.apache.hadoop.classification.InterfaceStability;
079import org.apache.hadoop.fs.FileSystem;
080import org.apache.hadoop.fs.Path;
081import org.apache.hadoop.fs.CommonConfigurationKeys;
082import org.apache.hadoop.io.Writable;
083import org.apache.hadoop.io.WritableUtils;
084import org.apache.hadoop.net.NetUtils;
085import org.apache.hadoop.security.alias.CredentialProvider;
086import org.apache.hadoop.security.alias.CredentialProvider.CredentialEntry;
087import org.apache.hadoop.security.alias.CredentialProviderFactory;
088import org.apache.hadoop.util.ReflectionUtils;
089import org.apache.hadoop.util.StringInterner;
090import org.apache.hadoop.util.StringUtils;
091import org.codehaus.jackson.JsonFactory;
092import org.codehaus.jackson.JsonGenerator;
093import org.w3c.dom.Attr;
094import org.w3c.dom.DOMException;
095import org.w3c.dom.Document;
096import org.w3c.dom.Element;
097import org.w3c.dom.Node;
098import org.w3c.dom.NodeList;
099import org.w3c.dom.Text;
100import org.xml.sax.SAXException;
101
102import com.google.common.base.Preconditions;
103
104/** 
105 * Provides access to configuration parameters.
106 *
107 * <h4 id="Resources">Resources</h4>
108 *
109 * <p>Configurations are specified by resources. A resource contains a set of
110 * name/value pairs as XML data. Each resource is named by either a 
111 * <code>String</code> or by a {@link Path}. If named by a <code>String</code>, 
112 * then the classpath is examined for a file with that name.  If named by a 
113 * <code>Path</code>, then the local filesystem is examined directly, without 
114 * referring to the classpath.
115 *
116 * <p>Unless explicitly turned off, Hadoop by default specifies two 
117 * resources, loaded in-order from the classpath: <ol>
118 * <li><tt>
119 * <a href="{@docRoot}/../hadoop-project-dist/hadoop-common/core-default.xml">
120 * core-default.xml</a></tt>: Read-only defaults for hadoop.</li>
121 * <li><tt>core-site.xml</tt>: Site-specific configuration for a given hadoop
122 * installation.</li>
123 * </ol>
124 * Applications may add additional resources, which are loaded
125 * subsequent to these resources in the order they are added.
126 * 
127 * <h4 id="FinalParams">Final Parameters</h4>
128 *
129 * <p>Configuration parameters may be declared <i>final</i>. 
130 * Once a resource declares a value final, no subsequently-loaded 
131 * resource can alter that value.  
132 * For example, one might define a final parameter with:
133 * <tt><pre>
134 *  &lt;property&gt;
135 *    &lt;name&gt;dfs.hosts.include&lt;/name&gt;
136 *    &lt;value&gt;/etc/hadoop/conf/hosts.include&lt;/value&gt;
137 *    <b>&lt;final&gt;true&lt;/final&gt;</b>
138 *  &lt;/property&gt;</pre></tt>
139 *
140 * Administrators typically define parameters as final in 
141 * <tt>core-site.xml</tt> for values that user applications may not alter.
142 *
143 * <h4 id="VariableExpansion">Variable Expansion</h4>
144 *
145 * <p>Value strings are first processed for <i>variable expansion</i>. The
146 * available properties are:<ol>
147 * <li>Other properties defined in this Configuration; and, if a name is
148 * undefined here,</li>
149 * <li>Environment variables in {@link System#getenv()} if a name starts with
150 * "env.", or</li>
151 * <li>Properties in {@link System#getProperties()}.</li>
152 * </ol>
153 *
154 * <p>For example, if a configuration resource contains the following property
155 * definitions: 
156 * <tt><pre>
157 *  &lt;property&gt;
158 *    &lt;name&gt;basedir&lt;/name&gt;
159 *    &lt;value&gt;/user/${<i>user.name</i>}&lt;/value&gt;
160 *  &lt;/property&gt;
161 *  
162 *  &lt;property&gt;
163 *    &lt;name&gt;tempdir&lt;/name&gt;
164 *    &lt;value&gt;${<i>basedir</i>}/tmp&lt;/value&gt;
165 *  &lt;/property&gt;
166 *
167 *  &lt;property&gt;
168 *    &lt;name&gt;otherdir&lt;/name&gt;
169 *    &lt;value&gt;${<i>env.BASE_DIR</i>}/other&lt;/value&gt;
170 *  &lt;/property&gt;
171 *  </pre></tt>
172 *
173 * <p>When <tt>conf.get("tempdir")</tt> is called, then <tt>${<i>basedir</i>}</tt>
174 * will be resolved to another property in this Configuration, while
175 * <tt>${<i>user.name</i>}</tt> would then ordinarily be resolved to the value
176 * of the System property with that name.
177 * <p>When <tt>conf.get("otherdir")</tt> is called, then <tt>${<i>env.BASE_DIR</i>}</tt>
178 * will be resolved to the value of the <tt>${<i>BASE_DIR</i>}</tt> environment variable.
179 * It supports <tt>${<i>env.NAME:-default</i>}</tt> and <tt>${<i>env.NAME-default</i>}</tt> notations.
180 * The former is resolved to ???default??? if <tt>${<i>NAME</i>}</tt> environment variable is undefined
181 * or its value is empty.
182 * The latter behaves the same way only if <tt>${<i>NAME</i>}</tt> is undefined.
183 * <p>By default, warnings will be given to any deprecated configuration 
184 * parameters and these are suppressible by configuring
185 * <tt>log4j.logger.org.apache.hadoop.conf.Configuration.deprecation</tt> in
186 * log4j.properties file.
187 */
188@InterfaceAudience.Public
189@InterfaceStability.Stable
190public class Configuration implements Iterable<Map.Entry<String,String>>,
191                                      Writable {
192  private static final Log LOG =
193    LogFactory.getLog(Configuration.class);
194
195  private static final Log LOG_DEPRECATION =
196    LogFactory.getLog("org.apache.hadoop.conf.Configuration.deprecation");
197
198  private boolean quietmode = true;
199
200  private static final String DEFAULT_STRING_CHECK =
201    "testingforemptydefaultvalue";
202
203  private boolean allowNullValueProperties = false;
204
205  private static class Resource {
206    private final Object resource;
207    private final String name;
208    
209    public Resource(Object resource) {
210      this(resource, resource.toString());
211    }
212    
213    public Resource(Object resource, String name) {
214      this.resource = resource;
215      this.name = name;
216    }
217    
218    public String getName(){
219      return name;
220    }
221    
222    public Object getResource() {
223      return resource;
224    }
225    
226    @Override
227    public String toString() {
228      return name;
229    }
230  }
231  
232  /**
233   * List of configuration resources.
234   */
235  private ArrayList<Resource> resources = new ArrayList<Resource>();
236  
237  /**
238   * The value reported as the setting resource when a key is set
239   * by code rather than a file resource by dumpConfiguration.
240   */
241  static final String UNKNOWN_RESOURCE = "Unknown";
242
243
244  /**
245   * List of configuration parameters marked <b>final</b>. 
246   */
247  private Set<String> finalParameters = Collections.newSetFromMap(
248      new ConcurrentHashMap<String, Boolean>());
249  
250  private boolean loadDefaults = true;
251  
252  /**
253   * Configuration objects
254   */
255  private static final WeakHashMap<Configuration,Object> REGISTRY = 
256    new WeakHashMap<Configuration,Object>();
257  
258  /**
259   * List of default Resources. Resources are loaded in the order of the list 
260   * entries
261   */
262  private static final CopyOnWriteArrayList<String> defaultResources =
263    new CopyOnWriteArrayList<String>();
264
265  private static final Map<ClassLoader, Map<String, WeakReference<Class<?>>>>
266    CACHE_CLASSES = new WeakHashMap<ClassLoader, Map<String, WeakReference<Class<?>>>>();
267
268  /**
269   * Sentinel value to store negative cache results in {@link #CACHE_CLASSES}.
270   */
271  private static final Class<?> NEGATIVE_CACHE_SENTINEL =
272    NegativeCacheSentinel.class;
273
274  /**
275   * Stores the mapping of key to the resource which modifies or loads 
276   * the key most recently
277   */
278  private Map<String, String[]> updatingResource;
279 
280  /**
281   * Class to keep the information about the keys which replace the deprecated
282   * ones.
283   * 
284   * This class stores the new keys which replace the deprecated keys and also
285   * gives a provision to have a custom message for each of the deprecated key
286   * that is being replaced. It also provides method to get the appropriate
287   * warning message which can be logged whenever the deprecated key is used.
288   */
289  private static class DeprecatedKeyInfo {
290    private final String[] newKeys;
291    private final String customMessage;
292    private final AtomicBoolean accessed = new AtomicBoolean(false);
293
294    DeprecatedKeyInfo(String[] newKeys, String customMessage) {
295      this.newKeys = newKeys;
296      this.customMessage = customMessage;
297    }
298
299    /**
300     * Method to provide the warning message. It gives the custom message if
301     * non-null, and default message otherwise.
302     * @param key the associated deprecated key.
303     * @return message that is to be logged when a deprecated key is used.
304     */
305    private final String getWarningMessage(String key) {
306      String warningMessage;
307      if(customMessage == null) {
308        StringBuilder message = new StringBuilder(key);
309        String deprecatedKeySuffix = " is deprecated. Instead, use ";
310        message.append(deprecatedKeySuffix);
311        for (int i = 0; i < newKeys.length; i++) {
312          message.append(newKeys[i]);
313          if(i != newKeys.length-1) {
314            message.append(", ");
315          }
316        }
317        warningMessage = message.toString();
318      }
319      else {
320        warningMessage = customMessage;
321      }
322      return warningMessage;
323    }
324
325    boolean getAndSetAccessed() {
326      return accessed.getAndSet(true);
327    }
328
329    public void clearAccessed() {
330      accessed.set(false);
331    }
332  }
333  
334  /**
335   * A pending addition to the global set of deprecated keys.
336   */
337  public static class DeprecationDelta {
338    private final String key;
339    private final String[] newKeys;
340    private final String customMessage;
341
342    DeprecationDelta(String key, String[] newKeys, String customMessage) {
343      Preconditions.checkNotNull(key);
344      Preconditions.checkNotNull(newKeys);
345      Preconditions.checkArgument(newKeys.length > 0);
346      this.key = key;
347      this.newKeys = newKeys;
348      this.customMessage = customMessage;
349    }
350
351    public DeprecationDelta(String key, String newKey, String customMessage) {
352      this(key, new String[] { newKey }, customMessage);
353    }
354
355    public DeprecationDelta(String key, String newKey) {
356      this(key, new String[] { newKey }, null);
357    }
358
359    public String getKey() {
360      return key;
361    }
362
363    public String[] getNewKeys() {
364      return newKeys;
365    }
366
367    public String getCustomMessage() {
368      return customMessage;
369    }
370  }
371
372  /**
373   * The set of all keys which are deprecated.
374   *
375   * DeprecationContext objects are immutable.
376   */
377  private static class DeprecationContext {
378    /**
379     * Stores the deprecated keys, the new keys which replace the deprecated keys
380     * and custom message(if any provided).
381     */
382    private final Map<String, DeprecatedKeyInfo> deprecatedKeyMap;
383
384    /**
385     * Stores a mapping from superseding keys to the keys which they deprecate.
386     */
387    private final Map<String, String> reverseDeprecatedKeyMap;
388
389    /**
390     * Create a new DeprecationContext by copying a previous DeprecationContext
391     * and adding some deltas.
392     *
393     * @param other   The previous deprecation context to copy, or null to start
394     *                from nothing.
395     * @param deltas  The deltas to apply.
396     */
397    @SuppressWarnings("unchecked")
398    DeprecationContext(DeprecationContext other, DeprecationDelta[] deltas) {
399      HashMap<String, DeprecatedKeyInfo> newDeprecatedKeyMap = 
400        new HashMap<String, DeprecatedKeyInfo>();
401      HashMap<String, String> newReverseDeprecatedKeyMap =
402        new HashMap<String, String>();
403      if (other != null) {
404        for (Entry<String, DeprecatedKeyInfo> entry :
405            other.deprecatedKeyMap.entrySet()) {
406          newDeprecatedKeyMap.put(entry.getKey(), entry.getValue());
407        }
408        for (Entry<String, String> entry :
409            other.reverseDeprecatedKeyMap.entrySet()) {
410          newReverseDeprecatedKeyMap.put(entry.getKey(), entry.getValue());
411        }
412      }
413      for (DeprecationDelta delta : deltas) {
414        if (!newDeprecatedKeyMap.containsKey(delta.getKey())) {
415          DeprecatedKeyInfo newKeyInfo =
416            new DeprecatedKeyInfo(delta.getNewKeys(), delta.getCustomMessage());
417          newDeprecatedKeyMap.put(delta.key, newKeyInfo);
418          for (String newKey : delta.getNewKeys()) {
419            newReverseDeprecatedKeyMap.put(newKey, delta.key);
420          }
421        }
422      }
423      this.deprecatedKeyMap =
424        UnmodifiableMap.decorate(newDeprecatedKeyMap);
425      this.reverseDeprecatedKeyMap =
426        UnmodifiableMap.decorate(newReverseDeprecatedKeyMap);
427    }
428
429    Map<String, DeprecatedKeyInfo> getDeprecatedKeyMap() {
430      return deprecatedKeyMap;
431    }
432
433    Map<String, String> getReverseDeprecatedKeyMap() {
434      return reverseDeprecatedKeyMap;
435    }
436  }
437  
438  private static DeprecationDelta[] defaultDeprecations = 
439    new DeprecationDelta[] {
440      new DeprecationDelta("topology.script.file.name", 
441        CommonConfigurationKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY),
442      new DeprecationDelta("topology.script.number.args", 
443        CommonConfigurationKeys.NET_TOPOLOGY_SCRIPT_NUMBER_ARGS_KEY),
444      new DeprecationDelta("hadoop.configured.node.mapping", 
445        CommonConfigurationKeys.NET_TOPOLOGY_CONFIGURED_NODE_MAPPING_KEY),
446      new DeprecationDelta("topology.node.switch.mapping.impl", 
447        CommonConfigurationKeys.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY),
448      new DeprecationDelta("dfs.df.interval", 
449        CommonConfigurationKeys.FS_DF_INTERVAL_KEY),
450      new DeprecationDelta("fs.default.name", 
451        CommonConfigurationKeys.FS_DEFAULT_NAME_KEY),
452      new DeprecationDelta("dfs.umaskmode",
453        CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY),
454      new DeprecationDelta("dfs.nfs.exports.allowed.hosts",
455          CommonConfigurationKeys.NFS_EXPORTS_ALLOWED_HOSTS_KEY)
456    };
457
458  /**
459   * The global DeprecationContext.
460   */
461  private static AtomicReference<DeprecationContext> deprecationContext =
462      new AtomicReference<DeprecationContext>(
463          new DeprecationContext(null, defaultDeprecations));
464
465  /**
466   * Adds a set of deprecated keys to the global deprecations.
467   *
468   * This method is lockless.  It works by means of creating a new
469   * DeprecationContext based on the old one, and then atomically swapping in
470   * the new context.  If someone else updated the context in between us reading
471   * the old context and swapping in the new one, we try again until we win the
472   * race.
473   *
474   * @param deltas   The deprecations to add.
475   */
476  public static void addDeprecations(DeprecationDelta[] deltas) {
477    DeprecationContext prev, next;
478    do {
479      prev = deprecationContext.get();
480      next = new DeprecationContext(prev, deltas);
481    } while (!deprecationContext.compareAndSet(prev, next));
482  }
483
484  /**
485   * Adds the deprecated key to the global deprecation map.
486   * It does not override any existing entries in the deprecation map.
487   * This is to be used only by the developers in order to add deprecation of
488   * keys, and attempts to call this method after loading resources once,
489   * would lead to <tt>UnsupportedOperationException</tt>
490   * 
491   * If a key is deprecated in favor of multiple keys, they are all treated as 
492   * aliases of each other, and setting any one of them resets all the others 
493   * to the new value.
494   *
495   * If you have multiple deprecation entries to add, it is more efficient to
496   * use #addDeprecations(DeprecationDelta[] deltas) instead.
497   * 
498   * @param key
499   * @param newKeys
500   * @param customMessage
501   * @deprecated use {@link #addDeprecation(String key, String newKey,
502      String customMessage)} instead
503   */
504  @Deprecated
505  public static void addDeprecation(String key, String[] newKeys,
506      String customMessage) {
507    addDeprecations(new DeprecationDelta[] {
508      new DeprecationDelta(key, newKeys, customMessage)
509    });
510  }
511
512  /**
513   * Adds the deprecated key to the global deprecation map.
514   * It does not override any existing entries in the deprecation map.
515   * This is to be used only by the developers in order to add deprecation of
516   * keys, and attempts to call this method after loading resources once,
517   * would lead to <tt>UnsupportedOperationException</tt>
518   * 
519   * If you have multiple deprecation entries to add, it is more efficient to
520   * use #addDeprecations(DeprecationDelta[] deltas) instead.
521   *
522   * @param key
523   * @param newKey
524   * @param customMessage
525   */
526  public static void addDeprecation(String key, String newKey,
527              String customMessage) {
528          addDeprecation(key, new String[] {newKey}, customMessage);
529  }
530
531  /**
532   * Adds the deprecated key to the global deprecation map when no custom
533   * message is provided.
534   * It does not override any existing entries in the deprecation map.
535   * This is to be used only by the developers in order to add deprecation of
536   * keys, and attempts to call this method after loading resources once,
537   * would lead to <tt>UnsupportedOperationException</tt>
538   * 
539   * If a key is deprecated in favor of multiple keys, they are all treated as 
540   * aliases of each other, and setting any one of them resets all the others 
541   * to the new value.
542   * 
543   * If you have multiple deprecation entries to add, it is more efficient to
544   * use #addDeprecations(DeprecationDelta[] deltas) instead.
545   *
546   * @param key Key that is to be deprecated
547   * @param newKeys list of keys that take up the values of deprecated key
548   * @deprecated use {@link #addDeprecation(String key, String newKey)} instead
549   */
550  @Deprecated
551  public static void addDeprecation(String key, String[] newKeys) {
552    addDeprecation(key, newKeys, null);
553  }
554  
555  /**
556   * Adds the deprecated key to the global deprecation map when no custom
557   * message is provided.
558   * It does not override any existing entries in the deprecation map.
559   * This is to be used only by the developers in order to add deprecation of
560   * keys, and attempts to call this method after loading resources once,
561   * would lead to <tt>UnsupportedOperationException</tt>
562   * 
563   * If you have multiple deprecation entries to add, it is more efficient to
564   * use #addDeprecations(DeprecationDelta[] deltas) instead.
565   *
566   * @param key Key that is to be deprecated
567   * @param newKey key that takes up the value of deprecated key
568   */
569  public static void addDeprecation(String key, String newKey) {
570    addDeprecation(key, new String[] {newKey}, null);
571  }
572  
573  /**
574   * checks whether the given <code>key</code> is deprecated.
575   * 
576   * @param key the parameter which is to be checked for deprecation
577   * @return <code>true</code> if the key is deprecated and 
578   *         <code>false</code> otherwise.
579   */
580  public static boolean isDeprecated(String key) {
581    return deprecationContext.get().getDeprecatedKeyMap().containsKey(key);
582  }
583
584  /**
585   * Sets all deprecated properties that are not currently set but have a
586   * corresponding new property that is set. Useful for iterating the
587   * properties when all deprecated properties for currently set properties
588   * need to be present.
589   */
590  public void setDeprecatedProperties() {
591    DeprecationContext deprecations = deprecationContext.get();
592    Properties props = getProps();
593    Properties overlay = getOverlay();
594    for (Map.Entry<String, DeprecatedKeyInfo> entry :
595        deprecations.getDeprecatedKeyMap().entrySet()) {
596      String depKey = entry.getKey();
597      if (!overlay.contains(depKey)) {
598        for (String newKey : entry.getValue().newKeys) {
599          String val = overlay.getProperty(newKey);
600          if (val != null) {
601            props.setProperty(depKey, val);
602            overlay.setProperty(depKey, val);
603            break;
604          }
605        }
606      }
607    }
608  }
609
610  /**
611   * Checks for the presence of the property <code>name</code> in the
612   * deprecation map. Returns the first of the list of new keys if present
613   * in the deprecation map or the <code>name</code> itself. If the property
614   * is not presently set but the property map contains an entry for the
615   * deprecated key, the value of the deprecated key is set as the value for
616   * the provided property name.
617   *
618   * @param name the property name
619   * @return the first property in the list of properties mapping
620   *         the <code>name</code> or the <code>name</code> itself.
621   */
622  private String[] handleDeprecation(DeprecationContext deprecations,
623      String name) {
624    if (null != name) {
625      name = name.trim();
626    }
627    ArrayList<String > names = new ArrayList<String>();
628        if (isDeprecated(name)) {
629      DeprecatedKeyInfo keyInfo = deprecations.getDeprecatedKeyMap().get(name);
630      warnOnceIfDeprecated(deprecations, name);
631      for (String newKey : keyInfo.newKeys) {
632        if(newKey != null) {
633          names.add(newKey);
634        }
635      }
636    }
637    if(names.size() == 0) {
638        names.add(name);
639    }
640    for(String n : names) {
641          String deprecatedKey = deprecations.getReverseDeprecatedKeyMap().get(n);
642          if (deprecatedKey != null && !getOverlay().containsKey(n) &&
643              getOverlay().containsKey(deprecatedKey)) {
644            getProps().setProperty(n, getOverlay().getProperty(deprecatedKey));
645            getOverlay().setProperty(n, getOverlay().getProperty(deprecatedKey));
646          }
647    }
648    return names.toArray(new String[names.size()]);
649  }
650 
651  private void handleDeprecation() {
652    LOG.debug("Handling deprecation for all properties in config...");
653    DeprecationContext deprecations = deprecationContext.get();
654    Set<Object> keys = new HashSet<Object>();
655    keys.addAll(getProps().keySet());
656    for (Object item: keys) {
657      LOG.debug("Handling deprecation for " + (String)item);
658      handleDeprecation(deprecations, (String)item);
659    }
660  }
661 
662  static{
663    //print deprecation warning if hadoop-site.xml is found in classpath
664    ClassLoader cL = Thread.currentThread().getContextClassLoader();
665    if (cL == null) {
666      cL = Configuration.class.getClassLoader();
667    }
668    if(cL.getResource("hadoop-site.xml")!=null) {
669      LOG.warn("DEPRECATED: hadoop-site.xml found in the classpath. " +
670          "Usage of hadoop-site.xml is deprecated. Instead use core-site.xml, "
671          + "mapred-site.xml and hdfs-site.xml to override properties of " +
672          "core-default.xml, mapred-default.xml and hdfs-default.xml " +
673          "respectively");
674    }
675    addDefaultResource("core-default.xml");
676    addDefaultResource("core-site.xml");
677  }
678  
679  private Properties properties;
680  private Properties overlay;
681  private ClassLoader classLoader;
682  {
683    classLoader = Thread.currentThread().getContextClassLoader();
684    if (classLoader == null) {
685      classLoader = Configuration.class.getClassLoader();
686    }
687  }
688  
689  /** A new configuration. */
690  public Configuration() {
691    this(true);
692  }
693
694  /** A new configuration where the behavior of reading from the default 
695   * resources can be turned off.
696   * 
697   * If the parameter {@code loadDefaults} is false, the new instance
698   * will not load resources from the default files. 
699   * @param loadDefaults specifies whether to load from the default files
700   */
701  public Configuration(boolean loadDefaults) {
702    this.loadDefaults = loadDefaults;
703    updatingResource = new ConcurrentHashMap<String, String[]>();
704    synchronized(Configuration.class) {
705      REGISTRY.put(this, null);
706    }
707  }
708  
709  /** 
710   * A new configuration with the same settings cloned from another.
711   * 
712   * @param other the configuration from which to clone settings.
713   */
714  @SuppressWarnings("unchecked")
715  public Configuration(Configuration other) {
716   this.resources = (ArrayList<Resource>) other.resources.clone();
717   synchronized(other) {
718     if (other.properties != null) {
719       this.properties = (Properties)other.properties.clone();
720     }
721
722     if (other.overlay!=null) {
723       this.overlay = (Properties)other.overlay.clone();
724     }
725
726     this.updatingResource = new ConcurrentHashMap<String, String[]>(
727         other.updatingResource);
728     this.finalParameters = Collections.newSetFromMap(
729         new ConcurrentHashMap<String, Boolean>());
730     this.finalParameters.addAll(other.finalParameters);
731   }
732   
733    synchronized(Configuration.class) {
734      REGISTRY.put(this, null);
735    }
736    this.classLoader = other.classLoader;
737    this.loadDefaults = other.loadDefaults;
738    setQuietMode(other.getQuietMode());
739  }
740  
741  /**
742   * Add a default resource. Resources are loaded in the order of the resources 
743   * added.
744   * @param name file name. File should be present in the classpath.
745   */
746  public static synchronized void addDefaultResource(String name) {
747    if(!defaultResources.contains(name)) {
748      defaultResources.add(name);
749      for(Configuration conf : REGISTRY.keySet()) {
750        if(conf.loadDefaults) {
751          conf.reloadConfiguration();
752        }
753      }
754    }
755  }
756
757  /**
758   * Add a configuration resource. 
759   * 
760   * The properties of this resource will override properties of previously 
761   * added resources, unless they were marked <a href="#Final">final</a>. 
762   * 
763   * @param name resource to be added, the classpath is examined for a file 
764   *             with that name.
765   */
766  public void addResource(String name) {
767    addResourceObject(new Resource(name));
768  }
769
770  /**
771   * Add a configuration resource. 
772   * 
773   * The properties of this resource will override properties of previously 
774   * added resources, unless they were marked <a href="#Final">final</a>. 
775   * 
776   * @param url url of the resource to be added, the local filesystem is 
777   *            examined directly to find the resource, without referring to 
778   *            the classpath.
779   */
780  public void addResource(URL url) {
781    addResourceObject(new Resource(url));
782  }
783
784  /**
785   * Add a configuration resource. 
786   * 
787   * The properties of this resource will override properties of previously 
788   * added resources, unless they were marked <a href="#Final">final</a>. 
789   * 
790   * @param file file-path of resource to be added, the local filesystem is
791   *             examined directly to find the resource, without referring to 
792   *             the classpath.
793   */
794  public void addResource(Path file) {
795    addResourceObject(new Resource(file));
796  }
797
798  /**
799   * Add a configuration resource. 
800   * 
801   * The properties of this resource will override properties of previously 
802   * added resources, unless they were marked <a href="#Final">final</a>. 
803   * 
804   * WARNING: The contents of the InputStream will be cached, by this method. 
805   * So use this sparingly because it does increase the memory consumption.
806   * 
807   * @param in InputStream to deserialize the object from. In will be read from
808   * when a get or set is called next.  After it is read the stream will be
809   * closed. 
810   */
811  public void addResource(InputStream in) {
812    addResourceObject(new Resource(in));
813  }
814
815  /**
816   * Add a configuration resource. 
817   * 
818   * The properties of this resource will override properties of previously 
819   * added resources, unless they were marked <a href="#Final">final</a>. 
820   * 
821   * @param in InputStream to deserialize the object from.
822   * @param name the name of the resource because InputStream.toString is not
823   * very descriptive some times.  
824   */
825  public void addResource(InputStream in, String name) {
826    addResourceObject(new Resource(in, name));
827  }
828  
829  /**
830   * Add a configuration resource.
831   *
832   * The properties of this resource will override properties of previously
833   * added resources, unless they were marked <a href="#Final">final</a>.
834   *
835   * @param conf Configuration object from which to load properties
836   */
837  public void addResource(Configuration conf) {
838    addResourceObject(new Resource(conf.getProps()));
839  }
840
841  
842  
843  /**
844   * Reload configuration from previously added resources.
845   *
846   * This method will clear all the configuration read from the added 
847   * resources, and final parameters. This will make the resources to 
848   * be read again before accessing the values. Values that are added
849   * via set methods will overlay values read from the resources.
850   */
851  public synchronized void reloadConfiguration() {
852    properties = null;                            // trigger reload
853    finalParameters.clear();                      // clear site-limits
854  }
855  
856  private synchronized void addResourceObject(Resource resource) {
857    resources.add(resource);                      // add to resources
858    reloadConfiguration();
859  }
860
861  private static final int MAX_SUBST = 20;
862
863  private static final int SUB_START_IDX = 0;
864  private static final int SUB_END_IDX = SUB_START_IDX + 1;
865
866  /**
867   * This is a manual implementation of the following regex
868   * "\\$\\{[^\\}\\$\u0020]+\\}". It can be 15x more efficient than
869   * a regex matcher as demonstrated by HADOOP-11506. This is noticeable with
870   * Hadoop apps building on the assumption Configuration#get is an O(1)
871   * hash table lookup, especially when the eval is a long string.
872   *
873   * @param eval a string that may contain variables requiring expansion.
874   * @return a 2-element int array res such that
875   * eval.substring(res[0], res[1]) is "var" for the left-most occurrence of
876   * ${var} in eval. If no variable is found -1, -1 is returned.
877   */
878  private static int[] findSubVariable(String eval) {
879    int[] result = {-1, -1};
880
881    int matchStart;
882    int leftBrace;
883
884    // scanning for a brace first because it's less frequent than $
885    // that can occur in nested class names
886    //
887    match_loop:
888    for (matchStart = 1, leftBrace = eval.indexOf('{', matchStart);
889         // minimum left brace position (follows '$')
890         leftBrace > 0
891         // right brace of a smallest valid expression "${c}"
892         && leftBrace + "{c".length() < eval.length();
893         leftBrace = eval.indexOf('{', matchStart)) {
894      int matchedLen = 0;
895      if (eval.charAt(leftBrace - 1) == '$') {
896        int subStart = leftBrace + 1; // after '{'
897        for (int i = subStart; i < eval.length(); i++) {
898          switch (eval.charAt(i)) {
899            case '}':
900              if (matchedLen > 0) { // match
901                result[SUB_START_IDX] = subStart;
902                result[SUB_END_IDX] = subStart + matchedLen;
903                break match_loop;
904              }
905              // fall through to skip 1 char
906            case ' ':
907            case '$':
908              matchStart = i + 1;
909              continue match_loop;
910            default:
911              matchedLen++;
912          }
913        }
914        // scanned from "${"  to the end of eval, and no reset via ' ', '$':
915        //    no match!
916        break match_loop;
917      } else {
918        // not a start of a variable
919        //
920        matchStart = leftBrace + 1;
921      }
922    }
923    return result;
924  }
925
926  /**
927   * Attempts to repeatedly expand the value {@code expr} by replacing the
928   * left-most substring of the form "${var}" in the following precedence order
929   * <ol>
930   *   <li>by the value of the environment variable "var" if defined</li>
931   *   <li>by the value of the Java system property "var" if defined</li>
932   *   <li>by the value of the configuration key "var" if defined</li>
933   * </ol>
934   *
935   * If var is unbounded the current state of expansion "prefix${var}suffix" is
936   * returned.
937   *
938   * If a cycle is detected: replacing var1 requires replacing var2 ... requires
939   * replacing var1, i.e., the cycle is shorter than
940   * {@link Configuration#MAX_SUBST} then the original expr is returned.
941   *
942   * @param expr the literal value of a config key
943   * @return null if expr is null, otherwise the value resulting from expanding
944   * expr using the algorithm above.
945   * @throws IllegalArgumentException when more than
946   * {@link Configuration#MAX_SUBST} replacements are required
947   */
948  private String substituteVars(String expr) {
949    if (expr == null) {
950      return null;
951    }
952    String eval = expr;
953    Set<String> evalSet = null;
954    for(int s = 0; s < MAX_SUBST; s++) {
955      final int[] varBounds = findSubVariable(eval);
956      if (varBounds[SUB_START_IDX] == -1) {
957        return eval;
958      }
959      final String var = eval.substring(varBounds[SUB_START_IDX],
960          varBounds[SUB_END_IDX]);
961      String val = null;
962      try {
963        if (var.startsWith("env.") && 4 < var.length()) {
964          String v = var.substring(4);
965          int i = 0;
966          for (; i < v.length(); i++) {
967            char c = v.charAt(i);
968            if (c == ':' && i < v.length() - 1 && v.charAt(i + 1) == '-') {
969              val = getenv(v.substring(0, i));
970              if (val == null || val.length() == 0) {
971                val = v.substring(i + 2);
972              }
973              break;
974            } else if (c == '-') {
975              val = getenv(v.substring(0, i));
976              if (val == null) {
977                val = v.substring(i + 1);
978              }
979              break;
980            }
981          }
982          if (i == v.length()) {
983            val = getenv(v);
984          }
985        } else {
986          val = getProperty(var);
987        }
988      } catch(SecurityException se) {
989        LOG.warn("Unexpected SecurityException in Configuration", se);
990      }
991      if (val == null || val.isEmpty()) {
992        String raw = getRaw(var);
993        if (raw != null) {
994          // if System.getProperty(var) returns an empty string, retain this
995          // value instead of return null
996          val = raw;
997        }
998      }
999      if (val == null) {
1000        return eval; // return literal ${var}: var is unbound
1001      }
1002
1003      // prevent recursive resolution
1004      //
1005      final int dollar = varBounds[SUB_START_IDX] - "${".length();
1006      final int afterRightBrace = varBounds[SUB_END_IDX] + "}".length();
1007      final String refVar = eval.substring(dollar, afterRightBrace);
1008      if (evalSet == null) {
1009        evalSet = new HashSet<String>();
1010      }
1011      if (!evalSet.add(refVar)) {
1012        return expr; // return original expression if there is a loop
1013      }
1014
1015      // substitute
1016      eval = eval.substring(0, dollar)
1017             + val
1018             + eval.substring(afterRightBrace);
1019    }
1020    throw new IllegalStateException("Variable substitution depth too large: " 
1021                                    + MAX_SUBST + " " + expr);
1022  }
1023  
1024  String getenv(String name) {
1025    return System.getenv(name);
1026  }
1027
1028  String getProperty(String key) {
1029    return System.getProperty(key);
1030  }
1031
1032  /**
1033   * Get the value of the <code>name</code> property, <code>null</code> if
1034   * no such property exists. If the key is deprecated, it returns the value of
1035   * the first key which replaces the deprecated key and is not null.
1036   * 
1037   * Values are processed for <a href="#VariableExpansion">variable expansion</a> 
1038   * before being returned. 
1039   * 
1040   * @param name the property name, will be trimmed before get value.
1041   * @return the value of the <code>name</code> or its replacing property, 
1042   *         or null if no such property exists.
1043   */
1044  public String get(String name) {
1045    String[] names = handleDeprecation(deprecationContext.get(), name);
1046    String result = null;
1047    for(String n : names) {
1048      result = substituteVars(getProps().getProperty(n));
1049    }
1050    return result;
1051  }
1052
1053  /**
1054   * Set Configuration to allow keys without values during setup.  Intended
1055   * for use during testing.
1056   *
1057   * @param val If true, will allow Configuration to store keys without values
1058   */
1059  @VisibleForTesting
1060  public void setAllowNullValueProperties( boolean val ) {
1061    this.allowNullValueProperties = val;
1062  }
1063
1064  /**
1065   * Return existence of the <code>name</code> property, but only for
1066   * names which have no valid value, usually non-existent or commented
1067   * out in XML.
1068   *
1069   * @param name the property name
1070   * @return true if the property <code>name</code> exists without value
1071   */
1072  @VisibleForTesting
1073  public boolean onlyKeyExists(String name) {
1074    String[] names = handleDeprecation(deprecationContext.get(), name);
1075    for(String n : names) {
1076      if ( getProps().getProperty(n,DEFAULT_STRING_CHECK)
1077               .equals(DEFAULT_STRING_CHECK) ) {
1078        return true;
1079      }
1080    }
1081    return false;
1082  }
1083
1084  /**
1085   * Get the value of the <code>name</code> property as a trimmed <code>String</code>, 
1086   * <code>null</code> if no such property exists. 
1087   * If the key is deprecated, it returns the value of
1088   * the first key which replaces the deprecated key and is not null
1089   * 
1090   * Values are processed for <a href="#VariableExpansion">variable expansion</a> 
1091   * before being returned. 
1092   * 
1093   * @param name the property name.
1094   * @return the value of the <code>name</code> or its replacing property, 
1095   *         or null if no such property exists.
1096   */
1097  public String getTrimmed(String name) {
1098    String value = get(name);
1099    
1100    if (null == value) {
1101      return null;
1102    } else {
1103      return value.trim();
1104    }
1105  }
1106  
1107  /**
1108   * Get the value of the <code>name</code> property as a trimmed <code>String</code>, 
1109   * <code>defaultValue</code> if no such property exists. 
1110   * See @{Configuration#getTrimmed} for more details.
1111   * 
1112   * @param name          the property name.
1113   * @param defaultValue  the property default value.
1114   * @return              the value of the <code>name</code> or defaultValue
1115   *                      if it is not set.
1116   */
1117  public String getTrimmed(String name, String defaultValue) {
1118    String ret = getTrimmed(name);
1119    return ret == null ? defaultValue : ret;
1120  }
1121
1122  /**
1123   * Get the value of the <code>name</code> property, without doing
1124   * <a href="#VariableExpansion">variable expansion</a>.If the key is 
1125   * deprecated, it returns the value of the first key which replaces 
1126   * the deprecated key and is not null.
1127   * 
1128   * @param name the property name.
1129   * @return the value of the <code>name</code> property or 
1130   *         its replacing property and null if no such property exists.
1131   */
1132  public String getRaw(String name) {
1133    String[] names = handleDeprecation(deprecationContext.get(), name);
1134    String result = null;
1135    for(String n : names) {
1136      result = getProps().getProperty(n);
1137    }
1138    return result;
1139  }
1140
1141  /**
1142   * Returns alternative names (non-deprecated keys or previously-set deprecated keys)
1143   * for a given non-deprecated key.
1144   * If the given key is deprecated, return null.
1145   *
1146   * @param name property name.
1147   * @return alternative names.
1148   */
1149  private String[] getAlternativeNames(String name) {
1150    String altNames[] = null;
1151    DeprecatedKeyInfo keyInfo = null;
1152    DeprecationContext cur = deprecationContext.get();
1153    String depKey = cur.getReverseDeprecatedKeyMap().get(name);
1154    if(depKey != null) {
1155      keyInfo = cur.getDeprecatedKeyMap().get(depKey);
1156      if(keyInfo.newKeys.length > 0) {
1157        if(getProps().containsKey(depKey)) {
1158          //if deprecated key is previously set explicitly
1159          List<String> list = new ArrayList<String>();
1160          list.addAll(Arrays.asList(keyInfo.newKeys));
1161          list.add(depKey);
1162          altNames = list.toArray(new String[list.size()]);
1163        }
1164        else {
1165          altNames = keyInfo.newKeys;
1166        }
1167      }
1168    }
1169    return altNames;
1170  }
1171
1172  /** 
1173   * Set the <code>value</code> of the <code>name</code> property. If 
1174   * <code>name</code> is deprecated or there is a deprecated name associated to it,
1175   * it sets the value to both names. Name will be trimmed before put into
1176   * configuration.
1177   * 
1178   * @param name property name.
1179   * @param value property value.
1180   */
1181  public void set(String name, String value) {
1182    set(name, value, null);
1183  }
1184  
1185  /** 
1186   * Set the <code>value</code> of the <code>name</code> property. If 
1187   * <code>name</code> is deprecated, it also sets the <code>value</code> to
1188   * the keys that replace the deprecated key. Name will be trimmed before put
1189   * into configuration.
1190   *
1191   * @param name property name.
1192   * @param value property value.
1193   * @param source the place that this configuration value came from 
1194   * (For debugging).
1195   * @throws IllegalArgumentException when the value or name is null.
1196   */
1197  public void set(String name, String value, String source) {
1198    Preconditions.checkArgument(
1199        name != null,
1200        "Property name must not be null");
1201    Preconditions.checkArgument(
1202        value != null,
1203        "The value of property " + name + " must not be null");
1204    name = name.trim();
1205    DeprecationContext deprecations = deprecationContext.get();
1206    if (deprecations.getDeprecatedKeyMap().isEmpty()) {
1207      getProps();
1208    }
1209    getOverlay().setProperty(name, value);
1210    getProps().setProperty(name, value);
1211    String newSource = (source == null ? "programmatically" : source);
1212
1213    if (!isDeprecated(name)) {
1214      updatingResource.put(name, new String[] {newSource});
1215      String[] altNames = getAlternativeNames(name);
1216      if(altNames != null) {
1217        for(String n: altNames) {
1218          if(!n.equals(name)) {
1219            getOverlay().setProperty(n, value);
1220            getProps().setProperty(n, value);
1221            updatingResource.put(n, new String[] {newSource});
1222          }
1223        }
1224      }
1225    }
1226    else {
1227      String[] names = handleDeprecation(deprecationContext.get(), name);
1228      String altSource = "because " + name + " is deprecated";
1229      for(String n : names) {
1230        getOverlay().setProperty(n, value);
1231        getProps().setProperty(n, value);
1232        updatingResource.put(n, new String[] {altSource});
1233      }
1234    }
1235  }
1236
1237  private void warnOnceIfDeprecated(DeprecationContext deprecations, String name) {
1238    DeprecatedKeyInfo keyInfo = deprecations.getDeprecatedKeyMap().get(name);
1239    if (keyInfo != null && !keyInfo.getAndSetAccessed()) {
1240      LOG_DEPRECATION.info(keyInfo.getWarningMessage(name));
1241    }
1242  }
1243
1244  /**
1245   * Unset a previously set property.
1246   */
1247  public synchronized void unset(String name) {
1248    String[] names = null;
1249    if (!isDeprecated(name)) {
1250      names = getAlternativeNames(name);
1251      if(names == null) {
1252          names = new String[]{name};
1253      }
1254    }
1255    else {
1256      names = handleDeprecation(deprecationContext.get(), name);
1257    }
1258
1259    for(String n: names) {
1260      getOverlay().remove(n);
1261      getProps().remove(n);
1262    }
1263  }
1264
1265  /**
1266   * Sets a property if it is currently unset.
1267   * @param name the property name
1268   * @param value the new value
1269   */
1270  public synchronized void setIfUnset(String name, String value) {
1271    if (get(name) == null) {
1272      set(name, value);
1273    }
1274  }
1275  
1276  private synchronized Properties getOverlay() {
1277    if (overlay==null){
1278      overlay=new Properties();
1279    }
1280    return overlay;
1281  }
1282
1283  /** 
1284   * Get the value of the <code>name</code>. If the key is deprecated,
1285   * it returns the value of the first key which replaces the deprecated key
1286   * and is not null.
1287   * If no such property exists,
1288   * then <code>defaultValue</code> is returned.
1289   * 
1290   * @param name property name, will be trimmed before get value.
1291   * @param defaultValue default value.
1292   * @return property value, or <code>defaultValue</code> if the property 
1293   *         doesn't exist.                    
1294   */
1295  public String get(String name, String defaultValue) {
1296    String[] names = handleDeprecation(deprecationContext.get(), name);
1297    String result = null;
1298    for(String n : names) {
1299      result = substituteVars(getProps().getProperty(n, defaultValue));
1300    }
1301    return result;
1302  }
1303
1304  /** 
1305   * Get the value of the <code>name</code> property as an <code>int</code>.
1306   *   
1307   * If no such property exists, the provided default value is returned,
1308   * or if the specified value is not a valid <code>int</code>,
1309   * then an error is thrown.
1310   * 
1311   * @param name property name.
1312   * @param defaultValue default value.
1313   * @throws NumberFormatException when the value is invalid
1314   * @return property value as an <code>int</code>, 
1315   *         or <code>defaultValue</code>. 
1316   */
1317  public int getInt(String name, int defaultValue) {
1318    String valueString = getTrimmed(name);
1319    if (valueString == null)
1320      return defaultValue;
1321    String hexString = getHexDigits(valueString);
1322    if (hexString != null) {
1323      return Integer.parseInt(hexString, 16);
1324    }
1325    return Integer.parseInt(valueString);
1326  }
1327  
1328  /**
1329   * Get the value of the <code>name</code> property as a set of comma-delimited
1330   * <code>int</code> values.
1331   * 
1332   * If no such property exists, an empty array is returned.
1333   * 
1334   * @param name property name
1335   * @return property value interpreted as an array of comma-delimited
1336   *         <code>int</code> values
1337   */
1338  public int[] getInts(String name) {
1339    String[] strings = getTrimmedStrings(name);
1340    int[] ints = new int[strings.length];
1341    for (int i = 0; i < strings.length; i++) {
1342      ints[i] = Integer.parseInt(strings[i]);
1343    }
1344    return ints;
1345  }
1346
1347  /** 
1348   * Set the value of the <code>name</code> property to an <code>int</code>.
1349   * 
1350   * @param name property name.
1351   * @param value <code>int</code> value of the property.
1352   */
1353  public void setInt(String name, int value) {
1354    set(name, Integer.toString(value));
1355  }
1356
1357
1358  /** 
1359   * Get the value of the <code>name</code> property as a <code>long</code>.  
1360   * If no such property exists, the provided default value is returned,
1361   * or if the specified value is not a valid <code>long</code>,
1362   * then an error is thrown.
1363   * 
1364   * @param name property name.
1365   * @param defaultValue default value.
1366   * @throws NumberFormatException when the value is invalid
1367   * @return property value as a <code>long</code>, 
1368   *         or <code>defaultValue</code>. 
1369   */
1370  public long getLong(String name, long defaultValue) {
1371    String valueString = getTrimmed(name);
1372    if (valueString == null)
1373      return defaultValue;
1374    String hexString = getHexDigits(valueString);
1375    if (hexString != null) {
1376      return Long.parseLong(hexString, 16);
1377    }
1378    return Long.parseLong(valueString);
1379  }
1380
1381  /**
1382   * Get the value of the <code>name</code> property as a <code>long</code> or
1383   * human readable format. If no such property exists, the provided default
1384   * value is returned, or if the specified value is not a valid
1385   * <code>long</code> or human readable format, then an error is thrown. You
1386   * can use the following suffix (case insensitive): k(kilo), m(mega), g(giga),
1387   * t(tera), p(peta), e(exa)
1388   *
1389   * @param name property name.
1390   * @param defaultValue default value.
1391   * @throws NumberFormatException when the value is invalid
1392   * @return property value as a <code>long</code>,
1393   *         or <code>defaultValue</code>.
1394   */
1395  public long getLongBytes(String name, long defaultValue) {
1396    String valueString = getTrimmed(name);
1397    if (valueString == null)
1398      return defaultValue;
1399    return StringUtils.TraditionalBinaryPrefix.string2long(valueString);
1400  }
1401
1402  private String getHexDigits(String value) {
1403    boolean negative = false;
1404    String str = value;
1405    String hexString = null;
1406    if (value.startsWith("-")) {
1407      negative = true;
1408      str = value.substring(1);
1409    }
1410    if (str.startsWith("0x") || str.startsWith("0X")) {
1411      hexString = str.substring(2);
1412      if (negative) {
1413        hexString = "-" + hexString;
1414      }
1415      return hexString;
1416    }
1417    return null;
1418  }
1419  
1420  /** 
1421   * Set the value of the <code>name</code> property to a <code>long</code>.
1422   * 
1423   * @param name property name.
1424   * @param value <code>long</code> value of the property.
1425   */
1426  public void setLong(String name, long value) {
1427    set(name, Long.toString(value));
1428  }
1429
1430  /** 
1431   * Get the value of the <code>name</code> property as a <code>float</code>.  
1432   * If no such property exists, the provided default value is returned,
1433   * or if the specified value is not a valid <code>float</code>,
1434   * then an error is thrown.
1435   *
1436   * @param name property name.
1437   * @param defaultValue default value.
1438   * @throws NumberFormatException when the value is invalid
1439   * @return property value as a <code>float</code>, 
1440   *         or <code>defaultValue</code>. 
1441   */
1442  public float getFloat(String name, float defaultValue) {
1443    String valueString = getTrimmed(name);
1444    if (valueString == null)
1445      return defaultValue;
1446    return Float.parseFloat(valueString);
1447  }
1448
1449  /**
1450   * Set the value of the <code>name</code> property to a <code>float</code>.
1451   * 
1452   * @param name property name.
1453   * @param value property value.
1454   */
1455  public void setFloat(String name, float value) {
1456    set(name,Float.toString(value));
1457  }
1458
1459  /** 
1460   * Get the value of the <code>name</code> property as a <code>double</code>.  
1461   * If no such property exists, the provided default value is returned,
1462   * or if the specified value is not a valid <code>double</code>,
1463   * then an error is thrown.
1464   *
1465   * @param name property name.
1466   * @param defaultValue default value.
1467   * @throws NumberFormatException when the value is invalid
1468   * @return property value as a <code>double</code>, 
1469   *         or <code>defaultValue</code>. 
1470   */
1471  public double getDouble(String name, double defaultValue) {
1472    String valueString = getTrimmed(name);
1473    if (valueString == null)
1474      return defaultValue;
1475    return Double.parseDouble(valueString);
1476  }
1477
1478  /**
1479   * Set the value of the <code>name</code> property to a <code>double</code>.
1480   * 
1481   * @param name property name.
1482   * @param value property value.
1483   */
1484  public void setDouble(String name, double value) {
1485    set(name,Double.toString(value));
1486  }
1487 
1488  /** 
1489   * Get the value of the <code>name</code> property as a <code>boolean</code>.  
1490   * If no such property is specified, or if the specified value is not a valid
1491   * <code>boolean</code>, then <code>defaultValue</code> is returned.
1492   * 
1493   * @param name property name.
1494   * @param defaultValue default value.
1495   * @return property value as a <code>boolean</code>, 
1496   *         or <code>defaultValue</code>. 
1497   */
1498  public boolean getBoolean(String name, boolean defaultValue) {
1499    String valueString = getTrimmed(name);
1500    if (null == valueString || valueString.isEmpty()) {
1501      return defaultValue;
1502    }
1503
1504    if (StringUtils.equalsIgnoreCase("true", valueString))
1505      return true;
1506    else if (StringUtils.equalsIgnoreCase("false", valueString))
1507      return false;
1508    else return defaultValue;
1509  }
1510
1511  /** 
1512   * Set the value of the <code>name</code> property to a <code>boolean</code>.
1513   * 
1514   * @param name property name.
1515   * @param value <code>boolean</code> value of the property.
1516   */
1517  public void setBoolean(String name, boolean value) {
1518    set(name, Boolean.toString(value));
1519  }
1520
1521  /**
1522   * Set the given property, if it is currently unset.
1523   * @param name property name
1524   * @param value new value
1525   */
1526  public void setBooleanIfUnset(String name, boolean value) {
1527    setIfUnset(name, Boolean.toString(value));
1528  }
1529
1530  /**
1531   * Set the value of the <code>name</code> property to the given type. This
1532   * is equivalent to <code>set(&lt;name&gt;, value.toString())</code>.
1533   * @param name property name
1534   * @param value new value
1535   */
1536  public <T extends Enum<T>> void setEnum(String name, T value) {
1537    set(name, value.toString());
1538  }
1539
1540  /**
1541   * Return value matching this enumerated type.
1542   * Note that the returned value is trimmed by this method.
1543   * @param name Property name
1544   * @param defaultValue Value returned if no mapping exists
1545   * @throws IllegalArgumentException If mapping is illegal for the type
1546   * provided
1547   */
1548  public <T extends Enum<T>> T getEnum(String name, T defaultValue) {
1549    final String val = getTrimmed(name);
1550    return null == val
1551      ? defaultValue
1552      : Enum.valueOf(defaultValue.getDeclaringClass(), val);
1553  }
1554
1555  enum ParsedTimeDuration {
1556    NS {
1557      TimeUnit unit() { return TimeUnit.NANOSECONDS; }
1558      String suffix() { return "ns"; }
1559    },
1560    US {
1561      TimeUnit unit() { return TimeUnit.MICROSECONDS; }
1562      String suffix() { return "us"; }
1563    },
1564    MS {
1565      TimeUnit unit() { return TimeUnit.MILLISECONDS; }
1566      String suffix() { return "ms"; }
1567    },
1568    S {
1569      TimeUnit unit() { return TimeUnit.SECONDS; }
1570      String suffix() { return "s"; }
1571    },
1572    M {
1573      TimeUnit unit() { return TimeUnit.MINUTES; }
1574      String suffix() { return "m"; }
1575    },
1576    H {
1577      TimeUnit unit() { return TimeUnit.HOURS; }
1578      String suffix() { return "h"; }
1579    },
1580    D {
1581      TimeUnit unit() { return TimeUnit.DAYS; }
1582      String suffix() { return "d"; }
1583    };
1584    abstract TimeUnit unit();
1585    abstract String suffix();
1586    static ParsedTimeDuration unitFor(String s) {
1587      for (ParsedTimeDuration ptd : values()) {
1588        // iteration order is in decl order, so SECONDS matched last
1589        if (s.endsWith(ptd.suffix())) {
1590          return ptd;
1591        }
1592      }
1593      return null;
1594    }
1595    static ParsedTimeDuration unitFor(TimeUnit unit) {
1596      for (ParsedTimeDuration ptd : values()) {
1597        if (ptd.unit() == unit) {
1598          return ptd;
1599        }
1600      }
1601      return null;
1602    }
1603  }
1604
1605  /**
1606   * Set the value of <code>name</code> to the given time duration. This
1607   * is equivalent to <code>set(&lt;name&gt;, value + &lt;time suffix&gt;)</code>.
1608   * @param name Property name
1609   * @param value Time duration
1610   * @param unit Unit of time
1611   */
1612  public void setTimeDuration(String name, long value, TimeUnit unit) {
1613    set(name, value + ParsedTimeDuration.unitFor(unit).suffix());
1614  }
1615
1616  /**
1617   * Return time duration in the given time unit. Valid units are encoded in
1618   * properties as suffixes: nanoseconds (ns), microseconds (us), milliseconds
1619   * (ms), seconds (s), minutes (m), hours (h), and days (d).
1620   * @param name Property name
1621   * @param defaultValue Value returned if no mapping exists.
1622   * @param unit Unit to convert the stored property, if it exists.
1623   * @throws NumberFormatException If the property stripped of its unit is not
1624   *         a number
1625   */
1626  public long getTimeDuration(String name, long defaultValue, TimeUnit unit) {
1627    String vStr = get(name);
1628    if (null == vStr) {
1629      return defaultValue;
1630    }
1631    vStr = vStr.trim();
1632    ParsedTimeDuration vUnit = ParsedTimeDuration.unitFor(vStr);
1633    if (null == vUnit) {
1634      LOG.warn("No unit for " + name + "(" + vStr + ") assuming " + unit);
1635      vUnit = ParsedTimeDuration.unitFor(unit);
1636    } else {
1637      vStr = vStr.substring(0, vStr.lastIndexOf(vUnit.suffix()));
1638    }
1639    return unit.convert(Long.parseLong(vStr), vUnit.unit());
1640  }
1641
1642  /**
1643   * Get the value of the <code>name</code> property as a <code>Pattern</code>.
1644   * If no such property is specified, or if the specified value is not a valid
1645   * <code>Pattern</code>, then <code>DefaultValue</code> is returned.
1646   * Note that the returned value is NOT trimmed by this method.
1647   *
1648   * @param name property name
1649   * @param defaultValue default value
1650   * @return property value as a compiled Pattern, or defaultValue
1651   */
1652  public Pattern getPattern(String name, Pattern defaultValue) {
1653    String valString = get(name);
1654    if (null == valString || valString.isEmpty()) {
1655      return defaultValue;
1656    }
1657    try {
1658      return Pattern.compile(valString);
1659    } catch (PatternSyntaxException pse) {
1660      LOG.warn("Regular expression '" + valString + "' for property '" +
1661               name + "' not valid. Using default", pse);
1662      return defaultValue;
1663    }
1664  }
1665
1666  /**
1667   * Set the given property to <code>Pattern</code>.
1668   * If the pattern is passed as null, sets the empty pattern which results in
1669   * further calls to getPattern(...) returning the default value.
1670   *
1671   * @param name property name
1672   * @param pattern new value
1673   */
1674  public void setPattern(String name, Pattern pattern) {
1675    assert pattern != null : "Pattern cannot be null";
1676    set(name, pattern.pattern());
1677  }
1678
1679  /**
1680   * Gets information about why a property was set.  Typically this is the 
1681   * path to the resource objects (file, URL, etc.) the property came from, but
1682   * it can also indicate that it was set programmatically, or because of the
1683   * command line.
1684   *
1685   * @param name - The property name to get the source of.
1686   * @return null - If the property or its source wasn't found. Otherwise, 
1687   * returns a list of the sources of the resource.  The older sources are
1688   * the first ones in the list.  So for example if a configuration is set from
1689   * the command line, and then written out to a file that is read back in the
1690   * first entry would indicate that it was set from the command line, while
1691   * the second one would indicate the file that the new configuration was read
1692   * in from.
1693   */
1694  @InterfaceStability.Unstable
1695  public synchronized String[] getPropertySources(String name) {
1696    if (properties == null) {
1697      // If properties is null, it means a resource was newly added
1698      // but the props were cleared so as to load it upon future
1699      // requests. So lets force a load by asking a properties list.
1700      getProps();
1701    }
1702    // Return a null right away if our properties still
1703    // haven't loaded or the resource mapping isn't defined
1704    if (properties == null || updatingResource == null) {
1705      return null;
1706    } else {
1707      String[] source = updatingResource.get(name);
1708      if(source == null) {
1709        return null;
1710      } else {
1711        return Arrays.copyOf(source, source.length);
1712      }
1713    }
1714  }
1715
1716  /**
1717   * A class that represents a set of positive integer ranges. It parses 
1718   * strings of the form: "2-3,5,7-" where ranges are separated by comma and 
1719   * the lower/upper bounds are separated by dash. Either the lower or upper 
1720   * bound may be omitted meaning all values up to or over. So the string 
1721   * above means 2, 3, 5, and 7, 8, 9, ...
1722   */
1723  public static class IntegerRanges implements Iterable<Integer>{
1724    private static class Range {
1725      int start;
1726      int end;
1727    }
1728    
1729    private static class RangeNumberIterator implements Iterator<Integer> {
1730      Iterator<Range> internal;
1731      int at;
1732      int end;
1733
1734      public RangeNumberIterator(List<Range> ranges) {
1735        if (ranges != null) {
1736          internal = ranges.iterator();
1737        }
1738        at = -1;
1739        end = -2;
1740      }
1741      
1742      @Override
1743      public boolean hasNext() {
1744        if (at <= end) {
1745          return true;
1746        } else if (internal != null){
1747          return internal.hasNext();
1748        }
1749        return false;
1750      }
1751
1752      @Override
1753      public Integer next() {
1754        if (at <= end) {
1755          at++;
1756          return at - 1;
1757        } else if (internal != null){
1758          Range found = internal.next();
1759          if (found != null) {
1760            at = found.start;
1761            end = found.end;
1762            at++;
1763            return at - 1;
1764          }
1765        }
1766        return null;
1767      }
1768
1769      @Override
1770      public void remove() {
1771        throw new UnsupportedOperationException();
1772      }
1773    };
1774
1775    List<Range> ranges = new ArrayList<Range>();
1776    
1777    public IntegerRanges() {
1778    }
1779    
1780    public IntegerRanges(String newValue) {
1781      StringTokenizer itr = new StringTokenizer(newValue, ",");
1782      while (itr.hasMoreTokens()) {
1783        String rng = itr.nextToken().trim();
1784        String[] parts = rng.split("-", 3);
1785        if (parts.length < 1 || parts.length > 2) {
1786          throw new IllegalArgumentException("integer range badly formed: " + 
1787                                             rng);
1788        }
1789        Range r = new Range();
1790        r.start = convertToInt(parts[0], 0);
1791        if (parts.length == 2) {
1792          r.end = convertToInt(parts[1], Integer.MAX_VALUE);
1793        } else {
1794          r.end = r.start;
1795        }
1796        if (r.start > r.end) {
1797          throw new IllegalArgumentException("IntegerRange from " + r.start + 
1798                                             " to " + r.end + " is invalid");
1799        }
1800        ranges.add(r);
1801      }
1802    }
1803
1804    /**
1805     * Convert a string to an int treating empty strings as the default value.
1806     * @param value the string value
1807     * @param defaultValue the value for if the string is empty
1808     * @return the desired integer
1809     */
1810    private static int convertToInt(String value, int defaultValue) {
1811      String trim = value.trim();
1812      if (trim.length() == 0) {
1813        return defaultValue;
1814      }
1815      return Integer.parseInt(trim);
1816    }
1817
1818    /**
1819     * Is the given value in the set of ranges
1820     * @param value the value to check
1821     * @return is the value in the ranges?
1822     */
1823    public boolean isIncluded(int value) {
1824      for(Range r: ranges) {
1825        if (r.start <= value && value <= r.end) {
1826          return true;
1827        }
1828      }
1829      return false;
1830    }
1831    
1832    /**
1833     * @return true if there are no values in this range, else false.
1834     */
1835    public boolean isEmpty() {
1836      return ranges == null || ranges.isEmpty();
1837    }
1838    
1839    @Override
1840    public String toString() {
1841      StringBuilder result = new StringBuilder();
1842      boolean first = true;
1843      for(Range r: ranges) {
1844        if (first) {
1845          first = false;
1846        } else {
1847          result.append(',');
1848        }
1849        result.append(r.start);
1850        result.append('-');
1851        result.append(r.end);
1852      }
1853      return result.toString();
1854    }
1855
1856    @Override
1857    public Iterator<Integer> iterator() {
1858      return new RangeNumberIterator(ranges);
1859    }
1860    
1861  }
1862
1863  /**
1864   * Parse the given attribute as a set of integer ranges
1865   * @param name the attribute name
1866   * @param defaultValue the default value if it is not set
1867   * @return a new set of ranges from the configured value
1868   */
1869  public IntegerRanges getRange(String name, String defaultValue) {
1870    return new IntegerRanges(get(name, defaultValue));
1871  }
1872
1873  /** 
1874   * Get the comma delimited values of the <code>name</code> property as 
1875   * a collection of <code>String</code>s.  
1876   * If no such property is specified then empty collection is returned.
1877   * <p>
1878   * This is an optimized version of {@link #getStrings(String)}
1879   * 
1880   * @param name property name.
1881   * @return property value as a collection of <code>String</code>s. 
1882   */
1883  public Collection<String> getStringCollection(String name) {
1884    String valueString = get(name);
1885    return StringUtils.getStringCollection(valueString);
1886  }
1887
1888  /** 
1889   * Get the comma delimited values of the <code>name</code> property as 
1890   * an array of <code>String</code>s.  
1891   * If no such property is specified then <code>null</code> is returned.
1892   * 
1893   * @param name property name.
1894   * @return property value as an array of <code>String</code>s, 
1895   *         or <code>null</code>. 
1896   */
1897  public String[] getStrings(String name) {
1898    String valueString = get(name);
1899    return StringUtils.getStrings(valueString);
1900  }
1901
1902  /** 
1903   * Get the comma delimited values of the <code>name</code> property as 
1904   * an array of <code>String</code>s.  
1905   * If no such property is specified then default value is returned.
1906   * 
1907   * @param name property name.
1908   * @param defaultValue The default value
1909   * @return property value as an array of <code>String</code>s, 
1910   *         or default value. 
1911   */
1912  public String[] getStrings(String name, String... defaultValue) {
1913    String valueString = get(name);
1914    if (valueString == null) {
1915      return defaultValue;
1916    } else {
1917      return StringUtils.getStrings(valueString);
1918    }
1919  }
1920  
1921  /** 
1922   * Get the comma delimited values of the <code>name</code> property as 
1923   * a collection of <code>String</code>s, trimmed of the leading and trailing whitespace.  
1924   * If no such property is specified then empty <code>Collection</code> is returned.
1925   *
1926   * @param name property name.
1927   * @return property value as a collection of <code>String</code>s, or empty <code>Collection</code> 
1928   */
1929  public Collection<String> getTrimmedStringCollection(String name) {
1930    String valueString = get(name);
1931    if (null == valueString) {
1932      Collection<String> empty = new ArrayList<String>();
1933      return empty;
1934    }
1935    return StringUtils.getTrimmedStringCollection(valueString);
1936  }
1937  
1938  /** 
1939   * Get the comma delimited values of the <code>name</code> property as 
1940   * an array of <code>String</code>s, trimmed of the leading and trailing whitespace.
1941   * If no such property is specified then an empty array is returned.
1942   * 
1943   * @param name property name.
1944   * @return property value as an array of trimmed <code>String</code>s, 
1945   *         or empty array. 
1946   */
1947  public String[] getTrimmedStrings(String name) {
1948    String valueString = get(name);
1949    return StringUtils.getTrimmedStrings(valueString);
1950  }
1951
1952  /** 
1953   * Get the comma delimited values of the <code>name</code> property as 
1954   * an array of <code>String</code>s, trimmed of the leading and trailing whitespace.
1955   * If no such property is specified then default value is returned.
1956   * 
1957   * @param name property name.
1958   * @param defaultValue The default value
1959   * @return property value as an array of trimmed <code>String</code>s, 
1960   *         or default value. 
1961   */
1962  public String[] getTrimmedStrings(String name, String... defaultValue) {
1963    String valueString = get(name);
1964    if (null == valueString) {
1965      return defaultValue;
1966    } else {
1967      return StringUtils.getTrimmedStrings(valueString);
1968    }
1969  }
1970
1971  /** 
1972   * Set the array of string values for the <code>name</code> property as 
1973   * as comma delimited values.  
1974   * 
1975   * @param name property name.
1976   * @param values The values
1977   */
1978  public void setStrings(String name, String... values) {
1979    set(name, StringUtils.arrayToString(values));
1980  }
1981
1982  /**
1983   * Get the value for a known password configuration element.
1984   * In order to enable the elimination of clear text passwords in config,
1985   * this method attempts to resolve the property name as an alias through
1986   * the CredentialProvider API and conditionally fallsback to config.
1987   * @param name property name
1988   * @return password
1989   */
1990  public char[] getPassword(String name) throws IOException {
1991    char[] pass = null;
1992
1993    pass = getPasswordFromCredentialProviders(name);
1994
1995    if (pass == null) {
1996      pass = getPasswordFromConfig(name);
1997    }
1998
1999    return pass;
2000  }
2001
2002  /**
2003   * Try and resolve the provided element name as a credential provider
2004   * alias.
2005   * @param name alias of the provisioned credential
2006   * @return password or null if not found
2007   * @throws IOException
2008   */
2009  protected char[] getPasswordFromCredentialProviders(String name)
2010      throws IOException {
2011    char[] pass = null;
2012    try {
2013      List<CredentialProvider> providers =
2014          CredentialProviderFactory.getProviders(this);
2015
2016      if (providers != null) {
2017        for (CredentialProvider provider : providers) {
2018          try {
2019            CredentialEntry entry = provider.getCredentialEntry(name);
2020            if (entry != null) {
2021              pass = entry.getCredential();
2022              break;
2023            }
2024          }
2025          catch (IOException ioe) {
2026            throw new IOException("Can't get key " + name + " from key provider" +
2027                        "of type: " + provider.getClass().getName() + ".", ioe);
2028          }
2029        }
2030      }
2031    }
2032    catch (IOException ioe) {
2033      throw new IOException("Configuration problem with provider path.", ioe);
2034    }
2035
2036    return pass;
2037  }
2038
2039  /**
2040   * Fallback to clear text passwords in configuration.
2041   * @param name
2042   * @return clear text password or null
2043   */
2044  protected char[] getPasswordFromConfig(String name) {
2045    char[] pass = null;
2046    if (getBoolean(CredentialProvider.CLEAR_TEXT_FALLBACK, true)) {
2047      String passStr = get(name);
2048      if (passStr != null) {
2049        pass = passStr.toCharArray();
2050      }
2051    }
2052    return pass;
2053  }
2054
2055  /**
2056   * Get the socket address for <code>hostProperty</code> as a
2057   * <code>InetSocketAddress</code>. If <code>hostProperty</code> is
2058   * <code>null</code>, <code>addressProperty</code> will be used. This
2059   * is useful for cases where we want to differentiate between host
2060   * bind address and address clients should use to establish connection.
2061   *
2062   * @param hostProperty bind host property name.
2063   * @param addressProperty address property name.
2064   * @param defaultAddressValue the default value
2065   * @param defaultPort the default port
2066   * @return InetSocketAddress
2067   */
2068  public InetSocketAddress getSocketAddr(
2069      String hostProperty,
2070      String addressProperty,
2071      String defaultAddressValue,
2072      int defaultPort) {
2073
2074    InetSocketAddress bindAddr = getSocketAddr(
2075      addressProperty, defaultAddressValue, defaultPort);
2076
2077    final String host = get(hostProperty);
2078
2079    if (host == null || host.isEmpty()) {
2080      return bindAddr;
2081    }
2082
2083    return NetUtils.createSocketAddr(
2084        host, bindAddr.getPort(), hostProperty);
2085  }
2086
2087  /**
2088   * Get the socket address for <code>name</code> property as a
2089   * <code>InetSocketAddress</code>.
2090   * @param name property name.
2091   * @param defaultAddress the default value
2092   * @param defaultPort the default port
2093   * @return InetSocketAddress
2094   */
2095  public InetSocketAddress getSocketAddr(
2096      String name, String defaultAddress, int defaultPort) {
2097    final String address = getTrimmed(name, defaultAddress);
2098    return NetUtils.createSocketAddr(address, defaultPort, name);
2099  }
2100
2101  /**
2102   * Set the socket address for the <code>name</code> property as
2103   * a <code>host:port</code>.
2104   */
2105  public void setSocketAddr(String name, InetSocketAddress addr) {
2106    set(name, NetUtils.getHostPortString(addr));
2107  }
2108
2109  /**
2110   * Set the socket address a client can use to connect for the
2111   * <code>name</code> property as a <code>host:port</code>.  The wildcard
2112   * address is replaced with the local host's address. If the host and address
2113   * properties are configured the host component of the address will be combined
2114   * with the port component of the addr to generate the address.  This is to allow
2115   * optional control over which host name is used in multi-home bind-host
2116   * cases where a host can have multiple names
2117   * @param hostProperty the bind-host configuration name
2118   * @param addressProperty the service address configuration name
2119   * @param defaultAddressValue the service default address configuration value
2120   * @param addr InetSocketAddress of the service listener
2121   * @return InetSocketAddress for clients to connect
2122   */
2123  public InetSocketAddress updateConnectAddr(
2124      String hostProperty,
2125      String addressProperty,
2126      String defaultAddressValue,
2127      InetSocketAddress addr) {
2128
2129    final String host = get(hostProperty);
2130    final String connectHostPort = getTrimmed(addressProperty, defaultAddressValue);
2131
2132    if (host == null || host.isEmpty() || connectHostPort == null || connectHostPort.isEmpty()) {
2133      //not our case, fall back to original logic
2134      return updateConnectAddr(addressProperty, addr);
2135    }
2136
2137    final String connectHost = connectHostPort.split(":")[0];
2138    // Create connect address using client address hostname and server port.
2139    return updateConnectAddr(addressProperty, NetUtils.createSocketAddrForHost(
2140        connectHost, addr.getPort()));
2141  }
2142  
2143  /**
2144   * Set the socket address a client can use to connect for the
2145   * <code>name</code> property as a <code>host:port</code>.  The wildcard
2146   * address is replaced with the local host's address.
2147   * @param name property name.
2148   * @param addr InetSocketAddress of a listener to store in the given property
2149   * @return InetSocketAddress for clients to connect
2150   */
2151  public InetSocketAddress updateConnectAddr(String name,
2152                                             InetSocketAddress addr) {
2153    final InetSocketAddress connectAddr = NetUtils.getConnectAddress(addr);
2154    setSocketAddr(name, connectAddr);
2155    return connectAddr;
2156  }
2157  
2158  /**
2159   * Load a class by name.
2160   * 
2161   * @param name the class name.
2162   * @return the class object.
2163   * @throws ClassNotFoundException if the class is not found.
2164   */
2165  public Class<?> getClassByName(String name) throws ClassNotFoundException {
2166    Class<?> ret = getClassByNameOrNull(name);
2167    if (ret == null) {
2168      throw new ClassNotFoundException("Class " + name + " not found");
2169    }
2170    return ret;
2171  }
2172  
2173  /**
2174   * Load a class by name, returning null rather than throwing an exception
2175   * if it couldn't be loaded. This is to avoid the overhead of creating
2176   * an exception.
2177   * 
2178   * @param name the class name
2179   * @return the class object, or null if it could not be found.
2180   */
2181  public Class<?> getClassByNameOrNull(String name) {
2182    Map<String, WeakReference<Class<?>>> map;
2183    
2184    synchronized (CACHE_CLASSES) {
2185      map = CACHE_CLASSES.get(classLoader);
2186      if (map == null) {
2187        map = Collections.synchronizedMap(
2188          new WeakHashMap<String, WeakReference<Class<?>>>());
2189        CACHE_CLASSES.put(classLoader, map);
2190      }
2191    }
2192
2193    Class<?> clazz = null;
2194    WeakReference<Class<?>> ref = map.get(name); 
2195    if (ref != null) {
2196       clazz = ref.get();
2197    }
2198     
2199    if (clazz == null) {
2200      try {
2201        clazz = Class.forName(name, true, classLoader);
2202      } catch (ClassNotFoundException e) {
2203        // Leave a marker that the class isn't found
2204        map.put(name, new WeakReference<Class<?>>(NEGATIVE_CACHE_SENTINEL));
2205        return null;
2206      }
2207      // two putters can race here, but they'll put the same class
2208      map.put(name, new WeakReference<Class<?>>(clazz));
2209      return clazz;
2210    } else if (clazz == NEGATIVE_CACHE_SENTINEL) {
2211      return null; // not found
2212    } else {
2213      // cache hit
2214      return clazz;
2215    }
2216  }
2217
2218  /** 
2219   * Get the value of the <code>name</code> property
2220   * as an array of <code>Class</code>.
2221   * The value of the property specifies a list of comma separated class names.  
2222   * If no such property is specified, then <code>defaultValue</code> is 
2223   * returned.
2224   * 
2225   * @param name the property name.
2226   * @param defaultValue default value.
2227   * @return property value as a <code>Class[]</code>, 
2228   *         or <code>defaultValue</code>. 
2229   */
2230  public Class<?>[] getClasses(String name, Class<?> ... defaultValue) {
2231    String valueString = getRaw(name);
2232    if (null == valueString) {
2233      return defaultValue;
2234    }
2235    String[] classnames = getTrimmedStrings(name);
2236    try {
2237      Class<?>[] classes = new Class<?>[classnames.length];
2238      for(int i = 0; i < classnames.length; i++) {
2239        classes[i] = getClassByName(classnames[i]);
2240      }
2241      return classes;
2242    } catch (ClassNotFoundException e) {
2243      throw new RuntimeException(e);
2244    }
2245  }
2246
2247  /** 
2248   * Get the value of the <code>name</code> property as a <code>Class</code>.  
2249   * If no such property is specified, then <code>defaultValue</code> is 
2250   * returned.
2251   * 
2252   * @param name the class name.
2253   * @param defaultValue default value.
2254   * @return property value as a <code>Class</code>, 
2255   *         or <code>defaultValue</code>. 
2256   */
2257  public Class<?> getClass(String name, Class<?> defaultValue) {
2258    String valueString = getTrimmed(name);
2259    if (valueString == null)
2260      return defaultValue;
2261    try {
2262      return getClassByName(valueString);
2263    } catch (ClassNotFoundException e) {
2264      throw new RuntimeException(e);
2265    }
2266  }
2267
2268  /** 
2269   * Get the value of the <code>name</code> property as a <code>Class</code>
2270   * implementing the interface specified by <code>xface</code>.
2271   *   
2272   * If no such property is specified, then <code>defaultValue</code> is 
2273   * returned.
2274   * 
2275   * An exception is thrown if the returned class does not implement the named
2276   * interface. 
2277   * 
2278   * @param name the class name.
2279   * @param defaultValue default value.
2280   * @param xface the interface implemented by the named class.
2281   * @return property value as a <code>Class</code>, 
2282   *         or <code>defaultValue</code>.
2283   */
2284  public <U> Class<? extends U> getClass(String name, 
2285                                         Class<? extends U> defaultValue, 
2286                                         Class<U> xface) {
2287    try {
2288      Class<?> theClass = getClass(name, defaultValue);
2289      if (theClass != null && !xface.isAssignableFrom(theClass))
2290        throw new RuntimeException(theClass+" not "+xface.getName());
2291      else if (theClass != null)
2292        return theClass.asSubclass(xface);
2293      else
2294        return null;
2295    } catch (Exception e) {
2296      throw new RuntimeException(e);
2297    }
2298  }
2299
2300  /**
2301   * Get the value of the <code>name</code> property as a <code>List</code>
2302   * of objects implementing the interface specified by <code>xface</code>.
2303   * 
2304   * An exception is thrown if any of the classes does not exist, or if it does
2305   * not implement the named interface.
2306   * 
2307   * @param name the property name.
2308   * @param xface the interface implemented by the classes named by
2309   *        <code>name</code>.
2310   * @return a <code>List</code> of objects implementing <code>xface</code>.
2311   */
2312  @SuppressWarnings("unchecked")
2313  public <U> List<U> getInstances(String name, Class<U> xface) {
2314    List<U> ret = new ArrayList<U>();
2315    Class<?>[] classes = getClasses(name);
2316    for (Class<?> cl: classes) {
2317      if (!xface.isAssignableFrom(cl)) {
2318        throw new RuntimeException(cl + " does not implement " + xface);
2319      }
2320      ret.add((U)ReflectionUtils.newInstance(cl, this));
2321    }
2322    return ret;
2323  }
2324
2325  /** 
2326   * Set the value of the <code>name</code> property to the name of a 
2327   * <code>theClass</code> implementing the given interface <code>xface</code>.
2328   * 
2329   * An exception is thrown if <code>theClass</code> does not implement the 
2330   * interface <code>xface</code>. 
2331   * 
2332   * @param name property name.
2333   * @param theClass property value.
2334   * @param xface the interface implemented by the named class.
2335   */
2336  public void setClass(String name, Class<?> theClass, Class<?> xface) {
2337    if (!xface.isAssignableFrom(theClass))
2338      throw new RuntimeException(theClass+" not "+xface.getName());
2339    set(name, theClass.getName());
2340  }
2341
2342  /** 
2343   * Get a local file under a directory named by <i>dirsProp</i> with
2344   * the given <i>path</i>.  If <i>dirsProp</i> contains multiple directories,
2345   * then one is chosen based on <i>path</i>'s hash code.  If the selected
2346   * directory does not exist, an attempt is made to create it.
2347   * 
2348   * @param dirsProp directory in which to locate the file.
2349   * @param path file-path.
2350   * @return local file under the directory with the given path.
2351   */
2352  public Path getLocalPath(String dirsProp, String path)
2353    throws IOException {
2354    String[] dirs = getTrimmedStrings(dirsProp);
2355    int hashCode = path.hashCode();
2356    FileSystem fs = FileSystem.getLocal(this);
2357    for (int i = 0; i < dirs.length; i++) {  // try each local dir
2358      int index = (hashCode+i & Integer.MAX_VALUE) % dirs.length;
2359      Path file = new Path(dirs[index], path);
2360      Path dir = file.getParent();
2361      if (fs.mkdirs(dir) || fs.exists(dir)) {
2362        return file;
2363      }
2364    }
2365    LOG.warn("Could not make " + path + 
2366             " in local directories from " + dirsProp);
2367    for(int i=0; i < dirs.length; i++) {
2368      int index = (hashCode+i & Integer.MAX_VALUE) % dirs.length;
2369      LOG.warn(dirsProp + "[" + index + "]=" + dirs[index]);
2370    }
2371    throw new IOException("No valid local directories in property: "+dirsProp);
2372  }
2373
2374  /** 
2375   * Get a local file name under a directory named in <i>dirsProp</i> with
2376   * the given <i>path</i>.  If <i>dirsProp</i> contains multiple directories,
2377   * then one is chosen based on <i>path</i>'s hash code.  If the selected
2378   * directory does not exist, an attempt is made to create it.
2379   * 
2380   * @param dirsProp directory in which to locate the file.
2381   * @param path file-path.
2382   * @return local file under the directory with the given path.
2383   */
2384  public File getFile(String dirsProp, String path)
2385    throws IOException {
2386    String[] dirs = getTrimmedStrings(dirsProp);
2387    int hashCode = path.hashCode();
2388    for (int i = 0; i < dirs.length; i++) {  // try each local dir
2389      int index = (hashCode+i & Integer.MAX_VALUE) % dirs.length;
2390      File file = new File(dirs[index], path);
2391      File dir = file.getParentFile();
2392      if (dir.exists() || dir.mkdirs()) {
2393        return file;
2394      }
2395    }
2396    throw new IOException("No valid local directories in property: "+dirsProp);
2397  }
2398
2399  /** 
2400   * Get the {@link URL} for the named resource.
2401   * 
2402   * @param name resource name.
2403   * @return the url for the named resource.
2404   */
2405  public URL getResource(String name) {
2406    return classLoader.getResource(name);
2407  }
2408  
2409  /** 
2410   * Get an input stream attached to the configuration resource with the
2411   * given <code>name</code>.
2412   * 
2413   * @param name configuration resource name.
2414   * @return an input stream attached to the resource.
2415   */
2416  public InputStream getConfResourceAsInputStream(String name) {
2417    try {
2418      URL url= getResource(name);
2419
2420      if (url == null) {
2421        LOG.info(name + " not found");
2422        return null;
2423      } else {
2424        LOG.info("found resource " + name + " at " + url);
2425      }
2426
2427      return url.openStream();
2428    } catch (Exception e) {
2429      return null;
2430    }
2431  }
2432
2433  /** 
2434   * Get a {@link Reader} attached to the configuration resource with the
2435   * given <code>name</code>.
2436   * 
2437   * @param name configuration resource name.
2438   * @return a reader attached to the resource.
2439   */
2440  public Reader getConfResourceAsReader(String name) {
2441    try {
2442      URL url= getResource(name);
2443
2444      if (url == null) {
2445        LOG.info(name + " not found");
2446        return null;
2447      } else {
2448        LOG.info("found resource " + name + " at " + url);
2449      }
2450
2451      return new InputStreamReader(url.openStream(), Charsets.UTF_8);
2452    } catch (Exception e) {
2453      return null;
2454    }
2455  }
2456
2457  /**
2458   * Get the set of parameters marked final.
2459   *
2460   * @return final parameter set.
2461   */
2462  public Set<String> getFinalParameters() {
2463    Set<String> setFinalParams = Collections.newSetFromMap(
2464        new ConcurrentHashMap<String, Boolean>());
2465    setFinalParams.addAll(finalParameters);
2466    return setFinalParams;
2467  }
2468
2469  protected synchronized Properties getProps() {
2470    if (properties == null) {
2471      properties = new Properties();
2472      Map<String, String[]> backup =
2473          new ConcurrentHashMap<String, String[]>(updatingResource);
2474      loadResources(properties, resources, quietmode);
2475
2476      if (overlay != null) {
2477        properties.putAll(overlay);
2478        for (Map.Entry<Object,Object> item: overlay.entrySet()) {
2479          String key = (String)item.getKey();
2480          String[] source = backup.get(key);
2481          if(source != null) {
2482            updatingResource.put(key, source);
2483          }
2484        }
2485      }
2486    }
2487    return properties;
2488  }
2489
2490  /**
2491   * Return the number of keys in the configuration.
2492   *
2493   * @return number of keys in the configuration.
2494   */
2495  public int size() {
2496    return getProps().size();
2497  }
2498
2499  /**
2500   * Clears all keys from the configuration.
2501   */
2502  public void clear() {
2503    getProps().clear();
2504    getOverlay().clear();
2505  }
2506
2507  /**
2508   * Get an {@link Iterator} to go through the list of <code>String</code> 
2509   * key-value pairs in the configuration.
2510   * 
2511   * @return an iterator over the entries.
2512   */
2513  @Override
2514  public Iterator<Map.Entry<String, String>> iterator() {
2515    // Get a copy of just the string to string pairs. After the old object
2516    // methods that allow non-strings to be put into configurations are removed,
2517    // we could replace properties with a Map<String,String> and get rid of this
2518    // code.
2519    Map<String,String> result = new HashMap<String,String>();
2520    for(Map.Entry<Object,Object> item: getProps().entrySet()) {
2521      if (item.getKey() instanceof String &&
2522          item.getValue() instanceof String) {
2523          result.put((String) item.getKey(), (String) item.getValue());
2524      }
2525    }
2526    return result.entrySet().iterator();
2527  }
2528
2529  private Document parse(DocumentBuilder builder, URL url)
2530      throws IOException, SAXException {
2531    if (!quietmode) {
2532      if (LOG.isDebugEnabled()) {
2533        LOG.debug("parsing URL " + url);
2534      }
2535    }
2536    if (url == null) {
2537      return null;
2538    }
2539    return parse(builder, url.openStream(), url.toString());
2540  }
2541
2542  private Document parse(DocumentBuilder builder, InputStream is,
2543      String systemId) throws IOException, SAXException {
2544    if (!quietmode) {
2545      LOG.debug("parsing input stream " + is);
2546    }
2547    if (is == null) {
2548      return null;
2549    }
2550    try {
2551      return (systemId == null) ? builder.parse(is) : builder.parse(is,
2552          systemId);
2553    } finally {
2554      is.close();
2555    }
2556  }
2557
2558  private void loadResources(Properties properties,
2559                             ArrayList<Resource> resources,
2560                             boolean quiet) {
2561    if(loadDefaults) {
2562      for (String resource : defaultResources) {
2563        loadResource(properties, new Resource(resource), quiet);
2564      }
2565    
2566      //support the hadoop-site.xml as a deprecated case
2567      if(getResource("hadoop-site.xml")!=null) {
2568        loadResource(properties, new Resource("hadoop-site.xml"), quiet);
2569      }
2570    }
2571    
2572    for (int i = 0; i < resources.size(); i++) {
2573      Resource ret = loadResource(properties, resources.get(i), quiet);
2574      if (ret != null) {
2575        resources.set(i, ret);
2576      }
2577    }
2578  }
2579  
2580  private Resource loadResource(Properties properties, Resource wrapper, boolean quiet) {
2581    String name = UNKNOWN_RESOURCE;
2582    try {
2583      Object resource = wrapper.getResource();
2584      name = wrapper.getName();
2585      
2586      DocumentBuilderFactory docBuilderFactory 
2587        = DocumentBuilderFactory.newInstance();
2588      //ignore all comments inside the xml file
2589      docBuilderFactory.setIgnoringComments(true);
2590
2591      //allow includes in the xml file
2592      docBuilderFactory.setNamespaceAware(true);
2593      try {
2594          docBuilderFactory.setXIncludeAware(true);
2595      } catch (UnsupportedOperationException e) {
2596        LOG.error("Failed to set setXIncludeAware(true) for parser "
2597                + docBuilderFactory
2598                + ":" + e,
2599                e);
2600      }
2601      DocumentBuilder builder = docBuilderFactory.newDocumentBuilder();
2602      Document doc = null;
2603      Element root = null;
2604      boolean returnCachedProperties = false;
2605      
2606      if (resource instanceof URL) {                  // an URL resource
2607        doc = parse(builder, (URL)resource);
2608      } else if (resource instanceof String) {        // a CLASSPATH resource
2609        URL url = getResource((String)resource);
2610        doc = parse(builder, url);
2611      } else if (resource instanceof Path) {          // a file resource
2612        // Can't use FileSystem API or we get an infinite loop
2613        // since FileSystem uses Configuration API.  Use java.io.File instead.
2614        File file = new File(((Path)resource).toUri().getPath())
2615          .getAbsoluteFile();
2616        if (file.exists()) {
2617          if (!quiet) {
2618            LOG.debug("parsing File " + file);
2619          }
2620          doc = parse(builder, new BufferedInputStream(
2621              new FileInputStream(file)), ((Path)resource).toString());
2622        }
2623      } else if (resource instanceof InputStream) {
2624        doc = parse(builder, (InputStream) resource, null);
2625        returnCachedProperties = true;
2626      } else if (resource instanceof Properties) {
2627        overlay(properties, (Properties)resource);
2628      } else if (resource instanceof Element) {
2629        root = (Element)resource;
2630      }
2631
2632      if (root == null) {
2633        if (doc == null) {
2634          if (quiet) {
2635            return null;
2636          }
2637          throw new RuntimeException(resource + " not found");
2638        }
2639        root = doc.getDocumentElement();
2640      }
2641      Properties toAddTo = properties;
2642      if(returnCachedProperties) {
2643        toAddTo = new Properties();
2644      }
2645      if (!"configuration".equals(root.getTagName()))
2646        LOG.fatal("bad conf file: top-level element not <configuration>");
2647      NodeList props = root.getChildNodes();
2648      DeprecationContext deprecations = deprecationContext.get();
2649      for (int i = 0; i < props.getLength(); i++) {
2650        Node propNode = props.item(i);
2651        if (!(propNode instanceof Element))
2652          continue;
2653        Element prop = (Element)propNode;
2654        if ("configuration".equals(prop.getTagName())) {
2655          loadResource(toAddTo, new Resource(prop, name), quiet);
2656          continue;
2657        }
2658        if (!"property".equals(prop.getTagName()))
2659          LOG.warn("bad conf file: element not <property>");
2660
2661        String attr = null;
2662        String value = null;
2663        boolean finalParameter = false;
2664        LinkedList<String> source = new LinkedList<String>();
2665
2666        Attr propAttr = prop.getAttributeNode("name");
2667        if (propAttr != null)
2668          attr = StringInterner.weakIntern(propAttr.getValue());
2669        propAttr = prop.getAttributeNode("value");
2670        if (propAttr != null)
2671          value = StringInterner.weakIntern(propAttr.getValue());
2672        propAttr = prop.getAttributeNode("final");
2673        if (propAttr != null)
2674          finalParameter = "true".equals(propAttr.getValue());
2675        propAttr = prop.getAttributeNode("source");
2676        if (propAttr != null)
2677          source.add(StringInterner.weakIntern(propAttr.getValue()));
2678
2679        NodeList fields = prop.getChildNodes();
2680        for (int j = 0; j < fields.getLength(); j++) {
2681          Node fieldNode = fields.item(j);
2682          if (!(fieldNode instanceof Element))
2683            continue;
2684          Element field = (Element)fieldNode;
2685          if ("name".equals(field.getTagName()) && field.hasChildNodes())
2686            attr = StringInterner.weakIntern(
2687                ((Text)field.getFirstChild()).getData().trim());
2688          if ("value".equals(field.getTagName()) && field.hasChildNodes())
2689            value = StringInterner.weakIntern(
2690                ((Text)field.getFirstChild()).getData());
2691          if ("final".equals(field.getTagName()) && field.hasChildNodes())
2692            finalParameter = "true".equals(((Text)field.getFirstChild()).getData());
2693          if ("source".equals(field.getTagName()) && field.hasChildNodes())
2694            source.add(StringInterner.weakIntern(
2695                ((Text)field.getFirstChild()).getData()));
2696        }
2697        source.add(name);
2698        
2699        // Ignore this parameter if it has already been marked as 'final'
2700        if (attr != null) {
2701          if (deprecations.getDeprecatedKeyMap().containsKey(attr)) {
2702            DeprecatedKeyInfo keyInfo =
2703                deprecations.getDeprecatedKeyMap().get(attr);
2704            keyInfo.clearAccessed();
2705            for (String key:keyInfo.newKeys) {
2706              // update new keys with deprecated key's value 
2707              loadProperty(toAddTo, name, key, value, finalParameter, 
2708                  source.toArray(new String[source.size()]));
2709            }
2710          }
2711          else {
2712            loadProperty(toAddTo, name, attr, value, finalParameter, 
2713                source.toArray(new String[source.size()]));
2714          }
2715        }
2716      }
2717      
2718      if (returnCachedProperties) {
2719        overlay(properties, toAddTo);
2720        return new Resource(toAddTo, name);
2721      }
2722      return null;
2723    } catch (IOException e) {
2724      LOG.fatal("error parsing conf " + name, e);
2725      throw new RuntimeException(e);
2726    } catch (DOMException e) {
2727      LOG.fatal("error parsing conf " + name, e);
2728      throw new RuntimeException(e);
2729    } catch (SAXException e) {
2730      LOG.fatal("error parsing conf " + name, e);
2731      throw new RuntimeException(e);
2732    } catch (ParserConfigurationException e) {
2733      LOG.fatal("error parsing conf " + name , e);
2734      throw new RuntimeException(e);
2735    }
2736  }
2737
2738  private void overlay(Properties to, Properties from) {
2739    for (Entry<Object, Object> entry: from.entrySet()) {
2740      to.put(entry.getKey(), entry.getValue());
2741    }
2742  }
2743
2744  private void loadProperty(Properties properties, String name, String attr,
2745      String value, boolean finalParameter, String[] source) {
2746    if (value != null || allowNullValueProperties) {
2747      if (value == null) {
2748        value = DEFAULT_STRING_CHECK;
2749      }
2750      if (!finalParameters.contains(attr)) {
2751        properties.setProperty(attr, value);
2752        if(source != null) {
2753          updatingResource.put(attr, source);
2754        }
2755      } else if (!value.equals(properties.getProperty(attr))) {
2756        LOG.warn(name+":an attempt to override final parameter: "+attr
2757            +";  Ignoring.");
2758      }
2759    }
2760    if (finalParameter && attr != null) {
2761      finalParameters.add(attr);
2762    }
2763  }
2764
2765  /** 
2766   * Write out the non-default properties in this configuration to the given
2767   * {@link OutputStream} using UTF-8 encoding.
2768   * 
2769   * @param out the output stream to write to.
2770   */
2771  public void writeXml(OutputStream out) throws IOException {
2772    writeXml(new OutputStreamWriter(out, "UTF-8"));
2773  }
2774
2775  /** 
2776   * Write out the non-default properties in this configuration to the given
2777   * {@link Writer}.
2778   * 
2779   * @param out the writer to write to.
2780   */
2781  public void writeXml(Writer out) throws IOException {
2782    Document doc = asXmlDocument();
2783
2784    try {
2785      DOMSource source = new DOMSource(doc);
2786      StreamResult result = new StreamResult(out);
2787      TransformerFactory transFactory = TransformerFactory.newInstance();
2788      Transformer transformer = transFactory.newTransformer();
2789
2790      // Important to not hold Configuration log while writing result, since
2791      // 'out' may be an HDFS stream which needs to lock this configuration
2792      // from another thread.
2793      transformer.transform(source, result);
2794    } catch (TransformerException te) {
2795      throw new IOException(te);
2796    }
2797  }
2798
2799  /**
2800   * Return the XML DOM corresponding to this Configuration.
2801   */
2802  private synchronized Document asXmlDocument() throws IOException {
2803    Document doc;
2804    try {
2805      doc =
2806        DocumentBuilderFactory.newInstance().newDocumentBuilder().newDocument();
2807    } catch (ParserConfigurationException pe) {
2808      throw new IOException(pe);
2809    }
2810    Element conf = doc.createElement("configuration");
2811    doc.appendChild(conf);
2812    conf.appendChild(doc.createTextNode("\n"));
2813    handleDeprecation(); //ensure properties is set and deprecation is handled
2814    for (Enumeration<Object> e = properties.keys(); e.hasMoreElements();) {
2815      String name = (String)e.nextElement();
2816      Object object = properties.get(name);
2817      String value = null;
2818      if (object instanceof String) {
2819        value = (String) object;
2820      }else {
2821        continue;
2822      }
2823      Element propNode = doc.createElement("property");
2824      conf.appendChild(propNode);
2825
2826      Element nameNode = doc.createElement("name");
2827      nameNode.appendChild(doc.createTextNode(name));
2828      propNode.appendChild(nameNode);
2829
2830      Element valueNode = doc.createElement("value");
2831      valueNode.appendChild(doc.createTextNode(value));
2832      propNode.appendChild(valueNode);
2833
2834      if (updatingResource != null) {
2835        String[] sources = updatingResource.get(name);
2836        if(sources != null) {
2837          for(String s : sources) {
2838            Element sourceNode = doc.createElement("source");
2839            sourceNode.appendChild(doc.createTextNode(s));
2840            propNode.appendChild(sourceNode);
2841          }
2842        }
2843      }
2844      
2845      conf.appendChild(doc.createTextNode("\n"));
2846    }
2847    return doc;
2848  }
2849
2850  /**
2851   *  Writes out all the parameters and their properties (final and resource) to
2852   *  the given {@link Writer}
2853   *  The format of the output would be 
2854   *  { "properties" : [ {key1,value1,key1.isFinal,key1.resource}, {key2,value2,
2855   *  key2.isFinal,key2.resource}... ] } 
2856   *  It does not output the parameters of the configuration object which is 
2857   *  loaded from an input stream.
2858   * @param out the Writer to write to
2859   * @throws IOException
2860   */
2861  public static void dumpConfiguration(Configuration config,
2862      Writer out) throws IOException {
2863    JsonFactory dumpFactory = new JsonFactory();
2864    JsonGenerator dumpGenerator = dumpFactory.createJsonGenerator(out);
2865    dumpGenerator.writeStartObject();
2866    dumpGenerator.writeFieldName("properties");
2867    dumpGenerator.writeStartArray();
2868    dumpGenerator.flush();
2869    synchronized (config) {
2870      for (Map.Entry<Object,Object> item: config.getProps().entrySet()) {
2871        dumpGenerator.writeStartObject();
2872        dumpGenerator.writeStringField("key", (String) item.getKey());
2873        dumpGenerator.writeStringField("value", 
2874                                       config.get((String) item.getKey()));
2875        dumpGenerator.writeBooleanField("isFinal",
2876                                        config.finalParameters.contains(item.getKey()));
2877        String[] resources = config.updatingResource.get(item.getKey());
2878        String resource = UNKNOWN_RESOURCE;
2879        if(resources != null && resources.length > 0) {
2880          resource = resources[0];
2881        }
2882        dumpGenerator.writeStringField("resource", resource);
2883        dumpGenerator.writeEndObject();
2884      }
2885    }
2886    dumpGenerator.writeEndArray();
2887    dumpGenerator.writeEndObject();
2888    dumpGenerator.flush();
2889  }
2890  
2891  /**
2892   * Get the {@link ClassLoader} for this job.
2893   * 
2894   * @return the correct class loader.
2895   */
2896  public ClassLoader getClassLoader() {
2897    return classLoader;
2898  }
2899  
2900  /**
2901   * Set the class loader that will be used to load the various objects.
2902   * 
2903   * @param classLoader the new class loader.
2904   */
2905  public void setClassLoader(ClassLoader classLoader) {
2906    this.classLoader = classLoader;
2907  }
2908  
2909  @Override
2910  public String toString() {
2911    StringBuilder sb = new StringBuilder();
2912    sb.append("Configuration: ");
2913    if(loadDefaults) {
2914      toString(defaultResources, sb);
2915      if(resources.size()>0) {
2916        sb.append(", ");
2917      }
2918    }
2919    toString(resources, sb);
2920    return sb.toString();
2921  }
2922  
2923  private <T> void toString(List<T> resources, StringBuilder sb) {
2924    ListIterator<T> i = resources.listIterator();
2925    while (i.hasNext()) {
2926      if (i.nextIndex() != 0) {
2927        sb.append(", ");
2928      }
2929      sb.append(i.next());
2930    }
2931  }
2932
2933  /** 
2934   * Set the quietness-mode. 
2935   * 
2936   * In the quiet-mode, error and informational messages might not be logged.
2937   * 
2938   * @param quietmode <code>true</code> to set quiet-mode on, <code>false</code>
2939   *              to turn it off.
2940   */
2941  public synchronized void setQuietMode(boolean quietmode) {
2942    this.quietmode = quietmode;
2943  }
2944
2945  synchronized boolean getQuietMode() {
2946    return this.quietmode;
2947  }
2948  
2949  /** For debugging.  List non-default properties to the terminal and exit. */
2950  public static void main(String[] args) throws Exception {
2951    new Configuration().writeXml(System.out);
2952  }
2953
2954  @Override
2955  public void readFields(DataInput in) throws IOException {
2956    clear();
2957    int size = WritableUtils.readVInt(in);
2958    for(int i=0; i < size; ++i) {
2959      String key = org.apache.hadoop.io.Text.readString(in);
2960      String value = org.apache.hadoop.io.Text.readString(in);
2961      set(key, value); 
2962      String sources[] = WritableUtils.readCompressedStringArray(in);
2963      if(sources != null) {
2964        updatingResource.put(key, sources);
2965      }
2966    }
2967  }
2968
2969  //@Override
2970  @Override
2971  public void write(DataOutput out) throws IOException {
2972    Properties props = getProps();
2973    WritableUtils.writeVInt(out, props.size());
2974    for(Map.Entry<Object, Object> item: props.entrySet()) {
2975      org.apache.hadoop.io.Text.writeString(out, (String) item.getKey());
2976      org.apache.hadoop.io.Text.writeString(out, (String) item.getValue());
2977      WritableUtils.writeCompressedStringArray(out, 
2978          updatingResource.get(item.getKey()));
2979    }
2980  }
2981  
2982  /**
2983   * get keys matching the the regex 
2984   * @param regex
2985   * @return Map<String,String> with matching keys
2986   */
2987  public Map<String,String> getValByRegex(String regex) {
2988    Pattern p = Pattern.compile(regex);
2989
2990    Map<String,String> result = new HashMap<String,String>();
2991    Matcher m;
2992
2993    for(Map.Entry<Object,Object> item: getProps().entrySet()) {
2994      if (item.getKey() instanceof String && 
2995          item.getValue() instanceof String) {
2996        m = p.matcher((String)item.getKey());
2997        if(m.find()) { // match
2998          result.put((String) item.getKey(),
2999              substituteVars(getProps().getProperty((String) item.getKey())));
3000        }
3001      }
3002    }
3003    return result;
3004  }
3005
3006  /**
3007   * A unique class which is used as a sentinel value in the caching
3008   * for getClassByName. {@link Configuration#getClassByNameOrNull(String)}
3009   */
3010  private static abstract class NegativeCacheSentinel {}
3011
3012  public static void dumpDeprecatedKeys() {
3013    DeprecationContext deprecations = deprecationContext.get();
3014    for (Map.Entry<String, DeprecatedKeyInfo> entry :
3015        deprecations.getDeprecatedKeyMap().entrySet()) {
3016      StringBuilder newKeys = new StringBuilder();
3017      for (String newKey : entry.getValue().newKeys) {
3018        newKeys.append(newKey).append("\t");
3019      }
3020      System.out.println(entry.getKey() + "\t" + newKeys.toString());
3021    }
3022  }
3023
3024  /**
3025   * Returns whether or not a deprecated name has been warned. If the name is not
3026   * deprecated then always return false
3027   */
3028  public static boolean hasWarnedDeprecation(String name) {
3029    DeprecationContext deprecations = deprecationContext.get();
3030    if(deprecations.getDeprecatedKeyMap().containsKey(name)) {
3031      if(deprecations.getDeprecatedKeyMap().get(name).accessed.get()) {
3032        return true;
3033      }
3034    }
3035    return false;
3036  }
3037}