You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by mm...@apache.org on 2018/12/07 18:33:20 UTC

[accumulo] branch master updated: Cleanup hadoop-mapreduce (#809)

This is an automated email from the ASF dual-hosted git repository.

mmiller pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/master by this push:
     new 779ba39  Cleanup hadoop-mapreduce (#809)
779ba39 is described below

commit 779ba392bad09486b23a3a0eea86ad639e208728
Author: Mike Miller <mm...@apache.org>
AuthorDate: Fri Dec 7 13:33:16 2018 -0500

    Cleanup hadoop-mapreduce (#809)
    
    * Drop unused unwrapping DelegationTokenStub and methods
    * Drop exceptions never thrown
---
 .../hadoop/mapred/AccumuloFileOutputFormat.java    |  2 +-
 .../hadoop/mapred/AccumuloInputFormat.java         |  2 +-
 .../hadoop/mapred/AccumuloRowInputFormat.java      |  3 +-
 .../hadoop/mapreduce/AccumuloFileOutputFormat.java |  2 +-
 .../hadoop/mapreduce/AccumuloInputFormat.java      |  2 +-
 .../hadoop/mapreduce/AccumuloOutputFormat.java     |  2 +-
 .../hadoop/mapreduce/AccumuloRowInputFormat.java   |  6 +-
 .../hadoopImpl/mapred/AbstractInputFormat.java     |  4 +-
 .../mapred/AccumuloOutputFormatImpl.java           |  2 +-
 .../hadoopImpl/mapreduce/AbstractInputFormat.java  |  6 +-
 .../mapreduce/AccumuloOutputFormatImpl.java        |  5 +-
 .../hadoopImpl/mapreduce/DelegationTokenStub.java  | 81 ----------------------
 .../hadoopImpl/mapreduce/RangeInputSplit.java      |  2 +-
 .../accumulo/hadoopImpl/mapreduce/SplitUtils.java  |  3 +-
 .../hadoopImpl/mapreduce/lib/ConfiguratorBase.java | 68 ------------------
 .../mapreduce/lib/InputConfigurator.java           |  3 +-
 .../mapreduce/lib/MapReduceClientOpts.java         |  4 +-
 .../its/mapred/AccumuloFileOutputFormatIT.java     |  6 +-
 .../hadoop/its/mapred/AccumuloInputFormatIT.java   |  6 +-
 .../its/mapred/AccumuloRowInputFormatIT.java       |  5 +-
 .../accumulo/hadoop/its/mapred/TokenFileIT.java    |  3 +-
 .../its/mapreduce/AccumuloFileOutputFormatIT.java  |  6 +-
 .../its/mapreduce/AccumuloInputFormatIT.java       |  7 +-
 .../its/mapreduce/AccumuloOutputFormatIT.java      |  2 +-
 .../its/mapreduce/AccumuloRowInputFormatIT.java    |  6 +-
 .../accumulo/hadoop/its/mapreduce/TokenFileIT.java |  2 +-
 .../mapred/AccumuloFileOutputFormatTest.java       |  3 +-
 .../hadoop/mapred/AccumuloOutputFormatTest.java    |  2 +-
 .../hadoop/mapreduce/AccumuloOutputFormatTest.java |  2 +-
 .../mapreduce/lib/ConfiguratorBaseTest.java        |  4 +-
 30 files changed, 41 insertions(+), 210 deletions(-)

diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapred/AccumuloFileOutputFormat.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapred/AccumuloFileOutputFormat.java
index ff531b9..0edcd84 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapred/AccumuloFileOutputFormat.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapred/AccumuloFileOutputFormat.java
@@ -46,7 +46,7 @@ public class AccumuloFileOutputFormat extends FileOutputFormat<Key,Value> {
 
   @Override
   public RecordWriter<Key,Value> getRecordWriter(FileSystem ignored, JobConf job, String name,
-      Progressable progress) throws IOException {
+      Progressable progress) {
     // get the path of the temporary output file
     final Configuration conf = job;
     final AccumuloConfiguration acuConf = FileOutputConfigurator
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapred/AccumuloInputFormat.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapred/AccumuloInputFormat.java
index 92df89d..6fabcf9 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapred/AccumuloInputFormat.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapred/AccumuloInputFormat.java
@@ -64,7 +64,7 @@ public class AccumuloInputFormat implements InputFormat<Key,Value> {
     RecordReaderBase<Key,Value> recordReader = new RecordReaderBase<Key,Value>() {
 
       @Override
-      public boolean next(Key key, Value value) throws IOException {
+      public boolean next(Key key, Value value) {
         if (scannerIterator.hasNext()) {
           ++numKeysRead;
           Entry<Key,Value> entry = scannerIterator.next();
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapred/AccumuloRowInputFormat.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapred/AccumuloRowInputFormat.java
index c1c6cbe..bdcfbdb 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapred/AccumuloRowInputFormat.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapred/AccumuloRowInputFormat.java
@@ -72,8 +72,7 @@ public class AccumuloRowInputFormat implements InputFormat<Text,PeekingIterator<
           }
 
           @Override
-          public boolean next(Text key, PeekingIterator<Entry<Key,Value>> value)
-              throws IOException {
+          public boolean next(Text key, PeekingIterator<Entry<Key,Value>> value) {
             if (!rowIterator.hasNext())
               return false;
             value.initialize(rowIterator.next());
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloFileOutputFormat.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloFileOutputFormat.java
index 2f869e9..7e3d56f 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloFileOutputFormat.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloFileOutputFormat.java
@@ -93,7 +93,7 @@ public class AccumuloFileOutputFormat extends FileOutputFormat<Key,Value> {
    * Sets all the information required for this map reduce job.
    */
   public static FileOutputFormatBuilder.PathParams<Job> configure() {
-    return new FileOutputFormatBuilderImpl<Job>();
+    return new FileOutputFormatBuilderImpl<>();
   }
 
 }
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloInputFormat.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloInputFormat.java
index 9ea2ff4..4f89bf0 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloInputFormat.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloInputFormat.java
@@ -77,7 +77,7 @@ public class AccumuloInputFormat extends InputFormat<Key,Value> {
 
     return new InputFormatBase.RecordReaderBase<Key,Value>() {
       @Override
-      public boolean nextKeyValue() throws IOException, InterruptedException {
+      public boolean nextKeyValue() {
         if (scannerIterator.hasNext()) {
           ++numKeysRead;
           Entry<Key,Value> entry = scannerIterator.next();
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloOutputFormat.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloOutputFormat.java
index 16f7395..0e7dfb6 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloOutputFormat.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloOutputFormat.java
@@ -90,7 +90,7 @@ public class AccumuloOutputFormat extends OutputFormat<Text,Mutation> {
    * Sets all the information required for this map reduce job.
    */
   public static OutputFormatBuilder.ClientParams<Job> configure() {
-    return new OutputFormatBuilderImpl<Job>();
+    return new OutputFormatBuilderImpl<>();
   }
 
 }
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloRowInputFormat.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloRowInputFormat.java
index 90147fd..d89e849 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloRowInputFormat.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloRowInputFormat.java
@@ -61,7 +61,7 @@ public class AccumuloRowInputFormat extends InputFormat<Text,PeekingIterator<Ent
 
   @Override
   public RecordReader<Text,PeekingIterator<Entry<Key,Value>>> createRecordReader(InputSplit split,
-      TaskAttemptContext context) throws IOException, InterruptedException {
+      TaskAttemptContext context) {
     return new InputFormatBase.RecordReaderBase<Text,PeekingIterator<Entry<Key,Value>>>() {
       RowIterator rowIterator;
 
@@ -74,7 +74,7 @@ public class AccumuloRowInputFormat extends InputFormat<Text,PeekingIterator<Ent
       }
 
       @Override
-      public boolean nextKeyValue() throws IOException, InterruptedException {
+      public boolean nextKeyValue() {
         if (!rowIterator.hasNext())
           return false;
         currentV = new PeekingIterator<>(rowIterator.next());
@@ -104,6 +104,6 @@ public class AccumuloRowInputFormat extends InputFormat<Text,PeekingIterator<Ent
    * Sets all the information required for this map reduce job.
    */
   public static InputFormatBuilder.ClientParams<Job> configure() {
-    return new InputFormatBuilderImpl<Job>(CLASS);
+    return new InputFormatBuilderImpl<>(CLASS);
   }
 }
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 c86c310..657f66c 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
@@ -408,12 +408,12 @@ public abstract class AbstractInputFormat {
     }
 
     @Override
-    public long getPos() throws IOException {
+    public long getPos() {
       return numKeysRead;
     }
 
     @Override
-    public float getProgress() throws IOException {
+    public float getProgress() {
       if (numKeysRead > 0 && currentKey == null)
         return 1.0f;
       return baseSplit.getProgress(currentKey);
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloOutputFormatImpl.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloOutputFormatImpl.java
index 2b6942d..99aa434 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloOutputFormatImpl.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloOutputFormatImpl.java
@@ -206,7 +206,7 @@ public class AccumuloOutputFormatImpl {
 
     private AccumuloClient client;
 
-    public AccumuloRecordWriter(JobConf job) throws AccumuloException, AccumuloSecurityException {
+    public AccumuloRecordWriter(JobConf job) {
       this.simulate = getSimulationMode(job);
       this.createTables = canCreateTables(job);
 
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AbstractInputFormat.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AbstractInputFormat.java
index 891a354..ea13fb3 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AbstractInputFormat.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AbstractInputFormat.java
@@ -412,7 +412,7 @@ public abstract class AbstractInputFormat {
     }
 
     @Override
-    public float getProgress() throws IOException {
+    public float getProgress() {
       if (numKeysRead > 0 && currentKey == null)
         return 1.0f;
       return split.getProgress(currentKey);
@@ -435,12 +435,12 @@ public abstract class AbstractInputFormat {
     protected Key currentKey = null;
 
     @Override
-    public K getCurrentKey() throws IOException, InterruptedException {
+    public K getCurrentKey() {
       return currentK;
     }
 
     @Override
-    public V getCurrentValue() throws IOException, InterruptedException {
+    public V getCurrentValue() {
       return currentV;
     }
   }
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloOutputFormatImpl.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloOutputFormatImpl.java
index 50b8925..11b57e2 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloOutputFormatImpl.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloOutputFormatImpl.java
@@ -208,8 +208,7 @@ public class AccumuloOutputFormatImpl {
 
     private AccumuloClient client;
 
-    public AccumuloRecordWriter(TaskAttemptContext context)
-        throws AccumuloException, AccumuloSecurityException {
+    public AccumuloRecordWriter(TaskAttemptContext context) {
       this.simulate = getSimulationMode(context);
       this.createTables = canCreateTables(context);
 
@@ -322,7 +321,7 @@ public class AccumuloOutputFormatImpl {
     }
 
     @Override
-    public void close(TaskAttemptContext attempt) throws IOException, InterruptedException {
+    public void close(TaskAttemptContext attempt) throws IOException {
       log.debug("mutations written: " + mutCount + ", values written: " + valCount);
       if (simulate)
         return;
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/DelegationTokenStub.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/DelegationTokenStub.java
deleted file mode 100644
index 62529f9..0000000
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/DelegationTokenStub.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.hadoopImpl.mapreduce;
-
-import static java.util.Objects.requireNonNull;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Set;
-
-import javax.security.auth.DestroyFailedException;
-
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-
-/**
- * An internal stub class for passing DelegationToken information out of the Configuration back up
- * to the appropriate implementation for mapreduce or mapred.
- */
-public class DelegationTokenStub implements AuthenticationToken {
-
-  private String serviceName;
-
-  public DelegationTokenStub(String serviceName) {
-    requireNonNull(serviceName);
-    this.serviceName = serviceName;
-  }
-
-  public String getServiceName() {
-    return serviceName;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void destroy() throws DestroyFailedException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public boolean isDestroyed() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void init(Properties properties) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public Set<TokenProperty> getProperties() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public AuthenticationToken clone() {
-    throw new UnsupportedOperationException();
-  }
-}
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/RangeInputSplit.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/RangeInputSplit.java
index e0751e2..3fcc881 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/RangeInputSplit.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/RangeInputSplit.java
@@ -117,7 +117,7 @@ public class RangeInputSplit extends InputSplit implements Writable {
   }
 
   @Override
-  public String[] getLocations() throws IOException {
+  public String[] getLocations() {
     return Arrays.copyOf(locations, locations.length);
   }
 
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/SplitUtils.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/SplitUtils.java
index 44855fb..bcd3af1 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/SplitUtils.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/SplitUtils.java
@@ -17,7 +17,6 @@
 
 package org.apache.accumulo.hadoopImpl.mapreduce;
 
-import java.io.IOException;
 import java.math.BigInteger;
 
 import org.apache.accumulo.core.data.ByteSequence;
@@ -46,7 +45,7 @@ public class SplitUtils {
         / endBI.subtract(startBI).doubleValue());
   }
 
-  public static long getRangeLength(Range range) throws IOException {
+  public static long getRangeLength(Range range) {
     Text startRow = range.isInfiniteStartKey() ? new Text(new byte[] {Byte.MIN_VALUE})
         : range.getStartKey().getRow();
     Text stopRow = range.isInfiniteStopKey() ? new Text(new byte[] {Byte.MAX_VALUE})
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/ConfiguratorBase.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/ConfiguratorBase.java
index 54bdb58..8d725bb 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/ConfiguratorBase.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/ConfiguratorBase.java
@@ -17,10 +17,7 @@
 package org.apache.accumulo.hadoopImpl.mapreduce.lib;
 
 import static com.google.common.base.Preconditions.checkArgument;
-import static java.util.Objects.requireNonNull;
 
-import java.io.ByteArrayInputStream;
-import java.io.DataInputStream;
 import java.io.IOException;
 import java.io.StringReader;
 import java.io.StringWriter;
@@ -40,16 +37,11 @@ import org.apache.accumulo.core.clientImpl.AuthenticationTokenIdentifier;
 import org.apache.accumulo.core.clientImpl.ClientInfoImpl;
 import org.apache.accumulo.core.clientImpl.DelegationTokenImpl;
 import org.apache.accumulo.core.conf.ClientProperty;
-import org.apache.accumulo.hadoopImpl.mapreduce.DelegationTokenStub;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.util.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -331,64 +323,4 @@ public class ConfiguratorBase {
     return conf.getInt(enumToConfKey(GeneralOpts.VISIBILITY_CACHE_SIZE),
         Constants.DEFAULT_VISIBILITY_CACHE_SIZE);
   }
-
-  /**
-   * Unwraps the provided {@link AuthenticationToken} if it is an instance of
-   * {@link DelegationTokenStub}, reconstituting it from the provided {@link JobConf}.
-   *
-   * @param job
-   *          The job
-   * @param token
-   *          The authentication token
-   */
-  public static AuthenticationToken unwrapAuthenticationToken(JobConf job,
-      AuthenticationToken token) {
-    requireNonNull(job);
-    requireNonNull(token);
-    if (token instanceof DelegationTokenStub) {
-      DelegationTokenStub delTokenStub = (DelegationTokenStub) token;
-      Token<? extends TokenIdentifier> hadoopToken = job.getCredentials()
-          .getToken(new Text(delTokenStub.getServiceName()));
-      AuthenticationTokenIdentifier identifier = new AuthenticationTokenIdentifier();
-      try {
-        identifier
-            .readFields(new DataInputStream(new ByteArrayInputStream(hadoopToken.getIdentifier())));
-        return new DelegationTokenImpl(hadoopToken.getPassword(), identifier);
-      } catch (IOException e) {
-        throw new RuntimeException("Could not construct DelegationToken from JobConf Credentials",
-            e);
-      }
-    }
-    return token;
-  }
-
-  /**
-   * Unwraps the provided {@link AuthenticationToken} if it is an instance of
-   * {@link DelegationTokenStub}, reconstituting it from the provided {@link JobConf}.
-   *
-   * @param job
-   *          The job
-   * @param token
-   *          The authentication token
-   */
-  public static AuthenticationToken unwrapAuthenticationToken(JobContext job,
-      AuthenticationToken token) {
-    requireNonNull(job);
-    requireNonNull(token);
-    if (token instanceof DelegationTokenStub) {
-      DelegationTokenStub delTokenStub = (DelegationTokenStub) token;
-      Token<? extends TokenIdentifier> hadoopToken = job.getCredentials()
-          .getToken(new Text(delTokenStub.getServiceName()));
-      AuthenticationTokenIdentifier identifier = new AuthenticationTokenIdentifier();
-      try {
-        identifier
-            .readFields(new DataInputStream(new ByteArrayInputStream(hadoopToken.getIdentifier())));
-        return new DelegationTokenImpl(hadoopToken.getPassword(), identifier);
-      } catch (IOException e) {
-        throw new RuntimeException("Could not construct DelegationToken from JobConf Credentials",
-            e);
-      }
-    }
-    return token;
-  }
 }
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/InputConfigurator.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/InputConfigurator.java
index 745ae41..46c1a06 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/InputConfigurator.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/InputConfigurator.java
@@ -810,8 +810,7 @@ public class InputConfigurator extends ConfiguratorBase {
   }
 
   public static Map<String,Map<KeyExtent,List<Range>>> binOffline(Table.ID tableId,
-      List<Range> ranges, ClientContext context)
-      throws AccumuloException, TableNotFoundException, AccumuloSecurityException {
+      List<Range> ranges, ClientContext context) throws AccumuloException, TableNotFoundException {
     Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<>();
 
     if (Tables.getTableState(context, tableId) != TableState.OFFLINE) {
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 4eb339b..49a7fa4 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,8 +16,6 @@
  */
 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;
@@ -39,7 +37,7 @@ public abstract class MapReduceClientOpts extends ClientOpts {
   private static final Logger log = LoggerFactory.getLogger(MapReduceClientOpts.class);
 
   public abstract void setAccumuloConfigs(Job job)
-      throws IOException, AccumuloException, AccumuloSecurityException;
+      throws AccumuloException, AccumuloSecurityException;
 
   @Override
   public AuthenticationToken getToken() {
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloFileOutputFormatIT.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloFileOutputFormatIT.java
index bc21ad4..225b461 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloFileOutputFormatIT.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloFileOutputFormatIT.java
@@ -24,7 +24,6 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.File;
-import java.io.IOException;
 
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.BatchWriter;
@@ -108,8 +107,7 @@ public class AccumuloFileOutputFormatIT extends AccumuloClusterHarness {
       int index = 0;
 
       @Override
-      public void map(Key key, Value value, OutputCollector<Key,Value> output, Reporter reporter)
-          throws IOException {
+      public void map(Key key, Value value, OutputCollector<Key,Value> output, Reporter reporter) {
         try {
           try {
             output.collect(key, value);
@@ -129,7 +127,7 @@ public class AccumuloFileOutputFormatIT extends AccumuloClusterHarness {
       public void configure(JobConf job) {}
 
       @Override
-      public void close() throws IOException {
+      public void close() {
         try {
           assertEquals(2, index);
         } catch (AssertionError e) {
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloInputFormatIT.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloInputFormatIT.java
index 6f7ca84..4c8ed2d 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloInputFormatIT.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloInputFormatIT.java
@@ -21,7 +21,6 @@ import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
 import java.io.File;
-import java.io.IOException;
 import java.util.Collection;
 import java.util.Collections;
 
@@ -73,8 +72,7 @@ public class AccumuloInputFormatIT extends AccumuloClusterHarness {
       int count = 0;
 
       @Override
-      public void map(Key k, Value v, OutputCollector<Key,Value> output, Reporter reporter)
-          throws IOException {
+      public void map(Key k, Value v, OutputCollector<Key,Value> output, Reporter reporter) {
         try {
           if (key != null)
             assertEquals(key.getRow().toString(), new String(v.get()));
@@ -92,7 +90,7 @@ public class AccumuloInputFormatIT extends AccumuloClusterHarness {
       public void configure(JobConf job) {}
 
       @Override
-      public void close() throws IOException {
+      public void close() {
         try {
           assertEquals(100, count);
         } catch (AssertionError e) {
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloRowInputFormatIT.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloRowInputFormatIT.java
index c5676e7..58beb78 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloRowInputFormatIT.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloRowInputFormatIT.java
@@ -21,7 +21,6 @@ import static org.junit.Assert.assertNull;
 import static org.junit.Assert.fail;
 
 import java.io.File;
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
@@ -113,7 +112,7 @@ public class AccumuloRowInputFormatIT extends AccumuloClusterHarness {
 
       @Override
       public void map(Text k, PeekingIterator<Entry<Key,Value>> v,
-          OutputCollector<Key,Value> output, Reporter reporter) throws IOException {
+          OutputCollector<Key,Value> output, Reporter reporter) {
         try {
           switch (count) {
             case 0:
@@ -141,7 +140,7 @@ public class AccumuloRowInputFormatIT extends AccumuloClusterHarness {
       public void configure(JobConf job) {}
 
       @Override
-      public void close() throws IOException {
+      public void close() {
         try {
           assertEquals(3, count);
         } catch (AssertionError e) {
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/TokenFileIT.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/TokenFileIT.java
index ba153a5..688146f 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/TokenFileIT.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/TokenFileIT.java
@@ -68,8 +68,7 @@ public class TokenFileIT extends AccumuloClusterHarness {
       OutputCollector<Text,Mutation> finalOutput;
 
       @Override
-      public void map(Key k, Value v, OutputCollector<Text,Mutation> output, Reporter reporter)
-          throws IOException {
+      public void map(Key k, Value v, OutputCollector<Text,Mutation> output, Reporter reporter) {
         finalOutput = output;
         try {
           if (key != null)
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloFileOutputFormatIT.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloFileOutputFormatIT.java
index 4797454..f036611 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloFileOutputFormatIT.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloFileOutputFormatIT.java
@@ -22,7 +22,6 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.File;
-import java.io.IOException;
 
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.BatchWriter;
@@ -118,8 +117,7 @@ public class AccumuloFileOutputFormatIT extends AccumuloClusterHarness {
       int index = 0;
 
       @Override
-      protected void map(Key key, Value value, Context context)
-          throws IOException, InterruptedException {
+      protected void map(Key key, Value value, Context context) {
         String table = context.getConfiguration().get("MRTester_tableName");
         assertNotNull(table);
         try {
@@ -137,7 +135,7 @@ public class AccumuloFileOutputFormatIT extends AccumuloClusterHarness {
       }
 
       @Override
-      protected void cleanup(Context context) throws IOException, InterruptedException {
+      protected void cleanup(Context context) {
         String table = context.getConfiguration().get("MRTester_tableName");
         assertNotNull(table);
         try {
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloInputFormatIT.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloInputFormatIT.java
index a1b4b6f..90d2571 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloInputFormatIT.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloInputFormatIT.java
@@ -33,7 +33,6 @@ import java.util.concurrent.TimeUnit;
 
 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.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.IteratorSetting;
@@ -202,7 +201,7 @@ public class AccumuloInputFormatIT extends AccumuloClusterHarness {
   }
 
   private void insertData(String tableName, long ts)
-      throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+      throws AccumuloException, TableNotFoundException {
     BatchWriter bw = getAccumuloClient().createBatchWriter(tableName, null);
 
     for (int i = 0; i < 10000; i++) {
@@ -227,7 +226,7 @@ public class AccumuloInputFormatIT extends AccumuloClusterHarness {
       int count = 0;
 
       @Override
-      protected void map(Key k, Value v, Context context) throws IOException, InterruptedException {
+      protected void map(Key k, Value v, Context context) {
         String table = context.getConfiguration().get("MRTester_tableName");
         assertNotNull(table);
         try {
@@ -243,7 +242,7 @@ public class AccumuloInputFormatIT extends AccumuloClusterHarness {
       }
 
       @Override
-      protected void cleanup(Context context) throws IOException, InterruptedException {
+      protected void cleanup(Context context) {
         String table = context.getConfiguration().get("MRTester_tableName");
         assertNotNull(table);
         try {
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloOutputFormatIT.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloOutputFormatIT.java
index 288fe70..2fbc1f2 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloOutputFormatIT.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloOutputFormatIT.java
@@ -55,7 +55,7 @@ public class AccumuloOutputFormatIT extends AccumuloClusterHarness {
       int count = 0;
 
       @Override
-      protected void map(Key k, Value v, Context context) throws IOException, InterruptedException {
+      protected void map(Key k, Value v, Context context) {
         try {
           if (key != null)
             assertEquals(key.getRow().toString(), new String(v.get()));
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloRowInputFormatIT.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloRowInputFormatIT.java
index 2d07dbd..17c3f6a 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloRowInputFormatIT.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloRowInputFormatIT.java
@@ -21,7 +21,6 @@ import static org.junit.Assert.assertNull;
 import static org.junit.Assert.fail;
 
 import java.io.File;
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
@@ -109,8 +108,7 @@ public class AccumuloRowInputFormatIT extends AccumuloClusterHarness {
       int count = 0;
 
       @Override
-      protected void map(Text k, PeekingIterator<Entry<Key,Value>> v, Context context)
-          throws IOException, InterruptedException {
+      protected void map(Text k, PeekingIterator<Entry<Key,Value>> v, Context context) {
         try {
           switch (count) {
             case 0:
@@ -135,7 +133,7 @@ public class AccumuloRowInputFormatIT extends AccumuloClusterHarness {
       }
 
       @Override
-      protected void cleanup(Context context) throws IOException, InterruptedException {
+      protected void cleanup(Context context) {
         try {
           assertEquals(3, count);
         } catch (AssertionError e) {
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/TokenFileIT.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/TokenFileIT.java
index dfb0103..c5d7e37 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/TokenFileIT.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/TokenFileIT.java
@@ -64,7 +64,7 @@ public class TokenFileIT extends AccumuloClusterHarness {
       int count = 0;
 
       @Override
-      protected void map(Key k, Value v, Context context) throws IOException, InterruptedException {
+      protected void map(Key k, Value v, Context context) {
         try {
           if (key != null)
             assertEquals(key.getRow().toString(), new String(v.get()));
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapred/AccumuloFileOutputFormatTest.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapred/AccumuloFileOutputFormatTest.java
index c170918..7c2efba 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapred/AccumuloFileOutputFormatTest.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapred/AccumuloFileOutputFormatTest.java
@@ -19,7 +19,6 @@ package org.apache.accumulo.hadoop.mapred;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
-import java.io.IOException;
 import java.util.Collection;
 
 import org.apache.accumulo.core.client.sample.RowSampler;
@@ -39,7 +38,7 @@ import org.junit.Test;
 public class AccumuloFileOutputFormatTest {
 
   @Test
-  public void validateConfiguration() throws IOException, InterruptedException {
+  public void validateConfiguration() {
 
     int a = 7;
     long b = 300L;
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapred/AccumuloOutputFormatTest.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapred/AccumuloOutputFormatTest.java
index 4cdd872..dc69dd9 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapred/AccumuloOutputFormatTest.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapred/AccumuloOutputFormatTest.java
@@ -54,7 +54,7 @@ public class AccumuloOutputFormatTest {
 
     AccumuloOutputFormat myAOF = new AccumuloOutputFormat() {
       @Override
-      public void checkOutputSpecs(FileSystem ignored, JobConf job) throws IOException {
+      public void checkOutputSpecs(FileSystem ignored, JobConf job) {
         BatchWriterConfig bwOpts = OutputConfigurator
             .getBatchWriterOptions(AccumuloOutputFormat.class, job);
 
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapreduce/AccumuloOutputFormatTest.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapreduce/AccumuloOutputFormatTest.java
index 35839be..6351396 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapreduce/AccumuloOutputFormatTest.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapreduce/AccumuloOutputFormatTest.java
@@ -56,7 +56,7 @@ public class AccumuloOutputFormatTest {
 
     AccumuloOutputFormat myAOF = new AccumuloOutputFormat() {
       @Override
-      public void checkOutputSpecs(JobContext job) throws IOException {
+      public void checkOutputSpecs(JobContext job) {
         BatchWriterConfig bwOpts = OutputConfigurator
             .getBatchWriterOptions(AccumuloOutputFormat.class, job.getConfiguration());
 
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/ConfiguratorBaseTest.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/ConfiguratorBaseTest.java
index 2c16354..95f1c09 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/ConfiguratorBaseTest.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/ConfiguratorBaseTest.java
@@ -23,7 +23,6 @@ import static org.junit.Assert.assertTrue;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.ClientInfo;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
@@ -59,8 +58,7 @@ public class ConfiguratorBaseTest {
   }
 
   @Test
-  public void testSetConnectorInfoClassOfQConfigurationStringString()
-      throws AccumuloSecurityException {
+  public void testSetConnectorInfoClassOfQConfigurationStringString() {
     Configuration conf = new Configuration();
     assertFalse(ConfiguratorBase.isConnectorInfoSet(this.getClass(), conf));
     ConfiguratorBase.setConnectorInfo(this.getClass(), conf, "testUser",