HBASE-26567 Remove IndexType from ChunkCreator (#3947)
[hbase.git] / hbase-server / src / test / java / org / apache / hadoop / hbase / TestMetaTableLocator.java
blob3bea0a742d298a1c18b8bde1fb1433b8f810ebe5
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;
20 import static org.junit.Assert.assertEquals;
21 import static org.junit.Assert.assertNull;
22 import static org.junit.Assert.assertTrue;
24 import java.io.IOException;
25 import org.apache.hadoop.hbase.master.RegionState;
26 import org.apache.hadoop.hbase.testclassification.MediumTests;
27 import org.apache.hadoop.hbase.testclassification.MiscTests;
28 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
29 import org.apache.hadoop.hbase.util.Threads;
30 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
31 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
32 import org.apache.zookeeper.KeeperException;
33 import org.junit.After;
34 import org.junit.AfterClass;
35 import org.junit.Before;
36 import org.junit.BeforeClass;
37 import org.junit.ClassRule;
38 import org.junit.Test;
39 import org.junit.experimental.categories.Category;
40 import org.mockito.Mockito;
41 import org.slf4j.Logger;
42 import org.slf4j.LoggerFactory;
44 import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
45 import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
47 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
48 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest;
49 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetResponse;
51 /**
52 * Test {@link org.apache.hadoop.hbase.zookeeper.MetaTableLocator}
54 @Category({ MiscTests.class, MediumTests.class })
55 public class TestMetaTableLocator {
57 @ClassRule
58 public static final HBaseClassTestRule CLASS_RULE =
59 HBaseClassTestRule.forClass(TestMetaTableLocator.class);
61 private static final Logger LOG = LoggerFactory.getLogger(TestMetaTableLocator.class);
62 private static final HBaseTestingUtil UTIL = new HBaseTestingUtil();
63 private static final ServerName SN =
64 ServerName.valueOf("example.org", 1234, EnvironmentEdgeManager.currentTime());
65 private ZKWatcher watcher;
66 private Abortable abortable;
68 @BeforeClass
69 public static void beforeClass() throws Exception {
70 // Set this down so tests run quicker
71 UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 3);
72 UTIL.startMiniZKCluster();
75 @AfterClass
76 public static void afterClass() throws IOException {
77 UTIL.getZkCluster().shutdown();
80 @Before
81 public void before() throws IOException {
82 this.abortable = new Abortable() {
83 @Override
84 public void abort(String why, Throwable e) {
85 LOG.info(why, e);
88 @Override
89 public boolean isAborted() {
90 return false;
93 this.watcher =
94 new ZKWatcher(UTIL.getConfiguration(), this.getClass().getSimpleName(), this.abortable, true);
97 @After
98 public void after() {
99 try {
100 // Clean out meta location or later tests will be confused... they presume
101 // start fresh in zk.
102 MetaTableLocator.deleteMetaLocation(this.watcher);
103 } catch (KeeperException e) {
104 LOG.warn("Unable to delete hbase:meta location", e);
107 this.watcher.close();
111 * Test normal operations
113 @Test
114 public void testMetaLookup()
115 throws IOException, InterruptedException, ServiceException, KeeperException {
116 final ClientProtos.ClientService.BlockingInterface client =
117 Mockito.mock(ClientProtos.ClientService.BlockingInterface.class);
119 Mockito.when(client.get((RpcController) Mockito.any(), (GetRequest) Mockito.any()))
120 .thenReturn(GetResponse.newBuilder().build());
122 assertNull(MetaTableLocator.getMetaRegionLocation(this.watcher));
123 for (RegionState.State state : RegionState.State.values()) {
124 if (state.equals(RegionState.State.OPEN)) {
125 continue;
127 MetaTableLocator.setMetaLocation(this.watcher, SN, state);
128 assertNull(MetaTableLocator.getMetaRegionLocation(this.watcher));
129 assertEquals(state, MetaTableLocator.getMetaRegionState(this.watcher).getState());
131 MetaTableLocator.setMetaLocation(this.watcher, SN, RegionState.State.OPEN);
132 assertEquals(SN, MetaTableLocator.getMetaRegionLocation(this.watcher));
133 assertEquals(RegionState.State.OPEN,
134 MetaTableLocator.getMetaRegionState(this.watcher).getState());
136 MetaTableLocator.deleteMetaLocation(this.watcher);
137 assertNull(MetaTableLocator.getMetaRegionState(this.watcher).getServerName());
138 assertEquals(RegionState.State.OFFLINE,
139 MetaTableLocator.getMetaRegionState(this.watcher).getState());
140 assertNull(MetaTableLocator.getMetaRegionLocation(this.watcher));
143 @Test(expected = NotAllMetaRegionsOnlineException.class)
144 public void testTimeoutWaitForMeta() throws IOException, InterruptedException {
145 MetaTableLocator.waitMetaRegionLocation(watcher, 100);
149 * Test waiting on meat w/ no timeout specified.
151 @Test
152 public void testNoTimeoutWaitForMeta() throws IOException, InterruptedException, KeeperException {
153 ServerName hsa = MetaTableLocator.getMetaRegionLocation(watcher);
154 assertNull(hsa);
156 // Now test waiting on meta location getting set.
157 Thread t = new WaitOnMetaThread();
158 startWaitAliveThenWaitItLives(t, 1);
159 // Set a meta location.
160 MetaTableLocator.setMetaLocation(this.watcher, SN, RegionState.State.OPEN);
161 hsa = SN;
162 // Join the thread... should exit shortly.
163 t.join();
164 // Now meta is available.
165 assertTrue(MetaTableLocator.getMetaRegionLocation(watcher).equals(hsa));
168 private void startWaitAliveThenWaitItLives(final Thread t, final int ms) {
169 t.start();
170 UTIL.waitFor(2000, t::isAlive);
171 // Wait one second.
172 Threads.sleep(ms);
173 assertTrue("Assert " + t.getName() + " still waiting", t.isAlive());
177 * Wait on META.
179 class WaitOnMetaThread extends Thread {
181 WaitOnMetaThread() {
182 super("WaitOnMeta");
185 @Override
186 public void run() {
187 try {
188 doWaiting();
189 } catch (InterruptedException e) {
190 throw new RuntimeException("Failed wait", e);
192 LOG.info("Exiting " + getName());
195 void doWaiting() throws InterruptedException {
196 try {
197 for (;;) {
198 if (MetaTableLocator.waitMetaRegionLocation(watcher, 10000) != null) {
199 break;
202 } catch (NotAllMetaRegionsOnlineException e) {
203 // Ignore