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 java.io.IOException;
22  import java.util.List;
23  
24  import org.apache.commons.logging.Log;
25  import org.apache.commons.logging.LogFactory;
26  import org.apache.hadoop.conf.Configuration;
27  import org.apache.hadoop.hbase.util.Writables;
28  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
29  import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
30  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
31  import org.apache.zookeeper.KeeperException;
32  
33  /**
34   * Synchronizes token encryption keys across cluster nodes.
35   */
36  public class ZKSecretWatcher extends ZooKeeperListener {
37    private static final String DEFAULT_ROOT_NODE = "tokenauth";
38    private static final String DEFAULT_KEYS_PARENT = "keys";
39    private static Log LOG = LogFactory.getLog(ZKSecretWatcher.class);
40  
41    private AuthenticationTokenSecretManager secretManager;
42    private String baseKeyZNode;
43    private String keysParentZNode;
44  
45    public ZKSecretWatcher(Configuration conf,
46        ZooKeeperWatcher watcher,
47        AuthenticationTokenSecretManager secretManager) {
48      super(watcher);
49      this.secretManager = secretManager;
50      String keyZNodeParent = conf.get("zookeeper.znode.tokenauth.parent", DEFAULT_ROOT_NODE);
51      this.baseKeyZNode = ZKUtil.joinZNode(watcher.baseZNode, keyZNodeParent);
52      this.keysParentZNode = ZKUtil.joinZNode(baseKeyZNode, DEFAULT_KEYS_PARENT);
53    }
54  
55    public void start() throws KeeperException {
56      watcher.registerListener(this);
57      // make sure the base node exists
58      ZKUtil.createWithParents(watcher, keysParentZNode);
59  
60      if (ZKUtil.watchAndCheckExists(watcher, keysParentZNode)) {
61        List<ZKUtil.NodeAndData> nodes =
62            ZKUtil.getChildDataAndWatchForNewChildren(watcher, keysParentZNode);
63        refreshNodes(nodes);
64      }
65    }
66  
67    @Override
68    public void nodeCreated(String path) {
69      if (path.equals(keysParentZNode)) {
70        try {
71          List<ZKUtil.NodeAndData> nodes =
72              ZKUtil.getChildDataAndWatchForNewChildren(watcher, keysParentZNode);
73          refreshNodes(nodes);
74        } catch (KeeperException ke) {
75          LOG.fatal("Error reading data from zookeeper", ke);
76          watcher.abort("Error reading new key znode "+path, ke);
77        }
78      }
79    }
80  
81    @Override
82    public void nodeDeleted(String path) {
83      if (keysParentZNode.equals(ZKUtil.getParent(path))) {
84        String keyId = ZKUtil.getNodeName(path);
85        try {
86          Integer id = new Integer(keyId);
87          secretManager.removeKey(id);
88        } catch (NumberFormatException nfe) {
89          LOG.error("Invalid znode name for key ID '"+keyId+"'", nfe);
90        }
91      }
92    }
93  
94    @Override
95    public void nodeDataChanged(String path) {
96      if (keysParentZNode.equals(ZKUtil.getParent(path))) {
97        try {
98          byte[] data = ZKUtil.getDataAndWatch(watcher, path);
99          if (data == null || data.length == 0) {
100           LOG.debug("Ignoring empty node "+path);
101           return;
102         }
103 
104         AuthenticationKey key = (AuthenticationKey)Writables.getWritable(data,
105             new AuthenticationKey());
106         secretManager.addKey(key);
107       } catch (KeeperException ke) {
108         LOG.fatal("Error reading data from zookeeper", ke);
109         watcher.abort("Error reading updated key znode "+path, ke);
110       } catch (IOException ioe) {
111         LOG.fatal("Error reading key writables", ioe);
112         watcher.abort("Error reading key writables from znode "+path, ioe);
113       }
114     }
115   }
116 
117   @Override
118   public void nodeChildrenChanged(String path) {
119     if (path.equals(keysParentZNode)) {
120       // keys changed
121       try {
122         List<ZKUtil.NodeAndData> nodes =
123             ZKUtil.getChildDataAndWatchForNewChildren(watcher, keysParentZNode);
124         refreshNodes(nodes);
125       } catch (KeeperException ke) {
126         LOG.fatal("Error reading data from zookeeper", ke);
127         watcher.abort("Error reading changed keys from zookeeper", ke);
128       }
129     }
130   }
131 
132   public String getRootKeyZNode() {
133     return baseKeyZNode;
134   }
135 
136   private void refreshNodes(List<ZKUtil.NodeAndData> nodes) {
137     for (ZKUtil.NodeAndData n : nodes) {
138       String path = n.getNode();
139       String keyId = ZKUtil.getNodeName(path);
140       try {
141         byte[] data = n.getData();
142         if (data == null || data.length == 0) {
143           LOG.debug("Ignoring empty node "+path);
144           continue;
145         }
146         AuthenticationKey key = (AuthenticationKey)Writables.getWritable(
147             data, new AuthenticationKey());
148         secretManager.addKey(key);
149       } catch (IOException ioe) {
150         LOG.fatal("Failed reading new secret key for id '" + keyId +
151             "' from zk", ioe);
152         watcher.abort("Error deserializing key from znode "+path, ioe);
153       }
154     }
155   }
156 
157   private String getKeyNode(int keyId) {
158     return ZKUtil.joinZNode(keysParentZNode, Integer.toString(keyId));
159   }
160 
161   public void removeKeyFromZK(AuthenticationKey key) {
162     String keyZNode = getKeyNode(key.getKeyId());
163     try {
164       ZKUtil.deleteNode(watcher, keyZNode);
165     } catch (KeeperException.NoNodeException nne) {
166       LOG.error("Non-existent znode "+keyZNode+" for key "+key.getKeyId(), nne);
167     } catch (KeeperException ke) {
168       LOG.fatal("Failed removing znode "+keyZNode+" for key "+key.getKeyId(),
169           ke);
170       watcher.abort("Unhandled zookeeper error removing znode "+keyZNode+
171           " for key "+key.getKeyId(), ke);
172     }
173   }
174 
175   public void addKeyToZK(AuthenticationKey key) {
176     String keyZNode = getKeyNode(key.getKeyId());
177     try {
178       byte[] keyData = Writables.getBytes(key);
179       // TODO: is there any point in retrying beyond what ZK client does?
180       ZKUtil.createSetData(watcher, keyZNode, keyData);
181     } catch (KeeperException ke) {
182       LOG.fatal("Unable to synchronize master key "+key.getKeyId()+
183           " to znode "+keyZNode, ke);
184       watcher.abort("Unable to synchronize secret key "+
185           key.getKeyId()+" in zookeeper", ke);
186     } catch (IOException ioe) {
187       // this can only happen from an error serializing the key
188       watcher.abort("Failed serializing key "+key.getKeyId(), ioe);
189     }
190   }
191 
192   public void updateKeyInZK(AuthenticationKey key) {
193     String keyZNode = getKeyNode(key.getKeyId());
194     try {
195       byte[] keyData = Writables.getBytes(key);
196       try {
197         ZKUtil.updateExistingNodeData(watcher, keyZNode, keyData, -1);
198       } catch (KeeperException.NoNodeException ne) {
199         // node was somehow removed, try adding it back
200         ZKUtil.createSetData(watcher, keyZNode, keyData);
201       }
202     } catch (KeeperException ke) {
203       LOG.fatal("Unable to update master key "+key.getKeyId()+
204           " in znode "+keyZNode);
205       watcher.abort("Unable to synchronize secret key "+
206           key.getKeyId()+" in zookeeper", ke);
207     } catch (IOException ioe) {
208       // this can only happen from an error serializing the key
209       watcher.abort("Failed serializing key "+key.getKeyId(), ioe);
210     }
211   }
212 }