View Javadoc

1   /*
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  package org.apache.hadoop.hbase.regionserver;
19  
20  import java.io.IOException;
21  import java.util.Arrays;
22  import java.util.List;
23  
24  import org.junit.Before;
25  import org.apache.hadoop.conf.Configuration;
26  import org.apache.hadoop.hbase.HBaseTestingUtility;
27  import org.apache.hadoop.hbase.HConstants;
28  import org.apache.hadoop.hbase.TableName;
29  import org.apache.hadoop.hbase.ipc.HBaseRpcController;
30  import org.apache.hadoop.hbase.testclassification.LargeTests;
31  import org.apache.hadoop.hbase.util.Bytes;
32  import org.junit.After;
33  import org.junit.Assert;
34  import org.junit.Test;
35  import org.junit.experimental.categories.Category;
36  import org.junit.runner.RunWith;
37  import org.junit.runners.Parameterized;
38  import org.mockito.Mockito;
39  
40  import com.google.protobuf.RpcController;
41  import com.google.protobuf.ServiceException;
42  
43  import org.apache.hadoop.hbase.protobuf.RequestConverter;
44  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
45  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action;
46  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest;
47  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction;
48  
49  /**
50   * Tests logging of large batch commands via Multi. Tests are fast, but uses a mini-cluster (to test
51   * via "Multi" commands) so classified as MediumTests
52   */
53  @RunWith(Parameterized.class)
54  @Category(LargeTests.class)
55  public class TestMultiLogThreshold {
56  
57    private static RSRpcServices SERVICES;
58  
59    private static HBaseTestingUtility TEST_UTIL;
60    private static Configuration CONF;
61    private static final byte[] TEST_FAM = Bytes.toBytes("fam");
62    private static RSRpcServices.LogDelegate LD;
63    private static HRegionServer RS;
64    private static int THRESHOLD;
65  
66    @Parameterized.Parameter
67    public static boolean rejectLargeBatchOp;
68  
69    @Parameterized.Parameters
70    public static List<Object[]> params() {
71      return Arrays.asList(new Object[] { false }, new Object[] { true });
72    }
73  
74    @Before
75    public void setupTest() throws Exception {
76      final TableName tableName = TableName.valueOf("tableName");
77      TEST_UTIL = HBaseTestingUtility.createLocalHTU();
78      CONF = TEST_UTIL.getConfiguration();
79      THRESHOLD = CONF.getInt(HConstants.BATCH_ROWS_THRESHOLD_NAME,
80        HConstants.BATCH_ROWS_THRESHOLD_DEFAULT);
81      CONF.setBoolean("hbase.rpc.rows.size.threshold.reject", rejectLargeBatchOp);
82      TEST_UTIL.startMiniCluster();
83      TEST_UTIL.createTable(tableName, TEST_FAM);
84      RS = TEST_UTIL.getRSForFirstRegionInTable(tableName);
85    }
86  
87    @After
88    public void tearDown() throws Exception {
89      TEST_UTIL.shutdownMiniCluster();
90    }
91  
92    private enum ActionType {
93      REGION_ACTIONS, ACTIONS;
94    }
95  
96    /**
97     * Sends a multi request with a certain amount of rows, will populate Multi command with either
98     * "rows" number of RegionActions with one Action each or one RegionAction with "rows" number of
99     * Actions
100    */
101   private void sendMultiRequest(int rows, ActionType actionType)
102       throws ServiceException, IOException {
103     RpcController rpcc = Mockito.mock(HBaseRpcController.class);
104     MultiRequest.Builder builder = MultiRequest.newBuilder();
105     int numRAs = 1;
106     int numAs = 1;
107     switch (actionType) {
108     case REGION_ACTIONS:
109       numRAs = rows;
110       break;
111     case ACTIONS:
112       numAs = rows;
113       break;
114     }
115     for (int i = 0; i < numRAs; i++) {
116       RegionAction.Builder rab = RegionAction.newBuilder();
117       rab.setRegion(RequestConverter.buildRegionSpecifier(
118         HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME,
119         new String("someStuff" + i).getBytes()));
120       for (int j = 0; j < numAs; j++) {
121         Action.Builder ab = Action.newBuilder();
122         rab.addAction(ab.build());
123       }
124       builder.addRegionAction(rab.build());
125     }
126     LD = Mockito.mock(RSRpcServices.LogDelegate.class);
127     SERVICES = new RSRpcServices(RS, LD);
128     SERVICES.multi(rpcc, builder.build());
129   }
130 
131   @Test
132   public void testMultiLogThresholdRegionActions() throws ServiceException, IOException {
133     try {
134       sendMultiRequest(THRESHOLD + 1, ActionType.REGION_ACTIONS);
135       Assert.assertFalse(rejectLargeBatchOp);
136     } catch (ServiceException e) {
137       Assert.assertTrue(rejectLargeBatchOp);
138     }
139     Mockito.verify(LD, Mockito.times(1))
140       .logBatchWarning(Mockito.anyString(), Mockito.anyInt(), Mockito.anyInt());
141 
142     sendMultiRequest(THRESHOLD, ActionType.REGION_ACTIONS);
143     Mockito.verify(LD, Mockito.never())
144       .logBatchWarning(Mockito.anyString(), Mockito.anyInt(), Mockito.anyInt());
145 
146     try {
147       sendMultiRequest(THRESHOLD + 1, ActionType.ACTIONS);
148       Assert.assertFalse(rejectLargeBatchOp);
149     } catch (ServiceException e) {
150       Assert.assertTrue(rejectLargeBatchOp);
151     }
152     Mockito.verify(LD, Mockito.times(1))
153       .logBatchWarning(Mockito.anyString(), Mockito.anyInt(), Mockito.anyInt());
154 
155     sendMultiRequest(THRESHOLD, ActionType.ACTIONS);
156     Mockito.verify(LD, Mockito.never())
157       .logBatchWarning(Mockito.anyString(), Mockito.anyInt(), Mockito.anyInt());
158   }
159 
160 }