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