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  
25  import org.apache.commons.logging.Log;
26  import org.apache.commons.logging.LogFactory;
27  import org.apache.hadoop.conf.Configuration;
28  import org.apache.hadoop.fs.FileSystem;
29  import org.apache.hadoop.hbase.HBaseConfiguration;
30  import org.apache.hadoop.hbase.Stoppable;
31  import org.apache.hadoop.hbase.util.Threads;
32  
33  /**
34   * Manage regionserver shutdown hooks.
35   * @see #install(Configuration, FileSystem, Stoppable, Thread)
36   */
37  class ShutdownHook {
38    private static final Log LOG = LogFactory.getLog(ShutdownHook.class);
39    private static final String CLIENT_FINALIZER_DATA_METHOD = "clientFinalizer";
40  
41    /**
42     * Key for boolean configuration whose default is true.
43     */
44    public static final String RUN_SHUTDOWN_HOOK = "hbase.shutdown.hook";
45  
46    /**
47     * Key for a long configuration on how much time to wait on the fs shutdown
48     * hook. Default is 30 seconds.
49     */
50    public static final String FS_SHUTDOWN_HOOK_WAIT = "hbase.fs.shutdown.hook.wait";
51  
52    /**
53     * Install a shutdown hook that calls stop on the passed Stoppable
54     * and then thread joins against the passed <code>threadToJoin</code>.
55     * When this thread completes, it then runs the hdfs thread (This install
56     * removes the hdfs shutdown hook keeping a handle on it to run it after
57     * <code>threadToJoin</code> has stopped).
58     *
59     * <p>To suppress all shutdown hook  handling -- both the running of the
60     * regionserver hook and of the hdfs hook code -- set
61     * {@link ShutdownHook#RUN_SHUTDOWN_HOOK} in {@link Configuration} to
62     * <code>false</code>.
63     * This configuration value is checked when the hook code runs.
64     * @param conf
65     * @param fs Instance of Filesystem used by the RegionServer
66     * @param stop Installed shutdown hook will call stop against this passed
67     * <code>Stoppable</code> instance.
68     * @param threadToJoin After calling stop on <code>stop</code> will then
69     * join this thread.
70     */
71    static void install(final Configuration conf, final FileSystem fs,
72        final Stoppable stop, final Thread threadToJoin) {
73      Thread fsShutdownHook = suppressHdfsShutdownHook(fs);
74      Thread t = new ShutdownHookThread(conf, stop, threadToJoin, fsShutdownHook);
75      Runtime.getRuntime().addShutdownHook(t);
76      LOG.info("Installed shutdown hook thread: " + t.getName());
77    }
78  
79    /*
80     * Thread run by shutdown hook.
81     */
82    private static class ShutdownHookThread extends Thread {
83      private final Stoppable stop;
84      private final Thread threadToJoin;
85      private final Thread fsShutdownHook;
86      private final Configuration conf;
87  
88      ShutdownHookThread(final Configuration conf, final Stoppable stop,
89          final Thread threadToJoin, final Thread fsShutdownHook) {
90        super("Shutdownhook:" + threadToJoin.getName());
91        this.stop = stop;
92        this.threadToJoin = threadToJoin;
93        this.conf = conf;
94        this.fsShutdownHook = fsShutdownHook;
95      }
96  
97      @Override
98      public void run() {
99        boolean b = this.conf.getBoolean(RUN_SHUTDOWN_HOOK, true);
100       LOG.info("Shutdown hook starting; " + RUN_SHUTDOWN_HOOK + "=" + b +
101         "; fsShutdownHook=" + this.fsShutdownHook);
102       if (b) {
103         this.stop.stop("Shutdown hook");
104         Threads.shutdown(this.threadToJoin);
105         if (this.fsShutdownHook != null) {
106           LOG.info("Starting fs shutdown hook thread.");
107           this.fsShutdownHook.start();
108           Threads.shutdown(this.fsShutdownHook,
109             this.conf.getLong(FS_SHUTDOWN_HOOK_WAIT, 30000));
110         }
111       }
112       LOG.info("Shutdown hook finished.");
113     }
114   }
115 
116   /*
117    * So, HDFS keeps a static map of all FS instances. In order to make sure
118    * things are cleaned up on our way out, it also creates a shutdown hook
119    * so that all filesystems can be closed when the process is terminated; it
120    * calls FileSystem.closeAll. This inconveniently runs concurrently with our
121    * own shutdown handler, and therefore causes all the filesystems to be closed
122    * before the server can do all its necessary cleanup.
123    *
124    * <p>The dirty reflection in this method sneaks into the FileSystem class
125    * and grabs the shutdown hook, removes it from the list of active shutdown
126    * hooks, and returns the hook for the caller to run at its convenience.
127    *
128    * <p>This seems quite fragile and susceptible to breaking if Hadoop changes
129    * anything about the way this cleanup is managed. Keep an eye on things.
130    * @return The fs shutdown hook
131    * @throws RuntimeException if we fail to find or grap the shutdown hook.
132    */
133   private static Thread suppressHdfsShutdownHook(final FileSystem fs) {
134     try {
135       // This introspection has been updated to work for hadoop 0.20, 0.21 and for
136       // cloudera 0.20.  0.21 and cloudera 0.20 both have hadoop-4829.  With the
137       // latter in place, things are a little messy in that there are now two
138       // instances of the data member clientFinalizer; an uninstalled one in
139       // FileSystem and one in the innner class named Cache that actually gets
140       // registered as a shutdown hook.  If the latter is present, then we are
141       // on 0.21 or cloudera patched 0.20.
142       Thread hdfsClientFinalizer = null;
143       // Look into the FileSystem#Cache class for clientFinalizer
144       Class<?> [] classes = FileSystem.class.getDeclaredClasses();
145       Class<?> cache = null;
146       for (Class<?> c: classes) {
147         if (c.getSimpleName().equals("Cache")) {
148           cache = c;
149           break;
150         }
151       }
152       Field field = null;
153       try {
154         field = cache.getDeclaredField(CLIENT_FINALIZER_DATA_METHOD);
155       } catch (NoSuchFieldException e) {
156         // We can get here if the Cache class does not have a clientFinalizer
157         // instance: i.e. we're running on straight 0.20 w/o hadoop-4829.
158       }
159       if (field != null) {
160         field.setAccessible(true);
161         Field cacheField = FileSystem.class.getDeclaredField("CACHE");
162         cacheField.setAccessible(true);
163         Object cacheInstance = cacheField.get(fs);
164         hdfsClientFinalizer = (Thread)field.get(cacheInstance);
165       } else {
166         // Then we didnt' find clientFinalizer in Cache.  Presume clean 0.20 hadoop.
167         field = FileSystem.class.getDeclaredField(CLIENT_FINALIZER_DATA_METHOD);
168         field.setAccessible(true);
169         hdfsClientFinalizer = (Thread)field.get(null);
170       }
171       if (hdfsClientFinalizer == null) {
172         throw new RuntimeException("Client finalizer is null, can't suppress!");
173       }
174       if (!Runtime.getRuntime().removeShutdownHook(hdfsClientFinalizer)) {
175         throw new RuntimeException("Failed suppression of fs shutdown hook: " +
176           hdfsClientFinalizer);
177       }
178       return hdfsClientFinalizer;
179     } catch (NoSuchFieldException nsfe) {
180       LOG.fatal("Couldn't find field 'clientFinalizer' in FileSystem!", nsfe);
181       throw new RuntimeException("Failed to suppress HDFS shutdown hook");
182     } catch (IllegalAccessException iae) {
183       LOG.fatal("Couldn't access field 'clientFinalizer' in FileSystem!", iae);
184       throw new RuntimeException("Failed to suppress HDFS shutdown hook");
185     }
186   }
187 
188   // Thread that does nothing. Used in below main testing.
189   static class DoNothingThread extends Thread {
190     DoNothingThread() {
191       super("donothing");
192     }
193     @Override
194     public void run() {
195       super.run();
196     }
197   }
198 
199   // Stoppable with nothing to stop.  Used below in main testing.
200   static class DoNothingStoppable implements Stoppable {
201     @Override
202     public boolean isStopped() {
203       // TODO Auto-generated method stub
204       return false;
205     }
206 
207     @Override
208     public void stop(String why) {
209       // TODO Auto-generated method stub
210     }
211   }
212 
213   /**
214    * Main to test basic functionality.  Run with clean hadoop 0.20 and hadoop
215    * 0.21 and cloudera patched hadoop to make sure our shutdown hook handling
216    * works for all compbinations.
217    * Pass '-Dhbase.shutdown.hook=false' to test turning off the running of
218    * shutdown hooks.
219    * @param args
220    * @throws IOException
221    */
222   public static void main(final String [] args) throws IOException {
223     Configuration conf = HBaseConfiguration.create();
224     String prop = System.getProperty(RUN_SHUTDOWN_HOOK);
225     if (prop != null) {
226       conf.setBoolean(RUN_SHUTDOWN_HOOK, Boolean.parseBoolean(prop));
227     }
228     // Instantiate a FileSystem. This will register the fs shutdown hook.
229     FileSystem fs = FileSystem.get(conf);
230     Thread donothing = new DoNothingThread();
231     donothing.start();
232     ShutdownHook.install(conf, fs, new DoNothingStoppable(), donothing);
233   }
234 }