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