HBASE-8439 [replication] Remove ReplicationZookeeper class
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1507497 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
50df1e2fd7
commit
ff5fdb671f
|
@ -25,7 +25,9 @@ import org.apache.hadoop.hbase.Abortable;
|
|||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.client.HConnection;
|
||||
import org.apache.hadoop.hbase.client.HConnectionManager;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationZookeeper;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationFactory;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationPeers;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
|
@ -59,8 +61,9 @@ import java.util.Map;
|
|||
public class ReplicationAdmin implements Closeable {
|
||||
private static final Log LOG = LogFactory.getLog(ReplicationAdmin.class);
|
||||
|
||||
private final ReplicationZookeeper replicationZk;
|
||||
private final HConnection connection;
|
||||
private final ReplicationQueuesClient replicationQueuesClient;
|
||||
private final ReplicationPeers replicationPeers;
|
||||
|
||||
/**
|
||||
* Constructor that creates a connection to the local ZooKeeper ensemble.
|
||||
|
@ -76,7 +79,12 @@ public class ReplicationAdmin implements Closeable {
|
|||
this.connection = HConnectionManager.getConnection(conf);
|
||||
ZooKeeperWatcher zkw = createZooKeeperWatcher();
|
||||
try {
|
||||
this.replicationZk = new ReplicationZookeeper(this.connection, conf, zkw);
|
||||
this.replicationPeers = ReplicationFactory.getReplicationPeers(zkw, conf, this.connection);
|
||||
this.replicationPeers.init();
|
||||
this.replicationQueuesClient =
|
||||
ReplicationFactory.getReplicationQueuesClient(zkw, conf, this.connection);
|
||||
this.replicationQueuesClient.init();
|
||||
|
||||
} catch (KeeperException e) {
|
||||
throw new IOException("Unable setup the ZooKeeper connection", e);
|
||||
}
|
||||
|
@ -109,7 +117,7 @@ public class ReplicationAdmin implements Closeable {
|
|||
* multi-slave isn't supported yet.
|
||||
*/
|
||||
public void addPeer(String id, String clusterKey) throws IOException {
|
||||
this.replicationZk.addPeer(id, clusterKey);
|
||||
this.replicationPeers.addPeer(id, clusterKey);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -117,7 +125,7 @@ public class ReplicationAdmin implements Closeable {
|
|||
* @param id a short that identifies the cluster
|
||||
*/
|
||||
public void removePeer(String id) throws IOException {
|
||||
this.replicationZk.removePeer(id);
|
||||
this.replicationPeers.removePeer(id);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -125,7 +133,7 @@ public class ReplicationAdmin implements Closeable {
|
|||
* @param id a short that identifies the cluster
|
||||
*/
|
||||
public void enablePeer(String id) throws IOException {
|
||||
this.replicationZk.enablePeer(id);
|
||||
this.replicationPeers.enablePeer(id);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -133,7 +141,7 @@ public class ReplicationAdmin implements Closeable {
|
|||
* @param id a short that identifies the cluster
|
||||
*/
|
||||
public void disablePeer(String id) throws IOException {
|
||||
this.replicationZk.disablePeer(id);
|
||||
this.replicationPeers.disablePeer(id);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -141,7 +149,7 @@ public class ReplicationAdmin implements Closeable {
|
|||
* @return number of slave clusters
|
||||
*/
|
||||
public int getPeersCount() {
|
||||
return this.replicationZk.listPeersIdsAndWatch().size();
|
||||
return this.replicationPeers.getAllPeerIds().size();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -149,15 +157,7 @@ public class ReplicationAdmin implements Closeable {
|
|||
* @return A map of peer ids to peer cluster keys
|
||||
*/
|
||||
public Map<String, String> listPeers() {
|
||||
return this.replicationZk.listPeers();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the ZK-support tool created and used by this object for replication.
|
||||
* @return the ZK-support tool
|
||||
*/
|
||||
ReplicationZookeeper getReplicationZk() {
|
||||
return replicationZk;
|
||||
return this.replicationPeers.getAllPeerClusterKeys();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -0,0 +1,51 @@
|
|||
/*
|
||||
*
|
||||
* 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.replication;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Abortable;
|
||||
import org.apache.hadoop.hbase.Stoppable;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
|
||||
/**
|
||||
* A factory class for instantiating replication objects that deal with replication state.
|
||||
*/
|
||||
public class ReplicationFactory {
|
||||
|
||||
public static ReplicationQueues getReplicationQueues(final ZooKeeperWatcher zk,
|
||||
Configuration conf, Abortable abortable) {
|
||||
return new ReplicationQueuesZKImpl(zk, conf, abortable);
|
||||
}
|
||||
|
||||
public static ReplicationQueuesClient getReplicationQueuesClient(final ZooKeeperWatcher zk,
|
||||
Configuration conf, Abortable abortable) {
|
||||
return new ReplicationQueuesClientZKImpl(zk, conf, abortable);
|
||||
}
|
||||
|
||||
public static ReplicationPeers getReplicationPeers(final ZooKeeperWatcher zk, Configuration conf,
|
||||
Abortable abortable) {
|
||||
return new ReplicationPeersZKImpl(zk, conf, abortable);
|
||||
}
|
||||
|
||||
public static ReplicationTracker getReplicationTracker(ZooKeeperWatcher zookeeper,
|
||||
final ReplicationPeers replicationPeers, Configuration conf, Abortable abortable,
|
||||
Stoppable stopper) {
|
||||
return new ReplicationTrackerZKImpl(zookeeper, replicationPeers, conf, abortable, stopper);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,51 @@
|
|||
/*
|
||||
*
|
||||
* 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.replication;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
||||
/**
|
||||
* The replication listener interface can be implemented if a class needs to subscribe to events
|
||||
* generated by the ReplicationTracker. These events include things like addition/deletion of peer
|
||||
* clusters or failure of a local region server. To receive events, the class also needs to register
|
||||
* itself with a Replication Tracker.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public interface ReplicationListener {
|
||||
|
||||
/**
|
||||
* A region server has been removed from the local cluster
|
||||
* @param regionServer the removed region server
|
||||
*/
|
||||
public void regionServerRemoved(String regionServer);
|
||||
|
||||
/**
|
||||
* A peer cluster has been removed (i.e. unregistered) from replication.
|
||||
* @param peerId The peer id of the cluster that has been removed
|
||||
*/
|
||||
public void peerRemoved(String peerId);
|
||||
|
||||
/**
|
||||
* The list of registered peer clusters has changed.
|
||||
* @param peerIds A list of all currently registered peer clusters
|
||||
*/
|
||||
public void peerListChanged(List<String> peerIds);
|
||||
}
|
|
@ -20,12 +20,19 @@ package org.apache.hadoop.hbase.replication;
|
|||
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
/**
|
||||
* This provides an interface for clients of replication to view replication queues. These queues
|
||||
* keep track of the HLogs that still need to be replicated to remote clusters.
|
||||
*/
|
||||
public interface ReplicationQueuesClient {
|
||||
|
||||
/**
|
||||
* Initialize the replication queue client interface.
|
||||
*/
|
||||
public void init() throws KeeperException;
|
||||
|
||||
/**
|
||||
* Get a list of all region servers that have outstanding replication queues. These servers could
|
||||
* be alive, dead or from a previous run of the cluster.
|
||||
|
|
|
@ -30,8 +30,12 @@ public class ReplicationQueuesClientZKImpl extends ReplicationStateZKBase implem
|
|||
ReplicationQueuesClient {
|
||||
|
||||
public ReplicationQueuesClientZKImpl(final ZooKeeperWatcher zk, Configuration conf,
|
||||
Abortable abortable) throws KeeperException {
|
||||
Abortable abortable) {
|
||||
super(zk, conf, abortable);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void init() throws KeeperException {
|
||||
ZKUtil.createWithParents(this.zookeeper, this.queuesZNode);
|
||||
}
|
||||
|
||||
|
|
|
@ -95,7 +95,16 @@ public abstract class ReplicationStateZKBase {
|
|||
return ProtobufUtil.prependPBMagic(bytes);
|
||||
}
|
||||
|
||||
public boolean peerExists(String id) throws KeeperException {
|
||||
protected boolean peerExists(String id) throws KeeperException {
|
||||
return ZKUtil.checkExists(this.zookeeper, ZKUtil.joinZNode(this.peersZNode, id)) >= 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* Determine if a ZK path points to a peer node.
|
||||
* @param path path to be checked
|
||||
* @return true if the path points to a peer node, otherwise false
|
||||
*/
|
||||
protected boolean isPeerPath(String path) {
|
||||
return path.split("/").length == peersZNode.split("/").length + 1;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,49 @@
|
|||
/*
|
||||
*
|
||||
* 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.replication;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* This is the interface for a Replication Tracker. A replication tracker provides the facility to
|
||||
* subscribe and track events that reflect a change in replication state. These events are used by
|
||||
* the ReplicationSourceManager to coordinate replication tasks such as addition/deletion of queues
|
||||
* and queue failover. These events are defined in the ReplicationListener interface. If a class
|
||||
* would like to listen to replication events it must implement the ReplicationListener interface
|
||||
* and register itself with a Replication Tracker.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public interface ReplicationTracker {
|
||||
|
||||
/**
|
||||
* Register a replication listener to receive replication events.
|
||||
* @param listener
|
||||
*/
|
||||
public void registerListener(ReplicationListener listener);
|
||||
|
||||
public void removeListener(ReplicationListener listener);
|
||||
|
||||
/**
|
||||
* Returns a list of other live region servers in the cluster.
|
||||
* @return
|
||||
*/
|
||||
public List<String> getListOfRegionServers();
|
||||
}
|
|
@ -0,0 +1,249 @@
|
|||
/*
|
||||
*
|
||||
* 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.replication;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
|
||||
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.Stoppable;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
/**
|
||||
* This class is a Zookeeper implementation of the ReplicationTracker interface. This class is
|
||||
* responsible for handling replication events that are defined in the ReplicationListener
|
||||
* interface.
|
||||
*/
|
||||
public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements ReplicationTracker {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(ReplicationTrackerZKImpl.class);
|
||||
// All about stopping
|
||||
private final Stoppable stopper;
|
||||
// listeners to be notified
|
||||
private final List<ReplicationListener> listeners =
|
||||
new CopyOnWriteArrayList<ReplicationListener>();
|
||||
// List of all the other region servers in this cluster
|
||||
private final ArrayList<String> otherRegionServers = new ArrayList<String>();
|
||||
private final ReplicationPeers replicationPeers;
|
||||
|
||||
public ReplicationTrackerZKImpl(ZooKeeperWatcher zookeeper,
|
||||
final ReplicationPeers replicationPeers, Configuration conf, Abortable abortable,
|
||||
Stoppable stopper) {
|
||||
super(zookeeper, conf, abortable);
|
||||
this.replicationPeers = replicationPeers;
|
||||
this.stopper = stopper;
|
||||
this.zookeeper.registerListener(new OtherRegionServerWatcher(this.zookeeper));
|
||||
this.zookeeper.registerListener(new PeersWatcher(this.zookeeper));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void registerListener(ReplicationListener listener) {
|
||||
listeners.add(listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void removeListener(ReplicationListener listener) {
|
||||
listeners.remove(listener);
|
||||
}
|
||||
|
||||
/**
|
||||
* Return a snapshot of the current region servers.
|
||||
*/
|
||||
@Override
|
||||
public List<String> getListOfRegionServers() {
|
||||
refreshOtherRegionServersList();
|
||||
|
||||
List<String> list = null;
|
||||
synchronized (otherRegionServers) {
|
||||
list = new ArrayList<String>(otherRegionServers);
|
||||
}
|
||||
return list;
|
||||
}
|
||||
|
||||
/**
|
||||
* Watcher used to be notified of the other region server's death in the local cluster. It
|
||||
* initiates the process to transfer the queues if it is able to grab the lock.
|
||||
*/
|
||||
public class OtherRegionServerWatcher extends ZooKeeperListener {
|
||||
|
||||
/**
|
||||
* Construct a ZooKeeper event listener.
|
||||
*/
|
||||
public OtherRegionServerWatcher(ZooKeeperWatcher watcher) {
|
||||
super(watcher);
|
||||
}
|
||||
|
||||
/**
|
||||
* Called when a new node has been created.
|
||||
* @param path full path of the new node
|
||||
*/
|
||||
public void nodeCreated(String path) {
|
||||
refreshListIfRightPath(path);
|
||||
}
|
||||
|
||||
/**
|
||||
* Called when a node has been deleted
|
||||
* @param path full path of the deleted node
|
||||
*/
|
||||
public void nodeDeleted(String path) {
|
||||
if (stopper.isStopped()) {
|
||||
return;
|
||||
}
|
||||
boolean cont = refreshListIfRightPath(path);
|
||||
if (!cont) {
|
||||
return;
|
||||
}
|
||||
LOG.info(path + " znode expired, triggering replicatorRemoved event");
|
||||
for (ReplicationListener rl : listeners) {
|
||||
rl.regionServerRemoved(getZNodeName(path));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Called when an existing node has a child node added or removed.
|
||||
* @param path full path of the node whose children have changed
|
||||
*/
|
||||
public void nodeChildrenChanged(String path) {
|
||||
if (stopper.isStopped()) {
|
||||
return;
|
||||
}
|
||||
refreshListIfRightPath(path);
|
||||
}
|
||||
|
||||
private boolean refreshListIfRightPath(String path) {
|
||||
if (!path.startsWith(this.watcher.rsZNode)) {
|
||||
return false;
|
||||
}
|
||||
return refreshOtherRegionServersList();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Watcher used to follow the creation and deletion of peer clusters.
|
||||
*/
|
||||
public class PeersWatcher extends ZooKeeperListener {
|
||||
|
||||
/**
|
||||
* Construct a ZooKeeper event listener.
|
||||
*/
|
||||
public PeersWatcher(ZooKeeperWatcher watcher) {
|
||||
super(watcher);
|
||||
}
|
||||
|
||||
/**
|
||||
* Called when a node has been deleted
|
||||
* @param path full path of the deleted node
|
||||
*/
|
||||
public void nodeDeleted(String path) {
|
||||
List<String> peers = refreshPeersList(path);
|
||||
if (peers == null) {
|
||||
return;
|
||||
}
|
||||
if (isPeerPath(path)) {
|
||||
String id = getZNodeName(path);
|
||||
LOG.info(path + " znode expired, triggering peerRemoved event");
|
||||
for (ReplicationListener rl : listeners) {
|
||||
rl.peerRemoved(id);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Called when an existing node has a child node added or removed.
|
||||
* @param path full path of the node whose children have changed
|
||||
*/
|
||||
public void nodeChildrenChanged(String path) {
|
||||
List<String> peers = refreshPeersList(path);
|
||||
if (peers == null) {
|
||||
return;
|
||||
}
|
||||
LOG.info(path + " znode expired, triggering peerListChanged event");
|
||||
for (ReplicationListener rl : listeners) {
|
||||
rl.peerListChanged(peers);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Verify if this event is meant for us, and if so then get the latest peers' list from ZK. Also
|
||||
* reset the watches.
|
||||
* @param path path to check against
|
||||
* @return A list of peers' identifiers if the event concerns this watcher, else null.
|
||||
*/
|
||||
private List<String> refreshPeersList(String path) {
|
||||
if (!path.startsWith(getPeersZNode())) {
|
||||
return null;
|
||||
}
|
||||
return this.replicationPeers.getAllPeerIds();
|
||||
}
|
||||
|
||||
private String getPeersZNode() {
|
||||
return this.peersZNode;
|
||||
}
|
||||
|
||||
/**
|
||||
* Extracts the znode name of a peer cluster from a ZK path
|
||||
* @param fullPath Path to extract the id from
|
||||
* @return the id or an empty string if path is invalid
|
||||
*/
|
||||
private String getZNodeName(String fullPath) {
|
||||
String[] parts = fullPath.split("/");
|
||||
return parts.length > 0 ? parts[parts.length - 1] : "";
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads the list of region servers from ZK and atomically clears our local view of it and
|
||||
* replaces it with the updated list.
|
||||
* @return true if the local list of the other region servers was updated with the ZK data (even
|
||||
* if it was empty), false if the data was missing in ZK
|
||||
*/
|
||||
private boolean refreshOtherRegionServersList() {
|
||||
List<String> newRsList = getRegisteredRegionServers();
|
||||
if (newRsList == null) {
|
||||
return false;
|
||||
} else {
|
||||
synchronized (otherRegionServers) {
|
||||
otherRegionServers.clear();
|
||||
otherRegionServers.addAll(newRsList);
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get a list of all the other region servers in this cluster and set a watch
|
||||
* @return a list of server nanes
|
||||
*/
|
||||
private List<String> getRegisteredRegionServers() {
|
||||
List<String> result = null;
|
||||
try {
|
||||
result = ZKUtil.listChildrenAndWatchThem(this.zookeeper, this.zookeeper.rsZNode);
|
||||
} catch (KeeperException e) {
|
||||
this.abortable.abort("Get list of registered region servers", e);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
}
|
|
@ -1,366 +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.replication;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Abortable;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.SortedMap;
|
||||
import java.util.SortedSet;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
/**
|
||||
* This class serves as a helper for all things related to zookeeper in
|
||||
* replication.
|
||||
* <p/>
|
||||
* The layout looks something like this under zookeeper.znode.parent for the
|
||||
* master cluster:
|
||||
* <p/>
|
||||
*
|
||||
* <pre>
|
||||
* replication/
|
||||
* state {contains true or false}
|
||||
* clusterId {contains a byte}
|
||||
* peers/
|
||||
* 1/ {contains a full cluster address}
|
||||
* peer-state {contains ENABLED or DISABLED}
|
||||
* 2/
|
||||
* ...
|
||||
* rs/ {lists all RS that replicate}
|
||||
* startcode1/ {lists all peer clusters}
|
||||
* 1/ {lists hlogs to process}
|
||||
* 10.10.1.76%3A53488.123456789 {contains nothing or a position}
|
||||
* 10.10.1.76%3A53488.123456790
|
||||
* ...
|
||||
* 2/
|
||||
* ...
|
||||
* startcode2/
|
||||
* ...
|
||||
* </pre>
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class ReplicationZookeeper extends ReplicationStateZKBase {
|
||||
private static final Log LOG = LogFactory.getLog(ReplicationZookeeper.class);
|
||||
|
||||
// Our handle on zookeeper
|
||||
private final ZooKeeperWatcher zookeeper;
|
||||
private String peersZNode;
|
||||
private final Configuration conf;
|
||||
// Abortable
|
||||
private Abortable abortable;
|
||||
private final ReplicationPeers replicationPeers;
|
||||
private final ReplicationQueues replicationQueues;
|
||||
|
||||
/**
|
||||
* Constructor used by clients of replication (like master and HBase clients)
|
||||
* @param conf conf to use
|
||||
* @param zk zk connection to use
|
||||
* @throws IOException
|
||||
*/
|
||||
public ReplicationZookeeper(final Abortable abortable, final Configuration conf,
|
||||
final ZooKeeperWatcher zk) throws KeeperException, IOException {
|
||||
super(zk, conf, abortable);
|
||||
this.conf = conf;
|
||||
this.zookeeper = zk;
|
||||
setZNodes(abortable);
|
||||
// TODO This interface is no longer used by anyone using this constructor. When this class goes
|
||||
// away, we will no longer have this null initialization business
|
||||
this.replicationQueues = null;
|
||||
this.replicationPeers = new ReplicationPeersZKImpl(this.zookeeper, this.conf, abortable);
|
||||
this.replicationPeers.init();
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructor used by region servers, connects to the peer cluster right away.
|
||||
*
|
||||
* @param server
|
||||
* @throws IOException
|
||||
* @throws KeeperException
|
||||
*/
|
||||
public ReplicationZookeeper(final Server server)
|
||||
throws IOException, KeeperException {
|
||||
super(server.getZooKeeper(), server.getConfiguration(), server);
|
||||
this.abortable = server;
|
||||
this.zookeeper = server.getZooKeeper();
|
||||
this.conf = server.getConfiguration();
|
||||
setZNodes(server);
|
||||
this.replicationQueues = new ReplicationQueuesZKImpl(this.zookeeper, this.conf, server);
|
||||
this.replicationQueues.init(server.getServerName().toString());
|
||||
this.replicationPeers = new ReplicationPeersZKImpl(this.zookeeper, this.conf, server);
|
||||
this.replicationPeers.init();
|
||||
}
|
||||
|
||||
private void setZNodes(Abortable abortable) throws KeeperException {
|
||||
String replicationZNodeName = conf.get("zookeeper.znode.replication", "replication");
|
||||
String peersZNodeName = conf.get("zookeeper.znode.replication.peers", "peers");
|
||||
String replicationZNode = ZKUtil.joinZNode(this.zookeeper.baseZNode, replicationZNodeName);
|
||||
this.peersZNode = ZKUtil.joinZNode(replicationZNode, peersZNodeName);
|
||||
}
|
||||
|
||||
/**
|
||||
* List this cluster's peers' IDs
|
||||
* @return list of all peers' identifiers
|
||||
*/
|
||||
public List<String> listPeersIdsAndWatch() {
|
||||
return this.replicationPeers.getAllPeerIds();
|
||||
}
|
||||
|
||||
/**
|
||||
* Map of this cluster's peers for display.
|
||||
* @return A map of peer ids to peer cluster keys
|
||||
*/
|
||||
public Map<String, String> listPeers() {
|
||||
return this.replicationPeers.getAllPeerClusterKeys();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns all region servers from given peer
|
||||
*
|
||||
* @param peerClusterId (byte) the cluster to interrogate
|
||||
* @return addresses of all region servers
|
||||
*/
|
||||
public List<ServerName> getSlavesAddresses(String peerClusterId) {
|
||||
return this.replicationPeers.getRegionServersOfConnectedPeer(peerClusterId);
|
||||
}
|
||||
|
||||
/**
|
||||
* This method connects this cluster to another one and registers it
|
||||
* in this region server's replication znode
|
||||
* @param peerId id of the peer cluster
|
||||
* @throws KeeperException
|
||||
*/
|
||||
public boolean connectToPeer(String peerId) throws IOException, KeeperException {
|
||||
return this.replicationPeers.connectToPeer(peerId);
|
||||
}
|
||||
|
||||
/**
|
||||
* Remove the peer from zookeeper. which will trigger the watchers on every
|
||||
* region server and close their sources
|
||||
* @param id
|
||||
* @throws IllegalArgumentException Thrown when the peer doesn't exist
|
||||
*/
|
||||
public void removePeer(String id) throws IOException {
|
||||
this.replicationPeers.removePeer(id);
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a new peer to this cluster
|
||||
* @param id peer's identifier
|
||||
* @param clusterKey ZK ensemble's addresses, client port and root znode
|
||||
* @throws IllegalArgumentException Thrown when the peer doesn't exist
|
||||
* @throws IllegalStateException Thrown when a peer already exists, since
|
||||
* multi-slave isn't supported yet.
|
||||
*/
|
||||
public void addPeer(String id, String clusterKey) throws IOException {
|
||||
this.replicationPeers.addPeer(id, clusterKey);
|
||||
}
|
||||
|
||||
/**
|
||||
* Enable replication to the peer
|
||||
*
|
||||
* @param id peer's identifier
|
||||
* @throws IllegalArgumentException
|
||||
* Thrown when the peer doesn't exist
|
||||
*/
|
||||
public void enablePeer(String id) throws IOException {
|
||||
this.replicationPeers.enablePeer(id);
|
||||
}
|
||||
|
||||
/**
|
||||
* Disable replication to the peer
|
||||
*
|
||||
* @param id peer's identifier
|
||||
* @throws IllegalArgumentException
|
||||
* Thrown when the peer doesn't exist
|
||||
*/
|
||||
public void disablePeer(String id) throws IOException {
|
||||
this.replicationPeers.disablePeer(id);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check whether the peer is enabled or not. This method checks the atomic
|
||||
* boolean of ReplicationPeer locally.
|
||||
*
|
||||
* @param id peer identifier
|
||||
* @return true if the peer is enabled, otherwise false
|
||||
* @throws IllegalArgumentException
|
||||
* Thrown when the peer doesn't exist
|
||||
*/
|
||||
public boolean getPeerEnabled(String id) {
|
||||
return this.replicationPeers.getStatusOfConnectedPeer(id);
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a new log to the list of hlogs in zookeeper
|
||||
* @param filename name of the hlog's znode
|
||||
* @param peerId name of the cluster's znode
|
||||
*/
|
||||
public void addLogToList(String filename, String peerId) throws KeeperException {
|
||||
this.replicationQueues.addLog(peerId, filename);
|
||||
}
|
||||
|
||||
/**
|
||||
* Remove a log from the list of hlogs in zookeeper
|
||||
* @param filename name of the hlog's znode
|
||||
* @param clusterId name of the cluster's znode
|
||||
*/
|
||||
public void removeLogFromList(String filename, String clusterId) {
|
||||
this.replicationQueues.removeLog(clusterId, filename);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the current position of the specified cluster in the current hlog
|
||||
* @param filename filename name of the hlog's znode
|
||||
* @param clusterId clusterId name of the cluster's znode
|
||||
* @param position the position in the file
|
||||
*/
|
||||
public void writeReplicationStatus(String filename, String clusterId, long position) {
|
||||
this.replicationQueues.setLogPosition(clusterId, filename, position);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get a list of all the other region servers in this cluster
|
||||
* and set a watch
|
||||
* @return a list of server nanes
|
||||
*/
|
||||
public List<String> getRegisteredRegionServers() {
|
||||
List<String> result = null;
|
||||
try {
|
||||
result = ZKUtil.listChildrenAndWatchThem(
|
||||
this.zookeeper, this.zookeeper.rsZNode);
|
||||
} catch (KeeperException e) {
|
||||
this.abortable.abort("Get list of registered region servers", e);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Take ownership for the set of queues belonging to a dead region server.
|
||||
* @param regionserver the id of the dead region server
|
||||
* @return A SortedMap of the queues that have been claimed, including a SortedSet of HLogs in
|
||||
* each queue.
|
||||
*/
|
||||
public SortedMap<String, SortedSet<String>> claimQueues(String regionserver) {
|
||||
return this.replicationQueues.claimQueues(regionserver);
|
||||
}
|
||||
|
||||
/**
|
||||
* Delete a complete queue of hlogs
|
||||
* @param peerZnode znode of the peer cluster queue of hlogs to delete
|
||||
*/
|
||||
public void deleteSource(String peerZnode, boolean closeConnection) {
|
||||
this.replicationQueues.removeQueue(peerZnode);
|
||||
if (closeConnection) {
|
||||
this.replicationPeers.disconnectFromPeer(peerZnode);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Delete this cluster's queues
|
||||
*/
|
||||
public void deleteOwnRSZNode() {
|
||||
this.replicationQueues.removeAllQueues();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the position of the specified hlog in the specified peer znode
|
||||
* @param peerId znode of the peer cluster
|
||||
* @param hlog name of the hlog
|
||||
* @return the position in that hlog
|
||||
* @throws KeeperException
|
||||
*/
|
||||
public long getHLogRepPosition(String peerId, String hlog) throws KeeperException {
|
||||
return this.replicationQueues.getLogPosition(peerId, hlog);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the UUID of the provided peer id. Should a connection loss or session
|
||||
* expiration happen, the ZK handler will be reopened once and if it still doesn't
|
||||
* work then it will bail and return null.
|
||||
* @param peerId the peer's ID that will be converted into a UUID
|
||||
* @return a UUID or null if there's a ZK connection issue
|
||||
*/
|
||||
public UUID getPeerUUID(String peerId) {
|
||||
return this.replicationPeers.getPeerUUID(peerId);
|
||||
}
|
||||
|
||||
public void registerRegionServerListener(ZooKeeperListener listener) {
|
||||
this.zookeeper.registerListener(listener);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get a map of all peer clusters
|
||||
* @return map of peer cluster keyed by id
|
||||
*/
|
||||
public Set<String> getPeerClusters() {
|
||||
return this.replicationPeers.getConnectedPeers();
|
||||
}
|
||||
|
||||
/**
|
||||
* Determine if a ZK path points to a peer node.
|
||||
* @param path path to be checked
|
||||
* @return true if the path points to a peer node, otherwise false
|
||||
*/
|
||||
public boolean isPeerPath(String path) {
|
||||
return path.split("/").length == peersZNode.split("/").length + 1;
|
||||
}
|
||||
|
||||
/**
|
||||
* Extracts the znode name of a peer cluster from a ZK path
|
||||
* @param fullPath Path to extract the id from
|
||||
* @return the id or an empty string if path is invalid
|
||||
*/
|
||||
public static String getZNodeName(String fullPath) {
|
||||
String[] parts = fullPath.split("/");
|
||||
return parts.length > 0 ? parts[parts.length-1] : "";
|
||||
}
|
||||
|
||||
/**
|
||||
* Get this cluster's zk connection
|
||||
* @return zk connection
|
||||
*/
|
||||
public ZooKeeperWatcher getZookeeperWatcher() {
|
||||
return this.zookeeper;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Get the full path to the peers' znode
|
||||
* @return path to peers in zk
|
||||
*/
|
||||
public String getPeersZNode() {
|
||||
return peersZNode;
|
||||
}
|
||||
}
|
|
@ -36,8 +36,6 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.HConnection;
|
||||
import org.apache.hadoop.hbase.client.HConnectionManager;
|
||||
import org.apache.hadoop.hbase.client.HTable;
|
||||
import org.apache.hadoop.hbase.client.Mutation;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
|
@ -45,8 +43,6 @@ import org.apache.hadoop.hbase.client.Result;
|
|||
import org.apache.hadoop.hbase.exceptions.ZooKeeperConnectionException;
|
||||
import org.apache.hadoop.hbase.filter.Filter;
|
||||
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationPeersZKImpl;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationZookeeper;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
|
|
|
@ -36,10 +36,10 @@ import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
|||
import org.apache.hadoop.hbase.mapreduce.TableInputFormat;
|
||||
import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
|
||||
import org.apache.hadoop.hbase.mapreduce.TableMapper;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationFactory;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationPeer;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationPeers;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationPeersZKImpl;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationZookeeper;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.hadoop.mapreduce.Job;
|
||||
|
@ -118,7 +118,8 @@ public class VerifyReplication {
|
|||
@Override public void abort(String why, Throwable e) {}
|
||||
@Override public boolean isAborted() {return false;}
|
||||
});
|
||||
ReplicationPeers rp = new ReplicationPeersZKImpl(localZKW, conf, localZKW);
|
||||
ReplicationPeers rp =
|
||||
ReplicationFactory.getReplicationPeers(localZKW, conf, localZKW);
|
||||
rp.init();
|
||||
Configuration peerConf = rp.getPeerConf(peerId);
|
||||
if (peerConf == null) {
|
||||
|
|
|
@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.Abortable;
|
|||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.client.HConnectionManager;
|
||||
import org.apache.hadoop.hbase.master.cleaner.BaseLogCleanerDelegate;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationFactory;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationQueuesClientZKImpl;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
|
@ -122,7 +123,8 @@ public class ReplicationLogCleaner extends BaseLogCleanerDelegate implements Abo
|
|||
super.setConf(conf);
|
||||
try {
|
||||
this.zkw = new ZooKeeperWatcher(conf, "replicationLogCleaner", null);
|
||||
this.replicationQueues = new ReplicationQueuesClientZKImpl(zkw, conf, this);
|
||||
this.replicationQueues = ReplicationFactory.getReplicationQueuesClient(zkw, conf, this);
|
||||
this.replicationQueues.init();
|
||||
} catch (KeeperException e) {
|
||||
LOG.error("Error while configuring " + this.getClass().getName(), e);
|
||||
} catch (IOException e) {
|
||||
|
|
|
@ -22,10 +22,10 @@ import java.io.IOException;
|
|||
import java.util.List;
|
||||
import java.util.NavigableMap;
|
||||
import java.util.TreeMap;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
import org.apache.commons.logging.Log;
|
||||
|
@ -45,11 +45,13 @@ import org.apache.hadoop.hbase.regionserver.ReplicationSinkService;
|
|||
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationFactory;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationPeers;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationQueues;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationQueuesZKImpl;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationZookeeper;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationTracker;
|
||||
import org.apache.hadoop.hbase.replication.master.ReplicationLogCleaner;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
import static org.apache.hadoop.hbase.HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS;
|
||||
|
@ -66,8 +68,9 @@ public class Replication implements WALActionsListener,
|
|||
LogFactory.getLog(Replication.class);
|
||||
private boolean replication;
|
||||
private ReplicationSourceManager replicationManager;
|
||||
private ReplicationZookeeper zkHelper;
|
||||
private ReplicationQueues replicationQueues;
|
||||
private ReplicationPeers replicationPeers;
|
||||
private ReplicationTracker replicationTracker;
|
||||
private Configuration conf;
|
||||
private ReplicationSink replicationSink;
|
||||
// Hosting server
|
||||
|
@ -107,23 +110,35 @@ public class Replication implements WALActionsListener,
|
|||
.build());
|
||||
if (replication) {
|
||||
try {
|
||||
this.zkHelper = new ReplicationZookeeper(server);
|
||||
this.replicationQueues =
|
||||
new ReplicationQueuesZKImpl(server.getZooKeeper(), this.conf, this.server);
|
||||
ReplicationFactory.getReplicationQueues(server.getZooKeeper(), this.conf, this.server);
|
||||
this.replicationQueues.init(this.server.getServerName().toString());
|
||||
this.replicationPeers =
|
||||
ReplicationFactory.getReplicationPeers(server.getZooKeeper(), this.conf, this.server);
|
||||
this.replicationPeers.init();
|
||||
this.replicationTracker =
|
||||
ReplicationFactory.getReplicationTracker(server.getZooKeeper(), this.replicationPeers,
|
||||
this.conf, this.server, this.server);
|
||||
} catch (KeeperException ke) {
|
||||
throw new IOException("Failed replication handler create", ke);
|
||||
}
|
||||
UUID clusterId = null;
|
||||
try {
|
||||
clusterId = ZKClusterId.getUUIDForCluster(this.server.getZooKeeper());
|
||||
} catch (KeeperException ke) {
|
||||
throw new IOException("Could not read cluster id", ke);
|
||||
}
|
||||
this.replicationManager =
|
||||
new ReplicationSourceManager(zkHelper, replicationQueues, conf, this.server, fs, logDir,
|
||||
oldLogDir);
|
||||
new ReplicationSourceManager(replicationQueues, replicationPeers, replicationTracker,
|
||||
conf, this.server, fs, logDir, oldLogDir, clusterId);
|
||||
this.statsThreadPeriod =
|
||||
this.conf.getInt("replication.stats.thread.period.seconds", 5 * 60);
|
||||
LOG.debug("ReplicationStatisticsThread " + this.statsThreadPeriod);
|
||||
} else {
|
||||
this.replicationManager = null;
|
||||
this.zkHelper = null;
|
||||
this.replicationQueues = null;
|
||||
this.replicationPeers = null;
|
||||
this.replicationTracker = null;
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -56,11 +56,11 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
|
|||
import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationPeers;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationZookeeper;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationQueues;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
|
||||
import org.apache.hadoop.ipc.RemoteException;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
|
@ -86,8 +86,8 @@ public class ReplicationSource extends Thread
|
|||
// container of entries to replicate
|
||||
private HLog.Entry[] entriesArray;
|
||||
private HConnection conn;
|
||||
// Helper class for zookeeper
|
||||
private ReplicationZookeeper zkHelper;
|
||||
private ReplicationQueues replicationQueues;
|
||||
private ReplicationPeers replicationPeers;
|
||||
private Configuration conf;
|
||||
// ratio of region servers to chose from a slave cluster
|
||||
private float ratio;
|
||||
|
@ -151,12 +151,10 @@ public class ReplicationSource extends Thread
|
|||
* @param peerClusterZnode the name of our znode
|
||||
* @throws IOException
|
||||
*/
|
||||
public void init(final Configuration conf,
|
||||
final FileSystem fs,
|
||||
final ReplicationSourceManager manager,
|
||||
final Stoppable stopper,
|
||||
final String peerClusterZnode)
|
||||
throws IOException {
|
||||
public void init(final Configuration conf, final FileSystem fs,
|
||||
final ReplicationSourceManager manager, final ReplicationQueues replicationQueues,
|
||||
final ReplicationPeers replicationPeers, final Stoppable stopper,
|
||||
final String peerClusterZnode, final UUID clusterId) throws IOException {
|
||||
this.stopper = stopper;
|
||||
this.conf = conf;
|
||||
this.replicationQueueSizeCapacity =
|
||||
|
@ -178,7 +176,8 @@ public class ReplicationSource extends Thread
|
|||
// replication and make replication specific settings such as compression or codec to use
|
||||
// passing Cells.
|
||||
this.conn = HConnectionManager.getConnection(conf);
|
||||
this.zkHelper = manager.getRepZkWrapper();
|
||||
this.replicationQueues = replicationQueues;
|
||||
this.replicationPeers = replicationPeers;
|
||||
this.ratio = this.conf.getFloat("replication.source.ratio", 0.1f);
|
||||
this.currentPeers = new ArrayList<ServerName>();
|
||||
this.random = new Random();
|
||||
|
@ -188,11 +187,8 @@ public class ReplicationSource extends Thread
|
|||
this.fs = fs;
|
||||
this.metrics = new MetricsSource(peerClusterZnode);
|
||||
this.repLogReader = new ReplicationHLogReaderManager(this.fs, this.conf);
|
||||
try {
|
||||
this.clusterId = ZKClusterId.getUUIDForCluster(zkHelper.getZookeeperWatcher());
|
||||
} catch (KeeperException ke) {
|
||||
throw new IOException("Could not read cluster id", ke);
|
||||
}
|
||||
this.clusterId = clusterId;
|
||||
|
||||
this.peerClusterZnode = peerClusterZnode;
|
||||
this.replicationQueueInfo = new ReplicationQueueInfo(peerClusterZnode);
|
||||
// ReplicationQueueInfo parses the peerId out of the znode for us
|
||||
|
@ -204,7 +200,7 @@ public class ReplicationSource extends Thread
|
|||
*/
|
||||
private void chooseSinks() {
|
||||
this.currentPeers.clear();
|
||||
List<ServerName> addresses = this.zkHelper.getSlavesAddresses(this.peerId);
|
||||
List<ServerName> addresses = this.replicationPeers.getRegionServersOfConnectedPeer(this.peerId);
|
||||
Set<ServerName> setOfAddr = new HashSet<ServerName>();
|
||||
int nbPeers = (int) (Math.ceil(addresses.size() * ratio));
|
||||
LOG.debug("Getting " + nbPeers +
|
||||
|
@ -238,7 +234,7 @@ public class ReplicationSource extends Thread
|
|||
int sleepMultiplier = 1;
|
||||
// delay this until we are in an asynchronous thread
|
||||
while (this.peerClusterId == null) {
|
||||
this.peerClusterId = zkHelper.getPeerUUID(this.peerId);
|
||||
this.peerClusterId = replicationPeers.getPeerUUID(this.peerId);
|
||||
if (this.peerClusterId == null) {
|
||||
if (sleepForRetries("Cannot contact the peer's zk ensemble", sleepMultiplier)) {
|
||||
sleepMultiplier++;
|
||||
|
@ -254,8 +250,8 @@ public class ReplicationSource extends Thread
|
|||
// normally has a position (unless the RS failed between 2 logs)
|
||||
if (this.replicationQueueInfo.isQueueRecovered()) {
|
||||
try {
|
||||
this.repLogReader.setPosition(this.zkHelper.getHLogRepPosition(
|
||||
this.peerClusterZnode, this.queue.peek().getName()));
|
||||
this.repLogReader.setPosition(this.replicationQueues.getLogPosition(this.peerClusterZnode,
|
||||
this.queue.peek().getName()));
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("Recovered queue started with log " + this.queue.peek() +
|
||||
" at position " + this.repLogReader.getPosition());
|
||||
|
@ -736,7 +732,7 @@ public class ReplicationSource extends Thread
|
|||
* @return true if the peer is enabled, otherwise false
|
||||
*/
|
||||
protected boolean isPeerEnabled() {
|
||||
return this.zkHelper.getPeerEnabled(this.peerId);
|
||||
return this.replicationPeers.getStatusOfConnectedPeer(this.peerId);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -19,12 +19,15 @@
|
|||
package org.apache.hadoop.hbase.replication.regionserver;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.UUID;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Stoppable;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationPeers;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationQueues;
|
||||
|
||||
/**
|
||||
* Interface that defines a replication source
|
||||
|
@ -41,13 +44,10 @@ public interface ReplicationSourceInterface {
|
|||
* @param peerClusterId the id of the peer cluster
|
||||
* @throws IOException
|
||||
*/
|
||||
void init(
|
||||
final Configuration conf,
|
||||
final FileSystem fs,
|
||||
final ReplicationSourceManager manager,
|
||||
final Stoppable stopper,
|
||||
final String peerClusterId
|
||||
) throws IOException;
|
||||
public void init(final Configuration conf, final FileSystem fs,
|
||||
final ReplicationSourceManager manager, final ReplicationQueues replicationQueues,
|
||||
final ReplicationPeers replicationPeers, final Stoppable stopper,
|
||||
final String peerClusterZnode, final UUID clusterId) throws IOException;
|
||||
|
||||
/**
|
||||
* Add a log to the list of logs to replicate
|
||||
|
|
|
@ -29,11 +29,11 @@ import java.util.Random;
|
|||
import java.util.SortedMap;
|
||||
import java.util.SortedSet;
|
||||
import java.util.TreeSet;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
import java.util.concurrent.RejectedExecutionException;
|
||||
import java.util.concurrent.ThreadPoolExecutor;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
@ -42,10 +42,10 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Stoppable;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationListener;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationPeers;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationQueues;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationZookeeper;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationTracker;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
|
@ -60,18 +60,23 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
|||
* When a region server dies, this class uses a watcher to get notified and it
|
||||
* tries to grab a lock in order to transfer all the queues in a local
|
||||
* old source.
|
||||
*
|
||||
* This class implements the ReplicationListener interface so that it can track changes in
|
||||
* replication state.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class ReplicationSourceManager {
|
||||
public class ReplicationSourceManager implements ReplicationListener {
|
||||
private static final Log LOG =
|
||||
LogFactory.getLog(ReplicationSourceManager.class);
|
||||
// List of all the sources that read this RS's logs
|
||||
private final List<ReplicationSourceInterface> sources;
|
||||
// List of all the sources we got from died RSs
|
||||
private final List<ReplicationSourceInterface> oldsources;
|
||||
// Helper for zookeeper
|
||||
private final ReplicationZookeeper zkHelper;
|
||||
private final ReplicationQueues replicationQueues;
|
||||
private final ReplicationTracker replicationTracker;
|
||||
private final ReplicationPeers replicationPeers;
|
||||
// UUID for this cluster
|
||||
private final UUID clusterId;
|
||||
// All about stopping
|
||||
private final Stoppable stopper;
|
||||
// All logs we are currently tracking
|
||||
|
@ -80,8 +85,6 @@ public class ReplicationSourceManager {
|
|||
private final FileSystem fs;
|
||||
// The path to the latest log we saw, for new coming sources
|
||||
private Path latestPath;
|
||||
// List of all the other region servers in this cluster
|
||||
private final List<String> otherRegionServers = new ArrayList<String>();
|
||||
// Path to the hlogs directories
|
||||
private final Path logDir;
|
||||
// Path to the hlog archive
|
||||
|
@ -104,12 +107,14 @@ public class ReplicationSourceManager {
|
|||
* @param logDir the directory that contains all hlog directories of live RSs
|
||||
* @param oldLogDir the directory where old logs are archived
|
||||
*/
|
||||
public ReplicationSourceManager(final ReplicationZookeeper zkHelper,
|
||||
final ReplicationQueues replicationQueues, final Configuration conf, final Stoppable stopper,
|
||||
final FileSystem fs, final Path logDir, final Path oldLogDir) {
|
||||
public ReplicationSourceManager(final ReplicationQueues replicationQueues,
|
||||
final ReplicationPeers replicationPeers, final ReplicationTracker replicationTracker,
|
||||
final Configuration conf, final Stoppable stopper, final FileSystem fs, final Path logDir,
|
||||
final Path oldLogDir, final UUID clusterId) {
|
||||
this.sources = new ArrayList<ReplicationSourceInterface>();
|
||||
this.zkHelper = zkHelper;
|
||||
this.replicationQueues = replicationQueues;
|
||||
this.replicationPeers = replicationPeers;
|
||||
this.replicationTracker = replicationTracker;
|
||||
this.stopper = stopper;
|
||||
this.hlogsById = new HashMap<String, SortedSet<String>>();
|
||||
this.oldsources = new ArrayList<ReplicationSourceInterface>();
|
||||
|
@ -118,11 +123,9 @@ public class ReplicationSourceManager {
|
|||
this.logDir = logDir;
|
||||
this.oldLogDir = oldLogDir;
|
||||
this.sleepBeforeFailover = conf.getLong("replication.sleep.before.failover", 2000);
|
||||
this.zkHelper.registerRegionServerListener(
|
||||
new OtherRegionServerWatcher(this.zkHelper.getZookeeperWatcher()));
|
||||
this.zkHelper.registerRegionServerListener(
|
||||
new PeersWatcher(this.zkHelper.getZookeeperWatcher()));
|
||||
this.zkHelper.listPeersIdsAndWatch();
|
||||
this.clusterId = clusterId;
|
||||
this.replicationTracker.registerListener(this);
|
||||
this.replicationPeers.getAllPeerIds();
|
||||
// It's preferable to failover 1 RS at a time, but with good zk servers
|
||||
// more could be processed at the same time.
|
||||
int nbWorkers = conf.getInt("replication.executor.workers", 1);
|
||||
|
@ -150,12 +153,12 @@ public class ReplicationSourceManager {
|
|||
*/
|
||||
public void logPositionAndCleanOldLogs(Path log, String id, long position,
|
||||
boolean queueRecovered, boolean holdLogInZK) {
|
||||
String key = log.getName();
|
||||
this.zkHelper.writeReplicationStatus(key, id, position);
|
||||
String fileName = log.getName();
|
||||
this.replicationQueues.setLogPosition(id, fileName, position);
|
||||
if (holdLogInZK) {
|
||||
return;
|
||||
}
|
||||
cleanOldLogs(key, id, queueRecovered);
|
||||
cleanOldLogs(fileName, id, queueRecovered);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -175,7 +178,7 @@ public class ReplicationSourceManager {
|
|||
}
|
||||
SortedSet<String> hlogSet = hlogs.headSet(key);
|
||||
for (String hlog : hlogSet) {
|
||||
this.zkHelper.removeLogFromList(hlog, id);
|
||||
this.replicationQueues.removeLog(id, hlog);
|
||||
}
|
||||
hlogSet.clear();
|
||||
}
|
||||
|
@ -186,24 +189,21 @@ public class ReplicationSourceManager {
|
|||
* old region server hlog queues
|
||||
*/
|
||||
public void init() throws IOException {
|
||||
for (String id : this.zkHelper.getPeerClusters()) {
|
||||
for (String id : this.replicationPeers.getConnectedPeers()) {
|
||||
addSource(id);
|
||||
}
|
||||
List<String> currentReplicators = this.replicationQueues.getListOfReplicators();
|
||||
if (currentReplicators == null || currentReplicators.size() == 0) {
|
||||
return;
|
||||
}
|
||||
synchronized (otherRegionServers) {
|
||||
refreshOtherRegionServersList();
|
||||
LOG.info("Current list of replicators: " + currentReplicators
|
||||
+ " other RSs: " + otherRegionServers);
|
||||
}
|
||||
List<String> otherRegionServers = replicationTracker.getListOfRegionServers();
|
||||
LOG.info("Current list of replicators: " + currentReplicators + " other RSs: "
|
||||
+ otherRegionServers);
|
||||
|
||||
// Look if there's anything to process after a restart
|
||||
for (String rs : currentReplicators) {
|
||||
synchronized (otherRegionServers) {
|
||||
if (!this.otherRegionServers.contains(rs)) {
|
||||
transferQueues(rs);
|
||||
}
|
||||
if (!otherRegionServers.contains(rs)) {
|
||||
transferQueues(rs);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -215,7 +215,9 @@ public class ReplicationSourceManager {
|
|||
* @throws IOException
|
||||
*/
|
||||
public ReplicationSourceInterface addSource(String id) throws IOException {
|
||||
ReplicationSourceInterface src = getReplicationSource(this.conf, this.fs, this, stopper, id);
|
||||
ReplicationSourceInterface src =
|
||||
getReplicationSource(this.conf, this.fs, this, this.replicationQueues,
|
||||
this.replicationPeers, stopper, id, this.clusterId);
|
||||
synchronized (this.hlogsById) {
|
||||
this.sources.add(src);
|
||||
this.hlogsById.put(id, new TreeSet<String>());
|
||||
|
@ -224,7 +226,7 @@ public class ReplicationSourceManager {
|
|||
String name = this.latestPath.getName();
|
||||
this.hlogsById.get(id).add(name);
|
||||
try {
|
||||
this.zkHelper.addLogToList(name, src.getPeerClusterZnode());
|
||||
this.replicationQueues.addLog(src.getPeerClusterZnode(), name);
|
||||
} catch (KeeperException ke) {
|
||||
String message = "Cannot add log to zk for" +
|
||||
" replication when creating a new source";
|
||||
|
@ -238,13 +240,24 @@ public class ReplicationSourceManager {
|
|||
return src;
|
||||
}
|
||||
|
||||
/**
|
||||
* Delete a complete queue of hlogs associated with a peer cluster
|
||||
* @param peerId Id of the peer cluster queue of hlogs to delete
|
||||
*/
|
||||
public void deleteSource(String peerId, boolean closeConnection) {
|
||||
this.replicationQueues.removeQueue(peerId);
|
||||
if (closeConnection) {
|
||||
this.replicationPeers.disconnectFromPeer(peerId);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Terminate the replication on this region server
|
||||
*/
|
||||
public void join() {
|
||||
this.executor.shutdown();
|
||||
if (this.sources.size() == 0) {
|
||||
this.zkHelper.deleteOwnRSZNode();
|
||||
this.replicationQueues.removeAllQueues();
|
||||
}
|
||||
for (ReplicationSourceInterface source : this.sources) {
|
||||
source.terminate("Region server is closing");
|
||||
|
@ -273,7 +286,7 @@ public class ReplicationSourceManager {
|
|||
String name = newLog.getName();
|
||||
for (ReplicationSourceInterface source : this.sources) {
|
||||
try {
|
||||
this.zkHelper.addLogToList(name, source.getPeerClusterZnode());
|
||||
this.replicationQueues.addLog(source.getPeerClusterZnode(), name);
|
||||
} catch (KeeperException ke) {
|
||||
throw new IOException("Cannot add log to zk for replication", ke);
|
||||
}
|
||||
|
@ -298,14 +311,6 @@ public class ReplicationSourceManager {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the ZK help of this manager
|
||||
* @return the helper
|
||||
*/
|
||||
public ReplicationZookeeper getRepZkWrapper() {
|
||||
return zkHelper;
|
||||
}
|
||||
|
||||
/**
|
||||
* Factory method to create a replication source
|
||||
* @param conf the configuration to use
|
||||
|
@ -316,12 +321,10 @@ public class ReplicationSourceManager {
|
|||
* @return the created source
|
||||
* @throws IOException
|
||||
*/
|
||||
public ReplicationSourceInterface getReplicationSource(
|
||||
final Configuration conf,
|
||||
final FileSystem fs,
|
||||
final ReplicationSourceManager manager,
|
||||
final Stoppable stopper,
|
||||
final String peerId) throws IOException {
|
||||
public ReplicationSourceInterface getReplicationSource(final Configuration conf,
|
||||
final FileSystem fs, final ReplicationSourceManager manager,
|
||||
final ReplicationQueues replicationQueues, final ReplicationPeers replicationPeers,
|
||||
final Stoppable stopper, final String peerId, final UUID clusterId) throws IOException {
|
||||
ReplicationSourceInterface src;
|
||||
try {
|
||||
@SuppressWarnings("rawtypes")
|
||||
|
@ -334,7 +337,7 @@ public class ReplicationSourceManager {
|
|||
src = new ReplicationSource();
|
||||
|
||||
}
|
||||
src.init(conf, fs, manager, stopper, peerId);
|
||||
src.init(conf, fs, manager, replicationQueues, replicationPeers, stopper, peerId, clusterId);
|
||||
return src;
|
||||
}
|
||||
|
||||
|
@ -347,7 +350,9 @@ public class ReplicationSourceManager {
|
|||
* @param rsZnode
|
||||
*/
|
||||
private void transferQueues(String rsZnode) {
|
||||
NodeFailoverWorker transfer = new NodeFailoverWorker(rsZnode);
|
||||
NodeFailoverWorker transfer =
|
||||
new NodeFailoverWorker(rsZnode, this.replicationQueues, this.replicationPeers,
|
||||
this.clusterId);
|
||||
try {
|
||||
this.executor.execute(transfer);
|
||||
} catch (RejectedExecutionException ex) {
|
||||
|
@ -362,7 +367,7 @@ public class ReplicationSourceManager {
|
|||
public void closeRecoveredQueue(ReplicationSourceInterface src) {
|
||||
LOG.info("Done with the recovered queue " + src.getPeerClusterZnode());
|
||||
this.oldsources.remove(src);
|
||||
this.zkHelper.deleteSource(src.getPeerClusterZnode(), false);
|
||||
deleteSource(src.getPeerClusterZnode(), false);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -403,151 +408,34 @@ public class ReplicationSourceManager {
|
|||
}
|
||||
srcToRemove.terminate(terminateMessage);
|
||||
this.sources.remove(srcToRemove);
|
||||
this.zkHelper.deleteSource(id, true);
|
||||
deleteSource(id, true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads the list of region servers from ZK and atomically clears our
|
||||
* local view of it and replaces it with the updated list.
|
||||
*
|
||||
* @return true if the local list of the other region servers was updated
|
||||
* with the ZK data (even if it was empty),
|
||||
* false if the data was missing in ZK
|
||||
*/
|
||||
private boolean refreshOtherRegionServersList() {
|
||||
List<String> newRsList = zkHelper.getRegisteredRegionServers();
|
||||
if (newRsList == null) {
|
||||
return false;
|
||||
} else {
|
||||
synchronized (otherRegionServers) {
|
||||
otherRegionServers.clear();
|
||||
otherRegionServers.addAll(newRsList);
|
||||
}
|
||||
}
|
||||
return true;
|
||||
@Override
|
||||
public void regionServerRemoved(String regionserver) {
|
||||
transferQueues(regionserver);
|
||||
}
|
||||
|
||||
/**
|
||||
* Watcher used to be notified of the other region server's death
|
||||
* in the local cluster. It initiates the process to transfer the queues
|
||||
* if it is able to grab the lock.
|
||||
*/
|
||||
public class OtherRegionServerWatcher extends ZooKeeperListener {
|
||||
|
||||
/**
|
||||
* Construct a ZooKeeper event listener.
|
||||
*/
|
||||
public OtherRegionServerWatcher(ZooKeeperWatcher watcher) {
|
||||
super(watcher);
|
||||
}
|
||||
|
||||
/**
|
||||
* Called when a new node has been created.
|
||||
* @param path full path of the new node
|
||||
*/
|
||||
public void nodeCreated(String path) {
|
||||
refreshListIfRightPath(path);
|
||||
}
|
||||
|
||||
/**
|
||||
* Called when a node has been deleted
|
||||
* @param path full path of the deleted node
|
||||
*/
|
||||
public void nodeDeleted(String path) {
|
||||
if (stopper.isStopped()) {
|
||||
return;
|
||||
}
|
||||
boolean cont = refreshListIfRightPath(path);
|
||||
if (!cont) {
|
||||
return;
|
||||
}
|
||||
LOG.info(path + " znode expired, trying to lock it");
|
||||
transferQueues(ReplicationZookeeper.getZNodeName(path));
|
||||
}
|
||||
|
||||
/**
|
||||
* Called when an existing node has a child node added or removed.
|
||||
* @param path full path of the node whose children have changed
|
||||
*/
|
||||
public void nodeChildrenChanged(String path) {
|
||||
if (stopper.isStopped()) {
|
||||
return;
|
||||
}
|
||||
refreshListIfRightPath(path);
|
||||
}
|
||||
|
||||
private boolean refreshListIfRightPath(String path) {
|
||||
if (!path.startsWith(zkHelper.getZookeeperWatcher().rsZNode)) {
|
||||
return false;
|
||||
}
|
||||
return refreshOtherRegionServersList();
|
||||
}
|
||||
@Override
|
||||
public void peerRemoved(String peerId) {
|
||||
removePeer(peerId);
|
||||
}
|
||||
|
||||
/**
|
||||
* Watcher used to follow the creation and deletion of peer clusters.
|
||||
*/
|
||||
public class PeersWatcher extends ZooKeeperListener {
|
||||
|
||||
/**
|
||||
* Construct a ZooKeeper event listener.
|
||||
*/
|
||||
public PeersWatcher(ZooKeeperWatcher watcher) {
|
||||
super(watcher);
|
||||
}
|
||||
|
||||
/**
|
||||
* Called when a node has been deleted
|
||||
* @param path full path of the deleted node
|
||||
*/
|
||||
public void nodeDeleted(String path) {
|
||||
List<String> peers = refreshPeersList(path);
|
||||
if (peers == null) {
|
||||
return;
|
||||
}
|
||||
if (zkHelper.isPeerPath(path)) {
|
||||
String id = ReplicationZookeeper.getZNodeName(path);
|
||||
removePeer(id);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Called when an existing node has a child node added or removed.
|
||||
* @param path full path of the node whose children have changed
|
||||
*/
|
||||
public void nodeChildrenChanged(String path) {
|
||||
List<String> peers = refreshPeersList(path);
|
||||
if (peers == null) {
|
||||
return;
|
||||
}
|
||||
for (String id : peers) {
|
||||
try {
|
||||
boolean added = zkHelper.connectToPeer(id);
|
||||
if (added) {
|
||||
addSource(id);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
// TODO manage better than that ?
|
||||
LOG.error("Error while adding a new peer", e);
|
||||
} catch (KeeperException e) {
|
||||
LOG.error("Error while adding a new peer", e);
|
||||
@Override
|
||||
public void peerListChanged(List<String> peerIds) {
|
||||
for (String id : peerIds) {
|
||||
try {
|
||||
boolean added = this.replicationPeers.connectToPeer(id);
|
||||
if (added) {
|
||||
addSource(id);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
// TODO manage better than that ?
|
||||
LOG.error("Error while adding a new peer", e);
|
||||
} catch (KeeperException e) {
|
||||
LOG.error("Error while adding a new peer", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Verify if this event is meant for us, and if so then get the latest
|
||||
* peers' list from ZK. Also reset the watches.
|
||||
* @param path path to check against
|
||||
* @return A list of peers' identifiers if the event concerns this watcher,
|
||||
* else null.
|
||||
*/
|
||||
private List<String> refreshPeersList(String path) {
|
||||
if (!path.startsWith(zkHelper.getPeersZNode())) {
|
||||
return null;
|
||||
}
|
||||
return zkHelper.listPeersIdsAndWatch();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -557,14 +445,21 @@ public class ReplicationSourceManager {
|
|||
class NodeFailoverWorker extends Thread {
|
||||
|
||||
private String rsZnode;
|
||||
private final ReplicationQueues rq;
|
||||
private final ReplicationPeers rp;
|
||||
private final UUID clusterId;
|
||||
|
||||
/**
|
||||
*
|
||||
* @param rsZnode
|
||||
*/
|
||||
public NodeFailoverWorker(String rsZnode) {
|
||||
public NodeFailoverWorker(String rsZnode, final ReplicationQueues replicationQueues,
|
||||
final ReplicationPeers replicationPeers, final UUID clusterId) {
|
||||
super("Failover-for-"+rsZnode);
|
||||
this.rsZnode = rsZnode;
|
||||
this.rq = replicationQueues;
|
||||
this.rp = replicationPeers;
|
||||
this.clusterId = clusterId;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -584,7 +479,7 @@ public class ReplicationSourceManager {
|
|||
}
|
||||
SortedMap<String, SortedSet<String>> newQueues = null;
|
||||
|
||||
newQueues = zkHelper.claimQueues(rsZnode);
|
||||
newQueues = this.rq.claimQueues(rsZnode);
|
||||
|
||||
// Copying over the failed queue is completed.
|
||||
if (newQueues.isEmpty()) {
|
||||
|
@ -597,8 +492,9 @@ public class ReplicationSourceManager {
|
|||
String peerId = entry.getKey();
|
||||
try {
|
||||
ReplicationSourceInterface src =
|
||||
getReplicationSource(conf, fs, ReplicationSourceManager.this, stopper, peerId);
|
||||
if (!zkHelper.getPeerClusters().contains((src.getPeerClusterId()))) {
|
||||
getReplicationSource(conf, fs, ReplicationSourceManager.this, this.rq, this.rp,
|
||||
stopper, peerId, this.clusterId);
|
||||
if (!this.rp.getConnectedPeers().contains((src.getPeerClusterId()))) {
|
||||
src.terminate("Recovered queue doesn't belong to any current peer");
|
||||
break;
|
||||
}
|
||||
|
|
|
@ -21,7 +21,6 @@ import static org.junit.Assert.assertEquals;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.net.URLEncoder;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
|
@ -30,7 +29,8 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.hbase.*;
|
||||
import org.apache.hadoop.hbase.catalog.CatalogTracker;
|
||||
import org.apache.hadoop.hbase.master.cleaner.LogCleaner;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationZookeeper;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationFactory;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationQueues;
|
||||
import org.apache.hadoop.hbase.replication.regionserver.Replication;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.junit.AfterClass;
|
||||
|
@ -68,8 +68,9 @@ public class TestLogsCleaner {
|
|||
conf.setBoolean(HConstants.REPLICATION_ENABLE_KEY, true);
|
||||
Replication.decorateMasterConfiguration(conf);
|
||||
Server server = new DummyServer();
|
||||
ReplicationZookeeper zkHelper = new ReplicationZookeeper(server);
|
||||
|
||||
ReplicationQueues repQueues =
|
||||
ReplicationFactory.getReplicationQueues(server.getZooKeeper(), conf, server);
|
||||
repQueues.init(server.getServerName().toString());
|
||||
Path oldLogDir = new Path(TEST_UTIL.getDataTestDir(),
|
||||
HConstants.HREGION_OLDLOGDIR_NAME);
|
||||
String fakeMachineName =
|
||||
|
@ -98,7 +99,7 @@ public class TestLogsCleaner {
|
|||
// (TimeToLiveLogCleaner) but would be rejected by the second
|
||||
// (ReplicationLogCleaner)
|
||||
if (i % (30/3) == 1) {
|
||||
zkHelper.addLogToList(fileName.getName(), fakeMachineName);
|
||||
repQueues.addLog(fakeMachineName, fileName.getName());
|
||||
System.out.println("Replication log file: " + fileName);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
package org.apache.hadoop.hbase.replication;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -39,7 +40,9 @@ public class ReplicationSourceDummy implements ReplicationSourceInterface {
|
|||
|
||||
@Override
|
||||
public void init(Configuration conf, FileSystem fs, ReplicationSourceManager manager,
|
||||
Stoppable stopper, String peerClusterId) throws IOException {
|
||||
ReplicationQueues rq, ReplicationPeers rp, Stoppable stopper, String peerClusterId,
|
||||
UUID clusterId) throws IOException {
|
||||
|
||||
this.manager = manager;
|
||||
this.peerClusterId = peerClusterId;
|
||||
}
|
||||
|
|
|
@ -69,6 +69,7 @@ public abstract class TestReplicationStateBasic {
|
|||
|
||||
@Test
|
||||
public void testReplicationQueuesClient() throws KeeperException {
|
||||
rqc.init();
|
||||
// Test methods with empty state
|
||||
assertEquals(0, rqc.getListOfReplicators().size());
|
||||
assertNull(rqc.getLogsInQueue(server1, "qId1"));
|
||||
|
|
|
@ -36,7 +36,10 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
|||
import org.apache.zookeeper.KeeperException;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
@ -50,6 +53,7 @@ public class TestReplicationStateZKImpl extends TestReplicationStateBasic {
|
|||
private static HBaseTestingUtility utility;
|
||||
private static ZooKeeperWatcher zkw;
|
||||
private static String replicationZNode;
|
||||
private ReplicationQueuesZKImpl rqZK;
|
||||
|
||||
@BeforeClass
|
||||
public static void setUpBeforeClass() throws Exception {
|
||||
|
@ -65,7 +69,7 @@ public class TestReplicationStateZKImpl extends TestReplicationStateBasic {
|
|||
|
||||
private static String initPeerClusterState(String baseZKNode)
|
||||
throws IOException, KeeperException {
|
||||
// Set up state nodes of peer clusters
|
||||
// Add a dummy region server and set up the cluster id
|
||||
Configuration testConf = new Configuration(conf);
|
||||
testConf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, baseZKNode);
|
||||
ZooKeeperWatcher zkw1 = new ZooKeeperWatcher(testConf, "test1", null);
|
||||
|
@ -82,16 +86,13 @@ public class TestReplicationStateZKImpl extends TestReplicationStateBasic {
|
|||
DummyServer ds1 = new DummyServer(server1);
|
||||
DummyServer ds2 = new DummyServer(server2);
|
||||
DummyServer ds3 = new DummyServer(server3);
|
||||
try {
|
||||
rq1 = new ReplicationQueuesZKImpl(zkw, conf, ds1);
|
||||
rq2 = new ReplicationQueuesZKImpl(zkw, conf, ds2);
|
||||
rq3 = new ReplicationQueuesZKImpl(zkw, conf, ds3);
|
||||
rqc = new ReplicationQueuesClientZKImpl(zkw, conf, ds1);
|
||||
rp = new ReplicationPeersZKImpl(zkw, conf, zkw);
|
||||
OUR_KEY = ZKUtil.getZooKeeperClusterKey(conf);
|
||||
} catch (KeeperException e) {
|
||||
fail("Exception thrown: " + e);
|
||||
}
|
||||
rq1 = ReplicationFactory.getReplicationQueues(zkw, conf, ds1);
|
||||
rq2 = ReplicationFactory.getReplicationQueues(zkw, conf, ds2);
|
||||
rq3 = ReplicationFactory.getReplicationQueues(zkw, conf, ds3);
|
||||
rqc = ReplicationFactory.getReplicationQueuesClient(zkw, conf, ds1);
|
||||
rp = ReplicationFactory.getReplicationPeers(zkw, conf, zkw);
|
||||
OUR_KEY = ZKUtil.getZooKeeperClusterKey(conf);
|
||||
rqZK = new ReplicationQueuesZKImpl(zkw, conf, ds1);
|
||||
}
|
||||
|
||||
@After
|
||||
|
@ -104,6 +105,23 @@ public class TestReplicationStateZKImpl extends TestReplicationStateBasic {
|
|||
utility.shutdownMiniZKCluster();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIsPeerPath_PathToParentOfPeerNode() {
|
||||
assertFalse(rqZK.isPeerPath(rqZK.peersZNode));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIsPeerPath_PathToChildOfPeerNode() {
|
||||
String peerChild = ZKUtil.joinZNode(ZKUtil.joinZNode(rqZK.peersZNode, "1"), "child");
|
||||
assertFalse(rqZK.isPeerPath(peerChild));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIsPeerPath_ActualPeerPath() {
|
||||
String peerPath = ZKUtil.joinZNode(rqZK.peersZNode, "1");
|
||||
assertTrue(rqZK.isPeerPath(peerPath));
|
||||
}
|
||||
|
||||
static class DummyServer implements Server {
|
||||
private String serverName;
|
||||
private boolean isAborted = false;
|
||||
|
|
|
@ -0,0 +1,251 @@
|
|||
/**
|
||||
* 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.replication;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.ClusterId;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.MediumTests;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.catalog.CatalogTracker;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
/**
|
||||
* This class tests the ReplicationTrackerZKImpl class and ReplicationListener interface. One
|
||||
* MiniZKCluster is used throughout the entire class. The cluster is initialized with the creation
|
||||
* of the rsZNode. All other znode creation/initialization is handled by the replication state
|
||||
* interfaces (i.e. ReplicationPeers, etc.). Each test case in this class should ensure that the
|
||||
* MiniZKCluster is cleaned and returned to it's initial state (i.e. nothing but the rsZNode).
|
||||
*/
|
||||
@Category(MediumTests.class)
|
||||
public class TestReplicationTrackerZKImpl {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(TestReplicationTrackerZKImpl.class);
|
||||
|
||||
private static Configuration conf;
|
||||
private static HBaseTestingUtility utility;
|
||||
|
||||
// Each one of the below variables are reinitialized before every test case
|
||||
private ZooKeeperWatcher zkw;
|
||||
private ReplicationPeers rp;
|
||||
private ReplicationTracker rt;
|
||||
private AtomicInteger rsRemovedCount;
|
||||
private String rsRemovedData;
|
||||
private AtomicInteger plChangedCount;
|
||||
private List<String> plChangedData;
|
||||
private AtomicInteger peerRemovedCount;
|
||||
private String peerRemovedData;
|
||||
|
||||
@BeforeClass
|
||||
public static void setUpBeforeClass() throws Exception {
|
||||
utility = new HBaseTestingUtility();
|
||||
utility.startMiniZKCluster();
|
||||
conf = utility.getConfiguration();
|
||||
ZooKeeperWatcher zk = HBaseTestingUtility.getZooKeeperWatcher(utility);
|
||||
ZKUtil.createWithParents(zk, zk.rsZNode);
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
zkw = HBaseTestingUtility.getZooKeeperWatcher(utility);
|
||||
String fakeRs1 = ZKUtil.joinZNode(zkw.rsZNode, "hostname1.example.org:1234");
|
||||
try {
|
||||
ZKClusterId.setClusterId(zkw, new ClusterId());
|
||||
rp = ReplicationFactory.getReplicationPeers(zkw, conf, zkw);
|
||||
rp.init();
|
||||
rt = ReplicationFactory.getReplicationTracker(zkw, rp, conf, zkw, new DummyServer(fakeRs1));
|
||||
} catch (Exception e) {
|
||||
fail("Exception during test setup: " + e);
|
||||
}
|
||||
rsRemovedCount = new AtomicInteger(0);
|
||||
rsRemovedData = "";
|
||||
plChangedCount = new AtomicInteger(0);
|
||||
plChangedData = new ArrayList<String>();
|
||||
peerRemovedCount = new AtomicInteger(0);
|
||||
peerRemovedData = "";
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDownAfterClass() throws Exception {
|
||||
utility.shutdownMiniZKCluster();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetListOfRegionServers() throws Exception {
|
||||
// 0 region servers
|
||||
assertEquals(0, rt.getListOfRegionServers().size());
|
||||
|
||||
// 1 region server
|
||||
ZKUtil.createWithParents(zkw, ZKUtil.joinZNode(zkw.rsZNode, "hostname1.example.org:1234"));
|
||||
assertEquals(1, rt.getListOfRegionServers().size());
|
||||
|
||||
// 2 region servers
|
||||
ZKUtil.createWithParents(zkw, ZKUtil.joinZNode(zkw.rsZNode, "hostname2.example.org:1234"));
|
||||
assertEquals(2, rt.getListOfRegionServers().size());
|
||||
|
||||
// 1 region server
|
||||
ZKUtil.deleteNode(zkw, ZKUtil.joinZNode(zkw.rsZNode, "hostname2.example.org:1234"));
|
||||
assertEquals(1, rt.getListOfRegionServers().size());
|
||||
|
||||
// 0 region server
|
||||
ZKUtil.deleteNode(zkw, ZKUtil.joinZNode(zkw.rsZNode, "hostname1.example.org:1234"));
|
||||
assertEquals(0, rt.getListOfRegionServers().size());
|
||||
}
|
||||
|
||||
@Test(timeout = 30000)
|
||||
public void testRegionServerRemovedEvent() throws Exception {
|
||||
ZKUtil.createAndWatch(zkw, ZKUtil.joinZNode(zkw.rsZNode, "hostname2.example.org:1234"),
|
||||
HConstants.EMPTY_BYTE_ARRAY);
|
||||
rt.registerListener(new DummyReplicationListener());
|
||||
// delete one
|
||||
ZKUtil.deleteNode(zkw, ZKUtil.joinZNode(zkw.rsZNode, "hostname2.example.org:1234"));
|
||||
// wait for event
|
||||
while (rsRemovedCount.get() < 1) {
|
||||
Thread.sleep(5);
|
||||
}
|
||||
assertEquals("hostname2.example.org:1234", rsRemovedData);
|
||||
}
|
||||
|
||||
@Test(timeout = 30000)
|
||||
public void testPeerRemovedEvent() throws Exception {
|
||||
rp.addPeer("5", utility.getClusterKey());
|
||||
rt.registerListener(new DummyReplicationListener());
|
||||
rp.removePeer("5");
|
||||
// wait for event
|
||||
while (peerRemovedCount.get() < 1) {
|
||||
Thread.sleep(5);
|
||||
}
|
||||
assertEquals("5", peerRemovedData);
|
||||
}
|
||||
|
||||
@Test(timeout = 30000)
|
||||
public void testPeerListChangedEvent() throws Exception {
|
||||
// add a peer
|
||||
rp.addPeer("5", utility.getClusterKey());
|
||||
zkw.getRecoverableZooKeeper().getZooKeeper().getChildren("/hbase/replication/peers/5", true);
|
||||
rt.registerListener(new DummyReplicationListener());
|
||||
rp.disablePeer("5");
|
||||
ZKUtil.deleteNode(zkw, "/hbase/replication/peers/5/peer-state");
|
||||
// wait for event
|
||||
int tmp = plChangedCount.get();
|
||||
while (plChangedCount.get() <= tmp) {
|
||||
Thread.sleep(5);
|
||||
}
|
||||
assertEquals(1, plChangedData.size());
|
||||
assertTrue(plChangedData.contains("5"));
|
||||
|
||||
// clean up
|
||||
ZKUtil.deleteNode(zkw, "/hbase/replication/peers/5");
|
||||
}
|
||||
|
||||
private class DummyReplicationListener implements ReplicationListener {
|
||||
|
||||
@Override
|
||||
public void regionServerRemoved(String regionServer) {
|
||||
rsRemovedData = regionServer;
|
||||
rsRemovedCount.getAndIncrement();
|
||||
LOG.debug("Received regionServerRemoved event: " + regionServer);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void peerRemoved(String peerId) {
|
||||
peerRemovedData = peerId;
|
||||
peerRemovedCount.getAndIncrement();
|
||||
LOG.debug("Received peerRemoved event: " + peerId);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void peerListChanged(List<String> peerIds) {
|
||||
plChangedData.clear();
|
||||
plChangedData.addAll(peerIds);
|
||||
plChangedCount.getAndIncrement();
|
||||
LOG.debug("Received peerListChanged event");
|
||||
}
|
||||
}
|
||||
|
||||
private class DummyServer implements Server {
|
||||
private String serverName;
|
||||
private boolean isAborted = false;
|
||||
private boolean isStopped = false;
|
||||
|
||||
public DummyServer(String serverName) {
|
||||
this.serverName = serverName;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Configuration getConfiguration() {
|
||||
return conf;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ZooKeeperWatcher getZooKeeper() {
|
||||
return zkw;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CatalogTracker getCatalogTracker() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ServerName getServerName() {
|
||||
return new ServerName(this.serverName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort(String why, Throwable e) {
|
||||
LOG.info("Aborting " + serverName);
|
||||
this.isAborted = true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isAborted() {
|
||||
return this.isAborted;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stop(String why) {
|
||||
this.isStopped = true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isStopped() {
|
||||
return this.isStopped;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -1,145 +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.replication;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.MediumTests;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.catalog.CatalogTracker;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
@Category(MediumTests.class)
|
||||
public class TestReplicationZookeeper {
|
||||
|
||||
private static Configuration conf;
|
||||
|
||||
private static HBaseTestingUtility utility;
|
||||
|
||||
private static ZooKeeperWatcher zkw;
|
||||
|
||||
private static ReplicationZookeeper repZk;
|
||||
|
||||
private static String slaveClusterKey;
|
||||
|
||||
private static String peersZNode;
|
||||
|
||||
@BeforeClass
|
||||
public static void setUpBeforeClass() throws Exception {
|
||||
utility = new HBaseTestingUtility();
|
||||
utility.startMiniZKCluster();
|
||||
conf = utility.getConfiguration();
|
||||
zkw = HBaseTestingUtility.getZooKeeperWatcher(utility);
|
||||
DummyServer server = new DummyServer();
|
||||
repZk = new ReplicationZookeeper(server);
|
||||
slaveClusterKey = conf.get(HConstants.ZOOKEEPER_QUORUM) + ":" +
|
||||
conf.get("hbase.zookeeper.property.clientPort") + ":/1";
|
||||
String replicationZNodeName = conf.get("zookeeper.znode.replication", "replication");
|
||||
String peersZNodeName = conf.get("zookeeper.znode.replication.peers", "peers");
|
||||
String replicationZNode = ZKUtil.joinZNode(zkw.baseZNode, replicationZNodeName);
|
||||
peersZNode = ZKUtil.joinZNode(replicationZNode, peersZNodeName);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDownAfterClass() throws Exception {
|
||||
utility.shutdownMiniZKCluster();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetAddressesMissingSlave()
|
||||
throws IOException, KeeperException {
|
||||
repZk.addPeer("1", slaveClusterKey);
|
||||
// HBASE-5586 used to get an NPE
|
||||
assertEquals(0, repZk.getSlavesAddresses("1").size());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIsPeerPath_PathToParentOfPeerNode() {
|
||||
String peerParentNode = peersZNode;
|
||||
assertFalse(repZk.isPeerPath(peerParentNode));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIsPeerPath_PathToChildOfPeerNode() {
|
||||
String peerChild = ZKUtil.joinZNode(ZKUtil.joinZNode(peersZNode, "1"), "child");
|
||||
assertFalse(repZk.isPeerPath(peerChild));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIsPeerPath_ActualPeerPath() {
|
||||
String peerPath = ZKUtil.joinZNode(peersZNode, "1");
|
||||
assertTrue(repZk.isPeerPath(peerPath));
|
||||
}
|
||||
|
||||
static class DummyServer implements Server {
|
||||
|
||||
@Override
|
||||
public Configuration getConfiguration() {
|
||||
return conf;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ZooKeeperWatcher getZooKeeper() {
|
||||
return zkw;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CatalogTracker getCatalogTracker() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ServerName getServerName() {
|
||||
return new ServerName("hostname.example.org", 1234, -1L);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort(String why, Throwable e) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isAborted() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stop(String why) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isStopped() {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -34,6 +34,7 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.ClusterId;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
|
@ -50,11 +51,13 @@ import org.apache.hadoop.hbase.regionserver.wal.HLogFactory;
|
|||
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationFactory;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationQueues;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationSourceDummy;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationStateZKBase;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationZookeeper;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.junit.After;
|
||||
|
@ -128,6 +131,8 @@ public class TestReplicationSourceManager {
|
|||
ZKUtil.createWithParents(zkw, "/hbase/replication/state");
|
||||
ZKUtil.setData(zkw, "/hbase/replication/state", ReplicationStateZKBase.ENABLED_ZNODE_BYTES);
|
||||
|
||||
ZKClusterId.setClusterId(zkw, new ClusterId());
|
||||
|
||||
replication = new Replication(new DummyServer(), fs, logDir, oldLogDir);
|
||||
manager = replication.getReplicationManager();
|
||||
fs = FileSystem.get(conf);
|
||||
|
@ -226,12 +231,15 @@ public class TestReplicationSourceManager {
|
|||
LOG.debug("testNodeFailoverWorkerCopyQueuesFromRSUsingMulti");
|
||||
conf.setBoolean(HConstants.ZOOKEEPER_USEMULTI, true);
|
||||
final Server server = new DummyServer("hostname0.example.org");
|
||||
ReplicationZookeeper rz = new ReplicationZookeeper(server);
|
||||
ReplicationQueues rq =
|
||||
ReplicationFactory.getReplicationQueues(server.getZooKeeper(), server.getConfiguration(),
|
||||
server);
|
||||
rq.init(server.getServerName().toString());
|
||||
// populate some znodes in the peer znode
|
||||
files.add("log1");
|
||||
files.add("log2");
|
||||
for (String file : files) {
|
||||
rz.addLogToList(file, "1");
|
||||
rq.addLog("1", file);
|
||||
}
|
||||
// create 3 DummyServers
|
||||
Server s1 = new DummyServer("dummyserver1.example.org");
|
||||
|
@ -266,12 +274,14 @@ public class TestReplicationSourceManager {
|
|||
LOG.debug("testNodeFailoverDeadServerParsing");
|
||||
conf.setBoolean(HConstants.ZOOKEEPER_USEMULTI, true);
|
||||
final Server server = new DummyServer("ec2-54-234-230-108.compute-1.amazonaws.com");
|
||||
ReplicationZookeeper rz = new ReplicationZookeeper(server);
|
||||
ReplicationQueues repQueues =
|
||||
ReplicationFactory.getReplicationQueues(server.getZooKeeper(), conf, server);
|
||||
repQueues.init(server.getServerName().toString());
|
||||
// populate some znodes in the peer znode
|
||||
files.add("log1");
|
||||
files.add("log2");
|
||||
for (String file : files) {
|
||||
rz.addLogToList(file, "1");
|
||||
repQueues.addLog("1", file);
|
||||
}
|
||||
|
||||
// create 3 DummyServers
|
||||
|
@ -280,13 +290,19 @@ public class TestReplicationSourceManager {
|
|||
Server s3 = new DummyServer("ec2-23-20-187-167.compute-1.amazonaws.com");
|
||||
|
||||
// simulate three servers fail sequentially
|
||||
ReplicationZookeeper rz1 = new ReplicationZookeeper(s1);
|
||||
ReplicationQueues rq1 =
|
||||
ReplicationFactory.getReplicationQueues(s1.getZooKeeper(), s1.getConfiguration(), s1);
|
||||
rq1.init(s1.getServerName().toString());
|
||||
SortedMap<String, SortedSet<String>> testMap =
|
||||
rz1.claimQueues(server.getServerName().getServerName());
|
||||
ReplicationZookeeper rz2 = new ReplicationZookeeper(s2);
|
||||
testMap = rz2.claimQueues(s1.getServerName().getServerName());
|
||||
ReplicationZookeeper rz3 = new ReplicationZookeeper(s3);
|
||||
testMap = rz3.claimQueues(s2.getServerName().getServerName());
|
||||
rq1.claimQueues(server.getServerName().getServerName());
|
||||
ReplicationQueues rq2 =
|
||||
ReplicationFactory.getReplicationQueues(s2.getZooKeeper(), s2.getConfiguration(), s2);
|
||||
rq2.init(s2.getServerName().toString());
|
||||
testMap = rq2.claimQueues(s1.getServerName().getServerName());
|
||||
ReplicationQueues rq3 =
|
||||
ReplicationFactory.getReplicationQueues(s3.getZooKeeper(), s3.getConfiguration(), s3);
|
||||
rq3.init(s3.getServerName().toString());
|
||||
testMap = rq3.claimQueues(s2.getServerName().getServerName());
|
||||
|
||||
ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(testMap.firstKey());
|
||||
List<String> result = replicationQueueInfo.getDeadRegionServers();
|
||||
|
@ -304,18 +320,21 @@ public class TestReplicationSourceManager {
|
|||
private SortedMap<String, SortedSet<String>> logZnodesMap;
|
||||
Server server;
|
||||
private String deadRsZnode;
|
||||
ReplicationZookeeper rz;
|
||||
ReplicationQueues rq;
|
||||
|
||||
public DummyNodeFailoverWorker(String znode, Server s) throws Exception {
|
||||
this.deadRsZnode = znode;
|
||||
this.server = s;
|
||||
rz = new ReplicationZookeeper(server);
|
||||
this.rq =
|
||||
ReplicationFactory.getReplicationQueues(server.getZooKeeper(), server.getConfiguration(),
|
||||
server);
|
||||
this.rq.init(this.server.getServerName().toString());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
logZnodesMap = rz.claimQueues(deadRsZnode);
|
||||
logZnodesMap = rq.claimQueues(deadRsZnode);
|
||||
server.abort("Done with testing", null);
|
||||
} catch (Exception e) {
|
||||
LOG.error("Got exception while running NodeFailoverWorker", e);
|
||||
|
|
Loading…
Reference in New Issue