View Javadoc

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