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
10 * http://www.apache.org/licenses/LICENSE-2.0
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.
19 package org
.apache
.hadoop
.hbase
.regionserver
;
21 import java
.io
.Closeable
;
22 import java
.io
.IOException
;
23 import java
.util
.HashMap
;
25 import java
.util
.OptionalDouble
;
26 import java
.util
.OptionalLong
;
27 import java
.util
.concurrent
.ScheduledExecutorService
;
28 import java
.util
.concurrent
.ScheduledFuture
;
29 import java
.util
.concurrent
.TimeUnit
;
31 import org
.apache
.hadoop
.hbase
.CompatibilitySingletonFactory
;
32 import org
.apache
.hadoop
.hbase
.client
.RegionInfo
;
33 import org
.apache
.hadoop
.hbase
.client
.TableDescriptor
;
34 import org
.apache
.hadoop
.hbase
.util
.EnvironmentEdgeManager
;
35 import org
.apache
.hadoop
.metrics2
.MetricsExecutor
;
36 import org
.apache
.yetus
.audience
.InterfaceAudience
;
37 import org
.slf4j
.Logger
;
38 import org
.slf4j
.LoggerFactory
;
40 @InterfaceAudience.Private
41 public class MetricsRegionWrapperImpl
implements MetricsRegionWrapper
, Closeable
{
43 private static final Logger LOG
= LoggerFactory
.getLogger(MetricsRegionWrapperImpl
.class);
45 public static final int PERIOD
= 45;
46 public static final String UNKNOWN
= "unknown";
48 private final HRegion region
;
49 private ScheduledExecutorService executor
;
50 private Runnable runnable
;
51 private long numStoreFiles
;
52 private long storeRefCount
;
53 private long maxCompactedStoreFileRefCount
;
54 private long memstoreSize
;
55 private long storeFileSize
;
56 private long maxStoreFileAge
;
57 private long minStoreFileAge
;
58 private long avgStoreFileAge
;
59 private long numReferenceFiles
;
60 private long maxFlushQueueSize
;
61 private long maxCompactionQueueSize
;
62 private Map
<String
, Long
> readsOnlyFromMemstore
;
63 private Map
<String
, Long
> mixedReadsOnStore
;
65 private ScheduledFuture
<?
> regionMetricsUpdateTask
;
67 public MetricsRegionWrapperImpl(HRegion region
) {
69 this.executor
= CompatibilitySingletonFactory
.getInstance(MetricsExecutor
.class).getExecutor();
70 this.runnable
= new HRegionMetricsWrapperRunnable();
71 this.regionMetricsUpdateTask
= this.executor
.scheduleWithFixedDelay(this.runnable
, PERIOD
,
72 PERIOD
, TimeUnit
.SECONDS
);
76 public String
getTableName() {
77 TableDescriptor tableDesc
= this.region
.getTableDescriptor();
78 if (tableDesc
== null) {
81 return tableDesc
.getTableName().getQualifierAsString();
85 public String
getNamespace() {
86 TableDescriptor tableDesc
= this.region
.getTableDescriptor();
87 if (tableDesc
== null) {
90 return tableDesc
.getTableName().getNamespaceAsString();
95 public String
getRegionName() {
96 RegionInfo regionInfo
= this.region
.getRegionInfo();
97 if (regionInfo
== null) {
100 return regionInfo
.getEncodedName();
104 public long getNumStores() {
105 Map
<byte[], HStore
> stores
= this.region
.stores
;
106 if (stores
== null) {
109 return stores
.size();
113 public long getNumStoreFiles() {
114 return numStoreFiles
;
118 public long getMemStoreSize() {
123 public long getStoreFileSize() {
124 return storeFileSize
;
128 public long getStoreRefCount() {
129 return storeRefCount
;
133 public long getMaxCompactedStoreFileRefCount() {
134 return maxCompactedStoreFileRefCount
;
138 public long getReadRequestCount() {
139 return this.region
.getReadRequestsCount();
143 public long getCpRequestCount() {
144 return this.region
.getCpRequestsCount();
148 public long getFilteredReadRequestCount() {
149 return this.region
.getFilteredReadRequestsCount();
153 public long getWriteRequestCount() {
154 return this.region
.getWriteRequestsCount();
158 public long getNumFilesCompacted() {
159 return this.region
.compactionNumFilesCompacted
.sum();
163 public long getNumBytesCompacted() {
164 return this.region
.compactionNumBytesCompacted
.sum();
168 public long getNumCompactionsCompleted() {
169 return this.region
.compactionsFinished
.sum();
173 public long getLastMajorCompactionAge() {
174 long lastMajorCompactionTs
= 0L;
176 lastMajorCompactionTs
= this.region
.getOldestHfileTs(true);
177 } catch (IOException ioe
) {
178 LOG
.error("Could not load HFile info ", ioe
);
180 long now
= EnvironmentEdgeManager
.currentTime();
181 return now
- lastMajorCompactionTs
;
185 public long getTotalRequestCount() {
186 return getReadRequestCount() + getWriteRequestCount();
190 public long getNumCompactionsFailed() {
191 return this.region
.compactionsFailed
.sum();
195 public long getNumCompactionsQueued() {
196 return this.region
.compactionsQueued
.sum();
200 public long getNumFlushesQueued() {
201 return this.region
.flushesQueued
.sum();
205 public long getMaxCompactionQueueSize() {
206 return maxCompactionQueueSize
;
210 public long getMaxFlushQueueSize() {
211 return maxFlushQueueSize
;
215 public long getMaxStoreFileAge() {
216 return maxStoreFileAge
;
220 public long getMinStoreFileAge() {
221 return minStoreFileAge
;
225 public long getAvgStoreFileAge() {
226 return avgStoreFileAge
;
230 public long getNumReferenceFiles() {
231 return numReferenceFiles
;
235 public int getRegionHashCode() {
236 return this.region
.hashCode();
240 public Map
<String
, Long
> getMemstoreOnlyRowReadsCount() {
241 return readsOnlyFromMemstore
;
245 public Map
<String
, Long
> getMixedRowReadsCount() {
246 return mixedReadsOnStore
;
249 public class HRegionMetricsWrapperRunnable
implements Runnable
{
253 long tempNumStoreFiles
= 0;
254 int tempStoreRefCount
= 0;
255 int tempMaxCompactedStoreFileRefCount
= 0;
256 long tempMemstoreSize
= 0;
257 long tempStoreFileSize
= 0;
258 long tempMaxStoreFileAge
= 0;
259 long tempMinStoreFileAge
= Long
.MAX_VALUE
;
260 long tempNumReferenceFiles
= 0;
261 long tempMaxCompactionQueueSize
= 0;
262 long tempMaxFlushQueueSize
= 0;
263 long avgAgeNumerator
= 0;
265 if (region
.stores
!= null) {
266 for (HStore store
: region
.stores
.values()) {
267 tempNumStoreFiles
+= store
.getStorefilesCount();
268 int currentStoreRefCount
= store
.getStoreRefCount();
269 tempStoreRefCount
+= currentStoreRefCount
;
270 int currentMaxCompactedStoreFileRefCount
= store
.getMaxCompactedStoreFileRefCount();
271 tempMaxCompactedStoreFileRefCount
= Math
.max(tempMaxCompactedStoreFileRefCount
,
272 currentMaxCompactedStoreFileRefCount
);
273 tempMemstoreSize
+= store
.getMemStoreSize().getDataSize();
274 tempStoreFileSize
+= store
.getStorefilesSize();
275 OptionalLong storeMaxStoreFileAge
= store
.getMaxStoreFileAge();
276 if (storeMaxStoreFileAge
.isPresent() &&
277 storeMaxStoreFileAge
.getAsLong() > tempMaxStoreFileAge
) {
278 tempMaxStoreFileAge
= storeMaxStoreFileAge
.getAsLong();
281 OptionalLong storeMinStoreFileAge
= store
.getMinStoreFileAge();
282 if (storeMinStoreFileAge
.isPresent() &&
283 storeMinStoreFileAge
.getAsLong() < tempMinStoreFileAge
) {
284 tempMinStoreFileAge
= storeMinStoreFileAge
.getAsLong();
287 long storeHFiles
= store
.getNumHFiles();
288 numHFiles
+= storeHFiles
;
289 tempNumReferenceFiles
+= store
.getNumReferenceFiles();
291 OptionalDouble storeAvgStoreFileAge
= store
.getAvgStoreFileAge();
292 if (storeAvgStoreFileAge
.isPresent()) {
293 avgAgeNumerator
+= (long) storeAvgStoreFileAge
.getAsDouble() * storeHFiles
;
295 if(mixedReadsOnStore
== null) {
296 mixedReadsOnStore
= new HashMap
<String
, Long
>();
298 Long tempVal
= mixedReadsOnStore
.get(store
.getColumnFamilyName());
299 if (tempVal
== null) {
302 tempVal
+= store
.getMixedRowReadsCount();
304 mixedReadsOnStore
.put(store
.getColumnFamilyName(), tempVal
);
305 if (readsOnlyFromMemstore
== null) {
306 readsOnlyFromMemstore
= new HashMap
<String
, Long
>();
308 tempVal
= readsOnlyFromMemstore
.get(store
.getColumnFamilyName());
309 if (tempVal
== null) {
312 tempVal
+= store
.getMemstoreOnlyRowReadsCount();
314 readsOnlyFromMemstore
.put(store
.getColumnFamilyName(), tempVal
);
318 numStoreFiles
= tempNumStoreFiles
;
319 storeRefCount
= tempStoreRefCount
;
320 maxCompactedStoreFileRefCount
= tempMaxCompactedStoreFileRefCount
;
321 memstoreSize
= tempMemstoreSize
;
322 storeFileSize
= tempStoreFileSize
;
323 maxStoreFileAge
= tempMaxStoreFileAge
;
324 if (tempMinStoreFileAge
!= Long
.MAX_VALUE
) {
325 minStoreFileAge
= tempMinStoreFileAge
;
328 if (numHFiles
!= 0) {
329 avgStoreFileAge
= avgAgeNumerator
/ numHFiles
;
332 numReferenceFiles
= tempNumReferenceFiles
;
333 tempMaxCompactionQueueSize
= getNumCompactionsQueued();
334 tempMaxFlushQueueSize
= getNumFlushesQueued();
335 if (tempMaxCompactionQueueSize
> maxCompactionQueueSize
) {
336 maxCompactionQueueSize
= tempMaxCompactionQueueSize
;
338 if (tempMaxFlushQueueSize
> maxFlushQueueSize
) {
339 maxFlushQueueSize
= tempMaxFlushQueueSize
;
345 public void close() throws IOException
{
346 regionMetricsUpdateTask
.cancel(true);
350 * Get the replica id of this region.
353 public int getReplicaId() {
354 return region
.getRegionInfo().getReplicaId();