1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.regionserver;
19
20 import java.util.Collection;
21 import java.util.concurrent.ScheduledExecutorService;
22 import java.util.concurrent.TimeUnit;
23
24 import org.apache.commons.lang.StringUtils;
25 import org.apache.commons.logging.Log;
26 import org.apache.commons.logging.LogFactory;
27 import org.apache.hadoop.hbase.classification.InterfaceAudience;
28 import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
29 import org.apache.hadoop.hbase.HConstants;
30 import org.apache.hadoop.hbase.HDFSBlocksDistribution;
31 import org.apache.hadoop.hbase.ServerName;
32 import org.apache.hadoop.hbase.io.hfile.BlockCache;
33 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
34 import org.apache.hadoop.hbase.io.hfile.CacheStats;
35 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
36 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
37 import org.apache.hadoop.metrics2.MetricsExecutor;
38
39
40
41
42 @InterfaceAudience.Private
43 class MetricsRegionServerWrapperImpl
44 implements MetricsRegionServerWrapper {
45
46 public static final Log LOG = LogFactory.getLog(MetricsRegionServerWrapperImpl.class);
47
48 private final HRegionServer regionServer;
49
50 private BlockCache blockCache;
51
52 private volatile long numStores = 0;
53 private volatile long numHLogFiles = 0;
54 private volatile long hlogFileSize = 0;
55 private volatile long numStoreFiles = 0;
56 private volatile long memstoreSize = 0;
57 private volatile long storeFileSize = 0;
58 private volatile double requestsPerSecond = 0.0;
59 private volatile long readRequestsCount = 0;
60 private volatile long writeRequestsCount = 0;
61 private volatile long checkAndMutateChecksFailed = 0;
62 private volatile long checkAndMutateChecksPassed = 0;
63 private volatile long storefileIndexSize = 0;
64 private volatile long totalStaticIndexSize = 0;
65 private volatile long totalStaticBloomSize = 0;
66 private volatile long numMutationsWithoutWAL = 0;
67 private volatile long dataInMemoryWithoutWAL = 0;
68 private volatile int percentFileLocal = 0;
69 private volatile long flushedCellsCount = 0;
70 private volatile long compactedCellsCount = 0;
71 private volatile long majorCompactedCellsCount = 0;
72 private volatile long flushedCellsSize = 0;
73 private volatile long compactedCellsSize = 0;
74 private volatile long majorCompactedCellsSize = 0;
75
76 private CacheStats cacheStats;
77 private ScheduledExecutorService executor;
78 private Runnable runnable;
79 private long period;
80
81 public MetricsRegionServerWrapperImpl(final HRegionServer regionServer) {
82 this.regionServer = regionServer;
83 initBlockCache();
84
85 this.period =
86 regionServer.conf.getLong(HConstants.REGIONSERVER_METRICS_PERIOD,
87 HConstants.DEFAULT_REGIONSERVER_METRICS_PERIOD);
88
89 this.executor = CompatibilitySingletonFactory.getInstance(MetricsExecutor.class).getExecutor();
90 this.runnable = new RegionServerMetricsWrapperRunnable();
91 this.executor.scheduleWithFixedDelay(this.runnable, this.period, this.period,
92 TimeUnit.MILLISECONDS);
93
94 if (LOG.isInfoEnabled()) {
95 LOG.info("Computing regionserver metrics every " + this.period + " milliseconds");
96 }
97 }
98
99
100
101
102
103
104 private synchronized void initBlockCache() {
105 CacheConfig cacheConfig = this.regionServer.cacheConfig;
106 if (cacheConfig != null && this.blockCache == null) {
107 this.blockCache = cacheConfig.getBlockCache();
108 }
109
110 if (this.blockCache != null && this.cacheStats == null) {
111 this.cacheStats = blockCache.getStats();
112 }
113 }
114
115 @Override
116 public String getClusterId() {
117 return regionServer.getClusterId();
118 }
119
120 @Override
121 public long getStartCode() {
122 return regionServer.getStartcode();
123 }
124
125 @Override
126 public String getZookeeperQuorum() {
127 ZooKeeperWatcher zk = regionServer.getZooKeeperWatcher();
128 if (zk == null) {
129 return "";
130 }
131 return zk.getQuorum();
132 }
133
134 @Override
135 public String getCoprocessors() {
136 String[] coprocessors = regionServer.getCoprocessors();
137 if (coprocessors == null || coprocessors.length == 0) {
138 return "";
139 }
140 return StringUtils.join(coprocessors, ", ");
141 }
142
143 @Override
144 public String getServerName() {
145 ServerName serverName = regionServer.getServerName();
146 if (serverName == null) {
147 return "";
148 }
149 return serverName.getServerName();
150 }
151
152 @Override
153 public long getNumOnlineRegions() {
154 Collection<HRegion> onlineRegionsLocalContext = regionServer.getOnlineRegionsLocalContext();
155 if (onlineRegionsLocalContext == null) {
156 return 0;
157 }
158 return onlineRegionsLocalContext.size();
159 }
160
161 @Override
162 public long getTotalRequestCount() {
163 return regionServer.requestCount.get();
164 }
165
166 @Override
167 public int getCompactionQueueSize() {
168
169 if (this.regionServer.compactSplitThread == null) {
170 return 0;
171 }
172 return this.regionServer.compactSplitThread.getCompactionQueueSize();
173 }
174
175 @Override
176 public int getSmallCompactionQueueSize() {
177
178 if (this.regionServer.compactSplitThread == null) {
179 return 0;
180 }
181 return this.regionServer.compactSplitThread.getSmallCompactionQueueSize();
182 }
183
184 @Override
185 public int getLargeCompactionQueueSize() {
186
187 if (this.regionServer.compactSplitThread == null) {
188 return 0;
189 }
190 return this.regionServer.compactSplitThread.getLargeCompactionQueueSize();
191 }
192
193 @Override
194 public int getFlushQueueSize() {
195
196 if (this.regionServer.cacheFlusher == null) {
197 return 0;
198 }
199 return this.regionServer.cacheFlusher.getFlushQueueSize();
200 }
201
202 @Override
203 public long getBlockCacheCount() {
204 if (this.blockCache == null) {
205 return 0;
206 }
207 return this.blockCache.getBlockCount();
208 }
209
210 @Override
211 public long getBlockCacheSize() {
212 if (this.blockCache == null) {
213 return 0;
214 }
215 return this.blockCache.getCurrentSize();
216 }
217
218 @Override
219 public long getBlockCacheFreeSize() {
220 if (this.blockCache == null) {
221 return 0;
222 }
223 return this.blockCache.getFreeSize();
224 }
225
226 @Override
227 public long getBlockCacheHitCount() {
228 if (this.cacheStats == null) {
229 return 0;
230 }
231 return this.cacheStats.getHitCount();
232 }
233
234 @Override
235 public long getBlockCacheMissCount() {
236 if (this.cacheStats == null) {
237 return 0;
238 }
239 return this.cacheStats.getMissCount();
240 }
241
242 @Override
243 public long getBlockCacheEvictedCount() {
244 if (this.cacheStats == null) {
245 return 0;
246 }
247 return this.cacheStats.getEvictedCount();
248 }
249
250 @Override
251 public int getBlockCacheHitPercent() {
252 if (this.cacheStats == null) {
253 return 0;
254 }
255 return (int) (this.cacheStats.getHitRatio() * 100);
256 }
257
258 @Override
259 public int getBlockCacheHitCachingPercent() {
260 if (this.cacheStats == null) {
261 return 0;
262 }
263 return (int) (this.cacheStats.getHitCachingRatio() * 100);
264 }
265
266 @Override public void forceRecompute() {
267 this.runnable.run();
268 }
269
270 @Override
271 public long getNumStores() {
272 return numStores;
273 }
274
275 @Override
276 public long getNumHLogFiles() {
277 return numHLogFiles;
278 }
279
280 @Override
281 public long getHLogFileSize() {
282 return hlogFileSize;
283 }
284
285 @Override
286 public long getNumStoreFiles() {
287 return numStoreFiles;
288 }
289
290 @Override
291 public long getMemstoreSize() {
292 return memstoreSize;
293 }
294
295 @Override
296 public long getStoreFileSize() {
297 return storeFileSize;
298 }
299
300 @Override public double getRequestsPerSecond() {
301 return requestsPerSecond;
302 }
303
304 @Override
305 public long getReadRequestsCount() {
306 return readRequestsCount;
307 }
308
309 @Override
310 public long getWriteRequestsCount() {
311 return writeRequestsCount;
312 }
313
314 @Override
315 public long getCheckAndMutateChecksFailed() {
316 return checkAndMutateChecksFailed;
317 }
318
319 @Override
320 public long getCheckAndMutateChecksPassed() {
321 return checkAndMutateChecksPassed;
322 }
323
324 @Override
325 public long getStoreFileIndexSize() {
326 return storefileIndexSize;
327 }
328
329 @Override
330 public long getTotalStaticIndexSize() {
331 return totalStaticIndexSize;
332 }
333
334 @Override
335 public long getTotalStaticBloomSize() {
336 return totalStaticBloomSize;
337 }
338
339 @Override
340 public long getNumMutationsWithoutWAL() {
341 return numMutationsWithoutWAL;
342 }
343
344 @Override
345 public long getDataInMemoryWithoutWAL() {
346 return dataInMemoryWithoutWAL;
347 }
348
349 @Override
350 public int getPercentFileLocal() {
351 return percentFileLocal;
352 }
353
354 @Override
355 public long getUpdatesBlockedTime() {
356 if (this.regionServer.cacheFlusher == null) {
357 return 0;
358 }
359 return this.regionServer.cacheFlusher.getUpdatesBlockedMsHighWater().get();
360 }
361
362 @Override
363 public long getFlushedCellsCount() {
364 return flushedCellsCount;
365 }
366
367 @Override
368 public long getCompactedCellsCount() {
369 return compactedCellsCount;
370 }
371
372 @Override
373 public long getMajorCompactedCellsCount() {
374 return majorCompactedCellsCount;
375 }
376
377 @Override
378 public long getFlushedCellsSize() {
379 return flushedCellsSize;
380 }
381
382 @Override
383 public long getCompactedCellsSize() {
384 return compactedCellsSize;
385 }
386
387 @Override
388 public long getMajorCompactedCellsSize() {
389 return majorCompactedCellsSize;
390 }
391
392
393
394
395
396
397 public class RegionServerMetricsWrapperRunnable implements Runnable {
398
399 private long lastRan = 0;
400 private long lastRequestCount = 0;
401
402 @Override
403 synchronized public void run() {
404 initBlockCache();
405 cacheStats = blockCache.getStats();
406
407 HDFSBlocksDistribution hdfsBlocksDistribution =
408 new HDFSBlocksDistribution();
409
410 long tempNumStores = 0;
411 long tempNumStoreFiles = 0;
412 long tempMemstoreSize = 0;
413 long tempStoreFileSize = 0;
414 long tempReadRequestsCount = 0;
415 long tempWriteRequestsCount = 0;
416 long tempCheckAndMutateChecksFailed = 0;
417 long tempCheckAndMutateChecksPassed = 0;
418 long tempStorefileIndexSize = 0;
419 long tempTotalStaticIndexSize = 0;
420 long tempTotalStaticBloomSize = 0;
421 long tempNumMutationsWithoutWAL = 0;
422 long tempDataInMemoryWithoutWAL = 0;
423 int tempPercentFileLocal = 0;
424 long tempFlushedCellsCount = 0;
425 long tempCompactedCellsCount = 0;
426 long tempMajorCompactedCellsCount = 0;
427 long tempFlushedCellsSize = 0;
428 long tempCompactedCellsSize = 0;
429 long tempMajorCompactedCellsSize = 0;
430
431 for (HRegion r : regionServer.getOnlineRegionsLocalContext()) {
432 tempNumMutationsWithoutWAL += r.numMutationsWithoutWAL.get();
433 tempDataInMemoryWithoutWAL += r.dataInMemoryWithoutWAL.get();
434 tempReadRequestsCount += r.readRequestsCount.get();
435 tempWriteRequestsCount += r.writeRequestsCount.get();
436 tempCheckAndMutateChecksFailed += r.checkAndMutateChecksFailed.get();
437 tempCheckAndMutateChecksPassed += r.checkAndMutateChecksPassed.get();
438 tempNumStores += r.stores.size();
439 for (Store store : r.stores.values()) {
440 tempNumStoreFiles += store.getStorefilesCount();
441 tempMemstoreSize += store.getMemStoreSize();
442 tempStoreFileSize += store.getStorefilesSize();
443 tempStorefileIndexSize += store.getStorefilesIndexSize();
444 tempTotalStaticBloomSize += store.getTotalStaticBloomSize();
445 tempTotalStaticIndexSize += store.getTotalStaticIndexSize();
446 tempFlushedCellsCount += store.getFlushedCellsCount();
447 tempCompactedCellsCount += store.getCompactedCellsCount();
448 tempMajorCompactedCellsCount += store.getMajorCompactedCellsCount();
449 tempFlushedCellsSize += store.getFlushedCellsSize();
450 tempCompactedCellsSize += store.getCompactedCellsSize();
451 tempMajorCompactedCellsSize += store.getMajorCompactedCellsSize();
452 }
453
454 hdfsBlocksDistribution.add(r.getHDFSBlocksDistribution());
455 }
456
457 float localityIndex = hdfsBlocksDistribution.getBlockLocalityIndex(
458 regionServer.getServerName().getHostname());
459 tempPercentFileLocal = (int) (localityIndex * 100);
460
461
462
463 long currentTime = EnvironmentEdgeManager.currentTimeMillis();
464
465
466
467 if (lastRan == 0) {
468 lastRan = currentTime - period;
469 }
470
471
472
473 if ((currentTime - lastRan) > 0) {
474 long currentRequestCount = getTotalRequestCount();
475 requestsPerSecond = (currentRequestCount - lastRequestCount) / ((currentTime - lastRan) / 1000.0);
476 lastRequestCount = currentRequestCount;
477 }
478 lastRan = currentTime;
479
480
481 numStores = tempNumStores;
482 long tempNumHLogFiles = regionServer.hlog.getNumLogFiles();
483
484 if (regionServer.hlogForMeta != null) {
485 tempNumHLogFiles += regionServer.hlogForMeta.getNumLogFiles();
486 }
487 numHLogFiles = tempNumHLogFiles;
488
489 long tempHlogFileSize = regionServer.hlog.getLogFileSize();
490 if (regionServer.hlogForMeta != null) {
491 tempHlogFileSize += regionServer.hlogForMeta.getLogFileSize();
492 }
493 hlogFileSize = tempHlogFileSize;
494
495 numStoreFiles = tempNumStoreFiles;
496 memstoreSize = tempMemstoreSize;
497 storeFileSize = tempStoreFileSize;
498 readRequestsCount = tempReadRequestsCount;
499 writeRequestsCount = tempWriteRequestsCount;
500 checkAndMutateChecksFailed = tempCheckAndMutateChecksFailed;
501 checkAndMutateChecksPassed = tempCheckAndMutateChecksPassed;
502 storefileIndexSize = tempStorefileIndexSize;
503 totalStaticIndexSize = tempTotalStaticIndexSize;
504 totalStaticBloomSize = tempTotalStaticBloomSize;
505 numMutationsWithoutWAL = tempNumMutationsWithoutWAL;
506 dataInMemoryWithoutWAL = tempDataInMemoryWithoutWAL;
507 percentFileLocal = tempPercentFileLocal;
508 flushedCellsCount = tempFlushedCellsCount;
509 compactedCellsCount = tempCompactedCellsCount;
510 majorCompactedCellsCount = tempMajorCompactedCellsCount;
511 flushedCellsSize = tempFlushedCellsSize;
512 compactedCellsSize = tempCompactedCellsSize;
513 majorCompactedCellsSize = tempMajorCompactedCellsSize;
514 }
515 }
516 }