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.Set;
025import java.util.concurrent.ConcurrentHashMap;
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.HConstants;
031import org.apache.hadoop.hbase.TableName;
032import org.apache.hadoop.metrics2.MetricsExecutor;
033import org.apache.yetus.audience.InterfaceAudience;
034
035import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
036
037@InterfaceAudience.Private
038public class MetricsTableWrapperAggregateImpl implements MetricsTableWrapperAggregate, Closeable {
039  private final HRegionServer regionServer;
040  private ScheduledExecutorService executor;
041  private Runnable runnable;
042  private long period;
043  private ScheduledFuture<?> tableMetricsUpdateTask;
044  private ConcurrentHashMap<TableName, MetricsTableValues> metricsTableMap =
045    new ConcurrentHashMap<>();
046
047  public MetricsTableWrapperAggregateImpl(final HRegionServer regionServer) {
048    this.regionServer = regionServer;
049    this.period = regionServer.getConfiguration().getLong(HConstants.REGIONSERVER_METRICS_PERIOD,
050      HConstants.DEFAULT_REGIONSERVER_METRICS_PERIOD) + 1000;
051    this.executor = CompatibilitySingletonFactory.getInstance(MetricsExecutor.class).getExecutor();
052    this.runnable = new TableMetricsWrapperRunnable();
053    this.tableMetricsUpdateTask =
054      this.executor.scheduleWithFixedDelay(this.runnable, period, period, TimeUnit.MILLISECONDS);
055  }
056
057  public class TableMetricsWrapperRunnable implements Runnable {
058
059    @Override
060    public void run() {
061      Map<TableName, MetricsTableValues> localMetricsTableMap = new HashMap<>();
062      for (Region r : regionServer.getOnlineRegionsLocalContext()) {
063        TableName tbl = r.getTableDescriptor().getTableName();
064        MetricsTableValues mt = localMetricsTableMap.get(tbl);
065        if (mt == null) {
066          mt = new MetricsTableValues();
067          localMetricsTableMap.put(tbl, mt);
068        }
069        long memstoreReadCount = 0L;
070        long mixedReadCount = 0L;
071        String tempKey = null;
072        if (r.getStores() != null) {
073          String familyName = null;
074          for (Store store : r.getStores()) {
075            familyName = store.getColumnFamilyName();
076
077            mt.storeFileCount += store.getStorefilesCount();
078            mt.maxStoreFileCount = Math.max(mt.maxStoreFileCount, store.getStorefilesCount());
079            final MemStoreSize memstoreSize = store.getMemStoreSize();
080            mt.memstoreSize += memstoreSize.getDataSize();
081            mt.memstoreHeapSize += memstoreSize.getHeapSize();
082            mt.memstoreOffHeapSize += memstoreSize.getOffHeapSize();
083            mt.storeFileSize += store.getStorefilesSize();
084            mt.referenceFileCount += store.getNumReferenceFiles();
085            if (store.getMaxStoreFileAge().isPresent()) {
086              mt.maxStoreFileAge =
087                Math.max(mt.maxStoreFileAge, store.getMaxStoreFileAge().getAsLong());
088            }
089            if (store.getMinStoreFileAge().isPresent()) {
090              mt.minStoreFileAge =
091                Math.min(mt.minStoreFileAge, store.getMinStoreFileAge().getAsLong());
092            }
093            if (store.getAvgStoreFileAge().isPresent()) {
094              mt.totalStoreFileAge =
095                (long) store.getAvgStoreFileAge().getAsDouble() * store.getStorefilesCount();
096            }
097            mt.storeCount += 1;
098
099            mt.staticIndexSize += store.getTotalStaticIndexSize();
100            mt.staticBloomSize += store.getTotalStaticBloomSize();
101
102            mt.bloomRequestsCount += store.getBloomFilterRequestsCount();
103            mt.bloomNegativeResultsCount += store.getBloomFilterNegativeResultsCount();
104            mt.bloomEligibleRequestsCount += store.getBloomFilterEligibleRequestsCount();
105
106            tempKey = tbl.getNameAsString() + HASH + familyName;
107            Long tempVal = mt.perStoreMemstoreOnlyReadCount.get(tempKey);
108            if (tempVal == null) {
109              tempVal = 0L;
110            }
111            memstoreReadCount = store.getMemstoreOnlyRowReadsCount() + tempVal;
112            tempVal = mt.perStoreMixedReadCount.get(tempKey);
113            if (tempVal == null) {
114              tempVal = 0L;
115            }
116            mixedReadCount = store.getMixedRowReadsCount() + tempVal;
117            // accumulate the count
118            mt.perStoreMemstoreOnlyReadCount.put(tempKey, memstoreReadCount);
119            mt.perStoreMixedReadCount.put(tempKey, mixedReadCount);
120            mt.perStoreFileSize.merge(tempKey, store.getStorefilesSize(), Long::sum);
121          }
122
123          mt.regionCount += 1;
124
125          mt.readRequestCount += r.getReadRequestsCount();
126          mt.filteredReadRequestCount += r.getFilteredReadRequestsCount();
127          mt.writeRequestCount += r.getWriteRequestsCount();
128        }
129      }
130
131      for (Map.Entry<TableName, MetricsTableValues> entry : localMetricsTableMap.entrySet()) {
132        TableName tbl = entry.getKey();
133        if (metricsTableMap.get(tbl) == null) {
134          // this will add the Wrapper to the list of TableMetrics
135          CompatibilitySingletonFactory.getInstance(MetricsRegionServerSourceFactory.class)
136            .getTableAggregate()
137            .getOrCreateTableSource(tbl.getNameAsString(), MetricsTableWrapperAggregateImpl.this);
138        }
139        metricsTableMap.put(entry.getKey(), entry.getValue());
140      }
141      Set<TableName> existingTableNames = Sets.newHashSet(metricsTableMap.keySet());
142      existingTableNames.removeAll(localMetricsTableMap.keySet());
143      MetricsTableAggregateSource agg = CompatibilitySingletonFactory
144        .getInstance(MetricsRegionServerSourceFactory.class).getTableAggregate();
145      for (TableName table : existingTableNames) {
146        agg.deleteTableSource(table.getNameAsString());
147        if (metricsTableMap.get(table) != null) {
148          metricsTableMap.remove(table);
149        }
150      }
151    }
152  }
153
154  @Override
155  public long getReadRequestCount(String table) {
156    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
157    if (metricsTable == null) {
158      return 0;
159    } else {
160      return metricsTable.readRequestCount;
161    }
162  }
163
164  @Override
165  public Map<String, Long> getMemstoreOnlyRowReadsCount(String table) {
166    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
167    if (metricsTable == null) {
168      return null;
169    } else {
170      return metricsTable.perStoreMemstoreOnlyReadCount;
171    }
172  }
173
174  @Override
175  public Map<String, Long> getMixedRowReadsCount(String table) {
176    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
177    if (metricsTable == null) {
178      return null;
179    } else {
180      return metricsTable.perStoreMixedReadCount;
181    }
182  }
183
184  @Override
185  public Map<String, Long> getStoreFileSizePerStore(String table) {
186    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
187    if (metricsTable == null) {
188      return null;
189    } else {
190      return metricsTable.perStoreFileSize;
191    }
192  }
193
194  @Override
195  public long getCpRequestsCount(String table) {
196    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
197    if (metricsTable == null) {
198      return 0;
199    } else {
200      return metricsTable.cpRequestCount;
201    }
202  }
203
204  public long getFilteredReadRequestCount(String table) {
205    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
206    if (metricsTable == null) {
207      return 0;
208    }
209    return metricsTable.filteredReadRequestCount;
210  }
211
212  @Override
213  public long getWriteRequestCount(String table) {
214    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
215    if (metricsTable == null) {
216      return 0;
217    } else {
218      return metricsTable.writeRequestCount;
219    }
220  }
221
222  @Override
223  public long getTotalRequestsCount(String table) {
224    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
225    if (metricsTable == null) {
226      return 0;
227    } else {
228      return metricsTable.readRequestCount + metricsTable.writeRequestCount;
229    }
230  }
231
232  @Override
233  public long getMemStoreSize(String table) {
234    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
235    if (metricsTable == null) {
236      return 0;
237    } else {
238      return metricsTable.memstoreSize;
239    }
240  }
241
242  @Override
243  public long getMemStoreHeapSize(String table) {
244    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
245    if (metricsTable == null) {
246      return 0;
247    } else {
248      return metricsTable.memstoreHeapSize;
249    }
250  }
251
252  @Override
253  public long getMemStoreOffHeapSize(String table) {
254    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
255    if (metricsTable == null) {
256      return 0;
257    } else {
258      return metricsTable.memstoreOffHeapSize;
259    }
260  }
261
262  @Override
263  public long getStoreFileSize(String table) {
264    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
265    if (metricsTable == null) {
266      return 0;
267    } else {
268      return metricsTable.storeFileSize;
269    }
270  }
271
272  @Override
273  public long getTableSize(String table) {
274    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
275    if (metricsTable == null) {
276      return 0;
277    } else {
278      return metricsTable.memstoreSize + metricsTable.storeFileSize;
279    }
280  }
281
282  public long getNumRegions(String table) {
283    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
284    if (metricsTable == null) {
285      return 0;
286    }
287    return metricsTable.regionCount;
288  }
289
290  @Override
291  public long getNumStores(String table) {
292    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
293    if (metricsTable == null) {
294      return 0;
295    }
296    return metricsTable.storeCount;
297  }
298
299  @Override
300  public long getNumStoreFiles(String table) {
301    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
302    if (metricsTable == null) {
303      return 0;
304    }
305    return metricsTable.storeFileCount;
306  }
307
308  @Override
309  public long getMaxStoreFiles(String table) {
310    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
311    if (metricsTable == null) {
312      return 0;
313    }
314    return metricsTable.maxStoreFileCount;
315  }
316
317  @Override
318  public long getMaxStoreFileAge(String table) {
319    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
320    if (metricsTable == null) {
321      return 0;
322    }
323    return metricsTable.maxStoreFileAge;
324  }
325
326  @Override
327  public long getMinStoreFileAge(String table) {
328    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
329    if (metricsTable == null) {
330      return 0;
331    }
332    return metricsTable.minStoreFileAge == Long.MAX_VALUE ? 0 : metricsTable.minStoreFileAge;
333  }
334
335  @Override
336  public long getAvgStoreFileAge(String table) {
337    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
338    if (metricsTable == null) {
339      return 0;
340    }
341
342    return metricsTable.storeFileCount == 0
343      ? 0
344      : (metricsTable.totalStoreFileAge / metricsTable.storeFileCount);
345  }
346
347  @Override
348  public long getStaticIndexSize(String table) {
349    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
350    if (metricsTable == null) {
351      return 0;
352    }
353
354    return metricsTable.staticIndexSize;
355  }
356
357  @Override
358  public long getStaticBloomSize(String table) {
359    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
360    if (metricsTable == null) {
361      return 0;
362    }
363
364    return metricsTable.staticBloomSize;
365  }
366
367  @Override
368  public long getBloomFilterRequestsCount(String table) {
369    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
370    if (metricsTable == null) {
371      return 0;
372    }
373
374    return metricsTable.bloomRequestsCount;
375  }
376
377  @Override
378  public long getBloomFilterNegativeResultsCount(String table) {
379    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
380    if (metricsTable == null) {
381      return 0;
382    }
383
384    return metricsTable.bloomNegativeResultsCount;
385  }
386
387  @Override
388  public long getBloomFilterEligibleRequestsCount(String table) {
389    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
390    if (metricsTable == null) {
391      return 0;
392    }
393
394    return metricsTable.bloomEligibleRequestsCount;
395  }
396
397  @Override
398  public long getNumReferenceFiles(String table) {
399    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
400    if (metricsTable == null) {
401      return 0;
402    }
403    return metricsTable.referenceFileCount;
404  }
405
406  @Override
407  public long getAvgRegionSize(String table) {
408    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
409    if (metricsTable == null) {
410      return 0;
411    }
412    return metricsTable.regionCount == 0
413      ? 0
414      : (metricsTable.memstoreSize + metricsTable.storeFileSize) / metricsTable.regionCount;
415  }
416
417  public long getCpRequestCount(String table) {
418    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
419    if (metricsTable == null) {
420      return 0;
421    }
422    return metricsTable.cpRequestCount;
423  }
424
425  @Override
426  public void close() throws IOException {
427    tableMetricsUpdateTask.cancel(true);
428  }
429
430  private static class MetricsTableValues {
431    long readRequestCount;
432    long filteredReadRequestCount;
433    long writeRequestCount;
434    long memstoreSize;
435    long memstoreHeapSize;
436    long memstoreOffHeapSize;
437    long regionCount;
438    long storeCount;
439    long storeFileCount;
440    long maxStoreFileCount;
441    long storeFileSize;
442    long maxStoreFileAge;
443    long minStoreFileAge = Long.MAX_VALUE;
444    long totalStoreFileAge;
445
446    long staticIndexSize;
447
448    long staticBloomSize;
449    long referenceFileCount;
450
451    long bloomRequestsCount;
452    long bloomNegativeResultsCount;
453    long bloomEligibleRequestsCount;
454    long cpRequestCount;
455    Map<String, Long> perStoreMemstoreOnlyReadCount = new HashMap<>();
456    Map<String, Long> perStoreMixedReadCount = new HashMap<>();
457    Map<String, Long> perStoreFileSize = new HashMap<>();
458  }
459
460}