001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.namequeues;
019
020import static org.junit.jupiter.api.Assertions.assertEquals;
021import static org.junit.jupiter.api.Assertions.assertNotEquals;
022
023import java.io.IOException;
024import java.util.List;
025import java.util.concurrent.CompletableFuture;
026import java.util.concurrent.TimeUnit;
027import org.apache.hadoop.conf.Configuration;
028import org.apache.hadoop.hbase.HBaseTestingUtil;
029import org.apache.hadoop.hbase.HConstants;
030import org.apache.hadoop.hbase.client.Connection;
031import org.apache.hadoop.hbase.client.Result;
032import org.apache.hadoop.hbase.client.ResultScanner;
033import org.apache.hadoop.hbase.client.Scan;
034import org.apache.hadoop.hbase.client.Table;
035import org.apache.hadoop.hbase.namequeues.request.NamedQueueGetRequest;
036import org.apache.hadoop.hbase.namequeues.response.NamedQueueGetResponse;
037import org.apache.hadoop.hbase.regionserver.HRegionServer;
038import org.apache.hadoop.hbase.slowlog.SlowLogTableAccessor;
039import org.apache.hadoop.hbase.testclassification.MasterTests;
040import org.apache.hadoop.hbase.testclassification.MediumTests;
041import org.junit.jupiter.api.AfterAll;
042import org.junit.jupiter.api.BeforeAll;
043import org.junit.jupiter.api.BeforeEach;
044import org.junit.jupiter.api.Tag;
045import org.junit.jupiter.api.Test;
046import org.slf4j.Logger;
047import org.slf4j.LoggerFactory;
048
049import org.apache.hbase.thirdparty.com.google.common.util.concurrent.Uninterruptibles;
050
051import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
052import org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog;
053
054/**
055 * Tests for SlowLog System Table
056 */
057@Tag(MediumTests.TAG)
058@Tag(MasterTests.TAG)
059public class TestSlowLogAccessor {
060
061  private static final Logger LOG = LoggerFactory.getLogger(TestSlowLogAccessor.class);
062
063  private static final HBaseTestingUtil HBASE_TESTING_UTILITY = new HBaseTestingUtil();
064
065  private NamedQueueRecorder namedQueueRecorder;
066
067  @BeforeAll
068  public static void setup() throws Exception {
069    try {
070      HBASE_TESTING_UTILITY.shutdownMiniHBaseCluster();
071    } catch (IOException e) {
072      LOG.debug("No worries.");
073    }
074    Configuration conf = HBASE_TESTING_UTILITY.getConfiguration();
075    conf.setBoolean(HConstants.SLOW_LOG_BUFFER_ENABLED_KEY, true);
076    conf.setBoolean(HConstants.SLOW_LOG_SYS_TABLE_ENABLED_KEY, true);
077    conf.setInt("hbase.slowlog.systable.chore.duration", 900);
078    conf.setInt("hbase.regionserver.slowlog.ringbuffer.size", 50000);
079    conf.setInt("hbase.regionserver.slowlog.systable.queue.size", 5000);
080    HBASE_TESTING_UTILITY.startMiniCluster();
081  }
082
083  @AfterAll
084  public static void teardown() throws Exception {
085    HBASE_TESTING_UTILITY.shutdownMiniHBaseCluster();
086  }
087
088  @BeforeEach
089  public void setUp() throws Exception {
090    HRegionServer hRegionServer = HBASE_TESTING_UTILITY.getMiniHBaseCluster().getRegionServer(0);
091    this.namedQueueRecorder = hRegionServer.getNamedQueueRecorder();
092  }
093
094  private List<TooSlowLog.SlowLogPayload>
095    getSlowLogPayloads(AdminProtos.SlowLogResponseRequest request) {
096    NamedQueueGetRequest namedQueueGetRequest = new NamedQueueGetRequest();
097    namedQueueGetRequest.setNamedQueueEvent(RpcLogDetails.SLOW_LOG_EVENT);
098    namedQueueGetRequest.setSlowLogResponseRequest(request);
099    NamedQueueGetResponse namedQueueGetResponse =
100      namedQueueRecorder.getNamedQueueRecords(namedQueueGetRequest);
101    return namedQueueGetResponse.getSlowLogPayloads();
102  }
103
104  @Test
105  public void testSlowLogRecords() throws Exception {
106
107    AdminProtos.SlowLogResponseRequest request =
108      AdminProtos.SlowLogResponseRequest.newBuilder().setLimit(15).build();
109
110    namedQueueRecorder.clearNamedQueue(NamedQueuePayload.NamedQueueEvent.SLOW_LOG);
111    assertEquals(0, getSlowLogPayloads(request).size());
112
113    int i = 0;
114
115    Connection connection = waitForSlowLogTableCreation();
116    // add 5 records initially
117    for (; i < 5; i++) {
118      RpcLogDetails rpcLogDetails = TestNamedQueueRecorder.getRpcLogDetails("userName_" + (i + 1),
119        "client_" + (i + 1), "class_" + (i + 1));
120      namedQueueRecorder.addRecord(rpcLogDetails);
121    }
122
123    // add 2 more records
124    for (; i < 7; i++) {
125      RpcLogDetails rpcLogDetails = TestNamedQueueRecorder.getRpcLogDetails("userName_" + (i + 1),
126        "client_" + (i + 1), "class_" + (i + 1));
127      namedQueueRecorder.addRecord(rpcLogDetails);
128    }
129
130    // add 3 more records
131    for (; i < 10; i++) {
132      RpcLogDetails rpcLogDetails = TestNamedQueueRecorder.getRpcLogDetails("userName_" + (i + 1),
133        "client_" + (i + 1), "class_" + (i + 1));
134      namedQueueRecorder.addRecord(rpcLogDetails);
135    }
136
137    // add 4 more records
138    for (; i < 14; i++) {
139      RpcLogDetails rpcLogDetails = TestNamedQueueRecorder.getRpcLogDetails("userName_" + (i + 1),
140        "client_" + (i + 1), "class_" + (i + 1));
141      namedQueueRecorder.addRecord(rpcLogDetails);
142    }
143
144    assertNotEquals(-1,
145      HBASE_TESTING_UTILITY.waitFor(3000, () -> getSlowLogPayloads(request).size() == 14));
146
147    assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(3000, () -> getTableCount(connection) == 14));
148  }
149
150  private int getTableCount(Connection connection) {
151    try (Table table = connection.getTable(SlowLogTableAccessor.SLOW_LOG_TABLE_NAME)) {
152      ResultScanner resultScanner = table.getScanner(new Scan().setReadType(Scan.ReadType.STREAM));
153      int count = 0;
154      for (Result result : resultScanner) {
155        ++count;
156      }
157      return count;
158    } catch (Exception e) {
159      return 0;
160    }
161  }
162
163  private Connection waitForSlowLogTableCreation() throws IOException {
164    assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(2000, () -> {
165      try {
166        return HBASE_TESTING_UTILITY.getAdmin()
167          .tableExists(SlowLogTableAccessor.SLOW_LOG_TABLE_NAME);
168      } catch (IOException e) {
169        return false;
170      }
171    }));
172    return HBASE_TESTING_UTILITY.getConnection();
173  }
174
175  @Test
176  public void testHigherSlowLogs() throws Exception {
177    Connection connection = waitForSlowLogTableCreation();
178
179    namedQueueRecorder.clearNamedQueue(NamedQueuePayload.NamedQueueEvent.SLOW_LOG);
180    AdminProtos.SlowLogResponseRequest request =
181      AdminProtos.SlowLogResponseRequest.newBuilder().setLimit(500000).build();
182    assertEquals(0, getSlowLogPayloads(request).size());
183
184    for (int j = 0; j < 100; j++) {
185      CompletableFuture.runAsync(() -> {
186        for (int i = 0; i < 350; i++) {
187          if (i == 300) {
188            Uninterruptibles.sleepUninterruptibly(500, TimeUnit.MILLISECONDS);
189          }
190          RpcLogDetails rpcLogDetails = TestNamedQueueRecorder
191            .getRpcLogDetails("userName_" + (i + 1), "client_" + (i + 1), "class_" + (i + 1));
192          namedQueueRecorder.addRecord(rpcLogDetails);
193        }
194      });
195    }
196
197    assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(7000, () -> {
198      int count = getSlowLogPayloads(request).size();
199      LOG.debug("RingBuffer records count: {}", count);
200      return count > 2000;
201    }));
202
203    assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(7000, () -> {
204      int count = getTableCount(connection);
205      LOG.debug("SlowLog Table records count: {}", count);
206      return count > 2000;
207    }));
208  }
209
210}