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