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
;
52 * Test {@link org.apache.hadoop.hbase.zookeeper.MetaTableLocator}
54 @Category({ MiscTests
.class, MediumTests
.class })
55 public class TestMetaTableLocator
{
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
;
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();
76 public static void afterClass() throws IOException
{
77 UTIL
.getZkCluster().shutdown();
81 public void before() throws IOException
{
82 this.abortable
= new Abortable() {
84 public void abort(String why
, Throwable e
) {
89 public boolean isAborted() {
94 new ZKWatcher(UTIL
.getConfiguration(), this.getClass().getSimpleName(), this.abortable
, true);
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
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
)) {
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.
152 public void testNoTimeoutWaitForMeta() throws IOException
, InterruptedException
, KeeperException
{
153 ServerName hsa
= MetaTableLocator
.getMetaRegionLocation(watcher
);
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
);
162 // Join the thread... should exit shortly.
164 // Now meta is available.
165 assertTrue(MetaTableLocator
.getMetaRegionLocation(watcher
).equals(hsa
));
168 private void startWaitAliveThenWaitItLives(final Thread t
, final int ms
) {
170 UTIL
.waitFor(2000, t
::isAlive
);
173 assertTrue("Assert " + t
.getName() + " still waiting", t
.isAlive());
179 class WaitOnMetaThread
extends Thread
{
189 } catch (InterruptedException e
) {
190 throw new RuntimeException("Failed wait", e
);
192 LOG
.info("Exiting " + getName());
195 void doWaiting() throws InterruptedException
{
198 if (MetaTableLocator
.waitMetaRegionLocation(watcher
, 10000) != null) {
202 } catch (NotAllMetaRegionsOnlineException e
) {