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.client;
019
020import static org.apache.hadoop.hbase.HConstants.NORMAL_QOS;
021import static org.apache.hadoop.hbase.HConstants.SYSTEMTABLE_QOS;
022import static org.apache.hadoop.hbase.NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR;
023import static org.junit.Assert.assertNotNull;
024import static org.mockito.ArgumentMatchers.any;
025import static org.mockito.ArgumentMatchers.anyInt;
026import static org.mockito.ArgumentMatchers.anyLong;
027import static org.mockito.ArgumentMatchers.argThat;
028import static org.mockito.Mockito.atLeast;
029import static org.mockito.Mockito.doAnswer;
030import static org.mockito.Mockito.mock;
031import static org.mockito.Mockito.times;
032import static org.mockito.Mockito.verify;
033
034import java.io.IOException;
035import java.util.Arrays;
036import java.util.concurrent.CompletableFuture;
037import java.util.concurrent.atomic.AtomicInteger;
038import org.apache.hadoop.conf.Configuration;
039import org.apache.hadoop.hbase.Cell;
040import org.apache.hadoop.hbase.Cell.Type;
041import org.apache.hadoop.hbase.CellBuilderFactory;
042import org.apache.hadoop.hbase.CellBuilderType;
043import org.apache.hadoop.hbase.HBaseClassTestRule;
044import org.apache.hadoop.hbase.HBaseConfiguration;
045import org.apache.hadoop.hbase.HRegionLocation;
046import org.apache.hadoop.hbase.ServerName;
047import org.apache.hadoop.hbase.TableName;
048import org.apache.hadoop.hbase.ipc.HBaseRpcController;
049import org.apache.hadoop.hbase.security.UserProvider;
050import org.apache.hadoop.hbase.testclassification.ClientTests;
051import org.apache.hadoop.hbase.testclassification.MediumTests;
052import org.apache.hadoop.hbase.util.Bytes;
053import org.junit.Before;
054import org.junit.ClassRule;
055import org.junit.Rule;
056import org.junit.Test;
057import org.junit.experimental.categories.Category;
058import org.junit.rules.TestName;
059import org.mockito.ArgumentMatcher;
060import org.mockito.invocation.InvocationOnMock;
061import org.mockito.stubbing.Answer;
062
063import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
064
065import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
066import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
067import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
068import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest;
069import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetResponse;
070import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest;
071import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse;
072import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto;
073import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue;
074import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue;
075import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult;
076import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException;
077import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest;
078import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse;
079
080/**
081 * Confirm that we will set the priority in {@link HBaseRpcController} for several table operations.
082 */
083@Category({ ClientTests.class, MediumTests.class })
084public class TestAsyncTableRpcPriority {
085
086  @ClassRule
087  public static final HBaseClassTestRule CLASS_RULE =
088    HBaseClassTestRule.forClass(TestAsyncTableRpcPriority.class);
089
090  private static Configuration CONF = HBaseConfiguration.create();
091
092  private ClientService.Interface stub;
093
094  private AsyncConnection conn;
095
096  @Rule
097  public TestName name = new TestName();
098
099  @Before
100  public void setUp() throws IOException {
101    stub = mock(ClientService.Interface.class);
102    AtomicInteger scanNextCalled = new AtomicInteger(0);
103    doAnswer(new Answer<Void>() {
104
105      @Override
106      public Void answer(InvocationOnMock invocation) throws Throwable {
107        ScanRequest req = invocation.getArgument(1);
108        RpcCallback<ScanResponse> done = invocation.getArgument(2);
109        if (!req.hasScannerId()) {
110          done.run(ScanResponse.newBuilder().setScannerId(1).setTtl(800)
111            .setMoreResultsInRegion(true).setMoreResults(true).build());
112        } else {
113          if (req.hasCloseScanner() && req.getCloseScanner()) {
114            done.run(ScanResponse.getDefaultInstance());
115          } else {
116            Cell cell = CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setType(Type.Put)
117              .setRow(Bytes.toBytes(scanNextCalled.incrementAndGet()))
118              .setFamily(Bytes.toBytes("cf")).setQualifier(Bytes.toBytes("cq"))
119              .setValue(Bytes.toBytes("v")).build();
120            Result result = Result.create(Arrays.asList(cell));
121            done.run(
122              ScanResponse.newBuilder().setScannerId(1).setTtl(800).setMoreResultsInRegion(true)
123                .setMoreResults(true).addResults(ProtobufUtil.toResult(result)).build());
124          }
125        }
126        return null;
127      }
128    }).when(stub).scan(any(HBaseRpcController.class), any(ScanRequest.class), any());
129    doAnswer(new Answer<Void>() {
130
131      @Override
132      public Void answer(InvocationOnMock invocation) throws Throwable {
133        ClientProtos.MultiResponse resp =
134          ClientProtos.MultiResponse.newBuilder()
135            .addRegionActionResult(RegionActionResult.newBuilder().addResultOrException(
136              ResultOrException.newBuilder().setResult(ProtobufUtil.toResult(new Result()))))
137            .build();
138        RpcCallback<ClientProtos.MultiResponse> done = invocation.getArgument(2);
139        done.run(resp);
140        return null;
141      }
142    }).when(stub).multi(any(HBaseRpcController.class), any(ClientProtos.MultiRequest.class), any());
143    doAnswer(new Answer<Void>() {
144
145      @Override
146      public Void answer(InvocationOnMock invocation) throws Throwable {
147        MutationProto req = ((MutateRequest) invocation.getArgument(1)).getMutation();
148        MutateResponse resp;
149        switch (req.getMutateType()) {
150          case INCREMENT:
151            ColumnValue value = req.getColumnValue(0);
152            QualifierValue qvalue = value.getQualifierValue(0);
153            Cell cell = CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setType(Type.Put)
154              .setRow(req.getRow().toByteArray()).setFamily(value.getFamily().toByteArray())
155              .setQualifier(qvalue.getQualifier().toByteArray())
156              .setValue(qvalue.getValue().toByteArray()).build();
157            resp = MutateResponse.newBuilder()
158              .setResult(ProtobufUtil.toResult(Result.create(Arrays.asList(cell)))).build();
159            break;
160          default:
161            resp = MutateResponse.getDefaultInstance();
162            break;
163        }
164        RpcCallback<MutateResponse> done = invocation.getArgument(2);
165        done.run(resp);
166        return null;
167      }
168    }).when(stub).mutate(any(HBaseRpcController.class), any(MutateRequest.class), any());
169    doAnswer(new Answer<Void>() {
170
171      @Override
172      public Void answer(InvocationOnMock invocation) throws Throwable {
173        RpcCallback<GetResponse> done = invocation.getArgument(2);
174        done.run(GetResponse.getDefaultInstance());
175        return null;
176      }
177    }).when(stub).get(any(HBaseRpcController.class), any(GetRequest.class), any());
178    conn = new AsyncConnectionImpl(CONF, new DoNothingAsyncRegistry(CONF), "test",
179      UserProvider.instantiate(CONF).getCurrent()) {
180
181      @Override
182      AsyncRegionLocator getLocator() {
183        AsyncRegionLocator locator = mock(AsyncRegionLocator.class);
184        Answer<CompletableFuture<HRegionLocation>> answer =
185          new Answer<CompletableFuture<HRegionLocation>>() {
186
187            @Override
188            public CompletableFuture<HRegionLocation> answer(InvocationOnMock invocation)
189                throws Throwable {
190              TableName tableName = invocation.getArgument(0);
191              RegionInfo info = RegionInfoBuilder.newBuilder(tableName).build();
192              ServerName serverName = ServerName.valueOf("rs", 16010, 12345);
193              HRegionLocation loc = new HRegionLocation(info, serverName);
194              return CompletableFuture.completedFuture(loc);
195            }
196          };
197        doAnswer(answer).when(locator).getRegionLocation(any(TableName.class), any(byte[].class),
198          any(RegionLocateType.class), anyLong());
199        doAnswer(answer).when(locator).getRegionLocation(any(TableName.class), any(byte[].class),
200          anyInt(), any(RegionLocateType.class), anyLong());
201        return locator;
202      }
203
204      @Override
205      ClientService.Interface getRegionServerStub(ServerName serverName) throws IOException {
206        return stub;
207      }
208    };
209  }
210
211  private HBaseRpcController assertPriority(int priority) {
212    return argThat(new ArgumentMatcher<HBaseRpcController>() {
213
214      @Override
215      public boolean matches(HBaseRpcController controller) {
216        return controller.getPriority() == priority;
217      }
218    });
219  }
220
221  @Test
222  public void testGet() {
223    conn.getTable(TableName.valueOf(name.getMethodName()))
224      .get(new Get(Bytes.toBytes(0)).setPriority(11)).join();
225    verify(stub, times(1)).get(assertPriority(11), any(GetRequest.class), any());
226  }
227
228  @Test
229  public void testGetNormalTable() {
230    conn.getTable(TableName.valueOf(name.getMethodName())).get(new Get(Bytes.toBytes(0))).join();
231    verify(stub, times(1)).get(assertPriority(NORMAL_QOS), any(GetRequest.class), any());
232  }
233
234  @Test
235  public void testGetSystemTable() {
236    conn.getTable(TableName.valueOf(SYSTEM_NAMESPACE_NAME_STR, name.getMethodName()))
237      .get(new Get(Bytes.toBytes(0))).join();
238    verify(stub, times(1)).get(assertPriority(SYSTEMTABLE_QOS), any(GetRequest.class), any());
239  }
240
241  @Test
242  public void testGetMetaTable() {
243    conn.getTable(TableName.META_TABLE_NAME).get(new Get(Bytes.toBytes(0))).join();
244    verify(stub, times(1)).get(assertPriority(SYSTEMTABLE_QOS), any(GetRequest.class), any());
245  }
246
247  @Test
248  public void testPut() {
249    conn
250      .getTable(TableName.valueOf(name.getMethodName())).put(new Put(Bytes.toBytes(0))
251        .setPriority(12).addColumn(Bytes.toBytes("cf"), Bytes.toBytes("cq"), Bytes.toBytes("v")))
252      .join();
253    verify(stub, times(1)).mutate(assertPriority(12), any(MutateRequest.class), any());
254  }
255
256  @Test
257  public void testPutNormalTable() {
258    conn.getTable(TableName.valueOf(name.getMethodName())).put(new Put(Bytes.toBytes(0))
259      .addColumn(Bytes.toBytes("cf"), Bytes.toBytes("cq"), Bytes.toBytes("v"))).join();
260    verify(stub, times(1)).mutate(assertPriority(NORMAL_QOS), any(MutateRequest.class), any());
261  }
262
263  @Test
264  public void testPutSystemTable() {
265    conn.getTable(TableName.valueOf(SYSTEM_NAMESPACE_NAME_STR, name.getMethodName()))
266      .put(new Put(Bytes.toBytes(0)).addColumn(Bytes.toBytes("cf"), Bytes.toBytes("cq"),
267        Bytes.toBytes("v")))
268      .join();
269    verify(stub, times(1)).mutate(assertPriority(SYSTEMTABLE_QOS), any(MutateRequest.class), any());
270  }
271
272  @Test
273  public void testPutMetaTable() {
274    conn.getTable(TableName.META_TABLE_NAME).put(new Put(Bytes.toBytes(0))
275      .addColumn(Bytes.toBytes("cf"), Bytes.toBytes("cq"), Bytes.toBytes("v"))).join();
276    verify(stub, times(1)).mutate(assertPriority(SYSTEMTABLE_QOS), any(MutateRequest.class), any());
277  }
278
279  @Test
280  public void testDelete() {
281    conn.getTable(TableName.valueOf(name.getMethodName()))
282      .delete(new Delete(Bytes.toBytes(0)).setPriority(13)).join();
283    verify(stub, times(1)).mutate(assertPriority(13), any(MutateRequest.class), any());
284  }
285
286  @Test
287  public void testDeleteNormalTable() {
288    conn.getTable(TableName.valueOf(name.getMethodName())).delete(new Delete(Bytes.toBytes(0)))
289      .join();
290    verify(stub, times(1)).mutate(assertPriority(NORMAL_QOS), any(MutateRequest.class), any());
291  }
292
293  @Test
294  public void testDeleteSystemTable() {
295    conn.getTable(TableName.valueOf(SYSTEM_NAMESPACE_NAME_STR, name.getMethodName()))
296      .delete(new Delete(Bytes.toBytes(0))).join();
297    verify(stub, times(1)).mutate(assertPriority(SYSTEMTABLE_QOS), any(MutateRequest.class), any());
298  }
299
300  @Test
301  public void testDeleteMetaTable() {
302    conn.getTable(TableName.META_TABLE_NAME).delete(new Delete(Bytes.toBytes(0))).join();
303    verify(stub, times(1)).mutate(assertPriority(SYSTEMTABLE_QOS), any(MutateRequest.class), any());
304  }
305
306  @Test
307  public void testAppend() {
308    conn
309      .getTable(TableName.valueOf(name.getMethodName())).append(new Append(Bytes.toBytes(0))
310        .setPriority(14).addColumn(Bytes.toBytes("cf"), Bytes.toBytes("cq"), Bytes.toBytes("v")))
311      .join();
312    verify(stub, times(1)).mutate(assertPriority(14), any(MutateRequest.class), any());
313  }
314
315  @Test
316  public void testAppendNormalTable() {
317    conn.getTable(TableName.valueOf(name.getMethodName())).append(new Append(Bytes.toBytes(0))
318      .addColumn(Bytes.toBytes("cf"), Bytes.toBytes("cq"), Bytes.toBytes("v"))).join();
319    verify(stub, times(1)).mutate(assertPriority(NORMAL_QOS), any(MutateRequest.class), any());
320  }
321
322  @Test
323  public void testAppendSystemTable() {
324    conn.getTable(TableName.valueOf(SYSTEM_NAMESPACE_NAME_STR, name.getMethodName()))
325      .append(new Append(Bytes.toBytes(0)).addColumn(Bytes.toBytes("cf"), Bytes.toBytes("cq"),
326        Bytes.toBytes("v")))
327      .join();
328    verify(stub, times(1)).mutate(assertPriority(SYSTEMTABLE_QOS), any(MutateRequest.class), any());
329  }
330
331  @Test
332  public void testAppendMetaTable() {
333    conn.getTable(TableName.META_TABLE_NAME).append(new Append(Bytes.toBytes(0))
334      .addColumn(Bytes.toBytes("cf"), Bytes.toBytes("cq"), Bytes.toBytes("v"))).join();
335    verify(stub, times(1)).mutate(assertPriority(SYSTEMTABLE_QOS), any(MutateRequest.class), any());
336  }
337
338  @Test
339  public void testIncrement() {
340    conn.getTable(TableName.valueOf(name.getMethodName())).increment(new Increment(Bytes.toBytes(0))
341      .addColumn(Bytes.toBytes("cf"), Bytes.toBytes("cq"), 1).setPriority(15)).join();
342    verify(stub, times(1)).mutate(assertPriority(15), any(MutateRequest.class), any());
343  }
344
345  @Test
346  public void testIncrementNormalTable() {
347    conn.getTable(TableName.valueOf(name.getMethodName()))
348      .incrementColumnValue(Bytes.toBytes(0), Bytes.toBytes("cf"), Bytes.toBytes("cq"), 1).join();
349    verify(stub, times(1)).mutate(assertPriority(NORMAL_QOS), any(MutateRequest.class), any());
350  }
351
352  @Test
353  public void testIncrementSystemTable() {
354    conn.getTable(TableName.valueOf(SYSTEM_NAMESPACE_NAME_STR, name.getMethodName()))
355      .incrementColumnValue(Bytes.toBytes(0), Bytes.toBytes("cf"), Bytes.toBytes("cq"), 1).join();
356    verify(stub, times(1)).mutate(assertPriority(SYSTEMTABLE_QOS), any(MutateRequest.class), any());
357  }
358
359  @Test
360  public void testIncrementMetaTable() {
361    conn.getTable(TableName.META_TABLE_NAME)
362      .incrementColumnValue(Bytes.toBytes(0), Bytes.toBytes("cf"), Bytes.toBytes("cq"), 1).join();
363    verify(stub, times(1)).mutate(assertPriority(SYSTEMTABLE_QOS), any(MutateRequest.class), any());
364  }
365
366  @Test
367  public void testCheckAndPut() {
368    conn.getTable(TableName.valueOf(name.getMethodName()))
369      .checkAndMutate(Bytes.toBytes(0), Bytes.toBytes("cf")).qualifier(Bytes.toBytes("cq"))
370      .ifNotExists()
371      .thenPut(new Put(Bytes.toBytes(0))
372        .addColumn(Bytes.toBytes("cf"), Bytes.toBytes("cq"), Bytes.toBytes("v")).setPriority(16))
373      .join();
374    verify(stub, times(1)).mutate(assertPriority(16), any(MutateRequest.class), any());
375  }
376
377  @Test
378  public void testCheckAndPutNormalTable() {
379    conn.getTable(TableName.valueOf(name.getMethodName()))
380      .checkAndMutate(Bytes.toBytes(0), Bytes.toBytes("cf")).qualifier(Bytes.toBytes("cq"))
381      .ifNotExists().thenPut(new Put(Bytes.toBytes(0)).addColumn(Bytes.toBytes("cf"),
382        Bytes.toBytes("cq"), Bytes.toBytes("v")))
383      .join();
384    verify(stub, times(1)).mutate(assertPriority(NORMAL_QOS), any(MutateRequest.class), any());
385  }
386
387  @Test
388  public void testCheckAndPutSystemTable() {
389    conn.getTable(TableName.valueOf(SYSTEM_NAMESPACE_NAME_STR, name.getMethodName()))
390      .checkAndMutate(Bytes.toBytes(0), Bytes.toBytes("cf")).qualifier(Bytes.toBytes("cq"))
391      .ifNotExists().thenPut(new Put(Bytes.toBytes(0)).addColumn(Bytes.toBytes("cf"),
392        Bytes.toBytes("cq"), Bytes.toBytes("v")))
393      .join();
394    verify(stub, times(1)).mutate(assertPriority(SYSTEMTABLE_QOS), any(MutateRequest.class), any());
395  }
396
397  @Test
398  public void testCheckAndPutMetaTable() {
399    conn.getTable(TableName.META_TABLE_NAME).checkAndMutate(Bytes.toBytes(0), Bytes.toBytes("cf"))
400      .qualifier(Bytes.toBytes("cq")).ifNotExists().thenPut(new Put(Bytes.toBytes(0))
401        .addColumn(Bytes.toBytes("cf"), Bytes.toBytes("cq"), Bytes.toBytes("v")))
402      .join();
403    verify(stub, times(1)).mutate(assertPriority(SYSTEMTABLE_QOS), any(MutateRequest.class), any());
404  }
405
406  @Test
407  public void testCheckAndDelete() {
408    conn.getTable(TableName.valueOf(name.getMethodName()))
409      .checkAndMutate(Bytes.toBytes(0), Bytes.toBytes("cf")).qualifier(Bytes.toBytes("cq"))
410      .ifEquals(Bytes.toBytes("v")).thenDelete(new Delete(Bytes.toBytes(0)).setPriority(17)).join();
411    verify(stub, times(1)).mutate(assertPriority(17), any(MutateRequest.class), any());
412  }
413
414  @Test
415  public void testCheckAndDeleteNormalTable() {
416    conn.getTable(TableName.valueOf(name.getMethodName()))
417      .checkAndMutate(Bytes.toBytes(0), Bytes.toBytes("cf")).qualifier(Bytes.toBytes("cq"))
418      .ifEquals(Bytes.toBytes("v")).thenDelete(new Delete(Bytes.toBytes(0))).join();
419    verify(stub, times(1)).mutate(assertPriority(NORMAL_QOS), any(MutateRequest.class), any());
420  }
421
422  @Test
423  public void testCheckAndDeleteSystemTable() {
424    conn.getTable(TableName.valueOf(SYSTEM_NAMESPACE_NAME_STR, name.getMethodName()))
425      .checkAndMutate(Bytes.toBytes(0), Bytes.toBytes("cf")).qualifier(Bytes.toBytes("cq"))
426      .ifEquals(Bytes.toBytes("v")).thenDelete(new Delete(Bytes.toBytes(0))).join();
427    verify(stub, times(1)).mutate(assertPriority(SYSTEMTABLE_QOS), any(MutateRequest.class), any());
428  }
429
430  @Test
431  public void testCheckAndDeleteMetaTable() {
432    conn.getTable(TableName.META_TABLE_NAME).checkAndMutate(Bytes.toBytes(0), Bytes.toBytes("cf"))
433      .qualifier(Bytes.toBytes("cq")).ifNotExists().thenPut(new Put(Bytes.toBytes(0))
434        .addColumn(Bytes.toBytes("cf"), Bytes.toBytes("cq"), Bytes.toBytes("v")))
435      .join();
436    verify(stub, times(1)).mutate(assertPriority(SYSTEMTABLE_QOS), any(MutateRequest.class), any());
437  }
438
439  @Test
440  public void testCheckAndMutate() throws IOException {
441    conn.getTable(TableName.valueOf(name.getMethodName()))
442      .checkAndMutate(Bytes.toBytes(0), Bytes.toBytes("cf")).qualifier(Bytes.toBytes("cq"))
443      .ifEquals(Bytes.toBytes("v")).thenMutate(new RowMutations(Bytes.toBytes(0))
444        .add((Mutation) new Delete(Bytes.toBytes(0)).setPriority(18)))
445      .join();
446    verify(stub, times(1)).multi(assertPriority(18), any(ClientProtos.MultiRequest.class), any());
447  }
448
449  @Test
450  public void testCheckAndMutateNormalTable() throws IOException {
451    conn.getTable(TableName.valueOf(name.getMethodName()))
452      .checkAndMutate(Bytes.toBytes(0), Bytes.toBytes("cf")).qualifier(Bytes.toBytes("cq"))
453      .ifEquals(Bytes.toBytes("v"))
454      .thenMutate(new RowMutations(Bytes.toBytes(0)).add((Mutation) new Delete(Bytes.toBytes(0))))
455      .join();
456    verify(stub, times(1)).multi(assertPriority(NORMAL_QOS), any(ClientProtos.MultiRequest.class),
457      any());
458  }
459
460  @Test
461  public void testCheckAndMutateSystemTable() throws IOException {
462    conn.getTable(TableName.valueOf(SYSTEM_NAMESPACE_NAME_STR, name.getMethodName()))
463      .checkAndMutate(Bytes.toBytes(0), Bytes.toBytes("cf")).qualifier(Bytes.toBytes("cq"))
464      .ifEquals(Bytes.toBytes("v"))
465      .thenMutate(new RowMutations(Bytes.toBytes(0)).add((Mutation) new Delete(Bytes.toBytes(0))))
466      .join();
467    verify(stub, times(1)).multi(assertPriority(SYSTEMTABLE_QOS),
468      any(ClientProtos.MultiRequest.class), any());
469  }
470
471  @Test
472  public void testCheckAndMutateMetaTable() throws IOException {
473    conn.getTable(TableName.META_TABLE_NAME).checkAndMutate(Bytes.toBytes(0), Bytes.toBytes("cf"))
474      .qualifier(Bytes.toBytes("cq")).ifEquals(Bytes.toBytes("v"))
475      .thenMutate(new RowMutations(Bytes.toBytes(0)).add((Mutation) new Delete(Bytes.toBytes(0))))
476      .join();
477    verify(stub, times(1)).multi(assertPriority(SYSTEMTABLE_QOS),
478      any(ClientProtos.MultiRequest.class), any());
479  }
480
481  @Test
482  public void testScan() throws IOException, InterruptedException {
483    try (ResultScanner scanner = conn.getTable(TableName.valueOf(name.getMethodName()))
484      .getScanner(new Scan().setCaching(1).setMaxResultSize(1).setPriority(19))) {
485      assertNotNull(scanner.next());
486      Thread.sleep(1000);
487    }
488    Thread.sleep(1000);
489    // open, next, several renew lease, and then close
490    verify(stub, atLeast(4)).scan(assertPriority(19), any(ScanRequest.class), any());
491  }
492
493  @Test
494  public void testScanNormalTable() throws IOException, InterruptedException {
495    try (ResultScanner scanner = conn.getTable(TableName.valueOf(name.getMethodName()))
496      .getScanner(new Scan().setCaching(1).setMaxResultSize(1))) {
497      assertNotNull(scanner.next());
498      Thread.sleep(1000);
499    }
500    Thread.sleep(1000);
501    // open, next, several renew lease, and then close
502    verify(stub, atLeast(4)).scan(assertPriority(NORMAL_QOS), any(ScanRequest.class), any());
503  }
504
505  @Test
506  public void testScanSystemTable() throws IOException, InterruptedException {
507    try (ResultScanner scanner =
508      conn.getTable(TableName.valueOf(SYSTEM_NAMESPACE_NAME_STR, name.getMethodName()))
509        .getScanner(new Scan().setCaching(1).setMaxResultSize(1))) {
510      assertNotNull(scanner.next());
511      Thread.sleep(1000);
512    }
513    Thread.sleep(1000);
514    // open, next, several renew lease, and then close
515    verify(stub, atLeast(4)).scan(assertPriority(SYSTEMTABLE_QOS), any(ScanRequest.class), any());
516  }
517
518  @Test
519  public void testScanMetaTable() throws IOException, InterruptedException {
520    try (ResultScanner scanner = conn.getTable(TableName.META_TABLE_NAME)
521      .getScanner(new Scan().setCaching(1).setMaxResultSize(1))) {
522      assertNotNull(scanner.next());
523      Thread.sleep(1000);
524    }
525    Thread.sleep(1000);
526    // open, next, several renew lease, and then close
527    verify(stub, atLeast(4)).scan(assertPriority(SYSTEMTABLE_QOS), any(ScanRequest.class), any());
528  }
529
530  @Test
531  public void testBatchNormalTable() {
532    conn.getTable(TableName.valueOf(name.getMethodName()))
533      .batchAll(Arrays.asList(new Delete(Bytes.toBytes(0)))).join();
534    verify(stub, times(1)).multi(assertPriority(NORMAL_QOS), any(ClientProtos.MultiRequest.class),
535      any());
536  }
537
538  @Test
539  public void testBatchSystemTable() {
540    conn.getTable(TableName.valueOf(SYSTEM_NAMESPACE_NAME_STR, name.getMethodName()))
541      .batchAll(Arrays.asList(new Delete(Bytes.toBytes(0)))).join();
542    verify(stub, times(1)).multi(assertPriority(SYSTEMTABLE_QOS),
543      any(ClientProtos.MultiRequest.class), any());
544  }
545
546  @Test
547  public void testBatchMetaTable() {
548    conn.getTable(TableName.META_TABLE_NAME).batchAll(Arrays.asList(new Delete(Bytes.toBytes(0))))
549      .join();
550    verify(stub, times(1)).multi(assertPriority(SYSTEMTABLE_QOS),
551      any(ClientProtos.MultiRequest.class), any());
552  }
553}