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