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