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 javax.crypto.SecretKey;
22  import java.io.IOException;
23  import java.util.Iterator;
24  import java.util.Map;
25  import java.util.concurrent.ConcurrentHashMap;
26  import java.util.concurrent.atomic.AtomicLong;
27  
28  import org.apache.commons.logging.Log;
29  import org.apache.commons.logging.LogFactory;
30  import org.apache.hadoop.classification.InterfaceAudience;
31  import org.apache.hadoop.conf.Configuration;
32  import org.apache.hadoop.hbase.Stoppable;
33  import org.apache.hadoop.hbase.util.Bytes;
34  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
35  import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
36  import org.apache.hadoop.hbase.zookeeper.ZKLeaderManager;
37  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
38  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
39  import org.apache.hadoop.io.Text;
40  import org.apache.hadoop.io.WritableUtils;
41  import org.apache.hadoop.security.token.SecretManager;
42  import org.apache.hadoop.security.token.Token;
43  import org.apache.zookeeper.KeeperException;
44  
45  /**
46   * Manages an internal list of secret keys used to sign new authentication
47   * tokens as they are generated, and to valid existing tokens used for
48   * authentication.
49   *
50   * <p>
51   * A single instance of {@code AuthenticationTokenSecretManager} will be
52   * running as the "leader" in a given HBase cluster.  The leader is responsible
53   * for periodically generating new secret keys, which are then distributed to
54   * followers via ZooKeeper, and for expiring previously used secret keys that
55   * are no longer needed (as any tokens using them have expired).
56   * </p>
57   */
58  @InterfaceAudience.Private
59  public class AuthenticationTokenSecretManager
60      extends SecretManager<AuthenticationTokenIdentifier> {
61  
62    static final String NAME_PREFIX = "SecretManager-";
63  
64    private static Log LOG = LogFactory.getLog(
65        AuthenticationTokenSecretManager.class);
66  
67    private long lastKeyUpdate;
68    private long keyUpdateInterval;
69    private long tokenMaxLifetime;
70    private ZKSecretWatcher zkWatcher;
71    private LeaderElector leaderElector;
72    private ZKClusterId clusterId;
73  
74    private Map<Integer,AuthenticationKey> allKeys =
75        new ConcurrentHashMap<Integer, AuthenticationKey>();
76    private AuthenticationKey currentKey;
77  
78    private int idSeq;
79    private AtomicLong tokenSeq = new AtomicLong();
80    private String name;
81  
82    /**
83     * Create a new secret manager instance for generating keys.
84     * @param conf Configuration to use
85     * @param zk Connection to zookeeper for handling leader elections
86     * @param keyUpdateInterval Time (in milliseconds) between rolling a new master key for token signing
87     * @param tokenMaxLifetime Maximum age (in milliseconds) before a token expires and is no longer valid
88     */
89    /* TODO: Restrict access to this constructor to make rogues instances more difficult.
90     * For the moment this class is instantiated from
91     * org.apache.hadoop.hbase.ipc.SecureServer so public access is needed.
92     */
93    public AuthenticationTokenSecretManager(Configuration conf,
94        ZooKeeperWatcher zk, String serverName,
95        long keyUpdateInterval, long tokenMaxLifetime) {
96      this.zkWatcher = new ZKSecretWatcher(conf, zk, this);
97      this.keyUpdateInterval = keyUpdateInterval;
98      this.tokenMaxLifetime = tokenMaxLifetime;
99      this.leaderElector = new LeaderElector(zk, serverName);
100     this.name = NAME_PREFIX+serverName;
101     this.clusterId = new ZKClusterId(zk, zk);
102   }
103 
104   public void start() {
105     try {
106       // populate any existing keys
107       this.zkWatcher.start();
108       // try to become leader
109       this.leaderElector.start();
110     } catch (KeeperException ke) {
111       LOG.error("Zookeeper initialization failed", ke);
112     }
113   }
114 
115   public void stop() {
116     this.leaderElector.stop("SecretManager stopping");
117   }
118 
119   public boolean isMaster() {
120     return leaderElector.isMaster();
121   }
122 
123   public String getName() {
124     return name;
125   }
126 
127   @Override
128   protected byte[] createPassword(AuthenticationTokenIdentifier identifier) {
129     long now = EnvironmentEdgeManager.currentTime();
130     AuthenticationKey secretKey = currentKey;
131     identifier.setKeyId(secretKey.getKeyId());
132     identifier.setIssueDate(now);
133     identifier.setExpirationDate(now + tokenMaxLifetime);
134     identifier.setSequenceNumber(tokenSeq.getAndIncrement());
135     return createPassword(identifier.getBytes(),
136         secretKey.getKey());
137   }
138 
139   @Override
140   public byte[] retrievePassword(AuthenticationTokenIdentifier identifier)
141       throws InvalidToken {
142     long now = EnvironmentEdgeManager.currentTime();
143     if (identifier.getExpirationDate() < now) {
144       throw new InvalidToken("Token has expired");
145     }
146     AuthenticationKey masterKey = allKeys.get(identifier.getKeyId());
147     if (masterKey == null) {
148       throw new InvalidToken("Unknown master key for token (id="+
149           identifier.getKeyId()+")");
150     }
151     // regenerate the password
152     return createPassword(identifier.getBytes(),
153         masterKey.getKey());
154   }
155 
156   @Override
157   public AuthenticationTokenIdentifier createIdentifier() {
158     return new AuthenticationTokenIdentifier();
159   }
160 
161   public Token<AuthenticationTokenIdentifier> generateToken(String username) {
162     AuthenticationTokenIdentifier ident =
163         new AuthenticationTokenIdentifier(username);
164     Token<AuthenticationTokenIdentifier> token =
165         new Token<AuthenticationTokenIdentifier>(ident, this);
166     if (clusterId.hasId()) {
167       token.setService(new Text(clusterId.getId()));
168     }
169     return token;
170   }
171 
172   public synchronized void addKey(AuthenticationKey key) throws IOException {
173     // ignore zk changes when running as master
174     if (leaderElector.isMaster()) {
175       if (LOG.isDebugEnabled()) {
176         LOG.debug("Running as master, ignoring new key "+key.getKeyId());
177       }
178       return;
179     }
180 
181     if (LOG.isDebugEnabled()) {
182       LOG.debug("Adding key "+key.getKeyId());
183     }
184 
185     allKeys.put(key.getKeyId(), key);
186     if (currentKey == null || key.getKeyId() > currentKey.getKeyId()) {
187       currentKey = key;
188     }
189     // update current sequence
190     if (key.getKeyId() > idSeq) {
191       idSeq = key.getKeyId();
192     }
193   }
194 
195   synchronized boolean removeKey(Integer keyId) {
196     // ignore zk changes when running as master
197     if (leaderElector.isMaster()) {
198       if (LOG.isDebugEnabled()) {
199         LOG.debug("Running as master, ignoring removed key "+keyId);
200       }
201       return false;
202     }
203 
204     if (LOG.isDebugEnabled()) {
205       LOG.debug("Removing key "+keyId);
206     }
207 
208     allKeys.remove(keyId);
209     return true;
210   }
211 
212   AuthenticationKey getCurrentKey() {
213     return currentKey;
214   }
215 
216   AuthenticationKey getKey(int keyId) {
217     return allKeys.get(keyId);
218   }
219 
220   synchronized void removeExpiredKeys() {
221     if (!leaderElector.isMaster()) {
222       LOG.info("Skipping removeExpiredKeys() because not running as master.");
223       return;
224     }
225 
226     long now = EnvironmentEdgeManager.currentTime();
227     Iterator<AuthenticationKey> iter = allKeys.values().iterator();
228     while (iter.hasNext()) {
229       AuthenticationKey key = iter.next();
230       if (key.getExpiration() < now) {
231         if (LOG.isDebugEnabled()) {
232           LOG.debug("Removing expired key "+key.getKeyId());
233         }
234         iter.remove();
235         zkWatcher.removeKeyFromZK(key);
236       }
237     }
238   }
239   
240   synchronized boolean isCurrentKeyRolled() {
241     return currentKey != null;
242   }
243 
244   synchronized void rollCurrentKey() {
245     if (!leaderElector.isMaster()) {
246       LOG.info("Skipping rollCurrentKey() because not running as master.");
247       return;
248     }
249 
250     long now = EnvironmentEdgeManager.currentTime();
251     AuthenticationKey prev = currentKey;
252     AuthenticationKey newKey = new AuthenticationKey(++idSeq,
253         Long.MAX_VALUE, // don't allow to expire until it's replaced by a new key
254         generateSecret());
255     allKeys.put(newKey.getKeyId(), newKey);
256     currentKey = newKey;
257     zkWatcher.addKeyToZK(newKey);
258     lastKeyUpdate = now;
259 
260     if (prev != null) {
261       // make sure previous key is still stored
262       prev.setExpiration(now + tokenMaxLifetime);
263       allKeys.put(prev.getKeyId(), prev);
264       zkWatcher.updateKeyInZK(prev);
265     }
266   }
267 
268   public static SecretKey createSecretKey(byte[] raw) {
269     return SecretManager.createSecretKey(raw);
270   }
271 
272   private class LeaderElector extends Thread implements Stoppable {
273     private boolean stopped = false;
274     /** Flag indicating whether we're in charge of rolling/expiring keys */
275     private boolean isMaster = false;
276     private ZKLeaderManager zkLeader;
277 
278     public LeaderElector(ZooKeeperWatcher watcher, String serverName) {
279       setDaemon(true);
280       setName("ZKSecretWatcher-leaderElector");
281       zkLeader = new ZKLeaderManager(watcher,
282           ZKUtil.joinZNode(zkWatcher.getRootKeyZNode(), "keymaster"),
283           Bytes.toBytes(serverName), this);
284     }
285 
286     public boolean isMaster() {
287       return isMaster;
288     }
289 
290     @Override
291     public boolean isStopped() {
292       return stopped;
293     }
294 
295     @Override
296     public void stop(String reason) {
297       if (stopped) {
298         return;
299       }
300 
301       stopped = true;
302       // prevent further key generation when stopping
303       if (isMaster) {
304         zkLeader.stepDownAsLeader();
305       }
306       isMaster = false;
307       LOG.info("Stopping leader election, because: "+reason);
308       interrupt();
309     }
310 
311     public void run() {
312       zkLeader.start();
313       zkLeader.waitToBecomeLeader();
314       isMaster = true;
315 
316       while (!stopped) {
317         long now = EnvironmentEdgeManager.currentTime();
318 
319         // clear any expired
320         removeExpiredKeys();
321 
322         if (lastKeyUpdate + keyUpdateInterval < now) {
323           // roll a new master key
324           rollCurrentKey();
325         }
326 
327         try {
328           Thread.sleep(5000);
329         } catch (InterruptedException ie) {
330           if (LOG.isDebugEnabled()) {
331             LOG.debug("Interrupted waiting for next update", ie);
332           }
333         }
334       }
335     }
336   }
337 }