View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  package org.apache.hadoop.hbase.conf;
19  
20  import org.apache.commons.logging.Log;
21  import org.apache.commons.logging.LogFactory;
22  import java.util.Collections;
23  import java.util.Set;
24  import java.util.WeakHashMap;
25  import org.apache.hadoop.conf.Configuration;
26  import org.apache.hadoop.hbase.classification.InterfaceAudience;
27  import org.apache.hadoop.hbase.classification.InterfaceStability;
28  
29  /**
30   * Maintains the set of all the classes which would like to get notified
31   * when the Configuration is reloaded from the disk in the Online Configuration
32   * Change mechanism, which lets you update certain configuration properties
33   * on-the-fly, without having to restart the cluster.
34   *
35   * If a class has configuration properties which you would like to be able to
36   * change on-the-fly, do the following:
37   * 1. Implement the {@link ConfigurationObserver} interface. This would require
38   *    you to implement the
39   *    {@link ConfigurationObserver#onConfigurationChange(Configuration)}
40   *    method.  This is a callback that is used to notify your class' instance
41   *    that the configuration has changed. In this method, you need to check
42   *    if the new values for the properties that are of interest to your class
43   *    are different from the cached values. If yes, update them.
44   *
45   *    However, be careful with this. Certain properties might be trivially
46   *    mutable online, but others might not. Two properties might be trivially
47   *    mutable by themselves, but not when changed together. For example, if a
48   *    method uses properties "a" and "b" to make some decision, and is running
49   *    in parallel when the notifyOnChange() method updates "a", but hasn't
50   *    yet updated "b", it might make a decision on the basis of a new value of
51   *    "a", and an old value of "b". This might introduce subtle bugs. This
52   *    needs to be dealt on a case-by-case basis, and this class does not provide
53   *    any protection from such cases.
54   *
55   * 2. Register the appropriate instance of the class with the
56   *    {@link ConfigurationManager} instance, using the
57   *    {@link ConfigurationManager#registerObserver(ConfigurationObserver)}
58   *    method. Be careful not to do this in the constructor, as you might cause
59   *    the 'this' reference to escape. Use a factory method, or an initialize()
60   *    method which is called after the construction of the object.
61   *
62   * 3. Deregister the instance using the
63   *    {@link ConfigurationManager#deregisterObserver(ConfigurationObserver)}
64   *    method when it is going out of scope. In case you are not able to do that
65   *    for any reason, it is still okay, since entries for dead observers are
66   *    automatically collected during GC. But nonetheless, it is still a good
67   *    practice to deregister your observer, whenever possible.
68   */
69  @InterfaceAudience.Private
70  @InterfaceStability.Evolving
71  public class ConfigurationManager {
72    private static final Log LOG = LogFactory.getLog(ConfigurationManager.class);
73  
74    // The set of Configuration Observers. These classes would like to get
75    // notified when the configuration is reloaded from disk. This is a set
76    // constructed from a WeakHashMap, whose entries would be removed if the
77    // observer classes go out of scope.
78    private final Set<ConfigurationObserver> configurationObservers =
79        Collections.newSetFromMap(new WeakHashMap<ConfigurationObserver, Boolean>());
80  
81    /**
82     * Register an observer class
83     * @param observer observer to be registered.
84     */
85    public void registerObserver(ConfigurationObserver observer) {
86      synchronized (configurationObservers) {
87        configurationObservers.add(observer);
88        if (observer instanceof PropagatingConfigurationObserver) {
89          ((PropagatingConfigurationObserver) observer).registerChildren(this);
90        }
91      }
92    }
93  
94    /**
95     * Deregister an observer class
96     * @param observer to be deregistered.
97     */
98    public void deregisterObserver(ConfigurationObserver observer) {
99      synchronized (configurationObservers) {
100       configurationObservers.remove(observer);
101       if (observer instanceof PropagatingConfigurationObserver) {
102         ((PropagatingConfigurationObserver) observer).deregisterChildren(this);
103       }
104     }
105   }
106 
107   /**
108    * The conf object has been repopulated from disk, and we have to notify
109    * all the observers that are expressed interest to do that.
110    */
111   public void notifyAllObservers(Configuration conf) {
112     LOG.info("Starting to notify all observers that config changed.");
113     synchronized (configurationObservers) {
114       for (ConfigurationObserver observer : configurationObservers) {
115         try {
116           if (observer != null) {
117             observer.onConfigurationChange(conf);
118           }
119         } catch (Throwable t) {
120           LOG.error("Encountered a throwable while notifying observers: " + " of type : " +
121               observer.getClass().getCanonicalName() + "(" + observer + ")", t);
122         }
123       }
124     }
125   }
126 
127   /**
128    * @return the number of observers. 
129    */
130   public int getNumObservers() {
131     synchronized (configurationObservers) {
132       return configurationObservers.size();
133     }
134   }
135 }