View Javadoc

1   /*
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
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   * NamedQueue recorder that maintains various named queues.
45   * The service uses LMAX Disruptor to save queue records which are then consumed by
46   * a queue and based on the ring buffer size, the available records are then fetched
47   * from the queue in thread-safe manner.
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     * Initialize disruptor with configurable ringbuffer size
64     */
65    private NamedQueueRecorder(Configuration conf) {
66  
67      // This is the 'writer' -- a single threaded executor. This single thread consumes what is
68      // put on the ringbuffer.
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      // disruptor initialization with BlockingWaitStrategy
76      this.disruptor = new Disruptor<>(getEventFactory(), getEventCount(eventCount), executorService,
77        ProducerType.MULTI, new BlockingWaitStrategy());
78      this.disruptor.handleExceptionsWith(new DisruptorExceptionHandler());
79  
80      // initialize ringbuffer event handler
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   // must be power of 2 for disruptor ringbuffer
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     // max capacity is 1 << 30
116     if (floor >= 1 << 29) {
117       return 1 << 30;
118     }
119     return floor << 1;
120   }
121 
122   /**
123    * Retrieve in memory queue records from ringbuffer
124    *
125    * @param request namedQueue request with event type
126    * @return queue records from ringbuffer after filter (if applied)
127    */
128   public NamedQueueGetResponse getNamedQueueRecords(NamedQueueGetRequest request) {
129     return this.logEventHandler.getNamedQueueRecords(request);
130   }
131 
132   /**
133    * clears queue records from ringbuffer
134    *
135    * @param namedQueueEvent type of queue to clear
136    * @return true if slow log payloads are cleaned up or
137    *   hbase.regionserver.slowlog.buffer.enabled is not set to true, false if failed to
138    *   clean up slow logs
139    */
140   public boolean clearNamedQueue(NamedQueuePayload.NamedQueueEvent namedQueueEvent) {
141     return this.logEventHandler.clearNamedQueue(namedQueueEvent);
142   }
143 
144   /**
145    * Add various NamedQueue records to ringbuffer. Based on the type of the event (e.g slowLog),
146    * consumer of disruptor ringbuffer will have specific logic.
147    * This method is producer of disruptor ringbuffer which is initialized in NamedQueueRecorder
148    * constructor.
149    *
150    * @param namedQueuePayload namedQueue payload sent by client of ring buffer
151    *   service
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    * Add all in memory queue records to system table. The implementors can use system table
167    * or direct HDFS file or ZK as persistence system.
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     // Setting closed flag to true so that we don't add more events to RingBuffer.
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     // With disruptor down, this is safe to let go.
191     if (this.executorService !=  null) {
192       // This will close the executor threads.
193       this.executorService.shutdownNow();
194     }
195   }
196 }