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 */ 018package org.apache.hadoop.hbase.regionserver; 019 020import java.io.Closeable; 021import java.io.IOException; 022import java.util.HashMap; 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; 029import org.apache.hadoop.hbase.CompatibilitySingletonFactory; 030import org.apache.hadoop.hbase.client.RegionInfo; 031import org.apache.hadoop.hbase.client.TableDescriptor; 032import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; 033import org.apache.hadoop.metrics2.MetricsExecutor; 034import org.apache.yetus.audience.InterfaceAudience; 035import org.slf4j.Logger; 036import org.slf4j.LoggerFactory; 037 038@InterfaceAudience.Private 039public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable { 040 041 private static final Logger LOG = LoggerFactory.getLogger(MetricsRegionWrapperImpl.class); 042 043 public static final int PERIOD = 45; 044 public static final String UNKNOWN = "unknown"; 045 046 private final HRegion region; 047 private ScheduledExecutorService executor; 048 private Runnable runnable; 049 private long numStoreFiles; 050 private long storeRefCount; 051 private long maxCompactedStoreFileRefCount; 052 private long memstoreSize; 053 private long storeFileSize; 054 private long maxStoreFileAge; 055 private long minStoreFileAge; 056 private long avgStoreFileAge; 057 private long numReferenceFiles; 058 private long maxFlushQueueSize; 059 private long maxCompactionQueueSize; 060 private Map<String, Long> readsOnlyFromMemstore; 061 private Map<String, Long> mixedReadsOnStore; 062 063 private ScheduledFuture<?> regionMetricsUpdateTask; 064 065 public MetricsRegionWrapperImpl(HRegion region) { 066 this.region = region; 067 this.executor = CompatibilitySingletonFactory.getInstance(MetricsExecutor.class).getExecutor(); 068 this.runnable = new HRegionMetricsWrapperRunnable(); 069 this.regionMetricsUpdateTask = 070 this.executor.scheduleWithFixedDelay(this.runnable, PERIOD, PERIOD, TimeUnit.SECONDS); 071 } 072 073 @Override 074 public String getTableName() { 075 TableDescriptor tableDesc = this.region.getTableDescriptor(); 076 if (tableDesc == null) { 077 return UNKNOWN; 078 } 079 return tableDesc.getTableName().getQualifierAsString(); 080 } 081 082 @Override 083 public String getNamespace() { 084 TableDescriptor tableDesc = this.region.getTableDescriptor(); 085 if (tableDesc == null) { 086 return UNKNOWN; 087 } 088 return tableDesc.getTableName().getNamespaceAsString(); 089 } 090 091 @Override 092 public String getRegionName() { 093 RegionInfo regionInfo = this.region.getRegionInfo(); 094 if (regionInfo == null) { 095 return UNKNOWN; 096 } 097 return regionInfo.getEncodedName(); 098 } 099 100 @Override 101 public long getNumStores() { 102 Map<byte[], HStore> stores = this.region.stores; 103 if (stores == null) { 104 return 0; 105 } 106 return stores.size(); 107 } 108 109 @Override 110 public long getNumStoreFiles() { 111 return numStoreFiles; 112 } 113 114 @Override 115 public long getMemStoreSize() { 116 return memstoreSize; 117 } 118 119 @Override 120 public long getStoreFileSize() { 121 return storeFileSize; 122 } 123 124 @Override 125 public long getStoreRefCount() { 126 return storeRefCount; 127 } 128 129 @Override 130 public long getMaxCompactedStoreFileRefCount() { 131 return maxCompactedStoreFileRefCount; 132 } 133 134 @Override 135 public long getReadRequestCount() { 136 return this.region.getReadRequestsCount(); 137 } 138 139 @Override 140 public long getFilteredReadRequestCount() { 141 return this.region.getFilteredReadRequestsCount(); 142 } 143 144 @Override 145 public long getWriteRequestCount() { 146 return this.region.getWriteRequestsCount(); 147 } 148 149 @Override 150 public long getNumFilesCompacted() { 151 return this.region.compactionNumFilesCompacted.sum(); 152 } 153 154 @Override 155 public long getNumBytesCompacted() { 156 return this.region.compactionNumBytesCompacted.sum(); 157 } 158 159 @Override 160 public long getNumCompactionsCompleted() { 161 return this.region.compactionsFinished.sum(); 162 } 163 164 @Override 165 public long getLastMajorCompactionAge() { 166 long lastMajorCompactionTs = 0L; 167 try { 168 lastMajorCompactionTs = this.region.getOldestHfileTs(true); 169 } catch (IOException ioe) { 170 LOG.error("Could not load HFile info ", ioe); 171 } 172 long now = EnvironmentEdgeManager.currentTime(); 173 return now - lastMajorCompactionTs; 174 } 175 176 @Override 177 public long getTotalRequestCount() { 178 return getReadRequestCount() + getWriteRequestCount(); 179 } 180 181 @Override 182 public long getNumCompactionsFailed() { 183 return this.region.compactionsFailed.sum(); 184 } 185 186 @Override 187 public long getNumCompactionsQueued() { 188 return this.region.compactionsQueued.sum(); 189 } 190 191 @Override 192 public long getNumFlushesQueued() { 193 return this.region.flushesQueued.sum(); 194 } 195 196 @Override 197 public long getMaxCompactionQueueSize() { 198 return maxCompactionQueueSize; 199 } 200 201 @Override 202 public long getMaxFlushQueueSize() { 203 return maxFlushQueueSize; 204 } 205 206 @Override 207 public long getMaxStoreFileAge() { 208 return maxStoreFileAge; 209 } 210 211 @Override 212 public long getMinStoreFileAge() { 213 return minStoreFileAge; 214 } 215 216 @Override 217 public long getAvgStoreFileAge() { 218 return avgStoreFileAge; 219 } 220 221 @Override 222 public long getNumReferenceFiles() { 223 return numReferenceFiles; 224 } 225 226 @Override 227 public int getRegionHashCode() { 228 return this.region.hashCode(); 229 } 230 231 @Override 232 public Map<String, Long> getMemstoreOnlyRowReadsCount() { 233 return readsOnlyFromMemstore; 234 } 235 236 @Override 237 public Map<String, Long> getMixedRowReadsCount() { 238 return mixedReadsOnStore; 239 } 240 241 public class HRegionMetricsWrapperRunnable implements Runnable { 242 243 @Override 244 public void run() { 245 long tempNumStoreFiles = 0; 246 int tempStoreRefCount = 0; 247 int tempMaxCompactedStoreFileRefCount = 0; 248 long tempMemstoreSize = 0; 249 long tempStoreFileSize = 0; 250 long tempMaxStoreFileAge = 0; 251 long tempMinStoreFileAge = Long.MAX_VALUE; 252 long tempNumReferenceFiles = 0; 253 long tempMaxCompactionQueueSize = 0; 254 long tempMaxFlushQueueSize = 0; 255 long avgAgeNumerator = 0; 256 long numHFiles = 0; 257 if (region.stores != null) { 258 for (HStore store : region.stores.values()) { 259 tempNumStoreFiles += store.getStorefilesCount(); 260 int currentStoreRefCount = store.getStoreRefCount(); 261 tempStoreRefCount += currentStoreRefCount; 262 int currentMaxCompactedStoreFileRefCount = store.getMaxCompactedStoreFileRefCount(); 263 tempMaxCompactedStoreFileRefCount = 264 Math.max(tempMaxCompactedStoreFileRefCount, currentMaxCompactedStoreFileRefCount); 265 tempMemstoreSize += store.getMemStoreSize().getDataSize(); 266 tempStoreFileSize += store.getStorefilesSize(); 267 OptionalLong storeMaxStoreFileAge = store.getMaxStoreFileAge(); 268 if ( 269 storeMaxStoreFileAge.isPresent() 270 && storeMaxStoreFileAge.getAsLong() > tempMaxStoreFileAge 271 ) { 272 tempMaxStoreFileAge = storeMaxStoreFileAge.getAsLong(); 273 } 274 275 OptionalLong storeMinStoreFileAge = store.getMinStoreFileAge(); 276 if ( 277 storeMinStoreFileAge.isPresent() 278 && storeMinStoreFileAge.getAsLong() < tempMinStoreFileAge 279 ) { 280 tempMinStoreFileAge = storeMinStoreFileAge.getAsLong(); 281 } 282 283 long storeHFiles = store.getNumHFiles(); 284 numHFiles += storeHFiles; 285 tempNumReferenceFiles += store.getNumReferenceFiles(); 286 287 OptionalDouble storeAvgStoreFileAge = store.getAvgStoreFileAge(); 288 if (storeAvgStoreFileAge.isPresent()) { 289 avgAgeNumerator += (long) storeAvgStoreFileAge.getAsDouble() * storeHFiles; 290 } 291 if (mixedReadsOnStore == null) { 292 mixedReadsOnStore = new HashMap<String, Long>(); 293 } 294 Long tempVal = mixedReadsOnStore.get(store.getColumnFamilyName()); 295 if (tempVal == null) { 296 tempVal = 0L; 297 } else { 298 tempVal += store.getMixedRowReadsCount(); 299 } 300 mixedReadsOnStore.put(store.getColumnFamilyName(), tempVal); 301 if (readsOnlyFromMemstore == null) { 302 readsOnlyFromMemstore = new HashMap<String, Long>(); 303 } 304 tempVal = readsOnlyFromMemstore.get(store.getColumnFamilyName()); 305 if (tempVal == null) { 306 tempVal = 0L; 307 } else { 308 tempVal += store.getMemstoreOnlyRowReadsCount(); 309 } 310 readsOnlyFromMemstore.put(store.getColumnFamilyName(), tempVal); 311 } 312 } 313 314 numStoreFiles = tempNumStoreFiles; 315 storeRefCount = tempStoreRefCount; 316 maxCompactedStoreFileRefCount = tempMaxCompactedStoreFileRefCount; 317 memstoreSize = tempMemstoreSize; 318 storeFileSize = tempStoreFileSize; 319 maxStoreFileAge = tempMaxStoreFileAge; 320 if (tempMinStoreFileAge != Long.MAX_VALUE) { 321 minStoreFileAge = tempMinStoreFileAge; 322 } 323 324 if (numHFiles != 0) { 325 avgStoreFileAge = avgAgeNumerator / numHFiles; 326 } 327 328 numReferenceFiles = tempNumReferenceFiles; 329 tempMaxCompactionQueueSize = getNumCompactionsQueued(); 330 tempMaxFlushQueueSize = getNumFlushesQueued(); 331 if (tempMaxCompactionQueueSize > maxCompactionQueueSize) { 332 maxCompactionQueueSize = tempMaxCompactionQueueSize; 333 } 334 if (tempMaxFlushQueueSize > maxFlushQueueSize) { 335 maxFlushQueueSize = tempMaxFlushQueueSize; 336 } 337 } 338 } 339 340 @Override 341 public void close() throws IOException { 342 regionMetricsUpdateTask.cancel(true); 343 } 344 345 /** 346 * Get the replica id of this region. 347 */ 348 @Override 349 public int getReplicaId() { 350 return region.getRegionInfo().getReplicaId(); 351 } 352 353}