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.impl;
21  
22  import com.google.protobuf.Descriptors;
23  import com.google.protobuf.Message;
24  import java.util.ArrayList;
25  import java.util.Collections;
26  import java.util.List;
27  import org.apache.commons.lang.StringUtils;
28  import org.apache.hadoop.conf.Configuration;
29  import org.apache.hadoop.hbase.HConstants;
30  import org.apache.hadoop.hbase.classification.InterfaceAudience;
31  import org.apache.hadoop.hbase.client.SlowLogParams;
32  import org.apache.hadoop.hbase.namequeues.LogHandlerUtils;
33  import org.apache.hadoop.hbase.namequeues.NamedQueuePayload;
34  import org.apache.hadoop.hbase.namequeues.NamedQueueService;
35  import org.apache.hadoop.hbase.namequeues.RpcLogDetails;
36  import org.apache.hadoop.hbase.namequeues.SlowLogPersistentService;
37  import org.apache.hadoop.hbase.namequeues.queue.EvictingQueue;
38  import org.apache.hadoop.hbase.namequeues.request.NamedQueueGetRequest;
39  import org.apache.hadoop.hbase.namequeues.response.NamedQueueGetResponse;
40  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
41  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
42  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
43  import org.apache.hadoop.hbase.protobuf.generated.TooSlowLog;
44  import org.slf4j.Logger;
45  import org.slf4j.LoggerFactory;
46  
47  
48  /**
49   * In-memory Queue service provider for Slow/LargeLog events
50   */
51  @InterfaceAudience.Private
52  public class SlowLogQueueService implements NamedQueueService {
53  
54    private static final Logger LOG = LoggerFactory.getLogger(SlowLogQueueService.class);
55  
56    private static final String SLOW_LOG_RING_BUFFER_SIZE =
57      "hbase.regionserver.slowlog.ringbuffer.size";
58  
59    private final boolean isOnlineLogProviderEnabled;
60    private final boolean isSlowLogTableEnabled;
61    private final SlowLogPersistentService slowLogPersistentService;
62    private final EvictingQueue<TooSlowLog.SlowLogPayload> slowLogQueue;
63  
64    public SlowLogQueueService(Configuration conf) {
65      this.isOnlineLogProviderEnabled = conf.getBoolean(HConstants.SLOW_LOG_BUFFER_ENABLED_KEY,
66        HConstants.DEFAULT_ONLINE_LOG_PROVIDER_ENABLED);
67  
68      if (!isOnlineLogProviderEnabled) {
69        this.isSlowLogTableEnabled = false;
70        this.slowLogPersistentService = null;
71        this.slowLogQueue = null;
72        return;
73      }
74  
75      // Initialize SlowLog Queue
76      int slowLogQueueSize =
77        conf.getInt(SLOW_LOG_RING_BUFFER_SIZE, HConstants.DEFAULT_SLOW_LOG_RING_BUFFER_SIZE);
78  
79      slowLogQueue = EvictingQueue.create(slowLogQueueSize);
80      this.isSlowLogTableEnabled = conf.getBoolean(HConstants.SLOW_LOG_SYS_TABLE_ENABLED_KEY,
81        HConstants.DEFAULT_SLOW_LOG_SYS_TABLE_ENABLED_KEY);
82      if (isSlowLogTableEnabled) {
83        slowLogPersistentService = new SlowLogPersistentService(conf);
84      } else {
85        slowLogPersistentService = null;
86      }
87    }
88  
89    @Override
90    public NamedQueuePayload.NamedQueueEvent getEvent() {
91      return NamedQueuePayload.NamedQueueEvent.SLOW_LOG;
92    }
93  
94    /**
95     * This implementation is specific to slowLog event. This consumes slowLog event from
96     * disruptor and inserts records to EvictingQueue.
97     *
98     * @param namedQueuePayload namedQueue payload from disruptor ring buffer
99     */
100   @Override
101   public void consumeEventFromDisruptor(NamedQueuePayload namedQueuePayload) {
102     if (!isOnlineLogProviderEnabled) {
103       return;
104     }
105     if (!(namedQueuePayload instanceof RpcLogDetails)) {
106       LOG.warn("SlowLogQueueService: NamedQueuePayload is not of type RpcLogDetails.");
107       return;
108     }
109     final RpcLogDetails rpcLogDetails = (RpcLogDetails) namedQueuePayload;
110     final Descriptors.MethodDescriptor methodDescriptor = rpcLogDetails.getMethodDescriptor();
111     final String clientAddress = rpcLogDetails.getClientAddress();
112     final long responseSize = rpcLogDetails.getResponseSize();
113     final String className = rpcLogDetails.getClassName();
114     final TooSlowLog.SlowLogPayload.Type type = getLogType(rpcLogDetails);
115     if (type == null) {
116       return;
117     }
118     Message param = rpcLogDetails.getParam();
119     long receiveTime = rpcLogDetails.getReceiveTime();
120     long startTime = rpcLogDetails.getStartTime();
121     long endTime = System.currentTimeMillis();
122     int processingTime = (int) (endTime - startTime);
123     int qTime = (int) (startTime - receiveTime);
124     final SlowLogParams slowLogParams = ProtobufUtil.getSlowLogParams(param);
125     int numGets = 0;
126     int numMutations = 0;
127     int numServiceCalls = 0;
128     if (param instanceof ClientProtos.MultiRequest) {
129       ClientProtos.MultiRequest multi = (ClientProtos.MultiRequest) param;
130       for (ClientProtos.RegionAction regionAction : multi.getRegionActionList()) {
131         for (ClientProtos.Action action : regionAction.getActionList()) {
132           if (action.hasMutation()) {
133             numMutations++;
134           }
135           if (action.hasGet()) {
136             numGets++;
137           }
138           if (action.hasServiceCall()) {
139             numServiceCalls++;
140           }
141         }
142       }
143     }
144     final String userName =
145       rpcLogDetails.getUserName() != null ? rpcLogDetails.getUserName() : StringUtils.EMPTY;
146     final String methodDescriptorName =
147       methodDescriptor != null ? methodDescriptor.getName() : StringUtils.EMPTY;
148     TooSlowLog.SlowLogPayload slowLogPayload = TooSlowLog.SlowLogPayload.newBuilder()
149       .setCallDetails(methodDescriptorName + "(" + param.getClass().getName() + ")")
150       .setClientAddress(clientAddress)
151       .setMethodName(methodDescriptorName)
152       .setMultiGets(numGets)
153       .setMultiMutations(numMutations)
154       .setMultiServiceCalls(numServiceCalls)
155       .setParam(slowLogParams != null ? slowLogParams.getParams() : StringUtils.EMPTY)
156       .setProcessingTime(processingTime)
157       .setQueueTime(qTime)
158       .setRegionName(slowLogParams != null ? slowLogParams.getRegionName() : StringUtils.EMPTY)
159       .setResponseSize(responseSize)
160       .setServerClass(className)
161       .setStartTime(startTime)
162       .setType(type)
163       .setUserName(userName)
164       .build();
165     slowLogQueue.add(slowLogPayload);
166     if (isSlowLogTableEnabled) {
167       if (!slowLogPayload.getRegionName().startsWith("hbase:slowlog")) {
168         slowLogPersistentService.addToQueueForSysTable(slowLogPayload);
169       }
170     }
171   }
172 
173   @Override
174   public boolean clearNamedQueue() {
175     if (!isOnlineLogProviderEnabled) {
176       return false;
177     }
178     LOG.debug("Received request to clean up online slowlog buffer.");
179     slowLogQueue.clear();
180     return true;
181   }
182 
183   @Override
184   public NamedQueueGetResponse getNamedQueueRecords(NamedQueueGetRequest request) {
185     if (!isOnlineLogProviderEnabled) {
186       return null;
187     }
188     final AdminProtos.SlowLogResponseRequest slowLogResponseRequest =
189       request.getSlowLogResponseRequest();
190     final List<TooSlowLog.SlowLogPayload> slowLogPayloads;
191     if (AdminProtos.SlowLogResponseRequest.LogType.LARGE_LOG
192         .equals(slowLogResponseRequest.getLogType())) {
193       slowLogPayloads = getLargeLogPayloads(slowLogResponseRequest);
194     } else {
195       slowLogPayloads = getSlowLogPayloads(slowLogResponseRequest);
196     }
197     NamedQueueGetResponse response = new NamedQueueGetResponse();
198     response.setNamedQueueEvent(RpcLogDetails.SLOW_LOG_EVENT);
199     response.setSlowLogPayloads(slowLogPayloads);
200     return response;
201   }
202 
203   private TooSlowLog.SlowLogPayload.Type getLogType(RpcLogDetails rpcCallDetails) {
204     final boolean isSlowLog = rpcCallDetails.isSlowLog();
205     final boolean isLargeLog = rpcCallDetails.isLargeLog();
206     final TooSlowLog.SlowLogPayload.Type type;
207     if (!isSlowLog && !isLargeLog) {
208       LOG.error("slowLog and largeLog both are false. Ignoring the event. rpcCallDetails: {}",
209         rpcCallDetails);
210       return null;
211     }
212     if (isSlowLog && isLargeLog) {
213       type = TooSlowLog.SlowLogPayload.Type.ALL;
214     } else if (isSlowLog) {
215       type = TooSlowLog.SlowLogPayload.Type.SLOW_LOG;
216     } else {
217       type = TooSlowLog.SlowLogPayload.Type.LARGE_LOG;
218     }
219     return type;
220   }
221 
222   /**
223    * Add all slowLog events to system table. This is only for slowLog event's persistence on
224    * system table.
225    */
226   @Override
227   public void persistAll() {
228     if (!isOnlineLogProviderEnabled) {
229       return;
230     }
231     if (slowLogPersistentService != null) {
232       slowLogPersistentService.addAllLogsToSysTable();
233     }
234   }
235 
236   private List<TooSlowLog.SlowLogPayload> getSlowLogPayloads(
237       final AdminProtos.SlowLogResponseRequest request) {
238     TooSlowLog.SlowLogPayload[] slowLogPayloads =
239       slowLogQueue.toArray(new TooSlowLog.SlowLogPayload[0]);
240     List<TooSlowLog.SlowLogPayload> slowLogPayloadList = new ArrayList<>();
241     for (TooSlowLog.SlowLogPayload slowLogPayload : slowLogPayloads) {
242       if (slowLogPayload.getType() == TooSlowLog.SlowLogPayload.Type.ALL
243         || slowLogPayload.getType() == TooSlowLog.SlowLogPayload.Type.SLOW_LOG) {
244         slowLogPayloadList.add(slowLogPayload);
245       }
246     }
247     // latest slow logs first, operator is interested in latest records from in-memory buffer
248     Collections.reverse(slowLogPayloadList);
249     return LogHandlerUtils.getFilteredLogs(request, slowLogPayloadList);
250   }
251 
252   private List<TooSlowLog.SlowLogPayload> getLargeLogPayloads(
253       final AdminProtos.SlowLogResponseRequest request) {
254     TooSlowLog.SlowLogPayload[] slowLogPayloads =
255       slowLogQueue.toArray(new TooSlowLog.SlowLogPayload[0]);
256     List<TooSlowLog.SlowLogPayload> slowLogPayloadList = new ArrayList<>();
257     for (TooSlowLog.SlowLogPayload slowLogPayload : slowLogPayloads) {
258       if (slowLogPayload.getType() == TooSlowLog.SlowLogPayload.Type.ALL
259         || slowLogPayload.getType() == TooSlowLog.SlowLogPayload.Type.LARGE_LOG) {
260         slowLogPayloadList.add(slowLogPayload);
261       }
262     }
263     // latest large logs first, operator is interested in latest records from in-memory buffer
264     Collections.reverse(slowLogPayloadList);
265     return LogHandlerUtils.getFilteredLogs(request, slowLogPayloadList);
266   }
267 
268 }