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