HBASE-23949 refactor loadBalancer implements for rsgroup balance by table to achieve...
[hbase.git] / hbase-server / src / test / java / org / apache / hadoop / hbase / master / TestWarmupRegion.java
blob79cde38b2b063049f57a3218c30f72f9e05f3ff0
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.master;
20 import static org.apache.hadoop.hbase.regionserver.HRegion.warmupHRegion;
21 import static org.junit.Assert.assertTrue;
23 import java.io.IOException;
24 import org.apache.hadoop.conf.Configuration;
25 import org.apache.hadoop.hbase.HBaseClassTestRule;
26 import org.apache.hadoop.hbase.HBaseTestingUtility;
27 import org.apache.hadoop.hbase.MiniHBaseCluster;
28 import org.apache.hadoop.hbase.TableName;
29 import org.apache.hadoop.hbase.Waiter;
30 import org.apache.hadoop.hbase.client.CompactionState;
31 import org.apache.hadoop.hbase.client.Put;
32 import org.apache.hadoop.hbase.client.RegionInfo;
33 import org.apache.hadoop.hbase.client.Table;
34 import org.apache.hadoop.hbase.client.TableDescriptor;
35 import org.apache.hadoop.hbase.regionserver.HRegion;
36 import org.apache.hadoop.hbase.regionserver.HRegionServer;
37 import org.apache.hadoop.hbase.testclassification.LargeTests;
38 import org.apache.hadoop.hbase.testclassification.MasterTests;
39 import org.apache.hadoop.hbase.util.Bytes;
40 import org.junit.After;
41 import org.junit.AfterClass;
42 import org.junit.Before;
43 import org.junit.BeforeClass;
44 import org.junit.ClassRule;
45 import org.junit.Test;
46 import org.junit.experimental.categories.Category;
47 import org.slf4j.Logger;
48 import org.slf4j.LoggerFactory;
50 /**
51 * Run tests that use the HBase clients; {@link org.apache.hadoop.hbase.client.TableBuilder}.
52 * Sets up the HBase mini cluster once at start and runs through all client tests.
53 * Each creates a table named for the method and does its stuff against that.
55 @Category({MasterTests.class, LargeTests.class})
56 @SuppressWarnings ("deprecation")
57 public class TestWarmupRegion {
59 @ClassRule
60 public static final HBaseClassTestRule CLASS_RULE =
61 HBaseClassTestRule.forClass(TestWarmupRegion.class);
63 private static final Logger LOG = LoggerFactory.getLogger(TestWarmupRegion.class);
64 protected TableName TABLENAME = TableName.valueOf("testPurgeFutureDeletes");
65 protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
66 private static byte [] ROW = Bytes.toBytes("testRow");
67 private static byte [] FAMILY = Bytes.toBytes("testFamily");
68 private static byte [] QUALIFIER = Bytes.toBytes("testQualifier");
69 private static byte [] VALUE = Bytes.toBytes("testValue");
70 private static byte[] COLUMN = Bytes.toBytes("column");
71 private static int numRows = 10000;
72 protected static int SLAVES = 3;
73 private static MiniHBaseCluster myCluster;
74 private static Table table;
76 /**
77 * @throws java.lang.Exception
79 @BeforeClass
80 public static void setUpBeforeClass() throws Exception {
81 Configuration conf = TEST_UTIL.getConfiguration();
82 TEST_UTIL.startMiniCluster(SLAVES);
85 /**
86 * @throws java.lang.Exception
88 @AfterClass
89 public static void tearDownAfterClass() throws Exception {
90 TEST_UTIL.shutdownMiniCluster();
93 /**
94 * @throws java.lang.Exception
96 @Before
97 public void setUp() throws Exception {
98 table = TEST_UTIL.createTable(TABLENAME, FAMILY);
100 // future timestamp
101 for (int i = 0; i < numRows; i++) {
102 long ts = System.currentTimeMillis() * 2;
103 Put put = new Put(ROW, ts);
104 put.addColumn(FAMILY, COLUMN, VALUE);
105 table.put(put);
108 // major compaction, purged future deletes
109 TEST_UTIL.getAdmin().flush(TABLENAME);
110 TEST_UTIL.getAdmin().majorCompact(TABLENAME);
112 // waiting for the major compaction to complete
113 TEST_UTIL.waitFor(6000, new Waiter.Predicate<IOException>() {
114 @Override
115 public boolean evaluate() throws IOException {
116 return TEST_UTIL.getAdmin().getCompactionState(TABLENAME) ==
117 CompactionState.NONE;
121 table.close();
126 * @throws java.lang.Exception
128 @After
129 public void tearDown() throws Exception {
130 // Nothing to do.
133 protected void runwarmup() throws InterruptedException{
134 Thread thread = new Thread(new Runnable() {
135 @Override
136 public void run() {
137 HRegionServer rs = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0);
138 HRegion region = TEST_UTIL.getMiniHBaseCluster().getRegions(TABLENAME).get(0);
139 RegionInfo info = region.getRegionInfo();
141 try {
142 TableDescriptor htd = table.getDescriptor();
143 for (int i = 0; i < 10; i++) {
144 warmupHRegion(info, htd, rs.getWAL(info), rs.getConfiguration(), rs, null);
146 } catch (IOException ie) {
147 LOG.error("Failed warming up region " + info.getRegionNameAsString(), ie);
151 thread.start();
152 thread.join();
156 * Basic client side validation of HBASE-4536
158 @Test
159 public void testWarmup() throws Exception {
160 int serverid = 0;
161 HRegion region = TEST_UTIL.getMiniHBaseCluster().getRegions(TABLENAME).get(0);
162 RegionInfo info = region.getRegionInfo();
163 runwarmup();
164 for (int i = 0; i < 10; i++) {
165 HRegionServer rs = TEST_UTIL.getMiniHBaseCluster().getRegionServer(serverid);
166 byte [] destName = Bytes.toBytes(rs.getServerName().toString());
167 assertTrue(destName != null);
168 LOG.info("i=" + i );
169 TEST_UTIL.getMiniHBaseCluster().getMaster().move(info.getEncodedNameAsBytes(), destName);
170 serverid = (serverid + 1) % 2;