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  
21  package org.apache.hadoop.hbase.coprocessor;
22  
23  import com.google.common.collect.MapMaker;
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.fs.Path;
30  import org.apache.hadoop.hbase.Coprocessor;
31  import org.apache.hadoop.hbase.CoprocessorEnvironment;
32  import org.apache.hadoop.hbase.DoNotRetryIOException;
33  import org.apache.hadoop.hbase.HBaseConfiguration;
34  import org.apache.hadoop.hbase.HTableDescriptor;
35  import org.apache.hadoop.hbase.client.*;
36  import org.apache.hadoop.hbase.client.coprocessor.Batch;
37  import org.apache.hadoop.hbase.ipc.CoprocessorProtocol;
38  import org.apache.hadoop.hbase.util.Bytes;
39  import org.apache.hadoop.hbase.util.SortedCopyOnWriteSet;
40  import org.apache.hadoop.hbase.util.VersionInfo;
41  import org.apache.hadoop.hbase.Server;
42  import org.apache.hadoop.io.IOUtils;
43  
44  import java.io.File;
45  import java.io.FileOutputStream;
46  import java.io.IOException;
47  import java.net.URL;
48  import java.util.*;
49  import java.util.concurrent.ConcurrentMap;
50  import java.util.jar.JarEntry;
51  import java.util.jar.JarFile;
52  
53  /**
54   * Provides the common setup framework and runtime services for coprocessor
55   * invocation from HBase services.
56   * @param <E> the specific environment extension that a concrete implementation
57   * provides
58   */
59  public abstract class CoprocessorHost<E extends CoprocessorEnvironment> {
60    public static final String REGION_COPROCESSOR_CONF_KEY =
61      "hbase.coprocessor.region.classes";
62    public static final String REGIONSERVER_COPROCESSOR_CONF_KEY =
63      "hbase.coprocessor.regionserver.classes";
64    public static final String USER_REGION_COPROCESSOR_CONF_KEY =
65      "hbase.coprocessor.user.region.classes";
66    public static final String MASTER_COPROCESSOR_CONF_KEY =
67      "hbase.coprocessor.master.classes";
68    public static final String WAL_COPROCESSOR_CONF_KEY =
69      "hbase.coprocessor.wal.classes";
70  
71    //coprocessor jars are put under ${hbase.local.dir}/coprocessor/jars/
72    private static final String COPROCESSOR_JARS_DIR = File.separator
73        + "coprocessor" + File.separator + "jars" + File.separator;
74  
75    private static final Log LOG = LogFactory.getLog(CoprocessorHost.class);
76    /** Ordered set of loaded coprocessors with lock */
77    protected SortedSet<E> coprocessors =
78        new SortedCopyOnWriteSet<E>(new EnvironmentPriorityComparator());
79    protected Configuration conf;
80    // unique file prefix to use for local copies of jars when classloading
81    protected String pathPrefix;
82    protected volatile int loadSequence;
83  
84    /*
85     * External classloaders cache keyed by external jar path.
86     * ClassLoader instance is stored as a weak-reference
87     * to allow GC'ing when no CoprocessorHost is using it
88     * (@see HBASE-7205)
89     */
90    static ConcurrentMap<Path, ClassLoader> classLoadersCache =
91        new MapMaker().concurrencyLevel(3).weakValues().makeMap();
92  
93    public CoprocessorHost() {
94      pathPrefix = UUID.randomUUID().toString();
95    }
96  
97    /**
98     * Not to be confused with the per-object _coprocessors_ (above),
99     * coprocessorNames is static and stores the set of all coprocessors ever
100    * loaded by any thread in this JVM. It is strictly additive: coprocessors are
101    * added to coprocessorNames, by loadInstance() but are never removed, since
102    * the intention is to preserve a history of all loaded coprocessors for
103    * diagnosis in case of server crash (HBASE-4014).
104    */
105   private static Set<String> coprocessorNames =
106       Collections.synchronizedSet(new HashSet<String>());
107   public static Set<String> getLoadedCoprocessors() {
108       return coprocessorNames;
109   }
110 
111   /**
112    * Used to create a parameter to the HServerLoad constructor so that
113    * HServerLoad can provide information about the coprocessors loaded by this
114    * regionserver.
115    * (HBASE-4070: Improve region server metrics to report loaded coprocessors
116    * to master).
117    */
118   public Set<String> getCoprocessors() {
119     Set<String> returnValue = new TreeSet<String>();
120     for(CoprocessorEnvironment e: coprocessors) {
121       returnValue.add(e.getInstance().getClass().getSimpleName());
122     }
123     return returnValue;
124   }
125 
126   /**
127    * Load system coprocessors. Read the class names from configuration.
128    * Called by constructor.
129    */
130   protected void loadSystemCoprocessors(Configuration conf, String confKey) {
131     Class<?> implClass = null;
132 
133     // load default coprocessors from configure file    
134     String[] defaultCPClasses = conf.getStrings(confKey);
135     if (defaultCPClasses == null || defaultCPClasses.length == 0)
136       return;
137 
138     int priority = Coprocessor.PRIORITY_SYSTEM;
139     List<E> configured = new ArrayList<E>();
140     for (String className : defaultCPClasses) {
141       className = className.trim();
142       if (findCoprocessor(className) != null) {
143         continue;
144       }
145       ClassLoader cl = this.getClass().getClassLoader();
146       Thread.currentThread().setContextClassLoader(cl);
147       try {
148         implClass = cl.loadClass(className);
149         configured.add(loadInstance(implClass, Coprocessor.PRIORITY_SYSTEM, conf));
150         LOG.info("System coprocessor " + className + " was loaded " +
151             "successfully with priority (" + priority++ + ").");
152       } catch (ClassNotFoundException e) {
153         LOG.warn("Class " + className + " cannot be found. " +
154             e.getMessage());
155       } catch (IOException e) {
156         LOG.warn("Load coprocessor " + className + " failed. " +
157             e.getMessage());
158       }
159     }
160 
161     // add entire set to the collection for COW efficiency
162     coprocessors.addAll(configured);
163   }
164 
165   /**
166    * Load a coprocessor implementation into the host
167    * @param path path to implementation jar
168    * @param className the main class name
169    * @param priority chaining priority
170    * @param conf configuration for coprocessor
171    * @throws java.io.IOException Exception
172    */
173   @SuppressWarnings("deprecation")
174   public E load(Path path, String className, int priority,
175       Configuration conf) throws IOException {
176     Class<?> implClass = null;
177     LOG.debug("Loading coprocessor class " + className + " with path " + 
178         path + " and priority " + priority);
179 
180     ClassLoader cl = null;
181     if (path == null) {
182       try {
183         implClass = getClass().getClassLoader().loadClass(className);
184       } catch (ClassNotFoundException e) {
185         throw new IOException("No jar path specified for " + className);
186       }
187     } else {
188       // Have we already loaded the class, perhaps from an earlier region open
189       // for the same table?
190       cl = classLoadersCache.get(path);
191       if (cl != null){
192         LOG.debug("Found classloader "+ cl + "for "+path.toString());
193         try {
194           implClass = cl.loadClass(className);
195         } catch (ClassNotFoundException e) {
196           LOG.info("Class " + className + " needs to be loaded from a file - " +
197               path + ".");
198           // go ahead to load from file system.
199         }
200       }
201     }
202 
203     // If not, load
204     if (implClass == null) {
205       if (path == null) {
206         throw new IOException("No jar path specified for " + className);
207       }
208       // copy the jar to the local filesystem
209       if (!path.toString().endsWith(".jar")) {
210         throw new IOException(path.toString() + ": not a jar file?");
211       }
212       FileSystem fs = path.getFileSystem(this.conf);
213       File parentDir = new File(this.conf.get("hbase.local.dir") + COPROCESSOR_JARS_DIR);
214       parentDir.mkdirs();
215       File dst = new File(parentDir, "." + pathPrefix +
216           "." + className + "." + System.currentTimeMillis() + ".jar");
217       fs.copyToLocalFile(path, new Path(dst.toString()));
218       dst.deleteOnExit();
219 
220       // TODO: code weaving goes here
221 
222       // TODO: wrap heap allocations and enforce maximum usage limits
223 
224       /* TODO: inject code into loop headers that monitors CPU use and
225          aborts runaway user code */
226 
227       // load the jar and get the implementation main class
228       // NOTE: Path.toURL is deprecated (toURI instead) but the URLClassLoader
229       // unsurprisingly wants URLs, not URIs; so we will use the deprecated
230       // method which returns URLs for as long as it is available
231       List<URL> paths = new ArrayList<URL>();
232       URL url = dst.getCanonicalFile().toURL();
233       paths.add(url);
234 
235       JarFile jarFile = new JarFile(dst.toString());
236       Enumeration<JarEntry> entries = jarFile.entries();
237       while (entries.hasMoreElements()) {
238         JarEntry entry = entries.nextElement();
239         if (entry.getName().matches("/lib/[^/]+\\.jar")) {
240           File file = new File(parentDir, "." + pathPrefix +
241               "." + className + "." + System.currentTimeMillis() + "." + entry.getName().substring(5));
242           IOUtils.copyBytes(jarFile.getInputStream(entry), new FileOutputStream(file), conf, true);
243           file.deleteOnExit();
244           paths.add(file.toURL());
245         }
246       }
247       jarFile.close();
248 
249       cl = new CoprocessorClassLoader(paths, this.getClass().getClassLoader());
250       // cache cp classloader as a weak value, will be GC'ed when no reference left
251       ClassLoader prev = classLoadersCache.putIfAbsent(path, cl);
252       if (prev != null) {
253         //lost update race, use already added classloader
254         cl = prev;
255       }
256 
257       try {
258         implClass = cl.loadClass(className);
259       } catch (ClassNotFoundException e) {
260         throw new IOException("Cannot load external coprocessor class " + className, e);
261       }
262     }
263 
264     //load custom code for coprocessor
265     Thread currentThread = Thread.currentThread();
266     ClassLoader hostClassLoader = currentThread.getContextClassLoader();
267     try{
268       // switch temporarily to the thread classloader for custom CP
269       currentThread.setContextClassLoader(cl);
270       E cpInstance = loadInstance(implClass, priority, conf);
271       return cpInstance;
272     } finally {
273       // restore the fresh (host) classloader
274       currentThread.setContextClassLoader(hostClassLoader);
275     }
276   }
277 
278   /**
279    * @param implClass Implementation class
280    * @param priority priority
281    * @param conf configuration
282    * @throws java.io.IOException Exception
283    */
284   public void load(Class<?> implClass, int priority, Configuration conf)
285       throws IOException {
286     E env = loadInstance(implClass, priority, conf);
287     coprocessors.add(env);
288   }
289 
290   /**
291    * @param implClass Implementation class
292    * @param priority priority
293    * @param conf configuration
294    * @throws java.io.IOException Exception
295    */
296   public E loadInstance(Class<?> implClass, int priority, Configuration conf)
297       throws IOException {
298     // create the instance
299     Coprocessor impl;
300     Object o = null;
301     try {
302       o = implClass.newInstance();
303       impl = (Coprocessor)o;
304     } catch (InstantiationException e) {
305       throw new IOException(e);
306     } catch (IllegalAccessException e) {
307       throw new IOException(e);
308     }
309     // create the environment
310     E env = createEnvironment(implClass, impl, priority, ++loadSequence, conf);
311     if (env instanceof Environment) {
312       ((Environment)env).startup();
313     }
314     // HBASE-4014: maintain list of loaded coprocessors for later crash analysis
315     // if server (master or regionserver) aborts.
316     coprocessorNames.add(implClass.getName());
317     return env;
318   }
319 
320   /**
321    * Called when a new Coprocessor class is loaded
322    */
323   public abstract E createEnvironment(Class<?> implClass, Coprocessor instance,
324       int priority, int sequence, Configuration conf);
325 
326   public void shutdown(CoprocessorEnvironment e) {
327     if (e instanceof Environment) {
328       ((Environment)e).shutdown();
329     } else {
330       LOG.warn("Shutdown called on unknown environment: "+
331           e.getClass().getName());
332     }
333   }
334 
335   /**
336    * Find a coprocessor implementation by class name
337    * @param className the class name
338    * @return the coprocessor, or null if not found
339    */
340   public Coprocessor findCoprocessor(String className) {
341     // initialize the coprocessors
342     for (E env: coprocessors) {
343       if (env.getInstance().getClass().getName().equals(className) ||
344           env.getInstance().getClass().getSimpleName().equals(className)) {
345         return env.getInstance();
346       }
347     }
348     return null;
349   }
350 
351   /**
352    * Retrieves the set of classloaders used to instantiate Coprocessor classes defined in external
353    * jar files.
354    * @return A set of ClassLoader instances
355    */
356   Set<ClassLoader> getExternalClassLoaders() {
357     Set<ClassLoader> externalClassLoaders = new HashSet<ClassLoader>();
358     final ClassLoader systemClassLoader = this.getClass().getClassLoader();
359     for (E env : coprocessors) {
360       ClassLoader cl = env.getInstance().getClass().getClassLoader();
361       if (cl != systemClassLoader ){
362         //do not include system classloader
363         externalClassLoaders.add(cl);
364       }
365     }
366     return externalClassLoaders;
367   }
368 
369   /**
370    * Find a coprocessor environment by class name
371    * @param className the class name
372    * @return the coprocessor, or null if not found
373    */
374   public CoprocessorEnvironment findCoprocessorEnvironment(String className) {
375     // initialize the coprocessors
376     for (E env: coprocessors) {
377       if (env.getInstance().getClass().getName().equals(className) ||
378           env.getInstance().getClass().getSimpleName().equals(className)) {
379         return env;
380       }
381     }
382     return null;
383   }
384 
385   /**
386    * Environment priority comparator.
387    * Coprocessors are chained in sorted order.
388    */
389   static class EnvironmentPriorityComparator
390       implements Comparator<CoprocessorEnvironment> {
391     public int compare(final CoprocessorEnvironment env1,
392         final CoprocessorEnvironment env2) {
393       if (env1.getPriority() < env2.getPriority()) {
394         return -1;
395       } else if (env1.getPriority() > env2.getPriority()) {
396         return 1;
397       }
398       if (env1.getLoadSequence() < env2.getLoadSequence()) {
399         return -1;
400       } else if (env1.getLoadSequence() > env2.getLoadSequence()) {
401         return 1;
402       }
403       return 0;
404     }
405   }
406 
407   /**
408    * Encapsulation of the environment of each coprocessor
409    */
410   public static class Environment implements CoprocessorEnvironment {
411 
412     /**
413      * A wrapper for HTable. Can be used to restrict privilege.
414      *
415      * Currently it just helps to track tables opened by a Coprocessor and
416      * facilitate close of them if it is aborted.
417      *
418      * We also disallow row locking.
419      *
420      * There is nothing now that will stop a coprocessor from using HTable
421      * objects directly instead of this API, but in the future we intend to
422      * analyze coprocessor implementations as they are loaded and reject those
423      * which attempt to use objects and methods outside the Environment
424      * sandbox.
425      */
426     class HTableWrapper implements HTableInterface {
427 
428       private byte[] tableName;
429       private HTable table;
430 
431       public HTableWrapper(byte[] tableName) throws IOException {
432         this.tableName = tableName;
433         this.table = new HTable(conf, tableName);
434         openTables.add(this);
435       }
436 
437       void internalClose() throws IOException {
438         table.close();
439       }
440 
441       public Configuration getConfiguration() {
442         return table.getConfiguration();
443       }
444 
445       public void close() throws IOException {
446         try {
447           internalClose();
448         } finally {
449           openTables.remove(this);
450         }
451       }
452 
453       public Result getRowOrBefore(byte[] row, byte[] family)
454           throws IOException {
455         return table.getRowOrBefore(row, family);
456       }
457 
458       public Result get(Get get) throws IOException {
459         return table.get(get);
460       }
461 
462       public boolean exists(Get get) throws IOException {
463         return table.exists(get);
464       }
465 
466       public void put(Put put) throws IOException {
467         table.put(put);
468       }
469 
470       public void put(List<Put> puts) throws IOException {
471         table.put(puts);
472       }
473 
474       public void delete(Delete delete) throws IOException {
475         table.delete(delete);
476       }
477 
478       public void delete(List<Delete> deletes) throws IOException {
479         table.delete(deletes);
480       }
481 
482       public boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier,
483           byte[] value, Put put) throws IOException {
484         return table.checkAndPut(row, family, qualifier, value, put);
485       }
486 
487       public boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
488           byte[] value, Delete delete) throws IOException {
489         return table.checkAndDelete(row, family, qualifier, value, delete);
490       }
491 
492       public long incrementColumnValue(byte[] row, byte[] family,
493           byte[] qualifier, long amount) throws IOException {
494         return table.incrementColumnValue(row, family, qualifier, amount);
495       }
496 
497       public long incrementColumnValue(byte[] row, byte[] family,
498           byte[] qualifier, long amount, boolean writeToWAL)
499           throws IOException {
500         return table.incrementColumnValue(row, family, qualifier, amount,
501           writeToWAL);
502       }
503 
504       @Override
505       public Result append(Append append) throws IOException {
506         return table.append(append);
507       }
508 
509       @Override
510       public Result increment(Increment increment) throws IOException {
511         return table.increment(increment);
512       }
513 
514       public void flushCommits() throws IOException {
515         table.flushCommits();
516       }
517 
518       public boolean isAutoFlush() {
519         return table.isAutoFlush();
520       }
521 
522       public ResultScanner getScanner(Scan scan) throws IOException {
523         return table.getScanner(scan);
524       }
525 
526       public ResultScanner getScanner(byte[] family) throws IOException {
527         return table.getScanner(family);
528       }
529 
530       public ResultScanner getScanner(byte[] family, byte[] qualifier)
531           throws IOException {
532         return table.getScanner(family, qualifier);
533       }
534 
535       public HTableDescriptor getTableDescriptor() throws IOException {
536         return table.getTableDescriptor();
537       }
538 
539       public byte[] getTableName() {
540         return tableName;
541       }
542 
543       /**
544        * @deprecated {@link RowLock} and associated operations are deprecated.
545        */
546       public RowLock lockRow(byte[] row) throws IOException {
547         throw new RuntimeException(
548           "row locking is not allowed within the coprocessor environment");
549       }
550 
551       /**
552        * @deprecated {@link RowLock} and associated operations are deprecated.
553        */
554       public void unlockRow(RowLock rl) throws IOException {
555         throw new RuntimeException(
556           "row locking is not allowed within the coprocessor environment");
557       }
558 
559       @Override
560       public void batch(List<? extends Row> actions, Object[] results)
561           throws IOException, InterruptedException {
562         table.batch(actions, results);
563       }
564 
565       @Override
566       public Object[] batch(List<? extends Row> actions)
567           throws IOException, InterruptedException {
568         return table.batch(actions);
569       }
570 
571       @Override
572       public Result[] get(List<Get> gets) throws IOException {
573         return table.get(gets);
574       }
575 
576       @Override
577       public <T extends CoprocessorProtocol, R> void coprocessorExec(Class<T> protocol,
578           byte[] startKey, byte[] endKey, Batch.Call<T, R> callable,
579           Batch.Callback<R> callback) throws IOException, Throwable {
580         table.coprocessorExec(protocol, startKey, endKey, callable, callback);
581       }
582 
583       @Override
584       public <T extends CoprocessorProtocol, R> Map<byte[], R> coprocessorExec(
585           Class<T> protocol, byte[] startKey, byte[] endKey, Batch.Call<T, R> callable)
586           throws IOException, Throwable {
587         return table.coprocessorExec(protocol, startKey, endKey, callable);
588       }
589 
590       @Override
591       public <T extends CoprocessorProtocol> T coprocessorProxy(Class<T> protocol,
592           byte[] row) {
593         return table.coprocessorProxy(protocol, row);
594       }
595 
596       @Override
597       public void mutateRow(RowMutations rm) throws IOException {
598         table.mutateRow(rm);
599       }
600 
601       @Override
602       public void setAutoFlush(boolean autoFlush) {
603         table.setAutoFlush(autoFlush);
604       }
605 
606       @Override
607       public void setAutoFlush(boolean autoFlush, boolean clearBufferOnFail) {
608         table.setAutoFlush(autoFlush, clearBufferOnFail);
609       }
610 
611       @Override
612       public long getWriteBufferSize() {
613          return table.getWriteBufferSize();
614       }
615 
616       @Override
617       public void setWriteBufferSize(long writeBufferSize) throws IOException {
618         table.setWriteBufferSize(writeBufferSize);
619       }
620     }
621 
622     /** The coprocessor */
623     public Coprocessor impl;
624     /** Chaining priority */
625     protected int priority = Coprocessor.PRIORITY_USER;
626     /** Current coprocessor state */
627     Coprocessor.State state = Coprocessor.State.UNINSTALLED;
628     /** Accounting for tables opened by the coprocessor */
629     protected List<HTableInterface> openTables =
630       Collections.synchronizedList(new ArrayList<HTableInterface>());
631     private int seq;
632     private Configuration conf;
633 
634     /**
635      * Constructor
636      * @param impl the coprocessor instance
637      * @param priority chaining priority
638      */
639     public Environment(final Coprocessor impl, final int priority,
640         final int seq, final Configuration conf) {
641       this.impl = impl;
642       this.priority = priority;
643       this.state = Coprocessor.State.INSTALLED;
644       this.seq = seq;
645       this.conf = conf;
646     }
647 
648     /** Initialize the environment */
649     public void startup() {
650       if (state == Coprocessor.State.INSTALLED ||
651           state == Coprocessor.State.STOPPED) {
652         state = Coprocessor.State.STARTING;
653         try {
654           impl.start(this);
655           state = Coprocessor.State.ACTIVE;
656         } catch (IOException ioe) {
657           LOG.error("Error starting coprocessor "+impl.getClass().getName(), ioe);
658         }
659       } else {
660         LOG.warn("Not starting coprocessor "+impl.getClass().getName()+
661             " because not inactive (state="+state.toString()+")");
662       }
663     }
664 
665     /** Clean up the environment */
666     protected void shutdown() {
667       if (state == Coprocessor.State.ACTIVE) {
668         state = Coprocessor.State.STOPPING;
669         try {
670           impl.stop(this);
671           state = Coprocessor.State.STOPPED;
672         } catch (IOException ioe) {
673           LOG.error("Error stopping coprocessor "+impl.getClass().getName(), ioe);
674         }
675       } else {
676         LOG.warn("Not stopping coprocessor "+impl.getClass().getName()+
677             " because not active (state="+state.toString()+")");
678       }
679       // clean up any table references
680       for (HTableInterface table: openTables) {
681         try {
682           ((HTableWrapper)table).internalClose();
683         } catch (IOException e) {
684           // nothing can be done here
685           LOG.warn("Failed to close " +
686               Bytes.toStringBinary(table.getTableName()), e);
687         }
688       }
689     }
690 
691     @Override
692     public Coprocessor getInstance() {
693       return impl;
694     }
695 
696     @Override
697     public int getPriority() {
698       return priority;
699     }
700 
701     @Override
702     public int getLoadSequence() {
703       return seq;
704     }
705 
706     /** @return the coprocessor environment version */
707     @Override
708     public int getVersion() {
709       return Coprocessor.VERSION;
710     }
711 
712     /** @return the HBase release */
713     @Override
714     public String getHBaseVersion() {
715       return VersionInfo.getVersion();
716     }
717 
718     @Override
719     public Configuration getConfiguration() {
720       return conf;
721     }
722 
723     /**
724      * Open a table from within the Coprocessor environment
725      * @param tableName the table name
726      * @return an interface for manipulating the table
727      * @exception java.io.IOException Exception
728      */
729     @Override
730     public HTableInterface getTable(byte[] tableName) throws IOException {
731       return new HTableWrapper(tableName);
732     }
733   }
734 
735   protected void abortServer(final String service,
736       final Server server,
737       final CoprocessorEnvironment environment,
738       final Throwable e) {
739     String coprocessorName = (environment.getInstance()).toString();
740     server.abort("Aborting service: " + service + " running on : "
741             + server.getServerName() + " because coprocessor: "
742             + coprocessorName + " threw an exception.", e);
743   }
744 
745   protected void abortServer(final CoprocessorEnvironment environment,
746                              final Throwable e) {
747     String coprocessorName = (environment.getInstance()).toString();
748     LOG.error("The coprocessor: " + coprocessorName + " threw an unexpected " +
749         "exception: " + e + ", but there's no specific implementation of " +
750         " abortServer() for this coprocessor's environment.");
751   }
752 
753 
754   /**
755    * This is used by coprocessor hooks which are declared to throw IOException
756    * (or its subtypes). For such hooks, we should handle throwable objects
757    * depending on the Throwable's type. Those which are instances of
758    * IOException should be passed on to the client. This is in conformance with
759    * the HBase idiom regarding IOException: that it represents a circumstance
760    * that should be passed along to the client for its own handling. For
761    * example, a coprocessor that implements access controls would throw a
762    * subclass of IOException, such as AccessDeniedException, in its preGet()
763    * method to prevent an unauthorized client's performing a Get on a particular
764    * table.
765    * @param env Coprocessor Environment
766    * @param e Throwable object thrown by coprocessor.
767    * @exception IOException Exception
768    */
769   protected void handleCoprocessorThrowable(final CoprocessorEnvironment env,
770                                             final Throwable e)
771       throws IOException {
772     if (e instanceof IOException) {
773       throw (IOException)e;
774     }
775     // If we got here, e is not an IOException. A loaded coprocessor has a
776     // fatal bug, and the server (master or regionserver) should remove the
777     // faulty coprocessor from its set of active coprocessors. Setting
778     // 'hbase.coprocessor.abortonerror' to true will cause abortServer(),
779     // which may be useful in development and testing environments where
780     // 'failing fast' for error analysis is desired.
781     if (env.getConfiguration().getBoolean("hbase.coprocessor.abortonerror",false)) {
782       // server is configured to abort.
783       abortServer(env, e);
784     } else {
785       LOG.error("Removing coprocessor '" + env.toString() + "' from " +
786           "environment because it threw:  " + e,e);
787       coprocessors.remove(env);
788       throw new DoNotRetryIOException("Coprocessor: '" + env.toString() +
789           "' threw: '" + e + "' and has been removed" + "from the active " +
790           "coprocessor set.", e);
791     }
792   }
793 }
794 
795