HDFS-13951. HDFS DelegationTokenFetcher can't print non-HDFS tokens in a tokenfile.

Contributed by Steve Loughran.
This commit is contained in:
Steve Loughran 2018-10-11 20:03:56 +01:00
parent 735a36afd5
commit 2addebb94f
No known key found for this signature in database
GPG Key ID: D22CF846DBB162A0
1 changed files with 43 additions and 15 deletions

View File

@ -17,8 +17,6 @@
*/
package org.apache.hadoop.hdfs.tools;
import java.io.ByteArrayInputStream;
import java.io.DataInputStream;
import java.io.IOException;
import java.io.PrintStream;
import java.net.URI;
@ -36,12 +34,12 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
import org.apache.hadoop.hdfs.web.WebHdfsConstants;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.util.ExitUtil;
import org.apache.hadoop.util.GenericOptionsParser;
@ -64,10 +62,29 @@ public class DelegationTokenFetcher {
private static final String RENEWER = "renewer";
private static final String VERBOSE = "verbose";
/**
* Command-line interface
* Command-line interface.
* @param args argument list.
* @throws Exception on a failure.
* @throws org.apache.hadoop.util.ExitUtil.ExitException if the command
* failed and exiting was disabled.
*/
public static void main(final String[] args) throws Exception {
final Configuration conf = new HdfsConfiguration();
main(new HdfsConfiguration(), args);
}
/**
* Command line interface with a specific configuration.
* Errors in this operation will call {@link ExitUtil#terminate(int)} to
* exit the process.
* @param conf configuration to create filesystems with.
* @param args argument list.
* @throws Exception on a failure.
* @throws org.apache.hadoop.util.ExitUtil.ExitException if the command
* failed and exiting was disabled.
*/
@VisibleForTesting
public static void main(Configuration conf, final String[] args)
throws Exception {
Options fetcherOptions = new Options();
fetcherOptions
.addOption(WEBSERVICE, true, "HTTP url to reach the NameNode at")
@ -96,17 +113,19 @@ public class DelegationTokenFetcher {
// check option validity
if (help) {
printUsage(System.out);
System.exit(0);
return;
}
int commandCount = (cancel ? 1 : 0) + (renew ? 1 : 0) + (print ? 1 : 0);
if (commandCount > 1) {
System.err.println("ERROR: Only specify cancel, renew or print.");
printUsage(System.err);
return;
}
if (remaining.length != 1 || remaining[0].charAt(0) == '-') {
System.err.println("ERROR: Must specify exactly one token file");
printUsage(System.err);
return;
}
// default to using the local file system
FileSystem local = FileSystem.getLocal(conf);
@ -195,19 +214,22 @@ public class DelegationTokenFetcher {
}
@VisibleForTesting
static String printTokensToString(
public static String printTokensToString(
final Configuration conf,
final Path tokenFile,
final boolean verbose) throws IOException {
StringBuilder sbld = new StringBuilder();
final String nl = System.getProperty("line.separator");
DelegationTokenIdentifier id = new DelegationTokenSecretManager(0, 0, 0,
0, null).createIdentifier();
for (Token<?> token : readTokens(tokenFile, conf)) {
DataInputStream in = new DataInputStream(new ByteArrayInputStream(token
.getIdentifier()));
id.readFields(in);
String idStr = (verbose? id.toString() : id.toStringStable());
TokenIdentifier tokenId = token.decodeIdentifier();
String idStr;
if (tokenId instanceof DelegationTokenIdentifier) {
DelegationTokenIdentifier id = (DelegationTokenIdentifier) tokenId;
idStr = (verbose? id.toString() : id.toStringStable());
} else {
idStr = tokenId.toString();
}
sbld
.append("Token (").append(idStr)
.append(") for ").append(token.getService()).append(nl);
@ -222,13 +244,19 @@ public class DelegationTokenFetcher {
System.out.print(printTokensToString(conf, tokenFile, verbose));
}
/**
* Print usage to the error stream, then
* call {@link ExitUtil#terminate(int)} with status code 1.
* This will exit or raise an exception if that's been disabled.
* @param err stream for the messages.
*/
private static void printUsage(PrintStream err) {
err.println("fetchdt retrieves delegation tokens from the NameNode");
err.println();
err.println("fetchdt <opts> <token file>");
err.println("Options:");
err.println(" --webservice <url> Url to contact NN on (starts with " +
"http:// or https://)");
err.println(" --webservice <url> URL to contact NN on (starts with " +
"http:// or https://), or other filesystem URL");
err.println(" --renewer <name> Name of the delegation token renewer");
err.println(" --cancel Cancel the delegation token");
err.println(" --renew Renew the delegation token. " +