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