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 */
018package org.apache.hadoop.hbase.security.token;
019
020import java.io.IOException;
021import java.util.List;
022import org.apache.hadoop.conf.Configuration;
023import org.apache.hadoop.hbase.log.HBaseMarkers;
024import org.apache.hadoop.hbase.util.Writables;
025import org.apache.hadoop.hbase.zookeeper.ZKListener;
026import org.apache.hadoop.hbase.zookeeper.ZKUtil;
027import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
028import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
029import org.apache.yetus.audience.InterfaceAudience;
030import org.apache.zookeeper.KeeperException;
031import org.slf4j.Logger;
032import org.slf4j.LoggerFactory;
033
034/**
035 * Synchronizes token encryption keys across cluster nodes.
036 */
037@InterfaceAudience.Private
038public class ZKSecretWatcher extends ZKListener {
039  private static final String DEFAULT_ROOT_NODE = "tokenauth";
040  private static final String DEFAULT_KEYS_PARENT = "keys";
041  private static final Logger LOG = LoggerFactory.getLogger(ZKSecretWatcher.class);
042
043  private AuthenticationTokenSecretManager secretManager;
044  private String baseKeyZNode;
045  private String keysParentZNode;
046
047  public ZKSecretWatcher(Configuration conf, ZKWatcher watcher,
048    AuthenticationTokenSecretManager secretManager) {
049    super(watcher);
050    this.secretManager = secretManager;
051    String keyZNodeParent = conf.get("zookeeper.znode.tokenauth.parent", DEFAULT_ROOT_NODE);
052    this.baseKeyZNode = ZNodePaths.joinZNode(watcher.getZNodePaths().baseZNode, keyZNodeParent);
053    this.keysParentZNode = ZNodePaths.joinZNode(baseKeyZNode, DEFAULT_KEYS_PARENT);
054  }
055
056  public void start() throws KeeperException {
057    watcher.registerListener(this);
058    // make sure the base node exists
059    ZKUtil.createWithParents(watcher, keysParentZNode);
060
061    if (ZKUtil.watchAndCheckExists(watcher, keysParentZNode)) {
062      List<ZKUtil.NodeAndData> nodes =
063        ZKUtil.getChildDataAndWatchForNewChildren(watcher, keysParentZNode);
064      refreshNodes(nodes);
065    }
066  }
067
068  @Override
069  public void nodeCreated(String path) {
070    if (path.equals(keysParentZNode)) {
071      try {
072        List<ZKUtil.NodeAndData> nodes =
073          ZKUtil.getChildDataAndWatchForNewChildren(watcher, keysParentZNode);
074        refreshNodes(nodes);
075      } catch (KeeperException ke) {
076        LOG.error(HBaseMarkers.FATAL, "Error reading data from zookeeper", ke);
077        watcher.abort("Error reading new key znode " + path, ke);
078      }
079    }
080  }
081
082  @Override
083  public void nodeDeleted(String path) {
084    if (keysParentZNode.equals(ZKUtil.getParent(path))) {
085      String keyId = ZKUtil.getNodeName(path);
086      try {
087        Integer id = Integer.valueOf(keyId);
088        secretManager.removeKey(id);
089        LOG.info("Node deleted id={}", id);
090      } catch (NumberFormatException nfe) {
091        LOG.error("Invalid znode name for key ID '" + keyId + "'", nfe);
092      }
093    }
094  }
095
096  @Override
097  public void nodeDataChanged(String path) {
098    if (keysParentZNode.equals(ZKUtil.getParent(path))) {
099      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 =
107          (AuthenticationKey) Writables.getWritable(data, new AuthenticationKey());
108        secretManager.addKey(key);
109      } catch (KeeperException ke) {
110        LOG.error(HBaseMarkers.FATAL, "Error reading data from zookeeper", ke);
111        watcher.abort("Error reading updated key znode " + path, ke);
112      } catch (IOException ioe) {
113        LOG.error(HBaseMarkers.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      // keys changed
123      try {
124        List<ZKUtil.NodeAndData> nodes =
125          ZKUtil.getChildDataAndWatchForNewChildren(watcher, keysParentZNode);
126        refreshNodes(nodes);
127      } catch (KeeperException ke) {
128        LOG.error(HBaseMarkers.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 =
149          (AuthenticationKey) Writables.getWritable(data, new AuthenticationKey());
150        secretManager.addKey(key);
151      } catch (IOException ioe) {
152        LOG.error(HBaseMarkers.FATAL,
153          "Failed reading new secret key for id '" + keyId + "' 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 ZNodePaths.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.error(HBaseMarkers.FATAL,
171        "Failed removing znode " + keyZNode + " for key " + key.getKeyId(), ke);
172      watcher.abort(
173        "Unhandled zookeeper error removing znode " + keyZNode + " 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      // TODO: is there any point in retrying beyond what ZK client does?
182      ZKUtil.createSetData(watcher, keyZNode, keyData);
183    } catch (KeeperException ke) {
184      LOG.error(HBaseMarkers.FATAL,
185        "Unable to synchronize master key " + key.getKeyId() + " to znode " + keyZNode, ke);
186      watcher.abort("Unable to synchronize secret key " + key.getKeyId() + " in zookeeper", ke);
187    } catch (IOException ioe) {
188      // this can only happen from an error serializing the key
189      watcher.abort("Failed serializing key " + key.getKeyId(), ioe);
190    }
191  }
192
193  public void updateKeyInZK(AuthenticationKey key) {
194    String keyZNode = getKeyNode(key.getKeyId());
195    try {
196      byte[] keyData = Writables.getBytes(key);
197      try {
198        ZKUtil.updateExistingNodeData(watcher, keyZNode, keyData, -1);
199      } catch (KeeperException.NoNodeException ne) {
200        // node was somehow removed, try adding it back
201        ZKUtil.createSetData(watcher, keyZNode, keyData);
202      }
203    } catch (KeeperException ke) {
204      LOG.error(HBaseMarkers.FATAL,
205        "Unable to update master key " + key.getKeyId() + " in znode " + keyZNode);
206      watcher.abort("Unable to synchronize secret key " + 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
213  /**
214   * refresh keys
215   */
216  synchronized void refreshKeys() {
217    try {
218      List<ZKUtil.NodeAndData> nodes =
219        ZKUtil.getChildDataAndWatchForNewChildren(watcher, keysParentZNode);
220      refreshNodes(nodes);
221    } catch (KeeperException ke) {
222      LOG.error(HBaseMarkers.FATAL, "Error reading data from zookeeper", ke);
223      watcher.abort("Error reading changed keys from zookeeper", ke);
224    }
225  }
226
227  /**
228   * get token keys parent node
229   * @return token keys parent node
230   */
231  String getKeysParentZNode() {
232    return keysParentZNode;
233  }
234}