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 getCompactionQueueSize() {
169
170 if (this.regionServer.compactSplitThread == null) {
171 return 0;
172 }
173 return this.regionServer.compactSplitThread.getCompactionQueueSize();
174 }
175
176 @Override
177 public int getSmallCompactionQueueSize() {
178
179 if (this.regionServer.compactSplitThread == null) {
180 return 0;
181 }
182 return this.regionServer.compactSplitThread.getSmallCompactionQueueSize();
183 }
184
185 @Override
186 public int getLargeCompactionQueueSize() {
187
188 if (this.regionServer.compactSplitThread == null) {
189 return 0;
190 }
191 return this.regionServer.compactSplitThread.getLargeCompactionQueueSize();
192 }
193
194 @Override
195 public int getFlushQueueSize() {
196
197 if (this.regionServer.cacheFlusher == null) {
198 return 0;
199 }
200 return this.regionServer.cacheFlusher.getFlushQueueSize();
201 }
202
203 @Override
204 public long getBlockCacheCount() {
205 if (this.blockCache == null) {
206 return 0;
207 }
208 return this.blockCache.getBlockCount();
209 }
210
211 @Override
212 public long getBlockCacheSize() {
213 if (this.blockCache == null) {
214 return 0;
215 }
216 return this.blockCache.getCurrentSize();
217 }
218
219 @Override
220 public long getBlockCacheFreeSize() {
221 if (this.blockCache == null) {
222 return 0;
223 }
224 return this.blockCache.getFreeSize();
225 }
226
227 @Override
228 public long getBlockCacheHitCount() {
229 if (this.cacheStats == null) {
230 return 0;
231 }
232 return this.cacheStats.getHitCount();
233 }
234
235 @Override
236 public long getBlockCacheMissCount() {
237 if (this.cacheStats == null) {
238 return 0;
239 }
240 return this.cacheStats.getMissCount();
241 }
242
243 @Override
244 public long getBlockCacheEvictedCount() {
245 if (this.cacheStats == null) {
246 return 0;
247 }
248 return this.cacheStats.getEvictedCount();
249 }
250
251 @Override
252 public int getBlockCacheHitPercent() {
253 if (this.cacheStats == null) {
254 return 0;
255 }
256 return (int) (this.cacheStats.getHitRatio() * 100);
257 }
258
259 @Override
260 public int getBlockCacheHitCachingPercent() {
261 if (this.cacheStats == null) {
262 return 0;
263 }
264 return (int) (this.cacheStats.getHitCachingRatio() * 100);
265 }
266
267 @Override public void forceRecompute() {
268 this.runnable.run();
269 }
270
271 @Override
272 public long getNumStores() {
273 return numStores;
274 }
275
276 @Override
277 public long getNumHLogFiles() {
278 return numHLogFiles;
279 }
280
281 @Override
282 public long getHLogFileSize() {
283 return hlogFileSize;
284 }
285
286 @Override
287 public long getNumStoreFiles() {
288 return numStoreFiles;
289 }
290
291 @Override
292 public long getMemstoreSize() {
293 return memstoreSize;
294 }
295
296 @Override
297 public long getStoreFileSize() {
298 return storeFileSize;
299 }
300
301 @Override public double getRequestsPerSecond() {
302 return requestsPerSecond;
303 }
304
305 @Override
306 public long getReadRequestsCount() {
307 return readRequestsCount;
308 }
309
310 @Override
311 public long getWriteRequestsCount() {
312 return writeRequestsCount;
313 }
314
315 @Override
316 public long getCheckAndMutateChecksFailed() {
317 return checkAndMutateChecksFailed;
318 }
319
320 @Override
321 public long getCheckAndMutateChecksPassed() {
322 return checkAndMutateChecksPassed;
323 }
324
325 @Override
326 public long getStoreFileIndexSize() {
327 return storefileIndexSize;
328 }
329
330 @Override
331 public long getTotalStaticIndexSize() {
332 return totalStaticIndexSize;
333 }
334
335 @Override
336 public long getTotalStaticBloomSize() {
337 return totalStaticBloomSize;
338 }
339
340 @Override
341 public long getNumMutationsWithoutWAL() {
342 return numMutationsWithoutWAL;
343 }
344
345 @Override
346 public long getDataInMemoryWithoutWAL() {
347 return dataInMemoryWithoutWAL;
348 }
349
350 @Override
351 public int getPercentFileLocal() {
352 return percentFileLocal;
353 }
354
355 @Override
356 public long getUpdatesBlockedTime() {
357 if (this.regionServer.cacheFlusher == null) {
358 return 0;
359 }
360 return this.regionServer.cacheFlusher.getUpdatesBlockedMsHighWater().get();
361 }
362
363 @Override
364 public long getFlushedCellsCount() {
365 return flushedCellsCount;
366 }
367
368 @Override
369 public long getCompactedCellsCount() {
370 return compactedCellsCount;
371 }
372
373 @Override
374 public long getMajorCompactedCellsCount() {
375 return majorCompactedCellsCount;
376 }
377
378 @Override
379 public long getFlushedCellsSize() {
380 return flushedCellsSize;
381 }
382
383 @Override
384 public long getCompactedCellsSize() {
385 return compactedCellsSize;
386 }
387
388 @Override
389 public long getMajorCompactedCellsSize() {
390 return majorCompactedCellsSize;
391 }
392
393
394
395
396
397
398 public class RegionServerMetricsWrapperRunnable implements Runnable {
399
400 private long lastRan = 0;
401 private long lastRequestCount = 0;
402
403 @Override
404 synchronized public void run() {
405 initBlockCache();
406 cacheStats = blockCache.getStats();
407
408 HDFSBlocksDistribution hdfsBlocksDistribution =
409 new HDFSBlocksDistribution();
410
411 long tempNumStores = 0;
412 long tempNumStoreFiles = 0;
413 long tempMemstoreSize = 0;
414 long tempStoreFileSize = 0;
415 long tempReadRequestsCount = 0;
416 long tempWriteRequestsCount = 0;
417 long tempCheckAndMutateChecksFailed = 0;
418 long tempCheckAndMutateChecksPassed = 0;
419 long tempStorefileIndexSize = 0;
420 long tempTotalStaticIndexSize = 0;
421 long tempTotalStaticBloomSize = 0;
422 long tempNumMutationsWithoutWAL = 0;
423 long tempDataInMemoryWithoutWAL = 0;
424 int tempPercentFileLocal = 0;
425 long tempFlushedCellsCount = 0;
426 long tempCompactedCellsCount = 0;
427 long tempMajorCompactedCellsCount = 0;
428 long tempFlushedCellsSize = 0;
429 long tempCompactedCellsSize = 0;
430 long tempMajorCompactedCellsSize = 0;
431 long tempBlockedRequestsCount = 0L;
432
433 for (HRegion r : regionServer.getOnlineRegionsLocalContext()) {
434 tempNumMutationsWithoutWAL += r.numMutationsWithoutWAL.get();
435 tempDataInMemoryWithoutWAL += r.dataInMemoryWithoutWAL.get();
436 tempReadRequestsCount += r.readRequestsCount.get();
437 tempWriteRequestsCount += r.writeRequestsCount.get();
438 tempCheckAndMutateChecksFailed += r.checkAndMutateChecksFailed.get();
439 tempCheckAndMutateChecksPassed += r.checkAndMutateChecksPassed.get();
440 tempBlockedRequestsCount += r.getBlockedRequestsCount();
441 tempNumStores += r.stores.size();
442 for (Store store : r.stores.values()) {
443 tempNumStoreFiles += store.getStorefilesCount();
444 tempMemstoreSize += store.getMemStoreSize();
445 tempStoreFileSize += store.getStorefilesSize();
446 tempStorefileIndexSize += store.getStorefilesIndexSize();
447 tempTotalStaticBloomSize += store.getTotalStaticBloomSize();
448 tempTotalStaticIndexSize += store.getTotalStaticIndexSize();
449 tempFlushedCellsCount += store.getFlushedCellsCount();
450 tempCompactedCellsCount += store.getCompactedCellsCount();
451 tempMajorCompactedCellsCount += store.getMajorCompactedCellsCount();
452 tempFlushedCellsSize += store.getFlushedCellsSize();
453 tempCompactedCellsSize += store.getCompactedCellsSize();
454 tempMajorCompactedCellsSize += store.getMajorCompactedCellsSize();
455 }
456
457 hdfsBlocksDistribution.add(r.getHDFSBlocksDistribution());
458 }
459
460 float localityIndex = hdfsBlocksDistribution.getBlockLocalityIndex(
461 regionServer.getServerName().getHostname());
462 tempPercentFileLocal = (int) (localityIndex * 100);
463
464
465
466 long currentTime = EnvironmentEdgeManager.currentTimeMillis();
467
468
469
470 if (lastRan == 0) {
471 lastRan = currentTime - period;
472 }
473
474
475
476 if ((currentTime - lastRan) > 0) {
477 long currentRequestCount = getTotalRequestCount();
478 requestsPerSecond = (currentRequestCount - lastRequestCount) / ((currentTime - lastRan) / 1000.0);
479 lastRequestCount = currentRequestCount;
480 }
481 lastRan = currentTime;
482
483
484 numStores = tempNumStores;
485 long tempNumHLogFiles = regionServer.hlog.getNumLogFiles();
486
487 if (regionServer.hlogForMeta != null) {
488 tempNumHLogFiles += regionServer.hlogForMeta.getNumLogFiles();
489 }
490 numHLogFiles = tempNumHLogFiles;
491
492 long tempHlogFileSize = regionServer.hlog.getLogFileSize();
493 if (regionServer.hlogForMeta != null) {
494 tempHlogFileSize += regionServer.hlogForMeta.getLogFileSize();
495 }
496 hlogFileSize = tempHlogFileSize;
497
498 numStoreFiles = tempNumStoreFiles;
499 memstoreSize = tempMemstoreSize;
500 storeFileSize = tempStoreFileSize;
501 readRequestsCount = tempReadRequestsCount;
502 writeRequestsCount = tempWriteRequestsCount;
503 checkAndMutateChecksFailed = tempCheckAndMutateChecksFailed;
504 checkAndMutateChecksPassed = tempCheckAndMutateChecksPassed;
505 storefileIndexSize = tempStorefileIndexSize;
506 totalStaticIndexSize = tempTotalStaticIndexSize;
507 totalStaticBloomSize = tempTotalStaticBloomSize;
508 numMutationsWithoutWAL = tempNumMutationsWithoutWAL;
509 dataInMemoryWithoutWAL = tempDataInMemoryWithoutWAL;
510 percentFileLocal = tempPercentFileLocal;
511 flushedCellsCount = tempFlushedCellsCount;
512 compactedCellsCount = tempCompactedCellsCount;
513 majorCompactedCellsCount = tempMajorCompactedCellsCount;
514 flushedCellsSize = tempFlushedCellsSize;
515 compactedCellsSize = tempCompactedCellsSize;
516 majorCompactedCellsSize = tempMajorCompactedCellsSize;
517 blockedRequestsCount = tempBlockedRequestsCount;
518 }
519 }
520
521 @Override
522 public long getBlockedRequestsCount() {
523 return blockedRequestsCount;
524 }
525 }