001/**
002 *
003 * Licensed to the Apache Software Foundation (ASF) under one
004 * or more contributor license agreements.  See the NOTICE file
005 * distributed with this work for additional information
006 * regarding copyright ownership.  The ASF licenses this file
007 * to you under the Apache License, Version 2.0 (the
008 * "License"); you may not use this file except in compliance
009 * with the License.  You may obtain a copy of the License at
010 *
011 *     http://www.apache.org/licenses/LICENSE-2.0
012 *
013 * Unless required by applicable law or agreed to in writing, software
014 * distributed under the License is distributed on an "AS IS" BASIS,
015 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
016 * See the License for the specific language governing permissions and
017 * limitations under the License.
018 */
019package org.apache.hadoop.hbase.regionserver;
020
021import java.io.IOException;
022import java.lang.reflect.Field;
023import java.util.HashMap;
024import java.util.Map;
025
026import org.apache.yetus.audience.InterfaceAudience;
027import org.slf4j.Logger;
028import org.slf4j.LoggerFactory;
029import org.apache.hadoop.conf.Configuration;
030import org.apache.hadoop.fs.FileSystem;
031import org.apache.hadoop.hbase.HBaseConfiguration;
032import org.apache.hadoop.hbase.Stoppable;
033import org.apache.hadoop.hbase.log.HBaseMarkers;
034import org.apache.hadoop.hbase.util.ShutdownHookManager;
035import org.apache.hadoop.hbase.util.Threads;
036
037/**
038 * Manage regionserver shutdown hooks.
039 * @see #install(Configuration, FileSystem, Stoppable, Thread)
040 */
041@InterfaceAudience.Private
042public class ShutdownHook {
043  private static final Logger LOG = LoggerFactory.getLogger(ShutdownHook.class);
044  private static final String CLIENT_FINALIZER_DATA_METHOD = "clientFinalizer";
045
046  /**
047   * Key for boolean configuration whose default is true.
048   */
049  public static final String RUN_SHUTDOWN_HOOK = "hbase.shutdown.hook";
050
051  /**
052   * Key for a long configuration on how much time to wait on the fs shutdown
053   * hook. Default is 30 seconds.
054   */
055  public static final String FS_SHUTDOWN_HOOK_WAIT = "hbase.fs.shutdown.hook.wait";
056
057  /**
058   * A place for keeping track of all the filesystem shutdown hooks that need
059   * to be executed after the last regionserver referring to a given filesystem
060   * stops. We keep track of the # of regionserver references in values of the map.
061   */
062  private final static Map<Runnable, Integer> fsShutdownHooks = new HashMap<>();
063
064  /**
065   * Install a shutdown hook that calls stop on the passed Stoppable
066   * and then thread joins against the passed <code>threadToJoin</code>.
067   * When this thread completes, it then runs the hdfs thread (This install
068   * removes the hdfs shutdown hook keeping a handle on it to run it after
069   * <code>threadToJoin</code> has stopped).
070   *
071   * <p>To suppress all shutdown hook  handling -- both the running of the
072   * regionserver hook and of the hdfs hook code -- set
073   * {@link ShutdownHook#RUN_SHUTDOWN_HOOK} in {@link Configuration} to
074   * <code>false</code>.
075   * This configuration value is checked when the hook code runs.
076   * @param conf
077   * @param fs Instance of Filesystem used by the RegionServer
078   * @param stop Installed shutdown hook will call stop against this passed
079   * <code>Stoppable</code> instance.
080   * @param threadToJoin After calling stop on <code>stop</code> will then
081   * join this thread.
082   */
083  public static void install(final Configuration conf, final FileSystem fs,
084      final Stoppable stop, final Thread threadToJoin) {
085    Runnable fsShutdownHook = suppressHdfsShutdownHook(fs);
086    Thread t = new ShutdownHookThread(conf, stop, threadToJoin, fsShutdownHook);
087    ShutdownHookManager.affixShutdownHook(t, 0);
088    LOG.debug("Installed shutdown hook thread: " + t.getName());
089  }
090
091  /*
092   * Thread run by shutdown hook.
093   */
094  private static class ShutdownHookThread extends Thread {
095    private final Stoppable stop;
096    private final Thread threadToJoin;
097    private final Runnable fsShutdownHook;
098    private final Configuration conf;
099
100    ShutdownHookThread(final Configuration conf, final Stoppable stop,
101        final Thread threadToJoin, final Runnable fsShutdownHook) {
102      super("Shutdownhook:" + threadToJoin.getName());
103      this.stop = stop;
104      this.threadToJoin = threadToJoin;
105      this.conf = conf;
106      this.fsShutdownHook = fsShutdownHook;
107    }
108
109    @Override
110    public void run() {
111      boolean b = this.conf.getBoolean(RUN_SHUTDOWN_HOOK, true);
112      LOG.info("Shutdown hook starting; " + RUN_SHUTDOWN_HOOK + "=" + b +
113        "; fsShutdownHook=" + this.fsShutdownHook);
114      if (b) {
115        this.stop.stop("Shutdown hook");
116        Threads.shutdown(this.threadToJoin);
117        if (this.fsShutdownHook != null) {
118          synchronized (fsShutdownHooks) {
119            int refs = fsShutdownHooks.get(fsShutdownHook);
120            if (refs == 1) {
121              LOG.info("Starting fs shutdown hook thread.");
122              Thread fsShutdownHookThread = (fsShutdownHook instanceof Thread) ?
123                (Thread)fsShutdownHook : new Thread(fsShutdownHook,
124                  fsShutdownHook.getClass().getSimpleName() + "-shutdown-hook");
125              fsShutdownHookThread.start();
126              Threads.shutdown(fsShutdownHookThread,
127              this.conf.getLong(FS_SHUTDOWN_HOOK_WAIT, 30000));
128            }
129            if (refs > 0) {
130              fsShutdownHooks.put(fsShutdownHook, refs - 1);
131            }
132          }
133        }
134      }
135      LOG.info("Shutdown hook finished.");
136    }
137  }
138
139  /*
140   * So, HDFS keeps a static map of all FS instances. In order to make sure
141   * things are cleaned up on our way out, it also creates a shutdown hook
142   * so that all filesystems can be closed when the process is terminated; it
143   * calls FileSystem.closeAll. This inconveniently runs concurrently with our
144   * own shutdown handler, and therefore causes all the filesystems to be closed
145   * before the server can do all its necessary cleanup.
146   *
147   * <p>The dirty reflection in this method sneaks into the FileSystem class
148   * and grabs the shutdown hook, removes it from the list of active shutdown
149   * hooks, and returns the hook for the caller to run at its convenience.
150   *
151   * <p>This seems quite fragile and susceptible to breaking if Hadoop changes
152   * anything about the way this cleanup is managed. Keep an eye on things.
153   * @return The fs shutdown hook
154   * @throws RuntimeException if we fail to find or grap the shutdown hook.
155   */
156  private static Runnable suppressHdfsShutdownHook(final FileSystem fs) {
157    try {
158      // This introspection has been updated to work for hadoop 0.20, 0.21 and for
159      // cloudera 0.20.  0.21 and cloudera 0.20 both have hadoop-4829.  With the
160      // latter in place, things are a little messy in that there are now two
161      // instances of the data member clientFinalizer; an uninstalled one in
162      // FileSystem and one in the innner class named Cache that actually gets
163      // registered as a shutdown hook.  If the latter is present, then we are
164      // on 0.21 or cloudera patched 0.20.
165      Runnable hdfsClientFinalizer = null;
166      // Look into the FileSystem#Cache class for clientFinalizer
167      Class<?> [] classes = FileSystem.class.getDeclaredClasses();
168      Class<?> cache = null;
169      for (Class<?> c: classes) {
170        if (c.getSimpleName().equals("Cache")) {
171          cache = c;
172          break;
173        }
174      }
175
176      if (cache == null) {
177        throw new RuntimeException(
178            "This should not happen. Could not find the cache class in FileSystem.");
179      }
180
181      Field field = null;
182      try {
183        field = cache.getDeclaredField(CLIENT_FINALIZER_DATA_METHOD);
184      } catch (NoSuchFieldException e) {
185        // We can get here if the Cache class does not have a clientFinalizer
186        // instance: i.e. we're running on straight 0.20 w/o hadoop-4829.
187      }
188      if (field != null) {
189        field.setAccessible(true);
190        Field cacheField = FileSystem.class.getDeclaredField("CACHE");
191        cacheField.setAccessible(true);
192        Object cacheInstance = cacheField.get(fs);
193        hdfsClientFinalizer = (Runnable)field.get(cacheInstance);
194      } else {
195        // Then we didnt' find clientFinalizer in Cache.  Presume clean 0.20 hadoop.
196        field = FileSystem.class.getDeclaredField(CLIENT_FINALIZER_DATA_METHOD);
197        field.setAccessible(true);
198        hdfsClientFinalizer = (Runnable)field.get(null);
199      }
200      if (hdfsClientFinalizer == null) {
201        throw new RuntimeException("Client finalizer is null, can't suppress!");
202      }
203      synchronized (fsShutdownHooks) {
204        boolean isFSCacheDisabled = fs.getConf().getBoolean("fs.hdfs.impl.disable.cache", false);
205        if (!isFSCacheDisabled && !fsShutdownHooks.containsKey(hdfsClientFinalizer)
206            && !ShutdownHookManager.deleteShutdownHook(hdfsClientFinalizer)) {
207          throw new RuntimeException(
208              "Failed suppression of fs shutdown hook: " + hdfsClientFinalizer);
209        }
210        Integer refs = fsShutdownHooks.get(hdfsClientFinalizer);
211        fsShutdownHooks.put(hdfsClientFinalizer, refs == null ? 1 : refs + 1);
212      }
213      return hdfsClientFinalizer;
214    } catch (NoSuchFieldException nsfe) {
215      LOG.error(HBaseMarkers.FATAL, "Couldn't find field 'clientFinalizer' in FileSystem!",
216          nsfe);
217      throw new RuntimeException("Failed to suppress HDFS shutdown hook");
218    } catch (IllegalAccessException iae) {
219      LOG.error(HBaseMarkers.FATAL, "Couldn't access field 'clientFinalizer' in FileSystem!",
220          iae);
221      throw new RuntimeException("Failed to suppress HDFS shutdown hook");
222    }
223  }
224
225  // Thread that does nothing. Used in below main testing.
226  static class DoNothingThread extends Thread {
227    DoNothingThread() {
228      super("donothing");
229    }
230    @Override
231    public void run() {
232      super.run();
233    }
234  }
235
236  // Stoppable with nothing to stop.  Used below in main testing.
237  static class DoNothingStoppable implements Stoppable {
238    @Override
239    public boolean isStopped() {
240      // TODO Auto-generated method stub
241      return false;
242    }
243
244    @Override
245    public void stop(String why) {
246      // TODO Auto-generated method stub
247    }
248  }
249
250  /**
251   * Main to test basic functionality.  Run with clean hadoop 0.20 and hadoop
252   * 0.21 and cloudera patched hadoop to make sure our shutdown hook handling
253   * works for all compbinations.
254   * Pass '-Dhbase.shutdown.hook=false' to test turning off the running of
255   * shutdown hooks.
256   * @param args
257   * @throws IOException
258   */
259  public static void main(final String [] args) throws IOException {
260    Configuration conf = HBaseConfiguration.create();
261    String prop = System.getProperty(RUN_SHUTDOWN_HOOK);
262    if (prop != null) {
263      conf.setBoolean(RUN_SHUTDOWN_HOOK, Boolean.parseBoolean(prop));
264    }
265    // Instantiate a FileSystem. This will register the fs shutdown hook.
266    FileSystem fs = FileSystem.get(conf);
267    Thread donothing = new DoNothingThread();
268    donothing.start();
269    ShutdownHook.install(conf, fs, new DoNothingStoppable(), donothing);
270  }
271}