blob: e634c16492fffe089b492f38ddc6c9e2e33a424f [file] [log] [blame]
<!DOCTYPE HTML PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
<html lang="en">
<head>
<title>Source code</title>
<link rel="stylesheet" type="text/css" href="../../../../../../stylesheet.css" title="Style">
</head>
<body>
<div class="sourceContainer">
<pre><span class="sourceLineNo">001</span>/*<a name="line.1"></a>
<span class="sourceLineNo">002</span> * Licensed to the Apache Software Foundation (ASF) under one<a name="line.2"></a>
<span class="sourceLineNo">003</span> * or more contributor license agreements. See the NOTICE file<a name="line.3"></a>
<span class="sourceLineNo">004</span> * distributed with this work for additional information<a name="line.4"></a>
<span class="sourceLineNo">005</span> * regarding copyright ownership. The ASF licenses this file<a name="line.5"></a>
<span class="sourceLineNo">006</span> * to you under the Apache License, Version 2.0 (the<a name="line.6"></a>
<span class="sourceLineNo">007</span> * "License"); you may not use this file except in compliance<a name="line.7"></a>
<span class="sourceLineNo">008</span> * with the License. You may obtain a copy of the License at<a name="line.8"></a>
<span class="sourceLineNo">009</span> *<a name="line.9"></a>
<span class="sourceLineNo">010</span> * http://www.apache.org/licenses/LICENSE-2.0<a name="line.10"></a>
<span class="sourceLineNo">011</span> *<a name="line.11"></a>
<span class="sourceLineNo">012</span> * Unless required by applicable law or agreed to in writing, software<a name="line.12"></a>
<span class="sourceLineNo">013</span> * distributed under the License is distributed on an "AS IS" BASIS,<a name="line.13"></a>
<span class="sourceLineNo">014</span> * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.<a name="line.14"></a>
<span class="sourceLineNo">015</span> * See the License for the specific language governing permissions and<a name="line.15"></a>
<span class="sourceLineNo">016</span> * limitations under the License.<a name="line.16"></a>
<span class="sourceLineNo">017</span> */<a name="line.17"></a>
<span class="sourceLineNo">018</span>package org.apache.hadoop.hbase.client;<a name="line.18"></a>
<span class="sourceLineNo">019</span><a name="line.19"></a>
<span class="sourceLineNo">020</span>import static org.apache.hadoop.hbase.client.ConnectionUtils.setCoprocessorError;<a name="line.20"></a>
<span class="sourceLineNo">021</span><a name="line.21"></a>
<span class="sourceLineNo">022</span>import io.opentelemetry.api.trace.Span;<a name="line.22"></a>
<span class="sourceLineNo">023</span>import io.opentelemetry.context.Context;<a name="line.23"></a>
<span class="sourceLineNo">024</span>import io.opentelemetry.context.Scope;<a name="line.24"></a>
<span class="sourceLineNo">025</span>import java.io.IOException;<a name="line.25"></a>
<span class="sourceLineNo">026</span>import java.io.InterruptedIOException;<a name="line.26"></a>
<span class="sourceLineNo">027</span>import java.util.ArrayList;<a name="line.27"></a>
<span class="sourceLineNo">028</span>import java.util.List;<a name="line.28"></a>
<span class="sourceLineNo">029</span>import java.util.Map;<a name="line.29"></a>
<span class="sourceLineNo">030</span>import java.util.TreeMap;<a name="line.30"></a>
<span class="sourceLineNo">031</span>import java.util.concurrent.CompletableFuture;<a name="line.31"></a>
<span class="sourceLineNo">032</span>import java.util.concurrent.ConcurrentLinkedQueue;<a name="line.32"></a>
<span class="sourceLineNo">033</span>import java.util.concurrent.CountDownLatch;<a name="line.33"></a>
<span class="sourceLineNo">034</span>import java.util.concurrent.ExecutionException;<a name="line.34"></a>
<span class="sourceLineNo">035</span>import java.util.concurrent.ExecutorService;<a name="line.35"></a>
<span class="sourceLineNo">036</span>import java.util.concurrent.Future;<a name="line.36"></a>
<span class="sourceLineNo">037</span>import java.util.concurrent.RejectedExecutionException;<a name="line.37"></a>
<span class="sourceLineNo">038</span>import java.util.concurrent.TimeUnit;<a name="line.38"></a>
<span class="sourceLineNo">039</span>import java.util.function.Supplier;<a name="line.39"></a>
<span class="sourceLineNo">040</span>import java.util.stream.Collectors;<a name="line.40"></a>
<span class="sourceLineNo">041</span>import org.apache.commons.lang3.ArrayUtils;<a name="line.41"></a>
<span class="sourceLineNo">042</span>import org.apache.hadoop.conf.Configuration;<a name="line.42"></a>
<span class="sourceLineNo">043</span>import org.apache.hadoop.hbase.CompareOperator;<a name="line.43"></a>
<span class="sourceLineNo">044</span>import org.apache.hadoop.hbase.DoNotRetryIOException;<a name="line.44"></a>
<span class="sourceLineNo">045</span>import org.apache.hadoop.hbase.HBaseIOException;<a name="line.45"></a>
<span class="sourceLineNo">046</span>import org.apache.hadoop.hbase.HConstants;<a name="line.46"></a>
<span class="sourceLineNo">047</span>import org.apache.hadoop.hbase.HRegionLocation;<a name="line.47"></a>
<span class="sourceLineNo">048</span>import org.apache.hadoop.hbase.TableName;<a name="line.48"></a>
<span class="sourceLineNo">049</span>import org.apache.hadoop.hbase.client.RetriesExhaustedException.ThrowableWithExtraContext;<a name="line.49"></a>
<span class="sourceLineNo">050</span>import org.apache.hadoop.hbase.client.coprocessor.Batch;<a name="line.50"></a>
<span class="sourceLineNo">051</span>import org.apache.hadoop.hbase.client.trace.TableOperationSpanBuilder;<a name="line.51"></a>
<span class="sourceLineNo">052</span>import org.apache.hadoop.hbase.filter.Filter;<a name="line.52"></a>
<span class="sourceLineNo">053</span>import org.apache.hadoop.hbase.io.TimeRange;<a name="line.53"></a>
<span class="sourceLineNo">054</span>import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;<a name="line.54"></a>
<span class="sourceLineNo">055</span>import org.apache.hadoop.hbase.trace.HBaseSemanticAttributes;<a name="line.55"></a>
<span class="sourceLineNo">056</span>import org.apache.hadoop.hbase.trace.TraceUtil;<a name="line.56"></a>
<span class="sourceLineNo">057</span>import org.apache.hadoop.hbase.util.Bytes;<a name="line.57"></a>
<span class="sourceLineNo">058</span>import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;<a name="line.58"></a>
<span class="sourceLineNo">059</span>import org.apache.hadoop.hbase.util.FutureUtils;<a name="line.59"></a>
<span class="sourceLineNo">060</span>import org.apache.hadoop.hbase.util.IOExceptionSupplier;<a name="line.60"></a>
<span class="sourceLineNo">061</span>import org.apache.hadoop.hbase.util.Pair;<a name="line.61"></a>
<span class="sourceLineNo">062</span>import org.apache.yetus.audience.InterfaceAudience;<a name="line.62"></a>
<span class="sourceLineNo">063</span>import org.slf4j.Logger;<a name="line.63"></a>
<span class="sourceLineNo">064</span>import org.slf4j.LoggerFactory;<a name="line.64"></a>
<span class="sourceLineNo">065</span><a name="line.65"></a>
<span class="sourceLineNo">066</span>import org.apache.hbase.thirdparty.com.google.common.primitives.Booleans;<a name="line.66"></a>
<span class="sourceLineNo">067</span>import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor;<a name="line.67"></a>
<span class="sourceLineNo">068</span>import org.apache.hbase.thirdparty.com.google.protobuf.Message;<a name="line.68"></a>
<span class="sourceLineNo">069</span>import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;<a name="line.69"></a>
<span class="sourceLineNo">070</span>import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;<a name="line.70"></a>
<span class="sourceLineNo">071</span>import org.apache.hbase.thirdparty.com.google.protobuf.Service;<a name="line.71"></a>
<span class="sourceLineNo">072</span>import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;<a name="line.72"></a>
<span class="sourceLineNo">073</span><a name="line.73"></a>
<span class="sourceLineNo">074</span>import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;<a name="line.74"></a>
<span class="sourceLineNo">075</span><a name="line.75"></a>
<span class="sourceLineNo">076</span>/**<a name="line.76"></a>
<span class="sourceLineNo">077</span> * The table implementation based on {@link AsyncTable}.<a name="line.77"></a>
<span class="sourceLineNo">078</span> */<a name="line.78"></a>
<span class="sourceLineNo">079</span>@InterfaceAudience.Private<a name="line.79"></a>
<span class="sourceLineNo">080</span>class TableOverAsyncTable implements Table {<a name="line.80"></a>
<span class="sourceLineNo">081</span><a name="line.81"></a>
<span class="sourceLineNo">082</span> private static final Logger LOG = LoggerFactory.getLogger(TableOverAsyncTable.class);<a name="line.82"></a>
<span class="sourceLineNo">083</span><a name="line.83"></a>
<span class="sourceLineNo">084</span> private final AsyncConnectionImpl conn;<a name="line.84"></a>
<span class="sourceLineNo">085</span><a name="line.85"></a>
<span class="sourceLineNo">086</span> private final AsyncTable&lt;?&gt; table;<a name="line.86"></a>
<span class="sourceLineNo">087</span><a name="line.87"></a>
<span class="sourceLineNo">088</span> private final IOExceptionSupplier&lt;ExecutorService&gt; poolSupplier;<a name="line.88"></a>
<span class="sourceLineNo">089</span><a name="line.89"></a>
<span class="sourceLineNo">090</span> TableOverAsyncTable(AsyncConnectionImpl conn, AsyncTable&lt;?&gt; table,<a name="line.90"></a>
<span class="sourceLineNo">091</span> IOExceptionSupplier&lt;ExecutorService&gt; poolSupplier) {<a name="line.91"></a>
<span class="sourceLineNo">092</span> this.conn = conn;<a name="line.92"></a>
<span class="sourceLineNo">093</span> this.table = table;<a name="line.93"></a>
<span class="sourceLineNo">094</span> this.poolSupplier = poolSupplier;<a name="line.94"></a>
<span class="sourceLineNo">095</span> }<a name="line.95"></a>
<span class="sourceLineNo">096</span><a name="line.96"></a>
<span class="sourceLineNo">097</span> @Override<a name="line.97"></a>
<span class="sourceLineNo">098</span> public TableName getName() {<a name="line.98"></a>
<span class="sourceLineNo">099</span> return table.getName();<a name="line.99"></a>
<span class="sourceLineNo">100</span> }<a name="line.100"></a>
<span class="sourceLineNo">101</span><a name="line.101"></a>
<span class="sourceLineNo">102</span> @Override<a name="line.102"></a>
<span class="sourceLineNo">103</span> public Configuration getConfiguration() {<a name="line.103"></a>
<span class="sourceLineNo">104</span> return table.getConfiguration();<a name="line.104"></a>
<span class="sourceLineNo">105</span> }<a name="line.105"></a>
<span class="sourceLineNo">106</span><a name="line.106"></a>
<span class="sourceLineNo">107</span> @Override<a name="line.107"></a>
<span class="sourceLineNo">108</span> public TableDescriptor getDescriptor() throws IOException {<a name="line.108"></a>
<span class="sourceLineNo">109</span> return FutureUtils.get(conn.getAdmin().getDescriptor(getName()));<a name="line.109"></a>
<span class="sourceLineNo">110</span> }<a name="line.110"></a>
<span class="sourceLineNo">111</span><a name="line.111"></a>
<span class="sourceLineNo">112</span> @Override<a name="line.112"></a>
<span class="sourceLineNo">113</span> public boolean exists(Get get) throws IOException {<a name="line.113"></a>
<span class="sourceLineNo">114</span> return FutureUtils.get(table.exists(get));<a name="line.114"></a>
<span class="sourceLineNo">115</span> }<a name="line.115"></a>
<span class="sourceLineNo">116</span><a name="line.116"></a>
<span class="sourceLineNo">117</span> @Override<a name="line.117"></a>
<span class="sourceLineNo">118</span> public boolean[] exists(List&lt;Get&gt; gets) throws IOException {<a name="line.118"></a>
<span class="sourceLineNo">119</span> return Booleans.toArray(FutureUtils.get(table.existsAll(gets)));<a name="line.119"></a>
<span class="sourceLineNo">120</span> }<a name="line.120"></a>
<span class="sourceLineNo">121</span><a name="line.121"></a>
<span class="sourceLineNo">122</span> @Override<a name="line.122"></a>
<span class="sourceLineNo">123</span> public void batch(List&lt;? extends Row&gt; actions, Object[] results) throws IOException {<a name="line.123"></a>
<span class="sourceLineNo">124</span> if (ArrayUtils.isEmpty(results)) {<a name="line.124"></a>
<span class="sourceLineNo">125</span> FutureUtils.get(table.batchAll(actions));<a name="line.125"></a>
<span class="sourceLineNo">126</span> return;<a name="line.126"></a>
<span class="sourceLineNo">127</span> }<a name="line.127"></a>
<span class="sourceLineNo">128</span> List&lt;ThrowableWithExtraContext&gt; errors = new ArrayList&lt;&gt;();<a name="line.128"></a>
<span class="sourceLineNo">129</span> List&lt;CompletableFuture&lt;Object&gt;&gt; futures = table.batch(actions);<a name="line.129"></a>
<span class="sourceLineNo">130</span> for (int i = 0, n = results.length; i &lt; n; i++) {<a name="line.130"></a>
<span class="sourceLineNo">131</span> try {<a name="line.131"></a>
<span class="sourceLineNo">132</span> results[i] = FutureUtils.get(futures.get(i));<a name="line.132"></a>
<span class="sourceLineNo">133</span> } catch (IOException e) {<a name="line.133"></a>
<span class="sourceLineNo">134</span> results[i] = e;<a name="line.134"></a>
<span class="sourceLineNo">135</span> errors.add(new ThrowableWithExtraContext(e, EnvironmentEdgeManager.currentTime(),<a name="line.135"></a>
<span class="sourceLineNo">136</span> "Error when processing " + actions.get(i)));<a name="line.136"></a>
<span class="sourceLineNo">137</span> }<a name="line.137"></a>
<span class="sourceLineNo">138</span> }<a name="line.138"></a>
<span class="sourceLineNo">139</span> if (!errors.isEmpty()) {<a name="line.139"></a>
<span class="sourceLineNo">140</span> throw new RetriesExhaustedException(errors.size(), errors);<a name="line.140"></a>
<span class="sourceLineNo">141</span> }<a name="line.141"></a>
<span class="sourceLineNo">142</span> }<a name="line.142"></a>
<span class="sourceLineNo">143</span><a name="line.143"></a>
<span class="sourceLineNo">144</span> @Override<a name="line.144"></a>
<span class="sourceLineNo">145</span> public &lt;R&gt; void batchCallback(List&lt;? extends Row&gt; actions, Object[] results,<a name="line.145"></a>
<span class="sourceLineNo">146</span> Batch.Callback&lt;R&gt; callback) throws IOException, InterruptedException {<a name="line.146"></a>
<span class="sourceLineNo">147</span> ConcurrentLinkedQueue&lt;ThrowableWithExtraContext&gt; errors = new ConcurrentLinkedQueue&lt;&gt;();<a name="line.147"></a>
<span class="sourceLineNo">148</span> CountDownLatch latch = new CountDownLatch(actions.size());<a name="line.148"></a>
<span class="sourceLineNo">149</span> AsyncTableRegionLocator locator = conn.getRegionLocator(getName());<a name="line.149"></a>
<span class="sourceLineNo">150</span> List&lt;CompletableFuture&lt;R&gt;&gt; futures = table.&lt;R&gt; batch(actions);<a name="line.150"></a>
<span class="sourceLineNo">151</span> for (int i = 0, n = futures.size(); i &lt; n; i++) {<a name="line.151"></a>
<span class="sourceLineNo">152</span> final int index = i;<a name="line.152"></a>
<span class="sourceLineNo">153</span> FutureUtils.addListener(futures.get(i), (r, e) -&gt; {<a name="line.153"></a>
<span class="sourceLineNo">154</span> if (e != null) {<a name="line.154"></a>
<span class="sourceLineNo">155</span> errors.add(new ThrowableWithExtraContext(e, EnvironmentEdgeManager.currentTime(),<a name="line.155"></a>
<span class="sourceLineNo">156</span> "Error when processing " + actions.get(index)));<a name="line.156"></a>
<span class="sourceLineNo">157</span> if (!ArrayUtils.isEmpty(results)) {<a name="line.157"></a>
<span class="sourceLineNo">158</span> results[index] = e;<a name="line.158"></a>
<span class="sourceLineNo">159</span> }<a name="line.159"></a>
<span class="sourceLineNo">160</span> latch.countDown();<a name="line.160"></a>
<span class="sourceLineNo">161</span> } else {<a name="line.161"></a>
<span class="sourceLineNo">162</span> if (!ArrayUtils.isEmpty(results)) {<a name="line.162"></a>
<span class="sourceLineNo">163</span> results[index] = r;<a name="line.163"></a>
<span class="sourceLineNo">164</span> }<a name="line.164"></a>
<span class="sourceLineNo">165</span> FutureUtils.addListener(locator.getRegionLocation(actions.get(index).getRow()),<a name="line.165"></a>
<span class="sourceLineNo">166</span> (l, le) -&gt; {<a name="line.166"></a>
<span class="sourceLineNo">167</span> if (le != null) {<a name="line.167"></a>
<span class="sourceLineNo">168</span> errors.add(new ThrowableWithExtraContext(le, EnvironmentEdgeManager.currentTime(),<a name="line.168"></a>
<span class="sourceLineNo">169</span> "Error when finding the region for row "<a name="line.169"></a>
<span class="sourceLineNo">170</span> + Bytes.toStringBinary(actions.get(index).getRow())));<a name="line.170"></a>
<span class="sourceLineNo">171</span> } else {<a name="line.171"></a>
<span class="sourceLineNo">172</span> callback.update(l.getRegion().getRegionName(), actions.get(index).getRow(), r);<a name="line.172"></a>
<span class="sourceLineNo">173</span> }<a name="line.173"></a>
<span class="sourceLineNo">174</span> latch.countDown();<a name="line.174"></a>
<span class="sourceLineNo">175</span> });<a name="line.175"></a>
<span class="sourceLineNo">176</span> }<a name="line.176"></a>
<span class="sourceLineNo">177</span> });<a name="line.177"></a>
<span class="sourceLineNo">178</span> }<a name="line.178"></a>
<span class="sourceLineNo">179</span> latch.await();<a name="line.179"></a>
<span class="sourceLineNo">180</span> if (!errors.isEmpty()) {<a name="line.180"></a>
<span class="sourceLineNo">181</span> throw new RetriesExhaustedException(errors.size(),<a name="line.181"></a>
<span class="sourceLineNo">182</span> errors.stream().collect(Collectors.toList()));<a name="line.182"></a>
<span class="sourceLineNo">183</span> }<a name="line.183"></a>
<span class="sourceLineNo">184</span> }<a name="line.184"></a>
<span class="sourceLineNo">185</span><a name="line.185"></a>
<span class="sourceLineNo">186</span> @Override<a name="line.186"></a>
<span class="sourceLineNo">187</span> public Result get(Get get) throws IOException {<a name="line.187"></a>
<span class="sourceLineNo">188</span> return FutureUtils.get(table.get(get));<a name="line.188"></a>
<span class="sourceLineNo">189</span> }<a name="line.189"></a>
<span class="sourceLineNo">190</span><a name="line.190"></a>
<span class="sourceLineNo">191</span> @Override<a name="line.191"></a>
<span class="sourceLineNo">192</span> public Result[] get(List&lt;Get&gt; gets) throws IOException {<a name="line.192"></a>
<span class="sourceLineNo">193</span> return FutureUtils.get(table.getAll(gets)).toArray(new Result[0]);<a name="line.193"></a>
<span class="sourceLineNo">194</span> }<a name="line.194"></a>
<span class="sourceLineNo">195</span><a name="line.195"></a>
<span class="sourceLineNo">196</span> @Override<a name="line.196"></a>
<span class="sourceLineNo">197</span> public ResultScanner getScanner(Scan scan) throws IOException {<a name="line.197"></a>
<span class="sourceLineNo">198</span> return table.getScanner(scan);<a name="line.198"></a>
<span class="sourceLineNo">199</span> }<a name="line.199"></a>
<span class="sourceLineNo">200</span><a name="line.200"></a>
<span class="sourceLineNo">201</span> @Override<a name="line.201"></a>
<span class="sourceLineNo">202</span> public ResultScanner getScanner(byte[] family) throws IOException {<a name="line.202"></a>
<span class="sourceLineNo">203</span> return table.getScanner(family);<a name="line.203"></a>
<span class="sourceLineNo">204</span> }<a name="line.204"></a>
<span class="sourceLineNo">205</span><a name="line.205"></a>
<span class="sourceLineNo">206</span> @Override<a name="line.206"></a>
<span class="sourceLineNo">207</span> public ResultScanner getScanner(byte[] family, byte[] qualifier) throws IOException {<a name="line.207"></a>
<span class="sourceLineNo">208</span> return table.getScanner(family, qualifier);<a name="line.208"></a>
<span class="sourceLineNo">209</span> }<a name="line.209"></a>
<span class="sourceLineNo">210</span><a name="line.210"></a>
<span class="sourceLineNo">211</span> @Override<a name="line.211"></a>
<span class="sourceLineNo">212</span> public void put(Put put) throws IOException {<a name="line.212"></a>
<span class="sourceLineNo">213</span> FutureUtils.get(table.put(put));<a name="line.213"></a>
<span class="sourceLineNo">214</span> }<a name="line.214"></a>
<span class="sourceLineNo">215</span><a name="line.215"></a>
<span class="sourceLineNo">216</span> @Override<a name="line.216"></a>
<span class="sourceLineNo">217</span> public void put(List&lt;Put&gt; puts) throws IOException {<a name="line.217"></a>
<span class="sourceLineNo">218</span> FutureUtils.get(table.putAll(puts));<a name="line.218"></a>
<span class="sourceLineNo">219</span> }<a name="line.219"></a>
<span class="sourceLineNo">220</span><a name="line.220"></a>
<span class="sourceLineNo">221</span> @Override<a name="line.221"></a>
<span class="sourceLineNo">222</span> public void delete(Delete delete) throws IOException {<a name="line.222"></a>
<span class="sourceLineNo">223</span> FutureUtils.get(table.delete(delete));<a name="line.223"></a>
<span class="sourceLineNo">224</span> }<a name="line.224"></a>
<span class="sourceLineNo">225</span><a name="line.225"></a>
<span class="sourceLineNo">226</span> @Override<a name="line.226"></a>
<span class="sourceLineNo">227</span> public void delete(List&lt;Delete&gt; deletes) throws IOException {<a name="line.227"></a>
<span class="sourceLineNo">228</span> FutureUtils.get(table.deleteAll(deletes));<a name="line.228"></a>
<span class="sourceLineNo">229</span> }<a name="line.229"></a>
<span class="sourceLineNo">230</span><a name="line.230"></a>
<span class="sourceLineNo">231</span> @Override<a name="line.231"></a>
<span class="sourceLineNo">232</span> public CheckAndMutateBuilder checkAndMutate(byte[] row, byte[] family) {<a name="line.232"></a>
<span class="sourceLineNo">233</span> return new CheckAndMutateBuilder() {<a name="line.233"></a>
<span class="sourceLineNo">234</span><a name="line.234"></a>
<span class="sourceLineNo">235</span> private final AsyncTable.CheckAndMutateBuilder builder = table.checkAndMutate(row, family);<a name="line.235"></a>
<span class="sourceLineNo">236</span><a name="line.236"></a>
<span class="sourceLineNo">237</span> @Override<a name="line.237"></a>
<span class="sourceLineNo">238</span> public CheckAndMutateBuilder qualifier(byte[] qualifier) {<a name="line.238"></a>
<span class="sourceLineNo">239</span> builder.qualifier(qualifier);<a name="line.239"></a>
<span class="sourceLineNo">240</span> return this;<a name="line.240"></a>
<span class="sourceLineNo">241</span> }<a name="line.241"></a>
<span class="sourceLineNo">242</span><a name="line.242"></a>
<span class="sourceLineNo">243</span> @Override<a name="line.243"></a>
<span class="sourceLineNo">244</span> public CheckAndMutateBuilder timeRange(TimeRange timeRange) {<a name="line.244"></a>
<span class="sourceLineNo">245</span> builder.timeRange(timeRange);<a name="line.245"></a>
<span class="sourceLineNo">246</span> return this;<a name="line.246"></a>
<span class="sourceLineNo">247</span> }<a name="line.247"></a>
<span class="sourceLineNo">248</span><a name="line.248"></a>
<span class="sourceLineNo">249</span> @Override<a name="line.249"></a>
<span class="sourceLineNo">250</span> public CheckAndMutateBuilder ifNotExists() {<a name="line.250"></a>
<span class="sourceLineNo">251</span> builder.ifNotExists();<a name="line.251"></a>
<span class="sourceLineNo">252</span> return this;<a name="line.252"></a>
<span class="sourceLineNo">253</span> }<a name="line.253"></a>
<span class="sourceLineNo">254</span><a name="line.254"></a>
<span class="sourceLineNo">255</span> @Override<a name="line.255"></a>
<span class="sourceLineNo">256</span> public CheckAndMutateBuilder ifMatches(CompareOperator compareOp, byte[] value) {<a name="line.256"></a>
<span class="sourceLineNo">257</span> builder.ifMatches(compareOp, value);<a name="line.257"></a>
<span class="sourceLineNo">258</span> return this;<a name="line.258"></a>
<span class="sourceLineNo">259</span> }<a name="line.259"></a>
<span class="sourceLineNo">260</span><a name="line.260"></a>
<span class="sourceLineNo">261</span> @Override<a name="line.261"></a>
<span class="sourceLineNo">262</span> public boolean thenPut(Put put) throws IOException {<a name="line.262"></a>
<span class="sourceLineNo">263</span> return FutureUtils.get(builder.thenPut(put));<a name="line.263"></a>
<span class="sourceLineNo">264</span> }<a name="line.264"></a>
<span class="sourceLineNo">265</span><a name="line.265"></a>
<span class="sourceLineNo">266</span> @Override<a name="line.266"></a>
<span class="sourceLineNo">267</span> public boolean thenDelete(Delete delete) throws IOException {<a name="line.267"></a>
<span class="sourceLineNo">268</span> return FutureUtils.get(builder.thenDelete(delete));<a name="line.268"></a>
<span class="sourceLineNo">269</span> }<a name="line.269"></a>
<span class="sourceLineNo">270</span><a name="line.270"></a>
<span class="sourceLineNo">271</span> @Override<a name="line.271"></a>
<span class="sourceLineNo">272</span> public boolean thenMutate(RowMutations mutation) throws IOException {<a name="line.272"></a>
<span class="sourceLineNo">273</span> return FutureUtils.get(builder.thenMutate(mutation));<a name="line.273"></a>
<span class="sourceLineNo">274</span> }<a name="line.274"></a>
<span class="sourceLineNo">275</span> };<a name="line.275"></a>
<span class="sourceLineNo">276</span> }<a name="line.276"></a>
<span class="sourceLineNo">277</span><a name="line.277"></a>
<span class="sourceLineNo">278</span> @Override<a name="line.278"></a>
<span class="sourceLineNo">279</span> public CheckAndMutateWithFilterBuilder checkAndMutate(byte[] row, Filter filter) {<a name="line.279"></a>
<span class="sourceLineNo">280</span> return new CheckAndMutateWithFilterBuilder() {<a name="line.280"></a>
<span class="sourceLineNo">281</span> private final AsyncTable.CheckAndMutateWithFilterBuilder builder =<a name="line.281"></a>
<span class="sourceLineNo">282</span> table.checkAndMutate(row, filter);<a name="line.282"></a>
<span class="sourceLineNo">283</span><a name="line.283"></a>
<span class="sourceLineNo">284</span> @Override<a name="line.284"></a>
<span class="sourceLineNo">285</span> public CheckAndMutateWithFilterBuilder timeRange(TimeRange timeRange) {<a name="line.285"></a>
<span class="sourceLineNo">286</span> builder.timeRange(timeRange);<a name="line.286"></a>
<span class="sourceLineNo">287</span> return this;<a name="line.287"></a>
<span class="sourceLineNo">288</span> }<a name="line.288"></a>
<span class="sourceLineNo">289</span><a name="line.289"></a>
<span class="sourceLineNo">290</span> @Override<a name="line.290"></a>
<span class="sourceLineNo">291</span> public boolean thenPut(Put put) throws IOException {<a name="line.291"></a>
<span class="sourceLineNo">292</span> return FutureUtils.get(builder.thenPut(put));<a name="line.292"></a>
<span class="sourceLineNo">293</span> }<a name="line.293"></a>
<span class="sourceLineNo">294</span><a name="line.294"></a>
<span class="sourceLineNo">295</span> @Override<a name="line.295"></a>
<span class="sourceLineNo">296</span> public boolean thenDelete(Delete delete) throws IOException {<a name="line.296"></a>
<span class="sourceLineNo">297</span> return FutureUtils.get(builder.thenDelete(delete));<a name="line.297"></a>
<span class="sourceLineNo">298</span> }<a name="line.298"></a>
<span class="sourceLineNo">299</span><a name="line.299"></a>
<span class="sourceLineNo">300</span> @Override<a name="line.300"></a>
<span class="sourceLineNo">301</span> public boolean thenMutate(RowMutations mutation) throws IOException {<a name="line.301"></a>
<span class="sourceLineNo">302</span> return FutureUtils.get(builder.thenMutate(mutation));<a name="line.302"></a>
<span class="sourceLineNo">303</span> }<a name="line.303"></a>
<span class="sourceLineNo">304</span> };<a name="line.304"></a>
<span class="sourceLineNo">305</span> }<a name="line.305"></a>
<span class="sourceLineNo">306</span><a name="line.306"></a>
<span class="sourceLineNo">307</span> @Override<a name="line.307"></a>
<span class="sourceLineNo">308</span> public CheckAndMutateResult checkAndMutate(CheckAndMutate checkAndMutate) throws IOException {<a name="line.308"></a>
<span class="sourceLineNo">309</span> return FutureUtils.get(table.checkAndMutate(checkAndMutate));<a name="line.309"></a>
<span class="sourceLineNo">310</span> }<a name="line.310"></a>
<span class="sourceLineNo">311</span><a name="line.311"></a>
<span class="sourceLineNo">312</span> @Override<a name="line.312"></a>
<span class="sourceLineNo">313</span> public List&lt;CheckAndMutateResult&gt; checkAndMutate(List&lt;CheckAndMutate&gt; checkAndMutates)<a name="line.313"></a>
<span class="sourceLineNo">314</span> throws IOException {<a name="line.314"></a>
<span class="sourceLineNo">315</span> return FutureUtils.get(table.checkAndMutateAll(checkAndMutates));<a name="line.315"></a>
<span class="sourceLineNo">316</span> }<a name="line.316"></a>
<span class="sourceLineNo">317</span><a name="line.317"></a>
<span class="sourceLineNo">318</span> @Override<a name="line.318"></a>
<span class="sourceLineNo">319</span> public Result mutateRow(RowMutations rm) throws IOException {<a name="line.319"></a>
<span class="sourceLineNo">320</span> return FutureUtils.get(table.mutateRow(rm));<a name="line.320"></a>
<span class="sourceLineNo">321</span> }<a name="line.321"></a>
<span class="sourceLineNo">322</span><a name="line.322"></a>
<span class="sourceLineNo">323</span> @Override<a name="line.323"></a>
<span class="sourceLineNo">324</span> public Result append(Append append) throws IOException {<a name="line.324"></a>
<span class="sourceLineNo">325</span> return FutureUtils.get(table.append(append));<a name="line.325"></a>
<span class="sourceLineNo">326</span> }<a name="line.326"></a>
<span class="sourceLineNo">327</span><a name="line.327"></a>
<span class="sourceLineNo">328</span> @Override<a name="line.328"></a>
<span class="sourceLineNo">329</span> public Result increment(Increment increment) throws IOException {<a name="line.329"></a>
<span class="sourceLineNo">330</span> return FutureUtils.get(table.increment(increment));<a name="line.330"></a>
<span class="sourceLineNo">331</span> }<a name="line.331"></a>
<span class="sourceLineNo">332</span><a name="line.332"></a>
<span class="sourceLineNo">333</span> @Override<a name="line.333"></a>
<span class="sourceLineNo">334</span> public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier, long amount)<a name="line.334"></a>
<span class="sourceLineNo">335</span> throws IOException {<a name="line.335"></a>
<span class="sourceLineNo">336</span> return FutureUtils.get(table.incrementColumnValue(row, family, qualifier, amount));<a name="line.336"></a>
<span class="sourceLineNo">337</span> }<a name="line.337"></a>
<span class="sourceLineNo">338</span><a name="line.338"></a>
<span class="sourceLineNo">339</span> @Override<a name="line.339"></a>
<span class="sourceLineNo">340</span> public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier, long amount,<a name="line.340"></a>
<span class="sourceLineNo">341</span> Durability durability) throws IOException {<a name="line.341"></a>
<span class="sourceLineNo">342</span> return FutureUtils.get(table.incrementColumnValue(row, family, qualifier, amount, durability));<a name="line.342"></a>
<span class="sourceLineNo">343</span> }<a name="line.343"></a>
<span class="sourceLineNo">344</span><a name="line.344"></a>
<span class="sourceLineNo">345</span> @Override<a name="line.345"></a>
<span class="sourceLineNo">346</span> public void close() {<a name="line.346"></a>
<span class="sourceLineNo">347</span> }<a name="line.347"></a>
<span class="sourceLineNo">348</span><a name="line.348"></a>
<span class="sourceLineNo">349</span> @SuppressWarnings("deprecation")<a name="line.349"></a>
<span class="sourceLineNo">350</span> private static final class RegionCoprocessorRpcChannel extends RegionCoprocessorRpcChannelImpl<a name="line.350"></a>
<span class="sourceLineNo">351</span> implements CoprocessorRpcChannel {<a name="line.351"></a>
<span class="sourceLineNo">352</span><a name="line.352"></a>
<span class="sourceLineNo">353</span> RegionCoprocessorRpcChannel(AsyncConnectionImpl conn, TableName tableName, RegionInfo region,<a name="line.353"></a>
<span class="sourceLineNo">354</span> byte[] row, long rpcTimeoutNs, long operationTimeoutNs) {<a name="line.354"></a>
<span class="sourceLineNo">355</span> super(conn, tableName, region, row, rpcTimeoutNs, operationTimeoutNs);<a name="line.355"></a>
<span class="sourceLineNo">356</span> }<a name="line.356"></a>
<span class="sourceLineNo">357</span><a name="line.357"></a>
<span class="sourceLineNo">358</span> @Override<a name="line.358"></a>
<span class="sourceLineNo">359</span> public void callMethod(MethodDescriptor method, RpcController controller, Message request,<a name="line.359"></a>
<span class="sourceLineNo">360</span> Message responsePrototype, RpcCallback&lt;Message&gt; done) {<a name="line.360"></a>
<span class="sourceLineNo">361</span> ClientCoprocessorRpcController c = new ClientCoprocessorRpcController();<a name="line.361"></a>
<span class="sourceLineNo">362</span> CoprocessorBlockingRpcCallback&lt;Message&gt; callback = new CoprocessorBlockingRpcCallback&lt;&gt;();<a name="line.362"></a>
<span class="sourceLineNo">363</span> super.callMethod(method, c, request, responsePrototype, callback);<a name="line.363"></a>
<span class="sourceLineNo">364</span> Message ret;<a name="line.364"></a>
<span class="sourceLineNo">365</span> try {<a name="line.365"></a>
<span class="sourceLineNo">366</span> ret = callback.get();<a name="line.366"></a>
<span class="sourceLineNo">367</span> } catch (IOException e) {<a name="line.367"></a>
<span class="sourceLineNo">368</span> setCoprocessorError(controller, e);<a name="line.368"></a>
<span class="sourceLineNo">369</span> return;<a name="line.369"></a>
<span class="sourceLineNo">370</span> }<a name="line.370"></a>
<span class="sourceLineNo">371</span> if (c.failed()) {<a name="line.371"></a>
<span class="sourceLineNo">372</span> setCoprocessorError(controller, c.getFailed());<a name="line.372"></a>
<span class="sourceLineNo">373</span> }<a name="line.373"></a>
<span class="sourceLineNo">374</span> done.run(ret);<a name="line.374"></a>
<span class="sourceLineNo">375</span> }<a name="line.375"></a>
<span class="sourceLineNo">376</span><a name="line.376"></a>
<span class="sourceLineNo">377</span> @Override<a name="line.377"></a>
<span class="sourceLineNo">378</span> public Message callBlockingMethod(MethodDescriptor method, RpcController controller,<a name="line.378"></a>
<span class="sourceLineNo">379</span> Message request, Message responsePrototype) throws ServiceException {<a name="line.379"></a>
<span class="sourceLineNo">380</span> ClientCoprocessorRpcController c = new ClientCoprocessorRpcController();<a name="line.380"></a>
<span class="sourceLineNo">381</span> CoprocessorBlockingRpcCallback&lt;Message&gt; done = new CoprocessorBlockingRpcCallback&lt;&gt;();<a name="line.381"></a>
<span class="sourceLineNo">382</span> callMethod(method, c, request, responsePrototype, done);<a name="line.382"></a>
<span class="sourceLineNo">383</span> Message ret;<a name="line.383"></a>
<span class="sourceLineNo">384</span> try {<a name="line.384"></a>
<span class="sourceLineNo">385</span> ret = done.get();<a name="line.385"></a>
<span class="sourceLineNo">386</span> } catch (IOException e) {<a name="line.386"></a>
<span class="sourceLineNo">387</span> throw new ServiceException(e);<a name="line.387"></a>
<span class="sourceLineNo">388</span> }<a name="line.388"></a>
<span class="sourceLineNo">389</span> if (c.failed()) {<a name="line.389"></a>
<span class="sourceLineNo">390</span> setCoprocessorError(controller, c.getFailed());<a name="line.390"></a>
<span class="sourceLineNo">391</span> throw new ServiceException(c.getFailed());<a name="line.391"></a>
<span class="sourceLineNo">392</span> }<a name="line.392"></a>
<span class="sourceLineNo">393</span> return ret;<a name="line.393"></a>
<span class="sourceLineNo">394</span> }<a name="line.394"></a>
<span class="sourceLineNo">395</span> }<a name="line.395"></a>
<span class="sourceLineNo">396</span><a name="line.396"></a>
<span class="sourceLineNo">397</span> @Override<a name="line.397"></a>
<span class="sourceLineNo">398</span> public RegionCoprocessorRpcChannel coprocessorService(byte[] row) {<a name="line.398"></a>
<span class="sourceLineNo">399</span> return new RegionCoprocessorRpcChannel(conn, getName(), null, row,<a name="line.399"></a>
<span class="sourceLineNo">400</span> getRpcTimeout(TimeUnit.NANOSECONDS), getOperationTimeout(TimeUnit.NANOSECONDS));<a name="line.400"></a>
<span class="sourceLineNo">401</span> }<a name="line.401"></a>
<span class="sourceLineNo">402</span><a name="line.402"></a>
<span class="sourceLineNo">403</span> /**<a name="line.403"></a>
<span class="sourceLineNo">404</span> * Get the corresponding start keys and regions for an arbitrary range of keys.<a name="line.404"></a>
<span class="sourceLineNo">405</span> * &lt;p&gt;<a name="line.405"></a>
<span class="sourceLineNo">406</span> * @param startKey Starting row in range, inclusive<a name="line.406"></a>
<span class="sourceLineNo">407</span> * @param endKey Ending row in range<a name="line.407"></a>
<span class="sourceLineNo">408</span> * @param includeEndKey true if endRow is inclusive, false if exclusive<a name="line.408"></a>
<span class="sourceLineNo">409</span> * @return A pair of list of start keys and list of HRegionLocations that contain the specified<a name="line.409"></a>
<span class="sourceLineNo">410</span> * range<a name="line.410"></a>
<span class="sourceLineNo">411</span> * @throws IOException if a remote or network exception occurs<a name="line.411"></a>
<span class="sourceLineNo">412</span> */<a name="line.412"></a>
<span class="sourceLineNo">413</span> private Pair&lt;List&lt;byte[]&gt;, List&lt;HRegionLocation&gt;&gt; getKeysAndRegionsInRange(final byte[] startKey,<a name="line.413"></a>
<span class="sourceLineNo">414</span> final byte[] endKey, final boolean includeEndKey) throws IOException {<a name="line.414"></a>
<span class="sourceLineNo">415</span> return getKeysAndRegionsInRange(startKey, endKey, includeEndKey, false);<a name="line.415"></a>
<span class="sourceLineNo">416</span> }<a name="line.416"></a>
<span class="sourceLineNo">417</span><a name="line.417"></a>
<span class="sourceLineNo">418</span> /**<a name="line.418"></a>
<span class="sourceLineNo">419</span> * Get the corresponding start keys and regions for an arbitrary range of keys.<a name="line.419"></a>
<span class="sourceLineNo">420</span> * &lt;p&gt;<a name="line.420"></a>
<span class="sourceLineNo">421</span> * @param startKey Starting row in range, inclusive<a name="line.421"></a>
<span class="sourceLineNo">422</span> * @param endKey Ending row in range<a name="line.422"></a>
<span class="sourceLineNo">423</span> * @param includeEndKey true if endRow is inclusive, false if exclusive<a name="line.423"></a>
<span class="sourceLineNo">424</span> * @param reload true to reload information or false to use cached information<a name="line.424"></a>
<span class="sourceLineNo">425</span> * @return A pair of list of start keys and list of HRegionLocations that contain the specified<a name="line.425"></a>
<span class="sourceLineNo">426</span> * range<a name="line.426"></a>
<span class="sourceLineNo">427</span> * @throws IOException if a remote or network exception occurs<a name="line.427"></a>
<span class="sourceLineNo">428</span> */<a name="line.428"></a>
<span class="sourceLineNo">429</span> private Pair&lt;List&lt;byte[]&gt;, List&lt;HRegionLocation&gt;&gt; getKeysAndRegionsInRange(final byte[] startKey,<a name="line.429"></a>
<span class="sourceLineNo">430</span> final byte[] endKey, final boolean includeEndKey, final boolean reload) throws IOException {<a name="line.430"></a>
<span class="sourceLineNo">431</span> final boolean endKeyIsEndOfTable = Bytes.equals(endKey, HConstants.EMPTY_END_ROW);<a name="line.431"></a>
<span class="sourceLineNo">432</span> if ((Bytes.compareTo(startKey, endKey) &gt; 0) &amp;&amp; !endKeyIsEndOfTable) {<a name="line.432"></a>
<span class="sourceLineNo">433</span> throw new IllegalArgumentException(<a name="line.433"></a>
<span class="sourceLineNo">434</span> "Invalid range: " + Bytes.toStringBinary(startKey) + " &gt; " + Bytes.toStringBinary(endKey));<a name="line.434"></a>
<span class="sourceLineNo">435</span> }<a name="line.435"></a>
<span class="sourceLineNo">436</span> List&lt;byte[]&gt; keysInRange = new ArrayList&lt;&gt;();<a name="line.436"></a>
<span class="sourceLineNo">437</span> List&lt;HRegionLocation&gt; regionsInRange = new ArrayList&lt;&gt;();<a name="line.437"></a>
<span class="sourceLineNo">438</span> byte[] currentKey = startKey;<a name="line.438"></a>
<span class="sourceLineNo">439</span> do {<a name="line.439"></a>
<span class="sourceLineNo">440</span> HRegionLocation regionLocation =<a name="line.440"></a>
<span class="sourceLineNo">441</span> FutureUtils.get(conn.getRegionLocator(getName()).getRegionLocation(currentKey, reload));<a name="line.441"></a>
<span class="sourceLineNo">442</span> keysInRange.add(currentKey);<a name="line.442"></a>
<span class="sourceLineNo">443</span> regionsInRange.add(regionLocation);<a name="line.443"></a>
<span class="sourceLineNo">444</span> currentKey = regionLocation.getRegion().getEndKey();<a name="line.444"></a>
<span class="sourceLineNo">445</span> } while (<a name="line.445"></a>
<span class="sourceLineNo">446</span> !Bytes.equals(currentKey, HConstants.EMPTY_END_ROW)<a name="line.446"></a>
<span class="sourceLineNo">447</span> &amp;&amp; (endKeyIsEndOfTable || Bytes.compareTo(currentKey, endKey) &lt; 0<a name="line.447"></a>
<span class="sourceLineNo">448</span> || (includeEndKey &amp;&amp; Bytes.compareTo(currentKey, endKey) == 0))<a name="line.448"></a>
<span class="sourceLineNo">449</span> );<a name="line.449"></a>
<span class="sourceLineNo">450</span> return new Pair&lt;&gt;(keysInRange, regionsInRange);<a name="line.450"></a>
<span class="sourceLineNo">451</span> }<a name="line.451"></a>
<span class="sourceLineNo">452</span><a name="line.452"></a>
<span class="sourceLineNo">453</span> private List&lt;byte[]&gt; getStartKeysInRange(byte[] start, byte[] end) throws IOException {<a name="line.453"></a>
<span class="sourceLineNo">454</span> if (start == null) {<a name="line.454"></a>
<span class="sourceLineNo">455</span> start = HConstants.EMPTY_START_ROW;<a name="line.455"></a>
<span class="sourceLineNo">456</span> }<a name="line.456"></a>
<span class="sourceLineNo">457</span> if (end == null) {<a name="line.457"></a>
<span class="sourceLineNo">458</span> end = HConstants.EMPTY_END_ROW;<a name="line.458"></a>
<span class="sourceLineNo">459</span> }<a name="line.459"></a>
<span class="sourceLineNo">460</span> return getKeysAndRegionsInRange(start, end, true).getFirst();<a name="line.460"></a>
<span class="sourceLineNo">461</span> }<a name="line.461"></a>
<span class="sourceLineNo">462</span><a name="line.462"></a>
<span class="sourceLineNo">463</span> @FunctionalInterface<a name="line.463"></a>
<span class="sourceLineNo">464</span> private interface StubCall&lt;R&gt; {<a name="line.464"></a>
<span class="sourceLineNo">465</span> R call(RegionCoprocessorRpcChannel channel) throws Exception;<a name="line.465"></a>
<span class="sourceLineNo">466</span> }<a name="line.466"></a>
<span class="sourceLineNo">467</span><a name="line.467"></a>
<span class="sourceLineNo">468</span> private &lt;R&gt; void coprocessorService(String serviceName, byte[] startKey, byte[] endKey,<a name="line.468"></a>
<span class="sourceLineNo">469</span> Batch.Callback&lt;R&gt; callback, StubCall&lt;R&gt; call) throws Throwable {<a name="line.469"></a>
<span class="sourceLineNo">470</span> // get regions covered by the row range<a name="line.470"></a>
<span class="sourceLineNo">471</span> ExecutorService pool = Context.current().wrap(this.poolSupplier.get());<a name="line.471"></a>
<span class="sourceLineNo">472</span> List&lt;byte[]&gt; keys = getStartKeysInRange(startKey, endKey);<a name="line.472"></a>
<span class="sourceLineNo">473</span> Map&lt;byte[], Future&lt;R&gt;&gt; futures = new TreeMap&lt;&gt;(Bytes.BYTES_COMPARATOR);<a name="line.473"></a>
<span class="sourceLineNo">474</span> try {<a name="line.474"></a>
<span class="sourceLineNo">475</span> for (byte[] r : keys) {<a name="line.475"></a>
<span class="sourceLineNo">476</span> RegionCoprocessorRpcChannel channel = coprocessorService(r);<a name="line.476"></a>
<span class="sourceLineNo">477</span> Future&lt;R&gt; future = pool.submit(() -&gt; {<a name="line.477"></a>
<span class="sourceLineNo">478</span> R result = call.call(channel);<a name="line.478"></a>
<span class="sourceLineNo">479</span> byte[] region = channel.getLastRegion();<a name="line.479"></a>
<span class="sourceLineNo">480</span> if (callback != null) {<a name="line.480"></a>
<span class="sourceLineNo">481</span> callback.update(region, r, result);<a name="line.481"></a>
<span class="sourceLineNo">482</span> }<a name="line.482"></a>
<span class="sourceLineNo">483</span> return result;<a name="line.483"></a>
<span class="sourceLineNo">484</span> });<a name="line.484"></a>
<span class="sourceLineNo">485</span> futures.put(r, future);<a name="line.485"></a>
<span class="sourceLineNo">486</span> }<a name="line.486"></a>
<span class="sourceLineNo">487</span> } catch (RejectedExecutionException e) {<a name="line.487"></a>
<span class="sourceLineNo">488</span> // maybe the connection has been closed, let's check<a name="line.488"></a>
<span class="sourceLineNo">489</span> if (conn.isClosed()) {<a name="line.489"></a>
<span class="sourceLineNo">490</span> throw new DoNotRetryIOException("Connection is closed", e);<a name="line.490"></a>
<span class="sourceLineNo">491</span> } else {<a name="line.491"></a>
<span class="sourceLineNo">492</span> throw new HBaseIOException("Coprocessor operation is rejected", e);<a name="line.492"></a>
<span class="sourceLineNo">493</span> }<a name="line.493"></a>
<span class="sourceLineNo">494</span> }<a name="line.494"></a>
<span class="sourceLineNo">495</span> for (Map.Entry&lt;byte[], Future&lt;R&gt;&gt; e : futures.entrySet()) {<a name="line.495"></a>
<span class="sourceLineNo">496</span> try {<a name="line.496"></a>
<span class="sourceLineNo">497</span> e.getValue().get();<a name="line.497"></a>
<span class="sourceLineNo">498</span> } catch (ExecutionException ee) {<a name="line.498"></a>
<span class="sourceLineNo">499</span> LOG.warn("Error calling coprocessor service {} for row {}", serviceName,<a name="line.499"></a>
<span class="sourceLineNo">500</span> Bytes.toStringBinary(e.getKey()), ee);<a name="line.500"></a>
<span class="sourceLineNo">501</span> throw ee.getCause();<a name="line.501"></a>
<span class="sourceLineNo">502</span> } catch (InterruptedException ie) {<a name="line.502"></a>
<span class="sourceLineNo">503</span> throw new InterruptedIOException("Interrupted calling coprocessor service " + serviceName<a name="line.503"></a>
<span class="sourceLineNo">504</span> + " for row " + Bytes.toStringBinary(e.getKey())).initCause(ie);<a name="line.504"></a>
<span class="sourceLineNo">505</span> }<a name="line.505"></a>
<span class="sourceLineNo">506</span> }<a name="line.506"></a>
<span class="sourceLineNo">507</span> }<a name="line.507"></a>
<span class="sourceLineNo">508</span><a name="line.508"></a>
<span class="sourceLineNo">509</span> @Override<a name="line.509"></a>
<span class="sourceLineNo">510</span> public &lt;T extends Service, R&gt; void coprocessorService(Class&lt;T&gt; service, byte[] startKey,<a name="line.510"></a>
<span class="sourceLineNo">511</span> byte[] endKey, Batch.Call&lt;T, R&gt; callable, Batch.Callback&lt;R&gt; callback)<a name="line.511"></a>
<span class="sourceLineNo">512</span> throws ServiceException, Throwable {<a name="line.512"></a>
<span class="sourceLineNo">513</span> final Supplier&lt;Span&gt; supplier = new TableOperationSpanBuilder(conn)<a name="line.513"></a>
<span class="sourceLineNo">514</span> .setTableName(table.getName()).setOperation(HBaseSemanticAttributes.Operation.COPROC_EXEC);<a name="line.514"></a>
<span class="sourceLineNo">515</span> TraceUtil.trace(() -&gt; {<a name="line.515"></a>
<span class="sourceLineNo">516</span> final Context context = Context.current();<a name="line.516"></a>
<span class="sourceLineNo">517</span> coprocessorService(service.getName(), startKey, endKey, callback, channel -&gt; {<a name="line.517"></a>
<span class="sourceLineNo">518</span> try (Scope ignored = context.makeCurrent()) {<a name="line.518"></a>
<span class="sourceLineNo">519</span> T instance = ProtobufUtil.newServiceStub(service, channel);<a name="line.519"></a>
<span class="sourceLineNo">520</span> return callable.call(instance);<a name="line.520"></a>
<span class="sourceLineNo">521</span> }<a name="line.521"></a>
<span class="sourceLineNo">522</span> });<a name="line.522"></a>
<span class="sourceLineNo">523</span> }, supplier);<a name="line.523"></a>
<span class="sourceLineNo">524</span> }<a name="line.524"></a>
<span class="sourceLineNo">525</span><a name="line.525"></a>
<span class="sourceLineNo">526</span> @SuppressWarnings("unchecked")<a name="line.526"></a>
<span class="sourceLineNo">527</span> @Override<a name="line.527"></a>
<span class="sourceLineNo">528</span> public &lt;R extends Message&gt; void batchCoprocessorService(MethodDescriptor methodDescriptor,<a name="line.528"></a>
<span class="sourceLineNo">529</span> Message request, byte[] startKey, byte[] endKey, R responsePrototype,<a name="line.529"></a>
<span class="sourceLineNo">530</span> Batch.Callback&lt;R&gt; callback) throws ServiceException, Throwable {<a name="line.530"></a>
<span class="sourceLineNo">531</span> final Supplier&lt;Span&gt; supplier = new TableOperationSpanBuilder(conn)<a name="line.531"></a>
<span class="sourceLineNo">532</span> .setTableName(table.getName()).setOperation(HBaseSemanticAttributes.Operation.COPROC_EXEC);<a name="line.532"></a>
<span class="sourceLineNo">533</span> TraceUtil.trace(() -&gt; {<a name="line.533"></a>
<span class="sourceLineNo">534</span> final Context context = Context.current();<a name="line.534"></a>
<span class="sourceLineNo">535</span> coprocessorService(methodDescriptor.getFullName(), startKey, endKey, callback, channel -&gt; {<a name="line.535"></a>
<span class="sourceLineNo">536</span> try (Scope ignored = context.makeCurrent()) {<a name="line.536"></a>
<span class="sourceLineNo">537</span> return (R) channel.callBlockingMethod(methodDescriptor, null, request, responsePrototype);<a name="line.537"></a>
<span class="sourceLineNo">538</span> }<a name="line.538"></a>
<span class="sourceLineNo">539</span> });<a name="line.539"></a>
<span class="sourceLineNo">540</span> }, supplier);<a name="line.540"></a>
<span class="sourceLineNo">541</span> }<a name="line.541"></a>
<span class="sourceLineNo">542</span><a name="line.542"></a>
<span class="sourceLineNo">543</span> @Override<a name="line.543"></a>
<span class="sourceLineNo">544</span> public long getRpcTimeout(TimeUnit unit) {<a name="line.544"></a>
<span class="sourceLineNo">545</span> return table.getRpcTimeout(unit);<a name="line.545"></a>
<span class="sourceLineNo">546</span> }<a name="line.546"></a>
<span class="sourceLineNo">547</span><a name="line.547"></a>
<span class="sourceLineNo">548</span> @Override<a name="line.548"></a>
<span class="sourceLineNo">549</span> public long getReadRpcTimeout(TimeUnit unit) {<a name="line.549"></a>
<span class="sourceLineNo">550</span> return table.getReadRpcTimeout(unit);<a name="line.550"></a>
<span class="sourceLineNo">551</span> }<a name="line.551"></a>
<span class="sourceLineNo">552</span><a name="line.552"></a>
<span class="sourceLineNo">553</span> @Override<a name="line.553"></a>
<span class="sourceLineNo">554</span> public long getWriteRpcTimeout(TimeUnit unit) {<a name="line.554"></a>
<span class="sourceLineNo">555</span> return table.getWriteRpcTimeout(unit);<a name="line.555"></a>
<span class="sourceLineNo">556</span> }<a name="line.556"></a>
<span class="sourceLineNo">557</span><a name="line.557"></a>
<span class="sourceLineNo">558</span> @Override<a name="line.558"></a>
<span class="sourceLineNo">559</span> public long getOperationTimeout(TimeUnit unit) {<a name="line.559"></a>
<span class="sourceLineNo">560</span> return table.getOperationTimeout(unit);<a name="line.560"></a>
<span class="sourceLineNo">561</span> }<a name="line.561"></a>
<span class="sourceLineNo">562</span><a name="line.562"></a>
<span class="sourceLineNo">563</span> @Override<a name="line.563"></a>
<span class="sourceLineNo">564</span> public Map&lt;String, byte[]&gt; getRequestAttributes() {<a name="line.564"></a>
<span class="sourceLineNo">565</span> return table.getRequestAttributes();<a name="line.565"></a>
<span class="sourceLineNo">566</span> }<a name="line.566"></a>
<span class="sourceLineNo">567</span><a name="line.567"></a>
<span class="sourceLineNo">568</span> @Override<a name="line.568"></a>
<span class="sourceLineNo">569</span> public RegionLocator getRegionLocator() throws IOException {<a name="line.569"></a>
<span class="sourceLineNo">570</span> return conn.toConnection().getRegionLocator(getName());<a name="line.570"></a>
<span class="sourceLineNo">571</span> }<a name="line.571"></a>
<span class="sourceLineNo">572</span>}<a name="line.572"></a>
</pre>
</div>
</body>
</html>