001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.client;
019
020import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.assertErrors;
021import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.doFsck;
022import static org.junit.Assert.assertNotEquals;
023import static org.junit.Assert.assertNotNull;
024import static org.junit.Assert.assertTrue;
025import static org.junit.Assert.fail;
026
027import edu.umd.cs.findbugs.annotations.Nullable;
028import java.io.IOException;
029import java.util.Arrays;
030import java.util.Collection;
031import java.util.EnumSet;
032import java.util.HashSet;
033import java.util.List;
034import java.util.Set;
035import java.util.concurrent.ExecutorService;
036import java.util.concurrent.atomic.AtomicBoolean;
037import org.apache.hadoop.conf.Configuration;
038import org.apache.hadoop.hbase.Abortable;
039import org.apache.hadoop.hbase.ClusterMetrics.Option;
040import org.apache.hadoop.hbase.HBaseClassTestRule;
041import org.apache.hadoop.hbase.HBaseTestingUtility;
042import org.apache.hadoop.hbase.HConstants;
043import org.apache.hadoop.hbase.HRegionLocation;
044import org.apache.hadoop.hbase.MetaTableAccessor;
045import org.apache.hadoop.hbase.RegionLocations;
046import org.apache.hadoop.hbase.ServerName;
047import org.apache.hadoop.hbase.TableName;
048import org.apache.hadoop.hbase.TableNotFoundException;
049import org.apache.hadoop.hbase.Waiter;
050import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
051import org.apache.hadoop.hbase.master.assignment.AssignmentTestingUtil;
052import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
053import org.apache.hadoop.hbase.regionserver.StorefileRefresherChore;
054import org.apache.hadoop.hbase.testclassification.LargeTests;
055import org.apache.hadoop.hbase.util.Bytes;
056import org.apache.hadoop.hbase.util.HBaseFsck;
057import org.apache.hadoop.hbase.util.HbckErrorReporter.ERROR_CODE;
058import org.apache.hadoop.hbase.util.HBaseFsckRepair;
059import org.apache.hadoop.hbase.util.hbck.HbckTestingUtil;
060import org.apache.hadoop.hbase.zookeeper.LoadBalancerTracker;
061import org.apache.hadoop.hbase.zookeeper.ZKUtil;
062import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
063import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
064import org.apache.zookeeper.KeeperException;
065import org.junit.After;
066import org.junit.Before;
067import org.junit.ClassRule;
068import org.junit.Ignore;
069import org.junit.Rule;
070import org.junit.Test;
071import org.junit.experimental.categories.Category;
072import org.junit.rules.TestName;
073import org.slf4j.Logger;
074import org.slf4j.LoggerFactory;
075
076/**
077 * Tests the scenarios where replicas are enabled for the meta table
078 */
079@Category(LargeTests.class)
080public class TestMetaWithReplicas {
081
082  @ClassRule
083  public static final HBaseClassTestRule CLASS_RULE =
084      HBaseClassTestRule.forClass(TestMetaWithReplicas.class);
085
086  private static final Logger LOG = LoggerFactory.getLogger(TestMetaWithReplicas.class);
087  private final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
088  private static final int REGIONSERVERS_COUNT = 3;
089
090  @Rule
091  public TestName name = new TestName();
092
093  @Before
094  public void setup() throws Exception {
095    TEST_UTIL.getConfiguration().setInt("zookeeper.session.timeout", 30000);
096    TEST_UTIL.getConfiguration().setInt(HConstants.META_REPLICAS_NUM, 3);
097    TEST_UTIL.getConfiguration().setInt(
098        StorefileRefresherChore.REGIONSERVER_STOREFILE_REFRESH_PERIOD, 1000);
099    TEST_UTIL.startMiniCluster(REGIONSERVERS_COUNT);
100    AssignmentManager am = TEST_UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager();
101    Set<ServerName> sns = new HashSet<ServerName>();
102    ServerName hbaseMetaServerName =
103        TEST_UTIL.getMiniHBaseCluster().getMaster().getMetaTableLocator().
104            getMetaRegionLocation(TEST_UTIL.getZooKeeperWatcher());
105    LOG.info("HBASE:META DEPLOY: on " + hbaseMetaServerName);
106    sns.add(hbaseMetaServerName);
107    for (int replicaId = 1; replicaId < 3; replicaId++) {
108      RegionInfo h = RegionReplicaUtil
109        .getRegionInfoForReplica(RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId);
110      AssignmentTestingUtil.waitForAssignment(am, h);
111      ServerName sn = am.getRegionStates().getRegionServerOfRegion(h);
112      assertNotNull(sn);
113      LOG.info("HBASE:META DEPLOY: " + h.getRegionNameAsString() + " on " + sn);
114      sns.add(sn);
115    }
116    // Fun. All meta region replicas have ended up on the one server. This will cause this test
117    // to fail ... sometimes.
118    if (sns.size() == 1) {
119      int count = TEST_UTIL.getMiniHBaseCluster().getLiveRegionServerThreads().size();
120      assertTrue("count=" + count, count == REGIONSERVERS_COUNT);
121      LOG.warn("All hbase:meta replicas are on the one server; moving hbase:meta: " + sns);
122      int metaServerIndex = TEST_UTIL.getHBaseCluster().getServerWithMeta();
123      int newServerIndex = metaServerIndex;
124      while (newServerIndex == metaServerIndex) {
125        newServerIndex = (newServerIndex + 1) % REGIONSERVERS_COUNT;
126      }
127      assertNotEquals(metaServerIndex, newServerIndex);
128      ServerName destinationServerName =
129          TEST_UTIL.getHBaseCluster().getRegionServer(newServerIndex).getServerName();
130      ServerName metaServerName =
131          TEST_UTIL.getHBaseCluster().getRegionServer(metaServerIndex).getServerName();
132      assertNotEquals(destinationServerName, metaServerName);
133      TEST_UTIL.getAdmin().move(RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes(),
134          Bytes.toBytes(destinationServerName.toString()));
135    }
136    // Disable the balancer
137    LoadBalancerTracker l = new LoadBalancerTracker(TEST_UTIL.getZooKeeperWatcher(),
138        new Abortable() {
139          AtomicBoolean aborted = new AtomicBoolean(false);
140          @Override
141          public boolean isAborted() {
142            return aborted.get();
143          }
144          @Override
145          public void abort(String why, Throwable e) {
146            aborted.set(true);
147          }
148        });
149    l.setBalancerOn(false);
150    LOG.debug("All meta replicas assigned");
151  }
152
153  @After
154  public void tearDown() throws Exception {
155    TEST_UTIL.shutdownMiniCluster();
156  }
157
158  @Test
159  public void testMetaHTDReplicaCount() throws Exception {
160    assertTrue(TEST_UTIL.getAdmin().getTableDescriptor(TableName.META_TABLE_NAME)
161        .getRegionReplication() == 3);
162  }
163
164  @Test
165  public void testZookeeperNodesForReplicas() throws Exception {
166    // Checks all the znodes exist when meta's replicas are enabled
167    ZKWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
168    Configuration conf = TEST_UTIL.getConfiguration();
169    String baseZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
170        HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
171    String primaryMetaZnode = ZNodePaths.joinZNode(baseZNode,
172        conf.get("zookeeper.znode.metaserver", "meta-region-server"));
173    // check that the data in the znode is parseable (this would also mean the znode exists)
174    byte[] data = ZKUtil.getData(zkw, primaryMetaZnode);
175    ProtobufUtil.toServerName(data);
176    for (int i = 1; i < 3; i++) {
177      String secZnode = ZNodePaths.joinZNode(baseZNode,
178          conf.get("zookeeper.znode.metaserver", "meta-region-server") + "-" + i);
179      String str = zkw.getZNodePaths().getZNodeForReplica(i);
180      assertTrue(str.equals(secZnode));
181      // check that the data in the znode is parseable (this would also mean the znode exists)
182      data = ZKUtil.getData(zkw, secZnode);
183      ProtobufUtil.toServerName(data);
184    }
185  }
186
187  @Test
188  public void testShutdownHandling() throws Exception {
189    // This test creates a table, flushes the meta (with 3 replicas), kills the
190    // server holding the primary meta replica. Then it does a put/get into/from
191    // the test table. The put/get operations would use the replicas to locate the
192    // location of the test table's region
193    shutdownMetaAndDoValidations(TEST_UTIL);
194  }
195
196  public static void shutdownMetaAndDoValidations(HBaseTestingUtility util) throws Exception {
197    // This test creates a table, flushes the meta (with 3 replicas), kills the
198    // server holding the primary meta replica. Then it does a put/get into/from
199    // the test table. The put/get operations would use the replicas to locate the
200    // location of the test table's region
201    ZKWatcher zkw = util.getZooKeeperWatcher();
202    Configuration conf = util.getConfiguration();
203    conf.setBoolean(HConstants.USE_META_REPLICAS, true);
204
205    String baseZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
206        HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
207    String primaryMetaZnode = ZNodePaths.joinZNode(baseZNode,
208        conf.get("zookeeper.znode.metaserver", "meta-region-server"));
209    byte[] data = ZKUtil.getData(zkw, primaryMetaZnode);
210    ServerName primary = ProtobufUtil.toServerName(data);
211    LOG.info("Primary=" + primary.toString());
212
213    TableName TABLE = TableName.valueOf("testShutdownHandling");
214    byte[][] FAMILIES = new byte[][] { Bytes.toBytes("foo") };
215    if (util.getAdmin().tableExists(TABLE)) {
216      util.getAdmin().disableTable(TABLE);
217      util.getAdmin().deleteTable(TABLE);
218    }
219    ServerName master = null;
220    try (Connection c = ConnectionFactory.createConnection(util.getConfiguration());) {
221      try (Table htable = util.createTable(TABLE, FAMILIES);) {
222        util.getAdmin().flush(TableName.META_TABLE_NAME);
223        Thread.sleep(conf.getInt(StorefileRefresherChore.REGIONSERVER_STOREFILE_REFRESH_PERIOD,
224            30000) * 6);
225        List<RegionInfo> regions = MetaTableAccessor.getTableRegions(c, TABLE);
226        HRegionLocation hrl = MetaTableAccessor.getRegionLocation(c, regions.get(0));
227        // Ensure that the primary server for test table is not the same one as the primary
228        // of the meta region since we will be killing the srv holding the meta's primary...
229        // We want to be able to write to the test table even when the meta is not present ..
230        // If the servers are the same, then move the test table's region out of the server
231        // to another random server
232        if (hrl.getServerName().equals(primary)) {
233          util.getAdmin().move(hrl.getRegionInfo().getEncodedNameAsBytes(), null);
234          // wait for the move to complete
235          do {
236            Thread.sleep(10);
237            hrl = MetaTableAccessor.getRegionLocation(c, regions.get(0));
238          } while (primary.equals(hrl.getServerName()));
239          util.getAdmin().flush(TableName.META_TABLE_NAME);
240          Thread.sleep(conf.getInt(StorefileRefresherChore.REGIONSERVER_STOREFILE_REFRESH_PERIOD,
241              30000) * 3);
242        }
243        // Ensure all metas are not on same hbase:meta replica=0 server!
244
245        master = util.getHBaseClusterInterface().getClusterMetrics().getMasterName();
246        // kill the master so that regionserver recovery is not triggered at all
247        // for the meta server
248        LOG.info("Stopping master=" + master.toString());
249        util.getHBaseClusterInterface().stopMaster(master);
250        util.getHBaseClusterInterface().waitForMasterToStop(master, 60000);
251        LOG.info("Master " + master + " stopped!");
252        if (!master.equals(primary)) {
253          util.getHBaseClusterInterface().killRegionServer(primary);
254          util.getHBaseClusterInterface().waitForRegionServerToStop(primary, 60000);
255        }
256        ((ClusterConnection)c).clearRegionCache();
257      }
258      LOG.info("Running GETs");
259      Get get = null;
260      Result r = null;
261      byte[] row = "test".getBytes();
262      try (Table htable = c.getTable(TABLE);) {
263        Put put = new Put(row);
264        put.addColumn("foo".getBytes(), row, row);
265        BufferedMutator m = c.getBufferedMutator(TABLE);
266        m.mutate(put);
267        m.flush();
268        // Try to do a get of the row that was just put
269        get = new Get(row);
270        r = htable.get(get);
271        assertTrue(Arrays.equals(r.getRow(), row));
272        // now start back the killed servers and disable use of replicas. That would mean
273        // calls go to the primary
274        LOG.info("Starting Master");
275        util.getHBaseClusterInterface().startMaster(master.getHostname(), 0);
276        util.getHBaseClusterInterface().startRegionServer(primary.getHostname(), 0);
277        util.getHBaseClusterInterface().waitForActiveAndReadyMaster();
278        LOG.info("Master active!");
279        ((ClusterConnection)c).clearRegionCache();
280      }
281      conf.setBoolean(HConstants.USE_META_REPLICAS, false);
282      LOG.info("Running GETs no replicas");
283      try (Table htable = c.getTable(TABLE);) {
284        r = htable.get(get);
285        assertTrue(Arrays.equals(r.getRow(), row));
286      }
287    }
288  }
289
290  @Test
291  public void testMetaLookupThreadPoolCreated() throws Exception {
292    final TableName tableName = TableName.valueOf(name.getMethodName());
293    byte[][] FAMILIES = new byte[][] { Bytes.toBytes("foo") };
294    if (TEST_UTIL.getAdmin().tableExists(tableName)) {
295      TEST_UTIL.getAdmin().disableTable(tableName);
296      TEST_UTIL.getAdmin().deleteTable(tableName);
297    }
298    try (Table htable = TEST_UTIL.createTable(tableName, FAMILIES);) {
299      byte[] row = "test".getBytes();
300      ConnectionImplementation c = ((ConnectionImplementation) TEST_UTIL.getConnection());
301      // check that metalookup pool would get created
302      c.relocateRegion(tableName, row);
303      ExecutorService ex = c.getCurrentMetaLookupPool();
304      assert(ex != null);
305    }
306  }
307
308  @Ignore @Test // Uses FSCK. Needs fixing after HBASE-14614.
309  public void testChangingReplicaCount() throws Exception {
310    // tests changing the replica count across master restarts
311    // reduce the replica count from 3 to 2
312    stopMasterAndValidateReplicaCount(3, 2);
313    // increase the replica count from 2 to 3
314    stopMasterAndValidateReplicaCount(2, 3);
315  }
316
317  private void stopMasterAndValidateReplicaCount(final int originalReplicaCount,
318      final int newReplicaCount)
319      throws Exception {
320    ServerName sn = TEST_UTIL.getHBaseClusterInterface().getClusterMetrics().getMasterName();
321    TEST_UTIL.getHBaseClusterInterface().stopMaster(sn);
322    TEST_UTIL.getHBaseClusterInterface().waitForMasterToStop(sn, 60000);
323    List<String> metaZnodes = TEST_UTIL.getZooKeeperWatcher().getMetaReplicaNodes();
324    assert(metaZnodes.size() == originalReplicaCount); //we should have what was configured before
325    TEST_UTIL.getHBaseClusterInterface().getConf().setInt(HConstants.META_REPLICAS_NUM,
326        newReplicaCount);
327    if (TEST_UTIL.getHBaseCluster().countServedRegions() < newReplicaCount) {
328      TEST_UTIL.getHBaseCluster().startRegionServer();
329    }
330    TEST_UTIL.getHBaseClusterInterface().startMaster(sn.getHostname(), 0);
331    TEST_UTIL.getHBaseClusterInterface().waitForActiveAndReadyMaster();
332    TEST_UTIL.waitFor(10000, predicateMetaHasReplicas(newReplicaCount));
333    // also check if hbck returns without errors
334    TEST_UTIL.getConfiguration().setInt(HConstants.META_REPLICAS_NUM,
335        newReplicaCount);
336    HBaseFsck hbck = HbckTestingUtil.doFsck(TEST_UTIL.getConfiguration(), false);
337    HbckTestingUtil.assertNoErrors(hbck);
338  }
339
340  private Waiter.ExplainingPredicate<Exception> predicateMetaHasReplicas(
341      final int newReplicaCount) {
342    return new Waiter.ExplainingPredicate<Exception>() {
343      @Override
344      public String explainFailure() throws Exception {
345        return checkMetaLocationAndExplain(newReplicaCount);
346      }
347
348      @Override
349      public boolean evaluate() throws Exception {
350        return checkMetaLocationAndExplain(newReplicaCount) == null;
351      }
352    };
353  }
354
355  @Nullable
356  private String checkMetaLocationAndExplain(int originalReplicaCount)
357      throws KeeperException, IOException {
358    List<String> metaZnodes = TEST_UTIL.getZooKeeperWatcher().getMetaReplicaNodes();
359    if (metaZnodes.size() == originalReplicaCount) {
360      RegionLocations rl = ((ClusterConnection) TEST_UTIL.getConnection())
361          .locateRegion(TableName.META_TABLE_NAME,
362              HConstants.EMPTY_START_ROW, false, false);
363      for (HRegionLocation location : rl.getRegionLocations()) {
364        if (location == null) {
365          return "Null location found in " + rl.toString();
366        }
367        if (location.getRegionInfo() == null) {
368          return "Null regionInfo for location " + location;
369        }
370        if (location.getHostname() == null) {
371          return "Null hostName for location " + location;
372        }
373      }
374      return null; // OK
375    }
376    return "Replica count is not as expected " + originalReplicaCount + " <> " + metaZnodes.size()
377        + "(" + metaZnodes.toString() + ")";
378  }
379
380  @Ignore @Test
381  public void testHBaseFsckWithMetaReplicas() throws Exception {
382    HBaseFsck hbck = HbckTestingUtil.doFsck(TEST_UTIL.getConfiguration(), false);
383    HbckTestingUtil.assertNoErrors(hbck);
384  }
385
386  @Ignore @Test // Disabled. Relies on FSCK which needs work for AMv2.
387  public void testHBaseFsckWithFewerMetaReplicas() throws Exception {
388    ClusterConnection c = (ClusterConnection)ConnectionFactory.createConnection(
389        TEST_UTIL.getConfiguration());
390    RegionLocations rl = c.locateRegion(TableName.META_TABLE_NAME, HConstants.EMPTY_START_ROW,
391        false, false);
392    HBaseFsckRepair.closeRegionSilentlyAndWait(c,
393        rl.getRegionLocation(1).getServerName(), rl.getRegionLocation(1).getRegionInfo());
394    // check that problem exists
395    HBaseFsck hbck = doFsck(TEST_UTIL.getConfiguration(), false);
396    assertErrors(hbck, new ERROR_CODE[]{ERROR_CODE.UNKNOWN,ERROR_CODE.NO_META_REGION});
397    // fix the problem
398    hbck = doFsck(TEST_UTIL.getConfiguration(), true);
399    // run hbck again to make sure we don't see any errors
400    hbck = doFsck(TEST_UTIL.getConfiguration(), false);
401    assertErrors(hbck, new ERROR_CODE[]{});
402  }
403
404  @Ignore @Test // The close silently doesn't work any more since HBASE-14614. Fix.
405  public void testHBaseFsckWithFewerMetaReplicaZnodes() throws Exception {
406    ClusterConnection c = (ClusterConnection)ConnectionFactory.createConnection(
407        TEST_UTIL.getConfiguration());
408    RegionLocations rl = c.locateRegion(TableName.META_TABLE_NAME, HConstants.EMPTY_START_ROW,
409        false, false);
410    HBaseFsckRepair.closeRegionSilentlyAndWait(c,
411        rl.getRegionLocation(2).getServerName(), rl.getRegionLocation(2).getRegionInfo());
412    ZKWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
413    ZKUtil.deleteNode(zkw, zkw.getZNodePaths().getZNodeForReplica(2));
414    // check that problem exists
415    HBaseFsck hbck = doFsck(TEST_UTIL.getConfiguration(), false);
416    assertErrors(hbck, new ERROR_CODE[]{ERROR_CODE.UNKNOWN,ERROR_CODE.NO_META_REGION});
417    // fix the problem
418    hbck = doFsck(TEST_UTIL.getConfiguration(), true);
419    // run hbck again to make sure we don't see any errors
420    hbck = doFsck(TEST_UTIL.getConfiguration(), false);
421    assertErrors(hbck, new ERROR_CODE[]{});
422  }
423
424  @Test
425  public void testAccessingUnknownTables() throws Exception {
426    Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
427    conf.setBoolean(HConstants.USE_META_REPLICAS, true);
428    Table table = TEST_UTIL.getConnection().getTable(TableName.valueOf(name.getMethodName()));
429    Get get = new Get(Bytes.toBytes("foo"));
430    try {
431      table.get(get);
432    } catch (TableNotFoundException t) {
433      return;
434    }
435    fail("Expected TableNotFoundException");
436  }
437
438  @Test
439  public void testMetaAddressChange() throws Exception {
440    // checks that even when the meta's location changes, the various
441    // caches update themselves. Uses the master operations to test
442    // this
443    Configuration conf = TEST_UTIL.getConfiguration();
444    ZKWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
445    String baseZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
446        HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
447    String primaryMetaZnode = ZNodePaths.joinZNode(baseZNode,
448        conf.get("zookeeper.znode.metaserver", "meta-region-server"));
449    // check that the data in the znode is parseable (this would also mean the znode exists)
450    byte[] data = ZKUtil.getData(zkw, primaryMetaZnode);
451    ServerName currentServer = ProtobufUtil.toServerName(data);
452    Collection<ServerName> liveServers = TEST_UTIL.getAdmin()
453        .getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics().keySet();
454    ServerName moveToServer = null;
455    for (ServerName s : liveServers) {
456      if (!currentServer.equals(s)) {
457        moveToServer = s;
458      }
459    }
460    assert(moveToServer != null);
461    final TableName tableName = TableName.valueOf(name.getMethodName());
462    TEST_UTIL.createTable(tableName, "f");
463    assertTrue(TEST_UTIL.getAdmin().tableExists(tableName));
464    TEST_UTIL.getAdmin().move(RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes(),
465        Bytes.toBytes(moveToServer.getServerName()));
466    int i = 0;
467    assert !moveToServer.equals(currentServer);
468    LOG.info("CurrentServer=" + currentServer + ", moveToServer=" + moveToServer);
469    final int max = 10000;
470    do {
471      Thread.sleep(10);
472      data = ZKUtil.getData(zkw, primaryMetaZnode);
473      currentServer = ProtobufUtil.toServerName(data);
474      i++;
475    } while (!moveToServer.equals(currentServer) && i < max); //wait for 10 seconds overall
476    assert(i != max);
477    TEST_UTIL.getAdmin().disableTable(tableName);
478    assertTrue(TEST_UTIL.getAdmin().isTableDisabled(tableName));
479  }
480
481  @Test
482  public void testShutdownOfReplicaHolder() throws Exception {
483    // checks that the when the server holding meta replica is shut down, the meta replica
484    // can be recovered
485    try (ClusterConnection conn = (ClusterConnection)
486        ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());) {
487      RegionLocations rl = conn.
488          locateRegion(TableName.META_TABLE_NAME, Bytes.toBytes(""), false, true);
489      HRegionLocation hrl = rl.getRegionLocation(1);
490      ServerName oldServer = hrl.getServerName();
491      TEST_UTIL.getHBaseClusterInterface().killRegionServer(oldServer);
492      int i = 0;
493      do {
494        LOG.debug("Waiting for the replica " + hrl.getRegionInfo() + " to come up");
495        Thread.sleep(10000); //wait for the detection/recovery
496        rl = conn.locateRegion(TableName.META_TABLE_NAME, Bytes.toBytes(""), false, true);
497        hrl = rl.getRegionLocation(1);
498        i++;
499      } while ((hrl == null || hrl.getServerName().equals(oldServer)) && i < 3);
500      assertTrue(i != 3);
501    }
502  }
503
504  @Ignore @Test // Disabled because fsck and this needs work for AMv2
505  public void testHBaseFsckWithExcessMetaReplicas() throws Exception {
506    // Create a meta replica (this will be the 4th one) and assign it
507    RegionInfo h = RegionReplicaUtil.getRegionInfoForReplica(
508        RegionInfoBuilder.FIRST_META_REGIONINFO, 3);
509    TEST_UTIL.assignRegion(h);
510    HBaseFsckRepair.waitUntilAssigned(TEST_UTIL.getAdmin(), h);
511    // check that problem exists
512    HBaseFsck hbck = doFsck(TEST_UTIL.getConfiguration(), false);
513    assertErrors(hbck, new ERROR_CODE[]{ERROR_CODE.UNKNOWN, ERROR_CODE.SHOULD_NOT_BE_DEPLOYED});
514    // fix the problem
515    hbck = doFsck(TEST_UTIL.getConfiguration(), true);
516    // run hbck again to make sure we don't see any errors
517    hbck = doFsck(TEST_UTIL.getConfiguration(), false);
518    assertErrors(hbck, new ERROR_CODE[]{});
519  }
520}