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.io.Closeable;
22  import java.io.IOException;
23  import java.util.HashMap;
24  import java.util.HashSet;
25  import java.util.List;
26  import java.util.Map;
27  import java.util.Set;
28  
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.TableName;
35  import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
36  import org.apache.hadoop.hbase.exceptions.DeserializationException;
37  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
38  import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
39  import org.apache.hadoop.hbase.util.Bytes;
40  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
41  import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker;
42  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
43  import org.apache.zookeeper.KeeperException;
44  import org.apache.zookeeper.KeeperException.NodeExistsException;
45  
46  @InterfaceAudience.Private
47  public class ReplicationPeerZKImpl extends ReplicationStateZKBase implements ReplicationPeer,
48      Abortable, Closeable {
49    private static final Log LOG = LogFactory.getLog(ReplicationPeerZKImpl.class);
50  
51    private ReplicationPeerConfig peerConfig;
52    private final String id;
53    private volatile PeerState peerState;
54    private volatile Map<TableName, List<String>> tableCFs = new HashMap<TableName, List<String>>();
55    private final Configuration conf;
56  
57    private PeerStateTracker peerStateTracker;
58    private PeerConfigTracker peerConfigTracker;
59  
60    /**
61     * Constructor that takes all the objects required to communicate with the specified peer, except
62     * for the region server addresses.
63     * @param conf configuration object to this peer
64     * @param id string representation of this peer's identifier
65     * @param peerConfig configuration for the replication peer
66     */
67    public ReplicationPeerZKImpl(ZooKeeperWatcher zkWatcher, Configuration conf, String id,
68        ReplicationPeerConfig peerConfig, Abortable abortable) throws ReplicationException {
69      super(zkWatcher, conf, abortable);
70      this.conf = conf;
71      this.peerConfig = peerConfig;
72      this.id = id;
73    }
74  
75    /**
76     * start a state tracker to check whether this peer is enabled or not
77     *
78     * @param peerStateNode path to zk node which stores peer state
79     * @throws KeeperException
80     */
81    public void startStateTracker(String peerStateNode)
82        throws KeeperException {
83      ensurePeerEnabled(peerStateNode);
84      this.peerStateTracker = new PeerStateTracker(peerStateNode, zookeeper, this);
85      this.peerStateTracker.start();
86      try {
87        this.readPeerStateZnode();
88      } catch (DeserializationException e) {
89        throw ZKUtil.convert(e);
90      }
91    }
92  
93    private void readPeerStateZnode() throws DeserializationException {
94      this.peerState =
95          isStateEnabled(this.peerStateTracker.getData(false))
96            ? PeerState.ENABLED
97            : PeerState.DISABLED;
98    }
99  
100   /**
101    * start a table-cfs tracker to listen the (table, cf-list) map change
102    * @param zookeeper
103    * @param peerConfigNode path to zk node which stores table-cfs
104    * @throws KeeperException
105    */
106   public void startPeerConfigTracker(ZooKeeperWatcher zookeeper, String peerConfigNode)
107       throws KeeperException {
108     this.peerConfigTracker = new PeerConfigTracker(peerConfigNode, zookeeper,
109         this);
110     this.peerConfigTracker.start();
111     this.readPeerConfig();
112   }
113 
114   private ReplicationPeerConfig readPeerConfig() {
115     try {
116       byte[] data = peerConfigTracker.getData(false);
117       if (data != null) {
118         this.peerConfig = ReplicationSerDeHelper.parsePeerFrom(data);
119       }
120     } catch (DeserializationException e) {
121       LOG.error("", e);
122     }
123     return this.peerConfig;
124   }
125 
126   @Override
127   public PeerState getPeerState() {
128     return peerState;
129   }
130 
131   /**
132    * Get the identifier of this peer
133    * @return string representation of the id (short)
134    */
135   @Override
136   public String getId() {
137     return id;
138   }
139 
140   /**
141    * Get the peer config object
142    * @return the ReplicationPeerConfig for this peer
143    */
144   @Override
145   public ReplicationPeerConfig getPeerConfig() {
146     return peerConfig;
147   }
148 
149   /**
150    * Get the configuration object required to communicate with this peer
151    * @return configuration object
152    */
153   @Override
154   public Configuration getConfiguration() {
155     return conf;
156   }
157 
158   /**
159    * Get replicable (table, cf-list) map of this peer
160    * @return the replicable (table, cf-list) map
161    */
162   @Override
163   public Map<TableName, List<String>> getTableCFs() {
164     this.tableCFs = peerConfig.getTableCFsMap();
165     return this.tableCFs;
166   }
167 
168 
169   @Override
170   public long getPeerBandwidth() {
171     return this.peerConfig.getBandwidth();
172   }
173 
174   @Override
175   public void trackPeerConfigChanges(ReplicationPeerConfigListener listener) {
176     if (this.peerConfigTracker != null){
177       this.peerConfigTracker.addListener(listener);
178     }
179   }
180 
181   @Override
182   public void removeListenerOfPeerConfig(ReplicationPeerConfigListener listener) {
183     if (this.peerConfigTracker != null){
184       this.peerConfigTracker.removeListener(listener);
185     }
186   }
187 
188   PeerConfigTracker getPeerConfigTracker() {
189     return this.peerConfigTracker;
190   }
191 
192   @Override
193   public void abort(String why, Throwable e) {
194     LOG.fatal("The ReplicationPeer corresponding to peer " + peerConfig
195         + " was aborted for the following reason(s):" + why, e);
196   }
197 
198   @Override
199   public boolean isAborted() {
200     // Currently the replication peer is never "Aborted", we just log when the
201     // abort method is called.
202     return false;
203   }
204 
205   @Override
206   public void close() throws IOException {
207     // TODO: stop zkw?
208   }
209 
210   /**
211    * Parse the raw data from ZK to get a peer's state
212    * @param bytes raw ZK data
213    * @return True if the passed in <code>bytes</code> are those of a pb serialized ENABLED state.
214    * @throws DeserializationException
215    */
216   public static boolean isStateEnabled(final byte[] bytes) throws DeserializationException {
217     ZooKeeperProtos.ReplicationState.State state = parseStateFrom(bytes);
218     return ZooKeeperProtos.ReplicationState.State.ENABLED == state;
219   }
220 
221   /**
222    * @param bytes Content of a state znode.
223    * @return State parsed from the passed bytes.
224    * @throws DeserializationException
225    */
226   private static ZooKeeperProtos.ReplicationState.State parseStateFrom(final byte[] bytes)
227       throws DeserializationException {
228     ProtobufUtil.expectPBMagicPrefix(bytes);
229     int pblen = ProtobufUtil.lengthOfPBMagic();
230     ZooKeeperProtos.ReplicationState.Builder builder =
231         ZooKeeperProtos.ReplicationState.newBuilder();
232     ZooKeeperProtos.ReplicationState state;
233     try {
234       ProtobufUtil.mergeFrom(builder, bytes, pblen, bytes.length - pblen);
235       state = builder.build();
236       return state.getState();
237     } catch (IOException e) {
238       throw new DeserializationException(e);
239     }
240   }
241 
242   /**
243    * Utility method to ensure an ENABLED znode is in place; if not present, we create it.
244    * @param zookeeper
245    * @param path Path to znode to check
246    * @return True if we created the znode.
247    * @throws NodeExistsException
248    * @throws KeeperException
249    */
250   private boolean ensurePeerEnabled(final String path)
251       throws NodeExistsException, KeeperException {
252     if (ZKUtil.checkExists(zookeeper, path) == -1) {
253       // There is a race b/w PeerWatcher and ReplicationZookeeper#add method to create the
254       // peer-state znode. This happens while adding a peer.
255       // The peer state data is set as "ENABLED" by default.
256       ZKUtil.createNodeIfNotExistsAndWatch(zookeeper, path,
257         ReplicationStateZKBase.ENABLED_ZNODE_BYTES);
258       return true;
259     }
260     return false;
261   }
262 
263   /**
264    * Tracker for state of this peer
265    */
266   public class PeerStateTracker extends ZooKeeperNodeTracker {
267 
268     public PeerStateTracker(String peerStateZNode, ZooKeeperWatcher watcher,
269         Abortable abortable) {
270       super(watcher, peerStateZNode, abortable);
271     }
272 
273     @Override
274     public synchronized void nodeDataChanged(String path) {
275       if (path.equals(node)) {
276         super.nodeDataChanged(path);
277         try {
278           readPeerStateZnode();
279         } catch (DeserializationException e) {
280           LOG.warn("Failed deserializing the content of " + path, e);
281         }
282       }
283     }
284   }
285 
286   /**
287    * Tracker for PeerConfigNode of this peer
288    */
289   public class PeerConfigTracker extends ZooKeeperNodeTracker {
290 
291     private Set<ReplicationPeerConfigListener> listeners;
292 
293     public PeerConfigTracker(String peerConfigNode, ZooKeeperWatcher watcher,
294                              Abortable abortable) {
295       super(watcher, peerConfigNode, abortable);
296       listeners = new HashSet<>();
297     }
298 
299     public synchronized void addListener(ReplicationPeerConfigListener listener){
300       listeners.add(listener);
301     }
302 
303     Set<ReplicationPeerConfigListener> getListeners(){
304       return this.listeners;
305     }
306 
307     public synchronized void removeListener(ReplicationPeerConfigListener listenerToRemove) {
308       listeners.remove(listenerToRemove);
309     }
310 
311     @Override
312     public synchronized void nodeCreated(String path) {
313       if (path.equals(node)) {
314         super.nodeCreated(path);
315         ReplicationPeerConfig config = readPeerConfig();
316         for (ReplicationPeerConfigListener listener : listeners) {
317           listener.peerConfigUpdated(config);
318         }
319       }
320     }
321 
322     @Override
323     public synchronized void nodeDataChanged(String path) {
324       //superclass calls nodeCreated
325       if (path.equals(node)) {
326         super.nodeDataChanged(path);
327       }
328     }
329   }
330 }