You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by el...@apache.org on 2015/05/14 19:18:03 UTC

[2/3] accumulo git commit: ACCUMULO-3810 Fix MultiTable CopyTable for Kerberos

ACCUMULO-3810 Fix MultiTable CopyTable for Kerberos

Need to make sure we do the KerberosToken + DelegationToken
dance for Kerberos instead of just assuming we have a PasswordToken.


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/c85361a9
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/c85361a9
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/c85361a9

Branch: refs/heads/master
Commit: c85361a98f066ece9e07bea7114532f512674d5b
Parents: 3f53e0d
Author: Josh Elser <el...@apache.org>
Authored: Thu May 14 13:12:44 2015 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Thu May 14 13:12:44 2015 -0400

----------------------------------------------------------------------
 .../test/randomwalk/multitable/CopyTable.java   |  3 ++
 .../test/randomwalk/multitable/CopyTool.java    | 56 +++++++++++++++++++-
 2 files changed, 57 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/c85361a9/test/src/main/java/org/apache/accumulo/test/randomwalk/multitable/CopyTable.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/multitable/CopyTable.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/multitable/CopyTable.java
index 24f83e1..d02cb42 100644
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/multitable/CopyTable.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/multitable/CopyTable.java
@@ -59,6 +59,9 @@ public class CopyTable extends Test {
     args[1] = getMapReduceJars();
     args[2] = env.getUserName();
     args[3] = env.getPassword();
+    if (null == args[3]) {
+      args[3] = env.getKeytab();
+    }
     args[4] = srcTableName;
     args[5] = env.getInstance().getInstanceName();
     args[6] = env.getConfigProperty("ZOOKEEPERS");

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c85361a9/test/src/main/java/org/apache/accumulo/test/randomwalk/multitable/CopyTool.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/multitable/CopyTool.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/multitable/CopyTool.java
index 515ea36..b4e509c 100644
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/multitable/CopyTool.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/multitable/CopyTool.java
@@ -19,17 +19,25 @@ package org.apache.accumulo.test.randomwalk.multitable;
 import java.io.IOException;
 
 import org.apache.accumulo.core.client.ClientConfiguration;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.ZooKeeperInstance;
+import org.apache.accumulo.core.client.ClientConfiguration.ClientProperty;
+import org.apache.accumulo.core.client.admin.DelegationTokenConfig;
 import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
 import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat;
+import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
+import org.apache.accumulo.core.client.security.tokens.KerberosToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Tool;
 import org.apache.log4j.Logger;
 
@@ -58,11 +66,56 @@ public class CopyTool extends Configured implements Tool {
     ClientConfiguration clientConf = new ClientConfiguration().withInstance(args[3]).withZkHosts(args[4]);
 
     job.setInputFormatClass(AccumuloInputFormat.class);
-    AccumuloInputFormat.setConnectorInfo(job, args[0], new PasswordToken(args[1]));
     AccumuloInputFormat.setInputTableName(job, args[2]);
     AccumuloInputFormat.setScanAuthorizations(job, Authorizations.EMPTY);
     AccumuloInputFormat.setZooKeeperInstance(job, clientConf);
 
+    final String principal;
+    final AuthenticationToken token;
+    if (clientConf.getBoolean(ClientProperty.INSTANCE_RPC_SASL_ENABLED.getKey(), false)) {
+      // Use the Kerberos creds to request a DelegationToken for MapReduce to use
+      final String keytab = args[1];
+
+      // Better be logged in. Could use the keytab, but we're already logged in soo..
+      KerberosToken kt = new KerberosToken();
+      try {
+        UserGroupInformation user = UserGroupInformation.getCurrentUser();
+        if (!user.hasKerberosCredentials()) {
+          throw new IllegalStateException("Expected current user to have Kerberos credentials");
+        }
+
+        // Get the principal via UGI
+        principal = user.getUserName();
+
+        // Connector w/ the Kerberos creds
+        ZooKeeperInstance inst = new ZooKeeperInstance(clientConf);
+        Connector conn = inst.getConnector(principal, kt);
+
+        // Do the explicit check to see if the user has the permission to get a delegation token
+        if (!conn.securityOperations().hasSystemPermission(conn.whoami(), SystemPermission.OBTAIN_DELEGATION_TOKEN)) {
+          log.error(principal + " doesn't have the " + SystemPermission.OBTAIN_DELEGATION_TOKEN.name()
+              + " SystemPermission neccesary to obtain a delegation token. MapReduce tasks cannot automatically use the client's"
+              + " credentials on remote servers. Delegation tokens provide a means to run MapReduce without distributing the user's credentials.");
+          throw new IllegalStateException(conn.whoami() + " does not have permission to obtain a delegation token");
+        }
+
+        // Fetch a delegation token from Accumulo
+        token = conn.securityOperations().getDelegationToken(new DelegationTokenConfig());
+
+      } catch (Exception e) {
+        final String msg = "Failed to acquire DelegationToken for use with MapReduce";
+        log.error(msg, e);
+        throw new RuntimeException(msg, e);
+      }
+    } else {
+      // Simple principal + password
+      principal = args[0];
+      token = new PasswordToken(args[1]);
+    }
+
+    AccumuloInputFormat.setConnectorInfo(job, principal, token);
+    AccumuloOutputFormat.setConnectorInfo(job, principal, token);
+
     job.setMapperClass(SeqMapClass.class);
     job.setMapOutputKeyClass(Text.class);
     job.setMapOutputValueClass(Mutation.class);
@@ -70,7 +123,6 @@ public class CopyTool extends Configured implements Tool {
     job.setNumReduceTasks(0);
 
     job.setOutputFormatClass(AccumuloOutputFormat.class);
-    AccumuloOutputFormat.setConnectorInfo(job, args[0], new PasswordToken(args[1]));
     AccumuloOutputFormat.setCreateTables(job, true);
     AccumuloOutputFormat.setDefaultTableName(job, args[5]);
     AccumuloOutputFormat.setZooKeeperInstance(job, clientConf);