blob: d3242bd4a7aafa68a2c096bece6cac9bb409ff11 [file] [log] [blame]
<!DOCTYPE HTML>
<html lang="en">
<head>
<!-- Generated by javadoc (17) -->
<title>Source code</title>
<meta name="viewport" content="width=device-width, initial-scale=1">
<meta name="description" content="source: package: org.apache.hadoop.hbase.regionserver, class: StoreFileReader">
<meta name="generator" content="javadoc/SourceToHTMLConverter">
<link rel="stylesheet" type="text/css" href="../../../../../../stylesheet.css" title="Style">
</head>
<body class="source-page">
<main role="main">
<div class="source-container">
<pre><span class="source-line-no">001</span><span id="line-1">/*</span>
<span class="source-line-no">002</span><span id="line-2"> * Licensed to the Apache Software Foundation (ASF) under one</span>
<span class="source-line-no">003</span><span id="line-3"> * or more contributor license agreements. See the NOTICE file</span>
<span class="source-line-no">004</span><span id="line-4"> * distributed with this work for additional information</span>
<span class="source-line-no">005</span><span id="line-5"> * regarding copyright ownership. The ASF licenses this file</span>
<span class="source-line-no">006</span><span id="line-6"> * to you under the Apache License, Version 2.0 (the</span>
<span class="source-line-no">007</span><span id="line-7"> * "License"); you may not use this file except in compliance</span>
<span class="source-line-no">008</span><span id="line-8"> * with the License. You may obtain a copy of the License at</span>
<span class="source-line-no">009</span><span id="line-9"> *</span>
<span class="source-line-no">010</span><span id="line-10"> * http://www.apache.org/licenses/LICENSE-2.0</span>
<span class="source-line-no">011</span><span id="line-11"> *</span>
<span class="source-line-no">012</span><span id="line-12"> * Unless required by applicable law or agreed to in writing, software</span>
<span class="source-line-no">013</span><span id="line-13"> * distributed under the License is distributed on an "AS IS" BASIS,</span>
<span class="source-line-no">014</span><span id="line-14"> * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.</span>
<span class="source-line-no">015</span><span id="line-15"> * See the License for the specific language governing permissions and</span>
<span class="source-line-no">016</span><span id="line-16"> * limitations under the License.</span>
<span class="source-line-no">017</span><span id="line-17"> */</span>
<span class="source-line-no">018</span><span id="line-18">package org.apache.hadoop.hbase.regionserver;</span>
<span class="source-line-no">019</span><span id="line-19"></span>
<span class="source-line-no">020</span><span id="line-20">import static org.apache.hadoop.hbase.regionserver.HStoreFile.BLOOM_FILTER_PARAM_KEY;</span>
<span class="source-line-no">021</span><span id="line-21">import static org.apache.hadoop.hbase.regionserver.HStoreFile.BLOOM_FILTER_TYPE_KEY;</span>
<span class="source-line-no">022</span><span id="line-22">import static org.apache.hadoop.hbase.regionserver.HStoreFile.DELETE_FAMILY_COUNT;</span>
<span class="source-line-no">023</span><span id="line-23">import static org.apache.hadoop.hbase.regionserver.HStoreFile.LAST_BLOOM_KEY;</span>
<span class="source-line-no">024</span><span id="line-24"></span>
<span class="source-line-no">025</span><span id="line-25">import com.google.errorprone.annotations.RestrictedApi;</span>
<span class="source-line-no">026</span><span id="line-26">import java.io.DataInput;</span>
<span class="source-line-no">027</span><span id="line-27">import java.io.IOException;</span>
<span class="source-line-no">028</span><span id="line-28">import java.util.Map;</span>
<span class="source-line-no">029</span><span id="line-29">import java.util.Optional;</span>
<span class="source-line-no">030</span><span id="line-30">import java.util.SortedSet;</span>
<span class="source-line-no">031</span><span id="line-31">import org.apache.hadoop.conf.Configuration;</span>
<span class="source-line-no">032</span><span id="line-32">import org.apache.hadoop.hbase.Cell;</span>
<span class="source-line-no">033</span><span id="line-33">import org.apache.hadoop.hbase.CellComparator;</span>
<span class="source-line-no">034</span><span id="line-34">import org.apache.hadoop.hbase.ExtendedCell;</span>
<span class="source-line-no">035</span><span id="line-35">import org.apache.hadoop.hbase.HBaseInterfaceAudience;</span>
<span class="source-line-no">036</span><span id="line-36">import org.apache.hadoop.hbase.HConstants;</span>
<span class="source-line-no">037</span><span id="line-37">import org.apache.hadoop.hbase.KeyValue;</span>
<span class="source-line-no">038</span><span id="line-38">import org.apache.hadoop.hbase.PrivateCellUtil;</span>
<span class="source-line-no">039</span><span id="line-39">import org.apache.hadoop.hbase.client.Scan;</span>
<span class="source-line-no">040</span><span id="line-40">import org.apache.hadoop.hbase.io.TimeRange;</span>
<span class="source-line-no">041</span><span id="line-41">import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;</span>
<span class="source-line-no">042</span><span id="line-42">import org.apache.hadoop.hbase.io.hfile.BlockType;</span>
<span class="source-line-no">043</span><span id="line-43">import org.apache.hadoop.hbase.io.hfile.BloomFilterMetrics;</span>
<span class="source-line-no">044</span><span id="line-44">import org.apache.hadoop.hbase.io.hfile.CacheConfig;</span>
<span class="source-line-no">045</span><span id="line-45">import org.apache.hadoop.hbase.io.hfile.HFile;</span>
<span class="source-line-no">046</span><span id="line-46">import org.apache.hadoop.hbase.io.hfile.HFileBlock;</span>
<span class="source-line-no">047</span><span id="line-47">import org.apache.hadoop.hbase.io.hfile.HFileInfo;</span>
<span class="source-line-no">048</span><span id="line-48">import org.apache.hadoop.hbase.io.hfile.HFileScanner;</span>
<span class="source-line-no">049</span><span id="line-49">import org.apache.hadoop.hbase.io.hfile.ReaderContext;</span>
<span class="source-line-no">050</span><span id="line-50">import org.apache.hadoop.hbase.io.hfile.ReaderContext.ReaderType;</span>
<span class="source-line-no">051</span><span id="line-51">import org.apache.hadoop.hbase.nio.ByteBuff;</span>
<span class="source-line-no">052</span><span id="line-52">import org.apache.hadoop.hbase.util.BloomFilter;</span>
<span class="source-line-no">053</span><span id="line-53">import org.apache.hadoop.hbase.util.BloomFilterFactory;</span>
<span class="source-line-no">054</span><span id="line-54">import org.apache.hadoop.hbase.util.Bytes;</span>
<span class="source-line-no">055</span><span id="line-55">import org.apache.yetus.audience.InterfaceAudience;</span>
<span class="source-line-no">056</span><span id="line-56">import org.apache.yetus.audience.InterfaceStability;</span>
<span class="source-line-no">057</span><span id="line-57">import org.slf4j.Logger;</span>
<span class="source-line-no">058</span><span id="line-58">import org.slf4j.LoggerFactory;</span>
<span class="source-line-no">059</span><span id="line-59"></span>
<span class="source-line-no">060</span><span id="line-60">/**</span>
<span class="source-line-no">061</span><span id="line-61"> * Reader for a StoreFile.</span>
<span class="source-line-no">062</span><span id="line-62"> */</span>
<span class="source-line-no">063</span><span id="line-63">@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.PHOENIX)</span>
<span class="source-line-no">064</span><span id="line-64">@InterfaceStability.Evolving</span>
<span class="source-line-no">065</span><span id="line-65">public class StoreFileReader {</span>
<span class="source-line-no">066</span><span id="line-66"> private static final Logger LOG = LoggerFactory.getLogger(StoreFileReader.class.getName());</span>
<span class="source-line-no">067</span><span id="line-67"></span>
<span class="source-line-no">068</span><span id="line-68"> protected BloomFilter generalBloomFilter = null;</span>
<span class="source-line-no">069</span><span id="line-69"> protected BloomFilter deleteFamilyBloomFilter = null;</span>
<span class="source-line-no">070</span><span id="line-70"> private BloomFilterMetrics bloomFilterMetrics = null;</span>
<span class="source-line-no">071</span><span id="line-71"> protected BloomType bloomFilterType;</span>
<span class="source-line-no">072</span><span id="line-72"> protected final HFile.Reader reader;</span>
<span class="source-line-no">073</span><span id="line-73"> protected long sequenceID = -1;</span>
<span class="source-line-no">074</span><span id="line-74"> protected TimeRange timeRange = null;</span>
<span class="source-line-no">075</span><span id="line-75"> private byte[] lastBloomKey;</span>
<span class="source-line-no">076</span><span id="line-76"> private long deleteFamilyCnt = -1;</span>
<span class="source-line-no">077</span><span id="line-77"> private boolean bulkLoadResult = false;</span>
<span class="source-line-no">078</span><span id="line-78"> private KeyValue.KeyOnlyKeyValue lastBloomKeyOnlyKV = null;</span>
<span class="source-line-no">079</span><span id="line-79"> private boolean skipResetSeqId = true;</span>
<span class="source-line-no">080</span><span id="line-80"> private int prefixLength = -1;</span>
<span class="source-line-no">081</span><span id="line-81"> protected Configuration conf;</span>
<span class="source-line-no">082</span><span id="line-82"></span>
<span class="source-line-no">083</span><span id="line-83"> /**</span>
<span class="source-line-no">084</span><span id="line-84"> * All {@link StoreFileReader} for the same StoreFile will share the</span>
<span class="source-line-no">085</span><span id="line-85"> * {@link StoreFileInfo#refCount}. Counter that is incremented every time a scanner is created on</span>
<span class="source-line-no">086</span><span id="line-86"> * the store file. It is decremented when the scan on the store file is done.</span>
<span class="source-line-no">087</span><span id="line-87"> */</span>
<span class="source-line-no">088</span><span id="line-88"> private final StoreFileInfo storeFileInfo;</span>
<span class="source-line-no">089</span><span id="line-89"> private final ReaderContext context;</span>
<span class="source-line-no">090</span><span id="line-90"></span>
<span class="source-line-no">091</span><span id="line-91"> private StoreFileReader(HFile.Reader reader, StoreFileInfo storeFileInfo, ReaderContext context,</span>
<span class="source-line-no">092</span><span id="line-92"> Configuration conf) {</span>
<span class="source-line-no">093</span><span id="line-93"> this.reader = reader;</span>
<span class="source-line-no">094</span><span id="line-94"> bloomFilterType = BloomType.NONE;</span>
<span class="source-line-no">095</span><span id="line-95"> this.storeFileInfo = storeFileInfo;</span>
<span class="source-line-no">096</span><span id="line-96"> this.context = context;</span>
<span class="source-line-no">097</span><span id="line-97"> this.conf = conf;</span>
<span class="source-line-no">098</span><span id="line-98"> }</span>
<span class="source-line-no">099</span><span id="line-99"></span>
<span class="source-line-no">100</span><span id="line-100"> public StoreFileReader(ReaderContext context, HFileInfo fileInfo, CacheConfig cacheConf,</span>
<span class="source-line-no">101</span><span id="line-101"> StoreFileInfo storeFileInfo, Configuration conf) throws IOException {</span>
<span class="source-line-no">102</span><span id="line-102"> this(HFile.createReader(context, fileInfo, cacheConf, conf), storeFileInfo, context, conf);</span>
<span class="source-line-no">103</span><span id="line-103"> }</span>
<span class="source-line-no">104</span><span id="line-104"></span>
<span class="source-line-no">105</span><span id="line-105"> void copyFields(StoreFileReader storeFileReader) throws IOException {</span>
<span class="source-line-no">106</span><span id="line-106"> this.generalBloomFilter = storeFileReader.generalBloomFilter;</span>
<span class="source-line-no">107</span><span id="line-107"> this.deleteFamilyBloomFilter = storeFileReader.deleteFamilyBloomFilter;</span>
<span class="source-line-no">108</span><span id="line-108"> this.bloomFilterType = storeFileReader.bloomFilterType;</span>
<span class="source-line-no">109</span><span id="line-109"> this.bloomFilterMetrics = storeFileReader.bloomFilterMetrics;</span>
<span class="source-line-no">110</span><span id="line-110"> this.sequenceID = storeFileReader.sequenceID;</span>
<span class="source-line-no">111</span><span id="line-111"> this.timeRange = storeFileReader.timeRange;</span>
<span class="source-line-no">112</span><span id="line-112"> this.lastBloomKey = storeFileReader.lastBloomKey;</span>
<span class="source-line-no">113</span><span id="line-113"> this.bulkLoadResult = storeFileReader.bulkLoadResult;</span>
<span class="source-line-no">114</span><span id="line-114"> this.lastBloomKeyOnlyKV = storeFileReader.lastBloomKeyOnlyKV;</span>
<span class="source-line-no">115</span><span id="line-115"> this.skipResetSeqId = storeFileReader.skipResetSeqId;</span>
<span class="source-line-no">116</span><span id="line-116"> this.prefixLength = storeFileReader.prefixLength;</span>
<span class="source-line-no">117</span><span id="line-117"> }</span>
<span class="source-line-no">118</span><span id="line-118"></span>
<span class="source-line-no">119</span><span id="line-119"> public boolean isPrimaryReplicaReader() {</span>
<span class="source-line-no">120</span><span id="line-120"> return reader.isPrimaryReplicaReader();</span>
<span class="source-line-no">121</span><span id="line-121"> }</span>
<span class="source-line-no">122</span><span id="line-122"></span>
<span class="source-line-no">123</span><span id="line-123"> /**</span>
<span class="source-line-no">124</span><span id="line-124"> * ONLY USE DEFAULT CONSTRUCTOR FOR UNIT TESTS</span>
<span class="source-line-no">125</span><span id="line-125"> */</span>
<span class="source-line-no">126</span><span id="line-126"> @InterfaceAudience.Private</span>
<span class="source-line-no">127</span><span id="line-127"> StoreFileReader() {</span>
<span class="source-line-no">128</span><span id="line-128"> this.storeFileInfo = null;</span>
<span class="source-line-no">129</span><span id="line-129"> this.reader = null;</span>
<span class="source-line-no">130</span><span id="line-130"> this.context = 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"> public CellComparator getComparator() {</span>
<span class="source-line-no">134</span><span id="line-134"> return reader.getComparator();</span>
<span class="source-line-no">135</span><span id="line-135"> }</span>
<span class="source-line-no">136</span><span id="line-136"></span>
<span class="source-line-no">137</span><span id="line-137"> /**</span>
<span class="source-line-no">138</span><span id="line-138"> * Get a scanner to scan over this StoreFile.</span>
<span class="source-line-no">139</span><span id="line-139"> * @param cacheBlocks should this scanner cache blocks?</span>
<span class="source-line-no">140</span><span id="line-140"> * @param pread use pread (for highly concurrent small readers)</span>
<span class="source-line-no">141</span><span id="line-141"> * @param isCompaction is scanner being used for compaction?</span>
<span class="source-line-no">142</span><span id="line-142"> * @param scannerOrder Order of this scanner relative to other scanners. See</span>
<span class="source-line-no">143</span><span id="line-143"> * {@link KeyValueScanner#getScannerOrder()}.</span>
<span class="source-line-no">144</span><span id="line-144"> * @param canOptimizeForNonNullColumn {@code true} if we can make sure there is no null column,</span>
<span class="source-line-no">145</span><span id="line-145"> * otherwise {@code false}. This is a hint for optimization.</span>
<span class="source-line-no">146</span><span id="line-146"> * @return a scanner</span>
<span class="source-line-no">147</span><span id="line-147"> */</span>
<span class="source-line-no">148</span><span id="line-148"> public StoreFileScanner getStoreFileScanner(boolean cacheBlocks, boolean pread,</span>
<span class="source-line-no">149</span><span id="line-149"> boolean isCompaction, long readPt, long scannerOrder, boolean canOptimizeForNonNullColumn) {</span>
<span class="source-line-no">150</span><span id="line-150"> return new StoreFileScanner(this, getScanner(cacheBlocks, pread, isCompaction), !isCompaction,</span>
<span class="source-line-no">151</span><span id="line-151"> reader.hasMVCCInfo(), readPt, scannerOrder, canOptimizeForNonNullColumn,</span>
<span class="source-line-no">152</span><span id="line-152"> reader.getDataBlockEncoding() == DataBlockEncoding.ROW_INDEX_V1);</span>
<span class="source-line-no">153</span><span id="line-153"> }</span>
<span class="source-line-no">154</span><span id="line-154"></span>
<span class="source-line-no">155</span><span id="line-155"> /**</span>
<span class="source-line-no">156</span><span id="line-156"> * Return the ref count associated with the reader whenever a scanner associated with the reader</span>
<span class="source-line-no">157</span><span id="line-157"> * is opened.</span>
<span class="source-line-no">158</span><span id="line-158"> */</span>
<span class="source-line-no">159</span><span id="line-159"> int getRefCount() {</span>
<span class="source-line-no">160</span><span id="line-160"> return storeFileInfo.getRefCount();</span>
<span class="source-line-no">161</span><span id="line-161"> }</span>
<span class="source-line-no">162</span><span id="line-162"></span>
<span class="source-line-no">163</span><span id="line-163"> /**</span>
<span class="source-line-no">164</span><span id="line-164"> * Indicate that the scanner has started reading with this reader. We need to increment the ref</span>
<span class="source-line-no">165</span><span id="line-165"> * count so reader is not close until some object is holding the lock</span>
<span class="source-line-no">166</span><span id="line-166"> */</span>
<span class="source-line-no">167</span><span id="line-167"> void incrementRefCount() {</span>
<span class="source-line-no">168</span><span id="line-168"> storeFileInfo.increaseRefCount();</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"> /**</span>
<span class="source-line-no">172</span><span id="line-172"> * Indicate that the scanner has finished reading with this reader. We need to decrement the ref</span>
<span class="source-line-no">173</span><span id="line-173"> * count, and also, if this is not the common pread reader, we should close it.</span>
<span class="source-line-no">174</span><span id="line-174"> */</span>
<span class="source-line-no">175</span><span id="line-175"> void readCompleted() {</span>
<span class="source-line-no">176</span><span id="line-176"> storeFileInfo.decreaseRefCount();</span>
<span class="source-line-no">177</span><span id="line-177"> if (context.getReaderType() == ReaderType.STREAM) {</span>
<span class="source-line-no">178</span><span id="line-178"> try {</span>
<span class="source-line-no">179</span><span id="line-179"> reader.close(false);</span>
<span class="source-line-no">180</span><span id="line-180"> } catch (IOException e) {</span>
<span class="source-line-no">181</span><span id="line-181"> LOG.warn("failed to close stream reader", e);</span>
<span class="source-line-no">182</span><span id="line-182"> }</span>
<span class="source-line-no">183</span><span id="line-183"> }</span>
<span class="source-line-no">184</span><span id="line-184"> }</span>
<span class="source-line-no">185</span><span id="line-185"></span>
<span class="source-line-no">186</span><span id="line-186"> /**</span>
<span class="source-line-no">187</span><span id="line-187"> * Will be overridden in HalfStoreFileReader</span>
<span class="source-line-no">188</span><span id="line-188"> */</span>
<span class="source-line-no">189</span><span id="line-189"> protected HFileScanner getScanner(boolean cacheBlocks, boolean pread, boolean isCompaction) {</span>
<span class="source-line-no">190</span><span id="line-190"> return reader.getScanner(conf, cacheBlocks, pread, isCompaction);</span>
<span class="source-line-no">191</span><span id="line-191"> }</span>
<span class="source-line-no">192</span><span id="line-192"></span>
<span class="source-line-no">193</span><span id="line-193"> public void close(boolean evictOnClose) throws IOException {</span>
<span class="source-line-no">194</span><span id="line-194"> reader.close(evictOnClose);</span>
<span class="source-line-no">195</span><span id="line-195"> }</span>
<span class="source-line-no">196</span><span id="line-196"></span>
<span class="source-line-no">197</span><span id="line-197"> /**</span>
<span class="source-line-no">198</span><span id="line-198"> * Check if this storeFile may contain keys within the TimeRange that have not expired (i.e. not</span>
<span class="source-line-no">199</span><span id="line-199"> * older than oldestUnexpiredTS).</span>
<span class="source-line-no">200</span><span id="line-200"> * @param tr the timeRange to restrict</span>
<span class="source-line-no">201</span><span id="line-201"> * @param oldestUnexpiredTS the oldest timestamp that is not expired, as determined by the column</span>
<span class="source-line-no">202</span><span id="line-202"> * family's TTL</span>
<span class="source-line-no">203</span><span id="line-203"> * @return false if queried keys definitely don't exist in this StoreFile</span>
<span class="source-line-no">204</span><span id="line-204"> */</span>
<span class="source-line-no">205</span><span id="line-205"> boolean passesTimerangeFilter(TimeRange tr, long oldestUnexpiredTS) {</span>
<span class="source-line-no">206</span><span id="line-206"> return this.timeRange == null</span>
<span class="source-line-no">207</span><span id="line-207"> ? true</span>
<span class="source-line-no">208</span><span id="line-208"> : this.timeRange.includesTimeRange(tr) &amp;&amp; this.timeRange.getMax() &gt;= oldestUnexpiredTS;</span>
<span class="source-line-no">209</span><span id="line-209"> }</span>
<span class="source-line-no">210</span><span id="line-210"></span>
<span class="source-line-no">211</span><span id="line-211"> /**</span>
<span class="source-line-no">212</span><span id="line-212"> * Checks whether the given scan passes the Bloom filter (if present). Only checks Bloom filters</span>
<span class="source-line-no">213</span><span id="line-213"> * for single-row or single-row-column scans. Bloom filter checking for multi-gets is implemented</span>
<span class="source-line-no">214</span><span id="line-214"> * as part of the store scanner system (see {@link StoreFileScanner#seek(Cell)} and uses the</span>
<span class="source-line-no">215</span><span id="line-215"> * lower-level API {@link #passesGeneralRowBloomFilter(byte[], int, int)} and</span>
<span class="source-line-no">216</span><span id="line-216"> * {@link #passesGeneralRowColBloomFilter(Cell)}.</span>
<span class="source-line-no">217</span><span id="line-217"> * @param scan the scan specification. Used to determine the row, and to check whether this is</span>
<span class="source-line-no">218</span><span id="line-218"> * a single-row ("get") scan.</span>
<span class="source-line-no">219</span><span id="line-219"> * @param columns the set of columns. Only used for row-column Bloom filters.</span>
<span class="source-line-no">220</span><span id="line-220"> * @return true if the scan with the given column set passes the Bloom filter, or if the Bloom</span>
<span class="source-line-no">221</span><span id="line-221"> * filter is not applicable for the scan. False if the Bloom filter is applicable and the</span>
<span class="source-line-no">222</span><span id="line-222"> * scan fails it.</span>
<span class="source-line-no">223</span><span id="line-223"> */</span>
<span class="source-line-no">224</span><span id="line-224"> boolean passesBloomFilter(Scan scan, final SortedSet&lt;byte[]&gt; columns) {</span>
<span class="source-line-no">225</span><span id="line-225"> byte[] row = scan.getStartRow();</span>
<span class="source-line-no">226</span><span id="line-226"> switch (this.bloomFilterType) {</span>
<span class="source-line-no">227</span><span id="line-227"> case ROW:</span>
<span class="source-line-no">228</span><span id="line-228"> if (!scan.isGetScan()) {</span>
<span class="source-line-no">229</span><span id="line-229"> return true;</span>
<span class="source-line-no">230</span><span id="line-230"> }</span>
<span class="source-line-no">231</span><span id="line-231"> return passesGeneralRowBloomFilter(row, 0, row.length);</span>
<span class="source-line-no">232</span><span id="line-232"></span>
<span class="source-line-no">233</span><span id="line-233"> case ROWCOL:</span>
<span class="source-line-no">234</span><span id="line-234"> if (!scan.isGetScan()) {</span>
<span class="source-line-no">235</span><span id="line-235"> return true;</span>
<span class="source-line-no">236</span><span id="line-236"> }</span>
<span class="source-line-no">237</span><span id="line-237"> if (columns != null &amp;&amp; columns.size() == 1) {</span>
<span class="source-line-no">238</span><span id="line-238"> byte[] column = columns.first();</span>
<span class="source-line-no">239</span><span id="line-239"> // create the required fake key</span>
<span class="source-line-no">240</span><span id="line-240"> Cell kvKey = PrivateCellUtil.createFirstOnRow(row, HConstants.EMPTY_BYTE_ARRAY, column);</span>
<span class="source-line-no">241</span><span id="line-241"> return passesGeneralRowColBloomFilter(kvKey);</span>
<span class="source-line-no">242</span><span id="line-242"> }</span>
<span class="source-line-no">243</span><span id="line-243"></span>
<span class="source-line-no">244</span><span id="line-244"> // For multi-column queries the Bloom filter is checked from the</span>
<span class="source-line-no">245</span><span id="line-245"> // seekExact operation.</span>
<span class="source-line-no">246</span><span id="line-246"> return true;</span>
<span class="source-line-no">247</span><span id="line-247"> case ROWPREFIX_FIXED_LENGTH:</span>
<span class="source-line-no">248</span><span id="line-248"> return passesGeneralRowPrefixBloomFilter(scan);</span>
<span class="source-line-no">249</span><span id="line-249"> default:</span>
<span class="source-line-no">250</span><span id="line-250"> if (scan.isGetScan()) {</span>
<span class="source-line-no">251</span><span id="line-251"> bloomFilterMetrics.incrementEligible();</span>
<span class="source-line-no">252</span><span id="line-252"> }</span>
<span class="source-line-no">253</span><span id="line-253"> return true;</span>
<span class="source-line-no">254</span><span id="line-254"> }</span>
<span class="source-line-no">255</span><span id="line-255"> }</span>
<span class="source-line-no">256</span><span id="line-256"></span>
<span class="source-line-no">257</span><span id="line-257"> public boolean passesDeleteFamilyBloomFilter(byte[] row, int rowOffset, int rowLen) {</span>
<span class="source-line-no">258</span><span id="line-258"> // Cache Bloom filter as a local variable in case it is set to null by</span>
<span class="source-line-no">259</span><span id="line-259"> // another thread on an IO error.</span>
<span class="source-line-no">260</span><span id="line-260"> BloomFilter bloomFilter = this.deleteFamilyBloomFilter;</span>
<span class="source-line-no">261</span><span id="line-261"></span>
<span class="source-line-no">262</span><span id="line-262"> // Empty file or there is no delete family at all</span>
<span class="source-line-no">263</span><span id="line-263"> if (reader.getTrailer().getEntryCount() == 0 || deleteFamilyCnt == 0) {</span>
<span class="source-line-no">264</span><span id="line-264"> return false;</span>
<span class="source-line-no">265</span><span id="line-265"> }</span>
<span class="source-line-no">266</span><span id="line-266"></span>
<span class="source-line-no">267</span><span id="line-267"> if (bloomFilter == null) {</span>
<span class="source-line-no">268</span><span id="line-268"> return true;</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"> try {</span>
<span class="source-line-no">272</span><span id="line-272"> if (!bloomFilter.supportsAutoLoading()) {</span>
<span class="source-line-no">273</span><span id="line-273"> return true;</span>
<span class="source-line-no">274</span><span id="line-274"> }</span>
<span class="source-line-no">275</span><span id="line-275"> return bloomFilter.contains(row, rowOffset, rowLen, null);</span>
<span class="source-line-no">276</span><span id="line-276"> } catch (IllegalArgumentException e) {</span>
<span class="source-line-no">277</span><span id="line-277"> LOG.error("Bad Delete Family bloom filter data -- proceeding without", e);</span>
<span class="source-line-no">278</span><span id="line-278"> setDeleteFamilyBloomFilterFaulty();</span>
<span class="source-line-no">279</span><span id="line-279"> }</span>
<span class="source-line-no">280</span><span id="line-280"></span>
<span class="source-line-no">281</span><span id="line-281"> return true;</span>
<span class="source-line-no">282</span><span id="line-282"> }</span>
<span class="source-line-no">283</span><span id="line-283"></span>
<span class="source-line-no">284</span><span id="line-284"> /**</span>
<span class="source-line-no">285</span><span id="line-285"> * A method for checking Bloom filters. Called directly from StoreFileScanner in case of a</span>
<span class="source-line-no">286</span><span id="line-286"> * multi-column query.</span>
<span class="source-line-no">287</span><span id="line-287"> * @return True if passes</span>
<span class="source-line-no">288</span><span id="line-288"> */</span>
<span class="source-line-no">289</span><span id="line-289"> private boolean passesGeneralRowBloomFilter(byte[] row, int rowOffset, int rowLen) {</span>
<span class="source-line-no">290</span><span id="line-290"> BloomFilter bloomFilter = this.generalBloomFilter;</span>
<span class="source-line-no">291</span><span id="line-291"> if (bloomFilter == null) {</span>
<span class="source-line-no">292</span><span id="line-292"> bloomFilterMetrics.incrementEligible();</span>
<span class="source-line-no">293</span><span id="line-293"> return true;</span>
<span class="source-line-no">294</span><span id="line-294"> }</span>
<span class="source-line-no">295</span><span id="line-295"></span>
<span class="source-line-no">296</span><span id="line-296"> // Used in ROW bloom</span>
<span class="source-line-no">297</span><span id="line-297"> byte[] key = null;</span>
<span class="source-line-no">298</span><span id="line-298"> if (rowOffset != 0 || rowLen != row.length) {</span>
<span class="source-line-no">299</span><span id="line-299"> throw new AssertionError("For row-only Bloom filters the row must occupy the whole array");</span>
<span class="source-line-no">300</span><span id="line-300"> }</span>
<span class="source-line-no">301</span><span id="line-301"> key = row;</span>
<span class="source-line-no">302</span><span id="line-302"> return checkGeneralBloomFilter(key, null, bloomFilter);</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"> * A method for checking Bloom filters. Called directly from StoreFileScanner in case of a</span>
<span class="source-line-no">307</span><span id="line-307"> * multi-column query. the cell to check if present in BloomFilter</span>
<span class="source-line-no">308</span><span id="line-308"> * @return True if passes</span>
<span class="source-line-no">309</span><span id="line-309"> */</span>
<span class="source-line-no">310</span><span id="line-310"> public boolean passesGeneralRowColBloomFilter(Cell cell) {</span>
<span class="source-line-no">311</span><span id="line-311"> BloomFilter bloomFilter = this.generalBloomFilter;</span>
<span class="source-line-no">312</span><span id="line-312"> if (bloomFilter == null) {</span>
<span class="source-line-no">313</span><span id="line-313"> bloomFilterMetrics.incrementEligible();</span>
<span class="source-line-no">314</span><span id="line-314"> return true;</span>
<span class="source-line-no">315</span><span id="line-315"> }</span>
<span class="source-line-no">316</span><span id="line-316"> // Used in ROW_COL bloom</span>
<span class="source-line-no">317</span><span id="line-317"> Cell kvKey = null;</span>
<span class="source-line-no">318</span><span id="line-318"> // Already if the incoming key is a fake rowcol key then use it as it is</span>
<span class="source-line-no">319</span><span id="line-319"> if (cell.getTypeByte() == KeyValue.Type.Maximum.getCode() &amp;&amp; cell.getFamilyLength() == 0) {</span>
<span class="source-line-no">320</span><span id="line-320"> kvKey = cell;</span>
<span class="source-line-no">321</span><span id="line-321"> } else {</span>
<span class="source-line-no">322</span><span id="line-322"> kvKey = PrivateCellUtil.createFirstOnRowCol(cell);</span>
<span class="source-line-no">323</span><span id="line-323"> }</span>
<span class="source-line-no">324</span><span id="line-324"> return checkGeneralBloomFilter(null, kvKey, bloomFilter);</span>
<span class="source-line-no">325</span><span id="line-325"> }</span>
<span class="source-line-no">326</span><span id="line-326"></span>
<span class="source-line-no">327</span><span id="line-327"> /**</span>
<span class="source-line-no">328</span><span id="line-328"> * A method for checking Bloom filters. Called directly from StoreFileScanner in case of a</span>
<span class="source-line-no">329</span><span id="line-329"> * multi-column query.</span>
<span class="source-line-no">330</span><span id="line-330"> * @return True if passes</span>
<span class="source-line-no">331</span><span id="line-331"> */</span>
<span class="source-line-no">332</span><span id="line-332"> private boolean passesGeneralRowPrefixBloomFilter(Scan scan) {</span>
<span class="source-line-no">333</span><span id="line-333"> BloomFilter bloomFilter = this.generalBloomFilter;</span>
<span class="source-line-no">334</span><span id="line-334"> if (bloomFilter == null) {</span>
<span class="source-line-no">335</span><span id="line-335"> bloomFilterMetrics.incrementEligible();</span>
<span class="source-line-no">336</span><span id="line-336"> return true;</span>
<span class="source-line-no">337</span><span id="line-337"> }</span>
<span class="source-line-no">338</span><span id="line-338"></span>
<span class="source-line-no">339</span><span id="line-339"> byte[] row = scan.getStartRow();</span>
<span class="source-line-no">340</span><span id="line-340"> byte[] rowPrefix;</span>
<span class="source-line-no">341</span><span id="line-341"> if (scan.isGetScan()) {</span>
<span class="source-line-no">342</span><span id="line-342"> rowPrefix = Bytes.copy(row, 0, Math.min(prefixLength, row.length));</span>
<span class="source-line-no">343</span><span id="line-343"> } else {</span>
<span class="source-line-no">344</span><span id="line-344"> // For non-get scans</span>
<span class="source-line-no">345</span><span id="line-345"> // Find out the common prefix of startRow and stopRow.</span>
<span class="source-line-no">346</span><span id="line-346"> int commonLength = Bytes.findCommonPrefix(scan.getStartRow(), scan.getStopRow(),</span>
<span class="source-line-no">347</span><span id="line-347"> scan.getStartRow().length, scan.getStopRow().length, 0, 0);</span>
<span class="source-line-no">348</span><span id="line-348"> // startRow and stopRow don't have the common prefix.</span>
<span class="source-line-no">349</span><span id="line-349"> // Or the common prefix length is less than prefixLength</span>
<span class="source-line-no">350</span><span id="line-350"> if (commonLength &lt;= 0 || commonLength &lt; prefixLength) {</span>
<span class="source-line-no">351</span><span id="line-351"> return true;</span>
<span class="source-line-no">352</span><span id="line-352"> }</span>
<span class="source-line-no">353</span><span id="line-353"> rowPrefix = Bytes.copy(row, 0, prefixLength);</span>
<span class="source-line-no">354</span><span id="line-354"> }</span>
<span class="source-line-no">355</span><span id="line-355"> return checkGeneralBloomFilter(rowPrefix, null, bloomFilter);</span>
<span class="source-line-no">356</span><span id="line-356"> }</span>
<span class="source-line-no">357</span><span id="line-357"></span>
<span class="source-line-no">358</span><span id="line-358"> private boolean checkGeneralBloomFilter(byte[] key, Cell kvKey, BloomFilter bloomFilter) {</span>
<span class="source-line-no">359</span><span id="line-359"> // Empty file</span>
<span class="source-line-no">360</span><span id="line-360"> if (reader.getTrailer().getEntryCount() == 0) {</span>
<span class="source-line-no">361</span><span id="line-361"> return false;</span>
<span class="source-line-no">362</span><span id="line-362"> }</span>
<span class="source-line-no">363</span><span id="line-363"> HFileBlock bloomBlock = null;</span>
<span class="source-line-no">364</span><span id="line-364"> try {</span>
<span class="source-line-no">365</span><span id="line-365"> boolean shouldCheckBloom;</span>
<span class="source-line-no">366</span><span id="line-366"> ByteBuff bloom;</span>
<span class="source-line-no">367</span><span id="line-367"> if (bloomFilter.supportsAutoLoading()) {</span>
<span class="source-line-no">368</span><span id="line-368"> bloom = null;</span>
<span class="source-line-no">369</span><span id="line-369"> shouldCheckBloom = true;</span>
<span class="source-line-no">370</span><span id="line-370"> } else {</span>
<span class="source-line-no">371</span><span id="line-371"> bloomBlock = reader.getMetaBlock(HFile.BLOOM_FILTER_DATA_KEY, true);</span>
<span class="source-line-no">372</span><span id="line-372"> bloom = bloomBlock.getBufferWithoutHeader();</span>
<span class="source-line-no">373</span><span id="line-373"> shouldCheckBloom = bloom != null;</span>
<span class="source-line-no">374</span><span id="line-374"> }</span>
<span class="source-line-no">375</span><span id="line-375"></span>
<span class="source-line-no">376</span><span id="line-376"> if (shouldCheckBloom) {</span>
<span class="source-line-no">377</span><span id="line-377"> boolean exists;</span>
<span class="source-line-no">378</span><span id="line-378"></span>
<span class="source-line-no">379</span><span id="line-379"> // Whether the primary Bloom key is greater than the last Bloom key</span>
<span class="source-line-no">380</span><span id="line-380"> // from the file info. For row-column Bloom filters this is not yet</span>
<span class="source-line-no">381</span><span id="line-381"> // a sufficient condition to return false.</span>
<span class="source-line-no">382</span><span id="line-382"> boolean keyIsAfterLast = (lastBloomKey != null);</span>
<span class="source-line-no">383</span><span id="line-383"> // hbase:meta does not have blooms. So we need not have special interpretation</span>
<span class="source-line-no">384</span><span id="line-384"> // of the hbase:meta cells. We can safely use Bytes.BYTES_RAWCOMPARATOR for ROW Bloom</span>
<span class="source-line-no">385</span><span id="line-385"> if (keyIsAfterLast) {</span>
<span class="source-line-no">386</span><span id="line-386"> if (bloomFilterType == BloomType.ROWCOL) {</span>
<span class="source-line-no">387</span><span id="line-387"> keyIsAfterLast = (getComparator().compare(kvKey, lastBloomKeyOnlyKV)) &gt; 0;</span>
<span class="source-line-no">388</span><span id="line-388"> } else {</span>
<span class="source-line-no">389</span><span id="line-389"> keyIsAfterLast = (Bytes.BYTES_RAWCOMPARATOR.compare(key, lastBloomKey) &gt; 0);</span>
<span class="source-line-no">390</span><span id="line-390"> }</span>
<span class="source-line-no">391</span><span id="line-391"> }</span>
<span class="source-line-no">392</span><span id="line-392"></span>
<span class="source-line-no">393</span><span id="line-393"> if (bloomFilterType == BloomType.ROWCOL) {</span>
<span class="source-line-no">394</span><span id="line-394"> // Since a Row Delete is essentially a DeleteFamily applied to all</span>
<span class="source-line-no">395</span><span id="line-395"> // columns, a file might be skipped if using row+col Bloom filter.</span>
<span class="source-line-no">396</span><span id="line-396"> // In order to ensure this file is included an additional check is</span>
<span class="source-line-no">397</span><span id="line-397"> // required looking only for a row bloom.</span>
<span class="source-line-no">398</span><span id="line-398"> Cell rowBloomKey = PrivateCellUtil.createFirstOnRow(kvKey);</span>
<span class="source-line-no">399</span><span id="line-399"> // hbase:meta does not have blooms. So we need not have special interpretation</span>
<span class="source-line-no">400</span><span id="line-400"> // of the hbase:meta cells. We can safely use Bytes.BYTES_RAWCOMPARATOR for ROW Bloom</span>
<span class="source-line-no">401</span><span id="line-401"> if (keyIsAfterLast &amp;&amp; (getComparator().compare(rowBloomKey, lastBloomKeyOnlyKV)) &gt; 0) {</span>
<span class="source-line-no">402</span><span id="line-402"> exists = false;</span>
<span class="source-line-no">403</span><span id="line-403"> } else {</span>
<span class="source-line-no">404</span><span id="line-404"> exists = bloomFilter.contains(kvKey, bloom, BloomType.ROWCOL)</span>
<span class="source-line-no">405</span><span id="line-405"> || bloomFilter.contains(rowBloomKey, bloom, BloomType.ROWCOL);</span>
<span class="source-line-no">406</span><span id="line-406"> }</span>
<span class="source-line-no">407</span><span id="line-407"> } else {</span>
<span class="source-line-no">408</span><span id="line-408"> exists = !keyIsAfterLast &amp;&amp; bloomFilter.contains(key, 0, key.length, bloom);</span>
<span class="source-line-no">409</span><span id="line-409"> }</span>
<span class="source-line-no">410</span><span id="line-410"></span>
<span class="source-line-no">411</span><span id="line-411"> return exists;</span>
<span class="source-line-no">412</span><span id="line-412"> }</span>
<span class="source-line-no">413</span><span id="line-413"> } catch (IOException e) {</span>
<span class="source-line-no">414</span><span id="line-414"> LOG.error("Error reading bloom filter data -- proceeding without", e);</span>
<span class="source-line-no">415</span><span id="line-415"> setGeneralBloomFilterFaulty();</span>
<span class="source-line-no">416</span><span id="line-416"> } catch (IllegalArgumentException e) {</span>
<span class="source-line-no">417</span><span id="line-417"> LOG.error("Bad bloom filter data -- proceeding without", e);</span>
<span class="source-line-no">418</span><span id="line-418"> setGeneralBloomFilterFaulty();</span>
<span class="source-line-no">419</span><span id="line-419"> } finally {</span>
<span class="source-line-no">420</span><span id="line-420"> // Release the bloom block so that its ref count can be decremented.</span>
<span class="source-line-no">421</span><span id="line-421"> if (bloomBlock != null) {</span>
<span class="source-line-no">422</span><span id="line-422"> bloomBlock.release();</span>
<span class="source-line-no">423</span><span id="line-423"> }</span>
<span class="source-line-no">424</span><span id="line-424"> }</span>
<span class="source-line-no">425</span><span id="line-425"> return true;</span>
<span class="source-line-no">426</span><span id="line-426"> }</span>
<span class="source-line-no">427</span><span id="line-427"></span>
<span class="source-line-no">428</span><span id="line-428"> /**</span>
<span class="source-line-no">429</span><span id="line-429"> * Checks whether the given scan rowkey range overlaps with the current storefile's</span>
<span class="source-line-no">430</span><span id="line-430"> * @param scan the scan specification. Used to determine the rowkey range.</span>
<span class="source-line-no">431</span><span id="line-431"> * @return true if there is overlap, false otherwise</span>
<span class="source-line-no">432</span><span id="line-432"> */</span>
<span class="source-line-no">433</span><span id="line-433"> public boolean passesKeyRangeFilter(Scan scan) {</span>
<span class="source-line-no">434</span><span id="line-434"> Optional&lt;ExtendedCell&gt; firstKeyKV = this.getFirstKey();</span>
<span class="source-line-no">435</span><span id="line-435"> Optional&lt;ExtendedCell&gt; lastKeyKV = this.getLastKey();</span>
<span class="source-line-no">436</span><span id="line-436"> if (!firstKeyKV.isPresent() || !lastKeyKV.isPresent()) {</span>
<span class="source-line-no">437</span><span id="line-437"> // the file is empty</span>
<span class="source-line-no">438</span><span id="line-438"> return false;</span>
<span class="source-line-no">439</span><span id="line-439"> }</span>
<span class="source-line-no">440</span><span id="line-440"> if (</span>
<span class="source-line-no">441</span><span id="line-441"> Bytes.equals(scan.getStartRow(), HConstants.EMPTY_START_ROW)</span>
<span class="source-line-no">442</span><span id="line-442"> &amp;&amp; Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW)</span>
<span class="source-line-no">443</span><span id="line-443"> ) {</span>
<span class="source-line-no">444</span><span id="line-444"> return true;</span>
<span class="source-line-no">445</span><span id="line-445"> }</span>
<span class="source-line-no">446</span><span id="line-446"> byte[] smallestScanRow = scan.isReversed() ? scan.getStopRow() : scan.getStartRow();</span>
<span class="source-line-no">447</span><span id="line-447"> byte[] largestScanRow = scan.isReversed() ? scan.getStartRow() : scan.getStopRow();</span>
<span class="source-line-no">448</span><span id="line-448"> boolean nonOverLapping =</span>
<span class="source-line-no">449</span><span id="line-449"> (getComparator().compareRows(firstKeyKV.get(), largestScanRow, 0, largestScanRow.length) &gt; 0</span>
<span class="source-line-no">450</span><span id="line-450"> &amp;&amp; !Bytes.equals(scan.isReversed() ? scan.getStartRow() : scan.getStopRow(),</span>
<span class="source-line-no">451</span><span id="line-451"> HConstants.EMPTY_END_ROW))</span>
<span class="source-line-no">452</span><span id="line-452"> || getComparator().compareRows(lastKeyKV.get(), smallestScanRow, 0, smallestScanRow.length)</span>
<span class="source-line-no">453</span><span id="line-453"> &lt; 0;</span>
<span class="source-line-no">454</span><span id="line-454"> return !nonOverLapping;</span>
<span class="source-line-no">455</span><span id="line-455"> }</span>
<span class="source-line-no">456</span><span id="line-456"></span>
<span class="source-line-no">457</span><span id="line-457"> public Map&lt;byte[], byte[]&gt; loadFileInfo() throws IOException {</span>
<span class="source-line-no">458</span><span id="line-458"> Map&lt;byte[], byte[]&gt; fi = reader.getHFileInfo();</span>
<span class="source-line-no">459</span><span id="line-459"></span>
<span class="source-line-no">460</span><span id="line-460"> byte[] b = fi.get(BLOOM_FILTER_TYPE_KEY);</span>
<span class="source-line-no">461</span><span id="line-461"> if (b != null) {</span>
<span class="source-line-no">462</span><span id="line-462"> bloomFilterType = BloomType.valueOf(Bytes.toString(b));</span>
<span class="source-line-no">463</span><span id="line-463"> }</span>
<span class="source-line-no">464</span><span id="line-464"></span>
<span class="source-line-no">465</span><span id="line-465"> byte[] p = fi.get(BLOOM_FILTER_PARAM_KEY);</span>
<span class="source-line-no">466</span><span id="line-466"> if (bloomFilterType == BloomType.ROWPREFIX_FIXED_LENGTH) {</span>
<span class="source-line-no">467</span><span id="line-467"> prefixLength = Bytes.toInt(p);</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"> lastBloomKey = fi.get(LAST_BLOOM_KEY);</span>
<span class="source-line-no">471</span><span id="line-471"> if (bloomFilterType == BloomType.ROWCOL) {</span>
<span class="source-line-no">472</span><span id="line-472"> lastBloomKeyOnlyKV = new KeyValue.KeyOnlyKeyValue(lastBloomKey, 0, lastBloomKey.length);</span>
<span class="source-line-no">473</span><span id="line-473"> }</span>
<span class="source-line-no">474</span><span id="line-474"> byte[] cnt = fi.get(DELETE_FAMILY_COUNT);</span>
<span class="source-line-no">475</span><span id="line-475"> if (cnt != null) {</span>
<span class="source-line-no">476</span><span id="line-476"> deleteFamilyCnt = Bytes.toLong(cnt);</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"> return fi;</span>
<span class="source-line-no">480</span><span id="line-480"> }</span>
<span class="source-line-no">481</span><span id="line-481"></span>
<span class="source-line-no">482</span><span id="line-482"> @RestrictedApi(explanation = "Should only be called in tests", link = "",</span>
<span class="source-line-no">483</span><span id="line-483"> allowedOnPath = ".*/src/test/.*")</span>
<span class="source-line-no">484</span><span id="line-484"> void loadBloomfilter() {</span>
<span class="source-line-no">485</span><span id="line-485"> this.loadBloomfilter(BlockType.GENERAL_BLOOM_META, null);</span>
<span class="source-line-no">486</span><span id="line-486"> this.loadBloomfilter(BlockType.DELETE_FAMILY_BLOOM_META, null);</span>
<span class="source-line-no">487</span><span id="line-487"> }</span>
<span class="source-line-no">488</span><span id="line-488"></span>
<span class="source-line-no">489</span><span id="line-489"> public void loadBloomfilter(BlockType blockType, BloomFilterMetrics metrics) {</span>
<span class="source-line-no">490</span><span id="line-490"> try {</span>
<span class="source-line-no">491</span><span id="line-491"> this.bloomFilterMetrics = metrics;</span>
<span class="source-line-no">492</span><span id="line-492"> if (blockType == BlockType.GENERAL_BLOOM_META) {</span>
<span class="source-line-no">493</span><span id="line-493"> if (this.generalBloomFilter != null) return; // Bloom has been loaded</span>
<span class="source-line-no">494</span><span id="line-494"></span>
<span class="source-line-no">495</span><span id="line-495"> DataInput bloomMeta = reader.getGeneralBloomFilterMetadata();</span>
<span class="source-line-no">496</span><span id="line-496"> if (bloomMeta != null) {</span>
<span class="source-line-no">497</span><span id="line-497"> // sanity check for NONE Bloom filter</span>
<span class="source-line-no">498</span><span id="line-498"> if (bloomFilterType == BloomType.NONE) {</span>
<span class="source-line-no">499</span><span id="line-499"> throw new IOException("valid bloom filter type not found in FileInfo");</span>
<span class="source-line-no">500</span><span id="line-500"> } else {</span>
<span class="source-line-no">501</span><span id="line-501"> generalBloomFilter = BloomFilterFactory.createFromMeta(bloomMeta, reader, metrics);</span>
<span class="source-line-no">502</span><span id="line-502"> if (LOG.isTraceEnabled()) {</span>
<span class="source-line-no">503</span><span id="line-503"> LOG.trace("Loaded " + bloomFilterType.toString() + " "</span>
<span class="source-line-no">504</span><span id="line-504"> + generalBloomFilter.getClass().getSimpleName() + " metadata for "</span>
<span class="source-line-no">505</span><span id="line-505"> + reader.getName());</span>
<span class="source-line-no">506</span><span id="line-506"> }</span>
<span class="source-line-no">507</span><span id="line-507"> }</span>
<span class="source-line-no">508</span><span id="line-508"> }</span>
<span class="source-line-no">509</span><span id="line-509"> } else if (blockType == BlockType.DELETE_FAMILY_BLOOM_META) {</span>
<span class="source-line-no">510</span><span id="line-510"> if (this.deleteFamilyBloomFilter != null) return; // Bloom has been loaded</span>
<span class="source-line-no">511</span><span id="line-511"></span>
<span class="source-line-no">512</span><span id="line-512"> DataInput bloomMeta = reader.getDeleteBloomFilterMetadata();</span>
<span class="source-line-no">513</span><span id="line-513"> if (bloomMeta != null) {</span>
<span class="source-line-no">514</span><span id="line-514"> // don't pass in metrics for the delete family bloom for now since the</span>
<span class="source-line-no">515</span><span id="line-515"> // goal is to give users insight into blooms _they_ configured.</span>
<span class="source-line-no">516</span><span id="line-516"> deleteFamilyBloomFilter = BloomFilterFactory.createFromMeta(bloomMeta, reader, null);</span>
<span class="source-line-no">517</span><span id="line-517"> LOG.info(</span>
<span class="source-line-no">518</span><span id="line-518"> "Loaded Delete Family Bloom (" + deleteFamilyBloomFilter.getClass().getSimpleName()</span>
<span class="source-line-no">519</span><span id="line-519"> + ") metadata for " + reader.getName());</span>
<span class="source-line-no">520</span><span id="line-520"> }</span>
<span class="source-line-no">521</span><span id="line-521"> } else {</span>
<span class="source-line-no">522</span><span id="line-522"> throw new RuntimeException(</span>
<span class="source-line-no">523</span><span id="line-523"> "Block Type: " + blockType.toString() + "is not supported for Bloom filter");</span>
<span class="source-line-no">524</span><span id="line-524"> }</span>
<span class="source-line-no">525</span><span id="line-525"> } catch (IOException e) {</span>
<span class="source-line-no">526</span><span id="line-526"> LOG.error("Error reading bloom filter meta for " + blockType + " -- proceeding without", e);</span>
<span class="source-line-no">527</span><span id="line-527"> setBloomFilterFaulty(blockType);</span>
<span class="source-line-no">528</span><span id="line-528"> } catch (IllegalArgumentException e) {</span>
<span class="source-line-no">529</span><span id="line-529"> LOG.error("Bad bloom filter meta " + blockType + " -- proceeding without", e);</span>
<span class="source-line-no">530</span><span id="line-530"> setBloomFilterFaulty(blockType);</span>
<span class="source-line-no">531</span><span id="line-531"> }</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"> @RestrictedApi(explanation = "Should only be called in tests", link = "",</span>
<span class="source-line-no">535</span><span id="line-535"> allowedOnPath = ".*/StoreFileReader.java|.*/src/test/.*")</span>
<span class="source-line-no">536</span><span id="line-536"> void setBloomFilterFaulty(BlockType blockType) {</span>
<span class="source-line-no">537</span><span id="line-537"> if (blockType == BlockType.GENERAL_BLOOM_META) {</span>
<span class="source-line-no">538</span><span id="line-538"> setGeneralBloomFilterFaulty();</span>
<span class="source-line-no">539</span><span id="line-539"> } else if (blockType == BlockType.DELETE_FAMILY_BLOOM_META) {</span>
<span class="source-line-no">540</span><span id="line-540"> setDeleteFamilyBloomFilterFaulty();</span>
<span class="source-line-no">541</span><span id="line-541"> }</span>
<span class="source-line-no">542</span><span id="line-542"> }</span>
<span class="source-line-no">543</span><span id="line-543"></span>
<span class="source-line-no">544</span><span id="line-544"> /**</span>
<span class="source-line-no">545</span><span id="line-545"> * The number of Bloom filter entries in this store file, or an estimate thereof, if the Bloom</span>
<span class="source-line-no">546</span><span id="line-546"> * filter is not loaded. This always returns an upper bound of the number of Bloom filter entries.</span>
<span class="source-line-no">547</span><span id="line-547"> * @return an estimate of the number of Bloom filter entries in this file</span>
<span class="source-line-no">548</span><span id="line-548"> */</span>
<span class="source-line-no">549</span><span id="line-549"> public long getFilterEntries() {</span>
<span class="source-line-no">550</span><span id="line-550"> return generalBloomFilter != null ? generalBloomFilter.getKeyCount() : reader.getEntries();</span>
<span class="source-line-no">551</span><span id="line-551"> }</span>
<span class="source-line-no">552</span><span id="line-552"></span>
<span class="source-line-no">553</span><span id="line-553"> private void setGeneralBloomFilterFaulty() {</span>
<span class="source-line-no">554</span><span id="line-554"> generalBloomFilter = null;</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"> private void setDeleteFamilyBloomFilterFaulty() {</span>
<span class="source-line-no">558</span><span id="line-558"> this.deleteFamilyBloomFilter = null;</span>
<span class="source-line-no">559</span><span id="line-559"> }</span>
<span class="source-line-no">560</span><span id="line-560"></span>
<span class="source-line-no">561</span><span id="line-561"> public Optional&lt;ExtendedCell&gt; getLastKey() {</span>
<span class="source-line-no">562</span><span id="line-562"> return reader.getLastKey();</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"> public Optional&lt;byte[]&gt; getLastRowKey() {</span>
<span class="source-line-no">566</span><span id="line-566"> return reader.getLastRowKey();</span>
<span class="source-line-no">567</span><span id="line-567"> }</span>
<span class="source-line-no">568</span><span id="line-568"></span>
<span class="source-line-no">569</span><span id="line-569"> public Optional&lt;ExtendedCell&gt; midKey() throws IOException {</span>
<span class="source-line-no">570</span><span id="line-570"> return reader.midKey();</span>
<span class="source-line-no">571</span><span id="line-571"> }</span>
<span class="source-line-no">572</span><span id="line-572"></span>
<span class="source-line-no">573</span><span id="line-573"> public long length() {</span>
<span class="source-line-no">574</span><span id="line-574"> return reader.length();</span>
<span class="source-line-no">575</span><span id="line-575"> }</span>
<span class="source-line-no">576</span><span id="line-576"></span>
<span class="source-line-no">577</span><span id="line-577"> public long getTotalUncompressedBytes() {</span>
<span class="source-line-no">578</span><span id="line-578"> return reader.getTrailer().getTotalUncompressedBytes();</span>
<span class="source-line-no">579</span><span id="line-579"> }</span>
<span class="source-line-no">580</span><span id="line-580"></span>
<span class="source-line-no">581</span><span id="line-581"> public long getEntries() {</span>
<span class="source-line-no">582</span><span id="line-582"> return reader.getEntries();</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"> public long getDeleteFamilyCnt() {</span>
<span class="source-line-no">586</span><span id="line-586"> return deleteFamilyCnt;</span>
<span class="source-line-no">587</span><span id="line-587"> }</span>
<span class="source-line-no">588</span><span id="line-588"></span>
<span class="source-line-no">589</span><span id="line-589"> public Optional&lt;ExtendedCell&gt; getFirstKey() {</span>
<span class="source-line-no">590</span><span id="line-590"> return reader.getFirstKey();</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"> public long indexSize() {</span>
<span class="source-line-no">594</span><span id="line-594"> return reader.indexSize();</span>
<span class="source-line-no">595</span><span id="line-595"> }</span>
<span class="source-line-no">596</span><span id="line-596"></span>
<span class="source-line-no">597</span><span id="line-597"> public BloomType getBloomFilterType() {</span>
<span class="source-line-no">598</span><span id="line-598"> return this.bloomFilterType;</span>
<span class="source-line-no">599</span><span id="line-599"> }</span>
<span class="source-line-no">600</span><span id="line-600"></span>
<span class="source-line-no">601</span><span id="line-601"> public long getSequenceID() {</span>
<span class="source-line-no">602</span><span id="line-602"> return sequenceID;</span>
<span class="source-line-no">603</span><span id="line-603"> }</span>
<span class="source-line-no">604</span><span id="line-604"></span>
<span class="source-line-no">605</span><span id="line-605"> public void setSequenceID(long sequenceID) {</span>
<span class="source-line-no">606</span><span id="line-606"> this.sequenceID = sequenceID;</span>
<span class="source-line-no">607</span><span id="line-607"> }</span>
<span class="source-line-no">608</span><span id="line-608"></span>
<span class="source-line-no">609</span><span id="line-609"> public void setBulkLoaded(boolean bulkLoadResult) {</span>
<span class="source-line-no">610</span><span id="line-610"> this.bulkLoadResult = bulkLoadResult;</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"> public boolean isBulkLoaded() {</span>
<span class="source-line-no">614</span><span id="line-614"> return this.bulkLoadResult;</span>
<span class="source-line-no">615</span><span id="line-615"> }</span>
<span class="source-line-no">616</span><span id="line-616"></span>
<span class="source-line-no">617</span><span id="line-617"> BloomFilter getGeneralBloomFilter() {</span>
<span class="source-line-no">618</span><span id="line-618"> return generalBloomFilter;</span>
<span class="source-line-no">619</span><span id="line-619"> }</span>
<span class="source-line-no">620</span><span id="line-620"></span>
<span class="source-line-no">621</span><span id="line-621"> long getUncompressedDataIndexSize() {</span>
<span class="source-line-no">622</span><span id="line-622"> return reader.getTrailer().getUncompressedDataIndexSize();</span>
<span class="source-line-no">623</span><span id="line-623"> }</span>
<span class="source-line-no">624</span><span id="line-624"></span>
<span class="source-line-no">625</span><span id="line-625"> public long getTotalBloomSize() {</span>
<span class="source-line-no">626</span><span id="line-626"> if (generalBloomFilter == null) return 0;</span>
<span class="source-line-no">627</span><span id="line-627"> return generalBloomFilter.getByteSize();</span>
<span class="source-line-no">628</span><span id="line-628"> }</span>
<span class="source-line-no">629</span><span id="line-629"></span>
<span class="source-line-no">630</span><span id="line-630"> public int getHFileVersion() {</span>
<span class="source-line-no">631</span><span id="line-631"> return reader.getTrailer().getMajorVersion();</span>
<span class="source-line-no">632</span><span id="line-632"> }</span>
<span class="source-line-no">633</span><span id="line-633"></span>
<span class="source-line-no">634</span><span id="line-634"> public int getHFileMinorVersion() {</span>
<span class="source-line-no">635</span><span id="line-635"> return reader.getTrailer().getMinorVersion();</span>
<span class="source-line-no">636</span><span id="line-636"> }</span>
<span class="source-line-no">637</span><span id="line-637"></span>
<span class="source-line-no">638</span><span id="line-638"> public HFile.Reader getHFileReader() {</span>
<span class="source-line-no">639</span><span id="line-639"> return reader;</span>
<span class="source-line-no">640</span><span id="line-640"> }</span>
<span class="source-line-no">641</span><span id="line-641"></span>
<span class="source-line-no">642</span><span id="line-642"> void disableBloomFilterForTesting() {</span>
<span class="source-line-no">643</span><span id="line-643"> generalBloomFilter = null;</span>
<span class="source-line-no">644</span><span id="line-644"> this.deleteFamilyBloomFilter = null;</span>
<span class="source-line-no">645</span><span id="line-645"> }</span>
<span class="source-line-no">646</span><span id="line-646"></span>
<span class="source-line-no">647</span><span id="line-647"> public long getMaxTimestamp() {</span>
<span class="source-line-no">648</span><span id="line-648"> return timeRange == null ? TimeRange.INITIAL_MAX_TIMESTAMP : timeRange.getMax();</span>
<span class="source-line-no">649</span><span id="line-649"> }</span>
<span class="source-line-no">650</span><span id="line-650"></span>
<span class="source-line-no">651</span><span id="line-651"> boolean isSkipResetSeqId() {</span>
<span class="source-line-no">652</span><span id="line-652"> return skipResetSeqId;</span>
<span class="source-line-no">653</span><span id="line-653"> }</span>
<span class="source-line-no">654</span><span id="line-654"></span>
<span class="source-line-no">655</span><span id="line-655"> void setSkipResetSeqId(boolean skipResetSeqId) {</span>
<span class="source-line-no">656</span><span id="line-656"> this.skipResetSeqId = skipResetSeqId;</span>
<span class="source-line-no">657</span><span id="line-657"> }</span>
<span class="source-line-no">658</span><span id="line-658"></span>
<span class="source-line-no">659</span><span id="line-659"> public int getPrefixLength() {</span>
<span class="source-line-no">660</span><span id="line-660"> return prefixLength;</span>
<span class="source-line-no">661</span><span id="line-661"> }</span>
<span class="source-line-no">662</span><span id="line-662"></span>
<span class="source-line-no">663</span><span id="line-663"> public ReaderContext getReaderContext() {</span>
<span class="source-line-no">664</span><span id="line-664"> return this.context;</span>
<span class="source-line-no">665</span><span id="line-665"> }</span>
<span class="source-line-no">666</span><span id="line-666">}</span>
</pre>
</div>
</main>
</body>
</html>