You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by GitBox <gi...@apache.org> on 2018/11/30 21:22:26 UTC

[GitHub] milleruntime closed pull request #789: Make auths optional in new MR API

milleruntime closed pull request #789: Make auths optional in new MR API
URL: https://github.com/apache/accumulo/pull/789
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/InputFormatBuilder.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/InputFormatBuilder.java
index 1751d9fd2f..a1e6fd907b 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/InputFormatBuilder.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/InputFormatBuilder.java
@@ -20,6 +20,8 @@
 import java.util.Map;
 import java.util.Properties;
 
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.ClientInfo;
 import org.apache.accumulo.core.client.ClientSideIteratorScanner;
 import org.apache.accumulo.core.client.IsolatedScanner;
@@ -65,23 +67,7 @@
      * @param tableName
      *          the table to use when the tablename is null in the write call
      */
-    AuthsParams<T> table(String tableName);
-  }
-
-  /**
-   * Required params for builder
-   *
-   * @since 2.0
-   */
-  interface AuthsParams<T> {
-    /**
-     * Sets the {@link Authorizations} used to scan. Must be a subset of the user's authorizations.
-     * If none present use {@link Authorizations#EMPTY}
-     *
-     * @param auths
-     *          the user's authorizations
-     */
-    InputFormatOptions<T> auths(Authorizations auths);
+    InputFormatOptions<T> table(String tableName);
   }
 
   /**
@@ -93,7 +79,7 @@
     /**
      * Finish configuring, verify and store options into the JobConf or Job
      */
