1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.namequeues;
21
22 import java.util.ArrayList;
23 import java.util.List;
24 import java.util.concurrent.locks.ReentrantLock;
25 import org.apache.hadoop.conf.Configuration;
26 import org.apache.hadoop.hbase.classification.InterfaceAudience;
27 import org.apache.hadoop.hbase.namequeues.queue.EvictingQueue;
28 import org.apache.hadoop.hbase.protobuf.generated.TooSlowLog;
29 import org.apache.hadoop.hbase.slowlog.SlowLogTableAccessor;
30 import org.slf4j.Logger;
31 import org.slf4j.LoggerFactory;
32
33
34
35
36 @InterfaceAudience.Private
37 public class SlowLogPersistentService {
38
39 private static final Logger LOG = LoggerFactory.getLogger(SlowLogPersistentService.class);
40
41 private static final ReentrantLock LOCK = new ReentrantLock();
42 private static final String SYS_TABLE_QUEUE_SIZE =
43 "hbase.regionserver.slowlog.systable.queue.size";
44 private static final int DEFAULT_SYS_TABLE_QUEUE_SIZE = 1000;
45 private static final int SYSTABLE_PUT_BATCH_SIZE = 100;
46
47 private final EvictingQueue<TooSlowLog.SlowLogPayload> queueForSysTable;
48
49 private final Configuration configuration;
50
51 public SlowLogPersistentService(final Configuration configuration) {
52 this.configuration = configuration;
53 int sysTableQueueSize =
54 configuration.getInt(SYS_TABLE_QUEUE_SIZE, DEFAULT_SYS_TABLE_QUEUE_SIZE);
55 queueForSysTable = EvictingQueue.create(sysTableQueueSize);
56 }
57
58 public void addToQueueForSysTable(TooSlowLog.SlowLogPayload slowLogPayload) {
59 queueForSysTable.add(slowLogPayload);
60 }
61
62
63
64
65 public void addAllLogsToSysTable() {
66 if (queueForSysTable == null) {
67 LOG.trace("hbase.regionserver.slowlog.systable.enabled is turned off. Exiting.");
68 return;
69 }
70 if (LOCK.isLocked()) {
71 return;
72 }
73 LOCK.lock();
74 try {
75 List<TooSlowLog.SlowLogPayload> slowLogPayloads = new ArrayList<>();
76 int i = 0;
77 while (!queueForSysTable.isEmpty()) {
78 slowLogPayloads.add(queueForSysTable.poll());
79 i++;
80 if (i == SYSTABLE_PUT_BATCH_SIZE) {
81 SlowLogTableAccessor.addSlowLogRecords(slowLogPayloads, this.configuration);
82 slowLogPayloads.clear();
83 i = 0;
84 }
85 }
86 if (slowLogPayloads.size() > 0) {
87 SlowLogTableAccessor.addSlowLogRecords(slowLogPayloads, this.configuration);
88 }
89 } finally {
90 LOCK.unlock();
91 }
92 }
93
94 }