View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.regionserver;
20  
21  import java.io.IOException;
22  import java.lang.reflect.Field;
23  import java.util.HashMap;
24  import java.util.Map;
25  
26  import org.apache.commons.logging.Log;
27  import org.apache.commons.logging.LogFactory;
28  import org.apache.hadoop.hbase.classification.InterfaceAudience;
29  import org.apache.hadoop.conf.Configuration;
30  import org.apache.hadoop.fs.FileSystem;
31  import org.apache.hadoop.hbase.HBaseConfiguration;
32  import org.apache.hadoop.hbase.Stoppable;
33  import org.apache.hadoop.hbase.util.ShutdownHookManager;
34  import org.apache.hadoop.hbase.util.Threads;
35  
36  /**
37   * Manage regionserver shutdown hooks.
38   * @see #install(Configuration, FileSystem, Stoppable, Thread)
39   */
40  @InterfaceAudience.Private
41  public class ShutdownHook {
42    private static final Log LOG = LogFactory.getLog(ShutdownHook.class);
43    private static final String CLIENT_FINALIZER_DATA_METHOD = "clientFinalizer";
44  
45    /**
46     * Key for boolean configuration whose default is true.
47     */
48    public static final String RUN_SHUTDOWN_HOOK = "hbase.shutdown.hook";
49  
50    /**
51     * Key for a long configuration on how much time to wait on the fs shutdown
52     * hook. Default is 30 seconds.
53     */
54    public static final String FS_SHUTDOWN_HOOK_WAIT = "hbase.fs.shutdown.hook.wait";
55  
56    /**
57     * A place for keeping track of all the filesystem shutdown hooks that need
58     * to be executed after the last regionserver referring to a given filesystem
59     * stops. We keep track of the # of regionserver references in values of the map.
60     */
61    private final static Map<Runnable, Integer> fsShutdownHooks = new HashMap<Runnable, Integer>();
62  
63    /**
64     * Install a shutdown hook that calls stop on the passed Stoppable
65     * and then thread joins against the passed <code>threadToJoin</code>.
66     * When this thread completes, it then runs the hdfs thread (This install
67     * removes the hdfs shutdown hook keeping a handle on it to run it after
68     * <code>threadToJoin</code> has stopped).
69     *
70     * <p>To suppress all shutdown hook  handling -- both the running of the
71     * regionserver hook and of the hdfs hook code -- set
72     * {@link ShutdownHook#RUN_SHUTDOWN_HOOK} in {@link Configuration} to
73     * <code>false</code>.
74     * This configuration value is checked when the hook code runs.
75     * @param conf
76     * @param fs Instance of Filesystem used by the RegionServer
77     * @param stop Installed shutdown hook will call stop against this passed
78     * <code>Stoppable</code> instance.
79     * @param threadToJoin After calling stop on <code>stop</code> will then
80     * join this thread.
81     */
82    public static void install(final Configuration conf, final FileSystem fs,
83        final Stoppable stop, final Thread threadToJoin) {
84      Runnable fsShutdownHook = suppressHdfsShutdownHook(fs);
85      Thread t = new ShutdownHookThread(conf, stop, threadToJoin, fsShutdownHook);
86      ShutdownHookManager.affixShutdownHook(t, 0);
87      LOG.debug("Installed shutdown hook thread: " + t.getName());
88    }
89  
90    /*
91     * Thread run by shutdown hook.
92     */
93    private static class ShutdownHookThread extends Thread {
94      private final Stoppable stop;
95      private final Thread threadToJoin;
96      private final Runnable fsShutdownHook;
97      private final Configuration conf;
98  
99      ShutdownHookThread(final Configuration conf, final Stoppable stop,
100         final Thread threadToJoin, final Runnable fsShutdownHook) {
101       super("Shutdownhook:" + threadToJoin.getName());
102       this.stop = stop;
103       this.threadToJoin = threadToJoin;
104       this.conf = conf;
105       this.fsShutdownHook = fsShutdownHook;
106     }
107 
108     @Override
109     public void run() {
110       boolean b = this.conf.getBoolean(RUN_SHUTDOWN_HOOK, true);
111       LOG.info("Shutdown hook starting; " + RUN_SHUTDOWN_HOOK + "=" + b +
112         "; fsShutdownHook=" + this.fsShutdownHook);
113       if (b) {
114         this.stop.stop("Shutdown hook");
115         Threads.shutdown(this.threadToJoin);
116         if (this.fsShutdownHook != null) {
117           synchronized (fsShutdownHooks) {
118             int refs = fsShutdownHooks.get(fsShutdownHook);
119             if (refs == 1) {
120               LOG.info("Starting fs shutdown hook thread.");
121               Thread fsShutdownHookThread = (fsShutdownHook instanceof Thread) ?
122                 (Thread)fsShutdownHook : new Thread(fsShutdownHook);
123               fsShutdownHookThread.start();
124               Threads.shutdown(fsShutdownHookThread,
125               this.conf.getLong(FS_SHUTDOWN_HOOK_WAIT, 30000));
126             }
127             if (refs > 0) {
128               fsShutdownHooks.put(fsShutdownHook, refs - 1);
129             }
130           }
131         }
132       }
133       LOG.info("Shutdown hook finished.");
134     }
135   }
136 
137   /*
138    * So, HDFS keeps a static map of all FS instances. In order to make sure
139    * things are cleaned up on our way out, it also creates a shutdown hook
140    * so that all filesystems can be closed when the process is terminated; it
141    * calls FileSystem.closeAll. This inconveniently runs concurrently with our
142    * own shutdown handler, and therefore causes all the filesystems to be closed
143    * before the server can do all its necessary cleanup.
144    *
145    * <p>The dirty reflection in this method sneaks into the FileSystem class
146    * and grabs the shutdown hook, removes it from the list of active shutdown
147    * hooks, and returns the hook for the caller to run at its convenience.
148    *
149    * <p>This seems quite fragile and susceptible to breaking if Hadoop changes
150    * anything about the way this cleanup is managed. Keep an eye on things.
151    * @return The fs shutdown hook
152    * @throws RuntimeException if we fail to find or grap the shutdown hook.
153    */
154   private static Runnable suppressHdfsShutdownHook(final FileSystem fs) {
155     try {
156       // This introspection has been updated to work for hadoop 0.20, 0.21 and for
157       // cloudera 0.20.  0.21 and cloudera 0.20 both have hadoop-4829.  With the
158       // latter in place, things are a little messy in that there are now two
159       // instances of the data member clientFinalizer; an uninstalled one in
160       // FileSystem and one in the innner class named Cache that actually gets
161       // registered as a shutdown hook.  If the latter is present, then we are
162       // on 0.21 or cloudera patched 0.20.
163       Runnable hdfsClientFinalizer = null;
164       // Look into the FileSystem#Cache class for clientFinalizer
165       Class<?> [] classes = FileSystem.class.getDeclaredClasses();
166       Class<?> cache = null;
167       for (Class<?> c: classes) {
168         if (c.getSimpleName().equals("Cache")) {
169           cache = c;
170           break;
171         }
172       }
173 
174       if (cache == null) {
175         throw new RuntimeException(
176             "This should not happen. Could not find the cache class in FileSystem.");
177       }
178 
179       Field field = null;
180       try {
181         field = cache.getDeclaredField(CLIENT_FINALIZER_DATA_METHOD);
182       } catch (NoSuchFieldException e) {
183         // We can get here if the Cache class does not have a clientFinalizer
184         // instance: i.e. we're running on straight 0.20 w/o hadoop-4829.
185       }
186       if (field != null) {
187         field.setAccessible(true);
188         Field cacheField = FileSystem.class.getDeclaredField("CACHE");
189         cacheField.setAccessible(true);
190         Object cacheInstance = cacheField.get(fs);
191         hdfsClientFinalizer = (Runnable)field.get(cacheInstance);
192       } else {
193         // Then we didnt' find clientFinalizer in Cache.  Presume clean 0.20 hadoop.
194         field = FileSystem.class.getDeclaredField(CLIENT_FINALIZER_DATA_METHOD);
195         field.setAccessible(true);
196         hdfsClientFinalizer = (Runnable)field.get(null);
197       }
198       if (hdfsClientFinalizer == null) {
199         throw new RuntimeException("Client finalizer is null, can't suppress!");
200       }
201       synchronized (fsShutdownHooks) {
202         if (!fsShutdownHooks.containsKey(hdfsClientFinalizer) &&
203             !ShutdownHookManager.deleteShutdownHook(hdfsClientFinalizer)) {
204           throw new RuntimeException("Failed suppression of fs shutdown hook: " +
205             hdfsClientFinalizer);
206         }
207         Integer refs = fsShutdownHooks.get(hdfsClientFinalizer);
208         fsShutdownHooks.put(hdfsClientFinalizer, refs == null ? 1 : refs + 1);
209       }
210       return hdfsClientFinalizer;
211     } catch (NoSuchFieldException nsfe) {
212       LOG.fatal("Couldn't find field 'clientFinalizer' in FileSystem!", nsfe);
213       throw new RuntimeException("Failed to suppress HDFS shutdown hook");
214     } catch (IllegalAccessException iae) {
215       LOG.fatal("Couldn't access field 'clientFinalizer' in FileSystem!", iae);
216       throw new RuntimeException("Failed to suppress HDFS shutdown hook");
217     }
218   }
219 
220   // Thread that does nothing. Used in below main testing.
221   static class DoNothingThread extends Thread {
222     DoNothingThread() {
223       super("donothing");
224     }
225     @Override
226     public void run() {
227       super.run();
228     }
229   }
230 
231   // Stoppable with nothing to stop.  Used below in main testing.
232   static class DoNothingStoppable implements Stoppable {
233     @Override
234     public boolean isStopped() {
235       // TODO Auto-generated method stub
236       return false;
237     }
238 
239     @Override
240     public void stop(String why) {
241       // TODO Auto-generated method stub
242     }
243   }
244 
245   /**
246    * Main to test basic functionality.  Run with clean hadoop 0.20 and hadoop
247    * 0.21 and cloudera patched hadoop to make sure our shutdown hook handling
248    * works for all compbinations.
249    * Pass '-Dhbase.shutdown.hook=false' to test turning off the running of
250    * shutdown hooks.
251    * @param args
252    * @throws IOException
253    */
254   public static void main(final String [] args) throws IOException {
255     Configuration conf = HBaseConfiguration.create();
256     String prop = System.getProperty(RUN_SHUTDOWN_HOOK);
257     if (prop != null) {
258       conf.setBoolean(RUN_SHUTDOWN_HOOK, Boolean.parseBoolean(prop));
259     }
260     // Instantiate a FileSystem. This will register the fs shutdown hook.
261     FileSystem fs = FileSystem.get(conf);
262     Thread donothing = new DoNothingThread();
263     donothing.start();
264     ShutdownHook.install(conf, fs, new DoNothingStoppable(), donothing);
265   }
266 }