blob: 421851b56ad216614b2e99d3679ad8805009c2ae [file] [log] [blame]
<!DOCTYPE HTML>
<html lang="en">
<head>
<!-- Generated by javadoc (17) -->
<title>Source code</title>
<meta name="viewport" content="width=device-width, initial-scale=1">
<meta name="description" content="source: package: org.apache.hadoop.hbase.regionserver, class: RegionCoprocessorHost, class: RegionEnvironment">
<meta name="generator" content="javadoc/SourceToHTMLConverter">
<link rel="stylesheet" type="text/css" href="../../../../../../stylesheet.css" title="Style">
</head>
<body class="source-page">
<main role="main">
<div class="source-container">
<pre><span class="source-line-no">001</span><span id="line-1">/*</span>
<span class="source-line-no">002</span><span id="line-2"> * Licensed to the Apache Software Foundation (ASF) under one</span>
<span class="source-line-no">003</span><span id="line-3"> * or more contributor license agreements. See the NOTICE file</span>
<span class="source-line-no">004</span><span id="line-4"> * distributed with this work for additional information</span>
<span class="source-line-no">005</span><span id="line-5"> * regarding copyright ownership. The ASF licenses this file</span>
<span class="source-line-no">006</span><span id="line-6"> * to you under the Apache License, Version 2.0 (the</span>
<span class="source-line-no">007</span><span id="line-7"> * "License"); you may not use this file except in compliance</span>
<span class="source-line-no">008</span><span id="line-8"> * with the License. You may obtain a copy of the License at</span>
<span class="source-line-no">009</span><span id="line-9"> *</span>
<span class="source-line-no">010</span><span id="line-10"> * http://www.apache.org/licenses/LICENSE-2.0</span>
<span class="source-line-no">011</span><span id="line-11"> *</span>
<span class="source-line-no">012</span><span id="line-12"> * Unless required by applicable law or agreed to in writing, software</span>
<span class="source-line-no">013</span><span id="line-13"> * distributed under the License is distributed on an "AS IS" BASIS,</span>
<span class="source-line-no">014</span><span id="line-14"> * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.</span>
<span class="source-line-no">015</span><span id="line-15"> * See the License for the specific language governing permissions and</span>
<span class="source-line-no">016</span><span id="line-16"> * limitations under the License.</span>
<span class="source-line-no">017</span><span id="line-17"> */</span>
<span class="source-line-no">018</span><span id="line-18">package org.apache.hadoop.hbase.regionserver;</span>
<span class="source-line-no">019</span><span id="line-19"></span>
<span class="source-line-no">020</span><span id="line-20">import java.io.IOException;</span>
<span class="source-line-no">021</span><span id="line-21">import java.lang.reflect.InvocationTargetException;</span>
<span class="source-line-no">022</span><span id="line-22">import java.util.ArrayList;</span>
<span class="source-line-no">023</span><span id="line-23">import java.util.List;</span>
<span class="source-line-no">024</span><span id="line-24">import java.util.Map;</span>
<span class="source-line-no">025</span><span id="line-25">import java.util.UUID;</span>
<span class="source-line-no">026</span><span id="line-26">import java.util.concurrent.ConcurrentHashMap;</span>
<span class="source-line-no">027</span><span id="line-27">import java.util.concurrent.ConcurrentMap;</span>
<span class="source-line-no">028</span><span id="line-28">import java.util.stream.Collectors;</span>
<span class="source-line-no">029</span><span id="line-29">import org.apache.hadoop.conf.Configuration;</span>
<span class="source-line-no">030</span><span id="line-30">import org.apache.hadoop.fs.FileSystem;</span>
<span class="source-line-no">031</span><span id="line-31">import org.apache.hadoop.fs.Path;</span>
<span class="source-line-no">032</span><span id="line-32">import org.apache.hadoop.hbase.Cell;</span>
<span class="source-line-no">033</span><span id="line-33">import org.apache.hadoop.hbase.HBaseConfiguration;</span>
<span class="source-line-no">034</span><span id="line-34">import org.apache.hadoop.hbase.HConstants;</span>
<span class="source-line-no">035</span><span id="line-35">import org.apache.hadoop.hbase.RawCellBuilder;</span>
<span class="source-line-no">036</span><span id="line-36">import org.apache.hadoop.hbase.RawCellBuilderFactory;</span>
<span class="source-line-no">037</span><span id="line-37">import org.apache.hadoop.hbase.ServerName;</span>
<span class="source-line-no">038</span><span id="line-38">import org.apache.hadoop.hbase.client.Append;</span>
<span class="source-line-no">039</span><span id="line-39">import org.apache.hadoop.hbase.client.CheckAndMutate;</span>
<span class="source-line-no">040</span><span id="line-40">import org.apache.hadoop.hbase.client.CheckAndMutateResult;</span>
<span class="source-line-no">041</span><span id="line-41">import org.apache.hadoop.hbase.client.Connection;</span>
<span class="source-line-no">042</span><span id="line-42">import org.apache.hadoop.hbase.client.Delete;</span>
<span class="source-line-no">043</span><span id="line-43">import org.apache.hadoop.hbase.client.Get;</span>
<span class="source-line-no">044</span><span id="line-44">import org.apache.hadoop.hbase.client.Increment;</span>
<span class="source-line-no">045</span><span id="line-45">import org.apache.hadoop.hbase.client.Mutation;</span>
<span class="source-line-no">046</span><span id="line-46">import org.apache.hadoop.hbase.client.Put;</span>
<span class="source-line-no">047</span><span id="line-47">import org.apache.hadoop.hbase.client.RegionInfo;</span>
<span class="source-line-no">048</span><span id="line-48">import org.apache.hadoop.hbase.client.Result;</span>
<span class="source-line-no">049</span><span id="line-49">import org.apache.hadoop.hbase.client.Scan;</span>
<span class="source-line-no">050</span><span id="line-50">import org.apache.hadoop.hbase.client.SharedConnection;</span>
<span class="source-line-no">051</span><span id="line-51">import org.apache.hadoop.hbase.client.TableDescriptor;</span>
<span class="source-line-no">052</span><span id="line-52">import org.apache.hadoop.hbase.coprocessor.BaseEnvironment;</span>
<span class="source-line-no">053</span><span id="line-53">import org.apache.hadoop.hbase.coprocessor.BulkLoadObserver;</span>
<span class="source-line-no">054</span><span id="line-54">import org.apache.hadoop.hbase.coprocessor.CoprocessorException;</span>
<span class="source-line-no">055</span><span id="line-55">import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;</span>
<span class="source-line-no">056</span><span id="line-56">import org.apache.hadoop.hbase.coprocessor.CoreCoprocessor;</span>
<span class="source-line-no">057</span><span id="line-57">import org.apache.hadoop.hbase.coprocessor.EndpointObserver;</span>
<span class="source-line-no">058</span><span id="line-58">import org.apache.hadoop.hbase.coprocessor.HasRegionServerServices;</span>
<span class="source-line-no">059</span><span id="line-59">import org.apache.hadoop.hbase.coprocessor.MetricsCoprocessor;</span>
<span class="source-line-no">060</span><span id="line-60">import org.apache.hadoop.hbase.coprocessor.ObserverContext;</span>
<span class="source-line-no">061</span><span id="line-61">import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;</span>
<span class="source-line-no">062</span><span id="line-62">import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;</span>
<span class="source-line-no">063</span><span id="line-63">import org.apache.hadoop.hbase.coprocessor.RegionObserver;</span>
<span class="source-line-no">064</span><span id="line-64">import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;</span>
<span class="source-line-no">065</span><span id="line-65">import org.apache.hadoop.hbase.io.Reference;</span>
<span class="source-line-no">066</span><span id="line-66">import org.apache.hadoop.hbase.io.hfile.CacheConfig;</span>
<span class="source-line-no">067</span><span id="line-67">import org.apache.hadoop.hbase.metrics.MetricRegistry;</span>
<span class="source-line-no">068</span><span id="line-68">import org.apache.hadoop.hbase.quotas.OperationQuota;</span>
<span class="source-line-no">069</span><span id="line-69">import org.apache.hadoop.hbase.quotas.RpcQuotaManager;</span>
<span class="source-line-no">070</span><span id="line-70">import org.apache.hadoop.hbase.quotas.RpcThrottlingException;</span>
<span class="source-line-no">071</span><span id="line-71">import org.apache.hadoop.hbase.regionserver.Region.Operation;</span>
<span class="source-line-no">072</span><span id="line-72">import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;</span>
<span class="source-line-no">073</span><span id="line-73">import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;</span>
<span class="source-line-no">074</span><span id="line-74">import org.apache.hadoop.hbase.regionserver.querymatcher.DeleteTracker;</span>
<span class="source-line-no">075</span><span id="line-75">import org.apache.hadoop.hbase.security.User;</span>
<span class="source-line-no">076</span><span id="line-76">import org.apache.hadoop.hbase.util.CoprocessorClassLoader;</span>
<span class="source-line-no">077</span><span id="line-77">import org.apache.hadoop.hbase.util.Pair;</span>
<span class="source-line-no">078</span><span id="line-78">import org.apache.hadoop.hbase.wal.WALEdit;</span>
<span class="source-line-no">079</span><span id="line-79">import org.apache.hadoop.hbase.wal.WALKey;</span>
<span class="source-line-no">080</span><span id="line-80">import org.apache.yetus.audience.InterfaceAudience;</span>
<span class="source-line-no">081</span><span id="line-81">import org.slf4j.Logger;</span>
<span class="source-line-no">082</span><span id="line-82">import org.slf4j.LoggerFactory;</span>
<span class="source-line-no">083</span><span id="line-83"></span>
<span class="source-line-no">084</span><span id="line-84">import org.apache.hbase.thirdparty.com.google.protobuf.Message;</span>
<span class="source-line-no">085</span><span id="line-85">import org.apache.hbase.thirdparty.com.google.protobuf.Service;</span>
<span class="source-line-no">086</span><span id="line-86">import org.apache.hbase.thirdparty.org.apache.commons.collections4.map.AbstractReferenceMap;</span>
<span class="source-line-no">087</span><span id="line-87">import org.apache.hbase.thirdparty.org.apache.commons.collections4.map.ReferenceMap;</span>
<span class="source-line-no">088</span><span id="line-88"></span>
<span class="source-line-no">089</span><span id="line-89">import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;</span>
<span class="source-line-no">090</span><span id="line-90">import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;</span>
<span class="source-line-no">091</span><span id="line-91"></span>
<span class="source-line-no">092</span><span id="line-92">/**</span>
<span class="source-line-no">093</span><span id="line-93"> * Implements the coprocessor environment and runtime support for coprocessors loaded within a</span>
<span class="source-line-no">094</span><span id="line-94"> * {@link Region}.</span>
<span class="source-line-no">095</span><span id="line-95"> */</span>
<span class="source-line-no">096</span><span id="line-96">@InterfaceAudience.Private</span>
<span class="source-line-no">097</span><span id="line-97">public class RegionCoprocessorHost</span>
<span class="source-line-no">098</span><span id="line-98"> extends CoprocessorHost&lt;RegionCoprocessor, RegionCoprocessorEnvironment&gt; {</span>
<span class="source-line-no">099</span><span id="line-99"></span>
<span class="source-line-no">100</span><span id="line-100"> private static final Logger LOG = LoggerFactory.getLogger(RegionCoprocessorHost.class);</span>
<span class="source-line-no">101</span><span id="line-101"> // The shared data map</span>
<span class="source-line-no">102</span><span id="line-102"> private static final ReferenceMap&lt;String, ConcurrentMap&lt;String, Object&gt;&gt; SHARED_DATA_MAP =</span>
<span class="source-line-no">103</span><span id="line-103"> new ReferenceMap&lt;&gt;(AbstractReferenceMap.ReferenceStrength.HARD,</span>
<span class="source-line-no">104</span><span id="line-104"> AbstractReferenceMap.ReferenceStrength.WEAK);</span>
<span class="source-line-no">105</span><span id="line-105"></span>
<span class="source-line-no">106</span><span id="line-106"> // optimization: no need to call postScannerFilterRow, if no coprocessor implements it</span>
<span class="source-line-no">107</span><span id="line-107"> private final boolean hasCustomPostScannerFilterRow;</span>
<span class="source-line-no">108</span><span id="line-108"></span>
<span class="source-line-no">109</span><span id="line-109"> /*</span>
<span class="source-line-no">110</span><span id="line-110"> * Whether any configured CPs override postScannerFilterRow hook</span>
<span class="source-line-no">111</span><span id="line-111"> */</span>
<span class="source-line-no">112</span><span id="line-112"> public boolean hasCustomPostScannerFilterRow() {</span>
<span class="source-line-no">113</span><span id="line-113"> return hasCustomPostScannerFilterRow;</span>
<span class="source-line-no">114</span><span id="line-114"> }</span>
<span class="source-line-no">115</span><span id="line-115"></span>
<span class="source-line-no">116</span><span id="line-116"> /**</span>
<span class="source-line-no">117</span><span id="line-117"> * Encapsulation of the environment of each coprocessor</span>
<span class="source-line-no">118</span><span id="line-118"> */</span>
<span class="source-line-no">119</span><span id="line-119"> private static class RegionEnvironment extends BaseEnvironment&lt;RegionCoprocessor&gt;</span>
<span class="source-line-no">120</span><span id="line-120"> implements RegionCoprocessorEnvironment {</span>
<span class="source-line-no">121</span><span id="line-121"> private Region region;</span>
<span class="source-line-no">122</span><span id="line-122"> ConcurrentMap&lt;String, Object&gt; sharedData;</span>
<span class="source-line-no">123</span><span id="line-123"> private final MetricRegistry metricRegistry;</span>
<span class="source-line-no">124</span><span id="line-124"> private final RegionServerServices services;</span>
<span class="source-line-no">125</span><span id="line-125"> private final RpcQuotaManager rpcQuotaManager;</span>
<span class="source-line-no">126</span><span id="line-126"></span>
<span class="source-line-no">127</span><span id="line-127"> /**</span>
<span class="source-line-no">128</span><span id="line-128"> * Constructor</span>
<span class="source-line-no">129</span><span id="line-129"> * @param impl the coprocessor instance</span>
<span class="source-line-no">130</span><span id="line-130"> * @param priority chaining priority</span>
<span class="source-line-no">131</span><span id="line-131"> */</span>
<span class="source-line-no">132</span><span id="line-132"> public RegionEnvironment(final RegionCoprocessor impl, final int priority, final int seq,</span>
<span class="source-line-no">133</span><span id="line-133"> final Configuration conf, final Region region, final RegionServerServices services,</span>
<span class="source-line-no">134</span><span id="line-134"> final ConcurrentMap&lt;String, Object&gt; sharedData) {</span>
<span class="source-line-no">135</span><span id="line-135"> super(impl, priority, seq, conf);</span>
<span class="source-line-no">136</span><span id="line-136"> this.region = region;</span>
<span class="source-line-no">137</span><span id="line-137"> this.sharedData = sharedData;</span>
<span class="source-line-no">138</span><span id="line-138"> this.services = services;</span>
<span class="source-line-no">139</span><span id="line-139"> this.metricRegistry =</span>
<span class="source-line-no">140</span><span id="line-140"> MetricsCoprocessor.createRegistryForRegionCoprocessor(impl.getClass().getName());</span>
<span class="source-line-no">141</span><span id="line-141"> // Some unit tests reach this line with services == null, and are okay with rpcQuotaManager</span>
<span class="source-line-no">142</span><span id="line-142"> // being null. Let these unit tests succeed. This should not happen in real usage.</span>
<span class="source-line-no">143</span><span id="line-143"> if (services != null) {</span>
<span class="source-line-no">144</span><span id="line-144"> this.rpcQuotaManager = services.getRegionServerRpcQuotaManager();</span>
<span class="source-line-no">145</span><span id="line-145"> } else {</span>
<span class="source-line-no">146</span><span id="line-146"> this.rpcQuotaManager = null;</span>
<span class="source-line-no">147</span><span id="line-147"> }</span>
<span class="source-line-no">148</span><span id="line-148"> }</span>
<span class="source-line-no">149</span><span id="line-149"></span>
<span class="source-line-no">150</span><span id="line-150"> /** Returns the region */</span>
<span class="source-line-no">151</span><span id="line-151"> @Override</span>
<span class="source-line-no">152</span><span id="line-152"> public Region getRegion() {</span>
<span class="source-line-no">153</span><span id="line-153"> return region;</span>
<span class="source-line-no">154</span><span id="line-154"> }</span>
<span class="source-line-no">155</span><span id="line-155"></span>
<span class="source-line-no">156</span><span id="line-156"> @Override</span>
<span class="source-line-no">157</span><span id="line-157"> public OnlineRegions getOnlineRegions() {</span>
<span class="source-line-no">158</span><span id="line-158"> return this.services;</span>
<span class="source-line-no">159</span><span id="line-159"> }</span>
<span class="source-line-no">160</span><span id="line-160"></span>
<span class="source-line-no">161</span><span id="line-161"> @Override</span>
<span class="source-line-no">162</span><span id="line-162"> public Connection getConnection() {</span>
<span class="source-line-no">163</span><span id="line-163"> // Mocks may have services as null at test time.</span>
<span class="source-line-no">164</span><span id="line-164"> return services != null ? new SharedConnection(services.getConnection()) : null;</span>
<span class="source-line-no">165</span><span id="line-165"> }</span>
<span class="source-line-no">166</span><span id="line-166"></span>
<span class="source-line-no">167</span><span id="line-167"> @Override</span>
<span class="source-line-no">168</span><span id="line-168"> public Connection createConnection(Configuration conf) throws IOException {</span>
<span class="source-line-no">169</span><span id="line-169"> return services != null ? this.services.createConnection(conf) : null;</span>
<span class="source-line-no">170</span><span id="line-170"> }</span>
<span class="source-line-no">171</span><span id="line-171"></span>
<span class="source-line-no">172</span><span id="line-172"> @Override</span>
<span class="source-line-no">173</span><span id="line-173"> public ServerName getServerName() {</span>
<span class="source-line-no">174</span><span id="line-174"> return services != null ? services.getServerName() : null;</span>
<span class="source-line-no">175</span><span id="line-175"> }</span>
<span class="source-line-no">176</span><span id="line-176"></span>
<span class="source-line-no">177</span><span id="line-177"> @Override</span>
<span class="source-line-no">178</span><span id="line-178"> public void shutdown() {</span>
<span class="source-line-no">179</span><span id="line-179"> super.shutdown();</span>
<span class="source-line-no">180</span><span id="line-180"> MetricsCoprocessor.removeRegistry(this.metricRegistry);</span>
<span class="source-line-no">181</span><span id="line-181"> }</span>
<span class="source-line-no">182</span><span id="line-182"></span>
<span class="source-line-no">183</span><span id="line-183"> @Override</span>
<span class="source-line-no">184</span><span id="line-184"> public ConcurrentMap&lt;String, Object&gt; getSharedData() {</span>
<span class="source-line-no">185</span><span id="line-185"> return sharedData;</span>
<span class="source-line-no">186</span><span id="line-186"> }</span>
<span class="source-line-no">187</span><span id="line-187"></span>
<span class="source-line-no">188</span><span id="line-188"> @Override</span>
<span class="source-line-no">189</span><span id="line-189"> public RegionInfo getRegionInfo() {</span>
<span class="source-line-no">190</span><span id="line-190"> return region.getRegionInfo();</span>
<span class="source-line-no">191</span><span id="line-191"> }</span>
<span class="source-line-no">192</span><span id="line-192"></span>
<span class="source-line-no">193</span><span id="line-193"> @Override</span>
<span class="source-line-no">194</span><span id="line-194"> public MetricRegistry getMetricRegistryForRegionServer() {</span>
<span class="source-line-no">195</span><span id="line-195"> return metricRegistry;</span>
<span class="source-line-no">196</span><span id="line-196"> }</span>
<span class="source-line-no">197</span><span id="line-197"></span>
<span class="source-line-no">198</span><span id="line-198"> @Override</span>
<span class="source-line-no">199</span><span id="line-199"> public RawCellBuilder getCellBuilder() {</span>
<span class="source-line-no">200</span><span id="line-200"> // We always do a DEEP_COPY only</span>
<span class="source-line-no">201</span><span id="line-201"> return RawCellBuilderFactory.create();</span>
<span class="source-line-no">202</span><span id="line-202"> }</span>
<span class="source-line-no">203</span><span id="line-203"></span>
<span class="source-line-no">204</span><span id="line-204"> @Override</span>
<span class="source-line-no">205</span><span id="line-205"> public RpcQuotaManager getRpcQuotaManager() {</span>
<span class="source-line-no">206</span><span id="line-206"> return rpcQuotaManager;</span>
<span class="source-line-no">207</span><span id="line-207"> }</span>
<span class="source-line-no">208</span><span id="line-208"></span>
<span class="source-line-no">209</span><span id="line-209"> @Override</span>
<span class="source-line-no">210</span><span id="line-210"> public OperationQuota checkScanQuota(Scan scan, long maxBlockBytesScanned,</span>
<span class="source-line-no">211</span><span id="line-211"> long prevBlockBytesScannedDifference) throws IOException, RpcThrottlingException {</span>
<span class="source-line-no">212</span><span id="line-212"> ClientProtos.ScanRequest scanRequest = RequestConverter</span>
<span class="source-line-no">213</span><span id="line-213"> .buildScanRequest(region.getRegionInfo().getRegionName(), scan, scan.getCaching(), false);</span>
<span class="source-line-no">214</span><span id="line-214"> long maxScannerResultSize =</span>
<span class="source-line-no">215</span><span id="line-215"> services.getConfiguration().getLong(HConstants.HBASE_SERVER_SCANNER_MAX_RESULT_SIZE_KEY,</span>
<span class="source-line-no">216</span><span id="line-216"> HConstants.DEFAULT_HBASE_SERVER_SCANNER_MAX_RESULT_SIZE);</span>
<span class="source-line-no">217</span><span id="line-217"> return rpcQuotaManager.checkScanQuota(region, scanRequest, maxScannerResultSize,</span>
<span class="source-line-no">218</span><span id="line-218"> maxBlockBytesScanned, prevBlockBytesScannedDifference);</span>
<span class="source-line-no">219</span><span id="line-219"> }</span>
<span class="source-line-no">220</span><span id="line-220"></span>
<span class="source-line-no">221</span><span id="line-221"> @Override</span>
<span class="source-line-no">222</span><span id="line-222"> public OperationQuota checkBatchQuota(Region region, OperationQuota.OperationType type)</span>
<span class="source-line-no">223</span><span id="line-223"> throws IOException, RpcThrottlingException {</span>
<span class="source-line-no">224</span><span id="line-224"> return rpcQuotaManager.checkBatchQuota(region, type);</span>
<span class="source-line-no">225</span><span id="line-225"> }</span>
<span class="source-line-no">226</span><span id="line-226"></span>
<span class="source-line-no">227</span><span id="line-227"> @Override</span>
<span class="source-line-no">228</span><span id="line-228"> public OperationQuota checkBatchQuota(final Region region, int numWrites, int numReads)</span>
<span class="source-line-no">229</span><span id="line-229"> throws IOException, RpcThrottlingException {</span>
<span class="source-line-no">230</span><span id="line-230"> return rpcQuotaManager.checkBatchQuota(region, numWrites, numReads);</span>
<span class="source-line-no">231</span><span id="line-231"> }</span>
<span class="source-line-no">232</span><span id="line-232"> }</span>
<span class="source-line-no">233</span><span id="line-233"></span>
<span class="source-line-no">234</span><span id="line-234"> /**</span>
<span class="source-line-no">235</span><span id="line-235"> * Special version of RegionEnvironment that exposes RegionServerServices for Core Coprocessors</span>
<span class="source-line-no">236</span><span id="line-236"> * only. Temporary hack until Core Coprocessors are integrated into Core.</span>
<span class="source-line-no">237</span><span id="line-237"> */</span>
<span class="source-line-no">238</span><span id="line-238"> private static class RegionEnvironmentForCoreCoprocessors extends RegionEnvironment</span>
<span class="source-line-no">239</span><span id="line-239"> implements HasRegionServerServices {</span>
<span class="source-line-no">240</span><span id="line-240"> private final RegionServerServices rsServices;</span>
<span class="source-line-no">241</span><span id="line-241"></span>
<span class="source-line-no">242</span><span id="line-242"> public RegionEnvironmentForCoreCoprocessors(final RegionCoprocessor impl, final int priority,</span>
<span class="source-line-no">243</span><span id="line-243"> final int seq, final Configuration conf, final Region region,</span>
<span class="source-line-no">244</span><span id="line-244"> final RegionServerServices services, final ConcurrentMap&lt;String, Object&gt; sharedData) {</span>
<span class="source-line-no">245</span><span id="line-245"> super(impl, priority, seq, conf, region, services, sharedData);</span>
<span class="source-line-no">246</span><span id="line-246"> this.rsServices = services;</span>
<span class="source-line-no">247</span><span id="line-247"> }</span>
<span class="source-line-no">248</span><span id="line-248"></span>
<span class="source-line-no">249</span><span id="line-249"> /**</span>
<span class="source-line-no">250</span><span id="line-250"> * @return An instance of RegionServerServices, an object NOT for general user-space Coprocessor</span>
<span class="source-line-no">251</span><span id="line-251"> * consumption.</span>
<span class="source-line-no">252</span><span id="line-252"> */</span>
<span class="source-line-no">253</span><span id="line-253"> @Override</span>
<span class="source-line-no">254</span><span id="line-254"> public RegionServerServices getRegionServerServices() {</span>
<span class="source-line-no">255</span><span id="line-255"> return this.rsServices;</span>
<span class="source-line-no">256</span><span id="line-256"> }</span>
<span class="source-line-no">257</span><span id="line-257"> }</span>
<span class="source-line-no">258</span><span id="line-258"></span>
<span class="source-line-no">259</span><span id="line-259"> static class TableCoprocessorAttribute {</span>
<span class="source-line-no">260</span><span id="line-260"> private Path path;</span>
<span class="source-line-no">261</span><span id="line-261"> private String className;</span>
<span class="source-line-no">262</span><span id="line-262"> private int priority;</span>
<span class="source-line-no">263</span><span id="line-263"> private Configuration conf;</span>
<span class="source-line-no">264</span><span id="line-264"></span>
<span class="source-line-no">265</span><span id="line-265"> public TableCoprocessorAttribute(Path path, String className, int priority,</span>
<span class="source-line-no">266</span><span id="line-266"> Configuration conf) {</span>
<span class="source-line-no">267</span><span id="line-267"> this.path = path;</span>
<span class="source-line-no">268</span><span id="line-268"> this.className = className;</span>
<span class="source-line-no">269</span><span id="line-269"> this.priority = priority;</span>
<span class="source-line-no">270</span><span id="line-270"> this.conf = conf;</span>
<span class="source-line-no">271</span><span id="line-271"> }</span>
<span class="source-line-no">272</span><span id="line-272"></span>
<span class="source-line-no">273</span><span id="line-273"> public Path getPath() {</span>
<span class="source-line-no">274</span><span id="line-274"> return path;</span>
<span class="source-line-no">275</span><span id="line-275"> }</span>
<span class="source-line-no">276</span><span id="line-276"></span>
<span class="source-line-no">277</span><span id="line-277"> public String getClassName() {</span>
<span class="source-line-no">278</span><span id="line-278"> return className;</span>
<span class="source-line-no">279</span><span id="line-279"> }</span>
<span class="source-line-no">280</span><span id="line-280"></span>
<span class="source-line-no">281</span><span id="line-281"> public int getPriority() {</span>
<span class="source-line-no">282</span><span id="line-282"> return priority;</span>
<span class="source-line-no">283</span><span id="line-283"> }</span>
<span class="source-line-no">284</span><span id="line-284"></span>
<span class="source-line-no">285</span><span id="line-285"> public Configuration getConf() {</span>
<span class="source-line-no">286</span><span id="line-286"> return conf;</span>
<span class="source-line-no">287</span><span id="line-287"> }</span>
<span class="source-line-no">288</span><span id="line-288"> }</span>
<span class="source-line-no">289</span><span id="line-289"></span>
<span class="source-line-no">290</span><span id="line-290"> /** The region server services */</span>
<span class="source-line-no">291</span><span id="line-291"> RegionServerServices rsServices;</span>
<span class="source-line-no">292</span><span id="line-292"> /** The region */</span>
<span class="source-line-no">293</span><span id="line-293"> HRegion region;</span>
<span class="source-line-no">294</span><span id="line-294"></span>
<span class="source-line-no">295</span><span id="line-295"> /**</span>
<span class="source-line-no">296</span><span id="line-296"> * Constructor</span>
<span class="source-line-no">297</span><span id="line-297"> * @param region the region</span>
<span class="source-line-no">298</span><span id="line-298"> * @param rsServices interface to available region server functionality</span>
<span class="source-line-no">299</span><span id="line-299"> * @param conf the configuration</span>
<span class="source-line-no">300</span><span id="line-300"> */</span>
<span class="source-line-no">301</span><span id="line-301"> @SuppressWarnings("ReturnValueIgnored") // Checking method exists as CPU optimization</span>
<span class="source-line-no">302</span><span id="line-302"> public RegionCoprocessorHost(final HRegion region, final RegionServerServices rsServices,</span>
<span class="source-line-no">303</span><span id="line-303"> final Configuration conf) {</span>
<span class="source-line-no">304</span><span id="line-304"> super(rsServices);</span>
<span class="source-line-no">305</span><span id="line-305"> this.conf = conf;</span>
<span class="source-line-no">306</span><span id="line-306"> this.rsServices = rsServices;</span>
<span class="source-line-no">307</span><span id="line-307"> this.region = region;</span>
<span class="source-line-no">308</span><span id="line-308"> this.pathPrefix = Integer.toString(this.region.getRegionInfo().hashCode());</span>
<span class="source-line-no">309</span><span id="line-309"></span>
<span class="source-line-no">310</span><span id="line-310"> // load system default cp's from configuration.</span>
<span class="source-line-no">311</span><span id="line-311"> loadSystemCoprocessors(conf, REGION_COPROCESSOR_CONF_KEY);</span>
<span class="source-line-no">312</span><span id="line-312"></span>
<span class="source-line-no">313</span><span id="line-313"> // load system default cp's for user tables from configuration.</span>
<span class="source-line-no">314</span><span id="line-314"> if (!region.getRegionInfo().getTable().isSystemTable()) {</span>
<span class="source-line-no">315</span><span id="line-315"> loadSystemCoprocessors(conf, USER_REGION_COPROCESSOR_CONF_KEY);</span>
<span class="source-line-no">316</span><span id="line-316"> }</span>
<span class="source-line-no">317</span><span id="line-317"></span>
<span class="source-line-no">318</span><span id="line-318"> // load Coprocessor From HDFS</span>
<span class="source-line-no">319</span><span id="line-319"> loadTableCoprocessors(conf);</span>
<span class="source-line-no">320</span><span id="line-320"></span>
<span class="source-line-no">321</span><span id="line-321"> // now check whether any coprocessor implements postScannerFilterRow</span>
<span class="source-line-no">322</span><span id="line-322"> boolean hasCustomPostScannerFilterRow = false;</span>
<span class="source-line-no">323</span><span id="line-323"> out: for (RegionCoprocessorEnvironment env : coprocEnvironments) {</span>
<span class="source-line-no">324</span><span id="line-324"> if (env.getInstance() instanceof RegionObserver) {</span>
<span class="source-line-no">325</span><span id="line-325"> Class&lt;?&gt; clazz = env.getInstance().getClass();</span>
<span class="source-line-no">326</span><span id="line-326"> for (;;) {</span>
<span class="source-line-no">327</span><span id="line-327"> if (clazz == Object.class) {</span>
<span class="source-line-no">328</span><span id="line-328"> // we dont need to look postScannerFilterRow into Object class</span>
<span class="source-line-no">329</span><span id="line-329"> break; // break the inner loop</span>
<span class="source-line-no">330</span><span id="line-330"> }</span>
<span class="source-line-no">331</span><span id="line-331"> try {</span>
<span class="source-line-no">332</span><span id="line-332"> clazz.getDeclaredMethod("postScannerFilterRow", ObserverContext.class,</span>
<span class="source-line-no">333</span><span id="line-333"> InternalScanner.class, Cell.class, boolean.class);</span>
<span class="source-line-no">334</span><span id="line-334"> // this coprocessor has a custom version of postScannerFilterRow</span>
<span class="source-line-no">335</span><span id="line-335"> hasCustomPostScannerFilterRow = true;</span>
<span class="source-line-no">336</span><span id="line-336"> break out;</span>
<span class="source-line-no">337</span><span id="line-337"> } catch (NoSuchMethodException ignore) {</span>
<span class="source-line-no">338</span><span id="line-338"> }</span>
<span class="source-line-no">339</span><span id="line-339"> // the deprecated signature still exists</span>
<span class="source-line-no">340</span><span id="line-340"> try {</span>
<span class="source-line-no">341</span><span id="line-341"> clazz.getDeclaredMethod("postScannerFilterRow", ObserverContext.class,</span>
<span class="source-line-no">342</span><span id="line-342"> InternalScanner.class, byte[].class, int.class, short.class, boolean.class);</span>
<span class="source-line-no">343</span><span id="line-343"> // this coprocessor has a custom version of postScannerFilterRow</span>
<span class="source-line-no">344</span><span id="line-344"> hasCustomPostScannerFilterRow = true;</span>
<span class="source-line-no">345</span><span id="line-345"> break out;</span>
<span class="source-line-no">346</span><span id="line-346"> } catch (NoSuchMethodException ignore) {</span>
<span class="source-line-no">347</span><span id="line-347"> }</span>
<span class="source-line-no">348</span><span id="line-348"> clazz = clazz.getSuperclass();</span>
<span class="source-line-no">349</span><span id="line-349"> }</span>
<span class="source-line-no">350</span><span id="line-350"> }</span>
<span class="source-line-no">351</span><span id="line-351"> }</span>
<span class="source-line-no">352</span><span id="line-352"> this.hasCustomPostScannerFilterRow = hasCustomPostScannerFilterRow;</span>
<span class="source-line-no">353</span><span id="line-353"> }</span>
<span class="source-line-no">354</span><span id="line-354"></span>
<span class="source-line-no">355</span><span id="line-355"> static List&lt;TableCoprocessorAttribute&gt; getTableCoprocessorAttrsFromSchema(Configuration conf,</span>
<span class="source-line-no">356</span><span id="line-356"> TableDescriptor htd) {</span>
<span class="source-line-no">357</span><span id="line-357"> return htd.getCoprocessorDescriptors().stream().map(cp -&gt; {</span>
<span class="source-line-no">358</span><span id="line-358"> Path path = cp.getJarPath().map(p -&gt; new Path(p)).orElse(null);</span>
<span class="source-line-no">359</span><span id="line-359"> Configuration ourConf;</span>
<span class="source-line-no">360</span><span id="line-360"> if (!cp.getProperties().isEmpty()) {</span>
<span class="source-line-no">361</span><span id="line-361"> // do an explicit deep copy of the passed configuration</span>
<span class="source-line-no">362</span><span id="line-362"> ourConf = new Configuration(false);</span>
<span class="source-line-no">363</span><span id="line-363"> HBaseConfiguration.merge(ourConf, conf);</span>
<span class="source-line-no">364</span><span id="line-364"> cp.getProperties().forEach((k, v) -&gt; ourConf.set(k, v));</span>
<span class="source-line-no">365</span><span id="line-365"> } else {</span>
<span class="source-line-no">366</span><span id="line-366"> ourConf = conf;</span>
<span class="source-line-no">367</span><span id="line-367"> }</span>
<span class="source-line-no">368</span><span id="line-368"> return new TableCoprocessorAttribute(path, cp.getClassName(), cp.getPriority(), ourConf);</span>
<span class="source-line-no">369</span><span id="line-369"> }).collect(Collectors.toList());</span>
<span class="source-line-no">370</span><span id="line-370"> }</span>
<span class="source-line-no">371</span><span id="line-371"></span>
<span class="source-line-no">372</span><span id="line-372"> /**</span>
<span class="source-line-no">373</span><span id="line-373"> * Sanity check the table coprocessor attributes of the supplied schema. Will throw an exception</span>
<span class="source-line-no">374</span><span id="line-374"> * if there is a problem.</span>
<span class="source-line-no">375</span><span id="line-375"> */</span>
<span class="source-line-no">376</span><span id="line-376"> public static void testTableCoprocessorAttrs(final Configuration conf, final TableDescriptor htd)</span>
<span class="source-line-no">377</span><span id="line-377"> throws IOException {</span>
<span class="source-line-no">378</span><span id="line-378"> String pathPrefix = UUID.randomUUID().toString();</span>
<span class="source-line-no">379</span><span id="line-379"> for (TableCoprocessorAttribute attr : getTableCoprocessorAttrsFromSchema(conf, htd)) {</span>
<span class="source-line-no">380</span><span id="line-380"> if (attr.getPriority() &lt; 0) {</span>
<span class="source-line-no">381</span><span id="line-381"> throw new IOException(</span>
<span class="source-line-no">382</span><span id="line-382"> "Priority for coprocessor " + attr.getClassName() + " cannot be less than 0");</span>
<span class="source-line-no">383</span><span id="line-383"> }</span>
<span class="source-line-no">384</span><span id="line-384"> ClassLoader old = Thread.currentThread().getContextClassLoader();</span>
<span class="source-line-no">385</span><span id="line-385"> try {</span>
<span class="source-line-no">386</span><span id="line-386"> ClassLoader cl;</span>
<span class="source-line-no">387</span><span id="line-387"> if (attr.getPath() != null) {</span>
<span class="source-line-no">388</span><span id="line-388"> cl = CoprocessorClassLoader.getClassLoader(attr.getPath(),</span>
<span class="source-line-no">389</span><span id="line-389"> CoprocessorHost.class.getClassLoader(), pathPrefix, conf);</span>
<span class="source-line-no">390</span><span id="line-390"> } else {</span>
<span class="source-line-no">391</span><span id="line-391"> cl = CoprocessorHost.class.getClassLoader();</span>
<span class="source-line-no">392</span><span id="line-392"> }</span>
<span class="source-line-no">393</span><span id="line-393"> Thread.currentThread().setContextClassLoader(cl);</span>
<span class="source-line-no">394</span><span id="line-394"> if (cl instanceof CoprocessorClassLoader) {</span>
<span class="source-line-no">395</span><span id="line-395"> String[] includedClassPrefixes = null;</span>
<span class="source-line-no">396</span><span id="line-396"> if (conf.get(HConstants.CP_HTD_ATTR_INCLUSION_KEY) != null) {</span>
<span class="source-line-no">397</span><span id="line-397"> String prefixes = attr.conf.get(HConstants.CP_HTD_ATTR_INCLUSION_KEY);</span>
<span class="source-line-no">398</span><span id="line-398"> includedClassPrefixes = prefixes.split(";");</span>
<span class="source-line-no">399</span><span id="line-399"> }</span>
<span class="source-line-no">400</span><span id="line-400"> ((CoprocessorClassLoader) cl).loadClass(attr.getClassName(), includedClassPrefixes);</span>
<span class="source-line-no">401</span><span id="line-401"> } else {</span>
<span class="source-line-no">402</span><span id="line-402"> cl.loadClass(attr.getClassName());</span>
<span class="source-line-no">403</span><span id="line-403"> }</span>
<span class="source-line-no">404</span><span id="line-404"> } catch (ClassNotFoundException e) {</span>
<span class="source-line-no">405</span><span id="line-405"> throw new IOException("Class " + attr.getClassName() + " cannot be loaded", e);</span>
<span class="source-line-no">406</span><span id="line-406"> } finally {</span>
<span class="source-line-no">407</span><span id="line-407"> Thread.currentThread().setContextClassLoader(old);</span>
<span class="source-line-no">408</span><span id="line-408"> }</span>
<span class="source-line-no">409</span><span id="line-409"> }</span>
<span class="source-line-no">410</span><span id="line-410"> }</span>
<span class="source-line-no">411</span><span id="line-411"></span>
<span class="source-line-no">412</span><span id="line-412"> void loadTableCoprocessors(final Configuration conf) {</span>
<span class="source-line-no">413</span><span id="line-413"> boolean coprocessorsEnabled =</span>
<span class="source-line-no">414</span><span id="line-414"> conf.getBoolean(COPROCESSORS_ENABLED_CONF_KEY, DEFAULT_COPROCESSORS_ENABLED);</span>
<span class="source-line-no">415</span><span id="line-415"> boolean tableCoprocessorsEnabled =</span>
<span class="source-line-no">416</span><span id="line-416"> conf.getBoolean(USER_COPROCESSORS_ENABLED_CONF_KEY, DEFAULT_USER_COPROCESSORS_ENABLED);</span>
<span class="source-line-no">417</span><span id="line-417"> if (!(coprocessorsEnabled &amp;&amp; tableCoprocessorsEnabled)) {</span>
<span class="source-line-no">418</span><span id="line-418"> return;</span>
<span class="source-line-no">419</span><span id="line-419"> }</span>
<span class="source-line-no">420</span><span id="line-420"></span>
<span class="source-line-no">421</span><span id="line-421"> // scan the table attributes for coprocessor load specifications</span>
<span class="source-line-no">422</span><span id="line-422"> // initialize the coprocessors</span>
<span class="source-line-no">423</span><span id="line-423"> List&lt;RegionCoprocessorEnvironment&gt; configured = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">424</span><span id="line-424"> for (TableCoprocessorAttribute attr : getTableCoprocessorAttrsFromSchema(conf,</span>
<span class="source-line-no">425</span><span id="line-425"> region.getTableDescriptor())) {</span>
<span class="source-line-no">426</span><span id="line-426"> // Load encompasses classloading and coprocessor initialization</span>
<span class="source-line-no">427</span><span id="line-427"> try {</span>
<span class="source-line-no">428</span><span id="line-428"> RegionCoprocessorEnvironment env =</span>
<span class="source-line-no">429</span><span id="line-429"> load(attr.getPath(), attr.getClassName(), attr.getPriority(), attr.getConf());</span>
<span class="source-line-no">430</span><span id="line-430"> if (env == null) {</span>
<span class="source-line-no">431</span><span id="line-431"> continue;</span>
<span class="source-line-no">432</span><span id="line-432"> }</span>
<span class="source-line-no">433</span><span id="line-433"> configured.add(env);</span>
<span class="source-line-no">434</span><span id="line-434"> LOG.info("Loaded coprocessor " + attr.getClassName() + " from HTD of "</span>
<span class="source-line-no">435</span><span id="line-435"> + region.getTableDescriptor().getTableName().getNameAsString() + " successfully.");</span>
<span class="source-line-no">436</span><span id="line-436"> } catch (Throwable t) {</span>
<span class="source-line-no">437</span><span id="line-437"> // Coprocessor failed to load, do we abort on error?</span>
<span class="source-line-no">438</span><span id="line-438"> if (conf.getBoolean(ABORT_ON_ERROR_KEY, DEFAULT_ABORT_ON_ERROR)) {</span>
<span class="source-line-no">439</span><span id="line-439"> abortServer(attr.getClassName(), t);</span>
<span class="source-line-no">440</span><span id="line-440"> } else {</span>
<span class="source-line-no">441</span><span id="line-441"> LOG.error("Failed to load coprocessor " + attr.getClassName(), t);</span>
<span class="source-line-no">442</span><span id="line-442"> }</span>
<span class="source-line-no">443</span><span id="line-443"> }</span>
<span class="source-line-no">444</span><span id="line-444"> }</span>
<span class="source-line-no">445</span><span id="line-445"> // add together to coprocessor set for COW efficiency</span>
<span class="source-line-no">446</span><span id="line-446"> coprocEnvironments.addAll(configured);</span>
<span class="source-line-no">447</span><span id="line-447"> }</span>
<span class="source-line-no">448</span><span id="line-448"></span>
<span class="source-line-no">449</span><span id="line-449"> @Override</span>
<span class="source-line-no">450</span><span id="line-450"> public RegionEnvironment createEnvironment(RegionCoprocessor instance, int priority, int seq,</span>
<span class="source-line-no">451</span><span id="line-451"> Configuration conf) {</span>
<span class="source-line-no">452</span><span id="line-452"> // If coprocessor exposes any services, register them.</span>
<span class="source-line-no">453</span><span id="line-453"> for (Service service : instance.getServices()) {</span>
<span class="source-line-no">454</span><span id="line-454"> region.registerService(service);</span>
<span class="source-line-no">455</span><span id="line-455"> }</span>
<span class="source-line-no">456</span><span id="line-456"> ConcurrentMap&lt;String, Object&gt; classData;</span>
<span class="source-line-no">457</span><span id="line-457"> // make sure only one thread can add maps</span>
<span class="source-line-no">458</span><span id="line-458"> synchronized (SHARED_DATA_MAP) {</span>
<span class="source-line-no">459</span><span id="line-459"> // as long as at least one RegionEnvironment holds on to its classData it will</span>
<span class="source-line-no">460</span><span id="line-460"> // remain in this map</span>
<span class="source-line-no">461</span><span id="line-461"> classData = SHARED_DATA_MAP.computeIfAbsent(instance.getClass().getName(),</span>
<span class="source-line-no">462</span><span id="line-462"> k -&gt; new ConcurrentHashMap&lt;&gt;());</span>
<span class="source-line-no">463</span><span id="line-463"> }</span>
<span class="source-line-no">464</span><span id="line-464"> // If a CoreCoprocessor, return a 'richer' environment, one laden with RegionServerServices.</span>
<span class="source-line-no">465</span><span id="line-465"> return instance.getClass().isAnnotationPresent(CoreCoprocessor.class)</span>
<span class="source-line-no">466</span><span id="line-466"> ? new RegionEnvironmentForCoreCoprocessors(instance, priority, seq, conf, region, rsServices,</span>
<span class="source-line-no">467</span><span id="line-467"> classData)</span>
<span class="source-line-no">468</span><span id="line-468"> : new RegionEnvironment(instance, priority, seq, conf, region, rsServices, classData);</span>
<span class="source-line-no">469</span><span id="line-469"> }</span>
<span class="source-line-no">470</span><span id="line-470"></span>
<span class="source-line-no">471</span><span id="line-471"> @Override</span>
<span class="source-line-no">472</span><span id="line-472"> public RegionCoprocessor checkAndGetInstance(Class&lt;?&gt; implClass)</span>
<span class="source-line-no">473</span><span id="line-473"> throws InstantiationException, IllegalAccessException {</span>
<span class="source-line-no">474</span><span id="line-474"> try {</span>
<span class="source-line-no">475</span><span id="line-475"> if (RegionCoprocessor.class.isAssignableFrom(implClass)) {</span>
<span class="source-line-no">476</span><span id="line-476"> return implClass.asSubclass(RegionCoprocessor.class).getDeclaredConstructor().newInstance();</span>
<span class="source-line-no">477</span><span id="line-477"> } else {</span>
<span class="source-line-no">478</span><span id="line-478"> LOG.error("{} is not of type RegionCoprocessor. Check the configuration of {}",</span>
<span class="source-line-no">479</span><span id="line-479"> implClass.getName(), CoprocessorHost.REGION_COPROCESSOR_CONF_KEY);</span>
<span class="source-line-no">480</span><span id="line-480"> return null;</span>
<span class="source-line-no">481</span><span id="line-481"> }</span>
<span class="source-line-no">482</span><span id="line-482"> } catch (NoSuchMethodException | InvocationTargetException e) {</span>
<span class="source-line-no">483</span><span id="line-483"> throw (InstantiationException) new InstantiationException(implClass.getName()).initCause(e);</span>
<span class="source-line-no">484</span><span id="line-484"> }</span>
<span class="source-line-no">485</span><span id="line-485"> }</span>
<span class="source-line-no">486</span><span id="line-486"></span>
<span class="source-line-no">487</span><span id="line-487"> private ObserverGetter&lt;RegionCoprocessor, RegionObserver&gt; regionObserverGetter =</span>
<span class="source-line-no">488</span><span id="line-488"> RegionCoprocessor::getRegionObserver;</span>
<span class="source-line-no">489</span><span id="line-489"></span>
<span class="source-line-no">490</span><span id="line-490"> private ObserverGetter&lt;RegionCoprocessor, EndpointObserver&gt; endpointObserverGetter =</span>
<span class="source-line-no">491</span><span id="line-491"> RegionCoprocessor::getEndpointObserver;</span>
<span class="source-line-no">492</span><span id="line-492"></span>
<span class="source-line-no">493</span><span id="line-493"> abstract class RegionObserverOperationWithoutResult</span>
<span class="source-line-no">494</span><span id="line-494"> extends ObserverOperationWithoutResult&lt;RegionObserver&gt; {</span>
<span class="source-line-no">495</span><span id="line-495"> public RegionObserverOperationWithoutResult() {</span>
<span class="source-line-no">496</span><span id="line-496"> super(regionObserverGetter);</span>
<span class="source-line-no">497</span><span id="line-497"> }</span>
<span class="source-line-no">498</span><span id="line-498"></span>
<span class="source-line-no">499</span><span id="line-499"> public RegionObserverOperationWithoutResult(User user) {</span>
<span class="source-line-no">500</span><span id="line-500"> super(regionObserverGetter, user);</span>
<span class="source-line-no">501</span><span id="line-501"> }</span>
<span class="source-line-no">502</span><span id="line-502"></span>
<span class="source-line-no">503</span><span id="line-503"> public RegionObserverOperationWithoutResult(boolean bypassable) {</span>
<span class="source-line-no">504</span><span id="line-504"> super(regionObserverGetter, null, bypassable);</span>
<span class="source-line-no">505</span><span id="line-505"> }</span>
<span class="source-line-no">506</span><span id="line-506"></span>
<span class="source-line-no">507</span><span id="line-507"> public RegionObserverOperationWithoutResult(User user, boolean bypassable) {</span>
<span class="source-line-no">508</span><span id="line-508"> super(regionObserverGetter, user, bypassable);</span>
<span class="source-line-no">509</span><span id="line-509"> }</span>
<span class="source-line-no">510</span><span id="line-510"> }</span>
<span class="source-line-no">511</span><span id="line-511"></span>
<span class="source-line-no">512</span><span id="line-512"> abstract class BulkLoadObserverOperation</span>
<span class="source-line-no">513</span><span id="line-513"> extends ObserverOperationWithoutResult&lt;BulkLoadObserver&gt; {</span>
<span class="source-line-no">514</span><span id="line-514"> public BulkLoadObserverOperation(User user) {</span>
<span class="source-line-no">515</span><span id="line-515"> super(RegionCoprocessor::getBulkLoadObserver, user);</span>
<span class="source-line-no">516</span><span id="line-516"> }</span>
<span class="source-line-no">517</span><span id="line-517"> }</span>
<span class="source-line-no">518</span><span id="line-518"></span>
<span class="source-line-no">519</span><span id="line-519"> //////////////////////////////////////////////////////////////////////////////////////////////////</span>
<span class="source-line-no">520</span><span id="line-520"> // Observer operations</span>
<span class="source-line-no">521</span><span id="line-521"> //////////////////////////////////////////////////////////////////////////////////////////////////</span>
<span class="source-line-no">522</span><span id="line-522"></span>
<span class="source-line-no">523</span><span id="line-523"> //////////////////////////////////////////////////////////////////////////////////////////////////</span>
<span class="source-line-no">524</span><span id="line-524"> // Observer operations</span>
<span class="source-line-no">525</span><span id="line-525"> //////////////////////////////////////////////////////////////////////////////////////////////////</span>
<span class="source-line-no">526</span><span id="line-526"></span>
<span class="source-line-no">527</span><span id="line-527"> /**</span>
<span class="source-line-no">528</span><span id="line-528"> * Invoked before a region open.</span>
<span class="source-line-no">529</span><span id="line-529"> * @throws IOException Signals that an I/O exception has occurred.</span>
<span class="source-line-no">530</span><span id="line-530"> */</span>
<span class="source-line-no">531</span><span id="line-531"> public void preOpen() throws IOException {</span>
<span class="source-line-no">532</span><span id="line-532"> if (coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">533</span><span id="line-533"> return;</span>
<span class="source-line-no">534</span><span id="line-534"> }</span>
<span class="source-line-no">535</span><span id="line-535"> execOperation(new RegionObserverOperationWithoutResult() {</span>
<span class="source-line-no">536</span><span id="line-536"> @Override</span>
<span class="source-line-no">537</span><span id="line-537"> public void call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">538</span><span id="line-538"> observer.preOpen(this);</span>
<span class="source-line-no">539</span><span id="line-539"> }</span>
<span class="source-line-no">540</span><span id="line-540"> });</span>
<span class="source-line-no">541</span><span id="line-541"> }</span>
<span class="source-line-no">542</span><span id="line-542"></span>
<span class="source-line-no">543</span><span id="line-543"> /**</span>
<span class="source-line-no">544</span><span id="line-544"> * Invoked after a region open</span>
<span class="source-line-no">545</span><span id="line-545"> */</span>
<span class="source-line-no">546</span><span id="line-546"> public void postOpen() {</span>
<span class="source-line-no">547</span><span id="line-547"> if (coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">548</span><span id="line-548"> return;</span>
<span class="source-line-no">549</span><span id="line-549"> }</span>
<span class="source-line-no">550</span><span id="line-550"> try {</span>
<span class="source-line-no">551</span><span id="line-551"> execOperation(new RegionObserverOperationWithoutResult() {</span>
<span class="source-line-no">552</span><span id="line-552"> @Override</span>
<span class="source-line-no">553</span><span id="line-553"> public void call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">554</span><span id="line-554"> observer.postOpen(this);</span>
<span class="source-line-no">555</span><span id="line-555"> }</span>
<span class="source-line-no">556</span><span id="line-556"> });</span>
<span class="source-line-no">557</span><span id="line-557"> } catch (IOException e) {</span>
<span class="source-line-no">558</span><span id="line-558"> LOG.warn(e.toString(), e);</span>
<span class="source-line-no">559</span><span id="line-559"> }</span>
<span class="source-line-no">560</span><span id="line-560"> }</span>
<span class="source-line-no">561</span><span id="line-561"></span>
<span class="source-line-no">562</span><span id="line-562"> /**</span>
<span class="source-line-no">563</span><span id="line-563"> * Invoked before a region is closed</span>
<span class="source-line-no">564</span><span id="line-564"> * @param abortRequested true if the server is aborting</span>
<span class="source-line-no">565</span><span id="line-565"> */</span>
<span class="source-line-no">566</span><span id="line-566"> public void preClose(final boolean abortRequested) throws IOException {</span>
<span class="source-line-no">567</span><span id="line-567"> execOperation(new RegionObserverOperationWithoutResult() {</span>
<span class="source-line-no">568</span><span id="line-568"> @Override</span>
<span class="source-line-no">569</span><span id="line-569"> public void call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">570</span><span id="line-570"> observer.preClose(this, abortRequested);</span>
<span class="source-line-no">571</span><span id="line-571"> }</span>
<span class="source-line-no">572</span><span id="line-572"> });</span>
<span class="source-line-no">573</span><span id="line-573"> }</span>
<span class="source-line-no">574</span><span id="line-574"></span>
<span class="source-line-no">575</span><span id="line-575"> /**</span>
<span class="source-line-no">576</span><span id="line-576"> * Invoked after a region is closed</span>
<span class="source-line-no">577</span><span id="line-577"> * @param abortRequested true if the server is aborting</span>
<span class="source-line-no">578</span><span id="line-578"> */</span>
<span class="source-line-no">579</span><span id="line-579"> public void postClose(final boolean abortRequested) {</span>
<span class="source-line-no">580</span><span id="line-580"> try {</span>
<span class="source-line-no">581</span><span id="line-581"> execOperation(new RegionObserverOperationWithoutResult() {</span>
<span class="source-line-no">582</span><span id="line-582"> @Override</span>
<span class="source-line-no">583</span><span id="line-583"> public void call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">584</span><span id="line-584"> observer.postClose(this, abortRequested);</span>
<span class="source-line-no">585</span><span id="line-585"> }</span>
<span class="source-line-no">586</span><span id="line-586"></span>
<span class="source-line-no">587</span><span id="line-587"> @Override</span>
<span class="source-line-no">588</span><span id="line-588"> public void postEnvCall() {</span>
<span class="source-line-no">589</span><span id="line-589"> shutdown(this.getEnvironment());</span>
<span class="source-line-no">590</span><span id="line-590"> }</span>
<span class="source-line-no">591</span><span id="line-591"> });</span>
<span class="source-line-no">592</span><span id="line-592"> } catch (IOException e) {</span>
<span class="source-line-no">593</span><span id="line-593"> LOG.warn(e.toString(), e);</span>
<span class="source-line-no">594</span><span id="line-594"> }</span>
<span class="source-line-no">595</span><span id="line-595"> }</span>
<span class="source-line-no">596</span><span id="line-596"></span>
<span class="source-line-no">597</span><span id="line-597"> /**</span>
<span class="source-line-no">598</span><span id="line-598"> * Called prior to selecting the {@link HStoreFile}s for compaction from the list of currently</span>
<span class="source-line-no">599</span><span id="line-599"> * available candidates.</span>
<span class="source-line-no">600</span><span id="line-600"> * &lt;p&gt;</span>
<span class="source-line-no">601</span><span id="line-601"> * Supports Coprocessor 'bypass' -- 'bypass' is how this method indicates that it changed the</span>
<span class="source-line-no">602</span><span id="line-602"> * passed in &lt;code&gt;candidates&lt;/code&gt;.</span>
<span class="source-line-no">603</span><span id="line-603"> * @param store The store where compaction is being requested</span>
<span class="source-line-no">604</span><span id="line-604"> * @param candidates The currently available store files</span>
<span class="source-line-no">605</span><span id="line-605"> * @param tracker used to track the life cycle of a compaction</span>
<span class="source-line-no">606</span><span id="line-606"> * @param user the user</span>
<span class="source-line-no">607</span><span id="line-607"> */</span>
<span class="source-line-no">608</span><span id="line-608"> public boolean preCompactSelection(final HStore store, final List&lt;HStoreFile&gt; candidates,</span>
<span class="source-line-no">609</span><span id="line-609"> final CompactionLifeCycleTracker tracker, final User user) throws IOException {</span>
<span class="source-line-no">610</span><span id="line-610"> if (coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">611</span><span id="line-611"> return false;</span>
<span class="source-line-no">612</span><span id="line-612"> }</span>
<span class="source-line-no">613</span><span id="line-613"> boolean bypassable = true;</span>
<span class="source-line-no">614</span><span id="line-614"> return execOperation(new RegionObserverOperationWithoutResult(user, bypassable) {</span>
<span class="source-line-no">615</span><span id="line-615"> @Override</span>
<span class="source-line-no">616</span><span id="line-616"> public void call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">617</span><span id="line-617"> observer.preCompactSelection(this, store, candidates, tracker);</span>
<span class="source-line-no">618</span><span id="line-618"> }</span>
<span class="source-line-no">619</span><span id="line-619"> });</span>
<span class="source-line-no">620</span><span id="line-620"> }</span>
<span class="source-line-no">621</span><span id="line-621"></span>
<span class="source-line-no">622</span><span id="line-622"> /**</span>
<span class="source-line-no">623</span><span id="line-623"> * Called after the {@link HStoreFile}s to be compacted have been selected from the available</span>
<span class="source-line-no">624</span><span id="line-624"> * candidates.</span>
<span class="source-line-no">625</span><span id="line-625"> * @param store The store where compaction is being requested</span>
<span class="source-line-no">626</span><span id="line-626"> * @param selected The store files selected to compact</span>
<span class="source-line-no">627</span><span id="line-627"> * @param tracker used to track the life cycle of a compaction</span>
<span class="source-line-no">628</span><span id="line-628"> * @param request the compaction request</span>
<span class="source-line-no">629</span><span id="line-629"> * @param user the user</span>
<span class="source-line-no">630</span><span id="line-630"> */</span>
<span class="source-line-no">631</span><span id="line-631"> public void postCompactSelection(final HStore store, final List&lt;HStoreFile&gt; selected,</span>
<span class="source-line-no">632</span><span id="line-632"> final CompactionLifeCycleTracker tracker, final CompactionRequest request, final User user)</span>
<span class="source-line-no">633</span><span id="line-633"> throws IOException {</span>
<span class="source-line-no">634</span><span id="line-634"> if (coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">635</span><span id="line-635"> return;</span>
<span class="source-line-no">636</span><span id="line-636"> }</span>
<span class="source-line-no">637</span><span id="line-637"> execOperation(new RegionObserverOperationWithoutResult(user) {</span>
<span class="source-line-no">638</span><span id="line-638"> @Override</span>
<span class="source-line-no">639</span><span id="line-639"> public void call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">640</span><span id="line-640"> observer.postCompactSelection(this, store, selected, tracker, request);</span>
<span class="source-line-no">641</span><span id="line-641"> }</span>
<span class="source-line-no">642</span><span id="line-642"> });</span>
<span class="source-line-no">643</span><span id="line-643"> }</span>
<span class="source-line-no">644</span><span id="line-644"></span>
<span class="source-line-no">645</span><span id="line-645"> /**</span>
<span class="source-line-no">646</span><span id="line-646"> * Called prior to opening store scanner for compaction.</span>
<span class="source-line-no">647</span><span id="line-647"> */</span>
<span class="source-line-no">648</span><span id="line-648"> public ScanInfo preCompactScannerOpen(HStore store, ScanType scanType,</span>
<span class="source-line-no">649</span><span id="line-649"> CompactionLifeCycleTracker tracker, CompactionRequest request, User user) throws IOException {</span>
<span class="source-line-no">650</span><span id="line-650"> if (coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">651</span><span id="line-651"> return store.getScanInfo();</span>
<span class="source-line-no">652</span><span id="line-652"> }</span>
<span class="source-line-no">653</span><span id="line-653"> CustomizedScanInfoBuilder builder = new CustomizedScanInfoBuilder(store.getScanInfo());</span>
<span class="source-line-no">654</span><span id="line-654"> execOperation(new RegionObserverOperationWithoutResult(user) {</span>
<span class="source-line-no">655</span><span id="line-655"> @Override</span>
<span class="source-line-no">656</span><span id="line-656"> public void call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">657</span><span id="line-657"> observer.preCompactScannerOpen(this, store, scanType, builder, tracker, request);</span>
<span class="source-line-no">658</span><span id="line-658"> }</span>
<span class="source-line-no">659</span><span id="line-659"> });</span>
<span class="source-line-no">660</span><span id="line-660"> return builder.build();</span>
<span class="source-line-no">661</span><span id="line-661"> }</span>
<span class="source-line-no">662</span><span id="line-662"></span>
<span class="source-line-no">663</span><span id="line-663"> /**</span>
<span class="source-line-no">664</span><span id="line-664"> * Called prior to rewriting the store files selected for compaction</span>
<span class="source-line-no">665</span><span id="line-665"> * @param store the store being compacted</span>
<span class="source-line-no">666</span><span id="line-666"> * @param scanner the scanner used to read store data during compaction</span>
<span class="source-line-no">667</span><span id="line-667"> * @param scanType type of Scan</span>
<span class="source-line-no">668</span><span id="line-668"> * @param tracker used to track the life cycle of a compaction</span>
<span class="source-line-no">669</span><span id="line-669"> * @param request the compaction request</span>
<span class="source-line-no">670</span><span id="line-670"> * @param user the user</span>
<span class="source-line-no">671</span><span id="line-671"> * @return Scanner to use (cannot be null!)</span>
<span class="source-line-no">672</span><span id="line-672"> */</span>
<span class="source-line-no">673</span><span id="line-673"> public InternalScanner preCompact(final HStore store, final InternalScanner scanner,</span>
<span class="source-line-no">674</span><span id="line-674"> final ScanType scanType, final CompactionLifeCycleTracker tracker,</span>
<span class="source-line-no">675</span><span id="line-675"> final CompactionRequest request, final User user) throws IOException {</span>
<span class="source-line-no">676</span><span id="line-676"> InternalScanner defaultResult = scanner;</span>
<span class="source-line-no">677</span><span id="line-677"> if (coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">678</span><span id="line-678"> return defaultResult;</span>
<span class="source-line-no">679</span><span id="line-679"> }</span>
<span class="source-line-no">680</span><span id="line-680"> return execOperationWithResult(new ObserverOperationWithResult&lt;RegionObserver, InternalScanner&gt;(</span>
<span class="source-line-no">681</span><span id="line-681"> regionObserverGetter, defaultResult, user) {</span>
<span class="source-line-no">682</span><span id="line-682"> @Override</span>
<span class="source-line-no">683</span><span id="line-683"> public InternalScanner call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">684</span><span id="line-684"> InternalScanner scanner =</span>
<span class="source-line-no">685</span><span id="line-685"> observer.preCompact(this, store, getResult(), scanType, tracker, request);</span>
<span class="source-line-no">686</span><span id="line-686"> if (scanner == null) {</span>
<span class="source-line-no">687</span><span id="line-687"> throw new CoprocessorException("Null Scanner return disallowed!");</span>
<span class="source-line-no">688</span><span id="line-688"> }</span>
<span class="source-line-no">689</span><span id="line-689"> return scanner;</span>
<span class="source-line-no">690</span><span id="line-690"> }</span>
<span class="source-line-no">691</span><span id="line-691"> });</span>
<span class="source-line-no">692</span><span id="line-692"> }</span>
<span class="source-line-no">693</span><span id="line-693"></span>
<span class="source-line-no">694</span><span id="line-694"> /**</span>
<span class="source-line-no">695</span><span id="line-695"> * Called after the store compaction has completed.</span>
<span class="source-line-no">696</span><span id="line-696"> * @param store the store being compacted</span>
<span class="source-line-no">697</span><span id="line-697"> * @param resultFile the new store file written during compaction</span>
<span class="source-line-no">698</span><span id="line-698"> * @param tracker used to track the life cycle of a compaction</span>
<span class="source-line-no">699</span><span id="line-699"> * @param request the compaction request</span>
<span class="source-line-no">700</span><span id="line-700"> * @param user the user</span>
<span class="source-line-no">701</span><span id="line-701"> */</span>
<span class="source-line-no">702</span><span id="line-702"> public void postCompact(final HStore store, final HStoreFile resultFile,</span>
<span class="source-line-no">703</span><span id="line-703"> final CompactionLifeCycleTracker tracker, final CompactionRequest request, final User user)</span>
<span class="source-line-no">704</span><span id="line-704"> throws IOException {</span>
<span class="source-line-no">705</span><span id="line-705"> execOperation(</span>
<span class="source-line-no">706</span><span id="line-706"> coprocEnvironments.isEmpty() ? null : new RegionObserverOperationWithoutResult(user) {</span>
<span class="source-line-no">707</span><span id="line-707"> @Override</span>
<span class="source-line-no">708</span><span id="line-708"> public void call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">709</span><span id="line-709"> observer.postCompact(this, store, resultFile, tracker, request);</span>
<span class="source-line-no">710</span><span id="line-710"> }</span>
<span class="source-line-no">711</span><span id="line-711"> });</span>
<span class="source-line-no">712</span><span id="line-712"> }</span>
<span class="source-line-no">713</span><span id="line-713"></span>
<span class="source-line-no">714</span><span id="line-714"> /**</span>
<span class="source-line-no">715</span><span id="line-715"> * Invoked before create StoreScanner for flush.</span>
<span class="source-line-no">716</span><span id="line-716"> */</span>
<span class="source-line-no">717</span><span id="line-717"> public ScanInfo preFlushScannerOpen(HStore store, FlushLifeCycleTracker tracker)</span>
<span class="source-line-no">718</span><span id="line-718"> throws IOException {</span>
<span class="source-line-no">719</span><span id="line-719"> if (coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">720</span><span id="line-720"> return store.getScanInfo();</span>
<span class="source-line-no">721</span><span id="line-721"> }</span>
<span class="source-line-no">722</span><span id="line-722"> CustomizedScanInfoBuilder builder = new CustomizedScanInfoBuilder(store.getScanInfo());</span>
<span class="source-line-no">723</span><span id="line-723"> execOperation(new RegionObserverOperationWithoutResult() {</span>
<span class="source-line-no">724</span><span id="line-724"> @Override</span>
<span class="source-line-no">725</span><span id="line-725"> public void call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">726</span><span id="line-726"> observer.preFlushScannerOpen(this, store, builder, tracker);</span>
<span class="source-line-no">727</span><span id="line-727"> }</span>
<span class="source-line-no">728</span><span id="line-728"> });</span>
<span class="source-line-no">729</span><span id="line-729"> return builder.build();</span>
<span class="source-line-no">730</span><span id="line-730"> }</span>
<span class="source-line-no">731</span><span id="line-731"></span>
<span class="source-line-no">732</span><span id="line-732"> /**</span>
<span class="source-line-no">733</span><span id="line-733"> * Invoked before a memstore flush</span>
<span class="source-line-no">734</span><span id="line-734"> * @return Scanner to use (cannot be null!)</span>
<span class="source-line-no">735</span><span id="line-735"> */</span>
<span class="source-line-no">736</span><span id="line-736"> public InternalScanner preFlush(HStore store, InternalScanner scanner,</span>
<span class="source-line-no">737</span><span id="line-737"> FlushLifeCycleTracker tracker) throws IOException {</span>
<span class="source-line-no">738</span><span id="line-738"> if (coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">739</span><span id="line-739"> return scanner;</span>
<span class="source-line-no">740</span><span id="line-740"> }</span>
<span class="source-line-no">741</span><span id="line-741"> return execOperationWithResult(new ObserverOperationWithResult&lt;RegionObserver, InternalScanner&gt;(</span>
<span class="source-line-no">742</span><span id="line-742"> regionObserverGetter, scanner) {</span>
<span class="source-line-no">743</span><span id="line-743"> @Override</span>
<span class="source-line-no">744</span><span id="line-744"> public InternalScanner call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">745</span><span id="line-745"> InternalScanner scanner = observer.preFlush(this, store, getResult(), tracker);</span>
<span class="source-line-no">746</span><span id="line-746"> if (scanner == null) {</span>
<span class="source-line-no">747</span><span id="line-747"> throw new CoprocessorException("Null Scanner return disallowed!");</span>
<span class="source-line-no">748</span><span id="line-748"> }</span>
<span class="source-line-no">749</span><span id="line-749"> return scanner;</span>
<span class="source-line-no">750</span><span id="line-750"> }</span>
<span class="source-line-no">751</span><span id="line-751"> });</span>
<span class="source-line-no">752</span><span id="line-752"> }</span>
<span class="source-line-no">753</span><span id="line-753"></span>
<span class="source-line-no">754</span><span id="line-754"> /**</span>
<span class="source-line-no">755</span><span id="line-755"> * Invoked before a memstore flush</span>
<span class="source-line-no">756</span><span id="line-756"> */</span>
<span class="source-line-no">757</span><span id="line-757"> public void preFlush(FlushLifeCycleTracker tracker) throws IOException {</span>
<span class="source-line-no">758</span><span id="line-758"> execOperation(coprocEnvironments.isEmpty() ? null : new RegionObserverOperationWithoutResult() {</span>
<span class="source-line-no">759</span><span id="line-759"> @Override</span>
<span class="source-line-no">760</span><span id="line-760"> public void call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">761</span><span id="line-761"> observer.preFlush(this, tracker);</span>
<span class="source-line-no">762</span><span id="line-762"> }</span>
<span class="source-line-no">763</span><span id="line-763"> });</span>
<span class="source-line-no">764</span><span id="line-764"> }</span>
<span class="source-line-no">765</span><span id="line-765"></span>
<span class="source-line-no">766</span><span id="line-766"> /**</span>
<span class="source-line-no">767</span><span id="line-767"> * Invoked after a memstore flush</span>
<span class="source-line-no">768</span><span id="line-768"> */</span>
<span class="source-line-no">769</span><span id="line-769"> public void postFlush(FlushLifeCycleTracker tracker) throws IOException {</span>
<span class="source-line-no">770</span><span id="line-770"> execOperation(coprocEnvironments.isEmpty() ? null : new RegionObserverOperationWithoutResult() {</span>
<span class="source-line-no">771</span><span id="line-771"> @Override</span>
<span class="source-line-no">772</span><span id="line-772"> public void call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">773</span><span id="line-773"> observer.postFlush(this, tracker);</span>
<span class="source-line-no">774</span><span id="line-774"> }</span>
<span class="source-line-no">775</span><span id="line-775"> });</span>
<span class="source-line-no">776</span><span id="line-776"> }</span>
<span class="source-line-no">777</span><span id="line-777"></span>
<span class="source-line-no">778</span><span id="line-778"> /**</span>
<span class="source-line-no">779</span><span id="line-779"> * Invoked before in memory compaction.</span>
<span class="source-line-no">780</span><span id="line-780"> */</span>
<span class="source-line-no">781</span><span id="line-781"> public void preMemStoreCompaction(HStore store) throws IOException {</span>
<span class="source-line-no">782</span><span id="line-782"> execOperation(coprocEnvironments.isEmpty() ? null : new RegionObserverOperationWithoutResult() {</span>
<span class="source-line-no">783</span><span id="line-783"> @Override</span>
<span class="source-line-no">784</span><span id="line-784"> public void call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">785</span><span id="line-785"> observer.preMemStoreCompaction(this, store);</span>
<span class="source-line-no">786</span><span id="line-786"> }</span>
<span class="source-line-no">787</span><span id="line-787"> });</span>
<span class="source-line-no">788</span><span id="line-788"> }</span>
<span class="source-line-no">789</span><span id="line-789"></span>
<span class="source-line-no">790</span><span id="line-790"> /**</span>
<span class="source-line-no">791</span><span id="line-791"> * Invoked before create StoreScanner for in memory compaction.</span>
<span class="source-line-no">792</span><span id="line-792"> */</span>
<span class="source-line-no">793</span><span id="line-793"> public ScanInfo preMemStoreCompactionCompactScannerOpen(HStore store) throws IOException {</span>
<span class="source-line-no">794</span><span id="line-794"> CustomizedScanInfoBuilder builder = new CustomizedScanInfoBuilder(store.getScanInfo());</span>
<span class="source-line-no">795</span><span id="line-795"> execOperation(coprocEnvironments.isEmpty() ? null : new RegionObserverOperationWithoutResult() {</span>
<span class="source-line-no">796</span><span id="line-796"> @Override</span>
<span class="source-line-no">797</span><span id="line-797"> public void call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">798</span><span id="line-798"> observer.preMemStoreCompactionCompactScannerOpen(this, store, builder);</span>
<span class="source-line-no">799</span><span id="line-799"> }</span>
<span class="source-line-no">800</span><span id="line-800"> });</span>
<span class="source-line-no">801</span><span id="line-801"> return builder.build();</span>
<span class="source-line-no">802</span><span id="line-802"> }</span>
<span class="source-line-no">803</span><span id="line-803"></span>
<span class="source-line-no">804</span><span id="line-804"> /**</span>
<span class="source-line-no">805</span><span id="line-805"> * Invoked before compacting memstore.</span>
<span class="source-line-no">806</span><span id="line-806"> */</span>
<span class="source-line-no">807</span><span id="line-807"> public InternalScanner preMemStoreCompactionCompact(HStore store, InternalScanner scanner)</span>
<span class="source-line-no">808</span><span id="line-808"> throws IOException {</span>
<span class="source-line-no">809</span><span id="line-809"> if (coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">810</span><span id="line-810"> return scanner;</span>
<span class="source-line-no">811</span><span id="line-811"> }</span>
<span class="source-line-no">812</span><span id="line-812"> return execOperationWithResult(new ObserverOperationWithResult&lt;RegionObserver, InternalScanner&gt;(</span>
<span class="source-line-no">813</span><span id="line-813"> regionObserverGetter, scanner) {</span>
<span class="source-line-no">814</span><span id="line-814"> @Override</span>
<span class="source-line-no">815</span><span id="line-815"> public InternalScanner call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">816</span><span id="line-816"> return observer.preMemStoreCompactionCompact(this, store, getResult());</span>
<span class="source-line-no">817</span><span id="line-817"> }</span>
<span class="source-line-no">818</span><span id="line-818"> });</span>
<span class="source-line-no">819</span><span id="line-819"> }</span>
<span class="source-line-no">820</span><span id="line-820"></span>
<span class="source-line-no">821</span><span id="line-821"> /**</span>
<span class="source-line-no">822</span><span id="line-822"> * Invoked after in memory compaction.</span>
<span class="source-line-no">823</span><span id="line-823"> */</span>
<span class="source-line-no">824</span><span id="line-824"> public void postMemStoreCompaction(HStore store) throws IOException {</span>
<span class="source-line-no">825</span><span id="line-825"> execOperation(coprocEnvironments.isEmpty() ? null : new RegionObserverOperationWithoutResult() {</span>
<span class="source-line-no">826</span><span id="line-826"> @Override</span>
<span class="source-line-no">827</span><span id="line-827"> public void call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">828</span><span id="line-828"> observer.postMemStoreCompaction(this, store);</span>
<span class="source-line-no">829</span><span id="line-829"> }</span>
<span class="source-line-no">830</span><span id="line-830"> });</span>
<span class="source-line-no">831</span><span id="line-831"> }</span>
<span class="source-line-no">832</span><span id="line-832"></span>
<span class="source-line-no">833</span><span id="line-833"> /**</span>
<span class="source-line-no">834</span><span id="line-834"> * Invoked after a memstore flush</span>
<span class="source-line-no">835</span><span id="line-835"> */</span>
<span class="source-line-no">836</span><span id="line-836"> public void postFlush(HStore store, HStoreFile storeFile, FlushLifeCycleTracker tracker)</span>
<span class="source-line-no">837</span><span id="line-837"> throws IOException {</span>
<span class="source-line-no">838</span><span id="line-838"> if (coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">839</span><span id="line-839"> return;</span>
<span class="source-line-no">840</span><span id="line-840"> }</span>
<span class="source-line-no">841</span><span id="line-841"> execOperation(new RegionObserverOperationWithoutResult() {</span>
<span class="source-line-no">842</span><span id="line-842"> @Override</span>
<span class="source-line-no">843</span><span id="line-843"> public void call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">844</span><span id="line-844"> observer.postFlush(this, store, storeFile, tracker);</span>
<span class="source-line-no">845</span><span id="line-845"> }</span>
<span class="source-line-no">846</span><span id="line-846"> });</span>
<span class="source-line-no">847</span><span id="line-847"> }</span>
<span class="source-line-no">848</span><span id="line-848"></span>
<span class="source-line-no">849</span><span id="line-849"> // RegionObserver support</span>
<span class="source-line-no">850</span><span id="line-850"> /**</span>
<span class="source-line-no">851</span><span id="line-851"> * Supports Coprocessor 'bypass'.</span>
<span class="source-line-no">852</span><span id="line-852"> * @param get the Get request</span>
<span class="source-line-no">853</span><span id="line-853"> * @param results What to return if return is true/'bypass'.</span>
<span class="source-line-no">854</span><span id="line-854"> * @return true if default processing should be bypassed.</span>
<span class="source-line-no">855</span><span id="line-855"> * @exception IOException Exception</span>
<span class="source-line-no">856</span><span id="line-856"> */</span>
<span class="source-line-no">857</span><span id="line-857"> public boolean preGet(final Get get, final List&lt;Cell&gt; results) throws IOException {</span>
<span class="source-line-no">858</span><span id="line-858"> if (coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">859</span><span id="line-859"> return false;</span>
<span class="source-line-no">860</span><span id="line-860"> }</span>
<span class="source-line-no">861</span><span id="line-861"> boolean bypassable = true;</span>
<span class="source-line-no">862</span><span id="line-862"> return execOperation(new RegionObserverOperationWithoutResult(bypassable) {</span>
<span class="source-line-no">863</span><span id="line-863"> @Override</span>
<span class="source-line-no">864</span><span id="line-864"> public void call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">865</span><span id="line-865"> observer.preGetOp(this, get, results);</span>
<span class="source-line-no">866</span><span id="line-866"> }</span>
<span class="source-line-no">867</span><span id="line-867"> });</span>
<span class="source-line-no">868</span><span id="line-868"> }</span>
<span class="source-line-no">869</span><span id="line-869"></span>
<span class="source-line-no">870</span><span id="line-870"> /**</span>
<span class="source-line-no">871</span><span id="line-871"> * @param get the Get request</span>
<span class="source-line-no">872</span><span id="line-872"> * @param results the result set</span>
<span class="source-line-no">873</span><span id="line-873"> * @exception IOException Exception</span>
<span class="source-line-no">874</span><span id="line-874"> */</span>
<span class="source-line-no">875</span><span id="line-875"> public void postGet(final Get get, final List&lt;Cell&gt; results) throws IOException {</span>
<span class="source-line-no">876</span><span id="line-876"> if (coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">877</span><span id="line-877"> return;</span>
<span class="source-line-no">878</span><span id="line-878"> }</span>
<span class="source-line-no">879</span><span id="line-879"> execOperation(new RegionObserverOperationWithoutResult() {</span>
<span class="source-line-no">880</span><span id="line-880"> @Override</span>
<span class="source-line-no">881</span><span id="line-881"> public void call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">882</span><span id="line-882"> observer.postGetOp(this, get, results);</span>
<span class="source-line-no">883</span><span id="line-883"> }</span>
<span class="source-line-no">884</span><span id="line-884"> });</span>
<span class="source-line-no">885</span><span id="line-885"> }</span>
<span class="source-line-no">886</span><span id="line-886"></span>
<span class="source-line-no">887</span><span id="line-887"> /**</span>
<span class="source-line-no">888</span><span id="line-888"> * Supports Coprocessor 'bypass'.</span>
<span class="source-line-no">889</span><span id="line-889"> * @param get the Get request</span>
<span class="source-line-no">890</span><span id="line-890"> * @return true or false to return to client if bypassing normal operation, or null otherwise</span>
<span class="source-line-no">891</span><span id="line-891"> * @exception IOException Exception</span>
<span class="source-line-no">892</span><span id="line-892"> */</span>
<span class="source-line-no">893</span><span id="line-893"> public Boolean preExists(final Get get) throws IOException {</span>
<span class="source-line-no">894</span><span id="line-894"> boolean bypassable = true;</span>
<span class="source-line-no">895</span><span id="line-895"> boolean defaultResult = false;</span>
<span class="source-line-no">896</span><span id="line-896"> if (coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">897</span><span id="line-897"> return null;</span>
<span class="source-line-no">898</span><span id="line-898"> }</span>
<span class="source-line-no">899</span><span id="line-899"> return execOperationWithResult(new ObserverOperationWithResult&lt;RegionObserver, Boolean&gt;(</span>
<span class="source-line-no">900</span><span id="line-900"> regionObserverGetter, defaultResult, bypassable) {</span>
<span class="source-line-no">901</span><span id="line-901"> @Override</span>
<span class="source-line-no">902</span><span id="line-902"> public Boolean call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">903</span><span id="line-903"> return observer.preExists(this, get, getResult());</span>
<span class="source-line-no">904</span><span id="line-904"> }</span>
<span class="source-line-no">905</span><span id="line-905"> });</span>
<span class="source-line-no">906</span><span id="line-906"> }</span>
<span class="source-line-no">907</span><span id="line-907"></span>
<span class="source-line-no">908</span><span id="line-908"> /**</span>
<span class="source-line-no">909</span><span id="line-909"> * @param get the Get request</span>
<span class="source-line-no">910</span><span id="line-910"> * @param result the result returned by the region server</span>
<span class="source-line-no">911</span><span id="line-911"> * @return the result to return to the client</span>
<span class="source-line-no">912</span><span id="line-912"> * @exception IOException Exception</span>
<span class="source-line-no">913</span><span id="line-913"> */</span>
<span class="source-line-no">914</span><span id="line-914"> public boolean postExists(final Get get, boolean result) throws IOException {</span>
<span class="source-line-no">915</span><span id="line-915"> if (this.coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">916</span><span id="line-916"> return result;</span>
<span class="source-line-no">917</span><span id="line-917"> }</span>
<span class="source-line-no">918</span><span id="line-918"> return execOperationWithResult(</span>
<span class="source-line-no">919</span><span id="line-919"> new ObserverOperationWithResult&lt;RegionObserver, Boolean&gt;(regionObserverGetter, result) {</span>
<span class="source-line-no">920</span><span id="line-920"> @Override</span>
<span class="source-line-no">921</span><span id="line-921"> public Boolean call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">922</span><span id="line-922"> return observer.postExists(this, get, getResult());</span>
<span class="source-line-no">923</span><span id="line-923"> }</span>
<span class="source-line-no">924</span><span id="line-924"> });</span>
<span class="source-line-no">925</span><span id="line-925"> }</span>
<span class="source-line-no">926</span><span id="line-926"></span>
<span class="source-line-no">927</span><span id="line-927"> /**</span>
<span class="source-line-no">928</span><span id="line-928"> * Supports Coprocessor 'bypass'.</span>
<span class="source-line-no">929</span><span id="line-929"> * @param put The Put object</span>
<span class="source-line-no">930</span><span id="line-930"> * @param edit The WALEdit object.</span>
<span class="source-line-no">931</span><span id="line-931"> * @return true if default processing should be bypassed</span>
<span class="source-line-no">932</span><span id="line-932"> * @exception IOException Exception</span>
<span class="source-line-no">933</span><span id="line-933"> */</span>
<span class="source-line-no">934</span><span id="line-934"> public boolean prePut(final Put put, final WALEdit edit) throws IOException {</span>
<span class="source-line-no">935</span><span id="line-935"> if (coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">936</span><span id="line-936"> return false;</span>
<span class="source-line-no">937</span><span id="line-937"> }</span>
<span class="source-line-no">938</span><span id="line-938"> boolean bypassable = true;</span>
<span class="source-line-no">939</span><span id="line-939"> return execOperation(new RegionObserverOperationWithoutResult(bypassable) {</span>
<span class="source-line-no">940</span><span id="line-940"> @Override</span>
<span class="source-line-no">941</span><span id="line-941"> public void call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">942</span><span id="line-942"> observer.prePut(this, put, edit);</span>
<span class="source-line-no">943</span><span id="line-943"> }</span>
<span class="source-line-no">944</span><span id="line-944"> });</span>
<span class="source-line-no">945</span><span id="line-945"> }</span>
<span class="source-line-no">946</span><span id="line-946"></span>
<span class="source-line-no">947</span><span id="line-947"> /**</span>
<span class="source-line-no">948</span><span id="line-948"> * Supports Coprocessor 'bypass'.</span>
<span class="source-line-no">949</span><span id="line-949"> * @param mutation - the current mutation</span>
<span class="source-line-no">950</span><span id="line-950"> * @param kv - the current cell</span>
<span class="source-line-no">951</span><span id="line-951"> * @param byteNow - current timestamp in bytes</span>
<span class="source-line-no">952</span><span id="line-952"> * @param get - the get that could be used Note that the get only does not specify the family</span>
<span class="source-line-no">953</span><span id="line-953"> * and qualifier that should be used</span>
<span class="source-line-no">954</span><span id="line-954"> * @return true if default processing should be bypassed</span>
<span class="source-line-no">955</span><span id="line-955"> * @deprecated In hbase-2.0.0. Will be removed in hbase-4.0.0. Added explicitly for a single</span>
<span class="source-line-no">956</span><span id="line-956"> * Coprocessor for its needs only. Will be removed. VisibilityController still needs</span>
<span class="source-line-no">957</span><span id="line-957"> * this, need to change the logic there first.</span>
<span class="source-line-no">958</span><span id="line-958"> */</span>
<span class="source-line-no">959</span><span id="line-959"> @Deprecated</span>
<span class="source-line-no">960</span><span id="line-960"> public boolean prePrepareTimeStampForDeleteVersion(final Mutation mutation, final Cell kv,</span>
<span class="source-line-no">961</span><span id="line-961"> final byte[] byteNow, final Get get) throws IOException {</span>
<span class="source-line-no">962</span><span id="line-962"> if (coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">963</span><span id="line-963"> return false;</span>
<span class="source-line-no">964</span><span id="line-964"> }</span>
<span class="source-line-no">965</span><span id="line-965"> boolean bypassable = true;</span>
<span class="source-line-no">966</span><span id="line-966"> return execOperation(new RegionObserverOperationWithoutResult(bypassable) {</span>
<span class="source-line-no">967</span><span id="line-967"> @Override</span>
<span class="source-line-no">968</span><span id="line-968"> public void call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">969</span><span id="line-969"> observer.prePrepareTimeStampForDeleteVersion(this, mutation, kv, byteNow, get);</span>
<span class="source-line-no">970</span><span id="line-970"> }</span>
<span class="source-line-no">971</span><span id="line-971"> });</span>
<span class="source-line-no">972</span><span id="line-972"> }</span>
<span class="source-line-no">973</span><span id="line-973"></span>
<span class="source-line-no">974</span><span id="line-974"> /**</span>
<span class="source-line-no">975</span><span id="line-975"> * @param put The Put object</span>
<span class="source-line-no">976</span><span id="line-976"> * @param edit The WALEdit object.</span>
<span class="source-line-no">977</span><span id="line-977"> * @exception IOException Exception</span>
<span class="source-line-no">978</span><span id="line-978"> */</span>
<span class="source-line-no">979</span><span id="line-979"> public void postPut(final Put put, final WALEdit edit) throws IOException {</span>
<span class="source-line-no">980</span><span id="line-980"> if (coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">981</span><span id="line-981"> return;</span>
<span class="source-line-no">982</span><span id="line-982"> }</span>
<span class="source-line-no">983</span><span id="line-983"> execOperation(new RegionObserverOperationWithoutResult() {</span>
<span class="source-line-no">984</span><span id="line-984"> @Override</span>
<span class="source-line-no">985</span><span id="line-985"> public void call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">986</span><span id="line-986"> observer.postPut(this, put, edit);</span>
<span class="source-line-no">987</span><span id="line-987"> }</span>
<span class="source-line-no">988</span><span id="line-988"> });</span>
<span class="source-line-no">989</span><span id="line-989"> }</span>
<span class="source-line-no">990</span><span id="line-990"></span>
<span class="source-line-no">991</span><span id="line-991"> /**</span>
<span class="source-line-no">992</span><span id="line-992"> * Supports Coprocessor 'bypass'.</span>
<span class="source-line-no">993</span><span id="line-993"> * @param delete The Delete object</span>
<span class="source-line-no">994</span><span id="line-994"> * @param edit The WALEdit object.</span>
<span class="source-line-no">995</span><span id="line-995"> * @return true if default processing should be bypassed</span>
<span class="source-line-no">996</span><span id="line-996"> * @exception IOException Exception</span>
<span class="source-line-no">997</span><span id="line-997"> */</span>
<span class="source-line-no">998</span><span id="line-998"> public boolean preDelete(final Delete delete, final WALEdit edit) throws IOException {</span>
<span class="source-line-no">999</span><span id="line-999"> if (this.coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">1000</span><span id="line-1000"> return false;</span>
<span class="source-line-no">1001</span><span id="line-1001"> }</span>
<span class="source-line-no">1002</span><span id="line-1002"> boolean bypassable = true;</span>
<span class="source-line-no">1003</span><span id="line-1003"> return execOperation(new RegionObserverOperationWithoutResult(bypassable) {</span>
<span class="source-line-no">1004</span><span id="line-1004"> @Override</span>
<span class="source-line-no">1005</span><span id="line-1005"> public void call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">1006</span><span id="line-1006"> observer.preDelete(this, delete, edit);</span>
<span class="source-line-no">1007</span><span id="line-1007"> }</span>
<span class="source-line-no">1008</span><span id="line-1008"> });</span>
<span class="source-line-no">1009</span><span id="line-1009"> }</span>
<span class="source-line-no">1010</span><span id="line-1010"></span>
<span class="source-line-no">1011</span><span id="line-1011"> /**</span>
<span class="source-line-no">1012</span><span id="line-1012"> * @param delete The Delete object</span>
<span class="source-line-no">1013</span><span id="line-1013"> * @param edit The WALEdit object.</span>
<span class="source-line-no">1014</span><span id="line-1014"> * @exception IOException Exception</span>
<span class="source-line-no">1015</span><span id="line-1015"> */</span>
<span class="source-line-no">1016</span><span id="line-1016"> public void postDelete(final Delete delete, final WALEdit edit) throws IOException {</span>
<span class="source-line-no">1017</span><span id="line-1017"> execOperation(coprocEnvironments.isEmpty() ? null : new RegionObserverOperationWithoutResult() {</span>
<span class="source-line-no">1018</span><span id="line-1018"> @Override</span>
<span class="source-line-no">1019</span><span id="line-1019"> public void call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">1020</span><span id="line-1020"> observer.postDelete(this, delete, edit);</span>
<span class="source-line-no">1021</span><span id="line-1021"> }</span>
<span class="source-line-no">1022</span><span id="line-1022"> });</span>
<span class="source-line-no">1023</span><span id="line-1023"> }</span>
<span class="source-line-no">1024</span><span id="line-1024"></span>
<span class="source-line-no">1025</span><span id="line-1025"> public void preBatchMutate(final MiniBatchOperationInProgress&lt;Mutation&gt; miniBatchOp)</span>
<span class="source-line-no">1026</span><span id="line-1026"> throws IOException {</span>
<span class="source-line-no">1027</span><span id="line-1027"> if (this.coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">1028</span><span id="line-1028"> return;</span>
<span class="source-line-no">1029</span><span id="line-1029"> }</span>
<span class="source-line-no">1030</span><span id="line-1030"> execOperation(new RegionObserverOperationWithoutResult() {</span>
<span class="source-line-no">1031</span><span id="line-1031"> @Override</span>
<span class="source-line-no">1032</span><span id="line-1032"> public void call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">1033</span><span id="line-1033"> observer.preBatchMutate(this, miniBatchOp);</span>
<span class="source-line-no">1034</span><span id="line-1034"> }</span>
<span class="source-line-no">1035</span><span id="line-1035"> });</span>
<span class="source-line-no">1036</span><span id="line-1036"> }</span>
<span class="source-line-no">1037</span><span id="line-1037"></span>
<span class="source-line-no">1038</span><span id="line-1038"> public void postBatchMutate(final MiniBatchOperationInProgress&lt;Mutation&gt; miniBatchOp)</span>
<span class="source-line-no">1039</span><span id="line-1039"> throws IOException {</span>
<span class="source-line-no">1040</span><span id="line-1040"> if (this.coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">1041</span><span id="line-1041"> return;</span>
<span class="source-line-no">1042</span><span id="line-1042"> }</span>
<span class="source-line-no">1043</span><span id="line-1043"> execOperation(new RegionObserverOperationWithoutResult() {</span>
<span class="source-line-no">1044</span><span id="line-1044"> @Override</span>
<span class="source-line-no">1045</span><span id="line-1045"> public void call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">1046</span><span id="line-1046"> observer.postBatchMutate(this, miniBatchOp);</span>
<span class="source-line-no">1047</span><span id="line-1047"> }</span>
<span class="source-line-no">1048</span><span id="line-1048"> });</span>
<span class="source-line-no">1049</span><span id="line-1049"> }</span>
<span class="source-line-no">1050</span><span id="line-1050"></span>
<span class="source-line-no">1051</span><span id="line-1051"> public void postBatchMutateIndispensably(final MiniBatchOperationInProgress&lt;Mutation&gt; miniBatchOp,</span>
<span class="source-line-no">1052</span><span id="line-1052"> final boolean success) throws IOException {</span>
<span class="source-line-no">1053</span><span id="line-1053"> if (this.coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">1054</span><span id="line-1054"> return;</span>
<span class="source-line-no">1055</span><span id="line-1055"> }</span>
<span class="source-line-no">1056</span><span id="line-1056"> execOperation(new RegionObserverOperationWithoutResult() {</span>
<span class="source-line-no">1057</span><span id="line-1057"> @Override</span>
<span class="source-line-no">1058</span><span id="line-1058"> public void call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">1059</span><span id="line-1059"> observer.postBatchMutateIndispensably(this, miniBatchOp, success);</span>
<span class="source-line-no">1060</span><span id="line-1060"> }</span>
<span class="source-line-no">1061</span><span id="line-1061"> });</span>
<span class="source-line-no">1062</span><span id="line-1062"> }</span>
<span class="source-line-no">1063</span><span id="line-1063"></span>
<span class="source-line-no">1064</span><span id="line-1064"> /**</span>
<span class="source-line-no">1065</span><span id="line-1065"> * Supports Coprocessor 'bypass'.</span>
<span class="source-line-no">1066</span><span id="line-1066"> * @param checkAndMutate the CheckAndMutate object</span>
<span class="source-line-no">1067</span><span id="line-1067"> * @return true or false to return to client if default processing should be bypassed, or null</span>
<span class="source-line-no">1068</span><span id="line-1068"> * otherwise</span>
<span class="source-line-no">1069</span><span id="line-1069"> * @throws IOException if an error occurred on the coprocessor</span>
<span class="source-line-no">1070</span><span id="line-1070"> */</span>
<span class="source-line-no">1071</span><span id="line-1071"> public CheckAndMutateResult preCheckAndMutate(CheckAndMutate checkAndMutate) throws IOException {</span>
<span class="source-line-no">1072</span><span id="line-1072"> boolean bypassable = true;</span>
<span class="source-line-no">1073</span><span id="line-1073"> CheckAndMutateResult defaultResult = new CheckAndMutateResult(false, null);</span>
<span class="source-line-no">1074</span><span id="line-1074"> if (coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">1075</span><span id="line-1075"> return null;</span>
<span class="source-line-no">1076</span><span id="line-1076"> }</span>
<span class="source-line-no">1077</span><span id="line-1077"> return execOperationWithResult(</span>
<span class="source-line-no">1078</span><span id="line-1078"> new ObserverOperationWithResult&lt;RegionObserver, CheckAndMutateResult&gt;(regionObserverGetter,</span>
<span class="source-line-no">1079</span><span id="line-1079"> defaultResult, bypassable) {</span>
<span class="source-line-no">1080</span><span id="line-1080"> @Override</span>
<span class="source-line-no">1081</span><span id="line-1081"> public CheckAndMutateResult call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">1082</span><span id="line-1082"> return observer.preCheckAndMutate(this, checkAndMutate, getResult());</span>
<span class="source-line-no">1083</span><span id="line-1083"> }</span>
<span class="source-line-no">1084</span><span id="line-1084"> });</span>
<span class="source-line-no">1085</span><span id="line-1085"> }</span>
<span class="source-line-no">1086</span><span id="line-1086"></span>
<span class="source-line-no">1087</span><span id="line-1087"> /**</span>
<span class="source-line-no">1088</span><span id="line-1088"> * Supports Coprocessor 'bypass'.</span>
<span class="source-line-no">1089</span><span id="line-1089"> * @param checkAndMutate the CheckAndMutate object</span>
<span class="source-line-no">1090</span><span id="line-1090"> * @return true or false to return to client if default processing should be bypassed, or null</span>
<span class="source-line-no">1091</span><span id="line-1091"> * otherwise</span>
<span class="source-line-no">1092</span><span id="line-1092"> * @throws IOException if an error occurred on the coprocessor</span>
<span class="source-line-no">1093</span><span id="line-1093"> */</span>
<span class="source-line-no">1094</span><span id="line-1094"> public CheckAndMutateResult preCheckAndMutateAfterRowLock(CheckAndMutate checkAndMutate)</span>
<span class="source-line-no">1095</span><span id="line-1095"> throws IOException {</span>
<span class="source-line-no">1096</span><span id="line-1096"> boolean bypassable = true;</span>
<span class="source-line-no">1097</span><span id="line-1097"> CheckAndMutateResult defaultResult = new CheckAndMutateResult(false, null);</span>
<span class="source-line-no">1098</span><span id="line-1098"> if (coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">1099</span><span id="line-1099"> return null;</span>
<span class="source-line-no">1100</span><span id="line-1100"> }</span>
<span class="source-line-no">1101</span><span id="line-1101"> return execOperationWithResult(</span>
<span class="source-line-no">1102</span><span id="line-1102"> new ObserverOperationWithResult&lt;RegionObserver, CheckAndMutateResult&gt;(regionObserverGetter,</span>
<span class="source-line-no">1103</span><span id="line-1103"> defaultResult, bypassable) {</span>
<span class="source-line-no">1104</span><span id="line-1104"> @Override</span>
<span class="source-line-no">1105</span><span id="line-1105"> public CheckAndMutateResult call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">1106</span><span id="line-1106"> return observer.preCheckAndMutateAfterRowLock(this, checkAndMutate, getResult());</span>
<span class="source-line-no">1107</span><span id="line-1107"> }</span>
<span class="source-line-no">1108</span><span id="line-1108"> });</span>
<span class="source-line-no">1109</span><span id="line-1109"> }</span>
<span class="source-line-no">1110</span><span id="line-1110"></span>
<span class="source-line-no">1111</span><span id="line-1111"> /**</span>
<span class="source-line-no">1112</span><span id="line-1112"> * @param checkAndMutate the CheckAndMutate object</span>
<span class="source-line-no">1113</span><span id="line-1113"> * @param result the result returned by the checkAndMutate</span>
<span class="source-line-no">1114</span><span id="line-1114"> * @return true or false to return to client if default processing should be bypassed, or null</span>
<span class="source-line-no">1115</span><span id="line-1115"> * otherwise</span>
<span class="source-line-no">1116</span><span id="line-1116"> * @throws IOException if an error occurred on the coprocessor</span>
<span class="source-line-no">1117</span><span id="line-1117"> */</span>
<span class="source-line-no">1118</span><span id="line-1118"> public CheckAndMutateResult postCheckAndMutate(CheckAndMutate checkAndMutate,</span>
<span class="source-line-no">1119</span><span id="line-1119"> CheckAndMutateResult result) throws IOException {</span>
<span class="source-line-no">1120</span><span id="line-1120"> if (this.coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">1121</span><span id="line-1121"> return result;</span>
<span class="source-line-no">1122</span><span id="line-1122"> }</span>
<span class="source-line-no">1123</span><span id="line-1123"> return execOperationWithResult(</span>
<span class="source-line-no">1124</span><span id="line-1124"> new ObserverOperationWithResult&lt;RegionObserver, CheckAndMutateResult&gt;(regionObserverGetter,</span>
<span class="source-line-no">1125</span><span id="line-1125"> result) {</span>
<span class="source-line-no">1126</span><span id="line-1126"> @Override</span>
<span class="source-line-no">1127</span><span id="line-1127"> public CheckAndMutateResult call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">1128</span><span id="line-1128"> return observer.postCheckAndMutate(this, checkAndMutate, getResult());</span>
<span class="source-line-no">1129</span><span id="line-1129"> }</span>
<span class="source-line-no">1130</span><span id="line-1130"> });</span>
<span class="source-line-no">1131</span><span id="line-1131"> }</span>
<span class="source-line-no">1132</span><span id="line-1132"></span>
<span class="source-line-no">1133</span><span id="line-1133"> /**</span>
<span class="source-line-no">1134</span><span id="line-1134"> * Supports Coprocessor 'bypass'.</span>
<span class="source-line-no">1135</span><span id="line-1135"> * @param append append object</span>
<span class="source-line-no">1136</span><span id="line-1136"> * @param edit The WALEdit object.</span>
<span class="source-line-no">1137</span><span id="line-1137"> * @return result to return to client if default operation should be bypassed, null otherwise</span>
<span class="source-line-no">1138</span><span id="line-1138"> * @throws IOException if an error occurred on the coprocessor</span>
<span class="source-line-no">1139</span><span id="line-1139"> */</span>
<span class="source-line-no">1140</span><span id="line-1140"> public Result preAppend(final Append append, final WALEdit edit) throws IOException {</span>
<span class="source-line-no">1141</span><span id="line-1141"> boolean bypassable = true;</span>
<span class="source-line-no">1142</span><span id="line-1142"> Result defaultResult = null;</span>
<span class="source-line-no">1143</span><span id="line-1143"> if (this.coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">1144</span><span id="line-1144"> return defaultResult;</span>
<span class="source-line-no">1145</span><span id="line-1145"> }</span>
<span class="source-line-no">1146</span><span id="line-1146"> return execOperationWithResult(new ObserverOperationWithResult&lt;RegionObserver, Result&gt;(</span>
<span class="source-line-no">1147</span><span id="line-1147"> regionObserverGetter, defaultResult, bypassable) {</span>
<span class="source-line-no">1148</span><span id="line-1148"> @Override</span>
<span class="source-line-no">1149</span><span id="line-1149"> public Result call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">1150</span><span id="line-1150"> return observer.preAppend(this, append, edit);</span>
<span class="source-line-no">1151</span><span id="line-1151"> }</span>
<span class="source-line-no">1152</span><span id="line-1152"> });</span>
<span class="source-line-no">1153</span><span id="line-1153"> }</span>
<span class="source-line-no">1154</span><span id="line-1154"></span>
<span class="source-line-no">1155</span><span id="line-1155"> /**</span>
<span class="source-line-no">1156</span><span id="line-1156"> * Supports Coprocessor 'bypass'.</span>
<span class="source-line-no">1157</span><span id="line-1157"> * @param append append object</span>
<span class="source-line-no">1158</span><span id="line-1158"> * @return result to return to client if default operation should be bypassed, null otherwise</span>
<span class="source-line-no">1159</span><span id="line-1159"> * @throws IOException if an error occurred on the coprocessor</span>
<span class="source-line-no">1160</span><span id="line-1160"> */</span>
<span class="source-line-no">1161</span><span id="line-1161"> public Result preAppendAfterRowLock(final Append append) throws IOException {</span>
<span class="source-line-no">1162</span><span id="line-1162"> boolean bypassable = true;</span>
<span class="source-line-no">1163</span><span id="line-1163"> Result defaultResult = null;</span>
<span class="source-line-no">1164</span><span id="line-1164"> if (this.coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">1165</span><span id="line-1165"> return defaultResult;</span>
<span class="source-line-no">1166</span><span id="line-1166"> }</span>
<span class="source-line-no">1167</span><span id="line-1167"> return execOperationWithResult(new ObserverOperationWithResult&lt;RegionObserver, Result&gt;(</span>
<span class="source-line-no">1168</span><span id="line-1168"> regionObserverGetter, defaultResult, bypassable) {</span>
<span class="source-line-no">1169</span><span id="line-1169"> @Override</span>
<span class="source-line-no">1170</span><span id="line-1170"> public Result call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">1171</span><span id="line-1171"> return observer.preAppendAfterRowLock(this, append);</span>
<span class="source-line-no">1172</span><span id="line-1172"> }</span>
<span class="source-line-no">1173</span><span id="line-1173"> });</span>
<span class="source-line-no">1174</span><span id="line-1174"> }</span>
<span class="source-line-no">1175</span><span id="line-1175"></span>
<span class="source-line-no">1176</span><span id="line-1176"> /**</span>
<span class="source-line-no">1177</span><span id="line-1177"> * Supports Coprocessor 'bypass'.</span>
<span class="source-line-no">1178</span><span id="line-1178"> * @param increment increment object</span>
<span class="source-line-no">1179</span><span id="line-1179"> * @param edit The WALEdit object.</span>
<span class="source-line-no">1180</span><span id="line-1180"> * @return result to return to client if default operation should be bypassed, null otherwise</span>
<span class="source-line-no">1181</span><span id="line-1181"> * @throws IOException if an error occurred on the coprocessor</span>
<span class="source-line-no">1182</span><span id="line-1182"> */</span>
<span class="source-line-no">1183</span><span id="line-1183"> public Result preIncrement(final Increment increment, final WALEdit edit) throws IOException {</span>
<span class="source-line-no">1184</span><span id="line-1184"> boolean bypassable = true;</span>
<span class="source-line-no">1185</span><span id="line-1185"> Result defaultResult = null;</span>
<span class="source-line-no">1186</span><span id="line-1186"> if (coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">1187</span><span id="line-1187"> return defaultResult;</span>
<span class="source-line-no">1188</span><span id="line-1188"> }</span>
<span class="source-line-no">1189</span><span id="line-1189"> return execOperationWithResult(new ObserverOperationWithResult&lt;RegionObserver, Result&gt;(</span>
<span class="source-line-no">1190</span><span id="line-1190"> regionObserverGetter, defaultResult, bypassable) {</span>
<span class="source-line-no">1191</span><span id="line-1191"> @Override</span>
<span class="source-line-no">1192</span><span id="line-1192"> public Result call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">1193</span><span id="line-1193"> return observer.preIncrement(this, increment, edit);</span>
<span class="source-line-no">1194</span><span id="line-1194"> }</span>
<span class="source-line-no">1195</span><span id="line-1195"> });</span>
<span class="source-line-no">1196</span><span id="line-1196"> }</span>
<span class="source-line-no">1197</span><span id="line-1197"></span>
<span class="source-line-no">1198</span><span id="line-1198"> /**</span>
<span class="source-line-no">1199</span><span id="line-1199"> * Supports Coprocessor 'bypass'.</span>
<span class="source-line-no">1200</span><span id="line-1200"> * @param increment increment object</span>
<span class="source-line-no">1201</span><span id="line-1201"> * @return result to return to client if default operation should be bypassed, null otherwise</span>
<span class="source-line-no">1202</span><span id="line-1202"> * @throws IOException if an error occurred on the coprocessor</span>
<span class="source-line-no">1203</span><span id="line-1203"> */</span>
<span class="source-line-no">1204</span><span id="line-1204"> public Result preIncrementAfterRowLock(final Increment increment) throws IOException {</span>
<span class="source-line-no">1205</span><span id="line-1205"> boolean bypassable = true;</span>
<span class="source-line-no">1206</span><span id="line-1206"> Result defaultResult = null;</span>
<span class="source-line-no">1207</span><span id="line-1207"> if (coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">1208</span><span id="line-1208"> return defaultResult;</span>
<span class="source-line-no">1209</span><span id="line-1209"> }</span>
<span class="source-line-no">1210</span><span id="line-1210"> return execOperationWithResult(new ObserverOperationWithResult&lt;RegionObserver, Result&gt;(</span>
<span class="source-line-no">1211</span><span id="line-1211"> regionObserverGetter, defaultResult, bypassable) {</span>
<span class="source-line-no">1212</span><span id="line-1212"> @Override</span>
<span class="source-line-no">1213</span><span id="line-1213"> public Result call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">1214</span><span id="line-1214"> return observer.preIncrementAfterRowLock(this, increment);</span>
<span class="source-line-no">1215</span><span id="line-1215"> }</span>
<span class="source-line-no">1216</span><span id="line-1216"> });</span>
<span class="source-line-no">1217</span><span id="line-1217"> }</span>
<span class="source-line-no">1218</span><span id="line-1218"></span>
<span class="source-line-no">1219</span><span id="line-1219"> /**</span>
<span class="source-line-no">1220</span><span id="line-1220"> * @param append Append object</span>
<span class="source-line-no">1221</span><span id="line-1221"> * @param result the result returned by the append</span>
<span class="source-line-no">1222</span><span id="line-1222"> * @param edit The WALEdit object.</span>
<span class="source-line-no">1223</span><span id="line-1223"> * @throws IOException if an error occurred on the coprocessor</span>
<span class="source-line-no">1224</span><span id="line-1224"> */</span>
<span class="source-line-no">1225</span><span id="line-1225"> public Result postAppend(final Append append, final Result result, final WALEdit edit)</span>
<span class="source-line-no">1226</span><span id="line-1226"> throws IOException {</span>
<span class="source-line-no">1227</span><span id="line-1227"> if (this.coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">1228</span><span id="line-1228"> return result;</span>
<span class="source-line-no">1229</span><span id="line-1229"> }</span>
<span class="source-line-no">1230</span><span id="line-1230"> return execOperationWithResult(</span>
<span class="source-line-no">1231</span><span id="line-1231"> new ObserverOperationWithResult&lt;RegionObserver, Result&gt;(regionObserverGetter, result) {</span>
<span class="source-line-no">1232</span><span id="line-1232"> @Override</span>
<span class="source-line-no">1233</span><span id="line-1233"> public Result call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">1234</span><span id="line-1234"> return observer.postAppend(this, append, result, edit);</span>
<span class="source-line-no">1235</span><span id="line-1235"> }</span>
<span class="source-line-no">1236</span><span id="line-1236"> });</span>
<span class="source-line-no">1237</span><span id="line-1237"> }</span>
<span class="source-line-no">1238</span><span id="line-1238"></span>
<span class="source-line-no">1239</span><span id="line-1239"> /**</span>
<span class="source-line-no">1240</span><span id="line-1240"> * @param increment increment object</span>
<span class="source-line-no">1241</span><span id="line-1241"> * @param result the result returned by postIncrement</span>
<span class="source-line-no">1242</span><span id="line-1242"> * @param edit The WALEdit object.</span>
<span class="source-line-no">1243</span><span id="line-1243"> * @throws IOException if an error occurred on the coprocessor</span>
<span class="source-line-no">1244</span><span id="line-1244"> */</span>
<span class="source-line-no">1245</span><span id="line-1245"> public Result postIncrement(final Increment increment, Result result, final WALEdit edit)</span>
<span class="source-line-no">1246</span><span id="line-1246"> throws IOException {</span>
<span class="source-line-no">1247</span><span id="line-1247"> if (this.coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">1248</span><span id="line-1248"> return result;</span>
<span class="source-line-no">1249</span><span id="line-1249"> }</span>
<span class="source-line-no">1250</span><span id="line-1250"> return execOperationWithResult(</span>
<span class="source-line-no">1251</span><span id="line-1251"> new ObserverOperationWithResult&lt;RegionObserver, Result&gt;(regionObserverGetter, result) {</span>
<span class="source-line-no">1252</span><span id="line-1252"> @Override</span>
<span class="source-line-no">1253</span><span id="line-1253"> public Result call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">1254</span><span id="line-1254"> return observer.postIncrement(this, increment, getResult(), edit);</span>
<span class="source-line-no">1255</span><span id="line-1255"> }</span>
<span class="source-line-no">1256</span><span id="line-1256"> });</span>
<span class="source-line-no">1257</span><span id="line-1257"> }</span>
<span class="source-line-no">1258</span><span id="line-1258"></span>
<span class="source-line-no">1259</span><span id="line-1259"> /**</span>
<span class="source-line-no">1260</span><span id="line-1260"> * @param scan the Scan specification</span>
<span class="source-line-no">1261</span><span id="line-1261"> * @exception IOException Exception</span>
<span class="source-line-no">1262</span><span id="line-1262"> */</span>
<span class="source-line-no">1263</span><span id="line-1263"> public void preScannerOpen(final Scan scan) throws IOException {</span>
<span class="source-line-no">1264</span><span id="line-1264"> execOperation(coprocEnvironments.isEmpty() ? null : new RegionObserverOperationWithoutResult() {</span>
<span class="source-line-no">1265</span><span id="line-1265"> @Override</span>
<span class="source-line-no">1266</span><span id="line-1266"> public void call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">1267</span><span id="line-1267"> observer.preScannerOpen(this, scan);</span>
<span class="source-line-no">1268</span><span id="line-1268"> }</span>
<span class="source-line-no">1269</span><span id="line-1269"> });</span>
<span class="source-line-no">1270</span><span id="line-1270"> }</span>
<span class="source-line-no">1271</span><span id="line-1271"></span>
<span class="source-line-no">1272</span><span id="line-1272"> /**</span>
<span class="source-line-no">1273</span><span id="line-1273"> * @param scan the Scan specification</span>
<span class="source-line-no">1274</span><span id="line-1274"> * @param s the scanner</span>
<span class="source-line-no">1275</span><span id="line-1275"> * @return the scanner instance to use</span>
<span class="source-line-no">1276</span><span id="line-1276"> * @exception IOException Exception</span>
<span class="source-line-no">1277</span><span id="line-1277"> */</span>
<span class="source-line-no">1278</span><span id="line-1278"> public RegionScanner postScannerOpen(final Scan scan, RegionScanner s) throws IOException {</span>
<span class="source-line-no">1279</span><span id="line-1279"> if (this.coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">1280</span><span id="line-1280"> return s;</span>
<span class="source-line-no">1281</span><span id="line-1281"> }</span>
<span class="source-line-no">1282</span><span id="line-1282"> return execOperationWithResult(</span>
<span class="source-line-no">1283</span><span id="line-1283"> new ObserverOperationWithResult&lt;RegionObserver, RegionScanner&gt;(regionObserverGetter, s) {</span>
<span class="source-line-no">1284</span><span id="line-1284"> @Override</span>
<span class="source-line-no">1285</span><span id="line-1285"> public RegionScanner call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">1286</span><span id="line-1286"> return observer.postScannerOpen(this, scan, getResult());</span>
<span class="source-line-no">1287</span><span id="line-1287"> }</span>
<span class="source-line-no">1288</span><span id="line-1288"> });</span>
<span class="source-line-no">1289</span><span id="line-1289"> }</span>
<span class="source-line-no">1290</span><span id="line-1290"></span>
<span class="source-line-no">1291</span><span id="line-1291"> /**</span>
<span class="source-line-no">1292</span><span id="line-1292"> * @param s the scanner</span>
<span class="source-line-no">1293</span><span id="line-1293"> * @param results the result set returned by the region server</span>
<span class="source-line-no">1294</span><span id="line-1294"> * @param limit the maximum number of results to return</span>
<span class="source-line-no">1295</span><span id="line-1295"> * @return 'has next' indication to client if bypassing default behavior, or null otherwise</span>
<span class="source-line-no">1296</span><span id="line-1296"> * @exception IOException Exception</span>
<span class="source-line-no">1297</span><span id="line-1297"> */</span>
<span class="source-line-no">1298</span><span id="line-1298"> public Boolean preScannerNext(final InternalScanner s, final List&lt;Result&gt; results,</span>
<span class="source-line-no">1299</span><span id="line-1299"> final int limit) throws IOException {</span>
<span class="source-line-no">1300</span><span id="line-1300"> boolean bypassable = true;</span>
<span class="source-line-no">1301</span><span id="line-1301"> boolean defaultResult = false;</span>
<span class="source-line-no">1302</span><span id="line-1302"> if (coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">1303</span><span id="line-1303"> return null;</span>
<span class="source-line-no">1304</span><span id="line-1304"> }</span>
<span class="source-line-no">1305</span><span id="line-1305"> return execOperationWithResult(new ObserverOperationWithResult&lt;RegionObserver, Boolean&gt;(</span>
<span class="source-line-no">1306</span><span id="line-1306"> regionObserverGetter, defaultResult, bypassable) {</span>
<span class="source-line-no">1307</span><span id="line-1307"> @Override</span>
<span class="source-line-no">1308</span><span id="line-1308"> public Boolean call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">1309</span><span id="line-1309"> return observer.preScannerNext(this, s, results, limit, getResult());</span>
<span class="source-line-no">1310</span><span id="line-1310"> }</span>
<span class="source-line-no">1311</span><span id="line-1311"> });</span>
<span class="source-line-no">1312</span><span id="line-1312"> }</span>
<span class="source-line-no">1313</span><span id="line-1313"></span>
<span class="source-line-no">1314</span><span id="line-1314"> /**</span>
<span class="source-line-no">1315</span><span id="line-1315"> * @param s the scanner</span>
<span class="source-line-no">1316</span><span id="line-1316"> * @param results the result set returned by the region server</span>
<span class="source-line-no">1317</span><span id="line-1317"> * @param limit the maximum number of results to return</span>
<span class="source-line-no">1318</span><span id="line-1318"> * @return 'has more' indication to give to client</span>
<span class="source-line-no">1319</span><span id="line-1319"> * @exception IOException Exception</span>
<span class="source-line-no">1320</span><span id="line-1320"> */</span>
<span class="source-line-no">1321</span><span id="line-1321"> public boolean postScannerNext(final InternalScanner s, final List&lt;Result&gt; results,</span>
<span class="source-line-no">1322</span><span id="line-1322"> final int limit, boolean hasMore) throws IOException {</span>
<span class="source-line-no">1323</span><span id="line-1323"> if (this.coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">1324</span><span id="line-1324"> return hasMore;</span>
<span class="source-line-no">1325</span><span id="line-1325"> }</span>
<span class="source-line-no">1326</span><span id="line-1326"> return execOperationWithResult(</span>
<span class="source-line-no">1327</span><span id="line-1327"> new ObserverOperationWithResult&lt;RegionObserver, Boolean&gt;(regionObserverGetter, hasMore) {</span>
<span class="source-line-no">1328</span><span id="line-1328"> @Override</span>
<span class="source-line-no">1329</span><span id="line-1329"> public Boolean call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">1330</span><span id="line-1330"> return observer.postScannerNext(this, s, results, limit, getResult());</span>
<span class="source-line-no">1331</span><span id="line-1331"> }</span>
<span class="source-line-no">1332</span><span id="line-1332"> });</span>
<span class="source-line-no">1333</span><span id="line-1333"> }</span>
<span class="source-line-no">1334</span><span id="line-1334"></span>
<span class="source-line-no">1335</span><span id="line-1335"> /**</span>
<span class="source-line-no">1336</span><span id="line-1336"> * This will be called by the scan flow when the current scanned row is being filtered out by the</span>
<span class="source-line-no">1337</span><span id="line-1337"> * filter.</span>
<span class="source-line-no">1338</span><span id="line-1338"> * @param s the scanner</span>
<span class="source-line-no">1339</span><span id="line-1339"> * @param curRowCell The cell in the current row which got filtered out</span>
<span class="source-line-no">1340</span><span id="line-1340"> * @return whether more rows are available for the scanner or not</span>
<span class="source-line-no">1341</span><span id="line-1341"> */</span>
<span class="source-line-no">1342</span><span id="line-1342"> public boolean postScannerFilterRow(final InternalScanner s, final Cell curRowCell)</span>
<span class="source-line-no">1343</span><span id="line-1343"> throws IOException {</span>
<span class="source-line-no">1344</span><span id="line-1344"> // short circuit for performance</span>
<span class="source-line-no">1345</span><span id="line-1345"> boolean defaultResult = true;</span>
<span class="source-line-no">1346</span><span id="line-1346"> if (!hasCustomPostScannerFilterRow) {</span>
<span class="source-line-no">1347</span><span id="line-1347"> return defaultResult;</span>
<span class="source-line-no">1348</span><span id="line-1348"> }</span>
<span class="source-line-no">1349</span><span id="line-1349"> if (this.coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">1350</span><span id="line-1350"> return defaultResult;</span>
<span class="source-line-no">1351</span><span id="line-1351"> }</span>
<span class="source-line-no">1352</span><span id="line-1352"> return execOperationWithResult(new ObserverOperationWithResult&lt;RegionObserver, Boolean&gt;(</span>
<span class="source-line-no">1353</span><span id="line-1353"> regionObserverGetter, defaultResult) {</span>
<span class="source-line-no">1354</span><span id="line-1354"> @Override</span>
<span class="source-line-no">1355</span><span id="line-1355"> public Boolean call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">1356</span><span id="line-1356"> return observer.postScannerFilterRow(this, s, curRowCell, getResult());</span>
<span class="source-line-no">1357</span><span id="line-1357"> }</span>
<span class="source-line-no">1358</span><span id="line-1358"> });</span>
<span class="source-line-no">1359</span><span id="line-1359"> }</span>
<span class="source-line-no">1360</span><span id="line-1360"></span>
<span class="source-line-no">1361</span><span id="line-1361"> /**</span>
<span class="source-line-no">1362</span><span id="line-1362"> * Supports Coprocessor 'bypass'.</span>
<span class="source-line-no">1363</span><span id="line-1363"> * @param s the scanner</span>
<span class="source-line-no">1364</span><span id="line-1364"> * @return true if default behavior should be bypassed, false otherwise</span>
<span class="source-line-no">1365</span><span id="line-1365"> * @exception IOException Exception</span>
<span class="source-line-no">1366</span><span id="line-1366"> */</span>
<span class="source-line-no">1367</span><span id="line-1367"> // Should this be bypassable?</span>
<span class="source-line-no">1368</span><span id="line-1368"> public boolean preScannerClose(final InternalScanner s) throws IOException {</span>
<span class="source-line-no">1369</span><span id="line-1369"> return execOperation(</span>
<span class="source-line-no">1370</span><span id="line-1370"> coprocEnvironments.isEmpty() ? null : new RegionObserverOperationWithoutResult(true) {</span>
<span class="source-line-no">1371</span><span id="line-1371"> @Override</span>
<span class="source-line-no">1372</span><span id="line-1372"> public void call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">1373</span><span id="line-1373"> observer.preScannerClose(this, s);</span>
<span class="source-line-no">1374</span><span id="line-1374"> }</span>
<span class="source-line-no">1375</span><span id="line-1375"> });</span>
<span class="source-line-no">1376</span><span id="line-1376"> }</span>
<span class="source-line-no">1377</span><span id="line-1377"></span>
<span class="source-line-no">1378</span><span id="line-1378"> /**</span>
<span class="source-line-no">1379</span><span id="line-1379"> * @exception IOException Exception</span>
<span class="source-line-no">1380</span><span id="line-1380"> */</span>
<span class="source-line-no">1381</span><span id="line-1381"> public void postScannerClose(final InternalScanner s) throws IOException {</span>
<span class="source-line-no">1382</span><span id="line-1382"> execOperation(coprocEnvironments.isEmpty() ? null : new RegionObserverOperationWithoutResult() {</span>
<span class="source-line-no">1383</span><span id="line-1383"> @Override</span>
<span class="source-line-no">1384</span><span id="line-1384"> public void call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">1385</span><span id="line-1385"> observer.postScannerClose(this, s);</span>
<span class="source-line-no">1386</span><span id="line-1386"> }</span>
<span class="source-line-no">1387</span><span id="line-1387"> });</span>
<span class="source-line-no">1388</span><span id="line-1388"> }</span>
<span class="source-line-no">1389</span><span id="line-1389"></span>
<span class="source-line-no">1390</span><span id="line-1390"> /**</span>
<span class="source-line-no">1391</span><span id="line-1391"> * Called before open store scanner for user scan.</span>
<span class="source-line-no">1392</span><span id="line-1392"> */</span>
<span class="source-line-no">1393</span><span id="line-1393"> public ScanInfo preStoreScannerOpen(HStore store, Scan scan) throws IOException {</span>
<span class="source-line-no">1394</span><span id="line-1394"> if (coprocEnvironments.isEmpty()) return store.getScanInfo();</span>
<span class="source-line-no">1395</span><span id="line-1395"> CustomizedScanInfoBuilder builder = new CustomizedScanInfoBuilder(store.getScanInfo(), scan);</span>
<span class="source-line-no">1396</span><span id="line-1396"> execOperation(new RegionObserverOperationWithoutResult() {</span>
<span class="source-line-no">1397</span><span id="line-1397"> @Override</span>
<span class="source-line-no">1398</span><span id="line-1398"> public void call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">1399</span><span id="line-1399"> observer.preStoreScannerOpen(this, store, builder);</span>
<span class="source-line-no">1400</span><span id="line-1400"> }</span>
<span class="source-line-no">1401</span><span id="line-1401"> });</span>
<span class="source-line-no">1402</span><span id="line-1402"> return builder.build();</span>
<span class="source-line-no">1403</span><span id="line-1403"> }</span>
<span class="source-line-no">1404</span><span id="line-1404"></span>
<span class="source-line-no">1405</span><span id="line-1405"> /**</span>
<span class="source-line-no">1406</span><span id="line-1406"> * @param info the RegionInfo for this region</span>
<span class="source-line-no">1407</span><span id="line-1407"> * @param edits the file of recovered edits</span>
<span class="source-line-no">1408</span><span id="line-1408"> */</span>
<span class="source-line-no">1409</span><span id="line-1409"> public void preReplayWALs(final RegionInfo info, final Path edits) throws IOException {</span>
<span class="source-line-no">1410</span><span id="line-1410"> execOperation(</span>
<span class="source-line-no">1411</span><span id="line-1411"> coprocEnvironments.isEmpty() ? null : new RegionObserverOperationWithoutResult(true) {</span>
<span class="source-line-no">1412</span><span id="line-1412"> @Override</span>
<span class="source-line-no">1413</span><span id="line-1413"> public void call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">1414</span><span id="line-1414"> observer.preReplayWALs(this, info, edits);</span>
<span class="source-line-no">1415</span><span id="line-1415"> }</span>
<span class="source-line-no">1416</span><span id="line-1416"> });</span>
<span class="source-line-no">1417</span><span id="line-1417"> }</span>
<span class="source-line-no">1418</span><span id="line-1418"></span>
<span class="source-line-no">1419</span><span id="line-1419"> /**</span>
<span class="source-line-no">1420</span><span id="line-1420"> * @param info the RegionInfo for this region</span>
<span class="source-line-no">1421</span><span id="line-1421"> * @param edits the file of recovered edits</span>
<span class="source-line-no">1422</span><span id="line-1422"> * @throws IOException Exception</span>
<span class="source-line-no">1423</span><span id="line-1423"> */</span>
<span class="source-line-no">1424</span><span id="line-1424"> public void postReplayWALs(final RegionInfo info, final Path edits) throws IOException {</span>
<span class="source-line-no">1425</span><span id="line-1425"> execOperation(coprocEnvironments.isEmpty() ? null : new RegionObserverOperationWithoutResult() {</span>
<span class="source-line-no">1426</span><span id="line-1426"> @Override</span>
<span class="source-line-no">1427</span><span id="line-1427"> public void call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">1428</span><span id="line-1428"> observer.postReplayWALs(this, info, edits);</span>
<span class="source-line-no">1429</span><span id="line-1429"> }</span>
<span class="source-line-no">1430</span><span id="line-1430"> });</span>
<span class="source-line-no">1431</span><span id="line-1431"> }</span>
<span class="source-line-no">1432</span><span id="line-1432"></span>
<span class="source-line-no">1433</span><span id="line-1433"> /**</span>
<span class="source-line-no">1434</span><span id="line-1434"> * @param familyPaths pairs of { CF, file path } submitted for bulk load</span>
<span class="source-line-no">1435</span><span id="line-1435"> */</span>
<span class="source-line-no">1436</span><span id="line-1436"> public void preBulkLoadHFile(final List&lt;Pair&lt;byte[], String&gt;&gt; familyPaths) throws IOException {</span>
<span class="source-line-no">1437</span><span id="line-1437"> execOperation(coprocEnvironments.isEmpty() ? null : new RegionObserverOperationWithoutResult() {</span>
<span class="source-line-no">1438</span><span id="line-1438"> @Override</span>
<span class="source-line-no">1439</span><span id="line-1439"> public void call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">1440</span><span id="line-1440"> observer.preBulkLoadHFile(this, familyPaths);</span>
<span class="source-line-no">1441</span><span id="line-1441"> }</span>
<span class="source-line-no">1442</span><span id="line-1442"> });</span>
<span class="source-line-no">1443</span><span id="line-1443"> }</span>
<span class="source-line-no">1444</span><span id="line-1444"></span>
<span class="source-line-no">1445</span><span id="line-1445"> public boolean preCommitStoreFile(final byte[] family, final List&lt;Pair&lt;Path, Path&gt;&gt; pairs)</span>
<span class="source-line-no">1446</span><span id="line-1446"> throws IOException {</span>
<span class="source-line-no">1447</span><span id="line-1447"> return execOperation(</span>
<span class="source-line-no">1448</span><span id="line-1448"> coprocEnvironments.isEmpty() ? null : new RegionObserverOperationWithoutResult() {</span>
<span class="source-line-no">1449</span><span id="line-1449"> @Override</span>
<span class="source-line-no">1450</span><span id="line-1450"> public void call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">1451</span><span id="line-1451"> observer.preCommitStoreFile(this, family, pairs);</span>
<span class="source-line-no">1452</span><span id="line-1452"> }</span>
<span class="source-line-no">1453</span><span id="line-1453"> });</span>
<span class="source-line-no">1454</span><span id="line-1454"> }</span>
<span class="source-line-no">1455</span><span id="line-1455"></span>
<span class="source-line-no">1456</span><span id="line-1456"> public void postCommitStoreFile(final byte[] family, Path srcPath, Path dstPath)</span>
<span class="source-line-no">1457</span><span id="line-1457"> throws IOException {</span>
<span class="source-line-no">1458</span><span id="line-1458"> execOperation(coprocEnvironments.isEmpty() ? null : new RegionObserverOperationWithoutResult() {</span>
<span class="source-line-no">1459</span><span id="line-1459"> @Override</span>
<span class="source-line-no">1460</span><span id="line-1460"> public void call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">1461</span><span id="line-1461"> observer.postCommitStoreFile(this, family, srcPath, dstPath);</span>
<span class="source-line-no">1462</span><span id="line-1462"> }</span>
<span class="source-line-no">1463</span><span id="line-1463"> });</span>
<span class="source-line-no">1464</span><span id="line-1464"> }</span>
<span class="source-line-no">1465</span><span id="line-1465"></span>
<span class="source-line-no">1466</span><span id="line-1466"> /**</span>
<span class="source-line-no">1467</span><span id="line-1467"> * @param familyPaths pairs of { CF, file path } submitted for bulk load</span>
<span class="source-line-no">1468</span><span id="line-1468"> * @param map Map of CF to List of file paths for the final loaded files</span>
<span class="source-line-no">1469</span><span id="line-1469"> */</span>
<span class="source-line-no">1470</span><span id="line-1470"> public void postBulkLoadHFile(final List&lt;Pair&lt;byte[], String&gt;&gt; familyPaths,</span>
<span class="source-line-no">1471</span><span id="line-1471"> Map&lt;byte[], List&lt;Path&gt;&gt; map) throws IOException {</span>
<span class="source-line-no">1472</span><span id="line-1472"> if (this.coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">1473</span><span id="line-1473"> return;</span>
<span class="source-line-no">1474</span><span id="line-1474"> }</span>
<span class="source-line-no">1475</span><span id="line-1475"> execOperation(coprocEnvironments.isEmpty() ? null : new RegionObserverOperationWithoutResult() {</span>
<span class="source-line-no">1476</span><span id="line-1476"> @Override</span>
<span class="source-line-no">1477</span><span id="line-1477"> public void call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">1478</span><span id="line-1478"> observer.postBulkLoadHFile(this, familyPaths, map);</span>
<span class="source-line-no">1479</span><span id="line-1479"> }</span>
<span class="source-line-no">1480</span><span id="line-1480"> });</span>
<span class="source-line-no">1481</span><span id="line-1481"> }</span>
<span class="source-line-no">1482</span><span id="line-1482"></span>
<span class="source-line-no">1483</span><span id="line-1483"> public void postStartRegionOperation(final Operation op) throws IOException {</span>
<span class="source-line-no">1484</span><span id="line-1484"> execOperation(coprocEnvironments.isEmpty() ? null : new RegionObserverOperationWithoutResult() {</span>
<span class="source-line-no">1485</span><span id="line-1485"> @Override</span>
<span class="source-line-no">1486</span><span id="line-1486"> public void call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">1487</span><span id="line-1487"> observer.postStartRegionOperation(this, op);</span>
<span class="source-line-no">1488</span><span id="line-1488"> }</span>
<span class="source-line-no">1489</span><span id="line-1489"> });</span>
<span class="source-line-no">1490</span><span id="line-1490"> }</span>
<span class="source-line-no">1491</span><span id="line-1491"></span>
<span class="source-line-no">1492</span><span id="line-1492"> public void postCloseRegionOperation(final Operation op) throws IOException {</span>
<span class="source-line-no">1493</span><span id="line-1493"> execOperation(coprocEnvironments.isEmpty() ? null : new RegionObserverOperationWithoutResult() {</span>
<span class="source-line-no">1494</span><span id="line-1494"> @Override</span>
<span class="source-line-no">1495</span><span id="line-1495"> public void call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">1496</span><span id="line-1496"> observer.postCloseRegionOperation(this, op);</span>
<span class="source-line-no">1497</span><span id="line-1497"> }</span>
<span class="source-line-no">1498</span><span id="line-1498"> });</span>
<span class="source-line-no">1499</span><span id="line-1499"> }</span>
<span class="source-line-no">1500</span><span id="line-1500"></span>
<span class="source-line-no">1501</span><span id="line-1501"> /**</span>
<span class="source-line-no">1502</span><span id="line-1502"> * @param fs fileystem to read from</span>
<span class="source-line-no">1503</span><span id="line-1503"> * @param p path to the file</span>
<span class="source-line-no">1504</span><span id="line-1504"> * @param in {@link FSDataInputStreamWrapper}</span>
<span class="source-line-no">1505</span><span id="line-1505"> * @param size Full size of the file</span>
<span class="source-line-no">1506</span><span id="line-1506"> * @param r original reference file. This will be not null only when reading a split file.</span>
<span class="source-line-no">1507</span><span id="line-1507"> * @return a Reader instance to use instead of the base reader if overriding default behavior,</span>
<span class="source-line-no">1508</span><span id="line-1508"> * null otherwise</span>
<span class="source-line-no">1509</span><span id="line-1509"> */</span>
<span class="source-line-no">1510</span><span id="line-1510"> public StoreFileReader preStoreFileReaderOpen(final FileSystem fs, final Path p,</span>
<span class="source-line-no">1511</span><span id="line-1511"> final FSDataInputStreamWrapper in, final long size, final CacheConfig cacheConf,</span>
<span class="source-line-no">1512</span><span id="line-1512"> final Reference r) throws IOException {</span>
<span class="source-line-no">1513</span><span id="line-1513"> if (coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">1514</span><span id="line-1514"> return null;</span>
<span class="source-line-no">1515</span><span id="line-1515"> }</span>
<span class="source-line-no">1516</span><span id="line-1516"> return execOperationWithResult(</span>
<span class="source-line-no">1517</span><span id="line-1517"> new ObserverOperationWithResult&lt;RegionObserver, StoreFileReader&gt;(regionObserverGetter, null) {</span>
<span class="source-line-no">1518</span><span id="line-1518"> @Override</span>
<span class="source-line-no">1519</span><span id="line-1519"> public StoreFileReader call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">1520</span><span id="line-1520"> return observer.preStoreFileReaderOpen(this, fs, p, in, size, cacheConf, r, getResult());</span>
<span class="source-line-no">1521</span><span id="line-1521"> }</span>
<span class="source-line-no">1522</span><span id="line-1522"> });</span>
<span class="source-line-no">1523</span><span id="line-1523"> }</span>
<span class="source-line-no">1524</span><span id="line-1524"></span>
<span class="source-line-no">1525</span><span id="line-1525"> /**</span>
<span class="source-line-no">1526</span><span id="line-1526"> * @param fs fileystem to read from</span>
<span class="source-line-no">1527</span><span id="line-1527"> * @param p path to the file</span>
<span class="source-line-no">1528</span><span id="line-1528"> * @param in {@link FSDataInputStreamWrapper}</span>
<span class="source-line-no">1529</span><span id="line-1529"> * @param size Full size of the file</span>
<span class="source-line-no">1530</span><span id="line-1530"> * @param r original reference file. This will be not null only when reading a split file.</span>
<span class="source-line-no">1531</span><span id="line-1531"> * @param reader the base reader instance</span>
<span class="source-line-no">1532</span><span id="line-1532"> * @return The reader to use</span>
<span class="source-line-no">1533</span><span id="line-1533"> */</span>
<span class="source-line-no">1534</span><span id="line-1534"> public StoreFileReader postStoreFileReaderOpen(final FileSystem fs, final Path p,</span>
<span class="source-line-no">1535</span><span id="line-1535"> final FSDataInputStreamWrapper in, final long size, final CacheConfig cacheConf,</span>
<span class="source-line-no">1536</span><span id="line-1536"> final Reference r, final StoreFileReader reader) throws IOException {</span>
<span class="source-line-no">1537</span><span id="line-1537"> if (this.coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">1538</span><span id="line-1538"> return reader;</span>
<span class="source-line-no">1539</span><span id="line-1539"> }</span>
<span class="source-line-no">1540</span><span id="line-1540"> return execOperationWithResult(new ObserverOperationWithResult&lt;RegionObserver, StoreFileReader&gt;(</span>
<span class="source-line-no">1541</span><span id="line-1541"> regionObserverGetter, reader) {</span>
<span class="source-line-no">1542</span><span id="line-1542"> @Override</span>
<span class="source-line-no">1543</span><span id="line-1543"> public StoreFileReader call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">1544</span><span id="line-1544"> return observer.postStoreFileReaderOpen(this, fs, p, in, size, cacheConf, r, getResult());</span>
<span class="source-line-no">1545</span><span id="line-1545"> }</span>
<span class="source-line-no">1546</span><span id="line-1546"> });</span>
<span class="source-line-no">1547</span><span id="line-1547"> }</span>
<span class="source-line-no">1548</span><span id="line-1548"></span>
<span class="source-line-no">1549</span><span id="line-1549"> public List&lt;Pair&lt;Cell, Cell&gt;&gt; postIncrementBeforeWAL(final Mutation mutation,</span>
<span class="source-line-no">1550</span><span id="line-1550"> final List&lt;Pair&lt;Cell, Cell&gt;&gt; cellPairs) throws IOException {</span>
<span class="source-line-no">1551</span><span id="line-1551"> if (this.coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">1552</span><span id="line-1552"> return cellPairs;</span>
<span class="source-line-no">1553</span><span id="line-1553"> }</span>
<span class="source-line-no">1554</span><span id="line-1554"> return execOperationWithResult(</span>
<span class="source-line-no">1555</span><span id="line-1555"> new ObserverOperationWithResult&lt;RegionObserver, List&lt;Pair&lt;Cell, Cell&gt;&gt;&gt;(regionObserverGetter,</span>
<span class="source-line-no">1556</span><span id="line-1556"> cellPairs) {</span>
<span class="source-line-no">1557</span><span id="line-1557"> @Override</span>
<span class="source-line-no">1558</span><span id="line-1558"> public List&lt;Pair&lt;Cell, Cell&gt;&gt; call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">1559</span><span id="line-1559"> return observer.postIncrementBeforeWAL(this, mutation, getResult());</span>
<span class="source-line-no">1560</span><span id="line-1560"> }</span>
<span class="source-line-no">1561</span><span id="line-1561"> });</span>
<span class="source-line-no">1562</span><span id="line-1562"> }</span>
<span class="source-line-no">1563</span><span id="line-1563"></span>
<span class="source-line-no">1564</span><span id="line-1564"> public List&lt;Pair&lt;Cell, Cell&gt;&gt; postAppendBeforeWAL(final Mutation mutation,</span>
<span class="source-line-no">1565</span><span id="line-1565"> final List&lt;Pair&lt;Cell, Cell&gt;&gt; cellPairs) throws IOException {</span>
<span class="source-line-no">1566</span><span id="line-1566"> if (this.coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">1567</span><span id="line-1567"> return cellPairs;</span>
<span class="source-line-no">1568</span><span id="line-1568"> }</span>
<span class="source-line-no">1569</span><span id="line-1569"> return execOperationWithResult(</span>
<span class="source-line-no">1570</span><span id="line-1570"> new ObserverOperationWithResult&lt;RegionObserver, List&lt;Pair&lt;Cell, Cell&gt;&gt;&gt;(regionObserverGetter,</span>
<span class="source-line-no">1571</span><span id="line-1571"> cellPairs) {</span>
<span class="source-line-no">1572</span><span id="line-1572"> @Override</span>
<span class="source-line-no">1573</span><span id="line-1573"> public List&lt;Pair&lt;Cell, Cell&gt;&gt; call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">1574</span><span id="line-1574"> return observer.postAppendBeforeWAL(this, mutation, getResult());</span>
<span class="source-line-no">1575</span><span id="line-1575"> }</span>
<span class="source-line-no">1576</span><span id="line-1576"> });</span>
<span class="source-line-no">1577</span><span id="line-1577"> }</span>
<span class="source-line-no">1578</span><span id="line-1578"></span>
<span class="source-line-no">1579</span><span id="line-1579"> public void preWALAppend(WALKey key, WALEdit edit) throws IOException {</span>
<span class="source-line-no">1580</span><span id="line-1580"> if (this.coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">1581</span><span id="line-1581"> return;</span>
<span class="source-line-no">1582</span><span id="line-1582"> }</span>
<span class="source-line-no">1583</span><span id="line-1583"> execOperation(new RegionObserverOperationWithoutResult() {</span>
<span class="source-line-no">1584</span><span id="line-1584"> @Override</span>
<span class="source-line-no">1585</span><span id="line-1585"> public void call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">1586</span><span id="line-1586"> observer.preWALAppend(this, key, edit);</span>
<span class="source-line-no">1587</span><span id="line-1587"> }</span>
<span class="source-line-no">1588</span><span id="line-1588"> });</span>
<span class="source-line-no">1589</span><span id="line-1589"> }</span>
<span class="source-line-no">1590</span><span id="line-1590"></span>
<span class="source-line-no">1591</span><span id="line-1591"> public Message preEndpointInvocation(final Service service, final String methodName,</span>
<span class="source-line-no">1592</span><span id="line-1592"> Message request) throws IOException {</span>
<span class="source-line-no">1593</span><span id="line-1593"> if (coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">1594</span><span id="line-1594"> return request;</span>
<span class="source-line-no">1595</span><span id="line-1595"> }</span>
<span class="source-line-no">1596</span><span id="line-1596"> return execOperationWithResult(</span>
<span class="source-line-no">1597</span><span id="line-1597"> new ObserverOperationWithResult&lt;EndpointObserver, Message&gt;(endpointObserverGetter, request) {</span>
<span class="source-line-no">1598</span><span id="line-1598"> @Override</span>
<span class="source-line-no">1599</span><span id="line-1599"> public Message call(EndpointObserver observer) throws IOException {</span>
<span class="source-line-no">1600</span><span id="line-1600"> return observer.preEndpointInvocation(this, service, methodName, getResult());</span>
<span class="source-line-no">1601</span><span id="line-1601"> }</span>
<span class="source-line-no">1602</span><span id="line-1602"> });</span>
<span class="source-line-no">1603</span><span id="line-1603"> }</span>
<span class="source-line-no">1604</span><span id="line-1604"></span>
<span class="source-line-no">1605</span><span id="line-1605"> public void postEndpointInvocation(final Service service, final String methodName,</span>
<span class="source-line-no">1606</span><span id="line-1606"> final Message request, final Message.Builder responseBuilder) throws IOException {</span>
<span class="source-line-no">1607</span><span id="line-1607"> execOperation(coprocEnvironments.isEmpty()</span>
<span class="source-line-no">1608</span><span id="line-1608"> ? null</span>
<span class="source-line-no">1609</span><span id="line-1609"> : new ObserverOperationWithoutResult&lt;EndpointObserver&gt;(endpointObserverGetter) {</span>
<span class="source-line-no">1610</span><span id="line-1610"> @Override</span>
<span class="source-line-no">1611</span><span id="line-1611"> public void call(EndpointObserver observer) throws IOException {</span>
<span class="source-line-no">1612</span><span id="line-1612"> observer.postEndpointInvocation(this, service, methodName, request, responseBuilder);</span>
<span class="source-line-no">1613</span><span id="line-1613"> }</span>
<span class="source-line-no">1614</span><span id="line-1614"> });</span>
<span class="source-line-no">1615</span><span id="line-1615"> }</span>
<span class="source-line-no">1616</span><span id="line-1616"></span>
<span class="source-line-no">1617</span><span id="line-1617"> /**</span>
<span class="source-line-no">1618</span><span id="line-1618"> * @deprecated Since 2.0 with out any replacement and will be removed in 3.0</span>
<span class="source-line-no">1619</span><span id="line-1619"> */</span>
<span class="source-line-no">1620</span><span id="line-1620"> @Deprecated</span>
<span class="source-line-no">1621</span><span id="line-1621"> public DeleteTracker postInstantiateDeleteTracker(DeleteTracker result) throws IOException {</span>
<span class="source-line-no">1622</span><span id="line-1622"> if (this.coprocEnvironments.isEmpty()) {</span>
<span class="source-line-no">1623</span><span id="line-1623"> return result;</span>
<span class="source-line-no">1624</span><span id="line-1624"> }</span>
<span class="source-line-no">1625</span><span id="line-1625"> return execOperationWithResult(</span>
<span class="source-line-no">1626</span><span id="line-1626"> new ObserverOperationWithResult&lt;RegionObserver, DeleteTracker&gt;(regionObserverGetter, result) {</span>
<span class="source-line-no">1627</span><span id="line-1627"> @Override</span>
<span class="source-line-no">1628</span><span id="line-1628"> public DeleteTracker call(RegionObserver observer) throws IOException {</span>
<span class="source-line-no">1629</span><span id="line-1629"> return observer.postInstantiateDeleteTracker(this, getResult());</span>
<span class="source-line-no">1630</span><span id="line-1630"> }</span>
<span class="source-line-no">1631</span><span id="line-1631"> });</span>
<span class="source-line-no">1632</span><span id="line-1632"> }</span>
<span class="source-line-no">1633</span><span id="line-1633"></span>
<span class="source-line-no">1634</span><span id="line-1634"> /////////////////////////////////////////////////////////////////////////////////////////////////</span>
<span class="source-line-no">1635</span><span id="line-1635"> // BulkLoadObserver hooks</span>
<span class="source-line-no">1636</span><span id="line-1636"> /////////////////////////////////////////////////////////////////////////////////////////////////</span>
<span class="source-line-no">1637</span><span id="line-1637"> public void prePrepareBulkLoad(User user) throws IOException {</span>
<span class="source-line-no">1638</span><span id="line-1638"> execOperation(coprocEnvironments.isEmpty() ? null : new BulkLoadObserverOperation(user) {</span>
<span class="source-line-no">1639</span><span id="line-1639"> @Override</span>
<span class="source-line-no">1640</span><span id="line-1640"> protected void call(BulkLoadObserver observer) throws IOException {</span>
<span class="source-line-no">1641</span><span id="line-1641"> observer.prePrepareBulkLoad(this);</span>
<span class="source-line-no">1642</span><span id="line-1642"> }</span>
<span class="source-line-no">1643</span><span id="line-1643"> });</span>
<span class="source-line-no">1644</span><span id="line-1644"> }</span>
<span class="source-line-no">1645</span><span id="line-1645"></span>
<span class="source-line-no">1646</span><span id="line-1646"> public void preCleanupBulkLoad(User user) throws IOException {</span>
<span class="source-line-no">1647</span><span id="line-1647"> execOperation(coprocEnvironments.isEmpty() ? null : new BulkLoadObserverOperation(user) {</span>
<span class="source-line-no">1648</span><span id="line-1648"> @Override</span>
<span class="source-line-no">1649</span><span id="line-1649"> protected void call(BulkLoadObserver observer) throws IOException {</span>
<span class="source-line-no">1650</span><span id="line-1650"> observer.preCleanupBulkLoad(this);</span>
<span class="source-line-no">1651</span><span id="line-1651"> }</span>
<span class="source-line-no">1652</span><span id="line-1652"> });</span>
<span class="source-line-no">1653</span><span id="line-1653"> }</span>
<span class="source-line-no">1654</span><span id="line-1654">}</span>
</pre>
</div>
</main>
</body>
</html>