blob: 8a8f3259a21fd091bade079b8e1dd60ffa57fa63 [file] [log] [blame]
<!DOCTYPE HTML>
<html lang="en">
<head>
<!-- Generated by javadoc (17) -->
<title>Source code</title>
<meta name="viewport" content="width=device-width, initial-scale=1">
<meta name="description" content="source: package: org.apache.hadoop.hbase.regionserver, class: RSRpcServices, class: RegionScannersCloseCallBack">
<meta name="generator" content="javadoc/SourceToHTMLConverter">
<link rel="stylesheet" type="text/css" href="../../../../../../stylesheet.css" title="Style">
</head>
<body class="source-page">
<main role="main">
<div class="source-container">
<pre><span class="source-line-no">001</span><span id="line-1">/*</span>
<span class="source-line-no">002</span><span id="line-2"> * Licensed to the Apache Software Foundation (ASF) under one</span>
<span class="source-line-no">003</span><span id="line-3"> * or more contributor license agreements. See the NOTICE file</span>
<span class="source-line-no">004</span><span id="line-4"> * distributed with this work for additional information</span>
<span class="source-line-no">005</span><span id="line-5"> * regarding copyright ownership. The ASF licenses this file</span>
<span class="source-line-no">006</span><span id="line-6"> * to you under the Apache License, Version 2.0 (the</span>
<span class="source-line-no">007</span><span id="line-7"> * "License"); you may not use this file except in compliance</span>
<span class="source-line-no">008</span><span id="line-8"> * with the License. You may obtain a copy of the License at</span>
<span class="source-line-no">009</span><span id="line-9"> *</span>
<span class="source-line-no">010</span><span id="line-10"> * http://www.apache.org/licenses/LICENSE-2.0</span>
<span class="source-line-no">011</span><span id="line-11"> *</span>
<span class="source-line-no">012</span><span id="line-12"> * Unless required by applicable law or agreed to in writing, software</span>
<span class="source-line-no">013</span><span id="line-13"> * distributed under the License is distributed on an "AS IS" BASIS,</span>
<span class="source-line-no">014</span><span id="line-14"> * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.</span>
<span class="source-line-no">015</span><span id="line-15"> * See the License for the specific language governing permissions and</span>
<span class="source-line-no">016</span><span id="line-16"> * limitations under the License.</span>
<span class="source-line-no">017</span><span id="line-17"> */</span>
<span class="source-line-no">018</span><span id="line-18">package org.apache.hadoop.hbase.regionserver;</span>
<span class="source-line-no">019</span><span id="line-19"></span>
<span class="source-line-no">020</span><span id="line-20">import com.google.errorprone.annotations.RestrictedApi;</span>
<span class="source-line-no">021</span><span id="line-21">import java.io.FileNotFoundException;</span>
<span class="source-line-no">022</span><span id="line-22">import java.io.IOException;</span>
<span class="source-line-no">023</span><span id="line-23">import java.io.UncheckedIOException;</span>
<span class="source-line-no">024</span><span id="line-24">import java.net.BindException;</span>
<span class="source-line-no">025</span><span id="line-25">import java.net.InetAddress;</span>
<span class="source-line-no">026</span><span id="line-26">import java.net.InetSocketAddress;</span>
<span class="source-line-no">027</span><span id="line-27">import java.util.ArrayList;</span>
<span class="source-line-no">028</span><span id="line-28">import java.util.Arrays;</span>
<span class="source-line-no">029</span><span id="line-29">import java.util.Collections;</span>
<span class="source-line-no">030</span><span id="line-30">import java.util.HashMap;</span>
<span class="source-line-no">031</span><span id="line-31">import java.util.Iterator;</span>
<span class="source-line-no">032</span><span id="line-32">import java.util.List;</span>
<span class="source-line-no">033</span><span id="line-33">import java.util.Map;</span>
<span class="source-line-no">034</span><span id="line-34">import java.util.Map.Entry;</span>
<span class="source-line-no">035</span><span id="line-35">import java.util.NavigableMap;</span>
<span class="source-line-no">036</span><span id="line-36">import java.util.Set;</span>
<span class="source-line-no">037</span><span id="line-37">import java.util.TreeSet;</span>
<span class="source-line-no">038</span><span id="line-38">import java.util.concurrent.ConcurrentHashMap;</span>
<span class="source-line-no">039</span><span id="line-39">import java.util.concurrent.ConcurrentMap;</span>
<span class="source-line-no">040</span><span id="line-40">import java.util.concurrent.TimeUnit;</span>
<span class="source-line-no">041</span><span id="line-41">import java.util.concurrent.atomic.AtomicBoolean;</span>
<span class="source-line-no">042</span><span id="line-42">import java.util.concurrent.atomic.AtomicLong;</span>
<span class="source-line-no">043</span><span id="line-43">import java.util.concurrent.atomic.LongAdder;</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.fs.FileSystem;</span>
<span class="source-line-no">046</span><span id="line-46">import org.apache.hadoop.fs.Path;</span>
<span class="source-line-no">047</span><span id="line-47">import org.apache.hadoop.hbase.CacheEvictionStats;</span>
<span class="source-line-no">048</span><span id="line-48">import org.apache.hadoop.hbase.CacheEvictionStatsBuilder;</span>
<span class="source-line-no">049</span><span id="line-49">import org.apache.hadoop.hbase.Cell;</span>
<span class="source-line-no">050</span><span id="line-50">import org.apache.hadoop.hbase.CellScanner;</span>
<span class="source-line-no">051</span><span id="line-51">import org.apache.hadoop.hbase.CellUtil;</span>
<span class="source-line-no">052</span><span id="line-52">import org.apache.hadoop.hbase.DoNotRetryIOException;</span>
<span class="source-line-no">053</span><span id="line-53">import org.apache.hadoop.hbase.DroppedSnapshotException;</span>
<span class="source-line-no">054</span><span id="line-54">import org.apache.hadoop.hbase.ExtendedCellScannable;</span>
<span class="source-line-no">055</span><span id="line-55">import org.apache.hadoop.hbase.ExtendedCellScanner;</span>
<span class="source-line-no">056</span><span id="line-56">import org.apache.hadoop.hbase.HBaseIOException;</span>
<span class="source-line-no">057</span><span id="line-57">import org.apache.hadoop.hbase.HBaseRpcServicesBase;</span>
<span class="source-line-no">058</span><span id="line-58">import org.apache.hadoop.hbase.HConstants;</span>
<span class="source-line-no">059</span><span id="line-59">import org.apache.hadoop.hbase.MultiActionResultTooLarge;</span>
<span class="source-line-no">060</span><span id="line-60">import org.apache.hadoop.hbase.NotServingRegionException;</span>
<span class="source-line-no">061</span><span id="line-61">import org.apache.hadoop.hbase.PrivateCellUtil;</span>
<span class="source-line-no">062</span><span id="line-62">import org.apache.hadoop.hbase.RegionTooBusyException;</span>
<span class="source-line-no">063</span><span id="line-63">import org.apache.hadoop.hbase.Server;</span>
<span class="source-line-no">064</span><span id="line-64">import org.apache.hadoop.hbase.ServerName;</span>
<span class="source-line-no">065</span><span id="line-65">import org.apache.hadoop.hbase.TableName;</span>
<span class="source-line-no">066</span><span id="line-66">import org.apache.hadoop.hbase.UnknownScannerException;</span>
<span class="source-line-no">067</span><span id="line-67">import org.apache.hadoop.hbase.client.Append;</span>
<span class="source-line-no">068</span><span id="line-68">import org.apache.hadoop.hbase.client.CheckAndMutate;</span>
<span class="source-line-no">069</span><span id="line-69">import org.apache.hadoop.hbase.client.CheckAndMutateResult;</span>
<span class="source-line-no">070</span><span id="line-70">import org.apache.hadoop.hbase.client.ClientInternalHelper;</span>
<span class="source-line-no">071</span><span id="line-71">import org.apache.hadoop.hbase.client.Delete;</span>
<span class="source-line-no">072</span><span id="line-72">import org.apache.hadoop.hbase.client.Durability;</span>
<span class="source-line-no">073</span><span id="line-73">import org.apache.hadoop.hbase.client.Get;</span>
<span class="source-line-no">074</span><span id="line-74">import org.apache.hadoop.hbase.client.Increment;</span>
<span class="source-line-no">075</span><span id="line-75">import org.apache.hadoop.hbase.client.Mutation;</span>
<span class="source-line-no">076</span><span id="line-76">import org.apache.hadoop.hbase.client.OperationWithAttributes;</span>
<span class="source-line-no">077</span><span id="line-77">import org.apache.hadoop.hbase.client.Put;</span>
<span class="source-line-no">078</span><span id="line-78">import org.apache.hadoop.hbase.client.RegionInfo;</span>
<span class="source-line-no">079</span><span id="line-79">import org.apache.hadoop.hbase.client.RegionReplicaUtil;</span>
<span class="source-line-no">080</span><span id="line-80">import org.apache.hadoop.hbase.client.Result;</span>
<span class="source-line-no">081</span><span id="line-81">import org.apache.hadoop.hbase.client.Row;</span>
<span class="source-line-no">082</span><span id="line-82">import org.apache.hadoop.hbase.client.Scan;</span>
<span class="source-line-no">083</span><span id="line-83">import org.apache.hadoop.hbase.client.TableDescriptor;</span>
<span class="source-line-no">084</span><span id="line-84">import org.apache.hadoop.hbase.client.VersionInfoUtil;</span>
<span class="source-line-no">085</span><span id="line-85">import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;</span>
<span class="source-line-no">086</span><span id="line-86">import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException;</span>
<span class="source-line-no">087</span><span id="line-87">import org.apache.hadoop.hbase.exceptions.ScannerResetException;</span>
<span class="source-line-no">088</span><span id="line-88">import org.apache.hadoop.hbase.exceptions.TimeoutIOException;</span>
<span class="source-line-no">089</span><span id="line-89">import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;</span>
<span class="source-line-no">090</span><span id="line-90">import org.apache.hadoop.hbase.io.ByteBuffAllocator;</span>
<span class="source-line-no">091</span><span id="line-91">import org.apache.hadoop.hbase.io.hfile.BlockCache;</span>
<span class="source-line-no">092</span><span id="line-92">import org.apache.hadoop.hbase.ipc.HBaseRpcController;</span>
<span class="source-line-no">093</span><span id="line-93">import org.apache.hadoop.hbase.ipc.PriorityFunction;</span>
<span class="source-line-no">094</span><span id="line-94">import org.apache.hadoop.hbase.ipc.QosPriority;</span>
<span class="source-line-no">095</span><span id="line-95">import org.apache.hadoop.hbase.ipc.RpcCall;</span>
<span class="source-line-no">096</span><span id="line-96">import org.apache.hadoop.hbase.ipc.RpcCallContext;</span>
<span class="source-line-no">097</span><span id="line-97">import org.apache.hadoop.hbase.ipc.RpcCallback;</span>
<span class="source-line-no">098</span><span id="line-98">import org.apache.hadoop.hbase.ipc.RpcServer;</span>
<span class="source-line-no">099</span><span id="line-99">import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface;</span>
<span class="source-line-no">100</span><span id="line-100">import org.apache.hadoop.hbase.ipc.RpcServerFactory;</span>
<span class="source-line-no">101</span><span id="line-101">import org.apache.hadoop.hbase.ipc.RpcServerInterface;</span>
<span class="source-line-no">102</span><span id="line-102">import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;</span>
<span class="source-line-no">103</span><span id="line-103">import org.apache.hadoop.hbase.ipc.ServerRpcController;</span>
<span class="source-line-no">104</span><span id="line-104">import org.apache.hadoop.hbase.net.Address;</span>
<span class="source-line-no">105</span><span id="line-105">import org.apache.hadoop.hbase.procedure2.RSProcedureCallable;</span>
<span class="source-line-no">106</span><span id="line-106">import org.apache.hadoop.hbase.quotas.ActivePolicyEnforcement;</span>
<span class="source-line-no">107</span><span id="line-107">import org.apache.hadoop.hbase.quotas.OperationQuota;</span>
<span class="source-line-no">108</span><span id="line-108">import org.apache.hadoop.hbase.quotas.QuotaUtil;</span>
<span class="source-line-no">109</span><span id="line-109">import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;</span>
<span class="source-line-no">110</span><span id="line-110">import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;</span>
<span class="source-line-no">111</span><span id="line-111">import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;</span>
<span class="source-line-no">112</span><span id="line-112">import org.apache.hadoop.hbase.quotas.SpaceViolationPolicyEnforcement;</span>
<span class="source-line-no">113</span><span id="line-113">import org.apache.hadoop.hbase.regionserver.LeaseManager.Lease;</span>
<span class="source-line-no">114</span><span id="line-114">import org.apache.hadoop.hbase.regionserver.LeaseManager.LeaseStillHeldException;</span>
<span class="source-line-no">115</span><span id="line-115">import org.apache.hadoop.hbase.regionserver.Region.Operation;</span>
<span class="source-line-no">116</span><span id="line-116">import org.apache.hadoop.hbase.regionserver.ScannerContext.LimitScope;</span>
<span class="source-line-no">117</span><span id="line-117">import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;</span>
<span class="source-line-no">118</span><span id="line-118">import org.apache.hadoop.hbase.regionserver.handler.AssignRegionHandler;</span>
<span class="source-line-no">119</span><span id="line-119">import org.apache.hadoop.hbase.regionserver.handler.OpenMetaHandler;</span>
<span class="source-line-no">120</span><span id="line-120">import org.apache.hadoop.hbase.regionserver.handler.OpenPriorityRegionHandler;</span>
<span class="source-line-no">121</span><span id="line-121">import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler;</span>
<span class="source-line-no">122</span><span id="line-122">import org.apache.hadoop.hbase.regionserver.handler.UnassignRegionHandler;</span>
<span class="source-line-no">123</span><span id="line-123">import org.apache.hadoop.hbase.replication.ReplicationUtils;</span>
<span class="source-line-no">124</span><span id="line-124">import org.apache.hadoop.hbase.replication.regionserver.RejectReplicationRequestStateChecker;</span>
<span class="source-line-no">125</span><span id="line-125">import org.apache.hadoop.hbase.replication.regionserver.RejectRequestsFromClientStateChecker;</span>
<span class="source-line-no">126</span><span id="line-126">import org.apache.hadoop.hbase.security.Superusers;</span>
<span class="source-line-no">127</span><span id="line-127">import org.apache.hadoop.hbase.security.access.Permission;</span>
<span class="source-line-no">128</span><span id="line-128">import org.apache.hadoop.hbase.util.Bytes;</span>
<span class="source-line-no">129</span><span id="line-129">import org.apache.hadoop.hbase.util.DNS;</span>
<span class="source-line-no">130</span><span id="line-130">import org.apache.hadoop.hbase.util.DNS.ServerType;</span>
<span class="source-line-no">131</span><span id="line-131">import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;</span>
<span class="source-line-no">132</span><span id="line-132">import org.apache.hadoop.hbase.util.Pair;</span>
<span class="source-line-no">133</span><span id="line-133">import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;</span>
<span class="source-line-no">134</span><span id="line-134">import org.apache.hadoop.hbase.wal.WAL;</span>
<span class="source-line-no">135</span><span id="line-135">import org.apache.hadoop.hbase.wal.WALEdit;</span>
<span class="source-line-no">136</span><span id="line-136">import org.apache.hadoop.hbase.wal.WALKey;</span>
<span class="source-line-no">137</span><span id="line-137">import org.apache.hadoop.hbase.wal.WALSplitUtil;</span>
<span class="source-line-no">138</span><span id="line-138">import org.apache.hadoop.hbase.wal.WALSplitUtil.MutationReplay;</span>
<span class="source-line-no">139</span><span id="line-139">import org.apache.hadoop.hbase.zookeeper.ZKWatcher;</span>
<span class="source-line-no">140</span><span id="line-140">import org.apache.yetus.audience.InterfaceAudience;</span>
<span class="source-line-no">141</span><span id="line-141">import org.slf4j.Logger;</span>
<span class="source-line-no">142</span><span id="line-142">import org.slf4j.LoggerFactory;</span>
<span class="source-line-no">143</span><span id="line-143"></span>
<span class="source-line-no">144</span><span id="line-144">import org.apache.hbase.thirdparty.com.google.common.cache.Cache;</span>
<span class="source-line-no">145</span><span id="line-145">import org.apache.hbase.thirdparty.com.google.common.cache.CacheBuilder;</span>
<span class="source-line-no">146</span><span id="line-146">import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;</span>
<span class="source-line-no">147</span><span id="line-147">import org.apache.hbase.thirdparty.com.google.common.collect.Lists;</span>
<span class="source-line-no">148</span><span id="line-148">import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;</span>
<span class="source-line-no">149</span><span id="line-149">import org.apache.hbase.thirdparty.com.google.protobuf.Message;</span>
<span class="source-line-no">150</span><span id="line-150">import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;</span>
<span class="source-line-no">151</span><span id="line-151">import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;</span>
<span class="source-line-no">152</span><span id="line-152">import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat;</span>
<span class="source-line-no">153</span><span id="line-153">import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;</span>
<span class="source-line-no">154</span><span id="line-154">import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;</span>
<span class="source-line-no">155</span><span id="line-155"></span>
<span class="source-line-no">156</span><span id="line-156">import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;</span>
<span class="source-line-no">157</span><span id="line-157">import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;</span>
<span class="source-line-no">158</span><span id="line-158">import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;</span>
<span class="source-line-no">159</span><span id="line-159">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;</span>
<span class="source-line-no">160</span><span id="line-160">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearCompactionQueuesRequest;</span>
<span class="source-line-no">161</span><span id="line-161">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearCompactionQueuesResponse;</span>
<span class="source-line-no">162</span><span id="line-162">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearRegionBlockCacheRequest;</span>
<span class="source-line-no">163</span><span id="line-163">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearRegionBlockCacheResponse;</span>
<span class="source-line-no">164</span><span id="line-164">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest;</span>
<span class="source-line-no">165</span><span id="line-165">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse;</span>
<span class="source-line-no">166</span><span id="line-166">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest;</span>
<span class="source-line-no">167</span><span id="line-167">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponse;</span>
<span class="source-line-no">168</span><span id="line-168">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactionSwitchRequest;</span>
<span class="source-line-no">169</span><span id="line-169">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactionSwitchResponse;</span>
<span class="source-line-no">170</span><span id="line-170">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresRequest;</span>
<span class="source-line-no">171</span><span id="line-171">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresResponse;</span>
<span class="source-line-no">172</span><span id="line-172">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest;</span>
<span class="source-line-no">173</span><span id="line-173">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse;</span>
<span class="source-line-no">174</span><span id="line-174">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetCachedFilesListRequest;</span>
<span class="source-line-no">175</span><span id="line-175">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetCachedFilesListResponse;</span>
<span class="source-line-no">176</span><span id="line-176">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest;</span>
<span class="source-line-no">177</span><span id="line-177">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse;</span>
<span class="source-line-no">178</span><span id="line-178">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest;</span>
<span class="source-line-no">179</span><span id="line-179">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;</span>
<span class="source-line-no">180</span><span id="line-180">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadRequest;</span>
<span class="source-line-no">181</span><span id="line-181">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadResponse;</span>
<span class="source-line-no">182</span><span id="line-182">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoRequest;</span>
<span class="source-line-no">183</span><span id="line-183">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse;</span>
<span class="source-line-no">184</span><span id="line-184">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileRequest;</span>
<span class="source-line-no">185</span><span id="line-185">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileResponse;</span>
<span class="source-line-no">186</span><span id="line-186">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest;</span>
<span class="source-line-no">187</span><span id="line-187">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfo;</span>
<span class="source-line-no">188</span><span id="line-188">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse;</span>
<span class="source-line-no">189</span><span id="line-189">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse.RegionOpeningState;</span>
<span class="source-line-no">190</span><span id="line-190">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RemoteProcedureRequest;</span>
<span class="source-line-no">191</span><span id="line-191">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest;</span>
<span class="source-line-no">192</span><span id="line-192">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse;</span>
<span class="source-line-no">193</span><span id="line-193">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterRequest;</span>
<span class="source-line-no">194</span><span id="line-194">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse;</span>
<span class="source-line-no">195</span><span id="line-195">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerRequest;</span>
<span class="source-line-no">196</span><span id="line-196">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerResponse;</span>
<span class="source-line-no">197</span><span id="line-197">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest;</span>
<span class="source-line-no">198</span><span id="line-198">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse;</span>
<span class="source-line-no">199</span><span id="line-199">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry;</span>
<span class="source-line-no">200</span><span id="line-200">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionRequest;</span>
<span class="source-line-no">201</span><span id="line-201">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionResponse;</span>
<span class="source-line-no">202</span><span id="line-202">import org.apache.hadoop.hbase.shaded.protobuf.generated.BootstrapNodeProtos.BootstrapNodeService;</span>
<span class="source-line-no">203</span><span id="line-203">import org.apache.hadoop.hbase.shaded.protobuf.generated.BootstrapNodeProtos.GetAllBootstrapNodesRequest;</span>
<span class="source-line-no">204</span><span id="line-204">import org.apache.hadoop.hbase.shaded.protobuf.generated.BootstrapNodeProtos.GetAllBootstrapNodesResponse;</span>
<span class="source-line-no">205</span><span id="line-205">import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;</span>
<span class="source-line-no">206</span><span id="line-206">import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action;</span>
<span class="source-line-no">207</span><span id="line-207">import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest;</span>
<span class="source-line-no">208</span><span id="line-208">import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest.FamilyPath;</span>
<span class="source-line-no">209</span><span id="line-209">import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileResponse;</span>
<span class="source-line-no">210</span><span id="line-210">import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest;</span>
<span class="source-line-no">211</span><span id="line-211">import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadResponse;</span>
<span class="source-line-no">212</span><span id="line-212">import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;</span>
<span class="source-line-no">213</span><span id="line-213">import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Condition;</span>
<span class="source-line-no">214</span><span id="line-214">import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest;</span>
<span class="source-line-no">215</span><span id="line-215">import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse;</span>
<span class="source-line-no">216</span><span id="line-216">import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest;</span>
<span class="source-line-no">217</span><span id="line-217">import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetResponse;</span>
<span class="source-line-no">218</span><span id="line-218">import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRegionLoadStats;</span>
<span class="source-line-no">219</span><span id="line-219">import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest;</span>
<span class="source-line-no">220</span><span id="line-220">import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiResponse;</span>
<span class="source-line-no">221</span><span id="line-221">import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest;</span>
<span class="source-line-no">222</span><span id="line-222">import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse;</span>
<span class="source-line-no">223</span><span id="line-223">import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto;</span>
<span class="source-line-no">224</span><span id="line-224">import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.MutationType;</span>
<span class="source-line-no">225</span><span id="line-225">import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest;</span>
<span class="source-line-no">226</span><span id="line-226">import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadResponse;</span>
<span class="source-line-no">227</span><span id="line-227">import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction;</span>
<span class="source-line-no">228</span><span id="line-228">import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult;</span>
<span class="source-line-no">229</span><span id="line-229">import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException;</span>
<span class="source-line-no">230</span><span id="line-230">import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest;</span>
<span class="source-line-no">231</span><span id="line-231">import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse;</span>
<span class="source-line-no">232</span><span id="line-232">import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;</span>
<span class="source-line-no">233</span><span id="line-233">import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad;</span>
<span class="source-line-no">234</span><span id="line-234">import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair;</span>
<span class="source-line-no">235</span><span id="line-235">import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameInt64Pair;</span>
<span class="source-line-no">236</span><span id="line-236">import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier;</span>
<span class="source-line-no">237</span><span id="line-237">import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;</span>
<span class="source-line-no">238</span><span id="line-238">import org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.ScanMetrics;</span>
<span class="source-line-no">239</span><span id="line-239">import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsRequest;</span>
<span class="source-line-no">240</span><span id="line-240">import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse;</span>
<span class="source-line-no">241</span><span id="line-241">import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshot;</span>
<span class="source-line-no">242</span><span id="line-242">import org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.ClientMetaService;</span>
<span class="source-line-no">243</span><span id="line-243">import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor;</span>
<span class="source-line-no">244</span><span id="line-244">import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;</span>
<span class="source-line-no">245</span><span id="line-245">import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor;</span>
<span class="source-line-no">246</span><span id="line-246">import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor;</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"> * Implements the regionserver RPC services.</span>
<span class="source-line-no">250</span><span id="line-250"> */</span>
<span class="source-line-no">251</span><span id="line-251">@InterfaceAudience.Private</span>
<span class="source-line-no">252</span><span id="line-252">public class RSRpcServices extends HBaseRpcServicesBase&lt;HRegionServer&gt;</span>
<span class="source-line-no">253</span><span id="line-253"> implements ClientService.BlockingInterface, BootstrapNodeService.BlockingInterface {</span>
<span class="source-line-no">254</span><span id="line-254"></span>
<span class="source-line-no">255</span><span id="line-255"> private static final Logger LOG = LoggerFactory.getLogger(RSRpcServices.class);</span>
<span class="source-line-no">256</span><span id="line-256"></span>
<span class="source-line-no">257</span><span id="line-257"> /** RPC scheduler to use for the region server. */</span>
<span class="source-line-no">258</span><span id="line-258"> public static final String REGION_SERVER_RPC_SCHEDULER_FACTORY_CLASS =</span>
<span class="source-line-no">259</span><span id="line-259"> "hbase.region.server.rpc.scheduler.factory.class";</span>
<span class="source-line-no">260</span><span id="line-260"></span>
<span class="source-line-no">261</span><span id="line-261"> /**</span>
<span class="source-line-no">262</span><span id="line-262"> * Minimum allowable time limit delta (in milliseconds) that can be enforced during scans. This</span>
<span class="source-line-no">263</span><span id="line-263"> * configuration exists to prevent the scenario where a time limit is specified to be so</span>
<span class="source-line-no">264</span><span id="line-264"> * restrictive that the time limit is reached immediately (before any cells are scanned).</span>
<span class="source-line-no">265</span><span id="line-265"> */</span>
<span class="source-line-no">266</span><span id="line-266"> private static final String REGION_SERVER_RPC_MINIMUM_SCAN_TIME_LIMIT_DELTA =</span>
<span class="source-line-no">267</span><span id="line-267"> "hbase.region.server.rpc.minimum.scan.time.limit.delta";</span>
<span class="source-line-no">268</span><span id="line-268"> /**</span>
<span class="source-line-no">269</span><span id="line-269"> * Default value of {@link RSRpcServices#REGION_SERVER_RPC_MINIMUM_SCAN_TIME_LIMIT_DELTA}</span>
<span class="source-line-no">270</span><span id="line-270"> */</span>
<span class="source-line-no">271</span><span id="line-271"> static final long DEFAULT_REGION_SERVER_RPC_MINIMUM_SCAN_TIME_LIMIT_DELTA = 10;</span>
<span class="source-line-no">272</span><span id="line-272"></span>
<span class="source-line-no">273</span><span id="line-273"> /**</span>
<span class="source-line-no">274</span><span id="line-274"> * Whether to reject rows with size &gt; threshold defined by</span>
<span class="source-line-no">275</span><span id="line-275"> * {@link HConstants#BATCH_ROWS_THRESHOLD_NAME}</span>
<span class="source-line-no">276</span><span id="line-276"> */</span>
<span class="source-line-no">277</span><span id="line-277"> private static final String REJECT_BATCH_ROWS_OVER_THRESHOLD =</span>
<span class="source-line-no">278</span><span id="line-278"> "hbase.rpc.rows.size.threshold.reject";</span>
<span class="source-line-no">279</span><span id="line-279"></span>
<span class="source-line-no">280</span><span id="line-280"> /**</span>
<span class="source-line-no">281</span><span id="line-281"> * Default value of config {@link RSRpcServices#REJECT_BATCH_ROWS_OVER_THRESHOLD}</span>
<span class="source-line-no">282</span><span id="line-282"> */</span>
<span class="source-line-no">283</span><span id="line-283"> private static final boolean DEFAULT_REJECT_BATCH_ROWS_OVER_THRESHOLD = false;</span>
<span class="source-line-no">284</span><span id="line-284"></span>
<span class="source-line-no">285</span><span id="line-285"> // Request counter. (Includes requests that are not serviced by regions.)</span>
<span class="source-line-no">286</span><span id="line-286"> // Count only once for requests with multiple actions like multi/caching-scan/replayBatch</span>
<span class="source-line-no">287</span><span id="line-287"> final LongAdder requestCount = new LongAdder();</span>
<span class="source-line-no">288</span><span id="line-288"></span>
<span class="source-line-no">289</span><span id="line-289"> // Request counter for rpc get</span>
<span class="source-line-no">290</span><span id="line-290"> final LongAdder rpcGetRequestCount = new LongAdder();</span>
<span class="source-line-no">291</span><span id="line-291"></span>
<span class="source-line-no">292</span><span id="line-292"> // Request counter for rpc scan</span>
<span class="source-line-no">293</span><span id="line-293"> final LongAdder rpcScanRequestCount = new LongAdder();</span>
<span class="source-line-no">294</span><span id="line-294"></span>
<span class="source-line-no">295</span><span id="line-295"> // Request counter for scans that might end up in full scans</span>
<span class="source-line-no">296</span><span id="line-296"> final LongAdder rpcFullScanRequestCount = new LongAdder();</span>
<span class="source-line-no">297</span><span id="line-297"></span>
<span class="source-line-no">298</span><span id="line-298"> // Request counter for rpc multi</span>
<span class="source-line-no">299</span><span id="line-299"> final LongAdder rpcMultiRequestCount = new LongAdder();</span>
<span class="source-line-no">300</span><span id="line-300"></span>
<span class="source-line-no">301</span><span id="line-301"> // Request counter for rpc mutate</span>
<span class="source-line-no">302</span><span id="line-302"> final LongAdder rpcMutateRequestCount = new LongAdder();</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 volatile long maxScannerResultSize;</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 ScannerIdGenerator scannerIdGenerator;</span>
<span class="source-line-no">307</span><span id="line-307"> private final ConcurrentMap&lt;String, RegionScannerHolder&gt; scanners = new ConcurrentHashMap&lt;&gt;();</span>
<span class="source-line-no">308</span><span id="line-308"> // Hold the name and last sequence number of a closed scanner for a while. This is used</span>
<span class="source-line-no">309</span><span id="line-309"> // to keep compatible for old clients which may send next or close request to a region</span>
<span class="source-line-no">310</span><span id="line-310"> // scanner which has already been exhausted. The entries will be removed automatically</span>
<span class="source-line-no">311</span><span id="line-311"> // after scannerLeaseTimeoutPeriod.</span>
<span class="source-line-no">312</span><span id="line-312"> private final Cache&lt;String, Long&gt; closedScanners;</span>
<span class="source-line-no">313</span><span id="line-313"> /**</span>
<span class="source-line-no">314</span><span id="line-314"> * The lease timeout period for client scanners (milliseconds).</span>
<span class="source-line-no">315</span><span id="line-315"> */</span>
<span class="source-line-no">316</span><span id="line-316"> private final int scannerLeaseTimeoutPeriod;</span>
<span class="source-line-no">317</span><span id="line-317"></span>
<span class="source-line-no">318</span><span id="line-318"> /**</span>
<span class="source-line-no">319</span><span id="line-319"> * The RPC timeout period (milliseconds)</span>
<span class="source-line-no">320</span><span id="line-320"> */</span>
<span class="source-line-no">321</span><span id="line-321"> private final int rpcTimeout;</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"> * The minimum allowable delta to use for the scan limit</span>
<span class="source-line-no">325</span><span id="line-325"> */</span>
<span class="source-line-no">326</span><span id="line-326"> private final long minimumScanTimeLimitDelta;</span>
<span class="source-line-no">327</span><span id="line-327"></span>
<span class="source-line-no">328</span><span id="line-328"> /**</span>
<span class="source-line-no">329</span><span id="line-329"> * Row size threshold for multi requests above which a warning is logged</span>
<span class="source-line-no">330</span><span id="line-330"> */</span>
<span class="source-line-no">331</span><span id="line-331"> private volatile int rowSizeWarnThreshold;</span>
<span class="source-line-no">332</span><span id="line-332"> /*</span>
<span class="source-line-no">333</span><span id="line-333"> * Whether we should reject requests with very high no of rows i.e. beyond threshold defined by</span>
<span class="source-line-no">334</span><span id="line-334"> * rowSizeWarnThreshold</span>
<span class="source-line-no">335</span><span id="line-335"> */</span>
<span class="source-line-no">336</span><span id="line-336"> private volatile boolean rejectRowsWithSizeOverThreshold;</span>
<span class="source-line-no">337</span><span id="line-337"></span>
<span class="source-line-no">338</span><span id="line-338"> final AtomicBoolean clearCompactionQueues = new AtomicBoolean(false);</span>
<span class="source-line-no">339</span><span id="line-339"></span>
<span class="source-line-no">340</span><span id="line-340"> /**</span>
<span class="source-line-no">341</span><span id="line-341"> * Services launched in RSRpcServices. By default they are on but you can use the below booleans</span>
<span class="source-line-no">342</span><span id="line-342"> * to selectively enable/disable these services (Rare is the case where you would ever turn off</span>
<span class="source-line-no">343</span><span id="line-343"> * one or the other).</span>
<span class="source-line-no">344</span><span id="line-344"> */</span>
<span class="source-line-no">345</span><span id="line-345"> public static final String REGIONSERVER_ADMIN_SERVICE_CONFIG =</span>
<span class="source-line-no">346</span><span id="line-346"> "hbase.regionserver.admin.executorService";</span>
<span class="source-line-no">347</span><span id="line-347"> public static final String REGIONSERVER_CLIENT_SERVICE_CONFIG =</span>
<span class="source-line-no">348</span><span id="line-348"> "hbase.regionserver.client.executorService";</span>
<span class="source-line-no">349</span><span id="line-349"> public static final String REGIONSERVER_CLIENT_META_SERVICE_CONFIG =</span>
<span class="source-line-no">350</span><span id="line-350"> "hbase.regionserver.client.meta.executorService";</span>
<span class="source-line-no">351</span><span id="line-351"> public static final String REGIONSERVER_BOOTSTRAP_NODES_SERVICE_CONFIG =</span>
<span class="source-line-no">352</span><span id="line-352"> "hbase.regionserver.bootstrap.nodes.executorService";</span>
<span class="source-line-no">353</span><span id="line-353"></span>
<span class="source-line-no">354</span><span id="line-354"> /**</span>
<span class="source-line-no">355</span><span id="line-355"> * An Rpc callback for closing a RegionScanner.</span>
<span class="source-line-no">356</span><span id="line-356"> */</span>
<span class="source-line-no">357</span><span id="line-357"> private static final class RegionScannerCloseCallBack implements RpcCallback {</span>
<span class="source-line-no">358</span><span id="line-358"></span>
<span class="source-line-no">359</span><span id="line-359"> private final RegionScanner scanner;</span>
<span class="source-line-no">360</span><span id="line-360"></span>
<span class="source-line-no">361</span><span id="line-361"> public RegionScannerCloseCallBack(RegionScanner scanner) {</span>
<span class="source-line-no">362</span><span id="line-362"> this.scanner = scanner;</span>
<span class="source-line-no">363</span><span id="line-363"> }</span>
<span class="source-line-no">364</span><span id="line-364"></span>
<span class="source-line-no">365</span><span id="line-365"> @Override</span>
<span class="source-line-no">366</span><span id="line-366"> public void run() throws IOException {</span>
<span class="source-line-no">367</span><span id="line-367"> this.scanner.close();</span>
<span class="source-line-no">368</span><span id="line-368"> }</span>
<span class="source-line-no">369</span><span id="line-369"> }</span>
<span class="source-line-no">370</span><span id="line-370"></span>
<span class="source-line-no">371</span><span id="line-371"> /**</span>
<span class="source-line-no">372</span><span id="line-372"> * An Rpc callback for doing shipped() call on a RegionScanner.</span>
<span class="source-line-no">373</span><span id="line-373"> */</span>
<span class="source-line-no">374</span><span id="line-374"> private class RegionScannerShippedCallBack implements RpcCallback {</span>
<span class="source-line-no">375</span><span id="line-375"> private final String scannerName;</span>
<span class="source-line-no">376</span><span id="line-376"> private final Shipper shipper;</span>
<span class="source-line-no">377</span><span id="line-377"> private final Lease lease;</span>
<span class="source-line-no">378</span><span id="line-378"></span>
<span class="source-line-no">379</span><span id="line-379"> public RegionScannerShippedCallBack(String scannerName, Shipper shipper, Lease lease) {</span>
<span class="source-line-no">380</span><span id="line-380"> this.scannerName = scannerName;</span>
<span class="source-line-no">381</span><span id="line-381"> this.shipper = shipper;</span>
<span class="source-line-no">382</span><span id="line-382"> this.lease = lease;</span>
<span class="source-line-no">383</span><span id="line-383"> }</span>
<span class="source-line-no">384</span><span id="line-384"></span>
<span class="source-line-no">385</span><span id="line-385"> @Override</span>
<span class="source-line-no">386</span><span id="line-386"> public void run() throws IOException {</span>
<span class="source-line-no">387</span><span id="line-387"> this.shipper.shipped();</span>
<span class="source-line-no">388</span><span id="line-388"> // We're done. On way out re-add the above removed lease. The lease was temp removed for this</span>
<span class="source-line-no">389</span><span id="line-389"> // Rpc call and we are at end of the call now. Time to add it back.</span>
<span class="source-line-no">390</span><span id="line-390"> if (scanners.containsKey(scannerName)) {</span>
<span class="source-line-no">391</span><span id="line-391"> if (lease != null) {</span>
<span class="source-line-no">392</span><span id="line-392"> server.getLeaseManager().addLease(lease);</span>
<span class="source-line-no">393</span><span id="line-393"> }</span>
<span class="source-line-no">394</span><span id="line-394"> }</span>
<span class="source-line-no">395</span><span id="line-395"> }</span>
<span class="source-line-no">396</span><span id="line-396"> }</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"> * An RpcCallBack that creates a list of scanners that needs to perform callBack operation on</span>
<span class="source-line-no">400</span><span id="line-400"> * completion of multiGets.</span>
<span class="source-line-no">401</span><span id="line-401"> */</span>
<span class="source-line-no">402</span><span id="line-402"> static class RegionScannersCloseCallBack implements RpcCallback {</span>
<span class="source-line-no">403</span><span id="line-403"> private final List&lt;RegionScanner&gt; scanners = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">404</span><span id="line-404"></span>
<span class="source-line-no">405</span><span id="line-405"> public void addScanner(RegionScanner scanner) {</span>
<span class="source-line-no">406</span><span id="line-406"> this.scanners.add(scanner);</span>
<span class="source-line-no">407</span><span id="line-407"> }</span>
<span class="source-line-no">408</span><span id="line-408"></span>
<span class="source-line-no">409</span><span id="line-409"> @Override</span>
<span class="source-line-no">410</span><span id="line-410"> public void run() {</span>
<span class="source-line-no">411</span><span id="line-411"> for (RegionScanner scanner : scanners) {</span>
<span class="source-line-no">412</span><span id="line-412"> try {</span>
<span class="source-line-no">413</span><span id="line-413"> scanner.close();</span>
<span class="source-line-no">414</span><span id="line-414"> } catch (IOException e) {</span>
<span class="source-line-no">415</span><span id="line-415"> LOG.error("Exception while closing the scanner " + scanner, e);</span>
<span class="source-line-no">416</span><span id="line-416"> }</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"> }</span>
<span class="source-line-no">420</span><span id="line-420"></span>
<span class="source-line-no">421</span><span id="line-421"> static class RegionScannerContext {</span>
<span class="source-line-no">422</span><span id="line-422"> final String scannerName;</span>
<span class="source-line-no">423</span><span id="line-423"> final RegionScannerHolder holder;</span>
<span class="source-line-no">424</span><span id="line-424"> final OperationQuota quota;</span>
<span class="source-line-no">425</span><span id="line-425"></span>
<span class="source-line-no">426</span><span id="line-426"> RegionScannerContext(String scannerName, RegionScannerHolder holder, OperationQuota quota) {</span>
<span class="source-line-no">427</span><span id="line-427"> this.scannerName = scannerName;</span>
<span class="source-line-no">428</span><span id="line-428"> this.holder = holder;</span>
<span class="source-line-no">429</span><span id="line-429"> this.quota = quota;</span>
<span class="source-line-no">430</span><span id="line-430"> }</span>
<span class="source-line-no">431</span><span id="line-431"> }</span>
<span class="source-line-no">432</span><span id="line-432"></span>
<span class="source-line-no">433</span><span id="line-433"> /**</span>
<span class="source-line-no">434</span><span id="line-434"> * Holder class which holds the RegionScanner, nextCallSeq and RpcCallbacks together.</span>
<span class="source-line-no">435</span><span id="line-435"> */</span>
<span class="source-line-no">436</span><span id="line-436"> static final class RegionScannerHolder {</span>
<span class="source-line-no">437</span><span id="line-437"> private final AtomicLong nextCallSeq = new AtomicLong(0);</span>
<span class="source-line-no">438</span><span id="line-438"> private final RegionScanner s;</span>
<span class="source-line-no">439</span><span id="line-439"> private final HRegion r;</span>
<span class="source-line-no">440</span><span id="line-440"> private final RpcCallback closeCallBack;</span>
<span class="source-line-no">441</span><span id="line-441"> private final RpcCallback shippedCallback;</span>
<span class="source-line-no">442</span><span id="line-442"> private byte[] rowOfLastPartialResult;</span>
<span class="source-line-no">443</span><span id="line-443"> private boolean needCursor;</span>
<span class="source-line-no">444</span><span id="line-444"> private boolean fullRegionScan;</span>
<span class="source-line-no">445</span><span id="line-445"> private final String clientIPAndPort;</span>
<span class="source-line-no">446</span><span id="line-446"> private final String userName;</span>
<span class="source-line-no">447</span><span id="line-447"> private volatile long maxBlockBytesScanned = 0;</span>
<span class="source-line-no">448</span><span id="line-448"> private volatile long prevBlockBytesScanned = 0;</span>
<span class="source-line-no">449</span><span id="line-449"> private volatile long prevBlockBytesScannedDifference = 0;</span>
<span class="source-line-no">450</span><span id="line-450"></span>
<span class="source-line-no">451</span><span id="line-451"> RegionScannerHolder(RegionScanner s, HRegion r, RpcCallback closeCallBack,</span>
<span class="source-line-no">452</span><span id="line-452"> RpcCallback shippedCallback, boolean needCursor, boolean fullRegionScan,</span>
<span class="source-line-no">453</span><span id="line-453"> String clientIPAndPort, String userName) {</span>
<span class="source-line-no">454</span><span id="line-454"> this.s = s;</span>
<span class="source-line-no">455</span><span id="line-455"> this.r = r;</span>
<span class="source-line-no">456</span><span id="line-456"> this.closeCallBack = closeCallBack;</span>
<span class="source-line-no">457</span><span id="line-457"> this.shippedCallback = shippedCallback;</span>
<span class="source-line-no">458</span><span id="line-458"> this.needCursor = needCursor;</span>
<span class="source-line-no">459</span><span id="line-459"> this.fullRegionScan = fullRegionScan;</span>
<span class="source-line-no">460</span><span id="line-460"> this.clientIPAndPort = clientIPAndPort;</span>
<span class="source-line-no">461</span><span id="line-461"> this.userName = userName;</span>
<span class="source-line-no">462</span><span id="line-462"> }</span>
<span class="source-line-no">463</span><span id="line-463"></span>
<span class="source-line-no">464</span><span id="line-464"> long getNextCallSeq() {</span>
<span class="source-line-no">465</span><span id="line-465"> return nextCallSeq.get();</span>
<span class="source-line-no">466</span><span id="line-466"> }</span>
<span class="source-line-no">467</span><span id="line-467"></span>
<span class="source-line-no">468</span><span id="line-468"> boolean incNextCallSeq(long currentSeq) {</span>
<span class="source-line-no">469</span><span id="line-469"> // Use CAS to prevent multiple scan request running on the same scanner.</span>
<span class="source-line-no">470</span><span id="line-470"> return nextCallSeq.compareAndSet(currentSeq, currentSeq + 1);</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"> long getMaxBlockBytesScanned() {</span>
<span class="source-line-no">474</span><span id="line-474"> return maxBlockBytesScanned;</span>
<span class="source-line-no">475</span><span id="line-475"> }</span>
<span class="source-line-no">476</span><span id="line-476"></span>
<span class="source-line-no">477</span><span id="line-477"> long getPrevBlockBytesScannedDifference() {</span>
<span class="source-line-no">478</span><span id="line-478"> return prevBlockBytesScannedDifference;</span>
<span class="source-line-no">479</span><span id="line-479"> }</span>
<span class="source-line-no">480</span><span id="line-480"></span>
<span class="source-line-no">481</span><span id="line-481"> void updateBlockBytesScanned(long blockBytesScanned) {</span>
<span class="source-line-no">482</span><span id="line-482"> prevBlockBytesScannedDifference = blockBytesScanned - prevBlockBytesScanned;</span>
<span class="source-line-no">483</span><span id="line-483"> prevBlockBytesScanned = blockBytesScanned;</span>
<span class="source-line-no">484</span><span id="line-484"> if (blockBytesScanned &gt; maxBlockBytesScanned) {</span>
<span class="source-line-no">485</span><span id="line-485"> maxBlockBytesScanned = blockBytesScanned;</span>
<span class="source-line-no">486</span><span id="line-486"> }</span>
<span class="source-line-no">487</span><span id="line-487"> }</span>
<span class="source-line-no">488</span><span id="line-488"></span>
<span class="source-line-no">489</span><span id="line-489"> // Should be called only when we need to print lease expired messages otherwise</span>
<span class="source-line-no">490</span><span id="line-490"> // cache the String once made.</span>
<span class="source-line-no">491</span><span id="line-491"> @Override</span>
<span class="source-line-no">492</span><span id="line-492"> public String toString() {</span>
<span class="source-line-no">493</span><span id="line-493"> return "clientIPAndPort=" + this.clientIPAndPort + ", userName=" + this.userName</span>
<span class="source-line-no">494</span><span id="line-494"> + ", regionInfo=" + this.r.getRegionInfo().getRegionNameAsString();</span>
<span class="source-line-no">495</span><span id="line-495"> }</span>
<span class="source-line-no">496</span><span id="line-496"> }</span>
<span class="source-line-no">497</span><span id="line-497"></span>
<span class="source-line-no">498</span><span id="line-498"> /**</span>
<span class="source-line-no">499</span><span id="line-499"> * Instantiated as a scanner lease. If the lease times out, the scanner is closed</span>
<span class="source-line-no">500</span><span id="line-500"> */</span>
<span class="source-line-no">501</span><span id="line-501"> private class ScannerListener implements LeaseListener {</span>
<span class="source-line-no">502</span><span id="line-502"> private final String scannerName;</span>
<span class="source-line-no">503</span><span id="line-503"></span>
<span class="source-line-no">504</span><span id="line-504"> ScannerListener(final String n) {</span>
<span class="source-line-no">505</span><span id="line-505"> this.scannerName = n;</span>
<span class="source-line-no">506</span><span id="line-506"> }</span>
<span class="source-line-no">507</span><span id="line-507"></span>
<span class="source-line-no">508</span><span id="line-508"> @Override</span>
<span class="source-line-no">509</span><span id="line-509"> public void leaseExpired() {</span>
<span class="source-line-no">510</span><span id="line-510"> RegionScannerHolder rsh = scanners.remove(this.scannerName);</span>
<span class="source-line-no">511</span><span id="line-511"> if (rsh == null) {</span>
<span class="source-line-no">512</span><span id="line-512"> LOG.warn("Scanner lease {} expired but no outstanding scanner", this.scannerName);</span>
<span class="source-line-no">513</span><span id="line-513"> return;</span>
<span class="source-line-no">514</span><span id="line-514"> }</span>
<span class="source-line-no">515</span><span id="line-515"> LOG.info("Scanner lease {} expired {}", this.scannerName, rsh);</span>
<span class="source-line-no">516</span><span id="line-516"> server.getMetrics().incrScannerLeaseExpired();</span>
<span class="source-line-no">517</span><span id="line-517"> RegionScanner s = rsh.s;</span>
<span class="source-line-no">518</span><span id="line-518"> HRegion region = null;</span>
<span class="source-line-no">519</span><span id="line-519"> try {</span>
<span class="source-line-no">520</span><span id="line-520"> region = server.getRegion(s.getRegionInfo().getRegionName());</span>
<span class="source-line-no">521</span><span id="line-521"> if (region != null &amp;&amp; region.getCoprocessorHost() != null) {</span>
<span class="source-line-no">522</span><span id="line-522"> region.getCoprocessorHost().preScannerClose(s);</span>
<span class="source-line-no">523</span><span id="line-523"> }</span>
<span class="source-line-no">524</span><span id="line-524"> } catch (IOException e) {</span>
<span class="source-line-no">525</span><span id="line-525"> LOG.error("Closing scanner {} {}", this.scannerName, rsh, e);</span>
<span class="source-line-no">526</span><span id="line-526"> } finally {</span>
<span class="source-line-no">527</span><span id="line-527"> try {</span>
<span class="source-line-no">528</span><span id="line-528"> s.close();</span>
<span class="source-line-no">529</span><span id="line-529"> if (region != null &amp;&amp; region.getCoprocessorHost() != null) {</span>
<span class="source-line-no">530</span><span id="line-530"> region.getCoprocessorHost().postScannerClose(s);</span>
<span class="source-line-no">531</span><span id="line-531"> }</span>
<span class="source-line-no">532</span><span id="line-532"> } catch (IOException e) {</span>
<span class="source-line-no">533</span><span id="line-533"> LOG.error("Closing scanner {} {}", this.scannerName, rsh, e);</span>
<span class="source-line-no">534</span><span id="line-534"> }</span>
<span class="source-line-no">535</span><span id="line-535"> }</span>
<span class="source-line-no">536</span><span id="line-536"> }</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"> private static ResultOrException getResultOrException(final ClientProtos.Result r,</span>
<span class="source-line-no">540</span><span id="line-540"> final int index) {</span>
<span class="source-line-no">541</span><span id="line-541"> return getResultOrException(ResponseConverter.buildActionResult(r), index);</span>
<span class="source-line-no">542</span><span id="line-542"> }</span>
<span class="source-line-no">543</span><span id="line-543"></span>
<span class="source-line-no">544</span><span id="line-544"> private static ResultOrException getResultOrException(final Exception e, final int index) {</span>
<span class="source-line-no">545</span><span id="line-545"> return getResultOrException(ResponseConverter.buildActionResult(e), index);</span>
<span class="source-line-no">546</span><span id="line-546"> }</span>
<span class="source-line-no">547</span><span id="line-547"></span>
<span class="source-line-no">548</span><span id="line-548"> private static ResultOrException getResultOrException(final ResultOrException.Builder builder,</span>
<span class="source-line-no">549</span><span id="line-549"> final int index) {</span>
<span class="source-line-no">550</span><span id="line-550"> return builder.setIndex(index).build();</span>
<span class="source-line-no">551</span><span id="line-551"> }</span>
<span class="source-line-no">552</span><span id="line-552"></span>
<span class="source-line-no">553</span><span id="line-553"> /**</span>
<span class="source-line-no">554</span><span id="line-554"> * Checks for the following pre-checks in order:</span>
<span class="source-line-no">555</span><span id="line-555"> * &lt;ol&gt;</span>
<span class="source-line-no">556</span><span id="line-556"> * &lt;li&gt;RegionServer is running&lt;/li&gt;</span>
<span class="source-line-no">557</span><span id="line-557"> * &lt;li&gt;If authorization is enabled, then RPC caller has ADMIN permissions&lt;/li&gt;</span>
<span class="source-line-no">558</span><span id="line-558"> * &lt;/ol&gt;</span>
<span class="source-line-no">559</span><span id="line-559"> * @param requestName name of rpc request. Used in reporting failures to provide context.</span>
<span class="source-line-no">560</span><span id="line-560"> * @throws ServiceException If any of the above listed pre-check fails.</span>
<span class="source-line-no">561</span><span id="line-561"> */</span>
<span class="source-line-no">562</span><span id="line-562"> private void rpcPreCheck(String requestName) throws ServiceException {</span>
<span class="source-line-no">563</span><span id="line-563"> try {</span>
<span class="source-line-no">564</span><span id="line-564"> checkOpen();</span>
<span class="source-line-no">565</span><span id="line-565"> requirePermission(requestName, Permission.Action.ADMIN);</span>
<span class="source-line-no">566</span><span id="line-566"> } catch (IOException ioe) {</span>
<span class="source-line-no">567</span><span id="line-567"> throw new ServiceException(ioe);</span>
<span class="source-line-no">568</span><span id="line-568"> }</span>
<span class="source-line-no">569</span><span id="line-569"> }</span>
<span class="source-line-no">570</span><span id="line-570"></span>
<span class="source-line-no">571</span><span id="line-571"> private boolean isClientCellBlockSupport(RpcCallContext context) {</span>
<span class="source-line-no">572</span><span id="line-572"> return context != null &amp;&amp; context.isClientCellBlockSupported();</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"> private void addResult(final MutateResponse.Builder builder, final Result result,</span>
<span class="source-line-no">576</span><span id="line-576"> final HBaseRpcController rpcc, boolean clientCellBlockSupported) {</span>
<span class="source-line-no">577</span><span id="line-577"> if (result == null) return;</span>
<span class="source-line-no">578</span><span id="line-578"> if (clientCellBlockSupported) {</span>
<span class="source-line-no">579</span><span id="line-579"> builder.setResult(ProtobufUtil.toResultNoData(result));</span>
<span class="source-line-no">580</span><span id="line-580"> rpcc.setCellScanner(result.cellScanner());</span>
<span class="source-line-no">581</span><span id="line-581"> } else {</span>
<span class="source-line-no">582</span><span id="line-582"> ClientProtos.Result pbr = ProtobufUtil.toResult(result);</span>
<span class="source-line-no">583</span><span id="line-583"> builder.setResult(pbr);</span>
<span class="source-line-no">584</span><span id="line-584"> }</span>
<span class="source-line-no">585</span><span id="line-585"> }</span>
<span class="source-line-no">586</span><span id="line-586"></span>
<span class="source-line-no">587</span><span id="line-587"> private void addResults(ScanResponse.Builder builder, List&lt;Result&gt; results,</span>
<span class="source-line-no">588</span><span id="line-588"> HBaseRpcController controller, boolean isDefaultRegion, boolean clientCellBlockSupported) {</span>
<span class="source-line-no">589</span><span id="line-589"> builder.setStale(!isDefaultRegion);</span>
<span class="source-line-no">590</span><span id="line-590"> if (results.isEmpty()) {</span>
<span class="source-line-no">591</span><span id="line-591"> return;</span>
<span class="source-line-no">592</span><span id="line-592"> }</span>
<span class="source-line-no">593</span><span id="line-593"> if (clientCellBlockSupported) {</span>
<span class="source-line-no">594</span><span id="line-594"> for (Result res : results) {</span>
<span class="source-line-no">595</span><span id="line-595"> builder.addCellsPerResult(res.size());</span>
<span class="source-line-no">596</span><span id="line-596"> builder.addPartialFlagPerResult(res.mayHaveMoreCellsInRow());</span>
<span class="source-line-no">597</span><span id="line-597"> }</span>
<span class="source-line-no">598</span><span id="line-598"> controller.setCellScanner(PrivateCellUtil.createExtendedCellScanner(results));</span>
<span class="source-line-no">599</span><span id="line-599"> } else {</span>
<span class="source-line-no">600</span><span id="line-600"> for (Result res : results) {</span>
<span class="source-line-no">601</span><span id="line-601"> ClientProtos.Result pbr = ProtobufUtil.toResult(res);</span>
<span class="source-line-no">602</span><span id="line-602"> builder.addResults(pbr);</span>
<span class="source-line-no">603</span><span id="line-603"> }</span>
<span class="source-line-no">604</span><span id="line-604"> }</span>
<span class="source-line-no">605</span><span id="line-605"> }</span>
<span class="source-line-no">606</span><span id="line-606"></span>
<span class="source-line-no">607</span><span id="line-607"> private CheckAndMutateResult checkAndMutate(HRegion region, List&lt;ClientProtos.Action&gt; actions,</span>
<span class="source-line-no">608</span><span id="line-608"> CellScanner cellScanner, Condition condition, long nonceGroup,</span>
<span class="source-line-no">609</span><span id="line-609"> ActivePolicyEnforcement spaceQuotaEnforcement) throws IOException {</span>
<span class="source-line-no">610</span><span id="line-610"> int countOfCompleteMutation = 0;</span>
<span class="source-line-no">611</span><span id="line-611"> try {</span>
<span class="source-line-no">612</span><span id="line-612"> if (!region.getRegionInfo().isMetaRegion()) {</span>
<span class="source-line-no">613</span><span id="line-613"> server.getMemStoreFlusher().reclaimMemStoreMemory();</span>
<span class="source-line-no">614</span><span id="line-614"> }</span>
<span class="source-line-no">615</span><span id="line-615"> List&lt;Mutation&gt; mutations = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">616</span><span id="line-616"> long nonce = HConstants.NO_NONCE;</span>
<span class="source-line-no">617</span><span id="line-617"> for (ClientProtos.Action action : actions) {</span>
<span class="source-line-no">618</span><span id="line-618"> if (action.hasGet()) {</span>
<span class="source-line-no">619</span><span id="line-619"> throw new DoNotRetryIOException(</span>
<span class="source-line-no">620</span><span id="line-620"> "Atomic put and/or delete only, not a Get=" + action.getGet());</span>
<span class="source-line-no">621</span><span id="line-621"> }</span>
<span class="source-line-no">622</span><span id="line-622"> MutationProto mutation = action.getMutation();</span>
<span class="source-line-no">623</span><span id="line-623"> MutationType type = mutation.getMutateType();</span>
<span class="source-line-no">624</span><span id="line-624"> switch (type) {</span>
<span class="source-line-no">625</span><span id="line-625"> case PUT:</span>
<span class="source-line-no">626</span><span id="line-626"> Put put = ProtobufUtil.toPut(mutation, cellScanner);</span>
<span class="source-line-no">627</span><span id="line-627"> ++countOfCompleteMutation;</span>
<span class="source-line-no">628</span><span id="line-628"> checkCellSizeLimit(region, put);</span>
<span class="source-line-no">629</span><span id="line-629"> spaceQuotaEnforcement.getPolicyEnforcement(region).check(put);</span>
<span class="source-line-no">630</span><span id="line-630"> mutations.add(put);</span>
<span class="source-line-no">631</span><span id="line-631"> break;</span>
<span class="source-line-no">632</span><span id="line-632"> case DELETE:</span>
<span class="source-line-no">633</span><span id="line-633"> Delete del = ProtobufUtil.toDelete(mutation, cellScanner);</span>
<span class="source-line-no">634</span><span id="line-634"> ++countOfCompleteMutation;</span>
<span class="source-line-no">635</span><span id="line-635"> spaceQuotaEnforcement.getPolicyEnforcement(region).check(del);</span>
<span class="source-line-no">636</span><span id="line-636"> mutations.add(del);</span>
<span class="source-line-no">637</span><span id="line-637"> break;</span>
<span class="source-line-no">638</span><span id="line-638"> case INCREMENT:</span>
<span class="source-line-no">639</span><span id="line-639"> Increment increment = ProtobufUtil.toIncrement(mutation, cellScanner);</span>
<span class="source-line-no">640</span><span id="line-640"> nonce = mutation.hasNonce() ? mutation.getNonce() : HConstants.NO_NONCE;</span>
<span class="source-line-no">641</span><span id="line-641"> ++countOfCompleteMutation;</span>
<span class="source-line-no">642</span><span id="line-642"> checkCellSizeLimit(region, increment);</span>
<span class="source-line-no">643</span><span id="line-643"> spaceQuotaEnforcement.getPolicyEnforcement(region).check(increment);</span>
<span class="source-line-no">644</span><span id="line-644"> mutations.add(increment);</span>
<span class="source-line-no">645</span><span id="line-645"> break;</span>
<span class="source-line-no">646</span><span id="line-646"> case APPEND:</span>
<span class="source-line-no">647</span><span id="line-647"> Append append = ProtobufUtil.toAppend(mutation, cellScanner);</span>
<span class="source-line-no">648</span><span id="line-648"> nonce = mutation.hasNonce() ? mutation.getNonce() : HConstants.NO_NONCE;</span>
<span class="source-line-no">649</span><span id="line-649"> ++countOfCompleteMutation;</span>
<span class="source-line-no">650</span><span id="line-650"> checkCellSizeLimit(region, append);</span>
<span class="source-line-no">651</span><span id="line-651"> spaceQuotaEnforcement.getPolicyEnforcement(region).check(append);</span>
<span class="source-line-no">652</span><span id="line-652"> mutations.add(append);</span>
<span class="source-line-no">653</span><span id="line-653"> break;</span>
<span class="source-line-no">654</span><span id="line-654"> default:</span>
<span class="source-line-no">655</span><span id="line-655"> throw new DoNotRetryIOException("invalid mutation type : " + type);</span>
<span class="source-line-no">656</span><span id="line-656"> }</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"> if (mutations.size() == 0) {</span>
<span class="source-line-no">660</span><span id="line-660"> return new CheckAndMutateResult(true, null);</span>
<span class="source-line-no">661</span><span id="line-661"> } else {</span>
<span class="source-line-no">662</span><span id="line-662"> CheckAndMutate checkAndMutate = ProtobufUtil.toCheckAndMutate(condition, mutations);</span>
<span class="source-line-no">663</span><span id="line-663"> CheckAndMutateResult result = null;</span>
<span class="source-line-no">664</span><span id="line-664"> if (region.getCoprocessorHost() != null) {</span>
<span class="source-line-no">665</span><span id="line-665"> result = region.getCoprocessorHost().preCheckAndMutate(checkAndMutate);</span>
<span class="source-line-no">666</span><span id="line-666"> }</span>
<span class="source-line-no">667</span><span id="line-667"> if (result == null) {</span>
<span class="source-line-no">668</span><span id="line-668"> result = region.checkAndMutate(checkAndMutate, nonceGroup, nonce);</span>
<span class="source-line-no">669</span><span id="line-669"> if (region.getCoprocessorHost() != null) {</span>
<span class="source-line-no">670</span><span id="line-670"> result = region.getCoprocessorHost().postCheckAndMutate(checkAndMutate, result);</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"> return result;</span>
<span class="source-line-no">674</span><span id="line-674"> }</span>
<span class="source-line-no">675</span><span id="line-675"> } finally {</span>
<span class="source-line-no">676</span><span id="line-676"> // Currently, the checkAndMutate isn't supported by batch so it won't mess up the cell scanner</span>
<span class="source-line-no">677</span><span id="line-677"> // even if the malformed cells are not skipped.</span>
<span class="source-line-no">678</span><span id="line-678"> for (int i = countOfCompleteMutation; i &lt; actions.size(); ++i) {</span>
<span class="source-line-no">679</span><span id="line-679"> skipCellsForMutation(actions.get(i), cellScanner);</span>
<span class="source-line-no">680</span><span id="line-680"> }</span>
<span class="source-line-no">681</span><span id="line-681"> }</span>
<span class="source-line-no">682</span><span id="line-682"> }</span>
<span class="source-line-no">683</span><span id="line-683"></span>
<span class="source-line-no">684</span><span id="line-684"> /**</span>
<span class="source-line-no">685</span><span id="line-685"> * Execute an append mutation.</span>
<span class="source-line-no">686</span><span id="line-686"> * @return result to return to client if default operation should be bypassed as indicated by</span>
<span class="source-line-no">687</span><span id="line-687"> * RegionObserver, null otherwise</span>
<span class="source-line-no">688</span><span id="line-688"> */</span>
<span class="source-line-no">689</span><span id="line-689"> private Result append(final HRegion region, final OperationQuota quota,</span>
<span class="source-line-no">690</span><span id="line-690"> final MutationProto mutation, final CellScanner cellScanner, long nonceGroup,</span>
<span class="source-line-no">691</span><span id="line-691"> ActivePolicyEnforcement spaceQuota, RpcCallContext context) throws IOException {</span>
<span class="source-line-no">692</span><span id="line-692"> long before = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">693</span><span id="line-693"> Append append = ProtobufUtil.toAppend(mutation, cellScanner);</span>
<span class="source-line-no">694</span><span id="line-694"> checkCellSizeLimit(region, append);</span>
<span class="source-line-no">695</span><span id="line-695"> spaceQuota.getPolicyEnforcement(region).check(append);</span>
<span class="source-line-no">696</span><span id="line-696"> quota.addMutation(append);</span>
<span class="source-line-no">697</span><span id="line-697"> long blockBytesScannedBefore = context != null ? context.getBlockBytesScanned() : 0;</span>
<span class="source-line-no">698</span><span id="line-698"> long nonce = mutation.hasNonce() ? mutation.getNonce() : HConstants.NO_NONCE;</span>
<span class="source-line-no">699</span><span id="line-699"> Result r = region.append(append, nonceGroup, nonce);</span>
<span class="source-line-no">700</span><span id="line-700"> if (server.getMetrics() != null) {</span>
<span class="source-line-no">701</span><span id="line-701"> long blockBytesScanned =</span>
<span class="source-line-no">702</span><span id="line-702"> context != null ? context.getBlockBytesScanned() - blockBytesScannedBefore : 0;</span>
<span class="source-line-no">703</span><span id="line-703"> server.getMetrics().updateAppend(region, EnvironmentEdgeManager.currentTime() - before,</span>
<span class="source-line-no">704</span><span id="line-704"> blockBytesScanned);</span>
<span class="source-line-no">705</span><span id="line-705"> }</span>
<span class="source-line-no">706</span><span id="line-706"> return r == null ? Result.EMPTY_RESULT : r;</span>
<span class="source-line-no">707</span><span id="line-707"> }</span>
<span class="source-line-no">708</span><span id="line-708"></span>
<span class="source-line-no">709</span><span id="line-709"> /**</span>
<span class="source-line-no">710</span><span id="line-710"> * Execute an increment mutation.</span>
<span class="source-line-no">711</span><span id="line-711"> */</span>
<span class="source-line-no">712</span><span id="line-712"> private Result increment(final HRegion region, final OperationQuota quota,</span>
<span class="source-line-no">713</span><span id="line-713"> final MutationProto mutation, final CellScanner cells, long nonceGroup,</span>
<span class="source-line-no">714</span><span id="line-714"> ActivePolicyEnforcement spaceQuota, RpcCallContext context) throws IOException {</span>
<span class="source-line-no">715</span><span id="line-715"> long before = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">716</span><span id="line-716"> Increment increment = ProtobufUtil.toIncrement(mutation, cells);</span>
<span class="source-line-no">717</span><span id="line-717"> checkCellSizeLimit(region, increment);</span>
<span class="source-line-no">718</span><span id="line-718"> spaceQuota.getPolicyEnforcement(region).check(increment);</span>
<span class="source-line-no">719</span><span id="line-719"> quota.addMutation(increment);</span>
<span class="source-line-no">720</span><span id="line-720"> long blockBytesScannedBefore = context != null ? context.getBlockBytesScanned() : 0;</span>
<span class="source-line-no">721</span><span id="line-721"> long nonce = mutation.hasNonce() ? mutation.getNonce() : HConstants.NO_NONCE;</span>
<span class="source-line-no">722</span><span id="line-722"> Result r = region.increment(increment, nonceGroup, nonce);</span>
<span class="source-line-no">723</span><span id="line-723"> final MetricsRegionServer metricsRegionServer = server.getMetrics();</span>
<span class="source-line-no">724</span><span id="line-724"> if (metricsRegionServer != null) {</span>
<span class="source-line-no">725</span><span id="line-725"> long blockBytesScanned =</span>
<span class="source-line-no">726</span><span id="line-726"> context != null ? context.getBlockBytesScanned() - blockBytesScannedBefore : 0;</span>
<span class="source-line-no">727</span><span id="line-727"> metricsRegionServer.updateIncrement(region, EnvironmentEdgeManager.currentTime() - before,</span>
<span class="source-line-no">728</span><span id="line-728"> blockBytesScanned);</span>
<span class="source-line-no">729</span><span id="line-729"> }</span>
<span class="source-line-no">730</span><span id="line-730"> return r == null ? Result.EMPTY_RESULT : r;</span>
<span class="source-line-no">731</span><span id="line-731"> }</span>
<span class="source-line-no">732</span><span id="line-732"></span>
<span class="source-line-no">733</span><span id="line-733"> /**</span>
<span class="source-line-no">734</span><span id="line-734"> * Run through the regionMutation &lt;code&gt;rm&lt;/code&gt; and per Mutation, do the work, and then when</span>
<span class="source-line-no">735</span><span id="line-735"> * done, add an instance of a {@link ResultOrException} that corresponds to each Mutation.</span>
<span class="source-line-no">736</span><span id="line-736"> * @param cellsToReturn Could be null. May be allocated in this method. This is what this method</span>
<span class="source-line-no">737</span><span id="line-737"> * returns as a 'result'.</span>
<span class="source-line-no">738</span><span id="line-738"> * @param closeCallBack the callback to be used with multigets</span>
<span class="source-line-no">739</span><span id="line-739"> * @param context the current RpcCallContext</span>
<span class="source-line-no">740</span><span id="line-740"> * @return Return the &lt;code&gt;cellScanner&lt;/code&gt; passed</span>
<span class="source-line-no">741</span><span id="line-741"> */</span>
<span class="source-line-no">742</span><span id="line-742"> private List&lt;ExtendedCellScannable&gt; doNonAtomicRegionMutation(final HRegion region,</span>
<span class="source-line-no">743</span><span id="line-743"> final OperationQuota quota, final RegionAction actions, final CellScanner cellScanner,</span>
<span class="source-line-no">744</span><span id="line-744"> final RegionActionResult.Builder builder, List&lt;ExtendedCellScannable&gt; cellsToReturn,</span>
<span class="source-line-no">745</span><span id="line-745"> long nonceGroup, final RegionScannersCloseCallBack closeCallBack, RpcCallContext context,</span>
<span class="source-line-no">746</span><span id="line-746"> ActivePolicyEnforcement spaceQuotaEnforcement) {</span>
<span class="source-line-no">747</span><span id="line-747"> // Gather up CONTIGUOUS Puts and Deletes in this mutations List. Idea is that rather than do</span>
<span class="source-line-no">748</span><span id="line-748"> // one at a time, we instead pass them in batch. Be aware that the corresponding</span>
<span class="source-line-no">749</span><span id="line-749"> // ResultOrException instance that matches each Put or Delete is then added down in the</span>
<span class="source-line-no">750</span><span id="line-750"> // doNonAtomicBatchOp call. We should be staying aligned though the Put and Delete are</span>
<span class="source-line-no">751</span><span id="line-751"> // deferred/batched</span>
<span class="source-line-no">752</span><span id="line-752"> List&lt;ClientProtos.Action&gt; mutations = null;</span>
<span class="source-line-no">753</span><span id="line-753"> long maxQuotaResultSize = Math.min(maxScannerResultSize, quota.getMaxResultSize());</span>
<span class="source-line-no">754</span><span id="line-754"> IOException sizeIOE = null;</span>
<span class="source-line-no">755</span><span id="line-755"> ClientProtos.ResultOrException.Builder resultOrExceptionBuilder =</span>
<span class="source-line-no">756</span><span id="line-756"> ResultOrException.newBuilder();</span>
<span class="source-line-no">757</span><span id="line-757"> boolean hasResultOrException = false;</span>
<span class="source-line-no">758</span><span id="line-758"> for (ClientProtos.Action action : actions.getActionList()) {</span>
<span class="source-line-no">759</span><span id="line-759"> hasResultOrException = false;</span>
<span class="source-line-no">760</span><span id="line-760"> resultOrExceptionBuilder.clear();</span>
<span class="source-line-no">761</span><span id="line-761"> try {</span>
<span class="source-line-no">762</span><span id="line-762"> Result r = null;</span>
<span class="source-line-no">763</span><span id="line-763"> long blockBytesScannedBefore = context != null ? context.getBlockBytesScanned() : 0;</span>
<span class="source-line-no">764</span><span id="line-764"> if (</span>
<span class="source-line-no">765</span><span id="line-765"> context != null &amp;&amp; context.isRetryImmediatelySupported()</span>
<span class="source-line-no">766</span><span id="line-766"> &amp;&amp; (context.getResponseCellSize() &gt; maxQuotaResultSize</span>
<span class="source-line-no">767</span><span id="line-767"> || blockBytesScannedBefore + context.getResponseExceptionSize() &gt; maxQuotaResultSize)</span>
<span class="source-line-no">768</span><span id="line-768"> ) {</span>
<span class="source-line-no">769</span><span id="line-769"></span>
<span class="source-line-no">770</span><span id="line-770"> // We're storing the exception since the exception and reason string won't</span>
<span class="source-line-no">771</span><span id="line-771"> // change after the response size limit is reached.</span>
<span class="source-line-no">772</span><span id="line-772"> if (sizeIOE == null) {</span>
<span class="source-line-no">773</span><span id="line-773"> // We don't need the stack un-winding do don't throw the exception.</span>
<span class="source-line-no">774</span><span id="line-774"> // Throwing will kill the JVM's JIT.</span>
<span class="source-line-no">775</span><span id="line-775"> //</span>
<span class="source-line-no">776</span><span id="line-776"> // Instead just create the exception and then store it.</span>
<span class="source-line-no">777</span><span id="line-777"> sizeIOE = new MultiActionResultTooLarge("Max size exceeded" + " CellSize: "</span>
<span class="source-line-no">778</span><span id="line-778"> + context.getResponseCellSize() + " BlockSize: " + blockBytesScannedBefore);</span>
<span class="source-line-no">779</span><span id="line-779"></span>
<span class="source-line-no">780</span><span id="line-780"> // Only report the exception once since there's only one request that</span>
<span class="source-line-no">781</span><span id="line-781"> // caused the exception. Otherwise this number will dominate the exceptions count.</span>
<span class="source-line-no">782</span><span id="line-782"> rpcServer.getMetrics().exception(sizeIOE);</span>
<span class="source-line-no">783</span><span id="line-783"> }</span>
<span class="source-line-no">784</span><span id="line-784"></span>
<span class="source-line-no">785</span><span id="line-785"> // Now that there's an exception is known to be created</span>
<span class="source-line-no">786</span><span id="line-786"> // use it for the response.</span>
<span class="source-line-no">787</span><span id="line-787"> //</span>
<span class="source-line-no">788</span><span id="line-788"> // This will create a copy in the builder.</span>
<span class="source-line-no">789</span><span id="line-789"> NameBytesPair pair = ResponseConverter.buildException(sizeIOE);</span>
<span class="source-line-no">790</span><span id="line-790"> resultOrExceptionBuilder.setException(pair);</span>
<span class="source-line-no">791</span><span id="line-791"> context.incrementResponseExceptionSize(pair.getSerializedSize());</span>
<span class="source-line-no">792</span><span id="line-792"> resultOrExceptionBuilder.setIndex(action.getIndex());</span>
<span class="source-line-no">793</span><span id="line-793"> builder.addResultOrException(resultOrExceptionBuilder.build());</span>
<span class="source-line-no">794</span><span id="line-794"> skipCellsForMutation(action, cellScanner);</span>
<span class="source-line-no">795</span><span id="line-795"> continue;</span>
<span class="source-line-no">796</span><span id="line-796"> }</span>
<span class="source-line-no">797</span><span id="line-797"> if (action.hasGet()) {</span>
<span class="source-line-no">798</span><span id="line-798"> long before = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">799</span><span id="line-799"> ClientProtos.Get pbGet = action.getGet();</span>
<span class="source-line-no">800</span><span id="line-800"> // An asynchbase client, https://github.com/OpenTSDB/asynchbase, starts by trying to do</span>
<span class="source-line-no">801</span><span id="line-801"> // a get closest before. Throwing the UnknownProtocolException signals it that it needs</span>
<span class="source-line-no">802</span><span id="line-802"> // to switch and do hbase2 protocol (HBase servers do not tell clients what versions</span>
<span class="source-line-no">803</span><span id="line-803"> // they are; its a problem for non-native clients like asynchbase. HBASE-20225.</span>
<span class="source-line-no">804</span><span id="line-804"> if (pbGet.hasClosestRowBefore() &amp;&amp; pbGet.getClosestRowBefore()) {</span>
<span class="source-line-no">805</span><span id="line-805"> throw new UnknownProtocolException("Is this a pre-hbase-1.0.0 or asynchbase client? "</span>
<span class="source-line-no">806</span><span id="line-806"> + "Client is invoking getClosestRowBefore removed in hbase-2.0.0 replaced by "</span>
<span class="source-line-no">807</span><span id="line-807"> + "reverse Scan.");</span>
<span class="source-line-no">808</span><span id="line-808"> }</span>
<span class="source-line-no">809</span><span id="line-809"> try {</span>
<span class="source-line-no">810</span><span id="line-810"> Get get = ProtobufUtil.toGet(pbGet);</span>
<span class="source-line-no">811</span><span id="line-811"> if (context != null) {</span>
<span class="source-line-no">812</span><span id="line-812"> r = get(get, (region), closeCallBack, context);</span>
<span class="source-line-no">813</span><span id="line-813"> } else {</span>
<span class="source-line-no">814</span><span id="line-814"> r = region.get(get);</span>
<span class="source-line-no">815</span><span id="line-815"> }</span>
<span class="source-line-no">816</span><span id="line-816"> } finally {</span>
<span class="source-line-no">817</span><span id="line-817"> final MetricsRegionServer metricsRegionServer = server.getMetrics();</span>
<span class="source-line-no">818</span><span id="line-818"> if (metricsRegionServer != null) {</span>
<span class="source-line-no">819</span><span id="line-819"> long blockBytesScanned =</span>
<span class="source-line-no">820</span><span id="line-820"> context != null ? context.getBlockBytesScanned() - blockBytesScannedBefore : 0;</span>
<span class="source-line-no">821</span><span id="line-821"> metricsRegionServer.updateGet(region, EnvironmentEdgeManager.currentTime() - before,</span>
<span class="source-line-no">822</span><span id="line-822"> blockBytesScanned);</span>
<span class="source-line-no">823</span><span id="line-823"> }</span>
<span class="source-line-no">824</span><span id="line-824"> }</span>
<span class="source-line-no">825</span><span id="line-825"> } else if (action.hasServiceCall()) {</span>
<span class="source-line-no">826</span><span id="line-826"> hasResultOrException = true;</span>
<span class="source-line-no">827</span><span id="line-827"> Message result = execServiceOnRegion(region, action.getServiceCall());</span>
<span class="source-line-no">828</span><span id="line-828"> ClientProtos.CoprocessorServiceResult.Builder serviceResultBuilder =</span>
<span class="source-line-no">829</span><span id="line-829"> ClientProtos.CoprocessorServiceResult.newBuilder();</span>
<span class="source-line-no">830</span><span id="line-830"> resultOrExceptionBuilder.setServiceResult(serviceResultBuilder</span>
<span class="source-line-no">831</span><span id="line-831"> .setValue(serviceResultBuilder.getValueBuilder().setName(result.getClass().getName())</span>
<span class="source-line-no">832</span><span id="line-832"> // TODO: Copy!!!</span>
<span class="source-line-no">833</span><span id="line-833"> .setValue(UnsafeByteOperations.unsafeWrap(result.toByteArray()))));</span>
<span class="source-line-no">834</span><span id="line-834"> } else if (action.hasMutation()) {</span>
<span class="source-line-no">835</span><span id="line-835"> MutationType type = action.getMutation().getMutateType();</span>
<span class="source-line-no">836</span><span id="line-836"> if (</span>
<span class="source-line-no">837</span><span id="line-837"> type != MutationType.PUT &amp;&amp; type != MutationType.DELETE &amp;&amp; mutations != null</span>
<span class="source-line-no">838</span><span id="line-838"> &amp;&amp; !mutations.isEmpty()</span>
<span class="source-line-no">839</span><span id="line-839"> ) {</span>
<span class="source-line-no">840</span><span id="line-840"> // Flush out any Puts or Deletes already collected.</span>
<span class="source-line-no">841</span><span id="line-841"> doNonAtomicBatchOp(builder, region, quota, mutations, cellScanner,</span>
<span class="source-line-no">842</span><span id="line-842"> spaceQuotaEnforcement);</span>
<span class="source-line-no">843</span><span id="line-843"> mutations.clear();</span>
<span class="source-line-no">844</span><span id="line-844"> }</span>
<span class="source-line-no">845</span><span id="line-845"> switch (type) {</span>
<span class="source-line-no">846</span><span id="line-846"> case APPEND:</span>
<span class="source-line-no">847</span><span id="line-847"> r = append(region, quota, action.getMutation(), cellScanner, nonceGroup,</span>
<span class="source-line-no">848</span><span id="line-848"> spaceQuotaEnforcement, context);</span>
<span class="source-line-no">849</span><span id="line-849"> break;</span>
<span class="source-line-no">850</span><span id="line-850"> case INCREMENT:</span>
<span class="source-line-no">851</span><span id="line-851"> r = increment(region, quota, action.getMutation(), cellScanner, nonceGroup,</span>
<span class="source-line-no">852</span><span id="line-852"> spaceQuotaEnforcement, context);</span>
<span class="source-line-no">853</span><span id="line-853"> break;</span>
<span class="source-line-no">854</span><span id="line-854"> case PUT:</span>
<span class="source-line-no">855</span><span id="line-855"> case DELETE:</span>
<span class="source-line-no">856</span><span id="line-856"> // Collect the individual mutations and apply in a batch</span>
<span class="source-line-no">857</span><span id="line-857"> if (mutations == null) {</span>
<span class="source-line-no">858</span><span id="line-858"> mutations = new ArrayList&lt;&gt;(actions.getActionCount());</span>
<span class="source-line-no">859</span><span id="line-859"> }</span>
<span class="source-line-no">860</span><span id="line-860"> mutations.add(action);</span>
<span class="source-line-no">861</span><span id="line-861"> break;</span>
<span class="source-line-no">862</span><span id="line-862"> default:</span>
<span class="source-line-no">863</span><span id="line-863"> throw new DoNotRetryIOException("Unsupported mutate type: " + type.name());</span>
<span class="source-line-no">864</span><span id="line-864"> }</span>
<span class="source-line-no">865</span><span id="line-865"> } else {</span>
<span class="source-line-no">866</span><span id="line-866"> throw new HBaseIOException("Unexpected Action type");</span>
<span class="source-line-no">867</span><span id="line-867"> }</span>
<span class="source-line-no">868</span><span id="line-868"> if (r != null) {</span>
<span class="source-line-no">869</span><span id="line-869"> ClientProtos.Result pbResult = null;</span>
<span class="source-line-no">870</span><span id="line-870"> if (isClientCellBlockSupport(context)) {</span>
<span class="source-line-no">871</span><span id="line-871"> pbResult = ProtobufUtil.toResultNoData(r);</span>
<span class="source-line-no">872</span><span id="line-872"> // Hard to guess the size here. Just make a rough guess.</span>
<span class="source-line-no">873</span><span id="line-873"> if (cellsToReturn == null) {</span>
<span class="source-line-no">874</span><span id="line-874"> cellsToReturn = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">875</span><span id="line-875"> }</span>
<span class="source-line-no">876</span><span id="line-876"> cellsToReturn.add(r);</span>
<span class="source-line-no">877</span><span id="line-877"> } else {</span>
<span class="source-line-no">878</span><span id="line-878"> pbResult = ProtobufUtil.toResult(r);</span>
<span class="source-line-no">879</span><span id="line-879"> }</span>
<span class="source-line-no">880</span><span id="line-880"> addSize(context, r);</span>
<span class="source-line-no">881</span><span id="line-881"> hasResultOrException = true;</span>
<span class="source-line-no">882</span><span id="line-882"> resultOrExceptionBuilder.setResult(pbResult);</span>
<span class="source-line-no">883</span><span id="line-883"> }</span>
<span class="source-line-no">884</span><span id="line-884"> // Could get to here and there was no result and no exception. Presumes we added</span>
<span class="source-line-no">885</span><span id="line-885"> // a Put or Delete to the collecting Mutations List for adding later. In this</span>
<span class="source-line-no">886</span><span id="line-886"> // case the corresponding ResultOrException instance for the Put or Delete will be added</span>
<span class="source-line-no">887</span><span id="line-887"> // down in the doNonAtomicBatchOp method call rather than up here.</span>
<span class="source-line-no">888</span><span id="line-888"> } catch (IOException ie) {</span>
<span class="source-line-no">889</span><span id="line-889"> rpcServer.getMetrics().exception(ie);</span>
<span class="source-line-no">890</span><span id="line-890"> hasResultOrException = true;</span>
<span class="source-line-no">891</span><span id="line-891"> NameBytesPair pair = ResponseConverter.buildException(ie);</span>
<span class="source-line-no">892</span><span id="line-892"> resultOrExceptionBuilder.setException(pair);</span>
<span class="source-line-no">893</span><span id="line-893"> context.incrementResponseExceptionSize(pair.getSerializedSize());</span>
<span class="source-line-no">894</span><span id="line-894"> }</span>
<span class="source-line-no">895</span><span id="line-895"> if (hasResultOrException) {</span>
<span class="source-line-no">896</span><span id="line-896"> // Propagate index.</span>
<span class="source-line-no">897</span><span id="line-897"> resultOrExceptionBuilder.setIndex(action.getIndex());</span>
<span class="source-line-no">898</span><span id="line-898"> builder.addResultOrException(resultOrExceptionBuilder.build());</span>
<span class="source-line-no">899</span><span id="line-899"> }</span>
<span class="source-line-no">900</span><span id="line-900"> }</span>
<span class="source-line-no">901</span><span id="line-901"> // Finish up any outstanding mutations</span>
<span class="source-line-no">902</span><span id="line-902"> if (!CollectionUtils.isEmpty(mutations)) {</span>
<span class="source-line-no">903</span><span id="line-903"> doNonAtomicBatchOp(builder, region, quota, mutations, cellScanner, spaceQuotaEnforcement);</span>
<span class="source-line-no">904</span><span id="line-904"> }</span>
<span class="source-line-no">905</span><span id="line-905"> return cellsToReturn;</span>
<span class="source-line-no">906</span><span id="line-906"> }</span>
<span class="source-line-no">907</span><span id="line-907"></span>
<span class="source-line-no">908</span><span id="line-908"> private void checkCellSizeLimit(final HRegion r, final Mutation m) throws IOException {</span>
<span class="source-line-no">909</span><span id="line-909"> if (r.maxCellSize &gt; 0) {</span>
<span class="source-line-no">910</span><span id="line-910"> CellScanner cells = m.cellScanner();</span>
<span class="source-line-no">911</span><span id="line-911"> while (cells.advance()) {</span>
<span class="source-line-no">912</span><span id="line-912"> int size = PrivateCellUtil.estimatedSerializedSizeOf(cells.current());</span>
<span class="source-line-no">913</span><span id="line-913"> if (size &gt; r.maxCellSize) {</span>
<span class="source-line-no">914</span><span id="line-914"> String msg = "Cell[" + cells.current() + "] with size " + size + " exceeds limit of "</span>
<span class="source-line-no">915</span><span id="line-915"> + r.maxCellSize + " bytes";</span>
<span class="source-line-no">916</span><span id="line-916"> LOG.debug(msg);</span>
<span class="source-line-no">917</span><span id="line-917"> throw new DoNotRetryIOException(msg);</span>
<span class="source-line-no">918</span><span id="line-918"> }</span>
<span class="source-line-no">919</span><span id="line-919"> }</span>
<span class="source-line-no">920</span><span id="line-920"> }</span>
<span class="source-line-no">921</span><span id="line-921"> }</span>
<span class="source-line-no">922</span><span id="line-922"></span>
<span class="source-line-no">923</span><span id="line-923"> private void doAtomicBatchOp(final RegionActionResult.Builder builder, final HRegion region,</span>
<span class="source-line-no">924</span><span id="line-924"> final OperationQuota quota, final List&lt;ClientProtos.Action&gt; mutations, final CellScanner cells,</span>
<span class="source-line-no">925</span><span id="line-925"> long nonceGroup, ActivePolicyEnforcement spaceQuotaEnforcement) throws IOException {</span>
<span class="source-line-no">926</span><span id="line-926"> // Just throw the exception. The exception will be caught and then added to region-level</span>
<span class="source-line-no">927</span><span id="line-927"> // exception for RegionAction. Leaving the null to action result is ok since the null</span>
<span class="source-line-no">928</span><span id="line-928"> // result is viewed as failure by hbase client. And the region-lever exception will be used</span>
<span class="source-line-no">929</span><span id="line-929"> // to replaced the null result. see AsyncRequestFutureImpl#receiveMultiAction and</span>
<span class="source-line-no">930</span><span id="line-930"> // AsyncBatchRpcRetryingCaller#onComplete for more details.</span>
<span class="source-line-no">931</span><span id="line-931"> doBatchOp(builder, region, quota, mutations, cells, nonceGroup, spaceQuotaEnforcement, true);</span>
<span class="source-line-no">932</span><span id="line-932"> }</span>
<span class="source-line-no">933</span><span id="line-933"></span>
<span class="source-line-no">934</span><span id="line-934"> private void doNonAtomicBatchOp(final RegionActionResult.Builder builder, final HRegion region,</span>
<span class="source-line-no">935</span><span id="line-935"> final OperationQuota quota, final List&lt;ClientProtos.Action&gt; mutations, final CellScanner cells,</span>
<span class="source-line-no">936</span><span id="line-936"> ActivePolicyEnforcement spaceQuotaEnforcement) {</span>
<span class="source-line-no">937</span><span id="line-937"> try {</span>
<span class="source-line-no">938</span><span id="line-938"> doBatchOp(builder, region, quota, mutations, cells, HConstants.NO_NONCE,</span>
<span class="source-line-no">939</span><span id="line-939"> spaceQuotaEnforcement, false);</span>
<span class="source-line-no">940</span><span id="line-940"> } catch (IOException e) {</span>
<span class="source-line-no">941</span><span id="line-941"> // Set the exception for each action. The mutations in same RegionAction are group to</span>
<span class="source-line-no">942</span><span id="line-942"> // different batch and then be processed individually. Hence, we don't set the region-level</span>
<span class="source-line-no">943</span><span id="line-943"> // exception here for whole RegionAction.</span>
<span class="source-line-no">944</span><span id="line-944"> for (Action mutation : mutations) {</span>
<span class="source-line-no">945</span><span id="line-945"> builder.addResultOrException(getResultOrException(e, mutation.getIndex()));</span>
<span class="source-line-no">946</span><span id="line-946"> }</span>
<span class="source-line-no">947</span><span id="line-947"> }</span>
<span class="source-line-no">948</span><span id="line-948"> }</span>
<span class="source-line-no">949</span><span id="line-949"></span>
<span class="source-line-no">950</span><span id="line-950"> /**</span>
<span class="source-line-no">951</span><span id="line-951"> * Execute a list of mutations.</span>
<span class="source-line-no">952</span><span id="line-952"> */</span>
<span class="source-line-no">953</span><span id="line-953"> private void doBatchOp(final RegionActionResult.Builder builder, final HRegion region,</span>
<span class="source-line-no">954</span><span id="line-954"> final OperationQuota quota, final List&lt;ClientProtos.Action&gt; mutations, final CellScanner cells,</span>
<span class="source-line-no">955</span><span id="line-955"> long nonceGroup, ActivePolicyEnforcement spaceQuotaEnforcement, boolean atomic)</span>
<span class="source-line-no">956</span><span id="line-956"> throws IOException {</span>
<span class="source-line-no">957</span><span id="line-957"> Mutation[] mArray = new Mutation[mutations.size()];</span>
<span class="source-line-no">958</span><span id="line-958"> long before = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">959</span><span id="line-959"> boolean batchContainsPuts = false, batchContainsDelete = false;</span>
<span class="source-line-no">960</span><span id="line-960"> try {</span>
<span class="source-line-no">961</span><span id="line-961"> /**</span>
<span class="source-line-no">962</span><span id="line-962"> * HBASE-17924 mutationActionMap is a map to map the relation between mutations and actions</span>
<span class="source-line-no">963</span><span id="line-963"> * since mutation array may have been reoredered.In order to return the right result or</span>
<span class="source-line-no">964</span><span id="line-964"> * exception to the corresponding actions, We need to know which action is the mutation belong</span>
<span class="source-line-no">965</span><span id="line-965"> * to. We can't sort ClientProtos.Action array, since they are bonded to cellscanners.</span>
<span class="source-line-no">966</span><span id="line-966"> */</span>
<span class="source-line-no">967</span><span id="line-967"> Map&lt;Mutation, ClientProtos.Action&gt; mutationActionMap = new HashMap&lt;&gt;();</span>
<span class="source-line-no">968</span><span id="line-968"> int i = 0;</span>
<span class="source-line-no">969</span><span id="line-969"> long nonce = HConstants.NO_NONCE;</span>
<span class="source-line-no">970</span><span id="line-970"> for (ClientProtos.Action action : mutations) {</span>
<span class="source-line-no">971</span><span id="line-971"> if (action.hasGet()) {</span>
<span class="source-line-no">972</span><span id="line-972"> throw new DoNotRetryIOException(</span>
<span class="source-line-no">973</span><span id="line-973"> "Atomic put and/or delete only, not a Get=" + action.getGet());</span>
<span class="source-line-no">974</span><span id="line-974"> }</span>
<span class="source-line-no">975</span><span id="line-975"> MutationProto m = action.getMutation();</span>
<span class="source-line-no">976</span><span id="line-976"> Mutation mutation;</span>
<span class="source-line-no">977</span><span id="line-977"> switch (m.getMutateType()) {</span>
<span class="source-line-no">978</span><span id="line-978"> case PUT:</span>
<span class="source-line-no">979</span><span id="line-979"> mutation = ProtobufUtil.toPut(m, cells);</span>
<span class="source-line-no">980</span><span id="line-980"> batchContainsPuts = true;</span>
<span class="source-line-no">981</span><span id="line-981"> break;</span>
<span class="source-line-no">982</span><span id="line-982"></span>
<span class="source-line-no">983</span><span id="line-983"> case DELETE:</span>
<span class="source-line-no">984</span><span id="line-984"> mutation = ProtobufUtil.toDelete(m, cells);</span>
<span class="source-line-no">985</span><span id="line-985"> batchContainsDelete = true;</span>
<span class="source-line-no">986</span><span id="line-986"> break;</span>
<span class="source-line-no">987</span><span id="line-987"></span>
<span class="source-line-no">988</span><span id="line-988"> case INCREMENT:</span>
<span class="source-line-no">989</span><span id="line-989"> mutation = ProtobufUtil.toIncrement(m, cells);</span>
<span class="source-line-no">990</span><span id="line-990"> nonce = m.hasNonce() ? m.getNonce() : HConstants.NO_NONCE;</span>
<span class="source-line-no">991</span><span id="line-991"> break;</span>
<span class="source-line-no">992</span><span id="line-992"></span>
<span class="source-line-no">993</span><span id="line-993"> case APPEND:</span>
<span class="source-line-no">994</span><span id="line-994"> mutation = ProtobufUtil.toAppend(m, cells);</span>
<span class="source-line-no">995</span><span id="line-995"> nonce = m.hasNonce() ? m.getNonce() : HConstants.NO_NONCE;</span>
<span class="source-line-no">996</span><span id="line-996"> break;</span>
<span class="source-line-no">997</span><span id="line-997"></span>
<span class="source-line-no">998</span><span id="line-998"> default:</span>
<span class="source-line-no">999</span><span id="line-999"> throw new DoNotRetryIOException("Invalid mutation type : " + m.getMutateType());</span>
<span class="source-line-no">1000</span><span id="line-1000"> }</span>
<span class="source-line-no">1001</span><span id="line-1001"> mutationActionMap.put(mutation, action);</span>
<span class="source-line-no">1002</span><span id="line-1002"> mArray[i++] = mutation;</span>
<span class="source-line-no">1003</span><span id="line-1003"> checkCellSizeLimit(region, mutation);</span>
<span class="source-line-no">1004</span><span id="line-1004"> // Check if a space quota disallows this mutation</span>
<span class="source-line-no">1005</span><span id="line-1005"> spaceQuotaEnforcement.getPolicyEnforcement(region).check(mutation);</span>
<span class="source-line-no">1006</span><span id="line-1006"> quota.addMutation(mutation);</span>
<span class="source-line-no">1007</span><span id="line-1007"> }</span>
<span class="source-line-no">1008</span><span id="line-1008"></span>
<span class="source-line-no">1009</span><span id="line-1009"> if (!region.getRegionInfo().isMetaRegion()) {</span>
<span class="source-line-no">1010</span><span id="line-1010"> server.getMemStoreFlusher().reclaimMemStoreMemory();</span>
<span class="source-line-no">1011</span><span id="line-1011"> }</span>
<span class="source-line-no">1012</span><span id="line-1012"></span>
<span class="source-line-no">1013</span><span id="line-1013"> // HBASE-17924</span>
<span class="source-line-no">1014</span><span id="line-1014"> // Sort to improve lock efficiency for non-atomic batch of operations. If atomic</span>
<span class="source-line-no">1015</span><span id="line-1015"> // order is preserved as its expected from the client</span>
<span class="source-line-no">1016</span><span id="line-1016"> if (!atomic) {</span>
<span class="source-line-no">1017</span><span id="line-1017"> Arrays.sort(mArray, (v1, v2) -&gt; Row.COMPARATOR.compare(v1, v2));</span>
<span class="source-line-no">1018</span><span id="line-1018"> }</span>
<span class="source-line-no">1019</span><span id="line-1019"></span>
<span class="source-line-no">1020</span><span id="line-1020"> OperationStatus[] codes = region.batchMutate(mArray, atomic, nonceGroup, nonce);</span>
<span class="source-line-no">1021</span><span id="line-1021"></span>
<span class="source-line-no">1022</span><span id="line-1022"> // When atomic is true, it indicates that the mutateRow API or the batch API with</span>
<span class="source-line-no">1023</span><span id="line-1023"> // RowMutations is called. In this case, we need to merge the results of the</span>
<span class="source-line-no">1024</span><span id="line-1024"> // Increment/Append operations if the mutations include those operations, and set the merged</span>
<span class="source-line-no">1025</span><span id="line-1025"> // result to the first element of the ResultOrException list</span>
<span class="source-line-no">1026</span><span id="line-1026"> if (atomic) {</span>
<span class="source-line-no">1027</span><span id="line-1027"> List&lt;ResultOrException&gt; resultOrExceptions = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">1028</span><span id="line-1028"> List&lt;Result&gt; results = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">1029</span><span id="line-1029"> for (i = 0; i &lt; codes.length; i++) {</span>
<span class="source-line-no">1030</span><span id="line-1030"> if (codes[i].getResult() != null) {</span>
<span class="source-line-no">1031</span><span id="line-1031"> results.add(codes[i].getResult());</span>
<span class="source-line-no">1032</span><span id="line-1032"> }</span>
<span class="source-line-no">1033</span><span id="line-1033"> if (i != 0) {</span>
<span class="source-line-no">1034</span><span id="line-1034"> resultOrExceptions</span>
<span class="source-line-no">1035</span><span id="line-1035"> .add(getResultOrException(ClientProtos.Result.getDefaultInstance(), i));</span>
<span class="source-line-no">1036</span><span id="line-1036"> }</span>
<span class="source-line-no">1037</span><span id="line-1037"> }</span>
<span class="source-line-no">1038</span><span id="line-1038"></span>
<span class="source-line-no">1039</span><span id="line-1039"> if (results.isEmpty()) {</span>
<span class="source-line-no">1040</span><span id="line-1040"> builder.addResultOrException(</span>
<span class="source-line-no">1041</span><span id="line-1041"> getResultOrException(ClientProtos.Result.getDefaultInstance(), 0));</span>
<span class="source-line-no">1042</span><span id="line-1042"> } else {</span>
<span class="source-line-no">1043</span><span id="line-1043"> // Merge the results of the Increment/Append operations</span>
<span class="source-line-no">1044</span><span id="line-1044"> List&lt;Cell&gt; cellList = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">1045</span><span id="line-1045"> for (Result result : results) {</span>
<span class="source-line-no">1046</span><span id="line-1046"> if (result.rawCells() != null) {</span>
<span class="source-line-no">1047</span><span id="line-1047"> cellList.addAll(Arrays.asList(result.rawCells()));</span>
<span class="source-line-no">1048</span><span id="line-1048"> }</span>
<span class="source-line-no">1049</span><span id="line-1049"> }</span>
<span class="source-line-no">1050</span><span id="line-1050"> Result result = Result.create(cellList);</span>
<span class="source-line-no">1051</span><span id="line-1051"></span>
<span class="source-line-no">1052</span><span id="line-1052"> // Set the merged result of the Increment/Append operations to the first element of the</span>
<span class="source-line-no">1053</span><span id="line-1053"> // ResultOrException list</span>
<span class="source-line-no">1054</span><span id="line-1054"> builder.addResultOrException(getResultOrException(ProtobufUtil.toResult(result), 0));</span>
<span class="source-line-no">1055</span><span id="line-1055"> }</span>
<span class="source-line-no">1056</span><span id="line-1056"></span>
<span class="source-line-no">1057</span><span id="line-1057"> builder.addAllResultOrException(resultOrExceptions);</span>
<span class="source-line-no">1058</span><span id="line-1058"> return;</span>
<span class="source-line-no">1059</span><span id="line-1059"> }</span>
<span class="source-line-no">1060</span><span id="line-1060"></span>
<span class="source-line-no">1061</span><span id="line-1061"> for (i = 0; i &lt; codes.length; i++) {</span>
<span class="source-line-no">1062</span><span id="line-1062"> Mutation currentMutation = mArray[i];</span>
<span class="source-line-no">1063</span><span id="line-1063"> ClientProtos.Action currentAction = mutationActionMap.get(currentMutation);</span>
<span class="source-line-no">1064</span><span id="line-1064"> int index = currentAction.hasIndex() ? currentAction.getIndex() : i;</span>
<span class="source-line-no">1065</span><span id="line-1065"> Exception e;</span>
<span class="source-line-no">1066</span><span id="line-1066"> switch (codes[i].getOperationStatusCode()) {</span>
<span class="source-line-no">1067</span><span id="line-1067"> case BAD_FAMILY:</span>
<span class="source-line-no">1068</span><span id="line-1068"> e = new NoSuchColumnFamilyException(codes[i].getExceptionMsg());</span>
<span class="source-line-no">1069</span><span id="line-1069"> builder.addResultOrException(getResultOrException(e, index));</span>
<span class="source-line-no">1070</span><span id="line-1070"> break;</span>
<span class="source-line-no">1071</span><span id="line-1071"></span>
<span class="source-line-no">1072</span><span id="line-1072"> case SANITY_CHECK_FAILURE:</span>
<span class="source-line-no">1073</span><span id="line-1073"> e = new FailedSanityCheckException(codes[i].getExceptionMsg());</span>
<span class="source-line-no">1074</span><span id="line-1074"> builder.addResultOrException(getResultOrException(e, index));</span>
<span class="source-line-no">1075</span><span id="line-1075"> break;</span>
<span class="source-line-no">1076</span><span id="line-1076"></span>
<span class="source-line-no">1077</span><span id="line-1077"> default:</span>
<span class="source-line-no">1078</span><span id="line-1078"> e = new DoNotRetryIOException(codes[i].getExceptionMsg());</span>
<span class="source-line-no">1079</span><span id="line-1079"> builder.addResultOrException(getResultOrException(e, index));</span>
<span class="source-line-no">1080</span><span id="line-1080"> break;</span>
<span class="source-line-no">1081</span><span id="line-1081"></span>
<span class="source-line-no">1082</span><span id="line-1082"> case SUCCESS:</span>
<span class="source-line-no">1083</span><span id="line-1083"> ClientProtos.Result result = codes[i].getResult() == null</span>
<span class="source-line-no">1084</span><span id="line-1084"> ? ClientProtos.Result.getDefaultInstance()</span>
<span class="source-line-no">1085</span><span id="line-1085"> : ProtobufUtil.toResult(codes[i].getResult());</span>
<span class="source-line-no">1086</span><span id="line-1086"> builder.addResultOrException(getResultOrException(result, index));</span>
<span class="source-line-no">1087</span><span id="line-1087"> break;</span>
<span class="source-line-no">1088</span><span id="line-1088"></span>
<span class="source-line-no">1089</span><span id="line-1089"> case STORE_TOO_BUSY:</span>
<span class="source-line-no">1090</span><span id="line-1090"> e = new RegionTooBusyException(codes[i].getExceptionMsg());</span>
<span class="source-line-no">1091</span><span id="line-1091"> builder.addResultOrException(getResultOrException(e, index));</span>
<span class="source-line-no">1092</span><span id="line-1092"> break;</span>
<span class="source-line-no">1093</span><span id="line-1093"> }</span>
<span class="source-line-no">1094</span><span id="line-1094"> }</span>
<span class="source-line-no">1095</span><span id="line-1095"> } finally {</span>
<span class="source-line-no">1096</span><span id="line-1096"> int processedMutationIndex = 0;</span>
<span class="source-line-no">1097</span><span id="line-1097"> for (Action mutation : mutations) {</span>
<span class="source-line-no">1098</span><span id="line-1098"> // The non-null mArray[i] means the cell scanner has been read.</span>
<span class="source-line-no">1099</span><span id="line-1099"> if (mArray[processedMutationIndex++] == null) {</span>
<span class="source-line-no">1100</span><span id="line-1100"> skipCellsForMutation(mutation, cells);</span>
<span class="source-line-no">1101</span><span id="line-1101"> }</span>
<span class="source-line-no">1102</span><span id="line-1102"> }</span>
<span class="source-line-no">1103</span><span id="line-1103"> updateMutationMetrics(region, before, batchContainsPuts, batchContainsDelete);</span>
<span class="source-line-no">1104</span><span id="line-1104"> }</span>
<span class="source-line-no">1105</span><span id="line-1105"> }</span>
<span class="source-line-no">1106</span><span id="line-1106"></span>
<span class="source-line-no">1107</span><span id="line-1107"> private void updateMutationMetrics(HRegion region, long starttime, boolean batchContainsPuts,</span>
<span class="source-line-no">1108</span><span id="line-1108"> boolean batchContainsDelete) {</span>
<span class="source-line-no">1109</span><span id="line-1109"> final MetricsRegionServer metricsRegionServer = server.getMetrics();</span>
<span class="source-line-no">1110</span><span id="line-1110"> if (metricsRegionServer != null) {</span>
<span class="source-line-no">1111</span><span id="line-1111"> long after = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">1112</span><span id="line-1112"> if (batchContainsPuts) {</span>
<span class="source-line-no">1113</span><span id="line-1113"> metricsRegionServer.updatePutBatch(region, after - starttime);</span>
<span class="source-line-no">1114</span><span id="line-1114"> }</span>
<span class="source-line-no">1115</span><span id="line-1115"> if (batchContainsDelete) {</span>
<span class="source-line-no">1116</span><span id="line-1116"> metricsRegionServer.updateDeleteBatch(region, after - starttime);</span>
<span class="source-line-no">1117</span><span id="line-1117"> }</span>
<span class="source-line-no">1118</span><span id="line-1118"> }</span>
<span class="source-line-no">1119</span><span id="line-1119"> }</span>
<span class="source-line-no">1120</span><span id="line-1120"></span>
<span class="source-line-no">1121</span><span id="line-1121"> /**</span>
<span class="source-line-no">1122</span><span id="line-1122"> * Execute a list of Put/Delete mutations. The function returns OperationStatus instead of</span>
<span class="source-line-no">1123</span><span id="line-1123"> * constructing MultiResponse to save a possible loop if caller doesn't need MultiResponse.</span>
<span class="source-line-no">1124</span><span id="line-1124"> * @return an array of OperationStatus which internally contains the OperationStatusCode and the</span>
<span class="source-line-no">1125</span><span id="line-1125"> * exceptionMessage if any</span>
<span class="source-line-no">1126</span><span id="line-1126"> * @deprecated Since 3.0.0, will be removed in 4.0.0. We do not use this method for replaying</span>
<span class="source-line-no">1127</span><span id="line-1127"> * edits for secondary replicas any more, see</span>
<span class="source-line-no">1128</span><span id="line-1128"> * {@link #replicateToReplica(RpcController, ReplicateWALEntryRequest)}.</span>
<span class="source-line-no">1129</span><span id="line-1129"> */</span>
<span class="source-line-no">1130</span><span id="line-1130"> @Deprecated</span>
<span class="source-line-no">1131</span><span id="line-1131"> private OperationStatus[] doReplayBatchOp(final HRegion region,</span>
<span class="source-line-no">1132</span><span id="line-1132"> final List&lt;MutationReplay&gt; mutations, long replaySeqId) throws IOException {</span>
<span class="source-line-no">1133</span><span id="line-1133"> long before = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">1134</span><span id="line-1134"> boolean batchContainsPuts = false, batchContainsDelete = false;</span>
<span class="source-line-no">1135</span><span id="line-1135"> try {</span>
<span class="source-line-no">1136</span><span id="line-1136"> for (Iterator&lt;MutationReplay&gt; it = mutations.iterator(); it.hasNext();) {</span>
<span class="source-line-no">1137</span><span id="line-1137"> MutationReplay m = it.next();</span>
<span class="source-line-no">1138</span><span id="line-1138"></span>
<span class="source-line-no">1139</span><span id="line-1139"> if (m.getType() == MutationType.PUT) {</span>
<span class="source-line-no">1140</span><span id="line-1140"> batchContainsPuts = true;</span>
<span class="source-line-no">1141</span><span id="line-1141"> } else {</span>
<span class="source-line-no">1142</span><span id="line-1142"> batchContainsDelete = true;</span>
<span class="source-line-no">1143</span><span id="line-1143"> }</span>
<span class="source-line-no">1144</span><span id="line-1144"></span>
<span class="source-line-no">1145</span><span id="line-1145"> NavigableMap&lt;byte[], List&lt;Cell&gt;&gt; map = m.mutation.getFamilyCellMap();</span>
<span class="source-line-no">1146</span><span id="line-1146"> List&lt;Cell&gt; metaCells = map.get(WALEdit.METAFAMILY);</span>
<span class="source-line-no">1147</span><span id="line-1147"> if (metaCells != null &amp;&amp; !metaCells.isEmpty()) {</span>
<span class="source-line-no">1148</span><span id="line-1148"> for (Cell metaCell : metaCells) {</span>
<span class="source-line-no">1149</span><span id="line-1149"> CompactionDescriptor compactionDesc = WALEdit.getCompaction(metaCell);</span>
<span class="source-line-no">1150</span><span id="line-1150"> boolean isDefaultReplica = RegionReplicaUtil.isDefaultReplica(region.getRegionInfo());</span>
<span class="source-line-no">1151</span><span id="line-1151"> HRegion hRegion = region;</span>
<span class="source-line-no">1152</span><span id="line-1152"> if (compactionDesc != null) {</span>
<span class="source-line-no">1153</span><span id="line-1153"> // replay the compaction. Remove the files from stores only if we are the primary</span>
<span class="source-line-no">1154</span><span id="line-1154"> // region replica (thus own the files)</span>
<span class="source-line-no">1155</span><span id="line-1155"> hRegion.replayWALCompactionMarker(compactionDesc, !isDefaultReplica, isDefaultReplica,</span>
<span class="source-line-no">1156</span><span id="line-1156"> replaySeqId);</span>
<span class="source-line-no">1157</span><span id="line-1157"> continue;</span>
<span class="source-line-no">1158</span><span id="line-1158"> }</span>
<span class="source-line-no">1159</span><span id="line-1159"> FlushDescriptor flushDesc = WALEdit.getFlushDescriptor(metaCell);</span>
<span class="source-line-no">1160</span><span id="line-1160"> if (flushDesc != null &amp;&amp; !isDefaultReplica) {</span>
<span class="source-line-no">1161</span><span id="line-1161"> hRegion.replayWALFlushMarker(flushDesc, replaySeqId);</span>
<span class="source-line-no">1162</span><span id="line-1162"> continue;</span>
<span class="source-line-no">1163</span><span id="line-1163"> }</span>
<span class="source-line-no">1164</span><span id="line-1164"> RegionEventDescriptor regionEvent = WALEdit.getRegionEventDescriptor(metaCell);</span>
<span class="source-line-no">1165</span><span id="line-1165"> if (regionEvent != null &amp;&amp; !isDefaultReplica) {</span>
<span class="source-line-no">1166</span><span id="line-1166"> hRegion.replayWALRegionEventMarker(regionEvent);</span>
<span class="source-line-no">1167</span><span id="line-1167"> continue;</span>
<span class="source-line-no">1168</span><span id="line-1168"> }</span>
<span class="source-line-no">1169</span><span id="line-1169"> BulkLoadDescriptor bulkLoadEvent = WALEdit.getBulkLoadDescriptor(metaCell);</span>
<span class="source-line-no">1170</span><span id="line-1170"> if (bulkLoadEvent != null) {</span>
<span class="source-line-no">1171</span><span id="line-1171"> hRegion.replayWALBulkLoadEventMarker(bulkLoadEvent);</span>
<span class="source-line-no">1172</span><span id="line-1172"> continue;</span>
<span class="source-line-no">1173</span><span id="line-1173"> }</span>
<span class="source-line-no">1174</span><span id="line-1174"> }</span>
<span class="source-line-no">1175</span><span id="line-1175"> it.remove();</span>
<span class="source-line-no">1176</span><span id="line-1176"> }</span>
<span class="source-line-no">1177</span><span id="line-1177"> }</span>
<span class="source-line-no">1178</span><span id="line-1178"> requestCount.increment();</span>
<span class="source-line-no">1179</span><span id="line-1179"> if (!region.getRegionInfo().isMetaRegion()) {</span>
<span class="source-line-no">1180</span><span id="line-1180"> server.getMemStoreFlusher().reclaimMemStoreMemory();</span>
<span class="source-line-no">1181</span><span id="line-1181"> }</span>
<span class="source-line-no">1182</span><span id="line-1182"> return region.batchReplay(mutations.toArray(new MutationReplay[mutations.size()]),</span>
<span class="source-line-no">1183</span><span id="line-1183"> replaySeqId);</span>
<span class="source-line-no">1184</span><span id="line-1184"> } finally {</span>
<span class="source-line-no">1185</span><span id="line-1185"> updateMutationMetrics(region, before, batchContainsPuts, batchContainsDelete);</span>
<span class="source-line-no">1186</span><span id="line-1186"> }</span>
<span class="source-line-no">1187</span><span id="line-1187"> }</span>
<span class="source-line-no">1188</span><span id="line-1188"></span>
<span class="source-line-no">1189</span><span id="line-1189"> private void closeAllScanners() {</span>
<span class="source-line-no">1190</span><span id="line-1190"> // Close any outstanding scanners. Means they'll get an UnknownScanner</span>
<span class="source-line-no">1191</span><span id="line-1191"> // exception next time they come in.</span>
<span class="source-line-no">1192</span><span id="line-1192"> for (Map.Entry&lt;String, RegionScannerHolder&gt; e : scanners.entrySet()) {</span>
<span class="source-line-no">1193</span><span id="line-1193"> try {</span>
<span class="source-line-no">1194</span><span id="line-1194"> e.getValue().s.close();</span>
<span class="source-line-no">1195</span><span id="line-1195"> } catch (IOException ioe) {</span>
<span class="source-line-no">1196</span><span id="line-1196"> LOG.warn("Closing scanner " + e.getKey(), ioe);</span>
<span class="source-line-no">1197</span><span id="line-1197"> }</span>
<span class="source-line-no">1198</span><span id="line-1198"> }</span>
<span class="source-line-no">1199</span><span id="line-1199"> }</span>
<span class="source-line-no">1200</span><span id="line-1200"></span>
<span class="source-line-no">1201</span><span id="line-1201"> // Directly invoked only for testing</span>
<span class="source-line-no">1202</span><span id="line-1202"> public RSRpcServices(final HRegionServer rs) throws IOException {</span>
<span class="source-line-no">1203</span><span id="line-1203"> super(rs, rs.getProcessName());</span>
<span class="source-line-no">1204</span><span id="line-1204"> final Configuration conf = rs.getConfiguration();</span>
<span class="source-line-no">1205</span><span id="line-1205"> setReloadableGuardrails(conf);</span>
<span class="source-line-no">1206</span><span id="line-1206"> scannerLeaseTimeoutPeriod = conf.getInt(HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD,</span>
<span class="source-line-no">1207</span><span id="line-1207"> HConstants.DEFAULT_HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD);</span>
<span class="source-line-no">1208</span><span id="line-1208"> rpcTimeout =</span>
<span class="source-line-no">1209</span><span id="line-1209"> conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT);</span>
<span class="source-line-no">1210</span><span id="line-1210"> minimumScanTimeLimitDelta = conf.getLong(REGION_SERVER_RPC_MINIMUM_SCAN_TIME_LIMIT_DELTA,</span>
<span class="source-line-no">1211</span><span id="line-1211"> DEFAULT_REGION_SERVER_RPC_MINIMUM_SCAN_TIME_LIMIT_DELTA);</span>
<span class="source-line-no">1212</span><span id="line-1212"> rpcServer.setNamedQueueRecorder(rs.getNamedQueueRecorder());</span>
<span class="source-line-no">1213</span><span id="line-1213"> closedScanners = CacheBuilder.newBuilder()</span>
<span class="source-line-no">1214</span><span id="line-1214"> .expireAfterAccess(scannerLeaseTimeoutPeriod, TimeUnit.MILLISECONDS).build();</span>
<span class="source-line-no">1215</span><span id="line-1215"> }</span>
<span class="source-line-no">1216</span><span id="line-1216"></span>
<span class="source-line-no">1217</span><span id="line-1217"> @Override</span>
<span class="source-line-no">1218</span><span id="line-1218"> protected boolean defaultReservoirEnabled() {</span>
<span class="source-line-no">1219</span><span id="line-1219"> return true;</span>
<span class="source-line-no">1220</span><span id="line-1220"> }</span>
<span class="source-line-no">1221</span><span id="line-1221"></span>
<span class="source-line-no">1222</span><span id="line-1222"> @Override</span>
<span class="source-line-no">1223</span><span id="line-1223"> protected ServerType getDNSServerType() {</span>
<span class="source-line-no">1224</span><span id="line-1224"> return DNS.ServerType.REGIONSERVER;</span>
<span class="source-line-no">1225</span><span id="line-1225"> }</span>
<span class="source-line-no">1226</span><span id="line-1226"></span>
<span class="source-line-no">1227</span><span id="line-1227"> @Override</span>
<span class="source-line-no">1228</span><span id="line-1228"> protected String getHostname(Configuration conf, String defaultHostname) {</span>
<span class="source-line-no">1229</span><span id="line-1229"> return conf.get("hbase.regionserver.ipc.address", defaultHostname);</span>
<span class="source-line-no">1230</span><span id="line-1230"> }</span>
<span class="source-line-no">1231</span><span id="line-1231"></span>
<span class="source-line-no">1232</span><span id="line-1232"> @Override</span>
<span class="source-line-no">1233</span><span id="line-1233"> protected String getPortConfigName() {</span>
<span class="source-line-no">1234</span><span id="line-1234"> return HConstants.REGIONSERVER_PORT;</span>
<span class="source-line-no">1235</span><span id="line-1235"> }</span>
<span class="source-line-no">1236</span><span id="line-1236"></span>
<span class="source-line-no">1237</span><span id="line-1237"> @Override</span>
<span class="source-line-no">1238</span><span id="line-1238"> protected int getDefaultPort() {</span>
<span class="source-line-no">1239</span><span id="line-1239"> return HConstants.DEFAULT_REGIONSERVER_PORT;</span>
<span class="source-line-no">1240</span><span id="line-1240"> }</span>
<span class="source-line-no">1241</span><span id="line-1241"></span>
<span class="source-line-no">1242</span><span id="line-1242"> @Override</span>
<span class="source-line-no">1243</span><span id="line-1243"> protected Class&lt;?&gt; getRpcSchedulerFactoryClass(Configuration conf) {</span>
<span class="source-line-no">1244</span><span id="line-1244"> return conf.getClass(REGION_SERVER_RPC_SCHEDULER_FACTORY_CLASS,</span>
<span class="source-line-no">1245</span><span id="line-1245"> SimpleRpcSchedulerFactory.class);</span>
<span class="source-line-no">1246</span><span id="line-1246"> }</span>
<span class="source-line-no">1247</span><span id="line-1247"></span>
<span class="source-line-no">1248</span><span id="line-1248"> protected RpcServerInterface createRpcServer(final Server server,</span>
<span class="source-line-no">1249</span><span id="line-1249"> final RpcSchedulerFactory rpcSchedulerFactory, final InetSocketAddress bindAddress,</span>
<span class="source-line-no">1250</span><span id="line-1250"> final String name) throws IOException {</span>
<span class="source-line-no">1251</span><span id="line-1251"> final Configuration conf = server.getConfiguration();</span>
<span class="source-line-no">1252</span><span id="line-1252"> boolean reservoirEnabled = conf.getBoolean(ByteBuffAllocator.ALLOCATOR_POOL_ENABLED_KEY, true);</span>
<span class="source-line-no">1253</span><span id="line-1253"> try {</span>
<span class="source-line-no">1254</span><span id="line-1254"> return RpcServerFactory.createRpcServer(server, name, getServices(), bindAddress, // use final</span>
<span class="source-line-no">1255</span><span id="line-1255"> // bindAddress</span>
<span class="source-line-no">1256</span><span id="line-1256"> // for this</span>
<span class="source-line-no">1257</span><span id="line-1257"> // server.</span>
<span class="source-line-no">1258</span><span id="line-1258"> conf, rpcSchedulerFactory.create(conf, this, server), reservoirEnabled);</span>
<span class="source-line-no">1259</span><span id="line-1259"> } catch (BindException be) {</span>
<span class="source-line-no">1260</span><span id="line-1260"> throw new IOException(be.getMessage() + ". To switch ports use the '"</span>
<span class="source-line-no">1261</span><span id="line-1261"> + HConstants.REGIONSERVER_PORT + "' configuration property.",</span>
<span class="source-line-no">1262</span><span id="line-1262"> be.getCause() != null ? be.getCause() : be);</span>
<span class="source-line-no">1263</span><span id="line-1263"> }</span>
<span class="source-line-no">1264</span><span id="line-1264"> }</span>
<span class="source-line-no">1265</span><span id="line-1265"></span>
<span class="source-line-no">1266</span><span id="line-1266"> protected Class&lt;?&gt; getRpcSchedulerFactoryClass() {</span>
<span class="source-line-no">1267</span><span id="line-1267"> final Configuration conf = server.getConfiguration();</span>
<span class="source-line-no">1268</span><span id="line-1268"> return conf.getClass(REGION_SERVER_RPC_SCHEDULER_FACTORY_CLASS,</span>
<span class="source-line-no">1269</span><span id="line-1269"> SimpleRpcSchedulerFactory.class);</span>
<span class="source-line-no">1270</span><span id="line-1270"> }</span>
<span class="source-line-no">1271</span><span id="line-1271"></span>
<span class="source-line-no">1272</span><span id="line-1272"> protected PriorityFunction createPriority() {</span>
<span class="source-line-no">1273</span><span id="line-1273"> return new RSAnnotationReadingPriorityFunction(this);</span>
<span class="source-line-no">1274</span><span id="line-1274"> }</span>
<span class="source-line-no">1275</span><span id="line-1275"></span>
<span class="source-line-no">1276</span><span id="line-1276"> public int getScannersCount() {</span>
<span class="source-line-no">1277</span><span id="line-1277"> return scanners.size();</span>
<span class="source-line-no">1278</span><span id="line-1278"> }</span>
<span class="source-line-no">1279</span><span id="line-1279"></span>
<span class="source-line-no">1280</span><span id="line-1280"> /** Returns The outstanding RegionScanner for &lt;code&gt;scannerId&lt;/code&gt; or null if none found. */</span>
<span class="source-line-no">1281</span><span id="line-1281"> RegionScanner getScanner(long scannerId) {</span>
<span class="source-line-no">1282</span><span id="line-1282"> RegionScannerHolder rsh = checkQuotaAndGetRegionScannerContext(scannerId);</span>
<span class="source-line-no">1283</span><span id="line-1283"> return rsh == null ? null : rsh.s;</span>
<span class="source-line-no">1284</span><span id="line-1284"> }</span>
<span class="source-line-no">1285</span><span id="line-1285"></span>
<span class="source-line-no">1286</span><span id="line-1286"> /** Returns The associated RegionScannerHolder for &lt;code&gt;scannerId&lt;/code&gt; or null. */</span>
<span class="source-line-no">1287</span><span id="line-1287"> private RegionScannerHolder checkQuotaAndGetRegionScannerContext(long scannerId) {</span>
<span class="source-line-no">1288</span><span id="line-1288"> return scanners.get(toScannerName(scannerId));</span>
<span class="source-line-no">1289</span><span id="line-1289"> }</span>
<span class="source-line-no">1290</span><span id="line-1290"></span>
<span class="source-line-no">1291</span><span id="line-1291"> public String getScanDetailsWithId(long scannerId) {</span>
<span class="source-line-no">1292</span><span id="line-1292"> RegionScanner scanner = getScanner(scannerId);</span>
<span class="source-line-no">1293</span><span id="line-1293"> if (scanner == null) {</span>
<span class="source-line-no">1294</span><span id="line-1294"> return null;</span>
<span class="source-line-no">1295</span><span id="line-1295"> }</span>
<span class="source-line-no">1296</span><span id="line-1296"> StringBuilder builder = new StringBuilder();</span>
<span class="source-line-no">1297</span><span id="line-1297"> builder.append("table: ").append(scanner.getRegionInfo().getTable().getNameAsString());</span>
<span class="source-line-no">1298</span><span id="line-1298"> builder.append(" region: ").append(scanner.getRegionInfo().getRegionNameAsString());</span>
<span class="source-line-no">1299</span><span id="line-1299"> builder.append(" operation_id: ").append(scanner.getOperationId());</span>
<span class="source-line-no">1300</span><span id="line-1300"> return builder.toString();</span>
<span class="source-line-no">1301</span><span id="line-1301"> }</span>
<span class="source-line-no">1302</span><span id="line-1302"></span>
<span class="source-line-no">1303</span><span id="line-1303"> public String getScanDetailsWithRequest(ScanRequest request) {</span>
<span class="source-line-no">1304</span><span id="line-1304"> try {</span>
<span class="source-line-no">1305</span><span id="line-1305"> if (!request.hasRegion()) {</span>
<span class="source-line-no">1306</span><span id="line-1306"> return null;</span>
<span class="source-line-no">1307</span><span id="line-1307"> }</span>
<span class="source-line-no">1308</span><span id="line-1308"> Region region = getRegion(request.getRegion());</span>
<span class="source-line-no">1309</span><span id="line-1309"> StringBuilder builder = new StringBuilder();</span>
<span class="source-line-no">1310</span><span id="line-1310"> builder.append("table: ").append(region.getRegionInfo().getTable().getNameAsString());</span>
<span class="source-line-no">1311</span><span id="line-1311"> builder.append(" region: ").append(region.getRegionInfo().getRegionNameAsString());</span>
<span class="source-line-no">1312</span><span id="line-1312"> for (NameBytesPair pair : request.getScan().getAttributeList()) {</span>
<span class="source-line-no">1313</span><span id="line-1313"> if (OperationWithAttributes.ID_ATRIBUTE.equals(pair.getName())) {</span>
<span class="source-line-no">1314</span><span id="line-1314"> builder.append(" operation_id: ").append(Bytes.toString(pair.getValue().toByteArray()));</span>
<span class="source-line-no">1315</span><span id="line-1315"> break;</span>
<span class="source-line-no">1316</span><span id="line-1316"> }</span>
<span class="source-line-no">1317</span><span id="line-1317"> }</span>
<span class="source-line-no">1318</span><span id="line-1318"> return builder.toString();</span>
<span class="source-line-no">1319</span><span id="line-1319"> } catch (IOException ignored) {</span>
<span class="source-line-no">1320</span><span id="line-1320"> return null;</span>
<span class="source-line-no">1321</span><span id="line-1321"> }</span>
<span class="source-line-no">1322</span><span id="line-1322"> }</span>
<span class="source-line-no">1323</span><span id="line-1323"></span>
<span class="source-line-no">1324</span><span id="line-1324"> /**</span>
<span class="source-line-no">1325</span><span id="line-1325"> * Get the vtime associated with the scanner. Currently the vtime is the number of "next" calls.</span>
<span class="source-line-no">1326</span><span id="line-1326"> */</span>
<span class="source-line-no">1327</span><span id="line-1327"> long getScannerVirtualTime(long scannerId) {</span>
<span class="source-line-no">1328</span><span id="line-1328"> RegionScannerHolder rsh = checkQuotaAndGetRegionScannerContext(scannerId);</span>
<span class="source-line-no">1329</span><span id="line-1329"> return rsh == null ? 0L : rsh.getNextCallSeq();</span>
<span class="source-line-no">1330</span><span id="line-1330"> }</span>
<span class="source-line-no">1331</span><span id="line-1331"></span>
<span class="source-line-no">1332</span><span id="line-1332"> /**</span>
<span class="source-line-no">1333</span><span id="line-1333"> * Method to account for the size of retained cells.</span>
<span class="source-line-no">1334</span><span id="line-1334"> * @param context rpc call context</span>
<span class="source-line-no">1335</span><span id="line-1335"> * @param r result to add size.</span>
<span class="source-line-no">1336</span><span id="line-1336"> * @return an object that represents the last referenced block from this response.</span>
<span class="source-line-no">1337</span><span id="line-1337"> */</span>
<span class="source-line-no">1338</span><span id="line-1338"> void addSize(RpcCallContext context, Result r) {</span>
<span class="source-line-no">1339</span><span id="line-1339"> if (context != null &amp;&amp; r != null &amp;&amp; !r.isEmpty()) {</span>
<span class="source-line-no">1340</span><span id="line-1340"> for (Cell c : r.rawCells()) {</span>
<span class="source-line-no">1341</span><span id="line-1341"> context.incrementResponseCellSize(PrivateCellUtil.estimatedSerializedSizeOf(c));</span>
<span class="source-line-no">1342</span><span id="line-1342"> }</span>
<span class="source-line-no">1343</span><span id="line-1343"> }</span>
<span class="source-line-no">1344</span><span id="line-1344"> }</span>
<span class="source-line-no">1345</span><span id="line-1345"></span>
<span class="source-line-no">1346</span><span id="line-1346"> /** Returns Remote client's ip and port else null if can't be determined. */</span>
<span class="source-line-no">1347</span><span id="line-1347"> @RestrictedApi(explanation = "Should only be called in TestRSRpcServices and RSRpcServices",</span>
<span class="source-line-no">1348</span><span id="line-1348"> link = "", allowedOnPath = ".*(TestRSRpcServices|RSRpcServices).java")</span>
<span class="source-line-no">1349</span><span id="line-1349"> static String getRemoteClientIpAndPort() {</span>
<span class="source-line-no">1350</span><span id="line-1350"> RpcCall rpcCall = RpcServer.getCurrentCall().orElse(null);</span>
<span class="source-line-no">1351</span><span id="line-1351"> if (rpcCall == null) {</span>
<span class="source-line-no">1352</span><span id="line-1352"> return HConstants.EMPTY_STRING;</span>
<span class="source-line-no">1353</span><span id="line-1353"> }</span>
<span class="source-line-no">1354</span><span id="line-1354"> InetAddress address = rpcCall.getRemoteAddress();</span>
<span class="source-line-no">1355</span><span id="line-1355"> if (address == null) {</span>
<span class="source-line-no">1356</span><span id="line-1356"> return HConstants.EMPTY_STRING;</span>
<span class="source-line-no">1357</span><span id="line-1357"> }</span>
<span class="source-line-no">1358</span><span id="line-1358"> // Be careful here with InetAddress. Do InetAddress#getHostAddress. It will not do a name</span>
<span class="source-line-no">1359</span><span id="line-1359"> // resolution. Just use the IP. It is generally a smaller amount of info to keep around while</span>
<span class="source-line-no">1360</span><span id="line-1360"> // scanning than a hostname anyways.</span>
<span class="source-line-no">1361</span><span id="line-1361"> return Address.fromParts(address.getHostAddress(), rpcCall.getRemotePort()).toString();</span>
<span class="source-line-no">1362</span><span id="line-1362"> }</span>
<span class="source-line-no">1363</span><span id="line-1363"></span>
<span class="source-line-no">1364</span><span id="line-1364"> /** Returns Remote client's username. */</span>
<span class="source-line-no">1365</span><span id="line-1365"> @RestrictedApi(explanation = "Should only be called in TestRSRpcServices and RSRpcServices",</span>
<span class="source-line-no">1366</span><span id="line-1366"> link = "", allowedOnPath = ".*(TestRSRpcServices|RSRpcServices).java")</span>
<span class="source-line-no">1367</span><span id="line-1367"> static String getUserName() {</span>
<span class="source-line-no">1368</span><span id="line-1368"> RpcCall rpcCall = RpcServer.getCurrentCall().orElse(null);</span>
<span class="source-line-no">1369</span><span id="line-1369"> if (rpcCall == null) {</span>
<span class="source-line-no">1370</span><span id="line-1370"> return HConstants.EMPTY_STRING;</span>
<span class="source-line-no">1371</span><span id="line-1371"> }</span>
<span class="source-line-no">1372</span><span id="line-1372"> return rpcCall.getRequestUserName().orElse(HConstants.EMPTY_STRING);</span>
<span class="source-line-no">1373</span><span id="line-1373"> }</span>
<span class="source-line-no">1374</span><span id="line-1374"></span>
<span class="source-line-no">1375</span><span id="line-1375"> private RegionScannerHolder addScanner(String scannerName, RegionScanner s, Shipper shipper,</span>
<span class="source-line-no">1376</span><span id="line-1376"> HRegion r, boolean needCursor, boolean fullRegionScan) throws LeaseStillHeldException {</span>
<span class="source-line-no">1377</span><span id="line-1377"> Lease lease = server.getLeaseManager().createLease(scannerName, this.scannerLeaseTimeoutPeriod,</span>
<span class="source-line-no">1378</span><span id="line-1378"> new ScannerListener(scannerName));</span>
<span class="source-line-no">1379</span><span id="line-1379"> RpcCallback shippedCallback = new RegionScannerShippedCallBack(scannerName, shipper, lease);</span>
<span class="source-line-no">1380</span><span id="line-1380"> RpcCallback closeCallback =</span>
<span class="source-line-no">1381</span><span id="line-1381"> s instanceof RpcCallback ? (RpcCallback) s : new RegionScannerCloseCallBack(s);</span>
<span class="source-line-no">1382</span><span id="line-1382"> RegionScannerHolder rsh = new RegionScannerHolder(s, r, closeCallback, shippedCallback,</span>
<span class="source-line-no">1383</span><span id="line-1383"> needCursor, fullRegionScan, getRemoteClientIpAndPort(), getUserName());</span>
<span class="source-line-no">1384</span><span id="line-1384"> RegionScannerHolder existing = scanners.putIfAbsent(scannerName, rsh);</span>
<span class="source-line-no">1385</span><span id="line-1385"> assert existing == null : "scannerId must be unique within regionserver's whole lifecycle! "</span>
<span class="source-line-no">1386</span><span id="line-1386"> + scannerName + ", " + existing;</span>
<span class="source-line-no">1387</span><span id="line-1387"> return rsh;</span>
<span class="source-line-no">1388</span><span id="line-1388"> }</span>
<span class="source-line-no">1389</span><span id="line-1389"></span>
<span class="source-line-no">1390</span><span id="line-1390"> private boolean isFullRegionScan(Scan scan, HRegion region) {</span>
<span class="source-line-no">1391</span><span id="line-1391"> // If the scan start row equals or less than the start key of the region</span>
<span class="source-line-no">1392</span><span id="line-1392"> // and stop row greater than equals end key (if stop row present)</span>
<span class="source-line-no">1393</span><span id="line-1393"> // or if the stop row is empty</span>
<span class="source-line-no">1394</span><span id="line-1394"> // account this as a full region scan</span>
<span class="source-line-no">1395</span><span id="line-1395"> if (</span>
<span class="source-line-no">1396</span><span id="line-1396"> Bytes.compareTo(scan.getStartRow(), region.getRegionInfo().getStartKey()) &lt;= 0</span>
<span class="source-line-no">1397</span><span id="line-1397"> &amp;&amp; (Bytes.compareTo(scan.getStopRow(), region.getRegionInfo().getEndKey()) &gt;= 0</span>
<span class="source-line-no">1398</span><span id="line-1398"> &amp;&amp; !Bytes.equals(region.getRegionInfo().getEndKey(), HConstants.EMPTY_END_ROW)</span>
<span class="source-line-no">1399</span><span id="line-1399"> || Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW))</span>
<span class="source-line-no">1400</span><span id="line-1400"> ) {</span>
<span class="source-line-no">1401</span><span id="line-1401"> return true;</span>
<span class="source-line-no">1402</span><span id="line-1402"> }</span>
<span class="source-line-no">1403</span><span id="line-1403"> return false;</span>
<span class="source-line-no">1404</span><span id="line-1404"> }</span>
<span class="source-line-no">1405</span><span id="line-1405"></span>
<span class="source-line-no">1406</span><span id="line-1406"> /**</span>
<span class="source-line-no">1407</span><span id="line-1407"> * Find the HRegion based on a region specifier</span>
<span class="source-line-no">1408</span><span id="line-1408"> * @param regionSpecifier the region specifier</span>
<span class="source-line-no">1409</span><span id="line-1409"> * @return the corresponding region</span>
<span class="source-line-no">1410</span><span id="line-1410"> * @throws IOException if the specifier is not null, but failed to find the region</span>
<span class="source-line-no">1411</span><span id="line-1411"> */</span>
<span class="source-line-no">1412</span><span id="line-1412"> public HRegion getRegion(final RegionSpecifier regionSpecifier) throws IOException {</span>
<span class="source-line-no">1413</span><span id="line-1413"> return server.getRegion(regionSpecifier.getValue().toByteArray());</span>
<span class="source-line-no">1414</span><span id="line-1414"> }</span>
<span class="source-line-no">1415</span><span id="line-1415"></span>
<span class="source-line-no">1416</span><span id="line-1416"> /**</span>
<span class="source-line-no">1417</span><span id="line-1417"> * Find the List of HRegions based on a list of region specifiers</span>
<span class="source-line-no">1418</span><span id="line-1418"> * @param regionSpecifiers the list of region specifiers</span>
<span class="source-line-no">1419</span><span id="line-1419"> * @return the corresponding list of regions</span>
<span class="source-line-no">1420</span><span id="line-1420"> * @throws IOException if any of the specifiers is not null, but failed to find the region</span>
<span class="source-line-no">1421</span><span id="line-1421"> */</span>
<span class="source-line-no">1422</span><span id="line-1422"> private List&lt;HRegion&gt; getRegions(final List&lt;RegionSpecifier&gt; regionSpecifiers,</span>
<span class="source-line-no">1423</span><span id="line-1423"> final CacheEvictionStatsBuilder stats) {</span>
<span class="source-line-no">1424</span><span id="line-1424"> List&lt;HRegion&gt; regions = Lists.newArrayListWithCapacity(regionSpecifiers.size());</span>
<span class="source-line-no">1425</span><span id="line-1425"> for (RegionSpecifier regionSpecifier : regionSpecifiers) {</span>
<span class="source-line-no">1426</span><span id="line-1426"> try {</span>
<span class="source-line-no">1427</span><span id="line-1427"> regions.add(server.getRegion(regionSpecifier.getValue().toByteArray()));</span>
<span class="source-line-no">1428</span><span id="line-1428"> } catch (NotServingRegionException e) {</span>
<span class="source-line-no">1429</span><span id="line-1429"> stats.addException(regionSpecifier.getValue().toByteArray(), e);</span>
<span class="source-line-no">1430</span><span id="line-1430"> }</span>
<span class="source-line-no">1431</span><span id="line-1431"> }</span>
<span class="source-line-no">1432</span><span id="line-1432"> return regions;</span>
<span class="source-line-no">1433</span><span id="line-1433"> }</span>
<span class="source-line-no">1434</span><span id="line-1434"></span>
<span class="source-line-no">1435</span><span id="line-1435"> PriorityFunction getPriority() {</span>
<span class="source-line-no">1436</span><span id="line-1436"> return priority;</span>
<span class="source-line-no">1437</span><span id="line-1437"> }</span>
<span class="source-line-no">1438</span><span id="line-1438"></span>
<span class="source-line-no">1439</span><span id="line-1439"> private RegionServerRpcQuotaManager getRpcQuotaManager() {</span>
<span class="source-line-no">1440</span><span id="line-1440"> return server.getRegionServerRpcQuotaManager();</span>
<span class="source-line-no">1441</span><span id="line-1441"> }</span>
<span class="source-line-no">1442</span><span id="line-1442"></span>
<span class="source-line-no">1443</span><span id="line-1443"> private RegionServerSpaceQuotaManager getSpaceQuotaManager() {</span>
<span class="source-line-no">1444</span><span id="line-1444"> return server.getRegionServerSpaceQuotaManager();</span>
<span class="source-line-no">1445</span><span id="line-1445"> }</span>
<span class="source-line-no">1446</span><span id="line-1446"></span>
<span class="source-line-no">1447</span><span id="line-1447"> void start(ZKWatcher zkWatcher) {</span>
<span class="source-line-no">1448</span><span id="line-1448"> this.scannerIdGenerator = new ScannerIdGenerator(this.server.getServerName());</span>
<span class="source-line-no">1449</span><span id="line-1449"> internalStart(zkWatcher);</span>
<span class="source-line-no">1450</span><span id="line-1450"> }</span>
<span class="source-line-no">1451</span><span id="line-1451"></span>
<span class="source-line-no">1452</span><span id="line-1452"> void stop() {</span>
<span class="source-line-no">1453</span><span id="line-1453"> closeAllScanners();</span>
<span class="source-line-no">1454</span><span id="line-1454"> internalStop();</span>
<span class="source-line-no">1455</span><span id="line-1455"> }</span>
<span class="source-line-no">1456</span><span id="line-1456"></span>
<span class="source-line-no">1457</span><span id="line-1457"> /**</span>
<span class="source-line-no">1458</span><span id="line-1458"> * Called to verify that this server is up and running.</span>
<span class="source-line-no">1459</span><span id="line-1459"> */</span>
<span class="source-line-no">1460</span><span id="line-1460"> // TODO : Rename this and HMaster#checkInitialized to isRunning() (or a better name).</span>
<span class="source-line-no">1461</span><span id="line-1461"> protected void checkOpen() throws IOException {</span>
<span class="source-line-no">1462</span><span id="line-1462"> if (server.isAborted()) {</span>
<span class="source-line-no">1463</span><span id="line-1463"> throw new RegionServerAbortedException("Server " + server.getServerName() + " aborting");</span>
<span class="source-line-no">1464</span><span id="line-1464"> }</span>
<span class="source-line-no">1465</span><span id="line-1465"> if (server.isStopped()) {</span>
<span class="source-line-no">1466</span><span id="line-1466"> throw new RegionServerStoppedException("Server " + server.getServerName() + " stopping");</span>
<span class="source-line-no">1467</span><span id="line-1467"> }</span>
<span class="source-line-no">1468</span><span id="line-1468"> if (!server.isDataFileSystemOk()) {</span>
<span class="source-line-no">1469</span><span id="line-1469"> throw new RegionServerStoppedException("File system not available");</span>
<span class="source-line-no">1470</span><span id="line-1470"> }</span>
<span class="source-line-no">1471</span><span id="line-1471"> if (!server.isOnline()) {</span>
<span class="source-line-no">1472</span><span id="line-1472"> throw new ServerNotRunningYetException(</span>
<span class="source-line-no">1473</span><span id="line-1473"> "Server " + server.getServerName() + " is not running yet");</span>
<span class="source-line-no">1474</span><span id="line-1474"> }</span>
<span class="source-line-no">1475</span><span id="line-1475"> }</span>
<span class="source-line-no">1476</span><span id="line-1476"></span>
<span class="source-line-no">1477</span><span id="line-1477"> /**</span>
<span class="source-line-no">1478</span><span id="line-1478"> * By default, put up an Admin and a Client Service. Set booleans</span>
<span class="source-line-no">1479</span><span id="line-1479"> * &lt;code&gt;hbase.regionserver.admin.executorService&lt;/code&gt; and</span>
<span class="source-line-no">1480</span><span id="line-1480"> * &lt;code&gt;hbase.regionserver.client.executorService&lt;/code&gt; if you want to enable/disable services.</span>
<span class="source-line-no">1481</span><span id="line-1481"> * Default is that both are enabled.</span>
<span class="source-line-no">1482</span><span id="line-1482"> * @return immutable list of blocking services and the security info classes that this server</span>
<span class="source-line-no">1483</span><span id="line-1483"> * supports</span>
<span class="source-line-no">1484</span><span id="line-1484"> */</span>
<span class="source-line-no">1485</span><span id="line-1485"> protected List&lt;BlockingServiceAndInterface&gt; getServices() {</span>
<span class="source-line-no">1486</span><span id="line-1486"> boolean admin = getConfiguration().getBoolean(REGIONSERVER_ADMIN_SERVICE_CONFIG, true);</span>
<span class="source-line-no">1487</span><span id="line-1487"> boolean client = getConfiguration().getBoolean(REGIONSERVER_CLIENT_SERVICE_CONFIG, true);</span>
<span class="source-line-no">1488</span><span id="line-1488"> boolean clientMeta =</span>
<span class="source-line-no">1489</span><span id="line-1489"> getConfiguration().getBoolean(REGIONSERVER_CLIENT_META_SERVICE_CONFIG, true);</span>
<span class="source-line-no">1490</span><span id="line-1490"> boolean bootstrapNodes =</span>
<span class="source-line-no">1491</span><span id="line-1491"> getConfiguration().getBoolean(REGIONSERVER_BOOTSTRAP_NODES_SERVICE_CONFIG, true);</span>
<span class="source-line-no">1492</span><span id="line-1492"> List&lt;BlockingServiceAndInterface&gt; bssi = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">1493</span><span id="line-1493"> if (client) {</span>
<span class="source-line-no">1494</span><span id="line-1494"> bssi.add(new BlockingServiceAndInterface(ClientService.newReflectiveBlockingService(this),</span>
<span class="source-line-no">1495</span><span id="line-1495"> ClientService.BlockingInterface.class));</span>
<span class="source-line-no">1496</span><span id="line-1496"> }</span>
<span class="source-line-no">1497</span><span id="line-1497"> if (admin) {</span>
<span class="source-line-no">1498</span><span id="line-1498"> bssi.add(new BlockingServiceAndInterface(AdminService.newReflectiveBlockingService(this),</span>
<span class="source-line-no">1499</span><span id="line-1499"> AdminService.BlockingInterface.class));</span>
<span class="source-line-no">1500</span><span id="line-1500"> }</span>
<span class="source-line-no">1501</span><span id="line-1501"> if (clientMeta) {</span>
<span class="source-line-no">1502</span><span id="line-1502"> bssi.add(new BlockingServiceAndInterface(ClientMetaService.newReflectiveBlockingService(this),</span>
<span class="source-line-no">1503</span><span id="line-1503"> ClientMetaService.BlockingInterface.class));</span>
<span class="source-line-no">1504</span><span id="line-1504"> }</span>
<span class="source-line-no">1505</span><span id="line-1505"> if (bootstrapNodes) {</span>
<span class="source-line-no">1506</span><span id="line-1506"> bssi.add(</span>
<span class="source-line-no">1507</span><span id="line-1507"> new BlockingServiceAndInterface(BootstrapNodeService.newReflectiveBlockingService(this),</span>
<span class="source-line-no">1508</span><span id="line-1508"> BootstrapNodeService.BlockingInterface.class));</span>
<span class="source-line-no">1509</span><span id="line-1509"> }</span>
<span class="source-line-no">1510</span><span id="line-1510"> return new ImmutableList.Builder&lt;BlockingServiceAndInterface&gt;().addAll(bssi).build();</span>
<span class="source-line-no">1511</span><span id="line-1511"> }</span>
<span class="source-line-no">1512</span><span id="line-1512"></span>
<span class="source-line-no">1513</span><span id="line-1513"> /**</span>
<span class="source-line-no">1514</span><span id="line-1514"> * Close a region on the region server.</span>
<span class="source-line-no">1515</span><span id="line-1515"> * @param controller the RPC controller</span>
<span class="source-line-no">1516</span><span id="line-1516"> * @param request the request</span>
<span class="source-line-no">1517</span><span id="line-1517"> */</span>
<span class="source-line-no">1518</span><span id="line-1518"> @Override</span>
<span class="source-line-no">1519</span><span id="line-1519"> @QosPriority(priority = HConstants.ADMIN_QOS)</span>
<span class="source-line-no">1520</span><span id="line-1520"> public CloseRegionResponse closeRegion(final RpcController controller,</span>
<span class="source-line-no">1521</span><span id="line-1521"> final CloseRegionRequest request) throws ServiceException {</span>
<span class="source-line-no">1522</span><span id="line-1522"> final ServerName sn = (request.hasDestinationServer()</span>
<span class="source-line-no">1523</span><span id="line-1523"> ? ProtobufUtil.toServerName(request.getDestinationServer())</span>
<span class="source-line-no">1524</span><span id="line-1524"> : null);</span>
<span class="source-line-no">1525</span><span id="line-1525"></span>
<span class="source-line-no">1526</span><span id="line-1526"> try {</span>
<span class="source-line-no">1527</span><span id="line-1527"> checkOpen();</span>
<span class="source-line-no">1528</span><span id="line-1528"> throwOnWrongStartCode(request);</span>
<span class="source-line-no">1529</span><span id="line-1529"> final String encodedRegionName = ProtobufUtil.getRegionEncodedName(request.getRegion());</span>
<span class="source-line-no">1530</span><span id="line-1530"></span>
<span class="source-line-no">1531</span><span id="line-1531"> requestCount.increment();</span>
<span class="source-line-no">1532</span><span id="line-1532"> if (sn == null) {</span>
<span class="source-line-no">1533</span><span id="line-1533"> LOG.info("Close " + encodedRegionName + " without moving");</span>
<span class="source-line-no">1534</span><span id="line-1534"> } else {</span>
<span class="source-line-no">1535</span><span id="line-1535"> LOG.info("Close " + encodedRegionName + ", moving to " + sn);</span>
<span class="source-line-no">1536</span><span id="line-1536"> }</span>
<span class="source-line-no">1537</span><span id="line-1537"> boolean closed = server.closeRegion(encodedRegionName, false, sn);</span>
<span class="source-line-no">1538</span><span id="line-1538"> CloseRegionResponse.Builder builder = CloseRegionResponse.newBuilder().setClosed(closed);</span>
<span class="source-line-no">1539</span><span id="line-1539"> return builder.build();</span>
<span class="source-line-no">1540</span><span id="line-1540"> } catch (IOException ie) {</span>
<span class="source-line-no">1541</span><span id="line-1541"> throw new ServiceException(ie);</span>
<span class="source-line-no">1542</span><span id="line-1542"> }</span>
<span class="source-line-no">1543</span><span id="line-1543"> }</span>
<span class="source-line-no">1544</span><span id="line-1544"></span>
<span class="source-line-no">1545</span><span id="line-1545"> /**</span>
<span class="source-line-no">1546</span><span id="line-1546"> * Compact a region on the region server.</span>
<span class="source-line-no">1547</span><span id="line-1547"> * @param controller the RPC controller</span>
<span class="source-line-no">1548</span><span id="line-1548"> * @param request the request</span>
<span class="source-line-no">1549</span><span id="line-1549"> */</span>
<span class="source-line-no">1550</span><span id="line-1550"> @Override</span>
<span class="source-line-no">1551</span><span id="line-1551"> @QosPriority(priority = HConstants.ADMIN_QOS)</span>
<span class="source-line-no">1552</span><span id="line-1552"> public CompactRegionResponse compactRegion(final RpcController controller,</span>
<span class="source-line-no">1553</span><span id="line-1553"> final CompactRegionRequest request) throws ServiceException {</span>
<span class="source-line-no">1554</span><span id="line-1554"> try {</span>
<span class="source-line-no">1555</span><span id="line-1555"> checkOpen();</span>
<span class="source-line-no">1556</span><span id="line-1556"> requestCount.increment();</span>
<span class="source-line-no">1557</span><span id="line-1557"> HRegion region = getRegion(request.getRegion());</span>
<span class="source-line-no">1558</span><span id="line-1558"> // Quota support is enabled, the requesting user is not system/super user</span>
<span class="source-line-no">1559</span><span id="line-1559"> // and a quota policy is enforced that disables compactions.</span>
<span class="source-line-no">1560</span><span id="line-1560"> if (</span>
<span class="source-line-no">1561</span><span id="line-1561"> QuotaUtil.isQuotaEnabled(getConfiguration())</span>
<span class="source-line-no">1562</span><span id="line-1562"> &amp;&amp; !Superusers.isSuperUser(RpcServer.getRequestUser().orElse(null))</span>
<span class="source-line-no">1563</span><span id="line-1563"> &amp;&amp; this.server.getRegionServerSpaceQuotaManager()</span>
<span class="source-line-no">1564</span><span id="line-1564"> .areCompactionsDisabled(region.getTableDescriptor().getTableName())</span>
<span class="source-line-no">1565</span><span id="line-1565"> ) {</span>
<span class="source-line-no">1566</span><span id="line-1566"> throw new DoNotRetryIOException(</span>
<span class="source-line-no">1567</span><span id="line-1567"> "Compactions on this region are " + "disabled due to a space quota violation.");</span>
<span class="source-line-no">1568</span><span id="line-1568"> }</span>
<span class="source-line-no">1569</span><span id="line-1569"> region.startRegionOperation(Operation.COMPACT_REGION);</span>
<span class="source-line-no">1570</span><span id="line-1570"> LOG.info("Compacting " + region.getRegionInfo().getRegionNameAsString());</span>
<span class="source-line-no">1571</span><span id="line-1571"> boolean major = request.hasMajor() &amp;&amp; request.getMajor();</span>
<span class="source-line-no">1572</span><span id="line-1572"> if (request.hasFamily()) {</span>
<span class="source-line-no">1573</span><span id="line-1573"> byte[] family = request.getFamily().toByteArray();</span>
<span class="source-line-no">1574</span><span id="line-1574"> String log = "User-triggered " + (major ? "major " : "") + "compaction for region "</span>
<span class="source-line-no">1575</span><span id="line-1575"> + region.getRegionInfo().getRegionNameAsString() + " and family "</span>
<span class="source-line-no">1576</span><span id="line-1576"> + Bytes.toString(family);</span>
<span class="source-line-no">1577</span><span id="line-1577"> LOG.trace(log);</span>
<span class="source-line-no">1578</span><span id="line-1578"> region.requestCompaction(family, log, Store.PRIORITY_USER, major,</span>
<span class="source-line-no">1579</span><span id="line-1579"> CompactionLifeCycleTracker.DUMMY);</span>
<span class="source-line-no">1580</span><span id="line-1580"> } else {</span>
<span class="source-line-no">1581</span><span id="line-1581"> String log = "User-triggered " + (major ? "major " : "") + "compaction for region "</span>
<span class="source-line-no">1582</span><span id="line-1582"> + region.getRegionInfo().getRegionNameAsString();</span>
<span class="source-line-no">1583</span><span id="line-1583"> LOG.trace(log);</span>
<span class="source-line-no">1584</span><span id="line-1584"> region.requestCompaction(log, Store.PRIORITY_USER, major, CompactionLifeCycleTracker.DUMMY);</span>
<span class="source-line-no">1585</span><span id="line-1585"> }</span>
<span class="source-line-no">1586</span><span id="line-1586"> return CompactRegionResponse.newBuilder().build();</span>
<span class="source-line-no">1587</span><span id="line-1587"> } catch (IOException ie) {</span>
<span class="source-line-no">1588</span><span id="line-1588"> throw new ServiceException(ie);</span>
<span class="source-line-no">1589</span><span id="line-1589"> }</span>
<span class="source-line-no">1590</span><span id="line-1590"> }</span>
<span class="source-line-no">1591</span><span id="line-1591"></span>
<span class="source-line-no">1592</span><span id="line-1592"> @Override</span>
<span class="source-line-no">1593</span><span id="line-1593"> @QosPriority(priority = HConstants.ADMIN_QOS)</span>
<span class="source-line-no">1594</span><span id="line-1594"> public CompactionSwitchResponse compactionSwitch(RpcController controller,</span>
<span class="source-line-no">1595</span><span id="line-1595"> CompactionSwitchRequest request) throws ServiceException {</span>
<span class="source-line-no">1596</span><span id="line-1596"> rpcPreCheck("compactionSwitch");</span>
<span class="source-line-no">1597</span><span id="line-1597"> final CompactSplit compactSplitThread = server.getCompactSplitThread();</span>
<span class="source-line-no">1598</span><span id="line-1598"> requestCount.increment();</span>
<span class="source-line-no">1599</span><span id="line-1599"> boolean prevState = compactSplitThread.isCompactionsEnabled();</span>
<span class="source-line-no">1600</span><span id="line-1600"> CompactionSwitchResponse response =</span>
<span class="source-line-no">1601</span><span id="line-1601"> CompactionSwitchResponse.newBuilder().setPrevState(prevState).build();</span>
<span class="source-line-no">1602</span><span id="line-1602"> if (prevState == request.getEnabled()) {</span>
<span class="source-line-no">1603</span><span id="line-1603"> // passed in requested state is same as current state. No action required</span>
<span class="source-line-no">1604</span><span id="line-1604"> return response;</span>
<span class="source-line-no">1605</span><span id="line-1605"> }</span>
<span class="source-line-no">1606</span><span id="line-1606"> compactSplitThread.switchCompaction(request.getEnabled());</span>
<span class="source-line-no">1607</span><span id="line-1607"> return response;</span>
<span class="source-line-no">1608</span><span id="line-1608"> }</span>
<span class="source-line-no">1609</span><span id="line-1609"></span>
<span class="source-line-no">1610</span><span id="line-1610"> /**</span>
<span class="source-line-no">1611</span><span id="line-1611"> * Flush a region on the region server.</span>
<span class="source-line-no">1612</span><span id="line-1612"> * @param controller the RPC controller</span>
<span class="source-line-no">1613</span><span id="line-1613"> * @param request the request</span>
<span class="source-line-no">1614</span><span id="line-1614"> */</span>
<span class="source-line-no">1615</span><span id="line-1615"> @Override</span>
<span class="source-line-no">1616</span><span id="line-1616"> @QosPriority(priority = HConstants.ADMIN_QOS)</span>
<span class="source-line-no">1617</span><span id="line-1617"> public FlushRegionResponse flushRegion(final RpcController controller,</span>
<span class="source-line-no">1618</span><span id="line-1618"> final FlushRegionRequest request) throws ServiceException {</span>
<span class="source-line-no">1619</span><span id="line-1619"> try {</span>
<span class="source-line-no">1620</span><span id="line-1620"> checkOpen();</span>
<span class="source-line-no">1621</span><span id="line-1621"> requestCount.increment();</span>
<span class="source-line-no">1622</span><span id="line-1622"> HRegion region = getRegion(request.getRegion());</span>
<span class="source-line-no">1623</span><span id="line-1623"> LOG.info("Flushing " + region.getRegionInfo().getRegionNameAsString());</span>
<span class="source-line-no">1624</span><span id="line-1624"> boolean shouldFlush = true;</span>
<span class="source-line-no">1625</span><span id="line-1625"> if (request.hasIfOlderThanTs()) {</span>
<span class="source-line-no">1626</span><span id="line-1626"> shouldFlush = region.getEarliestFlushTimeForAllStores() &lt; request.getIfOlderThanTs();</span>
<span class="source-line-no">1627</span><span id="line-1627"> }</span>
<span class="source-line-no">1628</span><span id="line-1628"> FlushRegionResponse.Builder builder = FlushRegionResponse.newBuilder();</span>
<span class="source-line-no">1629</span><span id="line-1629"> if (shouldFlush) {</span>
<span class="source-line-no">1630</span><span id="line-1630"> boolean writeFlushWalMarker =</span>
<span class="source-line-no">1631</span><span id="line-1631"> request.hasWriteFlushWalMarker() ? request.getWriteFlushWalMarker() : false;</span>
<span class="source-line-no">1632</span><span id="line-1632"> // Go behind the curtain so we can manage writing of the flush WAL marker</span>
<span class="source-line-no">1633</span><span id="line-1633"> HRegion.FlushResultImpl flushResult = null;</span>
<span class="source-line-no">1634</span><span id="line-1634"> if (request.hasFamily()) {</span>
<span class="source-line-no">1635</span><span id="line-1635"> List&lt;byte[]&gt; families = new ArrayList();</span>
<span class="source-line-no">1636</span><span id="line-1636"> families.add(request.getFamily().toByteArray());</span>
<span class="source-line-no">1637</span><span id="line-1637"> TableDescriptor tableDescriptor = region.getTableDescriptor();</span>
<span class="source-line-no">1638</span><span id="line-1638"> List&lt;String&gt; noSuchFamilies = families.stream()</span>
<span class="source-line-no">1639</span><span id="line-1639"> .filter(f -&gt; !tableDescriptor.hasColumnFamily(f)).map(Bytes::toString).toList();</span>
<span class="source-line-no">1640</span><span id="line-1640"> if (!noSuchFamilies.isEmpty()) {</span>
<span class="source-line-no">1641</span><span id="line-1641"> throw new NoSuchColumnFamilyException("Column families " + noSuchFamilies</span>
<span class="source-line-no">1642</span><span id="line-1642"> + " don't exist in table " + tableDescriptor.getTableName().getNameAsString());</span>
<span class="source-line-no">1643</span><span id="line-1643"> }</span>
<span class="source-line-no">1644</span><span id="line-1644"> flushResult =</span>
<span class="source-line-no">1645</span><span id="line-1645"> region.flushcache(families, writeFlushWalMarker, FlushLifeCycleTracker.DUMMY);</span>
<span class="source-line-no">1646</span><span id="line-1646"> } else {</span>
<span class="source-line-no">1647</span><span id="line-1647"> flushResult = region.flushcache(true, writeFlushWalMarker, FlushLifeCycleTracker.DUMMY);</span>
<span class="source-line-no">1648</span><span id="line-1648"> }</span>
<span class="source-line-no">1649</span><span id="line-1649"> boolean compactionNeeded = flushResult.isCompactionNeeded();</span>
<span class="source-line-no">1650</span><span id="line-1650"> if (compactionNeeded) {</span>
<span class="source-line-no">1651</span><span id="line-1651"> server.getCompactSplitThread().requestSystemCompaction(region,</span>
<span class="source-line-no">1652</span><span id="line-1652"> "Compaction through user triggered flush");</span>
<span class="source-line-no">1653</span><span id="line-1653"> }</span>
<span class="source-line-no">1654</span><span id="line-1654"> builder.setFlushed(flushResult.isFlushSucceeded());</span>
<span class="source-line-no">1655</span><span id="line-1655"> builder.setWroteFlushWalMarker(flushResult.wroteFlushWalMarker);</span>
<span class="source-line-no">1656</span><span id="line-1656"> }</span>
<span class="source-line-no">1657</span><span id="line-1657"> builder.setLastFlushTime(region.getEarliestFlushTimeForAllStores());</span>
<span class="source-line-no">1658</span><span id="line-1658"> return builder.build();</span>
<span class="source-line-no">1659</span><span id="line-1659"> } catch (DroppedSnapshotException ex) {</span>
<span class="source-line-no">1660</span><span id="line-1660"> // Cache flush can fail in a few places. If it fails in a critical</span>
<span class="source-line-no">1661</span><span id="line-1661"> // section, we get a DroppedSnapshotException and a replay of wal</span>
<span class="source-line-no">1662</span><span id="line-1662"> // is required. Currently the only way to do this is a restart of</span>
<span class="source-line-no">1663</span><span id="line-1663"> // the server.</span>
<span class="source-line-no">1664</span><span id="line-1664"> server.abort("Replay of WAL required. Forcing server shutdown", ex);</span>
<span class="source-line-no">1665</span><span id="line-1665"> throw new ServiceException(ex);</span>
<span class="source-line-no">1666</span><span id="line-1666"> } catch (IOException ie) {</span>
<span class="source-line-no">1667</span><span id="line-1667"> throw new ServiceException(ie);</span>
<span class="source-line-no">1668</span><span id="line-1668"> }</span>
<span class="source-line-no">1669</span><span id="line-1669"> }</span>
<span class="source-line-no">1670</span><span id="line-1670"></span>
<span class="source-line-no">1671</span><span id="line-1671"> @Override</span>
<span class="source-line-no">1672</span><span id="line-1672"> @QosPriority(priority = HConstants.ADMIN_QOS)</span>
<span class="source-line-no">1673</span><span id="line-1673"> public GetOnlineRegionResponse getOnlineRegion(final RpcController controller,</span>
<span class="source-line-no">1674</span><span id="line-1674"> final GetOnlineRegionRequest request) throws ServiceException {</span>
<span class="source-line-no">1675</span><span id="line-1675"> try {</span>
<span class="source-line-no">1676</span><span id="line-1676"> checkOpen();</span>
<span class="source-line-no">1677</span><span id="line-1677"> requestCount.increment();</span>
<span class="source-line-no">1678</span><span id="line-1678"> Map&lt;String, HRegion&gt; onlineRegions = server.getOnlineRegions();</span>
<span class="source-line-no">1679</span><span id="line-1679"> List&lt;RegionInfo&gt; list = new ArrayList&lt;&gt;(onlineRegions.size());</span>
<span class="source-line-no">1680</span><span id="line-1680"> for (HRegion region : onlineRegions.values()) {</span>
<span class="source-line-no">1681</span><span id="line-1681"> list.add(region.getRegionInfo());</span>
<span class="source-line-no">1682</span><span id="line-1682"> }</span>
<span class="source-line-no">1683</span><span id="line-1683"> list.sort(RegionInfo.COMPARATOR);</span>
<span class="source-line-no">1684</span><span id="line-1684"> return ResponseConverter.buildGetOnlineRegionResponse(list);</span>
<span class="source-line-no">1685</span><span id="line-1685"> } catch (IOException ie) {</span>
<span class="source-line-no">1686</span><span id="line-1686"> throw new ServiceException(ie);</span>
<span class="source-line-no">1687</span><span id="line-1687"> }</span>
<span class="source-line-no">1688</span><span id="line-1688"> }</span>
<span class="source-line-no">1689</span><span id="line-1689"></span>
<span class="source-line-no">1690</span><span id="line-1690"> // Master implementation of this Admin Service differs given it is not</span>
<span class="source-line-no">1691</span><span id="line-1691"> // able to supply detail only known to RegionServer. See note on</span>
<span class="source-line-no">1692</span><span id="line-1692"> // MasterRpcServers#getRegionInfo.</span>
<span class="source-line-no">1693</span><span id="line-1693"> @Override</span>
<span class="source-line-no">1694</span><span id="line-1694"> @QosPriority(priority = HConstants.ADMIN_QOS)</span>
<span class="source-line-no">1695</span><span id="line-1695"> public GetRegionInfoResponse getRegionInfo(final RpcController controller,</span>
<span class="source-line-no">1696</span><span id="line-1696"> final GetRegionInfoRequest request) throws ServiceException {</span>
<span class="source-line-no">1697</span><span id="line-1697"> try {</span>
<span class="source-line-no">1698</span><span id="line-1698"> checkOpen();</span>
<span class="source-line-no">1699</span><span id="line-1699"> requestCount.increment();</span>
<span class="source-line-no">1700</span><span id="line-1700"> HRegion region = getRegion(request.getRegion());</span>
<span class="source-line-no">1701</span><span id="line-1701"> RegionInfo info = region.getRegionInfo();</span>
<span class="source-line-no">1702</span><span id="line-1702"> byte[] bestSplitRow;</span>
<span class="source-line-no">1703</span><span id="line-1703"> if (request.hasBestSplitRow() &amp;&amp; request.getBestSplitRow()) {</span>
<span class="source-line-no">1704</span><span id="line-1704"> bestSplitRow = region.checkSplit(true).orElse(null);</span>
<span class="source-line-no">1705</span><span id="line-1705"> // when all table data are in memstore, bestSplitRow = null</span>
<span class="source-line-no">1706</span><span id="line-1706"> // try to flush region first</span>
<span class="source-line-no">1707</span><span id="line-1707"> if (bestSplitRow == null) {</span>
<span class="source-line-no">1708</span><span id="line-1708"> region.flush(true);</span>
<span class="source-line-no">1709</span><span id="line-1709"> bestSplitRow = region.checkSplit(true).orElse(null);</span>
<span class="source-line-no">1710</span><span id="line-1710"> }</span>
<span class="source-line-no">1711</span><span id="line-1711"> } else {</span>
<span class="source-line-no">1712</span><span id="line-1712"> bestSplitRow = null;</span>
<span class="source-line-no">1713</span><span id="line-1713"> }</span>
<span class="source-line-no">1714</span><span id="line-1714"> GetRegionInfoResponse.Builder builder = GetRegionInfoResponse.newBuilder();</span>
<span class="source-line-no">1715</span><span id="line-1715"> builder.setRegionInfo(ProtobufUtil.toRegionInfo(info));</span>
<span class="source-line-no">1716</span><span id="line-1716"> if (request.hasCompactionState() &amp;&amp; request.getCompactionState()) {</span>
<span class="source-line-no">1717</span><span id="line-1717"> builder.setCompactionState(ProtobufUtil.createCompactionState(region.getCompactionState()));</span>
<span class="source-line-no">1718</span><span id="line-1718"> }</span>
<span class="source-line-no">1719</span><span id="line-1719"> builder.setSplittable(region.isSplittable());</span>
<span class="source-line-no">1720</span><span id="line-1720"> builder.setMergeable(region.isMergeable());</span>
<span class="source-line-no">1721</span><span id="line-1721"> if (request.hasBestSplitRow() &amp;&amp; request.getBestSplitRow() &amp;&amp; bestSplitRow != null) {</span>
<span class="source-line-no">1722</span><span id="line-1722"> builder.setBestSplitRow(UnsafeByteOperations.unsafeWrap(bestSplitRow));</span>
<span class="source-line-no">1723</span><span id="line-1723"> }</span>
<span class="source-line-no">1724</span><span id="line-1724"> return builder.build();</span>
<span class="source-line-no">1725</span><span id="line-1725"> } catch (IOException ie) {</span>
<span class="source-line-no">1726</span><span id="line-1726"> throw new ServiceException(ie);</span>
<span class="source-line-no">1727</span><span id="line-1727"> }</span>
<span class="source-line-no">1728</span><span id="line-1728"> }</span>
<span class="source-line-no">1729</span><span id="line-1729"></span>
<span class="source-line-no">1730</span><span id="line-1730"> @Override</span>
<span class="source-line-no">1731</span><span id="line-1731"> @QosPriority(priority = HConstants.ADMIN_QOS)</span>
<span class="source-line-no">1732</span><span id="line-1732"> public GetRegionLoadResponse getRegionLoad(RpcController controller, GetRegionLoadRequest request)</span>
<span class="source-line-no">1733</span><span id="line-1733"> throws ServiceException {</span>
<span class="source-line-no">1734</span><span id="line-1734"></span>
<span class="source-line-no">1735</span><span id="line-1735"> List&lt;HRegion&gt; regions;</span>
<span class="source-line-no">1736</span><span id="line-1736"> if (request.hasTableName()) {</span>
<span class="source-line-no">1737</span><span id="line-1737"> TableName tableName = ProtobufUtil.toTableName(request.getTableName());</span>
<span class="source-line-no">1738</span><span id="line-1738"> regions = server.getRegions(tableName);</span>
<span class="source-line-no">1739</span><span id="line-1739"> } else {</span>
<span class="source-line-no">1740</span><span id="line-1740"> regions = server.getRegions();</span>
<span class="source-line-no">1741</span><span id="line-1741"> }</span>
<span class="source-line-no">1742</span><span id="line-1742"> List&lt;RegionLoad&gt; rLoads = new ArrayList&lt;&gt;(regions.size());</span>
<span class="source-line-no">1743</span><span id="line-1743"> RegionLoad.Builder regionLoadBuilder = ClusterStatusProtos.RegionLoad.newBuilder();</span>
<span class="source-line-no">1744</span><span id="line-1744"> RegionSpecifier.Builder regionSpecifier = RegionSpecifier.newBuilder();</span>
<span class="source-line-no">1745</span><span id="line-1745"></span>
<span class="source-line-no">1746</span><span id="line-1746"> try {</span>
<span class="source-line-no">1747</span><span id="line-1747"> for (HRegion region : regions) {</span>
<span class="source-line-no">1748</span><span id="line-1748"> rLoads.add(server.createRegionLoad(region, regionLoadBuilder, regionSpecifier));</span>
<span class="source-line-no">1749</span><span id="line-1749"> }</span>
<span class="source-line-no">1750</span><span id="line-1750"> } catch (IOException e) {</span>
<span class="source-line-no">1751</span><span id="line-1751"> throw new ServiceException(e);</span>
<span class="source-line-no">1752</span><span id="line-1752"> }</span>
<span class="source-line-no">1753</span><span id="line-1753"> GetRegionLoadResponse.Builder builder = GetRegionLoadResponse.newBuilder();</span>
<span class="source-line-no">1754</span><span id="line-1754"> builder.addAllRegionLoads(rLoads);</span>
<span class="source-line-no">1755</span><span id="line-1755"> return builder.build();</span>
<span class="source-line-no">1756</span><span id="line-1756"> }</span>
<span class="source-line-no">1757</span><span id="line-1757"></span>
<span class="source-line-no">1758</span><span id="line-1758"> @Override</span>
<span class="source-line-no">1759</span><span id="line-1759"> @QosPriority(priority = HConstants.ADMIN_QOS)</span>
<span class="source-line-no">1760</span><span id="line-1760"> public ClearCompactionQueuesResponse clearCompactionQueues(RpcController controller,</span>
<span class="source-line-no">1761</span><span id="line-1761"> ClearCompactionQueuesRequest request) throws ServiceException {</span>
<span class="source-line-no">1762</span><span id="line-1762"> LOG.debug("Client=" + RpcServer.getRequestUserName().orElse(null) + "/"</span>
<span class="source-line-no">1763</span><span id="line-1763"> + RpcServer.getRemoteAddress().orElse(null) + " clear compactions queue");</span>
<span class="source-line-no">1764</span><span id="line-1764"> ClearCompactionQueuesResponse.Builder respBuilder = ClearCompactionQueuesResponse.newBuilder();</span>
<span class="source-line-no">1765</span><span id="line-1765"> requestCount.increment();</span>
<span class="source-line-no">1766</span><span id="line-1766"> if (clearCompactionQueues.compareAndSet(false, true)) {</span>
<span class="source-line-no">1767</span><span id="line-1767"> final CompactSplit compactSplitThread = server.getCompactSplitThread();</span>
<span class="source-line-no">1768</span><span id="line-1768"> try {</span>
<span class="source-line-no">1769</span><span id="line-1769"> checkOpen();</span>
<span class="source-line-no">1770</span><span id="line-1770"> server.getRegionServerCoprocessorHost().preClearCompactionQueues();</span>
<span class="source-line-no">1771</span><span id="line-1771"> for (String queueName : request.getQueueNameList()) {</span>
<span class="source-line-no">1772</span><span id="line-1772"> LOG.debug("clear " + queueName + " compaction queue");</span>
<span class="source-line-no">1773</span><span id="line-1773"> switch (queueName) {</span>
<span class="source-line-no">1774</span><span id="line-1774"> case "long":</span>
<span class="source-line-no">1775</span><span id="line-1775"> compactSplitThread.clearLongCompactionsQueue();</span>
<span class="source-line-no">1776</span><span id="line-1776"> break;</span>
<span class="source-line-no">1777</span><span id="line-1777"> case "short":</span>
<span class="source-line-no">1778</span><span id="line-1778"> compactSplitThread.clearShortCompactionsQueue();</span>
<span class="source-line-no">1779</span><span id="line-1779"> break;</span>
<span class="source-line-no">1780</span><span id="line-1780"> default:</span>
<span class="source-line-no">1781</span><span id="line-1781"> LOG.warn("Unknown queue name " + queueName);</span>
<span class="source-line-no">1782</span><span id="line-1782"> throw new IOException("Unknown queue name " + queueName);</span>
<span class="source-line-no">1783</span><span id="line-1783"> }</span>
<span class="source-line-no">1784</span><span id="line-1784"> }</span>
<span class="source-line-no">1785</span><span id="line-1785"> server.getRegionServerCoprocessorHost().postClearCompactionQueues();</span>
<span class="source-line-no">1786</span><span id="line-1786"> } catch (IOException ie) {</span>
<span class="source-line-no">1787</span><span id="line-1787"> throw new ServiceException(ie);</span>
<span class="source-line-no">1788</span><span id="line-1788"> } finally {</span>
<span class="source-line-no">1789</span><span id="line-1789"> clearCompactionQueues.set(false);</span>
<span class="source-line-no">1790</span><span id="line-1790"> }</span>
<span class="source-line-no">1791</span><span id="line-1791"> } else {</span>
<span class="source-line-no">1792</span><span id="line-1792"> LOG.warn("Clear compactions queue is executing by other admin.");</span>
<span class="source-line-no">1793</span><span id="line-1793"> }</span>
<span class="source-line-no">1794</span><span id="line-1794"> return respBuilder.build();</span>
<span class="source-line-no">1795</span><span id="line-1795"> }</span>
<span class="source-line-no">1796</span><span id="line-1796"></span>
<span class="source-line-no">1797</span><span id="line-1797"> /**</span>
<span class="source-line-no">1798</span><span id="line-1798"> * Get some information of the region server.</span>
<span class="source-line-no">1799</span><span id="line-1799"> * @param controller the RPC controller</span>
<span class="source-line-no">1800</span><span id="line-1800"> * @param request the request</span>
<span class="source-line-no">1801</span><span id="line-1801"> */</span>
<span class="source-line-no">1802</span><span id="line-1802"> @Override</span>
<span class="source-line-no">1803</span><span id="line-1803"> @QosPriority(priority = HConstants.ADMIN_QOS)</span>
<span class="source-line-no">1804</span><span id="line-1804"> public GetServerInfoResponse getServerInfo(final RpcController controller,</span>
<span class="source-line-no">1805</span><span id="line-1805"> final GetServerInfoRequest request) throws ServiceException {</span>
<span class="source-line-no">1806</span><span id="line-1806"> try {</span>
<span class="source-line-no">1807</span><span id="line-1807"> checkOpen();</span>
<span class="source-line-no">1808</span><span id="line-1808"> } catch (IOException ie) {</span>
<span class="source-line-no">1809</span><span id="line-1809"> throw new ServiceException(ie);</span>
<span class="source-line-no">1810</span><span id="line-1810"> }</span>
<span class="source-line-no">1811</span><span id="line-1811"> requestCount.increment();</span>
<span class="source-line-no">1812</span><span id="line-1812"> int infoPort = server.getInfoServer() != null ? server.getInfoServer().getPort() : -1;</span>
<span class="source-line-no">1813</span><span id="line-1813"> return ResponseConverter.buildGetServerInfoResponse(server.getServerName(), infoPort);</span>
<span class="source-line-no">1814</span><span id="line-1814"> }</span>
<span class="source-line-no">1815</span><span id="line-1815"></span>
<span class="source-line-no">1816</span><span id="line-1816"> @Override</span>
<span class="source-line-no">1817</span><span id="line-1817"> @QosPriority(priority = HConstants.ADMIN_QOS)</span>
<span class="source-line-no">1818</span><span id="line-1818"> public GetStoreFileResponse getStoreFile(final RpcController controller,</span>
<span class="source-line-no">1819</span><span id="line-1819"> final GetStoreFileRequest request) throws ServiceException {</span>
<span class="source-line-no">1820</span><span id="line-1820"> try {</span>
<span class="source-line-no">1821</span><span id="line-1821"> checkOpen();</span>
<span class="source-line-no">1822</span><span id="line-1822"> HRegion region = getRegion(request.getRegion());</span>
<span class="source-line-no">1823</span><span id="line-1823"> requestCount.increment();</span>
<span class="source-line-no">1824</span><span id="line-1824"> Set&lt;byte[]&gt; columnFamilies;</span>
<span class="source-line-no">1825</span><span id="line-1825"> if (request.getFamilyCount() == 0) {</span>
<span class="source-line-no">1826</span><span id="line-1826"> columnFamilies = region.getTableDescriptor().getColumnFamilyNames();</span>
<span class="source-line-no">1827</span><span id="line-1827"> } else {</span>
<span class="source-line-no">1828</span><span id="line-1828"> columnFamilies = new TreeSet&lt;&gt;(Bytes.BYTES_RAWCOMPARATOR);</span>
<span class="source-line-no">1829</span><span id="line-1829"> for (ByteString cf : request.getFamilyList()) {</span>
<span class="source-line-no">1830</span><span id="line-1830"> columnFamilies.add(cf.toByteArray());</span>
<span class="source-line-no">1831</span><span id="line-1831"> }</span>
<span class="source-line-no">1832</span><span id="line-1832"> }</span>
<span class="source-line-no">1833</span><span id="line-1833"> int nCF = columnFamilies.size();</span>
<span class="source-line-no">1834</span><span id="line-1834"> List&lt;String&gt; fileList = region.getStoreFileList(columnFamilies.toArray(new byte[nCF][]));</span>
<span class="source-line-no">1835</span><span id="line-1835"> GetStoreFileResponse.Builder builder = GetStoreFileResponse.newBuilder();</span>
<span class="source-line-no">1836</span><span id="line-1836"> builder.addAllStoreFile(fileList);</span>
<span class="source-line-no">1837</span><span id="line-1837"> return builder.build();</span>
<span class="source-line-no">1838</span><span id="line-1838"> } catch (IOException ie) {</span>
<span class="source-line-no">1839</span><span id="line-1839"> throw new ServiceException(ie);</span>
<span class="source-line-no">1840</span><span id="line-1840"> }</span>
<span class="source-line-no">1841</span><span id="line-1841"> }</span>
<span class="source-line-no">1842</span><span id="line-1842"></span>
<span class="source-line-no">1843</span><span id="line-1843"> private void throwOnWrongStartCode(OpenRegionRequest request) throws ServiceException {</span>
<span class="source-line-no">1844</span><span id="line-1844"> if (!request.hasServerStartCode()) {</span>
<span class="source-line-no">1845</span><span id="line-1845"> LOG.warn("OpenRegionRequest for {} does not have a start code", request.getOpenInfoList());</span>
<span class="source-line-no">1846</span><span id="line-1846"> return;</span>
<span class="source-line-no">1847</span><span id="line-1847"> }</span>
<span class="source-line-no">1848</span><span id="line-1848"> throwOnWrongStartCode(request.getServerStartCode());</span>
<span class="source-line-no">1849</span><span id="line-1849"> }</span>
<span class="source-line-no">1850</span><span id="line-1850"></span>
<span class="source-line-no">1851</span><span id="line-1851"> private void throwOnWrongStartCode(CloseRegionRequest request) throws ServiceException {</span>
<span class="source-line-no">1852</span><span id="line-1852"> if (!request.hasServerStartCode()) {</span>
<span class="source-line-no">1853</span><span id="line-1853"> LOG.warn("CloseRegionRequest for {} does not have a start code", request.getRegion());</span>
<span class="source-line-no">1854</span><span id="line-1854"> return;</span>
<span class="source-line-no">1855</span><span id="line-1855"> }</span>
<span class="source-line-no">1856</span><span id="line-1856"> throwOnWrongStartCode(request.getServerStartCode());</span>
<span class="source-line-no">1857</span><span id="line-1857"> }</span>
<span class="source-line-no">1858</span><span id="line-1858"></span>
<span class="source-line-no">1859</span><span id="line-1859"> private void throwOnWrongStartCode(long serverStartCode) throws ServiceException {</span>
<span class="source-line-no">1860</span><span id="line-1860"> // check that we are the same server that this RPC is intended for.</span>
<span class="source-line-no">1861</span><span id="line-1861"> if (server.getServerName().getStartcode() != serverStartCode) {</span>
<span class="source-line-no">1862</span><span id="line-1862"> throw new ServiceException(new DoNotRetryIOException(</span>
<span class="source-line-no">1863</span><span id="line-1863"> "This RPC was intended for a " + "different server with startCode: " + serverStartCode</span>
<span class="source-line-no">1864</span><span id="line-1864"> + ", this server is: " + server.getServerName()));</span>
<span class="source-line-no">1865</span><span id="line-1865"> }</span>
<span class="source-line-no">1866</span><span id="line-1866"> }</span>
<span class="source-line-no">1867</span><span id="line-1867"></span>
<span class="source-line-no">1868</span><span id="line-1868"> private void throwOnWrongStartCode(ExecuteProceduresRequest req) throws ServiceException {</span>
<span class="source-line-no">1869</span><span id="line-1869"> if (req.getOpenRegionCount() &gt; 0) {</span>
<span class="source-line-no">1870</span><span id="line-1870"> for (OpenRegionRequest openReq : req.getOpenRegionList()) {</span>
<span class="source-line-no">1871</span><span id="line-1871"> throwOnWrongStartCode(openReq);</span>
<span class="source-line-no">1872</span><span id="line-1872"> }</span>
<span class="source-line-no">1873</span><span id="line-1873"> }</span>
<span class="source-line-no">1874</span><span id="line-1874"> if (req.getCloseRegionCount() &gt; 0) {</span>
<span class="source-line-no">1875</span><span id="line-1875"> for (CloseRegionRequest closeReq : req.getCloseRegionList()) {</span>
<span class="source-line-no">1876</span><span id="line-1876"> throwOnWrongStartCode(closeReq);</span>
<span class="source-line-no">1877</span><span id="line-1877"> }</span>
<span class="source-line-no">1878</span><span id="line-1878"> }</span>
<span class="source-line-no">1879</span><span id="line-1879"> }</span>
<span class="source-line-no">1880</span><span id="line-1880"></span>
<span class="source-line-no">1881</span><span id="line-1881"> /**</span>
<span class="source-line-no">1882</span><span id="line-1882"> * Open asynchronously a region or a set of regions on the region server. The opening is</span>
<span class="source-line-no">1883</span><span id="line-1883"> * coordinated by ZooKeeper, and this method requires the znode to be created before being called.</span>
<span class="source-line-no">1884</span><span id="line-1884"> * As a consequence, this method should be called only from the master.</span>
<span class="source-line-no">1885</span><span id="line-1885"> * &lt;p&gt;</span>
<span class="source-line-no">1886</span><span id="line-1886"> * Different manages states for the region are:</span>
<span class="source-line-no">1887</span><span id="line-1887"> * &lt;/p&gt;</span>
<span class="source-line-no">1888</span><span id="line-1888"> * &lt;ul&gt;</span>
<span class="source-line-no">1889</span><span id="line-1889"> * &lt;li&gt;region not opened: the region opening will start asynchronously.&lt;/li&gt;</span>
<span class="source-line-no">1890</span><span id="line-1890"> * &lt;li&gt;a close is already in progress: this is considered as an error.&lt;/li&gt;</span>
<span class="source-line-no">1891</span><span id="line-1891"> * &lt;li&gt;an open is already in progress: this new open request will be ignored. This is important</span>
<span class="source-line-no">1892</span><span id="line-1892"> * because the Master can do multiple requests if it crashes.&lt;/li&gt;</span>
<span class="source-line-no">1893</span><span id="line-1893"> * &lt;li&gt;the region is already opened: this new open request will be ignored.&lt;/li&gt;</span>
<span class="source-line-no">1894</span><span id="line-1894"> * &lt;/ul&gt;</span>
<span class="source-line-no">1895</span><span id="line-1895"> * &lt;p&gt;</span>
<span class="source-line-no">1896</span><span id="line-1896"> * Bulk assign: If there are more than 1 region to open, it will be considered as a bulk assign.</span>
<span class="source-line-no">1897</span><span id="line-1897"> * For a single region opening, errors are sent through a ServiceException. For bulk assign,</span>
<span class="source-line-no">1898</span><span id="line-1898"> * errors are put in the response as FAILED_OPENING.</span>
<span class="source-line-no">1899</span><span id="line-1899"> * &lt;/p&gt;</span>
<span class="source-line-no">1900</span><span id="line-1900"> * @param controller the RPC controller</span>
<span class="source-line-no">1901</span><span id="line-1901"> * @param request the request</span>
<span class="source-line-no">1902</span><span id="line-1902"> */</span>
<span class="source-line-no">1903</span><span id="line-1903"> @Override</span>
<span class="source-line-no">1904</span><span id="line-1904"> @QosPriority(priority = HConstants.ADMIN_QOS)</span>
<span class="source-line-no">1905</span><span id="line-1905"> public OpenRegionResponse openRegion(final RpcController controller,</span>
<span class="source-line-no">1906</span><span id="line-1906"> final OpenRegionRequest request) throws ServiceException {</span>
<span class="source-line-no">1907</span><span id="line-1907"> requestCount.increment();</span>
<span class="source-line-no">1908</span><span id="line-1908"> throwOnWrongStartCode(request);</span>
<span class="source-line-no">1909</span><span id="line-1909"></span>
<span class="source-line-no">1910</span><span id="line-1910"> OpenRegionResponse.Builder builder = OpenRegionResponse.newBuilder();</span>
<span class="source-line-no">1911</span><span id="line-1911"> final int regionCount = request.getOpenInfoCount();</span>
<span class="source-line-no">1912</span><span id="line-1912"> final Map&lt;TableName, TableDescriptor&gt; htds = new HashMap&lt;&gt;(regionCount);</span>
<span class="source-line-no">1913</span><span id="line-1913"> final boolean isBulkAssign = regionCount &gt; 1;</span>
<span class="source-line-no">1914</span><span id="line-1914"> try {</span>
<span class="source-line-no">1915</span><span id="line-1915"> checkOpen();</span>
<span class="source-line-no">1916</span><span id="line-1916"> } catch (IOException ie) {</span>
<span class="source-line-no">1917</span><span id="line-1917"> TableName tableName = null;</span>
<span class="source-line-no">1918</span><span id="line-1918"> if (regionCount == 1) {</span>
<span class="source-line-no">1919</span><span id="line-1919"> org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo ri =</span>
<span class="source-line-no">1920</span><span id="line-1920"> request.getOpenInfo(0).getRegion();</span>
<span class="source-line-no">1921</span><span id="line-1921"> if (ri != null) {</span>
<span class="source-line-no">1922</span><span id="line-1922"> tableName = ProtobufUtil.toTableName(ri.getTableName());</span>
<span class="source-line-no">1923</span><span id="line-1923"> }</span>
<span class="source-line-no">1924</span><span id="line-1924"> }</span>
<span class="source-line-no">1925</span><span id="line-1925"> if (!TableName.META_TABLE_NAME.equals(tableName)) {</span>
<span class="source-line-no">1926</span><span id="line-1926"> throw new ServiceException(ie);</span>
<span class="source-line-no">1927</span><span id="line-1927"> }</span>
<span class="source-line-no">1928</span><span id="line-1928"> // We are assigning meta, wait a little for regionserver to finish initialization.</span>
<span class="source-line-no">1929</span><span id="line-1929"> // Default to quarter of RPC timeout</span>
<span class="source-line-no">1930</span><span id="line-1930"> int timeout = server.getConfiguration().getInt(HConstants.HBASE_RPC_TIMEOUT_KEY,</span>
<span class="source-line-no">1931</span><span id="line-1931"> HConstants.DEFAULT_HBASE_RPC_TIMEOUT) &gt;&gt; 2;</span>
<span class="source-line-no">1932</span><span id="line-1932"> long endTime = EnvironmentEdgeManager.currentTime() + timeout;</span>
<span class="source-line-no">1933</span><span id="line-1933"> synchronized (server.online) {</span>
<span class="source-line-no">1934</span><span id="line-1934"> try {</span>
<span class="source-line-no">1935</span><span id="line-1935"> while (</span>
<span class="source-line-no">1936</span><span id="line-1936"> EnvironmentEdgeManager.currentTime() &lt;= endTime &amp;&amp; !server.isStopped()</span>
<span class="source-line-no">1937</span><span id="line-1937"> &amp;&amp; !server.isOnline()</span>
<span class="source-line-no">1938</span><span id="line-1938"> ) {</span>
<span class="source-line-no">1939</span><span id="line-1939"> server.online.wait(server.getMsgInterval());</span>
<span class="source-line-no">1940</span><span id="line-1940"> }</span>
<span class="source-line-no">1941</span><span id="line-1941"> checkOpen();</span>
<span class="source-line-no">1942</span><span id="line-1942"> } catch (InterruptedException t) {</span>
<span class="source-line-no">1943</span><span id="line-1943"> Thread.currentThread().interrupt();</span>
<span class="source-line-no">1944</span><span id="line-1944"> throw new ServiceException(t);</span>
<span class="source-line-no">1945</span><span id="line-1945"> } catch (IOException e) {</span>
<span class="source-line-no">1946</span><span id="line-1946"> throw new ServiceException(e);</span>
<span class="source-line-no">1947</span><span id="line-1947"> }</span>
<span class="source-line-no">1948</span><span id="line-1948"> }</span>
<span class="source-line-no">1949</span><span id="line-1949"> }</span>
<span class="source-line-no">1950</span><span id="line-1950"></span>
<span class="source-line-no">1951</span><span id="line-1951"> long masterSystemTime = request.hasMasterSystemTime() ? request.getMasterSystemTime() : -1;</span>
<span class="source-line-no">1952</span><span id="line-1952"></span>
<span class="source-line-no">1953</span><span id="line-1953"> for (RegionOpenInfo regionOpenInfo : request.getOpenInfoList()) {</span>
<span class="source-line-no">1954</span><span id="line-1954"> final RegionInfo region = ProtobufUtil.toRegionInfo(regionOpenInfo.getRegion());</span>
<span class="source-line-no">1955</span><span id="line-1955"> TableDescriptor htd;</span>
<span class="source-line-no">1956</span><span id="line-1956"> try {</span>
<span class="source-line-no">1957</span><span id="line-1957"> String encodedName = region.getEncodedName();</span>
<span class="source-line-no">1958</span><span id="line-1958"> byte[] encodedNameBytes = region.getEncodedNameAsBytes();</span>
<span class="source-line-no">1959</span><span id="line-1959"> final HRegion onlineRegion = server.getRegion(encodedName);</span>
<span class="source-line-no">1960</span><span id="line-1960"> if (onlineRegion != null) {</span>
<span class="source-line-no">1961</span><span id="line-1961"> // The region is already online. This should not happen any more.</span>
<span class="source-line-no">1962</span><span id="line-1962"> String error = "Received OPEN for the region:" + region.getRegionNameAsString()</span>
<span class="source-line-no">1963</span><span id="line-1963"> + ", which is already online";</span>
<span class="source-line-no">1964</span><span id="line-1964"> LOG.warn(error);</span>
<span class="source-line-no">1965</span><span id="line-1965"> // server.abort(error);</span>
<span class="source-line-no">1966</span><span id="line-1966"> // throw new IOException(error);</span>
<span class="source-line-no">1967</span><span id="line-1967"> builder.addOpeningState(RegionOpeningState.OPENED);</span>
<span class="source-line-no">1968</span><span id="line-1968"> continue;</span>
<span class="source-line-no">1969</span><span id="line-1969"> }</span>
<span class="source-line-no">1970</span><span id="line-1970"> LOG.info("Open " + region.getRegionNameAsString());</span>
<span class="source-line-no">1971</span><span id="line-1971"></span>
<span class="source-line-no">1972</span><span id="line-1972"> final Boolean previous =</span>
<span class="source-line-no">1973</span><span id="line-1973"> server.getRegionsInTransitionInRS().putIfAbsent(encodedNameBytes, Boolean.TRUE);</span>
<span class="source-line-no">1974</span><span id="line-1974"></span>
<span class="source-line-no">1975</span><span id="line-1975"> if (Boolean.FALSE.equals(previous)) {</span>
<span class="source-line-no">1976</span><span id="line-1976"> if (server.getRegion(encodedName) != null) {</span>
<span class="source-line-no">1977</span><span id="line-1977"> // There is a close in progress. This should not happen any more.</span>
<span class="source-line-no">1978</span><span id="line-1978"> String error = "Received OPEN for the region:" + region.getRegionNameAsString()</span>
<span class="source-line-no">1979</span><span id="line-1979"> + ", which we are already trying to CLOSE";</span>
<span class="source-line-no">1980</span><span id="line-1980"> server.abort(error);</span>
<span class="source-line-no">1981</span><span id="line-1981"> throw new IOException(error);</span>
<span class="source-line-no">1982</span><span id="line-1982"> }</span>
<span class="source-line-no">1983</span><span id="line-1983"> server.getRegionsInTransitionInRS().put(encodedNameBytes, Boolean.TRUE);</span>
<span class="source-line-no">1984</span><span id="line-1984"> }</span>
<span class="source-line-no">1985</span><span id="line-1985"></span>
<span class="source-line-no">1986</span><span id="line-1986"> if (Boolean.TRUE.equals(previous)) {</span>
<span class="source-line-no">1987</span><span id="line-1987"> // An open is in progress. This is supported, but let's log this.</span>
<span class="source-line-no">1988</span><span id="line-1988"> LOG.info("Receiving OPEN for the region:" + region.getRegionNameAsString()</span>
<span class="source-line-no">1989</span><span id="line-1989"> + ", which we are already trying to OPEN"</span>
<span class="source-line-no">1990</span><span id="line-1990"> + " - ignoring this new request for this region.");</span>
<span class="source-line-no">1991</span><span id="line-1991"> }</span>
<span class="source-line-no">1992</span><span id="line-1992"></span>
<span class="source-line-no">1993</span><span id="line-1993"> // We are opening this region. If it moves back and forth for whatever reason, we don't</span>
<span class="source-line-no">1994</span><span id="line-1994"> // want to keep returning the stale moved record while we are opening/if we close again.</span>
<span class="source-line-no">1995</span><span id="line-1995"> server.removeFromMovedRegions(region.getEncodedName());</span>
<span class="source-line-no">1996</span><span id="line-1996"></span>
<span class="source-line-no">1997</span><span id="line-1997"> if (previous == null || !previous.booleanValue()) {</span>
<span class="source-line-no">1998</span><span id="line-1998"> htd = htds.get(region.getTable());</span>
<span class="source-line-no">1999</span><span id="line-1999"> if (htd == null) {</span>
<span class="source-line-no">2000</span><span id="line-2000"> htd = server.getTableDescriptors().get(region.getTable());</span>
<span class="source-line-no">2001</span><span id="line-2001"> htds.put(region.getTable(), htd);</span>
<span class="source-line-no">2002</span><span id="line-2002"> }</span>
<span class="source-line-no">2003</span><span id="line-2003"> if (htd == null) {</span>
<span class="source-line-no">2004</span><span id="line-2004"> throw new IOException("Missing table descriptor for " + region.getEncodedName());</span>
<span class="source-line-no">2005</span><span id="line-2005"> }</span>
<span class="source-line-no">2006</span><span id="line-2006"> // If there is no action in progress, we can submit a specific handler.</span>
<span class="source-line-no">2007</span><span id="line-2007"> // Need to pass the expected version in the constructor.</span>
<span class="source-line-no">2008</span><span id="line-2008"> if (server.getExecutorService() == null) {</span>
<span class="source-line-no">2009</span><span id="line-2009"> LOG.info("No executor executorService; skipping open request");</span>
<span class="source-line-no">2010</span><span id="line-2010"> } else {</span>
<span class="source-line-no">2011</span><span id="line-2011"> if (region.isMetaRegion()) {</span>
<span class="source-line-no">2012</span><span id="line-2012"> server.getExecutorService()</span>
<span class="source-line-no">2013</span><span id="line-2013"> .submit(new OpenMetaHandler(server, server, region, htd, masterSystemTime));</span>
<span class="source-line-no">2014</span><span id="line-2014"> } else {</span>
<span class="source-line-no">2015</span><span id="line-2015"> if (regionOpenInfo.getFavoredNodesCount() &gt; 0) {</span>
<span class="source-line-no">2016</span><span id="line-2016"> server.updateRegionFavoredNodesMapping(region.getEncodedName(),</span>
<span class="source-line-no">2017</span><span id="line-2017"> regionOpenInfo.getFavoredNodesList());</span>
<span class="source-line-no">2018</span><span id="line-2018"> }</span>
<span class="source-line-no">2019</span><span id="line-2019"> if (htd.getPriority() &gt;= HConstants.ADMIN_QOS || region.getTable().isSystemTable()) {</span>
<span class="source-line-no">2020</span><span id="line-2020"> server.getExecutorService().submit(</span>
<span class="source-line-no">2021</span><span id="line-2021"> new OpenPriorityRegionHandler(server, server, region, htd, masterSystemTime));</span>
<span class="source-line-no">2022</span><span id="line-2022"> } else {</span>
<span class="source-line-no">2023</span><span id="line-2023"> server.getExecutorService()</span>
<span class="source-line-no">2024</span><span id="line-2024"> .submit(new OpenRegionHandler(server, server, region, htd, masterSystemTime));</span>
<span class="source-line-no">2025</span><span id="line-2025"> }</span>
<span class="source-line-no">2026</span><span id="line-2026"> }</span>
<span class="source-line-no">2027</span><span id="line-2027"> }</span>
<span class="source-line-no">2028</span><span id="line-2028"> }</span>
<span class="source-line-no">2029</span><span id="line-2029"></span>
<span class="source-line-no">2030</span><span id="line-2030"> builder.addOpeningState(RegionOpeningState.OPENED);</span>
<span class="source-line-no">2031</span><span id="line-2031"> } catch (IOException ie) {</span>
<span class="source-line-no">2032</span><span id="line-2032"> LOG.warn("Failed opening region " + region.getRegionNameAsString(), ie);</span>
<span class="source-line-no">2033</span><span id="line-2033"> if (isBulkAssign) {</span>
<span class="source-line-no">2034</span><span id="line-2034"> builder.addOpeningState(RegionOpeningState.FAILED_OPENING);</span>
<span class="source-line-no">2035</span><span id="line-2035"> } else {</span>
<span class="source-line-no">2036</span><span id="line-2036"> throw new ServiceException(ie);</span>
<span class="source-line-no">2037</span><span id="line-2037"> }</span>
<span class="source-line-no">2038</span><span id="line-2038"> }</span>
<span class="source-line-no">2039</span><span id="line-2039"> }</span>
<span class="source-line-no">2040</span><span id="line-2040"> return builder.build();</span>
<span class="source-line-no">2041</span><span id="line-2041"> }</span>
<span class="source-line-no">2042</span><span id="line-2042"></span>
<span class="source-line-no">2043</span><span id="line-2043"> /**</span>
<span class="source-line-no">2044</span><span id="line-2044"> * Warmup a region on this server. This method should only be called by Master. It synchronously</span>
<span class="source-line-no">2045</span><span id="line-2045"> * opens the region and closes the region bringing the most important pages in cache.</span>
<span class="source-line-no">2046</span><span id="line-2046"> */</span>
<span class="source-line-no">2047</span><span id="line-2047"> @Override</span>
<span class="source-line-no">2048</span><span id="line-2048"> public WarmupRegionResponse warmupRegion(final RpcController controller,</span>
<span class="source-line-no">2049</span><span id="line-2049"> final WarmupRegionRequest request) throws ServiceException {</span>
<span class="source-line-no">2050</span><span id="line-2050"> final RegionInfo region = ProtobufUtil.toRegionInfo(request.getRegionInfo());</span>
<span class="source-line-no">2051</span><span id="line-2051"> WarmupRegionResponse response = WarmupRegionResponse.getDefaultInstance();</span>
<span class="source-line-no">2052</span><span id="line-2052"> try {</span>
<span class="source-line-no">2053</span><span id="line-2053"> checkOpen();</span>
<span class="source-line-no">2054</span><span id="line-2054"> String encodedName = region.getEncodedName();</span>
<span class="source-line-no">2055</span><span id="line-2055"> byte[] encodedNameBytes = region.getEncodedNameAsBytes();</span>
<span class="source-line-no">2056</span><span id="line-2056"> final HRegion onlineRegion = server.getRegion(encodedName);</span>
<span class="source-line-no">2057</span><span id="line-2057"> if (onlineRegion != null) {</span>
<span class="source-line-no">2058</span><span id="line-2058"> LOG.info("{} is online; skipping warmup", region);</span>
<span class="source-line-no">2059</span><span id="line-2059"> return response;</span>
<span class="source-line-no">2060</span><span id="line-2060"> }</span>
<span class="source-line-no">2061</span><span id="line-2061"> TableDescriptor htd = server.getTableDescriptors().get(region.getTable());</span>
<span class="source-line-no">2062</span><span id="line-2062"> if (server.getRegionsInTransitionInRS().containsKey(encodedNameBytes)) {</span>
<span class="source-line-no">2063</span><span id="line-2063"> LOG.info("{} is in transition; skipping warmup", region);</span>
<span class="source-line-no">2064</span><span id="line-2064"> return response;</span>
<span class="source-line-no">2065</span><span id="line-2065"> }</span>
<span class="source-line-no">2066</span><span id="line-2066"> LOG.info("Warmup {}", region.getRegionNameAsString());</span>
<span class="source-line-no">2067</span><span id="line-2067"> HRegion.warmupHRegion(region, htd, server.getWAL(region), server.getConfiguration(), server,</span>
<span class="source-line-no">2068</span><span id="line-2068"> null);</span>
<span class="source-line-no">2069</span><span id="line-2069"> } catch (IOException ie) {</span>
<span class="source-line-no">2070</span><span id="line-2070"> LOG.error("Failed warmup of {}", region.getRegionNameAsString(), ie);</span>
<span class="source-line-no">2071</span><span id="line-2071"> throw new ServiceException(ie);</span>
<span class="source-line-no">2072</span><span id="line-2072"> }</span>
<span class="source-line-no">2073</span><span id="line-2073"></span>
<span class="source-line-no">2074</span><span id="line-2074"> return response;</span>
<span class="source-line-no">2075</span><span id="line-2075"> }</span>
<span class="source-line-no">2076</span><span id="line-2076"></span>
<span class="source-line-no">2077</span><span id="line-2077"> private ExtendedCellScanner getAndReset(RpcController controller) {</span>
<span class="source-line-no">2078</span><span id="line-2078"> HBaseRpcController hrc = (HBaseRpcController) controller;</span>
<span class="source-line-no">2079</span><span id="line-2079"> ExtendedCellScanner cells = hrc.cellScanner();</span>
<span class="source-line-no">2080</span><span id="line-2080"> hrc.setCellScanner(null);</span>
<span class="source-line-no">2081</span><span id="line-2081"> return cells;</span>
<span class="source-line-no">2082</span><span id="line-2082"> }</span>
<span class="source-line-no">2083</span><span id="line-2083"></span>
<span class="source-line-no">2084</span><span id="line-2084"> /**</span>
<span class="source-line-no">2085</span><span id="line-2085"> * Replay the given changes when distributedLogReplay WAL edits from a failed RS. The guarantee is</span>
<span class="source-line-no">2086</span><span id="line-2086"> * that the given mutations will be durable on the receiving RS if this method returns without any</span>
<span class="source-line-no">2087</span><span id="line-2087"> * exception.</span>
<span class="source-line-no">2088</span><span id="line-2088"> * @param controller the RPC controller</span>
<span class="source-line-no">2089</span><span id="line-2089"> * @param request the request</span>
<span class="source-line-no">2090</span><span id="line-2090"> * @deprecated Since 3.0.0, will be removed in 4.0.0. Not used any more, put here only for</span>
<span class="source-line-no">2091</span><span id="line-2091"> * compatibility with old region replica implementation. Now we will use</span>
<span class="source-line-no">2092</span><span id="line-2092"> * {@code replicateToReplica} method instead.</span>
<span class="source-line-no">2093</span><span id="line-2093"> */</span>
<span class="source-line-no">2094</span><span id="line-2094"> @Deprecated</span>
<span class="source-line-no">2095</span><span id="line-2095"> @Override</span>
<span class="source-line-no">2096</span><span id="line-2096"> @QosPriority(priority = HConstants.REPLAY_QOS)</span>
<span class="source-line-no">2097</span><span id="line-2097"> public ReplicateWALEntryResponse replay(final RpcController controller,</span>
<span class="source-line-no">2098</span><span id="line-2098"> final ReplicateWALEntryRequest request) throws ServiceException {</span>
<span class="source-line-no">2099</span><span id="line-2099"> long before = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">2100</span><span id="line-2100"> ExtendedCellScanner cells = getAndReset(controller);</span>
<span class="source-line-no">2101</span><span id="line-2101"> try {</span>
<span class="source-line-no">2102</span><span id="line-2102"> checkOpen();</span>
<span class="source-line-no">2103</span><span id="line-2103"> List&lt;WALEntry&gt; entries = request.getEntryList();</span>
<span class="source-line-no">2104</span><span id="line-2104"> if (entries == null || entries.isEmpty()) {</span>
<span class="source-line-no">2105</span><span id="line-2105"> // empty input</span>
<span class="source-line-no">2106</span><span id="line-2106"> return ReplicateWALEntryResponse.newBuilder().build();</span>
<span class="source-line-no">2107</span><span id="line-2107"> }</span>
<span class="source-line-no">2108</span><span id="line-2108"> ByteString regionName = entries.get(0).getKey().getEncodedRegionName();</span>
<span class="source-line-no">2109</span><span id="line-2109"> HRegion region = server.getRegionByEncodedName(regionName.toStringUtf8());</span>
<span class="source-line-no">2110</span><span id="line-2110"> RegionCoprocessorHost coprocessorHost =</span>
<span class="source-line-no">2111</span><span id="line-2111"> ServerRegionReplicaUtil.isDefaultReplica(region.getRegionInfo())</span>
<span class="source-line-no">2112</span><span id="line-2112"> ? region.getCoprocessorHost()</span>
<span class="source-line-no">2113</span><span id="line-2113"> : null; // do not invoke coprocessors if this is a secondary region replica</span>
<span class="source-line-no">2114</span><span id="line-2114"></span>
<span class="source-line-no">2115</span><span id="line-2115"> // Skip adding the edits to WAL if this is a secondary region replica</span>
<span class="source-line-no">2116</span><span id="line-2116"> boolean isPrimary = RegionReplicaUtil.isDefaultReplica(region.getRegionInfo());</span>
<span class="source-line-no">2117</span><span id="line-2117"> Durability durability = isPrimary ? Durability.USE_DEFAULT : Durability.SKIP_WAL;</span>
<span class="source-line-no">2118</span><span id="line-2118"></span>
<span class="source-line-no">2119</span><span id="line-2119"> for (WALEntry entry : entries) {</span>
<span class="source-line-no">2120</span><span id="line-2120"> if (!regionName.equals(entry.getKey().getEncodedRegionName())) {</span>
<span class="source-line-no">2121</span><span id="line-2121"> throw new NotServingRegionException("Replay request contains entries from multiple "</span>
<span class="source-line-no">2122</span><span id="line-2122"> + "regions. First region:" + regionName.toStringUtf8() + " , other region:"</span>
<span class="source-line-no">2123</span><span id="line-2123"> + entry.getKey().getEncodedRegionName());</span>
<span class="source-line-no">2124</span><span id="line-2124"> }</span>
<span class="source-line-no">2125</span><span id="line-2125"> if (server.nonceManager != null &amp;&amp; isPrimary) {</span>
<span class="source-line-no">2126</span><span id="line-2126"> long nonceGroup =</span>
<span class="source-line-no">2127</span><span id="line-2127"> entry.getKey().hasNonceGroup() ? entry.getKey().getNonceGroup() : HConstants.NO_NONCE;</span>
<span class="source-line-no">2128</span><span id="line-2128"> long nonce = entry.getKey().hasNonce() ? entry.getKey().getNonce() : HConstants.NO_NONCE;</span>
<span class="source-line-no">2129</span><span id="line-2129"> server.nonceManager.reportOperationFromWal(nonceGroup, nonce,</span>
<span class="source-line-no">2130</span><span id="line-2130"> entry.getKey().getWriteTime());</span>
<span class="source-line-no">2131</span><span id="line-2131"> }</span>
<span class="source-line-no">2132</span><span id="line-2132"> Pair&lt;WALKey, WALEdit&gt; walEntry = (coprocessorHost == null) ? null : new Pair&lt;&gt;();</span>
<span class="source-line-no">2133</span><span id="line-2133"> List&lt;MutationReplay&gt; edits =</span>
<span class="source-line-no">2134</span><span id="line-2134"> WALSplitUtil.getMutationsFromWALEntry(entry, cells, walEntry, durability);</span>
<span class="source-line-no">2135</span><span id="line-2135"> if (edits != null &amp;&amp; !edits.isEmpty()) {</span>
<span class="source-line-no">2136</span><span id="line-2136"> // HBASE-17924</span>
<span class="source-line-no">2137</span><span id="line-2137"> // sort to improve lock efficiency</span>
<span class="source-line-no">2138</span><span id="line-2138"> Collections.sort(edits, (v1, v2) -&gt; Row.COMPARATOR.compare(v1.mutation, v2.mutation));</span>
<span class="source-line-no">2139</span><span id="line-2139"> long replaySeqId = (entry.getKey().hasOrigSequenceNumber())</span>
<span class="source-line-no">2140</span><span id="line-2140"> ? entry.getKey().getOrigSequenceNumber()</span>
<span class="source-line-no">2141</span><span id="line-2141"> : entry.getKey().getLogSequenceNumber();</span>
<span class="source-line-no">2142</span><span id="line-2142"> OperationStatus[] result = doReplayBatchOp(region, edits, replaySeqId);</span>
<span class="source-line-no">2143</span><span id="line-2143"> // check if it's a partial success</span>
<span class="source-line-no">2144</span><span id="line-2144"> for (int i = 0; result != null &amp;&amp; i &lt; result.length; i++) {</span>
<span class="source-line-no">2145</span><span id="line-2145"> if (result[i] != OperationStatus.SUCCESS) {</span>
<span class="source-line-no">2146</span><span id="line-2146"> throw new IOException(result[i].getExceptionMsg());</span>
<span class="source-line-no">2147</span><span id="line-2147"> }</span>
<span class="source-line-no">2148</span><span id="line-2148"> }</span>
<span class="source-line-no">2149</span><span id="line-2149"> }</span>
<span class="source-line-no">2150</span><span id="line-2150"> }</span>
<span class="source-line-no">2151</span><span id="line-2151"></span>
<span class="source-line-no">2152</span><span id="line-2152"> // sync wal at the end because ASYNC_WAL is used above</span>
<span class="source-line-no">2153</span><span id="line-2153"> WAL wal = region.getWAL();</span>
<span class="source-line-no">2154</span><span id="line-2154"> if (wal != null) {</span>
<span class="source-line-no">2155</span><span id="line-2155"> wal.sync();</span>
<span class="source-line-no">2156</span><span id="line-2156"> }</span>
<span class="source-line-no">2157</span><span id="line-2157"> return ReplicateWALEntryResponse.newBuilder().build();</span>
<span class="source-line-no">2158</span><span id="line-2158"> } catch (IOException ie) {</span>
<span class="source-line-no">2159</span><span id="line-2159"> throw new ServiceException(ie);</span>
<span class="source-line-no">2160</span><span id="line-2160"> } finally {</span>
<span class="source-line-no">2161</span><span id="line-2161"> final MetricsRegionServer metricsRegionServer = server.getMetrics();</span>
<span class="source-line-no">2162</span><span id="line-2162"> if (metricsRegionServer != null) {</span>
<span class="source-line-no">2163</span><span id="line-2163"> metricsRegionServer.updateReplay(EnvironmentEdgeManager.currentTime() - before);</span>
<span class="source-line-no">2164</span><span id="line-2164"> }</span>
<span class="source-line-no">2165</span><span id="line-2165"> }</span>
<span class="source-line-no">2166</span><span id="line-2166"> }</span>
<span class="source-line-no">2167</span><span id="line-2167"></span>
<span class="source-line-no">2168</span><span id="line-2168"> /**</span>
<span class="source-line-no">2169</span><span id="line-2169"> * Replay the given changes on a secondary replica</span>
<span class="source-line-no">2170</span><span id="line-2170"> */</span>
<span class="source-line-no">2171</span><span id="line-2171"> @Override</span>
<span class="source-line-no">2172</span><span id="line-2172"> public ReplicateWALEntryResponse replicateToReplica(RpcController controller,</span>
<span class="source-line-no">2173</span><span id="line-2173"> ReplicateWALEntryRequest request) throws ServiceException {</span>
<span class="source-line-no">2174</span><span id="line-2174"> CellScanner cells = getAndReset(controller);</span>
<span class="source-line-no">2175</span><span id="line-2175"> try {</span>
<span class="source-line-no">2176</span><span id="line-2176"> checkOpen();</span>
<span class="source-line-no">2177</span><span id="line-2177"> List&lt;WALEntry&gt; entries = request.getEntryList();</span>
<span class="source-line-no">2178</span><span id="line-2178"> if (entries == null || entries.isEmpty()) {</span>
<span class="source-line-no">2179</span><span id="line-2179"> // empty input</span>
<span class="source-line-no">2180</span><span id="line-2180"> return ReplicateWALEntryResponse.newBuilder().build();</span>
<span class="source-line-no">2181</span><span id="line-2181"> }</span>
<span class="source-line-no">2182</span><span id="line-2182"> ByteString regionName = entries.get(0).getKey().getEncodedRegionName();</span>
<span class="source-line-no">2183</span><span id="line-2183"> HRegion region = server.getRegionByEncodedName(regionName.toStringUtf8());</span>
<span class="source-line-no">2184</span><span id="line-2184"> if (RegionReplicaUtil.isDefaultReplica(region.getRegionInfo())) {</span>
<span class="source-line-no">2185</span><span id="line-2185"> throw new DoNotRetryIOException(</span>
<span class="source-line-no">2186</span><span id="line-2186"> "Should not replicate to primary replica " + region.getRegionInfo() + ", CODE BUG?");</span>
<span class="source-line-no">2187</span><span id="line-2187"> }</span>
<span class="source-line-no">2188</span><span id="line-2188"> for (WALEntry entry : entries) {</span>
<span class="source-line-no">2189</span><span id="line-2189"> if (!regionName.equals(entry.getKey().getEncodedRegionName())) {</span>
<span class="source-line-no">2190</span><span id="line-2190"> throw new NotServingRegionException(</span>
<span class="source-line-no">2191</span><span id="line-2191"> "ReplicateToReplica request contains entries from multiple " + "regions. First region:"</span>
<span class="source-line-no">2192</span><span id="line-2192"> + regionName.toStringUtf8() + " , other region:"</span>
<span class="source-line-no">2193</span><span id="line-2193"> + entry.getKey().getEncodedRegionName());</span>
<span class="source-line-no">2194</span><span id="line-2194"> }</span>
<span class="source-line-no">2195</span><span id="line-2195"> region.replayWALEntry(entry, cells);</span>
<span class="source-line-no">2196</span><span id="line-2196"> }</span>
<span class="source-line-no">2197</span><span id="line-2197"> return ReplicateWALEntryResponse.newBuilder().build();</span>
<span class="source-line-no">2198</span><span id="line-2198"> } catch (IOException ie) {</span>
<span class="source-line-no">2199</span><span id="line-2199"> throw new ServiceException(ie);</span>
<span class="source-line-no">2200</span><span id="line-2200"> }</span>
<span class="source-line-no">2201</span><span id="line-2201"> }</span>
<span class="source-line-no">2202</span><span id="line-2202"></span>
<span class="source-line-no">2203</span><span id="line-2203"> private void checkShouldRejectReplicationRequest(List&lt;WALEntry&gt; entries) throws IOException {</span>
<span class="source-line-no">2204</span><span id="line-2204"> ReplicationSourceService replicationSource = server.getReplicationSourceService();</span>
<span class="source-line-no">2205</span><span id="line-2205"> if (replicationSource == null || entries.isEmpty()) {</span>
<span class="source-line-no">2206</span><span id="line-2206"> return;</span>
<span class="source-line-no">2207</span><span id="line-2207"> }</span>
<span class="source-line-no">2208</span><span id="line-2208"> // We can ensure that all entries are for one peer, so only need to check one entry's</span>
<span class="source-line-no">2209</span><span id="line-2209"> // table name. if the table hit sync replication at peer side and the peer cluster</span>
<span class="source-line-no">2210</span><span id="line-2210"> // is (or is transiting to) state ACTIVE or DOWNGRADE_ACTIVE, we should reject to apply</span>
<span class="source-line-no">2211</span><span id="line-2211"> // those entries according to the design doc.</span>
<span class="source-line-no">2212</span><span id="line-2212"> TableName table = TableName.valueOf(entries.get(0).getKey().getTableName().toByteArray());</span>
<span class="source-line-no">2213</span><span id="line-2213"> if (</span>
<span class="source-line-no">2214</span><span id="line-2214"> replicationSource.getSyncReplicationPeerInfoProvider().checkState(table,</span>
<span class="source-line-no">2215</span><span id="line-2215"> RejectReplicationRequestStateChecker.get())</span>
<span class="source-line-no">2216</span><span id="line-2216"> ) {</span>
<span class="source-line-no">2217</span><span id="line-2217"> throw new DoNotRetryIOException(</span>
<span class="source-line-no">2218</span><span id="line-2218"> "Reject to apply to sink cluster because sync replication state of sink cluster "</span>
<span class="source-line-no">2219</span><span id="line-2219"> + "is ACTIVE or DOWNGRADE_ACTIVE, table: " + table);</span>
<span class="source-line-no">2220</span><span id="line-2220"> }</span>
<span class="source-line-no">2221</span><span id="line-2221"> }</span>
<span class="source-line-no">2222</span><span id="line-2222"></span>
<span class="source-line-no">2223</span><span id="line-2223"> /**</span>
<span class="source-line-no">2224</span><span id="line-2224"> * Replicate WAL entries on the region server.</span>
<span class="source-line-no">2225</span><span id="line-2225"> * @param controller the RPC controller</span>
<span class="source-line-no">2226</span><span id="line-2226"> * @param request the request</span>
<span class="source-line-no">2227</span><span id="line-2227"> */</span>
<span class="source-line-no">2228</span><span id="line-2228"> @Override</span>
<span class="source-line-no">2229</span><span id="line-2229"> @QosPriority(priority = HConstants.REPLICATION_QOS)</span>
<span class="source-line-no">2230</span><span id="line-2230"> public ReplicateWALEntryResponse replicateWALEntry(final RpcController controller,</span>
<span class="source-line-no">2231</span><span id="line-2231"> final ReplicateWALEntryRequest request) throws ServiceException {</span>
<span class="source-line-no">2232</span><span id="line-2232"> try {</span>
<span class="source-line-no">2233</span><span id="line-2233"> checkOpen();</span>
<span class="source-line-no">2234</span><span id="line-2234"> if (server.getReplicationSinkService() != null) {</span>
<span class="source-line-no">2235</span><span id="line-2235"> requestCount.increment();</span>
<span class="source-line-no">2236</span><span id="line-2236"> List&lt;WALEntry&gt; entries = request.getEntryList();</span>
<span class="source-line-no">2237</span><span id="line-2237"> checkShouldRejectReplicationRequest(entries);</span>
<span class="source-line-no">2238</span><span id="line-2238"> ExtendedCellScanner cellScanner = getAndReset(controller);</span>
<span class="source-line-no">2239</span><span id="line-2239"> server.getRegionServerCoprocessorHost().preReplicateLogEntries();</span>
<span class="source-line-no">2240</span><span id="line-2240"> server.getReplicationSinkService().replicateLogEntries(entries, cellScanner,</span>
<span class="source-line-no">2241</span><span id="line-2241"> request.getReplicationClusterId(), request.getSourceBaseNamespaceDirPath(),</span>
<span class="source-line-no">2242</span><span id="line-2242"> request.getSourceHFileArchiveDirPath());</span>
<span class="source-line-no">2243</span><span id="line-2243"> server.getRegionServerCoprocessorHost().postReplicateLogEntries();</span>
<span class="source-line-no">2244</span><span id="line-2244"> return ReplicateWALEntryResponse.newBuilder().build();</span>
<span class="source-line-no">2245</span><span id="line-2245"> } else {</span>
<span class="source-line-no">2246</span><span id="line-2246"> throw new ServiceException("Replication services are not initialized yet");</span>
<span class="source-line-no">2247</span><span id="line-2247"> }</span>
<span class="source-line-no">2248</span><span id="line-2248"> } catch (IOException ie) {</span>
<span class="source-line-no">2249</span><span id="line-2249"> throw new ServiceException(ie);</span>
<span class="source-line-no">2250</span><span id="line-2250"> }</span>
<span class="source-line-no">2251</span><span id="line-2251"> }</span>
<span class="source-line-no">2252</span><span id="line-2252"></span>
<span class="source-line-no">2253</span><span id="line-2253"> /**</span>
<span class="source-line-no">2254</span><span id="line-2254"> * Roll the WAL writer of the region server.</span>
<span class="source-line-no">2255</span><span id="line-2255"> * @param controller the RPC controller</span>
<span class="source-line-no">2256</span><span id="line-2256"> * @param request the request</span>
<span class="source-line-no">2257</span><span id="line-2257"> */</span>
<span class="source-line-no">2258</span><span id="line-2258"> @Override</span>
<span class="source-line-no">2259</span><span id="line-2259"> @QosPriority(priority = HConstants.ADMIN_QOS)</span>
<span class="source-line-no">2260</span><span id="line-2260"> public RollWALWriterResponse rollWALWriter(final RpcController controller,</span>
<span class="source-line-no">2261</span><span id="line-2261"> final RollWALWriterRequest request) throws ServiceException {</span>
<span class="source-line-no">2262</span><span id="line-2262"> try {</span>
<span class="source-line-no">2263</span><span id="line-2263"> checkOpen();</span>
<span class="source-line-no">2264</span><span id="line-2264"> requestCount.increment();</span>
<span class="source-line-no">2265</span><span id="line-2265"> server.getRegionServerCoprocessorHost().preRollWALWriterRequest();</span>
<span class="source-line-no">2266</span><span id="line-2266"> server.getWalRoller().requestRollAll();</span>
<span class="source-line-no">2267</span><span id="line-2267"> server.getRegionServerCoprocessorHost().postRollWALWriterRequest();</span>
<span class="source-line-no">2268</span><span id="line-2268"> RollWALWriterResponse.Builder builder = RollWALWriterResponse.newBuilder();</span>
<span class="source-line-no">2269</span><span id="line-2269"> return builder.build();</span>
<span class="source-line-no">2270</span><span id="line-2270"> } catch (IOException ie) {</span>
<span class="source-line-no">2271</span><span id="line-2271"> throw new ServiceException(ie);</span>
<span class="source-line-no">2272</span><span id="line-2272"> }</span>
<span class="source-line-no">2273</span><span id="line-2273"> }</span>
<span class="source-line-no">2274</span><span id="line-2274"></span>
<span class="source-line-no">2275</span><span id="line-2275"> /**</span>
<span class="source-line-no">2276</span><span id="line-2276"> * Stop the region server.</span>
<span class="source-line-no">2277</span><span id="line-2277"> * @param controller the RPC controller</span>
<span class="source-line-no">2278</span><span id="line-2278"> * @param request the request</span>
<span class="source-line-no">2279</span><span id="line-2279"> */</span>
<span class="source-line-no">2280</span><span id="line-2280"> @Override</span>
<span class="source-line-no">2281</span><span id="line-2281"> @QosPriority(priority = HConstants.ADMIN_QOS)</span>
<span class="source-line-no">2282</span><span id="line-2282"> public StopServerResponse stopServer(final RpcController controller,</span>
<span class="source-line-no">2283</span><span id="line-2283"> final StopServerRequest request) throws ServiceException {</span>
<span class="source-line-no">2284</span><span id="line-2284"> rpcPreCheck("stopServer");</span>
<span class="source-line-no">2285</span><span id="line-2285"> requestCount.increment();</span>
<span class="source-line-no">2286</span><span id="line-2286"> String reason = request.getReason();</span>
<span class="source-line-no">2287</span><span id="line-2287"> server.stop(reason);</span>
<span class="source-line-no">2288</span><span id="line-2288"> return StopServerResponse.newBuilder().build();</span>
<span class="source-line-no">2289</span><span id="line-2289"> }</span>
<span class="source-line-no">2290</span><span id="line-2290"></span>
<span class="source-line-no">2291</span><span id="line-2291"> @Override</span>
<span class="source-line-no">2292</span><span id="line-2292"> public UpdateFavoredNodesResponse updateFavoredNodes(RpcController controller,</span>
<span class="source-line-no">2293</span><span id="line-2293"> UpdateFavoredNodesRequest request) throws ServiceException {</span>
<span class="source-line-no">2294</span><span id="line-2294"> rpcPreCheck("updateFavoredNodes");</span>
<span class="source-line-no">2295</span><span id="line-2295"> List&lt;UpdateFavoredNodesRequest.RegionUpdateInfo&gt; openInfoList = request.getUpdateInfoList();</span>
<span class="source-line-no">2296</span><span id="line-2296"> UpdateFavoredNodesResponse.Builder respBuilder = UpdateFavoredNodesResponse.newBuilder();</span>
<span class="source-line-no">2297</span><span id="line-2297"> for (UpdateFavoredNodesRequest.RegionUpdateInfo regionUpdateInfo : openInfoList) {</span>
<span class="source-line-no">2298</span><span id="line-2298"> RegionInfo hri = ProtobufUtil.toRegionInfo(regionUpdateInfo.getRegion());</span>
<span class="source-line-no">2299</span><span id="line-2299"> if (regionUpdateInfo.getFavoredNodesCount() &gt; 0) {</span>
<span class="source-line-no">2300</span><span id="line-2300"> server.updateRegionFavoredNodesMapping(hri.getEncodedName(),</span>
<span class="source-line-no">2301</span><span id="line-2301"> regionUpdateInfo.getFavoredNodesList());</span>
<span class="source-line-no">2302</span><span id="line-2302"> }</span>
<span class="source-line-no">2303</span><span id="line-2303"> }</span>
<span class="source-line-no">2304</span><span id="line-2304"> respBuilder.setResponse(openInfoList.size());</span>
<span class="source-line-no">2305</span><span id="line-2305"> return respBuilder.build();</span>
<span class="source-line-no">2306</span><span id="line-2306"> }</span>
<span class="source-line-no">2307</span><span id="line-2307"></span>
<span class="source-line-no">2308</span><span id="line-2308"> /**</span>
<span class="source-line-no">2309</span><span id="line-2309"> * Atomically bulk load several HFiles into an open region</span>
<span class="source-line-no">2310</span><span id="line-2310"> * @return true if successful, false is failed but recoverably (no action)</span>
<span class="source-line-no">2311</span><span id="line-2311"> * @throws ServiceException if failed unrecoverably</span>
<span class="source-line-no">2312</span><span id="line-2312"> */</span>
<span class="source-line-no">2313</span><span id="line-2313"> @Override</span>
<span class="source-line-no">2314</span><span id="line-2314"> public BulkLoadHFileResponse bulkLoadHFile(final RpcController controller,</span>
<span class="source-line-no">2315</span><span id="line-2315"> final BulkLoadHFileRequest request) throws ServiceException {</span>
<span class="source-line-no">2316</span><span id="line-2316"> long start = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">2317</span><span id="line-2317"> List&lt;String&gt; clusterIds = new ArrayList&lt;&gt;(request.getClusterIdsList());</span>
<span class="source-line-no">2318</span><span id="line-2318"> if (clusterIds.contains(this.server.getClusterId())) {</span>
<span class="source-line-no">2319</span><span id="line-2319"> return BulkLoadHFileResponse.newBuilder().setLoaded(true).build();</span>
<span class="source-line-no">2320</span><span id="line-2320"> } else {</span>
<span class="source-line-no">2321</span><span id="line-2321"> clusterIds.add(this.server.getClusterId());</span>
<span class="source-line-no">2322</span><span id="line-2322"> }</span>
<span class="source-line-no">2323</span><span id="line-2323"> try {</span>
<span class="source-line-no">2324</span><span id="line-2324"> checkOpen();</span>
<span class="source-line-no">2325</span><span id="line-2325"> requestCount.increment();</span>
<span class="source-line-no">2326</span><span id="line-2326"> HRegion region = getRegion(request.getRegion());</span>
<span class="source-line-no">2327</span><span id="line-2327"> final boolean spaceQuotaEnabled = QuotaUtil.isQuotaEnabled(getConfiguration());</span>
<span class="source-line-no">2328</span><span id="line-2328"> long sizeToBeLoaded = -1;</span>
<span class="source-line-no">2329</span><span id="line-2329"></span>
<span class="source-line-no">2330</span><span id="line-2330"> // Check to see if this bulk load would exceed the space quota for this table</span>
<span class="source-line-no">2331</span><span id="line-2331"> if (spaceQuotaEnabled) {</span>
<span class="source-line-no">2332</span><span id="line-2332"> ActivePolicyEnforcement activeSpaceQuotas = getSpaceQuotaManager().getActiveEnforcements();</span>
<span class="source-line-no">2333</span><span id="line-2333"> SpaceViolationPolicyEnforcement enforcement =</span>
<span class="source-line-no">2334</span><span id="line-2334"> activeSpaceQuotas.getPolicyEnforcement(region);</span>
<span class="source-line-no">2335</span><span id="line-2335"> if (enforcement != null) {</span>
<span class="source-line-no">2336</span><span id="line-2336"> // Bulk loads must still be atomic. We must enact all or none.</span>
<span class="source-line-no">2337</span><span id="line-2337"> List&lt;String&gt; filePaths = new ArrayList&lt;&gt;(request.getFamilyPathCount());</span>
<span class="source-line-no">2338</span><span id="line-2338"> for (FamilyPath familyPath : request.getFamilyPathList()) {</span>
<span class="source-line-no">2339</span><span id="line-2339"> filePaths.add(familyPath.getPath());</span>
<span class="source-line-no">2340</span><span id="line-2340"> }</span>
<span class="source-line-no">2341</span><span id="line-2341"> // Check if the batch of files exceeds the current quota</span>
<span class="source-line-no">2342</span><span id="line-2342"> sizeToBeLoaded = enforcement.computeBulkLoadSize(getFileSystem(filePaths), filePaths);</span>
<span class="source-line-no">2343</span><span id="line-2343"> }</span>
<span class="source-line-no">2344</span><span id="line-2344"> }</span>
<span class="source-line-no">2345</span><span id="line-2345"> // secure bulk load</span>
<span class="source-line-no">2346</span><span id="line-2346"> Map&lt;byte[], List&lt;Path&gt;&gt; map =</span>
<span class="source-line-no">2347</span><span id="line-2347"> server.getSecureBulkLoadManager().secureBulkLoadHFiles(region, request, clusterIds);</span>
<span class="source-line-no">2348</span><span id="line-2348"> BulkLoadHFileResponse.Builder builder = BulkLoadHFileResponse.newBuilder();</span>
<span class="source-line-no">2349</span><span id="line-2349"> builder.setLoaded(map != null);</span>
<span class="source-line-no">2350</span><span id="line-2350"> if (map != null) {</span>
<span class="source-line-no">2351</span><span id="line-2351"> // Treat any negative size as a flag to "ignore" updating the region size as that is</span>
<span class="source-line-no">2352</span><span id="line-2352"> // not possible to occur in real life (cannot bulk load a file with negative size)</span>
<span class="source-line-no">2353</span><span id="line-2353"> if (spaceQuotaEnabled &amp;&amp; sizeToBeLoaded &gt; 0) {</span>
<span class="source-line-no">2354</span><span id="line-2354"> if (LOG.isTraceEnabled()) {</span>
<span class="source-line-no">2355</span><span id="line-2355"> LOG.trace("Incrementing space use of " + region.getRegionInfo() + " by "</span>
<span class="source-line-no">2356</span><span id="line-2356"> + sizeToBeLoaded + " bytes");</span>
<span class="source-line-no">2357</span><span id="line-2357"> }</span>
<span class="source-line-no">2358</span><span id="line-2358"> // Inform space quotas of the new files for this region</span>
<span class="source-line-no">2359</span><span id="line-2359"> getSpaceQuotaManager().getRegionSizeStore().incrementRegionSize(region.getRegionInfo(),</span>
<span class="source-line-no">2360</span><span id="line-2360"> sizeToBeLoaded);</span>
<span class="source-line-no">2361</span><span id="line-2361"> }</span>
<span class="source-line-no">2362</span><span id="line-2362"> }</span>
<span class="source-line-no">2363</span><span id="line-2363"> return builder.build();</span>
<span class="source-line-no">2364</span><span id="line-2364"> } catch (IOException ie) {</span>
<span class="source-line-no">2365</span><span id="line-2365"> throw new ServiceException(ie);</span>
<span class="source-line-no">2366</span><span id="line-2366"> } finally {</span>
<span class="source-line-no">2367</span><span id="line-2367"> final MetricsRegionServer metricsRegionServer = server.getMetrics();</span>
<span class="source-line-no">2368</span><span id="line-2368"> if (metricsRegionServer != null) {</span>
<span class="source-line-no">2369</span><span id="line-2369"> metricsRegionServer.updateBulkLoad(EnvironmentEdgeManager.currentTime() - start);</span>
<span class="source-line-no">2370</span><span id="line-2370"> }</span>
<span class="source-line-no">2371</span><span id="line-2371"> }</span>
<span class="source-line-no">2372</span><span id="line-2372"> }</span>
<span class="source-line-no">2373</span><span id="line-2373"></span>
<span class="source-line-no">2374</span><span id="line-2374"> @Override</span>
<span class="source-line-no">2375</span><span id="line-2375"> public PrepareBulkLoadResponse prepareBulkLoad(RpcController controller,</span>
<span class="source-line-no">2376</span><span id="line-2376"> PrepareBulkLoadRequest request) throws ServiceException {</span>
<span class="source-line-no">2377</span><span id="line-2377"> try {</span>
<span class="source-line-no">2378</span><span id="line-2378"> checkOpen();</span>
<span class="source-line-no">2379</span><span id="line-2379"> requestCount.increment();</span>
<span class="source-line-no">2380</span><span id="line-2380"></span>
<span class="source-line-no">2381</span><span id="line-2381"> HRegion region = getRegion(request.getRegion());</span>
<span class="source-line-no">2382</span><span id="line-2382"></span>
<span class="source-line-no">2383</span><span id="line-2383"> String bulkToken = server.getSecureBulkLoadManager().prepareBulkLoad(region, request);</span>
<span class="source-line-no">2384</span><span id="line-2384"> PrepareBulkLoadResponse.Builder builder = PrepareBulkLoadResponse.newBuilder();</span>
<span class="source-line-no">2385</span><span id="line-2385"> builder.setBulkToken(bulkToken);</span>
<span class="source-line-no">2386</span><span id="line-2386"> return builder.build();</span>
<span class="source-line-no">2387</span><span id="line-2387"> } catch (IOException ie) {</span>
<span class="source-line-no">2388</span><span id="line-2388"> throw new ServiceException(ie);</span>
<span class="source-line-no">2389</span><span id="line-2389"> }</span>
<span class="source-line-no">2390</span><span id="line-2390"> }</span>
<span class="source-line-no">2391</span><span id="line-2391"></span>
<span class="source-line-no">2392</span><span id="line-2392"> @Override</span>
<span class="source-line-no">2393</span><span id="line-2393"> public CleanupBulkLoadResponse cleanupBulkLoad(RpcController controller,</span>
<span class="source-line-no">2394</span><span id="line-2394"> CleanupBulkLoadRequest request) throws ServiceException {</span>
<span class="source-line-no">2395</span><span id="line-2395"> try {</span>
<span class="source-line-no">2396</span><span id="line-2396"> checkOpen();</span>
<span class="source-line-no">2397</span><span id="line-2397"> requestCount.increment();</span>
<span class="source-line-no">2398</span><span id="line-2398"></span>
<span class="source-line-no">2399</span><span id="line-2399"> HRegion region = getRegion(request.getRegion());</span>
<span class="source-line-no">2400</span><span id="line-2400"></span>
<span class="source-line-no">2401</span><span id="line-2401"> server.getSecureBulkLoadManager().cleanupBulkLoad(region, request);</span>
<span class="source-line-no">2402</span><span id="line-2402"> return CleanupBulkLoadResponse.newBuilder().build();</span>
<span class="source-line-no">2403</span><span id="line-2403"> } catch (IOException ie) {</span>
<span class="source-line-no">2404</span><span id="line-2404"> throw new ServiceException(ie);</span>
<span class="source-line-no">2405</span><span id="line-2405"> }</span>
<span class="source-line-no">2406</span><span id="line-2406"> }</span>
<span class="source-line-no">2407</span><span id="line-2407"></span>
<span class="source-line-no">2408</span><span id="line-2408"> @Override</span>
<span class="source-line-no">2409</span><span id="line-2409"> public CoprocessorServiceResponse execService(final RpcController controller,</span>
<span class="source-line-no">2410</span><span id="line-2410"> final CoprocessorServiceRequest request) throws ServiceException {</span>
<span class="source-line-no">2411</span><span id="line-2411"> try {</span>
<span class="source-line-no">2412</span><span id="line-2412"> checkOpen();</span>
<span class="source-line-no">2413</span><span id="line-2413"> requestCount.increment();</span>
<span class="source-line-no">2414</span><span id="line-2414"> HRegion region = getRegion(request.getRegion());</span>
<span class="source-line-no">2415</span><span id="line-2415"> Message result = execServiceOnRegion(region, request.getCall());</span>
<span class="source-line-no">2416</span><span id="line-2416"> CoprocessorServiceResponse.Builder builder = CoprocessorServiceResponse.newBuilder();</span>
<span class="source-line-no">2417</span><span id="line-2417"> builder.setRegion(RequestConverter.buildRegionSpecifier(RegionSpecifierType.REGION_NAME,</span>
<span class="source-line-no">2418</span><span id="line-2418"> region.getRegionInfo().getRegionName()));</span>
<span class="source-line-no">2419</span><span id="line-2419"> // TODO: COPIES!!!!!!</span>
<span class="source-line-no">2420</span><span id="line-2420"> builder.setValue(builder.getValueBuilder().setName(result.getClass().getName()).setValue(</span>
<span class="source-line-no">2421</span><span id="line-2421"> org.apache.hbase.thirdparty.com.google.protobuf.ByteString.copyFrom(result.toByteArray())));</span>
<span class="source-line-no">2422</span><span id="line-2422"> return builder.build();</span>
<span class="source-line-no">2423</span><span id="line-2423"> } catch (IOException ie) {</span>
<span class="source-line-no">2424</span><span id="line-2424"> throw new ServiceException(ie);</span>
<span class="source-line-no">2425</span><span id="line-2425"> }</span>
<span class="source-line-no">2426</span><span id="line-2426"> }</span>
<span class="source-line-no">2427</span><span id="line-2427"></span>
<span class="source-line-no">2428</span><span id="line-2428"> private FileSystem getFileSystem(List&lt;String&gt; filePaths) throws IOException {</span>
<span class="source-line-no">2429</span><span id="line-2429"> if (filePaths.isEmpty()) {</span>
<span class="source-line-no">2430</span><span id="line-2430"> // local hdfs</span>
<span class="source-line-no">2431</span><span id="line-2431"> return server.getFileSystem();</span>
<span class="source-line-no">2432</span><span id="line-2432"> }</span>
<span class="source-line-no">2433</span><span id="line-2433"> // source hdfs</span>
<span class="source-line-no">2434</span><span id="line-2434"> return new Path(filePaths.get(0)).getFileSystem(server.getConfiguration());</span>
<span class="source-line-no">2435</span><span id="line-2435"> }</span>
<span class="source-line-no">2436</span><span id="line-2436"></span>
<span class="source-line-no">2437</span><span id="line-2437"> private Message execServiceOnRegion(HRegion region,</span>
<span class="source-line-no">2438</span><span id="line-2438"> final ClientProtos.CoprocessorServiceCall serviceCall) throws IOException {</span>
<span class="source-line-no">2439</span><span id="line-2439"> // ignore the passed in controller (from the serialized call)</span>
<span class="source-line-no">2440</span><span id="line-2440"> ServerRpcController execController = new ServerRpcController();</span>
<span class="source-line-no">2441</span><span id="line-2441"> return region.execService(execController, serviceCall);</span>
<span class="source-line-no">2442</span><span id="line-2442"> }</span>
<span class="source-line-no">2443</span><span id="line-2443"></span>
<span class="source-line-no">2444</span><span id="line-2444"> private boolean shouldRejectRequestsFromClient(HRegion region) {</span>
<span class="source-line-no">2445</span><span id="line-2445"> TableName table = region.getRegionInfo().getTable();</span>
<span class="source-line-no">2446</span><span id="line-2446"> ReplicationSourceService service = server.getReplicationSourceService();</span>
<span class="source-line-no">2447</span><span id="line-2447"> return service != null &amp;&amp; service.getSyncReplicationPeerInfoProvider().checkState(table,</span>
<span class="source-line-no">2448</span><span id="line-2448"> RejectRequestsFromClientStateChecker.get());</span>
<span class="source-line-no">2449</span><span id="line-2449"> }</span>
<span class="source-line-no">2450</span><span id="line-2450"></span>
<span class="source-line-no">2451</span><span id="line-2451"> private void rejectIfInStandByState(HRegion region) throws DoNotRetryIOException {</span>
<span class="source-line-no">2452</span><span id="line-2452"> if (shouldRejectRequestsFromClient(region)) {</span>
<span class="source-line-no">2453</span><span id="line-2453"> throw new DoNotRetryIOException(</span>
<span class="source-line-no">2454</span><span id="line-2454"> region.getRegionInfo().getRegionNameAsString() + " is in STANDBY state.");</span>
<span class="source-line-no">2455</span><span id="line-2455"> }</span>
<span class="source-line-no">2456</span><span id="line-2456"> }</span>
<span class="source-line-no">2457</span><span id="line-2457"></span>
<span class="source-line-no">2458</span><span id="line-2458"> /**</span>
<span class="source-line-no">2459</span><span id="line-2459"> * Get data from a table.</span>
<span class="source-line-no">2460</span><span id="line-2460"> * @param controller the RPC controller</span>
<span class="source-line-no">2461</span><span id="line-2461"> * @param request the get request</span>
<span class="source-line-no">2462</span><span id="line-2462"> */</span>
<span class="source-line-no">2463</span><span id="line-2463"> @Override</span>
<span class="source-line-no">2464</span><span id="line-2464"> public GetResponse get(final RpcController controller, final GetRequest request)</span>
<span class="source-line-no">2465</span><span id="line-2465"> throws ServiceException {</span>
<span class="source-line-no">2466</span><span id="line-2466"> long before = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">2467</span><span id="line-2467"> OperationQuota quota = null;</span>
<span class="source-line-no">2468</span><span id="line-2468"> HRegion region = null;</span>
<span class="source-line-no">2469</span><span id="line-2469"> RpcCallContext context = RpcServer.getCurrentCall().orElse(null);</span>
<span class="source-line-no">2470</span><span id="line-2470"> try {</span>
<span class="source-line-no">2471</span><span id="line-2471"> checkOpen();</span>
<span class="source-line-no">2472</span><span id="line-2472"> requestCount.increment();</span>
<span class="source-line-no">2473</span><span id="line-2473"> rpcGetRequestCount.increment();</span>
<span class="source-line-no">2474</span><span id="line-2474"> region = getRegion(request.getRegion());</span>
<span class="source-line-no">2475</span><span id="line-2475"> rejectIfInStandByState(region);</span>
<span class="source-line-no">2476</span><span id="line-2476"></span>
<span class="source-line-no">2477</span><span id="line-2477"> GetResponse.Builder builder = GetResponse.newBuilder();</span>
<span class="source-line-no">2478</span><span id="line-2478"> ClientProtos.Get get = request.getGet();</span>
<span class="source-line-no">2479</span><span id="line-2479"> // An asynchbase client, https://github.com/OpenTSDB/asynchbase, starts by trying to do</span>
<span class="source-line-no">2480</span><span id="line-2480"> // a get closest before. Throwing the UnknownProtocolException signals it that it needs</span>
<span class="source-line-no">2481</span><span id="line-2481"> // to switch and do hbase2 protocol (HBase servers do not tell clients what versions</span>
<span class="source-line-no">2482</span><span id="line-2482"> // they are; its a problem for non-native clients like asynchbase. HBASE-20225.</span>
<span class="source-line-no">2483</span><span id="line-2483"> if (get.hasClosestRowBefore() &amp;&amp; get.getClosestRowBefore()) {</span>
<span class="source-line-no">2484</span><span id="line-2484"> throw new UnknownProtocolException("Is this a pre-hbase-1.0.0 or asynchbase client? "</span>
<span class="source-line-no">2485</span><span id="line-2485"> + "Client is invoking getClosestRowBefore removed in hbase-2.0.0 replaced by "</span>
<span class="source-line-no">2486</span><span id="line-2486"> + "reverse Scan.");</span>
<span class="source-line-no">2487</span><span id="line-2487"> }</span>
<span class="source-line-no">2488</span><span id="line-2488"> Boolean existence = null;</span>
<span class="source-line-no">2489</span><span id="line-2489"> Result r = null;</span>
<span class="source-line-no">2490</span><span id="line-2490"> quota = getRpcQuotaManager().checkBatchQuota(region, OperationQuota.OperationType.GET);</span>
<span class="source-line-no">2491</span><span id="line-2491"></span>
<span class="source-line-no">2492</span><span id="line-2492"> Get clientGet = ProtobufUtil.toGet(get);</span>
<span class="source-line-no">2493</span><span id="line-2493"> if (get.getExistenceOnly() &amp;&amp; region.getCoprocessorHost() != null) {</span>
<span class="source-line-no">2494</span><span id="line-2494"> existence = region.getCoprocessorHost().preExists(clientGet);</span>
<span class="source-line-no">2495</span><span id="line-2495"> }</span>
<span class="source-line-no">2496</span><span id="line-2496"> if (existence == null) {</span>
<span class="source-line-no">2497</span><span id="line-2497"> if (context != null) {</span>
<span class="source-line-no">2498</span><span id="line-2498"> r = get(clientGet, (region), null, context);</span>
<span class="source-line-no">2499</span><span id="line-2499"> } else {</span>
<span class="source-line-no">2500</span><span id="line-2500"> // for test purpose</span>
<span class="source-line-no">2501</span><span id="line-2501"> r = region.get(clientGet);</span>
<span class="source-line-no">2502</span><span id="line-2502"> }</span>
<span class="source-line-no">2503</span><span id="line-2503"> if (get.getExistenceOnly()) {</span>
<span class="source-line-no">2504</span><span id="line-2504"> boolean exists = r.getExists();</span>
<span class="source-line-no">2505</span><span id="line-2505"> if (region.getCoprocessorHost() != null) {</span>
<span class="source-line-no">2506</span><span id="line-2506"> exists = region.getCoprocessorHost().postExists(clientGet, exists);</span>
<span class="source-line-no">2507</span><span id="line-2507"> }</span>
<span class="source-line-no">2508</span><span id="line-2508"> existence = exists;</span>
<span class="source-line-no">2509</span><span id="line-2509"> }</span>
<span class="source-line-no">2510</span><span id="line-2510"> }</span>
<span class="source-line-no">2511</span><span id="line-2511"> if (existence != null) {</span>
<span class="source-line-no">2512</span><span id="line-2512"> ClientProtos.Result pbr =</span>
<span class="source-line-no">2513</span><span id="line-2513"> ProtobufUtil.toResult(existence, region.getRegionInfo().getReplicaId() != 0);</span>
<span class="source-line-no">2514</span><span id="line-2514"> builder.setResult(pbr);</span>
<span class="source-line-no">2515</span><span id="line-2515"> } else if (r != null) {</span>
<span class="source-line-no">2516</span><span id="line-2516"> ClientProtos.Result pbr;</span>
<span class="source-line-no">2517</span><span id="line-2517"> if (</span>
<span class="source-line-no">2518</span><span id="line-2518"> isClientCellBlockSupport(context) &amp;&amp; controller instanceof HBaseRpcController</span>
<span class="source-line-no">2519</span><span id="line-2519"> &amp;&amp; VersionInfoUtil.hasMinimumVersion(context.getClientVersionInfo(), 1, 3)</span>
<span class="source-line-no">2520</span><span id="line-2520"> ) {</span>
<span class="source-line-no">2521</span><span id="line-2521"> pbr = ProtobufUtil.toResultNoData(r);</span>
<span class="source-line-no">2522</span><span id="line-2522"> ((HBaseRpcController) controller).setCellScanner(</span>
<span class="source-line-no">2523</span><span id="line-2523"> PrivateCellUtil.createExtendedCellScanner(ClientInternalHelper.getExtendedRawCells(r)));</span>
<span class="source-line-no">2524</span><span id="line-2524"> addSize(context, r);</span>
<span class="source-line-no">2525</span><span id="line-2525"> } else {</span>
<span class="source-line-no">2526</span><span id="line-2526"> pbr = ProtobufUtil.toResult(r);</span>
<span class="source-line-no">2527</span><span id="line-2527"> }</span>
<span class="source-line-no">2528</span><span id="line-2528"> builder.setResult(pbr);</span>
<span class="source-line-no">2529</span><span id="line-2529"> }</span>
<span class="source-line-no">2530</span><span id="line-2530"> // r.cells is null when an table.exists(get) call</span>
<span class="source-line-no">2531</span><span id="line-2531"> if (r != null &amp;&amp; r.rawCells() != null) {</span>
<span class="source-line-no">2532</span><span id="line-2532"> quota.addGetResult(r);</span>
<span class="source-line-no">2533</span><span id="line-2533"> }</span>
<span class="source-line-no">2534</span><span id="line-2534"> return builder.build();</span>
<span class="source-line-no">2535</span><span id="line-2535"> } catch (IOException ie) {</span>
<span class="source-line-no">2536</span><span id="line-2536"> throw new ServiceException(ie);</span>
<span class="source-line-no">2537</span><span id="line-2537"> } finally {</span>
<span class="source-line-no">2538</span><span id="line-2538"> final MetricsRegionServer metricsRegionServer = server.getMetrics();</span>
<span class="source-line-no">2539</span><span id="line-2539"> if (metricsRegionServer != null &amp;&amp; region != null) {</span>
<span class="source-line-no">2540</span><span id="line-2540"> long blockBytesScanned = context != null ? context.getBlockBytesScanned() : 0;</span>
<span class="source-line-no">2541</span><span id="line-2541"> metricsRegionServer.updateGet(region, EnvironmentEdgeManager.currentTime() - before,</span>
<span class="source-line-no">2542</span><span id="line-2542"> blockBytesScanned);</span>
<span class="source-line-no">2543</span><span id="line-2543"> }</span>
<span class="source-line-no">2544</span><span id="line-2544"> if (quota != null) {</span>
<span class="source-line-no">2545</span><span id="line-2545"> quota.close();</span>
<span class="source-line-no">2546</span><span id="line-2546"> }</span>
<span class="source-line-no">2547</span><span id="line-2547"> }</span>
<span class="source-line-no">2548</span><span id="line-2548"> }</span>
<span class="source-line-no">2549</span><span id="line-2549"></span>
<span class="source-line-no">2550</span><span id="line-2550"> private Result get(Get get, HRegion region, RegionScannersCloseCallBack closeCallBack,</span>
<span class="source-line-no">2551</span><span id="line-2551"> RpcCallContext context) throws IOException {</span>
<span class="source-line-no">2552</span><span id="line-2552"> region.prepareGet(get);</span>
<span class="source-line-no">2553</span><span id="line-2553"> boolean stale = region.getRegionInfo().getReplicaId() != 0;</span>
<span class="source-line-no">2554</span><span id="line-2554"></span>
<span class="source-line-no">2555</span><span id="line-2555"> // This method is almost the same as HRegion#get.</span>
<span class="source-line-no">2556</span><span id="line-2556"> List&lt;Cell&gt; results = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">2557</span><span id="line-2557"> // pre-get CP hook</span>
<span class="source-line-no">2558</span><span id="line-2558"> if (region.getCoprocessorHost() != null) {</span>
<span class="source-line-no">2559</span><span id="line-2559"> if (region.getCoprocessorHost().preGet(get, results)) {</span>
<span class="source-line-no">2560</span><span id="line-2560"> region.metricsUpdateForGet();</span>
<span class="source-line-no">2561</span><span id="line-2561"> return Result.create(results, get.isCheckExistenceOnly() ? !results.isEmpty() : null,</span>
<span class="source-line-no">2562</span><span id="line-2562"> stale);</span>
<span class="source-line-no">2563</span><span id="line-2563"> }</span>
<span class="source-line-no">2564</span><span id="line-2564"> }</span>
<span class="source-line-no">2565</span><span id="line-2565"> Scan scan = new Scan(get);</span>
<span class="source-line-no">2566</span><span id="line-2566"> if (scan.getLoadColumnFamiliesOnDemandValue() == null) {</span>
<span class="source-line-no">2567</span><span id="line-2567"> scan.setLoadColumnFamiliesOnDemand(region.isLoadingCfsOnDemandDefault());</span>
<span class="source-line-no">2568</span><span id="line-2568"> }</span>
<span class="source-line-no">2569</span><span id="line-2569"> RegionScannerImpl scanner = null;</span>
<span class="source-line-no">2570</span><span id="line-2570"> try {</span>
<span class="source-line-no">2571</span><span id="line-2571"> scanner = region.getScanner(scan);</span>
<span class="source-line-no">2572</span><span id="line-2572"> scanner.next(results);</span>
<span class="source-line-no">2573</span><span id="line-2573"> } finally {</span>
<span class="source-line-no">2574</span><span id="line-2574"> if (scanner != null) {</span>
<span class="source-line-no">2575</span><span id="line-2575"> if (closeCallBack == null) {</span>
<span class="source-line-no">2576</span><span id="line-2576"> // If there is a context then the scanner can be added to the current</span>
<span class="source-line-no">2577</span><span id="line-2577"> // RpcCallContext. The rpc callback will take care of closing the</span>
<span class="source-line-no">2578</span><span id="line-2578"> // scanner, for eg in case</span>
<span class="source-line-no">2579</span><span id="line-2579"> // of get()</span>
<span class="source-line-no">2580</span><span id="line-2580"> context.setCallBack(scanner);</span>
<span class="source-line-no">2581</span><span id="line-2581"> } else {</span>
<span class="source-line-no">2582</span><span id="line-2582"> // The call is from multi() where the results from the get() are</span>
<span class="source-line-no">2583</span><span id="line-2583"> // aggregated and then send out to the</span>
<span class="source-line-no">2584</span><span id="line-2584"> // rpc. The rpccall back will close all such scanners created as part</span>
<span class="source-line-no">2585</span><span id="line-2585"> // of multi().</span>
<span class="source-line-no">2586</span><span id="line-2586"> closeCallBack.addScanner(scanner);</span>
<span class="source-line-no">2587</span><span id="line-2587"> }</span>
<span class="source-line-no">2588</span><span id="line-2588"> }</span>
<span class="source-line-no">2589</span><span id="line-2589"> }</span>
<span class="source-line-no">2590</span><span id="line-2590"></span>
<span class="source-line-no">2591</span><span id="line-2591"> // post-get CP hook</span>
<span class="source-line-no">2592</span><span id="line-2592"> if (region.getCoprocessorHost() != null) {</span>
<span class="source-line-no">2593</span><span id="line-2593"> region.getCoprocessorHost().postGet(get, results);</span>
<span class="source-line-no">2594</span><span id="line-2594"> }</span>
<span class="source-line-no">2595</span><span id="line-2595"> region.metricsUpdateForGet();</span>
<span class="source-line-no">2596</span><span id="line-2596"></span>
<span class="source-line-no">2597</span><span id="line-2597"> return Result.create(results, get.isCheckExistenceOnly() ? !results.isEmpty() : null, stale);</span>
<span class="source-line-no">2598</span><span id="line-2598"> }</span>
<span class="source-line-no">2599</span><span id="line-2599"></span>
<span class="source-line-no">2600</span><span id="line-2600"> private void checkBatchSizeAndLogLargeSize(MultiRequest request) throws ServiceException {</span>
<span class="source-line-no">2601</span><span id="line-2601"> int sum = 0;</span>
<span class="source-line-no">2602</span><span id="line-2602"> String firstRegionName = null;</span>
<span class="source-line-no">2603</span><span id="line-2603"> for (RegionAction regionAction : request.getRegionActionList()) {</span>
<span class="source-line-no">2604</span><span id="line-2604"> if (sum == 0) {</span>
<span class="source-line-no">2605</span><span id="line-2605"> firstRegionName = Bytes.toStringBinary(regionAction.getRegion().getValue().toByteArray());</span>
<span class="source-line-no">2606</span><span id="line-2606"> }</span>
<span class="source-line-no">2607</span><span id="line-2607"> sum += regionAction.getActionCount();</span>
<span class="source-line-no">2608</span><span id="line-2608"> }</span>
<span class="source-line-no">2609</span><span id="line-2609"> if (sum &gt; rowSizeWarnThreshold) {</span>
<span class="source-line-no">2610</span><span id="line-2610"> LOG.warn("Large batch operation detected (greater than " + rowSizeWarnThreshold</span>
<span class="source-line-no">2611</span><span id="line-2611"> + ") (HBASE-18023)." + " Requested Number of Rows: " + sum + " Client: "</span>
<span class="source-line-no">2612</span><span id="line-2612"> + RpcServer.getRequestUserName().orElse(null) + "/"</span>
<span class="source-line-no">2613</span><span id="line-2613"> + RpcServer.getRemoteAddress().orElse(null) + " first region in multi=" + firstRegionName);</span>
<span class="source-line-no">2614</span><span id="line-2614"> if (rejectRowsWithSizeOverThreshold) {</span>
<span class="source-line-no">2615</span><span id="line-2615"> throw new ServiceException(</span>
<span class="source-line-no">2616</span><span id="line-2616"> "Rejecting large batch operation for current batch with firstRegionName: "</span>
<span class="source-line-no">2617</span><span id="line-2617"> + firstRegionName + " , Requested Number of Rows: " + sum + " , Size Threshold: "</span>
<span class="source-line-no">2618</span><span id="line-2618"> + rowSizeWarnThreshold);</span>
<span class="source-line-no">2619</span><span id="line-2619"> }</span>
<span class="source-line-no">2620</span><span id="line-2620"> }</span>
<span class="source-line-no">2621</span><span id="line-2621"> }</span>
<span class="source-line-no">2622</span><span id="line-2622"></span>
<span class="source-line-no">2623</span><span id="line-2623"> private void failRegionAction(MultiResponse.Builder responseBuilder,</span>
<span class="source-line-no">2624</span><span id="line-2624"> RegionActionResult.Builder regionActionResultBuilder, RegionAction regionAction,</span>
<span class="source-line-no">2625</span><span id="line-2625"> CellScanner cellScanner, Throwable error) {</span>
<span class="source-line-no">2626</span><span id="line-2626"> rpcServer.getMetrics().exception(error);</span>
<span class="source-line-no">2627</span><span id="line-2627"> regionActionResultBuilder.setException(ResponseConverter.buildException(error));</span>
<span class="source-line-no">2628</span><span id="line-2628"> responseBuilder.addRegionActionResult(regionActionResultBuilder.build());</span>
<span class="source-line-no">2629</span><span id="line-2629"> // All Mutations in this RegionAction not executed as we can not see the Region online here</span>
<span class="source-line-no">2630</span><span id="line-2630"> // in this RS. Will be retried from Client. Skipping all the Cells in CellScanner</span>
<span class="source-line-no">2631</span><span id="line-2631"> // corresponding to these Mutations.</span>
<span class="source-line-no">2632</span><span id="line-2632"> if (cellScanner != null) {</span>
<span class="source-line-no">2633</span><span id="line-2633"> skipCellsForMutations(regionAction.getActionList(), cellScanner);</span>
<span class="source-line-no">2634</span><span id="line-2634"> }</span>
<span class="source-line-no">2635</span><span id="line-2635"> }</span>
<span class="source-line-no">2636</span><span id="line-2636"></span>
<span class="source-line-no">2637</span><span id="line-2637"> private boolean isReplicationRequest(Action action) {</span>
<span class="source-line-no">2638</span><span id="line-2638"> // replication request can only be put or delete.</span>
<span class="source-line-no">2639</span><span id="line-2639"> if (!action.hasMutation()) {</span>
<span class="source-line-no">2640</span><span id="line-2640"> return false;</span>
<span class="source-line-no">2641</span><span id="line-2641"> }</span>
<span class="source-line-no">2642</span><span id="line-2642"> MutationProto mutation = action.getMutation();</span>
<span class="source-line-no">2643</span><span id="line-2643"> MutationType type = mutation.getMutateType();</span>
<span class="source-line-no">2644</span><span id="line-2644"> if (type != MutationType.PUT &amp;&amp; type != MutationType.DELETE) {</span>
<span class="source-line-no">2645</span><span id="line-2645"> return false;</span>
<span class="source-line-no">2646</span><span id="line-2646"> }</span>
<span class="source-line-no">2647</span><span id="line-2647"> // replication will set a special attribute so we can make use of it to decide whether a request</span>
<span class="source-line-no">2648</span><span id="line-2648"> // is for replication.</span>
<span class="source-line-no">2649</span><span id="line-2649"> return mutation.getAttributeList().stream().map(p -&gt; p.getName())</span>
<span class="source-line-no">2650</span><span id="line-2650"> .filter(n -&gt; n.equals(ReplicationUtils.REPLICATION_ATTR_NAME)).findAny().isPresent();</span>
<span class="source-line-no">2651</span><span id="line-2651"> }</span>
<span class="source-line-no">2652</span><span id="line-2652"></span>
<span class="source-line-no">2653</span><span id="line-2653"> /**</span>
<span class="source-line-no">2654</span><span id="line-2654"> * Execute multiple actions on a table: get, mutate, and/or execCoprocessor</span>
<span class="source-line-no">2655</span><span id="line-2655"> * @param rpcc the RPC controller</span>
<span class="source-line-no">2656</span><span id="line-2656"> * @param request the multi request</span>
<span class="source-line-no">2657</span><span id="line-2657"> */</span>
<span class="source-line-no">2658</span><span id="line-2658"> @Override</span>
<span class="source-line-no">2659</span><span id="line-2659"> public MultiResponse multi(final RpcController rpcc, final MultiRequest request)</span>
<span class="source-line-no">2660</span><span id="line-2660"> throws ServiceException {</span>
<span class="source-line-no">2661</span><span id="line-2661"> try {</span>
<span class="source-line-no">2662</span><span id="line-2662"> checkOpen();</span>
<span class="source-line-no">2663</span><span id="line-2663"> } catch (IOException ie) {</span>
<span class="source-line-no">2664</span><span id="line-2664"> throw new ServiceException(ie);</span>
<span class="source-line-no">2665</span><span id="line-2665"> }</span>
<span class="source-line-no">2666</span><span id="line-2666"></span>
<span class="source-line-no">2667</span><span id="line-2667"> checkBatchSizeAndLogLargeSize(request);</span>
<span class="source-line-no">2668</span><span id="line-2668"></span>
<span class="source-line-no">2669</span><span id="line-2669"> // rpc controller is how we bring in data via the back door; it is unprotobuf'ed data.</span>
<span class="source-line-no">2670</span><span id="line-2670"> // It is also the conduit via which we pass back data.</span>
<span class="source-line-no">2671</span><span id="line-2671"> HBaseRpcController controller = (HBaseRpcController) rpcc;</span>
<span class="source-line-no">2672</span><span id="line-2672"> CellScanner cellScanner = controller != null ? getAndReset(controller) : null;</span>
<span class="source-line-no">2673</span><span id="line-2673"></span>
<span class="source-line-no">2674</span><span id="line-2674"> long nonceGroup = request.hasNonceGroup() ? request.getNonceGroup() : HConstants.NO_NONCE;</span>
<span class="source-line-no">2675</span><span id="line-2675"></span>
<span class="source-line-no">2676</span><span id="line-2676"> MultiResponse.Builder responseBuilder = MultiResponse.newBuilder();</span>
<span class="source-line-no">2677</span><span id="line-2677"> RegionActionResult.Builder regionActionResultBuilder = RegionActionResult.newBuilder();</span>
<span class="source-line-no">2678</span><span id="line-2678"> this.rpcMultiRequestCount.increment();</span>
<span class="source-line-no">2679</span><span id="line-2679"> this.requestCount.increment();</span>
<span class="source-line-no">2680</span><span id="line-2680"> ActivePolicyEnforcement spaceQuotaEnforcement = getSpaceQuotaManager().getActiveEnforcements();</span>
<span class="source-line-no">2681</span><span id="line-2681"></span>
<span class="source-line-no">2682</span><span id="line-2682"> // We no longer use MultiRequest#condition. Instead, we use RegionAction#condition. The</span>
<span class="source-line-no">2683</span><span id="line-2683"> // following logic is for backward compatibility as old clients still use</span>
<span class="source-line-no">2684</span><span id="line-2684"> // MultiRequest#condition in case of checkAndMutate with RowMutations.</span>
<span class="source-line-no">2685</span><span id="line-2685"> if (request.hasCondition()) {</span>
<span class="source-line-no">2686</span><span id="line-2686"> if (request.getRegionActionList().isEmpty()) {</span>
<span class="source-line-no">2687</span><span id="line-2687"> // If the region action list is empty, do nothing.</span>
<span class="source-line-no">2688</span><span id="line-2688"> responseBuilder.setProcessed(true);</span>
<span class="source-line-no">2689</span><span id="line-2689"> return responseBuilder.build();</span>
<span class="source-line-no">2690</span><span id="line-2690"> }</span>
<span class="source-line-no">2691</span><span id="line-2691"></span>
<span class="source-line-no">2692</span><span id="line-2692"> RegionAction regionAction = request.getRegionAction(0);</span>
<span class="source-line-no">2693</span><span id="line-2693"></span>
<span class="source-line-no">2694</span><span id="line-2694"> // When request.hasCondition() is true, regionAction.getAtomic() should be always true. So</span>
<span class="source-line-no">2695</span><span id="line-2695"> // we can assume regionAction.getAtomic() is true here.</span>
<span class="source-line-no">2696</span><span id="line-2696"> assert regionAction.getAtomic();</span>
<span class="source-line-no">2697</span><span id="line-2697"></span>
<span class="source-line-no">2698</span><span id="line-2698"> OperationQuota quota;</span>
<span class="source-line-no">2699</span><span id="line-2699"> HRegion region;</span>
<span class="source-line-no">2700</span><span id="line-2700"> RegionSpecifier regionSpecifier = regionAction.getRegion();</span>
<span class="source-line-no">2701</span><span id="line-2701"></span>
<span class="source-line-no">2702</span><span id="line-2702"> try {</span>
<span class="source-line-no">2703</span><span id="line-2703"> region = getRegion(regionSpecifier);</span>
<span class="source-line-no">2704</span><span id="line-2704"> quota = getRpcQuotaManager().checkBatchQuota(region, regionAction.getActionList(),</span>
<span class="source-line-no">2705</span><span id="line-2705"> regionAction.hasCondition());</span>
<span class="source-line-no">2706</span><span id="line-2706"> } catch (IOException e) {</span>
<span class="source-line-no">2707</span><span id="line-2707"> failRegionAction(responseBuilder, regionActionResultBuilder, regionAction, cellScanner, e);</span>
<span class="source-line-no">2708</span><span id="line-2708"> return responseBuilder.build();</span>
<span class="source-line-no">2709</span><span id="line-2709"> }</span>
<span class="source-line-no">2710</span><span id="line-2710"></span>
<span class="source-line-no">2711</span><span id="line-2711"> try {</span>
<span class="source-line-no">2712</span><span id="line-2712"> boolean rejectIfFromClient = shouldRejectRequestsFromClient(region);</span>
<span class="source-line-no">2713</span><span id="line-2713"> // We only allow replication in standby state and it will not set the atomic flag.</span>
<span class="source-line-no">2714</span><span id="line-2714"> if (rejectIfFromClient) {</span>
<span class="source-line-no">2715</span><span id="line-2715"> failRegionAction(responseBuilder, regionActionResultBuilder, regionAction, cellScanner,</span>
<span class="source-line-no">2716</span><span id="line-2716"> new DoNotRetryIOException(</span>
<span class="source-line-no">2717</span><span id="line-2717"> region.getRegionInfo().getRegionNameAsString() + " is in STANDBY state"));</span>
<span class="source-line-no">2718</span><span id="line-2718"> return responseBuilder.build();</span>
<span class="source-line-no">2719</span><span id="line-2719"> }</span>
<span class="source-line-no">2720</span><span id="line-2720"></span>
<span class="source-line-no">2721</span><span id="line-2721"> try {</span>
<span class="source-line-no">2722</span><span id="line-2722"> CheckAndMutateResult result = checkAndMutate(region, regionAction.getActionList(),</span>
<span class="source-line-no">2723</span><span id="line-2723"> cellScanner, request.getCondition(), nonceGroup, spaceQuotaEnforcement);</span>
<span class="source-line-no">2724</span><span id="line-2724"> responseBuilder.setProcessed(result.isSuccess());</span>
<span class="source-line-no">2725</span><span id="line-2725"> ClientProtos.ResultOrException.Builder resultOrExceptionOrBuilder =</span>
<span class="source-line-no">2726</span><span id="line-2726"> ClientProtos.ResultOrException.newBuilder();</span>
<span class="source-line-no">2727</span><span id="line-2727"> for (int i = 0; i &lt; regionAction.getActionCount(); i++) {</span>
<span class="source-line-no">2728</span><span id="line-2728"> // To unify the response format with doNonAtomicRegionMutation and read through</span>
<span class="source-line-no">2729</span><span id="line-2729"> // client's AsyncProcess we have to add an empty result instance per operation</span>
<span class="source-line-no">2730</span><span id="line-2730"> resultOrExceptionOrBuilder.clear();</span>
<span class="source-line-no">2731</span><span id="line-2731"> resultOrExceptionOrBuilder.setIndex(i);</span>
<span class="source-line-no">2732</span><span id="line-2732"> regionActionResultBuilder.addResultOrException(resultOrExceptionOrBuilder.build());</span>
<span class="source-line-no">2733</span><span id="line-2733"> }</span>
<span class="source-line-no">2734</span><span id="line-2734"> } catch (IOException e) {</span>
<span class="source-line-no">2735</span><span id="line-2735"> rpcServer.getMetrics().exception(e);</span>
<span class="source-line-no">2736</span><span id="line-2736"> // As it's an atomic operation with a condition, we may expect it's a global failure.</span>
<span class="source-line-no">2737</span><span id="line-2737"> regionActionResultBuilder.setException(ResponseConverter.buildException(e));</span>
<span class="source-line-no">2738</span><span id="line-2738"> }</span>
<span class="source-line-no">2739</span><span id="line-2739"> } finally {</span>
<span class="source-line-no">2740</span><span id="line-2740"> quota.close();</span>
<span class="source-line-no">2741</span><span id="line-2741"> }</span>
<span class="source-line-no">2742</span><span id="line-2742"></span>
<span class="source-line-no">2743</span><span id="line-2743"> responseBuilder.addRegionActionResult(regionActionResultBuilder.build());</span>
<span class="source-line-no">2744</span><span id="line-2744"> ClientProtos.RegionLoadStats regionLoadStats = region.getLoadStatistics();</span>
<span class="source-line-no">2745</span><span id="line-2745"> if (regionLoadStats != null) {</span>
<span class="source-line-no">2746</span><span id="line-2746"> responseBuilder.setRegionStatistics(MultiRegionLoadStats.newBuilder()</span>
<span class="source-line-no">2747</span><span id="line-2747"> .addRegion(regionSpecifier).addStat(regionLoadStats).build());</span>
<span class="source-line-no">2748</span><span id="line-2748"> }</span>
<span class="source-line-no">2749</span><span id="line-2749"> return responseBuilder.build();</span>
<span class="source-line-no">2750</span><span id="line-2750"> }</span>
<span class="source-line-no">2751</span><span id="line-2751"></span>
<span class="source-line-no">2752</span><span id="line-2752"> // this will contain all the cells that we need to return. It's created later, if needed.</span>
<span class="source-line-no">2753</span><span id="line-2753"> List&lt;ExtendedCellScannable&gt; cellsToReturn = null;</span>
<span class="source-line-no">2754</span><span id="line-2754"> RegionScannersCloseCallBack closeCallBack = null;</span>
<span class="source-line-no">2755</span><span id="line-2755"> RpcCallContext context = RpcServer.getCurrentCall().orElse(null);</span>
<span class="source-line-no">2756</span><span id="line-2756"> Map&lt;RegionSpecifier, ClientProtos.RegionLoadStats&gt; regionStats =</span>
<span class="source-line-no">2757</span><span id="line-2757"> new HashMap&lt;&gt;(request.getRegionActionCount());</span>
<span class="source-line-no">2758</span><span id="line-2758"></span>
<span class="source-line-no">2759</span><span id="line-2759"> for (RegionAction regionAction : request.getRegionActionList()) {</span>
<span class="source-line-no">2760</span><span id="line-2760"> OperationQuota quota;</span>
<span class="source-line-no">2761</span><span id="line-2761"> HRegion region;</span>
<span class="source-line-no">2762</span><span id="line-2762"> RegionSpecifier regionSpecifier = regionAction.getRegion();</span>
<span class="source-line-no">2763</span><span id="line-2763"> regionActionResultBuilder.clear();</span>
<span class="source-line-no">2764</span><span id="line-2764"></span>
<span class="source-line-no">2765</span><span id="line-2765"> try {</span>
<span class="source-line-no">2766</span><span id="line-2766"> region = getRegion(regionSpecifier);</span>
<span class="source-line-no">2767</span><span id="line-2767"> quota = getRpcQuotaManager().checkBatchQuota(region, regionAction.getActionList(),</span>
<span class="source-line-no">2768</span><span id="line-2768"> regionAction.hasCondition());</span>
<span class="source-line-no">2769</span><span id="line-2769"> } catch (IOException e) {</span>
<span class="source-line-no">2770</span><span id="line-2770"> failRegionAction(responseBuilder, regionActionResultBuilder, regionAction, cellScanner, e);</span>
<span class="source-line-no">2771</span><span id="line-2771"> continue; // For this region it's a failure.</span>
<span class="source-line-no">2772</span><span id="line-2772"> }</span>
<span class="source-line-no">2773</span><span id="line-2773"></span>
<span class="source-line-no">2774</span><span id="line-2774"> try {</span>
<span class="source-line-no">2775</span><span id="line-2775"> boolean rejectIfFromClient = shouldRejectRequestsFromClient(region);</span>
<span class="source-line-no">2776</span><span id="line-2776"></span>
<span class="source-line-no">2777</span><span id="line-2777"> if (regionAction.hasCondition()) {</span>
<span class="source-line-no">2778</span><span id="line-2778"> // We only allow replication in standby state and it will not set the atomic flag.</span>
<span class="source-line-no">2779</span><span id="line-2779"> if (rejectIfFromClient) {</span>
<span class="source-line-no">2780</span><span id="line-2780"> failRegionAction(responseBuilder, regionActionResultBuilder, regionAction, cellScanner,</span>
<span class="source-line-no">2781</span><span id="line-2781"> new DoNotRetryIOException(</span>
<span class="source-line-no">2782</span><span id="line-2782"> region.getRegionInfo().getRegionNameAsString() + " is in STANDBY state"));</span>
<span class="source-line-no">2783</span><span id="line-2783"> continue;</span>
<span class="source-line-no">2784</span><span id="line-2784"> }</span>
<span class="source-line-no">2785</span><span id="line-2785"></span>
<span class="source-line-no">2786</span><span id="line-2786"> try {</span>
<span class="source-line-no">2787</span><span id="line-2787"> ClientProtos.ResultOrException.Builder resultOrExceptionOrBuilder =</span>
<span class="source-line-no">2788</span><span id="line-2788"> ClientProtos.ResultOrException.newBuilder();</span>
<span class="source-line-no">2789</span><span id="line-2789"> if (regionAction.getActionCount() == 1) {</span>
<span class="source-line-no">2790</span><span id="line-2790"> CheckAndMutateResult result =</span>
<span class="source-line-no">2791</span><span id="line-2791"> checkAndMutate(region, quota, regionAction.getAction(0).getMutation(), cellScanner,</span>
<span class="source-line-no">2792</span><span id="line-2792"> regionAction.getCondition(), nonceGroup, spaceQuotaEnforcement, context);</span>
<span class="source-line-no">2793</span><span id="line-2793"> regionActionResultBuilder.setProcessed(result.isSuccess());</span>
<span class="source-line-no">2794</span><span id="line-2794"> resultOrExceptionOrBuilder.setIndex(0);</span>
<span class="source-line-no">2795</span><span id="line-2795"> if (result.getResult() != null) {</span>
<span class="source-line-no">2796</span><span id="line-2796"> resultOrExceptionOrBuilder.setResult(ProtobufUtil.toResult(result.getResult()));</span>
<span class="source-line-no">2797</span><span id="line-2797"> }</span>
<span class="source-line-no">2798</span><span id="line-2798"> regionActionResultBuilder.addResultOrException(resultOrExceptionOrBuilder.build());</span>
<span class="source-line-no">2799</span><span id="line-2799"> } else {</span>
<span class="source-line-no">2800</span><span id="line-2800"> CheckAndMutateResult result = checkAndMutate(region, regionAction.getActionList(),</span>
<span class="source-line-no">2801</span><span id="line-2801"> cellScanner, regionAction.getCondition(), nonceGroup, spaceQuotaEnforcement);</span>
<span class="source-line-no">2802</span><span id="line-2802"> regionActionResultBuilder.setProcessed(result.isSuccess());</span>
<span class="source-line-no">2803</span><span id="line-2803"> for (int i = 0; i &lt; regionAction.getActionCount(); i++) {</span>
<span class="source-line-no">2804</span><span id="line-2804"> if (i == 0 &amp;&amp; result.getResult() != null) {</span>
<span class="source-line-no">2805</span><span id="line-2805"> // Set the result of the Increment/Append operations to the first element of the</span>
<span class="source-line-no">2806</span><span id="line-2806"> // ResultOrException list</span>
<span class="source-line-no">2807</span><span id="line-2807"> resultOrExceptionOrBuilder.setIndex(i);</span>
<span class="source-line-no">2808</span><span id="line-2808"> regionActionResultBuilder.addResultOrException(resultOrExceptionOrBuilder</span>
<span class="source-line-no">2809</span><span id="line-2809"> .setResult(ProtobufUtil.toResult(result.getResult())).build());</span>
<span class="source-line-no">2810</span><span id="line-2810"> continue;</span>
<span class="source-line-no">2811</span><span id="line-2811"> }</span>
<span class="source-line-no">2812</span><span id="line-2812"> // To unify the response format with doNonAtomicRegionMutation and read through</span>
<span class="source-line-no">2813</span><span id="line-2813"> // client's AsyncProcess we have to add an empty result instance per operation</span>
<span class="source-line-no">2814</span><span id="line-2814"> resultOrExceptionOrBuilder.clear();</span>
<span class="source-line-no">2815</span><span id="line-2815"> resultOrExceptionOrBuilder.setIndex(i);</span>
<span class="source-line-no">2816</span><span id="line-2816"> regionActionResultBuilder.addResultOrException(resultOrExceptionOrBuilder.build());</span>
<span class="source-line-no">2817</span><span id="line-2817"> }</span>
<span class="source-line-no">2818</span><span id="line-2818"> }</span>
<span class="source-line-no">2819</span><span id="line-2819"> } catch (IOException e) {</span>
<span class="source-line-no">2820</span><span id="line-2820"> rpcServer.getMetrics().exception(e);</span>
<span class="source-line-no">2821</span><span id="line-2821"> // As it's an atomic operation with a condition, we may expect it's a global failure.</span>
<span class="source-line-no">2822</span><span id="line-2822"> regionActionResultBuilder.setException(ResponseConverter.buildException(e));</span>
<span class="source-line-no">2823</span><span id="line-2823"> }</span>
<span class="source-line-no">2824</span><span id="line-2824"> } else if (regionAction.hasAtomic() &amp;&amp; regionAction.getAtomic()) {</span>
<span class="source-line-no">2825</span><span id="line-2825"> // We only allow replication in standby state and it will not set the atomic flag.</span>
<span class="source-line-no">2826</span><span id="line-2826"> if (rejectIfFromClient) {</span>
<span class="source-line-no">2827</span><span id="line-2827"> failRegionAction(responseBuilder, regionActionResultBuilder, regionAction, cellScanner,</span>
<span class="source-line-no">2828</span><span id="line-2828"> new DoNotRetryIOException(</span>
<span class="source-line-no">2829</span><span id="line-2829"> region.getRegionInfo().getRegionNameAsString() + " is in STANDBY state"));</span>
<span class="source-line-no">2830</span><span id="line-2830"> continue;</span>
<span class="source-line-no">2831</span><span id="line-2831"> }</span>
<span class="source-line-no">2832</span><span id="line-2832"> try {</span>
<span class="source-line-no">2833</span><span id="line-2833"> doAtomicBatchOp(regionActionResultBuilder, region, quota, regionAction.getActionList(),</span>
<span class="source-line-no">2834</span><span id="line-2834"> cellScanner, nonceGroup, spaceQuotaEnforcement);</span>
<span class="source-line-no">2835</span><span id="line-2835"> regionActionResultBuilder.setProcessed(true);</span>
<span class="source-line-no">2836</span><span id="line-2836"> // We no longer use MultiResponse#processed. Instead, we use</span>
<span class="source-line-no">2837</span><span id="line-2837"> // RegionActionResult#processed. This is for backward compatibility for old clients.</span>
<span class="source-line-no">2838</span><span id="line-2838"> responseBuilder.setProcessed(true);</span>
<span class="source-line-no">2839</span><span id="line-2839"> } catch (IOException e) {</span>
<span class="source-line-no">2840</span><span id="line-2840"> rpcServer.getMetrics().exception(e);</span>
<span class="source-line-no">2841</span><span id="line-2841"> // As it's atomic, we may expect it's a global failure.</span>
<span class="source-line-no">2842</span><span id="line-2842"> regionActionResultBuilder.setException(ResponseConverter.buildException(e));</span>
<span class="source-line-no">2843</span><span id="line-2843"> }</span>
<span class="source-line-no">2844</span><span id="line-2844"> } else {</span>
<span class="source-line-no">2845</span><span id="line-2845"> if (</span>
<span class="source-line-no">2846</span><span id="line-2846"> rejectIfFromClient &amp;&amp; regionAction.getActionCount() &gt; 0</span>
<span class="source-line-no">2847</span><span id="line-2847"> &amp;&amp; !isReplicationRequest(regionAction.getAction(0))</span>
<span class="source-line-no">2848</span><span id="line-2848"> ) {</span>
<span class="source-line-no">2849</span><span id="line-2849"> // fail if it is not a replication request</span>
<span class="source-line-no">2850</span><span id="line-2850"> failRegionAction(responseBuilder, regionActionResultBuilder, regionAction, cellScanner,</span>
<span class="source-line-no">2851</span><span id="line-2851"> new DoNotRetryIOException(</span>
<span class="source-line-no">2852</span><span id="line-2852"> region.getRegionInfo().getRegionNameAsString() + " is in STANDBY state"));</span>
<span class="source-line-no">2853</span><span id="line-2853"> continue;</span>
<span class="source-line-no">2854</span><span id="line-2854"> }</span>
<span class="source-line-no">2855</span><span id="line-2855"> // doNonAtomicRegionMutation manages the exception internally</span>
<span class="source-line-no">2856</span><span id="line-2856"> if (context != null &amp;&amp; closeCallBack == null) {</span>
<span class="source-line-no">2857</span><span id="line-2857"> // An RpcCallBack that creates a list of scanners that needs to perform callBack</span>
<span class="source-line-no">2858</span><span id="line-2858"> // operation on completion of multiGets.</span>
<span class="source-line-no">2859</span><span id="line-2859"> // Set this only once</span>
<span class="source-line-no">2860</span><span id="line-2860"> closeCallBack = new RegionScannersCloseCallBack();</span>
<span class="source-line-no">2861</span><span id="line-2861"> context.setCallBack(closeCallBack);</span>
<span class="source-line-no">2862</span><span id="line-2862"> }</span>
<span class="source-line-no">2863</span><span id="line-2863"> cellsToReturn = doNonAtomicRegionMutation(region, quota, regionAction, cellScanner,</span>
<span class="source-line-no">2864</span><span id="line-2864"> regionActionResultBuilder, cellsToReturn, nonceGroup, closeCallBack, context,</span>
<span class="source-line-no">2865</span><span id="line-2865"> spaceQuotaEnforcement);</span>
<span class="source-line-no">2866</span><span id="line-2866"> }</span>
<span class="source-line-no">2867</span><span id="line-2867"> } finally {</span>
<span class="source-line-no">2868</span><span id="line-2868"> quota.close();</span>
<span class="source-line-no">2869</span><span id="line-2869"> }</span>
<span class="source-line-no">2870</span><span id="line-2870"></span>
<span class="source-line-no">2871</span><span id="line-2871"> responseBuilder.addRegionActionResult(regionActionResultBuilder.build());</span>
<span class="source-line-no">2872</span><span id="line-2872"> ClientProtos.RegionLoadStats regionLoadStats = region.getLoadStatistics();</span>
<span class="source-line-no">2873</span><span id="line-2873"> if (regionLoadStats != null) {</span>
<span class="source-line-no">2874</span><span id="line-2874"> regionStats.put(regionSpecifier, regionLoadStats);</span>
<span class="source-line-no">2875</span><span id="line-2875"> }</span>
<span class="source-line-no">2876</span><span id="line-2876"> }</span>
<span class="source-line-no">2877</span><span id="line-2877"> // Load the controller with the Cells to return.</span>
<span class="source-line-no">2878</span><span id="line-2878"> if (cellsToReturn != null &amp;&amp; !cellsToReturn.isEmpty() &amp;&amp; controller != null) {</span>
<span class="source-line-no">2879</span><span id="line-2879"> controller.setCellScanner(PrivateCellUtil.createExtendedCellScanner(cellsToReturn));</span>
<span class="source-line-no">2880</span><span id="line-2880"> }</span>
<span class="source-line-no">2881</span><span id="line-2881"></span>
<span class="source-line-no">2882</span><span id="line-2882"> MultiRegionLoadStats.Builder builder = MultiRegionLoadStats.newBuilder();</span>
<span class="source-line-no">2883</span><span id="line-2883"> for (Entry&lt;RegionSpecifier, ClientProtos.RegionLoadStats&gt; stat : regionStats.entrySet()) {</span>
<span class="source-line-no">2884</span><span id="line-2884"> builder.addRegion(stat.getKey());</span>
<span class="source-line-no">2885</span><span id="line-2885"> builder.addStat(stat.getValue());</span>
<span class="source-line-no">2886</span><span id="line-2886"> }</span>
<span class="source-line-no">2887</span><span id="line-2887"> responseBuilder.setRegionStatistics(builder);</span>
<span class="source-line-no">2888</span><span id="line-2888"> return responseBuilder.build();</span>
<span class="source-line-no">2889</span><span id="line-2889"> }</span>
<span class="source-line-no">2890</span><span id="line-2890"></span>
<span class="source-line-no">2891</span><span id="line-2891"> private void skipCellsForMutations(List&lt;Action&gt; actions, CellScanner cellScanner) {</span>
<span class="source-line-no">2892</span><span id="line-2892"> if (cellScanner == null) {</span>
<span class="source-line-no">2893</span><span id="line-2893"> return;</span>
<span class="source-line-no">2894</span><span id="line-2894"> }</span>
<span class="source-line-no">2895</span><span id="line-2895"> for (Action action : actions) {</span>
<span class="source-line-no">2896</span><span id="line-2896"> skipCellsForMutation(action, cellScanner);</span>
<span class="source-line-no">2897</span><span id="line-2897"> }</span>
<span class="source-line-no">2898</span><span id="line-2898"> }</span>
<span class="source-line-no">2899</span><span id="line-2899"></span>
<span class="source-line-no">2900</span><span id="line-2900"> private void skipCellsForMutation(Action action, CellScanner cellScanner) {</span>
<span class="source-line-no">2901</span><span id="line-2901"> if (cellScanner == null) {</span>
<span class="source-line-no">2902</span><span id="line-2902"> return;</span>
<span class="source-line-no">2903</span><span id="line-2903"> }</span>
<span class="source-line-no">2904</span><span id="line-2904"> try {</span>
<span class="source-line-no">2905</span><span id="line-2905"> if (action.hasMutation()) {</span>
<span class="source-line-no">2906</span><span id="line-2906"> MutationProto m = action.getMutation();</span>
<span class="source-line-no">2907</span><span id="line-2907"> if (m.hasAssociatedCellCount()) {</span>
<span class="source-line-no">2908</span><span id="line-2908"> for (int i = 0; i &lt; m.getAssociatedCellCount(); i++) {</span>
<span class="source-line-no">2909</span><span id="line-2909"> cellScanner.advance();</span>
<span class="source-line-no">2910</span><span id="line-2910"> }</span>
<span class="source-line-no">2911</span><span id="line-2911"> }</span>
<span class="source-line-no">2912</span><span id="line-2912"> }</span>
<span class="source-line-no">2913</span><span id="line-2913"> } catch (IOException e) {</span>
<span class="source-line-no">2914</span><span id="line-2914"> // No need to handle these Individual Muatation level issue. Any way this entire RegionAction</span>
<span class="source-line-no">2915</span><span id="line-2915"> // marked as failed as we could not see the Region here. At client side the top level</span>
<span class="source-line-no">2916</span><span id="line-2916"> // RegionAction exception will be considered first.</span>
<span class="source-line-no">2917</span><span id="line-2917"> LOG.error("Error while skipping Cells in CellScanner for invalid Region Mutations", e);</span>
<span class="source-line-no">2918</span><span id="line-2918"> }</span>
<span class="source-line-no">2919</span><span id="line-2919"> }</span>
<span class="source-line-no">2920</span><span id="line-2920"></span>
<span class="source-line-no">2921</span><span id="line-2921"> /**</span>
<span class="source-line-no">2922</span><span id="line-2922"> * Mutate data in a table.</span>
<span class="source-line-no">2923</span><span id="line-2923"> * @param rpcc the RPC controller</span>
<span class="source-line-no">2924</span><span id="line-2924"> * @param request the mutate request</span>
<span class="source-line-no">2925</span><span id="line-2925"> */</span>
<span class="source-line-no">2926</span><span id="line-2926"> @Override</span>
<span class="source-line-no">2927</span><span id="line-2927"> public MutateResponse mutate(final RpcController rpcc, final MutateRequest request)</span>
<span class="source-line-no">2928</span><span id="line-2928"> throws ServiceException {</span>
<span class="source-line-no">2929</span><span id="line-2929"> // rpc controller is how we bring in data via the back door; it is unprotobuf'ed data.</span>
<span class="source-line-no">2930</span><span id="line-2930"> // It is also the conduit via which we pass back data.</span>
<span class="source-line-no">2931</span><span id="line-2931"> HBaseRpcController controller = (HBaseRpcController) rpcc;</span>
<span class="source-line-no">2932</span><span id="line-2932"> CellScanner cellScanner = controller != null ? controller.cellScanner() : null;</span>
<span class="source-line-no">2933</span><span id="line-2933"> OperationQuota quota = null;</span>
<span class="source-line-no">2934</span><span id="line-2934"> RpcCallContext context = RpcServer.getCurrentCall().orElse(null);</span>
<span class="source-line-no">2935</span><span id="line-2935"> // Clear scanner so we are not holding on to reference across call.</span>
<span class="source-line-no">2936</span><span id="line-2936"> if (controller != null) {</span>
<span class="source-line-no">2937</span><span id="line-2937"> controller.setCellScanner(null);</span>
<span class="source-line-no">2938</span><span id="line-2938"> }</span>
<span class="source-line-no">2939</span><span id="line-2939"> try {</span>
<span class="source-line-no">2940</span><span id="line-2940"> checkOpen();</span>
<span class="source-line-no">2941</span><span id="line-2941"> requestCount.increment();</span>
<span class="source-line-no">2942</span><span id="line-2942"> rpcMutateRequestCount.increment();</span>
<span class="source-line-no">2943</span><span id="line-2943"> HRegion region = getRegion(request.getRegion());</span>
<span class="source-line-no">2944</span><span id="line-2944"> rejectIfInStandByState(region);</span>
<span class="source-line-no">2945</span><span id="line-2945"> MutateResponse.Builder builder = MutateResponse.newBuilder();</span>
<span class="source-line-no">2946</span><span id="line-2946"> MutationProto mutation = request.getMutation();</span>
<span class="source-line-no">2947</span><span id="line-2947"> if (!region.getRegionInfo().isMetaRegion()) {</span>
<span class="source-line-no">2948</span><span id="line-2948"> server.getMemStoreFlusher().reclaimMemStoreMemory();</span>
<span class="source-line-no">2949</span><span id="line-2949"> }</span>
<span class="source-line-no">2950</span><span id="line-2950"> long nonceGroup = request.hasNonceGroup() ? request.getNonceGroup() : HConstants.NO_NONCE;</span>
<span class="source-line-no">2951</span><span id="line-2951"> OperationQuota.OperationType operationType = QuotaUtil.getQuotaOperationType(request);</span>
<span class="source-line-no">2952</span><span id="line-2952"> quota = getRpcQuotaManager().checkBatchQuota(region, operationType);</span>
<span class="source-line-no">2953</span><span id="line-2953"> ActivePolicyEnforcement spaceQuotaEnforcement =</span>
<span class="source-line-no">2954</span><span id="line-2954"> getSpaceQuotaManager().getActiveEnforcements();</span>
<span class="source-line-no">2955</span><span id="line-2955"></span>
<span class="source-line-no">2956</span><span id="line-2956"> if (request.hasCondition()) {</span>
<span class="source-line-no">2957</span><span id="line-2957"> CheckAndMutateResult result = checkAndMutate(region, quota, mutation, cellScanner,</span>
<span class="source-line-no">2958</span><span id="line-2958"> request.getCondition(), nonceGroup, spaceQuotaEnforcement, context);</span>
<span class="source-line-no">2959</span><span id="line-2959"> builder.setProcessed(result.isSuccess());</span>
<span class="source-line-no">2960</span><span id="line-2960"> boolean clientCellBlockSupported = isClientCellBlockSupport(context);</span>
<span class="source-line-no">2961</span><span id="line-2961"> addResult(builder, result.getResult(), controller, clientCellBlockSupported);</span>
<span class="source-line-no">2962</span><span id="line-2962"> if (clientCellBlockSupported) {</span>
<span class="source-line-no">2963</span><span id="line-2963"> addSize(context, result.getResult());</span>
<span class="source-line-no">2964</span><span id="line-2964"> }</span>
<span class="source-line-no">2965</span><span id="line-2965"> } else {</span>
<span class="source-line-no">2966</span><span id="line-2966"> Result r = null;</span>
<span class="source-line-no">2967</span><span id="line-2967"> Boolean processed = null;</span>
<span class="source-line-no">2968</span><span id="line-2968"> MutationType type = mutation.getMutateType();</span>
<span class="source-line-no">2969</span><span id="line-2969"> switch (type) {</span>
<span class="source-line-no">2970</span><span id="line-2970"> case APPEND:</span>
<span class="source-line-no">2971</span><span id="line-2971"> // TODO: this doesn't actually check anything.</span>
<span class="source-line-no">2972</span><span id="line-2972"> r = append(region, quota, mutation, cellScanner, nonceGroup, spaceQuotaEnforcement,</span>
<span class="source-line-no">2973</span><span id="line-2973"> context);</span>
<span class="source-line-no">2974</span><span id="line-2974"> break;</span>
<span class="source-line-no">2975</span><span id="line-2975"> case INCREMENT:</span>
<span class="source-line-no">2976</span><span id="line-2976"> // TODO: this doesn't actually check anything.</span>
<span class="source-line-no">2977</span><span id="line-2977"> r = increment(region, quota, mutation, cellScanner, nonceGroup, spaceQuotaEnforcement,</span>
<span class="source-line-no">2978</span><span id="line-2978"> context);</span>
<span class="source-line-no">2979</span><span id="line-2979"> break;</span>
<span class="source-line-no">2980</span><span id="line-2980"> case PUT:</span>
<span class="source-line-no">2981</span><span id="line-2981"> put(region, quota, mutation, cellScanner, spaceQuotaEnforcement);</span>
<span class="source-line-no">2982</span><span id="line-2982"> processed = Boolean.TRUE;</span>
<span class="source-line-no">2983</span><span id="line-2983"> break;</span>
<span class="source-line-no">2984</span><span id="line-2984"> case DELETE:</span>
<span class="source-line-no">2985</span><span id="line-2985"> delete(region, quota, mutation, cellScanner, spaceQuotaEnforcement);</span>
<span class="source-line-no">2986</span><span id="line-2986"> processed = Boolean.TRUE;</span>
<span class="source-line-no">2987</span><span id="line-2987"> break;</span>
<span class="source-line-no">2988</span><span id="line-2988"> default:</span>
<span class="source-line-no">2989</span><span id="line-2989"> throw new DoNotRetryIOException("Unsupported mutate type: " + type.name());</span>
<span class="source-line-no">2990</span><span id="line-2990"> }</span>
<span class="source-line-no">2991</span><span id="line-2991"> if (processed != null) {</span>
<span class="source-line-no">2992</span><span id="line-2992"> builder.setProcessed(processed);</span>
<span class="source-line-no">2993</span><span id="line-2993"> }</span>
<span class="source-line-no">2994</span><span id="line-2994"> boolean clientCellBlockSupported = isClientCellBlockSupport(context);</span>
<span class="source-line-no">2995</span><span id="line-2995"> addResult(builder, r, controller, clientCellBlockSupported);</span>
<span class="source-line-no">2996</span><span id="line-2996"> if (clientCellBlockSupported) {</span>
<span class="source-line-no">2997</span><span id="line-2997"> addSize(context, r);</span>
<span class="source-line-no">2998</span><span id="line-2998"> }</span>
<span class="source-line-no">2999</span><span id="line-2999"> }</span>
<span class="source-line-no">3000</span><span id="line-3000"> return builder.build();</span>
<span class="source-line-no">3001</span><span id="line-3001"> } catch (IOException ie) {</span>
<span class="source-line-no">3002</span><span id="line-3002"> server.checkFileSystem();</span>
<span class="source-line-no">3003</span><span id="line-3003"> throw new ServiceException(ie);</span>
<span class="source-line-no">3004</span><span id="line-3004"> } finally {</span>
<span class="source-line-no">3005</span><span id="line-3005"> if (quota != null) {</span>
<span class="source-line-no">3006</span><span id="line-3006"> quota.close();</span>
<span class="source-line-no">3007</span><span id="line-3007"> }</span>
<span class="source-line-no">3008</span><span id="line-3008"> }</span>
<span class="source-line-no">3009</span><span id="line-3009"> }</span>
<span class="source-line-no">3010</span><span id="line-3010"></span>
<span class="source-line-no">3011</span><span id="line-3011"> private void put(HRegion region, OperationQuota quota, MutationProto mutation,</span>
<span class="source-line-no">3012</span><span id="line-3012"> CellScanner cellScanner, ActivePolicyEnforcement spaceQuota) throws IOException {</span>
<span class="source-line-no">3013</span><span id="line-3013"> long before = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">3014</span><span id="line-3014"> Put put = ProtobufUtil.toPut(mutation, cellScanner);</span>
<span class="source-line-no">3015</span><span id="line-3015"> checkCellSizeLimit(region, put);</span>
<span class="source-line-no">3016</span><span id="line-3016"> spaceQuota.getPolicyEnforcement(region).check(put);</span>
<span class="source-line-no">3017</span><span id="line-3017"> quota.addMutation(put);</span>
<span class="source-line-no">3018</span><span id="line-3018"> region.put(put);</span>
<span class="source-line-no">3019</span><span id="line-3019"></span>
<span class="source-line-no">3020</span><span id="line-3020"> MetricsRegionServer metricsRegionServer = server.getMetrics();</span>
<span class="source-line-no">3021</span><span id="line-3021"> if (metricsRegionServer != null) {</span>
<span class="source-line-no">3022</span><span id="line-3022"> long after = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">3023</span><span id="line-3023"> metricsRegionServer.updatePut(region, after - before);</span>
<span class="source-line-no">3024</span><span id="line-3024"> }</span>
<span class="source-line-no">3025</span><span id="line-3025"> }</span>
<span class="source-line-no">3026</span><span id="line-3026"></span>
<span class="source-line-no">3027</span><span id="line-3027"> private void delete(HRegion region, OperationQuota quota, MutationProto mutation,</span>
<span class="source-line-no">3028</span><span id="line-3028"> CellScanner cellScanner, ActivePolicyEnforcement spaceQuota) throws IOException {</span>
<span class="source-line-no">3029</span><span id="line-3029"> long before = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">3030</span><span id="line-3030"> Delete delete = ProtobufUtil.toDelete(mutation, cellScanner);</span>
<span class="source-line-no">3031</span><span id="line-3031"> checkCellSizeLimit(region, delete);</span>
<span class="source-line-no">3032</span><span id="line-3032"> spaceQuota.getPolicyEnforcement(region).check(delete);</span>
<span class="source-line-no">3033</span><span id="line-3033"> quota.addMutation(delete);</span>
<span class="source-line-no">3034</span><span id="line-3034"> region.delete(delete);</span>
<span class="source-line-no">3035</span><span id="line-3035"></span>
<span class="source-line-no">3036</span><span id="line-3036"> MetricsRegionServer metricsRegionServer = server.getMetrics();</span>
<span class="source-line-no">3037</span><span id="line-3037"> if (metricsRegionServer != null) {</span>
<span class="source-line-no">3038</span><span id="line-3038"> long after = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">3039</span><span id="line-3039"> metricsRegionServer.updateDelete(region, after - before);</span>
<span class="source-line-no">3040</span><span id="line-3040"> }</span>
<span class="source-line-no">3041</span><span id="line-3041"> }</span>
<span class="source-line-no">3042</span><span id="line-3042"></span>
<span class="source-line-no">3043</span><span id="line-3043"> private CheckAndMutateResult checkAndMutate(HRegion region, OperationQuota quota,</span>
<span class="source-line-no">3044</span><span id="line-3044"> MutationProto mutation, CellScanner cellScanner, Condition condition, long nonceGroup,</span>
<span class="source-line-no">3045</span><span id="line-3045"> ActivePolicyEnforcement spaceQuota, RpcCallContext context) throws IOException {</span>
<span class="source-line-no">3046</span><span id="line-3046"> long before = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">3047</span><span id="line-3047"> long blockBytesScannedBefore = context != null ? context.getBlockBytesScanned() : 0;</span>
<span class="source-line-no">3048</span><span id="line-3048"> CheckAndMutate checkAndMutate = ProtobufUtil.toCheckAndMutate(condition, mutation, cellScanner);</span>
<span class="source-line-no">3049</span><span id="line-3049"> long nonce = mutation.hasNonce() ? mutation.getNonce() : HConstants.NO_NONCE;</span>
<span class="source-line-no">3050</span><span id="line-3050"> checkCellSizeLimit(region, (Mutation) checkAndMutate.getAction());</span>
<span class="source-line-no">3051</span><span id="line-3051"> spaceQuota.getPolicyEnforcement(region).check((Mutation) checkAndMutate.getAction());</span>
<span class="source-line-no">3052</span><span id="line-3052"> quota.addMutation((Mutation) checkAndMutate.getAction());</span>
<span class="source-line-no">3053</span><span id="line-3053"></span>
<span class="source-line-no">3054</span><span id="line-3054"> CheckAndMutateResult result = null;</span>
<span class="source-line-no">3055</span><span id="line-3055"> if (region.getCoprocessorHost() != null) {</span>
<span class="source-line-no">3056</span><span id="line-3056"> result = region.getCoprocessorHost().preCheckAndMutate(checkAndMutate);</span>
<span class="source-line-no">3057</span><span id="line-3057"> }</span>
<span class="source-line-no">3058</span><span id="line-3058"> if (result == null) {</span>
<span class="source-line-no">3059</span><span id="line-3059"> result = region.checkAndMutate(checkAndMutate, nonceGroup, nonce);</span>
<span class="source-line-no">3060</span><span id="line-3060"> if (region.getCoprocessorHost() != null) {</span>
<span class="source-line-no">3061</span><span id="line-3061"> result = region.getCoprocessorHost().postCheckAndMutate(checkAndMutate, result);</span>
<span class="source-line-no">3062</span><span id="line-3062"> }</span>
<span class="source-line-no">3063</span><span id="line-3063"> }</span>
<span class="source-line-no">3064</span><span id="line-3064"> MetricsRegionServer metricsRegionServer = server.getMetrics();</span>
<span class="source-line-no">3065</span><span id="line-3065"> if (metricsRegionServer != null) {</span>
<span class="source-line-no">3066</span><span id="line-3066"> long after = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">3067</span><span id="line-3067"> long blockBytesScanned =</span>
<span class="source-line-no">3068</span><span id="line-3068"> context != null ? context.getBlockBytesScanned() - blockBytesScannedBefore : 0;</span>
<span class="source-line-no">3069</span><span id="line-3069"> metricsRegionServer.updateCheckAndMutate(region, after - before, blockBytesScanned);</span>
<span class="source-line-no">3070</span><span id="line-3070"></span>
<span class="source-line-no">3071</span><span id="line-3071"> MutationType type = mutation.getMutateType();</span>
<span class="source-line-no">3072</span><span id="line-3072"> switch (type) {</span>
<span class="source-line-no">3073</span><span id="line-3073"> case PUT:</span>
<span class="source-line-no">3074</span><span id="line-3074"> metricsRegionServer.updateCheckAndPut(region, after - before);</span>
<span class="source-line-no">3075</span><span id="line-3075"> break;</span>
<span class="source-line-no">3076</span><span id="line-3076"> case DELETE:</span>
<span class="source-line-no">3077</span><span id="line-3077"> metricsRegionServer.updateCheckAndDelete(region, after - before);</span>
<span class="source-line-no">3078</span><span id="line-3078"> break;</span>
<span class="source-line-no">3079</span><span id="line-3079"> default:</span>
<span class="source-line-no">3080</span><span id="line-3080"> break;</span>
<span class="source-line-no">3081</span><span id="line-3081"> }</span>
<span class="source-line-no">3082</span><span id="line-3082"> }</span>
<span class="source-line-no">3083</span><span id="line-3083"> return result;</span>
<span class="source-line-no">3084</span><span id="line-3084"> }</span>
<span class="source-line-no">3085</span><span id="line-3085"></span>
<span class="source-line-no">3086</span><span id="line-3086"> // This is used to keep compatible with the old client implementation. Consider remove it if we</span>
<span class="source-line-no">3087</span><span id="line-3087"> // decide to drop the support of the client that still sends close request to a region scanner</span>
<span class="source-line-no">3088</span><span id="line-3088"> // which has already been exhausted.</span>
<span class="source-line-no">3089</span><span id="line-3089"> @Deprecated</span>
<span class="source-line-no">3090</span><span id="line-3090"> private static final IOException SCANNER_ALREADY_CLOSED = new IOException() {</span>
<span class="source-line-no">3091</span><span id="line-3091"></span>
<span class="source-line-no">3092</span><span id="line-3092"> private static final long serialVersionUID = -4305297078988180130L;</span>
<span class="source-line-no">3093</span><span id="line-3093"></span>
<span class="source-line-no">3094</span><span id="line-3094"> @Override</span>
<span class="source-line-no">3095</span><span id="line-3095"> public synchronized Throwable fillInStackTrace() {</span>
<span class="source-line-no">3096</span><span id="line-3096"> return this;</span>
<span class="source-line-no">3097</span><span id="line-3097"> }</span>
<span class="source-line-no">3098</span><span id="line-3098"> };</span>
<span class="source-line-no">3099</span><span id="line-3099"></span>
<span class="source-line-no">3100</span><span id="line-3100"> private RegionScannerHolder getRegionScanner(ScanRequest request) throws IOException {</span>
<span class="source-line-no">3101</span><span id="line-3101"> String scannerName = toScannerName(request.getScannerId());</span>
<span class="source-line-no">3102</span><span id="line-3102"> RegionScannerHolder rsh = this.scanners.get(scannerName);</span>
<span class="source-line-no">3103</span><span id="line-3103"> if (rsh == null) {</span>
<span class="source-line-no">3104</span><span id="line-3104"> // just ignore the next or close request if scanner does not exists.</span>
<span class="source-line-no">3105</span><span id="line-3105"> Long lastCallSeq = closedScanners.getIfPresent(scannerName);</span>
<span class="source-line-no">3106</span><span id="line-3106"> if (lastCallSeq != null) {</span>
<span class="source-line-no">3107</span><span id="line-3107"> // Check the sequence number to catch if the last call was incorrectly retried.</span>
<span class="source-line-no">3108</span><span id="line-3108"> // The only allowed scenario is when the scanner is exhausted and one more scan</span>
<span class="source-line-no">3109</span><span id="line-3109"> // request arrives - in this case returning 0 rows is correct.</span>
<span class="source-line-no">3110</span><span id="line-3110"> if (request.hasNextCallSeq() &amp;&amp; request.getNextCallSeq() != lastCallSeq + 1) {</span>
<span class="source-line-no">3111</span><span id="line-3111"> throw new OutOfOrderScannerNextException("Expected nextCallSeq for closed request: "</span>
<span class="source-line-no">3112</span><span id="line-3112"> + (lastCallSeq + 1) + " But the nextCallSeq got from client: "</span>
<span class="source-line-no">3113</span><span id="line-3113"> + request.getNextCallSeq() + "; request=" + TextFormat.shortDebugString(request));</span>
<span class="source-line-no">3114</span><span id="line-3114"> } else {</span>
<span class="source-line-no">3115</span><span id="line-3115"> throw SCANNER_ALREADY_CLOSED;</span>
<span class="source-line-no">3116</span><span id="line-3116"> }</span>
<span class="source-line-no">3117</span><span id="line-3117"> } else {</span>
<span class="source-line-no">3118</span><span id="line-3118"> LOG.warn("Client tried to access missing scanner " + scannerName);</span>
<span class="source-line-no">3119</span><span id="line-3119"> throw new UnknownScannerException(</span>
<span class="source-line-no">3120</span><span id="line-3120"> "Unknown scanner '" + scannerName + "'. This can happen due to any of the following "</span>
<span class="source-line-no">3121</span><span id="line-3121"> + "reasons: a) Scanner id given is wrong, b) Scanner lease expired because of "</span>
<span class="source-line-no">3122</span><span id="line-3122"> + "long wait between consecutive client checkins, c) Server may be closing down, "</span>
<span class="source-line-no">3123</span><span id="line-3123"> + "d) RegionServer restart during upgrade.\nIf the issue is due to reason (b), a "</span>
<span class="source-line-no">3124</span><span id="line-3124"> + "possible fix would be increasing the value of"</span>
<span class="source-line-no">3125</span><span id="line-3125"> + "'hbase.client.scanner.timeout.period' configuration.");</span>
<span class="source-line-no">3126</span><span id="line-3126"> }</span>
<span class="source-line-no">3127</span><span id="line-3127"> }</span>
<span class="source-line-no">3128</span><span id="line-3128"> rejectIfInStandByState(rsh.r);</span>
<span class="source-line-no">3129</span><span id="line-3129"> RegionInfo hri = rsh.s.getRegionInfo();</span>
<span class="source-line-no">3130</span><span id="line-3130"> // Yes, should be the same instance</span>
<span class="source-line-no">3131</span><span id="line-3131"> if (server.getOnlineRegion(hri.getRegionName()) != rsh.r) {</span>
<span class="source-line-no">3132</span><span id="line-3132"> String msg = "Region has changed on the scanner " + scannerName + ": regionName="</span>
<span class="source-line-no">3133</span><span id="line-3133"> + hri.getRegionNameAsString() + ", scannerRegionName=" + rsh.r;</span>
<span class="source-line-no">3134</span><span id="line-3134"> LOG.warn(msg + ", closing...");</span>
<span class="source-line-no">3135</span><span id="line-3135"> scanners.remove(scannerName);</span>
<span class="source-line-no">3136</span><span id="line-3136"> try {</span>
<span class="source-line-no">3137</span><span id="line-3137"> rsh.s.close();</span>
<span class="source-line-no">3138</span><span id="line-3138"> } catch (IOException e) {</span>
<span class="source-line-no">3139</span><span id="line-3139"> LOG.warn("Getting exception closing " + scannerName, e);</span>
<span class="source-line-no">3140</span><span id="line-3140"> } finally {</span>
<span class="source-line-no">3141</span><span id="line-3141"> try {</span>
<span class="source-line-no">3142</span><span id="line-3142"> server.getLeaseManager().cancelLease(scannerName);</span>
<span class="source-line-no">3143</span><span id="line-3143"> } catch (LeaseException e) {</span>
<span class="source-line-no">3144</span><span id="line-3144"> LOG.warn("Getting exception closing " + scannerName, e);</span>
<span class="source-line-no">3145</span><span id="line-3145"> }</span>
<span class="source-line-no">3146</span><span id="line-3146"> }</span>
<span class="source-line-no">3147</span><span id="line-3147"> throw new NotServingRegionException(msg);</span>
<span class="source-line-no">3148</span><span id="line-3148"> }</span>
<span class="source-line-no">3149</span><span id="line-3149"> return rsh;</span>
<span class="source-line-no">3150</span><span id="line-3150"> }</span>
<span class="source-line-no">3151</span><span id="line-3151"></span>
<span class="source-line-no">3152</span><span id="line-3152"> /**</span>
<span class="source-line-no">3153</span><span id="line-3153"> * @return Pair with scannerName key to use with this new Scanner and its RegionScannerHolder</span>
<span class="source-line-no">3154</span><span id="line-3154"> * value.</span>
<span class="source-line-no">3155</span><span id="line-3155"> */</span>
<span class="source-line-no">3156</span><span id="line-3156"> private Pair&lt;String, RegionScannerHolder&gt; newRegionScanner(ScanRequest request, HRegion region,</span>
<span class="source-line-no">3157</span><span id="line-3157"> ScanResponse.Builder builder) throws IOException {</span>
<span class="source-line-no">3158</span><span id="line-3158"> rejectIfInStandByState(region);</span>
<span class="source-line-no">3159</span><span id="line-3159"> ClientProtos.Scan protoScan = request.getScan();</span>
<span class="source-line-no">3160</span><span id="line-3160"> boolean isLoadingCfsOnDemandSet = protoScan.hasLoadColumnFamiliesOnDemand();</span>
<span class="source-line-no">3161</span><span id="line-3161"> Scan scan = ProtobufUtil.toScan(protoScan);</span>
<span class="source-line-no">3162</span><span id="line-3162"> // if the request doesn't set this, get the default region setting.</span>
<span class="source-line-no">3163</span><span id="line-3163"> if (!isLoadingCfsOnDemandSet) {</span>
<span class="source-line-no">3164</span><span id="line-3164"> scan.setLoadColumnFamiliesOnDemand(region.isLoadingCfsOnDemandDefault());</span>
<span class="source-line-no">3165</span><span id="line-3165"> }</span>
<span class="source-line-no">3166</span><span id="line-3166"></span>
<span class="source-line-no">3167</span><span id="line-3167"> if (!scan.hasFamilies()) {</span>
<span class="source-line-no">3168</span><span id="line-3168"> // Adding all families to scanner</span>
<span class="source-line-no">3169</span><span id="line-3169"> for (byte[] family : region.getTableDescriptor().getColumnFamilyNames()) {</span>
<span class="source-line-no">3170</span><span id="line-3170"> scan.addFamily(family);</span>
<span class="source-line-no">3171</span><span id="line-3171"> }</span>
<span class="source-line-no">3172</span><span id="line-3172"> }</span>
<span class="source-line-no">3173</span><span id="line-3173"> if (region.getCoprocessorHost() != null) {</span>
<span class="source-line-no">3174</span><span id="line-3174"> // preScannerOpen is not allowed to return a RegionScanner. Only post hook can create a</span>
<span class="source-line-no">3175</span><span id="line-3175"> // wrapper for the core created RegionScanner</span>
<span class="source-line-no">3176</span><span id="line-3176"> region.getCoprocessorHost().preScannerOpen(scan);</span>
<span class="source-line-no">3177</span><span id="line-3177"> }</span>
<span class="source-line-no">3178</span><span id="line-3178"> RegionScannerImpl coreScanner = region.getScanner(scan);</span>
<span class="source-line-no">3179</span><span id="line-3179"> Shipper shipper = coreScanner;</span>
<span class="source-line-no">3180</span><span id="line-3180"> RegionScanner scanner = coreScanner;</span>
<span class="source-line-no">3181</span><span id="line-3181"> try {</span>
<span class="source-line-no">3182</span><span id="line-3182"> if (region.getCoprocessorHost() != null) {</span>
<span class="source-line-no">3183</span><span id="line-3183"> scanner = region.getCoprocessorHost().postScannerOpen(scan, scanner);</span>
<span class="source-line-no">3184</span><span id="line-3184"> }</span>
<span class="source-line-no">3185</span><span id="line-3185"> } catch (Exception e) {</span>
<span class="source-line-no">3186</span><span id="line-3186"> // Although region coprocessor is for advanced users and they should take care of the</span>
<span class="source-line-no">3187</span><span id="line-3187"> // implementation to not damage the HBase system, closing the scanner on exception here does</span>
<span class="source-line-no">3188</span><span id="line-3188"> // not have any bad side effect, so let's do it</span>
<span class="source-line-no">3189</span><span id="line-3189"> scanner.close();</span>
<span class="source-line-no">3190</span><span id="line-3190"> throw e;</span>
<span class="source-line-no">3191</span><span id="line-3191"> }</span>
<span class="source-line-no">3192</span><span id="line-3192"> long scannerId = scannerIdGenerator.generateNewScannerId();</span>
<span class="source-line-no">3193</span><span id="line-3193"> builder.setScannerId(scannerId);</span>
<span class="source-line-no">3194</span><span id="line-3194"> builder.setMvccReadPoint(scanner.getMvccReadPoint());</span>
<span class="source-line-no">3195</span><span id="line-3195"> builder.setTtl(scannerLeaseTimeoutPeriod);</span>
<span class="source-line-no">3196</span><span id="line-3196"> String scannerName = toScannerName(scannerId);</span>
<span class="source-line-no">3197</span><span id="line-3197"></span>
<span class="source-line-no">3198</span><span id="line-3198"> boolean fullRegionScan =</span>
<span class="source-line-no">3199</span><span id="line-3199"> !region.getRegionInfo().getTable().isSystemTable() &amp;&amp; isFullRegionScan(scan, region);</span>
<span class="source-line-no">3200</span><span id="line-3200"></span>
<span class="source-line-no">3201</span><span id="line-3201"> return new Pair&lt;String, RegionScannerHolder&gt;(scannerName,</span>
<span class="source-line-no">3202</span><span id="line-3202"> addScanner(scannerName, scanner, shipper, region, scan.isNeedCursorResult(), fullRegionScan));</span>
<span class="source-line-no">3203</span><span id="line-3203"> }</span>
<span class="source-line-no">3204</span><span id="line-3204"></span>
<span class="source-line-no">3205</span><span id="line-3205"> /**</span>
<span class="source-line-no">3206</span><span id="line-3206"> * The returned String is used as key doing look up of outstanding Scanners in this Servers'</span>
<span class="source-line-no">3207</span><span id="line-3207"> * this.scanners, the Map of outstanding scanners and their current state.</span>
<span class="source-line-no">3208</span><span id="line-3208"> * @param scannerId A scanner long id.</span>
<span class="source-line-no">3209</span><span id="line-3209"> * @return The long id as a String.</span>
<span class="source-line-no">3210</span><span id="line-3210"> */</span>
<span class="source-line-no">3211</span><span id="line-3211"> private static String toScannerName(long scannerId) {</span>
<span class="source-line-no">3212</span><span id="line-3212"> return Long.toString(scannerId);</span>
<span class="source-line-no">3213</span><span id="line-3213"> }</span>
<span class="source-line-no">3214</span><span id="line-3214"></span>
<span class="source-line-no">3215</span><span id="line-3215"> private void checkScanNextCallSeq(ScanRequest request, RegionScannerHolder rsh)</span>
<span class="source-line-no">3216</span><span id="line-3216"> throws OutOfOrderScannerNextException {</span>
<span class="source-line-no">3217</span><span id="line-3217"> // if nextCallSeq does not match throw Exception straight away. This needs to be</span>
<span class="source-line-no">3218</span><span id="line-3218"> // performed even before checking of Lease.</span>
<span class="source-line-no">3219</span><span id="line-3219"> // See HBASE-5974</span>
<span class="source-line-no">3220</span><span id="line-3220"> if (request.hasNextCallSeq()) {</span>
<span class="source-line-no">3221</span><span id="line-3221"> long callSeq = request.getNextCallSeq();</span>
<span class="source-line-no">3222</span><span id="line-3222"> if (!rsh.incNextCallSeq(callSeq)) {</span>
<span class="source-line-no">3223</span><span id="line-3223"> throw new OutOfOrderScannerNextException(</span>
<span class="source-line-no">3224</span><span id="line-3224"> "Expected nextCallSeq: " + rsh.getNextCallSeq() + " But the nextCallSeq got from client: "</span>
<span class="source-line-no">3225</span><span id="line-3225"> + request.getNextCallSeq() + "; request=" + TextFormat.shortDebugString(request));</span>
<span class="source-line-no">3226</span><span id="line-3226"> }</span>
<span class="source-line-no">3227</span><span id="line-3227"> }</span>
<span class="source-line-no">3228</span><span id="line-3228"> }</span>
<span class="source-line-no">3229</span><span id="line-3229"></span>
<span class="source-line-no">3230</span><span id="line-3230"> private void addScannerLeaseBack(LeaseManager.Lease lease) {</span>
<span class="source-line-no">3231</span><span id="line-3231"> try {</span>
<span class="source-line-no">3232</span><span id="line-3232"> server.getLeaseManager().addLease(lease);</span>
<span class="source-line-no">3233</span><span id="line-3233"> } catch (LeaseStillHeldException e) {</span>
<span class="source-line-no">3234</span><span id="line-3234"> // should not happen as the scanner id is unique.</span>
<span class="source-line-no">3235</span><span id="line-3235"> throw new AssertionError(e);</span>
<span class="source-line-no">3236</span><span id="line-3236"> }</span>
<span class="source-line-no">3237</span><span id="line-3237"> }</span>
<span class="source-line-no">3238</span><span id="line-3238"></span>
<span class="source-line-no">3239</span><span id="line-3239"> // visible for testing only</span>
<span class="source-line-no">3240</span><span id="line-3240"> long getTimeLimit(RpcCall rpcCall, HBaseRpcController controller,</span>
<span class="source-line-no">3241</span><span id="line-3241"> boolean allowHeartbeatMessages) {</span>
<span class="source-line-no">3242</span><span id="line-3242"> // Set the time limit to be half of the more restrictive timeout value (one of the</span>
<span class="source-line-no">3243</span><span id="line-3243"> // timeout values must be positive). In the event that both values are positive, the</span>
<span class="source-line-no">3244</span><span id="line-3244"> // more restrictive of the two is used to calculate the limit.</span>
<span class="source-line-no">3245</span><span id="line-3245"> if (allowHeartbeatMessages) {</span>
<span class="source-line-no">3246</span><span id="line-3246"> long now = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">3247</span><span id="line-3247"> long remainingTimeout = getRemainingRpcTimeout(rpcCall, controller, now);</span>
<span class="source-line-no">3248</span><span id="line-3248"> if (scannerLeaseTimeoutPeriod &gt; 0 || remainingTimeout &gt; 0) {</span>
<span class="source-line-no">3249</span><span id="line-3249"> long timeLimitDelta;</span>
<span class="source-line-no">3250</span><span id="line-3250"> if (scannerLeaseTimeoutPeriod &gt; 0 &amp;&amp; remainingTimeout &gt; 0) {</span>
<span class="source-line-no">3251</span><span id="line-3251"> timeLimitDelta = Math.min(scannerLeaseTimeoutPeriod, remainingTimeout);</span>
<span class="source-line-no">3252</span><span id="line-3252"> } else {</span>
<span class="source-line-no">3253</span><span id="line-3253"> timeLimitDelta =</span>
<span class="source-line-no">3254</span><span id="line-3254"> scannerLeaseTimeoutPeriod &gt; 0 ? scannerLeaseTimeoutPeriod : remainingTimeout;</span>
<span class="source-line-no">3255</span><span id="line-3255"> }</span>
<span class="source-line-no">3256</span><span id="line-3256"></span>
<span class="source-line-no">3257</span><span id="line-3257"> // Use half of whichever timeout value was more restrictive... But don't allow</span>
<span class="source-line-no">3258</span><span id="line-3258"> // the time limit to be less than the allowable minimum (could cause an</span>
<span class="source-line-no">3259</span><span id="line-3259"> // immediate timeout before scanning any data).</span>
<span class="source-line-no">3260</span><span id="line-3260"> timeLimitDelta = Math.max(timeLimitDelta / 2, minimumScanTimeLimitDelta);</span>
<span class="source-line-no">3261</span><span id="line-3261"> return now + timeLimitDelta;</span>
<span class="source-line-no">3262</span><span id="line-3262"> }</span>
<span class="source-line-no">3263</span><span id="line-3263"> }</span>
<span class="source-line-no">3264</span><span id="line-3264"> // Default value of timeLimit is negative to indicate no timeLimit should be</span>
<span class="source-line-no">3265</span><span id="line-3265"> // enforced.</span>
<span class="source-line-no">3266</span><span id="line-3266"> return -1L;</span>
<span class="source-line-no">3267</span><span id="line-3267"> }</span>
<span class="source-line-no">3268</span><span id="line-3268"></span>
<span class="source-line-no">3269</span><span id="line-3269"> private long getRemainingRpcTimeout(RpcCall call, HBaseRpcController controller, long now) {</span>
<span class="source-line-no">3270</span><span id="line-3270"> long timeout;</span>
<span class="source-line-no">3271</span><span id="line-3271"> if (controller != null &amp;&amp; controller.getCallTimeout() &gt; 0) {</span>
<span class="source-line-no">3272</span><span id="line-3272"> timeout = controller.getCallTimeout();</span>
<span class="source-line-no">3273</span><span id="line-3273"> } else if (rpcTimeout &gt; 0) {</span>
<span class="source-line-no">3274</span><span id="line-3274"> timeout = rpcTimeout;</span>
<span class="source-line-no">3275</span><span id="line-3275"> } else {</span>
<span class="source-line-no">3276</span><span id="line-3276"> return -1;</span>
<span class="source-line-no">3277</span><span id="line-3277"> }</span>
<span class="source-line-no">3278</span><span id="line-3278"> if (call != null) {</span>
<span class="source-line-no">3279</span><span id="line-3279"> timeout -= (now - call.getReceiveTime());</span>
<span class="source-line-no">3280</span><span id="line-3280"> }</span>
<span class="source-line-no">3281</span><span id="line-3281"> // getTimeLimit ignores values &lt;= 0, but timeout may now be negative if queue time was high.</span>
<span class="source-line-no">3282</span><span id="line-3282"> // return minimum value here in that case so we count this in calculating the final delta.</span>
<span class="source-line-no">3283</span><span id="line-3283"> return Math.max(minimumScanTimeLimitDelta, timeout);</span>
<span class="source-line-no">3284</span><span id="line-3284"> }</span>
<span class="source-line-no">3285</span><span id="line-3285"></span>
<span class="source-line-no">3286</span><span id="line-3286"> private void checkLimitOfRows(int numOfCompleteRows, int limitOfRows, boolean moreRows,</span>
<span class="source-line-no">3287</span><span id="line-3287"> ScannerContext scannerContext, ScanResponse.Builder builder) {</span>
<span class="source-line-no">3288</span><span id="line-3288"> if (numOfCompleteRows &gt;= limitOfRows) {</span>
<span class="source-line-no">3289</span><span id="line-3289"> if (LOG.isTraceEnabled()) {</span>
<span class="source-line-no">3290</span><span id="line-3290"> LOG.trace("Done scanning, limit of rows reached, moreRows: " + moreRows</span>
<span class="source-line-no">3291</span><span id="line-3291"> + " scannerContext: " + scannerContext);</span>
<span class="source-line-no">3292</span><span id="line-3292"> }</span>
<span class="source-line-no">3293</span><span id="line-3293"> builder.setMoreResults(false);</span>
<span class="source-line-no">3294</span><span id="line-3294"> }</span>
<span class="source-line-no">3295</span><span id="line-3295"> }</span>
<span class="source-line-no">3296</span><span id="line-3296"></span>
<span class="source-line-no">3297</span><span id="line-3297"> // return whether we have more results in region.</span>
<span class="source-line-no">3298</span><span id="line-3298"> private void scan(HBaseRpcController controller, ScanRequest request, RegionScannerHolder rsh,</span>
<span class="source-line-no">3299</span><span id="line-3299"> long maxQuotaResultSize, int maxResults, int limitOfRows, List&lt;Result&gt; results,</span>
<span class="source-line-no">3300</span><span id="line-3300"> ScanResponse.Builder builder, RpcCall rpcCall) throws IOException {</span>
<span class="source-line-no">3301</span><span id="line-3301"> HRegion region = rsh.r;</span>
<span class="source-line-no">3302</span><span id="line-3302"> RegionScanner scanner = rsh.s;</span>
<span class="source-line-no">3303</span><span id="line-3303"> long maxResultSize;</span>
<span class="source-line-no">3304</span><span id="line-3304"> if (scanner.getMaxResultSize() &gt; 0) {</span>
<span class="source-line-no">3305</span><span id="line-3305"> maxResultSize = Math.min(scanner.getMaxResultSize(), maxQuotaResultSize);</span>
<span class="source-line-no">3306</span><span id="line-3306"> } else {</span>
<span class="source-line-no">3307</span><span id="line-3307"> maxResultSize = maxQuotaResultSize;</span>
<span class="source-line-no">3308</span><span id="line-3308"> }</span>
<span class="source-line-no">3309</span><span id="line-3309"> // This is cells inside a row. Default size is 10 so if many versions or many cfs,</span>
<span class="source-line-no">3310</span><span id="line-3310"> // then we'll resize. Resizings show in profiler. Set it higher than 10. For now</span>
<span class="source-line-no">3311</span><span id="line-3311"> // arbitrary 32. TODO: keep record of general size of results being returned.</span>
<span class="source-line-no">3312</span><span id="line-3312"> ArrayList&lt;Cell&gt; values = new ArrayList&lt;&gt;(32);</span>
<span class="source-line-no">3313</span><span id="line-3313"> region.startRegionOperation(Operation.SCAN);</span>
<span class="source-line-no">3314</span><span id="line-3314"> long before = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">3315</span><span id="line-3315"> // Used to check if we've matched the row limit set on the Scan</span>
<span class="source-line-no">3316</span><span id="line-3316"> int numOfCompleteRows = 0;</span>
<span class="source-line-no">3317</span><span id="line-3317"> // Count of times we call nextRaw; can be &gt; numOfCompleteRows.</span>
<span class="source-line-no">3318</span><span id="line-3318"> int numOfNextRawCalls = 0;</span>
<span class="source-line-no">3319</span><span id="line-3319"> try {</span>
<span class="source-line-no">3320</span><span id="line-3320"> int numOfResults = 0;</span>
<span class="source-line-no">3321</span><span id="line-3321"> synchronized (scanner) {</span>
<span class="source-line-no">3322</span><span id="line-3322"> boolean stale = (region.getRegionInfo().getReplicaId() != 0);</span>
<span class="source-line-no">3323</span><span id="line-3323"> boolean clientHandlesPartials =</span>
<span class="source-line-no">3324</span><span id="line-3324"> request.hasClientHandlesPartials() &amp;&amp; request.getClientHandlesPartials();</span>
<span class="source-line-no">3325</span><span id="line-3325"> boolean clientHandlesHeartbeats =</span>
<span class="source-line-no">3326</span><span id="line-3326"> request.hasClientHandlesHeartbeats() &amp;&amp; request.getClientHandlesHeartbeats();</span>
<span class="source-line-no">3327</span><span id="line-3327"></span>
<span class="source-line-no">3328</span><span id="line-3328"> // On the server side we must ensure that the correct ordering of partial results is</span>
<span class="source-line-no">3329</span><span id="line-3329"> // returned to the client to allow them to properly reconstruct the partial results.</span>
<span class="source-line-no">3330</span><span id="line-3330"> // If the coprocessor host is adding to the result list, we cannot guarantee the</span>
<span class="source-line-no">3331</span><span id="line-3331"> // correct ordering of partial results and so we prevent partial results from being</span>
<span class="source-line-no">3332</span><span id="line-3332"> // formed.</span>
<span class="source-line-no">3333</span><span id="line-3333"> boolean serverGuaranteesOrderOfPartials = results.isEmpty();</span>
<span class="source-line-no">3334</span><span id="line-3334"> boolean allowPartialResults = clientHandlesPartials &amp;&amp; serverGuaranteesOrderOfPartials;</span>
<span class="source-line-no">3335</span><span id="line-3335"> boolean moreRows = false;</span>
<span class="source-line-no">3336</span><span id="line-3336"></span>
<span class="source-line-no">3337</span><span id="line-3337"> // Heartbeat messages occur when the processing of the ScanRequest is exceeds a</span>
<span class="source-line-no">3338</span><span id="line-3338"> // certain time threshold on the server. When the time threshold is exceeded, the</span>
<span class="source-line-no">3339</span><span id="line-3339"> // server stops the scan and sends back whatever Results it has accumulated within</span>
<span class="source-line-no">3340</span><span id="line-3340"> // that time period (may be empty). Since heartbeat messages have the potential to</span>
<span class="source-line-no">3341</span><span id="line-3341"> // create partial Results (in the event that the timeout occurs in the middle of a</span>
<span class="source-line-no">3342</span><span id="line-3342"> // row), we must only generate heartbeat messages when the client can handle both</span>
<span class="source-line-no">3343</span><span id="line-3343"> // heartbeats AND partials</span>
<span class="source-line-no">3344</span><span id="line-3344"> boolean allowHeartbeatMessages = clientHandlesHeartbeats &amp;&amp; allowPartialResults;</span>
<span class="source-line-no">3345</span><span id="line-3345"></span>
<span class="source-line-no">3346</span><span id="line-3346"> long timeLimit = getTimeLimit(rpcCall, controller, allowHeartbeatMessages);</span>
<span class="source-line-no">3347</span><span id="line-3347"></span>
<span class="source-line-no">3348</span><span id="line-3348"> final LimitScope sizeScope =</span>
<span class="source-line-no">3349</span><span id="line-3349"> allowPartialResults ? LimitScope.BETWEEN_CELLS : LimitScope.BETWEEN_ROWS;</span>
<span class="source-line-no">3350</span><span id="line-3350"> final LimitScope timeScope =</span>
<span class="source-line-no">3351</span><span id="line-3351"> allowHeartbeatMessages ? LimitScope.BETWEEN_CELLS : LimitScope.BETWEEN_ROWS;</span>
<span class="source-line-no">3352</span><span id="line-3352"></span>
<span class="source-line-no">3353</span><span id="line-3353"> boolean trackMetrics = request.hasTrackScanMetrics() &amp;&amp; request.getTrackScanMetrics();</span>
<span class="source-line-no">3354</span><span id="line-3354"></span>
<span class="source-line-no">3355</span><span id="line-3355"> // Configure with limits for this RPC. Set keep progress true since size progress</span>
<span class="source-line-no">3356</span><span id="line-3356"> // towards size limit should be kept between calls to nextRaw</span>
<span class="source-line-no">3357</span><span id="line-3357"> ScannerContext.Builder contextBuilder = ScannerContext.newBuilder(true);</span>
<span class="source-line-no">3358</span><span id="line-3358"> // maxResultSize - either we can reach this much size for all cells(being read) data or sum</span>
<span class="source-line-no">3359</span><span id="line-3359"> // of heap size occupied by cells(being read). Cell data means its key and value parts.</span>
<span class="source-line-no">3360</span><span id="line-3360"> // maxQuotaResultSize - max results just from server side configuration and quotas, without</span>
<span class="source-line-no">3361</span><span id="line-3361"> // user's specified max. We use this for evaluating limits based on blocks (not cells).</span>
<span class="source-line-no">3362</span><span id="line-3362"> // We may have accumulated some results in coprocessor preScannerNext call. Subtract any</span>
<span class="source-line-no">3363</span><span id="line-3363"> // cell or block size from maximum here so we adhere to total limits of request.</span>
<span class="source-line-no">3364</span><span id="line-3364"> // Note: we track block size in StoreScanner. If the CP hook got cells from hbase, it will</span>
<span class="source-line-no">3365</span><span id="line-3365"> // have accumulated block bytes. If not, this will be 0 for block size.</span>
<span class="source-line-no">3366</span><span id="line-3366"> long maxCellSize = maxResultSize;</span>
<span class="source-line-no">3367</span><span id="line-3367"> long maxBlockSize = maxQuotaResultSize;</span>
<span class="source-line-no">3368</span><span id="line-3368"> if (rpcCall != null) {</span>
<span class="source-line-no">3369</span><span id="line-3369"> maxBlockSize -= rpcCall.getBlockBytesScanned();</span>
<span class="source-line-no">3370</span><span id="line-3370"> maxCellSize -= rpcCall.getResponseCellSize();</span>
<span class="source-line-no">3371</span><span id="line-3371"> }</span>
<span class="source-line-no">3372</span><span id="line-3372"></span>
<span class="source-line-no">3373</span><span id="line-3373"> contextBuilder.setSizeLimit(sizeScope, maxCellSize, maxCellSize, maxBlockSize);</span>
<span class="source-line-no">3374</span><span id="line-3374"> contextBuilder.setBatchLimit(scanner.getBatch());</span>
<span class="source-line-no">3375</span><span id="line-3375"> contextBuilder.setTimeLimit(timeScope, timeLimit);</span>
<span class="source-line-no">3376</span><span id="line-3376"> contextBuilder.setTrackMetrics(trackMetrics);</span>
<span class="source-line-no">3377</span><span id="line-3377"> ScannerContext scannerContext = contextBuilder.build();</span>
<span class="source-line-no">3378</span><span id="line-3378"> boolean limitReached = false;</span>
<span class="source-line-no">3379</span><span id="line-3379"> while (numOfResults &lt; maxResults) {</span>
<span class="source-line-no">3380</span><span id="line-3380"> // Reset the batch progress to 0 before every call to RegionScanner#nextRaw. The</span>
<span class="source-line-no">3381</span><span id="line-3381"> // batch limit is a limit on the number of cells per Result. Thus, if progress is</span>
<span class="source-line-no">3382</span><span id="line-3382"> // being tracked (i.e. scannerContext.keepProgress() is true) then we need to</span>
<span class="source-line-no">3383</span><span id="line-3383"> // reset the batch progress between nextRaw invocations since we don't want the</span>
<span class="source-line-no">3384</span><span id="line-3384"> // batch progress from previous calls to affect future calls</span>
<span class="source-line-no">3385</span><span id="line-3385"> scannerContext.setBatchProgress(0);</span>
<span class="source-line-no">3386</span><span id="line-3386"> assert values.isEmpty();</span>
<span class="source-line-no">3387</span><span id="line-3387"></span>
<span class="source-line-no">3388</span><span id="line-3388"> // Collect values to be returned here</span>
<span class="source-line-no">3389</span><span id="line-3389"> moreRows = scanner.nextRaw(values, scannerContext);</span>
<span class="source-line-no">3390</span><span id="line-3390"> if (rpcCall == null) {</span>
<span class="source-line-no">3391</span><span id="line-3391"> // When there is no RpcCallContext,copy EC to heap, then the scanner would close,</span>
<span class="source-line-no">3392</span><span id="line-3392"> // This can be an EXPENSIVE call. It may make an extra copy from offheap to onheap</span>
<span class="source-line-no">3393</span><span id="line-3393"> // buffers.See more details in HBASE-26036.</span>
<span class="source-line-no">3394</span><span id="line-3394"> CellUtil.cloneIfNecessary(values);</span>
<span class="source-line-no">3395</span><span id="line-3395"> }</span>
<span class="source-line-no">3396</span><span id="line-3396"> numOfNextRawCalls++;</span>
<span class="source-line-no">3397</span><span id="line-3397"></span>
<span class="source-line-no">3398</span><span id="line-3398"> if (!values.isEmpty()) {</span>
<span class="source-line-no">3399</span><span id="line-3399"> if (limitOfRows &gt; 0) {</span>
<span class="source-line-no">3400</span><span id="line-3400"> // First we need to check if the last result is partial and we have a row change. If</span>
<span class="source-line-no">3401</span><span id="line-3401"> // so then we need to increase the numOfCompleteRows.</span>
<span class="source-line-no">3402</span><span id="line-3402"> if (results.isEmpty()) {</span>
<span class="source-line-no">3403</span><span id="line-3403"> if (</span>
<span class="source-line-no">3404</span><span id="line-3404"> rsh.rowOfLastPartialResult != null</span>
<span class="source-line-no">3405</span><span id="line-3405"> &amp;&amp; !CellUtil.matchingRows(values.get(0), rsh.rowOfLastPartialResult)</span>
<span class="source-line-no">3406</span><span id="line-3406"> ) {</span>
<span class="source-line-no">3407</span><span id="line-3407"> numOfCompleteRows++;</span>
<span class="source-line-no">3408</span><span id="line-3408"> checkLimitOfRows(numOfCompleteRows, limitOfRows, moreRows, scannerContext,</span>
<span class="source-line-no">3409</span><span id="line-3409"> builder);</span>
<span class="source-line-no">3410</span><span id="line-3410"> }</span>
<span class="source-line-no">3411</span><span id="line-3411"> } else {</span>
<span class="source-line-no">3412</span><span id="line-3412"> Result lastResult = results.get(results.size() - 1);</span>
<span class="source-line-no">3413</span><span id="line-3413"> if (</span>
<span class="source-line-no">3414</span><span id="line-3414"> lastResult.mayHaveMoreCellsInRow()</span>
<span class="source-line-no">3415</span><span id="line-3415"> &amp;&amp; !CellUtil.matchingRows(values.get(0), lastResult.getRow())</span>
<span class="source-line-no">3416</span><span id="line-3416"> ) {</span>
<span class="source-line-no">3417</span><span id="line-3417"> numOfCompleteRows++;</span>
<span class="source-line-no">3418</span><span id="line-3418"> checkLimitOfRows(numOfCompleteRows, limitOfRows, moreRows, scannerContext,</span>
<span class="source-line-no">3419</span><span id="line-3419"> builder);</span>
<span class="source-line-no">3420</span><span id="line-3420"> }</span>
<span class="source-line-no">3421</span><span id="line-3421"> }</span>
<span class="source-line-no">3422</span><span id="line-3422"> if (builder.hasMoreResults() &amp;&amp; !builder.getMoreResults()) {</span>
<span class="source-line-no">3423</span><span id="line-3423"> break;</span>
<span class="source-line-no">3424</span><span id="line-3424"> }</span>
<span class="source-line-no">3425</span><span id="line-3425"> }</span>
<span class="source-line-no">3426</span><span id="line-3426"> boolean mayHaveMoreCellsInRow = scannerContext.mayHaveMoreCellsInRow();</span>
<span class="source-line-no">3427</span><span id="line-3427"> Result r = Result.create(values, null, stale, mayHaveMoreCellsInRow);</span>
<span class="source-line-no">3428</span><span id="line-3428"> results.add(r);</span>
<span class="source-line-no">3429</span><span id="line-3429"> numOfResults++;</span>
<span class="source-line-no">3430</span><span id="line-3430"> if (!mayHaveMoreCellsInRow &amp;&amp; limitOfRows &gt; 0) {</span>
<span class="source-line-no">3431</span><span id="line-3431"> numOfCompleteRows++;</span>
<span class="source-line-no">3432</span><span id="line-3432"> checkLimitOfRows(numOfCompleteRows, limitOfRows, moreRows, scannerContext, builder);</span>
<span class="source-line-no">3433</span><span id="line-3433"> if (builder.hasMoreResults() &amp;&amp; !builder.getMoreResults()) {</span>
<span class="source-line-no">3434</span><span id="line-3434"> break;</span>
<span class="source-line-no">3435</span><span id="line-3435"> }</span>
<span class="source-line-no">3436</span><span id="line-3436"> }</span>
<span class="source-line-no">3437</span><span id="line-3437"> } else if (!moreRows &amp;&amp; !results.isEmpty()) {</span>
<span class="source-line-no">3438</span><span id="line-3438"> // No more cells for the scan here, we need to ensure that the mayHaveMoreCellsInRow of</span>
<span class="source-line-no">3439</span><span id="line-3439"> // last result is false. Otherwise it's possible that: the first nextRaw returned</span>
<span class="source-line-no">3440</span><span id="line-3440"> // because BATCH_LIMIT_REACHED (BTW it happen to exhaust all cells of the scan),so the</span>
<span class="source-line-no">3441</span><span id="line-3441"> // last result's mayHaveMoreCellsInRow will be true. while the following nextRaw will</span>
<span class="source-line-no">3442</span><span id="line-3442"> // return with moreRows=false, which means moreResultsInRegion would be false, it will</span>
<span class="source-line-no">3443</span><span id="line-3443"> // be a contradictory state (HBASE-21206).</span>
<span class="source-line-no">3444</span><span id="line-3444"> int lastIdx = results.size() - 1;</span>
<span class="source-line-no">3445</span><span id="line-3445"> Result r = results.get(lastIdx);</span>
<span class="source-line-no">3446</span><span id="line-3446"> if (r.mayHaveMoreCellsInRow()) {</span>
<span class="source-line-no">3447</span><span id="line-3447"> results.set(lastIdx, ClientInternalHelper.createResult(</span>
<span class="source-line-no">3448</span><span id="line-3448"> ClientInternalHelper.getExtendedRawCells(r), r.getExists(), r.isStale(), false));</span>
<span class="source-line-no">3449</span><span id="line-3449"> }</span>
<span class="source-line-no">3450</span><span id="line-3450"> }</span>
<span class="source-line-no">3451</span><span id="line-3451"> boolean sizeLimitReached = scannerContext.checkSizeLimit(LimitScope.BETWEEN_ROWS);</span>
<span class="source-line-no">3452</span><span id="line-3452"> boolean timeLimitReached = scannerContext.checkTimeLimit(LimitScope.BETWEEN_ROWS);</span>
<span class="source-line-no">3453</span><span id="line-3453"> boolean resultsLimitReached = numOfResults &gt;= maxResults;</span>
<span class="source-line-no">3454</span><span id="line-3454"> limitReached = sizeLimitReached || timeLimitReached || resultsLimitReached;</span>
<span class="source-line-no">3455</span><span id="line-3455"></span>
<span class="source-line-no">3456</span><span id="line-3456"> if (limitReached || !moreRows) {</span>
<span class="source-line-no">3457</span><span id="line-3457"> // With block size limit, we may exceed size limit without collecting any results.</span>
<span class="source-line-no">3458</span><span id="line-3458"> // In this case we want to send heartbeat and/or cursor. We don't want to send heartbeat</span>
<span class="source-line-no">3459</span><span id="line-3459"> // or cursor if results were collected, for example for cell size or heap size limits.</span>
<span class="source-line-no">3460</span><span id="line-3460"> boolean sizeLimitReachedWithoutResults = sizeLimitReached &amp;&amp; results.isEmpty();</span>
<span class="source-line-no">3461</span><span id="line-3461"> // We only want to mark a ScanResponse as a heartbeat message in the event that</span>
<span class="source-line-no">3462</span><span id="line-3462"> // there are more values to be read server side. If there aren't more values,</span>
<span class="source-line-no">3463</span><span id="line-3463"> // marking it as a heartbeat is wasteful because the client will need to issue</span>
<span class="source-line-no">3464</span><span id="line-3464"> // another ScanRequest only to realize that they already have all the values</span>
<span class="source-line-no">3465</span><span id="line-3465"> if (moreRows &amp;&amp; (timeLimitReached || sizeLimitReachedWithoutResults)) {</span>
<span class="source-line-no">3466</span><span id="line-3466"> // Heartbeat messages occur when the time limit has been reached, or size limit has</span>
<span class="source-line-no">3467</span><span id="line-3467"> // been reached before collecting any results. This can happen for heavily filtered</span>
<span class="source-line-no">3468</span><span id="line-3468"> // scans which scan over too many blocks.</span>
<span class="source-line-no">3469</span><span id="line-3469"> builder.setHeartbeatMessage(true);</span>
<span class="source-line-no">3470</span><span id="line-3470"> if (rsh.needCursor) {</span>
<span class="source-line-no">3471</span><span id="line-3471"> Cell cursorCell = scannerContext.getLastPeekedCell();</span>
<span class="source-line-no">3472</span><span id="line-3472"> if (cursorCell != null) {</span>
<span class="source-line-no">3473</span><span id="line-3473"> builder.setCursor(ProtobufUtil.toCursor(cursorCell));</span>
<span class="source-line-no">3474</span><span id="line-3474"> }</span>
<span class="source-line-no">3475</span><span id="line-3475"> }</span>
<span class="source-line-no">3476</span><span id="line-3476"> }</span>
<span class="source-line-no">3477</span><span id="line-3477"> break;</span>
<span class="source-line-no">3478</span><span id="line-3478"> }</span>
<span class="source-line-no">3479</span><span id="line-3479"> values.clear();</span>
<span class="source-line-no">3480</span><span id="line-3480"> }</span>
<span class="source-line-no">3481</span><span id="line-3481"> if (rpcCall != null) {</span>
<span class="source-line-no">3482</span><span id="line-3482"> rpcCall.incrementResponseCellSize(scannerContext.getHeapSizeProgress());</span>
<span class="source-line-no">3483</span><span id="line-3483"> }</span>
<span class="source-line-no">3484</span><span id="line-3484"> builder.setMoreResultsInRegion(moreRows);</span>
<span class="source-line-no">3485</span><span id="line-3485"> // Check to see if the client requested that we track metrics server side. If the</span>
<span class="source-line-no">3486</span><span id="line-3486"> // client requested metrics, retrieve the metrics from the scanner context.</span>
<span class="source-line-no">3487</span><span id="line-3487"> if (trackMetrics) {</span>
<span class="source-line-no">3488</span><span id="line-3488"> // rather than increment yet another counter in StoreScanner, just set the value here</span>
<span class="source-line-no">3489</span><span id="line-3489"> // from block size progress before writing into the response</span>
<span class="source-line-no">3490</span><span id="line-3490"> scannerContext.getMetrics().countOfBlockBytesScanned</span>
<span class="source-line-no">3491</span><span id="line-3491"> .set(scannerContext.getBlockSizeProgress());</span>
<span class="source-line-no">3492</span><span id="line-3492"> if (rpcCall != null) {</span>
<span class="source-line-no">3493</span><span id="line-3493"> scannerContext.getMetrics().fsReadTime.set(rpcCall.getFsReadTime());</span>
<span class="source-line-no">3494</span><span id="line-3494"> }</span>
<span class="source-line-no">3495</span><span id="line-3495"> Map&lt;String, Long&gt; metrics = scannerContext.getMetrics().getMetricsMap();</span>
<span class="source-line-no">3496</span><span id="line-3496"> ScanMetrics.Builder metricBuilder = ScanMetrics.newBuilder();</span>
<span class="source-line-no">3497</span><span id="line-3497"> NameInt64Pair.Builder pairBuilder = NameInt64Pair.newBuilder();</span>
<span class="source-line-no">3498</span><span id="line-3498"></span>
<span class="source-line-no">3499</span><span id="line-3499"> for (Entry&lt;String, Long&gt; entry : metrics.entrySet()) {</span>
<span class="source-line-no">3500</span><span id="line-3500"> pairBuilder.setName(entry.getKey());</span>
<span class="source-line-no">3501</span><span id="line-3501"> pairBuilder.setValue(entry.getValue());</span>
<span class="source-line-no">3502</span><span id="line-3502"> metricBuilder.addMetrics(pairBuilder.build());</span>
<span class="source-line-no">3503</span><span id="line-3503"> }</span>
<span class="source-line-no">3504</span><span id="line-3504"></span>
<span class="source-line-no">3505</span><span id="line-3505"> builder.setScanMetrics(metricBuilder.build());</span>
<span class="source-line-no">3506</span><span id="line-3506"> }</span>
<span class="source-line-no">3507</span><span id="line-3507"> }</span>
<span class="source-line-no">3508</span><span id="line-3508"> } finally {</span>
<span class="source-line-no">3509</span><span id="line-3509"> region.closeRegionOperation();</span>
<span class="source-line-no">3510</span><span id="line-3510"> // Update serverside metrics, even on error.</span>
<span class="source-line-no">3511</span><span id="line-3511"> long end = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">3512</span><span id="line-3512"></span>
<span class="source-line-no">3513</span><span id="line-3513"> long responseCellSize = 0;</span>
<span class="source-line-no">3514</span><span id="line-3514"> long blockBytesScanned = 0;</span>
<span class="source-line-no">3515</span><span id="line-3515"> if (rpcCall != null) {</span>
<span class="source-line-no">3516</span><span id="line-3516"> responseCellSize = rpcCall.getResponseCellSize();</span>
<span class="source-line-no">3517</span><span id="line-3517"> blockBytesScanned = rpcCall.getBlockBytesScanned();</span>
<span class="source-line-no">3518</span><span id="line-3518"> rsh.updateBlockBytesScanned(blockBytesScanned);</span>
<span class="source-line-no">3519</span><span id="line-3519"> }</span>
<span class="source-line-no">3520</span><span id="line-3520"> region.getMetrics().updateScan();</span>
<span class="source-line-no">3521</span><span id="line-3521"> final MetricsRegionServer metricsRegionServer = server.getMetrics();</span>
<span class="source-line-no">3522</span><span id="line-3522"> if (metricsRegionServer != null) {</span>
<span class="source-line-no">3523</span><span id="line-3523"> metricsRegionServer.updateScan(region, end - before, responseCellSize, blockBytesScanned);</span>
<span class="source-line-no">3524</span><span id="line-3524"> metricsRegionServer.updateReadQueryMeter(region, numOfNextRawCalls);</span>
<span class="source-line-no">3525</span><span id="line-3525"> }</span>
<span class="source-line-no">3526</span><span id="line-3526"> }</span>
<span class="source-line-no">3527</span><span id="line-3527"> // coprocessor postNext hook</span>
<span class="source-line-no">3528</span><span id="line-3528"> if (region.getCoprocessorHost() != null) {</span>
<span class="source-line-no">3529</span><span id="line-3529"> region.getCoprocessorHost().postScannerNext(scanner, results, maxResults, true);</span>
<span class="source-line-no">3530</span><span id="line-3530"> }</span>
<span class="source-line-no">3531</span><span id="line-3531"> }</span>
<span class="source-line-no">3532</span><span id="line-3532"></span>
<span class="source-line-no">3533</span><span id="line-3533"> /**</span>
<span class="source-line-no">3534</span><span id="line-3534"> * Scan data in a table.</span>
<span class="source-line-no">3535</span><span id="line-3535"> * @param controller the RPC controller</span>
<span class="source-line-no">3536</span><span id="line-3536"> * @param request the scan request</span>
<span class="source-line-no">3537</span><span id="line-3537"> */</span>
<span class="source-line-no">3538</span><span id="line-3538"> @Override</span>
<span class="source-line-no">3539</span><span id="line-3539"> public ScanResponse scan(final RpcController controller, final ScanRequest request)</span>
<span class="source-line-no">3540</span><span id="line-3540"> throws ServiceException {</span>
<span class="source-line-no">3541</span><span id="line-3541"> if (controller != null &amp;&amp; !(controller instanceof HBaseRpcController)) {</span>
<span class="source-line-no">3542</span><span id="line-3542"> throw new UnsupportedOperationException(</span>
<span class="source-line-no">3543</span><span id="line-3543"> "We only do " + "HBaseRpcControllers! FIX IF A PROBLEM: " + controller);</span>
<span class="source-line-no">3544</span><span id="line-3544"> }</span>
<span class="source-line-no">3545</span><span id="line-3545"> if (!request.hasScannerId() &amp;&amp; !request.hasScan()) {</span>
<span class="source-line-no">3546</span><span id="line-3546"> throw new ServiceException(</span>
<span class="source-line-no">3547</span><span id="line-3547"> new DoNotRetryIOException("Missing required input: scannerId or scan"));</span>
<span class="source-line-no">3548</span><span id="line-3548"> }</span>
<span class="source-line-no">3549</span><span id="line-3549"> try {</span>
<span class="source-line-no">3550</span><span id="line-3550"> checkOpen();</span>
<span class="source-line-no">3551</span><span id="line-3551"> } catch (IOException e) {</span>
<span class="source-line-no">3552</span><span id="line-3552"> if (request.hasScannerId()) {</span>
<span class="source-line-no">3553</span><span id="line-3553"> String scannerName = toScannerName(request.getScannerId());</span>
<span class="source-line-no">3554</span><span id="line-3554"> if (LOG.isDebugEnabled()) {</span>
<span class="source-line-no">3555</span><span id="line-3555"> LOG.debug(</span>
<span class="source-line-no">3556</span><span id="line-3556"> "Server shutting down and client tried to access missing scanner " + scannerName);</span>
<span class="source-line-no">3557</span><span id="line-3557"> }</span>
<span class="source-line-no">3558</span><span id="line-3558"> final LeaseManager leaseManager = server.getLeaseManager();</span>
<span class="source-line-no">3559</span><span id="line-3559"> if (leaseManager != null) {</span>
<span class="source-line-no">3560</span><span id="line-3560"> try {</span>
<span class="source-line-no">3561</span><span id="line-3561"> leaseManager.cancelLease(scannerName);</span>
<span class="source-line-no">3562</span><span id="line-3562"> } catch (LeaseException le) {</span>
<span class="source-line-no">3563</span><span id="line-3563"> // No problem, ignore</span>
<span class="source-line-no">3564</span><span id="line-3564"> if (LOG.isTraceEnabled()) {</span>
<span class="source-line-no">3565</span><span id="line-3565"> LOG.trace("Un-able to cancel lease of scanner. It could already be closed.");</span>
<span class="source-line-no">3566</span><span id="line-3566"> }</span>
<span class="source-line-no">3567</span><span id="line-3567"> }</span>
<span class="source-line-no">3568</span><span id="line-3568"> }</span>
<span class="source-line-no">3569</span><span id="line-3569"> }</span>
<span class="source-line-no">3570</span><span id="line-3570"> throw new ServiceException(e);</span>
<span class="source-line-no">3571</span><span id="line-3571"> }</span>
<span class="source-line-no">3572</span><span id="line-3572"> requestCount.increment();</span>
<span class="source-line-no">3573</span><span id="line-3573"> rpcScanRequestCount.increment();</span>
<span class="source-line-no">3574</span><span id="line-3574"> RegionScannerContext rsx;</span>
<span class="source-line-no">3575</span><span id="line-3575"> ScanResponse.Builder builder = ScanResponse.newBuilder();</span>
<span class="source-line-no">3576</span><span id="line-3576"> try {</span>
<span class="source-line-no">3577</span><span id="line-3577"> rsx = checkQuotaAndGetRegionScannerContext(request, builder);</span>
<span class="source-line-no">3578</span><span id="line-3578"> } catch (IOException e) {</span>
<span class="source-line-no">3579</span><span id="line-3579"> if (e == SCANNER_ALREADY_CLOSED) {</span>
<span class="source-line-no">3580</span><span id="line-3580"> // Now we will close scanner automatically if there are no more results for this region but</span>
<span class="source-line-no">3581</span><span id="line-3581"> // the old client will still send a close request to us. Just ignore it and return.</span>
<span class="source-line-no">3582</span><span id="line-3582"> return builder.build();</span>
<span class="source-line-no">3583</span><span id="line-3583"> }</span>
<span class="source-line-no">3584</span><span id="line-3584"> throw new ServiceException(e);</span>
<span class="source-line-no">3585</span><span id="line-3585"> }</span>
<span class="source-line-no">3586</span><span id="line-3586"> String scannerName = rsx.scannerName;</span>
<span class="source-line-no">3587</span><span id="line-3587"> RegionScannerHolder rsh = rsx.holder;</span>
<span class="source-line-no">3588</span><span id="line-3588"> OperationQuota quota = rsx.quota;</span>
<span class="source-line-no">3589</span><span id="line-3589"> if (rsh.fullRegionScan) {</span>
<span class="source-line-no">3590</span><span id="line-3590"> rpcFullScanRequestCount.increment();</span>
<span class="source-line-no">3591</span><span id="line-3591"> }</span>
<span class="source-line-no">3592</span><span id="line-3592"> HRegion region = rsh.r;</span>
<span class="source-line-no">3593</span><span id="line-3593"> LeaseManager.Lease lease;</span>
<span class="source-line-no">3594</span><span id="line-3594"> try {</span>
<span class="source-line-no">3595</span><span id="line-3595"> // Remove lease while its being processed in server; protects against case</span>
<span class="source-line-no">3596</span><span id="line-3596"> // where processing of request takes &gt; lease expiration time. or null if none found.</span>
<span class="source-line-no">3597</span><span id="line-3597"> lease = server.getLeaseManager().removeLease(scannerName);</span>
<span class="source-line-no">3598</span><span id="line-3598"> } catch (LeaseException e) {</span>
<span class="source-line-no">3599</span><span id="line-3599"> throw new ServiceException(e);</span>
<span class="source-line-no">3600</span><span id="line-3600"> }</span>
<span class="source-line-no">3601</span><span id="line-3601"> if (request.hasRenew() &amp;&amp; request.getRenew()) {</span>
<span class="source-line-no">3602</span><span id="line-3602"> // add back and return</span>
<span class="source-line-no">3603</span><span id="line-3603"> addScannerLeaseBack(lease);</span>
<span class="source-line-no">3604</span><span id="line-3604"> try {</span>
<span class="source-line-no">3605</span><span id="line-3605"> checkScanNextCallSeq(request, rsh);</span>
<span class="source-line-no">3606</span><span id="line-3606"> } catch (OutOfOrderScannerNextException e) {</span>
<span class="source-line-no">3607</span><span id="line-3607"> throw new ServiceException(e);</span>
<span class="source-line-no">3608</span><span id="line-3608"> }</span>
<span class="source-line-no">3609</span><span id="line-3609"> return builder.build();</span>
<span class="source-line-no">3610</span><span id="line-3610"> }</span>
<span class="source-line-no">3611</span><span id="line-3611"> try {</span>
<span class="source-line-no">3612</span><span id="line-3612"> checkScanNextCallSeq(request, rsh);</span>
<span class="source-line-no">3613</span><span id="line-3613"> } catch (OutOfOrderScannerNextException e) {</span>
<span class="source-line-no">3614</span><span id="line-3614"> addScannerLeaseBack(lease);</span>
<span class="source-line-no">3615</span><span id="line-3615"> throw new ServiceException(e);</span>
<span class="source-line-no">3616</span><span id="line-3616"> }</span>
<span class="source-line-no">3617</span><span id="line-3617"> // Now we have increased the next call sequence. If we give client an error, the retry will</span>
<span class="source-line-no">3618</span><span id="line-3618"> // never success. So we'd better close the scanner and return a DoNotRetryIOException to client</span>
<span class="source-line-no">3619</span><span id="line-3619"> // and then client will try to open a new scanner.</span>
<span class="source-line-no">3620</span><span id="line-3620"> boolean closeScanner = request.hasCloseScanner() ? request.getCloseScanner() : false;</span>
<span class="source-line-no">3621</span><span id="line-3621"> int rows; // this is scan.getCaching</span>
<span class="source-line-no">3622</span><span id="line-3622"> if (request.hasNumberOfRows()) {</span>
<span class="source-line-no">3623</span><span id="line-3623"> rows = request.getNumberOfRows();</span>
<span class="source-line-no">3624</span><span id="line-3624"> } else {</span>
<span class="source-line-no">3625</span><span id="line-3625"> rows = closeScanner ? 0 : 1;</span>
<span class="source-line-no">3626</span><span id="line-3626"> }</span>
<span class="source-line-no">3627</span><span id="line-3627"> RpcCall rpcCall = RpcServer.getCurrentCall().orElse(null);</span>
<span class="source-line-no">3628</span><span id="line-3628"> // now let's do the real scan.</span>
<span class="source-line-no">3629</span><span id="line-3629"> long maxQuotaResultSize = Math.min(maxScannerResultSize, quota.getMaxResultSize());</span>
<span class="source-line-no">3630</span><span id="line-3630"> RegionScanner scanner = rsh.s;</span>
<span class="source-line-no">3631</span><span id="line-3631"> // this is the limit of rows for this scan, if we the number of rows reach this value, we will</span>
<span class="source-line-no">3632</span><span id="line-3632"> // close the scanner.</span>
<span class="source-line-no">3633</span><span id="line-3633"> int limitOfRows;</span>
<span class="source-line-no">3634</span><span id="line-3634"> if (request.hasLimitOfRows()) {</span>
<span class="source-line-no">3635</span><span id="line-3635"> limitOfRows = request.getLimitOfRows();</span>
<span class="source-line-no">3636</span><span id="line-3636"> } else {</span>
<span class="source-line-no">3637</span><span id="line-3637"> limitOfRows = -1;</span>
<span class="source-line-no">3638</span><span id="line-3638"> }</span>
<span class="source-line-no">3639</span><span id="line-3639"> boolean scannerClosed = false;</span>
<span class="source-line-no">3640</span><span id="line-3640"> try {</span>
<span class="source-line-no">3641</span><span id="line-3641"> List&lt;Result&gt; results = new ArrayList&lt;&gt;(Math.min(rows, 512));</span>
<span class="source-line-no">3642</span><span id="line-3642"> if (rows &gt; 0) {</span>
<span class="source-line-no">3643</span><span id="line-3643"> boolean done = false;</span>
<span class="source-line-no">3644</span><span id="line-3644"> // Call coprocessor. Get region info from scanner.</span>
<span class="source-line-no">3645</span><span id="line-3645"> if (region.getCoprocessorHost() != null) {</span>
<span class="source-line-no">3646</span><span id="line-3646"> Boolean bypass = region.getCoprocessorHost().preScannerNext(scanner, results, rows);</span>
<span class="source-line-no">3647</span><span id="line-3647"> if (!results.isEmpty()) {</span>
<span class="source-line-no">3648</span><span id="line-3648"> for (Result r : results) {</span>
<span class="source-line-no">3649</span><span id="line-3649"> // add cell size from CP results so we can track response size and update limits</span>
<span class="source-line-no">3650</span><span id="line-3650"> // when calling scan below if !done. We'll also have tracked block size if the CP</span>
<span class="source-line-no">3651</span><span id="line-3651"> // got results from hbase, since StoreScanner tracks that for all calls automatically.</span>
<span class="source-line-no">3652</span><span id="line-3652"> addSize(rpcCall, r);</span>
<span class="source-line-no">3653</span><span id="line-3653"> }</span>
<span class="source-line-no">3654</span><span id="line-3654"> }</span>
<span class="source-line-no">3655</span><span id="line-3655"> if (bypass != null &amp;&amp; bypass.booleanValue()) {</span>
<span class="source-line-no">3656</span><span id="line-3656"> done = true;</span>
<span class="source-line-no">3657</span><span id="line-3657"> }</span>
<span class="source-line-no">3658</span><span id="line-3658"> }</span>
<span class="source-line-no">3659</span><span id="line-3659"> if (!done) {</span>
<span class="source-line-no">3660</span><span id="line-3660"> scan((HBaseRpcController) controller, request, rsh, maxQuotaResultSize, rows, limitOfRows,</span>
<span class="source-line-no">3661</span><span id="line-3661"> results, builder, rpcCall);</span>
<span class="source-line-no">3662</span><span id="line-3662"> } else {</span>
<span class="source-line-no">3663</span><span id="line-3663"> builder.setMoreResultsInRegion(!results.isEmpty());</span>
<span class="source-line-no">3664</span><span id="line-3664"> }</span>
<span class="source-line-no">3665</span><span id="line-3665"> } else {</span>
<span class="source-line-no">3666</span><span id="line-3666"> // This is a open scanner call with numberOfRow = 0, so set more results in region to true.</span>
<span class="source-line-no">3667</span><span id="line-3667"> builder.setMoreResultsInRegion(true);</span>
<span class="source-line-no">3668</span><span id="line-3668"> }</span>
<span class="source-line-no">3669</span><span id="line-3669"></span>
<span class="source-line-no">3670</span><span id="line-3670"> quota.addScanResult(results);</span>
<span class="source-line-no">3671</span><span id="line-3671"> addResults(builder, results, (HBaseRpcController) controller,</span>
<span class="source-line-no">3672</span><span id="line-3672"> RegionReplicaUtil.isDefaultReplica(region.getRegionInfo()),</span>
<span class="source-line-no">3673</span><span id="line-3673"> isClientCellBlockSupport(rpcCall));</span>
<span class="source-line-no">3674</span><span id="line-3674"> if (scanner.isFilterDone() &amp;&amp; results.isEmpty()) {</span>
<span class="source-line-no">3675</span><span id="line-3675"> // If the scanner's filter - if any - is done with the scan</span>
<span class="source-line-no">3676</span><span id="line-3676"> // only set moreResults to false if the results is empty. This is used to keep compatible</span>
<span class="source-line-no">3677</span><span id="line-3677"> // with the old scan implementation where we just ignore the returned results if moreResults</span>
<span class="source-line-no">3678</span><span id="line-3678"> // is false. Can remove the isEmpty check after we get rid of the old implementation.</span>
<span class="source-line-no">3679</span><span id="line-3679"> builder.setMoreResults(false);</span>
<span class="source-line-no">3680</span><span id="line-3680"> }</span>
<span class="source-line-no">3681</span><span id="line-3681"> // Later we may close the scanner depending on this flag so here we need to make sure that we</span>
<span class="source-line-no">3682</span><span id="line-3682"> // have already set this flag.</span>
<span class="source-line-no">3683</span><span id="line-3683"> assert builder.hasMoreResultsInRegion();</span>
<span class="source-line-no">3684</span><span id="line-3684"> // we only set moreResults to false in the above code, so set it to true if we haven't set it</span>
<span class="source-line-no">3685</span><span id="line-3685"> // yet.</span>
<span class="source-line-no">3686</span><span id="line-3686"> if (!builder.hasMoreResults()) {</span>
<span class="source-line-no">3687</span><span id="line-3687"> builder.setMoreResults(true);</span>
<span class="source-line-no">3688</span><span id="line-3688"> }</span>
<span class="source-line-no">3689</span><span id="line-3689"> if (builder.getMoreResults() &amp;&amp; builder.getMoreResultsInRegion() &amp;&amp; !results.isEmpty()) {</span>
<span class="source-line-no">3690</span><span id="line-3690"> // Record the last cell of the last result if it is a partial result</span>
<span class="source-line-no">3691</span><span id="line-3691"> // We need this to calculate the complete rows we have returned to client as the</span>
<span class="source-line-no">3692</span><span id="line-3692"> // mayHaveMoreCellsInRow is true does not mean that there will be extra cells for the</span>
<span class="source-line-no">3693</span><span id="line-3693"> // current row. We may filter out all the remaining cells for the current row and just</span>
<span class="source-line-no">3694</span><span id="line-3694"> // return the cells of the nextRow when calling RegionScanner.nextRaw. So here we need to</span>
<span class="source-line-no">3695</span><span id="line-3695"> // check for row change.</span>
<span class="source-line-no">3696</span><span id="line-3696"> Result lastResult = results.get(results.size() - 1);</span>
<span class="source-line-no">3697</span><span id="line-3697"> if (lastResult.mayHaveMoreCellsInRow()) {</span>
<span class="source-line-no">3698</span><span id="line-3698"> rsh.rowOfLastPartialResult = lastResult.getRow();</span>
<span class="source-line-no">3699</span><span id="line-3699"> } else {</span>
<span class="source-line-no">3700</span><span id="line-3700"> rsh.rowOfLastPartialResult = null;</span>
<span class="source-line-no">3701</span><span id="line-3701"> }</span>
<span class="source-line-no">3702</span><span id="line-3702"> }</span>
<span class="source-line-no">3703</span><span id="line-3703"> if (!builder.getMoreResults() || !builder.getMoreResultsInRegion() || closeScanner) {</span>
<span class="source-line-no">3704</span><span id="line-3704"> scannerClosed = true;</span>
<span class="source-line-no">3705</span><span id="line-3705"> closeScanner(region, scanner, scannerName, rpcCall, false);</span>
<span class="source-line-no">3706</span><span id="line-3706"> }</span>
<span class="source-line-no">3707</span><span id="line-3707"></span>
<span class="source-line-no">3708</span><span id="line-3708"> // There's no point returning to a timed out client. Throwing ensures scanner is closed</span>
<span class="source-line-no">3709</span><span id="line-3709"> if (rpcCall != null &amp;&amp; EnvironmentEdgeManager.currentTime() &gt; rpcCall.getDeadline()) {</span>
<span class="source-line-no">3710</span><span id="line-3710"> throw new TimeoutIOException("Client deadline exceeded, cannot return results");</span>
<span class="source-line-no">3711</span><span id="line-3711"> }</span>
<span class="source-line-no">3712</span><span id="line-3712"></span>
<span class="source-line-no">3713</span><span id="line-3713"> return builder.build();</span>
<span class="source-line-no">3714</span><span id="line-3714"> } catch (IOException e) {</span>
<span class="source-line-no">3715</span><span id="line-3715"> try {</span>
<span class="source-line-no">3716</span><span id="line-3716"> // scanner is closed here</span>
<span class="source-line-no">3717</span><span id="line-3717"> scannerClosed = true;</span>
<span class="source-line-no">3718</span><span id="line-3718"> // The scanner state might be left in a dirty state, so we will tell the Client to</span>
<span class="source-line-no">3719</span><span id="line-3719"> // fail this RPC and close the scanner while opening up another one from the start of</span>
<span class="source-line-no">3720</span><span id="line-3720"> // row that the client has last seen.</span>
<span class="source-line-no">3721</span><span id="line-3721"> closeScanner(region, scanner, scannerName, rpcCall, true);</span>
<span class="source-line-no">3722</span><span id="line-3722"></span>
<span class="source-line-no">3723</span><span id="line-3723"> // If it is a DoNotRetryIOException already, throw as it is. Unfortunately, DNRIOE is</span>
<span class="source-line-no">3724</span><span id="line-3724"> // used in two different semantics.</span>
<span class="source-line-no">3725</span><span id="line-3725"> // (1) The first is to close the client scanner and bubble up the exception all the way</span>
<span class="source-line-no">3726</span><span id="line-3726"> // to the application. This is preferred when the exception is really un-recoverable</span>
<span class="source-line-no">3727</span><span id="line-3727"> // (like CorruptHFileException, etc). Plain DoNotRetryIOException also falls into this</span>
<span class="source-line-no">3728</span><span id="line-3728"> // bucket usually.</span>
<span class="source-line-no">3729</span><span id="line-3729"> // (2) Second semantics is to close the current region scanner only, but continue the</span>
<span class="source-line-no">3730</span><span id="line-3730"> // client scanner by overriding the exception. This is usually UnknownScannerException,</span>
<span class="source-line-no">3731</span><span id="line-3731"> // OutOfOrderScannerNextException, etc where the region scanner has to be closed, but the</span>
<span class="source-line-no">3732</span><span id="line-3732"> // application-level ClientScanner has to continue without bubbling up the exception to</span>
<span class="source-line-no">3733</span><span id="line-3733"> // the client. See ClientScanner code to see how it deals with these special exceptions.</span>
<span class="source-line-no">3734</span><span id="line-3734"> if (e instanceof DoNotRetryIOException) {</span>
<span class="source-line-no">3735</span><span id="line-3735"> throw e;</span>
<span class="source-line-no">3736</span><span id="line-3736"> }</span>
<span class="source-line-no">3737</span><span id="line-3737"></span>
<span class="source-line-no">3738</span><span id="line-3738"> // If it is a FileNotFoundException, wrap as a</span>
<span class="source-line-no">3739</span><span id="line-3739"> // DoNotRetryIOException. This can avoid the retry in ClientScanner.</span>
<span class="source-line-no">3740</span><span id="line-3740"> if (e instanceof FileNotFoundException) {</span>
<span class="source-line-no">3741</span><span id="line-3741"> throw new DoNotRetryIOException(e);</span>
<span class="source-line-no">3742</span><span id="line-3742"> }</span>
<span class="source-line-no">3743</span><span id="line-3743"></span>
<span class="source-line-no">3744</span><span id="line-3744"> // We closed the scanner already. Instead of throwing the IOException, and client</span>
<span class="source-line-no">3745</span><span id="line-3745"> // retrying with the same scannerId only to get USE on the next RPC, we directly throw</span>
<span class="source-line-no">3746</span><span id="line-3746"> // a special exception to save an RPC.</span>
<span class="source-line-no">3747</span><span id="line-3747"> if (VersionInfoUtil.hasMinimumVersion(rpcCall.getClientVersionInfo(), 1, 4)) {</span>
<span class="source-line-no">3748</span><span id="line-3748"> // 1.4.0+ clients know how to handle</span>
<span class="source-line-no">3749</span><span id="line-3749"> throw new ScannerResetException("Scanner is closed on the server-side", e);</span>
<span class="source-line-no">3750</span><span id="line-3750"> } else {</span>
<span class="source-line-no">3751</span><span id="line-3751"> // older clients do not know about SRE. Just throw USE, which they will handle</span>
<span class="source-line-no">3752</span><span id="line-3752"> throw new UnknownScannerException("Throwing UnknownScannerException to reset the client"</span>
<span class="source-line-no">3753</span><span id="line-3753"> + " scanner state for clients older than 1.3.", e);</span>
<span class="source-line-no">3754</span><span id="line-3754"> }</span>
<span class="source-line-no">3755</span><span id="line-3755"> } catch (IOException ioe) {</span>
<span class="source-line-no">3756</span><span id="line-3756"> throw new ServiceException(ioe);</span>
<span class="source-line-no">3757</span><span id="line-3757"> }</span>
<span class="source-line-no">3758</span><span id="line-3758"> } finally {</span>
<span class="source-line-no">3759</span><span id="line-3759"> if (!scannerClosed) {</span>
<span class="source-line-no">3760</span><span id="line-3760"> // Adding resets expiration time on lease.</span>
<span class="source-line-no">3761</span><span id="line-3761"> // the closeCallBack will be set in closeScanner so here we only care about shippedCallback</span>
<span class="source-line-no">3762</span><span id="line-3762"> if (rpcCall != null) {</span>
<span class="source-line-no">3763</span><span id="line-3763"> rpcCall.setCallBack(rsh.shippedCallback);</span>
<span class="source-line-no">3764</span><span id="line-3764"> } else {</span>
<span class="source-line-no">3765</span><span id="line-3765"> // If context is null,here we call rsh.shippedCallback directly to reuse the logic in</span>
<span class="source-line-no">3766</span><span id="line-3766"> // rsh.shippedCallback to release the internal resources in rsh,and lease is also added</span>
<span class="source-line-no">3767</span><span id="line-3767"> // back to regionserver's LeaseManager in rsh.shippedCallback.</span>
<span class="source-line-no">3768</span><span id="line-3768"> runShippedCallback(rsh);</span>
<span class="source-line-no">3769</span><span id="line-3769"> }</span>
<span class="source-line-no">3770</span><span id="line-3770"> }</span>
<span class="source-line-no">3771</span><span id="line-3771"> quota.close();</span>
<span class="source-line-no">3772</span><span id="line-3772"> }</span>
<span class="source-line-no">3773</span><span id="line-3773"> }</span>
<span class="source-line-no">3774</span><span id="line-3774"></span>
<span class="source-line-no">3775</span><span id="line-3775"> private void runShippedCallback(RegionScannerHolder rsh) throws ServiceException {</span>
<span class="source-line-no">3776</span><span id="line-3776"> assert rsh.shippedCallback != null;</span>
<span class="source-line-no">3777</span><span id="line-3777"> try {</span>
<span class="source-line-no">3778</span><span id="line-3778"> rsh.shippedCallback.run();</span>
<span class="source-line-no">3779</span><span id="line-3779"> } catch (IOException ioe) {</span>
<span class="source-line-no">3780</span><span id="line-3780"> throw new ServiceException(ioe);</span>
<span class="source-line-no">3781</span><span id="line-3781"> }</span>
<span class="source-line-no">3782</span><span id="line-3782"> }</span>
<span class="source-line-no">3783</span><span id="line-3783"></span>
<span class="source-line-no">3784</span><span id="line-3784"> private void closeScanner(HRegion region, RegionScanner scanner, String scannerName,</span>
<span class="source-line-no">3785</span><span id="line-3785"> RpcCallContext context, boolean isError) throws IOException {</span>
<span class="source-line-no">3786</span><span id="line-3786"> if (region.getCoprocessorHost() != null) {</span>
<span class="source-line-no">3787</span><span id="line-3787"> if (region.getCoprocessorHost().preScannerClose(scanner)) {</span>
<span class="source-line-no">3788</span><span id="line-3788"> // bypass the actual close.</span>
<span class="source-line-no">3789</span><span id="line-3789"> return;</span>
<span class="source-line-no">3790</span><span id="line-3790"> }</span>
<span class="source-line-no">3791</span><span id="line-3791"> }</span>
<span class="source-line-no">3792</span><span id="line-3792"> RegionScannerHolder rsh = scanners.remove(scannerName);</span>
<span class="source-line-no">3793</span><span id="line-3793"> if (rsh != null) {</span>
<span class="source-line-no">3794</span><span id="line-3794"> if (context != null) {</span>
<span class="source-line-no">3795</span><span id="line-3795"> context.setCallBack(rsh.closeCallBack);</span>
<span class="source-line-no">3796</span><span id="line-3796"> } else {</span>
<span class="source-line-no">3797</span><span id="line-3797"> rsh.s.close();</span>
<span class="source-line-no">3798</span><span id="line-3798"> }</span>
<span class="source-line-no">3799</span><span id="line-3799"> if (region.getCoprocessorHost() != null) {</span>
<span class="source-line-no">3800</span><span id="line-3800"> region.getCoprocessorHost().postScannerClose(scanner);</span>
<span class="source-line-no">3801</span><span id="line-3801"> }</span>
<span class="source-line-no">3802</span><span id="line-3802"> if (!isError) {</span>
<span class="source-line-no">3803</span><span id="line-3803"> closedScanners.put(scannerName, rsh.getNextCallSeq());</span>
<span class="source-line-no">3804</span><span id="line-3804"> }</span>
<span class="source-line-no">3805</span><span id="line-3805"> }</span>
<span class="source-line-no">3806</span><span id="line-3806"> }</span>
<span class="source-line-no">3807</span><span id="line-3807"></span>
<span class="source-line-no">3808</span><span id="line-3808"> @Override</span>
<span class="source-line-no">3809</span><span id="line-3809"> public CoprocessorServiceResponse execRegionServerService(RpcController controller,</span>
<span class="source-line-no">3810</span><span id="line-3810"> CoprocessorServiceRequest request) throws ServiceException {</span>
<span class="source-line-no">3811</span><span id="line-3811"> rpcPreCheck("execRegionServerService");</span>
<span class="source-line-no">3812</span><span id="line-3812"> return server.execRegionServerService(controller, request);</span>
<span class="source-line-no">3813</span><span id="line-3813"> }</span>
<span class="source-line-no">3814</span><span id="line-3814"></span>
<span class="source-line-no">3815</span><span id="line-3815"> @Override</span>
<span class="source-line-no">3816</span><span id="line-3816"> public GetSpaceQuotaSnapshotsResponse getSpaceQuotaSnapshots(RpcController controller,</span>
<span class="source-line-no">3817</span><span id="line-3817"> GetSpaceQuotaSnapshotsRequest request) throws ServiceException {</span>
<span class="source-line-no">3818</span><span id="line-3818"> try {</span>
<span class="source-line-no">3819</span><span id="line-3819"> final RegionServerSpaceQuotaManager manager = server.getRegionServerSpaceQuotaManager();</span>
<span class="source-line-no">3820</span><span id="line-3820"> final GetSpaceQuotaSnapshotsResponse.Builder builder =</span>
<span class="source-line-no">3821</span><span id="line-3821"> GetSpaceQuotaSnapshotsResponse.newBuilder();</span>
<span class="source-line-no">3822</span><span id="line-3822"> if (manager != null) {</span>
<span class="source-line-no">3823</span><span id="line-3823"> final Map&lt;TableName, SpaceQuotaSnapshot&gt; snapshots = manager.copyQuotaSnapshots();</span>
<span class="source-line-no">3824</span><span id="line-3824"> for (Entry&lt;TableName, SpaceQuotaSnapshot&gt; snapshot : snapshots.entrySet()) {</span>
<span class="source-line-no">3825</span><span id="line-3825"> builder.addSnapshots(TableQuotaSnapshot.newBuilder()</span>
<span class="source-line-no">3826</span><span id="line-3826"> .setTableName(ProtobufUtil.toProtoTableName(snapshot.getKey()))</span>
<span class="source-line-no">3827</span><span id="line-3827"> .setSnapshot(SpaceQuotaSnapshot.toProtoSnapshot(snapshot.getValue())).build());</span>
<span class="source-line-no">3828</span><span id="line-3828"> }</span>
<span class="source-line-no">3829</span><span id="line-3829"> }</span>
<span class="source-line-no">3830</span><span id="line-3830"> return builder.build();</span>
<span class="source-line-no">3831</span><span id="line-3831"> } catch (Exception e) {</span>
<span class="source-line-no">3832</span><span id="line-3832"> throw new ServiceException(e);</span>
<span class="source-line-no">3833</span><span id="line-3833"> }</span>
<span class="source-line-no">3834</span><span id="line-3834"> }</span>
<span class="source-line-no">3835</span><span id="line-3835"></span>
<span class="source-line-no">3836</span><span id="line-3836"> @Override</span>
<span class="source-line-no">3837</span><span id="line-3837"> public ClearRegionBlockCacheResponse clearRegionBlockCache(RpcController controller,</span>
<span class="source-line-no">3838</span><span id="line-3838"> ClearRegionBlockCacheRequest request) throws ServiceException {</span>
<span class="source-line-no">3839</span><span id="line-3839"> try {</span>
<span class="source-line-no">3840</span><span id="line-3840"> rpcPreCheck("clearRegionBlockCache");</span>
<span class="source-line-no">3841</span><span id="line-3841"> ClearRegionBlockCacheResponse.Builder builder = ClearRegionBlockCacheResponse.newBuilder();</span>
<span class="source-line-no">3842</span><span id="line-3842"> CacheEvictionStatsBuilder stats = CacheEvictionStats.builder();</span>
<span class="source-line-no">3843</span><span id="line-3843"> server.getRegionServerCoprocessorHost().preClearRegionBlockCache();</span>
<span class="source-line-no">3844</span><span id="line-3844"> List&lt;HRegion&gt; regions = getRegions(request.getRegionList(), stats);</span>
<span class="source-line-no">3845</span><span id="line-3845"> for (HRegion region : regions) {</span>
<span class="source-line-no">3846</span><span id="line-3846"> try {</span>
<span class="source-line-no">3847</span><span id="line-3847"> stats = stats.append(this.server.clearRegionBlockCache(region));</span>
<span class="source-line-no">3848</span><span id="line-3848"> } catch (Exception e) {</span>
<span class="source-line-no">3849</span><span id="line-3849"> stats.addException(region.getRegionInfo().getRegionName(), e);</span>
<span class="source-line-no">3850</span><span id="line-3850"> }</span>
<span class="source-line-no">3851</span><span id="line-3851"> }</span>
<span class="source-line-no">3852</span><span id="line-3852"> stats.withMaxCacheSize(server.getBlockCache().map(BlockCache::getMaxSize).orElse(0L));</span>
<span class="source-line-no">3853</span><span id="line-3853"> server.getRegionServerCoprocessorHost().postClearRegionBlockCache(stats.build());</span>
<span class="source-line-no">3854</span><span id="line-3854"> return builder.setStats(ProtobufUtil.toCacheEvictionStats(stats.build())).build();</span>
<span class="source-line-no">3855</span><span id="line-3855"> } catch (IOException e) {</span>
<span class="source-line-no">3856</span><span id="line-3856"> throw new ServiceException(e);</span>
<span class="source-line-no">3857</span><span id="line-3857"> }</span>
<span class="source-line-no">3858</span><span id="line-3858"> }</span>
<span class="source-line-no">3859</span><span id="line-3859"></span>
<span class="source-line-no">3860</span><span id="line-3860"> private void executeOpenRegionProcedures(OpenRegionRequest request,</span>
<span class="source-line-no">3861</span><span id="line-3861"> Map&lt;TableName, TableDescriptor&gt; tdCache) {</span>
<span class="source-line-no">3862</span><span id="line-3862"> long masterSystemTime = request.hasMasterSystemTime() ? request.getMasterSystemTime() : -1;</span>
<span class="source-line-no">3863</span><span id="line-3863"> long initiatingMasterActiveTime =</span>
<span class="source-line-no">3864</span><span id="line-3864"> request.hasInitiatingMasterActiveTime() ? request.getInitiatingMasterActiveTime() : -1;</span>
<span class="source-line-no">3865</span><span id="line-3865"> for (RegionOpenInfo regionOpenInfo : request.getOpenInfoList()) {</span>
<span class="source-line-no">3866</span><span id="line-3866"> RegionInfo regionInfo = ProtobufUtil.toRegionInfo(regionOpenInfo.getRegion());</span>
<span class="source-line-no">3867</span><span id="line-3867"> TableName tableName = regionInfo.getTable();</span>
<span class="source-line-no">3868</span><span id="line-3868"> TableDescriptor tableDesc = tdCache.get(tableName);</span>
<span class="source-line-no">3869</span><span id="line-3869"> if (tableDesc == null) {</span>
<span class="source-line-no">3870</span><span id="line-3870"> try {</span>
<span class="source-line-no">3871</span><span id="line-3871"> tableDesc = server.getTableDescriptors().get(regionInfo.getTable());</span>
<span class="source-line-no">3872</span><span id="line-3872"> } catch (IOException e) {</span>
<span class="source-line-no">3873</span><span id="line-3873"> // Here we do not fail the whole method since we also need deal with other</span>
<span class="source-line-no">3874</span><span id="line-3874"> // procedures, and we can not ignore this one, so we still schedule a</span>
<span class="source-line-no">3875</span><span id="line-3875"> // AssignRegionHandler and it will report back to master if we still can not get the</span>
<span class="source-line-no">3876</span><span id="line-3876"> // TableDescriptor.</span>
<span class="source-line-no">3877</span><span id="line-3877"> LOG.warn("Failed to get TableDescriptor of {}, will try again in the handler",</span>
<span class="source-line-no">3878</span><span id="line-3878"> regionInfo.getTable(), e);</span>
<span class="source-line-no">3879</span><span id="line-3879"> }</span>
<span class="source-line-no">3880</span><span id="line-3880"> if (tableDesc != null) {</span>
<span class="source-line-no">3881</span><span id="line-3881"> tdCache.put(tableName, tableDesc);</span>
<span class="source-line-no">3882</span><span id="line-3882"> }</span>
<span class="source-line-no">3883</span><span id="line-3883"> }</span>
<span class="source-line-no">3884</span><span id="line-3884"> if (regionOpenInfo.getFavoredNodesCount() &gt; 0) {</span>
<span class="source-line-no">3885</span><span id="line-3885"> server.updateRegionFavoredNodesMapping(regionInfo.getEncodedName(),</span>
<span class="source-line-no">3886</span><span id="line-3886"> regionOpenInfo.getFavoredNodesList());</span>
<span class="source-line-no">3887</span><span id="line-3887"> }</span>
<span class="source-line-no">3888</span><span id="line-3888"> long procId = regionOpenInfo.getOpenProcId();</span>
<span class="source-line-no">3889</span><span id="line-3889"> if (server.submitRegionProcedure(procId)) {</span>
<span class="source-line-no">3890</span><span id="line-3890"> server.getExecutorService().submit(AssignRegionHandler.create(server, regionInfo, procId,</span>
<span class="source-line-no">3891</span><span id="line-3891"> tableDesc, masterSystemTime, initiatingMasterActiveTime));</span>
<span class="source-line-no">3892</span><span id="line-3892"> }</span>
<span class="source-line-no">3893</span><span id="line-3893"> }</span>
<span class="source-line-no">3894</span><span id="line-3894"> }</span>
<span class="source-line-no">3895</span><span id="line-3895"></span>
<span class="source-line-no">3896</span><span id="line-3896"> private void executeCloseRegionProcedures(CloseRegionRequest request) {</span>
<span class="source-line-no">3897</span><span id="line-3897"> String encodedName;</span>
<span class="source-line-no">3898</span><span id="line-3898"> long initiatingMasterActiveTime =</span>
<span class="source-line-no">3899</span><span id="line-3899"> request.hasInitiatingMasterActiveTime() ? request.getInitiatingMasterActiveTime() : -1;</span>
<span class="source-line-no">3900</span><span id="line-3900"> try {</span>
<span class="source-line-no">3901</span><span id="line-3901"> encodedName = ProtobufUtil.getRegionEncodedName(request.getRegion());</span>
<span class="source-line-no">3902</span><span id="line-3902"> } catch (DoNotRetryIOException e) {</span>
<span class="source-line-no">3903</span><span id="line-3903"> throw new UncheckedIOException("Should not happen", e);</span>
<span class="source-line-no">3904</span><span id="line-3904"> }</span>
<span class="source-line-no">3905</span><span id="line-3905"> ServerName destination = request.hasDestinationServer()</span>
<span class="source-line-no">3906</span><span id="line-3906"> ? ProtobufUtil.toServerName(request.getDestinationServer())</span>
<span class="source-line-no">3907</span><span id="line-3907"> : null;</span>
<span class="source-line-no">3908</span><span id="line-3908"> long procId = request.getCloseProcId();</span>
<span class="source-line-no">3909</span><span id="line-3909"> boolean evictCache = request.getEvictCache();</span>
<span class="source-line-no">3910</span><span id="line-3910"> if (server.submitRegionProcedure(procId)) {</span>
<span class="source-line-no">3911</span><span id="line-3911"> server.getExecutorService().submit(UnassignRegionHandler.create(server, encodedName, procId,</span>
<span class="source-line-no">3912</span><span id="line-3912"> false, destination, evictCache, initiatingMasterActiveTime));</span>
<span class="source-line-no">3913</span><span id="line-3913"> }</span>
<span class="source-line-no">3914</span><span id="line-3914"> }</span>
<span class="source-line-no">3915</span><span id="line-3915"></span>
<span class="source-line-no">3916</span><span id="line-3916"> private void executeProcedures(RemoteProcedureRequest request) {</span>
<span class="source-line-no">3917</span><span id="line-3917"> RSProcedureCallable callable;</span>
<span class="source-line-no">3918</span><span id="line-3918"> try {</span>
<span class="source-line-no">3919</span><span id="line-3919"> callable = Class.forName(request.getProcClass()).asSubclass(RSProcedureCallable.class)</span>
<span class="source-line-no">3920</span><span id="line-3920"> .getDeclaredConstructor().newInstance();</span>
<span class="source-line-no">3921</span><span id="line-3921"> } catch (Exception e) {</span>
<span class="source-line-no">3922</span><span id="line-3922"> LOG.warn("Failed to instantiating remote procedure {}, pid={}", request.getProcClass(),</span>
<span class="source-line-no">3923</span><span id="line-3923"> request.getProcId(), e);</span>
<span class="source-line-no">3924</span><span id="line-3924"> server.remoteProcedureComplete(request.getProcId(), request.getInitiatingMasterActiveTime(),</span>
<span class="source-line-no">3925</span><span id="line-3925"> e);</span>
<span class="source-line-no">3926</span><span id="line-3926"> return;</span>
<span class="source-line-no">3927</span><span id="line-3927"> }</span>
<span class="source-line-no">3928</span><span id="line-3928"> callable.init(request.getProcData().toByteArray(), server);</span>
<span class="source-line-no">3929</span><span id="line-3929"> LOG.debug("Executing remote procedure {}, pid={}", callable.getClass(), request.getProcId());</span>
<span class="source-line-no">3930</span><span id="line-3930"> server.executeProcedure(request.getProcId(), request.getInitiatingMasterActiveTime(), callable);</span>
<span class="source-line-no">3931</span><span id="line-3931"> }</span>
<span class="source-line-no">3932</span><span id="line-3932"></span>
<span class="source-line-no">3933</span><span id="line-3933"> @Override</span>
<span class="source-line-no">3934</span><span id="line-3934"> @QosPriority(priority = HConstants.ADMIN_QOS)</span>
<span class="source-line-no">3935</span><span id="line-3935"> public ExecuteProceduresResponse executeProcedures(RpcController controller,</span>
<span class="source-line-no">3936</span><span id="line-3936"> ExecuteProceduresRequest request) throws ServiceException {</span>
<span class="source-line-no">3937</span><span id="line-3937"> try {</span>
<span class="source-line-no">3938</span><span id="line-3938"> checkOpen();</span>
<span class="source-line-no">3939</span><span id="line-3939"> throwOnWrongStartCode(request);</span>
<span class="source-line-no">3940</span><span id="line-3940"> server.getRegionServerCoprocessorHost().preExecuteProcedures();</span>
<span class="source-line-no">3941</span><span id="line-3941"> if (request.getOpenRegionCount() &gt; 0) {</span>
<span class="source-line-no">3942</span><span id="line-3942"> // Avoid reading from the TableDescritor every time(usually it will read from the file</span>
<span class="source-line-no">3943</span><span id="line-3943"> // system)</span>
<span class="source-line-no">3944</span><span id="line-3944"> Map&lt;TableName, TableDescriptor&gt; tdCache = new HashMap&lt;&gt;();</span>
<span class="source-line-no">3945</span><span id="line-3945"> request.getOpenRegionList().forEach(req -&gt; executeOpenRegionProcedures(req, tdCache));</span>
<span class="source-line-no">3946</span><span id="line-3946"> }</span>
<span class="source-line-no">3947</span><span id="line-3947"> if (request.getCloseRegionCount() &gt; 0) {</span>
<span class="source-line-no">3948</span><span id="line-3948"> request.getCloseRegionList().forEach(this::executeCloseRegionProcedures);</span>
<span class="source-line-no">3949</span><span id="line-3949"> }</span>
<span class="source-line-no">3950</span><span id="line-3950"> if (request.getProcCount() &gt; 0) {</span>
<span class="source-line-no">3951</span><span id="line-3951"> request.getProcList().forEach(this::executeProcedures);</span>
<span class="source-line-no">3952</span><span id="line-3952"> }</span>
<span class="source-line-no">3953</span><span id="line-3953"> server.getRegionServerCoprocessorHost().postExecuteProcedures();</span>
<span class="source-line-no">3954</span><span id="line-3954"> return ExecuteProceduresResponse.getDefaultInstance();</span>
<span class="source-line-no">3955</span><span id="line-3955"> } catch (IOException e) {</span>
<span class="source-line-no">3956</span><span id="line-3956"> throw new ServiceException(e);</span>
<span class="source-line-no">3957</span><span id="line-3957"> }</span>
<span class="source-line-no">3958</span><span id="line-3958"> }</span>
<span class="source-line-no">3959</span><span id="line-3959"></span>
<span class="source-line-no">3960</span><span id="line-3960"> @Override</span>
<span class="source-line-no">3961</span><span id="line-3961"> public GetAllBootstrapNodesResponse getAllBootstrapNodes(RpcController controller,</span>
<span class="source-line-no">3962</span><span id="line-3962"> GetAllBootstrapNodesRequest request) throws ServiceException {</span>
<span class="source-line-no">3963</span><span id="line-3963"> GetAllBootstrapNodesResponse.Builder builder = GetAllBootstrapNodesResponse.newBuilder();</span>
<span class="source-line-no">3964</span><span id="line-3964"> server.getBootstrapNodes()</span>
<span class="source-line-no">3965</span><span id="line-3965"> .forEachRemaining(server -&gt; builder.addNode(ProtobufUtil.toServerName(server)));</span>
<span class="source-line-no">3966</span><span id="line-3966"> return builder.build();</span>
<span class="source-line-no">3967</span><span id="line-3967"> }</span>
<span class="source-line-no">3968</span><span id="line-3968"></span>
<span class="source-line-no">3969</span><span id="line-3969"> private void setReloadableGuardrails(Configuration conf) {</span>
<span class="source-line-no">3970</span><span id="line-3970"> rowSizeWarnThreshold =</span>
<span class="source-line-no">3971</span><span id="line-3971"> conf.getInt(HConstants.BATCH_ROWS_THRESHOLD_NAME, HConstants.BATCH_ROWS_THRESHOLD_DEFAULT);</span>
<span class="source-line-no">3972</span><span id="line-3972"> rejectRowsWithSizeOverThreshold =</span>
<span class="source-line-no">3973</span><span id="line-3973"> conf.getBoolean(REJECT_BATCH_ROWS_OVER_THRESHOLD, DEFAULT_REJECT_BATCH_ROWS_OVER_THRESHOLD);</span>
<span class="source-line-no">3974</span><span id="line-3974"> maxScannerResultSize = conf.getLong(HConstants.HBASE_SERVER_SCANNER_MAX_RESULT_SIZE_KEY,</span>
<span class="source-line-no">3975</span><span id="line-3975"> HConstants.DEFAULT_HBASE_SERVER_SCANNER_MAX_RESULT_SIZE);</span>
<span class="source-line-no">3976</span><span id="line-3976"> }</span>
<span class="source-line-no">3977</span><span id="line-3977"></span>
<span class="source-line-no">3978</span><span id="line-3978"> @Override</span>
<span class="source-line-no">3979</span><span id="line-3979"> public void onConfigurationChange(Configuration conf) {</span>
<span class="source-line-no">3980</span><span id="line-3980"> super.onConfigurationChange(conf);</span>
<span class="source-line-no">3981</span><span id="line-3981"> setReloadableGuardrails(conf);</span>
<span class="source-line-no">3982</span><span id="line-3982"> }</span>
<span class="source-line-no">3983</span><span id="line-3983"></span>
<span class="source-line-no">3984</span><span id="line-3984"> @Override</span>
<span class="source-line-no">3985</span><span id="line-3985"> public GetCachedFilesListResponse getCachedFilesList(RpcController controller,</span>
<span class="source-line-no">3986</span><span id="line-3986"> GetCachedFilesListRequest request) throws ServiceException {</span>
<span class="source-line-no">3987</span><span id="line-3987"> GetCachedFilesListResponse.Builder responseBuilder = GetCachedFilesListResponse.newBuilder();</span>
<span class="source-line-no">3988</span><span id="line-3988"> List&lt;String&gt; fullyCachedFiles = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">3989</span><span id="line-3989"> server.getBlockCache().flatMap(BlockCache::getFullyCachedFiles).ifPresent(fcf -&gt; {</span>
<span class="source-line-no">3990</span><span id="line-3990"> fullyCachedFiles.addAll(fcf.keySet());</span>
<span class="source-line-no">3991</span><span id="line-3991"> });</span>
<span class="source-line-no">3992</span><span id="line-3992"> return responseBuilder.addAllCachedFiles(fullyCachedFiles).build();</span>
<span class="source-line-no">3993</span><span id="line-3993"> }</span>
<span class="source-line-no">3994</span><span id="line-3994"></span>
<span class="source-line-no">3995</span><span id="line-3995"> RegionScannerContext checkQuotaAndGetRegionScannerContext(ScanRequest request,</span>
<span class="source-line-no">3996</span><span id="line-3996"> ScanResponse.Builder builder) throws IOException {</span>
<span class="source-line-no">3997</span><span id="line-3997"> if (request.hasScannerId()) {</span>
<span class="source-line-no">3998</span><span id="line-3998"> // The downstream projects such as AsyncHBase in OpenTSDB need this value. See HBASE-18000</span>
<span class="source-line-no">3999</span><span id="line-3999"> // for more details.</span>
<span class="source-line-no">4000</span><span id="line-4000"> long scannerId = request.getScannerId();</span>
<span class="source-line-no">4001</span><span id="line-4001"> builder.setScannerId(scannerId);</span>
<span class="source-line-no">4002</span><span id="line-4002"> String scannerName = toScannerName(scannerId);</span>
<span class="source-line-no">4003</span><span id="line-4003"> RegionScannerHolder rsh = getRegionScanner(request);</span>
<span class="source-line-no">4004</span><span id="line-4004"> OperationQuota quota =</span>
<span class="source-line-no">4005</span><span id="line-4005"> getRpcQuotaManager().checkScanQuota(rsh.r, request, maxScannerResultSize,</span>
<span class="source-line-no">4006</span><span id="line-4006"> rsh.getMaxBlockBytesScanned(), rsh.getPrevBlockBytesScannedDifference());</span>
<span class="source-line-no">4007</span><span id="line-4007"> return new RegionScannerContext(scannerName, rsh, quota);</span>
<span class="source-line-no">4008</span><span id="line-4008"> }</span>
<span class="source-line-no">4009</span><span id="line-4009"></span>
<span class="source-line-no">4010</span><span id="line-4010"> HRegion region = getRegion(request.getRegion());</span>
<span class="source-line-no">4011</span><span id="line-4011"> OperationQuota quota =</span>
<span class="source-line-no">4012</span><span id="line-4012"> getRpcQuotaManager().checkScanQuota(region, request, maxScannerResultSize, 0L, 0L);</span>
<span class="source-line-no">4013</span><span id="line-4013"> Pair&lt;String, RegionScannerHolder&gt; pair = newRegionScanner(request, region, builder);</span>
<span class="source-line-no">4014</span><span id="line-4014"> return new RegionScannerContext(pair.getFirst(), pair.getSecond(), quota);</span>
<span class="source-line-no">4015</span><span id="line-4015"> }</span>
<span class="source-line-no">4016</span><span id="line-4016">}</span>
</pre>
</div>
</main>
</body>
</html>