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.SortedSet;
24  import java.util.TreeSet;
25  
26  import org.apache.commons.logging.Log;
27  import org.apache.commons.logging.LogFactory;
28  import org.apache.hadoop.conf.Configuration;
29  import org.apache.hadoop.fs.Path;
30  import org.apache.hadoop.hbase.Abortable;
31  import org.apache.hadoop.hbase.HConstants;
32  import org.apache.hadoop.hbase.classification.InterfaceAudience;
33  import org.apache.hadoop.hbase.exceptions.DeserializationException;
34  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
35  import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
36  import org.apache.hadoop.hbase.util.Bytes;
37  import org.apache.hadoop.hbase.util.Pair;
38  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
39  import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp;
40  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
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    public 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        if (ZKUtil.checkExists(this.zookeeper, this.myQueuesZnode) < 0) {
84          ZKUtil.createWithParents(this.zookeeper, this.myQueuesZnode);
85        }
86      } catch (KeeperException e) {
87        throw new ReplicationException("Could not initialize replication queues.", e);
88      }
89      if (conf.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY,
90        HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT)) {
91        try {
92          if (ZKUtil.checkExists(this.zookeeper, this.hfileRefsZNode) < 0) {
93            ZKUtil.createWithParents(this.zookeeper, this.hfileRefsZNode);
94          }
95        } catch (KeeperException e) {
96          throw new ReplicationException("Could not initialize hfile references replication queue.",
97              e);
98        }
99      }
100   }
101 
102   @Override
103   public List<String> getListOfReplicators() throws ReplicationException {
104     try {
105       return super.getListOfReplicatorsZK();
106     } catch (KeeperException e) {
107       LOG.warn("getListOfReplicators() from ZK failed", e);
108       throw new ReplicationException("getListOfReplicators() from ZK failed", e);
109     }
110   }
111 
112   @Override
113   public void removeQueue(String queueId) {
114     try {
115       ZKUtil.deleteNodeRecursively(this.zookeeper, ZKUtil.joinZNode(this.myQueuesZnode, queueId));
116     } catch (KeeperException e) {
117       this.abortable.abort("Failed to delete queue (queueId=" + queueId + ")", e);
118     }
119   }
120 
121   @Override
122   public void addLog(String queueId, String filename) throws ReplicationException {
123     String znode = ZKUtil.joinZNode(this.myQueuesZnode, queueId);
124     znode = ZKUtil.joinZNode(znode, filename);
125     try {
126       ZKUtil.createWithParents(this.zookeeper, znode);
127     } catch (KeeperException e) {
128       throw new ReplicationException(
129           "Could not add log because znode could not be created. queueId=" + queueId
130               + ", filename=" + filename);
131     }
132   }
133 
134   @Override
135   public void removeLog(String queueId, String filename)
136     throws ReplicationSourceWithoutPeerException {
137     try {
138       try {
139         String znode = ZKUtil.joinZNode(this.myQueuesZnode, queueId);
140         znode = ZKUtil.joinZNode(znode, filename);
141         ZKUtil.deleteNode(this.zookeeper, znode);
142       } catch (KeeperException.NoNodeException e) {
143         // in case of no node exception we should not crash the region server
144         // but instead check if the replication peer has been removed.
145         // If so, we can throw here so that the source can terminate itself.
146         // This situation can occur when the replication peer znodes has been
147         // removed but the sources not terminated due to any miss from zk node delete watcher.
148         if (!doesPeerExist(queueId)) {
149           LOG.warn("Replication peer " + queueId + " has been removed", e);
150           throw new ReplicationSourceWithoutPeerException(
151             "Znodes for peer has been delete while a source is still active", e);
152         } else {
153           throw e;
154         }
155       }
156     } catch (KeeperException ke) {
157       this.abortable.abort(
158         "Failed to remove wal from queue (queueId=" + queueId + ", filename=" + filename + ")", ke);
159     }
160   }
161 
162   private boolean doesPeerExist(String queueId) throws KeeperException {
163     String peerId = queueId;
164     if (peerId.contains("-")) {
165       // queueId will be in the form peerId + "-" + rsZNode.
166       // A peerId will not have "-" in its name, see HBASE-11394
167       peerId = queueId.split("-")[0];
168     }
169 
170     return peerExists(peerId);
171   }
172 
173   @Override
174   public void setLogPosition(String queueId, String filename, long position) {
175     try {
176       String znode = ZKUtil.joinZNode(this.myQueuesZnode, queueId);
177       znode = ZKUtil.joinZNode(znode, filename);
178       // Why serialize String of Long and not Long as bytes?
179       ZKUtil.setData(this.zookeeper, znode, ZKUtil.positionToByteArray(position));
180     } catch (KeeperException e) {
181       this.abortable.abort("Failed to write replication wal position (filename=" + filename
182           + ", position=" + position + ")", e);
183     }
184   }
185 
186   @Override
187   public long getLogPosition(String queueId, String filename) throws ReplicationException {
188     String clusterZnode = ZKUtil.joinZNode(this.myQueuesZnode, queueId);
189     String znode = ZKUtil.joinZNode(clusterZnode, filename);
190     byte[] bytes = null;
191     try {
192       bytes = ZKUtil.getData(this.zookeeper, znode);
193     } catch (KeeperException e) {
194       throw new ReplicationException("Internal Error: could not get position in log for queueId="
195           + queueId + ", filename=" + filename, e);
196     } catch (InterruptedException e) {
197       Thread.currentThread().interrupt();
198       return 0;
199     }
200     try {
201       return ZKUtil.parseWALPositionFrom(bytes);
202     } catch (DeserializationException de) {
203       LOG.warn("Failed to parse WALPosition for queueId=" + queueId + " and wal=" + filename
204           + " znode content, continuing.");
205     }
206     // if we can not parse the position, start at the beginning of the wal file
207     // again
208     return 0;
209   }
210 
211   @Override
212   public boolean isThisOurZnode(String znode) {
213     return ZKUtil.joinZNode(this.queuesZNode, znode).equals(this.myQueuesZnode);
214   }
215 
216   @Override
217   public void removeAllQueues() {
218     try {
219       ZKUtil.deleteNodeRecursively(this.zookeeper, this.myQueuesZnode);
220     } catch (KeeperException e) {
221       // if the znode is already expired, don't bother going further
222       if (e instanceof KeeperException.SessionExpiredException) {
223         return;
224       }
225       this.abortable.abort("Failed to delete replication queues for region server: "
226           + this.myQueuesZnode, e);
227     }
228   }
229 
230   @Override
231   public List<String> getLogsInQueue(String queueId) {
232     String znode = ZKUtil.joinZNode(this.myQueuesZnode, queueId);
233     List<String> result = null;
234     try {
235       result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);
236     } catch (KeeperException e) {
237       this.abortable.abort("Failed to get list of wals for queueId=" + queueId, e);
238     }
239     return result;
240   }
241 
242   @Override
243   public List<String> getAllQueues() {
244     List<String> listOfQueues = null;
245     try {
246       listOfQueues = ZKUtil.listChildrenNoWatch(this.zookeeper, this.myQueuesZnode);
247     } catch (KeeperException e) {
248       this.abortable.abort("Failed to get a list of queues for region server: "
249           + this.myQueuesZnode, e);
250     }
251     return listOfQueues;
252   }
253 
254   @Override
255   public boolean isThisOurRegionServer(String regionserver) {
256     return ZKUtil.joinZNode(this.queuesZNode, regionserver).equals(this.myQueuesZnode);
257   }
258 
259   @Override
260   public List<String> getUnClaimedQueueIds(String regionserver) {
261     if (isThisOurRegionServer(regionserver)) {
262       return null;
263     }
264     String rsZnodePath = ZKUtil.joinZNode(this.queuesZNode, regionserver);
265     List<String> queues = null;
266     try {
267       queues = ZKUtil.listChildrenNoWatch(this.zookeeper, rsZnodePath);
268     } catch (KeeperException e) {
269       this.abortable.abort("Failed to getUnClaimedQueueIds for " + regionserver, e);
270     }
271     if (queues != null) {
272       queues.remove(RS_LOCK_ZNODE);
273     }
274     return queues;
275   }
276 
277   @Override
278   public Pair<String, SortedSet<String>> claimQueue(String regionserver, String queueId) {
279     if (conf.getBoolean(HConstants.ZOOKEEPER_USEMULTI, true)) {
280       LOG.info("Atomically moving " + regionserver + "/" + queueId + "'s WALs to my queue");
281       return moveQueueUsingMulti(regionserver, queueId);
282     } else {
283       LOG.info("Moving " + regionserver + "/" + queueId + "'s wals to my queue");
284       if (!lockOtherRS(regionserver)) {
285         LOG.info("Can not take the lock now");
286         return null;
287       }
288       Pair<String, SortedSet<String>> newQueues;
289       try {
290         newQueues = copyQueueFromLockedRS(regionserver, queueId);
291         removeQueueFromLockedRS(regionserver, queueId);
292       } finally {
293         unlockOtherRS(regionserver);
294       }
295       return newQueues;
296     }
297   }
298 
299   private void removeQueueFromLockedRS(String znode, String peerId) {
300     String nodePath = ZKUtil.joinZNode(this.queuesZNode, znode);
301     String peerPath = ZKUtil.joinZNode(nodePath, peerId);
302     try {
303       ZKUtil.deleteNodeRecursively(this.zookeeper, peerPath);
304     } catch (KeeperException e) {
305       LOG.warn("Remove copied queue failed", e);
306     }
307   }
308 
309   @Override
310   public void removeReplicatorIfQueueIsEmpty(String regionserver) {
311     String rsPath = ZKUtil.joinZNode(this.queuesZNode, regionserver);
312     try {
313       List<String> list = ZKUtil.listChildrenNoWatch(this.zookeeper, rsPath);
314       if (list != null && list.size() == 0){
315         ZKUtil.deleteNode(this.zookeeper, rsPath);
316       }
317     } catch (KeeperException e) {
318       LOG.warn("Got error while removing replicator", e);
319     }
320   }
321 
322   /**
323    * Try to set a lock in another region server's znode.
324    * @param znode the server names of the other server
325    * @return true if the lock was acquired, false in every other cases
326    */
327   public boolean lockOtherRS(String znode) {
328     try {
329       String parent = ZKUtil.joinZNode(this.queuesZNode, znode);
330       if (parent.equals(this.myQueuesZnode)) {
331         LOG.warn("Won't lock because this is us, we're dead!");
332         return false;
333       }
334       String p = ZKUtil.joinZNode(parent, RS_LOCK_ZNODE);
335       ZKUtil.createAndWatch(this.zookeeper, p, lockToByteArray(this.myQueuesZnode));
336     } catch (KeeperException e) {
337       // This exception will pop up if the znode under which we're trying to
338       // create the lock is already deleted by another region server, meaning
339       // that the transfer already occurred.
340       // NoNode => transfer is done and znodes are already deleted
341       // NodeExists => lock znode already created by another RS
342       if (e instanceof KeeperException.NoNodeException
343           || e instanceof KeeperException.NodeExistsException) {
344         LOG.info("Won't transfer the queue," + " another RS took care of it because of: "
345             + e.getMessage());
346       } else {
347         LOG.info("Failed lock other rs", e);
348       }
349       return false;
350     }
351     return true;
352   }
353 
354   public String getLockZNode(String znode) {
355     return this.queuesZNode + "/" + znode + "/" + RS_LOCK_ZNODE;
356   }
357 
358   public boolean checkLockExists(String znode) throws KeeperException {
359     return ZKUtil.checkExists(zookeeper, getLockZNode(znode)) >= 0;
360   }
361 
362   private void unlockOtherRS(String znode){
363     String parent = ZKUtil.joinZNode(this.queuesZNode, znode);
364     String p = ZKUtil.joinZNode(parent, RS_LOCK_ZNODE);
365     try {
366       ZKUtil.deleteNode(this.zookeeper, p);
367     } catch (KeeperException e) {
368       this.abortable.abort("Remove lock failed", e);
369     }
370   }
371 
372   /**
373    * Delete all the replication queues for a given region server.
374    * @param regionserverZnode The znode of the region server to delete.
375    */
376   private void deleteAnotherRSQueues(String regionserverZnode) {
377     String fullpath = ZKUtil.joinZNode(this.queuesZNode, regionserverZnode);
378     try {
379       List<String> clusters = ZKUtil.listChildrenNoWatch(this.zookeeper, fullpath);
380       for (String cluster : clusters) {
381         // No need to delete, it will be deleted later.
382         if (cluster.equals(RS_LOCK_ZNODE)) {
383           continue;
384         }
385         String fullClusterPath = ZKUtil.joinZNode(fullpath, cluster);
386         ZKUtil.deleteNodeRecursively(this.zookeeper, fullClusterPath);
387       }
388       // Finish cleaning up
389       ZKUtil.deleteNodeRecursively(this.zookeeper, fullpath);
390     } catch (KeeperException e) {
391       if (e instanceof KeeperException.NoNodeException
392           || e instanceof KeeperException.NotEmptyException) {
393         // Testing a special case where another region server was able to
394         // create a lock just after we deleted it, but then was also able to
395         // delete the RS znode before us or its lock znode is still there.
396         if (e.getPath().equals(fullpath)) {
397           return;
398         }
399       }
400       this.abortable.abort("Failed to delete replication queues for region server: "
401           + regionserverZnode, e);
402     }
403   }
404 
405   /**
406    * It "atomically" copies one peer's wals queue from another dead region server and returns them
407    * all sorted. The new peer id is equal to the old peer id appended with the dead server's znode.
408    * @param znode pertaining to the region server to copy the queues from
409    * @peerId peerId pertaining to the queue need to be copied
410    */
411   private Pair<String, SortedSet<String>> moveQueueUsingMulti(String znode, String peerId) {
412     try {
413       // hbase/replication/rs/deadrs
414       String deadRSZnodePath = ZKUtil.joinZNode(this.queuesZNode, znode);
415       List<ZKUtilOp> listOfOps = new ArrayList<>();
416       ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(peerId);
417 
418       String newPeerId = peerId + "-" + znode;
419       String newPeerZnode = ZKUtil.joinZNode(this.myQueuesZnode, newPeerId);
420       // check the logs queue for the old peer cluster
421       String oldClusterZnode = ZKUtil.joinZNode(deadRSZnodePath, peerId);
422       List<String> wals = ZKUtil.listChildrenNoWatch(this.zookeeper, oldClusterZnode);
423 
424       if (!peerExists(replicationQueueInfo.getPeerId())) {
425         LOG.warn("Peer " + replicationQueueInfo.getPeerId() +
426                 " didn't exist, will move its queue to avoid the failure of multi op");
427         for (String wal : wals) {
428           String oldWalZnode = ZKUtil.joinZNode(oldClusterZnode, wal);
429           listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldWalZnode));
430         }
431         listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode));
432         ZKUtil.multiOrSequential(this.zookeeper, listOfOps, false);
433         return null;
434       }
435 
436       SortedSet<String> logQueue = new TreeSet<>();
437       if (wals == null || wals.size() == 0) {
438         listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode));
439       } else {
440         // create the new cluster znode
441         ZKUtilOp op = ZKUtilOp.createAndFailSilent(newPeerZnode, HConstants.EMPTY_BYTE_ARRAY);
442         listOfOps.add(op);
443         // get the offset of the logs and set it to new znodes
444         for (String wal : wals) {
445           String oldWalZnode = ZKUtil.joinZNode(oldClusterZnode, wal);
446           byte[] logOffset = ZKUtil.getData(this.zookeeper, oldWalZnode);
447           LOG.debug("Creating " + wal + " with data " + Bytes.toString(logOffset));
448           String newLogZnode = ZKUtil.joinZNode(newPeerZnode, wal);
449           listOfOps.add(ZKUtilOp.createAndFailSilent(newLogZnode, logOffset));
450           listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldWalZnode));
451           logQueue.add(wal);
452         }
453         // add delete op for peer
454         listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode));
455 
456         if (LOG.isTraceEnabled()) {
457           LOG.trace(" The multi list size is: " + listOfOps.size());
458         }
459       }
460       ZKUtil.multiOrSequential(this.zookeeper, listOfOps, false);
461 
462       LOG.info("Atomically moved " + znode + "/" + peerId + "'s WALs to my queue");
463       return new Pair<>(newPeerId, logQueue);
464     } catch (KeeperException e) {
465       // Multi call failed; it looks like some other regionserver took away the logs.
466       LOG.warn("Got exception in copyQueuesFromRSUsingMulti: ", e);
467     } catch (InterruptedException e) {
468       LOG.warn("Got exception in copyQueuesFromRSUsingMulti: ", e);
469       Thread.currentThread().interrupt();
470     }
471     return null;
472   }
473 
474   /**
475    * This methods moves all the wals queues from another region server and returns them all sorted
476    * per peer cluster (appended with the dead server's znode)
477    * @param znode server names to copy
478    * @return all wals for the peer of that cluster, null if an error occurred
479    */
480   private Pair<String, SortedSet<String>> copyQueueFromLockedRS(String znode, String peerId) {
481     // TODO this method isn't atomic enough, we could start copying and then
482     // TODO fail for some reason and we would end up with znodes we don't want.
483     try {
484       String nodePath = ZKUtil.joinZNode(this.queuesZNode, znode);
485       ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(peerId);
486       String clusterPath = ZKUtil.joinZNode(nodePath, peerId);
487       if (!peerExists(replicationQueueInfo.getPeerId())) {
488         LOG.warn("Peer " + peerId + " didn't exist, skipping the replay");
489         // Protection against moving orphaned queues
490         return null;
491       }
492       // We add the name of the recovered RS to the new znode, we can even
493       // do that for queues that were recovered 10 times giving a znode like
494       // number-startcode-number-otherstartcode-number-anotherstartcode-etc
495       String newCluster = peerId + "-" + znode;
496       String newClusterZnode = ZKUtil.joinZNode(this.myQueuesZnode, newCluster);
497 
498       List<String> wals = ZKUtil.listChildrenNoWatch(this.zookeeper, clusterPath);
499       // That region server didn't have anything to replicate for this cluster
500       if (wals == null || wals.size() == 0) {
501         return null;
502       }
503       ZKUtil.createNodeIfNotExistsAndWatch(this.zookeeper, newClusterZnode,
504           HConstants.EMPTY_BYTE_ARRAY);
505       SortedSet<String> logQueue = new TreeSet<>();
506       for (String wal : wals) {
507         String z = ZKUtil.joinZNode(clusterPath, wal);
508         byte[] positionBytes = ZKUtil.getData(this.zookeeper, z);
509         long position = 0;
510         try {
511           position = ZKUtil.parseWALPositionFrom(positionBytes);
512         } catch (DeserializationException e) {
513           LOG.warn("Failed parse of wal position from the following znode: " + z
514               + ", Exception: " + e);
515         }
516         LOG.debug("Creating " + wal + " with data " + position);
517         String child = ZKUtil.joinZNode(newClusterZnode, wal);
518         // Position doesn't actually change, we are just deserializing it for
519         // logging, so just use the already serialized version
520         ZKUtil.createNodeIfNotExistsAndWatch(this.zookeeper, child, positionBytes);
521         logQueue.add(wal);
522       }
523       return new Pair<>(newCluster, logQueue);
524     } catch (KeeperException e) {
525       LOG.warn("Got exception in copyQueueFromLockedRS: "+
526         " Possible problem: check if znode size exceeds jute.maxBuffer value. "
527           + "If so, increase it for both client and server side." ,e);
528 
529     } catch (InterruptedException e) {
530       LOG.warn(e);
531       Thread.currentThread().interrupt();
532     }
533     return null;
534   }
535 
536   /**
537    * @param lockOwner lock owner
538    * @return Serialized protobuf of <code>lockOwner</code> with pb magic prefix prepended suitable
539    *         for use as content of an replication lock during region server fail over.
540    */
541   static byte[] lockToByteArray(final String lockOwner) {
542     byte[] bytes =
543         ZooKeeperProtos.ReplicationLock.newBuilder().setLockOwner(lockOwner).build().toByteArray();
544     return ProtobufUtil.prependPBMagic(bytes);
545   }
546 
547   @Override
548   public void addHFileRefs(String peerId, List<Pair<Path, Path>> pairs)
549       throws ReplicationException {
550     String peerZnode = ZKUtil.joinZNode(this.hfileRefsZNode, peerId);
551     boolean debugEnabled = LOG.isDebugEnabled();
552     if (debugEnabled) {
553       LOG.debug("Adding hfile references " + pairs + " in queue " + peerZnode);
554     }
555     List<ZKUtilOp> listOfOps = new ArrayList<ZKUtil.ZKUtilOp>();
556     int size = pairs.size();
557     for (int i = 0; i < size; i++) {
558       listOfOps.add(ZKUtilOp.createAndFailSilent(
559         ZKUtil.joinZNode(peerZnode, pairs.get(i).getSecond().getName()),
560         HConstants.EMPTY_BYTE_ARRAY));
561     }
562     if (debugEnabled) {
563       LOG.debug(" The multi list size for adding hfile references in zk for node " + peerZnode
564           + " is " + listOfOps.size());
565     }
566     try {
567       ZKUtil.multiOrSequential(this.zookeeper, listOfOps, true);
568     } catch (KeeperException e) {
569       throw new ReplicationException("Failed to create hfile reference znode=" + e.getPath(), e);
570     }
571   }
572 
573   @Override
574   public void removeHFileRefs(String peerId, List<String> files) {
575     String peerZnode = ZKUtil.joinZNode(this.hfileRefsZNode, peerId);
576     boolean debugEnabled = LOG.isDebugEnabled();
577     if (debugEnabled) {
578       LOG.debug("Removing hfile references " + files + " from queue " + peerZnode);
579     }
580     List<ZKUtilOp> listOfOps = new ArrayList<ZKUtil.ZKUtilOp>();
581     int size = files.size();
582     for (int i = 0; i < size; i++) {
583       listOfOps.add(ZKUtilOp.deleteNodeFailSilent(ZKUtil.joinZNode(peerZnode, files.get(i))));
584     }
585     if (debugEnabled) {
586       LOG.debug(" The multi list size for removing hfile references in zk for node " + peerZnode
587           + " is " + listOfOps.size());
588     }
589     try {
590       ZKUtil.multiOrSequential(this.zookeeper, listOfOps, true);
591     } catch (KeeperException e) {
592       LOG.error("Failed to remove hfile reference znode=" + e.getPath(), e);
593     }
594   }
595 
596   @Override
597   public void addPeerToHFileRefs(String peerId) throws ReplicationException {
598     String peerZnode = ZKUtil.joinZNode(this.hfileRefsZNode, peerId);
599     try {
600       if (ZKUtil.checkExists(this.zookeeper, peerZnode) == -1) {
601         LOG.info("Adding peer " + peerId + " to hfile reference queue.");
602         ZKUtil.createWithParents(this.zookeeper, peerZnode);
603       }
604     } catch (KeeperException e) {
605       throw new ReplicationException("Failed to add peer " + peerId + " to hfile reference queue.",
606           e);
607     }
608   }
609 
610   @Override
611   public void removePeerFromHFileRefs(String peerId) {
612     final String peerZnode = ZKUtil.joinZNode(this.hfileRefsZNode, peerId);
613     try {
614       if (ZKUtil.checkExists(this.zookeeper, peerZnode) == -1) {
615         if (LOG.isDebugEnabled()) {
616           LOG.debug("Peer " + peerZnode + " not found in hfile reference queue.");
617         }
618         return;
619       } else {
620         LOG.info("Removing peer " + peerZnode + " from hfile reference queue.");
621         ZKUtil.deleteNodeRecursively(this.zookeeper, peerZnode);
622       }
623     } catch (KeeperException e) {
624       LOG.error("Ignoring the exception to remove peer " + peerId + " from hfile reference queue.",
625         e);
626     }
627   }
628 }