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