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  public long getCpRequestsCount(String table) {
182    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
183    if (metricsTable == null) {
184      return 0;
185    } else {
186      return metricsTable.cpRequestCount;
187    }
188  }
189
190  public long getFilteredReadRequestCount(String table) {
191    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
192    if (metricsTable == null) {
193      return 0;
194    }
195    return metricsTable.filteredReadRequestCount;
196  }
197
198  @Override
199  public long getWriteRequestCount(String table) {
200    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
201    if (metricsTable == null) {
202      return 0;
203    } else {
204      return metricsTable.writeRequestCount;
205    }
206  }
207
208  @Override
209  public long getTotalRequestsCount(String table) {
210    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
211    if (metricsTable == null) {
212      return 0;
213    } else {
214      return metricsTable.readRequestCount + metricsTable.writeRequestCount;
215    }
216  }
217
218  @Override
219  public long getMemStoreSize(String table) {
220    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
221    if (metricsTable == null) {
222      return 0;
223    } else {
224      return metricsTable.memstoreSize;
225    }
226  }
227
228  @Override
229  public long getStoreFileSize(String table) {
230    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
231    if (metricsTable == null) {
232      return 0;
233    } else {
234      return metricsTable.storeFileSize;
235    }
236  }
237
238  @Override
239  public long getTableSize(String table) {
240    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
241    if (metricsTable == null) {
242      return 0;
243    } else {
244      return metricsTable.memstoreSize + metricsTable.storeFileSize;
245    }
246  }
247
248  public long getNumRegions(String table) {
249    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
250    if (metricsTable == null) {
251      return 0;
252    }
253    return metricsTable.regionCount;
254  }
255
256  @Override
257  public long getNumStores(String table) {
258    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
259    if (metricsTable == null) {
260      return 0;
261    }
262    return metricsTable.storeCount;
263  }
264
265  @Override
266  public long getNumStoreFiles(String table) {
267    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
268    if (metricsTable == null) {
269      return 0;
270    }
271    return metricsTable.storeFileCount;
272  }
273
274  @Override
275  public long getMaxStoreFiles(String table) {
276    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
277    if (metricsTable == null) {
278      return 0;
279    }
280    return metricsTable.maxStoreFileCount;
281  }
282
283  @Override
284  public long getMaxStoreFileAge(String table) {
285    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
286    if (metricsTable == null) {
287      return 0;
288    }
289    return metricsTable.maxStoreFileAge;
290  }
291
292  @Override
293  public long getMinStoreFileAge(String table) {
294    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
295    if (metricsTable == null) {
296      return 0;
297    }
298    return metricsTable.minStoreFileAge == Long.MAX_VALUE ? 0 : metricsTable.minStoreFileAge;
299  }
300
301  @Override
302  public long getAvgStoreFileAge(String table) {
303    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
304    if (metricsTable == null) {
305      return 0;
306    }
307
308    return metricsTable.storeFileCount == 0
309      ? 0
310      : (metricsTable.totalStoreFileAge / metricsTable.storeFileCount);
311  }
312
313  @Override
314  public long getStaticIndexSize(String table) {
315    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
316    if (metricsTable == null) {
317      return 0;
318    }
319
320    return metricsTable.staticIndexSize;
321  }
322
323  @Override
324  public long getStaticBloomSize(String table) {
325    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
326    if (metricsTable == null) {
327      return 0;
328    }
329
330    return metricsTable.staticBloomSize;
331  }
332
333  @Override
334  public long getBloomFilterRequestsCount(String table) {
335    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
336    if (metricsTable == null) {
337      return 0;
338    }
339
340    return metricsTable.bloomRequestsCount;
341  }
342
343  @Override
344  public long getBloomFilterNegativeResultsCount(String table) {
345    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
346    if (metricsTable == null) {
347      return 0;
348    }
349
350    return metricsTable.bloomNegativeResultsCount;
351  }
352
353  @Override
354  public long getBloomFilterEligibleRequestsCount(String table) {
355    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
356    if (metricsTable == null) {
357      return 0;
358    }
359
360    return metricsTable.bloomEligibleRequestsCount;
361  }
362
363  @Override
364  public long getNumReferenceFiles(String table) {
365    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
366    if (metricsTable == null) {
367      return 0;
368    }
369    return metricsTable.referenceFileCount;
370  }
371
372  @Override
373  public long getAvgRegionSize(String table) {
374    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
375    if (metricsTable == null) {
376      return 0;
377    }
378    return metricsTable.regionCount == 0
379      ? 0
380      : (metricsTable.memstoreSize + metricsTable.storeFileSize) / metricsTable.regionCount;
381  }
382
383  public long getCpRequestCount(String table) {
384    MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
385    if (metricsTable == null) {
386      return 0;
387    }
388    return metricsTable.cpRequestCount;
389  }
390
391  @Override
392  public void close() throws IOException {
393    tableMetricsUpdateTask.cancel(true);
394  }
395
396  private static class MetricsTableValues {
397    long readRequestCount;
398    long filteredReadRequestCount;
399    long writeRequestCount;
400    long memstoreSize;
401    long regionCount;
402    long storeCount;
403    long storeFileCount;
404    long maxStoreFileCount;
405    long storeFileSize;
406    long maxStoreFileAge;
407    long minStoreFileAge = Long.MAX_VALUE;
408    long totalStoreFileAge;
409
410    long staticIndexSize;
411
412    long staticBloomSize;
413    long referenceFileCount;
414
415    long bloomRequestsCount;
416    long bloomNegativeResultsCount;
417    long bloomEligibleRequestsCount;
418    long cpRequestCount;
419    Map<String, Long> perStoreMemstoreOnlyReadCount = new HashMap<>();
420    Map<String, Long> perStoreMixedReadCount = new HashMap<>();
421  }
422
423}