1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.regionserver;
20
21 import com.google.common.util.concurrent.ThreadFactoryBuilder;
22 import java.lang.management.MemoryUsage;
23 import java.util.concurrent.BlockingQueue;
24 import java.util.concurrent.Executors;
25 import java.util.concurrent.LinkedBlockingQueue;
26 import java.util.concurrent.ScheduledExecutorService;
27 import java.util.concurrent.TimeUnit;
28 import java.util.concurrent.atomic.AtomicLong;
29 import org.apache.commons.logging.Log;
30 import org.apache.commons.logging.LogFactory;
31 import org.apache.hadoop.hbase.classification.InterfaceAudience;
32 import org.apache.hadoop.conf.Configuration;
33 import org.apache.hadoop.hbase.io.util.HeapMemorySizeUtil;
34 import org.apache.hadoop.hbase.regionserver.HeapMemStoreLAB.Chunk;
35 import org.apache.hadoop.util.StringUtils;
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51 @SuppressWarnings("javadoc")
52 @InterfaceAudience.Private
53 public class MemStoreChunkPool {
54 private static final Log LOG = LogFactory.getLog(MemStoreChunkPool.class);
55 final static String CHUNK_POOL_MAXSIZE_KEY = "hbase.hregion.memstore.chunkpool.maxsize";
56 final static String CHUNK_POOL_INITIALSIZE_KEY = "hbase.hregion.memstore.chunkpool.initialsize";
57 final static float POOL_MAX_SIZE_DEFAULT = 0.0f;
58 final static float POOL_INITIAL_SIZE_DEFAULT = 0.0f;
59
60
61 private static MemStoreChunkPool GLOBAL_INSTANCE;
62
63 static boolean chunkPoolDisabled = false;
64
65 private final int maxCount;
66
67
68 private final BlockingQueue<Chunk> reclaimedChunks;
69 private final int chunkSize;
70
71
72 private final ScheduledExecutorService scheduleThreadPool;
73
74 private static final int statThreadPeriod = 60 * 5;
75 private AtomicLong createdChunkCount = new AtomicLong();
76 private AtomicLong reusedChunkCount = new AtomicLong();
77 private AtomicLong requestedChunkCount = new AtomicLong();
78
79 MemStoreChunkPool(Configuration conf, int chunkSize, int maxCount,
80 int initialCount) {
81 this.maxCount = maxCount;
82 this.chunkSize = chunkSize;
83 this.reclaimedChunks = new LinkedBlockingQueue<Chunk>();
84 for (int i = 0; i < initialCount; i++) {
85 Chunk chunk = new Chunk(chunkSize);
86 chunk.init();
87 reclaimedChunks.add(chunk);
88 }
89 createdChunkCount.set(initialCount);
90 scheduleThreadPool = Executors.newScheduledThreadPool(1,
91 new ThreadFactoryBuilder().setNameFormat("MemStoreChunkPool Statistics")
92 .setDaemon(true).build());
93 this.scheduleThreadPool.scheduleAtFixedRate(new StatisticsThread(this),
94 statThreadPeriod, statThreadPeriod, TimeUnit.SECONDS);
95 }
96
97
98
99
100
101
102 Chunk getChunk() {
103 requestedChunkCount.incrementAndGet();
104 Chunk chunk = reclaimedChunks.poll();
105 if (chunk == null) {
106 chunk = new Chunk(chunkSize);
107 createdChunkCount.incrementAndGet();
108 } else {
109 chunk.reset();
110 reusedChunkCount.incrementAndGet();
111 }
112 return chunk;
113 }
114
115
116
117
118
119
120 void putbackChunks(BlockingQueue<Chunk> chunks) {
121 int maxNumToPutback = this.maxCount - reclaimedChunks.size();
122 if (maxNumToPutback <= 0) {
123 return;
124 }
125 chunks.drainTo(reclaimedChunks, maxNumToPutback);
126
127 if (chunks.size() > 0) {
128 if (LOG.isTraceEnabled()) {
129 LOG.trace("Left " + chunks.size() + " unreclaimable chunks, removing them from queue");
130 }
131 chunks.clear();
132 }
133 }
134
135
136
137
138
139
140 void putbackChunk(Chunk chunk) {
141 if (reclaimedChunks.size() >= this.maxCount) {
142 return;
143 }
144 reclaimedChunks.add(chunk);
145 }
146
147 int getPoolSize() {
148 return this.reclaimedChunks.size();
149 }
150
151
152
153
154 void clearChunks() {
155 this.reclaimedChunks.clear();
156 }
157
158 private static class StatisticsThread extends Thread {
159 MemStoreChunkPool mcp;
160
161 public StatisticsThread(MemStoreChunkPool mcp) {
162 super("MemStoreChunkPool.StatisticsThread");
163 setDaemon(true);
164 this.mcp = mcp;
165 }
166
167 @Override
168 public void run() {
169 mcp.logStats();
170 }
171 }
172
173 private void logStats() {
174 long total = createdChunkCount.get();
175 long reused = reusedChunkCount.get();
176 long available = reclaimedChunks.size();
177 long requested = requestedChunkCount.get();
178 LOG.info("Stats: chunk in pool=" + available
179 + ", chunk in use=" + (total - available)
180 + ", total chunk=" + total
181 + ", reused chunk=" + reused
182 + ", reuse ratio=" + (requested == 0 ? "0" : StringUtils.formatPercent(
183 (float) reused / (float) requested, 2)));
184 }
185
186
187
188
189
190 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="DC_DOUBLECHECK",
191 justification="Intentional")
192 static MemStoreChunkPool getPool(Configuration conf) {
193 if (GLOBAL_INSTANCE != null) return GLOBAL_INSTANCE;
194
195 synchronized (MemStoreChunkPool.class) {
196 if (chunkPoolDisabled) return null;
197 if (GLOBAL_INSTANCE != null) return GLOBAL_INSTANCE;
198 float poolSizePercentage = conf.getFloat(CHUNK_POOL_MAXSIZE_KEY, POOL_MAX_SIZE_DEFAULT);
199 if (poolSizePercentage <= 0) {
200 chunkPoolDisabled = true;
201 return null;
202 }
203 if (poolSizePercentage > 1.0) {
204 throw new IllegalArgumentException(CHUNK_POOL_MAXSIZE_KEY + " must be between 0.0 and 1.0");
205 }
206 long heapMax = -1L;
207 final MemoryUsage usage = HeapMemorySizeUtil.safeGetHeapMemoryUsage();
208 if (usage != null) {
209 heapMax = usage.getMax();
210 }
211 long globalMemStoreLimit = (long) (heapMax * HeapMemorySizeUtil.getGlobalMemStorePercent(conf,
212 false));
213 int chunkSize = conf.getInt(HeapMemStoreLAB.CHUNK_SIZE_KEY,
214 HeapMemStoreLAB.CHUNK_SIZE_DEFAULT);
215 int maxCount = (int) (globalMemStoreLimit * poolSizePercentage / chunkSize);
216
217 float initialCountPercentage = conf.getFloat(CHUNK_POOL_INITIALSIZE_KEY,
218 POOL_INITIAL_SIZE_DEFAULT);
219 if (initialCountPercentage > 1.0 || initialCountPercentage < 0) {
220 throw new IllegalArgumentException(CHUNK_POOL_INITIALSIZE_KEY
221 + " must be between 0.0 and 1.0");
222 }
223
224 int initialCount = (int) (initialCountPercentage * maxCount);
225 LOG.info("Allocating MemStoreChunkPool with chunk size " + StringUtils.byteDesc(chunkSize)
226 + ", max count " + maxCount + ", initial count " + initialCount);
227 GLOBAL_INSTANCE = new MemStoreChunkPool(conf, chunkSize, maxCount, initialCount);
228 return GLOBAL_INSTANCE;
229 }
230 }
231
232 int getMaxCount() {
233 return this.maxCount;
234 }
235
236 static void clearDisableFlag() {
237 chunkPoolDisabled = false;
238 }
239
240 }