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