001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018
019package org.apache.hadoop.hbase.regionserver;
020
021import java.io.Closeable;
022import java.io.IOException;
023import java.util.Map;
024import java.util.OptionalDouble;
025import java.util.OptionalLong;
026import java.util.concurrent.ScheduledExecutorService;
027import java.util.concurrent.ScheduledFuture;
028import java.util.concurrent.TimeUnit;
029
030import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
031import org.apache.hadoop.hbase.client.RegionInfo;
032import org.apache.hadoop.hbase.client.TableDescriptor;
033import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
034import org.apache.hadoop.metrics2.MetricsExecutor;
035import org.apache.yetus.audience.InterfaceAudience;
036import org.slf4j.Logger;
037import org.slf4j.LoggerFactory;
038
039@InterfaceAudience.Private
040public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable {
041
042  private static final Logger LOG = LoggerFactory.getLogger(MetricsRegionWrapperImpl.class);
043
044  public static final int PERIOD = 45;
045  public static final String UNKNOWN = "unknown";
046
047  private final HRegion region;
048  private ScheduledExecutorService executor;
049  private Runnable runnable;
050  private long numStoreFiles;
051  private long memstoreSize;
052  private long storeFileSize;
053  private long maxStoreFileAge;
054  private long minStoreFileAge;
055  private long avgStoreFileAge;
056  private long numReferenceFiles;
057  private long maxFlushQueueSize;
058  private long maxCompactionQueueSize;
059
060  private ScheduledFuture<?> regionMetricsUpdateTask;
061
062  public MetricsRegionWrapperImpl(HRegion region) {
063    this.region = region;
064    this.executor = CompatibilitySingletonFactory.getInstance(MetricsExecutor.class).getExecutor();
065    this.runnable = new HRegionMetricsWrapperRunnable();
066    this.regionMetricsUpdateTask = this.executor.scheduleWithFixedDelay(this.runnable, PERIOD,
067      PERIOD, TimeUnit.SECONDS);
068  }
069
070  @Override
071  public String getTableName() {
072    TableDescriptor tableDesc = this.region.getTableDescriptor();
073    if (tableDesc == null) {
074      return UNKNOWN;
075    }
076    return tableDesc.getTableName().getQualifierAsString();
077  }
078
079  @Override
080  public String getNamespace() {
081    TableDescriptor tableDesc = this.region.getTableDescriptor();
082    if (tableDesc == null) {
083      return UNKNOWN;
084    }
085    return tableDesc.getTableName().getNamespaceAsString();
086  }
087
088
089  @Override
090  public String getRegionName() {
091    RegionInfo regionInfo = this.region.getRegionInfo();
092    if (regionInfo == null) {
093      return UNKNOWN;
094    }
095    return regionInfo.getEncodedName();
096  }
097
098  @Override
099  public long getNumStores() {
100    Map<byte[], HStore> stores = this.region.stores;
101    if (stores == null) {
102      return 0;
103    }
104    return stores.size();
105  }
106
107  @Override
108  public long getNumStoreFiles() {
109    return numStoreFiles;
110  }
111
112  @Override
113  public long getMemStoreSize() {
114    return memstoreSize;
115  }
116
117  @Override
118  public long getStoreFileSize() {
119    return storeFileSize;
120  }
121
122  @Override
123  public long getReadRequestCount() {
124    return this.region.getReadRequestsCount();
125  }
126
127  @Override
128  public long getFilteredReadRequestCount() {
129    return this.region.getFilteredReadRequestsCount();
130  }
131
132  @Override
133  public long getWriteRequestCount() {
134    return this.region.getWriteRequestsCount();
135  }
136
137  @Override
138  public long getNumFilesCompacted() {
139    return this.region.compactionNumFilesCompacted.sum();
140  }
141
142  @Override
143  public long getNumBytesCompacted() {
144    return this.region.compactionNumBytesCompacted.sum();
145  }
146
147  @Override
148  public long getNumCompactionsCompleted() {
149    return this.region.compactionsFinished.sum();
150  }
151
152  @Override
153  public long getLastMajorCompactionAge() {
154    long lastMajorCompactionTs = 0L;
155    try {
156      lastMajorCompactionTs = this.region.getOldestHfileTs(true);
157    } catch (IOException ioe) {
158      LOG.error("Could not load HFile info ", ioe);
159    }
160    long now = EnvironmentEdgeManager.currentTime();
161    return now - lastMajorCompactionTs;
162  }
163
164  @Override
165  public long getNumCompactionsFailed() {
166    return this.region.compactionsFailed.sum();
167  }
168
169  @Override
170  public long getNumCompactionsQueued() {
171    return this.region.compactionsQueued.sum();
172  }
173
174  @Override
175  public long getNumFlushesQueued() {
176    return this.region.flushesQueued.sum();
177  }
178
179  @Override
180  public long getMaxCompactionQueueSize() {
181    return maxCompactionQueueSize;
182  }
183
184  @Override
185  public long getMaxFlushQueueSize() {
186    return maxFlushQueueSize;
187  }
188
189  @Override
190  public long getMaxStoreFileAge() {
191    return maxStoreFileAge;
192  }
193
194  @Override
195  public long getMinStoreFileAge() {
196    return minStoreFileAge;
197  }
198
199  @Override
200  public long getAvgStoreFileAge() {
201    return avgStoreFileAge;
202  }
203
204  @Override
205  public long getNumReferenceFiles() {
206    return numReferenceFiles;
207  }
208
209  @Override
210  public int getRegionHashCode() {
211    return this.region.hashCode();
212  }
213
214  public class HRegionMetricsWrapperRunnable implements Runnable {
215
216    @Override
217    public void run() {
218      long tempNumStoreFiles = 0;
219      long tempMemstoreSize = 0;
220      long tempStoreFileSize = 0;
221      long tempMaxStoreFileAge = 0;
222      long tempMinStoreFileAge = Long.MAX_VALUE;
223      long tempNumReferenceFiles = 0;
224      long tempMaxCompactionQueueSize = 0;
225      long tempMaxFlushQueueSize = 0;
226
227      long avgAgeNumerator = 0;
228      long numHFiles = 0;
229      if (region.stores != null) {
230        for (Store store : region.stores.values()) {
231          tempNumStoreFiles += store.getStorefilesCount();
232          tempMemstoreSize += store.getMemStoreSize().getDataSize();
233          tempStoreFileSize += store.getStorefilesSize();
234          OptionalLong storeMaxStoreFileAge = store.getMaxStoreFileAge();
235          if (storeMaxStoreFileAge.isPresent() &&
236              storeMaxStoreFileAge.getAsLong() > tempMaxStoreFileAge) {
237            tempMaxStoreFileAge = storeMaxStoreFileAge.getAsLong();
238          }
239
240          OptionalLong storeMinStoreFileAge = store.getMinStoreFileAge();
241          if (storeMinStoreFileAge.isPresent() &&
242              storeMinStoreFileAge.getAsLong() < tempMinStoreFileAge) {
243            tempMinStoreFileAge = storeMinStoreFileAge.getAsLong();
244          }
245
246          long storeHFiles = store.getNumHFiles();
247          numHFiles += storeHFiles;
248          tempNumReferenceFiles += store.getNumReferenceFiles();
249
250          OptionalDouble storeAvgStoreFileAge = store.getAvgStoreFileAge();
251          if (storeAvgStoreFileAge.isPresent()) {
252            avgAgeNumerator += (long) storeAvgStoreFileAge.getAsDouble() * storeHFiles;
253          }
254        }
255      }
256
257      numStoreFiles = tempNumStoreFiles;
258      memstoreSize = tempMemstoreSize;
259      storeFileSize = tempStoreFileSize;
260      maxStoreFileAge = tempMaxStoreFileAge;
261      if (tempMinStoreFileAge != Long.MAX_VALUE) {
262        minStoreFileAge = tempMinStoreFileAge;
263      }
264
265      if (numHFiles != 0) {
266        avgStoreFileAge = avgAgeNumerator / numHFiles;
267      }
268
269      numReferenceFiles = tempNumReferenceFiles;
270      tempMaxCompactionQueueSize = getNumCompactionsQueued();
271      tempMaxFlushQueueSize = getNumFlushesQueued();
272      if (tempMaxCompactionQueueSize > maxCompactionQueueSize) {
273        maxCompactionQueueSize = tempMaxCompactionQueueSize;
274      }
275      if (tempMaxFlushQueueSize > maxFlushQueueSize) {
276        maxFlushQueueSize = tempMaxFlushQueueSize;
277      }
278    }
279  }
280
281  @Override
282  public void close() throws IOException {
283    regionMetricsUpdateTask.cancel(true);
284  }
285
286  /**
287   * Get the replica id of this region.
288   */
289  @Override
290  public int getReplicaId() {
291    return region.getRegionInfo().getReplicaId();
292  }
293
294}