-    void store(T t);
+    void store(T t) throws AccumuloException, AccumuloSecurityException;
   }
 
   /**
@@ -124,6 +110,15 @@
    * @since 2.0
    */
   interface InputFormatOptions<T> {
+    /**
+     * Sets the {@link Authorizations} used to scan. Must be a subset of the user's authorizations.
+     * By Default, all of the users auths are set.
+     *
+     * @param auths
+     *          the user's authorizations
+     */
+    InputFormatOptions<T> auths(Authorizations auths);
+
     /**
      * Sets the name of the classloader context on this scanner
      *
@@ -260,6 +255,6 @@
     /**
      * Finish configuring, verify and serialize options into the JobConf or Job
      */
-    void store(T j);
+    void store(T j) throws AccumuloException, AccumuloSecurityException;
   }
 }
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AbstractInputFormat.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AbstractInputFormat.java
index 9095d91911..7b91397676 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AbstractInputFormat.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AbstractInputFormat.java
@@ -181,7 +181,7 @@ protected static Authorizations getScanAuthorizations(JobConf job) {
    *           if the context is improperly configured
    * @since 1.5.0
    */
-  protected static void validateOptions(JobConf job) throws IOException {
+  public static void validateOptions(JobConf job) throws IOException {
     AccumuloClient client = InputConfigurator.getClient(CLASS, job);
     InputConfigurator.validatePermissions(CLASS, job, client);
   }
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/InputFormatBuilderImpl.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/InputFormatBuilderImpl.java
index 79deca692a..e6b3a5137f 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/InputFormatBuilderImpl.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/InputFormatBuilderImpl.java
@@ -23,6 +23,10 @@
 import java.util.Objects;
 import java.util.Optional;
 
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.ClientInfo;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
@@ -36,9 +40,9 @@
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 
-public class InputFormatBuilderImpl<T> implements InputFormatBuilder,
-    InputFormatBuilder.ClientParams<T>, InputFormatBuilder.TableParams<T>,
-    InputFormatBuilder.AuthsParams<T>, InputFormatBuilder.InputFormatOptions<T>,
+public class InputFormatBuilderImpl<T>
+    implements InputFormatBuilder, InputFormatBuilder.ClientParams<T>,
+    InputFormatBuilder.TableParams<T>, InputFormatBuilder.InputFormatOptions<T>,
     InputFormatBuilder.ScanOptions<T>, InputFormatBuilder.BatchScanOptions<T> {
 
   Class<?> callingClass;
@@ -65,7 +69,7 @@ public InputFormatBuilderImpl(Class<?> callingClass) {
   }
 
   @Override
-  public InputFormatBuilder.AuthsParams<T> table(String tableName) {
+  public InputFormatBuilder.InputFormatOptions<T> table(String tableName) {
     this.tableName = Objects.requireNonNull(tableName, "Table name must not be null");
     return this;
   }
@@ -159,7 +163,7 @@ public InputFormatBuilderImpl(Class<?> callingClass) {
   }
 
   @Override
-  public void store(T j) {
+  public void store(T j) throws AccumuloException, AccumuloSecurityException {
     if (j instanceof Job) {
       store((Job) j);
     } else if (j instanceof JobConf) {
@@ -172,13 +176,13 @@ public void store(T j) {
   /**
    * Final builder method for mapreduce configuration
    */
-  private void store(Job job) {
-    // TODO validate params are set correctly, possibly call/modify
-    // AbstractInputFormat.validateOptions()
+  private void store(Job job) throws AccumuloException, AccumuloSecurityException {
     AbstractInputFormat.setClientInfo(job, clientInfo);
-    AbstractInputFormat.setScanAuthorizations(job, scanAuths);
     InputFormatBase.setInputTableName(job, tableName);
 
+    scanAuths = getUserAuths(scanAuths, clientInfo);
+    AbstractInputFormat.setScanAuthorizations(job, scanAuths);
+
     // all optional values
     if (context.isPresent())
       AbstractInputFormat.setClassLoaderContext(job, context.get());
@@ -203,13 +207,13 @@ private void store(Job job) {
   /**
    * Final builder method for legacy mapred configuration
    */
-  private void store(JobConf jobConf) {
-    // TODO validate params are set correctly, possibly call/modify
-    // AbstractInputFormat.validateOptions()
+  private void store(JobConf jobConf) throws AccumuloException, AccumuloSecurityException {
     org.apache.accumulo.hadoopImpl.mapred.AbstractInputFormat.setClientInfo(jobConf, clientInfo);
+    org.apache.accumulo.hadoopImpl.mapred.InputFormatBase.setInputTableName(jobConf, tableName);
+
+    scanAuths = getUserAuths(scanAuths, clientInfo);
     org.apache.accumulo.hadoopImpl.mapred.AbstractInputFormat.setScanAuthorizations(jobConf,
         scanAuths);
-    org.apache.accumulo.hadoopImpl.mapred.InputFormatBase.setInputTableName(jobConf, tableName);
 
     // all optional values
     if (context.isPresent())
@@ -237,6 +241,14 @@ private void store(JobConf jobConf) {
     org.apache.accumulo.hadoopImpl.mapred.InputFormatBase.setBatchScan(jobConf, bools.batchScan);
   }
 
+  private Authorizations getUserAuths(Authorizations scanAuths, ClientInfo clientInfo)
+      throws AccumuloSecurityException, AccumuloException {
+    if (scanAuths != null)
+      return scanAuths;
+    AccumuloClient c = Accumulo.newClient().from(clientInfo).build();
+    return c.securityOperations().getUserAuthorizations(clientInfo.getPrincipal());
+  }
+
   private static class BuilderBooleans {
     boolean autoAdjustRanges = true;
     boolean scanIsolation = false;
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/MapReduceClientOnDefaultTable.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/MapReduceClientOnDefaultTable.java
index 6f4ff2847c..adfa62c629 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/MapReduceClientOnDefaultTable.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/MapReduceClientOnDefaultTable.java
@@ -16,6 +16,10 @@
  */
 package org.apache.accumulo.hadoopImpl.mapreduce.lib;
 
+import java.io.IOException;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.ClientInfo;
 import org.apache.accumulo.hadoop.mapreduce.AccumuloInputFormat;
 import org.apache.accumulo.hadoop.mapreduce.AccumuloOutputFormat;
@@ -36,7 +40,8 @@ public String getTableName() {
   }
 
   @Override
-  public void setAccumuloConfigs(Job job) {
+  public void setAccumuloConfigs(Job job)
+      throws IOException, AccumuloException, AccumuloSecurityException {
     final String tableName = getTableName();
     final ClientInfo info = getClientInfo();
     AccumuloInputFormat.configure().clientInfo(info).table(tableName).auths(auths).store(job);
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/MapReduceClientOnRequiredTable.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/MapReduceClientOnRequiredTable.java
index 9e48706adf..aed3382d70 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/MapReduceClientOnRequiredTable.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/MapReduceClientOnRequiredTable.java
@@ -16,6 +16,10 @@
  */
 package org.apache.accumulo.hadoopImpl.mapreduce.lib;
 
+import java.io.IOException;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.ClientInfo;
 import org.apache.accumulo.hadoop.mapreduce.AccumuloInputFormat;
 import org.apache.accumulo.hadoop.mapreduce.AccumuloOutputFormat;
@@ -29,7 +33,8 @@
   private String tableName;
 
   @Override
-  public void setAccumuloConfigs(Job job) {
+  public void setAccumuloConfigs(Job job)
+      throws IOException, AccumuloException, AccumuloSecurityException {
     final String tableName = getTableName();
     final ClientInfo info = getClientInfo();
     AccumuloInputFormat.configure().clientInfo(info).table(tableName).auths(auths).store(job);
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/MapReduceClientOpts.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/MapReduceClientOpts.java
index bbea6c3e92..96a516ca31 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/MapReduceClientOpts.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/MapReduceClientOpts.java
@@ -16,9 +16,12 @@
  */
 package org.apache.accumulo.hadoopImpl.mapreduce.lib;
 
+import java.io.IOException;
+
 import org.apache.accumulo.core.cli.ClientOpts;
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.admin.DelegationTokenConfig;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
@@ -35,7 +38,8 @@
 public abstract class MapReduceClientOpts extends ClientOpts {
   private static final Logger log = LoggerFactory.getLogger(MapReduceClientOpts.class);
 
-  public abstract void setAccumuloConfigs(Job job) throws AccumuloSecurityException;
+  public abstract void setAccumuloConfigs(Job job)
+      throws IOException, AccumuloException, AccumuloSecurityException;
 
   @Override
   public AuthenticationToken getToken() {
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapred/AccumuloInputFormatTest.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapred/AccumuloInputFormatTest.java
index 6831869226..9f5b4fbf87 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapred/AccumuloInputFormatTest.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapred/AccumuloInputFormatTest.java
@@ -23,7 +23,6 @@
 
 import java.io.ByteArrayOutputStream;
 import java.io.DataOutputStream;
-import java.io.IOException;
 import java.util.Base64;
 import java.util.HashSet;
 import java.util.List;
@@ -76,7 +75,7 @@ public static void setupClientInfo() {
    * Check that the iterator configuration is getting stored in the Job conf correctly.
    */
   @Test
-  public void testSetIterator() throws IOException {
+  public void testSetIterator() throws Exception {
     InputFormatOptions<JobConf> opts = AccumuloInputFormat.configure().clientInfo(clientInfo)
         .table("test").auths(Authorizations.EMPTY);
 
@@ -89,7 +88,7 @@ public void testSetIterator() throws IOException {
   }
 
   @Test
-  public void testAddIterator() {
+  public void testAddIterator() throws Exception {
     InputFormatOptions<JobConf> opts = AccumuloInputFormat.configure().clientInfo(clientInfo)
         .table("test").auths(Authorizations.EMPTY);
 
@@ -168,7 +167,7 @@ public void testIteratorOptionEncoding() throws Throwable {
    * Test getting iterator settings for multiple iterators set
    */
   @Test
-  public void testGetIteratorSettings() {
+  public void testGetIteratorSettings() throws Exception {
     IteratorSetting iter1 = new IteratorSetting(1, "WholeRow", WholeRowIterator.class.getName());
     IteratorSetting iter2 = new IteratorSetting(2, "Versions", VersioningIterator.class.getName());
     IteratorSetting iter3 = new IteratorSetting(3, "Count", CountingIterator.class.getName());
@@ -199,7 +198,7 @@ public void testGetIteratorSettings() {
   }
 
   @Test
-  public void testSetRegex() {
+  public void testSetRegex() throws Exception {
     String regex = ">\"*%<>\'\\";
 
     IteratorSetting is = new IteratorSetting(50, regex, RegExFilter.class);
@@ -212,7 +211,7 @@ public void testSetRegex() {
   }
 
   @Test
-  public void testEmptyColumnFamily() throws IOException {
+  public void testEmptyColumnFamily() throws Exception {
     Set<IteratorSetting.Column> cols = new HashSet<>();
     cols.add(new IteratorSetting.Column(new Text(""), null));
     cols.add(new IteratorSetting.Column(new Text("foo"), new Text("bar")));
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapreduce/AccumuloInputFormatTest.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapreduce/AccumuloInputFormatTest.java
index dfac630162..0d9ecac905 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapreduce/AccumuloInputFormatTest.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapreduce/AccumuloInputFormatTest.java
@@ -23,7 +23,6 @@
 
 import java.io.ByteArrayOutputStream;
 import java.io.DataOutputStream;
-import java.io.IOException;
 import java.util.Base64;
 import java.util.HashSet;
 import java.util.List;
@@ -63,7 +62,7 @@ public static void setupClientInfo() {
    * Check that the iterator configuration is getting stored in the Job conf correctly.
    */
   @Test
-  public void testSetIterator() throws IOException {
+  public void testSetIterator() throws Exception {
     Job job = Job.getInstance();
 
     IteratorSetting is = new IteratorSetting(1, "WholeRow", WholeRowIterator.class);
@@ -77,7 +76,7 @@ public void testSetIterator() throws IOException {
   }
 
   @Test
-  public void testAddIterator() throws IOException {
+  public void testAddIterator() throws Exception {
     Job job = Job.getInstance();
 
     IteratorSetting iter1 = new IteratorSetting(1, "WholeRow", WholeRowIterator.class);
@@ -159,7 +158,7 @@ public void testIteratorOptionEncoding() throws Throwable {
    * Test getting iterator settings for multiple iterators set
    */
   @Test
-  public void testGetIteratorSettings() throws IOException {
+  public void testGetIteratorSettings() throws Exception {
     Job job = Job.getInstance();
 
     IteratorSetting iter1 = new IteratorSetting(1, "WholeRow", WholeRowIterator.class.getName());
@@ -193,7 +192,7 @@ public void testGetIteratorSettings() throws IOException {
   }
 
   @Test
-  public void testSetRegex() throws IOException {
+  public void testSetRegex() throws Exception {
     Job job = Job.getInstance();
 
     String regex = ">\"*%<>\'\\";
@@ -208,7 +207,7 @@ public void testSetRegex() throws IOException {
   }
 
   @Test
-  public void testEmptyColumnFamily() throws IOException {
+  public void testEmptyColumnFamily() throws Exception {
     Job job = Job.getInstance();
     Set<IteratorSetting.Column> cols = new HashSet<>();
     cols.add(new IteratorSetting.Column(new Text(""), null));


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services