View Javadoc

1   /*
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.replication;
20  
21  import java.util.ArrayList;
22  import java.util.List;
23  import java.util.SortedMap;
24  import java.util.SortedSet;
25  import java.util.TreeMap;
26  import java.util.TreeSet;
27  
28  import com.google.common.annotations.VisibleForTesting;
29  import org.apache.commons.logging.Log;
30  import org.apache.commons.logging.LogFactory;
31  import org.apache.hadoop.hbase.classification.InterfaceAudience;
32  import org.apache.hadoop.conf.Configuration;
33  import org.apache.hadoop.hbase.Abortable;
34  import org.apache.hadoop.hbase.HConstants;
35  import org.apache.hadoop.hbase.exceptions.DeserializationException;
36  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
37  import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
38  import org.apache.hadoop.hbase.util.Bytes;
39  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
40  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
41  import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp;
42  import org.apache.zookeeper.KeeperException;
43  
44  /**
45   * This class provides an implementation of the ReplicationQueues interface using Zookeeper. The
46   * base znode that this class works at is the myQueuesZnode. The myQueuesZnode contains a list of
47   * all outstanding WAL files on this region server that need to be replicated. The myQueuesZnode is
48   * the regionserver name (a concatenation of the region server’s hostname, client port and start
49   * code). For example:
50   *
51   * /hbase/replication/rs/hostname.example.org,6020,1234
52   *
53   * Within this znode, the region server maintains a set of WAL replication queues. These queues are
54   * represented by child znodes named using there give queue id. For example:
55   *
56   * /hbase/replication/rs/hostname.example.org,6020,1234/1
57   * /hbase/replication/rs/hostname.example.org,6020,1234/2
58   *
59   * Each queue has one child znode for every WAL that still needs to be replicated. The value of
60   * these WAL child znodes is the latest position that has been replicated. This position is updated
61   * every time a WAL entry is replicated. For example:
62   *
63   * /hbase/replication/rs/hostname.example.org,6020,1234/1/23522342.23422 [VALUE: 254]
64   */
65  @InterfaceAudience.Private
66  public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements ReplicationQueues {
67  
68    /** Znode containing all replication queues for this region server. */
69    private String myQueuesZnode;
70    /** Name of znode we use to lock during failover */
71    public final static String RS_LOCK_ZNODE = "lock";
72  
73    private static final Log LOG = LogFactory.getLog(ReplicationQueuesZKImpl.class);
74  
75    public ReplicationQueuesZKImpl(final ZooKeeperWatcher zk, Configuration conf,
76        Abortable abortable) {
77      super(zk, conf, abortable);
78    }
79  
80    @Override
81    public void init(String serverName) throws ReplicationException {
82      this.myQueuesZnode = ZKUtil.joinZNode(this.queuesZNode, serverName);
83      try {
84        ZKUtil.createWithParents(this.zookeeper, this.myQueuesZnode);
85      } catch (KeeperException e) {
86        throw new ReplicationException("Could not initialize replication queues.", e);
87      }
88    }
89  
90    @Override
91    public List<String> getListOfReplicators() throws ReplicationException {
92      try {
93        return super.getListOfReplicatorsZK();
94      } catch (KeeperException e) {
95        LOG.warn("getListOfReplicators() from ZK failed", e);
96        throw new ReplicationException("getListOfReplicators() from ZK failed", e);
97      }
98    }
99  
100   @Override
101   public void removeQueue(String queueId) {
102     try {
103       ZKUtil.deleteNodeRecursively(this.zookeeper, ZKUtil.joinZNode(this.myQueuesZnode, queueId));
104     } catch (KeeperException e) {
105       this.abortable.abort("Failed to delete queue (queueId=" + queueId + ")", e);
106     }
107   }
108 
109   @Override
110   public void addLog(String queueId, String filename) throws ReplicationException {
111     String znode = ZKUtil.joinZNode(this.myQueuesZnode, queueId);
112     znode = ZKUtil.joinZNode(znode, filename);
113     try {
114       ZKUtil.createWithParents(this.zookeeper, znode);
115     } catch (KeeperException e) {
116       throw new ReplicationException(
117           "Could not add log because znode could not be created. queueId=" + queueId
118               + ", filename=" + filename);
119     }
120   }
121 
122   @Override
123   public void removeLog(String queueId, String filename) {
124     try {
125       String znode = ZKUtil.joinZNode(this.myQueuesZnode, queueId);
126       znode = ZKUtil.joinZNode(znode, filename);
127       ZKUtil.deleteNode(this.zookeeper, znode);
128     } catch (KeeperException e) {
129       this.abortable.abort("Failed to remove wal from queue (queueId=" + queueId + ", filename="
130           + filename + ")", e);
131     }
132   }
133 
134   @Override
135   public void setLogPosition(String queueId, String filename, long position) {
136     try {
137       String znode = ZKUtil.joinZNode(this.myQueuesZnode, queueId);
138       znode = ZKUtil.joinZNode(znode, filename);
139       // Why serialize String of Long and not Long as bytes?
140       ZKUtil.setData(this.zookeeper, znode, ZKUtil.positionToByteArray(position));
141     } catch (KeeperException e) {
142       this.abortable.abort("Failed to write replication wal position (filename=" + filename
143           + ", position=" + position + ")", e);
144     }
145   }
146 
147   @Override
148   public long getLogPosition(String queueId, String filename) throws ReplicationException {
149     String clusterZnode = ZKUtil.joinZNode(this.myQueuesZnode, queueId);
150     String znode = ZKUtil.joinZNode(clusterZnode, filename);
151     byte[] bytes = null;
152     try {
153       bytes = ZKUtil.getData(this.zookeeper, znode);
154     } catch (KeeperException e) {
155       throw new ReplicationException("Internal Error: could not get position in log for queueId="
156           + queueId + ", filename=" + filename, e);
157     } catch (InterruptedException e) {
158       Thread.currentThread().interrupt();
159       return 0;
160     }
161     try {
162       return ZKUtil.parseWALPositionFrom(bytes);
163     } catch (DeserializationException de) {
164       LOG.warn("Failed to parse WALPosition for queueId=" + queueId + " and wal=" + filename
165           + "znode content, continuing.");
166     }
167     // if we can not parse the position, start at the beginning of the wal file
168     // again
169     return 0;
170   }
171 
172   @Override
173   public boolean isThisOurZnode(String znode) {
174     return ZKUtil.joinZNode(this.queuesZNode, znode).equals(this.myQueuesZnode);
175   }
176 
177   @Override
178   public SortedMap<String, SortedSet<String>> claimQueues(String regionserverZnode) {
179     SortedMap<String, SortedSet<String>> newQueues = new TreeMap<String, SortedSet<String>>();
180     // check whether there is multi support. If yes, use it.
181     if (conf.getBoolean(HConstants.ZOOKEEPER_USEMULTI, true)) {
182       LOG.info("Atomically moving " + regionserverZnode + "'s WALs to my queue");
183       newQueues = copyQueuesFromRSUsingMulti(regionserverZnode);
184     } else {
185       LOG.info("Moving " + regionserverZnode + "'s wals to my queue");
186       if (!lockOtherRS(regionserverZnode)) {
187         return newQueues;
188       }
189       newQueues = copyQueuesFromRS(regionserverZnode);
190       deleteAnotherRSQueues(regionserverZnode);
191     }
192     return newQueues;
193   }
194 
195   @Override
196   public void removeAllQueues() {
197     try {
198       ZKUtil.deleteNodeRecursively(this.zookeeper, this.myQueuesZnode);
199     } catch (KeeperException e) {
200       // if the znode is already expired, don't bother going further
201       if (e instanceof KeeperException.SessionExpiredException) {
202         return;
203       }
204       this.abortable.abort("Failed to delete replication queues for region server: "
205           + this.myQueuesZnode, e);
206     }
207   }
208 
209   @Override
210   public List<String> getLogsInQueue(String queueId) {
211     String znode = ZKUtil.joinZNode(this.myQueuesZnode, queueId);
212     List<String> result = null;
213     try {
214       result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);
215     } catch (KeeperException e) {
216       this.abortable.abort("Failed to get list of wals for queueId=" + queueId, e);
217     }
218     return result;
219   }
220 
221   @Override
222   public List<String> getAllQueues() {
223     List<String> listOfQueues = null;
224     try {
225       listOfQueues = ZKUtil.listChildrenNoWatch(this.zookeeper, this.myQueuesZnode);
226     } catch (KeeperException e) {
227       this.abortable.abort("Failed to get a list of queues for region server: "
228           + this.myQueuesZnode, e);
229     }
230     return listOfQueues;
231   }
232 
233   /**
234    * Try to set a lock in another region server's znode.
235    * @param znode the server names of the other server
236    * @return true if the lock was acquired, false in every other cases
237    */
238   @VisibleForTesting
239   public boolean lockOtherRS(String znode) {
240     try {
241       String parent = ZKUtil.joinZNode(this.queuesZNode, znode);
242       if (parent.equals(this.myQueuesZnode)) {
243         LOG.warn("Won't lock because this is us, we're dead!");
244         return false;
245       }
246       String p = ZKUtil.joinZNode(parent, RS_LOCK_ZNODE);
247       ZKUtil.createAndWatch(this.zookeeper, p, lockToByteArray(this.myQueuesZnode));
248     } catch (KeeperException e) {
249       // This exception will pop up if the znode under which we're trying to
250       // create the lock is already deleted by another region server, meaning
251       // that the transfer already occurred.
252       // NoNode => transfer is done and znodes are already deleted
253       // NodeExists => lock znode already created by another RS
254       if (e instanceof KeeperException.NoNodeException
255           || e instanceof KeeperException.NodeExistsException) {
256         LOG.info("Won't transfer the queue," + " another RS took care of it because of: "
257             + e.getMessage());
258       } else {
259         LOG.info("Failed lock other rs", e);
260       }
261       return false;
262     }
263     return true;
264   }
265 
266   public String getLockZNode(String znode) {
267     return this.queuesZNode + "/" + znode + "/" + RS_LOCK_ZNODE;
268   }
269 
270   @VisibleForTesting
271   public boolean checkLockExists(String znode) throws KeeperException {
272     return ZKUtil.checkExists(zookeeper, getLockZNode(znode)) >= 0;
273   }
274 
275   /**
276    * Delete all the replication queues for a given region server.
277    * @param regionserverZnode The znode of the region server to delete.
278    */
279   private void deleteAnotherRSQueues(String regionserverZnode) {
280     String fullpath = ZKUtil.joinZNode(this.queuesZNode, regionserverZnode);
281     try {
282       List<String> clusters = ZKUtil.listChildrenNoWatch(this.zookeeper, fullpath);
283       for (String cluster : clusters) {
284         // No need to delete, it will be deleted later.
285         if (cluster.equals(RS_LOCK_ZNODE)) {
286           continue;
287         }
288         String fullClusterPath = ZKUtil.joinZNode(fullpath, cluster);
289         ZKUtil.deleteNodeRecursively(this.zookeeper, fullClusterPath);
290       }
291       // Finish cleaning up
292       ZKUtil.deleteNodeRecursively(this.zookeeper, fullpath);
293     } catch (KeeperException e) {
294       if (e instanceof KeeperException.NoNodeException
295           || e instanceof KeeperException.NotEmptyException) {
296         // Testing a special case where another region server was able to
297         // create a lock just after we deleted it, but then was also able to
298         // delete the RS znode before us or its lock znode is still there.
299         if (e.getPath().equals(fullpath)) {
300           return;
301         }
302       }
303       this.abortable.abort("Failed to delete replication queues for region server: "
304           + regionserverZnode, e);
305     }
306   }
307 
308   /**
309    * It "atomically" copies all the wals queues from another region server and returns them all
310    * sorted per peer cluster (appended with the dead server's znode).
311    * @param znode pertaining to the region server to copy the queues from
312    * @return WAL queues sorted per peer cluster
313    */
314   private SortedMap<String, SortedSet<String>> copyQueuesFromRSUsingMulti(String znode) {
315     SortedMap<String, SortedSet<String>> queues = new TreeMap<String, SortedSet<String>>();
316     // hbase/replication/rs/deadrs
317     String deadRSZnodePath = ZKUtil.joinZNode(this.queuesZNode, znode);
318     List<String> peerIdsToProcess = null;
319     List<ZKUtilOp> listOfOps = new ArrayList<ZKUtil.ZKUtilOp>();
320     try {
321       peerIdsToProcess = ZKUtil.listChildrenNoWatch(this.zookeeper, deadRSZnodePath);
322       if (peerIdsToProcess == null) return queues; // node already processed
323       for (String peerId : peerIdsToProcess) {
324         ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(peerId);
325         if (!peerExists(replicationQueueInfo.getPeerId())) {
326           // the orphaned queues must be moved, otherwise the delete op of dead rs will fail,
327           // this will cause the whole multi op fail.
328           // NodeFailoverWorker will skip the orphaned queues.
329           LOG.warn("Peer " + peerId
330               + " didn't exist, will move its queue to avoid the failure of multi op");
331         }
332         String newPeerId = peerId + "-" + znode;
333         String newPeerZnode = ZKUtil.joinZNode(this.myQueuesZnode, newPeerId);
334         // check the logs queue for the old peer cluster
335         String oldClusterZnode = ZKUtil.joinZNode(deadRSZnodePath, peerId);
336         List<String> wals = ZKUtil.listChildrenNoWatch(this.zookeeper, oldClusterZnode);
337         if (wals == null || wals.size() == 0) {
338           listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode));
339           continue; // empty log queue.
340         }
341         // create the new cluster znode
342         SortedSet<String> logQueue = new TreeSet<String>();
343         queues.put(newPeerId, logQueue);
344         ZKUtilOp op = ZKUtilOp.createAndFailSilent(newPeerZnode, HConstants.EMPTY_BYTE_ARRAY);
345         listOfOps.add(op);
346         // get the offset of the logs and set it to new znodes
347         for (String wal : wals) {
348           String oldWalZnode = ZKUtil.joinZNode(oldClusterZnode, wal);
349           byte[] logOffset = ZKUtil.getData(this.zookeeper, oldWalZnode);
350           LOG.debug("Creating " + wal + " with data " + Bytes.toString(logOffset));
351           String newLogZnode = ZKUtil.joinZNode(newPeerZnode, wal);
352           listOfOps.add(ZKUtilOp.createAndFailSilent(newLogZnode, logOffset));
353           // add ops for deleting
354           listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldWalZnode));
355           logQueue.add(wal);
356         }
357         // add delete op for peer
358         listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode));
359       }
360       // add delete op for dead rs, this will update the cversion of the parent.
361       // The reader will make optimistic locking with this to get a consistent
362       // snapshot
363       listOfOps.add(ZKUtilOp.deleteNodeFailSilent(deadRSZnodePath));
364       if (LOG.isTraceEnabled()) LOG.trace(" The multi list size is: " + listOfOps.size());
365       ZKUtil.multiOrSequential(this.zookeeper, listOfOps, false);
366       if (LOG.isTraceEnabled()) LOG.trace("Atomically moved the dead regionserver logs. ");
367     } catch (KeeperException e) {
368       // Multi call failed; it looks like some other regionserver took away the logs.
369       LOG.warn("Got exception in copyQueuesFromRSUsingMulti: ", e);
370       queues.clear();
371     } catch (InterruptedException e) {
372       LOG.warn("Got exception in copyQueuesFromRSUsingMulti: ", e);
373       queues.clear();
374       Thread.currentThread().interrupt();
375     }
376     return queues;
377   }
378 
379   /**
380    * This methods copies all the wals queues from another region server and returns them all sorted
381    * per peer cluster (appended with the dead server's znode)
382    * @param znode server names to copy
383    * @return all wals for all peers of that cluster, null if an error occurred
384    */
385   private SortedMap<String, SortedSet<String>> copyQueuesFromRS(String znode) {
386     // TODO this method isn't atomic enough, we could start copying and then
387     // TODO fail for some reason and we would end up with znodes we don't want.
388     SortedMap<String, SortedSet<String>> queues = new TreeMap<String, SortedSet<String>>();
389     try {
390       String nodePath = ZKUtil.joinZNode(this.queuesZNode, znode);
391       List<String> clusters = ZKUtil.listChildrenNoWatch(this.zookeeper, nodePath);
392       // We have a lock znode in there, it will count as one.
393       if (clusters == null || clusters.size() <= 1) {
394         return queues;
395       }
396       // The lock isn't a peer cluster, remove it
397       clusters.remove(RS_LOCK_ZNODE);
398       for (String cluster : clusters) {
399         ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(cluster);
400         if (!peerExists(replicationQueueInfo.getPeerId())) {
401           LOG.warn("Peer " + cluster + " didn't exist, skipping the replay");
402           // Protection against moving orphaned queues
403           continue;
404         }
405         // We add the name of the recovered RS to the new znode, we can even
406         // do that for queues that were recovered 10 times giving a znode like
407         // number-startcode-number-otherstartcode-number-anotherstartcode-etc
408         String newCluster = cluster + "-" + znode;
409         String newClusterZnode = ZKUtil.joinZNode(this.myQueuesZnode, newCluster);
410         String clusterPath = ZKUtil.joinZNode(nodePath, cluster);
411         List<String> wals = ZKUtil.listChildrenNoWatch(this.zookeeper, clusterPath);
412         // That region server didn't have anything to replicate for this cluster
413         if (wals == null || wals.size() == 0) {
414           continue;
415         }
416         ZKUtil.createNodeIfNotExistsAndWatch(this.zookeeper, newClusterZnode,
417           HConstants.EMPTY_BYTE_ARRAY);
418         SortedSet<String> logQueue = new TreeSet<String>();
419         queues.put(newCluster, logQueue);
420         for (String wal : wals) {
421           String z = ZKUtil.joinZNode(clusterPath, wal);
422           byte[] positionBytes = ZKUtil.getData(this.zookeeper, z);
423           long position = 0;
424           try {
425             position = ZKUtil.parseWALPositionFrom(positionBytes);
426           } catch (DeserializationException e) {
427             LOG.warn("Failed parse of wal position from the following znode: " + z
428                 + ", Exception: " + e);
429           }
430           LOG.debug("Creating " + wal + " with data " + position);
431           String child = ZKUtil.joinZNode(newClusterZnode, wal);
432           // Position doesn't actually change, we are just deserializing it for
433           // logging, so just use the already serialized version
434           ZKUtil.createAndWatch(this.zookeeper, child, positionBytes);
435           logQueue.add(wal);
436         }
437       }
438     } catch (KeeperException e) {
439       this.abortable.abort("Copy queues from rs", e);
440     } catch (InterruptedException e) {
441       LOG.warn(e);
442       Thread.currentThread().interrupt();
443     }
444     return queues;
445   }
446 
447   /**
448    * @param lockOwner
449    * @return Serialized protobuf of <code>lockOwner</code> with pb magic prefix prepended suitable
450    *         for use as content of an replication lock during region server fail over.
451    */
452   static byte[] lockToByteArray(final String lockOwner) {
453     byte[] bytes =
454         ZooKeeperProtos.ReplicationLock.newBuilder().setLockOwner(lockOwner).build().toByteArray();
455     return ProtobufUtil.prependPBMagic(bytes);
456   }
457 }