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.assertTrue;
22  
23  import java.io.IOException;
24  import java.util.List;
25  import java.util.concurrent.ExecutorService;
26  import java.util.concurrent.Executors;
27  
28  import org.apache.hadoop.conf.Configuration;
29  import org.apache.hadoop.hbase.Cell;
30  import org.apache.hadoop.hbase.HConstants;
31  import org.apache.hadoop.hbase.HRegionInfo;
32  import org.apache.hadoop.hbase.KeyValue;
33  import org.apache.hadoop.hbase.KeyValue.Type;
34  import org.apache.hadoop.hbase.TableName;
35  import org.apache.hadoop.hbase.client.ClientSmallScanner.SmallScannerCallableFactory;
36  import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
37  import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
38  import org.apache.hadoop.hbase.testclassification.SmallTests;
39  import org.junit.After;
40  import org.junit.Before;
41  import org.junit.Test;
42  import org.junit.experimental.categories.Category;
43  import org.mockito.Mockito;
44  import org.mockito.invocation.InvocationOnMock;
45  import org.mockito.stubbing.Answer;
46  
47  /**
48   * Test the ClientSmallScanner.
49   */
50  @Category(SmallTests.class)
51  public class TestClientSmallScanner {
52  
53    Scan scan;
54    ExecutorService pool;
55    Configuration conf;
56  
57    HConnection clusterConn;
58    RpcRetryingCallerFactory rpcFactory;
59    RpcControllerFactory controllerFactory;
60    RpcRetryingCaller<Result[]> caller;
61  
62    @Before
63    @SuppressWarnings("unchecked")
64    public void setup() throws IOException {
65      clusterConn = Mockito.mock(HConnection.class);
66      rpcFactory = Mockito.mock(RpcRetryingCallerFactory.class);
67      controllerFactory = Mockito.mock(RpcControllerFactory.class);
68      pool = Executors.newSingleThreadExecutor();
69      scan = new Scan();
70      conf = new Configuration();
71      Mockito.when(clusterConn.getConfiguration()).thenReturn(conf);
72      // Mock out the RpcCaller
73      caller = Mockito.mock(RpcRetryingCaller.class);
74      // Return the mock from the factory
75      Mockito.when(rpcFactory.<Result[]> newCaller()).thenReturn(caller);
76    }
77  
78    @After
79    public void teardown() {
80      if (null != pool) {
81        pool.shutdownNow();
82      }
83    }
84  
85    /**
86     * Create a simple Answer which returns true the first time, and false every time after.
87     */
88    private Answer<Boolean> createTrueThenFalseAnswer() {
89      return new Answer<Boolean>() {
90        boolean first = true;
91  
92        @Override
93        public Boolean answer(InvocationOnMock invocation) {
94          if (first) {
95            first = false;
96            return true;
97          }
98          return false;
99        }
100     };
101   }
102 
103   private SmallScannerCallableFactory getFactory(
104       final RegionServerCallable<Result[]> callableWithReplicas) {
105     return new SmallScannerCallableFactory() {
106       @Override
107       public RegionServerCallable<Result[]> getCallable(final Scan sc, HConnection connection,
108           TableName table, ScanMetrics scanMetrics, byte[] localStartKey, final int cacheNum,
109           final RpcControllerFactory rpcControllerFactory) {
110         return callableWithReplicas;
111       }
112     };
113   }
114 
115   @Test
116   public void testContextPresent() throws Exception {
117     final KeyValue kv1 = new KeyValue("row1".getBytes(), "cf".getBytes(), "cq".getBytes(), 1,
118         Type.Maximum), kv2 = new KeyValue("row2".getBytes(), "cf".getBytes(), "cq".getBytes(), 1,
119         Type.Maximum), kv3 = new KeyValue("row3".getBytes(), "cf".getBytes(), "cq".getBytes(), 1,
120         Type.Maximum);
121 
122     @SuppressWarnings("unchecked")
123     RegionServerCallable<Result[]> callableWithReplicas = Mockito
124         .mock(RegionServerCallable.class);
125 
126     // Mock out the RpcCaller
127     @SuppressWarnings("unchecked")
128     RpcRetryingCaller<Result[]> caller = Mockito.mock(RpcRetryingCaller.class);
129     // Return the mock from the factory
130     Mockito.when(rpcFactory.<Result[]> newCaller()).thenReturn(caller);
131 
132     SmallScannerCallableFactory factory = getFactory(callableWithReplicas);
133 
134     // Intentionally leave a "default" caching size in the Scan. No matter the value, we
135     // should continue based on the server context
136 
137     ClientSmallScanner css = new ClientSmallScanner(conf, scan, TableName.valueOf("table"),
138         clusterConn);
139     try {
140       css.setRpcRetryingCaller(caller);
141       css.setRpcControllerFactory(controllerFactory);
142       css.setScannerCallableFactory(factory);
143 
144       // Return some data the first time, less the second, and none after that
145       Mockito.when(caller.callWithRetries(callableWithReplicas)).thenAnswer(new Answer<Result[]>() {
146         int count = 0;
147 
148         @Override
149         public Result[] answer(InvocationOnMock invocation) {
150           Result[] results;
151           if (0 == count) {
152             results = new Result[] {Result.create(new Cell[] {kv1}),
153                 Result.create(new Cell[] {kv2})};
154           } else if (1 == count) {
155             results = new Result[] {Result.create(new Cell[] {kv3})};
156           } else {
157             results = new Result[0];
158           }
159           count++;
160           return results;
161         }
162       });
163 
164       // Pass back the context always
165       Mockito.when(callableWithReplicas.hasMoreResultsContext()).thenReturn(true);
166       // Only have more results the first time
167       Mockito.when(callableWithReplicas.getServerHasMoreResults()).thenAnswer(
168           createTrueThenFalseAnswer());
169 
170       // A mocked HRegionInfo so ClientSmallScanner#nextScanner(...) works right
171       HRegionInfo regionInfo = Mockito.mock(HRegionInfo.class);
172       Mockito.when(callableWithReplicas.getHRegionInfo()).thenReturn(regionInfo);
173       // Trigger the "no more data" branch for #nextScanner(...)
174       Mockito.when(regionInfo.getEndKey()).thenReturn(HConstants.EMPTY_BYTE_ARRAY);
175 
176       css.loadCache();
177 
178       List<Result> results = css.cache;
179       assertEquals(3, results.size());
180       for (int i = 1; i <= 3; i++) {
181         Result result = results.get(i - 1);
182         byte[] row = result.getRow();
183         assertEquals("row" + i, new String(row, "UTF-8"));
184         assertEquals(1, result.getMap().size());
185       }
186 
187       assertTrue(css.closed);
188     } finally {
189       css.close();
190     }
191   }
192 
193   @Test
194   public void testNoContextFewerRecords() throws Exception {
195     final KeyValue kv1 = new KeyValue("row1".getBytes(), "cf".getBytes(), "cq".getBytes(), 1,
196         Type.Maximum), kv2 = new KeyValue("row2".getBytes(), "cf".getBytes(), "cq".getBytes(), 1,
197         Type.Maximum), kv3 = new KeyValue("row3".getBytes(), "cf".getBytes(), "cq".getBytes(), 1,
198         Type.Maximum);
199 
200     @SuppressWarnings("unchecked")
201     RegionServerCallable<Result[]> callableWithReplicas = Mockito
202         .mock(RegionServerCallable.class);
203 
204     // While the server returns 2 records per batch, we expect more records.
205     scan.setCaching(2);
206     SmallScannerCallableFactory factory = getFactory(callableWithReplicas);
207 
208     ClientSmallScanner css = new ClientSmallScanner(conf, scan, TableName.valueOf("table"),
209         clusterConn);
210     try {
211       css.setRpcRetryingCaller(caller);
212       css.setRpcControllerFactory(controllerFactory);
213       css.setScannerCallableFactory(factory);
214       // Return some data the first time, less the second, and none after that
215       Mockito.when(caller.callWithRetries(callableWithReplicas)).thenAnswer(new Answer<Result[]>() {
216         int count = 0;
217 
218         @Override
219         public Result[] answer(InvocationOnMock invocation) {
220           Result[] results;
221           if (0 == count) {
222             results = new Result[] {Result.create(new Cell[] {kv1}),
223                 Result.create(new Cell[] {kv2})};
224           } else if (1 == count) {
225             // Return fewer records than expected (2)
226             results = new Result[] {Result.create(new Cell[] {kv3})};
227           } else {
228             throw new RuntimeException("Should not fetch a third batch from the server");
229           }
230           count++;
231           return results;
232         }
233       });
234 
235       // Server doesn't return the context
236       Mockito.when(callableWithReplicas.hasMoreResultsContext()).thenReturn(false);
237       // Only have more results the first time
238       Mockito.when(callableWithReplicas.getServerHasMoreResults()).thenThrow(
239           new RuntimeException("Should not be called"));
240 
241       // A mocked HRegionInfo so ClientSmallScanner#nextScanner(...) works right
242       HRegionInfo regionInfo = Mockito.mock(HRegionInfo.class);
243       Mockito.when(callableWithReplicas.getHRegionInfo()).thenReturn(regionInfo);
244       // Trigger the "no more data" branch for #nextScanner(...)
245       Mockito.when(regionInfo.getEndKey()).thenReturn(HConstants.EMPTY_BYTE_ARRAY);
246 
247       css.loadCache();
248 
249       List<Result> results = css.cache;
250       assertEquals(2, results.size());
251       for (int i = 1; i <= 2; i++) {
252         Result result = results.get(i - 1);
253         byte[] row = result.getRow();
254         assertEquals("row" + i, new String(row, "UTF-8"));
255         assertEquals(1, result.getMap().size());
256       }
257 
258       // "consume" the results we verified
259       results.clear();
260 
261       css.loadCache();
262 
263       assertEquals(1, results.size());
264       Result result = results.get(0);
265       assertEquals("row3", new String(result.getRow(), "UTF-8"));
266       assertEquals(1, result.getMap().size());
267       assertTrue(css.closed);
268     } finally {
269       css.close();
270     }
271   }
272 
273   @Test
274   public void testNoContextNoRecords() throws Exception {
275     @SuppressWarnings("unchecked")
276     RegionServerCallable<Result[]> callableWithReplicas = Mockito
277         .mock(RegionServerCallable.class);
278 
279     // While the server return 2 records per RPC, we expect there to be more records.
280     scan.setCaching(2);
281 
282     SmallScannerCallableFactory factory = getFactory(callableWithReplicas);
283 
284     ClientSmallScanner css = new ClientSmallScanner(conf, scan, TableName.valueOf("table"),
285         clusterConn);
286     try {
287       css.setRpcRetryingCaller(caller);
288       css.setRpcControllerFactory(controllerFactory);
289       css.setScannerCallableFactory(factory);
290 
291       // Return some data the first time, less the second, and none after that
292       Mockito.when(caller.callWithRetries(callableWithReplicas)).thenReturn(new Result[0]);
293 
294       // Server doesn't return the context
295       Mockito.when(callableWithReplicas.hasMoreResultsContext()).thenReturn(false);
296       // Only have more results the first time
297       Mockito.when(callableWithReplicas.getServerHasMoreResults()).thenThrow(
298           new RuntimeException("Should not be called"));
299 
300       // A mocked HRegionInfo so ClientSmallScanner#nextScanner(...) works right
301       HRegionInfo regionInfo = Mockito.mock(HRegionInfo.class);
302       Mockito.when(callableWithReplicas.getHRegionInfo()).thenReturn(regionInfo);
303       // Trigger the "no more data" branch for #nextScanner(...)
304       Mockito.when(regionInfo.getEndKey()).thenReturn(HConstants.EMPTY_BYTE_ARRAY);
305 
306       css.loadCache();
307 
308       assertEquals(0, css.cache.size());
309       assertTrue(css.closed);
310     } finally {
311       css.close();
312     }
313   }
314 
315   @Test
316   public void testContextNoRecords() throws Exception {
317     @SuppressWarnings("unchecked")
318     RegionServerCallable<Result[]> callableWithReplicas = Mockito
319         .mock(RegionServerCallable.class);
320 
321     SmallScannerCallableFactory factory = getFactory(callableWithReplicas);
322 
323     ClientSmallScanner css = new ClientSmallScanner(conf, scan, TableName.valueOf("table"),
324         clusterConn);
325     try {
326       css.setRpcRetryingCaller(caller);
327       css.setRpcControllerFactory(controllerFactory);
328       css.setScannerCallableFactory(factory);
329 
330       // Return some data the first time, less the second, and none after that
331       Mockito.when(caller.callWithRetries(callableWithReplicas)).thenReturn(new Result[0]);
332 
333       // Server doesn't return the context
334       Mockito.when(callableWithReplicas.hasMoreResultsContext()).thenReturn(true);
335       // Only have more results the first time
336       Mockito.when(callableWithReplicas.getServerHasMoreResults()).thenReturn(false);
337 
338       // A mocked HRegionInfo so ClientSmallScanner#nextScanner(...) works right
339       HRegionInfo regionInfo = Mockito.mock(HRegionInfo.class);
340       Mockito.when(callableWithReplicas.getHRegionInfo()).thenReturn(regionInfo);
341       // Trigger the "no more data" branch for #nextScanner(...)
342       Mockito.when(regionInfo.getEndKey()).thenReturn(HConstants.EMPTY_BYTE_ARRAY);
343 
344       css.loadCache();
345 
346       assertEquals(0, css.cache.size());
347       assertTrue(css.closed);
348     } finally {
349       css.close();
350     }
351   }
352 }