001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018
019package org.apache.hadoop.hbase.security.token;
020
021import org.apache.hadoop.hbase.log.HBaseMarkers;
022import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
023
024import java.io.IOException;
025import java.util.List;
026
027import org.apache.hadoop.hbase.zookeeper.ZKListener;
028import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
029import org.apache.yetus.audience.InterfaceAudience;
030import org.apache.hadoop.conf.Configuration;
031import org.apache.hadoop.hbase.util.Writables;
032import org.apache.hadoop.hbase.zookeeper.ZKUtil;
033import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
034import org.apache.zookeeper.KeeperException;
035import org.slf4j.Logger;
036import org.slf4j.LoggerFactory;
037
038/**
039 * Synchronizes token encryption keys across cluster nodes.
040 */
041@InterfaceAudience.Private
042public class ZKSecretWatcher extends ZKListener {
043  private static final String DEFAULT_ROOT_NODE = "tokenauth";
044  private static final String DEFAULT_KEYS_PARENT = "keys";
045  private static final Logger LOG = LoggerFactory.getLogger(ZKSecretWatcher.class);
046
047  private AuthenticationTokenSecretManager secretManager;
048  private String baseKeyZNode;
049  private String keysParentZNode;
050
051  public ZKSecretWatcher(Configuration conf,
052      ZKWatcher watcher,
053      AuthenticationTokenSecretManager secretManager) {
054    super(watcher);
055    this.secretManager = secretManager;
056    String keyZNodeParent = conf.get("zookeeper.znode.tokenauth.parent", DEFAULT_ROOT_NODE);
057    this.baseKeyZNode = ZNodePaths.joinZNode(watcher.getZNodePaths().baseZNode, keyZNodeParent);
058    this.keysParentZNode = ZNodePaths.joinZNode(baseKeyZNode, DEFAULT_KEYS_PARENT);
059  }
060
061  public void start() throws KeeperException {
062    watcher.registerListener(this);
063    // make sure the base node exists
064    ZKUtil.createWithParents(watcher, keysParentZNode);
065
066    if (ZKUtil.watchAndCheckExists(watcher, keysParentZNode)) {
067      List<ZKUtil.NodeAndData> nodes =
068          ZKUtil.getChildDataAndWatchForNewChildren(watcher, keysParentZNode);
069      refreshNodes(nodes);
070    }
071  }
072
073  @Override
074  public void nodeCreated(String path) {
075    if (path.equals(keysParentZNode)) {
076      try {
077        List<ZKUtil.NodeAndData> nodes =
078            ZKUtil.getChildDataAndWatchForNewChildren(watcher, keysParentZNode);
079        refreshNodes(nodes);
080      } catch (KeeperException ke) {
081        LOG.error(HBaseMarkers.FATAL, "Error reading data from zookeeper", ke);
082        watcher.abort("Error reading new key znode "+path, ke);
083      }
084    }
085  }
086
087  @Override
088  public void nodeDeleted(String path) {
089    if (keysParentZNode.equals(ZKUtil.getParent(path))) {
090      String keyId = ZKUtil.getNodeName(path);
091      try {
092        Integer id = Integer.valueOf(keyId);
093        secretManager.removeKey(id);
094        LOG.info("Node deleted id={}", id);
095      } catch (NumberFormatException nfe) {
096        LOG.error("Invalid znode name for key ID '"+keyId+"'", nfe);
097      }
098    }
099  }
100
101  @Override
102  public void nodeDataChanged(String path) {
103    if (keysParentZNode.equals(ZKUtil.getParent(path))) {
104      try {
105        byte[] data = ZKUtil.getDataAndWatch(watcher, path);
106        if (data == null || data.length == 0) {
107          LOG.debug("Ignoring empty node "+path);
108          return;
109        }
110
111        AuthenticationKey key = (AuthenticationKey)Writables.getWritable(data,
112            new AuthenticationKey());
113        secretManager.addKey(key);
114      } catch (KeeperException ke) {
115        LOG.error(HBaseMarkers.FATAL, "Error reading data from zookeeper", ke);
116        watcher.abort("Error reading updated key znode "+path, ke);
117      } catch (IOException ioe) {
118        LOG.error(HBaseMarkers.FATAL, "Error reading key writables", ioe);
119        watcher.abort("Error reading key writables from znode "+path, ioe);
120      }
121    }
122  }
123
124  @Override
125  public void nodeChildrenChanged(String path) {
126    if (path.equals(keysParentZNode)) {
127      // keys changed
128      try {
129        List<ZKUtil.NodeAndData> nodes =
130            ZKUtil.getChildDataAndWatchForNewChildren(watcher, keysParentZNode);
131        refreshNodes(nodes);
132      } catch (KeeperException ke) {
133        LOG.error(HBaseMarkers.FATAL, "Error reading data from zookeeper", ke);
134        watcher.abort("Error reading changed keys from zookeeper", ke);
135      }
136    }
137  }
138
139  public String getRootKeyZNode() {
140    return baseKeyZNode;
141  }
142
143  private void refreshNodes(List<ZKUtil.NodeAndData> nodes) {
144    for (ZKUtil.NodeAndData n : nodes) {
145      String path = n.getNode();
146      String keyId = ZKUtil.getNodeName(path);
147      try {
148        byte[] data = n.getData();
149        if (data == null || data.length == 0) {
150          LOG.debug("Ignoring empty node "+path);
151          continue;
152        }
153        AuthenticationKey key = (AuthenticationKey)Writables.getWritable(
154            data, new AuthenticationKey());
155        secretManager.addKey(key);
156      } catch (IOException ioe) {
157        LOG.error(HBaseMarkers.FATAL, "Failed reading new secret key for id '" +
158            keyId + "' from zk", ioe);
159        watcher.abort("Error deserializing key from znode "+path, ioe);
160      }
161    }
162  }
163
164  private String getKeyNode(int keyId) {
165    return ZNodePaths.joinZNode(keysParentZNode, Integer.toString(keyId));
166  }
167
168  public void removeKeyFromZK(AuthenticationKey key) {
169    String keyZNode = getKeyNode(key.getKeyId());
170    try {
171      ZKUtil.deleteNode(watcher, keyZNode);
172    } catch (KeeperException.NoNodeException nne) {
173      LOG.error("Non-existent znode "+keyZNode+" for key "+key.getKeyId(), nne);
174    } catch (KeeperException ke) {
175      LOG.error(HBaseMarkers.FATAL, "Failed removing znode "+keyZNode+" for key "+
176          key.getKeyId(), ke);
177      watcher.abort("Unhandled zookeeper error removing znode "+keyZNode+
178          " for key "+key.getKeyId(), ke);
179    }
180  }
181
182  public void addKeyToZK(AuthenticationKey key) {
183    String keyZNode = getKeyNode(key.getKeyId());
184    try {
185      byte[] keyData = Writables.getBytes(key);
186      // TODO: is there any point in retrying beyond what ZK client does?
187      ZKUtil.createSetData(watcher, keyZNode, keyData);
188    } catch (KeeperException ke) {
189      LOG.error(HBaseMarkers.FATAL, "Unable to synchronize master key "+key.getKeyId()+
190          " to znode "+keyZNode, ke);
191      watcher.abort("Unable to synchronize secret key "+
192          key.getKeyId()+" in zookeeper", ke);
193    } catch (IOException ioe) {
194      // this can only happen from an error serializing the key
195      watcher.abort("Failed serializing key "+key.getKeyId(), ioe);
196    }
197  }
198
199  public void updateKeyInZK(AuthenticationKey key) {
200    String keyZNode = getKeyNode(key.getKeyId());
201    try {
202      byte[] keyData = Writables.getBytes(key);
203      try {
204        ZKUtil.updateExistingNodeData(watcher, keyZNode, keyData, -1);
205      } catch (KeeperException.NoNodeException ne) {
206        // node was somehow removed, try adding it back
207        ZKUtil.createSetData(watcher, keyZNode, keyData);
208      }
209    } catch (KeeperException ke) {
210      LOG.error(HBaseMarkers.FATAL, "Unable to update master key "+key.getKeyId()+
211          " in znode "+keyZNode);
212      watcher.abort("Unable to synchronize secret key "+
213          key.getKeyId()+" in zookeeper", ke);
214    } catch (IOException ioe) {
215      // this can only happen from an error serializing the key
216      watcher.abort("Failed serializing key "+key.getKeyId(), ioe);
217    }
218  }
219
220  /**
221   * refresh keys
222   */
223  synchronized void refreshKeys() {
224    try {
225      List<ZKUtil.NodeAndData> nodes =
226          ZKUtil.getChildDataAndWatchForNewChildren(watcher, keysParentZNode);
227      refreshNodes(nodes);
228    } catch (KeeperException ke) {
229      LOG.error(HBaseMarkers.FATAL, "Error reading data from zookeeper", ke);
230      watcher.abort("Error reading changed keys from zookeeper", ke);
231    }
232  }
233
234  /**
235   * get token keys parent node
236   * @return token keys parent node
237   */
238  @VisibleForTesting
239  String getKeysParentZNode() {
240    return keysParentZNode;
241  }
242}