blob: 7ca91a183ac294fcb1757829b14f680552f97ac3 [file] [log] [blame]
<!DOCTYPE HTML PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
<html lang="en">
<head>
<title>Source code</title>
<link rel="stylesheet" type="text/css" href="../../../../../../stylesheet.css" title="Style">
</head>
<body>
<div class="sourceContainer">
<pre><span class="sourceLineNo">001</span>/**<a name="line.1"></a>
<span class="sourceLineNo">002</span> * Licensed to the Apache Software Foundation (ASF) under one<a name="line.2"></a>
<span class="sourceLineNo">003</span> * or more contributor license agreements. See the NOTICE file<a name="line.3"></a>
<span class="sourceLineNo">004</span> * distributed with this work for additional information<a name="line.4"></a>
<span class="sourceLineNo">005</span> * regarding copyright ownership. The ASF licenses this file<a name="line.5"></a>
<span class="sourceLineNo">006</span> * to you under the Apache License, Version 2.0 (the<a name="line.6"></a>
<span class="sourceLineNo">007</span> * "License"); you may not use this file except in compliance<a name="line.7"></a>
<span class="sourceLineNo">008</span> * with the License. You may obtain a copy of the License at<a name="line.8"></a>
<span class="sourceLineNo">009</span> *<a name="line.9"></a>
<span class="sourceLineNo">010</span> * http://www.apache.org/licenses/LICENSE-2.0<a name="line.10"></a>
<span class="sourceLineNo">011</span> *<a name="line.11"></a>
<span class="sourceLineNo">012</span> * Unless required by applicable law or agreed to in writing, software<a name="line.12"></a>
<span class="sourceLineNo">013</span> * distributed under the License is distributed on an "AS IS" BASIS,<a name="line.13"></a>
<span class="sourceLineNo">014</span> * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.<a name="line.14"></a>
<span class="sourceLineNo">015</span> * See the License for the specific language governing permissions and<a name="line.15"></a>
<span class="sourceLineNo">016</span> * limitations under the License.<a name="line.16"></a>
<span class="sourceLineNo">017</span> */<a name="line.17"></a>
<span class="sourceLineNo">018</span>package org.apache.hadoop.hbase.regionserver;<a name="line.18"></a>
<span class="sourceLineNo">019</span><a name="line.19"></a>
<span class="sourceLineNo">020</span>import java.io.IOException;<a name="line.20"></a>
<span class="sourceLineNo">021</span>import java.util.AbstractList;<a name="line.21"></a>
<span class="sourceLineNo">022</span>import java.util.ArrayList;<a name="line.22"></a>
<span class="sourceLineNo">023</span>import java.util.Collection;<a name="line.23"></a>
<span class="sourceLineNo">024</span>import java.util.List;<a name="line.24"></a>
<span class="sourceLineNo">025</span>import java.util.Map;<a name="line.25"></a>
<span class="sourceLineNo">026</span>import java.util.NavigableSet;<a name="line.26"></a>
<span class="sourceLineNo">027</span>import java.util.Optional;<a name="line.27"></a>
<span class="sourceLineNo">028</span>import java.util.concurrent.ConcurrentHashMap;<a name="line.28"></a>
<span class="sourceLineNo">029</span>import org.apache.hadoop.hbase.Cell;<a name="line.29"></a>
<span class="sourceLineNo">030</span>import org.apache.hadoop.hbase.CellComparator;<a name="line.30"></a>
<span class="sourceLineNo">031</span>import org.apache.hadoop.hbase.CellUtil;<a name="line.31"></a>
<span class="sourceLineNo">032</span>import org.apache.hadoop.hbase.HConstants;<a name="line.32"></a>
<span class="sourceLineNo">033</span>import org.apache.hadoop.hbase.KeyValue;<a name="line.33"></a>
<span class="sourceLineNo">034</span>import org.apache.hadoop.hbase.PrivateCellUtil;<a name="line.34"></a>
<span class="sourceLineNo">035</span>import org.apache.hadoop.hbase.UnknownScannerException;<a name="line.35"></a>
<span class="sourceLineNo">036</span>import org.apache.hadoop.hbase.client.IsolationLevel;<a name="line.36"></a>
<span class="sourceLineNo">037</span>import org.apache.hadoop.hbase.client.PackagePrivateFieldAccessor;<a name="line.37"></a>
<span class="sourceLineNo">038</span>import org.apache.hadoop.hbase.client.RegionInfo;<a name="line.38"></a>
<span class="sourceLineNo">039</span>import org.apache.hadoop.hbase.client.Scan;<a name="line.39"></a>
<span class="sourceLineNo">040</span>import org.apache.hadoop.hbase.filter.FilterWrapper;<a name="line.40"></a>
<span class="sourceLineNo">041</span>import org.apache.hadoop.hbase.filter.IncompatibleFilterException;<a name="line.41"></a>
<span class="sourceLineNo">042</span>import org.apache.hadoop.hbase.ipc.CallerDisconnectedException;<a name="line.42"></a>
<span class="sourceLineNo">043</span>import org.apache.hadoop.hbase.ipc.RpcCall;<a name="line.43"></a>
<span class="sourceLineNo">044</span>import org.apache.hadoop.hbase.ipc.RpcCallback;<a name="line.44"></a>
<span class="sourceLineNo">045</span>import org.apache.hadoop.hbase.ipc.RpcServer;<a name="line.45"></a>
<span class="sourceLineNo">046</span>import org.apache.hadoop.hbase.regionserver.Region.Operation;<a name="line.46"></a>
<span class="sourceLineNo">047</span>import org.apache.hadoop.hbase.regionserver.ScannerContext.LimitScope;<a name="line.47"></a>
<span class="sourceLineNo">048</span>import org.apache.hadoop.hbase.regionserver.ScannerContext.NextState;<a name="line.48"></a>
<span class="sourceLineNo">049</span>import org.apache.hadoop.hbase.util.Bytes;<a name="line.49"></a>
<span class="sourceLineNo">050</span>import org.apache.yetus.audience.InterfaceAudience;<a name="line.50"></a>
<span class="sourceLineNo">051</span>import org.slf4j.Logger;<a name="line.51"></a>
<span class="sourceLineNo">052</span>import org.slf4j.LoggerFactory;<a name="line.52"></a>
<span class="sourceLineNo">053</span><a name="line.53"></a>
<span class="sourceLineNo">054</span>import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;<a name="line.54"></a>
<span class="sourceLineNo">055</span><a name="line.55"></a>
<span class="sourceLineNo">056</span>/**<a name="line.56"></a>
<span class="sourceLineNo">057</span> * RegionScannerImpl is used to combine scanners from multiple Stores (aka column families).<a name="line.57"></a>
<span class="sourceLineNo">058</span> */<a name="line.58"></a>
<span class="sourceLineNo">059</span>@InterfaceAudience.Private<a name="line.59"></a>
<span class="sourceLineNo">060</span>class RegionScannerImpl implements RegionScanner, Shipper, RpcCallback {<a name="line.60"></a>
<span class="sourceLineNo">061</span><a name="line.61"></a>
<span class="sourceLineNo">062</span> private static final Logger LOG = LoggerFactory.getLogger(RegionScannerImpl.class);<a name="line.62"></a>
<span class="sourceLineNo">063</span><a name="line.63"></a>
<span class="sourceLineNo">064</span> // Package local for testability<a name="line.64"></a>
<span class="sourceLineNo">065</span> KeyValueHeap storeHeap = null;<a name="line.65"></a>
<span class="sourceLineNo">066</span><a name="line.66"></a>
<span class="sourceLineNo">067</span> /**<a name="line.67"></a>
<span class="sourceLineNo">068</span> * Heap of key-values that are not essential for the provided filters and are thus read on demand,<a name="line.68"></a>
<span class="sourceLineNo">069</span> * if on-demand column family loading is enabled.<a name="line.69"></a>
<span class="sourceLineNo">070</span> */<a name="line.70"></a>
<span class="sourceLineNo">071</span> KeyValueHeap joinedHeap = null;<a name="line.71"></a>
<span class="sourceLineNo">072</span><a name="line.72"></a>
<span class="sourceLineNo">073</span> /**<a name="line.73"></a>
<span class="sourceLineNo">074</span> * If the joined heap data gathering is interrupted due to scan limits, this will contain the row<a name="line.74"></a>
<span class="sourceLineNo">075</span> * for which we are populating the values.<a name="line.75"></a>
<span class="sourceLineNo">076</span> */<a name="line.76"></a>
<span class="sourceLineNo">077</span> protected Cell joinedContinuationRow = null;<a name="line.77"></a>
<span class="sourceLineNo">078</span> private boolean filterClosed = false;<a name="line.78"></a>
<span class="sourceLineNo">079</span><a name="line.79"></a>
<span class="sourceLineNo">080</span> protected final byte[] stopRow;<a name="line.80"></a>
<span class="sourceLineNo">081</span> protected final boolean includeStopRow;<a name="line.81"></a>
<span class="sourceLineNo">082</span> protected final HRegion region;<a name="line.82"></a>
<span class="sourceLineNo">083</span> protected final CellComparator comparator;<a name="line.83"></a>
<span class="sourceLineNo">084</span><a name="line.84"></a>
<span class="sourceLineNo">085</span> private final ConcurrentHashMap&lt;RegionScanner, Long&gt; scannerReadPoints;<a name="line.85"></a>
<span class="sourceLineNo">086</span><a name="line.86"></a>
<span class="sourceLineNo">087</span> private final long readPt;<a name="line.87"></a>
<span class="sourceLineNo">088</span> private final long maxResultSize;<a name="line.88"></a>
<span class="sourceLineNo">089</span> private final ScannerContext defaultScannerContext;<a name="line.89"></a>
<span class="sourceLineNo">090</span> private final FilterWrapper filter;<a name="line.90"></a>
<span class="sourceLineNo">091</span><a name="line.91"></a>
<span class="sourceLineNo">092</span> private RegionServerServices rsServices;<a name="line.92"></a>
<span class="sourceLineNo">093</span><a name="line.93"></a>
<span class="sourceLineNo">094</span> @Override<a name="line.94"></a>
<span class="sourceLineNo">095</span> public RegionInfo getRegionInfo() {<a name="line.95"></a>
<span class="sourceLineNo">096</span> return region.getRegionInfo();<a name="line.96"></a>
<span class="sourceLineNo">097</span> }<a name="line.97"></a>
<span class="sourceLineNo">098</span><a name="line.98"></a>
<span class="sourceLineNo">099</span> private static boolean hasNonce(HRegion region, long nonce) {<a name="line.99"></a>
<span class="sourceLineNo">100</span> RegionServerServices rsServices = region.getRegionServerServices();<a name="line.100"></a>
<span class="sourceLineNo">101</span> return nonce != HConstants.NO_NONCE &amp;&amp; rsServices != null &amp;&amp;<a name="line.101"></a>
<span class="sourceLineNo">102</span> rsServices.getNonceManager() != null;<a name="line.102"></a>
<span class="sourceLineNo">103</span> }<a name="line.103"></a>
<span class="sourceLineNo">104</span><a name="line.104"></a>
<span class="sourceLineNo">105</span> RegionScannerImpl(Scan scan, List&lt;KeyValueScanner&gt; additionalScanners, HRegion region,<a name="line.105"></a>
<span class="sourceLineNo">106</span> long nonceGroup, long nonce) throws IOException {<a name="line.106"></a>
<span class="sourceLineNo">107</span> this.region = region;<a name="line.107"></a>
<span class="sourceLineNo">108</span> this.maxResultSize = scan.getMaxResultSize();<a name="line.108"></a>
<span class="sourceLineNo">109</span> if (scan.hasFilter()) {<a name="line.109"></a>
<span class="sourceLineNo">110</span> this.filter = new FilterWrapper(scan.getFilter());<a name="line.110"></a>
<span class="sourceLineNo">111</span> } else {<a name="line.111"></a>
<span class="sourceLineNo">112</span> this.filter = null;<a name="line.112"></a>
<span class="sourceLineNo">113</span> }<a name="line.113"></a>
<span class="sourceLineNo">114</span> this.comparator = region.getCellComparator();<a name="line.114"></a>
<span class="sourceLineNo">115</span> /**<a name="line.115"></a>
<span class="sourceLineNo">116</span> * By default, calls to next/nextRaw must enforce the batch limit. Thus, construct a default<a name="line.116"></a>
<span class="sourceLineNo">117</span> * scanner context that can be used to enforce the batch limit in the event that a<a name="line.117"></a>
<span class="sourceLineNo">118</span> * ScannerContext is not specified during an invocation of next/nextRaw<a name="line.118"></a>
<span class="sourceLineNo">119</span> */<a name="line.119"></a>
<span class="sourceLineNo">120</span> defaultScannerContext = ScannerContext.newBuilder().setBatchLimit(scan.getBatch()).build();<a name="line.120"></a>
<span class="sourceLineNo">121</span> this.stopRow = scan.getStopRow();<a name="line.121"></a>
<span class="sourceLineNo">122</span> this.includeStopRow = scan.includeStopRow();<a name="line.122"></a>
<span class="sourceLineNo">123</span><a name="line.123"></a>
<span class="sourceLineNo">124</span> // synchronize on scannerReadPoints so that nobody calculates<a name="line.124"></a>
<span class="sourceLineNo">125</span> // getSmallestReadPoint, before scannerReadPoints is updated.<a name="line.125"></a>
<span class="sourceLineNo">126</span> IsolationLevel isolationLevel = scan.getIsolationLevel();<a name="line.126"></a>
<span class="sourceLineNo">127</span> long mvccReadPoint = PackagePrivateFieldAccessor.getMvccReadPoint(scan);<a name="line.127"></a>
<span class="sourceLineNo">128</span> this.scannerReadPoints = region.scannerReadPoints;<a name="line.128"></a>
<span class="sourceLineNo">129</span> this.rsServices = region.getRegionServerServices();<a name="line.129"></a>
<span class="sourceLineNo">130</span> synchronized (scannerReadPoints) {<a name="line.130"></a>
<span class="sourceLineNo">131</span> if (mvccReadPoint &gt; 0) {<a name="line.131"></a>
<span class="sourceLineNo">132</span> this.readPt = mvccReadPoint;<a name="line.132"></a>
<span class="sourceLineNo">133</span> } else if (hasNonce(region, nonce)) {<a name="line.133"></a>
<span class="sourceLineNo">134</span> this.readPt = rsServices.getNonceManager().getMvccFromOperationContext(nonceGroup, nonce);<a name="line.134"></a>
<span class="sourceLineNo">135</span> } else {<a name="line.135"></a>
<span class="sourceLineNo">136</span> this.readPt = region.getReadPoint(isolationLevel);<a name="line.136"></a>
<span class="sourceLineNo">137</span> }<a name="line.137"></a>
<span class="sourceLineNo">138</span> scannerReadPoints.put(this, this.readPt);<a name="line.138"></a>
<span class="sourceLineNo">139</span> }<a name="line.139"></a>
<span class="sourceLineNo">140</span> initializeScanners(scan, additionalScanners);<a name="line.140"></a>
<span class="sourceLineNo">141</span> }<a name="line.141"></a>
<span class="sourceLineNo">142</span><a name="line.142"></a>
<span class="sourceLineNo">143</span> private void initializeScanners(Scan scan, List&lt;KeyValueScanner&gt; additionalScanners)<a name="line.143"></a>
<span class="sourceLineNo">144</span> throws IOException {<a name="line.144"></a>
<span class="sourceLineNo">145</span> // Here we separate all scanners into two lists - scanner that provide data required<a name="line.145"></a>
<span class="sourceLineNo">146</span> // by the filter to operate (scanners list) and all others (joinedScanners list).<a name="line.146"></a>
<span class="sourceLineNo">147</span> List&lt;KeyValueScanner&gt; scanners = new ArrayList&lt;&gt;(scan.getFamilyMap().size());<a name="line.147"></a>
<span class="sourceLineNo">148</span> List&lt;KeyValueScanner&gt; joinedScanners = new ArrayList&lt;&gt;(scan.getFamilyMap().size());<a name="line.148"></a>
<span class="sourceLineNo">149</span> // Store all already instantiated scanners for exception handling<a name="line.149"></a>
<span class="sourceLineNo">150</span> List&lt;KeyValueScanner&gt; instantiatedScanners = new ArrayList&lt;&gt;();<a name="line.150"></a>
<span class="sourceLineNo">151</span> // handle additionalScanners<a name="line.151"></a>
<span class="sourceLineNo">152</span> if (additionalScanners != null &amp;&amp; !additionalScanners.isEmpty()) {<a name="line.152"></a>
<span class="sourceLineNo">153</span> scanners.addAll(additionalScanners);<a name="line.153"></a>
<span class="sourceLineNo">154</span> instantiatedScanners.addAll(additionalScanners);<a name="line.154"></a>
<span class="sourceLineNo">155</span> }<a name="line.155"></a>
<span class="sourceLineNo">156</span><a name="line.156"></a>
<span class="sourceLineNo">157</span> try {<a name="line.157"></a>
<span class="sourceLineNo">158</span> for (Map.Entry&lt;byte[], NavigableSet&lt;byte[]&gt;&gt; entry : scan.getFamilyMap().entrySet()) {<a name="line.158"></a>
<span class="sourceLineNo">159</span> HStore store = region.getStore(entry.getKey());<a name="line.159"></a>
<span class="sourceLineNo">160</span> KeyValueScanner scanner = store.getScanner(scan, entry.getValue(), this.readPt);<a name="line.160"></a>
<span class="sourceLineNo">161</span> instantiatedScanners.add(scanner);<a name="line.161"></a>
<span class="sourceLineNo">162</span> if (this.filter == null || !scan.doLoadColumnFamiliesOnDemand() ||<a name="line.162"></a>
<span class="sourceLineNo">163</span> this.filter.isFamilyEssential(entry.getKey())) {<a name="line.163"></a>
<span class="sourceLineNo">164</span> scanners.add(scanner);<a name="line.164"></a>
<span class="sourceLineNo">165</span> } else {<a name="line.165"></a>
<span class="sourceLineNo">166</span> joinedScanners.add(scanner);<a name="line.166"></a>
<span class="sourceLineNo">167</span> }<a name="line.167"></a>
<span class="sourceLineNo">168</span> }<a name="line.168"></a>
<span class="sourceLineNo">169</span> initializeKVHeap(scanners, joinedScanners, region);<a name="line.169"></a>
<span class="sourceLineNo">170</span> } catch (Throwable t) {<a name="line.170"></a>
<span class="sourceLineNo">171</span> throw handleException(instantiatedScanners, t);<a name="line.171"></a>
<span class="sourceLineNo">172</span> }<a name="line.172"></a>
<span class="sourceLineNo">173</span> }<a name="line.173"></a>
<span class="sourceLineNo">174</span><a name="line.174"></a>
<span class="sourceLineNo">175</span> protected void initializeKVHeap(List&lt;KeyValueScanner&gt; scanners,<a name="line.175"></a>
<span class="sourceLineNo">176</span> List&lt;KeyValueScanner&gt; joinedScanners, HRegion region) throws IOException {<a name="line.176"></a>
<span class="sourceLineNo">177</span> this.storeHeap = new KeyValueHeap(scanners, comparator);<a name="line.177"></a>
<span class="sourceLineNo">178</span> if (!joinedScanners.isEmpty()) {<a name="line.178"></a>
<span class="sourceLineNo">179</span> this.joinedHeap = new KeyValueHeap(joinedScanners, comparator);<a name="line.179"></a>
<span class="sourceLineNo">180</span> }<a name="line.180"></a>
<span class="sourceLineNo">181</span> }<a name="line.181"></a>
<span class="sourceLineNo">182</span><a name="line.182"></a>
<span class="sourceLineNo">183</span> private IOException handleException(List&lt;KeyValueScanner&gt; instantiatedScanners, Throwable t) {<a name="line.183"></a>
<span class="sourceLineNo">184</span> // remove scaner read point before throw the exception<a name="line.184"></a>
<span class="sourceLineNo">185</span> scannerReadPoints.remove(this);<a name="line.185"></a>
<span class="sourceLineNo">186</span> if (storeHeap != null) {<a name="line.186"></a>
<span class="sourceLineNo">187</span> storeHeap.close();<a name="line.187"></a>
<span class="sourceLineNo">188</span> storeHeap = null;<a name="line.188"></a>
<span class="sourceLineNo">189</span> if (joinedHeap != null) {<a name="line.189"></a>
<span class="sourceLineNo">190</span> joinedHeap.close();<a name="line.190"></a>
<span class="sourceLineNo">191</span> joinedHeap = null;<a name="line.191"></a>
<span class="sourceLineNo">192</span> }<a name="line.192"></a>
<span class="sourceLineNo">193</span> } else {<a name="line.193"></a>
<span class="sourceLineNo">194</span> // close all already instantiated scanners before throwing the exception<a name="line.194"></a>
<span class="sourceLineNo">195</span> for (KeyValueScanner scanner : instantiatedScanners) {<a name="line.195"></a>
<span class="sourceLineNo">196</span> scanner.close();<a name="line.196"></a>
<span class="sourceLineNo">197</span> }<a name="line.197"></a>
<span class="sourceLineNo">198</span> }<a name="line.198"></a>
<span class="sourceLineNo">199</span> return t instanceof IOException ? (IOException) t : new IOException(t);<a name="line.199"></a>
<span class="sourceLineNo">200</span> }<a name="line.200"></a>
<span class="sourceLineNo">201</span><a name="line.201"></a>
<span class="sourceLineNo">202</span> @Override<a name="line.202"></a>
<span class="sourceLineNo">203</span> public long getMaxResultSize() {<a name="line.203"></a>
<span class="sourceLineNo">204</span> return maxResultSize;<a name="line.204"></a>
<span class="sourceLineNo">205</span> }<a name="line.205"></a>
<span class="sourceLineNo">206</span><a name="line.206"></a>
<span class="sourceLineNo">207</span> @Override<a name="line.207"></a>
<span class="sourceLineNo">208</span> public long getMvccReadPoint() {<a name="line.208"></a>
<span class="sourceLineNo">209</span> return this.readPt;<a name="line.209"></a>
<span class="sourceLineNo">210</span> }<a name="line.210"></a>
<span class="sourceLineNo">211</span><a name="line.211"></a>
<span class="sourceLineNo">212</span> @Override<a name="line.212"></a>
<span class="sourceLineNo">213</span> public int getBatch() {<a name="line.213"></a>
<span class="sourceLineNo">214</span> return this.defaultScannerContext.getBatchLimit();<a name="line.214"></a>
<span class="sourceLineNo">215</span> }<a name="line.215"></a>
<span class="sourceLineNo">216</span><a name="line.216"></a>
<span class="sourceLineNo">217</span> /**<a name="line.217"></a>
<span class="sourceLineNo">218</span> * Reset both the filter and the old filter.<a name="line.218"></a>
<span class="sourceLineNo">219</span> * @throws IOException in case a filter raises an I/O exception.<a name="line.219"></a>
<span class="sourceLineNo">220</span> */<a name="line.220"></a>
<span class="sourceLineNo">221</span> protected final void resetFilters() throws IOException {<a name="line.221"></a>
<span class="sourceLineNo">222</span> if (filter != null) {<a name="line.222"></a>
<span class="sourceLineNo">223</span> filter.reset();<a name="line.223"></a>
<span class="sourceLineNo">224</span> }<a name="line.224"></a>
<span class="sourceLineNo">225</span> }<a name="line.225"></a>
<span class="sourceLineNo">226</span><a name="line.226"></a>
<span class="sourceLineNo">227</span> @Override<a name="line.227"></a>
<span class="sourceLineNo">228</span> public boolean next(List&lt;Cell&gt; outResults) throws IOException {<a name="line.228"></a>
<span class="sourceLineNo">229</span> // apply the batching limit by default<a name="line.229"></a>
<span class="sourceLineNo">230</span> return next(outResults, defaultScannerContext);<a name="line.230"></a>
<span class="sourceLineNo">231</span> }<a name="line.231"></a>
<span class="sourceLineNo">232</span><a name="line.232"></a>
<span class="sourceLineNo">233</span> @Override<a name="line.233"></a>
<span class="sourceLineNo">234</span> public synchronized boolean next(List&lt;Cell&gt; outResults, ScannerContext scannerContext)<a name="line.234"></a>
<span class="sourceLineNo">235</span> throws IOException {<a name="line.235"></a>
<span class="sourceLineNo">236</span> if (this.filterClosed) {<a name="line.236"></a>
<span class="sourceLineNo">237</span> throw new UnknownScannerException("Scanner was closed (timed out?) " +<a name="line.237"></a>
<span class="sourceLineNo">238</span> "after we renewed it. Could be caused by a very slow scanner " +<a name="line.238"></a>
<span class="sourceLineNo">239</span> "or a lengthy garbage collection");<a name="line.239"></a>
<span class="sourceLineNo">240</span> }<a name="line.240"></a>
<span class="sourceLineNo">241</span> region.startRegionOperation(Operation.SCAN);<a name="line.241"></a>
<span class="sourceLineNo">242</span> try {<a name="line.242"></a>
<span class="sourceLineNo">243</span> return nextRaw(outResults, scannerContext);<a name="line.243"></a>
<span class="sourceLineNo">244</span> } finally {<a name="line.244"></a>
<span class="sourceLineNo">245</span> region.closeRegionOperation(Operation.SCAN);<a name="line.245"></a>
<span class="sourceLineNo">246</span> }<a name="line.246"></a>
<span class="sourceLineNo">247</span> }<a name="line.247"></a>
<span class="sourceLineNo">248</span><a name="line.248"></a>
<span class="sourceLineNo">249</span> @Override<a name="line.249"></a>
<span class="sourceLineNo">250</span> public boolean nextRaw(List&lt;Cell&gt; outResults) throws IOException {<a name="line.250"></a>
<span class="sourceLineNo">251</span> // Use the RegionScanner's context by default<a name="line.251"></a>
<span class="sourceLineNo">252</span> return nextRaw(outResults, defaultScannerContext);<a name="line.252"></a>
<span class="sourceLineNo">253</span> }<a name="line.253"></a>
<span class="sourceLineNo">254</span><a name="line.254"></a>
<span class="sourceLineNo">255</span> @Override<a name="line.255"></a>
<span class="sourceLineNo">256</span> public boolean nextRaw(List&lt;Cell&gt; outResults, ScannerContext scannerContext) throws IOException {<a name="line.256"></a>
<span class="sourceLineNo">257</span> if (storeHeap == null) {<a name="line.257"></a>
<span class="sourceLineNo">258</span> // scanner is closed<a name="line.258"></a>
<span class="sourceLineNo">259</span> throw new UnknownScannerException("Scanner was closed");<a name="line.259"></a>
<span class="sourceLineNo">260</span> }<a name="line.260"></a>
<span class="sourceLineNo">261</span> boolean moreValues = false;<a name="line.261"></a>
<span class="sourceLineNo">262</span> if (outResults.isEmpty()) {<a name="line.262"></a>
<span class="sourceLineNo">263</span> // Usually outResults is empty. This is true when next is called<a name="line.263"></a>
<span class="sourceLineNo">264</span> // to handle scan or get operation.<a name="line.264"></a>
<span class="sourceLineNo">265</span> moreValues = nextInternal(outResults, scannerContext);<a name="line.265"></a>
<span class="sourceLineNo">266</span> } else {<a name="line.266"></a>
<span class="sourceLineNo">267</span> List&lt;Cell&gt; tmpList = new ArrayList&lt;&gt;();<a name="line.267"></a>
<span class="sourceLineNo">268</span> moreValues = nextInternal(tmpList, scannerContext);<a name="line.268"></a>
<span class="sourceLineNo">269</span> outResults.addAll(tmpList);<a name="line.269"></a>
<span class="sourceLineNo">270</span> }<a name="line.270"></a>
<span class="sourceLineNo">271</span><a name="line.271"></a>
<span class="sourceLineNo">272</span> if (!outResults.isEmpty()) {<a name="line.272"></a>
<span class="sourceLineNo">273</span> region.addReadRequestsCount(1);<a name="line.273"></a>
<span class="sourceLineNo">274</span> if (region.getMetrics() != null) {<a name="line.274"></a>
<span class="sourceLineNo">275</span> region.getMetrics().updateReadRequestCount();<a name="line.275"></a>
<span class="sourceLineNo">276</span> }<a name="line.276"></a>
<span class="sourceLineNo">277</span> }<a name="line.277"></a>
<span class="sourceLineNo">278</span> if (rsServices != null &amp;&amp; rsServices.getMetrics() != null) {<a name="line.278"></a>
<span class="sourceLineNo">279</span> rsServices.getMetrics().updateReadQueryMeter(getRegionInfo().getTable());<a name="line.279"></a>
<span class="sourceLineNo">280</span> }<a name="line.280"></a>
<span class="sourceLineNo">281</span><a name="line.281"></a>
<span class="sourceLineNo">282</span> // If the size limit was reached it means a partial Result is being returned. Returning a<a name="line.282"></a>
<span class="sourceLineNo">283</span> // partial Result means that we should not reset the filters; filters should only be reset in<a name="line.283"></a>
<span class="sourceLineNo">284</span> // between rows<a name="line.284"></a>
<span class="sourceLineNo">285</span> if (!scannerContext.mayHaveMoreCellsInRow()) {<a name="line.285"></a>
<span class="sourceLineNo">286</span> resetFilters();<a name="line.286"></a>
<span class="sourceLineNo">287</span> }<a name="line.287"></a>
<span class="sourceLineNo">288</span><a name="line.288"></a>
<span class="sourceLineNo">289</span> if (isFilterDoneInternal()) {<a name="line.289"></a>
<span class="sourceLineNo">290</span> moreValues = false;<a name="line.290"></a>
<span class="sourceLineNo">291</span> }<a name="line.291"></a>
<span class="sourceLineNo">292</span> return moreValues;<a name="line.292"></a>
<span class="sourceLineNo">293</span> }<a name="line.293"></a>
<span class="sourceLineNo">294</span><a name="line.294"></a>
<span class="sourceLineNo">295</span> /**<a name="line.295"></a>
<span class="sourceLineNo">296</span> * @return true if more cells exist after this batch, false if scanner is done<a name="line.296"></a>
<span class="sourceLineNo">297</span> */<a name="line.297"></a>
<span class="sourceLineNo">298</span> private boolean populateFromJoinedHeap(List&lt;Cell&gt; results, ScannerContext scannerContext)<a name="line.298"></a>
<span class="sourceLineNo">299</span> throws IOException {<a name="line.299"></a>
<span class="sourceLineNo">300</span> assert joinedContinuationRow != null;<a name="line.300"></a>
<span class="sourceLineNo">301</span> boolean moreValues =<a name="line.301"></a>
<span class="sourceLineNo">302</span> populateResult(results, this.joinedHeap, scannerContext, joinedContinuationRow);<a name="line.302"></a>
<span class="sourceLineNo">303</span><a name="line.303"></a>
<span class="sourceLineNo">304</span> if (!scannerContext.checkAnyLimitReached(LimitScope.BETWEEN_CELLS)) {<a name="line.304"></a>
<span class="sourceLineNo">305</span> // We are done with this row, reset the continuation.<a name="line.305"></a>
<span class="sourceLineNo">306</span> joinedContinuationRow = null;<a name="line.306"></a>
<span class="sourceLineNo">307</span> }<a name="line.307"></a>
<span class="sourceLineNo">308</span> // As the data is obtained from two independent heaps, we need to<a name="line.308"></a>
<span class="sourceLineNo">309</span> // ensure that result list is sorted, because Result relies on that.<a name="line.309"></a>
<span class="sourceLineNo">310</span> results.sort(comparator);<a name="line.310"></a>
<span class="sourceLineNo">311</span> return moreValues;<a name="line.311"></a>
<span class="sourceLineNo">312</span> }<a name="line.312"></a>
<span class="sourceLineNo">313</span><a name="line.313"></a>
<span class="sourceLineNo">314</span> /**<a name="line.314"></a>
<span class="sourceLineNo">315</span> * Fetches records with currentRow into results list, until next row, batchLimit (if not -1) is<a name="line.315"></a>
<span class="sourceLineNo">316</span> * reached, or remainingResultSize (if not -1) is reaced<a name="line.316"></a>
<span class="sourceLineNo">317</span> * @param heap KeyValueHeap to fetch data from.It must be positioned on correct row before call.<a name="line.317"></a>
<span class="sourceLineNo">318</span> * @return state of last call to {@link KeyValueHeap#next()}<a name="line.318"></a>
<span class="sourceLineNo">319</span> */<a name="line.319"></a>
<span class="sourceLineNo">320</span> private boolean populateResult(List&lt;Cell&gt; results, KeyValueHeap heap,<a name="line.320"></a>
<span class="sourceLineNo">321</span> ScannerContext scannerContext, Cell currentRowCell) throws IOException {<a name="line.321"></a>
<span class="sourceLineNo">322</span> Cell nextKv;<a name="line.322"></a>
<span class="sourceLineNo">323</span> boolean moreCellsInRow = false;<a name="line.323"></a>
<span class="sourceLineNo">324</span> boolean tmpKeepProgress = scannerContext.getKeepProgress();<a name="line.324"></a>
<span class="sourceLineNo">325</span> // Scanning between column families and thus the scope is between cells<a name="line.325"></a>
<span class="sourceLineNo">326</span> LimitScope limitScope = LimitScope.BETWEEN_CELLS;<a name="line.326"></a>
<span class="sourceLineNo">327</span> do {<a name="line.327"></a>
<span class="sourceLineNo">328</span> // Check for thread interrupt status in case we have been signaled from<a name="line.328"></a>
<span class="sourceLineNo">329</span> // #interruptRegionOperation.<a name="line.329"></a>
<span class="sourceLineNo">330</span> region.checkInterrupt();<a name="line.330"></a>
<span class="sourceLineNo">331</span><a name="line.331"></a>
<span class="sourceLineNo">332</span> // We want to maintain any progress that is made towards the limits while scanning across<a name="line.332"></a>
<span class="sourceLineNo">333</span> // different column families. To do this, we toggle the keep progress flag on during calls<a name="line.333"></a>
<span class="sourceLineNo">334</span> // to the StoreScanner to ensure that any progress made thus far is not wiped away.<a name="line.334"></a>
<span class="sourceLineNo">335</span> scannerContext.setKeepProgress(true);<a name="line.335"></a>
<span class="sourceLineNo">336</span> heap.next(results, scannerContext);<a name="line.336"></a>
<span class="sourceLineNo">337</span> scannerContext.setKeepProgress(tmpKeepProgress);<a name="line.337"></a>
<span class="sourceLineNo">338</span><a name="line.338"></a>
<span class="sourceLineNo">339</span> nextKv = heap.peek();<a name="line.339"></a>
<span class="sourceLineNo">340</span> moreCellsInRow = moreCellsInRow(nextKv, currentRowCell);<a name="line.340"></a>
<span class="sourceLineNo">341</span> if (!moreCellsInRow) {<a name="line.341"></a>
<span class="sourceLineNo">342</span> incrementCountOfRowsScannedMetric(scannerContext);<a name="line.342"></a>
<span class="sourceLineNo">343</span> }<a name="line.343"></a>
<span class="sourceLineNo">344</span> if (moreCellsInRow &amp;&amp; scannerContext.checkBatchLimit(limitScope)) {<a name="line.344"></a>
<span class="sourceLineNo">345</span> return scannerContext.setScannerState(NextState.BATCH_LIMIT_REACHED).hasMoreValues();<a name="line.345"></a>
<span class="sourceLineNo">346</span> } else if (scannerContext.checkSizeLimit(limitScope)) {<a name="line.346"></a>
<span class="sourceLineNo">347</span> ScannerContext.NextState state =<a name="line.347"></a>
<span class="sourceLineNo">348</span> moreCellsInRow ? NextState.SIZE_LIMIT_REACHED_MID_ROW : NextState.SIZE_LIMIT_REACHED;<a name="line.348"></a>
<span class="sourceLineNo">349</span> return scannerContext.setScannerState(state).hasMoreValues();<a name="line.349"></a>
<span class="sourceLineNo">350</span> } else if (scannerContext.checkTimeLimit(limitScope)) {<a name="line.350"></a>
<span class="sourceLineNo">351</span> ScannerContext.NextState state =<a name="line.351"></a>
<span class="sourceLineNo">352</span> moreCellsInRow ? NextState.TIME_LIMIT_REACHED_MID_ROW : NextState.TIME_LIMIT_REACHED;<a name="line.352"></a>
<span class="sourceLineNo">353</span> return scannerContext.setScannerState(state).hasMoreValues();<a name="line.353"></a>
<span class="sourceLineNo">354</span> }<a name="line.354"></a>
<span class="sourceLineNo">355</span> } while (moreCellsInRow);<a name="line.355"></a>
<span class="sourceLineNo">356</span> return nextKv != null;<a name="line.356"></a>
<span class="sourceLineNo">357</span> }<a name="line.357"></a>
<span class="sourceLineNo">358</span><a name="line.358"></a>
<span class="sourceLineNo">359</span> /**<a name="line.359"></a>
<span class="sourceLineNo">360</span> * Based on the nextKv in the heap, and the current row, decide whether or not there are more<a name="line.360"></a>
<span class="sourceLineNo">361</span> * cells to be read in the heap. If the row of the nextKv in the heap matches the current row then<a name="line.361"></a>
<span class="sourceLineNo">362</span> * there are more cells to be read in the row.<a name="line.362"></a>
<span class="sourceLineNo">363</span> * @return true When there are more cells in the row to be read<a name="line.363"></a>
<span class="sourceLineNo">364</span> */<a name="line.364"></a>
<span class="sourceLineNo">365</span> private boolean moreCellsInRow(final Cell nextKv, Cell currentRowCell) {<a name="line.365"></a>
<span class="sourceLineNo">366</span> return nextKv != null &amp;&amp; CellUtil.matchingRows(nextKv, currentRowCell);<a name="line.366"></a>
<span class="sourceLineNo">367</span> }<a name="line.367"></a>
<span class="sourceLineNo">368</span><a name="line.368"></a>
<span class="sourceLineNo">369</span> /**<a name="line.369"></a>
<span class="sourceLineNo">370</span> * @return True if a filter rules the scanner is over, done.<a name="line.370"></a>
<span class="sourceLineNo">371</span> */<a name="line.371"></a>
<span class="sourceLineNo">372</span> @Override<a name="line.372"></a>
<span class="sourceLineNo">373</span> public synchronized boolean isFilterDone() throws IOException {<a name="line.373"></a>
<span class="sourceLineNo">374</span> return isFilterDoneInternal();<a name="line.374"></a>
<span class="sourceLineNo">375</span> }<a name="line.375"></a>
<span class="sourceLineNo">376</span><a name="line.376"></a>
<span class="sourceLineNo">377</span> private boolean isFilterDoneInternal() throws IOException {<a name="line.377"></a>
<span class="sourceLineNo">378</span> return this.filter != null &amp;&amp; this.filter.filterAllRemaining();<a name="line.378"></a>
<span class="sourceLineNo">379</span> }<a name="line.379"></a>
<span class="sourceLineNo">380</span><a name="line.380"></a>
<span class="sourceLineNo">381</span> private void checkClientDisconnect(Optional&lt;RpcCall&gt; rpcCall) throws CallerDisconnectedException {<a name="line.381"></a>
<span class="sourceLineNo">382</span> if (rpcCall.isPresent()) {<a name="line.382"></a>
<span class="sourceLineNo">383</span> // If a user specifies a too-restrictive or too-slow scanner, the<a name="line.383"></a>
<span class="sourceLineNo">384</span> // client might time out and disconnect while the server side<a name="line.384"></a>
<span class="sourceLineNo">385</span> // is still processing the request. We should abort aggressively<a name="line.385"></a>
<span class="sourceLineNo">386</span> // in that case.<a name="line.386"></a>
<span class="sourceLineNo">387</span> long afterTime = rpcCall.get().disconnectSince();<a name="line.387"></a>
<span class="sourceLineNo">388</span> if (afterTime &gt;= 0) {<a name="line.388"></a>
<span class="sourceLineNo">389</span> throw new CallerDisconnectedException(<a name="line.389"></a>
<span class="sourceLineNo">390</span> "Aborting on region " + getRegionInfo().getRegionNameAsString() + ", call " + this +<a name="line.390"></a>
<span class="sourceLineNo">391</span> " after " + afterTime + " ms, since " + "caller disconnected");<a name="line.391"></a>
<span class="sourceLineNo">392</span> }<a name="line.392"></a>
<span class="sourceLineNo">393</span> }<a name="line.393"></a>
<span class="sourceLineNo">394</span> }<a name="line.394"></a>
<span class="sourceLineNo">395</span><a name="line.395"></a>
<span class="sourceLineNo">396</span> private void resetProgress(ScannerContext scannerContext, int initialBatchProgress,<a name="line.396"></a>
<span class="sourceLineNo">397</span> long initialSizeProgress, long initialHeapSizeProgress) {<a name="line.397"></a>
<span class="sourceLineNo">398</span> // Starting to scan a new row. Reset the scanner progress according to whether or not<a name="line.398"></a>
<span class="sourceLineNo">399</span> // progress should be kept.<a name="line.399"></a>
<span class="sourceLineNo">400</span> if (scannerContext.getKeepProgress()) {<a name="line.400"></a>
<span class="sourceLineNo">401</span> // Progress should be kept. Reset to initial values seen at start of method invocation.<a name="line.401"></a>
<span class="sourceLineNo">402</span> scannerContext.setProgress(initialBatchProgress, initialSizeProgress,<a name="line.402"></a>
<span class="sourceLineNo">403</span> initialHeapSizeProgress);<a name="line.403"></a>
<span class="sourceLineNo">404</span> } else {<a name="line.404"></a>
<span class="sourceLineNo">405</span> scannerContext.clearProgress();<a name="line.405"></a>
<span class="sourceLineNo">406</span> }<a name="line.406"></a>
<span class="sourceLineNo">407</span> }<a name="line.407"></a>
<span class="sourceLineNo">408</span><a name="line.408"></a>
<span class="sourceLineNo">409</span> private boolean nextInternal(List&lt;Cell&gt; results, ScannerContext scannerContext)<a name="line.409"></a>
<span class="sourceLineNo">410</span> throws IOException {<a name="line.410"></a>
<span class="sourceLineNo">411</span> Preconditions.checkArgument(results.isEmpty(), "First parameter should be an empty list");<a name="line.411"></a>
<span class="sourceLineNo">412</span> Preconditions.checkArgument(scannerContext != null, "Scanner context cannot be null");<a name="line.412"></a>
<span class="sourceLineNo">413</span> Optional&lt;RpcCall&gt; rpcCall = RpcServer.getCurrentCall();<a name="line.413"></a>
<span class="sourceLineNo">414</span><a name="line.414"></a>
<span class="sourceLineNo">415</span> // Save the initial progress from the Scanner context in these local variables. The progress<a name="line.415"></a>
<span class="sourceLineNo">416</span> // may need to be reset a few times if rows are being filtered out so we save the initial<a name="line.416"></a>
<span class="sourceLineNo">417</span> // progress.<a name="line.417"></a>
<span class="sourceLineNo">418</span> int initialBatchProgress = scannerContext.getBatchProgress();<a name="line.418"></a>
<span class="sourceLineNo">419</span> long initialSizeProgress = scannerContext.getDataSizeProgress();<a name="line.419"></a>
<span class="sourceLineNo">420</span> long initialHeapSizeProgress = scannerContext.getHeapSizeProgress();<a name="line.420"></a>
<span class="sourceLineNo">421</span><a name="line.421"></a>
<span class="sourceLineNo">422</span> // Used to check time limit<a name="line.422"></a>
<span class="sourceLineNo">423</span> LimitScope limitScope = LimitScope.BETWEEN_CELLS;<a name="line.423"></a>
<span class="sourceLineNo">424</span><a name="line.424"></a>
<span class="sourceLineNo">425</span> // The loop here is used only when at some point during the next we determine<a name="line.425"></a>
<span class="sourceLineNo">426</span> // that due to effects of filters or otherwise, we have an empty row in the result.<a name="line.426"></a>
<span class="sourceLineNo">427</span> // Then we loop and try again. Otherwise, we must get out on the first iteration via return,<a name="line.427"></a>
<span class="sourceLineNo">428</span> // "true" if there's more data to read, "false" if there isn't (storeHeap is at a stop row,<a name="line.428"></a>
<span class="sourceLineNo">429</span> // and joinedHeap has no more data to read for the last row (if set, joinedContinuationRow).<a name="line.429"></a>
<span class="sourceLineNo">430</span> while (true) {<a name="line.430"></a>
<span class="sourceLineNo">431</span> resetProgress(scannerContext, initialBatchProgress, initialSizeProgress,<a name="line.431"></a>
<span class="sourceLineNo">432</span> initialHeapSizeProgress);<a name="line.432"></a>
<span class="sourceLineNo">433</span> checkClientDisconnect(rpcCall);<a name="line.433"></a>
<span class="sourceLineNo">434</span><a name="line.434"></a>
<span class="sourceLineNo">435</span> // Check for thread interrupt status in case we have been signaled from<a name="line.435"></a>
<span class="sourceLineNo">436</span> // #interruptRegionOperation.<a name="line.436"></a>
<span class="sourceLineNo">437</span> region.checkInterrupt();<a name="line.437"></a>
<span class="sourceLineNo">438</span><a name="line.438"></a>
<span class="sourceLineNo">439</span> // Let's see what we have in the storeHeap.<a name="line.439"></a>
<span class="sourceLineNo">440</span> Cell current = this.storeHeap.peek();<a name="line.440"></a>
<span class="sourceLineNo">441</span><a name="line.441"></a>
<span class="sourceLineNo">442</span> boolean shouldStop = shouldStop(current);<a name="line.442"></a>
<span class="sourceLineNo">443</span> // When has filter row is true it means that the all the cells for a particular row must be<a name="line.443"></a>
<span class="sourceLineNo">444</span> // read before a filtering decision can be made. This means that filters where hasFilterRow<a name="line.444"></a>
<span class="sourceLineNo">445</span> // run the risk of enLongAddering out of memory errors in the case that they are applied to a<a name="line.445"></a>
<span class="sourceLineNo">446</span> // table that has very large rows.<a name="line.446"></a>
<span class="sourceLineNo">447</span> boolean hasFilterRow = this.filter != null &amp;&amp; this.filter.hasFilterRow();<a name="line.447"></a>
<span class="sourceLineNo">448</span><a name="line.448"></a>
<span class="sourceLineNo">449</span> // If filter#hasFilterRow is true, partial results are not allowed since allowing them<a name="line.449"></a>
<span class="sourceLineNo">450</span> // would prevent the filters from being evaluated. Thus, if it is true, change the<a name="line.450"></a>
<span class="sourceLineNo">451</span> // scope of any limits that could potentially create partial results to<a name="line.451"></a>
<span class="sourceLineNo">452</span> // LimitScope.BETWEEN_ROWS so that those limits are not reached mid-row<a name="line.452"></a>
<span class="sourceLineNo">453</span> if (hasFilterRow) {<a name="line.453"></a>
<span class="sourceLineNo">454</span> if (LOG.isTraceEnabled()) {<a name="line.454"></a>
<span class="sourceLineNo">455</span> LOG.trace("filter#hasFilterRow is true which prevents partial results from being " +<a name="line.455"></a>
<span class="sourceLineNo">456</span> " formed. Changing scope of limits that may create partials");<a name="line.456"></a>
<span class="sourceLineNo">457</span> }<a name="line.457"></a>
<span class="sourceLineNo">458</span> scannerContext.setSizeLimitScope(LimitScope.BETWEEN_ROWS);<a name="line.458"></a>
<span class="sourceLineNo">459</span> scannerContext.setTimeLimitScope(LimitScope.BETWEEN_ROWS);<a name="line.459"></a>
<span class="sourceLineNo">460</span> limitScope = LimitScope.BETWEEN_ROWS;<a name="line.460"></a>
<span class="sourceLineNo">461</span> }<a name="line.461"></a>
<span class="sourceLineNo">462</span><a name="line.462"></a>
<span class="sourceLineNo">463</span> if (scannerContext.checkTimeLimit(LimitScope.BETWEEN_CELLS)) {<a name="line.463"></a>
<span class="sourceLineNo">464</span> if (hasFilterRow) {<a name="line.464"></a>
<span class="sourceLineNo">465</span> throw new IncompatibleFilterException(<a name="line.465"></a>
<span class="sourceLineNo">466</span> "Filter whose hasFilterRow() returns true is incompatible with scans that must " +<a name="line.466"></a>
<span class="sourceLineNo">467</span> " stop mid-row because of a limit. ScannerContext:" + scannerContext);<a name="line.467"></a>
<span class="sourceLineNo">468</span> }<a name="line.468"></a>
<span class="sourceLineNo">469</span> return true;<a name="line.469"></a>
<span class="sourceLineNo">470</span> }<a name="line.470"></a>
<span class="sourceLineNo">471</span><a name="line.471"></a>
<span class="sourceLineNo">472</span> // Check if we were getting data from the joinedHeap and hit the limit.<a name="line.472"></a>
<span class="sourceLineNo">473</span> // If not, then it's main path - getting results from storeHeap.<a name="line.473"></a>
<span class="sourceLineNo">474</span> if (joinedContinuationRow == null) {<a name="line.474"></a>
<span class="sourceLineNo">475</span> // First, check if we are at a stop row. If so, there are no more results.<a name="line.475"></a>
<span class="sourceLineNo">476</span> if (shouldStop) {<a name="line.476"></a>
<span class="sourceLineNo">477</span> if (hasFilterRow) {<a name="line.477"></a>
<span class="sourceLineNo">478</span> filter.filterRowCells(results);<a name="line.478"></a>
<span class="sourceLineNo">479</span> }<a name="line.479"></a>
<span class="sourceLineNo">480</span> return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();<a name="line.480"></a>
<span class="sourceLineNo">481</span> }<a name="line.481"></a>
<span class="sourceLineNo">482</span><a name="line.482"></a>
<span class="sourceLineNo">483</span> // Check if rowkey filter wants to exclude this row. If so, loop to next.<a name="line.483"></a>
<span class="sourceLineNo">484</span> // Technically, if we hit limits before on this row, we don't need this call.<a name="line.484"></a>
<span class="sourceLineNo">485</span> if (filterRowKey(current)) {<a name="line.485"></a>
<span class="sourceLineNo">486</span> incrementCountOfRowsFilteredMetric(scannerContext);<a name="line.486"></a>
<span class="sourceLineNo">487</span> // early check, see HBASE-16296<a name="line.487"></a>
<span class="sourceLineNo">488</span> if (isFilterDoneInternal()) {<a name="line.488"></a>
<span class="sourceLineNo">489</span> return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();<a name="line.489"></a>
<span class="sourceLineNo">490</span> }<a name="line.490"></a>
<span class="sourceLineNo">491</span> // Typically the count of rows scanned is incremented inside #populateResult. However,<a name="line.491"></a>
<span class="sourceLineNo">492</span> // here we are filtering a row based purely on its row key, preventing us from calling<a name="line.492"></a>
<span class="sourceLineNo">493</span> // #populateResult. Thus, perform the necessary increment here to rows scanned metric<a name="line.493"></a>
<span class="sourceLineNo">494</span> incrementCountOfRowsScannedMetric(scannerContext);<a name="line.494"></a>
<span class="sourceLineNo">495</span> boolean moreRows = nextRow(scannerContext, current);<a name="line.495"></a>
<span class="sourceLineNo">496</span> if (!moreRows) {<a name="line.496"></a>
<span class="sourceLineNo">497</span> return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();<a name="line.497"></a>
<span class="sourceLineNo">498</span> }<a name="line.498"></a>
<span class="sourceLineNo">499</span> results.clear();<a name="line.499"></a>
<span class="sourceLineNo">500</span><a name="line.500"></a>
<span class="sourceLineNo">501</span> // Read nothing as the rowkey was filtered, but still need to check time limit<a name="line.501"></a>
<span class="sourceLineNo">502</span> if (scannerContext.checkTimeLimit(limitScope)) {<a name="line.502"></a>
<span class="sourceLineNo">503</span> return true;<a name="line.503"></a>
<span class="sourceLineNo">504</span> }<a name="line.504"></a>
<span class="sourceLineNo">505</span> continue;<a name="line.505"></a>
<span class="sourceLineNo">506</span> }<a name="line.506"></a>
<span class="sourceLineNo">507</span><a name="line.507"></a>
<span class="sourceLineNo">508</span> // Ok, we are good, let's try to get some results from the main heap.<a name="line.508"></a>
<span class="sourceLineNo">509</span> populateResult(results, this.storeHeap, scannerContext, current);<a name="line.509"></a>
<span class="sourceLineNo">510</span> if (scannerContext.checkAnyLimitReached(LimitScope.BETWEEN_CELLS)) {<a name="line.510"></a>
<span class="sourceLineNo">511</span> if (hasFilterRow) {<a name="line.511"></a>
<span class="sourceLineNo">512</span> throw new IncompatibleFilterException(<a name="line.512"></a>
<span class="sourceLineNo">513</span> "Filter whose hasFilterRow() returns true is incompatible with scans that must " +<a name="line.513"></a>
<span class="sourceLineNo">514</span> " stop mid-row because of a limit. ScannerContext:" + scannerContext);<a name="line.514"></a>
<span class="sourceLineNo">515</span> }<a name="line.515"></a>
<span class="sourceLineNo">516</span> return true;<a name="line.516"></a>
<span class="sourceLineNo">517</span> }<a name="line.517"></a>
<span class="sourceLineNo">518</span><a name="line.518"></a>
<span class="sourceLineNo">519</span> // Check for thread interrupt status in case we have been signaled from<a name="line.519"></a>
<span class="sourceLineNo">520</span> // #interruptRegionOperation.<a name="line.520"></a>
<span class="sourceLineNo">521</span> region.checkInterrupt();<a name="line.521"></a>
<span class="sourceLineNo">522</span><a name="line.522"></a>
<span class="sourceLineNo">523</span> Cell nextKv = this.storeHeap.peek();<a name="line.523"></a>
<span class="sourceLineNo">524</span> shouldStop = shouldStop(nextKv);<a name="line.524"></a>
<span class="sourceLineNo">525</span> // save that the row was empty before filters applied to it.<a name="line.525"></a>
<span class="sourceLineNo">526</span> final boolean isEmptyRow = results.isEmpty();<a name="line.526"></a>
<span class="sourceLineNo">527</span><a name="line.527"></a>
<span class="sourceLineNo">528</span> // We have the part of the row necessary for filtering (all of it, usually).<a name="line.528"></a>
<span class="sourceLineNo">529</span> // First filter with the filterRow(List).<a name="line.529"></a>
<span class="sourceLineNo">530</span> FilterWrapper.FilterRowRetCode ret = FilterWrapper.FilterRowRetCode.NOT_CALLED;<a name="line.530"></a>
<span class="sourceLineNo">531</span> if (hasFilterRow) {<a name="line.531"></a>
<span class="sourceLineNo">532</span> ret = filter.filterRowCellsWithRet(results);<a name="line.532"></a>
<span class="sourceLineNo">533</span><a name="line.533"></a>
<span class="sourceLineNo">534</span> // We don't know how the results have changed after being filtered. Must set progress<a name="line.534"></a>
<span class="sourceLineNo">535</span> // according to contents of results now.<a name="line.535"></a>
<span class="sourceLineNo">536</span> if (scannerContext.getKeepProgress()) {<a name="line.536"></a>
<span class="sourceLineNo">537</span> scannerContext.setProgress(initialBatchProgress, initialSizeProgress,<a name="line.537"></a>
<span class="sourceLineNo">538</span> initialHeapSizeProgress);<a name="line.538"></a>
<span class="sourceLineNo">539</span> } else {<a name="line.539"></a>
<span class="sourceLineNo">540</span> scannerContext.clearProgress();<a name="line.540"></a>
<span class="sourceLineNo">541</span> }<a name="line.541"></a>
<span class="sourceLineNo">542</span> scannerContext.incrementBatchProgress(results.size());<a name="line.542"></a>
<span class="sourceLineNo">543</span> for (Cell cell : results) {<a name="line.543"></a>
<span class="sourceLineNo">544</span> scannerContext.incrementSizeProgress(PrivateCellUtil.estimatedSerializedSizeOf(cell),<a name="line.544"></a>
<span class="sourceLineNo">545</span> cell.heapSize());<a name="line.545"></a>
<span class="sourceLineNo">546</span> }<a name="line.546"></a>
<span class="sourceLineNo">547</span> }<a name="line.547"></a>
<span class="sourceLineNo">548</span><a name="line.548"></a>
<span class="sourceLineNo">549</span> if (isEmptyRow || ret == FilterWrapper.FilterRowRetCode.EXCLUDE || filterRow()) {<a name="line.549"></a>
<span class="sourceLineNo">550</span> incrementCountOfRowsFilteredMetric(scannerContext);<a name="line.550"></a>
<span class="sourceLineNo">551</span> results.clear();<a name="line.551"></a>
<span class="sourceLineNo">552</span> boolean moreRows = nextRow(scannerContext, current);<a name="line.552"></a>
<span class="sourceLineNo">553</span> if (!moreRows) {<a name="line.553"></a>
<span class="sourceLineNo">554</span> return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();<a name="line.554"></a>
<span class="sourceLineNo">555</span> }<a name="line.555"></a>
<span class="sourceLineNo">556</span><a name="line.556"></a>
<span class="sourceLineNo">557</span> // This row was totally filtered out, if this is NOT the last row,<a name="line.557"></a>
<span class="sourceLineNo">558</span> // we should continue on. Otherwise, nothing else to do.<a name="line.558"></a>
<span class="sourceLineNo">559</span> if (!shouldStop) {<a name="line.559"></a>
<span class="sourceLineNo">560</span> // Read nothing as the cells was filtered, but still need to check time limit<a name="line.560"></a>
<span class="sourceLineNo">561</span> if (scannerContext.checkTimeLimit(limitScope)) {<a name="line.561"></a>
<span class="sourceLineNo">562</span> return true;<a name="line.562"></a>
<span class="sourceLineNo">563</span> }<a name="line.563"></a>
<span class="sourceLineNo">564</span> continue;<a name="line.564"></a>
<span class="sourceLineNo">565</span> }<a name="line.565"></a>
<span class="sourceLineNo">566</span> return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();<a name="line.566"></a>
<span class="sourceLineNo">567</span> }<a name="line.567"></a>
<span class="sourceLineNo">568</span><a name="line.568"></a>
<span class="sourceLineNo">569</span> // Ok, we are done with storeHeap for this row.<a name="line.569"></a>
<span class="sourceLineNo">570</span> // Now we may need to fetch additional, non-essential data into row.<a name="line.570"></a>
<span class="sourceLineNo">571</span> // These values are not needed for filter to work, so we postpone their<a name="line.571"></a>
<span class="sourceLineNo">572</span> // fetch to (possibly) reduce amount of data loads from disk.<a name="line.572"></a>
<span class="sourceLineNo">573</span> if (this.joinedHeap != null) {<a name="line.573"></a>
<span class="sourceLineNo">574</span> boolean mayHaveData = joinedHeapMayHaveData(current);<a name="line.574"></a>
<span class="sourceLineNo">575</span> if (mayHaveData) {<a name="line.575"></a>
<span class="sourceLineNo">576</span> joinedContinuationRow = current;<a name="line.576"></a>
<span class="sourceLineNo">577</span> populateFromJoinedHeap(results, scannerContext);<a name="line.577"></a>
<span class="sourceLineNo">578</span><a name="line.578"></a>
<span class="sourceLineNo">579</span> if (scannerContext.checkAnyLimitReached(LimitScope.BETWEEN_CELLS)) {<a name="line.579"></a>
<span class="sourceLineNo">580</span> return true;<a name="line.580"></a>
<span class="sourceLineNo">581</span> }<a name="line.581"></a>
<span class="sourceLineNo">582</span> }<a name="line.582"></a>
<span class="sourceLineNo">583</span> }<a name="line.583"></a>
<span class="sourceLineNo">584</span> } else {<a name="line.584"></a>
<span class="sourceLineNo">585</span> // Populating from the joined heap was stopped by limits, populate some more.<a name="line.585"></a>
<span class="sourceLineNo">586</span> populateFromJoinedHeap(results, scannerContext);<a name="line.586"></a>
<span class="sourceLineNo">587</span> if (scannerContext.checkAnyLimitReached(LimitScope.BETWEEN_CELLS)) {<a name="line.587"></a>
<span class="sourceLineNo">588</span> return true;<a name="line.588"></a>
<span class="sourceLineNo">589</span> }<a name="line.589"></a>
<span class="sourceLineNo">590</span> }<a name="line.590"></a>
<span class="sourceLineNo">591</span> // We may have just called populateFromJoinedMap and hit the limits. If that is<a name="line.591"></a>
<span class="sourceLineNo">592</span> // the case, we need to call it again on the next next() invocation.<a name="line.592"></a>
<span class="sourceLineNo">593</span> if (joinedContinuationRow != null) {<a name="line.593"></a>
<span class="sourceLineNo">594</span> return scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues();<a name="line.594"></a>
<span class="sourceLineNo">595</span> }<a name="line.595"></a>
<span class="sourceLineNo">596</span><a name="line.596"></a>
<span class="sourceLineNo">597</span> // Finally, we are done with both joinedHeap and storeHeap.<a name="line.597"></a>
<span class="sourceLineNo">598</span> // Double check to prevent empty rows from appearing in result. It could be<a name="line.598"></a>
<span class="sourceLineNo">599</span> // the case when SingleColumnValueExcludeFilter is used.<a name="line.599"></a>
<span class="sourceLineNo">600</span> if (results.isEmpty()) {<a name="line.600"></a>
<span class="sourceLineNo">601</span> incrementCountOfRowsFilteredMetric(scannerContext);<a name="line.601"></a>
<span class="sourceLineNo">602</span> boolean moreRows = nextRow(scannerContext, current);<a name="line.602"></a>
<span class="sourceLineNo">603</span> if (!moreRows) {<a name="line.603"></a>
<span class="sourceLineNo">604</span> return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();<a name="line.604"></a>
<span class="sourceLineNo">605</span> }<a name="line.605"></a>
<span class="sourceLineNo">606</span> if (!shouldStop) {<a name="line.606"></a>
<span class="sourceLineNo">607</span> continue;<a name="line.607"></a>
<span class="sourceLineNo">608</span> }<a name="line.608"></a>
<span class="sourceLineNo">609</span> }<a name="line.609"></a>
<span class="sourceLineNo">610</span><a name="line.610"></a>
<span class="sourceLineNo">611</span> if (shouldStop) {<a name="line.611"></a>
<span class="sourceLineNo">612</span> return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();<a name="line.612"></a>
<span class="sourceLineNo">613</span> } else {<a name="line.613"></a>
<span class="sourceLineNo">614</span> return scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues();<a name="line.614"></a>
<span class="sourceLineNo">615</span> }<a name="line.615"></a>
<span class="sourceLineNo">616</span> }<a name="line.616"></a>
<span class="sourceLineNo">617</span> }<a name="line.617"></a>
<span class="sourceLineNo">618</span><a name="line.618"></a>
<span class="sourceLineNo">619</span> private void incrementCountOfRowsFilteredMetric(ScannerContext scannerContext) {<a name="line.619"></a>
<span class="sourceLineNo">620</span> region.filteredReadRequestsCount.increment();<a name="line.620"></a>
<span class="sourceLineNo">621</span> if (region.getMetrics() != null) {<a name="line.621"></a>
<span class="sourceLineNo">622</span> region.getMetrics().updateFilteredRecords();<a name="line.622"></a>
<span class="sourceLineNo">623</span> }<a name="line.623"></a>
<span class="sourceLineNo">624</span><a name="line.624"></a>
<span class="sourceLineNo">625</span> if (scannerContext == null || !scannerContext.isTrackingMetrics()) {<a name="line.625"></a>
<span class="sourceLineNo">626</span> return;<a name="line.626"></a>
<span class="sourceLineNo">627</span> }<a name="line.627"></a>
<span class="sourceLineNo">628</span><a name="line.628"></a>
<span class="sourceLineNo">629</span> scannerContext.getMetrics().countOfRowsFiltered.incrementAndGet();<a name="line.629"></a>
<span class="sourceLineNo">630</span> }<a name="line.630"></a>
<span class="sourceLineNo">631</span><a name="line.631"></a>
<span class="sourceLineNo">632</span> private void incrementCountOfRowsScannedMetric(ScannerContext scannerContext) {<a name="line.632"></a>
<span class="sourceLineNo">633</span> if (scannerContext == null || !scannerContext.isTrackingMetrics()) {<a name="line.633"></a>
<span class="sourceLineNo">634</span> return;<a name="line.634"></a>
<span class="sourceLineNo">635</span> }<a name="line.635"></a>
<span class="sourceLineNo">636</span><a name="line.636"></a>
<span class="sourceLineNo">637</span> scannerContext.getMetrics().countOfRowsScanned.incrementAndGet();<a name="line.637"></a>
<span class="sourceLineNo">638</span> }<a name="line.638"></a>
<span class="sourceLineNo">639</span><a name="line.639"></a>
<span class="sourceLineNo">640</span> /**<a name="line.640"></a>
<span class="sourceLineNo">641</span> * @return true when the joined heap may have data for the current row<a name="line.641"></a>
<span class="sourceLineNo">642</span> */<a name="line.642"></a>
<span class="sourceLineNo">643</span> private boolean joinedHeapMayHaveData(Cell currentRowCell) throws IOException {<a name="line.643"></a>
<span class="sourceLineNo">644</span> Cell nextJoinedKv = joinedHeap.peek();<a name="line.644"></a>
<span class="sourceLineNo">645</span> boolean matchCurrentRow =<a name="line.645"></a>
<span class="sourceLineNo">646</span> nextJoinedKv != null &amp;&amp; CellUtil.matchingRows(nextJoinedKv, currentRowCell);<a name="line.646"></a>
<span class="sourceLineNo">647</span> boolean matchAfterSeek = false;<a name="line.647"></a>
<span class="sourceLineNo">648</span><a name="line.648"></a>
<span class="sourceLineNo">649</span> // If the next value in the joined heap does not match the current row, try to seek to the<a name="line.649"></a>
<span class="sourceLineNo">650</span> // correct row<a name="line.650"></a>
<span class="sourceLineNo">651</span> if (!matchCurrentRow) {<a name="line.651"></a>
<span class="sourceLineNo">652</span> Cell firstOnCurrentRow = PrivateCellUtil.createFirstOnRow(currentRowCell);<a name="line.652"></a>
<span class="sourceLineNo">653</span> boolean seekSuccessful = this.joinedHeap.requestSeek(firstOnCurrentRow, true, true);<a name="line.653"></a>
<span class="sourceLineNo">654</span> matchAfterSeek = seekSuccessful &amp;&amp; joinedHeap.peek() != null &amp;&amp;<a name="line.654"></a>
<span class="sourceLineNo">655</span> CellUtil.matchingRows(joinedHeap.peek(), currentRowCell);<a name="line.655"></a>
<span class="sourceLineNo">656</span> }<a name="line.656"></a>
<span class="sourceLineNo">657</span><a name="line.657"></a>
<span class="sourceLineNo">658</span> return matchCurrentRow || matchAfterSeek;<a name="line.658"></a>
<span class="sourceLineNo">659</span> }<a name="line.659"></a>
<span class="sourceLineNo">660</span><a name="line.660"></a>
<span class="sourceLineNo">661</span> /**<a name="line.661"></a>
<span class="sourceLineNo">662</span> * This function is to maintain backward compatibility for 0.94 filters. HBASE-6429 combines both<a name="line.662"></a>
<span class="sourceLineNo">663</span> * filterRow &amp; filterRow({@code List&lt;KeyValue&gt; kvs}) functions. While 0.94 code or older, it may<a name="line.663"></a>
<span class="sourceLineNo">664</span> * not implement hasFilterRow as HBase-6429 expects because 0.94 hasFilterRow() only returns true<a name="line.664"></a>
<span class="sourceLineNo">665</span> * when filterRow({@code List&lt;KeyValue&gt; kvs}) is overridden not the filterRow(). Therefore, the<a name="line.665"></a>
<span class="sourceLineNo">666</span> * filterRow() will be skipped.<a name="line.666"></a>
<span class="sourceLineNo">667</span> */<a name="line.667"></a>
<span class="sourceLineNo">668</span> private boolean filterRow() throws IOException {<a name="line.668"></a>
<span class="sourceLineNo">669</span> // when hasFilterRow returns true, filter.filterRow() will be called automatically inside<a name="line.669"></a>
<span class="sourceLineNo">670</span> // filterRowCells(List&lt;Cell&gt; kvs) so we skip that scenario here.<a name="line.670"></a>
<span class="sourceLineNo">671</span> return filter != null &amp;&amp; (!filter.hasFilterRow()) &amp;&amp; filter.filterRow();<a name="line.671"></a>
<span class="sourceLineNo">672</span> }<a name="line.672"></a>
<span class="sourceLineNo">673</span><a name="line.673"></a>
<span class="sourceLineNo">674</span> private boolean filterRowKey(Cell current) throws IOException {<a name="line.674"></a>
<span class="sourceLineNo">675</span> return filter != null &amp;&amp; filter.filterRowKey(current);<a name="line.675"></a>
<span class="sourceLineNo">676</span> }<a name="line.676"></a>
<span class="sourceLineNo">677</span><a name="line.677"></a>
<span class="sourceLineNo">678</span> /**<a name="line.678"></a>
<span class="sourceLineNo">679</span> * A mocked list implementation - discards all updates.<a name="line.679"></a>
<span class="sourceLineNo">680</span> */<a name="line.680"></a>
<span class="sourceLineNo">681</span> private static final List&lt;Cell&gt; MOCKED_LIST = new AbstractList&lt;Cell&gt;() {<a name="line.681"></a>
<span class="sourceLineNo">682</span><a name="line.682"></a>
<span class="sourceLineNo">683</span> @Override<a name="line.683"></a>
<span class="sourceLineNo">684</span> public void add(int index, Cell element) {<a name="line.684"></a>
<span class="sourceLineNo">685</span> // do nothing<a name="line.685"></a>
<span class="sourceLineNo">686</span> }<a name="line.686"></a>
<span class="sourceLineNo">687</span><a name="line.687"></a>
<span class="sourceLineNo">688</span> @Override<a name="line.688"></a>
<span class="sourceLineNo">689</span> public boolean addAll(int index, Collection&lt;? extends Cell&gt; c) {<a name="line.689"></a>
<span class="sourceLineNo">690</span> return false; // this list is never changed as a result of an update<a name="line.690"></a>
<span class="sourceLineNo">691</span> }<a name="line.691"></a>
<span class="sourceLineNo">692</span><a name="line.692"></a>
<span class="sourceLineNo">693</span> @Override<a name="line.693"></a>
<span class="sourceLineNo">694</span> public KeyValue get(int index) {<a name="line.694"></a>
<span class="sourceLineNo">695</span> throw new UnsupportedOperationException();<a name="line.695"></a>
<span class="sourceLineNo">696</span> }<a name="line.696"></a>
<span class="sourceLineNo">697</span><a name="line.697"></a>
<span class="sourceLineNo">698</span> @Override<a name="line.698"></a>
<span class="sourceLineNo">699</span> public int size() {<a name="line.699"></a>
<span class="sourceLineNo">700</span> return 0;<a name="line.700"></a>
<span class="sourceLineNo">701</span> }<a name="line.701"></a>
<span class="sourceLineNo">702</span> };<a name="line.702"></a>
<span class="sourceLineNo">703</span><a name="line.703"></a>
<span class="sourceLineNo">704</span> protected boolean nextRow(ScannerContext scannerContext, Cell curRowCell) throws IOException {<a name="line.704"></a>
<span class="sourceLineNo">705</span> assert this.joinedContinuationRow == null : "Trying to go to next row during joinedHeap read.";<a name="line.705"></a>
<span class="sourceLineNo">706</span> Cell next;<a name="line.706"></a>
<span class="sourceLineNo">707</span> while ((next = this.storeHeap.peek()) != null &amp;&amp; CellUtil.matchingRows(next, curRowCell)) {<a name="line.707"></a>
<span class="sourceLineNo">708</span> // Check for thread interrupt status in case we have been signaled from<a name="line.708"></a>
<span class="sourceLineNo">709</span> // #interruptRegionOperation.<a name="line.709"></a>
<span class="sourceLineNo">710</span> region.checkInterrupt();<a name="line.710"></a>
<span class="sourceLineNo">711</span> this.storeHeap.next(MOCKED_LIST);<a name="line.711"></a>
<span class="sourceLineNo">712</span> }<a name="line.712"></a>
<span class="sourceLineNo">713</span> resetFilters();<a name="line.713"></a>
<span class="sourceLineNo">714</span><a name="line.714"></a>
<span class="sourceLineNo">715</span> // Calling the hook in CP which allows it to do a fast forward<a name="line.715"></a>
<span class="sourceLineNo">716</span> return this.region.getCoprocessorHost() == null ||<a name="line.716"></a>
<span class="sourceLineNo">717</span> this.region.getCoprocessorHost().postScannerFilterRow(this, curRowCell);<a name="line.717"></a>
<span class="sourceLineNo">718</span> }<a name="line.718"></a>
<span class="sourceLineNo">719</span><a name="line.719"></a>
<span class="sourceLineNo">720</span> protected boolean shouldStop(Cell currentRowCell) {<a name="line.720"></a>
<span class="sourceLineNo">721</span> if (currentRowCell == null) {<a name="line.721"></a>
<span class="sourceLineNo">722</span> return true;<a name="line.722"></a>
<span class="sourceLineNo">723</span> }<a name="line.723"></a>
<span class="sourceLineNo">724</span> if (stopRow == null || Bytes.equals(stopRow, HConstants.EMPTY_END_ROW)) {<a name="line.724"></a>
<span class="sourceLineNo">725</span> return false;<a name="line.725"></a>
<span class="sourceLineNo">726</span> }<a name="line.726"></a>
<span class="sourceLineNo">727</span> int c = comparator.compareRows(currentRowCell, stopRow, 0, stopRow.length);<a name="line.727"></a>
<span class="sourceLineNo">728</span> return c &gt; 0 || (c == 0 &amp;&amp; !includeStopRow);<a name="line.728"></a>
<span class="sourceLineNo">729</span> }<a name="line.729"></a>
<span class="sourceLineNo">730</span><a name="line.730"></a>
<span class="sourceLineNo">731</span> @Override<a name="line.731"></a>
<span class="sourceLineNo">732</span> public synchronized void close() {<a name="line.732"></a>
<span class="sourceLineNo">733</span> if (storeHeap != null) {<a name="line.733"></a>
<span class="sourceLineNo">734</span> storeHeap.close();<a name="line.734"></a>
<span class="sourceLineNo">735</span> storeHeap = null;<a name="line.735"></a>
<span class="sourceLineNo">736</span> }<a name="line.736"></a>
<span class="sourceLineNo">737</span> if (joinedHeap != null) {<a name="line.737"></a>
<span class="sourceLineNo">738</span> joinedHeap.close();<a name="line.738"></a>
<span class="sourceLineNo">739</span> joinedHeap = null;<a name="line.739"></a>
<span class="sourceLineNo">740</span> }<a name="line.740"></a>
<span class="sourceLineNo">741</span> // no need to synchronize here.<a name="line.741"></a>
<span class="sourceLineNo">742</span> scannerReadPoints.remove(this);<a name="line.742"></a>
<span class="sourceLineNo">743</span> this.filterClosed = true;<a name="line.743"></a>
<span class="sourceLineNo">744</span> }<a name="line.744"></a>
<span class="sourceLineNo">745</span><a name="line.745"></a>
<span class="sourceLineNo">746</span> @Override<a name="line.746"></a>
<span class="sourceLineNo">747</span> public synchronized boolean reseek(byte[] row) throws IOException {<a name="line.747"></a>
<span class="sourceLineNo">748</span> if (row == null) {<a name="line.748"></a>
<span class="sourceLineNo">749</span> throw new IllegalArgumentException("Row cannot be null.");<a name="line.749"></a>
<span class="sourceLineNo">750</span> }<a name="line.750"></a>
<span class="sourceLineNo">751</span> boolean result = false;<a name="line.751"></a>
<span class="sourceLineNo">752</span> region.startRegionOperation();<a name="line.752"></a>
<span class="sourceLineNo">753</span> Cell kv = PrivateCellUtil.createFirstOnRow(row, 0, (short) row.length);<a name="line.753"></a>
<span class="sourceLineNo">754</span> try {<a name="line.754"></a>
<span class="sourceLineNo">755</span> // use request seek to make use of the lazy seek option. See HBASE-5520<a name="line.755"></a>
<span class="sourceLineNo">756</span> result = this.storeHeap.requestSeek(kv, true, true);<a name="line.756"></a>
<span class="sourceLineNo">757</span> if (this.joinedHeap != null) {<a name="line.757"></a>
<span class="sourceLineNo">758</span> result = this.joinedHeap.requestSeek(kv, true, true) || result;<a name="line.758"></a>
<span class="sourceLineNo">759</span> }<a name="line.759"></a>
<span class="sourceLineNo">760</span> } finally {<a name="line.760"></a>
<span class="sourceLineNo">761</span> region.closeRegionOperation();<a name="line.761"></a>
<span class="sourceLineNo">762</span> }<a name="line.762"></a>
<span class="sourceLineNo">763</span> return result;<a name="line.763"></a>
<span class="sourceLineNo">764</span> }<a name="line.764"></a>
<span class="sourceLineNo">765</span><a name="line.765"></a>
<span class="sourceLineNo">766</span> @Override<a name="line.766"></a>
<span class="sourceLineNo">767</span> public void shipped() throws IOException {<a name="line.767"></a>
<span class="sourceLineNo">768</span> if (storeHeap != null) {<a name="line.768"></a>
<span class="sourceLineNo">769</span> storeHeap.shipped();<a name="line.769"></a>
<span class="sourceLineNo">770</span> }<a name="line.770"></a>
<span class="sourceLineNo">771</span> if (joinedHeap != null) {<a name="line.771"></a>
<span class="sourceLineNo">772</span> joinedHeap.shipped();<a name="line.772"></a>
<span class="sourceLineNo">773</span> }<a name="line.773"></a>
<span class="sourceLineNo">774</span> }<a name="line.774"></a>
<span class="sourceLineNo">775</span><a name="line.775"></a>
<span class="sourceLineNo">776</span> @Override<a name="line.776"></a>
<span class="sourceLineNo">777</span> public void run() throws IOException {<a name="line.777"></a>
<span class="sourceLineNo">778</span> // This is the RPC callback method executed. We do the close in of the scanner in this<a name="line.778"></a>
<span class="sourceLineNo">779</span> // callback<a name="line.779"></a>
<span class="sourceLineNo">780</span> this.close();<a name="line.780"></a>
<span class="sourceLineNo">781</span> }<a name="line.781"></a>
<span class="sourceLineNo">782</span>}<a name="line.782"></a>
</pre>
</div>
</body>
</html>