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 package org.apache.hadoop.hdfs.security.token.block;
19 
20 import java.io.IOException;
21 import java.util.EnumSet;
22 import java.util.HashMap;
23 import java.util.Map;
24 
25 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
26 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode;
27 import org.apache.hadoop.security.token.SecretManager;
28 import org.apache.hadoop.security.token.Token;
29 
30 import com.google.common.annotations.VisibleForTesting;
31 
32 /**
33  * Manages a {@link BlockTokenSecretManager} per block pool. Routes the requests
34  * given a block pool Id to corresponding {@link BlockTokenSecretManager}
35  */
36 public class BlockPoolTokenSecretManager extends
37     SecretManager<BlockTokenIdentifier> {
38 
39   private final Map<String, BlockTokenSecretManager> map =
40     new HashMap<String, BlockTokenSecretManager>();
41 
42   /**
43    * Add a block pool Id and corresponding {@link BlockTokenSecretManager} to map
44    * @param bpid block pool Id
45    * @param secretMgr {@link BlockTokenSecretManager}
46    */
addBlockPool(String bpid, BlockTokenSecretManager secretMgr)47   public synchronized void addBlockPool(String bpid,
48       BlockTokenSecretManager secretMgr) {
49     map.put(bpid, secretMgr);
50   }
51 
get(String bpid)52   synchronized BlockTokenSecretManager get(String bpid) {
53     BlockTokenSecretManager secretMgr = map.get(bpid);
54     if (secretMgr == null) {
55       throw new IllegalArgumentException("Block pool " + bpid
56           + " is not found");
57     }
58     return secretMgr;
59   }
60 
isBlockPoolRegistered(String bpid)61   public synchronized boolean isBlockPoolRegistered(String bpid) {
62     return map.containsKey(bpid);
63   }
64 
65   /** Return an empty BlockTokenIdentifer */
66   @Override
createIdentifier()67   public BlockTokenIdentifier createIdentifier() {
68     return new BlockTokenIdentifier();
69   }
70 
71   @Override
createPassword(BlockTokenIdentifier identifier)72   public byte[] createPassword(BlockTokenIdentifier identifier) {
73     return get(identifier.getBlockPoolId()).createPassword(identifier);
74   }
75 
76   @Override
retrievePassword(BlockTokenIdentifier identifier)77   public byte[] retrievePassword(BlockTokenIdentifier identifier)
78       throws InvalidToken {
79     return get(identifier.getBlockPoolId()).retrievePassword(identifier);
80   }
81 
82   /**
83    * See {@link BlockTokenSecretManager#checkAccess(BlockTokenIdentifier,
84    *                String, ExtendedBlock, AccessMode)}
85    */
checkAccess(BlockTokenIdentifier id, String userId, ExtendedBlock block, AccessMode mode)86   public void checkAccess(BlockTokenIdentifier id, String userId,
87       ExtendedBlock block, AccessMode mode) throws InvalidToken {
88     get(block.getBlockPoolId()).checkAccess(id, userId, block, mode);
89   }
90 
91   /**
92    * See {@link BlockTokenSecretManager#checkAccess(Token, String,
93    *                ExtendedBlock, AccessMode)}
94    */
checkAccess(Token<BlockTokenIdentifier> token, String userId, ExtendedBlock block, AccessMode mode)95   public void checkAccess(Token<BlockTokenIdentifier> token,
96       String userId, ExtendedBlock block, AccessMode mode) throws InvalidToken {
97     get(block.getBlockPoolId()).checkAccess(token, userId, block, mode);
98   }
99 
100   /**
101    * See {@link BlockTokenSecretManager#addKeys(ExportedBlockKeys)}
102    */
addKeys(String bpid, ExportedBlockKeys exportedKeys)103   public void addKeys(String bpid, ExportedBlockKeys exportedKeys)
104       throws IOException {
105     get(bpid).addKeys(exportedKeys);
106   }
107 
108   /**
109    * See {@link BlockTokenSecretManager#generateToken(ExtendedBlock, EnumSet)}
110    */
generateToken(ExtendedBlock b, EnumSet<AccessMode> of)111   public Token<BlockTokenIdentifier> generateToken(ExtendedBlock b,
112       EnumSet<AccessMode> of) throws IOException {
113     return get(b.getBlockPoolId()).generateToken(b, of);
114   }
115 
116   @VisibleForTesting
clearAllKeysForTesting()117   public void clearAllKeysForTesting() {
118     for (BlockTokenSecretManager btsm : map.values()) {
119       btsm.clearAllKeysForTesting();
120     }
121   }
122 
generateDataEncryptionKey(String blockPoolId)123   public DataEncryptionKey generateDataEncryptionKey(String blockPoolId) {
124     return get(blockPoolId).generateDataEncryptionKey();
125   }
126 
retrieveDataEncryptionKey(int keyId, String blockPoolId, byte[] nonce)127   public byte[] retrieveDataEncryptionKey(int keyId, String blockPoolId,
128       byte[] nonce) throws IOException {
129     return get(blockPoolId).retrieveDataEncryptionKey(keyId, nonce);
130   }
131 }
132