View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
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   * Impl for exposing HRegionServer Information through Hadoop's metrics 2 system.
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    * It's possible that due to threading the block cache could not be initialized
107    * yet (testing multiple region servers in one jvm).  So we need to try and initialize
108    * the blockCache and cacheStats reference multiple times until we succeed.
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     //The thread could be zero.  if so assume there is no queue.
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     //The thread could be zero.  if so assume there is no queue.
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     //The thread could be zero.  if so assume there is no queue.
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     //If there is no flusher there should be no queue.
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     if (this.cacheStats == null) {
317       return 0;
318     }
319     return this.cacheStats.getFailedInserts();
320   }
321 
322   @Override public void forceRecompute() {
323     this.runnable.run();
324   }
325 
326   @Override
327   public long getNumStores() {
328     return numStores;
329   }
330   
331   @Override
332   public long getNumWALFiles() {
333     return numWALFiles;
334   }
335 
336   @Override
337   public long getWALFileSize() {
338     return walFileSize;
339   }
340   
341   @Override
342   public long getNumStoreFiles() {
343     return numStoreFiles;
344   }
345 
346   @Override
347   public long getMemstoreSize() {
348     return memstoreSize;
349   }
350 
351   @Override
352   public long getStoreFileSize() {
353     return storeFileSize;
354   }
355 
356   @Override public double getRequestsPerSecond() {
357     return requestsPerSecond;
358   }
359 
360   @Override
361   public long getReadRequestsCount() {
362     return readRequestsCount;
363   }
364 
365   @Override
366   public long getWriteRequestsCount() {
367     return writeRequestsCount;
368   }
369 
370   @Override
371   public long getCheckAndMutateChecksFailed() {
372     return checkAndMutateChecksFailed;
373   }
374 
375   @Override
376   public long getCheckAndMutateChecksPassed() {
377     return checkAndMutateChecksPassed;
378   }
379 
380   @Override
381   public long getStoreFileIndexSize() {
382     return storefileIndexSize;
383   }
384 
385   @Override
386   public long getTotalStaticIndexSize() {
387     return totalStaticIndexSize;
388   }
389 
390   @Override
391   public long getTotalStaticBloomSize() {
392     return totalStaticBloomSize;
393   }
394 
395   @Override
396   public long getNumMutationsWithoutWAL() {
397     return numMutationsWithoutWAL;
398   }
399 
400   @Override
401   public long getDataInMemoryWithoutWAL() {
402     return dataInMemoryWithoutWAL;
403   }
404 
405   @Override
406   public double getPercentFileLocal() {
407     return percentFileLocal;
408   }
409 
410   @Override
411   public double getPercentFileLocalSecondaryRegions() {
412     return percentFileLocalSecondaryRegions;
413   }
414 
415   @Override
416   public long getUpdatesBlockedTime() {
417     if (this.regionServer.cacheFlusher == null) {
418       return 0;
419     }
420     return this.regionServer.cacheFlusher.getUpdatesBlockedMsHighWater().get();
421   }
422 
423   @Override
424   public long getFlushedCellsCount() {
425     return flushedCellsCount;
426   }
427 
428   @Override
429   public long getCompactedCellsCount() {
430     return compactedCellsCount;
431   }
432 
433   @Override
434   public long getMajorCompactedCellsCount() {
435     return majorCompactedCellsCount;
436   }
437 
438   @Override
439   public long getFlushedCellsSize() {
440     return flushedCellsSize;
441   }
442 
443   @Override
444   public long getCompactedCellsSize() {
445     return compactedCellsSize;
446   }
447 
448   @Override
449   public long getMajorCompactedCellsSize() {
450     return majorCompactedCellsSize;
451   }
452 
453   /**
454    * This is the runnable that will be executed on the executor every PERIOD number of seconds
455    * It will take metrics/numbers from all of the regions and use them to compute point in
456    * time metrics.
457    */
458   public class RegionServerMetricsWrapperRunnable implements Runnable {
459 
460     private long lastRan = 0;
461     private long lastRequestCount = 0;
462 
463     @Override
464     synchronized public void run() {
465       try {
466         initBlockCache();
467 
468         HDFSBlocksDistribution hdfsBlocksDistribution = new HDFSBlocksDistribution();
469         HDFSBlocksDistribution hdfsBlocksDistributionSecondaryRegions =
470             new HDFSBlocksDistribution();
471 
472         long tempNumStores = 0;
473         long tempNumStoreFiles = 0;
474         long tempMemstoreSize = 0;
475         long tempStoreFileSize = 0;
476         long tempReadRequestsCount = 0;
477         long tempWriteRequestsCount = 0;
478         long tempCheckAndMutateChecksFailed = 0;
479         long tempCheckAndMutateChecksPassed = 0;
480         long tempStorefileIndexSize = 0;
481         long tempTotalStaticIndexSize = 0;
482         long tempTotalStaticBloomSize = 0;
483         long tempNumMutationsWithoutWAL = 0;
484         long tempDataInMemoryWithoutWAL = 0;
485         double tempPercentFileLocal = 0;
486         double tempPercentFileLocalSecondaryRegions = 0;
487         long tempFlushedCellsCount = 0;
488         long tempCompactedCellsCount = 0;
489         long tempMajorCompactedCellsCount = 0;
490         long tempFlushedCellsSize = 0;
491         long tempCompactedCellsSize = 0;
492         long tempMajorCompactedCellsSize = 0;
493         long tempBlockedRequestsCount = 0L;
494 
495         for (Region r : regionServer.getOnlineRegionsLocalContext()) {
496           tempNumMutationsWithoutWAL += r.getNumMutationsWithoutWAL();
497           tempDataInMemoryWithoutWAL += r.getDataInMemoryWithoutWAL();
498           tempReadRequestsCount += r.getReadRequestsCount();
499           tempWriteRequestsCount += r.getWriteRequestsCount();
500           tempCheckAndMutateChecksFailed += r.getCheckAndMutateChecksFailed();
501           tempCheckAndMutateChecksPassed += r.getCheckAndMutateChecksPassed();
502           tempBlockedRequestsCount += r.getBlockedRequestsCount();
503           List<Store> storeList = r.getStores();
504           tempNumStores += storeList.size();
505           for (Store store : storeList) {
506             tempNumStoreFiles += store.getStorefilesCount();
507             tempMemstoreSize += store.getMemStoreSize();
508             tempStoreFileSize += store.getStorefilesSize();
509             tempStorefileIndexSize += store.getStorefilesIndexSize();
510             tempTotalStaticBloomSize += store.getTotalStaticBloomSize();
511             tempTotalStaticIndexSize += store.getTotalStaticIndexSize();
512             tempFlushedCellsCount += store.getFlushedCellsCount();
513             tempCompactedCellsCount += store.getCompactedCellsCount();
514             tempMajorCompactedCellsCount += store.getMajorCompactedCellsCount();
515             tempFlushedCellsSize += store.getFlushedCellsSize();
516             tempCompactedCellsSize += store.getCompactedCellsSize();
517             tempMajorCompactedCellsSize += store.getMajorCompactedCellsSize();
518           }
519 
520           HDFSBlocksDistribution distro = r.getHDFSBlocksDistribution();
521           hdfsBlocksDistribution.add(distro);
522           if (r.getRegionInfo().getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
523             hdfsBlocksDistributionSecondaryRegions.add(distro);
524           }
525         }
526 
527         float localityIndex =
528             hdfsBlocksDistribution
529                 .getBlockLocalityIndex(regionServer.getServerName().getHostname());
530         tempPercentFileLocal = Double.isNaN(tempBlockedRequestsCount) ? 0 : (localityIndex * 100);
531 
532         float localityIndexSecondaryRegions =
533             hdfsBlocksDistributionSecondaryRegions.getBlockLocalityIndex(regionServer
534                 .getServerName().getHostname());
535         tempPercentFileLocalSecondaryRegions = Double
536             .isNaN(localityIndexSecondaryRegions) ? 0 : (localityIndexSecondaryRegions * 100);
537 
538         // Compute the number of requests per second
539         long currentTime = EnvironmentEdgeManager.currentTime();
540 
541         // assume that it took PERIOD seconds to start the executor.
542         // this is a guess but it's a pretty good one.
543         if (lastRan == 0) {
544           lastRan = currentTime - period;
545         }
546 
547         // If we've time traveled keep the last requests per second.
548         if ((currentTime - lastRan) > 0) {
549           long currentRequestCount = getTotalRequestCount();
550           requestsPerSecond =
551               (currentRequestCount - lastRequestCount) / ((currentTime - lastRan) / 1000.0);
552           lastRequestCount = currentRequestCount;
553         }
554         lastRan = currentTime;
555 
556         numWALFiles =
557             DefaultWALProvider.getNumLogFiles(regionServer.walFactory)
558                 + BoundedRegionGroupingProvider.getNumLogFiles(regionServer.walFactory);
559         walFileSize =
560             DefaultWALProvider.getLogFileSize(regionServer.walFactory)
561                 + BoundedRegionGroupingProvider.getLogFileSize(regionServer.walFactory);
562         // Copy over computed values so that no thread sees half computed values.
563         numStores = tempNumStores;
564         numStoreFiles = tempNumStoreFiles;
565         memstoreSize = tempMemstoreSize;
566         storeFileSize = tempStoreFileSize;
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         percentFileLocalSecondaryRegions = tempPercentFileLocalSecondaryRegions;
578         flushedCellsCount = tempFlushedCellsCount;
579         compactedCellsCount = tempCompactedCellsCount;
580         majorCompactedCellsCount = tempMajorCompactedCellsCount;
581         flushedCellsSize = tempFlushedCellsSize;
582         compactedCellsSize = tempCompactedCellsSize;
583         majorCompactedCellsSize = tempMajorCompactedCellsSize;
584         blockedRequestsCount = tempBlockedRequestsCount;
585       } catch (Throwable e) {
586         LOG.warn("Caught exception! Will suppress and retry.", e);
587       }
588     }
589   }
590 
591   @Override
592   public long getBlockedRequestsCount() {
593     return blockedRequestsCount;
594   }
595 }