blob: 0240ad78744840a051d6f069f670df77adcdfbe5 [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> *<a name="line.2"></a>
<span class="sourceLineNo">003</span> * Licensed to the Apache Software Foundation (ASF) under one<a name="line.3"></a>
<span class="sourceLineNo">004</span> * or more contributor license agreements. See the NOTICE file<a name="line.4"></a>
<span class="sourceLineNo">005</span> * distributed with this work for additional information<a name="line.5"></a>
<span class="sourceLineNo">006</span> * regarding copyright ownership. The ASF licenses this file<a name="line.6"></a>
<span class="sourceLineNo">007</span> * to you under the Apache License, Version 2.0 (the<a name="line.7"></a>
<span class="sourceLineNo">008</span> * "License"); you may not use this file except in compliance<a name="line.8"></a>
<span class="sourceLineNo">009</span> * with the License. You may obtain a copy of the License at<a name="line.9"></a>
<span class="sourceLineNo">010</span> *<a name="line.10"></a>
<span class="sourceLineNo">011</span> * http://www.apache.org/licenses/LICENSE-2.0<a name="line.11"></a>
<span class="sourceLineNo">012</span> *<a name="line.12"></a>
<span class="sourceLineNo">013</span> * Unless required by applicable law or agreed to in writing, software<a name="line.13"></a>
<span class="sourceLineNo">014</span> * distributed under the License is distributed on an "AS IS" BASIS,<a name="line.14"></a>
<span class="sourceLineNo">015</span> * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.<a name="line.15"></a>
<span class="sourceLineNo">016</span> * See the License for the specific language governing permissions and<a name="line.16"></a>
<span class="sourceLineNo">017</span> * limitations under the License.<a name="line.17"></a>
<span class="sourceLineNo">018</span> */<a name="line.18"></a>
<span class="sourceLineNo">019</span>package org.apache.hadoop.hbase.regionserver;<a name="line.19"></a>
<span class="sourceLineNo">020</span><a name="line.20"></a>
<span class="sourceLineNo">021</span>import java.nio.ByteBuffer;<a name="line.21"></a>
<span class="sourceLineNo">022</span>import java.util.Set;<a name="line.22"></a>
<span class="sourceLineNo">023</span>import java.util.concurrent.BlockingQueue;<a name="line.23"></a>
<span class="sourceLineNo">024</span>import java.util.concurrent.ConcurrentSkipListSet;<a name="line.24"></a>
<span class="sourceLineNo">025</span>import java.util.concurrent.LinkedBlockingQueue;<a name="line.25"></a>
<span class="sourceLineNo">026</span>import java.util.concurrent.atomic.AtomicBoolean;<a name="line.26"></a>
<span class="sourceLineNo">027</span>import java.util.concurrent.atomic.AtomicInteger;<a name="line.27"></a>
<span class="sourceLineNo">028</span>import java.util.concurrent.atomic.AtomicReference;<a name="line.28"></a>
<span class="sourceLineNo">029</span>import java.util.concurrent.locks.ReentrantLock;<a name="line.29"></a>
<span class="sourceLineNo">030</span><a name="line.30"></a>
<span class="sourceLineNo">031</span>import org.apache.hadoop.conf.Configuration;<a name="line.31"></a>
<span class="sourceLineNo">032</span>import org.apache.hadoop.hbase.ByteBufferExtendedCell;<a name="line.32"></a>
<span class="sourceLineNo">033</span>import org.apache.hadoop.hbase.Cell;<a name="line.33"></a>
<span class="sourceLineNo">034</span>import org.apache.hadoop.hbase.ExtendedCell;<a name="line.34"></a>
<span class="sourceLineNo">035</span>import org.apache.hadoop.hbase.KeyValueUtil;<a name="line.35"></a>
<span class="sourceLineNo">036</span>import org.apache.yetus.audience.InterfaceAudience;<a name="line.36"></a>
<span class="sourceLineNo">037</span>import org.slf4j.Logger;<a name="line.37"></a>
<span class="sourceLineNo">038</span>import org.slf4j.LoggerFactory;<a name="line.38"></a>
<span class="sourceLineNo">039</span>import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;<a name="line.39"></a>
<span class="sourceLineNo">040</span>import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;<a name="line.40"></a>
<span class="sourceLineNo">041</span>/**<a name="line.41"></a>
<span class="sourceLineNo">042</span> * A memstore-local allocation buffer.<a name="line.42"></a>
<span class="sourceLineNo">043</span> * &lt;p&gt;<a name="line.43"></a>
<span class="sourceLineNo">044</span> * The MemStoreLAB is basically a bump-the-pointer allocator that allocates<a name="line.44"></a>
<span class="sourceLineNo">045</span> * big (2MB) byte[] chunks from and then doles it out to threads that request<a name="line.45"></a>
<span class="sourceLineNo">046</span> * slices into the array.<a name="line.46"></a>
<span class="sourceLineNo">047</span> * &lt;p&gt;<a name="line.47"></a>
<span class="sourceLineNo">048</span> * The purpose of this class is to combat heap fragmentation in the<a name="line.48"></a>
<span class="sourceLineNo">049</span> * regionserver. By ensuring that all Cells in a given memstore refer<a name="line.49"></a>
<span class="sourceLineNo">050</span> * only to large chunks of contiguous memory, we ensure that large blocks<a name="line.50"></a>
<span class="sourceLineNo">051</span> * get freed up when the memstore is flushed.<a name="line.51"></a>
<span class="sourceLineNo">052</span> * &lt;p&gt;<a name="line.52"></a>
<span class="sourceLineNo">053</span> * Without the MSLAB, the byte array allocated during insertion end up<a name="line.53"></a>
<span class="sourceLineNo">054</span> * interleaved throughout the heap, and the old generation gets progressively<a name="line.54"></a>
<span class="sourceLineNo">055</span> * more fragmented until a stop-the-world compacting collection occurs.<a name="line.55"></a>
<span class="sourceLineNo">056</span> * &lt;p&gt;<a name="line.56"></a>
<span class="sourceLineNo">057</span> * TODO: we should probably benchmark whether word-aligning the allocations<a name="line.57"></a>
<span class="sourceLineNo">058</span> * would provide a performance improvement - probably would speed up the<a name="line.58"></a>
<span class="sourceLineNo">059</span> * Bytes.toLong/Bytes.toInt calls in KeyValue, but some of those are cached<a name="line.59"></a>
<span class="sourceLineNo">060</span> * anyway.<a name="line.60"></a>
<span class="sourceLineNo">061</span> * The chunks created by this MemStoreLAB can get pooled at {@link ChunkCreator}.<a name="line.61"></a>
<span class="sourceLineNo">062</span> * When the Chunk comes from pool, it can be either an on heap or an off heap backed chunk. The chunks,<a name="line.62"></a>
<span class="sourceLineNo">063</span> * which this MemStoreLAB creates on its own (when no chunk available from pool), those will be<a name="line.63"></a>
<span class="sourceLineNo">064</span> * always on heap backed.<a name="line.64"></a>
<span class="sourceLineNo">065</span> */<a name="line.65"></a>
<span class="sourceLineNo">066</span>@InterfaceAudience.Private<a name="line.66"></a>
<span class="sourceLineNo">067</span>public class MemStoreLABImpl implements MemStoreLAB {<a name="line.67"></a>
<span class="sourceLineNo">068</span><a name="line.68"></a>
<span class="sourceLineNo">069</span> static final Logger LOG = LoggerFactory.getLogger(MemStoreLABImpl.class);<a name="line.69"></a>
<span class="sourceLineNo">070</span><a name="line.70"></a>
<span class="sourceLineNo">071</span> private AtomicReference&lt;Chunk&gt; currChunk = new AtomicReference&lt;&gt;();<a name="line.71"></a>
<span class="sourceLineNo">072</span> // Lock to manage multiple handlers requesting for a chunk<a name="line.72"></a>
<span class="sourceLineNo">073</span> private ReentrantLock lock = new ReentrantLock();<a name="line.73"></a>
<span class="sourceLineNo">074</span><a name="line.74"></a>
<span class="sourceLineNo">075</span> // A set of chunks contained by this memstore LAB<a name="line.75"></a>
<span class="sourceLineNo">076</span> @VisibleForTesting<a name="line.76"></a>
<span class="sourceLineNo">077</span> Set&lt;Integer&gt; chunks = new ConcurrentSkipListSet&lt;Integer&gt;();<a name="line.77"></a>
<span class="sourceLineNo">078</span> private final int dataChunkSize;<a name="line.78"></a>
<span class="sourceLineNo">079</span> private final int maxAlloc;<a name="line.79"></a>
<span class="sourceLineNo">080</span> private final ChunkCreator chunkCreator;<a name="line.80"></a>
<span class="sourceLineNo">081</span> private final CompactingMemStore.IndexType idxType; // what index is used for corresponding segment<a name="line.81"></a>
<span class="sourceLineNo">082</span><a name="line.82"></a>
<span class="sourceLineNo">083</span> // This flag is for closing this instance, its set when clearing snapshot of<a name="line.83"></a>
<span class="sourceLineNo">084</span> // memstore<a name="line.84"></a>
<span class="sourceLineNo">085</span> private volatile boolean closed = false;<a name="line.85"></a>
<span class="sourceLineNo">086</span> // This flag is for reclaiming chunks. Its set when putting chunks back to<a name="line.86"></a>
<span class="sourceLineNo">087</span> // pool<a name="line.87"></a>
<span class="sourceLineNo">088</span> private AtomicBoolean reclaimed = new AtomicBoolean(false);<a name="line.88"></a>
<span class="sourceLineNo">089</span> // Current count of open scanners which reading data from this MemStoreLAB<a name="line.89"></a>
<span class="sourceLineNo">090</span> private final AtomicInteger openScannerCount = new AtomicInteger();<a name="line.90"></a>
<span class="sourceLineNo">091</span><a name="line.91"></a>
<span class="sourceLineNo">092</span> // Used in testing<a name="line.92"></a>
<span class="sourceLineNo">093</span> public MemStoreLABImpl() {<a name="line.93"></a>
<span class="sourceLineNo">094</span> this(new Configuration());<a name="line.94"></a>
<span class="sourceLineNo">095</span> }<a name="line.95"></a>
<span class="sourceLineNo">096</span><a name="line.96"></a>
<span class="sourceLineNo">097</span> public MemStoreLABImpl(Configuration conf) {<a name="line.97"></a>
<span class="sourceLineNo">098</span> dataChunkSize = conf.getInt(CHUNK_SIZE_KEY, CHUNK_SIZE_DEFAULT);<a name="line.98"></a>
<span class="sourceLineNo">099</span> maxAlloc = conf.getInt(MAX_ALLOC_KEY, MAX_ALLOC_DEFAULT);<a name="line.99"></a>
<span class="sourceLineNo">100</span> this.chunkCreator = ChunkCreator.getInstance();<a name="line.100"></a>
<span class="sourceLineNo">101</span> // if we don't exclude allocations &gt;CHUNK_SIZE, we'd infiniteloop on one!<a name="line.101"></a>
<span class="sourceLineNo">102</span> Preconditions.checkArgument(maxAlloc &lt;= dataChunkSize,<a name="line.102"></a>
<span class="sourceLineNo">103</span> MAX_ALLOC_KEY + " must be less than " + CHUNK_SIZE_KEY);<a name="line.103"></a>
<span class="sourceLineNo">104</span><a name="line.104"></a>
<span class="sourceLineNo">105</span> // if user requested to work with MSLABs (whether on- or off-heap), then the<a name="line.105"></a>
<span class="sourceLineNo">106</span> // immutable segments are going to use CellChunkMap as their index<a name="line.106"></a>
<span class="sourceLineNo">107</span> idxType = CompactingMemStore.IndexType.CHUNK_MAP;<a name="line.107"></a>
<span class="sourceLineNo">108</span> }<a name="line.108"></a>
<span class="sourceLineNo">109</span><a name="line.109"></a>
<span class="sourceLineNo">110</span> @Override<a name="line.110"></a>
<span class="sourceLineNo">111</span> public Cell copyCellInto(Cell cell) {<a name="line.111"></a>
<span class="sourceLineNo">112</span> // See head of copyBBECellInto for how it differs from copyCellInto<a name="line.112"></a>
<span class="sourceLineNo">113</span> return (cell instanceof ByteBufferExtendedCell)?<a name="line.113"></a>
<span class="sourceLineNo">114</span> copyBBECellInto((ByteBufferExtendedCell)cell, maxAlloc):<a name="line.114"></a>
<span class="sourceLineNo">115</span> copyCellInto(cell, maxAlloc);<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> /**<a name="line.118"></a>
<span class="sourceLineNo">119</span> * When a cell's size is too big (bigger than maxAlloc),<a name="line.119"></a>
<span class="sourceLineNo">120</span> * copyCellInto does not allocate it on MSLAB.<a name="line.120"></a>
<span class="sourceLineNo">121</span> * Since the process of flattening to CellChunkMap assumes that<a name="line.121"></a>
<span class="sourceLineNo">122</span> * all cells are allocated on MSLAB, during this process,<a name="line.122"></a>
<span class="sourceLineNo">123</span> * the big cells are copied into MSLAB using this method.<a name="line.123"></a>
<span class="sourceLineNo">124</span> */<a name="line.124"></a>
<span class="sourceLineNo">125</span> @Override<a name="line.125"></a>
<span class="sourceLineNo">126</span> public Cell forceCopyOfBigCellInto(Cell cell) {<a name="line.126"></a>
<span class="sourceLineNo">127</span> int size = Segment.getCellLength(cell);<a name="line.127"></a>
<span class="sourceLineNo">128</span> size += ChunkCreator.SIZEOF_CHUNK_HEADER;<a name="line.128"></a>
<span class="sourceLineNo">129</span> Preconditions.checkArgument(size &gt;= 0, "negative size");<a name="line.129"></a>
<span class="sourceLineNo">130</span> if (size &lt;= dataChunkSize) {<a name="line.130"></a>
<span class="sourceLineNo">131</span> // Using copyCellInto for cells which are bigger than the original maxAlloc<a name="line.131"></a>
<span class="sourceLineNo">132</span> return copyCellInto(cell, dataChunkSize);<a name="line.132"></a>
<span class="sourceLineNo">133</span> } else {<a name="line.133"></a>
<span class="sourceLineNo">134</span> Chunk c = getNewExternalChunk(size);<a name="line.134"></a>
<span class="sourceLineNo">135</span> int allocOffset = c.alloc(size);<a name="line.135"></a>
<span class="sourceLineNo">136</span> return copyToChunkCell(cell, c.getData(), allocOffset, size);<a name="line.136"></a>
<span class="sourceLineNo">137</span> }<a name="line.137"></a>
<span class="sourceLineNo">138</span> }<a name="line.138"></a>
<span class="sourceLineNo">139</span><a name="line.139"></a>
<span class="sourceLineNo">140</span> /**<a name="line.140"></a>
<span class="sourceLineNo">141</span> * Mostly a duplicate of {@link #copyCellInto(Cell, int)}} done for perf sake. It presumes<a name="line.141"></a>
<span class="sourceLineNo">142</span> * ByteBufferExtendedCell instead of Cell so we deal with a specific type rather than the<a name="line.142"></a>
<span class="sourceLineNo">143</span> * super generic Cell. Removes instanceof checks. Shrinkage is enough to make this inline where<a name="line.143"></a>
<span class="sourceLineNo">144</span> * before it was too big. Uses less CPU. See HBASE-20875 for evidence.<a name="line.144"></a>
<span class="sourceLineNo">145</span> * @see #copyCellInto(Cell, int)<a name="line.145"></a>
<span class="sourceLineNo">146</span> */<a name="line.146"></a>
<span class="sourceLineNo">147</span> private Cell copyBBECellInto(ByteBufferExtendedCell cell, int maxAlloc) {<a name="line.147"></a>
<span class="sourceLineNo">148</span> int size = cell.getSerializedSize();<a name="line.148"></a>
<span class="sourceLineNo">149</span> Preconditions.checkArgument(size &gt;= 0, "negative size");<a name="line.149"></a>
<span class="sourceLineNo">150</span> // Callers should satisfy large allocations from JVM heap so limit fragmentation.<a name="line.150"></a>
<span class="sourceLineNo">151</span> if (size &gt; maxAlloc) {<a name="line.151"></a>
<span class="sourceLineNo">152</span> return null;<a name="line.152"></a>
<span class="sourceLineNo">153</span> }<a name="line.153"></a>
<span class="sourceLineNo">154</span> Chunk c = null;<a name="line.154"></a>
<span class="sourceLineNo">155</span> int allocOffset = 0;<a name="line.155"></a>
<span class="sourceLineNo">156</span> while (true) {<a name="line.156"></a>
<span class="sourceLineNo">157</span> // Try to get the chunk<a name="line.157"></a>
<span class="sourceLineNo">158</span> c = getOrMakeChunk();<a name="line.158"></a>
<span class="sourceLineNo">159</span> // We may get null because the some other thread succeeded in getting the lock<a name="line.159"></a>
<span class="sourceLineNo">160</span> // and so the current thread has to try again to make its chunk or grab the chunk<a name="line.160"></a>
<span class="sourceLineNo">161</span> // that the other thread created<a name="line.161"></a>
<span class="sourceLineNo">162</span> // Try to allocate from this chunk<a name="line.162"></a>
<span class="sourceLineNo">163</span> if (c != null) {<a name="line.163"></a>
<span class="sourceLineNo">164</span> allocOffset = c.alloc(size);<a name="line.164"></a>
<span class="sourceLineNo">165</span> if (allocOffset != -1) {<a name="line.165"></a>
<span class="sourceLineNo">166</span> // We succeeded - this is the common case - small alloc<a name="line.166"></a>
<span class="sourceLineNo">167</span> // from a big buffer<a name="line.167"></a>
<span class="sourceLineNo">168</span> break;<a name="line.168"></a>
<span class="sourceLineNo">169</span> }<a name="line.169"></a>
<span class="sourceLineNo">170</span> // not enough space!<a name="line.170"></a>
<span class="sourceLineNo">171</span> // try to retire this chunk<a name="line.171"></a>
<span class="sourceLineNo">172</span> tryRetireChunk(c);<a name="line.172"></a>
<span class="sourceLineNo">173</span> }<a name="line.173"></a>
<span class="sourceLineNo">174</span> }<a name="line.174"></a>
<span class="sourceLineNo">175</span> return copyBBECToChunkCell(cell, c.getData(), allocOffset, size);<a name="line.175"></a>
<span class="sourceLineNo">176</span> }<a name="line.176"></a>
<span class="sourceLineNo">177</span><a name="line.177"></a>
<span class="sourceLineNo">178</span> /**<a name="line.178"></a>
<span class="sourceLineNo">179</span> * @see #copyBBECellInto(ByteBufferExtendedCell, int)<a name="line.179"></a>
<span class="sourceLineNo">180</span> */<a name="line.180"></a>
<span class="sourceLineNo">181</span> private Cell copyCellInto(Cell cell, int maxAlloc) {<a name="line.181"></a>
<span class="sourceLineNo">182</span> int size = Segment.getCellLength(cell);<a name="line.182"></a>
<span class="sourceLineNo">183</span> Preconditions.checkArgument(size &gt;= 0, "negative size");<a name="line.183"></a>
<span class="sourceLineNo">184</span> // Callers should satisfy large allocations directly from JVM since they<a name="line.184"></a>
<span class="sourceLineNo">185</span> // don't cause fragmentation as badly.<a name="line.185"></a>
<span class="sourceLineNo">186</span> if (size &gt; maxAlloc) {<a name="line.186"></a>
<span class="sourceLineNo">187</span> return null;<a name="line.187"></a>
<span class="sourceLineNo">188</span> }<a name="line.188"></a>
<span class="sourceLineNo">189</span> Chunk c = null;<a name="line.189"></a>
<span class="sourceLineNo">190</span> int allocOffset = 0;<a name="line.190"></a>
<span class="sourceLineNo">191</span> while (true) {<a name="line.191"></a>
<span class="sourceLineNo">192</span> // Try to get the chunk<a name="line.192"></a>
<span class="sourceLineNo">193</span> c = getOrMakeChunk();<a name="line.193"></a>
<span class="sourceLineNo">194</span> // we may get null because the some other thread succeeded in getting the lock<a name="line.194"></a>
<span class="sourceLineNo">195</span> // and so the current thread has to try again to make its chunk or grab the chunk<a name="line.195"></a>
<span class="sourceLineNo">196</span> // that the other thread created<a name="line.196"></a>
<span class="sourceLineNo">197</span> // Try to allocate from this chunk<a name="line.197"></a>
<span class="sourceLineNo">198</span> if (c != null) {<a name="line.198"></a>
<span class="sourceLineNo">199</span> allocOffset = c.alloc(size);<a name="line.199"></a>
<span class="sourceLineNo">200</span> if (allocOffset != -1) {<a name="line.200"></a>
<span class="sourceLineNo">201</span> // We succeeded - this is the common case - small alloc<a name="line.201"></a>
<span class="sourceLineNo">202</span> // from a big buffer<a name="line.202"></a>
<span class="sourceLineNo">203</span> break;<a name="line.203"></a>
<span class="sourceLineNo">204</span> }<a name="line.204"></a>
<span class="sourceLineNo">205</span> // not enough space!<a name="line.205"></a>
<span class="sourceLineNo">206</span> // try to retire this chunk<a name="line.206"></a>
<span class="sourceLineNo">207</span> tryRetireChunk(c);<a name="line.207"></a>
<span class="sourceLineNo">208</span> }<a name="line.208"></a>
<span class="sourceLineNo">209</span> }<a name="line.209"></a>
<span class="sourceLineNo">210</span> return copyToChunkCell(cell, c.getData(), allocOffset, size);<a name="line.210"></a>
<span class="sourceLineNo">211</span> }<a name="line.211"></a>
<span class="sourceLineNo">212</span><a name="line.212"></a>
<span class="sourceLineNo">213</span> /**<a name="line.213"></a>
<span class="sourceLineNo">214</span> * Clone the passed cell by copying its data into the passed buf and create a cell with a chunkid<a name="line.214"></a>
<span class="sourceLineNo">215</span> * out of it<a name="line.215"></a>
<span class="sourceLineNo">216</span> * @see #copyBBECToChunkCell(ByteBufferExtendedCell, ByteBuffer, int, int)<a name="line.216"></a>
<span class="sourceLineNo">217</span> */<a name="line.217"></a>
<span class="sourceLineNo">218</span> private static Cell copyToChunkCell(Cell cell, ByteBuffer buf, int offset, int len) {<a name="line.218"></a>
<span class="sourceLineNo">219</span> int tagsLen = cell.getTagsLength();<a name="line.219"></a>
<span class="sourceLineNo">220</span> if (cell instanceof ExtendedCell) {<a name="line.220"></a>
<span class="sourceLineNo">221</span> ((ExtendedCell) cell).write(buf, offset);<a name="line.221"></a>
<span class="sourceLineNo">222</span> } else {<a name="line.222"></a>
<span class="sourceLineNo">223</span> // Normally all Cell impls within Server will be of type ExtendedCell. Just considering the<a name="line.223"></a>
<span class="sourceLineNo">224</span> // other case also. The data fragments within Cell is copied into buf as in KeyValue<a name="line.224"></a>
<span class="sourceLineNo">225</span> // serialization format only.<a name="line.225"></a>
<span class="sourceLineNo">226</span> KeyValueUtil.appendTo(cell, buf, offset, true);<a name="line.226"></a>
<span class="sourceLineNo">227</span> }<a name="line.227"></a>
<span class="sourceLineNo">228</span> return createChunkCell(buf, offset, len, tagsLen, cell.getSequenceId());<a name="line.228"></a>
<span class="sourceLineNo">229</span> }<a name="line.229"></a>
<span class="sourceLineNo">230</span><a name="line.230"></a>
<span class="sourceLineNo">231</span> /**<a name="line.231"></a>
<span class="sourceLineNo">232</span> * Clone the passed cell by copying its data into the passed buf and create a cell with a chunkid<a name="line.232"></a>
<span class="sourceLineNo">233</span> * out of it<a name="line.233"></a>
<span class="sourceLineNo">234</span> * @see #copyToChunkCell(Cell, ByteBuffer, int, int)<a name="line.234"></a>
<span class="sourceLineNo">235</span> */<a name="line.235"></a>
<span class="sourceLineNo">236</span> private static Cell copyBBECToChunkCell(ByteBufferExtendedCell cell, ByteBuffer buf, int offset,<a name="line.236"></a>
<span class="sourceLineNo">237</span> int len) {<a name="line.237"></a>
<span class="sourceLineNo">238</span> int tagsLen = cell.getTagsLength();<a name="line.238"></a>
<span class="sourceLineNo">239</span> cell.write(buf, offset);<a name="line.239"></a>
<span class="sourceLineNo">240</span> return createChunkCell(buf, offset, len, tagsLen, cell.getSequenceId());<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> private static Cell createChunkCell(ByteBuffer buf, int offset, int len, int tagsLen,<a name="line.243"></a>
<span class="sourceLineNo">244</span> long sequenceId) {<a name="line.244"></a>
<span class="sourceLineNo">245</span> // TODO : write the seqid here. For writing seqId we should create a new cell type so<a name="line.245"></a>
<span class="sourceLineNo">246</span> // that seqId is not used as the state<a name="line.246"></a>
<span class="sourceLineNo">247</span> if (tagsLen == 0) {<a name="line.247"></a>
<span class="sourceLineNo">248</span> // When tagsLen is 0, make a NoTagsByteBufferKeyValue version. This is an optimized class<a name="line.248"></a>
<span class="sourceLineNo">249</span> // which directly return tagsLen as 0. So we avoid parsing many length components in<a name="line.249"></a>
<span class="sourceLineNo">250</span> // reading the tagLength stored in the backing buffer. The Memstore addition of every Cell<a name="line.250"></a>
<span class="sourceLineNo">251</span> // call getTagsLength().<a name="line.251"></a>
<span class="sourceLineNo">252</span> return new NoTagByteBufferChunkKeyValue(buf, offset, len, sequenceId);<a name="line.252"></a>
<span class="sourceLineNo">253</span> } else {<a name="line.253"></a>
<span class="sourceLineNo">254</span> return new ByteBufferChunkKeyValue(buf, offset, len, sequenceId);<a name="line.254"></a>
<span class="sourceLineNo">255</span> }<a name="line.255"></a>
<span class="sourceLineNo">256</span> }<a name="line.256"></a>
<span class="sourceLineNo">257</span><a name="line.257"></a>
<span class="sourceLineNo">258</span> /**<a name="line.258"></a>
<span class="sourceLineNo">259</span> * Close this instance since it won't be used any more, try to put the chunks<a name="line.259"></a>
<span class="sourceLineNo">260</span> * back to pool<a name="line.260"></a>
<span class="sourceLineNo">261</span> */<a name="line.261"></a>
<span class="sourceLineNo">262</span> @Override<a name="line.262"></a>
<span class="sourceLineNo">263</span> public void close() {<a name="line.263"></a>
<span class="sourceLineNo">264</span> this.closed = true;<a name="line.264"></a>
<span class="sourceLineNo">265</span> // We could put back the chunks to pool for reusing only when there is no<a name="line.265"></a>
<span class="sourceLineNo">266</span> // opening scanner which will read their data<a name="line.266"></a>
<span class="sourceLineNo">267</span> int count = openScannerCount.get();<a name="line.267"></a>
<span class="sourceLineNo">268</span> if(count == 0) {<a name="line.268"></a>
<span class="sourceLineNo">269</span> recycleChunks();<a name="line.269"></a>
<span class="sourceLineNo">270</span> }<a name="line.270"></a>
<span class="sourceLineNo">271</span> }<a name="line.271"></a>
<span class="sourceLineNo">272</span><a name="line.272"></a>
<span class="sourceLineNo">273</span> @VisibleForTesting<a name="line.273"></a>
<span class="sourceLineNo">274</span> int getOpenScannerCount() {<a name="line.274"></a>
<span class="sourceLineNo">275</span> return this.openScannerCount.get();<a name="line.275"></a>
<span class="sourceLineNo">276</span> }<a name="line.276"></a>
<span class="sourceLineNo">277</span><a name="line.277"></a>
<span class="sourceLineNo">278</span> /**<a name="line.278"></a>
<span class="sourceLineNo">279</span> * Called when opening a scanner on the data of this MemStoreLAB<a name="line.279"></a>
<span class="sourceLineNo">280</span> */<a name="line.280"></a>
<span class="sourceLineNo">281</span> @Override<a name="line.281"></a>
<span class="sourceLineNo">282</span> public void incScannerCount() {<a name="line.282"></a>
<span class="sourceLineNo">283</span> this.openScannerCount.incrementAndGet();<a name="line.283"></a>
<span class="sourceLineNo">284</span> }<a name="line.284"></a>
<span class="sourceLineNo">285</span><a name="line.285"></a>
<span class="sourceLineNo">286</span> /**<a name="line.286"></a>
<span class="sourceLineNo">287</span> * Called when closing a scanner on the data of this MemStoreLAB<a name="line.287"></a>
<span class="sourceLineNo">288</span> */<a name="line.288"></a>
<span class="sourceLineNo">289</span> @Override<a name="line.289"></a>
<span class="sourceLineNo">290</span> public void decScannerCount() {<a name="line.290"></a>
<span class="sourceLineNo">291</span> int count = this.openScannerCount.decrementAndGet();<a name="line.291"></a>
<span class="sourceLineNo">292</span> if (this.closed &amp;&amp; count == 0) {<a name="line.292"></a>
<span class="sourceLineNo">293</span> recycleChunks();<a name="line.293"></a>
<span class="sourceLineNo">294</span> }<a name="line.294"></a>
<span class="sourceLineNo">295</span> }<a name="line.295"></a>
<span class="sourceLineNo">296</span><a name="line.296"></a>
<span class="sourceLineNo">297</span> private void recycleChunks() {<a name="line.297"></a>
<span class="sourceLineNo">298</span> if (reclaimed.compareAndSet(false, true)) {<a name="line.298"></a>
<span class="sourceLineNo">299</span> chunkCreator.putbackChunks(chunks);<a name="line.299"></a>
<span class="sourceLineNo">300</span> }<a name="line.300"></a>
<span class="sourceLineNo">301</span> }<a name="line.301"></a>
<span class="sourceLineNo">302</span><a name="line.302"></a>
<span class="sourceLineNo">303</span> /**<a name="line.303"></a>
<span class="sourceLineNo">304</span> * Try to retire the current chunk if it is still<a name="line.304"></a>
<span class="sourceLineNo">305</span> * &lt;code&gt;c&lt;/code&gt;. Postcondition is that curChunk.get()<a name="line.305"></a>
<span class="sourceLineNo">306</span> * != c<a name="line.306"></a>
<span class="sourceLineNo">307</span> * @param c the chunk to retire<a name="line.307"></a>
<span class="sourceLineNo">308</span> */<a name="line.308"></a>
<span class="sourceLineNo">309</span> private void tryRetireChunk(Chunk c) {<a name="line.309"></a>
<span class="sourceLineNo">310</span> currChunk.compareAndSet(c, null);<a name="line.310"></a>
<span class="sourceLineNo">311</span> // If the CAS succeeds, that means that we won the race<a name="line.311"></a>
<span class="sourceLineNo">312</span> // to retire the chunk. We could use this opportunity to<a name="line.312"></a>
<span class="sourceLineNo">313</span> // update metrics on external fragmentation.<a name="line.313"></a>
<span class="sourceLineNo">314</span> //<a name="line.314"></a>
<span class="sourceLineNo">315</span> // If the CAS fails, that means that someone else already<a name="line.315"></a>
<span class="sourceLineNo">316</span> // retired the chunk for us.<a name="line.316"></a>
<span class="sourceLineNo">317</span> }<a name="line.317"></a>
<span class="sourceLineNo">318</span><a name="line.318"></a>
<span class="sourceLineNo">319</span> /**<a name="line.319"></a>
<span class="sourceLineNo">320</span> * Get the current chunk, or, if there is no current chunk,<a name="line.320"></a>
<span class="sourceLineNo">321</span> * allocate a new one from the JVM.<a name="line.321"></a>
<span class="sourceLineNo">322</span> */<a name="line.322"></a>
<span class="sourceLineNo">323</span> private Chunk getOrMakeChunk() {<a name="line.323"></a>
<span class="sourceLineNo">324</span> // Try to get the chunk<a name="line.324"></a>
<span class="sourceLineNo">325</span> Chunk c = currChunk.get();<a name="line.325"></a>
<span class="sourceLineNo">326</span> if (c != null) {<a name="line.326"></a>
<span class="sourceLineNo">327</span> return c;<a name="line.327"></a>
<span class="sourceLineNo">328</span> }<a name="line.328"></a>
<span class="sourceLineNo">329</span> // No current chunk, so we want to allocate one. We race<a name="line.329"></a>
<span class="sourceLineNo">330</span> // against other allocators to CAS in an uninitialized chunk<a name="line.330"></a>
<span class="sourceLineNo">331</span> // (which is cheap to allocate)<a name="line.331"></a>
<span class="sourceLineNo">332</span> if (lock.tryLock()) {<a name="line.332"></a>
<span class="sourceLineNo">333</span> try {<a name="line.333"></a>
<span class="sourceLineNo">334</span> // once again check inside the lock<a name="line.334"></a>
<span class="sourceLineNo">335</span> c = currChunk.get();<a name="line.335"></a>
<span class="sourceLineNo">336</span> if (c != null) {<a name="line.336"></a>
<span class="sourceLineNo">337</span> return c;<a name="line.337"></a>
<span class="sourceLineNo">338</span> }<a name="line.338"></a>
<span class="sourceLineNo">339</span> c = this.chunkCreator.getChunk(idxType);<a name="line.339"></a>
<span class="sourceLineNo">340</span> if (c != null) {<a name="line.340"></a>
<span class="sourceLineNo">341</span> // set the curChunk. No need of CAS as only one thread will be here<a name="line.341"></a>
<span class="sourceLineNo">342</span> currChunk.set(c);<a name="line.342"></a>
<span class="sourceLineNo">343</span> chunks.add(c.getId());<a name="line.343"></a>
<span class="sourceLineNo">344</span> return c;<a name="line.344"></a>
<span class="sourceLineNo">345</span> }<a name="line.345"></a>
<span class="sourceLineNo">346</span> } finally {<a name="line.346"></a>
<span class="sourceLineNo">347</span> lock.unlock();<a name="line.347"></a>
<span class="sourceLineNo">348</span> }<a name="line.348"></a>
<span class="sourceLineNo">349</span> }<a name="line.349"></a>
<span class="sourceLineNo">350</span> return null;<a name="line.350"></a>
<span class="sourceLineNo">351</span> }<a name="line.351"></a>
<span class="sourceLineNo">352</span><a name="line.352"></a>
<span class="sourceLineNo">353</span> /* Returning a new pool chunk, without replacing current chunk,<a name="line.353"></a>
<span class="sourceLineNo">354</span> ** meaning MSLABImpl does not make the returned chunk as CurChunk.<a name="line.354"></a>
<span class="sourceLineNo">355</span> ** The space on this chunk will be allocated externally.<a name="line.355"></a>
<span class="sourceLineNo">356</span> ** The interface is only for external callers.<a name="line.356"></a>
<span class="sourceLineNo">357</span> */<a name="line.357"></a>
<span class="sourceLineNo">358</span> @Override<a name="line.358"></a>
<span class="sourceLineNo">359</span> public Chunk getNewExternalChunk(ChunkCreator.ChunkType chunkType) {<a name="line.359"></a>
<span class="sourceLineNo">360</span> switch (chunkType) {<a name="line.360"></a>
<span class="sourceLineNo">361</span> case INDEX_CHUNK:<a name="line.361"></a>
<span class="sourceLineNo">362</span> case DATA_CHUNK:<a name="line.362"></a>
<span class="sourceLineNo">363</span> Chunk c = this.chunkCreator.getChunk(chunkType);<a name="line.363"></a>
<span class="sourceLineNo">364</span> chunks.add(c.getId());<a name="line.364"></a>
<span class="sourceLineNo">365</span> return c;<a name="line.365"></a>
<span class="sourceLineNo">366</span> case JUMBO_CHUNK: // a jumbo chunk doesn't have a fixed size<a name="line.366"></a>
<span class="sourceLineNo">367</span> default:<a name="line.367"></a>
<span class="sourceLineNo">368</span> return null;<a name="line.368"></a>
<span class="sourceLineNo">369</span> }<a name="line.369"></a>
<span class="sourceLineNo">370</span> }<a name="line.370"></a>
<span class="sourceLineNo">371</span><a name="line.371"></a>
<span class="sourceLineNo">372</span> /* Returning a new chunk, without replacing current chunk,<a name="line.372"></a>
<span class="sourceLineNo">373</span> ** meaning MSLABImpl does not make the returned chunk as CurChunk.<a name="line.373"></a>
<span class="sourceLineNo">374</span> ** The space on this chunk will be allocated externally.<a name="line.374"></a>
<span class="sourceLineNo">375</span> ** The interface is only for external callers.<a name="line.375"></a>
<span class="sourceLineNo">376</span> ** Chunks from pools are not allocated from here, since they have fixed sizes<a name="line.376"></a>
<span class="sourceLineNo">377</span> */<a name="line.377"></a>
<span class="sourceLineNo">378</span> @Override<a name="line.378"></a>
<span class="sourceLineNo">379</span> public Chunk getNewExternalChunk(int size) {<a name="line.379"></a>
<span class="sourceLineNo">380</span> int allocSize = size + ChunkCreator.SIZEOF_CHUNK_HEADER;<a name="line.380"></a>
<span class="sourceLineNo">381</span> if (allocSize &lt;= ChunkCreator.getInstance().getChunkSize()) {<a name="line.381"></a>
<span class="sourceLineNo">382</span> return getNewExternalChunk(ChunkCreator.ChunkType.DATA_CHUNK);<a name="line.382"></a>
<span class="sourceLineNo">383</span> } else {<a name="line.383"></a>
<span class="sourceLineNo">384</span> Chunk c = this.chunkCreator.getJumboChunk(size);<a name="line.384"></a>
<span class="sourceLineNo">385</span> chunks.add(c.getId());<a name="line.385"></a>
<span class="sourceLineNo">386</span> return c;<a name="line.386"></a>
<span class="sourceLineNo">387</span> }<a name="line.387"></a>
<span class="sourceLineNo">388</span> }<a name="line.388"></a>
<span class="sourceLineNo">389</span><a name="line.389"></a>
<span class="sourceLineNo">390</span> @Override<a name="line.390"></a>
<span class="sourceLineNo">391</span> public boolean isOnHeap() {<a name="line.391"></a>
<span class="sourceLineNo">392</span> return !isOffHeap();<a name="line.392"></a>
<span class="sourceLineNo">393</span> }<a name="line.393"></a>
<span class="sourceLineNo">394</span><a name="line.394"></a>
<span class="sourceLineNo">395</span> @Override<a name="line.395"></a>
<span class="sourceLineNo">396</span> public boolean isOffHeap() {<a name="line.396"></a>
<span class="sourceLineNo">397</span> return this.chunkCreator.isOffheap();<a name="line.397"></a>
<span class="sourceLineNo">398</span> }<a name="line.398"></a>
<span class="sourceLineNo">399</span><a name="line.399"></a>
<span class="sourceLineNo">400</span> @VisibleForTesting<a name="line.400"></a>
<span class="sourceLineNo">401</span> Chunk getCurrentChunk() {<a name="line.401"></a>
<span class="sourceLineNo">402</span> return currChunk.get();<a name="line.402"></a>
<span class="sourceLineNo">403</span> }<a name="line.403"></a>
<span class="sourceLineNo">404</span><a name="line.404"></a>
<span class="sourceLineNo">405</span> @VisibleForTesting<a name="line.405"></a>
<span class="sourceLineNo">406</span> BlockingQueue&lt;Chunk&gt; getPooledChunks() {<a name="line.406"></a>
<span class="sourceLineNo">407</span> BlockingQueue&lt;Chunk&gt; pooledChunks = new LinkedBlockingQueue&lt;&gt;();<a name="line.407"></a>
<span class="sourceLineNo">408</span> for (Integer id : this.chunks) {<a name="line.408"></a>
<span class="sourceLineNo">409</span> Chunk chunk = chunkCreator.getChunk(id);<a name="line.409"></a>
<span class="sourceLineNo">410</span> if (chunk != null &amp;&amp; chunk.isFromPool()) {<a name="line.410"></a>
<span class="sourceLineNo">411</span> pooledChunks.add(chunk);<a name="line.411"></a>
<span class="sourceLineNo">412</span> }<a name="line.412"></a>
<span class="sourceLineNo">413</span> }<a name="line.413"></a>
<span class="sourceLineNo">414</span> return pooledChunks;<a name="line.414"></a>
<span class="sourceLineNo">415</span> }<a name="line.415"></a>
<span class="sourceLineNo">416</span><a name="line.416"></a>
<span class="sourceLineNo">417</span> @VisibleForTesting Integer getNumOfChunksReturnedToPool() {<a name="line.417"></a>
<span class="sourceLineNo">418</span> int i = 0;<a name="line.418"></a>
<span class="sourceLineNo">419</span> for (Integer id : this.chunks) {<a name="line.419"></a>
<span class="sourceLineNo">420</span> if (chunkCreator.isChunkInPool(id)) {<a name="line.420"></a>
<span class="sourceLineNo">421</span> i++;<a name="line.421"></a>
<span class="sourceLineNo">422</span> }<a name="line.422"></a>
<span class="sourceLineNo">423</span> }<a name="line.423"></a>
<span class="sourceLineNo">424</span> return i;<a name="line.424"></a>
<span class="sourceLineNo">425</span> }<a name="line.425"></a>
<span class="sourceLineNo">426</span>}<a name="line.426"></a>
</pre>
</div>
</body>
</html>