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.znodePaths.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      } catch (NumberFormatException nfe) {
095        LOG.error("Invalid znode name for key ID '"+keyId+"'", nfe);
096      }
097    }
098  }
099
100  @Override
101  public void nodeDataChanged(String path) {
102    if (keysParentZNode.equals(ZKUtil.getParent(path))) {
103      try {
104        byte[] data = ZKUtil.getDataAndWatch(watcher, path);
105        if (data == null || data.length == 0) {
106          LOG.debug("Ignoring empty node "+path);
107          return;
108        }
109
110        AuthenticationKey key = (AuthenticationKey)Writables.getWritable(data,
111            new AuthenticationKey());
112        secretManager.addKey(key);
113      } catch (KeeperException ke) {
114        LOG.error(HBaseMarkers.FATAL, "Error reading data from zookeeper", ke);
115        watcher.abort("Error reading updated key znode "+path, ke);
116      } catch (IOException ioe) {
117        LOG.error(HBaseMarkers.FATAL, "Error reading key writables", ioe);
118        watcher.abort("Error reading key writables from znode "+path, ioe);
119      }
120    }
121  }
122
123  @Override
124  public void nodeChildrenChanged(String path) {
125    if (path.equals(keysParentZNode)) {
126      // keys changed
127      try {
128        List<ZKUtil.NodeAndData> nodes =
129            ZKUtil.getChildDataAndWatchForNewChildren(watcher, keysParentZNode);
130        refreshNodes(nodes);
131      } catch (KeeperException ke) {
132        LOG.error(HBaseMarkers.FATAL, "Error reading data from zookeeper", ke);
133        watcher.abort("Error reading changed keys from zookeeper", ke);
134      }
135    }
136  }
137
138  public String getRootKeyZNode() {
139    return baseKeyZNode;
140  }
141
142  private void refreshNodes(List<ZKUtil.NodeAndData> nodes) {
143    for (ZKUtil.NodeAndData n : nodes) {
144      String path = n.getNode();
145      String keyId = ZKUtil.getNodeName(path);
146      try {
147        byte[] data = n.getData();
148        if (data == null || data.length == 0) {
149          LOG.debug("Ignoring empty node "+path);
150          continue;
151        }
152        AuthenticationKey key = (AuthenticationKey)Writables.getWritable(
153            data, new AuthenticationKey());
154        secretManager.addKey(key);
155      } catch (IOException ioe) {
156        LOG.error(HBaseMarkers.FATAL, "Failed reading new secret key for id '" +
157            keyId + "' from zk", ioe);
158        watcher.abort("Error deserializing key from znode "+path, ioe);
159      }
160    }
161  }
162
163  private String getKeyNode(int keyId) {
164    return ZNodePaths.joinZNode(keysParentZNode, Integer.toString(keyId));
165  }
166
167  public void removeKeyFromZK(AuthenticationKey key) {
168    String keyZNode = getKeyNode(key.getKeyId());
169    try {
170      ZKUtil.deleteNode(watcher, keyZNode);
171    } catch (KeeperException.NoNodeException nne) {
172      LOG.error("Non-existent znode "+keyZNode+" for key "+key.getKeyId(), nne);
173    } catch (KeeperException ke) {
174      LOG.error(HBaseMarkers.FATAL, "Failed removing znode "+keyZNode+" for key "+
175          key.getKeyId(), ke);
176      watcher.abort("Unhandled zookeeper error removing znode "+keyZNode+
177          " for key "+key.getKeyId(), ke);
178    }
179  }
180
181  public void addKeyToZK(AuthenticationKey key) {
182    String keyZNode = getKeyNode(key.getKeyId());
183    try {
184      byte[] keyData = Writables.getBytes(key);
185      // TODO: is there any point in retrying beyond what ZK client does?
186      ZKUtil.createSetData(watcher, keyZNode, keyData);
187    } catch (KeeperException ke) {
188      LOG.error(HBaseMarkers.FATAL, "Unable to synchronize master key "+key.getKeyId()+
189          " to znode "+keyZNode, ke);
190      watcher.abort("Unable to synchronize secret key "+
191          key.getKeyId()+" in zookeeper", ke);
192    } catch (IOException ioe) {
193      // this can only happen from an error serializing the key
194      watcher.abort("Failed serializing key "+key.getKeyId(), ioe);
195    }
196  }
197
198  public void updateKeyInZK(AuthenticationKey key) {
199    String keyZNode = getKeyNode(key.getKeyId());
200    try {
201      byte[] keyData = Writables.getBytes(key);
202      try {
203        ZKUtil.updateExistingNodeData(watcher, keyZNode, keyData, -1);
204      } catch (KeeperException.NoNodeException ne) {
205        // node was somehow removed, try adding it back
206        ZKUtil.createSetData(watcher, keyZNode, keyData);
207      }
208    } catch (KeeperException ke) {
209      LOG.error(HBaseMarkers.FATAL, "Unable to update master key "+key.getKeyId()+
210          " in znode "+keyZNode);
211      watcher.abort("Unable to synchronize secret key "+
212          key.getKeyId()+" in zookeeper", ke);
213    } catch (IOException ioe) {
214      // this can only happen from an error serializing the key
215      watcher.abort("Failed serializing key "+key.getKeyId(), ioe);
216    }
217  }
218
219  /**
220   * refresh keys
221   */
222  synchronized void refreshKeys() {
223    try {
224      List<ZKUtil.NodeAndData> nodes =
225          ZKUtil.getChildDataAndWatchForNewChildren(watcher, keysParentZNode);
226      refreshNodes(nodes);
227    } catch (KeeperException ke) {
228      LOG.error(HBaseMarkers.FATAL, "Error reading data from zookeeper", ke);
229      watcher.abort("Error reading changed keys from zookeeper", ke);
230    }
231  }
232
233  /**
234   * get token keys parent node
235   * @return token keys parent node
236   */
237  @VisibleForTesting
238  String getKeysParentZNode() {
239    return keysParentZNode;
240  }
241}