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.client;
19  
20  import static org.junit.Assert.assertEquals;
21  import static org.junit.Assert.assertFalse;
22  import static org.junit.Assert.assertNotNull;
23  import static org.junit.Assert.assertTrue;
24  import static org.junit.Assert.fail;
25  
26  import java.io.IOException;
27  import java.util.ArrayList;
28  import java.util.List;
29  import org.apache.commons.logging.Log;
30  import org.apache.commons.logging.LogFactory;
31  import org.apache.hadoop.hbase.Cell;
32  import org.apache.hadoop.hbase.CellUtil;
33  import org.apache.hadoop.hbase.DoNotRetryIOException;
34  import org.apache.hadoop.hbase.HBaseTestingUtility;
35  import org.apache.hadoop.hbase.HColumnDescriptor;
36  import org.apache.hadoop.hbase.HConstants;
37  import org.apache.hadoop.hbase.HTableDescriptor;
38  import org.apache.hadoop.hbase.TableName;
39  import org.apache.hadoop.hbase.filter.BinaryComparator;
40  import org.apache.hadoop.hbase.ipc.HBaseRpcController;
41  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
42  import org.apache.hadoop.hbase.protobuf.RequestConverter;
43  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
44  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
45  import org.apache.hadoop.hbase.regionserver.HRegion;
46  import org.apache.hadoop.hbase.regionserver.HRegionServer;
47  import org.apache.hadoop.hbase.testclassification.ClientTests;
48  import org.apache.hadoop.hbase.testclassification.MediumTests;
49  import org.apache.hadoop.hbase.util.Bytes;
50  import org.junit.After;
51  import org.junit.AfterClass;
52  import org.junit.Before;
53  import org.junit.BeforeClass;
54  import org.junit.Test;
55  import org.junit.experimental.categories.Category;
56  import org.mockito.Mockito;
57  
58  /**
59   * The purpose of this test is to ensure whether rs deals with the malformed cells correctly.
60   */
61  @Category({ MediumTests.class, ClientTests.class })
62  public class TestMalformedCellFromClient {
63    private static final Log LOG = LogFactory.getLog(TestMalformedCellFromClient.class);
64    private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
65    private static final byte[] FAMILY = Bytes.toBytes("testFamily");
66    private static final int CELL_SIZE = 100;
67    private static final TableName TABLE_NAME = TableName.valueOf("TestMalformedCellFromClient");
68  
69    @BeforeClass
70    public static void setUpBeforeClass() throws Exception {
71      // disable the retry
72      TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
73      TEST_UTIL.startMiniCluster(1);
74    }
75  
76    @Before
77    public void before() throws Exception {
78      HTableDescriptor desc =
79        new HTableDescriptor(TABLE_NAME).addFamily(new HColumnDescriptor(FAMILY))
80          .setValue(HRegion.HBASE_MAX_CELL_SIZE_KEY, String.valueOf(CELL_SIZE));
81      TEST_UTIL.getConnection().getAdmin().createTable(desc);
82    }
83  
84    @After
85    public void tearDown() throws Exception {
86      for (HTableDescriptor htd : TEST_UTIL.getHBaseAdmin().listTables()) {
87        TEST_UTIL.deleteTable(htd.getTableName());
88      }
89    }
90  
91    @AfterClass
92    public static void tearDownAfterClass() throws Exception {
93      TEST_UTIL.shutdownMiniCluster();
94    }
95  
96    /**
97     * This test verifies region exception doesn't corrupt the results of batch. The prescription is
98     * shown below. 1) honor the action result rather than region exception. If the action have both
99     * of true result and region exception, the action is fine as the exception is caused by other
100    * actions which are in the same region. 2) honor the action exception rather than region
101    * exception. If the action have both of action exception and region exception, we deal with the
102    * action exception only. If we also handle the region exception for the same action, it will
103    * introduce the negative count of actions in progress. The AsyncRequestFuture#waitUntilDone will
104    * block forever. If the RetriesExhaustedWithDetailsException contains the whole batch, each
105    * result should be of IOE. Otherwise, the row operation which is not in the exception should have
106    * a true result. The no-cluster test is in TestAsyncProcessWithRegionException.
107    */
108   @Test(timeout = 60000)
109   public void testRegionException() throws InterruptedException, IOException {
110     List<Row> batches = new ArrayList<>();
111     batches.add(new Put(Bytes.toBytes("good")).addColumn(FAMILY, null, new byte[10]));
112     // the rm is used to prompt the region exception.
113     // see RSRpcServices#multi
114     RowMutations rm = new RowMutations(Bytes.toBytes("fail"));
115     rm.add(new Put(rm.getRow()).addColumn(FAMILY, null, new byte[CELL_SIZE]));
116     batches.add(rm);
117     Object[] results = new Object[batches.size()];
118 
119     try (Table table = TEST_UTIL.getConnection().getTable(TABLE_NAME)) {
120       Throwable exceptionByCaught = null;
121       try {
122         table.batch(batches, results);
123         fail("Where is the exception? We put the malformed cells!!!");
124       } catch (RetriesExhaustedWithDetailsException e) {
125         for (Throwable throwable : e.getCauses()) {
126           assertNotNull(throwable);
127         }
128         assertEquals(1, e.getNumExceptions());
129         exceptionByCaught = e.getCause(0);
130       }
131       for (Object obj : results) {
132         assertNotNull(obj);
133       }
134       assertEquals(Result.class, results[0].getClass());
135       assertEquals(exceptionByCaught.getClass(), results[1].getClass());
136       Result result = table.get(new Get(Bytes.toBytes("good")));
137       assertEquals(1, result.size());
138       Cell cell = result.getColumnLatestCell(FAMILY, null);
139       assertTrue(Bytes.equals(CellUtil.cloneValue(cell), new byte[10]));
140     }
141   }
142 
143   /**
144    * The invalid cells is in rm. The rm should fail but the subsequent mutations should succeed.
145    * Currently, we have no client api to submit the request consisting of condition-rm and mutation.
146    * Hence, this test build the request manually.
147    */
148   @Test
149   public void testAtomicOperations() throws Exception {
150     RowMutations rm = new RowMutations(Bytes.toBytes("fail"));
151     rm.add(new Put(rm.getRow()).addColumn(FAMILY, null, new byte[CELL_SIZE]));
152     rm.add(new Put(rm.getRow()).addColumn(FAMILY, null, new byte[10]));
153     Put put = new Put(Bytes.toBytes("good")).addColumn(FAMILY, null, new byte[10]);
154 
155     // build the request
156     HRegion r = TEST_UTIL.getMiniHBaseCluster().getRegions(TABLE_NAME).get(0);
157     ClientProtos.MultiRequest request =
158       ClientProtos.MultiRequest.newBuilder(createRequest(rm, r.getRegionInfo().getRegionName()))
159         .addRegionAction(ClientProtos.RegionAction.newBuilder().setRegion(RequestConverter
160           .buildRegionSpecifier(HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME,
161             r.getRegionInfo().getRegionName())).addAction(ClientProtos.Action.newBuilder()
162           .setMutation(
163             ProtobufUtil.toMutationNoData(ClientProtos.MutationProto.MutationType.PUT, put))))
164         .build();
165 
166     List<Cell> cells = new ArrayList<>();
167     for (Mutation m : rm.getMutations()) {
168       cells.addAll(m.getCellList(FAMILY));
169     }
170     cells.addAll(put.getCellList(FAMILY));
171     assertEquals(3, cells.size());
172     HBaseRpcController controller = Mockito.mock(HBaseRpcController.class);
173     Mockito.when(controller.cellScanner()).thenReturn(CellUtil.createCellScanner(cells));
174     HRegionServer rs = TEST_UTIL.getMiniHBaseCluster().getRegionServer(
175       TEST_UTIL.getMiniHBaseCluster()
176         .getServerHoldingRegion(TABLE_NAME, r.getRegionInfo().getRegionName()));
177     ClientProtos.MultiResponse response = rs.getRSRpcServices().multi(controller, request);
178     assertEquals(2, response.getRegionActionResultCount());
179     assertTrue(response.getRegionActionResultList().get(0).hasException());
180     assertFalse(response.getRegionActionResultList().get(1).hasException());
181     assertEquals(1, response.getRegionActionResultList().get(1).getResultOrExceptionCount());
182     assertTrue(
183       response.getRegionActionResultList().get(1).getResultOrExceptionList().get(0).hasResult());
184     try (Table table = TEST_UTIL.getConnection().getTable(TABLE_NAME)) {
185       Result result = table.get(new Get(Bytes.toBytes("good")));
186       assertEquals(1, result.size());
187       Cell cell = result.getColumnLatestCell(FAMILY, null);
188       assertTrue(Bytes.equals(CellUtil.cloneValue(cell), new byte[10]));
189     }
190   }
191 
192   private static ClientProtos.MultiRequest createRequest(RowMutations rm, byte[] regionName)
193     throws IOException {
194     ClientProtos.RegionAction.Builder builder = RequestConverter
195       .getRegionActionBuilderWithRegion(ClientProtos.RegionAction.newBuilder(), regionName);
196     builder.setAtomic(true);
197     ClientProtos.Action.Builder actionBuilder = ClientProtos.Action.newBuilder();
198     ClientProtos.MutationProto.Builder mutationBuilder = ClientProtos.MutationProto.newBuilder();
199     ClientProtos.Condition condition = RequestConverter
200       .buildCondition(rm.getRow(), FAMILY, FAMILY, new BinaryComparator(new byte[10]),
201         HBaseProtos.CompareType.EQUAL);
202     for (Mutation mutation : rm.getMutations()) {
203       ClientProtos.MutationProto.MutationType mutateType = null;
204       if (mutation instanceof Put) {
205         mutateType = ClientProtos.MutationProto.MutationType.PUT;
206       } else if (mutation instanceof Delete) {
207         mutateType = ClientProtos.MutationProto.MutationType.DELETE;
208       } else {
209         throw new DoNotRetryIOException(
210           "RowMutations supports only put and delete, not " + mutation.getClass().getName());
211       }
212       mutationBuilder.clear();
213       ClientProtos.MutationProto mp =
214         ProtobufUtil.toMutationNoData(mutateType, mutation, mutationBuilder);
215       actionBuilder.clear();
216       actionBuilder.setMutation(mp);
217       builder.addAction(actionBuilder.build());
218     }
219     ClientProtos.MultiRequest request =
220       ClientProtos.MultiRequest.newBuilder().addRegionAction(builder.build())
221         .setCondition(condition).build();
222     return request;
223   }
224 
225   /**
226    * This test depends on how regionserver process the batch ops. 1) group the put/delete until
227    * meeting the increment 2) process the batch of put/delete 3) process the increment see
228    * RSRpcServices#doNonAtomicRegionMutation
229    */
230   @Test
231   public void testNonAtomicOperations() throws InterruptedException, IOException {
232     Increment inc = new Increment(Bytes.toBytes("good")).addColumn(FAMILY, FAMILY, 100);
233     List<Row> batches = new ArrayList<>();
234     // the first and second puts will be group by regionserver
235     batches.add(new Put(Bytes.toBytes("fail")).addColumn(FAMILY, null, new byte[CELL_SIZE]));
236     batches.add(new Put(Bytes.toBytes("fail")).addColumn(FAMILY, null, new byte[CELL_SIZE]));
237     // this Increment should succeed
238     batches.add(inc);
239     // this put should succeed
240     batches.add(new Put(Bytes.toBytes("good")).addColumn(FAMILY, null, new byte[1]));
241     Object[] objs = new Object[batches.size()];
242     try (Table table = TEST_UTIL.getConnection().getTable(TABLE_NAME)) {
243       table.batch(batches, objs);
244       fail("Where is the exception? We put the malformed cells!!!");
245     } catch (RetriesExhaustedWithDetailsException e) {
246       assertEquals(2, e.getNumExceptions());
247       for (int i = 0; i != e.getNumExceptions(); ++i) {
248         assertNotNull(e.getCause(i));
249         assertEquals(DoNotRetryIOException.class, e.getCause(i).getClass());
250         assertEquals("fail", Bytes.toString(e.getRow(i).getRow()));
251       }
252     } finally {
253       assertObjects(objs, batches.size());
254       assertTrue(objs[0] instanceof IOException);
255       assertTrue(objs[1] instanceof IOException);
256       assertEquals(Result.class, objs[2].getClass());
257       assertEquals(Result.class, objs[3].getClass());
258     }
259   }
260 
261   @Test
262   public void testRowMutations() throws InterruptedException, IOException {
263     Put put = new Put(Bytes.toBytes("good")).addColumn(FAMILY, null, new byte[1]);
264     List<Row> batches = new ArrayList<>();
265     RowMutations mutations = new RowMutations(Bytes.toBytes("fail"));
266     // the first and second puts will be group by regionserver
267     mutations.add(new Put(Bytes.toBytes("fail")).addColumn(FAMILY, null, new byte[CELL_SIZE]));
268     mutations.add(new Put(Bytes.toBytes("fail")).addColumn(FAMILY, null, new byte[CELL_SIZE]));
269     batches.add(mutations);
270     // this bm should succeed
271     mutations = new RowMutations(Bytes.toBytes("good"));
272     mutations.add(put);
273     mutations.add(put);
274     batches.add(mutations);
275     Object[] objs = new Object[batches.size()];
276     try (Table table = TEST_UTIL.getConnection().getTable(TABLE_NAME)) {
277       table.batch(batches, objs);
278       fail("Where is the exception? We put the malformed cells!!!");
279     } catch (RetriesExhaustedWithDetailsException e) {
280       assertEquals(1, e.getNumExceptions());
281       for (int i = 0; i != e.getNumExceptions(); ++i) {
282         assertNotNull(e.getCause(i));
283         assertTrue(e.getCause(i) instanceof IOException);
284         assertEquals("fail", Bytes.toString(e.getRow(i).getRow()));
285       }
286     } finally {
287       assertObjects(objs, batches.size());
288       assertTrue(objs[0] instanceof IOException);
289       assertEquals(Result.class, objs[1].getClass());
290     }
291   }
292 
293   private static void assertObjects(Object[] objs, int expectedSize) {
294     int count = 0;
295     for (Object obj : objs) {
296       assertNotNull(obj);
297       ++count;
298     }
299     assertEquals(expectedSize, count);
300   }
301 }