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