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 com.google.common.base.Preconditions;
23 import com.lmax.disruptor.BlockingWaitStrategy;
24 import com.lmax.disruptor.EventFactory;
25 import com.lmax.disruptor.RingBuffer;
26 import com.lmax.disruptor.dsl.Disruptor;
27 import com.lmax.disruptor.dsl.ProducerType;
28 import java.util.concurrent.Executors;
29 import org.apache.hadoop.conf.Configuration;
30 import org.apache.hadoop.hbase.classification.InterfaceAudience;
31 import org.apache.hadoop.hbase.classification.InterfaceStability;
32 import org.apache.hadoop.hbase.namequeues.request.NamedQueueGetRequest;
33 import org.apache.hadoop.hbase.namequeues.response.NamedQueueGetResponse;
34 import org.apache.hadoop.hbase.util.Threads;
35
36
37
38
39
40
41
42 @InterfaceAudience.Private
43 @InterfaceStability.Evolving
44 public final class NamedQueueRecorder {
45
46 private final Disruptor<RingBufferEnvelope> disruptor;
47 private final LogEventHandler logEventHandler;
48
49 private static NamedQueueRecorder namedQueueRecorder;
50 private static boolean isInit = false;
51 private static final Object LOCK = new Object();
52
53
54
55
56 private NamedQueueRecorder(Configuration conf) {
57
58
59
60 final String hostingThreadName = Thread.currentThread().getName();
61
62 int eventCount = conf.getInt("hbase.namedqueue.ringbuffer.size", 1024);
63
64
65 this.disruptor = new Disruptor<>(getEventFactory(), getEventCount(eventCount), Executors.
66 newSingleThreadExecutor(Threads.getNamedThreadFactory(hostingThreadName
67 + ".slowlog.append-pool")),
68 ProducerType.MULTI, new BlockingWaitStrategy());
69 this.disruptor.handleExceptionsWith(new DisruptorExceptionHandler());
70
71
72 this.logEventHandler = new LogEventHandler(conf);
73 this.disruptor.handleEventsWith(new LogEventHandler[]{this.logEventHandler});
74 this.disruptor.start();
75 }
76
77 private EventFactory<RingBufferEnvelope> getEventFactory() {
78 return new EventFactory<RingBufferEnvelope>() {
79 @Override
80 public RingBufferEnvelope newInstance() {
81 return new RingBufferEnvelope();
82 }
83 };
84 }
85
86 public static NamedQueueRecorder getInstance(Configuration conf) {
87 if (namedQueueRecorder != null) {
88 return namedQueueRecorder;
89 }
90 synchronized (LOCK) {
91 if (!isInit) {
92 namedQueueRecorder = new NamedQueueRecorder(conf);
93 isInit = true;
94 }
95 }
96 return namedQueueRecorder;
97 }
98
99
100 private int getEventCount(int eventCount) {
101 Preconditions.checkArgument(eventCount >= 0, "hbase.namedqueue.ringbuffer.size must be > 0");
102 int floor = Integer.highestOneBit(eventCount);
103 if (floor == eventCount) {
104 return floor;
105 }
106
107 if (floor >= 1 << 29) {
108 return 1 << 30;
109 }
110 return floor << 1;
111 }
112
113
114
115
116
117
118
119 public NamedQueueGetResponse getNamedQueueRecords(NamedQueueGetRequest request) {
120 return this.logEventHandler.getNamedQueueRecords(request);
121 }
122
123
124
125
126
127
128
129
130
131 public boolean clearNamedQueue(NamedQueuePayload.NamedQueueEvent namedQueueEvent) {
132 return this.logEventHandler.clearNamedQueue(namedQueueEvent);
133 }
134
135
136
137
138
139
140
141
142
143
144 public void addRecord(NamedQueuePayload namedQueuePayload) {
145 RingBuffer<RingBufferEnvelope> ringBuffer = this.disruptor.getRingBuffer();
146 long seqId = ringBuffer.next();
147 try {
148 ringBuffer.get(seqId).load(namedQueuePayload);
149 } finally {
150 ringBuffer.publish(seqId);
151 }
152 }
153
154
155
156
157
158 public void persistAll(NamedQueuePayload.NamedQueueEvent namedQueueEvent) {
159 if (this.logEventHandler != null) {
160 this.logEventHandler.persistAll(namedQueueEvent);
161 }
162 }
163
164 }