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    private 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 void removeQueue(String queueId) {
92      try {
93        ZKUtil.deleteNodeRecursively(this.zookeeper, ZKUtil.joinZNode(this.myQueuesZnode, queueId));
94      } catch (KeeperException e) {
95        this.abortable.abort("Failed to delete queue (queueId=" + queueId + ")", e);
96      }
97    }
98  
99    @Override
100   public void addLog(String queueId, String filename) throws ReplicationException {
101     String znode = ZKUtil.joinZNode(this.myQueuesZnode, queueId);
102     znode = ZKUtil.joinZNode(znode, filename);
103     try {
104       ZKUtil.createWithParents(this.zookeeper, znode);
105     } catch (KeeperException e) {
106       throw new ReplicationException(
107           "Could not add log because znode could not be created. queueId=" + queueId
108               + ", filename=" + filename);
109     }
110   }
111 
112   @Override
113   public void removeLog(String queueId, String filename) {
114     try {
115       String znode = ZKUtil.joinZNode(this.myQueuesZnode, queueId);
116       znode = ZKUtil.joinZNode(znode, filename);
117       ZKUtil.deleteNode(this.zookeeper, znode);
118     } catch (KeeperException e) {
119       this.abortable.abort("Failed to remove wal from queue (queueId=" + queueId + ", filename="
120           + filename + ")", e);
121     }
122   }
123 
124   @Override
125   public void setLogPosition(String queueId, String filename, long position) {
126     try {
127       String znode = ZKUtil.joinZNode(this.myQueuesZnode, queueId);
128       znode = ZKUtil.joinZNode(znode, filename);
129       // Why serialize String of Long and not Long as bytes?
130       ZKUtil.setData(this.zookeeper, znode, ZKUtil.positionToByteArray(position));
131     } catch (KeeperException e) {
132       this.abortable.abort("Failed to write replication wal position (filename=" + filename
133           + ", position=" + position + ")", e);
134     }
135   }
136 
137   @Override
138   public long getLogPosition(String queueId, String filename) throws ReplicationException {
139     String clusterZnode = ZKUtil.joinZNode(this.myQueuesZnode, queueId);
140     String znode = ZKUtil.joinZNode(clusterZnode, filename);
141     byte[] bytes = null;
142     try {
143       bytes = ZKUtil.getData(this.zookeeper, znode);
144     } catch (KeeperException e) {
145       throw new ReplicationException("Internal Error: could not get position in log for queueId="
146           + queueId + ", filename=" + filename, e);
147     } catch (InterruptedException e) {
148       Thread.currentThread().interrupt();
149       return 0;
150     }
151     try {
152       return ZKUtil.parseWALPositionFrom(bytes);
153     } catch (DeserializationException de) {
154       LOG.warn("Failed to parse WALPosition for queueId=" + queueId + " and wal=" + filename
155           + "znode content, continuing.");
156     }
157     // if we can not parse the position, start at the beginning of the wal file
158     // again
159     return 0;
160   }
161 
162   @Override
163   public boolean isThisOurZnode(String znode) {
164     return ZKUtil.joinZNode(this.queuesZNode, znode).equals(this.myQueuesZnode);
165   }
166 
167   @Override
168   public SortedMap<String, SortedSet<String>> claimQueues(String regionserverZnode) {
169     SortedMap<String, SortedSet<String>> newQueues = new TreeMap<String, SortedSet<String>>();
170     // check whether there is multi support. If yes, use it.
171     if (conf.getBoolean(HConstants.ZOOKEEPER_USEMULTI, true)) {
172       LOG.info("Atomically moving " + regionserverZnode + "'s wals to my queue");
173       newQueues = copyQueuesFromRSUsingMulti(regionserverZnode);
174     } else {
175       LOG.info("Moving " + regionserverZnode + "'s wals to my queue");
176       if (!lockOtherRS(regionserverZnode)) {
177         return newQueues;
178       }
179       newQueues = copyQueuesFromRS(regionserverZnode);
180       deleteAnotherRSQueues(regionserverZnode);
181     }
182     return newQueues;
183   }
184 
185   @Override
186   public void removeAllQueues() {
187     try {
188       ZKUtil.deleteNodeRecursively(this.zookeeper, this.myQueuesZnode);
189     } catch (KeeperException e) {
190       // if the znode is already expired, don't bother going further
191       if (e instanceof KeeperException.SessionExpiredException) {
192         return;
193       }
194       this.abortable.abort("Failed to delete replication queues for region server: "
195           + this.myQueuesZnode, e);
196     }
197   }
198 
199   @Override
200   public List<String> getLogsInQueue(String queueId) {
201     String znode = ZKUtil.joinZNode(this.myQueuesZnode, queueId);
202     List<String> result = null;
203     try {
204       result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);
205     } catch (KeeperException e) {
206       this.abortable.abort("Failed to get list of wals for queueId=" + queueId, e);
207     }
208     return result;
209   }
210 
211   @Override
212   public List<String> getAllQueues() {
213     List<String> listOfQueues = null;
214     try {
215       listOfQueues = ZKUtil.listChildrenNoWatch(this.zookeeper, this.myQueuesZnode);
216     } catch (KeeperException e) {
217       this.abortable.abort("Failed to get a list of queues for region server: "
218           + this.myQueuesZnode, e);
219     }
220     return listOfQueues;
221   }
222 
223   /**
224    * Try to set a lock in another region server's znode.
225    * @param znode the server names of the other server
226    * @return true if the lock was acquired, false in every other cases
227    */
228   @VisibleForTesting
229   public boolean lockOtherRS(String znode) {
230     try {
231       String parent = ZKUtil.joinZNode(this.queuesZNode, znode);
232       if (parent.equals(this.myQueuesZnode)) {
233         LOG.warn("Won't lock because this is us, we're dead!");
234         return false;
235       }
236       String p = ZKUtil.joinZNode(parent, RS_LOCK_ZNODE);
237       ZKUtil.createAndWatch(this.zookeeper, p, lockToByteArray(this.myQueuesZnode));
238     } catch (KeeperException e) {
239       // This exception will pop up if the znode under which we're trying to
240       // create the lock is already deleted by another region server, meaning
241       // that the transfer already occurred.
242       // NoNode => transfer is done and znodes are already deleted
243       // NodeExists => lock znode already created by another RS
244       if (e instanceof KeeperException.NoNodeException
245           || e instanceof KeeperException.NodeExistsException) {
246         LOG.info("Won't transfer the queue," + " another RS took care of it because of: "
247             + e.getMessage());
248       } else {
249         LOG.info("Failed lock other rs", e);
250       }
251       return false;
252     }
253     return true;
254   }
255 
256   public String getLockZNode(String znode) {
257     return this.queuesZNode + "/" + znode + "/" + RS_LOCK_ZNODE;
258   }
259 
260   @VisibleForTesting
261   public boolean checkLockExists(String znode) throws KeeperException {
262     return ZKUtil.checkExists(zookeeper, getLockZNode(znode)) >= 0;
263   }
264 
265   /**
266    * Delete all the replication queues for a given region server.
267    * @param regionserverZnode The znode of the region server to delete.
268    */
269   private void deleteAnotherRSQueues(String regionserverZnode) {
270     String fullpath = ZKUtil.joinZNode(this.queuesZNode, regionserverZnode);
271     try {
272       List<String> clusters = ZKUtil.listChildrenNoWatch(this.zookeeper, fullpath);
273       for (String cluster : clusters) {
274         // No need to delete, it will be deleted later.
275         if (cluster.equals(RS_LOCK_ZNODE)) {
276           continue;
277         }
278         String fullClusterPath = ZKUtil.joinZNode(fullpath, cluster);
279         ZKUtil.deleteNodeRecursively(this.zookeeper, fullClusterPath);
280       }
281       // Finish cleaning up
282       ZKUtil.deleteNodeRecursively(this.zookeeper, fullpath);
283     } catch (KeeperException e) {
284       if (e instanceof KeeperException.NoNodeException
285           || e instanceof KeeperException.NotEmptyException) {
286         // Testing a special case where another region server was able to
287         // create a lock just after we deleted it, but then was also able to
288         // delete the RS znode before us or its lock znode is still there.
289         if (e.getPath().equals(fullpath)) {
290           return;
291         }
292       }
293       this.abortable.abort("Failed to delete replication queues for region server: "
294           + regionserverZnode, e);
295     }
296   }
297 
298   /**
299    * It "atomically" copies all the wals queues from another region server and returns them all
300    * sorted per peer cluster (appended with the dead server's znode).
301    * @param znode pertaining to the region server to copy the queues from
302    * @return WAL queues sorted per peer cluster
303    */
304   private SortedMap<String, SortedSet<String>> copyQueuesFromRSUsingMulti(String znode) {
305     SortedMap<String, SortedSet<String>> queues = new TreeMap<String, SortedSet<String>>();
306     // hbase/replication/rs/deadrs
307     String deadRSZnodePath = ZKUtil.joinZNode(this.queuesZNode, znode);
308     List<String> peerIdsToProcess = null;
309     List<ZKUtilOp> listOfOps = new ArrayList<ZKUtil.ZKUtilOp>();
310     try {
311       peerIdsToProcess = ZKUtil.listChildrenNoWatch(this.zookeeper, deadRSZnodePath);
312       if (peerIdsToProcess == null) return queues; // node already processed
313       for (String peerId : peerIdsToProcess) {
314         ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(peerId);
315         if (!peerExists(replicationQueueInfo.getPeerId())) {
316           // the orphaned queues must be moved, otherwise the delete op of dead rs will fail,
317           // this will cause the whole multi op fail.
318           // NodeFailoverWorker will skip the orphaned queues.
319           LOG.warn("Peer " + peerId
320               + " didn't exist, will move its queue to avoid the failure of multi op");
321         }
322         String newPeerId = peerId + "-" + znode;
323         String newPeerZnode = ZKUtil.joinZNode(this.myQueuesZnode, newPeerId);
324         // check the logs queue for the old peer cluster
325         String oldClusterZnode = ZKUtil.joinZNode(deadRSZnodePath, peerId);
326         List<String> wals = ZKUtil.listChildrenNoWatch(this.zookeeper, oldClusterZnode);
327         if (wals == null || wals.size() == 0) {
328           listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode));
329           continue; // empty log queue.
330         }
331         // create the new cluster znode
332         SortedSet<String> logQueue = new TreeSet<String>();
333         queues.put(newPeerId, logQueue);
334         ZKUtilOp op = ZKUtilOp.createAndFailSilent(newPeerZnode, HConstants.EMPTY_BYTE_ARRAY);
335         listOfOps.add(op);
336         // get the offset of the logs and set it to new znodes
337         for (String wal : wals) {
338           String oldWalZnode = ZKUtil.joinZNode(oldClusterZnode, wal);
339           byte[] logOffset = ZKUtil.getData(this.zookeeper, oldWalZnode);
340           LOG.debug("Creating " + wal + " with data " + Bytes.toString(logOffset));
341           String newLogZnode = ZKUtil.joinZNode(newPeerZnode, wal);
342           listOfOps.add(ZKUtilOp.createAndFailSilent(newLogZnode, logOffset));
343           // add ops for deleting
344           listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldWalZnode));
345           logQueue.add(wal);
346         }
347         // add delete op for peer
348         listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode));
349       }
350       // add delete op for dead rs, this will update the cversion of the parent.
351       // The reader will make optimistic locking with this to get a consistent
352       // snapshot
353       listOfOps.add(ZKUtilOp.deleteNodeFailSilent(deadRSZnodePath));
354       LOG.debug(" The multi list size is: " + listOfOps.size());
355       ZKUtil.multiOrSequential(this.zookeeper, listOfOps, false);
356       LOG.info("Atomically moved the dead regionserver logs. ");
357     } catch (KeeperException e) {
358       // Multi call failed; it looks like some other regionserver took away the logs.
359       LOG.warn("Got exception in copyQueuesFromRSUsingMulti: ", e);
360       queues.clear();
361     } catch (InterruptedException e) {
362       LOG.warn("Got exception in copyQueuesFromRSUsingMulti: ", e);
363       queues.clear();
364       Thread.currentThread().interrupt();
365     }
366     return queues;
367   }
368 
369   /**
370    * This methods copies all the wals queues from another region server and returns them all sorted
371    * per peer cluster (appended with the dead server's znode)
372    * @param znode server names to copy
373    * @return all wals for all peers of that cluster, null if an error occurred
374    */
375   private SortedMap<String, SortedSet<String>> copyQueuesFromRS(String znode) {
376     // TODO this method isn't atomic enough, we could start copying and then
377     // TODO fail for some reason and we would end up with znodes we don't want.
378     SortedMap<String, SortedSet<String>> queues = new TreeMap<String, SortedSet<String>>();
379     try {
380       String nodePath = ZKUtil.joinZNode(this.queuesZNode, znode);
381       List<String> clusters = ZKUtil.listChildrenNoWatch(this.zookeeper, nodePath);
382       // We have a lock znode in there, it will count as one.
383       if (clusters == null || clusters.size() <= 1) {
384         return queues;
385       }
386       // The lock isn't a peer cluster, remove it
387       clusters.remove(RS_LOCK_ZNODE);
388       for (String cluster : clusters) {
389         ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(cluster);
390         if (!peerExists(replicationQueueInfo.getPeerId())) {
391           LOG.warn("Peer " + cluster + " didn't exist, skipping the replay");
392           // Protection against moving orphaned queues
393           continue;
394         }
395         // We add the name of the recovered RS to the new znode, we can even
396         // do that for queues that were recovered 10 times giving a znode like
397         // number-startcode-number-otherstartcode-number-anotherstartcode-etc
398         String newCluster = cluster + "-" + znode;
399         String newClusterZnode = ZKUtil.joinZNode(this.myQueuesZnode, newCluster);
400         String clusterPath = ZKUtil.joinZNode(nodePath, cluster);
401         List<String> wals = ZKUtil.listChildrenNoWatch(this.zookeeper, clusterPath);
402         // That region server didn't have anything to replicate for this cluster
403         if (wals == null || wals.size() == 0) {
404           continue;
405         }
406         ZKUtil.createNodeIfNotExistsAndWatch(this.zookeeper, newClusterZnode,
407           HConstants.EMPTY_BYTE_ARRAY);
408         SortedSet<String> logQueue = new TreeSet<String>();
409         queues.put(newCluster, logQueue);
410         for (String wal : wals) {
411           String z = ZKUtil.joinZNode(clusterPath, wal);
412           byte[] positionBytes = ZKUtil.getData(this.zookeeper, z);
413           long position = 0;
414           try {
415             position = ZKUtil.parseWALPositionFrom(positionBytes);
416           } catch (DeserializationException e) {
417             LOG.warn("Failed parse of wal position from the following znode: " + z
418                 + ", Exception: " + e);
419           }
420           LOG.debug("Creating " + wal + " with data " + position);
421           String child = ZKUtil.joinZNode(newClusterZnode, wal);
422           // Position doesn't actually change, we are just deserializing it for
423           // logging, so just use the already serialized version
424           ZKUtil.createAndWatch(this.zookeeper, child, positionBytes);
425           logQueue.add(wal);
426         }
427       }
428     } catch (KeeperException e) {
429       this.abortable.abort("Copy queues from rs", e);
430     } catch (InterruptedException e) {
431       LOG.warn(e);
432       Thread.currentThread().interrupt();
433     }
434     return queues;
435   }
436 
437   /**
438    * @param lockOwner
439    * @return Serialized protobuf of <code>lockOwner</code> with pb magic prefix prepended suitable
440    *         for use as content of an replication lock during region server fail over.
441    */
442   static byte[] lockToByteArray(final String lockOwner) {
443     byte[] bytes =
444         ZooKeeperProtos.ReplicationLock.newBuilder().setLockOwner(lockOwner).build().toByteArray();
445     return ProtobufUtil.prependPBMagic(bytes);
446   }
447 }