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