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 java.lang.management.ManagementFactory;
22 import java.util.concurrent.BlockingQueue;
23 import java.util.concurrent.Executors;
24 import java.util.concurrent.LinkedBlockingQueue;
25 import java.util.concurrent.ScheduledExecutorService;
26 import java.util.concurrent.TimeUnit;
27 import java.util.concurrent.atomic.AtomicLong;
28
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.regionserver.MemStoreLAB.Chunk;
34 import org.apache.hadoop.util.StringUtils;
35
36 import com.google.common.util.concurrent.ThreadFactoryBuilder;
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
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 globalInstance;
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
78 MemStoreChunkPool(Configuration conf, int chunkSize, int maxCount,
79 int initialCount) {
80 this.maxCount = maxCount;
81 this.chunkSize = chunkSize;
82 this.reclaimedChunks = new LinkedBlockingQueue<Chunk>();
83 for (int i = 0; i < initialCount; i++) {
84 Chunk chunk = new Chunk(chunkSize);
85 chunk.init();
86 reclaimedChunks.add(chunk);
87 }
88 final String n = Thread.currentThread().getName();
89 scheduleThreadPool = Executors.newScheduledThreadPool(1,
90 new ThreadFactoryBuilder().setNameFormat(n+"-MemStoreChunkPool Statistics")
91 .setDaemon(true).build());
92 this.scheduleThreadPool.scheduleAtFixedRate(new StatisticsThread(this),
93 statThreadPeriod, statThreadPeriod, TimeUnit.SECONDS);
94 }
95
96
97
98
99
100
101 Chunk getChunk() {
102 Chunk chunk = reclaimedChunks.poll();
103 if (chunk == null) {
104 chunk = new Chunk(chunkSize);
105 createdChunkCount.incrementAndGet();
106 } else {
107 chunk.reset();
108 reusedChunkCount.incrementAndGet();
109 }
110 return chunk;
111 }
112
113
114
115
116
117
118 void putbackChunks(BlockingQueue<Chunk> chunks) {
119 int maxNumToPutback = this.maxCount - reclaimedChunks.size();
120 if (maxNumToPutback <= 0) {
121 return;
122 }
123 chunks.drainTo(reclaimedChunks, maxNumToPutback);
124 }
125
126
127
128
129
130
131 void putbackChunk(Chunk chunk) {
132 if (reclaimedChunks.size() >= this.maxCount) {
133 return;
134 }
135 reclaimedChunks.add(chunk);
136 }
137
138 int getPoolSize() {
139 return this.reclaimedChunks.size();
140 }
141
142
143
144
145 void clearChunks() {
146 this.reclaimedChunks.clear();
147 }
148
149 private static class StatisticsThread extends Thread {
150 MemStoreChunkPool mcp;
151
152 public StatisticsThread(MemStoreChunkPool mcp) {
153 super("MemStoreChunkPool.StatisticsThread");
154 setDaemon(true);
155 this.mcp = mcp;
156 }
157
158 @Override
159 public void run() {
160 mcp.logStats();
161 }
162 }
163
164 private void logStats() {
165 if (!LOG.isDebugEnabled()) return;
166 long created = createdChunkCount.get();
167 long reused = reusedChunkCount.get();
168 long total = created + reused;
169 LOG.debug("Stats: current pool size=" + reclaimedChunks.size()
170 + ",created chunk count=" + created
171 + ",reused chunk count=" + reused
172 + ",reuseRatio=" + (total == 0 ? "0" : StringUtils.formatPercent(
173 (float) reused / (float) total, 2)));
174 }
175
176
177
178
179
180 static synchronized MemStoreChunkPool getPool(Configuration conf) {
181 if (globalInstance != null) return globalInstance;
182 if (chunkPoolDisabled) return null;
183
184
185 float poolSizePercentage = conf.getFloat(CHUNK_POOL_MAXSIZE_KEY,
186 POOL_MAX_SIZE_DEFAULT);
187 if (poolSizePercentage <= 0) {
188 chunkPoolDisabled = true;
189 return null;
190 }
191 if (poolSizePercentage > 1.0) {
192 throw new IllegalArgumentException(CHUNK_POOL_MAXSIZE_KEY
193 + " must be between 0.0 and 1.0");
194 }
195 long heapMax = ManagementFactory.getMemoryMXBean().getHeapMemoryUsage()
196 .getMax();
197 long globalMemStoreLimit = MemStoreFlusher.globalMemStoreLimit(heapMax,
198 MemStoreFlusher.DEFAULT_UPPER, MemStoreFlusher.UPPER_KEY, conf);
199 int chunkSize = conf.getInt(MemStoreLAB.CHUNK_SIZE_KEY,
200 MemStoreLAB.CHUNK_SIZE_DEFAULT);
201 int maxCount = (int) (globalMemStoreLimit * poolSizePercentage / chunkSize);
202
203 float initialCountPercentage = conf.getFloat(CHUNK_POOL_INITIALSIZE_KEY,
204 POOL_INITIAL_SIZE_DEFAULT);
205 if (initialCountPercentage > 1.0 || initialCountPercentage < 0) {
206 throw new IllegalArgumentException(CHUNK_POOL_INITIALSIZE_KEY
207 + " must be between 0.0 and 1.0");
208 }
209
210 int initialCount = (int) (initialCountPercentage * maxCount);
211 LOG.info("Allocating MemStoreChunkPool with chunk size "
212 + StringUtils.byteDesc(chunkSize) + ", max count " + maxCount
213 + ", initial count " + initialCount);
214 globalInstance = new MemStoreChunkPool(conf, chunkSize, maxCount,
215 initialCount);
216 return globalInstance;
217 }
218
219 }