You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by te...@apache.org on 2016/07/05 20:19:39 UTC

hbase git commit: HBASE-16144 Revert - test failure in TestMultiSlaveReplication#testMultiSlaveReplication seems to be related

Repository: hbase
Updated Branches:
  refs/heads/master 2eef33930 -> 20a99b4c0


HBASE-16144 Revert - test failure in TestMultiSlaveReplication#testMultiSlaveReplication seems to be related


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/20a99b4c
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/20a99b4c
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/20a99b4c

Branch: refs/heads/master
Commit: 20a99b4c06ecb77c29c3ff173052a00174b9af8c
Parents: 2eef339
Author: tedyu <yu...@gmail.com>
Authored: Tue Jul 5 13:19:33 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Tue Jul 5 13:19:33 2016 -0700

----------------------------------------------------------------------
 .../hbase/replication/ReplicationFactory.java   |   4 +-
 .../replication/ReplicationQueuesZKImpl.java    |  13 +--
 .../org/apache/hadoop/hbase/master/HMaster.java |  16 ---
 .../cleaner/ReplicationZKLockCleanerChore.java  | 112 -------------------
 .../replication/TestMultiSlaveReplication.java  |  42 -------
 5 files changed, 2 insertions(+), 185 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/20a99b4c/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
index bc7a4ce..b84641c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
@@ -31,12 +31,10 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 @InterfaceAudience.Private
 public class ReplicationFactory {
 
-  public static final Class defaultReplicationQueueClass = ReplicationQueuesZKImpl.class;
-
   public static ReplicationQueues getReplicationQueues(ReplicationQueuesArguments args)
       throws Exception {
     Class<?> classToBuild = args.getConf().getClass("hbase.region.replica." +
-        "replication.replicationQueues.class", defaultReplicationQueueClass);
+        "replication.replicationQueues.class", ReplicationQueuesZKImpl.class);
     return (ReplicationQueues) ConstructorUtils.invokeConstructor(classToBuild, args);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/20a99b4c/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
index 655aaae..baea74f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
@@ -25,7 +25,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -239,8 +238,7 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
    * @param znode the server names of the other server
    * @return true if the lock was acquired, false in every other cases
    */
-  @VisibleForTesting
-  public boolean lockOtherRS(String znode) {
+  private boolean lockOtherRS(String znode) {
     try {
       String parent = ZKUtil.joinZNode(this.queuesZNode, znode);
       if (parent.equals(this.myQueuesZnode)) {
@@ -267,15 +265,6 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
     return true;
   }
 
-  public String getLockZNode(String znode) {
-    return this.queuesZNode + "/" + znode + "/" + RS_LOCK_ZNODE;
-  }
-
-  @VisibleForTesting
-  public boolean checkLockExists(String znode) throws KeeperException {
-    return ZKUtil.checkExists(zookeeper, getLockZNode(znode)) >= 0;
-  }
-
   /**
    * Delete all the replication queues for a given region server.
    * @param regionserverZnode The znode of the region server to delete.

http://git-wip-us.apache.org/repos/asf/hbase/blob/20a99b4c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 87daa16..a18a51f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -101,7 +101,6 @@ import org.apache.hadoop.hbase.master.balancer.ClusterStatusChore;
 import org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory;
 import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
 import org.apache.hadoop.hbase.master.cleaner.LogCleaner;
-import org.apache.hadoop.hbase.master.cleaner.ReplicationZKLockCleanerChore;
 import org.apache.hadoop.hbase.master.handler.DispatchMergingRegionHandler;
 import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan;
 import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
@@ -143,8 +142,6 @@ import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
 import org.apache.hadoop.hbase.regionserver.RegionSplitPolicy;
 import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy;
 import org.apache.hadoop.hbase.regionserver.compactions.FIFOCompactionPolicy;
-import org.apache.hadoop.hbase.replication.ReplicationFactory;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesZKImpl;
 import org.apache.hadoop.hbase.replication.master.TableCFsUpdater;
 import org.apache.hadoop.hbase.replication.regionserver.Replication;
 import org.apache.hadoop.hbase.security.User;
@@ -311,7 +308,6 @@ public class HMaster extends HRegionServer implements MasterServices {
   private PeriodicDoMetrics periodicDoMetricsChore = null;
 
   CatalogJanitor catalogJanitorChore;
-  private ReplicationZKLockCleanerChore replicationZKLockCleanerChore;
   private LogCleaner logCleaner;
   private HFileCleaner hfileCleaner;
   private ExpiredMobFileCleanerChore expiredMobFileCleanerChore;
@@ -1123,17 +1119,6 @@ public class HMaster extends HRegionServer implements MasterServices {
     if (LOG.isTraceEnabled()) {
       LOG.trace("Started service threads");
     }
-    if (conf.getClass("hbase.region.replica.replication.replicationQueues.class",
-        ReplicationFactory.defaultReplicationQueueClass) == ReplicationQueuesZKImpl.class && !conf
-        .getBoolean(HConstants.ZOOKEEPER_USEMULTI, true)) {
-      try {
-        replicationZKLockCleanerChore = new ReplicationZKLockCleanerChore(this, this,
-            cleanerInterval, this.getZooKeeper(), this.conf);
-        getChoreService().scheduleChore(replicationZKLockCleanerChore);
-      } catch (Exception e) {
-        LOG.error("start replicationZKLockCleanerChore failed", e);
-      }
-    }
   }
 
   @Override
@@ -1167,7 +1152,6 @@ public class HMaster extends HRegionServer implements MasterServices {
     // Clean up and close up shop
     if (this.logCleaner != null) this.logCleaner.cancel(true);
     if (this.hfileCleaner != null) this.hfileCleaner.cancel(true);
-    if (this.replicationZKLockCleanerChore != null) this.replicationZKLockCleanerChore.cancel(true);
     if (this.quotaManager != null) this.quotaManager.stop();
     if (this.activeMasterManager != null) this.activeMasterManager.stop();
     if (this.serverManager != null) this.serverManager.stop();

http://git-wip-us.apache.org/repos/asf/hbase/blob/20a99b4c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKLockCleanerChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKLockCleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKLockCleanerChore.java
deleted file mode 100644
index dc5338e..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKLockCleanerChore.java
+++ /dev/null
@@ -1,112 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.master.cleaner;
-
-import com.google.common.annotations.VisibleForTesting;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.ScheduledChore;
-import org.apache.hadoop.hbase.Stoppable;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.replication.ReplicationFactory;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesZKImpl;
-import org.apache.hadoop.hbase.replication.ReplicationTracker;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.data.Stat;
-
-
-/**
- * A cleaner that cleans replication locks on zk which is locked by dead region servers
- */
-@InterfaceAudience.Private
-public class ReplicationZKLockCleanerChore extends ScheduledChore {
-  private static final Log LOG = LogFactory.getLog(ReplicationZKLockCleanerChore.class);
-  private ZooKeeperWatcher zk;
-  private ReplicationTracker tracker;
-  private long ttl;
-  private ReplicationQueuesZKImpl queues;
-
-  // Wait some times before delete lock to prevent a session expired RS not dead fully.
-  private static final long DEFAULT_TTL = 60 * 10 * 1000;//10 min
-
-  @VisibleForTesting
-  public static final String TTL_CONFIG_KEY = "hbase.replication.zk.deadrs.lock.ttl";
-
-  public ReplicationZKLockCleanerChore(Stoppable stopper, Abortable abortable, int period,
-      ZooKeeperWatcher zk, Configuration conf) throws Exception {
-    super("ReplicationZKLockCleanerChore", stopper, period);
-
-    this.zk = zk;
-    this.ttl = conf.getLong(TTL_CONFIG_KEY, DEFAULT_TTL);
-    tracker = ReplicationFactory.getReplicationTracker(zk,
-        ReplicationFactory.getReplicationPeers(zk, conf, abortable), conf, abortable, stopper);
-    queues = new ReplicationQueuesZKImpl(zk, conf, abortable);
-  }
-
-  @Override protected void chore() {
-    try {
-      List<String> regionServers = tracker.getListOfRegionServers();
-      if (regionServers == null) {
-        return;
-      }
-      Set<String> rsSet = new HashSet<String>(regionServers);
-      List<String> replicators = queues.getListOfReplicators();
-
-      for (String replicator: replicators) {
-        try {
-          String lockNode = queues.getLockZNode(replicator);
-          byte[] data = ZKUtil.getData(zk, lockNode);
-          if (data == null) {
-            continue;
-          }
-          String rsServerNameZnode = Bytes.toString(data);
-          String[] array = rsServerNameZnode.split("/");
-          String znode = array[array.length - 1];
-          if (!rsSet.contains(znode)) {
-            Stat s = zk.getRecoverableZooKeeper().exists(lockNode, false);
-            if (s != null && EnvironmentEdgeManager.currentTime() - s.getMtime() > this.ttl) {
-              // server is dead, but lock is still there, we have to delete the lock.
-              ZKUtil.deleteNode(zk, lockNode);
-              LOG.info("Remove lock acquired by dead RS: " + lockNode + " by " + znode);
-            }
-            continue;
-          }
-          LOG.info("Skip lock acquired by live RS: " + lockNode + " by " + znode);
-
-        } catch (KeeperException.NoNodeException ignore) {
-        } catch (InterruptedException e) {
-          LOG.warn("zk operation interrupted", e);
-          Thread.currentThread().interrupt();
-        }
-      }
-    } catch (KeeperException e) {
-      LOG.warn("zk operation interrupted", e);
-    }
-
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/20a99b4c/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java
index 5d9ca9b..544006c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java
@@ -21,14 +21,11 @@ package org.apache.hadoop.hbase.replication;
 
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.util.Arrays;
-import java.util.List;
 import java.util.concurrent.CountDownLatch;
 
 import org.apache.commons.logging.Log;
@@ -39,15 +36,12 @@ import org.apache.hadoop.hbase.*;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-import org.apache.hadoop.hbase.master.cleaner.ReplicationZKLockCleanerChore;
 import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
@@ -98,12 +92,6 @@ public class TestMultiSlaveReplication {
     conf1.setLong(HConstants.THREAD_WAKE_FREQUENCY, 100);
     conf1.setStrings(CoprocessorHost.USER_REGION_COPROCESSOR_CONF_KEY,
         "org.apache.hadoop.hbase.replication.TestMasterReplication$CoprocessorCounter");
-    conf1.setBoolean(HConstants.ZOOKEEPER_USEMULTI , false);// for testZKLockCleaner
-    conf1.setInt("hbase.master.cleaner.interval", 5 * 1000);
-    conf1.setClass("hbase.region.replica.replication.replicationQueues.class",
-        ReplicationQueuesZKImpl.class, ReplicationQueues.class);
-    conf1.setLong(ReplicationZKLockCleanerChore.TTL_CONFIG_KEY, 0L);
-
 
     utility1 = new HBaseTestingUtility(conf1);
     utility1.startMiniZKCluster();
@@ -210,36 +198,6 @@ public class TestMultiSlaveReplication {
     utility1.shutdownMiniCluster();
   }
 
-  @Test
-  public void testZKLockCleaner() throws Exception {
-    MiniHBaseCluster cluster = utility1.startMiniCluster(1, 1);
-    HBaseAdmin admin = utility1.getHBaseAdmin();
-    admin.createTable(table);
-    ReplicationAdmin replicationAdmin = new ReplicationAdmin(conf1);
-    ReplicationPeerConfig rpc = new ReplicationPeerConfig();
-    rpc.setClusterKey(utility2.getClusterKey());
-    replicationAdmin.addPeer("2", rpc, null);
-    HRegionServer rs = cluster.getRegionServer(0);
-    ReplicationQueuesZKImpl zk = new ReplicationQueuesZKImpl(rs.getZooKeeper(), conf1, rs);
-    zk.init(rs.getServerName().toString());
-    List<String> replicators = zk.getListOfReplicators();
-    assertEquals(2, replicators.size());
-    String zNode = cluster.getMaster().getServerName().toString();
-
-    assertTrue(zk.lockOtherRS(zNode));
-    assertTrue(zk.checkLockExists(zNode));
-    Thread.sleep(10000);
-    assertTrue(zk.checkLockExists(zNode));
-    cluster.abortRegionServer(0);
-    Thread.sleep(10000);
-    HRegionServer rs1 = cluster.getMaster();
-    zk = new ReplicationQueuesZKImpl(rs1.getZooKeeper(), conf1, rs1);
-    zk.init(rs1.getServerName().toString());
-    assertFalse(zk.checkLockExists(zNode));
-
-    utility1.shutdownMiniCluster();
-  }
-
   private void rollWALAndWait(final HBaseTestingUtility utility, final TableName table,
       final byte[] row) throws IOException {
     final Admin admin = utility.getHBaseAdmin();