1   /*
2    * Copyright 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.regionserver;
21  
22  import static org.junit.Assert.assertEquals;
23  import static org.junit.Assert.assertTrue;
24  
25  import java.io.IOException;
26  import java.util.ArrayList;
27  import java.util.List;
28  
29  import org.apache.commons.logging.Log;
30  import org.apache.commons.logging.LogFactory;
31  import org.apache.hadoop.conf.Configuration;
32  import org.apache.hadoop.hbase.HBaseTestingUtility;
33  import org.apache.hadoop.hbase.HColumnDescriptor;
34  import org.apache.hadoop.hbase.HConstants;
35  import org.apache.hadoop.hbase.HRegionInfo;
36  import org.apache.hadoop.hbase.HTableDescriptor;
37  import org.apache.hadoop.hbase.LargeTests;
38  import org.apache.hadoop.hbase.MiniHBaseCluster;
39  import org.apache.hadoop.hbase.ServerName;
40  import org.apache.hadoop.hbase.client.HBaseAdmin;
41  import org.apache.hadoop.hbase.client.HTable;
42  import org.apache.hadoop.hbase.client.Put;
43  import org.apache.hadoop.hbase.client.ResultScanner;
44  import org.apache.hadoop.hbase.client.Scan;
45  import org.apache.hadoop.hbase.master.AssignmentManager;
46  import org.apache.hadoop.hbase.master.HMaster;
47  import org.apache.hadoop.hbase.master.ServerManager;
48  import org.apache.hadoop.hbase.master.TestMasterFailover;
49  import org.apache.hadoop.hbase.util.Bytes;
50  import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
51  import org.apache.hadoop.hbase.zookeeper.ZKAssign;
52  import org.apache.hadoop.hbase.zookeeper.ZKTable;
53  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
54  import org.apache.zookeeper.KeeperException;
55  import org.junit.AfterClass;
56  import org.junit.BeforeClass;
57  import org.junit.Test;
58  import org.junit.experimental.categories.Category;
59  
60  @Category(LargeTests.class)
61  public class TestRSKilledWhenMasterInitializing {
62    private static final Log LOG = LogFactory.getLog(TestMasterFailover.class);
63  
64    private static final HBaseTestingUtility TESTUTIL = new HBaseTestingUtility();
65    private static final int NUM_MASTERS = 1;
66    private static final int NUM_RS = 5;
67  
68    @BeforeClass
69    public static void setUpBeforeClass() throws Exception {
70      // Set it so that this test runs with my custom master
71      Configuration conf = TESTUTIL.getConfiguration();
72      conf.setClass(HConstants.MASTER_IMPL, TestingMaster.class, HMaster.class);
73      conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, 3);
74      conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MAXTOSTART, 5);
75  
76      // Start up the cluster.
77      TESTUTIL.startMiniCluster(NUM_MASTERS, NUM_RS);
78    }
79  
80    @AfterClass
81    public static void tearDownAfterClass() throws Exception {
82      if (!TESTUTIL.getHBaseCluster().getMaster().isInitialized()) {
83        // master is not initialized and is waiting something forever.
84        for (MasterThread mt : TESTUTIL.getHBaseCluster().getLiveMasterThreads()) {
85          mt.interrupt();
86        }
87      }
88      TESTUTIL.shutdownMiniCluster();
89    }
90  
91    /**
92     * An HMaster instance used in this test. If 'TestingMaster.sleep' is set in
93     * the Configuration, then we'll sleep after log is split and we'll also
94     * return a custom RegionServerTracker.
95     */
96    public static class TestingMaster extends HMaster {
97      private boolean logSplit = false;
98  
99      public TestingMaster(Configuration conf) throws IOException,
100         KeeperException, InterruptedException {
101       super(conf);
102     }
103 
104     public boolean isLogSplitAfterStartup() {
105       return logSplit;
106     }
107   }
108 
109   @Test(timeout = 180000)
110   public void testCorrectnessWhenMasterFailOver() throws Exception {
111     final byte[] TABLENAME = Bytes.toBytes("testCorrectnessWhenMasterFailOver");
112     final byte[] FAMILY = Bytes.toBytes("family");
113     final byte[][] SPLITKEYS = { Bytes.toBytes("b"), Bytes.toBytes("i") };
114 
115     MiniHBaseCluster cluster = TESTUTIL.getHBaseCluster();
116 
117     HTableDescriptor desc = new HTableDescriptor(TABLENAME);
118     desc.addFamily(new HColumnDescriptor(FAMILY));
119     HBaseAdmin hbaseAdmin = TESTUTIL.getHBaseAdmin();
120     hbaseAdmin.createTable(desc, SPLITKEYS);
121 
122     assertTrue(hbaseAdmin.isTableAvailable(TABLENAME));
123 
124     HTable table = new HTable(TESTUTIL.getConfiguration(), TABLENAME);
125     List<Put> puts = new ArrayList<Put>();
126     Put put1 = new Put(Bytes.toBytes("a"));
127     put1.add(FAMILY, Bytes.toBytes("q1"), Bytes.toBytes("value"));
128     Put put2 = new Put(Bytes.toBytes("h"));
129     put2.add(FAMILY, Bytes.toBytes("q1"), Bytes.toBytes("value"));
130     Put put3 = new Put(Bytes.toBytes("o"));
131     put3.add(FAMILY, Bytes.toBytes("q1"), Bytes.toBytes("value"));
132     puts.add(put1);
133     puts.add(put2);
134     puts.add(put3);
135     table.put(puts);
136     ResultScanner resultScanner = table.getScanner(new Scan());
137     int count = 0;
138     while (resultScanner.next() != null) {
139       count++;
140     }
141     resultScanner.close();
142     table.close();
143     assertEquals(3, count);
144 
145     /* Starting test */
146     cluster.getConfiguration().setBoolean("TestingMaster.sleep", true);
147     cluster.getConfiguration().setInt("TestingMaster.sleep.duration", 10000);
148 
149     /* NO.1 .META. region correctness */
150     // First abort master
151     abortMaster(cluster);
152     TestingMaster master = startMasterAndWaitTillMetaRegionAssignment(cluster);
153 
154     // Second kill meta server
155     int metaServerNum = cluster.getServerWithMeta();
156     int rootServerNum = cluster.getServerWith(HRegionInfo.ROOT_REGIONINFO
157         .getRegionName());
158     HRegionServer metaRS = cluster.getRegionServer(metaServerNum);
159     LOG.debug("Killing metaRS and carryingRoot = "
160         + (metaServerNum == rootServerNum));
161     metaRS.kill();
162     metaRS.join();
163 
164     /*
165      * Sleep double time of TestingMaster.sleep.duration, so we can ensure that
166      * master has already assigned ROOTandMETA or is blocking on assigning
167      * ROOTandMETA
168      */
169     Thread.sleep(10000 * 2);
170 
171     waitUntilMasterIsInitialized(master);
172 
173     // Third check whether data is correct in meta region
174     assertTrue(hbaseAdmin.isTableAvailable(TABLENAME));
175 
176     /*
177      * NO.2 -ROOT- region correctness . If the .META. server killed in the NO.1
178      * is also carrying -ROOT- region, it is not needed
179      */
180     if (rootServerNum != metaServerNum) {
181       // First abort master
182       abortMaster(cluster);
183       master = startMasterAndWaitTillMetaRegionAssignment(cluster);
184 
185       // Second kill meta server
186       HRegionServer rootRS = cluster.getRegionServer(rootServerNum);
187       LOG.debug("Killing rootRS");
188       rootRS.kill();
189       rootRS.join();
190 
191       /*
192        * Sleep double time of TestingMaster.sleep.duration, so we can ensure
193        * that master has already assigned ROOTandMETA or is blocking on
194        * assigning ROOTandMETA
195        */
196       Thread.sleep(10000 * 2);
197       waitUntilMasterIsInitialized(master);
198 
199       // Third check whether data is correct in meta region
200       assertTrue(hbaseAdmin.isTableAvailable(TABLENAME));
201     }
202 
203 
204     /* NO.3 data region correctness */
205     ServerManager serverManager = cluster.getMaster().getServerManager();
206     while (serverManager.areDeadServersInProgress()) {
207       Thread.sleep(100);
208     }
209     // Create a ZKW to use in the test
210     ZooKeeperWatcher zkw = HBaseTestingUtility.getZooKeeperWatcher(TESTUTIL);
211     ZKAssign.blockUntilNoRIT(zkw);
212 
213     table = new HTable(TESTUTIL.getConfiguration(), TABLENAME);
214     resultScanner = table.getScanner(new Scan());
215     count = 0;
216     while (resultScanner.next() != null) {
217       count++;
218     }
219     resultScanner.close();
220     table.close();
221     assertEquals(3, count);
222   }
223 
224   @Test (timeout=180000)
225   public void testMasterFailoverWhenDisablingTableRegionsInRITOnDeadRS() throws Exception {
226     MiniHBaseCluster cluster = TESTUTIL.getHBaseCluster();
227     HMaster master = cluster.getMaster();
228     // disable load balancing on this master
229     master.balanceSwitch(false);
230 
231     final String table = "testMasterFailoverWhenDisablingTableRegionsInRITOnDeadRS";
232     byte [] FAMILY = Bytes.toBytes("family");
233     byte[][] SPLIT_KEYS =
234         new byte[][] {Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c"),
235             Bytes.toBytes("d") };
236     HTableDescriptor htd = new HTableDescriptor(table);
237     HColumnDescriptor hcd = new HColumnDescriptor(FAMILY);
238     htd.addFamily(hcd);
239     TESTUTIL.getHBaseAdmin().createTable(htd, SPLIT_KEYS);
240     AssignmentManager am = cluster.getMaster().getAssignmentManager();
241     List<HRegionInfo> regionsOfTable = null;
242     while ((regionsOfTable = am.getRegionsOfTable(table.getBytes())).size()
243         != (SPLIT_KEYS.length + 1)) {
244       Thread.sleep(10);
245     }
246     HRegionInfo closingRegion = regionsOfTable.get(0);
247     ServerName serverName = am.getRegionServerOfRegion(closingRegion);
248     HRegionServer deadRS = null;
249     for (int i = 0; i < cluster.getRegionServerThreads().size(); i++) {
250       deadRS = cluster.getRegionServer(i);
251       if (deadRS.getServerName().equals(serverName)) {
252         break;
253       }
254     }
255 
256     // Disable the table in ZK
257     ZKTable zkTable = am.getZKTable();
258     zkTable.setDisablingTable(table);
259     ZKAssign.createNodeClosing(master.getZooKeeper(), closingRegion, serverName);
260 
261     // Stop the master
262     abortMaster(cluster);
263     master = startMasterAndWaitTillMetaRegionAssignment(cluster);
264     deadRS.kill();
265     deadRS.join();
266     waitUntilMasterIsInitialized(master);
267     am = cluster.getMaster().getAssignmentManager();
268     zkTable = am.getZKTable();
269     // wait for no more RIT
270     ZKAssign.blockUntilNoRIT(master.getZooKeeper());
271     while (!master.getAssignmentManager().getZKTable().isDisabledTable(table)) {
272       Thread.sleep(10);
273     }
274     assertTrue("Table should be disabled state.", zkTable.isDisabledTable(table));
275     HBaseAdmin admin = new HBaseAdmin(master.getConfiguration());
276     admin.deleteTable(table);
277   }
278 
279   private void abortMaster(MiniHBaseCluster cluster)
280       throws InterruptedException {
281     for (MasterThread mt : cluster.getLiveMasterThreads()) {
282       if (mt.getMaster().isActiveMaster()) {
283         mt.getMaster().abort("Aborting for tests", new Exception("Trace info"));
284         mt.join();
285         break;
286       }
287     }
288     LOG.debug("Master is aborted");
289   }
290 
291   private TestingMaster startMasterAndWaitTillMetaRegionAssignment(MiniHBaseCluster cluster)
292       throws IOException, InterruptedException {
293     TestingMaster master = (TestingMaster) cluster.startMaster().getMaster();
294     while (!master.isInitializationStartsMetaRegoinAssignment()) {
295       Thread.sleep(100);
296     }
297     return master;
298   }
299 
300   private void waitUntilMasterIsInitialized(HMaster master)
301       throws InterruptedException {
302     while (!master.isInitialized()) {
303       Thread.sleep(100);
304     }
305     while (master.getServerManager().areDeadServersInProgress()) {
306       Thread.sleep(100);
307     }
308     LOG.debug("master isInitialized");
309   }
310   
311   @org.junit.Rule
312   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
313     new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
314 
315 }