Skip to content

Commit cb75b6b

Browse files
committed
Merge r1609845 through r1618416 from trunk.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-6584@1618417 13f79535-47bb-0310-9956-ffa450edef68
2 parents 9b250d7 + c3084d6 commit cb75b6b

File tree

129 files changed

+7039
-3240
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

129 files changed

+7039
-3240
lines changed

hadoop-common-project/hadoop-common/CHANGES.txt

Lines changed: 36 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -202,6 +202,10 @@ Trunk (Unreleased)
202202
HADOOP-10224. JavaKeyStoreProvider has to protect against corrupting
203203
underlying store. (asuresh via tucu)
204204

205+
HADOOP-10770. KMS add delegation token support. (tucu)
206+
207+
HADOOP-10698. KMS, add proxyuser support. (tucu)
208+
205209
BUG FIXES
206210

207211
HADOOP-9451. Fault single-layer config if node group topology is enabled.
@@ -427,6 +431,9 @@ Trunk (Unreleased)
427431
HADOOP-10862. Miscellaneous trivial corrections to KMS classes.
428432
(asuresh via tucu)
429433

434+
HADOOP-10967. Improve DefaultCryptoExtension#generateEncryptedKey
435+
performance. (hitliuyi via tucu)
436+
430437
OPTIMIZATIONS
431438

432439
HADOOP-7761. Improve the performance of raw comparisons. (todd)
@@ -505,8 +512,19 @@ Release 2.6.0 - UNRELEASED
505512
HADOOP-10820. Throw an exception in GenericOptionsParser when passed
506513
an empty Path. (Alex Holmes and Zhihai Xu via wang)
507514

515+
HADOOP-10281. Create a scheduler, which assigns schedulables a priority
516+
level. (Chris Li via Arpit Agarwal)
517+
518+
HADOOP-8944. Shell command fs -count should include human readable option
519+
(Jonathan Allen via aw)
520+
521+
HADOOP-10231. Add some components in Native Libraries document (Akira
522+
AJISAKA via aw)
523+
508524
OPTIMIZATIONS
509525

526+
HADOOP-10838. Byte array native checksumming. (James Thomas via todd)
527+
510528
BUG FIXES
511529

