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