blob: 63e6e48b3beea756a37b376ef9dde57f7af442ba [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.client, class: Scan">
<meta name="generator" content="javadoc/SourceToHTMLConverter">
<link rel="stylesheet" type="text/css" href="../../../../../../stylesheet.css" title="Style">
</head>
<body class="source-page">
<main role="main">
<div class="source-container">
<pre><span class="source-line-no">001</span><span id="line-1">/*</span>
<span class="source-line-no">002</span><span id="line-2"> * Licensed to the Apache Software Foundation (ASF) under one</span>
<span class="source-line-no">003</span><span id="line-3"> * or more contributor license agreements. See the NOTICE file</span>
<span class="source-line-no">004</span><span id="line-4"> * distributed with this work for additional information</span>
<span class="source-line-no">005</span><span id="line-5"> * regarding copyright ownership. The ASF licenses this file</span>
<span class="source-line-no">006</span><span id="line-6"> * to you under the Apache License, Version 2.0 (the</span>
<span class="source-line-no">007</span><span id="line-7"> * "License"); you may not use this file except in compliance</span>
<span class="source-line-no">008</span><span id="line-8"> * with the License. You may obtain a copy of the License at</span>
<span class="source-line-no">009</span><span id="line-9"> *</span>
<span class="source-line-no">010</span><span id="line-10"> * http://www.apache.org/licenses/LICENSE-2.0</span>
<span class="source-line-no">011</span><span id="line-11"> *</span>
<span class="source-line-no">012</span><span id="line-12"> * Unless required by applicable law or agreed to in writing, software</span>
<span class="source-line-no">013</span><span id="line-13"> * distributed under the License is distributed on an "AS IS" BASIS,</span>
<span class="source-line-no">014</span><span id="line-14"> * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.</span>
<span class="source-line-no">015</span><span id="line-15"> * See the License for the specific language governing permissions and</span>
<span class="source-line-no">016</span><span id="line-16"> * limitations under the License.</span>
<span class="source-line-no">017</span><span id="line-17"> */</span>
<span class="source-line-no">018</span><span id="line-18">package org.apache.hadoop.hbase.client;</span>
<span class="source-line-no">019</span><span id="line-19"></span>
<span class="source-line-no">020</span><span id="line-20">import java.io.IOException;</span>
<span class="source-line-no">021</span><span id="line-21">import java.util.ArrayList;</span>
<span class="source-line-no">022</span><span id="line-22">import java.util.HashMap;</span>
<span class="source-line-no">023</span><span id="line-23">import java.util.List;</span>
<span class="source-line-no">024</span><span id="line-24">import java.util.Map;</span>
<span class="source-line-no">025</span><span id="line-25">import java.util.NavigableSet;</span>
<span class="source-line-no">026</span><span id="line-26">import java.util.TreeMap;</span>
<span class="source-line-no">027</span><span id="line-27">import java.util.TreeSet;</span>
<span class="source-line-no">028</span><span id="line-28">import java.util.stream.Collectors;</span>
<span class="source-line-no">029</span><span id="line-29">import org.apache.hadoop.hbase.HConstants;</span>
<span class="source-line-no">030</span><span id="line-30">import org.apache.hadoop.hbase.client.metrics.ScanMetrics;</span>
<span class="source-line-no">031</span><span id="line-31">import org.apache.hadoop.hbase.filter.Filter;</span>
<span class="source-line-no">032</span><span id="line-32">import org.apache.hadoop.hbase.filter.IncompatibleFilterException;</span>
<span class="source-line-no">033</span><span id="line-33">import org.apache.hadoop.hbase.io.TimeRange;</span>
<span class="source-line-no">034</span><span id="line-34">import org.apache.hadoop.hbase.security.access.Permission;</span>
<span class="source-line-no">035</span><span id="line-35">import org.apache.hadoop.hbase.security.visibility.Authorizations;</span>
<span class="source-line-no">036</span><span id="line-36">import org.apache.hadoop.hbase.util.Bytes;</span>
<span class="source-line-no">037</span><span id="line-37">import org.apache.yetus.audience.InterfaceAudience;</span>
<span class="source-line-no">038</span><span id="line-38">import org.slf4j.Logger;</span>
<span class="source-line-no">039</span><span id="line-39">import org.slf4j.LoggerFactory;</span>
<span class="source-line-no">040</span><span id="line-40"></span>
<span class="source-line-no">041</span><span id="line-41">/**</span>
<span class="source-line-no">042</span><span id="line-42"> * Used to perform Scan operations.</span>
<span class="source-line-no">043</span><span id="line-43"> * &lt;p&gt;</span>
<span class="source-line-no">044</span><span id="line-44"> * All operations are identical to {@link Get} with the exception of instantiation. Rather than</span>
<span class="source-line-no">045</span><span id="line-45"> * specifying a single row, an optional startRow and stopRow may be defined. If rows are not</span>
<span class="source-line-no">046</span><span id="line-46"> * specified, the Scanner will iterate over all rows.</span>
<span class="source-line-no">047</span><span id="line-47"> * &lt;p&gt;</span>
<span class="source-line-no">048</span><span id="line-48"> * To get all columns from all rows of a Table, create an instance with no constraints; use the</span>
<span class="source-line-no">049</span><span id="line-49"> * {@link #Scan()} constructor. To constrain the scan to specific column families, call</span>
<span class="source-line-no">050</span><span id="line-50"> * {@link #addFamily(byte[]) addFamily} for each family to retrieve on your Scan instance.</span>
<span class="source-line-no">051</span><span id="line-51"> * &lt;p&gt;</span>
<span class="source-line-no">052</span><span id="line-52"> * To get specific columns, call {@link #addColumn(byte[], byte[]) addColumn} for each column to</span>
<span class="source-line-no">053</span><span id="line-53"> * retrieve.</span>
<span class="source-line-no">054</span><span id="line-54"> * &lt;p&gt;</span>
<span class="source-line-no">055</span><span id="line-55"> * To only retrieve columns within a specific range of version timestamps, call</span>
<span class="source-line-no">056</span><span id="line-56"> * {@link #setTimeRange(long, long) setTimeRange}.</span>
<span class="source-line-no">057</span><span id="line-57"> * &lt;p&gt;</span>
<span class="source-line-no">058</span><span id="line-58"> * To only retrieve columns with a specific timestamp, call {@link #setTimestamp(long) setTimestamp}</span>
<span class="source-line-no">059</span><span id="line-59"> * .</span>
<span class="source-line-no">060</span><span id="line-60"> * &lt;p&gt;</span>
<span class="source-line-no">061</span><span id="line-61"> * To limit the number of versions of each column to be returned, call {@link #readVersions(int)}.</span>
<span class="source-line-no">062</span><span id="line-62"> * &lt;p&gt;</span>
<span class="source-line-no">063</span><span id="line-63"> * To limit the maximum number of values returned for each call to next(), call</span>
<span class="source-line-no">064</span><span id="line-64"> * {@link #setBatch(int) setBatch}.</span>
<span class="source-line-no">065</span><span id="line-65"> * &lt;p&gt;</span>
<span class="source-line-no">066</span><span id="line-66"> * To add a filter, call {@link #setFilter(org.apache.hadoop.hbase.filter.Filter) setFilter}.</span>
<span class="source-line-no">067</span><span id="line-67"> * &lt;p&gt;</span>
<span class="source-line-no">068</span><span id="line-68"> * For small scan, it is deprecated in 2.0.0. Now we have a {@link #setLimit(int)} method in Scan</span>
<span class="source-line-no">069</span><span id="line-69"> * object which is used to tell RS how many rows we want. If the rows return reaches the limit, the</span>
<span class="source-line-no">070</span><span id="line-70"> * RS will close the RegionScanner automatically. And we will also fetch data when openScanner in</span>
<span class="source-line-no">071</span><span id="line-71"> * the new implementation, this means we can also finish a scan operation in one rpc call. And we</span>
<span class="source-line-no">072</span><span id="line-72"> * have also introduced a {@link #setReadType(ReadType)} method. You can use this method to tell RS</span>
<span class="source-line-no">073</span><span id="line-73"> * to use pread explicitly.</span>
<span class="source-line-no">074</span><span id="line-74"> * &lt;p&gt;</span>
<span class="source-line-no">075</span><span id="line-75"> * Expert: To explicitly disable server-side block caching for this scan, execute</span>
<span class="source-line-no">076</span><span id="line-76"> * {@link #setCacheBlocks(boolean)}.</span>
<span class="source-line-no">077</span><span id="line-77"> * &lt;p&gt;</span>
<span class="source-line-no">078</span><span id="line-78"> * &lt;em&gt;Note:&lt;/em&gt; Usage alters Scan instances. Internally, attributes are updated as the Scan runs</span>
<span class="source-line-no">079</span><span id="line-79"> * and if enabled, metrics accumulate in the Scan instance. Be aware this is the case when you go to</span>
<span class="source-line-no">080</span><span id="line-80"> * clone a Scan instance or if you go to reuse a created Scan instance; safer is create a Scan</span>
<span class="source-line-no">081</span><span id="line-81"> * instance per usage.</span>
<span class="source-line-no">082</span><span id="line-82"> */</span>
<span class="source-line-no">083</span><span id="line-83">@InterfaceAudience.Public</span>
<span class="source-line-no">084</span><span id="line-84">public class Scan extends Query {</span>
<span class="source-line-no">085</span><span id="line-85"> private static final Logger LOG = LoggerFactory.getLogger(Scan.class);</span>
<span class="source-line-no">086</span><span id="line-86"></span>
<span class="source-line-no">087</span><span id="line-87"> private static final String RAW_ATTR = "_raw_";</span>
<span class="source-line-no">088</span><span id="line-88"></span>
<span class="source-line-no">089</span><span id="line-89"> private byte[] startRow = HConstants.EMPTY_START_ROW;</span>
<span class="source-line-no">090</span><span id="line-90"> private boolean includeStartRow = true;</span>
<span class="source-line-no">091</span><span id="line-91"> private byte[] stopRow = HConstants.EMPTY_END_ROW;</span>
<span class="source-line-no">092</span><span id="line-92"> private boolean includeStopRow = false;</span>
<span class="source-line-no">093</span><span id="line-93"> private int maxVersions = 1;</span>
<span class="source-line-no">094</span><span id="line-94"> private int batch = -1;</span>
<span class="source-line-no">095</span><span id="line-95"></span>
<span class="source-line-no">096</span><span id="line-96"> /**</span>
<span class="source-line-no">097</span><span id="line-97"> * Partial {@link Result}s are {@link Result}s must be combined to form a complete {@link Result}.</span>
<span class="source-line-no">098</span><span id="line-98"> * The {@link Result}s had to be returned in fragments (i.e. as partials) because the size of the</span>
<span class="source-line-no">099</span><span id="line-99"> * cells in the row exceeded max result size on the server. Typically partial results will be</span>
<span class="source-line-no">100</span><span id="line-100"> * combined client side into complete results before being delivered to the caller. However, if</span>
<span class="source-line-no">101</span><span id="line-101"> * this flag is set, the caller is indicating that they do not mind seeing partial results (i.e.</span>
<span class="source-line-no">102</span><span id="line-102"> * they understand that the results returned from the Scanner may only represent part of a</span>
<span class="source-line-no">103</span><span id="line-103"> * particular row). In such a case, any attempt to combine the partials into a complete result on</span>
<span class="source-line-no">104</span><span id="line-104"> * the client side will be skipped, and the caller will be able to see the exact results returned</span>
<span class="source-line-no">105</span><span id="line-105"> * from the server.</span>
<span class="source-line-no">106</span><span id="line-106"> */</span>
<span class="source-line-no">107</span><span id="line-107"> private boolean allowPartialResults = false;</span>
<span class="source-line-no">108</span><span id="line-108"></span>
<span class="source-line-no">109</span><span id="line-109"> private int storeLimit = -1;</span>
<span class="source-line-no">110</span><span id="line-110"> private int storeOffset = 0;</span>
<span class="source-line-no">111</span><span id="line-111"></span>
<span class="source-line-no">112</span><span id="line-112"> private static final String SCAN_ATTRIBUTES_METRICS_ENABLE = "scan.attributes.metrics.enable";</span>
<span class="source-line-no">113</span><span id="line-113"></span>
<span class="source-line-no">114</span><span id="line-114"> // If an application wants to use multiple scans over different tables each scan must</span>
<span class="source-line-no">115</span><span id="line-115"> // define this attribute with the appropriate table name by calling</span>
<span class="source-line-no">116</span><span id="line-116"> // scan.setAttribute(Scan.SCAN_ATTRIBUTES_TABLE_NAME, Bytes.toBytes(tableName))</span>
<span class="source-line-no">117</span><span id="line-117"> static public final String SCAN_ATTRIBUTES_TABLE_NAME = "scan.attributes.table.name";</span>
<span class="source-line-no">118</span><span id="line-118"></span>
<span class="source-line-no">119</span><span id="line-119"> /**</span>
<span class="source-line-no">120</span><span id="line-120"> * -1 means no caching specified and the value of {@link HConstants#HBASE_CLIENT_SCANNER_CACHING}</span>
<span class="source-line-no">121</span><span id="line-121"> * (default to {@link HConstants#DEFAULT_HBASE_CLIENT_SCANNER_CACHING}) will be used</span>
<span class="source-line-no">122</span><span id="line-122"> */</span>
<span class="source-line-no">123</span><span id="line-123"> private int caching = -1;</span>
<span class="source-line-no">124</span><span id="line-124"> private long maxResultSize = -1;</span>
<span class="source-line-no">125</span><span id="line-125"> private boolean cacheBlocks = true;</span>
<span class="source-line-no">126</span><span id="line-126"> private boolean reversed = false;</span>
<span class="source-line-no">127</span><span id="line-127"> private TimeRange tr = TimeRange.allTime();</span>
<span class="source-line-no">128</span><span id="line-128"> private Map&lt;byte[], NavigableSet&lt;byte[]&gt;&gt; familyMap =</span>
<span class="source-line-no">129</span><span id="line-129"> new TreeMap&lt;byte[], NavigableSet&lt;byte[]&gt;&gt;(Bytes.BYTES_COMPARATOR);</span>
<span class="source-line-no">130</span><span id="line-130"> private Boolean asyncPrefetch = null;</span>
<span class="source-line-no">131</span><span id="line-131"></span>
<span class="source-line-no">132</span><span id="line-132"> /**</span>
<span class="source-line-no">133</span><span id="line-133"> * Parameter name for client scanner sync/async prefetch toggle. When using async scanner,</span>
<span class="source-line-no">134</span><span id="line-134"> * prefetching data from the server is done at the background. The parameter currently won't have</span>
<span class="source-line-no">135</span><span id="line-135"> * any effect in the case that the user has set Scan#setSmall or Scan#setReversed</span>
<span class="source-line-no">136</span><span id="line-136"> */</span>
<span class="source-line-no">137</span><span id="line-137"> public static final String HBASE_CLIENT_SCANNER_ASYNC_PREFETCH =</span>
<span class="source-line-no">138</span><span id="line-138"> "hbase.client.scanner.async.prefetch";</span>
<span class="source-line-no">139</span><span id="line-139"></span>
<span class="source-line-no">140</span><span id="line-140"> /**</span>
<span class="source-line-no">141</span><span id="line-141"> * Default value of {@link #HBASE_CLIENT_SCANNER_ASYNC_PREFETCH}.</span>
<span class="source-line-no">142</span><span id="line-142"> */</span>
<span class="source-line-no">143</span><span id="line-143"> public static final boolean DEFAULT_HBASE_CLIENT_SCANNER_ASYNC_PREFETCH = false;</span>
<span class="source-line-no">144</span><span id="line-144"></span>
<span class="source-line-no">145</span><span id="line-145"> /**</span>
<span class="source-line-no">146</span><span id="line-146"> * The mvcc read point to use when open a scanner. Remember to clear it after switching regions as</span>
<span class="source-line-no">147</span><span id="line-147"> * the mvcc is only valid within region scope.</span>
<span class="source-line-no">148</span><span id="line-148"> */</span>
<span class="source-line-no">149</span><span id="line-149"> private long mvccReadPoint = -1L;</span>
<span class="source-line-no">150</span><span id="line-150"></span>
<span class="source-line-no">151</span><span id="line-151"> /**</span>
<span class="source-line-no">152</span><span id="line-152"> * The number of rows we want for this scan. We will terminate the scan if the number of return</span>
<span class="source-line-no">153</span><span id="line-153"> * rows reaches this value.</span>
<span class="source-line-no">154</span><span id="line-154"> */</span>
<span class="source-line-no">155</span><span id="line-155"> private int limit = -1;</span>
<span class="source-line-no">156</span><span id="line-156"></span>
<span class="source-line-no">157</span><span id="line-157"> /**</span>
<span class="source-line-no">158</span><span id="line-158"> * Control whether to use pread at server side.</span>
<span class="source-line-no">159</span><span id="line-159"> */</span>
<span class="source-line-no">160</span><span id="line-160"> private ReadType readType = ReadType.DEFAULT;</span>
<span class="source-line-no">161</span><span id="line-161"></span>
<span class="source-line-no">162</span><span id="line-162"> private boolean needCursorResult = false;</span>
<span class="source-line-no">163</span><span id="line-163"></span>
<span class="source-line-no">164</span><span id="line-164"> /**</span>
<span class="source-line-no">165</span><span id="line-165"> * Create a Scan operation across all rows.</span>
<span class="source-line-no">166</span><span id="line-166"> */</span>
<span class="source-line-no">167</span><span id="line-167"> public Scan() {</span>
<span class="source-line-no">168</span><span id="line-168"> }</span>
<span class="source-line-no">169</span><span id="line-169"></span>
<span class="source-line-no">170</span><span id="line-170"> /**</span>
<span class="source-line-no">171</span><span id="line-171"> * Creates a new instance of this class while copying all values.</span>
<span class="source-line-no">172</span><span id="line-172"> * @param scan The scan instance to copy from.</span>
<span class="source-line-no">173</span><span id="line-173"> * @throws IOException When copying the values fails.</span>
<span class="source-line-no">174</span><span id="line-174"> */</span>
<span class="source-line-no">175</span><span id="line-175"> public Scan(Scan scan) throws IOException {</span>
<span class="source-line-no">176</span><span id="line-176"> startRow = scan.getStartRow();</span>
<span class="source-line-no">177</span><span id="line-177"> includeStartRow = scan.includeStartRow();</span>
<span class="source-line-no">178</span><span id="line-178"> stopRow = scan.getStopRow();</span>
<span class="source-line-no">179</span><span id="line-179"> includeStopRow = scan.includeStopRow();</span>
<span class="source-line-no">180</span><span id="line-180"> maxVersions = scan.getMaxVersions();</span>
<span class="source-line-no">181</span><span id="line-181"> batch = scan.getBatch();</span>
<span class="source-line-no">182</span><span id="line-182"> storeLimit = scan.getMaxResultsPerColumnFamily();</span>
<span class="source-line-no">183</span><span id="line-183"> storeOffset = scan.getRowOffsetPerColumnFamily();</span>
<span class="source-line-no">184</span><span id="line-184"> caching = scan.getCaching();</span>
<span class="source-line-no">185</span><span id="line-185"> maxResultSize = scan.getMaxResultSize();</span>
<span class="source-line-no">186</span><span id="line-186"> cacheBlocks = scan.getCacheBlocks();</span>
<span class="source-line-no">187</span><span id="line-187"> filter = scan.getFilter(); // clone?</span>
<span class="source-line-no">188</span><span id="line-188"> loadColumnFamiliesOnDemand = scan.getLoadColumnFamiliesOnDemandValue();</span>
<span class="source-line-no">189</span><span id="line-189"> consistency = scan.getConsistency();</span>
<span class="source-line-no">190</span><span id="line-190"> this.setIsolationLevel(scan.getIsolationLevel());</span>
<span class="source-line-no">191</span><span id="line-191"> reversed = scan.isReversed();</span>
<span class="source-line-no">192</span><span id="line-192"> asyncPrefetch = scan.isAsyncPrefetch();</span>
<span class="source-line-no">193</span><span id="line-193"> allowPartialResults = scan.getAllowPartialResults();</span>
<span class="source-line-no">194</span><span id="line-194"> tr = scan.getTimeRange(); // TimeRange is immutable</span>
<span class="source-line-no">195</span><span id="line-195"> Map&lt;byte[], NavigableSet&lt;byte[]&gt;&gt; fams = scan.getFamilyMap();</span>
<span class="source-line-no">196</span><span id="line-196"> for (Map.Entry&lt;byte[], NavigableSet&lt;byte[]&gt;&gt; entry : fams.entrySet()) {</span>
<span class="source-line-no">197</span><span id="line-197"> byte[] fam = entry.getKey();</span>
<span class="source-line-no">198</span><span id="line-198"> NavigableSet&lt;byte[]&gt; cols = entry.getValue();</span>
<span class="source-line-no">199</span><span id="line-199"> if (cols != null &amp;&amp; cols.size() &gt; 0) {</span>
<span class="source-line-no">200</span><span id="line-200"> for (byte[] col : cols) {</span>
<span class="source-line-no">201</span><span id="line-201"> addColumn(fam, col);</span>
<span class="source-line-no">202</span><span id="line-202"> }</span>
<span class="source-line-no">203</span><span id="line-203"> } else {</span>
<span class="source-line-no">204</span><span id="line-204"> addFamily(fam);</span>
<span class="source-line-no">205</span><span id="line-205"> }</span>
<span class="source-line-no">206</span><span id="line-206"> }</span>
<span class="source-line-no">207</span><span id="line-207"> for (Map.Entry&lt;String, byte[]&gt; attr : scan.getAttributesMap().entrySet()) {</span>
<span class="source-line-no">208</span><span id="line-208"> setAttribute(attr.getKey(), attr.getValue());</span>
<span class="source-line-no">209</span><span id="line-209"> }</span>
<span class="source-line-no">210</span><span id="line-210"> for (Map.Entry&lt;byte[], TimeRange&gt; entry : scan.getColumnFamilyTimeRange().entrySet()) {</span>
<span class="source-line-no">211</span><span id="line-211"> TimeRange tr = entry.getValue();</span>
<span class="source-line-no">212</span><span id="line-212"> setColumnFamilyTimeRange(entry.getKey(), tr.getMin(), tr.getMax());</span>
<span class="source-line-no">213</span><span id="line-213"> }</span>
<span class="source-line-no">214</span><span id="line-214"> this.mvccReadPoint = scan.getMvccReadPoint();</span>
<span class="source-line-no">215</span><span id="line-215"> this.limit = scan.getLimit();</span>
<span class="source-line-no">216</span><span id="line-216"> this.needCursorResult = scan.isNeedCursorResult();</span>
<span class="source-line-no">217</span><span id="line-217"> setPriority(scan.getPriority());</span>
<span class="source-line-no">218</span><span id="line-218"> readType = scan.getReadType();</span>
<span class="source-line-no">219</span><span id="line-219"> super.setReplicaId(scan.getReplicaId());</span>
<span class="source-line-no">220</span><span id="line-220"> }</span>
<span class="source-line-no">221</span><span id="line-221"></span>
<span class="source-line-no">222</span><span id="line-222"> /**</span>
<span class="source-line-no">223</span><span id="line-223"> * Builds a scan object with the same specs as get.</span>
<span class="source-line-no">224</span><span id="line-224"> * @param get get to model scan after</span>
<span class="source-line-no">225</span><span id="line-225"> */</span>
<span class="source-line-no">226</span><span id="line-226"> public Scan(Get get) {</span>
<span class="source-line-no">227</span><span id="line-227"> this.startRow = get.getRow();</span>
<span class="source-line-no">228</span><span id="line-228"> this.includeStartRow = true;</span>
<span class="source-line-no">229</span><span id="line-229"> this.stopRow = get.getRow();</span>
<span class="source-line-no">230</span><span id="line-230"> this.includeStopRow = true;</span>
<span class="source-line-no">231</span><span id="line-231"> this.filter = get.getFilter();</span>
<span class="source-line-no">232</span><span id="line-232"> this.cacheBlocks = get.getCacheBlocks();</span>
<span class="source-line-no">233</span><span id="line-233"> this.maxVersions = get.getMaxVersions();</span>
<span class="source-line-no">234</span><span id="line-234"> this.storeLimit = get.getMaxResultsPerColumnFamily();</span>
<span class="source-line-no">235</span><span id="line-235"> this.storeOffset = get.getRowOffsetPerColumnFamily();</span>
<span class="source-line-no">236</span><span id="line-236"> this.tr = get.getTimeRange();</span>
<span class="source-line-no">237</span><span id="line-237"> this.familyMap = get.getFamilyMap();</span>
<span class="source-line-no">238</span><span id="line-238"> this.asyncPrefetch = false;</span>
<span class="source-line-no">239</span><span id="line-239"> this.consistency = get.getConsistency();</span>
<span class="source-line-no">240</span><span id="line-240"> this.setIsolationLevel(get.getIsolationLevel());</span>
<span class="source-line-no">241</span><span id="line-241"> this.loadColumnFamiliesOnDemand = get.getLoadColumnFamiliesOnDemandValue();</span>
<span class="source-line-no">242</span><span id="line-242"> for (Map.Entry&lt;String, byte[]&gt; attr : get.getAttributesMap().entrySet()) {</span>
<span class="source-line-no">243</span><span id="line-243"> setAttribute(attr.getKey(), attr.getValue());</span>
<span class="source-line-no">244</span><span id="line-244"> }</span>
<span class="source-line-no">245</span><span id="line-245"> for (Map.Entry&lt;byte[], TimeRange&gt; entry : get.getColumnFamilyTimeRange().entrySet()) {</span>
<span class="source-line-no">246</span><span id="line-246"> TimeRange tr = entry.getValue();</span>
<span class="source-line-no">247</span><span id="line-247"> setColumnFamilyTimeRange(entry.getKey(), tr.getMin(), tr.getMax());</span>
<span class="source-line-no">248</span><span id="line-248"> }</span>
<span class="source-line-no">249</span><span id="line-249"> this.mvccReadPoint = -1L;</span>
<span class="source-line-no">250</span><span id="line-250"> setPriority(get.getPriority());</span>
<span class="source-line-no">251</span><span id="line-251"> super.setReplicaId(get.getReplicaId());</span>
<span class="source-line-no">252</span><span id="line-252"> }</span>
<span class="source-line-no">253</span><span id="line-253"></span>
<span class="source-line-no">254</span><span id="line-254"> public boolean isGetScan() {</span>
<span class="source-line-no">255</span><span id="line-255"> return includeStartRow &amp;&amp; includeStopRow</span>
<span class="source-line-no">256</span><span id="line-256"> &amp;&amp; ClientUtil.areScanStartRowAndStopRowEqual(this.startRow, this.stopRow);</span>
<span class="source-line-no">257</span><span id="line-257"> }</span>
<span class="source-line-no">258</span><span id="line-258"></span>
<span class="source-line-no">259</span><span id="line-259"> /**</span>
<span class="source-line-no">260</span><span id="line-260"> * Get all columns from the specified family.</span>
<span class="source-line-no">261</span><span id="line-261"> * &lt;p&gt;</span>
<span class="source-line-no">262</span><span id="line-262"> * Overrides previous calls to addColumn for this family.</span>
<span class="source-line-no">263</span><span id="line-263"> * @param family family name</span>
<span class="source-line-no">264</span><span id="line-264"> */</span>
<span class="source-line-no">265</span><span id="line-265"> public Scan addFamily(byte[] family) {</span>
<span class="source-line-no">266</span><span id="line-266"> familyMap.remove(family);</span>
<span class="source-line-no">267</span><span id="line-267"> familyMap.put(family, null);</span>
<span class="source-line-no">268</span><span id="line-268"> return this;</span>
<span class="source-line-no">269</span><span id="line-269"> }</span>
<span class="source-line-no">270</span><span id="line-270"></span>
<span class="source-line-no">271</span><span id="line-271"> /**</span>
<span class="source-line-no">272</span><span id="line-272"> * Get the column from the specified family with the specified qualifier.</span>
<span class="source-line-no">273</span><span id="line-273"> * &lt;p&gt;</span>
<span class="source-line-no">274</span><span id="line-274"> * Overrides previous calls to addFamily for this family.</span>
<span class="source-line-no">275</span><span id="line-275"> * @param family family name</span>
<span class="source-line-no">276</span><span id="line-276"> * @param qualifier column qualifier</span>
<span class="source-line-no">277</span><span id="line-277"> */</span>
<span class="source-line-no">278</span><span id="line-278"> public Scan addColumn(byte[] family, byte[] qualifier) {</span>
<span class="source-line-no">279</span><span id="line-279"> NavigableSet&lt;byte[]&gt; set = familyMap.get(family);</span>
<span class="source-line-no">280</span><span id="line-280"> if (set == null) {</span>
<span class="source-line-no">281</span><span id="line-281"> set = new TreeSet&lt;&gt;(Bytes.BYTES_COMPARATOR);</span>
<span class="source-line-no">282</span><span id="line-282"> familyMap.put(family, set);</span>
<span class="source-line-no">283</span><span id="line-283"> }</span>
<span class="source-line-no">284</span><span id="line-284"> if (qualifier == null) {</span>
<span class="source-line-no">285</span><span id="line-285"> qualifier = HConstants.EMPTY_BYTE_ARRAY;</span>
<span class="source-line-no">286</span><span id="line-286"> }</span>
<span class="source-line-no">287</span><span id="line-287"> set.add(qualifier);</span>
<span class="source-line-no">288</span><span id="line-288"> return this;</span>
<span class="source-line-no">289</span><span id="line-289"> }</span>
<span class="source-line-no">290</span><span id="line-290"></span>
<span class="source-line-no">291</span><span id="line-291"> /**</span>
<span class="source-line-no">292</span><span id="line-292"> * Get versions of columns only within the specified timestamp range, [minStamp, maxStamp). Note,</span>
<span class="source-line-no">293</span><span id="line-293"> * default maximum versions to return is 1. If your time range spans more than one version and you</span>
<span class="source-line-no">294</span><span id="line-294"> * want all versions returned, up the number of versions beyond the default.</span>
<span class="source-line-no">295</span><span id="line-295"> * @param minStamp minimum timestamp value, inclusive</span>
<span class="source-line-no">296</span><span id="line-296"> * @param maxStamp maximum timestamp value, exclusive</span>
<span class="source-line-no">297</span><span id="line-297"> * @see #readAllVersions()</span>
<span class="source-line-no">298</span><span id="line-298"> * @see #readVersions(int)</span>
<span class="source-line-no">299</span><span id="line-299"> */</span>
<span class="source-line-no">300</span><span id="line-300"> public Scan setTimeRange(long minStamp, long maxStamp) throws IOException {</span>
<span class="source-line-no">301</span><span id="line-301"> tr = TimeRange.between(minStamp, maxStamp);</span>
<span class="source-line-no">302</span><span id="line-302"> return this;</span>
<span class="source-line-no">303</span><span id="line-303"> }</span>
<span class="source-line-no">304</span><span id="line-304"></span>
<span class="source-line-no">305</span><span id="line-305"> /**</span>
<span class="source-line-no">306</span><span id="line-306"> * Get versions of columns with the specified timestamp. Note, default maximum versions to return</span>
<span class="source-line-no">307</span><span id="line-307"> * is 1. If your time range spans more than one version and you want all versions returned, up the</span>
<span class="source-line-no">308</span><span id="line-308"> * number of versions beyond the defaut.</span>
<span class="source-line-no">309</span><span id="line-309"> * @param timestamp version timestamp</span>
<span class="source-line-no">310</span><span id="line-310"> * @see #readAllVersions()</span>
<span class="source-line-no">311</span><span id="line-311"> * @see #readVersions(int)</span>
<span class="source-line-no">312</span><span id="line-312"> */</span>
<span class="source-line-no">313</span><span id="line-313"> public Scan setTimestamp(long timestamp) {</span>
<span class="source-line-no">314</span><span id="line-314"> try {</span>
<span class="source-line-no">315</span><span id="line-315"> tr = TimeRange.at(timestamp);</span>
<span class="source-line-no">316</span><span id="line-316"> } catch (Exception e) {</span>
<span class="source-line-no">317</span><span id="line-317"> // This should never happen, unless integer overflow or something extremely wrong...</span>
<span class="source-line-no">318</span><span id="line-318"> LOG.error("TimeRange failed, likely caused by integer overflow. ", e);</span>
<span class="source-line-no">319</span><span id="line-319"> throw e;</span>
<span class="source-line-no">320</span><span id="line-320"> }</span>
<span class="source-line-no">321</span><span id="line-321"></span>
<span class="source-line-no">322</span><span id="line-322"> return this;</span>
<span class="source-line-no">323</span><span id="line-323"> }</span>
<span class="source-line-no">324</span><span id="line-324"></span>
<span class="source-line-no">325</span><span id="line-325"> @Override</span>
<span class="source-line-no">326</span><span id="line-326"> public Scan setColumnFamilyTimeRange(byte[] cf, long minStamp, long maxStamp) {</span>
<span class="source-line-no">327</span><span id="line-327"> return (Scan) super.setColumnFamilyTimeRange(cf, minStamp, maxStamp);</span>
<span class="source-line-no">328</span><span id="line-328"> }</span>
<span class="source-line-no">329</span><span id="line-329"></span>
<span class="source-line-no">330</span><span id="line-330"> /**</span>
<span class="source-line-no">331</span><span id="line-331"> * Set the start row of the scan.</span>
<span class="source-line-no">332</span><span id="line-332"> * &lt;p&gt;</span>
<span class="source-line-no">333</span><span id="line-333"> * If the specified row does not exist, the Scanner will start from the next closest row after the</span>
<span class="source-line-no">334</span><span id="line-334"> * specified row.</span>
<span class="source-line-no">335</span><span id="line-335"> * &lt;p&gt;</span>
<span class="source-line-no">336</span><span id="line-336"> * &lt;b&gt;Note:&lt;/b&gt; &lt;strong&gt;Do NOT use this in combination with {@link #setRowPrefixFilter(byte[])} or</span>
<span class="source-line-no">337</span><span id="line-337"> * {@link #setStartStopRowForPrefixScan(byte[])}.&lt;/strong&gt; Doing so will make the scan result</span>
<span class="source-line-no">338</span><span id="line-338"> * unexpected or even undefined.</span>
<span class="source-line-no">339</span><span id="line-339"> * &lt;/p&gt;</span>
<span class="source-line-no">340</span><span id="line-340"> * @param startRow row to start scanner at or after</span>
<span class="source-line-no">341</span><span id="line-341"> * @throws IllegalArgumentException if startRow does not meet criteria for a row key (when length</span>
<span class="source-line-no">342</span><span id="line-342"> * exceeds {@link HConstants#MAX_ROW_LENGTH})</span>
<span class="source-line-no">343</span><span id="line-343"> */</span>
<span class="source-line-no">344</span><span id="line-344"> public Scan withStartRow(byte[] startRow) {</span>
<span class="source-line-no">345</span><span id="line-345"> return withStartRow(startRow, true);</span>
<span class="source-line-no">346</span><span id="line-346"> }</span>
<span class="source-line-no">347</span><span id="line-347"></span>
<span class="source-line-no">348</span><span id="line-348"> /**</span>
<span class="source-line-no">349</span><span id="line-349"> * Set the start row of the scan.</span>
<span class="source-line-no">350</span><span id="line-350"> * &lt;p&gt;</span>
<span class="source-line-no">351</span><span id="line-351"> * If the specified row does not exist, or the {@code inclusive} is {@code false}, the Scanner</span>
<span class="source-line-no">352</span><span id="line-352"> * will start from the next closest row after the specified row.</span>
<span class="source-line-no">353</span><span id="line-353"> * &lt;p&gt;</span>
<span class="source-line-no">354</span><span id="line-354"> * &lt;b&gt;Note:&lt;/b&gt; &lt;strong&gt;Do NOT use this in combination with {@link #setRowPrefixFilter(byte[])} or</span>
<span class="source-line-no">355</span><span id="line-355"> * {@link #setStartStopRowForPrefixScan(byte[])}.&lt;/strong&gt; Doing so will make the scan result</span>
<span class="source-line-no">356</span><span id="line-356"> * unexpected or even undefined.</span>
<span class="source-line-no">357</span><span id="line-357"> * &lt;/p&gt;</span>
<span class="source-line-no">358</span><span id="line-358"> * @param startRow row to start scanner at or after</span>
<span class="source-line-no">359</span><span id="line-359"> * @param inclusive whether we should include the start row when scan</span>
<span class="source-line-no">360</span><span id="line-360"> * @throws IllegalArgumentException if startRow does not meet criteria for a row key (when length</span>
<span class="source-line-no">361</span><span id="line-361"> * exceeds {@link HConstants#MAX_ROW_LENGTH})</span>
<span class="source-line-no">362</span><span id="line-362"> */</span>
<span class="source-line-no">363</span><span id="line-363"> public Scan withStartRow(byte[] startRow, boolean inclusive) {</span>
<span class="source-line-no">364</span><span id="line-364"> if (Bytes.len(startRow) &gt; HConstants.MAX_ROW_LENGTH) {</span>
<span class="source-line-no">365</span><span id="line-365"> throw new IllegalArgumentException("startRow's length must be less than or equal to "</span>
<span class="source-line-no">366</span><span id="line-366"> + HConstants.MAX_ROW_LENGTH + " to meet the criteria" + " for a row key.");</span>
<span class="source-line-no">367</span><span id="line-367"> }</span>
<span class="source-line-no">368</span><span id="line-368"> this.startRow = startRow;</span>
<span class="source-line-no">369</span><span id="line-369"> this.includeStartRow = inclusive;</span>
<span class="source-line-no">370</span><span id="line-370"> return this;</span>
<span class="source-line-no">371</span><span id="line-371"> }</span>
<span class="source-line-no">372</span><span id="line-372"></span>
<span class="source-line-no">373</span><span id="line-373"> /**</span>
<span class="source-line-no">374</span><span id="line-374"> * Set the stop row of the scan.</span>
<span class="source-line-no">375</span><span id="line-375"> * &lt;p&gt;</span>
<span class="source-line-no">376</span><span id="line-376"> * The scan will include rows that are lexicographically less than the provided stopRow.</span>
<span class="source-line-no">377</span><span id="line-377"> * &lt;p&gt;</span>
<span class="source-line-no">378</span><span id="line-378"> * &lt;b&gt;Note:&lt;/b&gt; &lt;strong&gt;Do NOT use this in combination with {@link #setRowPrefixFilter(byte[])} or</span>
<span class="source-line-no">379</span><span id="line-379"> * {@link #setStartStopRowForPrefixScan(byte[])}.&lt;/strong&gt; Doing so will make the scan result</span>
<span class="source-line-no">380</span><span id="line-380"> * unexpected or even undefined.</span>
<span class="source-line-no">381</span><span id="line-381"> * &lt;/p&gt;</span>
<span class="source-line-no">382</span><span id="line-382"> * @param stopRow row to end at (exclusive)</span>
<span class="source-line-no">383</span><span id="line-383"> * @throws IllegalArgumentException if stopRow does not meet criteria for a row key (when length</span>
<span class="source-line-no">384</span><span id="line-384"> * exceeds {@link HConstants#MAX_ROW_LENGTH})</span>
<span class="source-line-no">385</span><span id="line-385"> */</span>
<span class="source-line-no">386</span><span id="line-386"> public Scan withStopRow(byte[] stopRow) {</span>
<span class="source-line-no">387</span><span id="line-387"> return withStopRow(stopRow, false);</span>
<span class="source-line-no">388</span><span id="line-388"> }</span>
<span class="source-line-no">389</span><span id="line-389"></span>
<span class="source-line-no">390</span><span id="line-390"> /**</span>
<span class="source-line-no">391</span><span id="line-391"> * Set the stop row of the scan.</span>
<span class="source-line-no">392</span><span id="line-392"> * &lt;p&gt;</span>
<span class="source-line-no">393</span><span id="line-393"> * The scan will include rows that are lexicographically less than (or equal to if</span>
<span class="source-line-no">394</span><span id="line-394"> * {@code inclusive} is {@code true}) the provided stopRow.</span>
<span class="source-line-no">395</span><span id="line-395"> * &lt;p&gt;</span>
<span class="source-line-no">396</span><span id="line-396"> * &lt;b&gt;Note:&lt;/b&gt; &lt;strong&gt;Do NOT use this in combination with {@link #setRowPrefixFilter(byte[])} or</span>
<span class="source-line-no">397</span><span id="line-397"> * {@link #setStartStopRowForPrefixScan(byte[])}.&lt;/strong&gt; Doing so will make the scan result</span>
<span class="source-line-no">398</span><span id="line-398"> * unexpected or even undefined.</span>
<span class="source-line-no">399</span><span id="line-399"> * &lt;/p&gt;</span>
<span class="source-line-no">400</span><span id="line-400"> * @param stopRow row to end at</span>
<span class="source-line-no">401</span><span id="line-401"> * @param inclusive whether we should include the stop row when scan</span>
<span class="source-line-no">402</span><span id="line-402"> * @throws IllegalArgumentException if stopRow does not meet criteria for a row key (when length</span>
<span class="source-line-no">403</span><span id="line-403"> * exceeds {@link HConstants#MAX_ROW_LENGTH})</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 Scan withStopRow(byte[] stopRow, boolean inclusive) {</span>
<span class="source-line-no">406</span><span id="line-406"> if (Bytes.len(stopRow) &gt; HConstants.MAX_ROW_LENGTH) {</span>
<span class="source-line-no">407</span><span id="line-407"> throw new IllegalArgumentException("stopRow's length must be less than or equal to "</span>
<span class="source-line-no">408</span><span id="line-408"> + HConstants.MAX_ROW_LENGTH + " to meet the criteria" + " for a row key.");</span>
<span class="source-line-no">409</span><span id="line-409"> }</span>
<span class="source-line-no">410</span><span id="line-410"> this.stopRow = stopRow;</span>
<span class="source-line-no">411</span><span id="line-411"> this.includeStopRow = inclusive;</span>
<span class="source-line-no">412</span><span id="line-412"> return this;</span>
<span class="source-line-no">413</span><span id="line-413"> }</span>
<span class="source-line-no">414</span><span id="line-414"></span>
<span class="source-line-no">415</span><span id="line-415"> /**</span>
<span class="source-line-no">416</span><span id="line-416"> * &lt;p&gt;</span>
<span class="source-line-no">417</span><span id="line-417"> * Set a filter (using stopRow and startRow) so the result set only contains rows where the rowKey</span>
<span class="source-line-no">418</span><span id="line-418"> * starts with the specified prefix.</span>
<span class="source-line-no">419</span><span id="line-419"> * &lt;/p&gt;</span>
<span class="source-line-no">420</span><span id="line-420"> * &lt;p&gt;</span>
<span class="source-line-no">421</span><span id="line-421"> * This is a utility method that converts the desired rowPrefix into the appropriate values for</span>
<span class="source-line-no">422</span><span id="line-422"> * the startRow and stopRow to achieve the desired result.</span>
<span class="source-line-no">423</span><span id="line-423"> * &lt;/p&gt;</span>
<span class="source-line-no">424</span><span id="line-424"> * &lt;p&gt;</span>
<span class="source-line-no">425</span><span id="line-425"> * This can safely be used in combination with setFilter.</span>
<span class="source-line-no">426</span><span id="line-426"> * &lt;/p&gt;</span>
<span class="source-line-no">427</span><span id="line-427"> * &lt;p&gt;</span>
<span class="source-line-no">428</span><span id="line-428"> * &lt;strong&gt;This CANNOT be used in combination with withStartRow and/or withStopRow.&lt;/strong&gt; Such</span>
<span class="source-line-no">429</span><span id="line-429"> * a combination will yield unexpected and even undefined results.</span>
<span class="source-line-no">430</span><span id="line-430"> * &lt;/p&gt;</span>
<span class="source-line-no">431</span><span id="line-431"> * @param rowPrefix the prefix all rows must start with. (Set &lt;i&gt;null&lt;/i&gt; to remove the filter.)</span>
<span class="source-line-no">432</span><span id="line-432"> * @deprecated since 2.5.0, will be removed in 4.0.0. The name of this method is considered to be</span>
<span class="source-line-no">433</span><span id="line-433"> * confusing as it does not use a {@link Filter} but uses setting the startRow and</span>
<span class="source-line-no">434</span><span id="line-434"> * stopRow instead. Use {@link #setStartStopRowForPrefixScan(byte[])} instead.</span>
<span class="source-line-no">435</span><span id="line-435"> */</span>
<span class="source-line-no">436</span><span id="line-436"> @Deprecated</span>
<span class="source-line-no">437</span><span id="line-437"> public Scan setRowPrefixFilter(byte[] rowPrefix) {</span>
<span class="source-line-no">438</span><span id="line-438"> return setStartStopRowForPrefixScan(rowPrefix);</span>
<span class="source-line-no">439</span><span id="line-439"> }</span>
<span class="source-line-no">440</span><span id="line-440"></span>
<span class="source-line-no">441</span><span id="line-441"> /**</span>
<span class="source-line-no">442</span><span id="line-442"> * &lt;p&gt;</span>
<span class="source-line-no">443</span><span id="line-443"> * Set a filter (using stopRow and startRow) so the result set only contains rows where the rowKey</span>
<span class="source-line-no">444</span><span id="line-444"> * starts with the specified prefix.</span>
<span class="source-line-no">445</span><span id="line-445"> * &lt;/p&gt;</span>
<span class="source-line-no">446</span><span id="line-446"> * &lt;p&gt;</span>
<span class="source-line-no">447</span><span id="line-447"> * This is a utility method that converts the desired rowPrefix into the appropriate values for</span>
<span class="source-line-no">448</span><span id="line-448"> * the startRow and stopRow to achieve the desired result.</span>
<span class="source-line-no">449</span><span id="line-449"> * &lt;/p&gt;</span>
<span class="source-line-no">450</span><span id="line-450"> * &lt;p&gt;</span>
<span class="source-line-no">451</span><span id="line-451"> * This can safely be used in combination with setFilter.</span>
<span class="source-line-no">452</span><span id="line-452"> * &lt;/p&gt;</span>
<span class="source-line-no">453</span><span id="line-453"> * &lt;p&gt;</span>
<span class="source-line-no">454</span><span id="line-454"> * &lt;strong&gt;This CANNOT be used in combination with withStartRow and/or withStopRow.&lt;/strong&gt; Such</span>
<span class="source-line-no">455</span><span id="line-455"> * a combination will yield unexpected and even undefined results.</span>
<span class="source-line-no">456</span><span id="line-456"> * &lt;/p&gt;</span>
<span class="source-line-no">457</span><span id="line-457"> * @param rowPrefix the prefix all rows must start with. (Set &lt;i&gt;null&lt;/i&gt; to remove the filter.)</span>
<span class="source-line-no">458</span><span id="line-458"> */</span>
<span class="source-line-no">459</span><span id="line-459"> public Scan setStartStopRowForPrefixScan(byte[] rowPrefix) {</span>
<span class="source-line-no">460</span><span id="line-460"> if (rowPrefix == null) {</span>
<span class="source-line-no">461</span><span id="line-461"> withStartRow(HConstants.EMPTY_START_ROW);</span>
<span class="source-line-no">462</span><span id="line-462"> withStopRow(HConstants.EMPTY_END_ROW);</span>
<span class="source-line-no">463</span><span id="line-463"> } else {</span>
<span class="source-line-no">464</span><span id="line-464"> this.withStartRow(rowPrefix);</span>
<span class="source-line-no">465</span><span id="line-465"> this.withStopRow(ClientUtil.calculateTheClosestNextRowKeyForPrefix(rowPrefix));</span>
<span class="source-line-no">466</span><span id="line-466"> }</span>
<span class="source-line-no">467</span><span id="line-467"> return this;</span>
<span class="source-line-no">468</span><span id="line-468"> }</span>
<span class="source-line-no">469</span><span id="line-469"></span>
<span class="source-line-no">470</span><span id="line-470"> /**</span>
<span class="source-line-no">471</span><span id="line-471"> * Get all available versions.</span>
<span class="source-line-no">472</span><span id="line-472"> */</span>
<span class="source-line-no">473</span><span id="line-473"> public Scan readAllVersions() {</span>
<span class="source-line-no">474</span><span id="line-474"> this.maxVersions = Integer.MAX_VALUE;</span>
<span class="source-line-no">475</span><span id="line-475"> return this;</span>
<span class="source-line-no">476</span><span id="line-476"> }</span>
<span class="source-line-no">477</span><span id="line-477"></span>
<span class="source-line-no">478</span><span id="line-478"> /**</span>
<span class="source-line-no">479</span><span id="line-479"> * Get up to the specified number of versions of each column.</span>
<span class="source-line-no">480</span><span id="line-480"> * @param versions specified number of versions for each column</span>
<span class="source-line-no">481</span><span id="line-481"> */</span>
<span class="source-line-no">482</span><span id="line-482"> public Scan readVersions(int versions) {</span>
<span class="source-line-no">483</span><span id="line-483"> this.maxVersions = versions;</span>
<span class="source-line-no">484</span><span id="line-484"> return this;</span>
<span class="source-line-no">485</span><span id="line-485"> }</span>
<span class="source-line-no">486</span><span id="line-486"></span>
<span class="source-line-no">487</span><span id="line-487"> /**</span>
<span class="source-line-no">488</span><span id="line-488"> * Set the maximum number of cells to return for each call to next(). Callers should be aware that</span>
<span class="source-line-no">489</span><span id="line-489"> * this is not equivalent to calling {@link #setAllowPartialResults(boolean)}. If you don't allow</span>
<span class="source-line-no">490</span><span id="line-490"> * partial results, the number of cells in each Result must equal to your batch setting unless it</span>
<span class="source-line-no">491</span><span id="line-491"> * is the last Result for current row. So this method is helpful in paging queries. If you just</span>
<span class="source-line-no">492</span><span id="line-492"> * want to prevent OOM at client, use setAllowPartialResults(true) is better.</span>
<span class="source-line-no">493</span><span id="line-493"> * @param batch the maximum number of values</span>
<span class="source-line-no">494</span><span id="line-494"> * @see Result#mayHaveMoreCellsInRow()</span>
<span class="source-line-no">495</span><span id="line-495"> */</span>
<span class="source-line-no">496</span><span id="line-496"> public Scan setBatch(int batch) {</span>
<span class="source-line-no">497</span><span id="line-497"> if (this.hasFilter() &amp;&amp; this.filter.hasFilterRow()) {</span>
<span class="source-line-no">498</span><span id="line-498"> throw new IncompatibleFilterException(</span>
<span class="source-line-no">499</span><span id="line-499"> "Cannot set batch on a scan using a filter" + " that returns true for filter.hasFilterRow");</span>
<span class="source-line-no">500</span><span id="line-500"> }</span>
<span class="source-line-no">501</span><span id="line-501"> this.batch = batch;</span>
<span class="source-line-no">502</span><span id="line-502"> return this;</span>
<span class="source-line-no">503</span><span id="line-503"> }</span>
<span class="source-line-no">504</span><span id="line-504"></span>
<span class="source-line-no">505</span><span id="line-505"> /**</span>
<span class="source-line-no">506</span><span id="line-506"> * Set the maximum number of values to return per row per Column Family</span>
<span class="source-line-no">507</span><span id="line-507"> * @param limit the maximum number of values returned / row / CF</span>
<span class="source-line-no">508</span><span id="line-508"> */</span>
<span class="source-line-no">509</span><span id="line-509"> public Scan setMaxResultsPerColumnFamily(int limit) {</span>
<span class="source-line-no">510</span><span id="line-510"> this.storeLimit = limit;</span>
<span class="source-line-no">511</span><span id="line-511"> return this;</span>
<span class="source-line-no">512</span><span id="line-512"> }</span>
<span class="source-line-no">513</span><span id="line-513"></span>
<span class="source-line-no">514</span><span id="line-514"> /**</span>
<span class="source-line-no">515</span><span id="line-515"> * Set offset for the row per Column Family.</span>
<span class="source-line-no">516</span><span id="line-516"> * @param offset is the number of kvs that will be skipped.</span>
<span class="source-line-no">517</span><span id="line-517"> */</span>
<span class="source-line-no">518</span><span id="line-518"> public Scan setRowOffsetPerColumnFamily(int offset) {</span>
<span class="source-line-no">519</span><span id="line-519"> this.storeOffset = offset;</span>
<span class="source-line-no">520</span><span id="line-520"> return this;</span>
<span class="source-line-no">521</span><span id="line-521"> }</span>
<span class="source-line-no">522</span><span id="line-522"></span>
<span class="source-line-no">523</span><span id="line-523"> /**</span>
<span class="source-line-no">524</span><span id="line-524"> * Set the number of rows for caching that will be passed to scanners. If not set, the</span>
<span class="source-line-no">525</span><span id="line-525"> * Configuration setting {@link HConstants#HBASE_CLIENT_SCANNER_CACHING} will apply. Higher</span>
<span class="source-line-no">526</span><span id="line-526"> * caching values will enable faster scanners but will use more memory.</span>
<span class="source-line-no">527</span><span id="line-527"> * @param caching the number of rows for caching</span>
<span class="source-line-no">528</span><span id="line-528"> */</span>
<span class="source-line-no">529</span><span id="line-529"> public Scan setCaching(int caching) {</span>
<span class="source-line-no">530</span><span id="line-530"> this.caching = caching;</span>
<span class="source-line-no">531</span><span id="line-531"> return this;</span>
<span class="source-line-no">532</span><span id="line-532"> }</span>
<span class="source-line-no">533</span><span id="line-533"></span>
<span class="source-line-no">534</span><span id="line-534"> /** Returns the maximum result size in bytes. See {@link #setMaxResultSize(long)} */</span>
<span class="source-line-no">535</span><span id="line-535"> public long getMaxResultSize() {</span>
<span class="source-line-no">536</span><span id="line-536"> return maxResultSize;</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"> /**</span>
<span class="source-line-no">540</span><span id="line-540"> * Set the maximum result size. The default is -1; this means that no specific maximum result size</span>
<span class="source-line-no">541</span><span id="line-541"> * will be set for this scan, and the global configured value will be used instead. (Defaults to</span>
<span class="source-line-no">542</span><span id="line-542"> * unlimited).</span>
<span class="source-line-no">543</span><span id="line-543"> * @param maxResultSize The maximum result size in bytes.</span>
<span class="source-line-no">544</span><span id="line-544"> */</span>
<span class="source-line-no">545</span><span id="line-545"> public Scan setMaxResultSize(long maxResultSize) {</span>
<span class="source-line-no">546</span><span id="line-546"> this.maxResultSize = maxResultSize;</span>
<span class="source-line-no">547</span><span id="line-547"> return this;</span>
<span class="source-line-no">548</span><span id="line-548"> }</span>
<span class="source-line-no">549</span><span id="line-549"></span>
<span class="source-line-no">550</span><span id="line-550"> @Override</span>
<span class="source-line-no">551</span><span id="line-551"> public Scan setFilter(Filter filter) {</span>
<span class="source-line-no">552</span><span id="line-552"> super.setFilter(filter);</span>
<span class="source-line-no">553</span><span id="line-553"> return this;</span>
<span class="source-line-no">554</span><span id="line-554"> }</span>
<span class="source-line-no">555</span><span id="line-555"></span>
<span class="source-line-no">556</span><span id="line-556"> /**</span>
<span class="source-line-no">557</span><span id="line-557"> * Setting the familyMap</span>
<span class="source-line-no">558</span><span id="line-558"> * @param familyMap map of family to qualifier</span>
<span class="source-line-no">559</span><span id="line-559"> */</span>
<span class="source-line-no">560</span><span id="line-560"> public Scan setFamilyMap(Map&lt;byte[], NavigableSet&lt;byte[]&gt;&gt; familyMap) {</span>
<span class="source-line-no">561</span><span id="line-561"> this.familyMap = familyMap;</span>
<span class="source-line-no">562</span><span id="line-562"> return this;</span>
<span class="source-line-no">563</span><span id="line-563"> }</span>
<span class="source-line-no">564</span><span id="line-564"></span>
<span class="source-line-no">565</span><span id="line-565"> /**</span>
<span class="source-line-no">566</span><span id="line-566"> * Getting the familyMap</span>
<span class="source-line-no">567</span><span id="line-567"> */</span>
<span class="source-line-no">568</span><span id="line-568"> public Map&lt;byte[], NavigableSet&lt;byte[]&gt;&gt; getFamilyMap() {</span>
<span class="source-line-no">569</span><span id="line-569"> return this.familyMap;</span>
<span class="source-line-no">570</span><span id="line-570"> }</span>
<span class="source-line-no">571</span><span id="line-571"></span>
<span class="source-line-no">572</span><span id="line-572"> /** Returns the number of families in familyMap */</span>
<span class="source-line-no">573</span><span id="line-573"> public int numFamilies() {</span>
<span class="source-line-no">574</span><span id="line-574"> if (hasFamilies()) {</span>
<span class="source-line-no">575</span><span id="line-575"> return this.familyMap.size();</span>
<span class="source-line-no">576</span><span id="line-576"> }</span>
<span class="source-line-no">577</span><span id="line-577"> return 0;</span>
<span class="source-line-no">578</span><span id="line-578"> }</span>
<span class="source-line-no">579</span><span id="line-579"></span>
<span class="source-line-no">580</span><span id="line-580"> /** Returns true if familyMap is non empty, false otherwise */</span>
<span class="source-line-no">581</span><span id="line-581"> public boolean hasFamilies() {</span>
<span class="source-line-no">582</span><span id="line-582"> return !this.familyMap.isEmpty();</span>
<span class="source-line-no">583</span><span id="line-583"> }</span>
<span class="source-line-no">584</span><span id="line-584"></span>
<span class="source-line-no">585</span><span id="line-585"> /** Returns the keys of the familyMap */</span>
<span class="source-line-no">586</span><span id="line-586"> public byte[][] getFamilies() {</span>
<span class="source-line-no">587</span><span id="line-587"> if (hasFamilies()) {</span>
<span class="source-line-no">588</span><span id="line-588"> return this.familyMap.keySet().toArray(new byte[0][0]);</span>
<span class="source-line-no">589</span><span id="line-589"> }</span>
<span class="source-line-no">590</span><span id="line-590"> return null;</span>
<span class="source-line-no">591</span><span id="line-591"> }</span>
<span class="source-line-no">592</span><span id="line-592"></span>
<span class="source-line-no">593</span><span id="line-593"> /** Returns the startrow */</span>
<span class="source-line-no">594</span><span id="line-594"> public byte[] getStartRow() {</span>
<span class="source-line-no">595</span><span id="line-595"> return this.startRow;</span>
<span class="source-line-no">596</span><span id="line-596"> }</span>
<span class="source-line-no">597</span><span id="line-597"></span>
<span class="source-line-no">598</span><span id="line-598"> /** Returns if we should include start row when scan */</span>
<span class="source-line-no">599</span><span id="line-599"> public boolean includeStartRow() {</span>
<span class="source-line-no">600</span><span id="line-600"> return includeStartRow;</span>
<span class="source-line-no">601</span><span id="line-601"> }</span>
<span class="source-line-no">602</span><span id="line-602"></span>
<span class="source-line-no">603</span><span id="line-603"> /** Returns the stoprow */</span>
<span class="source-line-no">604</span><span id="line-604"> public byte[] getStopRow() {</span>
<span class="source-line-no">605</span><span id="line-605"> return this.stopRow;</span>
<span class="source-line-no">606</span><span id="line-606"> }</span>
<span class="source-line-no">607</span><span id="line-607"></span>
<span class="source-line-no">608</span><span id="line-608"> /** Returns if we should include stop row when scan */</span>
<span class="source-line-no">609</span><span id="line-609"> public boolean includeStopRow() {</span>
<span class="source-line-no">610</span><span id="line-610"> return includeStopRow;</span>
<span class="source-line-no">611</span><span id="line-611"> }</span>
<span class="source-line-no">612</span><span id="line-612"></span>
<span class="source-line-no">613</span><span id="line-613"> /** Returns the max number of versions to fetch */</span>
<span class="source-line-no">614</span><span id="line-614"> public int getMaxVersions() {</span>
<span class="source-line-no">615</span><span id="line-615"> return this.maxVersions;</span>
<span class="source-line-no">616</span><span id="line-616"> }</span>
<span class="source-line-no">617</span><span id="line-617"></span>
<span class="source-line-no">618</span><span id="line-618"> /** Returns maximum number of values to return for a single call to next() */</span>
<span class="source-line-no">619</span><span id="line-619"> public int getBatch() {</span>
<span class="source-line-no">620</span><span id="line-620"> return this.batch;</span>
<span class="source-line-no">621</span><span id="line-621"> }</span>
<span class="source-line-no">622</span><span id="line-622"></span>
<span class="source-line-no">623</span><span id="line-623"> /** Returns maximum number of values to return per row per CF */</span>
<span class="source-line-no">624</span><span id="line-624"> public int getMaxResultsPerColumnFamily() {</span>
<span class="source-line-no">625</span><span id="line-625"> return this.storeLimit;</span>
<span class="source-line-no">626</span><span id="line-626"> }</span>
<span class="source-line-no">627</span><span id="line-627"></span>
<span class="source-line-no">628</span><span id="line-628"> /**</span>
<span class="source-line-no">629</span><span id="line-629"> * Method for retrieving the scan's offset per row per column family (#kvs to be skipped)</span>
<span class="source-line-no">630</span><span id="line-630"> * @return row offset</span>
<span class="source-line-no">631</span><span id="line-631"> */</span>
<span class="source-line-no">632</span><span id="line-632"> public int getRowOffsetPerColumnFamily() {</span>
<span class="source-line-no">633</span><span id="line-633"> return this.storeOffset;</span>
<span class="source-line-no">634</span><span id="line-634"> }</span>
<span class="source-line-no">635</span><span id="line-635"></span>
<span class="source-line-no">636</span><span id="line-636"> /** Returns caching the number of rows fetched when calling next on a scanner */</span>
<span class="source-line-no">637</span><span id="line-637"> public int getCaching() {</span>
<span class="source-line-no">638</span><span id="line-638"> return this.caching;</span>
<span class="source-line-no">639</span><span id="line-639"> }</span>
<span class="source-line-no">640</span><span id="line-640"></span>
<span class="source-line-no">641</span><span id="line-641"> /** Returns TimeRange */</span>
<span class="source-line-no">642</span><span id="line-642"> public TimeRange getTimeRange() {</span>
<span class="source-line-no">643</span><span id="line-643"> return this.tr;</span>
<span class="source-line-no">644</span><span id="line-644"> }</span>
<span class="source-line-no">645</span><span id="line-645"></span>
<span class="source-line-no">646</span><span id="line-646"> /** Returns RowFilter */</span>
<span class="source-line-no">647</span><span id="line-647"> @Override</span>
<span class="source-line-no">648</span><span id="line-648"> public Filter getFilter() {</span>
<span class="source-line-no">649</span><span id="line-649"> return filter;</span>
<span class="source-line-no">650</span><span id="line-650"> }</span>
<span class="source-line-no">651</span><span id="line-651"></span>
<span class="source-line-no">652</span><span id="line-652"> /** Returns true is a filter has been specified, false if not */</span>
<span class="source-line-no">653</span><span id="line-653"> public boolean hasFilter() {</span>
<span class="source-line-no">654</span><span id="line-654"> return filter != null;</span>
<span class="source-line-no">655</span><span id="line-655"> }</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"> * Set whether blocks should be cached for this Scan.</span>
<span class="source-line-no">659</span><span id="line-659"> * &lt;p&gt;</span>
<span class="source-line-no">660</span><span id="line-660"> * This is true by default. When true, default settings of the table and family are used (this</span>
<span class="source-line-no">661</span><span id="line-661"> * will never override caching blocks if the block cache is disabled for that family or entirely).</span>
<span class="source-line-no">662</span><span id="line-662"> * @param cacheBlocks if false, default settings are overridden and blocks will not be cached</span>
<span class="source-line-no">663</span><span id="line-663"> */</span>
<span class="source-line-no">664</span><span id="line-664"> public Scan setCacheBlocks(boolean cacheBlocks) {</span>
<span class="source-line-no">665</span><span id="line-665"> this.cacheBlocks = cacheBlocks;</span>
<span class="source-line-no">666</span><span id="line-666"> return this;</span>
<span class="source-line-no">667</span><span id="line-667"> }</span>
<span class="source-line-no">668</span><span id="line-668"></span>
<span class="source-line-no">669</span><span id="line-669"> /**</span>
<span class="source-line-no">670</span><span id="line-670"> * Get whether blocks should be cached for this Scan.</span>
<span class="source-line-no">671</span><span id="line-671"> * @return true if default caching should be used, false if blocks should not be cached</span>
<span class="source-line-no">672</span><span id="line-672"> */</span>
<span class="source-line-no">673</span><span id="line-673"> public boolean getCacheBlocks() {</span>
<span class="source-line-no">674</span><span id="line-674"> return cacheBlocks;</span>
<span class="source-line-no">675</span><span id="line-675"> }</span>
<span class="source-line-no">676</span><span id="line-676"></span>
<span class="source-line-no">677</span><span id="line-677"> /**</span>
<span class="source-line-no">678</span><span id="line-678"> * Set whether this scan is a reversed one</span>
<span class="source-line-no">679</span><span id="line-679"> * &lt;p&gt;</span>
<span class="source-line-no">680</span><span id="line-680"> * This is false by default which means forward(normal) scan.</span>
<span class="source-line-no">681</span><span id="line-681"> * @param reversed if true, scan will be backward order</span>
<span class="source-line-no">682</span><span id="line-682"> */</span>
<span class="source-line-no">683</span><span id="line-683"> public Scan setReversed(boolean reversed) {</span>
<span class="source-line-no">684</span><span id="line-684"> this.reversed = reversed;</span>
<span class="source-line-no">685</span><span id="line-685"> return this;</span>
<span class="source-line-no">686</span><span id="line-686"> }</span>
<span class="source-line-no">687</span><span id="line-687"></span>
<span class="source-line-no">688</span><span id="line-688"> /**</span>
<span class="source-line-no">689</span><span id="line-689"> * Get whether this scan is a reversed one.</span>
<span class="source-line-no">690</span><span id="line-690"> * @return true if backward scan, false if forward(default) scan</span>
<span class="source-line-no">691</span><span id="line-691"> */</span>
<span class="source-line-no">692</span><span id="line-692"> public boolean isReversed() {</span>
<span class="source-line-no">693</span><span id="line-693"> return reversed;</span>
<span class="source-line-no">694</span><span id="line-694"> }</span>
<span class="source-line-no">695</span><span id="line-695"></span>
<span class="source-line-no">696</span><span id="line-696"> /**</span>
<span class="source-line-no">697</span><span id="line-697"> * Setting whether the caller wants to see the partial results when server returns</span>
<span class="source-line-no">698</span><span id="line-698"> * less-than-expected cells. It is helpful while scanning a huge row to prevent OOM at client. By</span>
<span class="source-line-no">699</span><span id="line-699"> * default this value is false and the complete results will be assembled client side before being</span>
<span class="source-line-no">700</span><span id="line-700"> * delivered to the caller.</span>
<span class="source-line-no">701</span><span id="line-701"> * @see Result#mayHaveMoreCellsInRow()</span>
<span class="source-line-no">702</span><span id="line-702"> * @see #setBatch(int)</span>
<span class="source-line-no">703</span><span id="line-703"> */</span>
<span class="source-line-no">704</span><span id="line-704"> public Scan setAllowPartialResults(final boolean allowPartialResults) {</span>
<span class="source-line-no">705</span><span id="line-705"> this.allowPartialResults = allowPartialResults;</span>
<span class="source-line-no">706</span><span id="line-706"> return this;</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"> * Returns true when the constructor of this scan understands that the results they will see may</span>
<span class="source-line-no">711</span><span id="line-711"> * only represent a partial portion of a row. The entire row would be retrieved by subsequent</span>
<span class="source-line-no">712</span><span id="line-712"> * calls to {@link ResultScanner#next()}</span>
<span class="source-line-no">713</span><span id="line-713"> */</span>
<span class="source-line-no">714</span><span id="line-714"> public boolean getAllowPartialResults() {</span>
<span class="source-line-no">715</span><span id="line-715"> return allowPartialResults;</span>
<span class="source-line-no">716</span><span id="line-716"> }</span>
<span class="source-line-no">717</span><span id="line-717"></span>
<span class="source-line-no">718</span><span id="line-718"> @Override</span>
<span class="source-line-no">719</span><span id="line-719"> public Scan setLoadColumnFamiliesOnDemand(boolean value) {</span>
<span class="source-line-no">720</span><span id="line-720"> return (Scan) super.setLoadColumnFamiliesOnDemand(value);</span>
<span class="source-line-no">721</span><span id="line-721"> }</span>
<span class="source-line-no">722</span><span id="line-722"></span>
<span class="source-line-no">723</span><span id="line-723"> /**</span>
<span class="source-line-no">724</span><span id="line-724"> * Compile the table and column family (i.e. schema) information into a String. Useful for parsing</span>
<span class="source-line-no">725</span><span id="line-725"> * and aggregation by debugging, logging, and administration tools.</span>
<span class="source-line-no">726</span><span id="line-726"> */</span>
<span class="source-line-no">727</span><span id="line-727"> @Override</span>
<span class="source-line-no">728</span><span id="line-728"> public Map&lt;String, Object&gt; getFingerprint() {</span>
<span class="source-line-no">729</span><span id="line-729"> Map&lt;String, Object&gt; map = new HashMap&lt;&gt;();</span>
<span class="source-line-no">730</span><span id="line-730"> List&lt;String&gt; families = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">731</span><span id="line-731"> if (this.familyMap.isEmpty()) {</span>
<span class="source-line-no">732</span><span id="line-732"> map.put("families", "ALL");</span>
<span class="source-line-no">733</span><span id="line-733"> return map;</span>
<span class="source-line-no">734</span><span id="line-734"> } else {</span>
<span class="source-line-no">735</span><span id="line-735"> map.put("families", families);</span>
<span class="source-line-no">736</span><span id="line-736"> }</span>
<span class="source-line-no">737</span><span id="line-737"> for (Map.Entry&lt;byte[], NavigableSet&lt;byte[]&gt;&gt; entry : this.familyMap.entrySet()) {</span>
<span class="source-line-no">738</span><span id="line-738"> families.add(Bytes.toStringBinary(entry.getKey()));</span>
<span class="source-line-no">739</span><span id="line-739"> }</span>
<span class="source-line-no">740</span><span id="line-740"> return map;</span>
<span class="source-line-no">741</span><span id="line-741"> }</span>
<span class="source-line-no">742</span><span id="line-742"></span>
<span class="source-line-no">743</span><span id="line-743"> /**</span>
<span class="source-line-no">744</span><span id="line-744"> * Compile the details beyond the scope of getFingerprint (row, columns, timestamps, etc.) into a</span>
<span class="source-line-no">745</span><span id="line-745"> * Map along with the fingerprinted information. Useful for debugging, logging, and administration</span>
<span class="source-line-no">746</span><span id="line-746"> * tools.</span>
<span class="source-line-no">747</span><span id="line-747"> * @param maxCols a limit on the number of columns output prior to truncation</span>
<span class="source-line-no">748</span><span id="line-748"> */</span>
<span class="source-line-no">749</span><span id="line-749"> @Override</span>
<span class="source-line-no">750</span><span id="line-750"> public Map&lt;String, Object&gt; toMap(int maxCols) {</span>
<span class="source-line-no">751</span><span id="line-751"> // start with the fingerprint map and build on top of it</span>
<span class="source-line-no">752</span><span id="line-752"> Map&lt;String, Object&gt; map = getFingerprint();</span>
<span class="source-line-no">753</span><span id="line-753"> // map from families to column list replaces fingerprint's list of families</span>
<span class="source-line-no">754</span><span id="line-754"> Map&lt;String, List&lt;String&gt;&gt; familyColumns = new HashMap&lt;&gt;();</span>
<span class="source-line-no">755</span><span id="line-755"> map.put("families", familyColumns);</span>
<span class="source-line-no">756</span><span id="line-756"> // add scalar information first</span>
<span class="source-line-no">757</span><span id="line-757"> map.put("startRow", Bytes.toStringBinary(this.startRow));</span>
<span class="source-line-no">758</span><span id="line-758"> map.put("stopRow", Bytes.toStringBinary(this.stopRow));</span>
<span class="source-line-no">759</span><span id="line-759"> map.put("maxVersions", this.maxVersions);</span>
<span class="source-line-no">760</span><span id="line-760"> map.put("batch", this.batch);</span>
<span class="source-line-no">761</span><span id="line-761"> map.put("caching", this.caching);</span>
<span class="source-line-no">762</span><span id="line-762"> map.put("maxResultSize", this.maxResultSize);</span>
<span class="source-line-no">763</span><span id="line-763"> map.put("cacheBlocks", this.cacheBlocks);</span>
<span class="source-line-no">764</span><span id="line-764"> map.put("loadColumnFamiliesOnDemand", this.loadColumnFamiliesOnDemand);</span>
<span class="source-line-no">765</span><span id="line-765"> List&lt;Long&gt; timeRange = new ArrayList&lt;&gt;(2);</span>
<span class="source-line-no">766</span><span id="line-766"> timeRange.add(this.tr.getMin());</span>
<span class="source-line-no">767</span><span id="line-767"> timeRange.add(this.tr.getMax());</span>
<span class="source-line-no">768</span><span id="line-768"> map.put("timeRange", timeRange);</span>
<span class="source-line-no">769</span><span id="line-769"> int colCount = 0;</span>
<span class="source-line-no">770</span><span id="line-770"> // iterate through affected families and list out up to maxCols columns</span>
<span class="source-line-no">771</span><span id="line-771"> for (Map.Entry&lt;byte[], NavigableSet&lt;byte[]&gt;&gt; entry : this.familyMap.entrySet()) {</span>
<span class="source-line-no">772</span><span id="line-772"> List&lt;String&gt; columns = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">773</span><span id="line-773"> familyColumns.put(Bytes.toStringBinary(entry.getKey()), columns);</span>
<span class="source-line-no">774</span><span id="line-774"> if (entry.getValue() == null) {</span>
<span class="source-line-no">775</span><span id="line-775"> colCount++;</span>
<span class="source-line-no">776</span><span id="line-776"> --maxCols;</span>
<span class="source-line-no">777</span><span id="line-777"> columns.add("ALL");</span>
<span class="source-line-no">778</span><span id="line-778"> } else {</span>
<span class="source-line-no">779</span><span id="line-779"> colCount += entry.getValue().size();</span>
<span class="source-line-no">780</span><span id="line-780"> if (maxCols &lt;= 0) {</span>
<span class="source-line-no">781</span><span id="line-781"> continue;</span>
<span class="source-line-no">782</span><span id="line-782"> }</span>
<span class="source-line-no">783</span><span id="line-783"> for (byte[] column : entry.getValue()) {</span>
<span class="source-line-no">784</span><span id="line-784"> if (--maxCols &lt;= 0) {</span>
<span class="source-line-no">785</span><span id="line-785"> continue;</span>
<span class="source-line-no">786</span><span id="line-786"> }</span>
<span class="source-line-no">787</span><span id="line-787"> columns.add(Bytes.toStringBinary(column));</span>
<span class="source-line-no">788</span><span id="line-788"> }</span>
<span class="source-line-no">789</span><span id="line-789"> }</span>
<span class="source-line-no">790</span><span id="line-790"> }</span>
<span class="source-line-no">791</span><span id="line-791"> map.put("totalColumns", colCount);</span>
<span class="source-line-no">792</span><span id="line-792"> if (this.filter != null) {</span>
<span class="source-line-no">793</span><span id="line-793"> map.put("filter", this.filter.toString());</span>
<span class="source-line-no">794</span><span id="line-794"> }</span>
<span class="source-line-no">795</span><span id="line-795"> // add the id if set</span>
<span class="source-line-no">796</span><span id="line-796"> if (getId() != null) {</span>
<span class="source-line-no">797</span><span id="line-797"> map.put("id", getId());</span>
<span class="source-line-no">798</span><span id="line-798"> }</span>
<span class="source-line-no">799</span><span id="line-799"> map.put("includeStartRow", includeStartRow);</span>
<span class="source-line-no">800</span><span id="line-800"> map.put("includeStopRow", includeStopRow);</span>
<span class="source-line-no">801</span><span id="line-801"> map.put("allowPartialResults", allowPartialResults);</span>
<span class="source-line-no">802</span><span id="line-802"> map.put("storeLimit", storeLimit);</span>
<span class="source-line-no">803</span><span id="line-803"> map.put("storeOffset", storeOffset);</span>
<span class="source-line-no">804</span><span id="line-804"> map.put("reversed", reversed);</span>
<span class="source-line-no">805</span><span id="line-805"> if (null != asyncPrefetch) {</span>
<span class="source-line-no">806</span><span id="line-806"> map.put("asyncPrefetch", asyncPrefetch);</span>
<span class="source-line-no">807</span><span id="line-807"> }</span>
<span class="source-line-no">808</span><span id="line-808"> map.put("mvccReadPoint", mvccReadPoint);</span>
<span class="source-line-no">809</span><span id="line-809"> map.put("limit", limit);</span>
<span class="source-line-no">810</span><span id="line-810"> map.put("readType", readType);</span>
<span class="source-line-no">811</span><span id="line-811"> map.put("needCursorResult", needCursorResult);</span>
<span class="source-line-no">812</span><span id="line-812"> map.put("targetReplicaId", targetReplicaId);</span>
<span class="source-line-no">813</span><span id="line-813"> map.put("consistency", consistency);</span>
<span class="source-line-no">814</span><span id="line-814"> if (!colFamTimeRangeMap.isEmpty()) {</span>
<span class="source-line-no">815</span><span id="line-815"> Map&lt;String, List&lt;Long&gt;&gt; colFamTimeRangeMapStr = colFamTimeRangeMap.entrySet().stream()</span>
<span class="source-line-no">816</span><span id="line-816"> .collect(Collectors.toMap((e) -&gt; Bytes.toStringBinary(e.getKey()), e -&gt; {</span>
<span class="source-line-no">817</span><span id="line-817"> TimeRange value = e.getValue();</span>
<span class="source-line-no">818</span><span id="line-818"> List&lt;Long&gt; rangeList = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">819</span><span id="line-819"> rangeList.add(value.getMin());</span>
<span class="source-line-no">820</span><span id="line-820"> rangeList.add(value.getMax());</span>
<span class="source-line-no">821</span><span id="line-821"> return rangeList;</span>
<span class="source-line-no">822</span><span id="line-822"> }));</span>
<span class="source-line-no">823</span><span id="line-823"></span>
<span class="source-line-no">824</span><span id="line-824"> map.put("colFamTimeRangeMap", colFamTimeRangeMapStr);</span>
<span class="source-line-no">825</span><span id="line-825"> }</span>
<span class="source-line-no">826</span><span id="line-826"> map.put("priority", getPriority());</span>
<span class="source-line-no">827</span><span id="line-827"> return map;</span>
<span class="source-line-no">828</span><span id="line-828"> }</span>
<span class="source-line-no">829</span><span id="line-829"></span>
<span class="source-line-no">830</span><span id="line-830"> /**</span>
<span class="source-line-no">831</span><span id="line-831"> * Enable/disable "raw" mode for this scan. If "raw" is enabled the scan will return all delete</span>
<span class="source-line-no">832</span><span id="line-832"> * marker and deleted rows that have not been collected, yet. This is mostly useful for Scan on</span>
<span class="source-line-no">833</span><span id="line-833"> * column families that have KEEP_DELETED_ROWS enabled. It is an error to specify any column when</span>
<span class="source-line-no">834</span><span id="line-834"> * "raw" is set.</span>
<span class="source-line-no">835</span><span id="line-835"> * @param raw True/False to enable/disable "raw" mode.</span>
<span class="source-line-no">836</span><span id="line-836"> */</span>
<span class="source-line-no">837</span><span id="line-837"> public Scan setRaw(boolean raw) {</span>
<span class="source-line-no">838</span><span id="line-838"> setAttribute(RAW_ATTR, Bytes.toBytes(raw));</span>
<span class="source-line-no">839</span><span id="line-839"> return this;</span>
<span class="source-line-no">840</span><span id="line-840"> }</span>
<span class="source-line-no">841</span><span id="line-841"></span>
<span class="source-line-no">842</span><span id="line-842"> /** Returns True if this Scan is in "raw" mode. */</span>
<span class="source-line-no">843</span><span id="line-843"> public boolean isRaw() {</span>
<span class="source-line-no">844</span><span id="line-844"> byte[] attr = getAttribute(RAW_ATTR);</span>
<span class="source-line-no">845</span><span id="line-845"> return attr == null ? false : Bytes.toBoolean(attr);</span>
<span class="source-line-no">846</span><span id="line-846"> }</span>
<span class="source-line-no">847</span><span id="line-847"></span>
<span class="source-line-no">848</span><span id="line-848"> @Override</span>
<span class="source-line-no">849</span><span id="line-849"> public Scan setAttribute(String name, byte[] value) {</span>
<span class="source-line-no">850</span><span id="line-850"> return (Scan) super.setAttribute(name, value);</span>
<span class="source-line-no">851</span><span id="line-851"> }</span>
<span class="source-line-no">852</span><span id="line-852"></span>
<span class="source-line-no">853</span><span id="line-853"> @Override</span>
<span class="source-line-no">854</span><span id="line-854"> public Scan setId(String id) {</span>
<span class="source-line-no">855</span><span id="line-855"> return (Scan) super.setId(id);</span>
<span class="source-line-no">856</span><span id="line-856"> }</span>
<span class="source-line-no">857</span><span id="line-857"></span>
<span class="source-line-no">858</span><span id="line-858"> @Override</span>
<span class="source-line-no">859</span><span id="line-859"> public Scan setAuthorizations(Authorizations authorizations) {</span>
<span class="source-line-no">860</span><span id="line-860"> return (Scan) super.setAuthorizations(authorizations);</span>
<span class="source-line-no">861</span><span id="line-861"> }</span>
<span class="source-line-no">862</span><span id="line-862"></span>
<span class="source-line-no">863</span><span id="line-863"> @Override</span>
<span class="source-line-no">864</span><span id="line-864"> public Scan setACL(Map&lt;String, Permission&gt; perms) {</span>
<span class="source-line-no">865</span><span id="line-865"> return (Scan) super.setACL(perms);</span>
<span class="source-line-no">866</span><span id="line-866"> }</span>
<span class="source-line-no">867</span><span id="line-867"></span>
<span class="source-line-no">868</span><span id="line-868"> @Override</span>
<span class="source-line-no">869</span><span id="line-869"> public Scan setACL(String user, Permission perms) {</span>
<span class="source-line-no">870</span><span id="line-870"> return (Scan) super.setACL(user, perms);</span>
<span class="source-line-no">871</span><span id="line-871"> }</span>
<span class="source-line-no">872</span><span id="line-872"></span>
<span class="source-line-no">873</span><span id="line-873"> @Override</span>
<span class="source-line-no">874</span><span id="line-874"> public Scan setConsistency(Consistency consistency) {</span>
<span class="source-line-no">875</span><span id="line-875"> return (Scan) super.setConsistency(consistency);</span>
<span class="source-line-no">876</span><span id="line-876"> }</span>
<span class="source-line-no">877</span><span id="line-877"></span>
<span class="source-line-no">878</span><span id="line-878"> @Override</span>
<span class="source-line-no">879</span><span id="line-879"> public Scan setReplicaId(int Id) {</span>
<span class="source-line-no">880</span><span id="line-880"> return (Scan) super.setReplicaId(Id);</span>
<span class="source-line-no">881</span><span id="line-881"> }</span>
<span class="source-line-no">882</span><span id="line-882"></span>
<span class="source-line-no">883</span><span id="line-883"> @Override</span>
<span class="source-line-no">884</span><span id="line-884"> public Scan setIsolationLevel(IsolationLevel level) {</span>
<span class="source-line-no">885</span><span id="line-885"> return (Scan) super.setIsolationLevel(level);</span>
<span class="source-line-no">886</span><span id="line-886"> }</span>
<span class="source-line-no">887</span><span id="line-887"></span>
<span class="source-line-no">888</span><span id="line-888"> @Override</span>
<span class="source-line-no">889</span><span id="line-889"> public Scan setPriority(int priority) {</span>
<span class="source-line-no">890</span><span id="line-890"> return (Scan) super.setPriority(priority);</span>
<span class="source-line-no">891</span><span id="line-891"> }</span>
<span class="source-line-no">892</span><span id="line-892"></span>
<span class="source-line-no">893</span><span id="line-893"> /**</span>
<span class="source-line-no">894</span><span id="line-894"> * Enable collection of {@link ScanMetrics}. For advanced users.</span>
<span class="source-line-no">895</span><span id="line-895"> * @param enabled Set to true to enable accumulating scan metrics</span>
<span class="source-line-no">896</span><span id="line-896"> */</span>
<span class="source-line-no">897</span><span id="line-897"> public Scan setScanMetricsEnabled(final boolean enabled) {</span>
<span class="source-line-no">898</span><span id="line-898"> setAttribute(Scan.SCAN_ATTRIBUTES_METRICS_ENABLE, Bytes.toBytes(Boolean.valueOf(enabled)));</span>
<span class="source-line-no">899</span><span id="line-899"> return this;</span>
<span class="source-line-no">900</span><span id="line-900"> }</span>
<span class="source-line-no">901</span><span id="line-901"></span>
<span class="source-line-no">902</span><span id="line-902"> /** Returns True if collection of scan metrics is enabled. For advanced users. */</span>
<span class="source-line-no">903</span><span id="line-903"> public boolean isScanMetricsEnabled() {</span>
<span class="source-line-no">904</span><span id="line-904"> byte[] attr = getAttribute(Scan.SCAN_ATTRIBUTES_METRICS_ENABLE);</span>
<span class="source-line-no">905</span><span id="line-905"> return attr == null ? false : Bytes.toBoolean(attr);</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"> public Boolean isAsyncPrefetch() {</span>
<span class="source-line-no">909</span><span id="line-909"> return asyncPrefetch;</span>
<span class="source-line-no">910</span><span id="line-910"> }</span>
<span class="source-line-no">911</span><span id="line-911"></span>
<span class="source-line-no">912</span><span id="line-912"> /**</span>
<span class="source-line-no">913</span><span id="line-913"> * @deprecated Since 3.0.0, will be removed in 4.0.0. After building sync client upon async</span>
<span class="source-line-no">914</span><span id="line-914"> * client, the implementation is always 'async prefetch', so this flag is useless now.</span>
<span class="source-line-no">915</span><span id="line-915"> */</span>
<span class="source-line-no">916</span><span id="line-916"> @Deprecated</span>
<span class="source-line-no">917</span><span id="line-917"> public Scan setAsyncPrefetch(boolean asyncPrefetch) {</span>
<span class="source-line-no">918</span><span id="line-918"> this.asyncPrefetch = asyncPrefetch;</span>
<span class="source-line-no">919</span><span id="line-919"> return this;</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"> /** Returns the limit of rows for this scan */</span>
<span class="source-line-no">923</span><span id="line-923"> public int getLimit() {</span>
<span class="source-line-no">924</span><span id="line-924"> return limit;</span>
<span class="source-line-no">925</span><span id="line-925"> }</span>
<span class="source-line-no">926</span><span id="line-926"></span>
<span class="source-line-no">927</span><span id="line-927"> /**</span>
<span class="source-line-no">928</span><span id="line-928"> * Set the limit of rows for this scan. We will terminate the scan if the number of returned rows</span>
<span class="source-line-no">929</span><span id="line-929"> * reaches this value.</span>
<span class="source-line-no">930</span><span id="line-930"> * &lt;p&gt;</span>
<span class="source-line-no">931</span><span id="line-931"> * This condition will be tested at last, after all other conditions such as stopRow, filter, etc.</span>
<span class="source-line-no">932</span><span id="line-932"> * @param limit the limit of rows for this scan</span>
<span class="source-line-no">933</span><span id="line-933"> */</span>
<span class="source-line-no">934</span><span id="line-934"> public Scan setLimit(int limit) {</span>
<span class="source-line-no">935</span><span id="line-935"> this.limit = limit;</span>
<span class="source-line-no">936</span><span id="line-936"> return this;</span>
<span class="source-line-no">937</span><span id="line-937"> }</span>
<span class="source-line-no">938</span><span id="line-938"></span>
<span class="source-line-no">939</span><span id="line-939"> /**</span>
<span class="source-line-no">940</span><span id="line-940"> * Call this when you only want to get one row. It will set {@code limit} to {@code 1}, and also</span>
<span class="source-line-no">941</span><span id="line-941"> * set {@code readType} to {@link ReadType#PREAD}.</span>
<span class="source-line-no">942</span><span id="line-942"> */</span>
<span class="source-line-no">943</span><span id="line-943"> public Scan setOneRowLimit() {</span>
<span class="source-line-no">944</span><span id="line-944"> return setLimit(1).setReadType(ReadType.PREAD);</span>
<span class="source-line-no">945</span><span id="line-945"> }</span>
<span class="source-line-no">946</span><span id="line-946"></span>
<span class="source-line-no">947</span><span id="line-947"> @InterfaceAudience.Public</span>
<span class="source-line-no">948</span><span id="line-948"> public enum ReadType {</span>
<span class="source-line-no">949</span><span id="line-949"> DEFAULT,</span>
<span class="source-line-no">950</span><span id="line-950"> STREAM,</span>
<span class="source-line-no">951</span><span id="line-951"> PREAD</span>
<span class="source-line-no">952</span><span id="line-952"> }</span>
<span class="source-line-no">953</span><span id="line-953"></span>
<span class="source-line-no">954</span><span id="line-954"> /** Returns the read type for this scan */</span>
<span class="source-line-no">955</span><span id="line-955"> public ReadType getReadType() {</span>
<span class="source-line-no">956</span><span id="line-956"> return readType;</span>
<span class="source-line-no">957</span><span id="line-957"> }</span>
<span class="source-line-no">958</span><span id="line-958"></span>
<span class="source-line-no">959</span><span id="line-959"> /**</span>
<span class="source-line-no">960</span><span id="line-960"> * Set the read type for this scan.</span>
<span class="source-line-no">961</span><span id="line-961"> * &lt;p&gt;</span>
<span class="source-line-no">962</span><span id="line-962"> * Notice that we may choose to use pread even if you specific {@link ReadType#STREAM} here. For</span>
<span class="source-line-no">963</span><span id="line-963"> * example, we will always use pread if this is a get scan.</span>
<span class="source-line-no">964</span><span id="line-964"> */</span>
<span class="source-line-no">965</span><span id="line-965"> public Scan setReadType(ReadType readType) {</span>
<span class="source-line-no">966</span><span id="line-966"> this.readType = readType;</span>
<span class="source-line-no">967</span><span id="line-967"> return this;</span>
<span class="source-line-no">968</span><span id="line-968"> }</span>
<span class="source-line-no">969</span><span id="line-969"></span>
<span class="source-line-no">970</span><span id="line-970"> /**</span>
<span class="source-line-no">971</span><span id="line-971"> * Get the mvcc read point used to open a scanner.</span>
<span class="source-line-no">972</span><span id="line-972"> */</span>
<span class="source-line-no">973</span><span id="line-973"> long getMvccReadPoint() {</span>
<span class="source-line-no">974</span><span id="line-974"> return mvccReadPoint;</span>
<span class="source-line-no">975</span><span id="line-975"> }</span>
<span class="source-line-no">976</span><span id="line-976"></span>
<span class="source-line-no">977</span><span id="line-977"> /**</span>
<span class="source-line-no">978</span><span id="line-978"> * Set the mvcc read point used to open a scanner.</span>
<span class="source-line-no">979</span><span id="line-979"> */</span>
<span class="source-line-no">980</span><span id="line-980"> Scan setMvccReadPoint(long mvccReadPoint) {</span>
<span class="source-line-no">981</span><span id="line-981"> this.mvccReadPoint = mvccReadPoint;</span>
<span class="source-line-no">982</span><span id="line-982"> return this;</span>
<span class="source-line-no">983</span><span id="line-983"> }</span>
<span class="source-line-no">984</span><span id="line-984"></span>
<span class="source-line-no">985</span><span id="line-985"> /**</span>
<span class="source-line-no">986</span><span id="line-986"> * Set the mvcc read point to -1 which means do not use it.</span>
<span class="source-line-no">987</span><span id="line-987"> */</span>
<span class="source-line-no">988</span><span id="line-988"> Scan resetMvccReadPoint() {</span>
<span class="source-line-no">989</span><span id="line-989"> return setMvccReadPoint(-1L);</span>
<span class="source-line-no">990</span><span id="line-990"> }</span>
<span class="source-line-no">991</span><span id="line-991"></span>
<span class="source-line-no">992</span><span id="line-992"> /**</span>
<span class="source-line-no">993</span><span id="line-993"> * When the server is slow or we scan a table with many deleted data or we use a sparse filter,</span>
<span class="source-line-no">994</span><span id="line-994"> * the server will response heartbeat to prevent timeout. However the scanner will return a Result</span>
<span class="source-line-no">995</span><span id="line-995"> * only when client can do it. So if there are many heartbeats, the blocking time on</span>
<span class="source-line-no">996</span><span id="line-996"> * ResultScanner#next() may be very long, which is not friendly to online services. Set this to</span>
<span class="source-line-no">997</span><span id="line-997"> * true then you can get a special Result whose #isCursor() returns true and is not contains any</span>
<span class="source-line-no">998</span><span id="line-998"> * real data. It only tells you where the server has scanned. You can call next to continue</span>
<span class="source-line-no">999</span><span id="line-999"> * scanning or open a new scanner with this row key as start row whenever you want. Users can get</span>
<span class="source-line-no">1000</span><span id="line-1000"> * a cursor when and only when there is a response from the server but we can not return a Result</span>
<span class="source-line-no">1001</span><span id="line-1001"> * to users, for example, this response is a heartbeat or there are partial cells but users do not</span>
<span class="source-line-no">1002</span><span id="line-1002"> * allow partial result. Now the cursor is in row level which means the special Result will only</span>
<span class="source-line-no">1003</span><span id="line-1003"> * contains a row key. {@link Result#isCursor()} {@link Result#getCursor()} {@link Cursor}</span>
<span class="source-line-no">1004</span><span id="line-1004"> */</span>
<span class="source-line-no">1005</span><span id="line-1005"> public Scan setNeedCursorResult(boolean needCursorResult) {</span>
<span class="source-line-no">1006</span><span id="line-1006"> this.needCursorResult = needCursorResult;</span>
<span class="source-line-no">1007</span><span id="line-1007"> return this;</span>
<span class="source-line-no">1008</span><span id="line-1008"> }</span>
<span class="source-line-no">1009</span><span id="line-1009"></span>
<span class="source-line-no">1010</span><span id="line-1010"> public boolean isNeedCursorResult() {</span>
<span class="source-line-no">1011</span><span id="line-1011"> return needCursorResult;</span>
<span class="source-line-no">1012</span><span id="line-1012"> }</span>
<span class="source-line-no">1013</span><span id="line-1013"></span>
<span class="source-line-no">1014</span><span id="line-1014"> /**</span>
<span class="source-line-no">1015</span><span id="line-1015"> * Create a new Scan with a cursor. It only set the position information like start row key. The</span>
<span class="source-line-no">1016</span><span id="line-1016"> * others (like cfs, stop row, limit) should still be filled in by the user.</span>
<span class="source-line-no">1017</span><span id="line-1017"> * {@link Result#isCursor()} {@link Result#getCursor()} {@link Cursor}</span>
<span class="source-line-no">1018</span><span id="line-1018"> */</span>
<span class="source-line-no">1019</span><span id="line-1019"> public static Scan createScanFromCursor(Cursor cursor) {</span>
<span class="source-line-no">1020</span><span id="line-1020"> return new Scan().withStartRow(cursor.getRow());</span>
<span class="source-line-no">1021</span><span id="line-1021"> }</span>
<span class="source-line-no">1022</span><span id="line-1022">}</span>
</pre>
</div>
</main>
</body>
</html>