512530
HADOOP-10781. Unportable getgrouplist() usage breaks FreeBSD (Dmitry
@@ -563,6 +581,24 @@ Release 2.6.0 - UNRELEASED
563581
HADOOP-10402. Configuration.getValByRegex does not substitute for
564582
variables. (Robert Kanter via kasha)
565583

584+
HADOOP-10851. NetgroupCache does not remove group memberships. (Benoy
585+
Antony via Arpit Agarwal)
586+
587+
HADOOP-10962. Flags for posix_fadvise are not valid in some architectures
588+
(David Villegas via Colin Patrick McCabe)
589+
590+
HADOOP-10966. Hadoop Common native compilation broken in windows.
591+
(David Villegas via Arpit Agarwal)
592+
593+
HADOOP-10843. TestGridmixRecord unit tests failure on PowerPC (Jinghui Wang
594+
via Colin Patrick McCabe)
595+
596+
HADOOP-10121. Fix javadoc spelling for HadoopArchives#writeTopLevelDirs
597+
(Akira AJISAKA via aw)
598+
599+
HADOOP-10964. Small fix for NetworkTopologyWithNodeGroup#sortByDistance.
600+
(Yi Liu via wang)
601+
566602
Release 2.5.0 - UNRELEASED
567603

568604
INCOMPATIBLE CHANGES

hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java

Lines changed: 10 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -219,6 +219,13 @@ public KeyVersion decryptEncryptedKey(
219219
private static class DefaultCryptoExtension implements CryptoExtension {
220220

221221
private final KeyProvider keyProvider;
222+
private static final ThreadLocal<SecureRandom> RANDOM =
223+
new ThreadLocal<SecureRandom>() {
224+
@Override
225+
protected SecureRandom initialValue() {
226+
return new SecureRandom();
227+
}
228+
};
222229

223230
private DefaultCryptoExtension(KeyProvider keyProvider) {
224231
this.keyProvider = keyProvider;
@@ -233,10 +240,10 @@ public EncryptedKeyVersion generateEncryptedKey(String encryptionKeyName)
233240
"No KeyVersion exists for key '%s' ", encryptionKeyName);
234241
// Generate random bytes for new key and IV
235242
Cipher cipher = Cipher.getInstance("AES/CTR/NoPadding");
236-
SecureRandom random = SecureRandom.getInstance("SHA1PRNG");
237243
final byte[] newKey = new byte[encryptionKey.getMaterial().length];
238-
random.nextBytes(newKey);
239-
final byte[] iv = random.generateSeed(cipher.getBlockSize());
244+
RANDOM.get().nextBytes(newKey);
245+
final byte[] iv = new byte[cipher.getBlockSize()];
246+
RANDOM.get().nextBytes(iv);
240247
// Encryption key IV is derived from new key's IV
241248
final byte[] encryptionIV = EncryptedKeyVersion.deriveIV(iv);
242249
// Encrypt the new key

hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderDelegationTokenExtension.java

Lines changed: 6 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,8 @@
2020
import org.apache.hadoop.security.Credentials;
2121
import org.apache.hadoop.security.token.Token;
2222

23+
import java.io.IOException;
24+
2325
/**
2426
* A KeyProvider extension with the ability to add a renewer's Delegation
2527
* Tokens to the provided Credentials.
@@ -45,9 +47,10 @@ public interface DelegationTokenExtension extends
4547
* @param renewer the user allowed to renew the delegation tokens
4648
* @param credentials cache in which to add new delegation tokens
4749
* @return list of new delegation tokens
50+
* @throws IOException thrown if IOException if an IO error occurs.
4851
*/
4952
public Token<?>[] addDelegationTokens(final String renewer,
50-
Credentials credentials);
53+
Credentials credentials) throws IOException;
5154
}
5255

5356
/**
@@ -76,9 +79,10 @@ private KeyProviderDelegationTokenExtension(KeyProvider keyProvider,
7679
* @param renewer the user allowed to renew the delegation tokens
7780
* @param credentials cache in which to add new delegation tokens
7881
* @return list of new delegation tokens
82+
* @throws IOException thrown if IOException if an IO error occurs.
7983
*/
8084
public Token<?>[] addDelegationTokens(final String renewer,
81-
Credentials credentials) {
85+
Credentials credentials) throws IOException {
8286
return getExtension().addDelegationTokens(renewer, credentials);
8387
}
8488

hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java

Lines changed: 83 additions & 27 deletions
Original file line numberDiff line numberDiff line change
@@ -22,15 +22,18 @@
2222
import org.apache.hadoop.conf.Configuration;
2323
import org.apache.hadoop.crypto.key.KeyProvider;
2424
import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion;
25+
import org.apache.hadoop.crypto.key.KeyProviderDelegationTokenExtension;
2526
import org.apache.hadoop.crypto.key.KeyProviderFactory;
2627
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
2728
import org.apache.hadoop.fs.Path;
29+
import org.apache.hadoop.security.Credentials;
2830
import org.apache.hadoop.security.ProviderUtils;
29-
import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
31+
import org.apache.hadoop.security.UserGroupInformation;
3032
import org.apache.hadoop.security.authentication.client.AuthenticationException;
3133
import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
32-
import org.apache.hadoop.security.authentication.client.PseudoAuthenticator;
3334
import org.apache.hadoop.security.ssl.SSLFactory;
35+
import org.apache.hadoop.security.token.Token;
36+
import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL;
3437
import org.apache.http.client.utils.URIBuilder;
3538
import org.codehaus.jackson.map.ObjectMapper;
3639

@@ -50,6 +53,7 @@
5053
import java.net.URLEncoder;
5154
import java.security.GeneralSecurityException;
5255
import java.security.NoSuchAlgorithmException;
56+
import java.security.PrivilegedExceptionAction;
5357
import java.text.MessageFormat;
5458
import java.util.ArrayList;
5559
import java.util.Date;
@@ -69,7 +73,10 @@
6973
* KMS client <code>KeyProvider</code> implementation.
7074
*/
7175
@InterfaceAudience.Private
72-
public class KMSClientProvider extends KeyProvider implements CryptoExtension {
76+
public class KMSClientProvider extends KeyProvider implements CryptoExtension,
77+
KeyProviderDelegationTokenExtension.DelegationTokenExtension {
78+
79+
public static final String TOKEN_KIND = "kms-dt";
7380

7481
public static final String SCHEME_NAME = "kms";
7582

@@ -229,6 +236,8 @@ public static String checkNotEmpty(String s, String name)
229236
private String kmsUrl;
230237
private SSLFactory sslFactory;
231238
private ConnectionConfigurator configurator;
239+
private DelegationTokenAuthenticatedURL.Token authToken;
240+
private UserGroupInformation loginUgi;
232241

233242
@Override
234243
public String toString() {
@@ -309,6 +318,8 @@ public KMSClientProvider(URI uri, Configuration conf) throws IOException {
309318
CommonConfigurationKeysPublic.
310319
KMS_CLIENT_ENC_KEY_CACHE_NUM_REFILL_THREADS_DEFAULT),
311320
new EncryptedQueueRefiller());
321+
authToken = new DelegationTokenAuthenticatedURL.Token();
322+
loginUgi = UserGroupInformation.getCurrentUser();
312323
}
313324

314325
private String createServiceURL(URL url) throws IOException {
@@ -325,12 +336,14 @@ private URL createURL(String collection, String resource, String subResource,
325336
try {
326337
StringBuilder sb = new StringBuilder();
327338
sb.append(kmsUrl);
328-
sb.append(collection);
329-
if (resource != null) {
330-
sb.append("/").append(URLEncoder.encode(resource, UTF8));
331-
}
332-
if (subResource != null) {
333-
sb.append("/").append(subResource);
339+
if (collection != null) {
340+
sb.append(collection);
341+
if (resource != null) {
342+
sb.append("/").append(URLEncoder.encode(resource, UTF8));
343+
if (subResource != null) {
344+
sb.append("/").append(subResource);
345+
}
346+
}
334347
}
335348
URIBuilder uriBuilder = new URIBuilder(sb.toString());
336349
if (parameters != null) {
@@ -365,14 +378,29 @@ private HttpURLConnection configureConnection(HttpURLConnection conn)
365378
return conn;
366379
}
367380

368-
private HttpURLConnection createConnection(URL url, String method)
381+
private HttpURLConnection createConnection(final URL url, String method)
369382
throws IOException {
370383
HttpURLConnection conn;
371384
try {
372-
AuthenticatedURL authUrl = new AuthenticatedURL(new PseudoAuthenticator(),
373-
configurator);
374-
conn = authUrl.openConnection(url, new AuthenticatedURL.Token());
375-
} catch (AuthenticationException ex) {
385+
// if current UGI is different from UGI at constructor time, behave as
386+
// proxyuser
387+
UserGroupInformation currentUgi = UserGroupInformation.getCurrentUser();
388+
final String doAsUser =
389+
(loginUgi.getShortUserName().equals(currentUgi.getShortUserName()))
390+
? null : currentUgi.getShortUserName();
391+
392+
// creating the HTTP connection using the current UGI at constructor time
393+
conn = loginUgi.doAs(new PrivilegedExceptionAction<HttpURLConnection>() {
394+
@Override
395+
public HttpURLConnection run() throws Exception {
396+
DelegationTokenAuthenticatedURL authUrl =
397+
new DelegationTokenAuthenticatedURL(configurator);
398+
return authUrl.openConnection(url, authToken, doAsUser);
399+
}
400+
});
401+
} catch (IOException ex) {
402+
throw ex;
403+
} catch (Exception ex) {
376404
throw new IOException(ex);
377405
}
378406
conn.setUseCaches(false);
@@ -403,20 +431,27 @@ private static void validateResponse(HttpURLConnection conn, int expected)
403431
if (status != expected) {
404432
InputStream es = null;
405433
try {
406-
es = conn.getErrorStream();
407-
ObjectMapper mapper = new ObjectMapper();
408-
Map json = mapper.readValue(es, Map.class);
409-
String exClass = (String) json.get(
410-
KMSRESTConstants.ERROR_EXCEPTION_JSON);
411-
String exMsg = (String)
412-
json.get(KMSRESTConstants.ERROR_MESSAGE_JSON);
413434
Exception toThrow;
414-
try {
415-
ClassLoader cl = KMSClientProvider.class.getClassLoader();
416-
Class klass = cl.loadClass(exClass);
417-
Constructor constr = klass.getConstructor(String.class);
418-
toThrow = (Exception) constr.newInstance(exMsg);
419-
} catch (Exception ex) {
435+
String contentType = conn.getHeaderField(CONTENT_TYPE);
436+
if (contentType != null &&
437+
contentType.toLowerCase().startsWith(APPLICATION_JSON_MIME)) {
438+
es = conn.getErrorStream();
439+
ObjectMapper mapper = new ObjectMapper();
440+
Map json = mapper.readValue(es, Map.class);
441+
String exClass = (String) json.get(
442+
KMSRESTConstants.ERROR_EXCEPTION_JSON);
443+
String exMsg = (String)
444+
json.get(KMSRESTConstants.ERROR_MESSAGE_JSON);
445+
try {
446+
ClassLoader cl = KMSClientProvider.class.getClassLoader();
447+
Class klass = cl.loadClass(exClass);
448+
Constructor constr = klass.getConstructor(String.class);
449+
toThrow = (Exception) constr.newInstance(exMsg);
450+
} catch (Exception ex) {
451+
toThrow = new IOException(MessageFormat.format(
452+
"HTTP status [{0}], {1}", status, conn.getResponseMessage()));
453+
}
454+
} else {
420455
toThrow = new IOException(MessageFormat.format(
421456
"HTTP status [{0}], {1}", status, conn.getResponseMessage()));
422457
}
@@ -729,4 +764,25 @@ public void warmUpEncryptedKeys(String... keyNames)
729764
}
730765
}
731766

767+
@Override
768+
public Token<?>[] addDelegationTokens(String renewer,
769+
Credentials credentials) throws IOException {
770+
Token<?>[] tokens;
771+
URL url = createURL(null, null, null, null);
772+
DelegationTokenAuthenticatedURL authUrl =
773+
new DelegationTokenAuthenticatedURL(configurator);
774+
try {
775+
Token<?> token = authUrl.getDelegationToken(url, authToken, renewer);
776+
if (token != null) {
777+
credentials.addToken(token.getService(), token);
778+
tokens = new Token<?>[] { token };
779+
} else {
780+
throw new IOException("Got NULL as delegation token");
781+
}
782+
} catch (AuthenticationException ex) {
783+
throw new IOException(ex);
784+
}
785+
return tokens;
786+
}
787+
732788
}

0 commit comments

Comments
 (0)