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