blob: 1e3ac964c25257b7970acd9c525dc4e02da9f498 [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.io.hfile, class: LruBlockCache">
<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.io.hfile;</span>
<span class="source-line-no">019</span><span id="line-19"></span>
<span class="source-line-no">020</span><span id="line-20">import static java.util.Objects.requireNonNull;</span>
<span class="source-line-no">021</span><span id="line-21"></span>
<span class="source-line-no">022</span><span id="line-22">import java.lang.ref.WeakReference;</span>
<span class="source-line-no">023</span><span id="line-23">import java.util.EnumMap;</span>
<span class="source-line-no">024</span><span id="line-24">import java.util.Iterator;</span>
<span class="source-line-no">025</span><span id="line-25">import java.util.List;</span>
<span class="source-line-no">026</span><span id="line-26">import java.util.Map;</span>
<span class="source-line-no">027</span><span id="line-27">import java.util.PriorityQueue;</span>
<span class="source-line-no">028</span><span id="line-28">import java.util.SortedSet;</span>
<span class="source-line-no">029</span><span id="line-29">import java.util.TreeSet;</span>
<span class="source-line-no">030</span><span id="line-30">import java.util.concurrent.ConcurrentHashMap;</span>
<span class="source-line-no">031</span><span id="line-31">import java.util.concurrent.Executors;</span>
<span class="source-line-no">032</span><span id="line-32">import java.util.concurrent.ScheduledExecutorService;</span>
<span class="source-line-no">033</span><span id="line-33">import java.util.concurrent.TimeUnit;</span>
<span class="source-line-no">034</span><span id="line-34">import java.util.concurrent.atomic.AtomicLong;</span>
<span class="source-line-no">035</span><span id="line-35">import java.util.concurrent.atomic.LongAdder;</span>
<span class="source-line-no">036</span><span id="line-36">import java.util.concurrent.locks.ReentrantLock;</span>
<span class="source-line-no">037</span><span id="line-37">import org.apache.commons.lang3.mutable.MutableBoolean;</span>
<span class="source-line-no">038</span><span id="line-38">import org.apache.hadoop.conf.Configuration;</span>
<span class="source-line-no">039</span><span id="line-39">import org.apache.hadoop.hbase.io.HeapSize;</span>
<span class="source-line-no">040</span><span id="line-40">import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;</span>
<span class="source-line-no">041</span><span id="line-41">import org.apache.hadoop.hbase.util.ClassSize;</span>
<span class="source-line-no">042</span><span id="line-42">import org.apache.hadoop.util.StringUtils;</span>
<span class="source-line-no">043</span><span id="line-43">import org.apache.yetus.audience.InterfaceAudience;</span>
<span class="source-line-no">044</span><span id="line-44">import org.slf4j.Logger;</span>
<span class="source-line-no">045</span><span id="line-45">import org.slf4j.LoggerFactory;</span>
<span class="source-line-no">046</span><span id="line-46"></span>
<span class="source-line-no">047</span><span id="line-47">import org.apache.hbase.thirdparty.com.google.common.base.MoreObjects;</span>
<span class="source-line-no">048</span><span id="line-48">import org.apache.hbase.thirdparty.com.google.common.base.Objects;</span>
<span class="source-line-no">049</span><span id="line-49">import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;</span>
<span class="source-line-no">050</span><span id="line-50"></span>
<span class="source-line-no">051</span><span id="line-51">/**</span>
<span class="source-line-no">052</span><span id="line-52"> * A block cache implementation that is memory-aware using {@link HeapSize}, memory-bound using an</span>
<span class="source-line-no">053</span><span id="line-53"> * LRU eviction algorithm, and concurrent: backed by a {@link ConcurrentHashMap} and with a</span>
<span class="source-line-no">054</span><span id="line-54"> * non-blocking eviction thread giving constant-time {@link #cacheBlock} and {@link #getBlock}</span>
<span class="source-line-no">055</span><span id="line-55"> * operations.</span>
<span class="source-line-no">056</span><span id="line-56"> * &lt;/p&gt;</span>
<span class="source-line-no">057</span><span id="line-57"> * Contains three levels of block priority to allow for scan-resistance and in-memory families</span>
<span class="source-line-no">058</span><span id="line-58"> * {@link org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder#setInMemory(boolean)} (An</span>
<span class="source-line-no">059</span><span id="line-59"> * in-memory column family is a column family that should be served from memory if possible):</span>
<span class="source-line-no">060</span><span id="line-60"> * single-access, multiple-accesses, and in-memory priority. A block is added with an in-memory</span>
<span class="source-line-no">061</span><span id="line-61"> * priority flag if {@link org.apache.hadoop.hbase.client.ColumnFamilyDescriptor#isInMemory()},</span>
<span class="source-line-no">062</span><span id="line-62"> * otherwise a block becomes a single access priority the first time it is read into this block</span>
<span class="source-line-no">063</span><span id="line-63"> * cache. If a block is accessed again while in cache, it is marked as a multiple access priority</span>
<span class="source-line-no">064</span><span id="line-64"> * block. This delineation of blocks is used to prevent scans from thrashing the cache adding a</span>
<span class="source-line-no">065</span><span id="line-65"> * least-frequently-used element to the eviction algorithm.</span>
<span class="source-line-no">066</span><span id="line-66"> * &lt;p/&gt;</span>
<span class="source-line-no">067</span><span id="line-67"> * Each priority is given its own chunk of the total cache to ensure fairness during eviction. Each</span>
<span class="source-line-no">068</span><span id="line-68"> * priority will retain close to its maximum size, however, if any priority is not using its entire</span>
<span class="source-line-no">069</span><span id="line-69"> * chunk the others are able to grow beyond their chunk size.</span>
<span class="source-line-no">070</span><span id="line-70"> * &lt;p/&gt;</span>
<span class="source-line-no">071</span><span id="line-71"> * Instantiated at a minimum with the total size and average block size. All sizes are in bytes. The</span>
<span class="source-line-no">072</span><span id="line-72"> * block size is not especially important as this cache is fully dynamic in its sizing of blocks. It</span>
<span class="source-line-no">073</span><span id="line-73"> * is only used for pre-allocating data structures and in initial heap estimation of the map.</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"> * The detailed constructor defines the sizes for the three priorities (they should total to the</span>
<span class="source-line-no">076</span><span id="line-76"> * &lt;code&gt;maximum size&lt;/code&gt; defined). It also sets the levels that trigger and control the eviction</span>
<span class="source-line-no">077</span><span id="line-77"> * thread.</span>
<span class="source-line-no">078</span><span id="line-78"> * &lt;p/&gt;</span>
<span class="source-line-no">079</span><span id="line-79"> * The &lt;code&gt;acceptable size&lt;/code&gt; is the cache size level which triggers the eviction process to</span>
<span class="source-line-no">080</span><span id="line-80"> * start. It evicts enough blocks to get the size below the minimum size specified.</span>
<span class="source-line-no">081</span><span id="line-81"> * &lt;p/&gt;</span>
<span class="source-line-no">082</span><span id="line-82"> * Eviction happens in a separate thread and involves a single full-scan of the map. It determines</span>
<span class="source-line-no">083</span><span id="line-83"> * how many bytes must be freed to reach the minimum size, and then while scanning determines the</span>
<span class="source-line-no">084</span><span id="line-84"> * fewest least-recently-used blocks necessary from each of the three priorities (would be 3 times</span>
<span class="source-line-no">085</span><span id="line-85"> * bytes to free). It then uses the priority chunk sizes to evict fairly according to the relative</span>
<span class="source-line-no">086</span><span id="line-86"> * sizes and usage.</span>
<span class="source-line-no">087</span><span id="line-87"> */</span>
<span class="source-line-no">088</span><span id="line-88">@InterfaceAudience.Private</span>
<span class="source-line-no">089</span><span id="line-89">public class LruBlockCache implements FirstLevelBlockCache {</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 static final Logger LOG = LoggerFactory.getLogger(LruBlockCache.class);</span>
<span class="source-line-no">092</span><span id="line-92"></span>
<span class="source-line-no">093</span><span id="line-93"> /**</span>
<span class="source-line-no">094</span><span id="line-94"> * Percentage of total size that eviction will evict until; e.g. if set to .8, then we will keep</span>
<span class="source-line-no">095</span><span id="line-95"> * evicting during an eviction run till the cache size is down to 80% of the total.</span>
<span class="source-line-no">096</span><span id="line-96"> */</span>
<span class="source-line-no">097</span><span id="line-97"> private static final String LRU_MIN_FACTOR_CONFIG_NAME = "hbase.lru.blockcache.min.factor";</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"> * Acceptable size of cache (no evictions if size &lt; acceptable)</span>
<span class="source-line-no">101</span><span id="line-101"> */</span>
<span class="source-line-no">102</span><span id="line-102"> private static final String LRU_ACCEPTABLE_FACTOR_CONFIG_NAME =</span>
<span class="source-line-no">103</span><span id="line-103"> "hbase.lru.blockcache.acceptable.factor";</span>
<span class="source-line-no">104</span><span id="line-104"></span>
<span class="source-line-no">105</span><span id="line-105"> /**</span>
<span class="source-line-no">106</span><span id="line-106"> * Hard capacity limit of cache, will reject any put if size &gt; this * acceptable</span>
<span class="source-line-no">107</span><span id="line-107"> */</span>
<span class="source-line-no">108</span><span id="line-108"> static final String LRU_HARD_CAPACITY_LIMIT_FACTOR_CONFIG_NAME =</span>
<span class="source-line-no">109</span><span id="line-109"> "hbase.lru.blockcache.hard.capacity.limit.factor";</span>
<span class="source-line-no">110</span><span id="line-110"> private static final String LRU_SINGLE_PERCENTAGE_CONFIG_NAME =</span>
<span class="source-line-no">111</span><span id="line-111"> "hbase.lru.blockcache.single.percentage";</span>
<span class="source-line-no">112</span><span id="line-112"> private static final String LRU_MULTI_PERCENTAGE_CONFIG_NAME =</span>
<span class="source-line-no">113</span><span id="line-113"> "hbase.lru.blockcache.multi.percentage";</span>
<span class="source-line-no">114</span><span id="line-114"> private static final String LRU_MEMORY_PERCENTAGE_CONFIG_NAME =</span>
<span class="source-line-no">115</span><span id="line-115"> "hbase.lru.blockcache.memory.percentage";</span>
<span class="source-line-no">116</span><span id="line-116"></span>
<span class="source-line-no">117</span><span id="line-117"> /**</span>
<span class="source-line-no">118</span><span id="line-118"> * Configuration key to force data-block always (except in-memory are too much) cached in memory</span>
<span class="source-line-no">119</span><span id="line-119"> * for in-memory hfile, unlike inMemory, which is a column-family configuration, inMemoryForceMode</span>
<span class="source-line-no">120</span><span id="line-120"> * is a cluster-wide configuration</span>
<span class="source-line-no">121</span><span id="line-121"> */</span>
<span class="source-line-no">122</span><span id="line-122"> private static final String LRU_IN_MEMORY_FORCE_MODE_CONFIG_NAME =</span>
<span class="source-line-no">123</span><span id="line-123"> "hbase.lru.rs.inmemoryforcemode";</span>
<span class="source-line-no">124</span><span id="line-124"></span>
<span class="source-line-no">125</span><span id="line-125"> /* Default Configuration Parameters */</span>
<span class="source-line-no">126</span><span id="line-126"></span>
<span class="source-line-no">127</span><span id="line-127"> /* Backing Concurrent Map Configuration */</span>
<span class="source-line-no">128</span><span id="line-128"> static final float DEFAULT_LOAD_FACTOR = 0.75f;</span>
<span class="source-line-no">129</span><span id="line-129"> static final int DEFAULT_CONCURRENCY_LEVEL = 16;</span>
<span class="source-line-no">130</span><span id="line-130"></span>
<span class="source-line-no">131</span><span id="line-131"> /* Eviction thresholds */</span>
<span class="source-line-no">132</span><span id="line-132"> private static final float DEFAULT_MIN_FACTOR = 0.95f;</span>
<span class="source-line-no">133</span><span id="line-133"> static final float DEFAULT_ACCEPTABLE_FACTOR = 0.99f;</span>
<span class="source-line-no">134</span><span id="line-134"></span>
<span class="source-line-no">135</span><span id="line-135"> /* Priority buckets */</span>
<span class="source-line-no">136</span><span id="line-136"> private static final float DEFAULT_SINGLE_FACTOR = 0.25f;</span>
<span class="source-line-no">137</span><span id="line-137"> private static final float DEFAULT_MULTI_FACTOR = 0.50f;</span>
<span class="source-line-no">138</span><span id="line-138"> private static final float DEFAULT_MEMORY_FACTOR = 0.25f;</span>
<span class="source-line-no">139</span><span id="line-139"></span>
<span class="source-line-no">140</span><span id="line-140"> private static final float DEFAULT_HARD_CAPACITY_LIMIT_FACTOR = 1.2f;</span>
<span class="source-line-no">141</span><span id="line-141"></span>
<span class="source-line-no">142</span><span id="line-142"> private static final boolean DEFAULT_IN_MEMORY_FORCE_MODE = false;</span>
<span class="source-line-no">143</span><span id="line-143"></span>
<span class="source-line-no">144</span><span id="line-144"> /* Statistics thread */</span>
<span class="source-line-no">145</span><span id="line-145"> private static final int STAT_THREAD_PERIOD = 60 * 5;</span>
<span class="source-line-no">146</span><span id="line-146"> private static final String LRU_MAX_BLOCK_SIZE = "hbase.lru.max.block.size";</span>
<span class="source-line-no">147</span><span id="line-147"> private static final long DEFAULT_MAX_BLOCK_SIZE = 16L * 1024L * 1024L;</span>
<span class="source-line-no">148</span><span id="line-148"></span>
<span class="source-line-no">149</span><span id="line-149"> /**</span>
<span class="source-line-no">150</span><span id="line-150"> * Defined the cache map as {@link ConcurrentHashMap} here, because in</span>
<span class="source-line-no">151</span><span id="line-151"> * {@link LruBlockCache#getBlock}, we need to guarantee the atomicity of map#k (key, func).</span>
<span class="source-line-no">152</span><span id="line-152"> * Besides, the func method must execute exactly once only when the key is present and under the</span>
<span class="source-line-no">153</span><span id="line-153"> * lock context, otherwise the reference count will be messed up. Notice that the</span>
<span class="source-line-no">154</span><span id="line-154"> * {@link java.util.concurrent.ConcurrentSkipListMap} can not guarantee that. Some code using</span>
<span class="source-line-no">155</span><span id="line-155"> * #computeIfPresent also expects the supplier to be executed only once. ConcurrentHashMap can</span>
<span class="source-line-no">156</span><span id="line-156"> * guarantee that. Other types may not.</span>
<span class="source-line-no">157</span><span id="line-157"> */</span>
<span class="source-line-no">158</span><span id="line-158"> private transient final ConcurrentHashMap&lt;BlockCacheKey, LruCachedBlock&gt; map;</span>
<span class="source-line-no">159</span><span id="line-159"></span>
<span class="source-line-no">160</span><span id="line-160"> /** Eviction lock (locked when eviction in process) */</span>
<span class="source-line-no">161</span><span id="line-161"> private transient final ReentrantLock evictionLock = new ReentrantLock(true);</span>
<span class="source-line-no">162</span><span id="line-162"></span>
<span class="source-line-no">163</span><span id="line-163"> private final long maxBlockSize;</span>
<span class="source-line-no">164</span><span id="line-164"></span>
<span class="source-line-no">165</span><span id="line-165"> /** Volatile boolean to track if we are in an eviction process or not */</span>
<span class="source-line-no">166</span><span id="line-166"> private volatile boolean evictionInProgress = false;</span>
<span class="source-line-no">167</span><span id="line-167"></span>
<span class="source-line-no">168</span><span id="line-168"> /** Eviction thread */</span>
<span class="source-line-no">169</span><span id="line-169"> private transient final EvictionThread evictionThread;</span>
<span class="source-line-no">170</span><span id="line-170"></span>
<span class="source-line-no">171</span><span id="line-171"> /** Statistics thread schedule pool (for heavy debugging, could remove) */</span>
<span class="source-line-no">172</span><span id="line-172"> private transient final ScheduledExecutorService scheduleThreadPool =</span>
<span class="source-line-no">173</span><span id="line-173"> Executors.newScheduledThreadPool(1, new ThreadFactoryBuilder()</span>
<span class="source-line-no">174</span><span id="line-174"> .setNameFormat("LruBlockCacheStatsExecutor").setDaemon(true).build());</span>
<span class="source-line-no">175</span><span id="line-175"></span>
<span class="source-line-no">176</span><span id="line-176"> /** Current size of cache */</span>
<span class="source-line-no">177</span><span id="line-177"> private final AtomicLong size;</span>
<span class="source-line-no">178</span><span id="line-178"></span>
<span class="source-line-no">179</span><span id="line-179"> /** Current size of data blocks */</span>
<span class="source-line-no">180</span><span id="line-180"> private final LongAdder dataBlockSize = new LongAdder();</span>
<span class="source-line-no">181</span><span id="line-181"></span>
<span class="source-line-no">182</span><span id="line-182"> /** Current size of index blocks */</span>
<span class="source-line-no">183</span><span id="line-183"> private final LongAdder indexBlockSize = new LongAdder();</span>
<span class="source-line-no">184</span><span id="line-184"></span>
<span class="source-line-no">185</span><span id="line-185"> /** Current size of bloom blocks */</span>
<span class="source-line-no">186</span><span id="line-186"> private final LongAdder bloomBlockSize = new LongAdder();</span>
<span class="source-line-no">187</span><span id="line-187"></span>
<span class="source-line-no">188</span><span id="line-188"> /** Current number of cached elements */</span>
<span class="source-line-no">189</span><span id="line-189"> private final AtomicLong elements;</span>
<span class="source-line-no">190</span><span id="line-190"></span>
<span class="source-line-no">191</span><span id="line-191"> /** Current number of cached data block elements */</span>
<span class="source-line-no">192</span><span id="line-192"> private final LongAdder dataBlockElements = new LongAdder();</span>
<span class="source-line-no">193</span><span id="line-193"></span>
<span class="source-line-no">194</span><span id="line-194"> /** Current number of cached index block elements */</span>
<span class="source-line-no">195</span><span id="line-195"> private final LongAdder indexBlockElements = new LongAdder();</span>
<span class="source-line-no">196</span><span id="line-196"></span>
<span class="source-line-no">197</span><span id="line-197"> /** Current number of cached bloom block elements */</span>
<span class="source-line-no">198</span><span id="line-198"> private final LongAdder bloomBlockElements = new LongAdder();</span>
<span class="source-line-no">199</span><span id="line-199"></span>
<span class="source-line-no">200</span><span id="line-200"> /** Cache access count (sequential ID) */</span>
<span class="source-line-no">201</span><span id="line-201"> private final AtomicLong count;</span>
<span class="source-line-no">202</span><span id="line-202"></span>
<span class="source-line-no">203</span><span id="line-203"> /** hard capacity limit */</span>
<span class="source-line-no">204</span><span id="line-204"> private float hardCapacityLimitFactor;</span>
<span class="source-line-no">205</span><span id="line-205"></span>
<span class="source-line-no">206</span><span id="line-206"> /** Cache statistics */</span>
<span class="source-line-no">207</span><span id="line-207"> private final CacheStats stats;</span>
<span class="source-line-no">208</span><span id="line-208"></span>
<span class="source-line-no">209</span><span id="line-209"> /** Maximum allowable size of cache (block put if size &gt; max, evict) */</span>
<span class="source-line-no">210</span><span id="line-210"> private long maxSize;</span>
<span class="source-line-no">211</span><span id="line-211"></span>
<span class="source-line-no">212</span><span id="line-212"> /** Approximate block size */</span>
<span class="source-line-no">213</span><span id="line-213"> private long blockSize;</span>
<span class="source-line-no">214</span><span id="line-214"></span>
<span class="source-line-no">215</span><span id="line-215"> /** Acceptable size of cache (no evictions if size &lt; acceptable) */</span>
<span class="source-line-no">216</span><span id="line-216"> private float acceptableFactor;</span>
<span class="source-line-no">217</span><span id="line-217"></span>
<span class="source-line-no">218</span><span id="line-218"> /** Minimum threshold of cache (when evicting, evict until size &lt; min) */</span>
<span class="source-line-no">219</span><span id="line-219"> private float minFactor;</span>
<span class="source-line-no">220</span><span id="line-220"></span>
<span class="source-line-no">221</span><span id="line-221"> /** Single access bucket size */</span>
<span class="source-line-no">222</span><span id="line-222"> private float singleFactor;</span>
<span class="source-line-no">223</span><span id="line-223"></span>
<span class="source-line-no">224</span><span id="line-224"> /** Multiple access bucket size */</span>
<span class="source-line-no">225</span><span id="line-225"> private float multiFactor;</span>
<span class="source-line-no">226</span><span id="line-226"></span>
<span class="source-line-no">227</span><span id="line-227"> /** In-memory bucket size */</span>
<span class="source-line-no">228</span><span id="line-228"> private float memoryFactor;</span>
<span class="source-line-no">229</span><span id="line-229"></span>
<span class="source-line-no">230</span><span id="line-230"> /** Overhead of the structure itself */</span>
<span class="source-line-no">231</span><span id="line-231"> private long overhead;</span>
<span class="source-line-no">232</span><span id="line-232"></span>
<span class="source-line-no">233</span><span id="line-233"> /** Whether in-memory hfile's data block has higher priority when evicting */</span>
<span class="source-line-no">234</span><span id="line-234"> private boolean forceInMemory;</span>
<span class="source-line-no">235</span><span id="line-235"></span>
<span class="source-line-no">236</span><span id="line-236"> /**</span>
<span class="source-line-no">237</span><span id="line-237"> * Where to send victims (blocks evicted/missing from the cache). This is used only when we use an</span>
<span class="source-line-no">238</span><span id="line-238"> * external cache as L2. Note: See org.apache.hadoop.hbase.io.hfile.MemcachedBlockCache</span>
<span class="source-line-no">239</span><span id="line-239"> */</span>
<span class="source-line-no">240</span><span id="line-240"> private transient BlockCache victimHandler = null;</span>
<span class="source-line-no">241</span><span id="line-241"></span>
<span class="source-line-no">242</span><span id="line-242"> /**</span>
<span class="source-line-no">243</span><span id="line-243"> * Default constructor. Specify maximum size and expected average block size (approximation is</span>
<span class="source-line-no">244</span><span id="line-244"> * fine).</span>
<span class="source-line-no">245</span><span id="line-245"> * &lt;p&gt;</span>
<span class="source-line-no">246</span><span id="line-246"> * All other factors will be calculated based on defaults specified in this class.</span>
<span class="source-line-no">247</span><span id="line-247"> * @param maxSize maximum size of cache, in bytes</span>
<span class="source-line-no">248</span><span id="line-248"> * @param blockSize approximate size of each block, in bytes</span>
<span class="source-line-no">249</span><span id="line-249"> */</span>
<span class="source-line-no">250</span><span id="line-250"> public LruBlockCache(long maxSize, long blockSize) {</span>
<span class="source-line-no">251</span><span id="line-251"> this(maxSize, blockSize, true);</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"> /**</span>
<span class="source-line-no">255</span><span id="line-255"> * Constructor used for testing. Allows disabling of the eviction thread.</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 LruBlockCache(long maxSize, long blockSize, boolean evictionThread) {</span>
<span class="source-line-no">258</span><span id="line-258"> this(maxSize, blockSize, evictionThread, (int) Math.ceil(1.2 * maxSize / blockSize),</span>
<span class="source-line-no">259</span><span id="line-259"> DEFAULT_LOAD_FACTOR, DEFAULT_CONCURRENCY_LEVEL, DEFAULT_MIN_FACTOR, DEFAULT_ACCEPTABLE_FACTOR,</span>
<span class="source-line-no">260</span><span id="line-260"> DEFAULT_SINGLE_FACTOR, DEFAULT_MULTI_FACTOR, DEFAULT_MEMORY_FACTOR,</span>
<span class="source-line-no">261</span><span id="line-261"> DEFAULT_HARD_CAPACITY_LIMIT_FACTOR, false, DEFAULT_MAX_BLOCK_SIZE);</span>
<span class="source-line-no">262</span><span id="line-262"> }</span>
<span class="source-line-no">263</span><span id="line-263"></span>
<span class="source-line-no">264</span><span id="line-264"> public LruBlockCache(long maxSize, long blockSize, boolean evictionThread, Configuration conf) {</span>
<span class="source-line-no">265</span><span id="line-265"> this(maxSize, blockSize, evictionThread, (int) Math.ceil(1.2 * maxSize / blockSize),</span>
<span class="source-line-no">266</span><span id="line-266"> DEFAULT_LOAD_FACTOR, DEFAULT_CONCURRENCY_LEVEL,</span>
<span class="source-line-no">267</span><span id="line-267"> conf.getFloat(LRU_MIN_FACTOR_CONFIG_NAME, DEFAULT_MIN_FACTOR),</span>
<span class="source-line-no">268</span><span id="line-268"> conf.getFloat(LRU_ACCEPTABLE_FACTOR_CONFIG_NAME, DEFAULT_ACCEPTABLE_FACTOR),</span>
<span class="source-line-no">269</span><span id="line-269"> conf.getFloat(LRU_SINGLE_PERCENTAGE_CONFIG_NAME, DEFAULT_SINGLE_FACTOR),</span>
<span class="source-line-no">270</span><span id="line-270"> conf.getFloat(LRU_MULTI_PERCENTAGE_CONFIG_NAME, DEFAULT_MULTI_FACTOR),</span>
<span class="source-line-no">271</span><span id="line-271"> conf.getFloat(LRU_MEMORY_PERCENTAGE_CONFIG_NAME, DEFAULT_MEMORY_FACTOR),</span>
<span class="source-line-no">272</span><span id="line-272"> conf.getFloat(LRU_HARD_CAPACITY_LIMIT_FACTOR_CONFIG_NAME, DEFAULT_HARD_CAPACITY_LIMIT_FACTOR),</span>
<span class="source-line-no">273</span><span id="line-273"> conf.getBoolean(LRU_IN_MEMORY_FORCE_MODE_CONFIG_NAME, DEFAULT_IN_MEMORY_FORCE_MODE),</span>
<span class="source-line-no">274</span><span id="line-274"> conf.getLong(LRU_MAX_BLOCK_SIZE, DEFAULT_MAX_BLOCK_SIZE));</span>
<span class="source-line-no">275</span><span id="line-275"> }</span>
<span class="source-line-no">276</span><span id="line-276"></span>
<span class="source-line-no">277</span><span id="line-277"> public LruBlockCache(long maxSize, long blockSize, Configuration conf) {</span>
<span class="source-line-no">278</span><span id="line-278"> this(maxSize, blockSize, true, conf);</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"> /**</span>
<span class="source-line-no">282</span><span id="line-282"> * Configurable constructor. Use this constructor if not using defaults.</span>
<span class="source-line-no">283</span><span id="line-283"> * @param maxSize maximum size of this cache, in bytes</span>
<span class="source-line-no">284</span><span id="line-284"> * @param blockSize expected average size of blocks, in bytes</span>
<span class="source-line-no">285</span><span id="line-285"> * @param evictionThread whether to run evictions in a bg thread or not</span>
<span class="source-line-no">286</span><span id="line-286"> * @param mapInitialSize initial size of backing ConcurrentHashMap</span>
<span class="source-line-no">287</span><span id="line-287"> * @param mapLoadFactor initial load factor of backing ConcurrentHashMap</span>
<span class="source-line-no">288</span><span id="line-288"> * @param mapConcurrencyLevel initial concurrency factor for backing CHM</span>
<span class="source-line-no">289</span><span id="line-289"> * @param minFactor percentage of total size that eviction will evict until</span>
<span class="source-line-no">290</span><span id="line-290"> * @param acceptableFactor percentage of total size that triggers eviction</span>
<span class="source-line-no">291</span><span id="line-291"> * @param singleFactor percentage of total size for single-access blocks</span>
<span class="source-line-no">292</span><span id="line-292"> * @param multiFactor percentage of total size for multiple-access blocks</span>
<span class="source-line-no">293</span><span id="line-293"> * @param memoryFactor percentage of total size for in-memory blocks</span>
<span class="source-line-no">294</span><span id="line-294"> */</span>
<span class="source-line-no">295</span><span id="line-295"> public LruBlockCache(long maxSize, long blockSize, boolean evictionThread, int mapInitialSize,</span>
<span class="source-line-no">296</span><span id="line-296"> float mapLoadFactor, int mapConcurrencyLevel, float minFactor, float acceptableFactor,</span>
<span class="source-line-no">297</span><span id="line-297"> float singleFactor, float multiFactor, float memoryFactor, float hardLimitFactor,</span>
<span class="source-line-no">298</span><span id="line-298"> boolean forceInMemory, long maxBlockSize) {</span>
<span class="source-line-no">299</span><span id="line-299"> this.maxBlockSize = maxBlockSize;</span>
<span class="source-line-no">300</span><span id="line-300"> if (</span>
<span class="source-line-no">301</span><span id="line-301"> singleFactor + multiFactor + memoryFactor != 1 || singleFactor &lt; 0 || multiFactor &lt; 0</span>
<span class="source-line-no">302</span><span id="line-302"> || memoryFactor &lt; 0</span>
<span class="source-line-no">303</span><span id="line-303"> ) {</span>
<span class="source-line-no">304</span><span id="line-304"> throw new IllegalArgumentException(</span>
<span class="source-line-no">305</span><span id="line-305"> "Single, multi, and memory factors " + " should be non-negative and total 1.0");</span>
<span class="source-line-no">306</span><span id="line-306"> }</span>
<span class="source-line-no">307</span><span id="line-307"> if (minFactor &gt;= acceptableFactor) {</span>
<span class="source-line-no">308</span><span id="line-308"> throw new IllegalArgumentException("minFactor must be smaller than acceptableFactor");</span>
<span class="source-line-no">309</span><span id="line-309"> }</span>
<span class="source-line-no">310</span><span id="line-310"> if (minFactor &gt;= 1.0f || acceptableFactor &gt;= 1.0f) {</span>
<span class="source-line-no">311</span><span id="line-311"> throw new IllegalArgumentException("all factors must be &lt; 1");</span>
<span class="source-line-no">312</span><span id="line-312"> }</span>
<span class="source-line-no">313</span><span id="line-313"> this.maxSize = maxSize;</span>
<span class="source-line-no">314</span><span id="line-314"> this.blockSize = blockSize;</span>
<span class="source-line-no">315</span><span id="line-315"> this.forceInMemory = forceInMemory;</span>
<span class="source-line-no">316</span><span id="line-316"> map = new ConcurrentHashMap&lt;&gt;(mapInitialSize, mapLoadFactor, mapConcurrencyLevel);</span>
<span class="source-line-no">317</span><span id="line-317"> this.minFactor = minFactor;</span>
<span class="source-line-no">318</span><span id="line-318"> this.acceptableFactor = acceptableFactor;</span>
<span class="source-line-no">319</span><span id="line-319"> this.singleFactor = singleFactor;</span>
<span class="source-line-no">320</span><span id="line-320"> this.multiFactor = multiFactor;</span>
<span class="source-line-no">321</span><span id="line-321"> this.memoryFactor = memoryFactor;</span>
<span class="source-line-no">322</span><span id="line-322"> this.stats = new CacheStats(this.getClass().getSimpleName());</span>
<span class="source-line-no">323</span><span id="line-323"> this.count = new AtomicLong(0);</span>
<span class="source-line-no">324</span><span id="line-324"> this.elements = new AtomicLong(0);</span>
<span class="source-line-no">325</span><span id="line-325"> this.overhead = calculateOverhead(maxSize, blockSize, mapConcurrencyLevel);</span>
<span class="source-line-no">326</span><span id="line-326"> this.size = new AtomicLong(this.overhead);</span>
<span class="source-line-no">327</span><span id="line-327"> this.hardCapacityLimitFactor = hardLimitFactor;</span>
<span class="source-line-no">328</span><span id="line-328"> if (evictionThread) {</span>
<span class="source-line-no">329</span><span id="line-329"> this.evictionThread = new EvictionThread(this);</span>
<span class="source-line-no">330</span><span id="line-330"> this.evictionThread.start(); // FindBugs SC_START_IN_CTOR</span>
<span class="source-line-no">331</span><span id="line-331"> } else {</span>
<span class="source-line-no">332</span><span id="line-332"> this.evictionThread = null;</span>
<span class="source-line-no">333</span><span id="line-333"> }</span>
<span class="source-line-no">334</span><span id="line-334"> // TODO: Add means of turning this off. Bit obnoxious running thread just to make a log</span>
<span class="source-line-no">335</span><span id="line-335"> // every five minutes.</span>
<span class="source-line-no">336</span><span id="line-336"> this.scheduleThreadPool.scheduleAtFixedRate(new StatisticsThread(this), STAT_THREAD_PERIOD,</span>
<span class="source-line-no">337</span><span id="line-337"> STAT_THREAD_PERIOD, TimeUnit.SECONDS);</span>
<span class="source-line-no">338</span><span id="line-338"> }</span>
<span class="source-line-no">339</span><span id="line-339"></span>
<span class="source-line-no">340</span><span id="line-340"> @Override</span>
<span class="source-line-no">341</span><span id="line-341"> public void setVictimCache(BlockCache victimCache) {</span>
<span class="source-line-no">342</span><span id="line-342"> if (victimHandler != null) {</span>
<span class="source-line-no">343</span><span id="line-343"> throw new IllegalArgumentException("The victim cache has already been set");</span>
<span class="source-line-no">344</span><span id="line-344"> }</span>
<span class="source-line-no">345</span><span id="line-345"> victimHandler = requireNonNull(victimCache);</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"> @Override</span>
<span class="source-line-no">349</span><span id="line-349"> public void setMaxSize(long maxSize) {</span>
<span class="source-line-no">350</span><span id="line-350"> this.maxSize = maxSize;</span>
<span class="source-line-no">351</span><span id="line-351"> if (this.size.get() &gt; acceptableSize() &amp;&amp; !evictionInProgress) {</span>
<span class="source-line-no">352</span><span id="line-352"> runEviction();</span>
<span class="source-line-no">353</span><span id="line-353"> }</span>
<span class="source-line-no">354</span><span id="line-354"> }</span>
<span class="source-line-no">355</span><span id="line-355"></span>
<span class="source-line-no">356</span><span id="line-356"> /**</span>
<span class="source-line-no">357</span><span id="line-357"> * The block cached in LRUBlockCache will always be an heap block: on the one side, the heap</span>
<span class="source-line-no">358</span><span id="line-358"> * access will be more faster then off-heap, the small index block or meta block cached in</span>
<span class="source-line-no">359</span><span id="line-359"> * CombinedBlockCache will benefit a lot. on other side, the LRUBlockCache size is always</span>
<span class="source-line-no">360</span><span id="line-360"> * calculated based on the total heap size, if caching an off-heap block in LRUBlockCache, the</span>
<span class="source-line-no">361</span><span id="line-361"> * heap size will be messed up. Here we will clone the block into an heap block if it's an</span>
<span class="source-line-no">362</span><span id="line-362"> * off-heap block, otherwise just use the original block. The key point is maintain the refCnt of</span>
<span class="source-line-no">363</span><span id="line-363"> * the block (HBASE-22127): &lt;br&gt;</span>
<span class="source-line-no">364</span><span id="line-364"> * 1. if cache the cloned heap block, its refCnt is an totally new one, it's easy to handle; &lt;br&gt;</span>
<span class="source-line-no">365</span><span id="line-365"> * 2. if cache the original heap block, we're sure that it won't be tracked in ByteBuffAllocator's</span>
<span class="source-line-no">366</span><span id="line-366"> * reservoir, if both RPC and LRUBlockCache release the block, then it can be garbage collected by</span>
<span class="source-line-no">367</span><span id="line-367"> * JVM, so need a retain here.</span>
<span class="source-line-no">368</span><span id="line-368"> * @param buf the original block</span>
<span class="source-line-no">369</span><span id="line-369"> * @return an block with an heap memory backend.</span>
<span class="source-line-no">370</span><span id="line-370"> */</span>
<span class="source-line-no">371</span><span id="line-371"> private Cacheable asReferencedHeapBlock(Cacheable buf) {</span>
<span class="source-line-no">372</span><span id="line-372"> if (buf instanceof HFileBlock) {</span>
<span class="source-line-no">373</span><span id="line-373"> HFileBlock blk = ((HFileBlock) buf);</span>
<span class="source-line-no">374</span><span id="line-374"> if (blk.isSharedMem()) {</span>
<span class="source-line-no">375</span><span id="line-375"> return HFileBlock.deepCloneOnHeap(blk);</span>
<span class="source-line-no">376</span><span id="line-376"> }</span>
<span class="source-line-no">377</span><span id="line-377"> }</span>
<span class="source-line-no">378</span><span id="line-378"> // The block will be referenced by this LRUBlockCache, so should increase its refCnt here.</span>
<span class="source-line-no">379</span><span id="line-379"> return buf.retain();</span>
<span class="source-line-no">380</span><span id="line-380"> }</span>
<span class="source-line-no">381</span><span id="line-381"></span>
<span class="source-line-no">382</span><span id="line-382"> // BlockCache implementation</span>
<span class="source-line-no">383</span><span id="line-383"></span>
<span class="source-line-no">384</span><span id="line-384"> /**</span>
<span class="source-line-no">385</span><span id="line-385"> * Cache the block with the specified name and buffer.</span>
<span class="source-line-no">386</span><span id="line-386"> * &lt;p&gt;</span>
<span class="source-line-no">387</span><span id="line-387"> * It is assumed this will NOT be called on an already cached block. In rare cases (HBASE-8547)</span>
<span class="source-line-no">388</span><span id="line-388"> * this can happen, for which we compare the buffer contents.</span>
<span class="source-line-no">389</span><span id="line-389"> * @param cacheKey block's cache key</span>
<span class="source-line-no">390</span><span id="line-390"> * @param buf block buffer</span>
<span class="source-line-no">391</span><span id="line-391"> * @param inMemory if block is in-memory</span>
<span class="source-line-no">392</span><span id="line-392"> */</span>
<span class="source-line-no">393</span><span id="line-393"> @Override</span>
<span class="source-line-no">394</span><span id="line-394"> public void cacheBlock(BlockCacheKey cacheKey, Cacheable buf, boolean inMemory) {</span>
<span class="source-line-no">395</span><span id="line-395"> if (buf.heapSize() &gt; maxBlockSize) {</span>
<span class="source-line-no">396</span><span id="line-396"> // If there are a lot of blocks that are too</span>
<span class="source-line-no">397</span><span id="line-397"> // big this can make the logs way too noisy.</span>
<span class="source-line-no">398</span><span id="line-398"> // So we log 2%</span>
<span class="source-line-no">399</span><span id="line-399"> if (stats.failInsert() % 50 == 0) {</span>
<span class="source-line-no">400</span><span id="line-400"> LOG.warn("Trying to cache too large a block " + cacheKey.getHfileName() + " @ "</span>
<span class="source-line-no">401</span><span id="line-401"> + cacheKey.getOffset() + " is " + buf.heapSize() + " which is larger than "</span>
<span class="source-line-no">402</span><span id="line-402"> + maxBlockSize);</span>
<span class="source-line-no">403</span><span id="line-403"> }</span>
<span class="source-line-no">404</span><span id="line-404"> return;</span>
<span class="source-line-no">405</span><span id="line-405"> }</span>
<span class="source-line-no">406</span><span id="line-406"></span>
<span class="source-line-no">407</span><span id="line-407"> LruCachedBlock cb = map.get(cacheKey);</span>
<span class="source-line-no">408</span><span id="line-408"> if (cb != null &amp;&amp; !BlockCacheUtil.shouldReplaceExistingCacheBlock(this, cacheKey, buf)) {</span>
<span class="source-line-no">409</span><span id="line-409"> return;</span>
<span class="source-line-no">410</span><span id="line-410"> }</span>
<span class="source-line-no">411</span><span id="line-411"> long currentSize = size.get();</span>
<span class="source-line-no">412</span><span id="line-412"> long currentAcceptableSize = acceptableSize();</span>
<span class="source-line-no">413</span><span id="line-413"> long hardLimitSize = (long) (hardCapacityLimitFactor * currentAcceptableSize);</span>
<span class="source-line-no">414</span><span id="line-414"> if (currentSize &gt;= hardLimitSize) {</span>
<span class="source-line-no">415</span><span id="line-415"> stats.failInsert();</span>
<span class="source-line-no">416</span><span id="line-416"> if (LOG.isTraceEnabled()) {</span>
<span class="source-line-no">417</span><span id="line-417"> LOG.trace("LruBlockCache current size " + StringUtils.byteDesc(currentSize)</span>
<span class="source-line-no">418</span><span id="line-418"> + " has exceeded acceptable size " + StringUtils.byteDesc(currentAcceptableSize) + "."</span>
<span class="source-line-no">419</span><span id="line-419"> + " The hard limit size is " + StringUtils.byteDesc(hardLimitSize)</span>
<span class="source-line-no">420</span><span id="line-420"> + ", failed to put cacheKey:" + cacheKey + " into LruBlockCache.");</span>
<span class="source-line-no">421</span><span id="line-421"> }</span>
<span class="source-line-no">422</span><span id="line-422"> if (!evictionInProgress) {</span>
<span class="source-line-no">423</span><span id="line-423"> runEviction();</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;</span>
<span class="source-line-no">426</span><span id="line-426"> }</span>
<span class="source-line-no">427</span><span id="line-427"> // Ensure that the block is an heap one.</span>
<span class="source-line-no">428</span><span id="line-428"> buf = asReferencedHeapBlock(buf);</span>
<span class="source-line-no">429</span><span id="line-429"> cb = new LruCachedBlock(cacheKey, buf, count.incrementAndGet(), inMemory);</span>
<span class="source-line-no">430</span><span id="line-430"> long newSize = updateSizeMetrics(cb, false);</span>
<span class="source-line-no">431</span><span id="line-431"> map.put(cacheKey, cb);</span>
<span class="source-line-no">432</span><span id="line-432"> long val = elements.incrementAndGet();</span>
<span class="source-line-no">433</span><span id="line-433"> if (buf.getBlockType().isBloom()) {</span>
<span class="source-line-no">434</span><span id="line-434"> bloomBlockElements.increment();</span>
<span class="source-line-no">435</span><span id="line-435"> } else if (buf.getBlockType().isIndex()) {</span>
<span class="source-line-no">436</span><span id="line-436"> indexBlockElements.increment();</span>
<span class="source-line-no">437</span><span id="line-437"> } else if (buf.getBlockType().isData()) {</span>
<span class="source-line-no">438</span><span id="line-438"> dataBlockElements.increment();</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 (LOG.isTraceEnabled()) {</span>
<span class="source-line-no">441</span><span id="line-441"> long size = map.size();</span>
<span class="source-line-no">442</span><span id="line-442"> assertCounterSanity(size, val);</span>
<span class="source-line-no">443</span><span id="line-443"> }</span>
<span class="source-line-no">444</span><span id="line-444"> if (newSize &gt; currentAcceptableSize &amp;&amp; !evictionInProgress) {</span>
<span class="source-line-no">445</span><span id="line-445"> runEviction();</span>
<span class="source-line-no">446</span><span id="line-446"> }</span>
<span class="source-line-no">447</span><span id="line-447"> }</span>
<span class="source-line-no">448</span><span id="line-448"></span>
<span class="source-line-no">449</span><span id="line-449"> /**</span>
<span class="source-line-no">450</span><span id="line-450"> * Sanity-checking for parity between actual block cache content and metrics. Intended only for</span>
<span class="source-line-no">451</span><span id="line-451"> * use with TRACE level logging and -ea JVM.</span>
<span class="source-line-no">452</span><span id="line-452"> */</span>
<span class="source-line-no">453</span><span id="line-453"> private static void assertCounterSanity(long mapSize, long counterVal) {</span>
<span class="source-line-no">454</span><span id="line-454"> if (counterVal &lt; 0) {</span>
<span class="source-line-no">455</span><span id="line-455"> LOG.trace("counterVal overflow. Assertions unreliable. counterVal=" + counterVal</span>
<span class="source-line-no">456</span><span id="line-456"> + ", mapSize=" + mapSize);</span>
<span class="source-line-no">457</span><span id="line-457"> return;</span>
<span class="source-line-no">458</span><span id="line-458"> }</span>
<span class="source-line-no">459</span><span id="line-459"> if (mapSize &lt; Integer.MAX_VALUE) {</span>
<span class="source-line-no">460</span><span id="line-460"> double pct_diff = Math.abs((((double) counterVal) / ((double) mapSize)) - 1.);</span>
<span class="source-line-no">461</span><span id="line-461"> if (pct_diff &gt; 0.05) {</span>
<span class="source-line-no">462</span><span id="line-462"> LOG.trace("delta between reported and actual size &gt; 5%. counterVal=" + counterVal</span>
<span class="source-line-no">463</span><span id="line-463"> + ", mapSize=" + mapSize);</span>
<span class="source-line-no">464</span><span id="line-464"> }</span>
<span class="source-line-no">465</span><span id="line-465"> }</span>
<span class="source-line-no">466</span><span id="line-466"> }</span>
<span class="source-line-no">467</span><span id="line-467"></span>
<span class="source-line-no">468</span><span id="line-468"> /**</span>
<span class="source-line-no">469</span><span id="line-469"> * Cache the block with the specified name and buffer.</span>
<span class="source-line-no">470</span><span id="line-470"> * &lt;p&gt;</span>
<span class="source-line-no">471</span><span id="line-471"> * TODO after HBASE-22005, we may cache an block which allocated from off-heap, but our LRU cache</span>
<span class="source-line-no">472</span><span id="line-472"> * sizing is based on heap size, so we should handle this in HBASE-22127. It will introduce an</span>
<span class="source-line-no">473</span><span id="line-473"> * switch whether make the LRU on-heap or not, if so we may need copy the memory to on-heap,</span>
<span class="source-line-no">474</span><span id="line-474"> * otherwise the caching size is based on off-heap.</span>
<span class="source-line-no">475</span><span id="line-475"> * @param cacheKey block's cache key</span>
<span class="source-line-no">476</span><span id="line-476"> * @param buf block buffer</span>
<span class="source-line-no">477</span><span id="line-477"> */</span>
<span class="source-line-no">478</span><span id="line-478"> @Override</span>
<span class="source-line-no">479</span><span id="line-479"> public void cacheBlock(BlockCacheKey cacheKey, Cacheable buf) {</span>
<span class="source-line-no">480</span><span id="line-480"> cacheBlock(cacheKey, buf, false);</span>
<span class="source-line-no">481</span><span id="line-481"> }</span>
<span class="source-line-no">482</span><span id="line-482"></span>
<span class="source-line-no">483</span><span id="line-483"> /**</span>
<span class="source-line-no">484</span><span id="line-484"> * Helper function that updates the local size counter and also updates any per-cf or</span>
<span class="source-line-no">485</span><span id="line-485"> * per-blocktype metrics it can discern from given {@link LruCachedBlock}</span>
<span class="source-line-no">486</span><span id="line-486"> */</span>
<span class="source-line-no">487</span><span id="line-487"> private long updateSizeMetrics(LruCachedBlock cb, boolean evict) {</span>
<span class="source-line-no">488</span><span id="line-488"> long heapsize = cb.heapSize();</span>
<span class="source-line-no">489</span><span id="line-489"> BlockType bt = cb.getBuffer().getBlockType();</span>
<span class="source-line-no">490</span><span id="line-490"> if (evict) {</span>
<span class="source-line-no">491</span><span id="line-491"> heapsize *= -1;</span>
<span class="source-line-no">492</span><span id="line-492"> }</span>
<span class="source-line-no">493</span><span id="line-493"> if (bt != null) {</span>
<span class="source-line-no">494</span><span id="line-494"> if (bt.isBloom()) {</span>
<span class="source-line-no">495</span><span id="line-495"> bloomBlockSize.add(heapsize);</span>
<span class="source-line-no">496</span><span id="line-496"> } else if (bt.isIndex()) {</span>
<span class="source-line-no">497</span><span id="line-497"> indexBlockSize.add(heapsize);</span>
<span class="source-line-no">498</span><span id="line-498"> } else if (bt.isData()) {</span>
<span class="source-line-no">499</span><span id="line-499"> dataBlockSize.add(heapsize);</span>
<span class="source-line-no">500</span><span id="line-500"> }</span>
<span class="source-line-no">501</span><span id="line-501"> }</span>
<span class="source-line-no">502</span><span id="line-502"> return size.addAndGet(heapsize);</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"> * Get the buffer of the block with the specified name.</span>
<span class="source-line-no">507</span><span id="line-507"> * @param cacheKey block's cache key</span>
<span class="source-line-no">508</span><span id="line-508"> * @param caching true if the caller caches blocks on cache misses</span>
<span class="source-line-no">509</span><span id="line-509"> * @param repeat Whether this is a repeat lookup for the same block (used to avoid</span>
<span class="source-line-no">510</span><span id="line-510"> * double counting cache misses when doing double-check locking)</span>
<span class="source-line-no">511</span><span id="line-511"> * @param updateCacheMetrics Whether to update cache metrics or not</span>
<span class="source-line-no">512</span><span id="line-512"> * @return buffer of specified cache key, or null if not in cache</span>
<span class="source-line-no">513</span><span id="line-513"> */</span>
<span class="source-line-no">514</span><span id="line-514"> @Override</span>
<span class="source-line-no">515</span><span id="line-515"> public Cacheable getBlock(BlockCacheKey cacheKey, boolean caching, boolean repeat,</span>
<span class="source-line-no">516</span><span id="line-516"> boolean updateCacheMetrics) {</span>
<span class="source-line-no">517</span><span id="line-517"> // Note: 'map' must be a ConcurrentHashMap or the supplier may be invoked more than once.</span>
<span class="source-line-no">518</span><span id="line-518"> LruCachedBlock cb = map.computeIfPresent(cacheKey, (key, val) -&gt; {</span>
<span class="source-line-no">519</span><span id="line-519"> // It will be referenced by RPC path, so increase here. NOTICE: Must do the retain inside</span>
<span class="source-line-no">520</span><span id="line-520"> // this block. because if retain outside the map#computeIfPresent, the evictBlock may remove</span>
<span class="source-line-no">521</span><span id="line-521"> // the block and release, then we're retaining a block with refCnt=0 which is disallowed.</span>
<span class="source-line-no">522</span><span id="line-522"> // see HBASE-22422.</span>
<span class="source-line-no">523</span><span id="line-523"> val.getBuffer().retain();</span>
<span class="source-line-no">524</span><span id="line-524"> return val;</span>
<span class="source-line-no">525</span><span id="line-525"> });</span>
<span class="source-line-no">526</span><span id="line-526"> if (cb == null) {</span>
<span class="source-line-no">527</span><span id="line-527"> if (!repeat &amp;&amp; updateCacheMetrics) {</span>
<span class="source-line-no">528</span><span id="line-528"> stats.miss(caching, cacheKey.isPrimary(), cacheKey.getBlockType());</span>
<span class="source-line-no">529</span><span id="line-529"> }</span>
<span class="source-line-no">530</span><span id="line-530"> // If there is another block cache then try and read there.</span>
<span class="source-line-no">531</span><span id="line-531"> // However if this is a retry ( second time in double checked locking )</span>
<span class="source-line-no">532</span><span id="line-532"> // And it's already a miss then the l2 will also be a miss.</span>
<span class="source-line-no">533</span><span id="line-533"> if (victimHandler != null &amp;&amp; !repeat) {</span>
<span class="source-line-no">534</span><span id="line-534"> // The handler will increase result's refCnt for RPC, so need no extra retain.</span>
<span class="source-line-no">535</span><span id="line-535"> Cacheable result = victimHandler.getBlock(cacheKey, caching, repeat, updateCacheMetrics);</span>
<span class="source-line-no">536</span><span id="line-536"> // Promote this to L1.</span>
<span class="source-line-no">537</span><span id="line-537"> if (result != null) {</span>
<span class="source-line-no">538</span><span id="line-538"> if (caching) {</span>
<span class="source-line-no">539</span><span id="line-539"> cacheBlock(cacheKey, result, /* inMemory = */ false);</span>
<span class="source-line-no">540</span><span id="line-540"> }</span>
<span class="source-line-no">541</span><span id="line-541"> }</span>
<span class="source-line-no">542</span><span id="line-542"> return result;</span>
<span class="source-line-no">543</span><span id="line-543"> }</span>
<span class="source-line-no">544</span><span id="line-544"> return null;</span>
<span class="source-line-no">545</span><span id="line-545"> }</span>
<span class="source-line-no">546</span><span id="line-546"> if (updateCacheMetrics) {</span>
<span class="source-line-no">547</span><span id="line-547"> stats.hit(caching, cacheKey.isPrimary(), cacheKey.getBlockType());</span>
<span class="source-line-no">548</span><span id="line-548"> }</span>
<span class="source-line-no">549</span><span id="line-549"> cb.access(count.incrementAndGet());</span>
<span class="source-line-no">550</span><span id="line-550"> return cb.getBuffer();</span>
<span class="source-line-no">551</span><span id="line-551"> }</span>
<span class="source-line-no">552</span><span id="line-552"></span>
<span class="source-line-no">553</span><span id="line-553"> /**</span>
<span class="source-line-no">554</span><span id="line-554"> * Whether the cache contains block with specified cacheKey</span>
<span class="source-line-no">555</span><span id="line-555"> * @return true if contains the block</span>
<span class="source-line-no">556</span><span id="line-556"> */</span>
<span class="source-line-no">557</span><span id="line-557"> @Override</span>
<span class="source-line-no">558</span><span id="line-558"> public boolean containsBlock(BlockCacheKey cacheKey) {</span>
<span class="source-line-no">559</span><span id="line-559"> return map.containsKey(cacheKey);</span>
<span class="source-line-no">560</span><span id="line-560"> }</span>
<span class="source-line-no">561</span><span id="line-561"></span>
<span class="source-line-no">562</span><span id="line-562"> @Override</span>
<span class="source-line-no">563</span><span id="line-563"> public boolean evictBlock(BlockCacheKey cacheKey) {</span>
<span class="source-line-no">564</span><span id="line-564"> LruCachedBlock cb = map.get(cacheKey);</span>
<span class="source-line-no">565</span><span id="line-565"> return cb != null &amp;&amp; evictBlock(cb, false) &gt; 0;</span>
<span class="source-line-no">566</span><span id="line-566"> }</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"> * Evicts all blocks for a specific HFile. This is an expensive operation implemented as a</span>
<span class="source-line-no">570</span><span id="line-570"> * linear-time search through all blocks in the cache. Ideally this should be a search in a</span>
<span class="source-line-no">571</span><span id="line-571"> * log-access-time map.</span>
<span class="source-line-no">572</span><span id="line-572"> * &lt;p&gt;</span>
<span class="source-line-no">573</span><span id="line-573"> * This is used for evict-on-close to remove all blocks of a specific HFile.</span>
<span class="source-line-no">574</span><span id="line-574"> * @return the number of blocks evicted</span>
<span class="source-line-no">575</span><span id="line-575"> */</span>
<span class="source-line-no">576</span><span id="line-576"> @Override</span>
<span class="source-line-no">577</span><span id="line-577"> public int evictBlocksByHfileName(String hfileName) {</span>
<span class="source-line-no">578</span><span id="line-578"> int numEvicted = 0;</span>
<span class="source-line-no">579</span><span id="line-579"> for (BlockCacheKey key : map.keySet()) {</span>
<span class="source-line-no">580</span><span id="line-580"> if (key.getHfileName().equals(hfileName)) {</span>
<span class="source-line-no">581</span><span id="line-581"> if (evictBlock(key)) {</span>
<span class="source-line-no">582</span><span id="line-582"> ++numEvicted;</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"> }</span>
<span class="source-line-no">586</span><span id="line-586"> if (victimHandler != null) {</span>
<span class="source-line-no">587</span><span id="line-587"> numEvicted += victimHandler.evictBlocksByHfileName(hfileName);</span>
<span class="source-line-no">588</span><span id="line-588"> }</span>
<span class="source-line-no">589</span><span id="line-589"> return numEvicted;</span>
<span class="source-line-no">590</span><span id="line-590"> }</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"> * Evict the block, and it will be cached by the victim handler if exists &amp;amp;&amp;amp; block may be</span>
<span class="source-line-no">594</span><span id="line-594"> * read again later</span>
<span class="source-line-no">595</span><span id="line-595"> * @param evictedByEvictionProcess true if the given block is evicted by EvictionThread</span>
<span class="source-line-no">596</span><span id="line-596"> * @return the heap size of evicted block</span>
<span class="source-line-no">597</span><span id="line-597"> */</span>
<span class="source-line-no">598</span><span id="line-598"> protected long evictBlock(LruCachedBlock block, boolean evictedByEvictionProcess) {</span>
<span class="source-line-no">599</span><span id="line-599"> final MutableBoolean evicted = new MutableBoolean(false);</span>
<span class="source-line-no">600</span><span id="line-600"> // Note: 'map' must be a ConcurrentHashMap or the supplier may be invoked more than once.</span>
<span class="source-line-no">601</span><span id="line-601"> map.computeIfPresent(block.getCacheKey(), (k, v) -&gt; {</span>
<span class="source-line-no">602</span><span id="line-602"> // Run the victim handler before we remove the mapping in the L1 map. It must complete</span>
<span class="source-line-no">603</span><span id="line-603"> // quickly because other removal or insertion operations can be blocked in the meantime.</span>
<span class="source-line-no">604</span><span id="line-604"> if (evictedByEvictionProcess &amp;&amp; victimHandler != null) {</span>
<span class="source-line-no">605</span><span id="line-605"> victimHandler.cacheBlock(k, v.getBuffer());</span>
<span class="source-line-no">606</span><span id="line-606"> }</span>
<span class="source-line-no">607</span><span id="line-607"> // Decrease the block's reference count, and if refCount is 0, then it'll auto-deallocate. DO</span>
<span class="source-line-no">608</span><span id="line-608"> // NOT move this up because if we do that then the victimHandler may access the buffer with</span>
<span class="source-line-no">609</span><span id="line-609"> // refCnt = 0 which is disallowed.</span>
<span class="source-line-no">610</span><span id="line-610"> v.getBuffer().release();</span>
<span class="source-line-no">611</span><span id="line-611"> evicted.setTrue();</span>
<span class="source-line-no">612</span><span id="line-612"> // By returning null from the supplier we remove the mapping from the L1 map.</span>
<span class="source-line-no">613</span><span id="line-613"> return null;</span>
<span class="source-line-no">614</span><span id="line-614"> });</span>
<span class="source-line-no">615</span><span id="line-615"> // If we didn't find anything to evict there is nothing more to do here.</span>
<span class="source-line-no">616</span><span id="line-616"> if (evicted.isFalse()) {</span>
<span class="source-line-no">617</span><span id="line-617"> return 0;</span>
<span class="source-line-no">618</span><span id="line-618"> }</span>
<span class="source-line-no">619</span><span id="line-619"> // We evicted the block so update L1 statistics.</span>
<span class="source-line-no">620</span><span id="line-620"> updateSizeMetrics(block, true);</span>
<span class="source-line-no">621</span><span id="line-621"> long val = elements.decrementAndGet();</span>
<span class="source-line-no">622</span><span id="line-622"> if (LOG.isTraceEnabled()) {</span>
<span class="source-line-no">623</span><span id="line-623"> long size = map.size();</span>
<span class="source-line-no">624</span><span id="line-624"> assertCounterSanity(size, val);</span>
<span class="source-line-no">625</span><span id="line-625"> }</span>
<span class="source-line-no">626</span><span id="line-626"> BlockType bt = block.getBuffer().getBlockType();</span>
<span class="source-line-no">627</span><span id="line-627"> if (bt.isBloom()) {</span>
<span class="source-line-no">628</span><span id="line-628"> bloomBlockElements.decrement();</span>
<span class="source-line-no">629</span><span id="line-629"> } else if (bt.isIndex()) {</span>
<span class="source-line-no">630</span><span id="line-630"> indexBlockElements.decrement();</span>
<span class="source-line-no">631</span><span id="line-631"> } else if (bt.isData()) {</span>
<span class="source-line-no">632</span><span id="line-632"> dataBlockElements.decrement();</span>
<span class="source-line-no">633</span><span id="line-633"> }</span>
<span class="source-line-no">634</span><span id="line-634"> if (evictedByEvictionProcess) {</span>
<span class="source-line-no">635</span><span id="line-635"> // When the eviction of the block happened because of invalidation of HFiles, no need to</span>
<span class="source-line-no">636</span><span id="line-636"> // update the stats counter.</span>
<span class="source-line-no">637</span><span id="line-637"> stats.evicted(block.getCachedTime(), block.getCacheKey().isPrimary());</span>
<span class="source-line-no">638</span><span id="line-638"> }</span>
<span class="source-line-no">639</span><span id="line-639"> return block.heapSize();</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"> /**</span>
<span class="source-line-no">643</span><span id="line-643"> * Multi-threaded call to run the eviction process.</span>
<span class="source-line-no">644</span><span id="line-644"> */</span>
<span class="source-line-no">645</span><span id="line-645"> private void runEviction() {</span>
<span class="source-line-no">646</span><span id="line-646"> if (evictionThread == null || !evictionThread.isGo()) {</span>
<span class="source-line-no">647</span><span id="line-647"> evict();</span>
<span class="source-line-no">648</span><span id="line-648"> } else {</span>
<span class="source-line-no">649</span><span id="line-649"> evictionThread.evict();</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"></span>
<span class="source-line-no">653</span><span id="line-653"> boolean isEvictionInProgress() {</span>
<span class="source-line-no">654</span><span id="line-654"> return evictionInProgress;</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"> long getOverhead() {</span>
<span class="source-line-no">658</span><span id="line-658"> return overhead;</span>
<span class="source-line-no">659</span><span id="line-659"> }</span>
<span class="source-line-no">660</span><span id="line-660"></span>
<span class="source-line-no">661</span><span id="line-661"> /**</span>
<span class="source-line-no">662</span><span id="line-662"> * Eviction method.</span>
<span class="source-line-no">663</span><span id="line-663"> */</span>
<span class="source-line-no">664</span><span id="line-664"> void evict() {</span>
<span class="source-line-no">665</span><span id="line-665"></span>
<span class="source-line-no">666</span><span id="line-666"> // Ensure only one eviction at a time</span>
<span class="source-line-no">667</span><span id="line-667"> if (!evictionLock.tryLock()) {</span>
<span class="source-line-no">668</span><span id="line-668"> return;</span>
<span class="source-line-no">669</span><span id="line-669"> }</span>
<span class="source-line-no">670</span><span id="line-670"></span>
<span class="source-line-no">671</span><span id="line-671"> try {</span>
<span class="source-line-no">672</span><span id="line-672"> evictionInProgress = true;</span>
<span class="source-line-no">673</span><span id="line-673"> long currentSize = this.size.get();</span>
<span class="source-line-no">674</span><span id="line-674"> long bytesToFree = currentSize - minSize();</span>
<span class="source-line-no">675</span><span id="line-675"></span>
<span class="source-line-no">676</span><span id="line-676"> if (LOG.isTraceEnabled()) {</span>
<span class="source-line-no">677</span><span id="line-677"> LOG.trace("Block cache LRU eviction started; Attempting to free "</span>
<span class="source-line-no">678</span><span id="line-678"> + StringUtils.byteDesc(bytesToFree) + " of total=" + StringUtils.byteDesc(currentSize));</span>
<span class="source-line-no">679</span><span id="line-679"> }</span>
<span class="source-line-no">680</span><span id="line-680"></span>
<span class="source-line-no">681</span><span id="line-681"> if (bytesToFree &lt;= 0) {</span>
<span class="source-line-no">682</span><span id="line-682"> return;</span>
<span class="source-line-no">683</span><span id="line-683"> }</span>
<span class="source-line-no">684</span><span id="line-684"></span>
<span class="source-line-no">685</span><span id="line-685"> // Instantiate priority buckets</span>
<span class="source-line-no">686</span><span id="line-686"> BlockBucket bucketSingle = new BlockBucket("single", bytesToFree, blockSize, singleSize());</span>
<span class="source-line-no">687</span><span id="line-687"> BlockBucket bucketMulti = new BlockBucket("multi", bytesToFree, blockSize, multiSize());</span>
<span class="source-line-no">688</span><span id="line-688"> BlockBucket bucketMemory = new BlockBucket("memory", bytesToFree, blockSize, memorySize());</span>
<span class="source-line-no">689</span><span id="line-689"></span>
<span class="source-line-no">690</span><span id="line-690"> // Scan entire map putting into appropriate buckets</span>
<span class="source-line-no">691</span><span id="line-691"> for (LruCachedBlock cachedBlock : map.values()) {</span>
<span class="source-line-no">692</span><span id="line-692"> switch (cachedBlock.getPriority()) {</span>
<span class="source-line-no">693</span><span id="line-693"> case SINGLE: {</span>
<span class="source-line-no">694</span><span id="line-694"> bucketSingle.add(cachedBlock);</span>
<span class="source-line-no">695</span><span id="line-695"> break;</span>
<span class="source-line-no">696</span><span id="line-696"> }</span>
<span class="source-line-no">697</span><span id="line-697"> case MULTI: {</span>
<span class="source-line-no">698</span><span id="line-698"> bucketMulti.add(cachedBlock);</span>
<span class="source-line-no">699</span><span id="line-699"> break;</span>
<span class="source-line-no">700</span><span id="line-700"> }</span>
<span class="source-line-no">701</span><span id="line-701"> case MEMORY: {</span>
<span class="source-line-no">702</span><span id="line-702"> bucketMemory.add(cachedBlock);</span>
<span class="source-line-no">703</span><span id="line-703"> break;</span>
<span class="source-line-no">704</span><span id="line-704"> }</span>
<span class="source-line-no">705</span><span id="line-705"> }</span>
<span class="source-line-no">706</span><span id="line-706"> }</span>
<span class="source-line-no">707</span><span id="line-707"></span>
<span class="source-line-no">708</span><span id="line-708"> long bytesFreed = 0;</span>
<span class="source-line-no">709</span><span id="line-709"> if (forceInMemory || memoryFactor &gt; 0.999f) {</span>
<span class="source-line-no">710</span><span id="line-710"> long s = bucketSingle.totalSize();</span>
<span class="source-line-no">711</span><span id="line-711"> long m = bucketMulti.totalSize();</span>
<span class="source-line-no">712</span><span id="line-712"> if (bytesToFree &gt; (s + m)) {</span>
<span class="source-line-no">713</span><span id="line-713"> // this means we need to evict blocks in memory bucket to make room,</span>
<span class="source-line-no">714</span><span id="line-714"> // so the single and multi buckets will be emptied</span>
<span class="source-line-no">715</span><span id="line-715"> bytesFreed = bucketSingle.free(s);</span>
<span class="source-line-no">716</span><span id="line-716"> bytesFreed += bucketMulti.free(m);</span>
<span class="source-line-no">717</span><span id="line-717"> if (LOG.isTraceEnabled()) {</span>
<span class="source-line-no">718</span><span id="line-718"> LOG.trace(</span>
<span class="source-line-no">719</span><span id="line-719"> "freed " + StringUtils.byteDesc(bytesFreed) + " from single and multi buckets");</span>
<span class="source-line-no">720</span><span id="line-720"> }</span>
<span class="source-line-no">721</span><span id="line-721"> bytesFreed += bucketMemory.free(bytesToFree - bytesFreed);</span>
<span class="source-line-no">722</span><span id="line-722"> if (LOG.isTraceEnabled()) {</span>
<span class="source-line-no">723</span><span id="line-723"> LOG.trace(</span>
<span class="source-line-no">724</span><span id="line-724"> "freed " + StringUtils.byteDesc(bytesFreed) + " total from all three buckets ");</span>
<span class="source-line-no">725</span><span id="line-725"> }</span>
<span class="source-line-no">726</span><span id="line-726"> } else {</span>
<span class="source-line-no">727</span><span id="line-727"> // this means no need to evict block in memory bucket,</span>
<span class="source-line-no">728</span><span id="line-728"> // and we try best to make the ratio between single-bucket and</span>
<span class="source-line-no">729</span><span id="line-729"> // multi-bucket is 1:2</span>
<span class="source-line-no">730</span><span id="line-730"> long bytesRemain = s + m - bytesToFree;</span>
<span class="source-line-no">731</span><span id="line-731"> if (3 * s &lt;= bytesRemain) {</span>
<span class="source-line-no">732</span><span id="line-732"> // single-bucket is small enough that no eviction happens for it</span>
<span class="source-line-no">733</span><span id="line-733"> // hence all eviction goes from multi-bucket</span>
<span class="source-line-no">734</span><span id="line-734"> bytesFreed = bucketMulti.free(bytesToFree);</span>
<span class="source-line-no">735</span><span id="line-735"> } else if (3 * m &lt;= 2 * bytesRemain) {</span>
<span class="source-line-no">736</span><span id="line-736"> // multi-bucket is small enough that no eviction happens for it</span>
<span class="source-line-no">737</span><span id="line-737"> // hence all eviction goes from single-bucket</span>
<span class="source-line-no">738</span><span id="line-738"> bytesFreed = bucketSingle.free(bytesToFree);</span>
<span class="source-line-no">739</span><span id="line-739"> } else {</span>
<span class="source-line-no">740</span><span id="line-740"> // both buckets need to evict some blocks</span>
<span class="source-line-no">741</span><span id="line-741"> bytesFreed = bucketSingle.free(s - bytesRemain / 3);</span>
<span class="source-line-no">742</span><span id="line-742"> if (bytesFreed &lt; bytesToFree) {</span>
<span class="source-line-no">743</span><span id="line-743"> bytesFreed += bucketMulti.free(bytesToFree - bytesFreed);</span>
<span class="source-line-no">744</span><span id="line-744"> }</span>
<span class="source-line-no">745</span><span id="line-745"> }</span>
<span class="source-line-no">746</span><span id="line-746"> }</span>
<span class="source-line-no">747</span><span id="line-747"> } else {</span>
<span class="source-line-no">748</span><span id="line-748"> PriorityQueue&lt;BlockBucket&gt; bucketQueue = new PriorityQueue&lt;&gt;(3);</span>
<span class="source-line-no">749</span><span id="line-749"></span>
<span class="source-line-no">750</span><span id="line-750"> bucketQueue.add(bucketSingle);</span>
<span class="source-line-no">751</span><span id="line-751"> bucketQueue.add(bucketMulti);</span>
<span class="source-line-no">752</span><span id="line-752"> bucketQueue.add(bucketMemory);</span>
<span class="source-line-no">753</span><span id="line-753"></span>
<span class="source-line-no">754</span><span id="line-754"> int remainingBuckets = bucketQueue.size();</span>
<span class="source-line-no">755</span><span id="line-755"></span>
<span class="source-line-no">756</span><span id="line-756"> BlockBucket bucket;</span>
<span class="source-line-no">757</span><span id="line-757"> while ((bucket = bucketQueue.poll()) != null) {</span>
<span class="source-line-no">758</span><span id="line-758"> long overflow = bucket.overflow();</span>
<span class="source-line-no">759</span><span id="line-759"> if (overflow &gt; 0) {</span>
<span class="source-line-no">760</span><span id="line-760"> long bucketBytesToFree =</span>
<span class="source-line-no">761</span><span id="line-761"> Math.min(overflow, (bytesToFree - bytesFreed) / remainingBuckets);</span>
<span class="source-line-no">762</span><span id="line-762"> bytesFreed += bucket.free(bucketBytesToFree);</span>
<span class="source-line-no">763</span><span id="line-763"> }</span>
<span class="source-line-no">764</span><span id="line-764"> remainingBuckets--;</span>
<span class="source-line-no">765</span><span id="line-765"> }</span>
<span class="source-line-no">766</span><span id="line-766"> }</span>
<span class="source-line-no">767</span><span id="line-767"> if (LOG.isTraceEnabled()) {</span>
<span class="source-line-no">768</span><span id="line-768"> long single = bucketSingle.totalSize();</span>
<span class="source-line-no">769</span><span id="line-769"> long multi = bucketMulti.totalSize();</span>
<span class="source-line-no">770</span><span id="line-770"> long memory = bucketMemory.totalSize();</span>
<span class="source-line-no">771</span><span id="line-771"> LOG.trace(</span>
<span class="source-line-no">772</span><span id="line-772"> "Block cache LRU eviction completed; " + "freed=" + StringUtils.byteDesc(bytesFreed)</span>
<span class="source-line-no">773</span><span id="line-773"> + ", " + "total=" + StringUtils.byteDesc(this.size.get()) + ", " + "single="</span>
<span class="source-line-no">774</span><span id="line-774"> + StringUtils.byteDesc(single) + ", " + "multi=" + StringUtils.byteDesc(multi) + ", "</span>
<span class="source-line-no">775</span><span id="line-775"> + "memory=" + StringUtils.byteDesc(memory));</span>
<span class="source-line-no">776</span><span id="line-776"> }</span>
<span class="source-line-no">777</span><span id="line-777"> } finally {</span>
<span class="source-line-no">778</span><span id="line-778"> stats.evict();</span>
<span class="source-line-no">779</span><span id="line-779"> evictionInProgress = false;</span>
<span class="source-line-no">780</span><span id="line-780"> evictionLock.unlock();</span>
<span class="source-line-no">781</span><span id="line-781"> }</span>
<span class="source-line-no">782</span><span id="line-782"> }</span>
<span class="source-line-no">783</span><span id="line-783"></span>
<span class="source-line-no">784</span><span id="line-784"> @Override</span>
<span class="source-line-no">785</span><span id="line-785"> public String toString() {</span>
<span class="source-line-no">786</span><span id="line-786"> return MoreObjects.toStringHelper(this).add("blockCount", getBlockCount())</span>
<span class="source-line-no">787</span><span id="line-787"> .add("currentSize", StringUtils.byteDesc(getCurrentSize()))</span>
<span class="source-line-no">788</span><span id="line-788"> .add("freeSize", StringUtils.byteDesc(getFreeSize()))</span>
<span class="source-line-no">789</span><span id="line-789"> .add("maxSize", StringUtils.byteDesc(getMaxSize()))</span>
<span class="source-line-no">790</span><span id="line-790"> .add("heapSize", StringUtils.byteDesc(heapSize()))</span>
<span class="source-line-no">791</span><span id="line-791"> .add("minSize", StringUtils.byteDesc(minSize())).add("minFactor", minFactor)</span>
<span class="source-line-no">792</span><span id="line-792"> .add("multiSize", StringUtils.byteDesc(multiSize())).add("multiFactor", multiFactor)</span>
<span class="source-line-no">793</span><span id="line-793"> .add("singleSize", StringUtils.byteDesc(singleSize())).add("singleFactor", singleFactor)</span>
<span class="source-line-no">794</span><span id="line-794"> .toString();</span>
<span class="source-line-no">795</span><span id="line-795"> }</span>
<span class="source-line-no">796</span><span id="line-796"></span>
<span class="source-line-no">797</span><span id="line-797"> /**</span>
<span class="source-line-no">798</span><span id="line-798"> * Used to group blocks into priority buckets. There will be a BlockBucket for each priority</span>
<span class="source-line-no">799</span><span id="line-799"> * (single, multi, memory). Once bucketed, the eviction algorithm takes the appropriate number of</span>
<span class="source-line-no">800</span><span id="line-800"> * elements out of each according to configuration parameters and their relatives sizes.</span>
<span class="source-line-no">801</span><span id="line-801"> */</span>
<span class="source-line-no">802</span><span id="line-802"> private class BlockBucket implements Comparable&lt;BlockBucket&gt; {</span>
<span class="source-line-no">803</span><span id="line-803"></span>
<span class="source-line-no">804</span><span id="line-804"> private final String name;</span>
<span class="source-line-no">805</span><span id="line-805"> private LruCachedBlockQueue queue;</span>
<span class="source-line-no">806</span><span id="line-806"> private long totalSize = 0;</span>
<span class="source-line-no">807</span><span id="line-807"> private long bucketSize;</span>
<span class="source-line-no">808</span><span id="line-808"></span>
<span class="source-line-no">809</span><span id="line-809"> public BlockBucket(String name, long bytesToFree, long blockSize, long bucketSize) {</span>
<span class="source-line-no">810</span><span id="line-810"> this.name = name;</span>
<span class="source-line-no">811</span><span id="line-811"> this.bucketSize = bucketSize;</span>
<span class="source-line-no">812</span><span id="line-812"> queue = new LruCachedBlockQueue(bytesToFree, blockSize);</span>
<span class="source-line-no">813</span><span id="line-813"> totalSize = 0;</span>
<span class="source-line-no">814</span><span id="line-814"> }</span>
<span class="source-line-no">815</span><span id="line-815"></span>
<span class="source-line-no">816</span><span id="line-816"> public void add(LruCachedBlock block) {</span>
<span class="source-line-no">817</span><span id="line-817"> totalSize += block.heapSize();</span>
<span class="source-line-no">818</span><span id="line-818"> queue.add(block);</span>
<span class="source-line-no">819</span><span id="line-819"> }</span>
<span class="source-line-no">820</span><span id="line-820"></span>
<span class="source-line-no">821</span><span id="line-821"> public long free(long toFree) {</span>
<span class="source-line-no">822</span><span id="line-822"> if (LOG.isTraceEnabled()) {</span>
<span class="source-line-no">823</span><span id="line-823"> LOG.trace("freeing " + StringUtils.byteDesc(toFree) + " from " + this);</span>
<span class="source-line-no">824</span><span id="line-824"> }</span>
<span class="source-line-no">825</span><span id="line-825"> LruCachedBlock cb;</span>
<span class="source-line-no">826</span><span id="line-826"> long freedBytes = 0;</span>
<span class="source-line-no">827</span><span id="line-827"> while ((cb = queue.pollLast()) != null) {</span>
<span class="source-line-no">828</span><span id="line-828"> freedBytes += evictBlock(cb, true);</span>
<span class="source-line-no">829</span><span id="line-829"> if (freedBytes &gt;= toFree) {</span>
<span class="source-line-no">830</span><span id="line-830"> return freedBytes;</span>
<span class="source-line-no">831</span><span id="line-831"> }</span>
<span class="source-line-no">832</span><span id="line-832"> }</span>
<span class="source-line-no">833</span><span id="line-833"> if (LOG.isTraceEnabled()) {</span>
<span class="source-line-no">834</span><span id="line-834"> LOG.trace("freed " + StringUtils.byteDesc(freedBytes) + " from " + this);</span>
<span class="source-line-no">835</span><span id="line-835"> }</span>
<span class="source-line-no">836</span><span id="line-836"> return freedBytes;</span>
<span class="source-line-no">837</span><span id="line-837"> }</span>
<span class="source-line-no">838</span><span id="line-838"></span>
<span class="source-line-no">839</span><span id="line-839"> public long overflow() {</span>
<span class="source-line-no">840</span><span id="line-840"> return totalSize - bucketSize;</span>
<span class="source-line-no">841</span><span id="line-841"> }</span>
<span class="source-line-no">842</span><span id="line-842"></span>
<span class="source-line-no">843</span><span id="line-843"> public long totalSize() {</span>
<span class="source-line-no">844</span><span id="line-844"> return totalSize;</span>
<span class="source-line-no">845</span><span id="line-845"> }</span>
<span class="source-line-no">846</span><span id="line-846"></span>
<span class="source-line-no">847</span><span id="line-847"> @Override</span>
<span class="source-line-no">848</span><span id="line-848"> public int compareTo(BlockBucket that) {</span>
<span class="source-line-no">849</span><span id="line-849"> return Long.compare(this.overflow(), that.overflow());</span>
<span class="source-line-no">850</span><span id="line-850"> }</span>
<span class="source-line-no">851</span><span id="line-851"></span>
<span class="source-line-no">852</span><span id="line-852"> @Override</span>
<span class="source-line-no">853</span><span id="line-853"> public boolean equals(Object that) {</span>
<span class="source-line-no">854</span><span id="line-854"> if (that == null || !(that instanceof BlockBucket)) {</span>
<span class="source-line-no">855</span><span id="line-855"> return false;</span>
<span class="source-line-no">856</span><span id="line-856"> }</span>
<span class="source-line-no">857</span><span id="line-857"> return compareTo((BlockBucket) that) == 0;</span>
<span class="source-line-no">858</span><span id="line-858"> }</span>
<span class="source-line-no">859</span><span id="line-859"></span>
<span class="source-line-no">860</span><span id="line-860"> @Override</span>
<span class="source-line-no">861</span><span id="line-861"> public int hashCode() {</span>
<span class="source-line-no">862</span><span id="line-862"> return Objects.hashCode(name, bucketSize, queue, totalSize);</span>
<span class="source-line-no">863</span><span id="line-863"> }</span>
<span class="source-line-no">864</span><span id="line-864"></span>
<span class="source-line-no">865</span><span id="line-865"> @Override</span>
<span class="source-line-no">866</span><span id="line-866"> public String toString() {</span>
<span class="source-line-no">867</span><span id="line-867"> return MoreObjects.toStringHelper(this).add("name", name)</span>
<span class="source-line-no">868</span><span id="line-868"> .add("totalSize", StringUtils.byteDesc(totalSize))</span>
<span class="source-line-no">869</span><span id="line-869"> .add("bucketSize", StringUtils.byteDesc(bucketSize)).toString();</span>
<span class="source-line-no">870</span><span id="line-870"> }</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"> /**</span>
<span class="source-line-no">874</span><span id="line-874"> * Get the maximum size of this cache.</span>
<span class="source-line-no">875</span><span id="line-875"> * @return max size in bytes</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 long getMaxSize() {</span>
<span class="source-line-no">880</span><span id="line-880"> return this.maxSize;</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 long getCurrentSize() {</span>
<span class="source-line-no">885</span><span id="line-885"> return this.size.get();</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 long getCurrentDataSize() {</span>
<span class="source-line-no">890</span><span id="line-890"> return this.dataBlockSize.sum();</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"> public long getCurrentIndexSize() {</span>
<span class="source-line-no">894</span><span id="line-894"> return this.indexBlockSize.sum();</span>
<span class="source-line-no">895</span><span id="line-895"> }</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 long getCurrentBloomSize() {</span>
<span class="source-line-no">898</span><span id="line-898"> return this.bloomBlockSize.sum();</span>
<span class="source-line-no">899</span><span id="line-899"> }</span>
<span class="source-line-no">900</span><span id="line-900"></span>
<span class="source-line-no">901</span><span id="line-901"> @Override</span>
<span class="source-line-no">902</span><span id="line-902"> public long getFreeSize() {</span>
<span class="source-line-no">903</span><span id="line-903"> return getMaxSize() - getCurrentSize();</span>
<span class="source-line-no">904</span><span id="line-904"> }</span>
<span class="source-line-no">905</span><span id="line-905"></span>
<span class="source-line-no">906</span><span id="line-906"> @Override</span>
<span class="source-line-no">907</span><span id="line-907"> public long size() {</span>
<span class="source-line-no">908</span><span id="line-908"> return getMaxSize();</span>
<span class="source-line-no">909</span><span id="line-909"> }</span>
<span class="source-line-no">910</span><span id="line-910"></span>
<span class="source-line-no">911</span><span id="line-911"> @Override</span>
<span class="source-line-no">912</span><span id="line-912"> public long getBlockCount() {</span>
<span class="source-line-no">913</span><span id="line-913"> return this.elements.get();</span>
<span class="source-line-no">914</span><span id="line-914"> }</span>
<span class="source-line-no">915</span><span id="line-915"></span>
<span class="source-line-no">916</span><span id="line-916"> @Override</span>
<span class="source-line-no">917</span><span id="line-917"> public long getDataBlockCount() {</span>
<span class="source-line-no">918</span><span id="line-918"> return this.dataBlockElements.sum();</span>
<span class="source-line-no">919</span><span id="line-919"> }</span>
<span class="source-line-no">920</span><span id="line-920"></span>
<span class="source-line-no">921</span><span id="line-921"> public long getIndexBlockCount() {</span>
<span class="source-line-no">922</span><span id="line-922"> return this.indexBlockElements.sum();</span>
<span class="source-line-no">923</span><span id="line-923"> }</span>
<span class="source-line-no">924</span><span id="line-924"></span>
<span class="source-line-no">925</span><span id="line-925"> public long getBloomBlockCount() {</span>
<span class="source-line-no">926</span><span id="line-926"> return this.bloomBlockElements.sum();</span>
<span class="source-line-no">927</span><span id="line-927"> }</span>
<span class="source-line-no">928</span><span id="line-928"></span>
<span class="source-line-no">929</span><span id="line-929"> EvictionThread getEvictionThread() {</span>
<span class="source-line-no">930</span><span id="line-930"> return this.evictionThread;</span>
<span class="source-line-no">931</span><span id="line-931"> }</span>
<span class="source-line-no">932</span><span id="line-932"></span>
<span class="source-line-no">933</span><span id="line-933"> /*</span>
<span class="source-line-no">934</span><span id="line-934"> * Eviction thread. Sits in waiting state until an eviction is triggered when the cache size grows</span>
<span class="source-line-no">935</span><span id="line-935"> * above the acceptable level.&lt;p&gt; Thread is triggered into action by {@link</span>
<span class="source-line-no">936</span><span id="line-936"> * LruBlockCache#runEviction()}</span>
<span class="source-line-no">937</span><span id="line-937"> */</span>
<span class="source-line-no">938</span><span id="line-938"> static class EvictionThread extends Thread {</span>
<span class="source-line-no">939</span><span id="line-939"></span>
<span class="source-line-no">940</span><span id="line-940"> private WeakReference&lt;LruBlockCache&gt; cache;</span>
<span class="source-line-no">941</span><span id="line-941"> private volatile boolean go = true;</span>
<span class="source-line-no">942</span><span id="line-942"> // flag set after enter the run method, used for test</span>
<span class="source-line-no">943</span><span id="line-943"> private boolean enteringRun = false;</span>
<span class="source-line-no">944</span><span id="line-944"></span>
<span class="source-line-no">945</span><span id="line-945"> public EvictionThread(LruBlockCache cache) {</span>
<span class="source-line-no">946</span><span id="line-946"> super(Thread.currentThread().getName() + ".LruBlockCache.EvictionThread");</span>
<span class="source-line-no">947</span><span id="line-947"> setDaemon(true);</span>
<span class="source-line-no">948</span><span id="line-948"> this.cache = new WeakReference&lt;&gt;(cache);</span>
<span class="source-line-no">949</span><span id="line-949"> }</span>
<span class="source-line-no">950</span><span id="line-950"></span>
<span class="source-line-no">951</span><span id="line-951"> @Override</span>
<span class="source-line-no">952</span><span id="line-952"> public void run() {</span>
<span class="source-line-no">953</span><span id="line-953"> enteringRun = true;</span>
<span class="source-line-no">954</span><span id="line-954"> while (this.go) {</span>
<span class="source-line-no">955</span><span id="line-955"> synchronized (this) {</span>
<span class="source-line-no">956</span><span id="line-956"> try {</span>
<span class="source-line-no">957</span><span id="line-957"> this.wait(1000 * 10/* Don't wait for ever */);</span>
<span class="source-line-no">958</span><span id="line-958"> } catch (InterruptedException e) {</span>
<span class="source-line-no">959</span><span id="line-959"> LOG.warn("Interrupted eviction thread ", e);</span>
<span class="source-line-no">960</span><span id="line-960"> Thread.currentThread().interrupt();</span>
<span class="source-line-no">961</span><span id="line-961"> }</span>
<span class="source-line-no">962</span><span id="line-962"> }</span>
<span class="source-line-no">963</span><span id="line-963"> LruBlockCache cache = this.cache.get();</span>
<span class="source-line-no">964</span><span id="line-964"> if (cache == null) {</span>
<span class="source-line-no">965</span><span id="line-965"> this.go = false;</span>
<span class="source-line-no">966</span><span id="line-966"> break;</span>
<span class="source-line-no">967</span><span id="line-967"> }</span>
<span class="source-line-no">968</span><span id="line-968"> cache.evict();</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"></span>
<span class="source-line-no">972</span><span id="line-972"> @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "NN_NAKED_NOTIFY",</span>
<span class="source-line-no">973</span><span id="line-973"> justification = "This is what we want")</span>
<span class="source-line-no">974</span><span id="line-974"> public void evict() {</span>
<span class="source-line-no">975</span><span id="line-975"> synchronized (this) {</span>
<span class="source-line-no">976</span><span id="line-976"> this.notifyAll();</span>
<span class="source-line-no">977</span><span id="line-977"> }</span>
<span class="source-line-no">978</span><span id="line-978"> }</span>
<span class="source-line-no">979</span><span id="line-979"></span>
<span class="source-line-no">980</span><span id="line-980"> synchronized void shutdown() {</span>
<span class="source-line-no">981</span><span id="line-981"> this.go = false;</span>
<span class="source-line-no">982</span><span id="line-982"> this.notifyAll();</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"> public boolean isGo() {</span>
<span class="source-line-no">986</span><span id="line-986"> return go;</span>
<span class="source-line-no">987</span><span id="line-987"> }</span>
<span class="source-line-no">988</span><span id="line-988"></span>
<span class="source-line-no">989</span><span id="line-989"> /**</span>
<span class="source-line-no">990</span><span id="line-990"> * Used for the test.</span>
<span class="source-line-no">991</span><span id="line-991"> */</span>
<span class="source-line-no">992</span><span id="line-992"> boolean isEnteringRun() {</span>
<span class="source-line-no">993</span><span id="line-993"> return this.enteringRun;</span>
<span class="source-line-no">994</span><span id="line-994"> }</span>
<span class="source-line-no">995</span><span id="line-995"> }</span>
<span class="source-line-no">996</span><span id="line-996"></span>
<span class="source-line-no">997</span><span id="line-997"> /*</span>
<span class="source-line-no">998</span><span id="line-998"> * Statistics thread. Periodically prints the cache statistics to the log.</span>
<span class="source-line-no">999</span><span id="line-999"> */</span>
<span class="source-line-no">1000</span><span id="line-1000"> static class StatisticsThread extends Thread {</span>
<span class="source-line-no">1001</span><span id="line-1001"></span>
<span class="source-line-no">1002</span><span id="line-1002"> private final LruBlockCache lru;</span>
<span class="source-line-no">1003</span><span id="line-1003"></span>
<span class="source-line-no">1004</span><span id="line-1004"> public StatisticsThread(LruBlockCache lru) {</span>
<span class="source-line-no">1005</span><span id="line-1005"> super("LruBlockCacheStats");</span>
<span class="source-line-no">1006</span><span id="line-1006"> setDaemon(true);</span>
<span class="source-line-no">1007</span><span id="line-1007"> this.lru = lru;</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"> @Override</span>
<span class="source-line-no">1011</span><span id="line-1011"> public void run() {</span>
<span class="source-line-no">1012</span><span id="line-1012"> lru.logStats();</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"></span>
<span class="source-line-no">1016</span><span id="line-1016"> public void logStats() {</span>
<span class="source-line-no">1017</span><span id="line-1017"> // Log size</span>
<span class="source-line-no">1018</span><span id="line-1018"> long usedSize = heapSize();</span>
<span class="source-line-no">1019</span><span id="line-1019"> long freeSize = maxSize - usedSize;</span>
<span class="source-line-no">1020</span><span id="line-1020"> LruBlockCache.LOG.info("totalSize=" + StringUtils.byteDesc(maxSize) + ", " + "usedSize="</span>
<span class="source-line-no">1021</span><span id="line-1021"> + StringUtils.byteDesc(usedSize) + ", " + "freeSize=" + StringUtils.byteDesc(freeSize) + ", "</span>
<span class="source-line-no">1022</span><span id="line-1022"> + "max=" + StringUtils.byteDesc(this.maxSize) + ", " + "blockCount=" + getBlockCount() + ", "</span>
<span class="source-line-no">1023</span><span id="line-1023"> + "accesses=" + stats.getRequestCount() + ", " + "hits=" + stats.getHitCount() + ", "</span>
<span class="source-line-no">1024</span><span id="line-1024"> + "hitRatio="</span>
<span class="source-line-no">1025</span><span id="line-1025"> + (stats.getHitCount() == 0</span>
<span class="source-line-no">1026</span><span id="line-1026"> ? "0"</span>
<span class="source-line-no">1027</span><span id="line-1027"> : (StringUtils.formatPercent(stats.getHitRatio(), 2) + ", "))</span>
<span class="source-line-no">1028</span><span id="line-1028"> + ", " + "cachingAccesses=" + stats.getRequestCachingCount() + ", " + "cachingHits="</span>
<span class="source-line-no">1029</span><span id="line-1029"> + stats.getHitCachingCount() + ", " + "cachingHitsRatio="</span>
<span class="source-line-no">1030</span><span id="line-1030"> + (stats.getHitCachingCount() == 0</span>
<span class="source-line-no">1031</span><span id="line-1031"> ? "0,"</span>
<span class="source-line-no">1032</span><span id="line-1032"> : (StringUtils.formatPercent(stats.getHitCachingRatio(), 2) + ", "))</span>
<span class="source-line-no">1033</span><span id="line-1033"> + "evictions=" + stats.getEvictionCount() + ", " + "evicted=" + stats.getEvictedCount() + ", "</span>
<span class="source-line-no">1034</span><span id="line-1034"> + "evictedPerRun=" + stats.evictedPerEviction());</span>
<span class="source-line-no">1035</span><span id="line-1035"> }</span>
<span class="source-line-no">1036</span><span id="line-1036"></span>
<span class="source-line-no">1037</span><span id="line-1037"> /**</span>
<span class="source-line-no">1038</span><span id="line-1038"> * Get counter statistics for this cache.</span>
<span class="source-line-no">1039</span><span id="line-1039"> * &lt;p&gt;</span>
<span class="source-line-no">1040</span><span id="line-1040"> * Includes: total accesses, hits, misses, evicted blocks, and runs of the eviction processes.</span>
<span class="source-line-no">1041</span><span id="line-1041"> */</span>
<span class="source-line-no">1042</span><span id="line-1042"> @Override</span>
<span class="source-line-no">1043</span><span id="line-1043"> public CacheStats getStats() {</span>
<span class="source-line-no">1044</span><span id="line-1044"> return this.stats;</span>
<span class="source-line-no">1045</span><span id="line-1045"> }</span>
<span class="source-line-no">1046</span><span id="line-1046"></span>
<span class="source-line-no">1047</span><span id="line-1047"> public final static long CACHE_FIXED_OVERHEAD =</span>
<span class="source-line-no">1048</span><span id="line-1048"> ClassSize.estimateBase(LruBlockCache.class, false);</span>
<span class="source-line-no">1049</span><span id="line-1049"></span>
<span class="source-line-no">1050</span><span id="line-1050"> @Override</span>
<span class="source-line-no">1051</span><span id="line-1051"> public long heapSize() {</span>
<span class="source-line-no">1052</span><span id="line-1052"> return getCurrentSize();</span>
<span class="source-line-no">1053</span><span id="line-1053"> }</span>
<span class="source-line-no">1054</span><span id="line-1054"></span>
<span class="source-line-no">1055</span><span id="line-1055"> private static long calculateOverhead(long maxSize, long blockSize, int concurrency) {</span>
<span class="source-line-no">1056</span><span id="line-1056"> // FindBugs ICAST_INTEGER_MULTIPLY_CAST_TO_LONG</span>
<span class="source-line-no">1057</span><span id="line-1057"> return CACHE_FIXED_OVERHEAD + ClassSize.CONCURRENT_HASHMAP</span>
<span class="source-line-no">1058</span><span id="line-1058"> + ((long) Math.ceil(maxSize * 1.2 / blockSize) * ClassSize.CONCURRENT_HASHMAP_ENTRY)</span>
<span class="source-line-no">1059</span><span id="line-1059"> + ((long) concurrency * ClassSize.CONCURRENT_HASHMAP_SEGMENT);</span>
<span class="source-line-no">1060</span><span id="line-1060"> }</span>
<span class="source-line-no">1061</span><span id="line-1061"></span>
<span class="source-line-no">1062</span><span id="line-1062"> @Override</span>
<span class="source-line-no">1063</span><span id="line-1063"> public Iterator&lt;CachedBlock&gt; iterator() {</span>
<span class="source-line-no">1064</span><span id="line-1064"> final Iterator&lt;LruCachedBlock&gt; iterator = map.values().iterator();</span>
<span class="source-line-no">1065</span><span id="line-1065"></span>
<span class="source-line-no">1066</span><span id="line-1066"> return new Iterator&lt;CachedBlock&gt;() {</span>
<span class="source-line-no">1067</span><span id="line-1067"> private final long now = System.nanoTime();</span>
<span class="source-line-no">1068</span><span id="line-1068"></span>
<span class="source-line-no">1069</span><span id="line-1069"> @Override</span>
<span class="source-line-no">1070</span><span id="line-1070"> public boolean hasNext() {</span>
<span class="source-line-no">1071</span><span id="line-1071"> return iterator.hasNext();</span>
<span class="source-line-no">1072</span><span id="line-1072"> }</span>
<span class="source-line-no">1073</span><span id="line-1073"></span>
<span class="source-line-no">1074</span><span id="line-1074"> @Override</span>
<span class="source-line-no">1075</span><span id="line-1075"> public CachedBlock next() {</span>
<span class="source-line-no">1076</span><span id="line-1076"> final LruCachedBlock b = iterator.next();</span>
<span class="source-line-no">1077</span><span id="line-1077"> return new CachedBlock() {</span>
<span class="source-line-no">1078</span><span id="line-1078"> @Override</span>
<span class="source-line-no">1079</span><span id="line-1079"> public String toString() {</span>
<span class="source-line-no">1080</span><span id="line-1080"> return BlockCacheUtil.toString(this, now);</span>
<span class="source-line-no">1081</span><span id="line-1081"> }</span>
<span class="source-line-no">1082</span><span id="line-1082"></span>
<span class="source-line-no">1083</span><span id="line-1083"> @Override</span>
<span class="source-line-no">1084</span><span id="line-1084"> public BlockPriority getBlockPriority() {</span>
<span class="source-line-no">1085</span><span id="line-1085"> return b.getPriority();</span>
<span class="source-line-no">1086</span><span id="line-1086"> }</span>
<span class="source-line-no">1087</span><span id="line-1087"></span>
<span class="source-line-no">1088</span><span id="line-1088"> @Override</span>
<span class="source-line-no">1089</span><span id="line-1089"> public BlockType getBlockType() {</span>
<span class="source-line-no">1090</span><span id="line-1090"> return b.getBuffer().getBlockType();</span>
<span class="source-line-no">1091</span><span id="line-1091"> }</span>
<span class="source-line-no">1092</span><span id="line-1092"></span>
<span class="source-line-no">1093</span><span id="line-1093"> @Override</span>
<span class="source-line-no">1094</span><span id="line-1094"> public long getOffset() {</span>
<span class="source-line-no">1095</span><span id="line-1095"> return b.getCacheKey().getOffset();</span>
<span class="source-line-no">1096</span><span id="line-1096"> }</span>
<span class="source-line-no">1097</span><span id="line-1097"></span>
<span class="source-line-no">1098</span><span id="line-1098"> @Override</span>
<span class="source-line-no">1099</span><span id="line-1099"> public long getSize() {</span>
<span class="source-line-no">1100</span><span id="line-1100"> return b.getBuffer().heapSize();</span>
<span class="source-line-no">1101</span><span id="line-1101"> }</span>
<span class="source-line-no">1102</span><span id="line-1102"></span>
<span class="source-line-no">1103</span><span id="line-1103"> @Override</span>
<span class="source-line-no">1104</span><span id="line-1104"> public long getCachedTime() {</span>
<span class="source-line-no">1105</span><span id="line-1105"> return b.getCachedTime();</span>
<span class="source-line-no">1106</span><span id="line-1106"> }</span>
<span class="source-line-no">1107</span><span id="line-1107"></span>
<span class="source-line-no">1108</span><span id="line-1108"> @Override</span>
<span class="source-line-no">1109</span><span id="line-1109"> public String getFilename() {</span>
<span class="source-line-no">1110</span><span id="line-1110"> return b.getCacheKey().getHfileName();</span>
<span class="source-line-no">1111</span><span id="line-1111"> }</span>
<span class="source-line-no">1112</span><span id="line-1112"></span>
<span class="source-line-no">1113</span><span id="line-1113"> @Override</span>
<span class="source-line-no">1114</span><span id="line-1114"> public int compareTo(CachedBlock other) {</span>
<span class="source-line-no">1115</span><span id="line-1115"> int diff = this.getFilename().compareTo(other.getFilename());</span>
<span class="source-line-no">1116</span><span id="line-1116"> if (diff != 0) {</span>
<span class="source-line-no">1117</span><span id="line-1117"> return diff;</span>
<span class="source-line-no">1118</span><span id="line-1118"> }</span>
<span class="source-line-no">1119</span><span id="line-1119"> diff = Long.compare(this.getOffset(), other.getOffset());</span>
<span class="source-line-no">1120</span><span id="line-1120"> if (diff != 0) {</span>
<span class="source-line-no">1121</span><span id="line-1121"> return diff;</span>
<span class="source-line-no">1122</span><span id="line-1122"> }</span>
<span class="source-line-no">1123</span><span id="line-1123"> if (other.getCachedTime() &lt; 0 || this.getCachedTime() &lt; 0) {</span>
<span class="source-line-no">1124</span><span id="line-1124"> throw new IllegalStateException(this.getCachedTime() + ", " + other.getCachedTime());</span>
<span class="source-line-no">1125</span><span id="line-1125"> }</span>
<span class="source-line-no">1126</span><span id="line-1126"> return Long.compare(other.getCachedTime(), this.getCachedTime());</span>
<span class="source-line-no">1127</span><span id="line-1127"> }</span>
<span class="source-line-no">1128</span><span id="line-1128"></span>
<span class="source-line-no">1129</span><span id="line-1129"> @Override</span>
<span class="source-line-no">1130</span><span id="line-1130"> public int hashCode() {</span>
<span class="source-line-no">1131</span><span id="line-1131"> return b.hashCode();</span>
<span class="source-line-no">1132</span><span id="line-1132"> }</span>
<span class="source-line-no">1133</span><span id="line-1133"></span>
<span class="source-line-no">1134</span><span id="line-1134"> @Override</span>
<span class="source-line-no">1135</span><span id="line-1135"> public boolean equals(Object obj) {</span>
<span class="source-line-no">1136</span><span id="line-1136"> if (obj instanceof CachedBlock) {</span>
<span class="source-line-no">1137</span><span id="line-1137"> CachedBlock cb = (CachedBlock) obj;</span>
<span class="source-line-no">1138</span><span id="line-1138"> return compareTo(cb) == 0;</span>
<span class="source-line-no">1139</span><span id="line-1139"> } else {</span>
<span class="source-line-no">1140</span><span id="line-1140"> return false;</span>
<span class="source-line-no">1141</span><span id="line-1141"> }</span>
<span class="source-line-no">1142</span><span id="line-1142"> }</span>
<span class="source-line-no">1143</span><span id="line-1143"> };</span>
<span class="source-line-no">1144</span><span id="line-1144"> }</span>
<span class="source-line-no">1145</span><span id="line-1145"></span>
<span class="source-line-no">1146</span><span id="line-1146"> @Override</span>
<span class="source-line-no">1147</span><span id="line-1147"> public void remove() {</span>
<span class="source-line-no">1148</span><span id="line-1148"> throw new UnsupportedOperationException();</span>
<span class="source-line-no">1149</span><span id="line-1149"> }</span>
<span class="source-line-no">1150</span><span id="line-1150"> };</span>
<span class="source-line-no">1151</span><span id="line-1151"> }</span>
<span class="source-line-no">1152</span><span id="line-1152"></span>
<span class="source-line-no">1153</span><span id="line-1153"> // Simple calculators of sizes given factors and maxSize</span>
<span class="source-line-no">1154</span><span id="line-1154"></span>
<span class="source-line-no">1155</span><span id="line-1155"> long acceptableSize() {</span>
<span class="source-line-no">1156</span><span id="line-1156"> return (long) Math.floor(this.maxSize * this.acceptableFactor);</span>
<span class="source-line-no">1157</span><span id="line-1157"> }</span>
<span class="source-line-no">1158</span><span id="line-1158"></span>
<span class="source-line-no">1159</span><span id="line-1159"> private long minSize() {</span>
<span class="source-line-no">1160</span><span id="line-1160"> return (long) Math.floor(this.maxSize * this.minFactor);</span>
<span class="source-line-no">1161</span><span id="line-1161"> }</span>
<span class="source-line-no">1162</span><span id="line-1162"></span>
<span class="source-line-no">1163</span><span id="line-1163"> private long singleSize() {</span>
<span class="source-line-no">1164</span><span id="line-1164"> return (long) Math.floor(this.maxSize * this.singleFactor * this.minFactor);</span>
<span class="source-line-no">1165</span><span id="line-1165"> }</span>
<span class="source-line-no">1166</span><span id="line-1166"></span>
<span class="source-line-no">1167</span><span id="line-1167"> private long multiSize() {</span>
<span class="source-line-no">1168</span><span id="line-1168"> return (long) Math.floor(this.maxSize * this.multiFactor * this.minFactor);</span>
<span class="source-line-no">1169</span><span id="line-1169"> }</span>
<span class="source-line-no">1170</span><span id="line-1170"></span>
<span class="source-line-no">1171</span><span id="line-1171"> private long memorySize() {</span>
<span class="source-line-no">1172</span><span id="line-1172"> return (long) Math.floor(this.maxSize * this.memoryFactor * this.minFactor);</span>
<span class="source-line-no">1173</span><span id="line-1173"> }</span>
<span class="source-line-no">1174</span><span id="line-1174"></span>
<span class="source-line-no">1175</span><span id="line-1175"> @Override</span>
<span class="source-line-no">1176</span><span id="line-1176"> public void shutdown() {</span>
<span class="source-line-no">1177</span><span id="line-1177"> if (victimHandler != null) {</span>
<span class="source-line-no">1178</span><span id="line-1178"> victimHandler.shutdown();</span>
<span class="source-line-no">1179</span><span id="line-1179"> }</span>
<span class="source-line-no">1180</span><span id="line-1180"> this.scheduleThreadPool.shutdown();</span>
<span class="source-line-no">1181</span><span id="line-1181"> for (int i = 0; i &lt; 10; i++) {</span>
<span class="source-line-no">1182</span><span id="line-1182"> if (!this.scheduleThreadPool.isShutdown()) {</span>
<span class="source-line-no">1183</span><span id="line-1183"> try {</span>
<span class="source-line-no">1184</span><span id="line-1184"> Thread.sleep(10);</span>
<span class="source-line-no">1185</span><span id="line-1185"> } catch (InterruptedException e) {</span>
<span class="source-line-no">1186</span><span id="line-1186"> LOG.warn("Interrupted while sleeping");</span>
<span class="source-line-no">1187</span><span id="line-1187"> Thread.currentThread().interrupt();</span>
<span class="source-line-no">1188</span><span id="line-1188"> break;</span>
<span class="source-line-no">1189</span><span id="line-1189"> }</span>
<span class="source-line-no">1190</span><span id="line-1190"> }</span>
<span class="source-line-no">1191</span><span id="line-1191"> }</span>
<span class="source-line-no">1192</span><span id="line-1192"></span>
<span class="source-line-no">1193</span><span id="line-1193"> if (!this.scheduleThreadPool.isShutdown()) {</span>
<span class="source-line-no">1194</span><span id="line-1194"> List&lt;Runnable&gt; runnables = this.scheduleThreadPool.shutdownNow();</span>
<span class="source-line-no">1195</span><span id="line-1195"> LOG.debug("Still running " + runnables);</span>
<span class="source-line-no">1196</span><span id="line-1196"> }</span>
<span class="source-line-no">1197</span><span id="line-1197"> this.evictionThread.shutdown();</span>
<span class="source-line-no">1198</span><span id="line-1198"> }</span>
<span class="source-line-no">1199</span><span id="line-1199"></span>
<span class="source-line-no">1200</span><span id="line-1200"> /** Clears the cache. Used in tests. */</span>
<span class="source-line-no">1201</span><span id="line-1201"> public void clearCache() {</span>
<span class="source-line-no">1202</span><span id="line-1202"> this.map.clear();</span>
<span class="source-line-no">1203</span><span id="line-1203"> this.elements.set(0);</span>
<span class="source-line-no">1204</span><span id="line-1204"> }</span>
<span class="source-line-no">1205</span><span id="line-1205"></span>
<span class="source-line-no">1206</span><span id="line-1206"> /**</span>
<span class="source-line-no">1207</span><span id="line-1207"> * Used in testing. May be very inefficient.</span>
<span class="source-line-no">1208</span><span id="line-1208"> * @return the set of cached file names</span>
<span class="source-line-no">1209</span><span id="line-1209"> */</span>
<span class="source-line-no">1210</span><span id="line-1210"> SortedSet&lt;String&gt; getCachedFileNamesForTest() {</span>
<span class="source-line-no">1211</span><span id="line-1211"> SortedSet&lt;String&gt; fileNames = new TreeSet&lt;&gt;();</span>
<span class="source-line-no">1212</span><span id="line-1212"> for (BlockCacheKey cacheKey : map.keySet()) {</span>
<span class="source-line-no">1213</span><span id="line-1213"> fileNames.add(cacheKey.getHfileName());</span>
<span class="source-line-no">1214</span><span id="line-1214"> }</span>
<span class="source-line-no">1215</span><span id="line-1215"> return fileNames;</span>
<span class="source-line-no">1216</span><span id="line-1216"> }</span>
<span class="source-line-no">1217</span><span id="line-1217"></span>
<span class="source-line-no">1218</span><span id="line-1218"> public Map&lt;DataBlockEncoding, Integer&gt; getEncodingCountsForTest() {</span>
<span class="source-line-no">1219</span><span id="line-1219"> Map&lt;DataBlockEncoding, Integer&gt; counts = new EnumMap&lt;&gt;(DataBlockEncoding.class);</span>
<span class="source-line-no">1220</span><span id="line-1220"> for (LruCachedBlock block : map.values()) {</span>
<span class="source-line-no">1221</span><span id="line-1221"> DataBlockEncoding encoding = ((HFileBlock) block.getBuffer()).getDataBlockEncoding();</span>
<span class="source-line-no">1222</span><span id="line-1222"> Integer count = counts.get(encoding);</span>
<span class="source-line-no">1223</span><span id="line-1223"> counts.put(encoding, (count == null ? 0 : count) + 1);</span>
<span class="source-line-no">1224</span><span id="line-1224"> }</span>
<span class="source-line-no">1225</span><span id="line-1225"> return counts;</span>
<span class="source-line-no">1226</span><span id="line-1226"> }</span>
<span class="source-line-no">1227</span><span id="line-1227"></span>
<span class="source-line-no">1228</span><span id="line-1228"> Map&lt;BlockCacheKey, LruCachedBlock&gt; getMapForTests() {</span>
<span class="source-line-no">1229</span><span id="line-1229"> return map;</span>
<span class="source-line-no">1230</span><span id="line-1230"> }</span>
<span class="source-line-no">1231</span><span id="line-1231"></span>
<span class="source-line-no">1232</span><span id="line-1232"> @Override</span>
<span class="source-line-no">1233</span><span id="line-1233"> public BlockCache[] getBlockCaches() {</span>
<span class="source-line-no">1234</span><span id="line-1234"> if (victimHandler != null) {</span>
<span class="source-line-no">1235</span><span id="line-1235"> return new BlockCache[] { this, this.victimHandler };</span>
<span class="source-line-no">1236</span><span id="line-1236"> }</span>
<span class="source-line-no">1237</span><span id="line-1237"> return null;</span>
<span class="source-line-no">1238</span><span id="line-1238"> }</span>
<span class="source-line-no">1239</span><span id="line-1239">}</span>
</pre>
</div>
</main>
</body>
</html>