1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.regionserver;
20
21 import java.io.Closeable;
22 import java.io.IOException;
23 import java.util.HashMap;
24 import java.util.Map;
25 import java.util.concurrent.ScheduledExecutorService;
26 import java.util.concurrent.ScheduledFuture;
27 import java.util.concurrent.TimeUnit;
28
29 import org.apache.commons.math.stat.descriptive.DescriptiveStatistics;
30 import org.apache.hadoop.hbase.classification.InterfaceAudience;
31 import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
32 import org.apache.hadoop.hbase.HRegionInfo;
33 import org.apache.hadoop.hbase.HTableDescriptor;
34 import org.apache.hadoop.metrics2.MetricsExecutor;
35
36 @InterfaceAudience.Private
37 public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable {
38
39 public static final int PERIOD = 45;
40 public static final String UNKNOWN = "unknown";
41
42 private final HRegion region;
43 private ScheduledExecutorService executor;
44 private Runnable runnable;
45 private long numStoreFiles;
46 private long memstoreSize;
47 private long storeFileSize;
48 private Map<String, DescriptiveStatistics> coprocessorTimes;
49
50 private ScheduledFuture<?> regionMetricsUpdateTask;
51
52 public MetricsRegionWrapperImpl(HRegion region) {
53 this.region = region;
54 this.executor = CompatibilitySingletonFactory.getInstance(MetricsExecutor.class).getExecutor();
55 this.runnable = new HRegionMetricsWrapperRunnable();
56 this.regionMetricsUpdateTask = this.executor.scheduleWithFixedDelay(this.runnable, PERIOD,
57 PERIOD, TimeUnit.SECONDS);
58 this.coprocessorTimes = new HashMap<String, DescriptiveStatistics>();
59 }
60
61 @Override
62 public String getTableName() {
63 HTableDescriptor tableDesc = this.region.getTableDesc();
64 if (tableDesc == null) {
65 return UNKNOWN;
66 }
67 return tableDesc.getTableName().getQualifierAsString();
68 }
69
70 @Override
71 public String getNamespace() {
72 HTableDescriptor tableDesc = this.region.getTableDesc();
73 if (tableDesc == null) {
74 return UNKNOWN;
75 }
76 return tableDesc.getTableName().getNamespaceAsString();
77 }
78
79
80 @Override
81 public String getRegionName() {
82 HRegionInfo regionInfo = this.region.getRegionInfo();
83 if (regionInfo == null) {
84 return UNKNOWN;
85 }
86 return regionInfo.getEncodedName();
87 }
88
89 @Override
90 public long getNumStores() {
91 Map<byte[],Store> stores = this.region.stores;
92 if (stores == null) {
93 return 0;
94 }
95 return stores.size();
96 }
97
98 @Override
99 public long getNumStoreFiles() {
100 return numStoreFiles;
101 }
102
103 @Override
104 public long getMemstoreSize() {
105 return memstoreSize;
106 }
107
108 @Override
109 public long getStoreFileSize() {
110 return storeFileSize;
111 }
112
113 @Override
114 public long getReadRequestCount() {
115 return this.region.getReadRequestsCount();
116 }
117
118 @Override
119 public long getWriteRequestCount() {
120 return this.region.getWriteRequestsCount();
121 }
122
123 @Override
124 public long getNumFilesCompacted() {
125 return this.region.compactionNumFilesCompacted.get();
126 }
127
128 @Override
129 public long getNumBytesCompacted() {
130 return this.region.compactionNumBytesCompacted.get();
131 }
132
133 @Override
134 public long getNumCompactionsCompleted() {
135 return this.region.compactionsFinished.get();
136 }
137
138 public class HRegionMetricsWrapperRunnable implements Runnable {
139
140 @Override
141 public void run() {
142 long tempNumStoreFiles = 0;
143 long tempMemstoreSize = 0;
144 long tempStoreFileSize = 0;
145
146 if (region.stores != null) {
147 for (Store store : region.stores.values()) {
148 tempNumStoreFiles += store.getStorefilesCount();
149 tempMemstoreSize += store.getMemStoreSize();
150 tempStoreFileSize += store.getStorefilesSize();
151 }
152 }
153
154 numStoreFiles = tempNumStoreFiles;
155 memstoreSize = tempMemstoreSize;
156 storeFileSize = tempStoreFileSize;
157 coprocessorTimes = region.getCoprocessorHost().getCoprocessorExecutionStatistics();
158
159 }
160 }
161
162 @Override
163 public void close() throws IOException {
164 regionMetricsUpdateTask.cancel(true);
165 }
166
167 @Override
168 public Map<String, DescriptiveStatistics> getCoprocessorExecutionStatistics() {
169 return coprocessorTimes;
170 }
171
172 }