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