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.TimeoutException;
27 import com.lmax.disruptor.dsl.Disruptor;
28 import com.lmax.disruptor.dsl.ProducerType;
29 import java.io.Closeable;
30 import java.io.IOException;
31 import java.util.concurrent.ExecutorService;
32 import java.util.concurrent.Executors;
33 import java.util.concurrent.TimeUnit;
34 import org.apache.hadoop.conf.Configuration;
35 import org.apache.hadoop.hbase.classification.InterfaceAudience;
36 import org.apache.hadoop.hbase.classification.InterfaceStability;
37 import org.apache.hadoop.hbase.namequeues.request.NamedQueueGetRequest;
38 import org.apache.hadoop.hbase.namequeues.response.NamedQueueGetResponse;
39 import org.apache.hadoop.hbase.util.Threads;
40 import org.slf4j.Logger;
41 import org.slf4j.LoggerFactory;
42
43
44
45
46
47
48
49 @InterfaceAudience.Private
50 @InterfaceStability.Evolving
51 public final class NamedQueueRecorder implements Closeable {
52 private static final Logger LOG = LoggerFactory.getLogger(NamedQueueRecorder.class);
53 private final Disruptor<RingBufferEnvelope> disruptor;
54 private final LogEventHandler logEventHandler;
55 private final ExecutorService executorService;
56
57 private static NamedQueueRecorder namedQueueRecorder;
58 private static boolean isInit = false;
59 private static final Object LOCK = new Object();
60 private volatile boolean closed = false;
61
62
63
64
65 private NamedQueueRecorder(Configuration conf) {
66
67
68
69 final String hostingThreadName = Thread.currentThread().getName();
70
71 int eventCount = conf.getInt("hbase.namedqueue.ringbuffer.size", 1024);
72
73 this.executorService = Executors.newSingleThreadExecutor(Threads.getNamedThreadFactory(
74 hostingThreadName + ".slowlog.append-pool"));
75
76 this.disruptor = new Disruptor<>(getEventFactory(), getEventCount(eventCount), executorService,
77 ProducerType.MULTI, new BlockingWaitStrategy());
78 this.disruptor.handleExceptionsWith(new DisruptorExceptionHandler());
79
80
81 this.logEventHandler = new LogEventHandler(conf);
82 this.disruptor.handleEventsWith(new LogEventHandler[]{this.logEventHandler});
83 this.disruptor.start();
84 }
85
86 private EventFactory<RingBufferEnvelope> getEventFactory() {
87 return new EventFactory<RingBufferEnvelope>() {
88 @Override
89 public RingBufferEnvelope newInstance() {
90 return new RingBufferEnvelope();
91 }
92 };
93 }
94
95 public static NamedQueueRecorder getInstance(Configuration conf) {
96 if (namedQueueRecorder != null) {
97 return namedQueueRecorder;
98 }
99 synchronized (LOCK) {
100 if (!isInit) {
101 namedQueueRecorder = new NamedQueueRecorder(conf);
102 isInit = true;
103 }
104 }
105 return namedQueueRecorder;
106 }
107
108
109 private int getEventCount(int eventCount) {
110 Preconditions.checkArgument(eventCount >= 0, "hbase.namedqueue.ringbuffer.size must be > 0");
111 int floor = Integer.highestOneBit(eventCount);
112 if (floor == eventCount) {
113 return floor;
114 }
115
116 if (floor >= 1 << 29) {
117 return 1 << 30;
118 }
119 return floor << 1;
120 }
121
122
123
124
125
126
127
128 public NamedQueueGetResponse getNamedQueueRecords(NamedQueueGetRequest request) {
129 return this.logEventHandler.getNamedQueueRecords(request);
130 }
131
132
133
134
135
136
137
138
139
140 public boolean clearNamedQueue(NamedQueuePayload.NamedQueueEvent namedQueueEvent) {
141 return this.logEventHandler.clearNamedQueue(namedQueueEvent);
142 }
143
144
145
146
147
148
149
150
151
152
153 public void addRecord(NamedQueuePayload namedQueuePayload) {
154 if (!closed) {
155 RingBuffer<RingBufferEnvelope> ringBuffer = this.disruptor.getRingBuffer();
156 long seqId = ringBuffer.next();
157 try {
158 ringBuffer.get(seqId).load(namedQueuePayload);
159 } finally {
160 ringBuffer.publish(seqId);
161 }
162 }
163 }
164
165
166
167
168
169 public void persistAll(NamedQueuePayload.NamedQueueEvent namedQueueEvent) {
170 if (this.logEventHandler != null) {
171 this.logEventHandler.persistAll(namedQueueEvent);
172 }
173 }
174
175 @Override
176 public void close() throws IOException {
177
178 this.closed = true;
179 LOG.info("Closing NamedQueueRecorder");
180 if (this.disruptor != null) {
181 long timeoutms = 5000;
182 try {
183 this.disruptor.shutdown(timeoutms, TimeUnit.MILLISECONDS);
184 } catch (TimeoutException e) {
185 LOG.warn("Timed out bringing down disruptor after " + timeoutms + " ms; forcing halt", e);
186 this.disruptor.halt();
187 this.disruptor.shutdown();
188 }
189 }
190
191 if (this.executorService != null) {
192
193 this.executorService.shutdownNow();
194 }
195 }
196 }