| <!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.client, class: RawAsyncTableImpl, interface: NoncedConverter"> |
| <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.client;</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 static java.util.stream.Collectors.toList;</span> |
| <span class="source-line-no">021</span><span id="line-21">import static org.apache.hadoop.hbase.client.ConnectionUtils.checkHasFamilies;</span> |
| <span class="source-line-no">022</span><span id="line-22">import static org.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStopRow;</span> |
| <span class="source-line-no">023</span><span id="line-23">import static org.apache.hadoop.hbase.client.ConnectionUtils.timelineConsistentRead;</span> |
| <span class="source-line-no">024</span><span id="line-24">import static org.apache.hadoop.hbase.client.ConnectionUtils.validatePut;</span> |
| <span class="source-line-no">025</span><span id="line-25">import static org.apache.hadoop.hbase.client.ConnectionUtils.validatePutsInRowMutations;</span> |
| <span class="source-line-no">026</span><span id="line-26">import static org.apache.hadoop.hbase.trace.TraceUtil.tracedFuture;</span> |
| <span class="source-line-no">027</span><span id="line-27">import static org.apache.hadoop.hbase.trace.TraceUtil.tracedFutures;</span> |
| <span class="source-line-no">028</span><span id="line-28">import static org.apache.hadoop.hbase.util.FutureUtils.addListener;</span> |
| <span class="source-line-no">029</span><span id="line-29"></span> |
| <span class="source-line-no">030</span><span id="line-30">import io.opentelemetry.api.trace.Span;</span> |
| <span class="source-line-no">031</span><span id="line-31">import io.opentelemetry.api.trace.StatusCode;</span> |
| <span class="source-line-no">032</span><span id="line-32">import io.opentelemetry.context.Scope;</span> |
| <span class="source-line-no">033</span><span id="line-33">import java.io.IOException;</span> |
| <span class="source-line-no">034</span><span id="line-34">import java.util.ArrayList;</span> |
| <span class="source-line-no">035</span><span id="line-35">import java.util.Arrays;</span> |
| <span class="source-line-no">036</span><span id="line-36">import java.util.List;</span> |
| <span class="source-line-no">037</span><span id="line-37">import java.util.Map;</span> |
| <span class="source-line-no">038</span><span id="line-38">import java.util.concurrent.CompletableFuture;</span> |
| <span class="source-line-no">039</span><span id="line-39">import java.util.concurrent.TimeUnit;</span> |
| <span class="source-line-no">040</span><span id="line-40">import java.util.concurrent.atomic.AtomicBoolean;</span> |
| <span class="source-line-no">041</span><span id="line-41">import java.util.concurrent.atomic.AtomicInteger;</span> |
| <span class="source-line-no">042</span><span id="line-42">import java.util.function.Function;</span> |
| <span class="source-line-no">043</span><span id="line-43">import java.util.function.Supplier;</span> |
| <span class="source-line-no">044</span><span id="line-44">import org.apache.hadoop.conf.Configuration;</span> |
| <span class="source-line-no">045</span><span id="line-45">import org.apache.hadoop.hbase.CompareOperator;</span> |
| <span class="source-line-no">046</span><span id="line-46">import org.apache.hadoop.hbase.DoNotRetryIOException;</span> |
| <span class="source-line-no">047</span><span id="line-47">import org.apache.hadoop.hbase.HConstants;</span> |
| <span class="source-line-no">048</span><span id="line-48">import org.apache.hadoop.hbase.HRegionLocation;</span> |
| <span class="source-line-no">049</span><span id="line-49">import org.apache.hadoop.hbase.TableName;</span> |
| <span class="source-line-no">050</span><span id="line-50">import org.apache.hadoop.hbase.client.AsyncRpcRetryingCallerFactory.SingleRequestCallerBuilder;</span> |
| <span class="source-line-no">051</span><span id="line-51">import org.apache.hadoop.hbase.client.ConnectionUtils.Converter;</span> |
| <span class="source-line-no">052</span><span id="line-52">import org.apache.hadoop.hbase.client.trace.TableOperationSpanBuilder;</span> |
| <span class="source-line-no">053</span><span id="line-53">import org.apache.hadoop.hbase.filter.Filter;</span> |
| <span class="source-line-no">054</span><span id="line-54">import org.apache.hadoop.hbase.io.TimeRange;</span> |
| <span class="source-line-no">055</span><span id="line-55">import org.apache.hadoop.hbase.ipc.HBaseRpcController;</span> |
| <span class="source-line-no">056</span><span id="line-56">import org.apache.hadoop.hbase.trace.HBaseSemanticAttributes;</span> |
| <span class="source-line-no">057</span><span id="line-57">import org.apache.hadoop.hbase.trace.TraceUtil;</span> |
| <span class="source-line-no">058</span><span id="line-58">import org.apache.hadoop.hbase.util.Bytes;</span> |
| <span class="source-line-no">059</span><span id="line-59">import org.apache.hadoop.hbase.util.ReflectionUtils;</span> |
| <span class="source-line-no">060</span><span id="line-60">import org.apache.yetus.audience.InterfaceAudience;</span> |
| <span class="source-line-no">061</span><span id="line-61">import org.slf4j.Logger;</span> |
| <span class="source-line-no">062</span><span id="line-62">import org.slf4j.LoggerFactory;</span> |
| <span class="source-line-no">063</span><span id="line-63"></span> |
| <span class="source-line-no">064</span><span id="line-64">import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;</span> |
| <span class="source-line-no">065</span><span id="line-65">import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;</span> |
| <span class="source-line-no">066</span><span id="line-66">import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;</span> |
| <span class="source-line-no">067</span><span id="line-67">import org.apache.hbase.thirdparty.io.netty.util.Timer;</span> |
| <span class="source-line-no">068</span><span id="line-68"></span> |
| <span class="source-line-no">069</span><span id="line-69">import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;</span> |
| <span class="source-line-no">070</span><span id="line-70">import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;</span> |
| <span class="source-line-no">071</span><span id="line-71">import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;</span> |
| <span class="source-line-no">072</span><span id="line-72">import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;</span> |
| <span class="source-line-no">073</span><span id="line-73">import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest;</span> |
| <span class="source-line-no">074</span><span id="line-74">import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetResponse;</span> |
| <span class="source-line-no">075</span><span id="line-75">import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest;</span> |
| <span class="source-line-no">076</span><span id="line-76">import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiResponse;</span> |
| <span class="source-line-no">077</span><span id="line-77">import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest;</span> |
| <span class="source-line-no">078</span><span id="line-78">import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse;</span> |
| <span class="source-line-no">079</span><span id="line-79"></span> |
| <span class="source-line-no">080</span><span id="line-80">/**</span> |
| <span class="source-line-no">081</span><span id="line-81"> * The implementation of RawAsyncTable.</span> |
| <span class="source-line-no">082</span><span id="line-82"> * <p/></span> |
| <span class="source-line-no">083</span><span id="line-83"> * The word 'Raw' means that this is a low level class. The returned {@link CompletableFuture} will</span> |
| <span class="source-line-no">084</span><span id="line-84"> * be finished inside the rpc framework thread, which means that the callbacks registered to the</span> |
| <span class="source-line-no">085</span><span id="line-85"> * {@link CompletableFuture} will also be executed inside the rpc framework thread. So users who use</span> |
| <span class="source-line-no">086</span><span id="line-86"> * this class should not try to do time consuming tasks in the callbacks.</span> |
| <span class="source-line-no">087</span><span id="line-87"> * @since 2.0.0</span> |
| <span class="source-line-no">088</span><span id="line-88"> * @see AsyncTableImpl</span> |
| <span class="source-line-no">089</span><span id="line-89"> */</span> |
| <span class="source-line-no">090</span><span id="line-90">@InterfaceAudience.Private</span> |
| <span class="source-line-no">091</span><span id="line-91">class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {</span> |
| <span class="source-line-no">092</span><span id="line-92"></span> |
| <span class="source-line-no">093</span><span id="line-93"> private static final Logger LOG = LoggerFactory.getLogger(RawAsyncTableImpl.class);</span> |
| <span class="source-line-no">094</span><span id="line-94"></span> |
| <span class="source-line-no">095</span><span id="line-95"> private final AsyncConnectionImpl conn;</span> |
| <span class="source-line-no">096</span><span id="line-96"></span> |
| <span class="source-line-no">097</span><span id="line-97"> private final Timer retryTimer;</span> |
| <span class="source-line-no">098</span><span id="line-98"></span> |
| <span class="source-line-no">099</span><span id="line-99"> private final TableName tableName;</span> |
| <span class="source-line-no">100</span><span id="line-100"></span> |
| <span class="source-line-no">101</span><span id="line-101"> private final int defaultScannerCaching;</span> |
| <span class="source-line-no">102</span><span id="line-102"></span> |
| <span class="source-line-no">103</span><span id="line-103"> private final long defaultScannerMaxResultSize;</span> |
| <span class="source-line-no">104</span><span id="line-104"></span> |
| <span class="source-line-no">105</span><span id="line-105"> private final long rpcTimeoutNs;</span> |
| <span class="source-line-no">106</span><span id="line-106"></span> |
| <span class="source-line-no">107</span><span id="line-107"> private final long readRpcTimeoutNs;</span> |
| <span class="source-line-no">108</span><span id="line-108"></span> |
| <span class="source-line-no">109</span><span id="line-109"> private final long writeRpcTimeoutNs;</span> |
| <span class="source-line-no">110</span><span id="line-110"></span> |
| <span class="source-line-no">111</span><span id="line-111"> private final long operationTimeoutNs;</span> |
| <span class="source-line-no">112</span><span id="line-112"></span> |
| <span class="source-line-no">113</span><span id="line-113"> private final long scanTimeoutNs;</span> |
| <span class="source-line-no">114</span><span id="line-114"></span> |
| <span class="source-line-no">115</span><span id="line-115"> private final long pauseNs;</span> |
| <span class="source-line-no">116</span><span id="line-116"></span> |
| <span class="source-line-no">117</span><span id="line-117"> private final long pauseNsForServerOverloaded;</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 final int maxAttempts;</span> |
| <span class="source-line-no">120</span><span id="line-120"></span> |
| <span class="source-line-no">121</span><span id="line-121"> private final int startLogErrorsCnt;</span> |
| <span class="source-line-no">122</span><span id="line-122"></span> |
| <span class="source-line-no">123</span><span id="line-123"> private final Map<String, byte[]> requestAttributes;</span> |
| <span class="source-line-no">124</span><span id="line-124"></span> |
| <span class="source-line-no">125</span><span id="line-125"> RawAsyncTableImpl(AsyncConnectionImpl conn, Timer retryTimer, AsyncTableBuilderBase<?> builder) {</span> |
| <span class="source-line-no">126</span><span id="line-126"> this.conn = conn;</span> |
| <span class="source-line-no">127</span><span id="line-127"> this.retryTimer = retryTimer;</span> |
| <span class="source-line-no">128</span><span id="line-128"> this.tableName = builder.tableName;</span> |
| <span class="source-line-no">129</span><span id="line-129"> this.rpcTimeoutNs = builder.rpcTimeoutNs;</span> |
| <span class="source-line-no">130</span><span id="line-130"> this.readRpcTimeoutNs = builder.readRpcTimeoutNs;</span> |
| <span class="source-line-no">131</span><span id="line-131"> this.writeRpcTimeoutNs = builder.writeRpcTimeoutNs;</span> |
| <span class="source-line-no">132</span><span id="line-132"> this.operationTimeoutNs = builder.operationTimeoutNs;</span> |
| <span class="source-line-no">133</span><span id="line-133"> this.scanTimeoutNs = builder.scanTimeoutNs;</span> |
| <span class="source-line-no">134</span><span id="line-134"> this.pauseNs = builder.pauseNs;</span> |
| <span class="source-line-no">135</span><span id="line-135"> if (builder.pauseNsForServerOverloaded < builder.pauseNs) {</span> |
| <span class="source-line-no">136</span><span id="line-136"> LOG.warn(</span> |
| <span class="source-line-no">137</span><span id="line-137"> "Configured value of pauseNsForServerOverloaded is {} ms, which is less than"</span> |
| <span class="source-line-no">138</span><span id="line-138"> + " the normal pause value {} ms, use the greater one instead",</span> |
| <span class="source-line-no">139</span><span id="line-139"> TimeUnit.NANOSECONDS.toMillis(builder.pauseNsForServerOverloaded),</span> |
| <span class="source-line-no">140</span><span id="line-140"> TimeUnit.NANOSECONDS.toMillis(builder.pauseNs));</span> |
| <span class="source-line-no">141</span><span id="line-141"> this.pauseNsForServerOverloaded = builder.pauseNs;</span> |
| <span class="source-line-no">142</span><span id="line-142"> } else {</span> |
| <span class="source-line-no">143</span><span id="line-143"> this.pauseNsForServerOverloaded = builder.pauseNsForServerOverloaded;</span> |
| <span class="source-line-no">144</span><span id="line-144"> }</span> |
| <span class="source-line-no">145</span><span id="line-145"> this.maxAttempts = builder.maxAttempts;</span> |
| <span class="source-line-no">146</span><span id="line-146"> this.startLogErrorsCnt = builder.startLogErrorsCnt;</span> |
| <span class="source-line-no">147</span><span id="line-147"> this.defaultScannerCaching = tableName.isSystemTable()</span> |
| <span class="source-line-no">148</span><span id="line-148"> ? conn.connConf.getMetaScannerCaching()</span> |
| <span class="source-line-no">149</span><span id="line-149"> : conn.connConf.getScannerCaching();</span> |
| <span class="source-line-no">150</span><span id="line-150"> this.defaultScannerMaxResultSize = conn.connConf.getScannerMaxResultSize();</span> |
| <span class="source-line-no">151</span><span id="line-151"> this.requestAttributes = builder.requestAttributes;</span> |
| <span class="source-line-no">152</span><span id="line-152"> }</span> |
| <span class="source-line-no">153</span><span id="line-153"></span> |
| <span class="source-line-no">154</span><span id="line-154"> @Override</span> |
| <span class="source-line-no">155</span><span id="line-155"> public TableName getName() {</span> |
| <span class="source-line-no">156</span><span id="line-156"> return tableName;</span> |
| <span class="source-line-no">157</span><span id="line-157"> }</span> |
| <span class="source-line-no">158</span><span id="line-158"></span> |
| <span class="source-line-no">159</span><span id="line-159"> @Override</span> |
| <span class="source-line-no">160</span><span id="line-160"> public Configuration getConfiguration() {</span> |
| <span class="source-line-no">161</span><span id="line-161"> return conn.getConfiguration();</span> |
| <span class="source-line-no">162</span><span id="line-162"> }</span> |
| <span class="source-line-no">163</span><span id="line-163"></span> |
| <span class="source-line-no">164</span><span id="line-164"> @Override</span> |
| <span class="source-line-no">165</span><span id="line-165"> public CompletableFuture<TableDescriptor> getDescriptor() {</span> |
| <span class="source-line-no">166</span><span id="line-166"> return conn.getAdmin().getDescriptor(tableName);</span> |
| <span class="source-line-no">167</span><span id="line-167"> }</span> |
| <span class="source-line-no">168</span><span id="line-168"></span> |
| <span class="source-line-no">169</span><span id="line-169"> @Override</span> |
| <span class="source-line-no">170</span><span id="line-170"> public AsyncTableRegionLocator getRegionLocator() {</span> |
| <span class="source-line-no">171</span><span id="line-171"> return conn.getRegionLocator(tableName);</span> |
| <span class="source-line-no">172</span><span id="line-172"> }</span> |
| <span class="source-line-no">173</span><span id="line-173"></span> |
| <span class="source-line-no">174</span><span id="line-174"> private static <REQ, RESP> CompletableFuture<RESP> mutate(HBaseRpcController controller,</span> |
| <span class="source-line-no">175</span><span id="line-175"> HRegionLocation loc, ClientService.Interface stub, REQ req,</span> |
| <span class="source-line-no">176</span><span id="line-176"> Converter<MutateRequest, byte[], REQ> reqConvert,</span> |
| <span class="source-line-no">177</span><span id="line-177"> Converter<RESP, HBaseRpcController, MutateResponse> respConverter) {</span> |
| <span class="source-line-no">178</span><span id="line-178"> return ConnectionUtils.call(controller, loc, stub, req, reqConvert,</span> |
| <span class="source-line-no">179</span><span id="line-179"> (s, c, r, done) -> s.mutate(c, r, done), respConverter);</span> |
| <span class="source-line-no">180</span><span id="line-180"> }</span> |
| <span class="source-line-no">181</span><span id="line-181"></span> |
| <span class="source-line-no">182</span><span id="line-182"> private static <REQ> CompletableFuture<Void> voidMutate(HBaseRpcController controller,</span> |
| <span class="source-line-no">183</span><span id="line-183"> HRegionLocation loc, ClientService.Interface stub, REQ req,</span> |
| <span class="source-line-no">184</span><span id="line-184"> Converter<MutateRequest, byte[], REQ> reqConvert) {</span> |
| <span class="source-line-no">185</span><span id="line-185"> return mutate(controller, loc, stub, req, reqConvert, (c, resp) -> {</span> |
| <span class="source-line-no">186</span><span id="line-186"> return null;</span> |
| <span class="source-line-no">187</span><span id="line-187"> });</span> |
| <span class="source-line-no">188</span><span id="line-188"> }</span> |
| <span class="source-line-no">189</span><span id="line-189"></span> |
| <span class="source-line-no">190</span><span id="line-190"> private static Result toResult(HBaseRpcController controller, MutateResponse resp)</span> |
| <span class="source-line-no">191</span><span id="line-191"> throws IOException {</span> |
| <span class="source-line-no">192</span><span id="line-192"> if (!resp.hasResult()) {</span> |
| <span class="source-line-no">193</span><span id="line-193"> return null;</span> |
| <span class="source-line-no">194</span><span id="line-194"> }</span> |
| <span class="source-line-no">195</span><span id="line-195"> return ProtobufUtil.toResult(resp.getResult(), controller.cellScanner());</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"> @FunctionalInterface</span> |
| <span class="source-line-no">199</span><span id="line-199"> private interface NoncedConverter<D, I, S> {</span> |
| <span class="source-line-no">200</span><span id="line-200"> D convert(I info, S src, long nonceGroup, long nonce) throws IOException;</span> |
| <span class="source-line-no">201</span><span id="line-201"> }</span> |
| <span class="source-line-no">202</span><span id="line-202"></span> |
| <span class="source-line-no">203</span><span id="line-203"> private <REQ, RESP> CompletableFuture<RESP> noncedMutate(long nonceGroup, long nonce,</span> |
| <span class="source-line-no">204</span><span id="line-204"> HBaseRpcController controller, HRegionLocation loc, ClientService.Interface stub, REQ req,</span> |
| <span class="source-line-no">205</span><span id="line-205"> NoncedConverter<MutateRequest, byte[], REQ> reqConvert,</span> |
| <span class="source-line-no">206</span><span id="line-206"> Converter<RESP, HBaseRpcController, MutateResponse> respConverter) {</span> |
| <span class="source-line-no">207</span><span id="line-207"> return mutate(controller, loc, stub, req,</span> |
| <span class="source-line-no">208</span><span id="line-208"> (info, src) -> reqConvert.convert(info, src, nonceGroup, nonce), respConverter);</span> |
| <span class="source-line-no">209</span><span id="line-209"> }</span> |
| <span class="source-line-no">210</span><span id="line-210"></span> |
| <span class="source-line-no">211</span><span id="line-211"> private <T> SingleRequestCallerBuilder<T> newCaller(byte[] row, int priority, long rpcTimeoutNs) {</span> |
| <span class="source-line-no">212</span><span id="line-212"> return conn.callerFactory.<T> single().table(tableName).row(row).priority(priority)</span> |
| <span class="source-line-no">213</span><span id="line-213"> .rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS)</span> |
| <span class="source-line-no">214</span><span id="line-214"> .operationTimeout(operationTimeoutNs, TimeUnit.NANOSECONDS)</span> |
| <span class="source-line-no">215</span><span id="line-215"> .pause(pauseNs, TimeUnit.NANOSECONDS)</span> |
| <span class="source-line-no">216</span><span id="line-216"> .pauseForServerOverloaded(pauseNsForServerOverloaded, TimeUnit.NANOSECONDS)</span> |
| <span class="source-line-no">217</span><span id="line-217"> .maxAttempts(maxAttempts).setRequestAttributes(requestAttributes)</span> |
| <span class="source-line-no">218</span><span id="line-218"> .startLogErrorsCnt(startLogErrorsCnt).setRequestAttributes(requestAttributes);</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"> private <T, R extends OperationWithAttributes & Row> SingleRequestCallerBuilder<T></span> |
| <span class="source-line-no">222</span><span id="line-222"> newCaller(R row, long rpcTimeoutNs) {</span> |
| <span class="source-line-no">223</span><span id="line-223"> return newCaller(row.getRow(), row.getPriority(), rpcTimeoutNs);</span> |
| <span class="source-line-no">224</span><span id="line-224"> }</span> |
| <span class="source-line-no">225</span><span id="line-225"></span> |
| <span class="source-line-no">226</span><span id="line-226"> private CompletableFuture<Result> get(Get get, int replicaId) {</span> |
| <span class="source-line-no">227</span><span id="line-227"> return this.<Result, Get> newCaller(get, readRpcTimeoutNs)</span> |
| <span class="source-line-no">228</span><span id="line-228"> .action((controller, loc, stub) -> ConnectionUtils.<Get, GetRequest, GetResponse,</span> |
| <span class="source-line-no">229</span><span id="line-229"> Result> call(controller, loc, stub, get, RequestConverter::buildGetRequest,</span> |
| <span class="source-line-no">230</span><span id="line-230"> (s, c, req, done) -> s.get(c, req, done),</span> |
| <span class="source-line-no">231</span><span id="line-231"> (c, resp) -> ProtobufUtil.toResult(resp.getResult(), c.cellScanner())))</span> |
| <span class="source-line-no">232</span><span id="line-232"> .replicaId(replicaId).call();</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"> private TableOperationSpanBuilder newTableOperationSpanBuilder() {</span> |
| <span class="source-line-no">236</span><span id="line-236"> return new TableOperationSpanBuilder(conn).setTableName(tableName);</span> |
| <span class="source-line-no">237</span><span id="line-237"> }</span> |
| <span class="source-line-no">238</span><span id="line-238"></span> |
| <span class="source-line-no">239</span><span id="line-239"> @Override</span> |
| <span class="source-line-no">240</span><span id="line-240"> public CompletableFuture<Result> get(Get get) {</span> |
| <span class="source-line-no">241</span><span id="line-241"> final Supplier<Span> supplier = newTableOperationSpanBuilder().setOperation(get);</span> |
| <span class="source-line-no">242</span><span id="line-242"> return tracedFuture(</span> |
| <span class="source-line-no">243</span><span id="line-243"> () -> timelineConsistentRead(conn.getLocator(), tableName, get, get.getRow(),</span> |
| <span class="source-line-no">244</span><span id="line-244"> RegionLocateType.CURRENT, replicaId -> get(get, replicaId), readRpcTimeoutNs,</span> |
| <span class="source-line-no">245</span><span id="line-245"> conn.connConf.getPrimaryCallTimeoutNs(), retryTimer, conn.getConnectionMetrics()),</span> |
| <span class="source-line-no">246</span><span id="line-246"> supplier);</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"> @Override</span> |
| <span class="source-line-no">250</span><span id="line-250"> public CompletableFuture<Void> put(Put put) {</span> |
| <span class="source-line-no">251</span><span id="line-251"> validatePut(put, conn.connConf.getMaxKeyValueSize());</span> |
| <span class="source-line-no">252</span><span id="line-252"> final Supplier<Span> supplier = newTableOperationSpanBuilder().setOperation(put);</span> |
| <span class="source-line-no">253</span><span id="line-253"> return tracedFuture(() -> this.<Void, Put> newCaller(put, writeRpcTimeoutNs)</span> |
| <span class="source-line-no">254</span><span id="line-254"> .action((controller, loc, stub) -> RawAsyncTableImpl.<Put> voidMutate(controller, loc, stub,</span> |
| <span class="source-line-no">255</span><span id="line-255"> put, RequestConverter::buildMutateRequest))</span> |
| <span class="source-line-no">256</span><span id="line-256"> .call(), supplier);</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"> @Override</span> |
| <span class="source-line-no">260</span><span id="line-260"> public CompletableFuture<Void> delete(Delete delete) {</span> |
| <span class="source-line-no">261</span><span id="line-261"> final Supplier<Span> supplier = newTableOperationSpanBuilder().setOperation(delete);</span> |
| <span class="source-line-no">262</span><span id="line-262"> return tracedFuture(() -> this.<Void, Delete> newCaller(delete, writeRpcTimeoutNs)</span> |
| <span class="source-line-no">263</span><span id="line-263"> .action((controller, loc, stub) -> RawAsyncTableImpl.<Delete> voidMutate(controller, loc,</span> |
| <span class="source-line-no">264</span><span id="line-264"> stub, delete, RequestConverter::buildMutateRequest))</span> |
| <span class="source-line-no">265</span><span id="line-265"> .call(), supplier);</span> |
| <span class="source-line-no">266</span><span id="line-266"> }</span> |
| <span class="source-line-no">267</span><span id="line-267"></span> |
| <span class="source-line-no">268</span><span id="line-268"> @Override</span> |
| <span class="source-line-no">269</span><span id="line-269"> public CompletableFuture<Result> append(Append append) {</span> |
| <span class="source-line-no">270</span><span id="line-270"> checkHasFamilies(append);</span> |
| <span class="source-line-no">271</span><span id="line-271"> final Supplier<Span> supplier = newTableOperationSpanBuilder().setOperation(append);</span> |
| <span class="source-line-no">272</span><span id="line-272"> return tracedFuture(() -> {</span> |
| <span class="source-line-no">273</span><span id="line-273"> long nonceGroup = conn.getNonceGenerator().getNonceGroup();</span> |
| <span class="source-line-no">274</span><span id="line-274"> long nonce = conn.getNonceGenerator().newNonce();</span> |
| <span class="source-line-no">275</span><span id="line-275"> return this.<Result, Append> newCaller(append, rpcTimeoutNs)</span> |
| <span class="source-line-no">276</span><span id="line-276"> .action((controller, loc, stub) -> this.<Append, Result> noncedMutate(nonceGroup, nonce,</span> |
| <span class="source-line-no">277</span><span id="line-277"> controller, loc, stub, append, RequestConverter::buildMutateRequest,</span> |
| <span class="source-line-no">278</span><span id="line-278"> RawAsyncTableImpl::toResult))</span> |
| <span class="source-line-no">279</span><span id="line-279"> .call();</span> |
| <span class="source-line-no">280</span><span id="line-280"> }, supplier);</span> |
| <span class="source-line-no">281</span><span id="line-281"> }</span> |
| <span class="source-line-no">282</span><span id="line-282"></span> |
| <span class="source-line-no">283</span><span id="line-283"> @Override</span> |
| <span class="source-line-no">284</span><span id="line-284"> public CompletableFuture<Result> increment(Increment increment) {</span> |
| <span class="source-line-no">285</span><span id="line-285"> checkHasFamilies(increment);</span> |
| <span class="source-line-no">286</span><span id="line-286"> final Supplier<Span> supplier = newTableOperationSpanBuilder().setOperation(increment);</span> |
| <span class="source-line-no">287</span><span id="line-287"> return tracedFuture(() -> {</span> |
| <span class="source-line-no">288</span><span id="line-288"> long nonceGroup = conn.getNonceGenerator().getNonceGroup();</span> |
| <span class="source-line-no">289</span><span id="line-289"> long nonce = conn.getNonceGenerator().newNonce();</span> |
| <span class="source-line-no">290</span><span id="line-290"> return this.<Result, Increment> newCaller(increment, rpcTimeoutNs)</span> |
| <span class="source-line-no">291</span><span id="line-291"> .action((controller, loc, stub) -> this.<Increment, Result> noncedMutate(nonceGroup, nonce,</span> |
| <span class="source-line-no">292</span><span id="line-292"> controller, loc, stub, increment, RequestConverter::buildMutateRequest,</span> |
| <span class="source-line-no">293</span><span id="line-293"> RawAsyncTableImpl::toResult))</span> |
| <span class="source-line-no">294</span><span id="line-294"> .call();</span> |
| <span class="source-line-no">295</span><span id="line-295"> }, supplier);</span> |
| <span class="source-line-no">296</span><span id="line-296"> }</span> |
| <span class="source-line-no">297</span><span id="line-297"></span> |
| <span class="source-line-no">298</span><span id="line-298"> private final class CheckAndMutateBuilderImpl implements CheckAndMutateBuilder {</span> |
| <span class="source-line-no">299</span><span id="line-299"></span> |
| <span class="source-line-no">300</span><span id="line-300"> private final byte[] row;</span> |
| <span class="source-line-no">301</span><span id="line-301"></span> |
| <span class="source-line-no">302</span><span id="line-302"> private final byte[] family;</span> |
| <span class="source-line-no">303</span><span id="line-303"></span> |
| <span class="source-line-no">304</span><span id="line-304"> private byte[] qualifier;</span> |
| <span class="source-line-no">305</span><span id="line-305"></span> |
| <span class="source-line-no">306</span><span id="line-306"> private TimeRange timeRange;</span> |
| <span class="source-line-no">307</span><span id="line-307"></span> |
| <span class="source-line-no">308</span><span id="line-308"> private CompareOperator op;</span> |
| <span class="source-line-no">309</span><span id="line-309"></span> |
| <span class="source-line-no">310</span><span id="line-310"> private byte[] value;</span> |
| <span class="source-line-no">311</span><span id="line-311"></span> |
| <span class="source-line-no">312</span><span id="line-312"> public CheckAndMutateBuilderImpl(byte[] row, byte[] family) {</span> |
| <span class="source-line-no">313</span><span id="line-313"> this.row = Preconditions.checkNotNull(row, "row is null");</span> |
| <span class="source-line-no">314</span><span id="line-314"> this.family = Preconditions.checkNotNull(family, "family is null");</span> |
| <span class="source-line-no">315</span><span id="line-315"> }</span> |
| <span class="source-line-no">316</span><span id="line-316"></span> |
| <span class="source-line-no">317</span><span id="line-317"> @Override</span> |
| <span class="source-line-no">318</span><span id="line-318"> public CheckAndMutateBuilder qualifier(byte[] qualifier) {</span> |
| <span class="source-line-no">319</span><span id="line-319"> this.qualifier = Preconditions.checkNotNull(qualifier, "qualifier is null. Consider using"</span> |
| <span class="source-line-no">320</span><span id="line-320"> + " an empty byte array, or just do not call this method if you want a null qualifier");</span> |
| <span class="source-line-no">321</span><span id="line-321"> return this;</span> |
| <span class="source-line-no">322</span><span id="line-322"> }</span> |
| <span class="source-line-no">323</span><span id="line-323"></span> |
| <span class="source-line-no">324</span><span id="line-324"> @Override</span> |
| <span class="source-line-no">325</span><span id="line-325"> public CheckAndMutateBuilder timeRange(TimeRange timeRange) {</span> |
| <span class="source-line-no">326</span><span id="line-326"> this.timeRange = timeRange;</span> |
| <span class="source-line-no">327</span><span id="line-327"> return this;</span> |
| <span class="source-line-no">328</span><span id="line-328"> }</span> |
| <span class="source-line-no">329</span><span id="line-329"></span> |
| <span class="source-line-no">330</span><span id="line-330"> @Override</span> |
| <span class="source-line-no">331</span><span id="line-331"> public CheckAndMutateBuilder ifNotExists() {</span> |
| <span class="source-line-no">332</span><span id="line-332"> this.op = CompareOperator.EQUAL;</span> |
| <span class="source-line-no">333</span><span id="line-333"> this.value = null;</span> |
| <span class="source-line-no">334</span><span id="line-334"> return this;</span> |
| <span class="source-line-no">335</span><span id="line-335"> }</span> |
| <span class="source-line-no">336</span><span id="line-336"></span> |
| <span class="source-line-no">337</span><span id="line-337"> @Override</span> |
| <span class="source-line-no">338</span><span id="line-338"> public CheckAndMutateBuilder ifMatches(CompareOperator compareOp, byte[] value) {</span> |
| <span class="source-line-no">339</span><span id="line-339"> this.op = Preconditions.checkNotNull(compareOp, "compareOp is null");</span> |
| <span class="source-line-no">340</span><span id="line-340"> this.value = Preconditions.checkNotNull(value, "value is null");</span> |
| <span class="source-line-no">341</span><span id="line-341"> return this;</span> |
| <span class="source-line-no">342</span><span id="line-342"> }</span> |
| <span class="source-line-no">343</span><span id="line-343"></span> |
| <span class="source-line-no">344</span><span id="line-344"> private void preCheck() {</span> |
| <span class="source-line-no">345</span><span id="line-345"> Preconditions.checkNotNull(op, "condition is null. You need to specify the condition by"</span> |
| <span class="source-line-no">346</span><span id="line-346"> + " calling ifNotExists/ifEquals/ifMatches before executing the request");</span> |
| <span class="source-line-no">347</span><span id="line-347"> }</span> |
| <span class="source-line-no">348</span><span id="line-348"></span> |
| <span class="source-line-no">349</span><span id="line-349"> @Override</span> |
| <span class="source-line-no">350</span><span id="line-350"> public CompletableFuture<Boolean> thenPut(Put put) {</span> |
| <span class="source-line-no">351</span><span id="line-351"> validatePut(put, conn.connConf.getMaxKeyValueSize());</span> |
| <span class="source-line-no">352</span><span id="line-352"> preCheck();</span> |
| <span class="source-line-no">353</span><span id="line-353"> final Supplier<Span> supplier = newTableOperationSpanBuilder()</span> |
| <span class="source-line-no">354</span><span id="line-354"> .setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE)</span> |
| <span class="source-line-no">355</span><span id="line-355"> .setContainerOperations(put);</span> |
| <span class="source-line-no">356</span><span id="line-356"> return tracedFuture(</span> |
| <span class="source-line-no">357</span><span id="line-357"> () -> RawAsyncTableImpl.this.<Boolean> newCaller(row, put.getPriority(), rpcTimeoutNs)</span> |
| <span class="source-line-no">358</span><span id="line-358"> .action((controller, loc, stub) -> RawAsyncTableImpl.mutate(controller, loc, stub, put,</span> |
| <span class="source-line-no">359</span><span id="line-359"> (rn, p) -> RequestConverter.buildMutateRequest(rn, row, family, qualifier, op, value,</span> |
| <span class="source-line-no">360</span><span id="line-360"> null, timeRange, p, HConstants.NO_NONCE, HConstants.NO_NONCE),</span> |
| <span class="source-line-no">361</span><span id="line-361"> (c, r) -> r.getProcessed()))</span> |
| <span class="source-line-no">362</span><span id="line-362"> .call(),</span> |
| <span class="source-line-no">363</span><span id="line-363"> supplier);</span> |
| <span class="source-line-no">364</span><span id="line-364"> }</span> |
| <span class="source-line-no">365</span><span id="line-365"></span> |
| <span class="source-line-no">366</span><span id="line-366"> @Override</span> |
| <span class="source-line-no">367</span><span id="line-367"> public CompletableFuture<Boolean> thenDelete(Delete delete) {</span> |
| <span class="source-line-no">368</span><span id="line-368"> preCheck();</span> |
| <span class="source-line-no">369</span><span id="line-369"> final Supplier<Span> supplier = newTableOperationSpanBuilder()</span> |
| <span class="source-line-no">370</span><span id="line-370"> .setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE)</span> |
| <span class="source-line-no">371</span><span id="line-371"> .setContainerOperations(delete);</span> |
| <span class="source-line-no">372</span><span id="line-372"> return tracedFuture(</span> |
| <span class="source-line-no">373</span><span id="line-373"> () -> RawAsyncTableImpl.this.<Boolean> newCaller(row, delete.getPriority(), rpcTimeoutNs)</span> |
| <span class="source-line-no">374</span><span id="line-374"> .action((controller, loc, stub) -> RawAsyncTableImpl.mutate(controller, loc, stub, delete,</span> |
| <span class="source-line-no">375</span><span id="line-375"> (rn, d) -> RequestConverter.buildMutateRequest(rn, row, family, qualifier, op, value,</span> |
| <span class="source-line-no">376</span><span id="line-376"> null, timeRange, d, HConstants.NO_NONCE, HConstants.NO_NONCE),</span> |
| <span class="source-line-no">377</span><span id="line-377"> (c, r) -> r.getProcessed()))</span> |
| <span class="source-line-no">378</span><span id="line-378"> .call(),</span> |
| <span class="source-line-no">379</span><span id="line-379"> supplier);</span> |
| <span class="source-line-no">380</span><span id="line-380"> }</span> |
| <span class="source-line-no">381</span><span id="line-381"></span> |
| <span class="source-line-no">382</span><span id="line-382"> @Override</span> |
| <span class="source-line-no">383</span><span id="line-383"> public CompletableFuture<Boolean> thenMutate(RowMutations mutations) {</span> |
| <span class="source-line-no">384</span><span id="line-384"> preCheck();</span> |
| <span class="source-line-no">385</span><span id="line-385"> validatePutsInRowMutations(mutations, conn.connConf.getMaxKeyValueSize());</span> |
| <span class="source-line-no">386</span><span id="line-386"> final Supplier<Span> supplier = newTableOperationSpanBuilder()</span> |
| <span class="source-line-no">387</span><span id="line-387"> .setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE)</span> |
| <span class="source-line-no">388</span><span id="line-388"> .setContainerOperations(mutations);</span> |
| <span class="source-line-no">389</span><span id="line-389"> return tracedFuture(() -> RawAsyncTableImpl.this</span> |
| <span class="source-line-no">390</span><span id="line-390"> .<Boolean> newCaller(row, mutations.getMaxPriority(), rpcTimeoutNs)</span> |
| <span class="source-line-no">391</span><span id="line-391"> .action((controller, loc, stub) -> RawAsyncTableImpl.this.mutateRow(controller, loc, stub,</span> |
| <span class="source-line-no">392</span><span id="line-392"> mutations,</span> |
| <span class="source-line-no">393</span><span id="line-393"> (rn, rm) -> RequestConverter.buildMultiRequest(rn, row, family, qualifier, op, value,</span> |
| <span class="source-line-no">394</span><span id="line-394"> null, timeRange, rm, HConstants.NO_NONCE, HConstants.NO_NONCE),</span> |
| <span class="source-line-no">395</span><span id="line-395"> CheckAndMutateResult::isSuccess))</span> |
| <span class="source-line-no">396</span><span id="line-396"> .call(), supplier);</span> |
| <span class="source-line-no">397</span><span id="line-397"> }</span> |
| <span class="source-line-no">398</span><span id="line-398"> }</span> |
| <span class="source-line-no">399</span><span id="line-399"></span> |
| <span class="source-line-no">400</span><span id="line-400"> @Override</span> |
| <span class="source-line-no">401</span><span id="line-401"> public CheckAndMutateBuilder checkAndMutate(byte[] row, byte[] family) {</span> |
| <span class="source-line-no">402</span><span id="line-402"> return new CheckAndMutateBuilderImpl(row, family);</span> |
| <span class="source-line-no">403</span><span id="line-403"> }</span> |
| <span class="source-line-no">404</span><span id="line-404"></span> |
| <span class="source-line-no">405</span><span id="line-405"> private final class CheckAndMutateWithFilterBuilderImpl</span> |
| <span class="source-line-no">406</span><span id="line-406"> implements CheckAndMutateWithFilterBuilder {</span> |
| <span class="source-line-no">407</span><span id="line-407"></span> |
| <span class="source-line-no">408</span><span id="line-408"> private final byte[] row;</span> |
| <span class="source-line-no">409</span><span id="line-409"></span> |
| <span class="source-line-no">410</span><span id="line-410"> private final Filter filter;</span> |
| <span class="source-line-no">411</span><span id="line-411"></span> |
| <span class="source-line-no">412</span><span id="line-412"> private TimeRange timeRange;</span> |
| <span class="source-line-no">413</span><span id="line-413"></span> |
| <span class="source-line-no">414</span><span id="line-414"> public CheckAndMutateWithFilterBuilderImpl(byte[] row, Filter filter) {</span> |
| <span class="source-line-no">415</span><span id="line-415"> this.row = Preconditions.checkNotNull(row, "row is null");</span> |
| <span class="source-line-no">416</span><span id="line-416"> this.filter = Preconditions.checkNotNull(filter, "filter is null");</span> |
| <span class="source-line-no">417</span><span id="line-417"> }</span> |
| <span class="source-line-no">418</span><span id="line-418"></span> |
| <span class="source-line-no">419</span><span id="line-419"> @Override</span> |
| <span class="source-line-no">420</span><span id="line-420"> public CheckAndMutateWithFilterBuilder timeRange(TimeRange timeRange) {</span> |
| <span class="source-line-no">421</span><span id="line-421"> this.timeRange = timeRange;</span> |
| <span class="source-line-no">422</span><span id="line-422"> return this;</span> |
| <span class="source-line-no">423</span><span id="line-423"> }</span> |
| <span class="source-line-no">424</span><span id="line-424"></span> |
| <span class="source-line-no">425</span><span id="line-425"> @Override</span> |
| <span class="source-line-no">426</span><span id="line-426"> public CompletableFuture<Boolean> thenPut(Put put) {</span> |
| <span class="source-line-no">427</span><span id="line-427"> validatePut(put, conn.connConf.getMaxKeyValueSize());</span> |
| <span class="source-line-no">428</span><span id="line-428"> final Supplier<Span> supplier = newTableOperationSpanBuilder()</span> |
| <span class="source-line-no">429</span><span id="line-429"> .setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE)</span> |
| <span class="source-line-no">430</span><span id="line-430"> .setContainerOperations(put);</span> |
| <span class="source-line-no">431</span><span id="line-431"> return tracedFuture(</span> |
| <span class="source-line-no">432</span><span id="line-432"> () -> RawAsyncTableImpl.this.<Boolean> newCaller(row, put.getPriority(), rpcTimeoutNs)</span> |
| <span class="source-line-no">433</span><span id="line-433"> .action((controller, loc, stub) -> RawAsyncTableImpl.mutate(controller, loc, stub, put,</span> |
| <span class="source-line-no">434</span><span id="line-434"> (rn, p) -> RequestConverter.buildMutateRequest(rn, row, null, null, null, null, filter,</span> |
| <span class="source-line-no">435</span><span id="line-435"> timeRange, p, HConstants.NO_NONCE, HConstants.NO_NONCE),</span> |
| <span class="source-line-no">436</span><span id="line-436"> (c, r) -> r.getProcessed()))</span> |
| <span class="source-line-no">437</span><span id="line-437"> .call(),</span> |
| <span class="source-line-no">438</span><span id="line-438"> supplier);</span> |
| <span class="source-line-no">439</span><span id="line-439"> }</span> |
| <span class="source-line-no">440</span><span id="line-440"></span> |
| <span class="source-line-no">441</span><span id="line-441"> @Override</span> |
| <span class="source-line-no">442</span><span id="line-442"> public CompletableFuture<Boolean> thenDelete(Delete delete) {</span> |
| <span class="source-line-no">443</span><span id="line-443"> final Supplier<Span> supplier = newTableOperationSpanBuilder()</span> |
| <span class="source-line-no">444</span><span id="line-444"> .setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE)</span> |
| <span class="source-line-no">445</span><span id="line-445"> .setContainerOperations(delete);</span> |
| <span class="source-line-no">446</span><span id="line-446"> return tracedFuture(</span> |
| <span class="source-line-no">447</span><span id="line-447"> () -> RawAsyncTableImpl.this.<Boolean> newCaller(row, delete.getPriority(), rpcTimeoutNs)</span> |
| <span class="source-line-no">448</span><span id="line-448"> .action((controller, loc, stub) -> RawAsyncTableImpl.mutate(controller, loc, stub, delete,</span> |
| <span class="source-line-no">449</span><span id="line-449"> (rn, d) -> RequestConverter.buildMutateRequest(rn, row, null, null, null, null, filter,</span> |
| <span class="source-line-no">450</span><span id="line-450"> timeRange, d, HConstants.NO_NONCE, HConstants.NO_NONCE),</span> |
| <span class="source-line-no">451</span><span id="line-451"> (c, r) -> r.getProcessed()))</span> |
| <span class="source-line-no">452</span><span id="line-452"> .call(),</span> |
| <span class="source-line-no">453</span><span id="line-453"> supplier);</span> |
| <span class="source-line-no">454</span><span id="line-454"> }</span> |
| <span class="source-line-no">455</span><span id="line-455"></span> |
| <span class="source-line-no">456</span><span id="line-456"> @Override</span> |
| <span class="source-line-no">457</span><span id="line-457"> public CompletableFuture<Boolean> thenMutate(RowMutations mutations) {</span> |
| <span class="source-line-no">458</span><span id="line-458"> validatePutsInRowMutations(mutations, conn.connConf.getMaxKeyValueSize());</span> |
| <span class="source-line-no">459</span><span id="line-459"> final Supplier<Span> supplier = newTableOperationSpanBuilder()</span> |
| <span class="source-line-no">460</span><span id="line-460"> .setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE)</span> |
| <span class="source-line-no">461</span><span id="line-461"> .setContainerOperations(mutations);</span> |
| <span class="source-line-no">462</span><span id="line-462"> return tracedFuture(() -> RawAsyncTableImpl.this</span> |
| <span class="source-line-no">463</span><span id="line-463"> .<Boolean> newCaller(row, mutations.getMaxPriority(), rpcTimeoutNs)</span> |
| <span class="source-line-no">464</span><span id="line-464"> .action((controller, loc, stub) -> RawAsyncTableImpl.this.mutateRow(controller, loc, stub,</span> |
| <span class="source-line-no">465</span><span id="line-465"> mutations,</span> |
| <span class="source-line-no">466</span><span id="line-466"> (rn, rm) -> RequestConverter.buildMultiRequest(rn, row, null, null, null, null, filter,</span> |
| <span class="source-line-no">467</span><span id="line-467"> timeRange, rm, HConstants.NO_NONCE, HConstants.NO_NONCE),</span> |
| <span class="source-line-no">468</span><span id="line-468"> CheckAndMutateResult::isSuccess))</span> |
| <span class="source-line-no">469</span><span id="line-469"> .call(), supplier);</span> |
| <span class="source-line-no">470</span><span id="line-470"> }</span> |
| <span class="source-line-no">471</span><span id="line-471"> }</span> |
| <span class="source-line-no">472</span><span id="line-472"></span> |
| <span class="source-line-no">473</span><span id="line-473"> @Override</span> |
| <span class="source-line-no">474</span><span id="line-474"> public CheckAndMutateWithFilterBuilder checkAndMutate(byte[] row, Filter filter) {</span> |
| <span class="source-line-no">475</span><span id="line-475"> return new CheckAndMutateWithFilterBuilderImpl(row, filter);</span> |
| <span class="source-line-no">476</span><span id="line-476"> }</span> |
| <span class="source-line-no">477</span><span id="line-477"></span> |
| <span class="source-line-no">478</span><span id="line-478"> @Override</span> |
| <span class="source-line-no">479</span><span id="line-479"> public CompletableFuture<CheckAndMutateResult> checkAndMutate(CheckAndMutate checkAndMutate) {</span> |
| <span class="source-line-no">480</span><span id="line-480"> final Supplier<Span> supplier = newTableOperationSpanBuilder().setOperation(checkAndMutate)</span> |
| <span class="source-line-no">481</span><span id="line-481"> .setContainerOperations(checkAndMutate.getAction());</span> |
| <span class="source-line-no">482</span><span id="line-482"> return tracedFuture(() -> {</span> |
| <span class="source-line-no">483</span><span id="line-483"> if (</span> |
| <span class="source-line-no">484</span><span id="line-484"> checkAndMutate.getAction() instanceof Put || checkAndMutate.getAction() instanceof Delete</span> |
| <span class="source-line-no">485</span><span id="line-485"> || checkAndMutate.getAction() instanceof Increment</span> |
| <span class="source-line-no">486</span><span id="line-486"> || checkAndMutate.getAction() instanceof Append</span> |
| <span class="source-line-no">487</span><span id="line-487"> ) {</span> |
| <span class="source-line-no">488</span><span id="line-488"> Mutation mutation = (Mutation) checkAndMutate.getAction();</span> |
| <span class="source-line-no">489</span><span id="line-489"> if (mutation instanceof Put) {</span> |
| <span class="source-line-no">490</span><span id="line-490"> validatePut((Put) mutation, conn.connConf.getMaxKeyValueSize());</span> |
| <span class="source-line-no">491</span><span id="line-491"> }</span> |
| <span class="source-line-no">492</span><span id="line-492"> long nonceGroup = conn.getNonceGenerator().getNonceGroup();</span> |
| <span class="source-line-no">493</span><span id="line-493"> long nonce = conn.getNonceGenerator().newNonce();</span> |
| <span class="source-line-no">494</span><span id="line-494"> return RawAsyncTableImpl.this</span> |
| <span class="source-line-no">495</span><span id="line-495"> .<CheckAndMutateResult> newCaller(checkAndMutate.getRow(), mutation.getPriority(),</span> |
| <span class="source-line-no">496</span><span id="line-496"> rpcTimeoutNs)</span> |
| <span class="source-line-no">497</span><span id="line-497"> .action(</span> |
| <span class="source-line-no">498</span><span id="line-498"> (controller, loc, stub) -> RawAsyncTableImpl.mutate(controller, loc, stub, mutation,</span> |
| <span class="source-line-no">499</span><span id="line-499"> (rn, m) -> RequestConverter.buildMutateRequest(rn, checkAndMutate.getRow(),</span> |
| <span class="source-line-no">500</span><span id="line-500"> checkAndMutate.getFamily(), checkAndMutate.getQualifier(),</span> |
| <span class="source-line-no">501</span><span id="line-501"> checkAndMutate.getCompareOp(), checkAndMutate.getValue(),</span> |
| <span class="source-line-no">502</span><span id="line-502"> checkAndMutate.getFilter(), checkAndMutate.getTimeRange(), m, nonceGroup, nonce),</span> |
| <span class="source-line-no">503</span><span id="line-503"> (c, r) -> ResponseConverter.getCheckAndMutateResult(r, c.cellScanner())))</span> |
| <span class="source-line-no">504</span><span id="line-504"> .call();</span> |
| <span class="source-line-no">505</span><span id="line-505"> } else if (checkAndMutate.getAction() instanceof RowMutations) {</span> |
| <span class="source-line-no">506</span><span id="line-506"> RowMutations rowMutations = (RowMutations) checkAndMutate.getAction();</span> |
| <span class="source-line-no">507</span><span id="line-507"> validatePutsInRowMutations(rowMutations, conn.connConf.getMaxKeyValueSize());</span> |
| <span class="source-line-no">508</span><span id="line-508"> long nonceGroup = conn.getNonceGenerator().getNonceGroup();</span> |
| <span class="source-line-no">509</span><span id="line-509"> long nonce = conn.getNonceGenerator().newNonce();</span> |
| <span class="source-line-no">510</span><span id="line-510"> return RawAsyncTableImpl.this</span> |
| <span class="source-line-no">511</span><span id="line-511"> .<CheckAndMutateResult> newCaller(checkAndMutate.getRow(), rowMutations.getMaxPriority(),</span> |
| <span class="source-line-no">512</span><span id="line-512"> rpcTimeoutNs)</span> |
| <span class="source-line-no">513</span><span id="line-513"> .action((controller, loc, stub) -> RawAsyncTableImpl.this.<CheckAndMutateResult,</span> |
| <span class="source-line-no">514</span><span id="line-514"> CheckAndMutateResult> mutateRow(controller, loc, stub, rowMutations,</span> |
| <span class="source-line-no">515</span><span id="line-515"> (rn, rm) -> RequestConverter.buildMultiRequest(rn, checkAndMutate.getRow(),</span> |
| <span class="source-line-no">516</span><span id="line-516"> checkAndMutate.getFamily(), checkAndMutate.getQualifier(),</span> |
| <span class="source-line-no">517</span><span id="line-517"> checkAndMutate.getCompareOp(), checkAndMutate.getValue(),</span> |
| <span class="source-line-no">518</span><span id="line-518"> checkAndMutate.getFilter(), checkAndMutate.getTimeRange(), rm, nonceGroup, nonce),</span> |
| <span class="source-line-no">519</span><span id="line-519"> resp -> resp))</span> |
| <span class="source-line-no">520</span><span id="line-520"> .call();</span> |
| <span class="source-line-no">521</span><span id="line-521"> } else {</span> |
| <span class="source-line-no">522</span><span id="line-522"> CompletableFuture<CheckAndMutateResult> future = new CompletableFuture<>();</span> |
| <span class="source-line-no">523</span><span id="line-523"> future.completeExceptionally(new DoNotRetryIOException(</span> |
| <span class="source-line-no">524</span><span id="line-524"> "CheckAndMutate doesn't support " + checkAndMutate.getAction().getClass().getName()));</span> |
| <span class="source-line-no">525</span><span id="line-525"> return future;</span> |
| <span class="source-line-no">526</span><span id="line-526"> }</span> |
| <span class="source-line-no">527</span><span id="line-527"> }, supplier);</span> |
| <span class="source-line-no">528</span><span id="line-528"> }</span> |
| <span class="source-line-no">529</span><span id="line-529"></span> |
| <span class="source-line-no">530</span><span id="line-530"> @Override</span> |
| <span class="source-line-no">531</span><span id="line-531"> public List<CompletableFuture<CheckAndMutateResult>></span> |
| <span class="source-line-no">532</span><span id="line-532"> checkAndMutate(List<CheckAndMutate> checkAndMutates) {</span> |
| <span class="source-line-no">533</span><span id="line-533"> final Supplier<Span> supplier = newTableOperationSpanBuilder().setOperation(checkAndMutates)</span> |
| <span class="source-line-no">534</span><span id="line-534"> .setContainerOperations(checkAndMutates);</span> |
| <span class="source-line-no">535</span><span id="line-535"> return tracedFutures(() -> batch(checkAndMutates, rpcTimeoutNs).stream()</span> |
| <span class="source-line-no">536</span><span id="line-536"> .map(f -> f.thenApply(r -> (CheckAndMutateResult) r)).collect(toList()), supplier);</span> |
| <span class="source-line-no">537</span><span id="line-537"> }</span> |
| <span class="source-line-no">538</span><span id="line-538"></span> |
| <span class="source-line-no">539</span><span id="line-539"> // We need the MultiRequest when constructing the org.apache.hadoop.hbase.client.MultiResponse,</span> |
| <span class="source-line-no">540</span><span id="line-540"> // so here I write a new method as I do not want to change the abstraction of call method.</span> |
| <span class="source-line-no">541</span><span id="line-541"> @SuppressWarnings("unchecked")</span> |
| <span class="source-line-no">542</span><span id="line-542"> private <RES, RESP> CompletableFuture<RESP> mutateRow(HBaseRpcController controller,</span> |
| <span class="source-line-no">543</span><span id="line-543"> HRegionLocation loc, ClientService.Interface stub, RowMutations mutation,</span> |
| <span class="source-line-no">544</span><span id="line-544"> Converter<MultiRequest, byte[], RowMutations> reqConvert, Function<RES, RESP> respConverter) {</span> |
| <span class="source-line-no">545</span><span id="line-545"> CompletableFuture<RESP> future = new CompletableFuture<>();</span> |
| <span class="source-line-no">546</span><span id="line-546"> try {</span> |
| <span class="source-line-no">547</span><span id="line-547"> byte[] regionName = loc.getRegion().getRegionName();</span> |
| <span class="source-line-no">548</span><span id="line-548"> MultiRequest req = reqConvert.convert(regionName, mutation);</span> |
| <span class="source-line-no">549</span><span id="line-549"> stub.multi(controller, req, new RpcCallback<MultiResponse>() {</span> |
| <span class="source-line-no">550</span><span id="line-550"></span> |
| <span class="source-line-no">551</span><span id="line-551"> @Override</span> |
| <span class="source-line-no">552</span><span id="line-552"> public void run(MultiResponse resp) {</span> |
| <span class="source-line-no">553</span><span id="line-553"> if (controller.failed()) {</span> |
| <span class="source-line-no">554</span><span id="line-554"> future.completeExceptionally(controller.getFailed());</span> |
| <span class="source-line-no">555</span><span id="line-555"> } else {</span> |
| <span class="source-line-no">556</span><span id="line-556"> try {</span> |
| <span class="source-line-no">557</span><span id="line-557"> org.apache.hadoop.hbase.client.MultiResponse multiResp =</span> |
| <span class="source-line-no">558</span><span id="line-558"> ResponseConverter.getResults(req, resp, controller.cellScanner());</span> |
| <span class="source-line-no">559</span><span id="line-559"> ConnectionUtils.updateStats(conn.getStatisticsTracker(), conn.getConnectionMetrics(),</span> |
| <span class="source-line-no">560</span><span id="line-560"> loc.getServerName(), multiResp);</span> |
| <span class="source-line-no">561</span><span id="line-561"> Throwable ex = multiResp.getException(regionName);</span> |
| <span class="source-line-no">562</span><span id="line-562"> if (ex != null) {</span> |
| <span class="source-line-no">563</span><span id="line-563"> future.completeExceptionally(ex instanceof IOException</span> |
| <span class="source-line-no">564</span><span id="line-564"> ? ex</span> |
| <span class="source-line-no">565</span><span id="line-565"> : new IOException(</span> |
| <span class="source-line-no">566</span><span id="line-566"> "Failed to mutate row: " + Bytes.toStringBinary(mutation.getRow()), ex));</span> |
| <span class="source-line-no">567</span><span id="line-567"> } else {</span> |
| <span class="source-line-no">568</span><span id="line-568"> future.complete(</span> |
| <span class="source-line-no">569</span><span id="line-569"> respConverter.apply((RES) multiResp.getResults().get(regionName).result.get(0)));</span> |
| <span class="source-line-no">570</span><span id="line-570"> }</span> |
| <span class="source-line-no">571</span><span id="line-571"> } catch (IOException e) {</span> |
| <span class="source-line-no">572</span><span id="line-572"> future.completeExceptionally(e);</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"> });</span> |
| <span class="source-line-no">577</span><span id="line-577"> } catch (IOException e) {</span> |
| <span class="source-line-no">578</span><span id="line-578"> future.completeExceptionally(e);</span> |
| <span class="source-line-no">579</span><span id="line-579"> }</span> |
| <span class="source-line-no">580</span><span id="line-580"> return future;</span> |
| <span class="source-line-no">581</span><span id="line-581"> }</span> |
| <span class="source-line-no">582</span><span id="line-582"></span> |
| <span class="source-line-no">583</span><span id="line-583"> @Override</span> |
| <span class="source-line-no">584</span><span id="line-584"> public CompletableFuture<Result> mutateRow(RowMutations mutations) {</span> |
| <span class="source-line-no">585</span><span id="line-585"> validatePutsInRowMutations(mutations, conn.connConf.getMaxKeyValueSize());</span> |
| <span class="source-line-no">586</span><span id="line-586"> long nonceGroup = conn.getNonceGenerator().getNonceGroup();</span> |
| <span class="source-line-no">587</span><span id="line-587"> long nonce = conn.getNonceGenerator().newNonce();</span> |
| <span class="source-line-no">588</span><span id="line-588"> final Supplier<Span> supplier =</span> |
| <span class="source-line-no">589</span><span id="line-589"> newTableOperationSpanBuilder().setOperation(mutations).setContainerOperations(mutations);</span> |
| <span class="source-line-no">590</span><span id="line-590"> return tracedFuture(</span> |
| <span class="source-line-no">591</span><span id="line-591"> () -> this</span> |
| <span class="source-line-no">592</span><span id="line-592"> .<Result> newCaller(mutations.getRow(), mutations.getMaxPriority(), writeRpcTimeoutNs)</span> |
| <span class="source-line-no">593</span><span id="line-593"> .action((controller, loc, stub) -> this.<Result, Result> mutateRow(controller, loc, stub,</span> |
| <span class="source-line-no">594</span><span id="line-594"> mutations, (rn, rm) -> RequestConverter.buildMultiRequest(rn, rm, nonceGroup, nonce),</span> |
| <span class="source-line-no">595</span><span id="line-595"> resp -> resp))</span> |
| <span class="source-line-no">596</span><span id="line-596"> .call(),</span> |
| <span class="source-line-no">597</span><span id="line-597"> supplier);</span> |
| <span class="source-line-no">598</span><span id="line-598"> }</span> |
| <span class="source-line-no">599</span><span id="line-599"></span> |
| <span class="source-line-no">600</span><span id="line-600"> private Scan setDefaultScanConfig(Scan scan) {</span> |
| <span class="source-line-no">601</span><span id="line-601"> // always create a new scan object as we may reset the start row later.</span> |
| <span class="source-line-no">602</span><span id="line-602"> Scan newScan = ReflectionUtils.newInstance(scan.getClass(), scan);</span> |
| <span class="source-line-no">603</span><span id="line-603"> if (newScan.getCaching() <= 0) {</span> |
| <span class="source-line-no">604</span><span id="line-604"> newScan.setCaching(defaultScannerCaching);</span> |
| <span class="source-line-no">605</span><span id="line-605"> }</span> |
| <span class="source-line-no">606</span><span id="line-606"> if (newScan.getMaxResultSize() <= 0) {</span> |
| <span class="source-line-no">607</span><span id="line-607"> newScan.setMaxResultSize(defaultScannerMaxResultSize);</span> |
| <span class="source-line-no">608</span><span id="line-608"> }</span> |
| <span class="source-line-no">609</span><span id="line-609"> return newScan;</span> |
| <span class="source-line-no">610</span><span id="line-610"> }</span> |
| <span class="source-line-no">611</span><span id="line-611"></span> |
| <span class="source-line-no">612</span><span id="line-612"> @Override</span> |
| <span class="source-line-no">613</span><span id="line-613"> public void scan(Scan scan, AdvancedScanResultConsumer consumer) {</span> |
| <span class="source-line-no">614</span><span id="line-614"> new AsyncClientScanner(setDefaultScanConfig(scan), consumer, tableName, conn, retryTimer,</span> |
| <span class="source-line-no">615</span><span id="line-615"> pauseNs, pauseNsForServerOverloaded, maxAttempts, scanTimeoutNs, readRpcTimeoutNs,</span> |
| <span class="source-line-no">616</span><span id="line-616"> startLogErrorsCnt, requestAttributes).start();</span> |
| <span class="source-line-no">617</span><span id="line-617"> }</span> |
| <span class="source-line-no">618</span><span id="line-618"></span> |
| <span class="source-line-no">619</span><span id="line-619"> private long resultSize2CacheSize(long maxResultSize) {</span> |
| <span class="source-line-no">620</span><span id="line-620"> // * 2 if possible</span> |
| <span class="source-line-no">621</span><span id="line-621"> return maxResultSize > Long.MAX_VALUE / 2 ? maxResultSize : maxResultSize * 2;</span> |
| <span class="source-line-no">622</span><span id="line-622"> }</span> |
| <span class="source-line-no">623</span><span id="line-623"></span> |
| <span class="source-line-no">624</span><span id="line-624"> @Override</span> |
| <span class="source-line-no">625</span><span id="line-625"> public AsyncTableResultScanner getScanner(Scan scan) {</span> |
| <span class="source-line-no">626</span><span id="line-626"> final long maxCacheSize = resultSize2CacheSize(</span> |
| <span class="source-line-no">627</span><span id="line-627"> scan.getMaxResultSize() > 0 ? scan.getMaxResultSize() : defaultScannerMaxResultSize);</span> |
| <span class="source-line-no">628</span><span id="line-628"> final Scan scanCopy = ReflectionUtils.newInstance(scan.getClass(), scan);</span> |
| <span class="source-line-no">629</span><span id="line-629"> final AsyncTableResultScanner scanner =</span> |
| <span class="source-line-no">630</span><span id="line-630"> new AsyncTableResultScanner(tableName, scanCopy, maxCacheSize);</span> |
| <span class="source-line-no">631</span><span id="line-631"> scan(scan, scanner);</span> |
| <span class="source-line-no">632</span><span id="line-632"> return scanner;</span> |
| <span class="source-line-no">633</span><span id="line-633"> }</span> |
| <span class="source-line-no">634</span><span id="line-634"></span> |
| <span class="source-line-no">635</span><span id="line-635"> @Override</span> |
| <span class="source-line-no">636</span><span id="line-636"> public CompletableFuture<List<Result>> scanAll(Scan scan) {</span> |
| <span class="source-line-no">637</span><span id="line-637"> CompletableFuture<List<Result>> future = new CompletableFuture<>();</span> |
| <span class="source-line-no">638</span><span id="line-638"> List<Result> scanResults = new ArrayList<>();</span> |
| <span class="source-line-no">639</span><span id="line-639"> scan(scan, new AdvancedScanResultConsumer() {</span> |
| <span class="source-line-no">640</span><span id="line-640"></span> |
| <span class="source-line-no">641</span><span id="line-641"> @Override</span> |
| <span class="source-line-no">642</span><span id="line-642"> public void onNext(Result[] results, ScanController controller) {</span> |
| <span class="source-line-no">643</span><span id="line-643"> scanResults.addAll(Arrays.asList(results));</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"> @Override</span> |
| <span class="source-line-no">647</span><span id="line-647"> public void onError(Throwable error) {</span> |
| <span class="source-line-no">648</span><span id="line-648"> future.completeExceptionally(error);</span> |
| <span class="source-line-no">649</span><span id="line-649"> }</span> |
| <span class="source-line-no">650</span><span id="line-650"></span> |
| <span class="source-line-no">651</span><span id="line-651"> @Override</span> |
| <span class="source-line-no">652</span><span id="line-652"> public void onComplete() {</span> |
| <span class="source-line-no">653</span><span id="line-653"> future.complete(scanResults);</span> |
| <span class="source-line-no">654</span><span id="line-654"> }</span> |
| <span class="source-line-no">655</span><span id="line-655"> });</span> |
| <span class="source-line-no">656</span><span id="line-656"> return future;</span> |
| <span class="source-line-no">657</span><span id="line-657"> }</span> |
| <span class="source-line-no">658</span><span id="line-658"></span> |
| <span class="source-line-no">659</span><span id="line-659"> @Override</span> |
| <span class="source-line-no">660</span><span id="line-660"> public List<CompletableFuture<Result>> get(List<Get> gets) {</span> |
| <span class="source-line-no">661</span><span id="line-661"> final Supplier<Span> supplier = newTableOperationSpanBuilder().setOperation(gets)</span> |
| <span class="source-line-no">662</span><span id="line-662"> .setContainerOperations(HBaseSemanticAttributes.Operation.GET);</span> |
| <span class="source-line-no">663</span><span id="line-663"> return tracedFutures(() -> batch(gets, readRpcTimeoutNs), supplier);</span> |
| <span class="source-line-no">664</span><span id="line-664"> }</span> |
| <span class="source-line-no">665</span><span id="line-665"></span> |
| <span class="source-line-no">666</span><span id="line-666"> @Override</span> |
| <span class="source-line-no">667</span><span id="line-667"> public List<CompletableFuture<Void>> put(List<Put> puts) {</span> |
| <span class="source-line-no">668</span><span id="line-668"> final Supplier<Span> supplier = newTableOperationSpanBuilder().setOperation(puts)</span> |
| <span class="source-line-no">669</span><span id="line-669"> .setContainerOperations(HBaseSemanticAttributes.Operation.PUT);</span> |
| <span class="source-line-no">670</span><span id="line-670"> return tracedFutures(() -> voidMutate(puts), supplier);</span> |
| <span class="source-line-no">671</span><span id="line-671"> }</span> |
| <span class="source-line-no">672</span><span id="line-672"></span> |
| <span class="source-line-no">673</span><span id="line-673"> @Override</span> |
| <span class="source-line-no">674</span><span id="line-674"> public List<CompletableFuture<Void>> delete(List<Delete> deletes) {</span> |
| <span class="source-line-no">675</span><span id="line-675"> final Supplier<Span> supplier = newTableOperationSpanBuilder().setOperation(deletes)</span> |
| <span class="source-line-no">676</span><span id="line-676"> .setContainerOperations(HBaseSemanticAttributes.Operation.DELETE);</span> |
| <span class="source-line-no">677</span><span id="line-677"> return tracedFutures(() -> voidMutate(deletes), supplier);</span> |
| <span class="source-line-no">678</span><span id="line-678"> }</span> |
| <span class="source-line-no">679</span><span id="line-679"></span> |
| <span class="source-line-no">680</span><span id="line-680"> @Override</span> |
| <span class="source-line-no">681</span><span id="line-681"> public <T> List<CompletableFuture<T>> batch(List<? extends Row> actions) {</span> |
| <span class="source-line-no">682</span><span id="line-682"> final Supplier<Span> supplier =</span> |
| <span class="source-line-no">683</span><span id="line-683"> newTableOperationSpanBuilder().setOperation(actions).setContainerOperations(actions);</span> |
| <span class="source-line-no">684</span><span id="line-684"> return tracedFutures(() -> batch(actions, rpcTimeoutNs), supplier);</span> |
| <span class="source-line-no">685</span><span id="line-685"> }</span> |
| <span class="source-line-no">686</span><span id="line-686"></span> |
| <span class="source-line-no">687</span><span id="line-687"> private List<CompletableFuture<Void>> voidMutate(List<? extends Row> actions) {</span> |
| <span class="source-line-no">688</span><span id="line-688"> return this.<Object> batch(actions, writeRpcTimeoutNs).stream()</span> |
| <span class="source-line-no">689</span><span id="line-689"> .map(f -> f.<Void> thenApply(r -> null)).collect(toList());</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"> private <T> List<CompletableFuture<T>> batch(List<? extends Row> actions, long rpcTimeoutNs) {</span> |
| <span class="source-line-no">693</span><span id="line-693"> for (Row action : actions) {</span> |
| <span class="source-line-no">694</span><span id="line-694"> if (action instanceof Put) {</span> |
| <span class="source-line-no">695</span><span id="line-695"> validatePut((Put) action, conn.connConf.getMaxKeyValueSize());</span> |
| <span class="source-line-no">696</span><span id="line-696"> } else if (action instanceof CheckAndMutate) {</span> |
| <span class="source-line-no">697</span><span id="line-697"> CheckAndMutate checkAndMutate = (CheckAndMutate) action;</span> |
| <span class="source-line-no">698</span><span id="line-698"> if (checkAndMutate.getAction() instanceof Put) {</span> |
| <span class="source-line-no">699</span><span id="line-699"> validatePut((Put) checkAndMutate.getAction(), conn.connConf.getMaxKeyValueSize());</span> |
| <span class="source-line-no">700</span><span id="line-700"> } else if (checkAndMutate.getAction() instanceof RowMutations) {</span> |
| <span class="source-line-no">701</span><span id="line-701"> validatePutsInRowMutations((RowMutations) checkAndMutate.getAction(),</span> |
| <span class="source-line-no">702</span><span id="line-702"> conn.connConf.getMaxKeyValueSize());</span> |
| <span class="source-line-no">703</span><span id="line-703"> }</span> |
| <span class="source-line-no">704</span><span id="line-704"> } else if (action instanceof RowMutations) {</span> |
| <span class="source-line-no">705</span><span id="line-705"> validatePutsInRowMutations((RowMutations) action, conn.connConf.getMaxKeyValueSize());</span> |
| <span class="source-line-no">706</span><span id="line-706"> }</span> |
| <span class="source-line-no">707</span><span id="line-707"> }</span> |
| <span class="source-line-no">708</span><span id="line-708"> return conn.callerFactory.batch().table(tableName).actions(actions)</span> |
| <span class="source-line-no">709</span><span id="line-709"> .operationTimeout(operationTimeoutNs, TimeUnit.NANOSECONDS)</span> |
| <span class="source-line-no">710</span><span id="line-710"> .rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS).pause(pauseNs, TimeUnit.NANOSECONDS)</span> |
| <span class="source-line-no">711</span><span id="line-711"> .pauseForServerOverloaded(pauseNsForServerOverloaded, TimeUnit.NANOSECONDS)</span> |
| <span class="source-line-no">712</span><span id="line-712"> .maxAttempts(maxAttempts).startLogErrorsCnt(startLogErrorsCnt)</span> |
| <span class="source-line-no">713</span><span id="line-713"> .setRequestAttributes(requestAttributes).call();</span> |
| <span class="source-line-no">714</span><span id="line-714"> }</span> |
| <span class="source-line-no">715</span><span id="line-715"></span> |
| <span class="source-line-no">716</span><span id="line-716"> @Override</span> |
| <span class="source-line-no">717</span><span id="line-717"> public long getRpcTimeout(TimeUnit unit) {</span> |
| <span class="source-line-no">718</span><span id="line-718"> return unit.convert(rpcTimeoutNs, TimeUnit.NANOSECONDS);</span> |
| <span class="source-line-no">719</span><span id="line-719"> }</span> |
| <span class="source-line-no">720</span><span id="line-720"></span> |
| <span class="source-line-no">721</span><span id="line-721"> @Override</span> |
| <span class="source-line-no">722</span><span id="line-722"> public long getReadRpcTimeout(TimeUnit unit) {</span> |
| <span class="source-line-no">723</span><span id="line-723"> return unit.convert(readRpcTimeoutNs, TimeUnit.NANOSECONDS);</span> |
| <span class="source-line-no">724</span><span id="line-724"> }</span> |
| <span class="source-line-no">725</span><span id="line-725"></span> |
| <span class="source-line-no">726</span><span id="line-726"> @Override</span> |
| <span class="source-line-no">727</span><span id="line-727"> public long getWriteRpcTimeout(TimeUnit unit) {</span> |
| <span class="source-line-no">728</span><span id="line-728"> return unit.convert(writeRpcTimeoutNs, TimeUnit.NANOSECONDS);</span> |
| <span class="source-line-no">729</span><span id="line-729"> }</span> |
| <span class="source-line-no">730</span><span id="line-730"></span> |
| <span class="source-line-no">731</span><span id="line-731"> @Override</span> |
| <span class="source-line-no">732</span><span id="line-732"> public long getOperationTimeout(TimeUnit unit) {</span> |
| <span class="source-line-no">733</span><span id="line-733"> return unit.convert(operationTimeoutNs, TimeUnit.NANOSECONDS);</span> |
| <span class="source-line-no">734</span><span id="line-734"> }</span> |
| <span class="source-line-no">735</span><span id="line-735"></span> |
| <span class="source-line-no">736</span><span id="line-736"> @Override</span> |
| <span class="source-line-no">737</span><span id="line-737"> public long getScanTimeout(TimeUnit unit) {</span> |
| <span class="source-line-no">738</span><span id="line-738"> return unit.convert(scanTimeoutNs, TimeUnit.NANOSECONDS);</span> |
| <span class="source-line-no">739</span><span id="line-739"> }</span> |
| <span class="source-line-no">740</span><span id="line-740"></span> |
| <span class="source-line-no">741</span><span id="line-741"> @Override</span> |
| <span class="source-line-no">742</span><span id="line-742"> public Map<String, byte[]> getRequestAttributes() {</span> |
| <span class="source-line-no">743</span><span id="line-743"> return requestAttributes;</span> |
| <span class="source-line-no">744</span><span id="line-744"> }</span> |
| <span class="source-line-no">745</span><span id="line-745"></span> |
| <span class="source-line-no">746</span><span id="line-746"> private <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,</span> |
| <span class="source-line-no">747</span><span id="line-747"> ServiceCaller<S, R> callable, RegionInfo region, byte[] row) {</span> |
| <span class="source-line-no">748</span><span id="line-748"> RegionCoprocessorRpcChannelImpl channel = new RegionCoprocessorRpcChannelImpl(conn, tableName,</span> |
| <span class="source-line-no">749</span><span id="line-749"> region, row, rpcTimeoutNs, operationTimeoutNs);</span> |
| <span class="source-line-no">750</span><span id="line-750"> final Span span = Span.current();</span> |
| <span class="source-line-no">751</span><span id="line-751"> S stub = stubMaker.apply(channel);</span> |
| <span class="source-line-no">752</span><span id="line-752"> CompletableFuture<R> future = new CompletableFuture<>();</span> |
| <span class="source-line-no">753</span><span id="line-753"> ClientCoprocessorRpcController controller = new ClientCoprocessorRpcController();</span> |
| <span class="source-line-no">754</span><span id="line-754"> callable.call(stub, controller, resp -> {</span> |
| <span class="source-line-no">755</span><span id="line-755"> try (Scope ignored = span.makeCurrent()) {</span> |
| <span class="source-line-no">756</span><span id="line-756"> if (controller.failed()) {</span> |
| <span class="source-line-no">757</span><span id="line-757"> final Throwable failure = controller.getFailed();</span> |
| <span class="source-line-no">758</span><span id="line-758"> future.completeExceptionally(failure);</span> |
| <span class="source-line-no">759</span><span id="line-759"> TraceUtil.setError(span, failure);</span> |
| <span class="source-line-no">760</span><span id="line-760"> } else {</span> |
| <span class="source-line-no">761</span><span id="line-761"> future.complete(resp);</span> |
| <span class="source-line-no">762</span><span id="line-762"> span.setStatus(StatusCode.OK);</span> |
| <span class="source-line-no">763</span><span id="line-763"> }</span> |
| <span class="source-line-no">764</span><span id="line-764"> } finally {</span> |
| <span class="source-line-no">765</span><span id="line-765"> span.end();</span> |
| <span class="source-line-no">766</span><span id="line-766"> }</span> |
| <span class="source-line-no">767</span><span id="line-767"> });</span> |
| <span class="source-line-no">768</span><span id="line-768"> return future;</span> |
| <span class="source-line-no">769</span><span id="line-769"> }</span> |
| <span class="source-line-no">770</span><span id="line-770"></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 <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,</span> |
| <span class="source-line-no">773</span><span id="line-773"> ServiceCaller<S, R> callable, byte[] row) {</span> |
| <span class="source-line-no">774</span><span id="line-774"> return coprocessorService(stubMaker, callable, null, row);</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"> private boolean locateFinished(RegionInfo region, byte[] endKey, boolean endKeyInclusive) {</span> |
| <span class="source-line-no">778</span><span id="line-778"> if (isEmptyStopRow(endKey)) {</span> |
| <span class="source-line-no">779</span><span id="line-779"> if (isEmptyStopRow(region.getEndKey())) {</span> |
| <span class="source-line-no">780</span><span id="line-780"> return true;</span> |
| <span class="source-line-no">781</span><span id="line-781"> }</span> |
| <span class="source-line-no">782</span><span id="line-782"> return false;</span> |
| <span class="source-line-no">783</span><span id="line-783"> } else {</span> |
| <span class="source-line-no">784</span><span id="line-784"> if (isEmptyStopRow(region.getEndKey())) {</span> |
| <span class="source-line-no">785</span><span id="line-785"> return true;</span> |
| <span class="source-line-no">786</span><span id="line-786"> }</span> |
| <span class="source-line-no">787</span><span id="line-787"> int c = Bytes.compareTo(endKey, region.getEndKey());</span> |
| <span class="source-line-no">788</span><span id="line-788"> // 1. if the region contains endKey</span> |
| <span class="source-line-no">789</span><span id="line-789"> // 2. endKey is equal to the region's endKey and we do not want to include endKey.</span> |
| <span class="source-line-no">790</span><span id="line-790"> return c < 0 || (c == 0 && !endKeyInclusive);</span> |
| <span class="source-line-no">791</span><span id="line-791"> }</span> |
| <span class="source-line-no">792</span><span id="line-792"> }</span> |
| <span class="source-line-no">793</span><span id="line-793"></span> |
| <span class="source-line-no">794</span><span id="line-794"> private <S, R> void onLocateComplete(Function<RpcChannel, S> stubMaker,</span> |
| <span class="source-line-no">795</span><span id="line-795"> ServiceCaller<S, R> callable, CoprocessorCallback<R> callback, List<HRegionLocation> locs,</span> |
| <span class="source-line-no">796</span><span id="line-796"> byte[] endKey, boolean endKeyInclusive, AtomicBoolean locateFinished,</span> |
| <span class="source-line-no">797</span><span id="line-797"> AtomicInteger unfinishedRequest, HRegionLocation loc, Throwable error) {</span> |
| <span class="source-line-no">798</span><span id="line-798"> final Span span = Span.current();</span> |
| <span class="source-line-no">799</span><span id="line-799"> if (error != null) {</span> |
| <span class="source-line-no">800</span><span id="line-800"> callback.onError(error);</span> |
| <span class="source-line-no">801</span><span id="line-801"> TraceUtil.setError(span, error);</span> |
| <span class="source-line-no">802</span><span id="line-802"> span.end();</span> |
| <span class="source-line-no">803</span><span id="line-803"> return;</span> |
| <span class="source-line-no">804</span><span id="line-804"> }</span> |
| <span class="source-line-no">805</span><span id="line-805"> unfinishedRequest.incrementAndGet();</span> |
| <span class="source-line-no">806</span><span id="line-806"> RegionInfo region = loc.getRegion();</span> |
| <span class="source-line-no">807</span><span id="line-807"> if (locateFinished(region, endKey, endKeyInclusive)) {</span> |
| <span class="source-line-no">808</span><span id="line-808"> locateFinished.set(true);</span> |
| <span class="source-line-no">809</span><span id="line-809"> } else {</span> |
| <span class="source-line-no">810</span><span id="line-810"> addListener(conn.getLocator().getRegionLocation(tableName, region.getEndKey(),</span> |
| <span class="source-line-no">811</span><span id="line-811"> RegionLocateType.CURRENT, operationTimeoutNs), (l, e) -> {</span> |
| <span class="source-line-no">812</span><span id="line-812"> try (Scope ignored = span.makeCurrent()) {</span> |
| <span class="source-line-no">813</span><span id="line-813"> onLocateComplete(stubMaker, callable, callback, locs, endKey, endKeyInclusive,</span> |
| <span class="source-line-no">814</span><span id="line-814"> locateFinished, unfinishedRequest, l, e);</span> |
| <span class="source-line-no">815</span><span id="line-815"> }</span> |
| <span class="source-line-no">816</span><span id="line-816"> });</span> |
| <span class="source-line-no">817</span><span id="line-817"> }</span> |
| <span class="source-line-no">818</span><span id="line-818"> addListener(coprocessorService(stubMaker, callable, region, region.getStartKey()), (r, e) -> {</span> |
| <span class="source-line-no">819</span><span id="line-819"> try (Scope ignored = span.makeCurrent()) {</span> |
| <span class="source-line-no">820</span><span id="line-820"> if (e != null) {</span> |
| <span class="source-line-no">821</span><span id="line-821"> callback.onRegionError(region, e);</span> |
| <span class="source-line-no">822</span><span id="line-822"> } else {</span> |
| <span class="source-line-no">823</span><span id="line-823"> callback.onRegionComplete(region, r);</span> |
| <span class="source-line-no">824</span><span id="line-824"> }</span> |
| <span class="source-line-no">825</span><span id="line-825"> if (unfinishedRequest.decrementAndGet() == 0 && locateFinished.get()) {</span> |
| <span class="source-line-no">826</span><span id="line-826"> callback.onComplete();</span> |
| <span class="source-line-no">827</span><span id="line-827"> }</span> |
| <span class="source-line-no">828</span><span id="line-828"> }</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"> private final class CoprocessorServiceBuilderImpl<S, R></span> |
| <span class="source-line-no">833</span><span id="line-833"> implements CoprocessorServiceBuilder<S, R> {</span> |
| <span class="source-line-no">834</span><span id="line-834"></span> |
| <span class="source-line-no">835</span><span id="line-835"> private final Function<RpcChannel, S> stubMaker;</span> |
| <span class="source-line-no">836</span><span id="line-836"></span> |
| <span class="source-line-no">837</span><span id="line-837"> private final ServiceCaller<S, R> callable;</span> |
| <span class="source-line-no">838</span><span id="line-838"></span> |
| <span class="source-line-no">839</span><span id="line-839"> private final CoprocessorCallback<R> callback;</span> |
| <span class="source-line-no">840</span><span id="line-840"></span> |
| <span class="source-line-no">841</span><span id="line-841"> private byte[] startKey = HConstants.EMPTY_START_ROW;</span> |
| <span class="source-line-no">842</span><span id="line-842"></span> |
| <span class="source-line-no">843</span><span id="line-843"> private boolean startKeyInclusive;</span> |
| <span class="source-line-no">844</span><span id="line-844"></span> |
| <span class="source-line-no">845</span><span id="line-845"> private byte[] endKey = HConstants.EMPTY_END_ROW;</span> |
| <span class="source-line-no">846</span><span id="line-846"></span> |
| <span class="source-line-no">847</span><span id="line-847"> private boolean endKeyInclusive;</span> |
| <span class="source-line-no">848</span><span id="line-848"></span> |
| <span class="source-line-no">849</span><span id="line-849"> public CoprocessorServiceBuilderImpl(Function<RpcChannel, S> stubMaker,</span> |
| <span class="source-line-no">850</span><span id="line-850"> ServiceCaller<S, R> callable, CoprocessorCallback<R> callback) {</span> |
| <span class="source-line-no">851</span><span id="line-851"> this.stubMaker = Preconditions.checkNotNull(stubMaker, "stubMaker is null");</span> |
| <span class="source-line-no">852</span><span id="line-852"> this.callable = Preconditions.checkNotNull(callable, "callable is null");</span> |
| <span class="source-line-no">853</span><span id="line-853"> this.callback = Preconditions.checkNotNull(callback, "callback is null");</span> |
| <span class="source-line-no">854</span><span id="line-854"> }</span> |
| <span class="source-line-no">855</span><span id="line-855"></span> |
| <span class="source-line-no">856</span><span id="line-856"> @Override</span> |
| <span class="source-line-no">857</span><span id="line-857"> public CoprocessorServiceBuilderImpl<S, R> fromRow(byte[] startKey, boolean inclusive) {</span> |
| <span class="source-line-no">858</span><span id="line-858"> this.startKey = Preconditions.checkNotNull(startKey,</span> |
| <span class="source-line-no">859</span><span id="line-859"> "startKey is null. Consider using"</span> |
| <span class="source-line-no">860</span><span id="line-860"> + " an empty byte array, or just do not call this method if you want to start selection"</span> |
| <span class="source-line-no">861</span><span id="line-861"> + " from the first region");</span> |
| <span class="source-line-no">862</span><span id="line-862"> this.startKeyInclusive = inclusive;</span> |
| <span class="source-line-no">863</span><span id="line-863"> return this;</span> |
| <span class="source-line-no">864</span><span id="line-864"> }</span> |
| <span class="source-line-no">865</span><span id="line-865"></span> |
| <span class="source-line-no">866</span><span id="line-866"> @Override</span> |
| <span class="source-line-no">867</span><span id="line-867"> public CoprocessorServiceBuilderImpl<S, R> toRow(byte[] endKey, boolean inclusive) {</span> |
| <span class="source-line-no">868</span><span id="line-868"> this.endKey = Preconditions.checkNotNull(endKey,</span> |
| <span class="source-line-no">869</span><span id="line-869"> "endKey is null. Consider using"</span> |
| <span class="source-line-no">870</span><span id="line-870"> + " an empty byte array, or just do not call this method if you want to continue"</span> |
| <span class="source-line-no">871</span><span id="line-871"> + " selection to the last region");</span> |
| <span class="source-line-no">872</span><span id="line-872"> this.endKeyInclusive = inclusive;</span> |
| <span class="source-line-no">873</span><span id="line-873"> return this;</span> |
| <span class="source-line-no">874</span><span id="line-874"> }</span> |
| <span class="source-line-no">875</span><span id="line-875"></span> |
| <span class="source-line-no">876</span><span id="line-876"> @Override</span> |
| <span class="source-line-no">877</span><span id="line-877"> public void execute() {</span> |
| <span class="source-line-no">878</span><span id="line-878"> final Span span = newTableOperationSpanBuilder()</span> |
| <span class="source-line-no">879</span><span id="line-879"> .setOperation(HBaseSemanticAttributes.Operation.COPROC_EXEC).build();</span> |
| <span class="source-line-no">880</span><span id="line-880"> try (Scope ignored = span.makeCurrent()) {</span> |
| <span class="source-line-no">881</span><span id="line-881"> final RegionLocateType regionLocateType =</span> |
| <span class="source-line-no">882</span><span id="line-882"> startKeyInclusive ? RegionLocateType.CURRENT : RegionLocateType.AFTER;</span> |
| <span class="source-line-no">883</span><span id="line-883"> final CompletableFuture<HRegionLocation> future = conn.getLocator()</span> |
| <span class="source-line-no">884</span><span id="line-884"> .getRegionLocation(tableName, startKey, regionLocateType, operationTimeoutNs);</span> |
| <span class="source-line-no">885</span><span id="line-885"> addListener(future, (loc, error) -> {</span> |
| <span class="source-line-no">886</span><span id="line-886"> try (Scope ignored1 = span.makeCurrent()) {</span> |
| <span class="source-line-no">887</span><span id="line-887"> onLocateComplete(stubMaker, callable, callback, new ArrayList<>(), endKey,</span> |
| <span class="source-line-no">888</span><span id="line-888"> endKeyInclusive, new AtomicBoolean(false), new AtomicInteger(0), loc, error);</span> |
| <span class="source-line-no">889</span><span id="line-889"> }</span> |
| <span class="source-line-no">890</span><span id="line-890"> });</span> |
| <span class="source-line-no">891</span><span id="line-891"> }</span> |
| <span class="source-line-no">892</span><span id="line-892"> }</span> |
| <span class="source-line-no">893</span><span id="line-893"> }</span> |
| <span class="source-line-no">894</span><span id="line-894"></span> |
| <span class="source-line-no">895</span><span id="line-895"> @Override</span> |
| <span class="source-line-no">896</span><span id="line-896"> public <S, R> CoprocessorServiceBuilder<S, R> coprocessorService(</span> |
| <span class="source-line-no">897</span><span id="line-897"> Function<RpcChannel, S> stubMaker, ServiceCaller<S, R> callable,</span> |
| <span class="source-line-no">898</span><span id="line-898"> CoprocessorCallback<R> callback) {</span> |
| <span class="source-line-no">899</span><span id="line-899"> return new CoprocessorServiceBuilderImpl<>(stubMaker, callable, callback);</span> |
| <span class="source-line-no">900</span><span id="line-900"> }</span> |
| <span class="source-line-no">901</span><span id="line-901">}</span> |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| </pre> |
| </div> |
| </main> |
| </body> |
| </html> |