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