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 long maxStoreFileAge = 0;
59 private volatile long minStoreFileAge = 0;
60 private volatile long avgStoreFileAge = 0;
61 private volatile long numReferenceFiles = 0;
62 private volatile double requestsPerSecond = 0.0;
63 private volatile long readRequestsCount = 0;
64 private volatile long writeRequestsCount = 0;
65 private volatile long checkAndMutateChecksFailed = 0;
66 private volatile long checkAndMutateChecksPassed = 0;
67 private volatile long storefileIndexSize = 0;
68 private volatile long totalStaticIndexSize = 0;
69 private volatile long totalStaticBloomSize = 0;
70 private volatile long numMutationsWithoutWAL = 0;
71 private volatile long dataInMemoryWithoutWAL = 0;
72 private volatile int percentFileLocal = 0;
73 private volatile long flushedCellsCount = 0;
74 private volatile long compactedCellsCount = 0;
75 private volatile long majorCompactedCellsCount = 0;
76 private volatile long flushedCellsSize = 0;
77 private volatile long compactedCellsSize = 0;
78 private volatile long majorCompactedCellsSize = 0;
79 private volatile long blockedRequestsCount = 0L;
80
81 private CacheStats cacheStats;
82 private ScheduledExecutorService executor;
83 private Runnable runnable;
84 private long period;
85
86 public MetricsRegionServerWrapperImpl(final HRegionServer regionServer) {
87 this.regionServer = regionServer;
88 initBlockCache();
89
90 this.period =
91 regionServer.conf.getLong(HConstants.REGIONSERVER_METRICS_PERIOD,
92 HConstants.DEFAULT_REGIONSERVER_METRICS_PERIOD);
93
94 this.executor = CompatibilitySingletonFactory.getInstance(MetricsExecutor.class).getExecutor();
95 this.runnable = new RegionServerMetricsWrapperRunnable();
96 this.executor.scheduleWithFixedDelay(this.runnable, this.period, this.period,
97 TimeUnit.MILLISECONDS);
98
99 if (LOG.isInfoEnabled()) {
100 LOG.info("Computing regionserver metrics every " + this.period + " milliseconds");
101 }
102 }
103
104
105
106
107
108
109 private synchronized void initBlockCache() {
110 CacheConfig cacheConfig = this.regionServer.cacheConfig;
111 if (cacheConfig != null && this.blockCache == null) {
112 this.blockCache = cacheConfig.getBlockCache();
113 }
114
115 if (this.blockCache != null && this.cacheStats == null) {
116 this.cacheStats = blockCache.getStats();
117 }
118 }
119
120 @Override
121 public String getClusterId() {
122 return regionServer.getClusterId();
123 }
124
125 @Override
126 public long getStartCode() {
127 return regionServer.getStartcode();
128 }
129
130 @Override
131 public String getZookeeperQuorum() {
132 ZooKeeperWatcher zk = regionServer.getZooKeeperWatcher();
133 if (zk == null) {
134 return "";
135 }
136 return zk.getQuorum();
137 }
138
139 @Override
140 public String getCoprocessors() {
141 String[] coprocessors = regionServer.getCoprocessors();
142 if (coprocessors == null || coprocessors.length == 0) {
143 return "";
144 }
145 return StringUtils.join(coprocessors, ", ");
146 }
147
148 @Override
149 public String getServerName() {
150 ServerName serverName = regionServer.getServerName();
151 if (serverName == null) {
152 return "";
153 }
154 return serverName.getServerName();
155 }
156
157 @Override
158 public long getNumOnlineRegions() {
159 Collection<HRegion> onlineRegionsLocalContext = regionServer.getOnlineRegionsLocalContext();
160 if (onlineRegionsLocalContext == null) {
161 return 0;
162 }
163 return onlineRegionsLocalContext.size();
164 }
165
166 @Override
167 public long getTotalRequestCount() {
168 return regionServer.requestCount.get();
169 }
170
171 @Override
172 public int getSplitQueueSize() {
173 if (this.regionServer.compactSplitThread == null) {
174 return 0;
175 }
176 return this.regionServer.compactSplitThread.getSplitQueueSize();
177 }
178
179 @Override
180 public int getCompactionQueueSize() {
181
182 if (this.regionServer.compactSplitThread == null) {
183 return 0;
184 }
185 return this.regionServer.compactSplitThread.getCompactionQueueSize();
186 }
187
188 @Override
189 public int getSmallCompactionQueueSize() {
190
191 if (this.regionServer.compactSplitThread == null) {
192 return 0;
193 }
194 return this.regionServer.compactSplitThread.getSmallCompactionQueueSize();
195 }
196
197 @Override
198 public int getLargeCompactionQueueSize() {
199
200 if (this.regionServer.compactSplitThread == null) {
201 return 0;
202 }
203 return this.regionServer.compactSplitThread.getLargeCompactionQueueSize();
204 }
205
206 @Override
207 public int getFlushQueueSize() {
208
209 if (this.regionServer.cacheFlusher == null) {
210 return 0;
211 }
212 return this.regionServer.cacheFlusher.getFlushQueueSize();
213 }
214
215 @Override
216 public long getBlockCacheCount() {
217 if (this.blockCache == null) {
218 return 0;
219 }
220 return this.blockCache.getBlockCount();
221 }
222
223 @Override
224 public long getBlockCacheSize() {
225 if (this.blockCache == null) {
226 return 0;
227 }
228 return this.blockCache.getCurrentSize();
229 }
230
231 @Override
232 public long getBlockCacheFreeSize() {
233 if (this.blockCache == null) {
234 return 0;
235 }
236 return this.blockCache.getFreeSize();
237 }
238
239 @Override
240 public long getBlockCacheHitCount() {
241 if (this.cacheStats == null) {
242 return 0;
243 }
244 return this.cacheStats.getHitCount();
245 }
246
247 @Override
248 public long getBlockCacheMissCount() {
249 if (this.cacheStats == null) {
250 return 0;
251 }
252 return this.cacheStats.getMissCount();
253 }
254
255 @Override
256 public long getBlockCacheEvictedCount() {
257 if (this.cacheStats == null) {
258 return 0;
259 }
260 return this.cacheStats.getEvictedCount();
261 }
262
263 @Override
264 public int getBlockCacheHitPercent() {
265 if (this.cacheStats == null) {
266 return 0;
267 }
268 return (int) (this.cacheStats.getHitRatio() * 100);
269 }
270
271 @Override
272 public int getBlockCacheHitCachingPercent() {
273 if (this.cacheStats == null) {
274 return 0;
275 }
276 return (int) (this.cacheStats.getHitCachingRatio() * 100);
277 }
278
279 @Override
280 public long getBlockCacheFailedInsertions() {
281 return this.cacheStats.getFailedInserts();
282 }
283
284 @Override public void forceRecompute() {
285 this.runnable.run();
286 }
287
288 @Override
289 public long getNumStores() {
290 return numStores;
291 }
292
293 @Override
294 public long getNumHLogFiles() {
295 return numHLogFiles;
296 }
297
298 @Override
299 public long getHLogFileSize() {
300 return hlogFileSize;
301 }
302
303 @Override
304 public long getNumStoreFiles() {
305 return numStoreFiles;
306 }
307
308 @Override
309 public long getMaxStoreFileAge() {
310 return maxStoreFileAge;
311 }
312
313 @Override
314 public long getMinStoreFileAge() {
315 return minStoreFileAge;
316 }
317
318 @Override
319 public long getAvgStoreFileAge() {
320 return avgStoreFileAge;
321 }
322
323 @Override
324 public long getNumReferenceFiles() {
325 return numReferenceFiles;
326 }
327
328 @Override
329 public long getMemstoreSize() {
330 return memstoreSize;
331 }
332
333 @Override
334 public long getStoreFileSize() {
335 return storeFileSize;
336 }
337
338 @Override public double getRequestsPerSecond() {
339 return requestsPerSecond;
340 }
341
342 @Override
343 public long getReadRequestsCount() {
344 return readRequestsCount;
345 }
346
347 @Override
348 public long getWriteRequestsCount() {
349 return writeRequestsCount;
350 }
351
352 @Override
353 public long getCheckAndMutateChecksFailed() {
354 return checkAndMutateChecksFailed;
355 }
356
357 @Override
358 public long getCheckAndMutateChecksPassed() {
359 return checkAndMutateChecksPassed;
360 }
361
362 @Override
363 public long getStoreFileIndexSize() {
364 return storefileIndexSize;
365 }
366
367 @Override
368 public long getTotalStaticIndexSize() {
369 return totalStaticIndexSize;
370 }
371
372 @Override
373 public long getTotalStaticBloomSize() {
374 return totalStaticBloomSize;
375 }
376
377 @Override
378 public long getNumMutationsWithoutWAL() {
379 return numMutationsWithoutWAL;
380 }
381
382 @Override
383 public long getDataInMemoryWithoutWAL() {
384 return dataInMemoryWithoutWAL;
385 }
386
387 @Override
388 public int getPercentFileLocal() {
389 return percentFileLocal;
390 }
391
392 @Override
393 public long getUpdatesBlockedTime() {
394 if (this.regionServer.cacheFlusher == null) {
395 return 0;
396 }
397 return this.regionServer.cacheFlusher.getUpdatesBlockedMsHighWater().get();
398 }
399
400 @Override
401 public long getFlushedCellsCount() {
402 return flushedCellsCount;
403 }
404
405 @Override
406 public long getCompactedCellsCount() {
407 return compactedCellsCount;
408 }
409
410 @Override
411 public long getMajorCompactedCellsCount() {
412 return majorCompactedCellsCount;
413 }
414
415 @Override
416 public long getFlushedCellsSize() {
417 return flushedCellsSize;
418 }
419
420 @Override
421 public long getCompactedCellsSize() {
422 return compactedCellsSize;
423 }
424
425 @Override
426 public long getMajorCompactedCellsSize() {
427 return majorCompactedCellsSize;
428 }
429
430
431
432
433
434
435 public class RegionServerMetricsWrapperRunnable implements Runnable {
436
437 private long lastRan = 0;
438 private long lastRequestCount = 0;
439
440 @Override
441 synchronized public void run() {
442 initBlockCache();
443 cacheStats = blockCache.getStats();
444
445 HDFSBlocksDistribution hdfsBlocksDistribution =
446 new HDFSBlocksDistribution();
447
448 long tempNumStores = 0;
449 long tempNumStoreFiles = 0;
450 long tempMemstoreSize = 0;
451 long tempStoreFileSize = 0;
452 long tempMaxStoreFileAge = 0;
453 long tempNumReferenceFiles = 0;
454 long avgAgeNumerator = 0;
455 long numHFiles = 0;
456 long tempMinStoreFileAge = Long.MAX_VALUE;
457 long tempReadRequestsCount = 0;
458 long tempWriteRequestsCount = 0;
459 long tempCheckAndMutateChecksFailed = 0;
460 long tempCheckAndMutateChecksPassed = 0;
461 long tempStorefileIndexSize = 0;
462 long tempTotalStaticIndexSize = 0;
463 long tempTotalStaticBloomSize = 0;
464 long tempNumMutationsWithoutWAL = 0;
465 long tempDataInMemoryWithoutWAL = 0;
466 int tempPercentFileLocal = 0;
467 long tempFlushedCellsCount = 0;
468 long tempCompactedCellsCount = 0;
469 long tempMajorCompactedCellsCount = 0;
470 long tempFlushedCellsSize = 0;
471 long tempCompactedCellsSize = 0;
472 long tempMajorCompactedCellsSize = 0;
473 long tempBlockedRequestsCount = 0L;
474
475 for (HRegion r : regionServer.getOnlineRegionsLocalContext()) {
476 tempNumMutationsWithoutWAL += r.numMutationsWithoutWAL.get();
477 tempDataInMemoryWithoutWAL += r.dataInMemoryWithoutWAL.get();
478 tempReadRequestsCount += r.readRequestsCount.get();
479 tempWriteRequestsCount += r.writeRequestsCount.get();
480 tempCheckAndMutateChecksFailed += r.checkAndMutateChecksFailed.get();
481 tempCheckAndMutateChecksPassed += r.checkAndMutateChecksPassed.get();
482 tempBlockedRequestsCount += r.getBlockedRequestsCount();
483 tempNumStores += r.stores.size();
484 for (Store store : r.stores.values()) {
485 tempNumStoreFiles += store.getStorefilesCount();
486 tempMemstoreSize += store.getMemStoreSize();
487 tempStoreFileSize += store.getStorefilesSize();
488
489 long storeMaxStoreFileAge = store.getMaxStoreFileAge();
490 tempMaxStoreFileAge = (storeMaxStoreFileAge > tempMaxStoreFileAge) ?
491 storeMaxStoreFileAge : tempMaxStoreFileAge;
492
493 long storeMinStoreFileAge = store.getMinStoreFileAge();
494 tempMinStoreFileAge = (storeMinStoreFileAge < tempMinStoreFileAge) ?
495 storeMinStoreFileAge : tempMinStoreFileAge;
496
497 long storeHFiles = store.getNumHFiles();
498 avgAgeNumerator += store.getAvgStoreFileAge() * storeHFiles;
499 numHFiles += storeHFiles;
500 tempNumReferenceFiles += store.getNumReferenceFiles();
501
502 tempStorefileIndexSize += store.getStorefilesIndexSize();
503 tempTotalStaticBloomSize += store.getTotalStaticBloomSize();
504 tempTotalStaticIndexSize += store.getTotalStaticIndexSize();
505 tempFlushedCellsCount += store.getFlushedCellsCount();
506 tempCompactedCellsCount += store.getCompactedCellsCount();
507 tempMajorCompactedCellsCount += store.getMajorCompactedCellsCount();
508 tempFlushedCellsSize += store.getFlushedCellsSize();
509 tempCompactedCellsSize += store.getCompactedCellsSize();
510 tempMajorCompactedCellsSize += store.getMajorCompactedCellsSize();
511 }
512
513 hdfsBlocksDistribution.add(r.getHDFSBlocksDistribution());
514 }
515
516 float localityIndex = hdfsBlocksDistribution.getBlockLocalityIndex(
517 regionServer.getServerName().getHostname());
518 tempPercentFileLocal = (int) (localityIndex * 100);
519
520
521
522 long currentTime = EnvironmentEdgeManager.currentTimeMillis();
523
524
525
526 if (lastRan == 0) {
527 lastRan = currentTime - period;
528 }
529
530
531
532 if ((currentTime - lastRan) > 0) {
533 long currentRequestCount = getTotalRequestCount();
534 requestsPerSecond = (currentRequestCount - lastRequestCount) / ((currentTime - lastRan) / 1000.0);
535 lastRequestCount = currentRequestCount;
536 }
537 lastRan = currentTime;
538
539
540 numStores = tempNumStores;
541 long tempNumHLogFiles = regionServer.hlog.getNumLogFiles();
542
543 if (regionServer.hlogForMeta != null) {
544 tempNumHLogFiles += regionServer.hlogForMeta.getNumLogFiles();
545 }
546 numHLogFiles = tempNumHLogFiles;
547
548 long tempHlogFileSize = regionServer.hlog.getLogFileSize();
549 if (regionServer.hlogForMeta != null) {
550 tempHlogFileSize += regionServer.hlogForMeta.getLogFileSize();
551 }
552 hlogFileSize = tempHlogFileSize;
553
554 numStoreFiles = tempNumStoreFiles;
555 memstoreSize = tempMemstoreSize;
556 storeFileSize = tempStoreFileSize;
557 maxStoreFileAge = tempMaxStoreFileAge;
558 if (tempMinStoreFileAge != Long.MAX_VALUE) {
559 minStoreFileAge = tempMinStoreFileAge;
560 }
561
562 if (numHFiles != 0) {
563 avgStoreFileAge = avgAgeNumerator / numHFiles;
564 }
565
566 numReferenceFiles= tempNumReferenceFiles;
567 readRequestsCount = tempReadRequestsCount;
568 writeRequestsCount = tempWriteRequestsCount;
569 checkAndMutateChecksFailed = tempCheckAndMutateChecksFailed;
570 checkAndMutateChecksPassed = tempCheckAndMutateChecksPassed;
571 storefileIndexSize = tempStorefileIndexSize;
572 totalStaticIndexSize = tempTotalStaticIndexSize;
573 totalStaticBloomSize = tempTotalStaticBloomSize;
574 numMutationsWithoutWAL = tempNumMutationsWithoutWAL;
575 dataInMemoryWithoutWAL = tempDataInMemoryWithoutWAL;
576 percentFileLocal = tempPercentFileLocal;
577 flushedCellsCount = tempFlushedCellsCount;
578 compactedCellsCount = tempCompactedCellsCount;
579 majorCompactedCellsCount = tempMajorCompactedCellsCount;
580 flushedCellsSize = tempFlushedCellsSize;
581 compactedCellsSize = tempCompactedCellsSize;
582 majorCompactedCellsSize = tempMajorCompactedCellsSize;
583 blockedRequestsCount = tempBlockedRequestsCount;
584 }
585 }
586
587 @Override
588 public long getBlockedRequestsCount() {
589 return blockedRequestsCount;
590 }
591 }