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/12/07 18:33:18 UTC

[GitHub] milleruntime closed pull request #809: Minor hadoop-mapreduce cleanup

milleruntime closed pull request #809: Minor hadoop-mapreduce cleanup
URL: https://github.com/apache/accumulo/pull/809
 
 
   

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/mapred/AccumuloFileOutputFormat.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapred/AccumuloFileOutputFormat.java
index ff531b96d0..0edcd844fd 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 @@
 
   @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 92df89d1b0..6fabcf90fd 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 @@
     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 c1c6cbe2fc..bdcfbdb99e 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 void initialize(InputSplit inSplit, JobConf job) throws IOException {
           }
 
           @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 2f869e9773..7e3d56f2c9 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 void write(Key key, Value value) throws IOException {
    * 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 9ea2ff46b3..4f89bf058f 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 @@
 
     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 16f7395ceb..0e7dfb6527 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 OutputCommitter getOutputCommitter(TaskAttemptContext context) {
    * 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 90147fde60..d89e8492ad 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 @@
 
   @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 void initialize(InputSplit inSplit, TaskAttemptContext attempt) throws IO
       }
 
       @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 boolean nextKeyValue() throws IOException, InterruptedException {
    * 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 c86c310971..657f66c96a 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 void close() {
     }
 
     @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 2b6942d026..99aa434787 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 @@ protected static Boolean getSimulationMode(JobConf job) {
 
     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 891a354f50..ea13fb3295 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 void close() {
     }
 
     @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 float getProgress() throws IOException {
     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 50b892599f..11b57e2fcb 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 @@ protected static Boolean getSimulationMode(JobContext context) {
 
     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 @@ private String hexDump(byte[] ba) {
     }
 
     @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 62529f962e..0000000000
--- 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 e0751e26c4..3fcc88189c 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 long getLength() throws IOException {
   }
 
   @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 44855fb039..bcd3af14fe 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 static float getProgress(ByteSequence start, ByteSequence end, ByteSequen
         / 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 54bdb58c92..8d725bb89d 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.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 static int getVisibilityCacheSize(Configuration conf) {
     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 745ae4146d..46c1a06731 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 static void validatePermissions(Class<?> implementingClass, Configuration
   }
 
   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 4eb339b5b1..49a7fa427c 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 @@
   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 bc21ad4c7e..225b461e00 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.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 void testRealWrite() throws Exception {
       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 void map(Key key, Value value, OutputCollector<Key,Value> output, Reporte
       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 6f7ca84ca7..4c8ed2dff4 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.assertTrue;
 
 import java.io.File;
-import java.io.IOException;
 import java.util.Collection;
 import java.util.Collections;
 
@@ -73,8 +72,7 @@ public static void setupClass() {
       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 void map(Key k, Value v, OutputCollector<Key,Value> output, Reporter repo
       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 c5676e70c0..58beb78819 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.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 @@ private static void insertList(final BatchWriter writer, final List<Entry<Key,Va
 
       @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 void map(Text k, PeekingIterator<Entry<Key,Value>> v,
       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 ba153a5c06..688146fc89 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 @@
       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 4797454084..f0366110a4 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.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 void testRealWrite() throws Exception {
       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 @@ protected void map(Key key, Value value, Context context)
       }
 
       @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 a1b4b6f3df..90d2571a01 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 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 void testGetSplits() throws Exception {
   }
 
   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 @@ private void insertData(String tableName, long ts)
       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 @@ protected void map(Key k, Value v, Context context) throws IOException, Interrup
       }
 
       @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 288fe70f00..2fbc1f286c 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 @@
       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 2d07dbd30d..17c3f6a56b 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.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 @@ private static void insertList(final BatchWriter writer, final List<Entry<Key,Va
       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 @@ protected void map(Text k, PeekingIterator<Entry<Key,Value>> v, Context context)
       }
 
       @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 dfb010324e..c5d7e37853 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 @@
       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 c170918199..7c2efba6e6 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 @@
 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 @@
 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 4cdd872991..dc69dd97f4 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 void testBWSettings() throws IOException {
 
     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 35839be2da..635139608f 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 void testBWSettings() throws IOException {
 
     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 2c16354f2b..95f1c09427 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 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 void testSetConnectorInfoClassOfQConfigurationStringAuthenticationToken()
   }
 
   @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",


 

----------------------------------------------------------------
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