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