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          }
121
122          mt.regionCount += 1;
123
124          mt.readRequestCount += r.getReadRequestsCount();
125          mt.filteredReadRequestCount += r.getFilteredReadRequestsCount();
126          mt.writeRequestCount += r.getWriteRequestsCount();
127        }
128      }
129
130      for (Map.Entry<TableName, MetricsTableValues> entry : localMetricsTableMap.entrySet()) {
131        TableName tbl = entry.getKey();
132        if (metricsTableMap.get(tbl) == null) {
133          // this will add the Wrapper to the list of TableMetrics
134          CompatibilitySingletonFactory.getInstance(MetricsRegionServerSourceFactory.class)
135            .getTableAggregate()
136            .getOrCreateTableSource(tbl.getNameAsString(), MetricsTableWrapperAggregateImpl.this);
137        }
138        metricsTableMap.put(entry.getKey(), entry.getValue());
139      }
140      Set<TableName> existingTableNames = Sets.newHashSet(metricsTableMap.keySet());
141      existingTableNames.removeAll(localMetricsTableMap.keySet());
142      MetricsTableAggregateSource agg = CompatibilitySingletonFactory
143        .getInstance(MetricsRegionServerSourceFactory.class).getTableAggregate();
144      for (TableName table : existingTableNames) {
145        agg.deleteTableSource(table.getNameAsString());
146        if (metricsTableMap.get(table) != null) {
147          metricsTableMap.remove(table);
148        }
149      }
150    }
151  }
152
153  @Override
154  public long getReadRequestCount(String table) {
155    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
156    if (metricsTable == null) {
157      return 0;
158    } else {
159      return metricsTable.readRequestCount;
160    }
161  }
162
163  @Override
164  public Map<String, Long> getMemstoreOnlyRowReadsCount(String table) {
165    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
166    if (metricsTable == null) {
167      return null;
168    } else {
169      return metricsTable.perStoreMemstoreOnlyReadCount;
170    }
171  }
172
173  @Override
174  public Map<String, Long> getMixedRowReadsCount(String table) {
175    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
176    if (metricsTable == null) {
177      return null;
178    } else {
179      return metricsTable.perStoreMixedReadCount;
180    }
181  }
182
183  @Override
184  public long getCpRequestsCount(String table) {
185    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
186    if (metricsTable == null) {
187      return 0;
188    } else {
189      return metricsTable.cpRequestCount;
190    }
191  }
192
193  public long getFilteredReadRequestCount(String table) {
194    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
195    if (metricsTable == null) {
196      return 0;
197    }
198    return metricsTable.filteredReadRequestCount;
199  }
200
201  @Override
202  public long getWriteRequestCount(String table) {
203    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
204    if (metricsTable == null) {
205      return 0;
206    } else {
207      return metricsTable.writeRequestCount;
208    }
209  }
210
211  @Override
212  public long getTotalRequestsCount(String table) {
213    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
214    if (metricsTable == null) {
215      return 0;
216    } else {
217      return metricsTable.readRequestCount + metricsTable.writeRequestCount;
218    }
219  }
220
221  @Override
222  public long getMemStoreSize(String table) {
223    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
224    if (metricsTable == null) {
225      return 0;
226    } else {
227      return metricsTable.memstoreSize;
228    }
229  }
230
231  @Override
232  public long getMemStoreHeapSize(String table) {
233    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
234    if (metricsTable == null) {
235      return 0;
236    } else {
237      return metricsTable.memstoreHeapSize;
238    }
239  }
240
241  @Override
242  public long getMemStoreOffHeapSize(String table) {
243    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
244    if (metricsTable == null) {
245      return 0;
246    } else {
247      return metricsTable.memstoreOffHeapSize;
248    }
249  }
250
251  @Override
252  public long getStoreFileSize(String table) {
253    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
254    if (metricsTable == null) {
255      return 0;
256    } else {
257      return metricsTable.storeFileSize;
258    }
259  }
260
261  @Override
262  public long getTableSize(String table) {
263    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
264    if (metricsTable == null) {
265      return 0;
266    } else {
267      return metricsTable.memstoreSize + metricsTable.storeFileSize;
268    }
269  }
270
271  public long getNumRegions(String table) {
272    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
273    if (metricsTable == null) {
274      return 0;
275    }
276    return metricsTable.regionCount;
277  }
278
279  @Override
280  public long getNumStores(String table) {
281    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
282    if (metricsTable == null) {
283      return 0;
284    }
285    return metricsTable.storeCount;
286  }
287
288  @Override
289  public long getNumStoreFiles(String table) {
290    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
291    if (metricsTable == null) {
292      return 0;
293    }
294    return metricsTable.storeFileCount;
295  }
296
297  @Override
298  public long getMaxStoreFiles(String table) {
299    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
300    if (metricsTable == null) {
301      return 0;
302    }
303    return metricsTable.maxStoreFileCount;
304  }
305
306  @Override
307  public long getMaxStoreFileAge(String table) {
308    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
309    if (metricsTable == null) {
310      return 0;
311    }
312    return metricsTable.maxStoreFileAge;
313  }
314
315  @Override
316  public long getMinStoreFileAge(String table) {
317    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
318    if (metricsTable == null) {
319      return 0;
320    }
321    return metricsTable.minStoreFileAge == Long.MAX_VALUE ? 0 : metricsTable.minStoreFileAge;
322  }
323
324  @Override
325  public long getAvgStoreFileAge(String table) {
326    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
327    if (metricsTable == null) {
328      return 0;
329    }
330
331    return metricsTable.storeFileCount == 0
332      ? 0
333      : (metricsTable.totalStoreFileAge / metricsTable.storeFileCount);
334  }
335
336  @Override
337  public long getStaticIndexSize(String table) {
338    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
339    if (metricsTable == null) {
340      return 0;
341    }
342
343    return metricsTable.staticIndexSize;
344  }
345
346  @Override
347  public long getStaticBloomSize(String table) {
348    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
349    if (metricsTable == null) {
350      return 0;
351    }
352
353    return metricsTable.staticBloomSize;
354  }
355
356  @Override
357  public long getBloomFilterRequestsCount(String table) {
358    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
359    if (metricsTable == null) {
360      return 0;
361    }
362
363    return metricsTable.bloomRequestsCount;
364  }
365
366  @Override
367  public long getBloomFilterNegativeResultsCount(String table) {
368    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
369    if (metricsTable == null) {
370      return 0;
371    }
372
373    return metricsTable.bloomNegativeResultsCount;
374  }
375
376  @Override
377  public long getBloomFilterEligibleRequestsCount(String table) {
378    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
379    if (metricsTable == null) {
380      return 0;
381    }
382
383    return metricsTable.bloomEligibleRequestsCount;
384  }
385
386  @Override
387  public long getNumReferenceFiles(String table) {
388    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
389    if (metricsTable == null) {
390      return 0;
391    }
392    return metricsTable.referenceFileCount;
393  }
394
395  @Override
396  public long getAvgRegionSize(String table) {
397    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
398    if (metricsTable == null) {
399      return 0;
400    }
401    return metricsTable.regionCount == 0
402      ? 0
403      : (metricsTable.memstoreSize + metricsTable.storeFileSize) / metricsTable.regionCount;
404  }
405
406  public long getCpRequestCount(String table) {
407    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
408    if (metricsTable == null) {
409      return 0;
410    }
411    return metricsTable.cpRequestCount;
412  }
413
414  @Override
415  public void close() throws IOException {
416    tableMetricsUpdateTask.cancel(true);
417  }
418
419  private static class MetricsTableValues {
420    long readRequestCount;
421    long filteredReadRequestCount;
422    long writeRequestCount;
423    long memstoreSize;
424    long memstoreHeapSize;
425    long memstoreOffHeapSize;
426    long regionCount;
427    long storeCount;
428    long storeFileCount;
429    long maxStoreFileCount;
430    long storeFileSize;
431    long maxStoreFileAge;
432    long minStoreFileAge = Long.MAX_VALUE;
433    long totalStoreFileAge;
434
435    long staticIndexSize;
436
437    long staticBloomSize;
438    long referenceFileCount;
439
440    long bloomRequestsCount;
441    long bloomNegativeResultsCount;
442    long bloomEligibleRequestsCount;
443    long cpRequestCount;
444    Map<String, Long> perStoreMemstoreOnlyReadCount = new HashMap<>();
445    Map<String, Long> perStoreMixedReadCount = new HashMap<>();
446  }
447
448}