View Javadoc

1   /*
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  
19  package org.apache.hadoop.hbase.security.token;
20  
21  import com.google.common.annotations.VisibleForTesting;
22  
23  import java.io.IOException;
24  import java.util.List;
25  
26  import org.apache.commons.logging.Log;
27  import org.apache.commons.logging.LogFactory;
28  import org.apache.hadoop.hbase.classification.InterfaceAudience;
29  import org.apache.hadoop.conf.Configuration;
30  import org.apache.hadoop.hbase.util.Writables;
31  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
32  import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
33  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
34  import org.apache.zookeeper.KeeperException;
35  
36  /**
37   * Synchronizes token encryption keys across cluster nodes.
38   */
39  @InterfaceAudience.Private
40  public class ZKSecretWatcher extends ZooKeeperListener {
41    private static final String DEFAULT_ROOT_NODE = "tokenauth";
42    private static final String DEFAULT_KEYS_PARENT = "keys";
43    private static final Log LOG = LogFactory.getLog(ZKSecretWatcher.class);
44  
45    private AuthenticationTokenSecretManager secretManager;
46    private String baseKeyZNode;
47    private String keysParentZNode;
48  
49    public ZKSecretWatcher(Configuration conf,
50        ZooKeeperWatcher watcher,
51        AuthenticationTokenSecretManager secretManager) {
52      super(watcher);
53      this.secretManager = secretManager;
54      String keyZNodeParent = conf.get("zookeeper.znode.tokenauth.parent", DEFAULT_ROOT_NODE);
55      this.baseKeyZNode = ZKUtil.joinZNode(watcher.baseZNode, keyZNodeParent);
56      this.keysParentZNode = ZKUtil.joinZNode(baseKeyZNode, DEFAULT_KEYS_PARENT);
57    }
58  
59    public void start() throws KeeperException {
60      watcher.registerListener(this);
61      // make sure the base node exists
62      ZKUtil.createWithParents(watcher, keysParentZNode);
63  
64      if (ZKUtil.watchAndCheckExists(watcher, keysParentZNode)) {
65        List<ZKUtil.NodeAndData> nodes =
66            ZKUtil.getChildDataAndWatchForNewChildren(watcher, keysParentZNode);
67        refreshNodes(nodes);
68      }
69    }
70  
71    @Override
72    public void nodeCreated(String path) {
73      if (path.equals(keysParentZNode)) {
74        try {
75          List<ZKUtil.NodeAndData> nodes =
76              ZKUtil.getChildDataAndWatchForNewChildren(watcher, keysParentZNode);
77          refreshNodes(nodes);
78        } catch (KeeperException ke) {
79          LOG.fatal("Error reading data from zookeeper", ke);
80          watcher.abort("Error reading new key znode "+path, ke);
81        }
82      }
83    }
84  
85    @Override
86    public void nodeDeleted(String path) {
87      if (keysParentZNode.equals(ZKUtil.getParent(path))) {
88        String keyId = ZKUtil.getNodeName(path);
89        try {
90          Integer id = Integer.valueOf(keyId);
91          secretManager.removeKey(id);
92        } catch (NumberFormatException nfe) {
93          LOG.error("Invalid znode name for key ID '"+keyId+"'", nfe);
94        }
95      }
96    }
97  
98    @Override
99    public void nodeDataChanged(String path) {
100     if (keysParentZNode.equals(ZKUtil.getParent(path))) {
101       try {
102         byte[] data = ZKUtil.getDataAndWatch(watcher, path);
103         if (data == null || data.length == 0) {
104           LOG.debug("Ignoring empty node "+path);
105           return;
106         }
107 
108         AuthenticationKey key = (AuthenticationKey)Writables.getWritable(data,
109             new AuthenticationKey());
110         secretManager.addKey(key);
111       } catch (KeeperException ke) {
112         LOG.fatal("Error reading data from zookeeper", ke);
113         watcher.abort("Error reading updated key znode "+path, ke);
114       } catch (IOException ioe) {
115         LOG.fatal("Error reading key writables", ioe);
116         watcher.abort("Error reading key writables from znode "+path, ioe);
117       }
118     }
119   }
120 
121   @Override
122   public void nodeChildrenChanged(String path) {
123     if (path.equals(keysParentZNode)) {
124       // keys changed
125       try {
126         List<ZKUtil.NodeAndData> nodes =
127             ZKUtil.getChildDataAndWatchForNewChildren(watcher, keysParentZNode);
128         refreshNodes(nodes);
129       } catch (KeeperException ke) {
130         LOG.fatal("Error reading data from zookeeper", ke);
131         watcher.abort("Error reading changed keys from zookeeper", ke);
132       }
133     }
134   }
135 
136   public String getRootKeyZNode() {
137     return baseKeyZNode;
138   }
139 
140   private void refreshNodes(List<ZKUtil.NodeAndData> nodes) {
141     for (ZKUtil.NodeAndData n : nodes) {
142       String path = n.getNode();
143       String keyId = ZKUtil.getNodeName(path);
144       try {
145         byte[] data = n.getData();
146         if (data == null || data.length == 0) {
147           LOG.debug("Ignoring empty node "+path);
148           continue;
149         }
150         AuthenticationKey key = (AuthenticationKey)Writables.getWritable(
151             data, new AuthenticationKey());
152         secretManager.addKey(key);
153       } catch (IOException ioe) {
154         LOG.fatal("Failed reading new secret key for id '" + keyId +
155             "' from zk", ioe);
156         watcher.abort("Error deserializing key from znode "+path, ioe);
157       }
158     }
159   }
160 
161   private String getKeyNode(int keyId) {
162     return ZKUtil.joinZNode(keysParentZNode, Integer.toString(keyId));
163   }
164 
165   public void removeKeyFromZK(AuthenticationKey key) {
166     String keyZNode = getKeyNode(key.getKeyId());
167     try {
168       ZKUtil.deleteNode(watcher, keyZNode);
169     } catch (KeeperException.NoNodeException nne) {
170       LOG.error("Non-existent znode "+keyZNode+" for key "+key.getKeyId(), nne);
171     } catch (KeeperException ke) {
172       LOG.fatal("Failed removing znode "+keyZNode+" for key "+key.getKeyId(),
173           ke);
174       watcher.abort("Unhandled zookeeper error removing znode "+keyZNode+
175           " for key "+key.getKeyId(), ke);
176     }
177   }
178 
179   public void addKeyToZK(AuthenticationKey key) {
180     String keyZNode = getKeyNode(key.getKeyId());
181     try {
182       byte[] keyData = Writables.getBytes(key);
183       // TODO: is there any point in retrying beyond what ZK client does?
184       ZKUtil.createSetData(watcher, keyZNode, keyData);
185     } catch (KeeperException ke) {
186       LOG.fatal("Unable to synchronize master key "+key.getKeyId()+
187           " to znode "+keyZNode, ke);
188       watcher.abort("Unable to synchronize secret key "+
189           key.getKeyId()+" in zookeeper", ke);
190     } catch (IOException ioe) {
191       // this can only happen from an error serializing the key
192       watcher.abort("Failed serializing key "+key.getKeyId(), ioe);
193     }
194   }
195 
196   public void updateKeyInZK(AuthenticationKey key) {
197     String keyZNode = getKeyNode(key.getKeyId());
198     try {
199       byte[] keyData = Writables.getBytes(key);
200       try {
201         ZKUtil.updateExistingNodeData(watcher, keyZNode, keyData, -1);
202       } catch (KeeperException.NoNodeException ne) {
203         // node was somehow removed, try adding it back
204         ZKUtil.createSetData(watcher, keyZNode, keyData);
205       }
206     } catch (KeeperException ke) {
207       LOG.fatal("Unable to update master key "+key.getKeyId()+
208           " in znode "+keyZNode);
209       watcher.abort("Unable to synchronize secret key "+
210           key.getKeyId()+" in zookeeper", ke);
211     } catch (IOException ioe) {
212       // this can only happen from an error serializing the key
213       watcher.abort("Failed serializing key "+key.getKeyId(), ioe);
214     }
215   }
216   
217   /**
218    * refresh keys
219    */
220   synchronized void refreshKeys() {
221     try {
222       List<ZKUtil.NodeAndData> nodes =
223           ZKUtil.getChildDataAndWatchForNewChildren(watcher, keysParentZNode);
224       refreshNodes(nodes);
225     } catch (KeeperException ke) {
226       LOG.fatal("Error reading data from zookeeper", ke);
227       watcher.abort("Error reading changed keys from zookeeper", ke);
228     }
229   }
230   
231   /**
232    * get token keys parent node
233    * @return token keys parent node
234    */
235   @VisibleForTesting
236   String getKeysParentZNode() {
237     return keysParentZNode;
238   }
239 }