001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018
019package org.apache.hadoop.hbase.replication;
020
021import java.io.IOException;
022import java.util.ArrayList;
023import java.util.Collections;
024import java.util.List;
025import java.util.UUID;
026import org.apache.hadoop.hbase.zookeeper.ZKListener;
027import org.apache.hadoop.hbase.Abortable;
028import org.apache.hadoop.hbase.ServerName;
029import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
030import org.apache.hadoop.hbase.zookeeper.ZKUtil;
031import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
032import org.apache.yetus.audience.InterfaceAudience;
033import org.apache.zookeeper.KeeperException;
034import org.apache.zookeeper.KeeperException.AuthFailedException;
035import org.apache.zookeeper.KeeperException.ConnectionLossException;
036import org.apache.zookeeper.KeeperException.SessionExpiredException;
037import org.slf4j.Logger;
038import org.slf4j.LoggerFactory;
039
040/**
041 * A {@link BaseReplicationEndpoint} for replication endpoints whose
042 * target cluster is an HBase cluster.
043 */
044@InterfaceAudience.Private
045public abstract class HBaseReplicationEndpoint extends BaseReplicationEndpoint
046  implements Abortable {
047
048  private static final Logger LOG = LoggerFactory.getLogger(HBaseReplicationEndpoint.class);
049
050  private ZKWatcher zkw = null;
051
052  private List<ServerName> regionServers = new ArrayList<>(0);
053  private long lastRegionServerUpdate;
054
055  protected synchronized void disconnect() {
056    if (zkw != null) {
057      zkw.close();
058    }
059  }
060
061  /**
062   * A private method used to re-establish a zookeeper session with a peer cluster.
063   * @param ke
064   */
065  protected void reconnect(KeeperException ke) {
066    if (ke instanceof ConnectionLossException || ke instanceof SessionExpiredException
067        || ke instanceof AuthFailedException) {
068      String clusterKey = ctx.getPeerConfig().getClusterKey();
069      LOG.warn("Lost the ZooKeeper connection for peer " + clusterKey, ke);
070      try {
071        reloadZkWatcher();
072      } catch (IOException io) {
073        LOG.warn("Creation of ZookeeperWatcher failed for peer " + clusterKey, io);
074      }
075    }
076  }
077
078  @Override
079  public void start() {
080    startAsync();
081  }
082
083  @Override
084  public void stop() {
085    stopAsync();
086  }
087
088  @Override
089  protected void doStart() {
090    try {
091      reloadZkWatcher();
092      notifyStarted();
093    } catch (IOException e) {
094      notifyFailed(e);
095    }
096  }
097
098  @Override
099  protected void doStop() {
100    disconnect();
101    notifyStopped();
102  }
103
104  @Override
105  // Synchronize peer cluster connection attempts to avoid races and rate
106  // limit connections when multiple replication sources try to connect to
107  // the peer cluster. If the peer cluster is down we can get out of control
108  // over time.
109  public synchronized UUID getPeerUUID() {
110    UUID peerUUID = null;
111    try {
112      peerUUID = ZKClusterId.getUUIDForCluster(zkw);
113    } catch (KeeperException ke) {
114      reconnect(ke);
115    }
116    return peerUUID;
117  }
118
119  /**
120   * Get the ZK connection to this peer
121   * @return zk connection
122   */
123  protected synchronized ZKWatcher getZkw() {
124    return zkw;
125  }
126
127  /**
128   * Closes the current ZKW (if not null) and creates a new one
129   * @throws IOException If anything goes wrong connecting
130   */
131  synchronized void reloadZkWatcher() throws IOException {
132    if (zkw != null) zkw.close();
133    zkw = new ZKWatcher(ctx.getConfiguration(),
134        "connection to cluster: " + ctx.getPeerId(), this);
135    getZkw().registerListener(new PeerRegionServerListener(this));
136  }
137
138  @Override
139  public void abort(String why, Throwable e) {
140    LOG.error("The HBaseReplicationEndpoint corresponding to peer " + ctx.getPeerId()
141        + " was aborted for the following reason(s):" + why, e);
142  }
143
144  @Override
145  public boolean isAborted() {
146    // Currently this is never "Aborted", we just log when the abort method is called.
147    return false;
148  }
149
150  /**
151   * Get the list of all the region servers from the specified peer
152   * @param zkw zk connection to use
153   * @return list of region server addresses or an empty list if the slave is unavailable
154   */
155  protected static List<ServerName> fetchSlavesAddresses(ZKWatcher zkw)
156      throws KeeperException {
157    List<String> children = ZKUtil.listChildrenAndWatchForNewChildren(zkw,
158            zkw.getZNodePaths().rsZNode);
159    if (children == null) {
160      return Collections.emptyList();
161    }
162    List<ServerName> addresses = new ArrayList<>(children.size());
163    for (String child : children) {
164      addresses.add(ServerName.parseServerName(child));
165    }
166    return addresses;
167  }
168
169  /**
170   * Get a list of all the addresses of all the available region servers
171   * for this peer cluster, or an empty list if no region servers available at peer cluster.
172   * @return list of addresses
173   */
174  // Synchronize peer cluster connection attempts to avoid races and rate
175  // limit connections when multiple replication sources try to connect to
176  // the peer cluster. If the peer cluster is down we can get out of control
177  // over time.
178  public synchronized List<ServerName> getRegionServers() {
179    try {
180      setRegionServers(fetchSlavesAddresses(this.getZkw()));
181    } catch (KeeperException ke) {
182      if (LOG.isDebugEnabled()) {
183        LOG.debug("Fetch slaves addresses failed", ke);
184      }
185      reconnect(ke);
186    }
187    return regionServers;
188  }
189
190  /**
191   * Set the list of region servers for that peer
192   * @param regionServers list of addresses for the region servers
193   */
194  public synchronized void setRegionServers(List<ServerName> regionServers) {
195    this.regionServers = regionServers;
196    lastRegionServerUpdate = System.currentTimeMillis();
197  }
198
199  /**
200   * Get the timestamp at which the last change occurred to the list of region servers to replicate
201   * to.
202   * @return The System.currentTimeMillis at the last time the list of peer region servers changed.
203   */
204  public long getLastRegionServerUpdate() {
205    return lastRegionServerUpdate;
206  }
207
208  /**
209   * Tracks changes to the list of region servers in a peer's cluster.
210   */
211  public static class PeerRegionServerListener extends ZKListener {
212
213    private final HBaseReplicationEndpoint replicationEndpoint;
214    private final String regionServerListNode;
215
216    public PeerRegionServerListener(HBaseReplicationEndpoint replicationPeer) {
217      super(replicationPeer.getZkw());
218      this.replicationEndpoint = replicationPeer;
219      this.regionServerListNode = replicationEndpoint.getZkw().getZNodePaths().rsZNode;
220    }
221
222    @Override
223    public synchronized void nodeChildrenChanged(String path) {
224      if (path.equals(regionServerListNode)) {
225        try {
226          LOG.info("Detected change to peer region servers, fetching updated list");
227          replicationEndpoint.setRegionServers(fetchSlavesAddresses(replicationEndpoint.getZkw()));
228        } catch (KeeperException e) {
229          LOG.error("Error reading slave addresses", e);
230        }
231      }
232    }
233  }
234}