1   /**
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
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.*;
25  import org.apache.hadoop.hbase.catalog.MetaReader;
26  import org.apache.hadoop.hbase.client.HTable;
27  import org.apache.hadoop.hbase.executor.EventHandler;
28  import org.apache.hadoop.hbase.executor.EventHandler.EventHandlerListener;
29  import org.apache.hadoop.hbase.executor.EventHandler.EventType;
30  import org.apache.hadoop.hbase.util.Bytes;
31  import org.apache.hadoop.hbase.util.Pair;
32  
33  import java.io.IOException;
34  import java.util.List;
35  import java.util.concurrent.CountDownLatch;
36  import java.util.concurrent.TimeUnit;
37  
38  import org.junit.AfterClass;
39  import org.junit.BeforeClass;
40  import org.junit.Test;
41  
42  import com.google.common.base.Joiner;
43  import org.junit.experimental.categories.Category;
44  
45  import static org.junit.Assert.*;
46  
47  @Category(MediumTests.class)
48  public class TestMaster {
49    private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
50    private static final Log LOG = LogFactory.getLog(TestMaster.class);
51    private static final byte[] TABLENAME = Bytes.toBytes("TestMaster");
52    private static final byte[] FAMILYNAME = Bytes.toBytes("fam");
53  
54    @BeforeClass
55    public static void beforeAllTests() throws Exception {
56      // Start a cluster of two regionservers.
57      TEST_UTIL.startMiniCluster(2);
58    }
59  
60    @AfterClass
61    public static void afterAllTests() throws Exception {
62      TEST_UTIL.shutdownMiniCluster();
63    }
64  
65    @Test
66    public void testMasterOpsWhileSplitting() throws Exception {
67      MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
68      HMaster m = cluster.getMaster();
69  
70      HTable ht = TEST_UTIL.createTable(TABLENAME, FAMILYNAME);
71      assertTrue(m.assignmentManager.getZKTable().isEnabledTable
72          (Bytes.toString(TABLENAME)));
73      TEST_UTIL.loadTable(ht, FAMILYNAME);
74      ht.close();
75  
76      List<Pair<HRegionInfo, ServerName>> tableRegions =
77        MetaReader.getTableRegionsAndLocations(m.getCatalogTracker(),
78            Bytes.toString(TABLENAME));
79      LOG.info("Regions after load: " + Joiner.on(',').join(tableRegions));
80      assertEquals(1, tableRegions.size());
81      assertArrayEquals(HConstants.EMPTY_START_ROW,
82          tableRegions.get(0).getFirst().getStartKey());
83      assertArrayEquals(HConstants.EMPTY_END_ROW,
84          tableRegions.get(0).getFirst().getEndKey());
85  
86      // Now trigger a split and stop when the split is in progress
87      CountDownLatch split = new CountDownLatch(1);
88      CountDownLatch proceed = new CountDownLatch(1);
89      RegionSplitListener list = new RegionSplitListener(split, proceed);
90      cluster.getMaster().executorService.
91        registerListener(EventType.RS_ZK_REGION_SPLIT, list);
92  
93      LOG.info("Splitting table");
94      TEST_UTIL.getHBaseAdmin().split(TABLENAME);
95      LOG.info("Waiting for split result to be about to open");
96      split.await(60, TimeUnit.SECONDS);
97      try {
98        LOG.info("Making sure we can call getTableRegions while opening");
99        tableRegions = MetaReader.getTableRegionsAndLocations(m.getCatalogTracker(),
100         TABLENAME, false);
101 
102       LOG.info("Regions: " + Joiner.on(',').join(tableRegions));
103       // We have three regions because one is split-in-progress
104       assertEquals(3, tableRegions.size());
105       LOG.info("Making sure we can call getTableRegionClosest while opening");
106       Pair<HRegionInfo, ServerName> pair =
107         m.getTableRegionForRow(TABLENAME, Bytes.toBytes("cde"));
108       LOG.info("Result is: " + pair);
109       Pair<HRegionInfo, ServerName> tableRegionFromName =
110         MetaReader.getRegion(m.getCatalogTracker(),
111             pair.getFirst().getRegionName());
112       assertEquals(tableRegionFromName.getFirst(), pair.getFirst());
113     } finally {
114       proceed.countDown();
115     }
116   }
117 
118   @Test
119   public void testMoveRegionWhenNotInitialized() {
120     MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
121     HMaster m = cluster.getMaster();
122     try {
123       m.initialized = false; // fake it, set back later
124       HRegionInfo meta = HRegionInfo.FIRST_META_REGIONINFO;
125       m.move(meta.getEncodedNameAsBytes(), null);
126       fail("Region should not be moved since master is not initialized");
127     } catch (IOException ioe) {
128       assertTrue(ioe.getCause() instanceof PleaseHoldException);
129     } finally {
130       m.initialized = true;
131     }
132   }
133 
134   static class RegionSplitListener implements EventHandlerListener {
135     CountDownLatch split, proceed;
136 
137     public RegionSplitListener(CountDownLatch split, CountDownLatch proceed) {
138       this.split = split;
139       this.proceed = proceed;
140     }
141 
142     @Override
143     public void afterProcess(EventHandler event) {
144       if (event.getEventType() != EventType.RS_ZK_REGION_SPLIT) {
145         return;
146       }
147       try {
148         split.countDown();
149         proceed.await(60, TimeUnit.SECONDS);
150       } catch (InterruptedException ie) {
151         throw new RuntimeException(ie);
152       }
153       return;
154     }
155 
156     @Override
157     public void beforeProcess(EventHandler event) {
158     }
159   }
160 
161   @org.junit.Rule
162   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
163     new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
164 }