View Javadoc

1   
2   /*
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.regionserver.wal;
22  
23  import java.io.IOException;
24  
25  import org.apache.hadoop.hbase.Coprocessor;
26  import org.apache.hadoop.hbase.HRegionInfo;
27  import org.apache.hadoop.hbase.coprocessor.*;
28  import org.apache.hadoop.classification.InterfaceAudience;
29  import org.apache.hadoop.conf.Configuration;
30  
31  /**
32   * Implements the coprocessor environment and runtime support for coprocessors
33   * loaded within a {@link FSHLog}.
34   */
35  @InterfaceAudience.Private
36  public class WALCoprocessorHost
37      extends CoprocessorHost<WALCoprocessorHost.WALEnvironment> {
38    
39    /**
40     * Encapsulation of the environment of each coprocessor
41     */
42    static class WALEnvironment extends CoprocessorHost.Environment
43      implements WALCoprocessorEnvironment {
44  
45      private FSHLog wal;
46  
47      @Override
48      public FSHLog getWAL() {
49        return wal;
50      }
51  
52      /**
53       * Constructor
54       * @param implClass - not used
55       * @param impl the coprocessor instance
56       * @param priority chaining priority
57       * @param seq load sequence
58       * @param conf configuration
59       * @param hlog HLog
60       */
61      public WALEnvironment(Class<?> implClass, final Coprocessor impl,
62          final int priority, final int seq, final Configuration conf,
63          final FSHLog hlog) {
64        super(impl, priority, seq, conf);
65        this.wal = hlog;
66      }
67    }
68  
69    FSHLog wal;
70    /**
71     * Constructor
72     * @param log the write ahead log
73     * @param conf the configuration
74     */
75    public WALCoprocessorHost(final FSHLog log, final Configuration conf) {
76      // We don't want to require an Abortable passed down through (FS)HLog, so
77      // this means that a failure to load of a WAL coprocessor won't abort the
78      // server. This isn't ideal, and means that security components that
79      // utilize a WALObserver will have to check the observer initialization
80      // state manually. However, WALObservers will eventually go away so it
81      // should be an acceptable state of affairs.
82      super(null);
83      this.wal = log;
84      // load system default cp's from configuration.
85      loadSystemCoprocessors(conf, WAL_COPROCESSOR_CONF_KEY);
86    }
87  
88    @Override
89    public WALEnvironment createEnvironment(final Class<?> implClass,
90        final Coprocessor instance, final int priority, final int seq,
91        final Configuration conf) {
92      return new WALEnvironment(implClass, instance, priority, seq, conf,
93          this.wal);
94    }
95  
96    /**
97     * @param info
98     * @param logKey
99     * @param logEdit
100    * @return true if default behavior should be bypassed, false otherwise
101    * @throws IOException
102    */
103   public boolean preWALWrite(final HRegionInfo info, final HLogKey logKey, final WALEdit logEdit)
104       throws IOException {
105     boolean bypass = false;
106     if (this.coprocessors == null || this.coprocessors.isEmpty()) return bypass;
107     ObserverContext<WALCoprocessorEnvironment> ctx = null;
108     for (WALEnvironment env: coprocessors) {
109       if (env.getInstance() instanceof
110           org.apache.hadoop.hbase.coprocessor.WALObserver) {
111         ctx = ObserverContext.createAndPrepare(env, ctx);
112         Thread currentThread = Thread.currentThread();
113         ClassLoader cl = currentThread.getContextClassLoader();
114         try {
115           currentThread.setContextClassLoader(env.getClassLoader());
116           ((org.apache.hadoop.hbase.coprocessor.WALObserver)env.getInstance()).
117             preWALWrite(ctx, info, logKey, logEdit);
118         } catch (Throwable e) {
119           handleCoprocessorThrowable(env, e);
120         } finally {
121           currentThread.setContextClassLoader(cl);
122         }
123         bypass |= ctx.shouldBypass();
124         if (ctx.shouldComplete()) {
125           break;
126         }
127       }
128     }
129     return bypass;
130   }
131 
132   /**
133    * @param info
134    * @param logKey
135    * @param logEdit
136    * @throws IOException
137    */
138   public void postWALWrite(final HRegionInfo info, final HLogKey logKey, final WALEdit logEdit)
139       throws IOException {
140     if (this.coprocessors == null || this.coprocessors.isEmpty()) return;
141     ObserverContext<WALCoprocessorEnvironment> ctx = null;
142     for (WALEnvironment env: coprocessors) {
143       if (env.getInstance() instanceof
144           org.apache.hadoop.hbase.coprocessor.WALObserver) {
145         ctx = ObserverContext.createAndPrepare(env, ctx);
146         Thread currentThread = Thread.currentThread();
147         ClassLoader cl = currentThread.getContextClassLoader();
148         try {
149           currentThread.setContextClassLoader(env.getClassLoader());
150           ((org.apache.hadoop.hbase.coprocessor.WALObserver)env.getInstance()).
151             postWALWrite(ctx, info, logKey, logEdit);
152         } catch (Throwable e) {
153           handleCoprocessorThrowable(env, e);
154         } finally {
155           currentThread.setContextClassLoader(cl);
156         }
157         if (ctx.shouldComplete()) {
158           break;
159         }
160       }
161     }
162   }
163 }