HBASE-26921 Rewrite the counting cells part in TestMultiVersions (#4316)
[hbase.git] / hbase-server / src / test / java / org / apache / hadoop / hbase / client / TestCheckAndMutateWithByteBuff.java
blob1489c1f0400bec36720cd4bc1751f6a770852a6e
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.encoding.DataBlockEncoding;
36 import org.apache.hadoop.hbase.io.hfile.BlockCacheFactory;
37 import org.apache.hadoop.hbase.regionserver.HRegion;
38 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
39 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
40 import org.apache.hadoop.hbase.testclassification.LargeTests;
41 import org.apache.hadoop.hbase.util.Bytes;
42 import org.apache.hadoop.hbase.wal.WAL;
43 import org.junit.AfterClass;
44 import org.junit.BeforeClass;
45 import org.junit.ClassRule;
46 import org.junit.Rule;
47 import org.junit.Test;
48 import org.junit.experimental.categories.Category;
49 import org.junit.rules.TestName;
50 import org.slf4j.Logger;
51 import org.slf4j.LoggerFactory;
53 @Category(LargeTests.class)
54 public class TestCheckAndMutateWithByteBuff {
55 private static final Logger LOG = LoggerFactory.getLogger(TestCheckAndMutateWithByteBuff.class);
57 @ClassRule
58 public static final HBaseClassTestRule CLASS_RULE =
59 HBaseClassTestRule.forClass(TestCheckAndMutateWithByteBuff.class);
61 @Rule
62 public TestName name = new TestName();
64 private static final byte[] CF = Bytes.toBytes("CF");
65 private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil();
66 private static final Configuration conf = TEST_UTIL.getConfiguration();
67 private static Admin admin = null;
69 @BeforeClass
70 public static void setupBeforeClass() throws Exception {
71 conf.set(HConstants.REGION_IMPL, TestCheckAndMutateRegion.class.getName());
72 conf.set(ByteBuffAllocator.BYTEBUFF_ALLOCATOR_CLASS,
73 DeallocateRewriteByteBuffAllocator.class.getName());
74 conf.setBoolean(ByteBuffAllocator.ALLOCATOR_POOL_ENABLED_KEY, true);
75 conf.setInt(ByteBuffAllocator.MIN_ALLOCATE_SIZE_KEY, 1);
76 conf.setInt(BlockCacheFactory.BUCKET_CACHE_WRITER_THREADS_KEY, 20);
77 conf.setInt(ByteBuffAllocator.BUFFER_SIZE_KEY, 1024);
78 conf.set(HConstants.BUCKET_CACHE_IOENGINE_KEY, "offheap");
79 conf.setInt(HConstants.BUCKET_CACHE_SIZE_KEY, 64);
80 conf.setInt("hbase.client.retries.number", 1);
81 TEST_UTIL.startMiniCluster();
82 admin = TEST_UTIL.getAdmin();
85 @AfterClass
86 public static void tearDownAfterClass() throws Exception {
87 TEST_UTIL.shutdownMiniCluster();
90 @Test
91 public void testCheckAndMutateWithByteBuffNoEncode() throws Exception {
92 testCheckAndMutateWithByteBuff(TableName.valueOf(name.getMethodName()), DataBlockEncoding.NONE);
95 @Test
96 public void testCheckAndMutateWithByteBuffEncode() throws Exception {
97 // Tests for HBASE-26777.
98 // As most HBase.getRegion() calls have been factored out from HBase, you'd need to revert
99 // both HBASE-26777, and the HBase.get() replacements from HBASE-26036 for this test to fail
100 testCheckAndMutateWithByteBuff(TableName.valueOf(name.getMethodName()), DataBlockEncoding.FAST_DIFF);
103 private void testCheckAndMutateWithByteBuff(TableName tableName, DataBlockEncoding dbe) throws Exception {
104 Table testTable = createTable(tableName, dbe);
105 byte[] checkRow = Bytes.toBytes("checkRow");
106 byte[] checkQualifier = Bytes.toBytes("cq");
107 byte[] checkValue = Bytes.toBytes("checkValue");
109 Put put = new Put(checkRow);
110 put.addColumn(CF, checkQualifier, checkValue);
111 testTable.put(put);
112 admin.flush(testTable.getName());
114 assertTrue(testTable.checkAndMutate(checkRow, CF).qualifier(checkQualifier).
115 ifEquals(checkValue)
116 .thenPut(new Put(checkRow).addColumn(CF, Bytes.toBytes("q1"),
117 Bytes.toBytes("testValue"))));
120 private Table createTable(TableName tableName, DataBlockEncoding dbe)
121 throws IOException {
122 TableDescriptor td = TableDescriptorBuilder.newBuilder(tableName)
123 .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(CF)
124 .setBlocksize(100)
125 .setDataBlockEncoding(dbe)
126 .build())
127 .build();
128 return TEST_UTIL.createTable(td, null);
132 * An override of HRegion to allow sleep after get(), waiting for the release of DBB
134 public static class TestCheckAndMutateRegion extends HRegion {
135 public TestCheckAndMutateRegion(Path tableDir, WAL log, FileSystem fs, Configuration confParam,
136 RegionInfo info, TableDescriptor htd, RegionServerServices rsServices) {
137 super(tableDir, log, fs, confParam, info, htd, rsServices);
140 public TestCheckAndMutateRegion(HRegionFileSystem fs, WAL wal, Configuration confParam,
141 TableDescriptor htd, RegionServerServices rsServices) {
142 super(fs, wal, confParam, htd, rsServices);
145 @Override
146 public List<Cell> get(Get get, boolean withCoprocessor) throws IOException {
147 List<Cell> cells = super.get(get, withCoprocessor);
148 sleep(600);
149 return cells;