1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.master;
21
22 import org.apache.commons.logging.Log;
23 import org.apache.commons.logging.LogFactory;
24 import org.apache.hadoop.hbase.HConstants;
25 import org.apache.hadoop.hbase.MiniHBaseCluster;
26 import org.apache.hadoop.hbase.HMsg;
27 import org.apache.hadoop.hbase.HServerInfo;
28 import org.apache.hadoop.hbase.HBaseTestingUtility;
29 import org.apache.hadoop.hbase.HRegionInfo;
30 import org.apache.hadoop.hbase.HServerAddress;
31 import org.apache.hadoop.hbase.client.HBaseAdmin;
32 import org.apache.hadoop.hbase.client.HTable;
33 import org.apache.hadoop.hbase.executor.HBaseEventHandler;
34 import org.apache.hadoop.hbase.executor.HBaseEventHandler.HBaseEventHandlerListener;
35 import org.apache.hadoop.hbase.executor.HBaseEventHandler.HBaseEventType;
36 import org.apache.hadoop.hbase.util.Bytes;
37 import org.apache.hadoop.hbase.util.Pair;
38
39 import java.io.IOException;
40 import java.util.List;
41 import java.util.concurrent.CountDownLatch;
42 import java.util.concurrent.TimeUnit;
43
44 import org.junit.AfterClass;
45 import org.junit.BeforeClass;
46 import org.junit.Test;
47
48 import com.google.common.base.Joiner;
49
50 import static org.junit.Assert.*;
51
52 public class TestMaster {
53 private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
54 private static final Log LOG = LogFactory.getLog(TestMasterWithDisabling.class);
55 private static final byte[] TABLENAME = Bytes.toBytes("TestMaster");
56 private static final byte[] FAMILYNAME = Bytes.toBytes("fam");
57
58 @BeforeClass
59 public static void beforeAllTests() throws Exception {
60
61 TEST_UTIL.startMiniCluster(1);
62 }
63
64 @AfterClass
65 public static void afterAllTests() throws IOException {
66 TEST_UTIL.shutdownMiniCluster();
67 }
68
69 @Test
70 public void testMasterOpsWhileSplitting() throws Exception {
71 MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
72 HMaster m = cluster.getMaster();
73 HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
74
75 TEST_UTIL.createTable(TABLENAME, FAMILYNAME);
76 TEST_UTIL.loadTable(new HTable(TABLENAME), FAMILYNAME);
77
78 List<Pair<HRegionInfo, HServerAddress>> tableRegions =
79 m.getTableRegions(TABLENAME);
80 LOG.info("Regions after load: " + Joiner.on(',').join(tableRegions));
81 assertEquals(1, tableRegions.size());
82 assertArrayEquals(HConstants.EMPTY_START_ROW,
83 tableRegions.get(0).getFirst().getStartKey());
84 assertArrayEquals(HConstants.EMPTY_END_ROW,
85 tableRegions.get(0).getFirst().getEndKey());
86
87
88
89 CountDownLatch aboutToOpen = new CountDownLatch(1);
90 CountDownLatch proceed = new CountDownLatch(1);
91 RegionOpenListener list = new RegionOpenListener(aboutToOpen, proceed);
92 HBaseEventHandler.registerListener(list);
93
94 LOG.info("Splitting table");
95 admin.split(TABLENAME);
96 LOG.info("Waiting for split result to be about to open");
97 aboutToOpen.await(60, TimeUnit.SECONDS);
98 try {
99 LOG.info("Making sure we can call getTableRegions while opening");
100 tableRegions = m.getTableRegions(TABLENAME);
101 LOG.info("Regions: " + Joiner.on(',').join(tableRegions));
102
103 assertEquals(3, tableRegions.size());
104 LOG.info("Making sure we can call getTableRegionClosest while opening");
105 Pair<HRegionInfo,HServerAddress> pair =
106 m.getTableRegionForRow(TABLENAME, Bytes.toBytes("cde"));
107 LOG.info("Result is: " + pair);
108 Pair<HRegionInfo, HServerAddress> tableRegionFromName = m.getTableRegionFromName(pair.getFirst().getRegionName());
109 assertEquals(tableRegionFromName.getFirst(), pair.getFirst());
110 } finally {
111 proceed.countDown();
112 }
113 }
114
115 static class RegionOpenListener implements HBaseEventHandlerListener {
116 CountDownLatch aboutToOpen, proceed;
117
118 public RegionOpenListener(CountDownLatch aboutToOpen, CountDownLatch proceed)
119 {
120 this.aboutToOpen = aboutToOpen;
121 this.proceed = proceed;
122 }
123
124 @Override
125 public void afterProcess(HBaseEventHandler event) {
126 if (event.getHBEvent() != HBaseEventType.RS2ZK_REGION_OPENED) {
127 return;
128 }
129 try {
130 aboutToOpen.countDown();
131 proceed.await(60, TimeUnit.SECONDS);
132 } catch (InterruptedException ie) {
133 throw new RuntimeException(ie);
134 }
135 return;
136 }
137
138 @Override
139 public void beforeProcess(HBaseEventHandler event) {
140 }
141 }
142
143 }