HBASE-26481 Consider rolling upgrading from old region replication framework (#3880)
[hbase.git] / hbase-server / src / test / java / org / apache / hadoop / hbase / client / TestCheckAndMutateWithByteBuff.java
blobde1d02bfe7341a54892ef3707881857c262866d8
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
10 * http://www.apache.org/licenses/LICENSE-2.0
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.
18 package org.apache.hadoop.hbase.client;
20 import static org.apache.hadoop.hbase.util.Threads.sleep;
21 import static org.junit.Assert.assertTrue;
23 import java.io.IOException;
24 import java.util.List;
25 import org.apache.hadoop.conf.Configuration;
26 import org.apache.hadoop.fs.FileSystem;
27 import org.apache.hadoop.fs.Path;
28 import org.apache.hadoop.hbase.Cell;
29 import org.apache.hadoop.hbase.HBaseClassTestRule;
30 import org.apache.hadoop.hbase.HBaseTestingUtil;
31 import org.apache.hadoop.hbase.HConstants;
32 import org.apache.hadoop.hbase.TableName;
33 import org.apache.hadoop.hbase.io.ByteBuffAllocator;
34 import org.apache.hadoop.hbase.io.DeallocateRewriteByteBuffAllocator;
35 import org.apache.hadoop.hbase.io.hfile.BlockCacheFactory;
36 import org.apache.hadoop.hbase.regionserver.HRegion;
37 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
38 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
39 import org.apache.hadoop.hbase.testclassification.LargeTests;
40 import org.apache.hadoop.hbase.util.Bytes;
41 import org.apache.hadoop.hbase.wal.WAL;
42 import org.junit.AfterClass;
43 import org.junit.BeforeClass;
44 import org.junit.ClassRule;
45 import org.junit.Rule;
46 import org.junit.Test;
47 import org.junit.experimental.categories.Category;
48 import org.junit.rules.TestName;
49 import org.slf4j.Logger;
50 import org.slf4j.LoggerFactory;
52 @Category(LargeTests.class)
53 public class TestCheckAndMutateWithByteBuff {
54 private static final Logger LOG = LoggerFactory.getLogger(TestCheckAndMutateWithByteBuff.class);
56 @ClassRule
57 public static final HBaseClassTestRule CLASS_RULE =
58 HBaseClassTestRule.forClass(TestCheckAndMutateWithByteBuff.class);
60 @Rule
61 public TestName name = new TestName();
63 private static final byte[] CF = Bytes.toBytes("CF");
64 private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil();
65 private static final Configuration conf = TEST_UTIL.getConfiguration();
66 private static Admin admin = null;
68 @BeforeClass
69 public static void setupBeforeClass() throws Exception {
70 conf.set(HConstants.REGION_IMPL, TestCheckAndMutateRegion.class.getName());
71 conf.set(ByteBuffAllocator.BYTEBUFF_ALLOCATOR_CLASS,
72 DeallocateRewriteByteBuffAllocator.class.getName());
73 conf.setBoolean(ByteBuffAllocator.ALLOCATOR_POOL_ENABLED_KEY, true);
74 conf.setInt(ByteBuffAllocator.MIN_ALLOCATE_SIZE_KEY, 1);
75 conf.setInt(BlockCacheFactory.BUCKET_CACHE_WRITER_THREADS_KEY, 20);
76 conf.setInt(ByteBuffAllocator.BUFFER_SIZE_KEY, 1024);
77 conf.set(HConstants.BUCKET_CACHE_IOENGINE_KEY, "offheap");
78 conf.setInt(HConstants.BUCKET_CACHE_SIZE_KEY, 64);
79 conf.setInt("hbase.client.retries.number", 1);
80 TEST_UTIL.startMiniCluster();
81 admin = TEST_UTIL.getAdmin();
84 @AfterClass
85 public static void tearDownAfterClass() throws Exception {
86 TEST_UTIL.shutdownMiniCluster();
89 @Test
90 public void testCheckAndMutateWithByteBuff() throws Exception {
91 Table testTable = createTable(TableName.valueOf(name.getMethodName()));
92 byte[] checkRow = Bytes.toBytes("checkRow");
93 byte[] checkQualifier = Bytes.toBytes("cq");
94 byte[] checkValue = Bytes.toBytes("checkValue");
96 Put put = new Put(checkRow);
97 put.addColumn(CF, checkQualifier, checkValue);
98 testTable.put(put);
99 admin.flush(testTable.getName());
101 assertTrue(testTable.checkAndMutate(checkRow, CF).qualifier(checkQualifier).
102 ifEquals(checkValue)
103 .thenPut(new Put(checkRow).addColumn(CF, Bytes.toBytes("q1"),
104 Bytes.toBytes("testValue"))));
107 private Table createTable(TableName tableName)
108 throws IOException {
109 TableDescriptor td = TableDescriptorBuilder.newBuilder(tableName)
110 .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(CF).setBlocksize(100).build())
111 .build();
112 return TEST_UTIL.createTable(td, null);
116 * An override of HRegion to allow sleep after get(), waiting for the release of DBB
118 public static class TestCheckAndMutateRegion extends HRegion {
119 public TestCheckAndMutateRegion(Path tableDir, WAL log, FileSystem fs, Configuration confParam,
120 RegionInfo info, TableDescriptor htd, RegionServerServices rsServices) {
121 super(tableDir, log, fs, confParam, info, htd, rsServices);
124 public TestCheckAndMutateRegion(HRegionFileSystem fs, WAL wal, Configuration confParam,
125 TableDescriptor htd, RegionServerServices rsServices) {
126 super(fs, wal, confParam, htd, rsServices);
129 @Override
130 public List<Cell> get(Get get, boolean withCoprocessor) throws IOException {
131 List<Cell> cells = super.get(get, withCoprocessor);
132 sleep(600);
133 return cells;