You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ct...@apache.org on 2017/01/13 17:59:06 UTC

[1/2] accumulo git commit: ACCUMULO-4568 Remove non-public API leakage

Repository: accumulo
Updated Branches:
  refs/heads/master fd0a4ebf1 -> dca2ff939


http://git-wip-us.apache.org/repos/asf/accumulo/blob/dca2ff93/core/src/main/java/org/apache/accumulo/core/data/ComparableBytes.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/data/ComparableBytes.java b/core/src/main/java/org/apache/accumulo/core/data/ComparableBytes.java
deleted file mode 100644
index 78c0e56..0000000
--- a/core/src/main/java/org/apache/accumulo/core/data/ComparableBytes.java
+++ /dev/null
@@ -1,58 +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.core.data;
-
-import org.apache.hadoop.io.BinaryComparable;
-
-/**
- * An array of bytes wrapped so as to extend Hadoop's <code>BinaryComparable</code> class.
- *
- * @deprecated since 1.7.0 In an attempt to clean up types in the data package that were not intended to be in public API this type was deprecated. Technically
- *             this method was not considered part of the public API in 1.6.0 and earlier, therefore it could have been deleted. However a decision was made to
- *             deprecate in order to be cautious and avoid confusion between 1.6.0 and 1.7.0.
- */
-@Deprecated
-public class ComparableBytes extends BinaryComparable {
-
-  public byte[] data;
-
-  /**
-   * Creates a new byte wrapper. The given byte array is used directly as a backing array, so later changes made to the array reflect into the new object.
-   *
-   * @param b
-   *          bytes to wrap
-   */
-  public ComparableBytes(byte[] b) {
-    this.data = b;
-  }
-
-  /**
-   * Gets the wrapped bytes in this object.
-   *
-   * @return bytes
-   */
-  @Override
-  public byte[] getBytes() {
-    return data;
-  }
-
-  @Override
-  public int getLength() {
-    return data.length;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dca2ff93/core/src/main/java/org/apache/accumulo/core/data/KeyExtent.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/data/KeyExtent.java b/core/src/main/java/org/apache/accumulo/core/data/KeyExtent.java
deleted file mode 100644
index 4e3d058..0000000
--- a/core/src/main/java/org/apache/accumulo/core/data/KeyExtent.java
+++ /dev/null
@@ -1,259 +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.core.data;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.SortedMap;
-import java.util.SortedSet;
-import java.util.TreeMap;
-import java.util.TreeSet;
-import java.util.UUID;
-
-import org.apache.accumulo.core.data.thrift.TKeyExtent;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema;
-import org.apache.hadoop.io.BinaryComparable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.WritableComparable;
-
-/**
- * keeps track of information needed to identify a tablet
- *
- * @deprecated since 1.7.0 use {@link TabletId}
- */
-@Deprecated
-public class KeyExtent implements WritableComparable<KeyExtent> {
-
-  // Wrapping impl.KeyExtent to resuse code. Did not want to extend impl.KeyExtent because any changes to impl.KeyExtent would be reflected in this class.
-  // Wrapping impl.KeyExtent allows the API of this deprecated class to be frozen.
-  private org.apache.accumulo.core.data.impl.KeyExtent wrapped;
-
-  public KeyExtent() {
-    this.wrapped = new org.apache.accumulo.core.data.impl.KeyExtent();
-  }
-
-  public KeyExtent(Text table, Text endRow, Text prevEndRow) {
-    this.wrapped = new org.apache.accumulo.core.data.impl.KeyExtent(table.toString(), endRow, prevEndRow);
-  }
-
-  public KeyExtent(KeyExtent extent) {
-    this.wrapped = new org.apache.accumulo.core.data.impl.KeyExtent(extent.getTableId().toString(), extent.getEndRow(), extent.getPrevEndRow());
-  }
-
-  public KeyExtent(TKeyExtent tke) {
-    this.wrapped = new org.apache.accumulo.core.data.impl.KeyExtent(tke);
-  }
-
-  // constructor for loading extents from metadata rows
-  public KeyExtent(Text flattenedExtent, Value prevEndRow) {
-    this.wrapped = new org.apache.accumulo.core.data.impl.KeyExtent(flattenedExtent, prevEndRow);
-  }
-
-  // recreates an encoded extent from a string representation
-  // this encoding is what is stored as the row id of the metadata table
-  public KeyExtent(Text flattenedExtent, Text prevEndRow) {
-    this.wrapped = new org.apache.accumulo.core.data.impl.KeyExtent(flattenedExtent, prevEndRow);
-  }
-
-  public Text getMetadataEntry() {
-    return wrapped.getMetadataEntry();
-  }
-
-  public void setTableId(Text tId) {
-    wrapped.setTableId(tId.toString());
-  }
-
-  public Text getTableId() {
-    return new Text(wrapped.getTableId());
-  }
-
-  public void setEndRow(Text endRow) {
-    wrapped.setEndRow(endRow);
-  }
-
-  public Text getEndRow() {
-    return wrapped.getEndRow();
-  }
-
-  public Text getPrevEndRow() {
-    return wrapped.getPrevEndRow();
-  }
-
-  public void setPrevEndRow(Text prevEndRow) {
-    wrapped.setPrevEndRow(prevEndRow);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    wrapped.readFields(in);
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    wrapped.write(out);
-  }
-
-  public Mutation getPrevRowUpdateMutation() {
-    return wrapped.getPrevRowUpdateMutation();
-  }
-
-  @Override
-  public int compareTo(KeyExtent other) {
-    return wrapped.compareTo(other.wrapped);
-  }
-
-  @Override
-  public int hashCode() {
-    return wrapped.hashCode();
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (o instanceof KeyExtent) {
-      return wrapped.equals(((KeyExtent) o).wrapped);
-    }
-
-    return false;
-  }
-
-  @Override
-  public String toString() {
-    return wrapped.toString();
-  }
-
-  public UUID getUUID() {
-    return wrapped.getUUID();
-  }
-
-  public boolean contains(ByteSequence bsrow) {
-    return wrapped.contains(bsrow);
-  }
-
-  public boolean contains(BinaryComparable row) {
-    return wrapped.contains(row);
-  }
-
-  public Range toDataRange() {
-    return wrapped.toDataRange();
-  }
-
-  public Range toMetadataRange() {
-    return wrapped.toMetadataRange();
-  }
-
-  public boolean overlaps(KeyExtent other) {
-    return wrapped.overlaps(other.wrapped);
-  }
-
-  public TKeyExtent toThrift() {
-    return wrapped.toThrift();
-  }
-
-  public boolean isPreviousExtent(KeyExtent prevExtent) {
-    return wrapped.isPreviousExtent(prevExtent.wrapped);
-  }
-
-  public boolean isMeta() {
-    return wrapped.isMeta();
-  }
-
-  public boolean isRootTablet() {
-    return wrapped.isRootTablet();
-  }
-
-  private static SortedSet<org.apache.accumulo.core.data.impl.KeyExtent> unwrap(Set<KeyExtent> tablets) {
-    SortedSet<org.apache.accumulo.core.data.impl.KeyExtent> trans = new TreeSet<>();
-    for (KeyExtent wrapper : tablets) {
-      trans.add(wrapper.wrapped);
-    }
-
-    return trans;
-  }
-
-  private static KeyExtent wrap(org.apache.accumulo.core.data.impl.KeyExtent ke) {
-    return new KeyExtent(new Text(ke.getTableId()), ke.getEndRow(), ke.getPrevEndRow());
-  }
-
-  private static SortedSet<KeyExtent> wrap(Collection<org.apache.accumulo.core.data.impl.KeyExtent> unwrapped) {
-    SortedSet<KeyExtent> wrapped = new TreeSet<>();
-    for (org.apache.accumulo.core.data.impl.KeyExtent wrappee : unwrapped) {
-      wrapped.add(wrap(wrappee));
-    }
-
-    return wrapped;
-  }
-
-  public static Text getMetadataEntry(Text tableId, Text endRow) {
-    return MetadataSchema.TabletsSection.getRow(tableId.toString(), endRow);
-  }
-
-  /**
-   * Empty start or end rows tell the method there are no start or end rows, and to use all the keyextents that are before the end row if no start row etc.
-   *
-   * @deprecated this method not intended for public use and is likely to be removed in a future version.
-   * @return all the key extents that the rows cover
-   */
-  @Deprecated
-  public static Collection<KeyExtent> getKeyExtentsForRange(Text startRow, Text endRow, Set<KeyExtent> kes) {
-    return wrap(org.apache.accumulo.core.data.impl.KeyExtent.getKeyExtentsForRange(startRow, endRow, unwrap(kes)));
-  }
-
-  public static Text decodePrevEndRow(Value ibw) {
-    return org.apache.accumulo.core.data.impl.KeyExtent.decodePrevEndRow(ibw);
-  }
-
-  public static Value encodePrevEndRow(Text per) {
-    return org.apache.accumulo.core.data.impl.KeyExtent.encodePrevEndRow(per);
-  }
-
-  public static Mutation getPrevRowUpdateMutation(KeyExtent ke) {
-    return org.apache.accumulo.core.data.impl.KeyExtent.getPrevRowUpdateMutation(ke.wrapped);
-  }
-
-  public static byte[] tableOfMetadataRow(Text row) {
-    return org.apache.accumulo.core.data.impl.KeyExtent.tableOfMetadataRow(row);
-  }
-
-  public static SortedSet<KeyExtent> findChildren(KeyExtent ke, SortedSet<KeyExtent> tablets) {
-    return wrap(org.apache.accumulo.core.data.impl.KeyExtent.findChildren(ke.wrapped, unwrap(tablets)));
-  }
-
-  public static KeyExtent findContainingExtent(KeyExtent extent, SortedSet<KeyExtent> extents) {
-    return wrap(org.apache.accumulo.core.data.impl.KeyExtent.findContainingExtent(extent.wrapped, unwrap(extents)));
-  }
-
-  public static Set<KeyExtent> findOverlapping(KeyExtent nke, SortedSet<KeyExtent> extents) {
-    return wrap(org.apache.accumulo.core.data.impl.KeyExtent.findOverlapping(nke.wrapped, unwrap(extents)));
-  }
-
-  public static Set<KeyExtent> findOverlapping(KeyExtent nke, SortedMap<KeyExtent,?> extents) {
-    SortedMap<org.apache.accumulo.core.data.impl.KeyExtent,Object> trans = new TreeMap<>();
-    for (Entry<KeyExtent,?> entry : extents.entrySet()) {
-      trans.put(entry.getKey().wrapped, entry.getValue());
-    }
-
-    return wrap(org.apache.accumulo.core.data.impl.KeyExtent.findOverlapping(nke.wrapped, trans));
-  }
-
-  public static Text getMetadataEntry(KeyExtent extent) {
-    return org.apache.accumulo.core.data.impl.KeyExtent.getMetadataEntry(extent.wrapped);
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dca2ff93/core/src/main/java/org/apache/accumulo/core/data/impl/TabletIdImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/data/impl/TabletIdImpl.java b/core/src/main/java/org/apache/accumulo/core/data/impl/TabletIdImpl.java
index d28a1ee..d34e379 100644
--- a/core/src/main/java/org/apache/accumulo/core/data/impl/TabletIdImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/data/impl/TabletIdImpl.java
@@ -17,8 +17,6 @@
 
 package org.apache.accumulo.core.data.impl;
 
-import java.util.function.Function;
-
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.TabletId;
 import org.apache.hadoop.io.Text;
@@ -27,35 +25,6 @@ public class TabletIdImpl implements TabletId {
 
   private KeyExtent ke;
 
-  @SuppressWarnings("deprecation")
-  public static final Function<org.apache.accumulo.core.data.KeyExtent,TabletId> KE_2_TID_OLD = new Function<org.apache.accumulo.core.data.KeyExtent,TabletId>() {
-    @Override
-    public TabletId apply(org.apache.accumulo.core.data.KeyExtent input) {
-      // the following if null check is to appease findbugs... grumble grumble spent a good part of my morning looking into this
-      // http://sourceforge.net/p/findbugs/bugs/1139/
-      // https://code.google.com/p/guava-libraries/issues/detail?id=920
-      if (input == null)
-        return null;
-      return new TabletIdImpl(input);
-    }
-  };
-
-  @SuppressWarnings("deprecation")
-  public static final Function<TabletId,org.apache.accumulo.core.data.KeyExtent> TID_2_KE_OLD = new Function<TabletId,org.apache.accumulo.core.data.KeyExtent>() {
-    @Override
-    public org.apache.accumulo.core.data.KeyExtent apply(TabletId input) {
-      if (input == null)
-        return null;
-      return new org.apache.accumulo.core.data.KeyExtent(input.getTableId(), input.getEndRow(), input.getPrevEndRow());
-    }
-
-  };
-
-  @Deprecated
-  public TabletIdImpl(org.apache.accumulo.core.data.KeyExtent ke) {
-    this.ke = new KeyExtent(ke.getTableId().toString(), ke.getEndRow(), ke.getPrevEndRow());
-  }
-
   public TabletIdImpl(KeyExtent ke) {
     this.ke = ke;
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dca2ff93/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java b/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java
index afe9349..8bd72bf 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java
@@ -70,17 +70,6 @@ public class IteratorUtil {
    */
   public static enum IteratorScope {
     majc, minc, scan;
-
-    /**
-     * Fetch the correct configuration key prefix for the given scope. Throws an IllegalArgumentException if no property exists for the given scope.
-     *
-     * @deprecated since 1.7.0 This method returns a type that is not part of the public API and is not guaranteed to be stable. The method was deprecated to
-     *             discourage its use.
-     */
-    @Deprecated
-    public static Property getProperty(IteratorScope scope) {
-      return IteratorUtil.getProperty(scope);
-    }
   }
 
   public static class IterInfoComparator implements Comparator<IterInfo>, Serializable {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dca2ff93/core/src/main/java/org/apache/accumulo/core/security/VisibilityConstraint.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/security/VisibilityConstraint.java b/core/src/main/java/org/apache/accumulo/core/security/VisibilityConstraint.java
deleted file mode 100644
index 611c8d4..0000000
--- a/core/src/main/java/org/apache/accumulo/core/security/VisibilityConstraint.java
+++ /dev/null
@@ -1,29 +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.core.security;
-
-/**
- *
- * @deprecated since 1.7.0 This is server side code not intended to exist in a public API package. This class references types that are not in the public API
- *             and therefore is not guaranteed to be stable. It was deprecated to clearly communicate this. Use
- *             {@link org.apache.accumulo.core.constraints.VisibilityConstraint} instead.
- */
-@Deprecated
-public class VisibilityConstraint extends org.apache.accumulo.core.constraints.VisibilityConstraint {
-
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dca2ff93/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java b/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
index bab52f6..39f1621 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
@@ -45,7 +45,6 @@ import org.apache.accumulo.core.client.impl.TabletLocator.TabletServerMutations;
 import org.apache.accumulo.core.client.impl.TabletLocatorImpl.TabletLocationObtainer;
 import org.apache.accumulo.core.client.impl.TabletLocatorImpl.TabletServerLockChecker;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.PartialKey;
@@ -451,18 +450,6 @@ public class TabletLocatorImplTest {
       throw new UnsupportedOperationException();
     }
 
-    @Deprecated
-    @Override
-    public AccumuloConfiguration getConfiguration() {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    @Deprecated
-    public void setConfiguration(AccumuloConfiguration conf) {
-      throw new UnsupportedOperationException();
-    }
-
     @Override
     @Deprecated
     public Connector getConnector(String user, CharSequence pass) throws AccumuloException, AccumuloSecurityException {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dca2ff93/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 9128dd0..f813c51 100644
--- a/pom.xml
+++ b/pom.xml
@@ -116,6 +116,7 @@
   <properties>
     <!-- used for filtering the java source with the current version -->
     <accumulo.release.version>${project.version}</accumulo.release.version>
+    <apilyzer.ignoreDeprecated>false</apilyzer.ignoreDeprecated>
     <assembly.tarLongFileMode>posix</assembly.tarLongFileMode>
     <!-- bouncycastle version for test dependencies -->
     <bouncycastle.version>1.54</bouncycastle.version>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dca2ff93/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java b/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
index 49bf1f4..745bd6f 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
@@ -24,7 +24,6 @@ import java.util.Collections;
 import java.util.List;
 import java.util.UUID;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
@@ -48,16 +47,16 @@ import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
 import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.accumulo.server.Accumulo;
-import org.apache.accumulo.server.conf.ServerConfigurationFactory;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.accumulo.server.zookeeper.ZooLock;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
-import com.google.common.base.Joiner;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.base.Joiner;
+
 /**
  * An implementation of Instance that looks in HDFS and ZooKeeper to find the master and root tablet location.
  *
@@ -195,30 +194,6 @@ public class HdfsZooInstance implements Instance {
     return getConnector(user, TextUtil.getBytes(new Text(pass.toString())));
   }
 
-  private final AtomicReference<AccumuloConfiguration> conf = new AtomicReference<>();
-
-  @Deprecated
-  @Override
-  public AccumuloConfiguration getConfiguration() {
-    AccumuloConfiguration conf = this.conf.get();
-    if (conf == null) {
-      // conf hasn't been set before, get an instance
-      conf = new ServerConfigurationFactory(this).getConfiguration();
-      // if the shared variable is still null, we're done.
-      if (!(this.conf.compareAndSet(null, conf))) {
-        // if it wasn't null, then we need to return the value that won.
-        conf = this.conf.get();
-      }
-    }
-    return conf;
-  }
-
-  @Override
-  @Deprecated
-  public void setConfiguration(AccumuloConfiguration conf) {
-    this.conf.set(conf);
-  }
-
   public static void main(String[] args) {
     Instance instance = HdfsZooInstance.getInstance();
     System.out.println("Instance Name: " + instance.getInstanceName());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dca2ff93/server/base/src/test/java/org/apache/accumulo/server/master/balancer/BaseHostRegexTableLoadBalancerTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/BaseHostRegexTableLoadBalancerTest.java b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/BaseHostRegexTableLoadBalancerTest.java
index 3ed6a1b..9360c82 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/BaseHostRegexTableLoadBalancerTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/BaseHostRegexTableLoadBalancerTest.java
@@ -98,16 +98,6 @@ public abstract class BaseHostRegexTableLoadBalancerTest extends HostRegexTableL
       throw new UnsupportedOperationException();
     }
 
-    @Deprecated
-    @Override
-    public AccumuloConfiguration getConfiguration() {
-      throw new UnsupportedOperationException();
-    }
-
-    @Deprecated
-    @Override
-    public void setConfiguration(AccumuloConfiguration conf) {}
-
     @Override
     public Connector getConnector(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
       throw new UnsupportedOperationException();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dca2ff93/server/base/src/test/java/org/apache/accumulo/server/util/TServerUtilsTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/util/TServerUtilsTest.java b/server/base/src/test/java/org/apache/accumulo/server/util/TServerUtilsTest.java
index 37d127a..bb0b177 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/util/TServerUtilsTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/util/TServerUtilsTest.java
@@ -106,16 +106,6 @@ public class TServerUtilsTest {
       throw new UnsupportedOperationException();
     }
 
-    @Deprecated
-    @Override
-    public AccumuloConfiguration getConfiguration() {
-      throw new UnsupportedOperationException();
-    }
-
-    @Deprecated
-    @Override
-    public void setConfiguration(AccumuloConfiguration conf) {}
-
     @Override
     public Connector getConnector(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
       throw new UnsupportedOperationException();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dca2ff93/test/src/main/java/org/apache/accumulo/test/server/security/SystemCredentialsIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/server/security/SystemCredentialsIT.java b/test/src/main/java/org/apache/accumulo/test/server/security/SystemCredentialsIT.java
index 9752916..20f7390 100644
--- a/test/src/main/java/org/apache/accumulo/test/server/security/SystemCredentialsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/server/security/SystemCredentialsIT.java
@@ -32,7 +32,6 @@ import org.apache.accumulo.core.client.impl.Credentials;
 import org.apache.accumulo.core.client.security.SecurityErrorCode;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.metadata.RootTable;
@@ -65,12 +64,6 @@ public class SystemCredentialsIT extends ConfigurableMacBase {
     if (args[0].equals("bad")) {
       Instance inst = new Instance() {
 
-        @Deprecated
-        @Override
-        public void setConfiguration(AccumuloConfiguration conf) {
-          throw new UnsupportedOperationException();
-        }
-
         @Override
         public int getZooKeepersSessionTimeOut() {
           throw new UnsupportedOperationException();
@@ -124,12 +117,6 @@ public class SystemCredentialsIT extends ConfigurableMacBase {
           throw new UnsupportedOperationException();
         }
 
-        @Deprecated
-        @Override
-        public AccumuloConfiguration getConfiguration() {
-          throw new UnsupportedOperationException();
-        }
-
       };
       creds = SystemCredentials.get(inst);
     } else if (args[0].equals("good")) {
@@ -190,18 +177,6 @@ public class SystemCredentialsIT extends ConfigurableMacBase {
           throw new UnsupportedOperationException();
         }
 
-        @Deprecated
-        @Override
-        public AccumuloConfiguration getConfiguration() {
-          throw new UnsupportedOperationException();
-        }
-
-        @Deprecated
-        @Override
-        public void setConfiguration(AccumuloConfiguration conf) {
-          throw new UnsupportedOperationException();
-        }
-
       };
       creds = new SystemCredentials(inst, "!SYSTEM", new PasswordToken("fake"));
     } else {


[2/2] accumulo git commit: ACCUMULO-4568 Remove non-public API leakage

Posted by ct...@apache.org.
ACCUMULO-4568 Remove non-public API leakage

Remove previously deprecated non-public API from being leaked in the
public API. Prevent that from happening again by watching more closely
with the apilyzer-maven-plugin (no longer ignores deprecations).


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

Branch: refs/heads/master
Commit: dca2ff93976957b40cae19cdf464586060cfcb5c
Parents: fd0a4eb
Author: Christopher Tubbs <ct...@apache.org>
Authored: Thu Jan 12 16:25:23 2017 -0500
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Fri Jan 13 12:56:24 2017 -0500

----------------------------------------------------------------------
 core/pom.xml                                    |   2 +
 .../core/client/ClientConfiguration.java        |  17 +-
 .../core/client/ClientSideIteratorScanner.java  |  15 -
 .../apache/accumulo/core/client/Instance.java   |  23 -
 .../core/client/MutationsRejectedException.java |  77 ----
 .../accumulo/core/client/ZooKeeperInstance.java |  25 -
 .../core/client/admin/ActiveCompaction.java     |   7 -
 .../accumulo/core/client/admin/ActiveScan.java  |   7 -
 .../core/client/impl/ActiveCompactionImpl.java  |   9 -
 .../core/client/impl/ActiveScanImpl.java        |   7 -
 .../core/client/mapred/AbstractInputFormat.java |  17 -
 .../client/mapred/AccumuloFileOutputFormat.java |  15 -
 .../core/client/mapred/InputFormatBase.java     |  18 -
 .../client/mapreduce/AbstractInputFormat.java   |  19 -
 .../mapreduce/AccumuloFileOutputFormat.java     |  16 -
 .../core/client/mapreduce/InputFormatBase.java  |  18 -
 .../mapreduce/lib/util/ConfiguratorBase.java    | 275 -----------
 .../lib/util/FileOutputConfigurator.java        | 170 -------
 .../mapreduce/lib/util/InputConfigurator.java   | 461 -------------------
 .../mapreduce/lib/util/OutputConfigurator.java  | 196 --------
 .../client/mapreduce/lib/util/package-info.java |  22 -
 .../accumulo/core/client/mock/MockInstance.java |  16 -
 .../accumulo/core/data/ComparableBytes.java     |  58 ---
 .../apache/accumulo/core/data/KeyExtent.java    | 259 -----------
 .../accumulo/core/data/impl/TabletIdImpl.java   |  31 --
 .../accumulo/core/iterators/IteratorUtil.java   |  11 -
 .../core/security/VisibilityConstraint.java     |  29 --
 .../core/client/impl/TabletLocatorImplTest.java |  13 -
 pom.xml                                         |   1 +
 .../accumulo/server/client/HdfsZooInstance.java |  29 +-
 .../BaseHostRegexTableLoadBalancerTest.java     |  10 -
 .../accumulo/server/util/TServerUtilsTest.java  |  10 -
 .../server/security/SystemCredentialsIT.java    |  25 -
 33 files changed, 6 insertions(+), 1902 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/dca2ff93/core/pom.xml
----------------------------------------------------------------------
diff --git a/core/pom.xml b/core/pom.xml
index 2e858b8..80a328f 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -208,7 +208,9 @@
                 <allow>org[.]apache[.]hadoop[.]fs[.](FileSystem|Path)</allow>
                 <allow>org[.]apache[.]hadoop[.]io[.](Text|Writable|WritableComparable|WritableComparator)</allow>
                 <allow>org[.]apache[.]hadoop[.]mapred[.](JobConf|RecordReader|InputSplit|RecordWriter|Reporter)</allow>
+                <allow>org[.]apache[.]hadoop[.]mapred[.]FileOutputFormat[$]Counter</allow>
                 <allow>org[.]apache[.]hadoop[.]mapreduce[.](Job|JobContext|RecordReader|InputSplit|TaskAttemptContext|RecordWriter|OutputCommitter|TaskInputOutputContext)</allow>
+                <allow>org[.]apache[.]hadoop[.]mapreduce[.]lib[.]output[.]FileOutputFormat[$]Counter</allow>
                 <allow>org[.]apache[.]hadoop[.]util[.]Progressable</allow>
                 <!--ugghhh-->
                 <allow>org[.]apache[.]log4j[.](Level|Logger)</allow>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dca2ff93/core/src/main/java/org/apache/accumulo/core/client/ClientConfiguration.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/ClientConfiguration.java b/core/src/main/java/org/apache/accumulo/core/client/ClientConfiguration.java
index c3dfb8b..a58d8c0 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/ClientConfiguration.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/ClientConfiguration.java
@@ -97,11 +97,8 @@ public class ClientConfiguration extends CompositeConfiguration {
     private PropertyType type;
     private String description;
 
-    private Property accumuloProperty = null;
-
     private ClientProperty(Property prop) {
       this(prop.getKey(), prop.getDefaultValue(), prop.getType(), prop.getDescription());
-      accumuloProperty = prop;
     }
 
     private ClientProperty(String key, String defaultValue, PropertyType type, String description) {
@@ -119,11 +116,7 @@ public class ClientConfiguration extends CompositeConfiguration {
       return defaultValue;
     }
 
-    /**
-     * @deprecated since 1.7.0 This method returns a type that is not part of the public API and not guaranteed to be stable.
-     */
-    @Deprecated
-    public PropertyType getType() {
+    private PropertyType getType() {
       return type;
     }
 
@@ -131,14 +124,6 @@ public class ClientConfiguration extends CompositeConfiguration {
       return description;
     }
 
-    /**
-     * @deprecated since 1.7.0 This method returns a type that is not part of the public API and not guaranteed to be stable.
-     */
-    @Deprecated
-    public Property getAccumuloProperty() {
-      return accumuloProperty;
-    }
-
     public static ClientProperty getPropertyByKey(String key) {
       for (ClientProperty prop : ClientProperty.values())
         if (prop.getKey().equals(key))

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dca2ff93/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java b/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java
index d4622c6..0b8aadb 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java
@@ -65,21 +65,6 @@ public class ClientSideIteratorScanner extends ScannerOptions implements Scanner
   private long readaheadThreshold = Constants.SCANNER_DEFAULT_READAHEAD_THRESHOLD;
   private SamplerConfiguration iteratorSamplerConfig;
 
-  /**
-   * @deprecated since 1.7.0 was never intended for public use. However this could have been used by anything extending this class.
-   */
-  @Deprecated
-  public class ScannerTranslator extends ScannerTranslatorImpl {
-    public ScannerTranslator(Scanner scanner) {
-      super(scanner, scanner.getSamplerConfiguration());
-    }
-
-    @Override
-    public SortedKeyValueIterator<Key,Value> deepCopy(final IteratorEnvironment env) {
-      return new ScannerTranslator(scanner);
-    }
-  }
-
   private class ClientSideIteratorEnvironment implements IteratorEnvironment {
 
     private SamplerConfiguration samplerConfig;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dca2ff93/core/src/main/java/org/apache/accumulo/core/client/Instance.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/Instance.java b/core/src/main/java/org/apache/accumulo/core/client/Instance.java
index 8a70d4c..ff6375b 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/Instance.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/Instance.java
@@ -19,10 +19,8 @@ package org.apache.accumulo.core.client;
 import java.nio.ByteBuffer;
 import java.util.List;
 
-import org.apache.accumulo.core.client.admin.InstanceOperations;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
 
 /**
  * This class represents the information a client needs to know to connect to an instance of accumulo.
@@ -123,27 +121,6 @@ public interface Instance {
   Connector getConnector(String user, CharSequence pass) throws AccumuloException, AccumuloSecurityException;
 
   /**
-   * Returns the AccumuloConfiguration to use when interacting with this instance.
-   *
-   * @return the AccumuloConfiguration that specifies properties related to interacting with this instance
-   * @deprecated since 1.6.0. This method makes very little sense in the context of the client API and never should have been exposed.
-   * @see InstanceOperations#getSystemConfiguration() for client-side reading of the server-side configuration.
-   */
-  @Deprecated
-  AccumuloConfiguration getConfiguration();
-
-  /**
-   * Set the AccumuloConfiguration to use when interacting with this instance.
-   *
-   * @param conf
-   *          accumulo configuration
-   * @deprecated since 1.6.0. This method makes very little sense in the context of the client API and never should have been exposed.
-   * @see InstanceOperations#setProperty(String, String)
-   */
-  @Deprecated
-  void setConfiguration(AccumuloConfiguration conf);
-
-  /**
    * Returns a connection to this instance of accumulo.
    *
    * @param principal

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dca2ff93/core/src/main/java/org/apache/accumulo/core/client/MutationsRejectedException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/MutationsRejectedException.java b/core/src/main/java/org/apache/accumulo/core/client/MutationsRejectedException.java
index b67a3a0..8f8720a 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/MutationsRejectedException.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/MutationsRejectedException.java
@@ -23,14 +23,11 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
-import java.util.function.Function;
-import java.util.stream.Collectors;
 
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.security.SecurityErrorCode;
 import org.apache.accumulo.core.data.ConstraintViolationSummary;
 import org.apache.accumulo.core.data.TabletId;
-import org.apache.accumulo.core.data.impl.TabletIdImpl;
 
 /**
  * Communicate the failed mutations of a BatchWriter back to the client.
@@ -44,61 +41,6 @@ public class MutationsRejectedException extends AccumuloException {
   private Collection<String> es;
   private int unknownErrors;
 
-  private static <K,V,L> Map<L,V> transformKeys(Map<K,V> map, Function<K,L> keyFunction) {
-    HashMap<L,V> ret = new HashMap<>();
-    for (Entry<K,V> entry : map.entrySet()) {
-      ret.put(keyFunction.apply(entry.getKey()), entry.getValue());
-    }
-
-    return ret;
-  }
-
-  /**
-   * @param cvsList
-   *          list of constraint violations
-   * @param hashMap
-   *          authorization failures
-   * @param serverSideErrors
-   *          server side errors
-   * @param unknownErrors
-   *          number of unknown errors
-   *
-   * @deprecated since 1.6.0, see {@link #MutationsRejectedException(Instance, List, Map, Collection, int, Throwable)}
-   */
-  @Deprecated
-  public MutationsRejectedException(List<ConstraintViolationSummary> cvsList, HashMap<org.apache.accumulo.core.data.KeyExtent,Set<SecurityErrorCode>> hashMap,
-      Collection<String> serverSideErrors, int unknownErrors, Throwable cause) {
-    super("# constraint violations : " + cvsList.size() + "  security codes: " + hashMap.values() + "  # server errors " + serverSideErrors.size()
-        + " # exceptions " + unknownErrors, cause);
-    this.cvsl = cvsList;
-    this.af = transformKeys(hashMap, TabletIdImpl.KE_2_TID_OLD);
-    this.es = serverSideErrors;
-    this.unknownErrors = unknownErrors;
-  }
-
-  /**
-   * @param cvsList
-   *          list of constraint violations
-   * @param hashMap
-   *          authorization failures
-   * @param serverSideErrors
-   *          server side errors
-   * @param unknownErrors
-   *          number of unknown errors
-   *
-   * @deprecated since 1.7.0 see {@link #MutationsRejectedException(Instance, List, Map, Collection, int, Throwable)}
-   */
-  @Deprecated
-  public MutationsRejectedException(Instance instance, List<ConstraintViolationSummary> cvsList,
-      HashMap<org.apache.accumulo.core.data.KeyExtent,Set<SecurityErrorCode>> hashMap, Collection<String> serverSideErrors, int unknownErrors, Throwable cause) {
-    super("# constraint violations : " + cvsList.size() + "  security codes: " + format(transformKeys(hashMap, TabletIdImpl.KE_2_TID_OLD), instance)
-        + "  # server errors " + serverSideErrors.size() + " # exceptions " + unknownErrors, cause);
-    this.cvsl = cvsList;
-    this.af = transformKeys(hashMap, TabletIdImpl.KE_2_TID_OLD);
-    this.es = serverSideErrors;
-    this.unknownErrors = unknownErrors;
-  }
-
   /**
    *
    * @param cvsList
@@ -146,25 +88,6 @@ public class MutationsRejectedException extends AccumuloException {
   }
 
   /**
-   * @return the internal list of authorization failures
-   * @deprecated since 1.5, see {@link #getAuthorizationFailuresMap()}
-   */
-  @Deprecated
-  public List<org.apache.accumulo.core.data.KeyExtent> getAuthorizationFailures() {
-    return af.keySet().stream().map(TabletIdImpl.TID_2_KE_OLD).collect(Collectors.toList());
-  }
-
-  /**
-   * @return the internal mapping of keyextent mappings to SecurityErrorCode
-   * @since 1.5.0
-   * @deprecated since 1.7.0 see {@link #getSecurityErrorCodes()}
-   */
-  @Deprecated
-  public Map<org.apache.accumulo.core.data.KeyExtent,Set<SecurityErrorCode>> getAuthorizationFailuresMap() {
-    return transformKeys(af, TabletIdImpl.TID_2_KE_OLD);
-  }
-
-  /**
    * @return the internal mapping of TabletID to SecurityErrorCodes
    */
   public Map<TabletId,Set<SecurityErrorCode>> getSecurityErrorCodes() {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dca2ff93/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java b/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
index 4a4dd5f..b983a20 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
@@ -34,7 +34,6 @@ import org.apache.accumulo.core.client.impl.InstanceOperationsImpl;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.util.ByteBufferUtil;
 import org.apache.accumulo.core.util.OpTimer;
@@ -73,7 +72,6 @@ public class ZooKeeperInstance implements Instance {
 
   private final int zooKeepersSessionTimeOut;
 
-  private AccumuloConfiguration conf;
   private ClientConfiguration clientConf;
 
   /**
@@ -277,29 +275,6 @@ public class ZooKeeperInstance implements Instance {
   }
 
   @Override
-  @Deprecated
-  public AccumuloConfiguration getConfiguration() {
-    return conf = conf == null ? DefaultConfiguration.getInstance() : ClientContext.convertClientConfig(clientConf);
-  }
-
-  @Override
-  @Deprecated
-  public void setConfiguration(AccumuloConfiguration conf) {
-    this.conf = conf;
-  }
-
-  /**
-   * Given a zooCache and instanceId, look up the instance name.
-   *
-   * @deprecated since 1.7.0 {@link ZooCache} is not part of the public API, but its a parameter to this method. Therefore code that uses this method is not
-   *             guaranteed to be stable. This method was deprecated to discourage its use.
-   */
-  @Deprecated
-  public static String lookupInstanceName(ZooCache zooCache, UUID instanceId) {
-    return InstanceOperationsImpl.lookupInstanceName(zooCache, instanceId);
-  }
-
-  @Override
   public String toString() {
     StringBuilder sb = new StringBuilder(64);
     sb.append("ZooKeeperInstance: ").append(getInstanceName()).append(" ").append(getZooKeepers());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dca2ff93/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveCompaction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveCompaction.java b/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveCompaction.java
index 5228391..ddb7fa9 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveCompaction.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveCompaction.java
@@ -78,13 +78,6 @@ public abstract class ActiveCompaction {
 
   /**
    * @return tablet thats is compacting
-   * @deprecated since 1.7.0 use {@link #getTablet()}
-   */
-  @Deprecated
-  public abstract org.apache.accumulo.core.data.KeyExtent getExtent();
-
-  /**
-   * @return tablet thats is compacting
    * @since 1.7.0
    */
   public abstract TabletId getTablet();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dca2ff93/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveScan.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveScan.java b/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveScan.java
index 81bb1cc..9510895 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveScan.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveScan.java
@@ -65,13 +65,6 @@ public abstract class ActiveScan {
 
   /**
    * @return tablet the scan is running against, if a batch scan may be one of many or null
-   * @deprecated since 1.7.0 use {@link #getTablet()}
-   */
-  @Deprecated
-  public abstract org.apache.accumulo.core.data.KeyExtent getExtent();
-
-  /**
-   * @return tablet the scan is running against, if a batch scan may be one of many or null
    * @since 1.7.0
    */
   public abstract TabletId getTablet();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dca2ff93/core/src/main/java/org/apache/accumulo/core/client/impl/ActiveCompactionImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ActiveCompactionImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ActiveCompactionImpl.java
index bdd5d51..a34bcf7 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ActiveCompactionImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ActiveCompactionImpl.java
@@ -28,7 +28,6 @@ import org.apache.accumulo.core.data.TabletId;
 import org.apache.accumulo.core.data.impl.KeyExtent;
 import org.apache.accumulo.core.data.impl.TabletIdImpl;
 import org.apache.accumulo.core.data.thrift.IterInfo;
-import org.apache.hadoop.io.Text;
 
 /**
  *
@@ -50,14 +49,6 @@ public class ActiveCompactionImpl extends ActiveCompaction {
   }
 
   @Override
-  @Deprecated
-  public org.apache.accumulo.core.data.KeyExtent getExtent() {
-    KeyExtent ke = new KeyExtent(tac.getExtent());
-    org.apache.accumulo.core.data.KeyExtent oke = new org.apache.accumulo.core.data.KeyExtent(new Text(ke.getTableId()), ke.getEndRow(), ke.getPrevEndRow());
-    return oke;
-  }
-
-  @Override
   public TabletId getTablet() {
     return new TabletIdImpl(new KeyExtent(tac.getExtent()));
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dca2ff93/core/src/main/java/org/apache/accumulo/core/client/impl/ActiveScanImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ActiveScanImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ActiveScanImpl.java
index 9021190..dd96aa3 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ActiveScanImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ActiveScanImpl.java
@@ -32,7 +32,6 @@ import org.apache.accumulo.core.data.impl.TabletIdImpl;
 import org.apache.accumulo.core.data.thrift.IterInfo;
 import org.apache.accumulo.core.data.thrift.TColumn;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.hadoop.io.Text;
 
 /**
  * A class that contains information about an ActiveScan
@@ -120,12 +119,6 @@ public class ActiveScanImpl extends ActiveScan {
   }
 
   @Override
-  @Deprecated
-  public org.apache.accumulo.core.data.KeyExtent getExtent() {
-    return new org.apache.accumulo.core.data.KeyExtent(new Text(extent.getTableId()), extent.getEndRow(), extent.getPrevEndRow());
-  }
-
-  @Override
   public TabletId getTablet() {
     return new TabletIdImpl(extent);
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dca2ff93/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
index 6165346..311d271 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
@@ -328,23 +328,6 @@ public abstract class AbstractInputFormat<K,V> implements InputFormat<K,V> {
   }
 
   /**
-   * Initializes an Accumulo {@link org.apache.accumulo.core.client.impl.TabletLocator} based on the configuration.
-   *
-   * @param job
-   *          the Hadoop context for the configured job
-   * @return an Accumulo tablet locator
-   * @throws org.apache.accumulo.core.client.TableNotFoundException
-   *           if the table name set on the configuration doesn't exist
-   * @since 1.6.0
-   * @deprecated since 1.7.0 This method returns a type that is not part of the public API and is not guaranteed to be stable. The method was deprecated to
-   *             discourage its use.
-   */
-  @Deprecated
-  protected static TabletLocator getTabletLocator(JobConf job, String tableId) throws TableNotFoundException {
-    return InputConfigurator.getTabletLocator(CLASS, job, tableId);
-  }
-
-  /**
    * Fetch the client configuration from the job.
    *
    * @param job

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dca2ff93/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormat.java
index f2bc4cd..640a85d 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormat.java
@@ -53,21 +53,6 @@ public class AccumuloFileOutputFormat extends FileOutputFormat<Key,Value> {
   protected static final Logger log = Logger.getLogger(CLASS);
 
   /**
-   * This helper method provides an AccumuloConfiguration object constructed from the Accumulo defaults, and overridden with Accumulo properties that have been
-   * stored in the Job's configuration.
-   *
-   * @param job
-   *          the Hadoop context for the configured job
-   * @since 1.5.0
-   * @deprecated since 1.7.0 This method returns a type that is not part of the public API and is not guaranteed to be stable. The method was deprecated to
-   *             discourage its use.
-   */
-  @Deprecated
-  protected static AccumuloConfiguration getAccumuloConfiguration(JobConf job) {
-    return FileOutputConfigurator.getAccumuloConfiguration(CLASS, job);
-  }
-
-  /**
    * Sets the compression type to use for data blocks. Specifying a compression may require additional libraries to be available to your Job.
    *
    * @param job

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dca2ff93/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
index 0cf57d2..a6223fc 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
@@ -26,8 +26,6 @@ import org.apache.accumulo.core.client.IsolatedScanner;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.ScannerBase;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.impl.TabletLocator;
 import org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator;
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
 import org.apache.accumulo.core.data.Key;
@@ -355,22 +353,6 @@ public abstract class InputFormatBase<K,V> extends AbstractInputFormat<K,V> {
     InputConfigurator.setSamplerConfiguration(CLASS, job, samplerConfig);
   }
 
-  /**
-   * Initializes an Accumulo {@link org.apache.accumulo.core.client.impl.TabletLocator} based on the configuration.
-   *
-   * @param job
-   *          the Hadoop job for the configured job
-   * @return an Accumulo tablet locator
-   * @throws org.apache.accumulo.core.client.TableNotFoundException
-   *           if the table name set on the job doesn't exist
-   * @since 1.5.0
-   * @deprecated since 1.6.0
-   */
-  @Deprecated
-  protected static TabletLocator getTabletLocator(JobConf job) throws TableNotFoundException {
-    return InputConfigurator.getTabletLocator(CLASS, job, InputConfigurator.getInputTableName(CLASS, job));
-  }
-
   protected abstract static class RecordReaderBase<K,V> extends AbstractRecordReader<K,V> {
 
     @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dca2ff93/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
index 9ccf78a..254d538 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
@@ -381,25 +381,6 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
     return InputConfigurator.getInputTableConfig(CLASS, context.getConfiguration(), tableName);
   }
 
-  /**
-   * Initializes an Accumulo {@link org.apache.accumulo.core.client.impl.TabletLocator} based on the configuration.
-   *
-   * @param context
-   *          the Hadoop context for the configured job
-   * @param table
-   *          the table for which to initialize the locator
-   * @return an Accumulo tablet locator
-   * @throws org.apache.accumulo.core.client.TableNotFoundException
-   *           if the table name set on the configuration doesn't exist
-   * @since 1.6.0
-   * @deprecated since 1.7.0 This method returns a type that is not part of the public API and is not guaranteed to be stable. The method was deprecated to
-   *             discourage its use.
-   */
-  @Deprecated
-  protected static TabletLocator getTabletLocator(JobContext context, String table) throws TableNotFoundException {
-    return InputConfigurator.getTabletLocator(CLASS, context.getConfiguration(), table);
-  }
-
   // InputFormat doesn't have the equivalent of OutputFormat's checkOutputSpecs(JobContext job)
   /**
    * Check whether a configuration is fully configured to be used with an Accumulo {@link org.apache.hadoop.mapreduce.InputFormat}.

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dca2ff93/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java
index 75afe2b..656dba7 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java
@@ -30,7 +30,6 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.RecordWriter;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
@@ -52,21 +51,6 @@ public class AccumuloFileOutputFormat extends FileOutputFormat<Key,Value> {
   protected static final Logger log = Logger.getLogger(CLASS);
 
   /**
-   * This helper method provides an AccumuloConfiguration object constructed from the Accumulo defaults, and overridden with Accumulo properties that have been
-   * stored in the Job's configuration.
-   *
-   * @param context
-   *          the Hadoop context for the configured job
-   * @since 1.5.0
-   * @deprecated since 1.7.0 This method returns a type that is not part of the public API and is not guaranteed to be stable. The method was deprecated to
-   *             discourage its use.
-   */
-  @Deprecated
-  protected static AccumuloConfiguration getAccumuloConfiguration(JobContext context) {
-    return FileOutputConfigurator.getAccumuloConfiguration(CLASS, context.getConfiguration());
-  }
-
-  /**
    * Sets the compression type to use for data blocks. Specifying a compression may require additional libraries to be available to your Job.
    *
    * @param job

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dca2ff93/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
index 324d5c7..0071966 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
@@ -26,8 +26,6 @@ import org.apache.accumulo.core.client.IsolatedScanner;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.ScannerBase;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.impl.TabletLocator;
 import org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator;
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
 import org.apache.accumulo.core.data.Key;
@@ -354,22 +352,6 @@ public abstract class InputFormatBase<K,V> extends AbstractInputFormat<K,V> {
     InputConfigurator.setSamplerConfiguration(CLASS, job.getConfiguration(), samplerConfig);
   }
 
-  /**
-   * Initializes an Accumulo {@link org.apache.accumulo.core.client.impl.TabletLocator} based on the configuration.
-   *
-   * @param context
-   *          the Hadoop context for the configured job
-   * @return an Accumulo tablet locator
-   * @throws org.apache.accumulo.core.client.TableNotFoundException
-   *           if the table name set on the configuration doesn't exist
-   * @since 1.5.0
-   * @deprecated since 1.6.0
-   */
-  @Deprecated
-  protected static TabletLocator getTabletLocator(JobContext context) throws TableNotFoundException {
-    return InputConfigurator.getTabletLocator(CLASS, context.getConfiguration(), InputConfigurator.getInputTableName(CLASS, context.getConfiguration()));
-  }
-
   protected abstract static class RecordReaderBase<K,V> extends AbstractRecordReader<K,V> {
 
     @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dca2ff93/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/ConfiguratorBase.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/ConfiguratorBase.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/ConfiguratorBase.java
deleted file mode 100644
index 6914071..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/ConfiguratorBase.java
+++ /dev/null
@@ -1,275 +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.core.client.mapreduce.lib.util;
-
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.ClientConfiguration;
-import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.ZooKeeperInstance;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.AuthenticationTokenSerializer;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.log4j.Level;
-
-/**
- * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
- * @since 1.5.0
- */
-@Deprecated
-public class ConfiguratorBase {
-
-  /**
-   * Configuration keys for {@link Instance#getConnector(String, AuthenticationToken)}.
-   *
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static enum ConnectorInfo {
-    IS_CONFIGURED, PRINCIPAL, TOKEN, TOKEN_CLASS
-  }
-
-  /**
-   * Configuration keys for {@link Instance}, {@link ZooKeeperInstance}, and {@link org.apache.accumulo.core.client.mock.MockInstance}.
-   *
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  protected static enum InstanceOpts {
-    TYPE, NAME, ZOO_KEEPERS;
-  }
-
-  /**
-   * Configuration keys for general configuration options.
-   *
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  protected static enum GeneralOpts {
-    LOG_LEVEL
-  }
-
-  /**
-   * Provides a configuration key for a given feature enum, prefixed by the implementingClass
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param e
-   *          the enum used to provide the unique part of the configuration key
-   * @return the configuration key
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  protected static String enumToConfKey(Class<?> implementingClass, Enum<?> e) {
-    return implementingClass.getSimpleName() + "." + e.getDeclaringClass().getSimpleName() + "." + StringUtils.camelize(e.name().toLowerCase());
-  }
-
-  /**
-   * Sets the connector information needed to communicate with Accumulo in this job.
-   *
-   * <p>
-   * <b>WARNING:</b> The serialized token is stored in the configuration and shared with all MapReduce tasks. It is BASE64 encoded to provide a charset safe
-   * conversion to a string, and is not intended to be secure.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param principal
-   *          a valid Accumulo user name
-   * @param token
-   *          the user's password
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void setConnectorInfo(Class<?> implementingClass, Configuration conf, String principal, AuthenticationToken token)
-      throws AccumuloSecurityException {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase.setConnectorInfo(implementingClass, conf, principal, token);
-  }
-
-  /**
-   * Determines if the connector info has already been set for this instance.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @return true if the connector info has already been set, false otherwise
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   * @see #setConnectorInfo(Class, Configuration, String, AuthenticationToken)
-   */
-  @Deprecated
-  public static Boolean isConnectorInfoSet(Class<?> implementingClass, Configuration conf) {
-    return org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase.isConnectorInfoSet(implementingClass, conf);
-  }
-
-  /**
-   * Gets the user name from the configuration.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @return the principal
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   * @see #setConnectorInfo(Class, Configuration, String, AuthenticationToken)
-   */
-  @Deprecated
-  public static String getPrincipal(Class<?> implementingClass, Configuration conf) {
-    return org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase.getPrincipal(implementingClass, conf);
-  }
-
-  /**
-   * DON'T USE THIS. No, really, don't use this. You already have an {@link AuthenticationToken} with
-   * {@link org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase#getAuthenticationToken(Class, Configuration)}. You don't need to construct it
-   * yourself.
-   * <p>
-   * Gets the serialized token class from the configuration.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @return the principal
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   * @see #setConnectorInfo(Class, Configuration, String, AuthenticationToken)
-   */
-  @Deprecated
-  public static String getTokenClass(Class<?> implementingClass, Configuration conf) {
-    return org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase.getAuthenticationToken(implementingClass, conf).getClass().getName();
-  }
-
-  /**
-   * DON'T USE THIS. No, really, don't use this. You already have an {@link AuthenticationToken} with
-   * {@link org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase#getAuthenticationToken(Class, Configuration)}. You don't need to construct it
-   * yourself.
-   * <p>
-   * Gets the password from the configuration. WARNING: The password is stored in the Configuration and shared with all MapReduce tasks; It is BASE64 encoded to
-   * provide a charset safe conversion to a string, and is not intended to be secure.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @return the decoded principal's authentication token
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   * @see #setConnectorInfo(Class, Configuration, String, AuthenticationToken)
-   */
-  @Deprecated
-  public static byte[] getToken(Class<?> implementingClass, Configuration conf) {
-    return AuthenticationTokenSerializer.serialize(org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase.getAuthenticationToken(
-        implementingClass, conf));
-  }
-
-  /**
-   * Configures a {@link ZooKeeperInstance} for this job.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param instanceName
-   *          the Accumulo instance name
-   * @param zooKeepers
-   *          a comma-separated list of zookeeper servers
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void setZooKeeperInstance(Class<?> implementingClass, Configuration conf, String instanceName, String zooKeepers) {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase.setZooKeeperInstance(implementingClass, conf,
-        new ClientConfiguration().withInstance(instanceName).withZkHosts(zooKeepers));
-  }
-
-  /**
-   * Configures a {@link org.apache.accumulo.core.client.mock.MockInstance} for this job.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param instanceName
-   *          the Accumulo instance name
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void setMockInstance(Class<?> implementingClass, Configuration conf, String instanceName) {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase.setMockInstance(implementingClass, conf, instanceName);
-  }
-
-  /**
-   * Initializes an Accumulo {@link Instance} based on the configuration.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @return an Accumulo instance
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   * @see #setZooKeeperInstance(Class, Configuration, String, String)
-   */
-  @Deprecated
-  public static Instance getInstance(Class<?> implementingClass, Configuration conf) {
-    return org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase.getInstance(implementingClass, conf);
-  }
-
-  /**
-   * Sets the log level for this job.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param level
-   *          the logging level
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void setLogLevel(Class<?> implementingClass, Configuration conf, Level level) {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase.setLogLevel(implementingClass, conf, level);
-  }
-
-  /**
-   * Gets the log level from this configuration.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @return the log level
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   * @see #setLogLevel(Class, Configuration, Level)
-   */
-  @Deprecated
-  public static Level getLogLevel(Class<?> implementingClass, Configuration conf) {
-    return org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase.getLogLevel(implementingClass, conf);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dca2ff93/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/FileOutputConfigurator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/FileOutputConfigurator.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/FileOutputConfigurator.java
deleted file mode 100644
index b4f6b8a..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/FileOutputConfigurator.java
+++ /dev/null
@@ -1,170 +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.core.client.mapreduce.lib.util;
-
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
- * @since 1.5.0
- */
-@Deprecated
-public class FileOutputConfigurator extends ConfiguratorBase {
-
-  /**
-   * Configuration keys for {@link AccumuloConfiguration}.
-   *
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static enum Opts {
-    ACCUMULO_PROPERTIES;
-  }
-
-  /**
-   * The supported Accumulo properties we set in this OutputFormat, that change the behavior of the RecordWriter.<br>
-   * These properties correspond to the supported public static setter methods available to this class.
-   *
-   * @param property
-   *          the Accumulo property to check
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  protected static Boolean isSupportedAccumuloProperty(Property property) {
-    switch (property) {
-      case TABLE_FILE_COMPRESSION_TYPE:
-      case TABLE_FILE_COMPRESSED_BLOCK_SIZE:
-      case TABLE_FILE_BLOCK_SIZE:
-      case TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX:
-      case TABLE_FILE_REPLICATION:
-        return true;
-      default:
-        return false;
-    }
-  }
-
-  /**
-   * This helper method provides an AccumuloConfiguration object constructed from the Accumulo defaults, and overridden with Accumulo properties that have been
-   * stored in the Job's configuration.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static AccumuloConfiguration getAccumuloConfiguration(Class<?> implementingClass, Configuration conf) {
-    return org.apache.accumulo.core.client.mapreduce.lib.impl.FileOutputConfigurator.getAccumuloConfiguration(implementingClass, conf);
-  }
-
-  /**
-   * Sets the compression type to use for data blocks. Specifying a compression may require additional libraries to be available to your Job.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param compressionType
-   *          one of "none", "gz", "lzo", or "snappy"
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void setCompressionType(Class<?> implementingClass, Configuration conf, String compressionType) {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.FileOutputConfigurator.setCompressionType(implementingClass, conf, compressionType);
-  }
-
-  /**
-   * Sets the size for data blocks within each file.<br>
-   * Data blocks are a span of key/value pairs stored in the file that are compressed and indexed as a group.
-   *
-   * <p>
-   * Making this value smaller may increase seek performance, but at the cost of increasing the size of the indexes (which can also affect seek performance).
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param dataBlockSize
-   *          the block size, in bytes
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void setDataBlockSize(Class<?> implementingClass, Configuration conf, long dataBlockSize) {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.FileOutputConfigurator.setDataBlockSize(implementingClass, conf, dataBlockSize);
-  }
-
-  /**
-   * Sets the size for file blocks in the file system; file blocks are managed, and replicated, by the underlying file system.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param fileBlockSize
-   *          the block size, in bytes
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void setFileBlockSize(Class<?> implementingClass, Configuration conf, long fileBlockSize) {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.FileOutputConfigurator.setFileBlockSize(implementingClass, conf, fileBlockSize);
-  }
-
-  /**
-   * Sets the size for index blocks within each file; smaller blocks means a deeper index hierarchy within the file, while larger blocks mean a more shallow
-   * index hierarchy within the file. This can affect the performance of queries.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param indexBlockSize
-   *          the block size, in bytes
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void setIndexBlockSize(Class<?> implementingClass, Configuration conf, long indexBlockSize) {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.FileOutputConfigurator.setIndexBlockSize(implementingClass, conf, indexBlockSize);
-  }
-
-  /**
-   * Sets the file system replication factor for the resulting file, overriding the file system default.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param replication
-   *          the number of replicas for produced files
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void setReplication(Class<?> implementingClass, Configuration conf, int replication) {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.FileOutputConfigurator.setReplication(implementingClass, conf, replication);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dca2ff93/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java
deleted file mode 100644
index b85253c..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java
+++ /dev/null
@@ -1,461 +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.core.client.mapreduce.lib.util;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.ClientSideIteratorScanner;
-import org.apache.accumulo.core.client.IsolatedScanner;
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.impl.Tables;
-import org.apache.accumulo.core.client.impl.TabletLocator;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.Text;
-
-/**
- * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
- * @since 1.5.0
- */
-@Deprecated
-public class InputConfigurator extends ConfiguratorBase {
-
-  /**
-   * Configuration keys for {@link Scanner}.
-   *
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static enum ScanOpts {
-    TABLE_NAME, AUTHORIZATIONS, RANGES, COLUMNS, ITERATORS
-  }
-
-  /**
-   * Configuration keys for various features.
-   *
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static enum Features {
-    AUTO_ADJUST_RANGES, SCAN_ISOLATION, USE_LOCAL_ITERATORS, SCAN_OFFLINE
-  }
-
-  /**
-   * Sets the name of the input table, over which this job will scan.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param tableName
-   *          the table to use when the tablename is null in the write call
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void setInputTableName(Class<?> implementingClass, Configuration conf, String tableName) {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.setInputTableName(implementingClass, conf, tableName);
-  }
-
-  /**
-   * Gets the table name from the configuration.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @return the table name
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   * @see #setInputTableName(Class, Configuration, String)
-   */
-  @Deprecated
-  public static String getInputTableName(Class<?> implementingClass, Configuration conf) {
-    return org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.getInputTableName(implementingClass, conf);
-  }
-
-  /**
-   * Sets the {@link Authorizations} used to scan. Must be a subset of the user's authorization. Defaults to the empty set.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param auths
-   *          the user's authorizations
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void setScanAuthorizations(Class<?> implementingClass, Configuration conf, Authorizations auths) {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.setScanAuthorizations(implementingClass, conf, auths);
-  }
-
-  /**
-   * Gets the authorizations to set for the scans from the configuration.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @return the Accumulo scan authorizations
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   * @see #setScanAuthorizations(Class, Configuration, Authorizations)
-   */
-  @Deprecated
-  public static Authorizations getScanAuthorizations(Class<?> implementingClass, Configuration conf) {
-    return org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.getScanAuthorizations(implementingClass, conf);
-  }
-
-  /**
-   * Sets the input ranges to scan for this job. If not set, the entire table will be scanned.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param ranges
-   *          the ranges that will be mapped over
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void setRanges(Class<?> implementingClass, Configuration conf, Collection<Range> ranges) {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.setRanges(implementingClass, conf, ranges);
-  }
-
-  /**
-   * Gets the ranges to scan over from a job.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @return the ranges
-   * @throws IOException
-   *           if the ranges have been encoded improperly
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   * @see #setRanges(Class, Configuration, Collection)
-   */
-  @Deprecated
-  public static List<Range> getRanges(Class<?> implementingClass, Configuration conf) throws IOException {
-    return org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.getRanges(implementingClass, conf);
-  }
-
-  /**
-   * Restricts the columns that will be mapped over for this job.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param columnFamilyColumnQualifierPairs
-   *          a pair of {@link Text} objects corresponding to column family and column qualifier. If the column qualifier is null, the entire column family is
-   *          selected. An empty set is the default and is equivalent to scanning the all columns.
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void fetchColumns(Class<?> implementingClass, Configuration conf, Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.fetchColumns(implementingClass, conf, columnFamilyColumnQualifierPairs);
-  }
-
-  /**
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   */
-  @Deprecated
-  public static String[] serializeColumns(Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
-    return org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.serializeColumns(columnFamilyColumnQualifierPairs);
-  }
-
-  /**
-   * Gets the columns to be mapped over from this job.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @return a set of columns
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   * @see #fetchColumns(Class, Configuration, Collection)
-   */
-  @Deprecated
-  public static Set<Pair<Text,Text>> getFetchedColumns(Class<?> implementingClass, Configuration conf) {
-    return org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.getFetchedColumns(implementingClass, conf);
-  }
-
-  /**
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   */
-  @Deprecated
-  public static Set<Pair<Text,Text>> deserializeFetchedColumns(Collection<String> serialized) {
-    return org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.deserializeFetchedColumns(serialized);
-  }
-
-  /**
-   * Encode an iterator on the input for this job.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param cfg
-   *          the configuration of the iterator
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void addIterator(Class<?> implementingClass, Configuration conf, IteratorSetting cfg) {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.addIterator(implementingClass, conf, cfg);
-  }
-
-  /**
-   * Gets a list of the iterator settings (for iterators to apply to a scanner) from this configuration.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @return a list of iterators
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   * @see #addIterator(Class, Configuration, IteratorSetting)
-   */
-  @Deprecated
-  public static List<IteratorSetting> getIterators(Class<?> implementingClass, Configuration conf) {
-    return org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.getIterators(implementingClass, conf);
-  }
-
-  /**
-   * Controls the automatic adjustment of ranges for this job. This feature merges overlapping ranges, then splits them to align with tablet boundaries.
-   * Disabling this feature will cause exactly one Map task to be created for each specified range. The default setting is enabled. *
-   *
-   * <p>
-   * By default, this feature is <b>enabled</b>.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param enableFeature
-   *          the feature is enabled if true, disabled otherwise
-   * @see #setRanges(Class, Configuration, Collection)
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void setAutoAdjustRanges(Class<?> implementingClass, Configuration conf, boolean enableFeature) {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.setAutoAdjustRanges(implementingClass, conf, enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration has auto-adjust ranges enabled.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @return false if the feature is disabled, true otherwise
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   * @see #setAutoAdjustRanges(Class, Configuration, boolean)
-   */
-  @Deprecated
-  public static Boolean getAutoAdjustRanges(Class<?> implementingClass, Configuration conf) {
-    return org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.getAutoAdjustRanges(implementingClass, conf);
-  }
-
-  /**
-   * Controls the use of the {@link IsolatedScanner} in this job.
-   *
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param enableFeature
-   *          the feature is enabled if true, disabled otherwise
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void setScanIsolation(Class<?> implementingClass, Configuration conf, boolean enableFeature) {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.setScanIsolation(implementingClass, conf, enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration has isolation enabled.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @return true if the feature is enabled, false otherwise
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   * @see #setScanIsolation(Class, Configuration, boolean)
-   */
-  @Deprecated
-  public static Boolean isIsolated(Class<?> implementingClass, Configuration conf) {
-    return org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.isIsolated(implementingClass, conf);
-  }
-
-  /**
-   * Controls the use of the {@link ClientSideIteratorScanner} in this job. Enabling this feature will cause the iterator stack to be constructed within the Map
-   * task, rather than within the Accumulo TServer. To use this feature, all classes needed for those iterators must be available on the classpath for the task.
-   *
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param enableFeature
-   *          the feature is enabled if true, disabled otherwise
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void setLocalIterators(Class<?> implementingClass, Configuration conf, boolean enableFeature) {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.setLocalIterators(implementingClass, conf, enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration uses local iterators.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @return true if the feature is enabled, false otherwise
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   * @see #setLocalIterators(Class, Configuration, boolean)
-   */
-  @Deprecated
-  public static Boolean usesLocalIterators(Class<?> implementingClass, Configuration conf) {
-    return org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.usesLocalIterators(implementingClass, conf);
-  }
-
-  /**
-   * Enable reading offline tables. By default, this feature is disabled and only online tables are scanned. This will make the map reduce job directly read the
-   * table's files. If the table is not offline, then the job will fail. If the table comes online during the map reduce job, it is likely that the job will
-   * fail.
-   *
-   * <p>
-   * To use this option, the map reduce user will need access to read the Accumulo directory in HDFS.
-   *
-   * <p>
-   * Reading the offline table will create the scan time iterator stack in the map process. So any iterators that are configured for the table will need to be
-   * on the mapper's classpath. The accumulo-site.xml may need to be on the mapper's classpath if HDFS or the Accumulo directory in HDFS are non-standard.
-   *
-   * <p>
-   * One way to use this feature is to clone a table, take the clone offline, and use the clone as the input table for a map reduce job. If you plan to map
-   * reduce over the data many times, it may be better to the compact the table, clone it, take it offline, and use the clone for all map reduce jobs. The
-   * reason to do this is that compaction will reduce each tablet in the table to one file, and it is faster to read from one file.
-   *
-   * <p>
-   * There are two possible advantages to reading a tables file directly out of HDFS. First, you may see better read performance. Second, it will support
-   * speculative execution better. When reading an online table speculative execution can put more load on an already slow tablet server.
-   *
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param enableFeature
-   *          the feature is enabled if true, disabled otherwise
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void setOfflineTableScan(Class<?> implementingClass, Configuration conf, boolean enableFeature) {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.setOfflineTableScan(implementingClass, conf, enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration has the offline table scan feature enabled.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @return true if the feature is enabled, false otherwise
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   * @see #setOfflineTableScan(Class, Configuration, boolean)
-   */
-  @Deprecated
-  public static Boolean isOfflineScan(Class<?> implementingClass, Configuration conf) {
-    return org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.isOfflineScan(implementingClass, conf);
-  }
-
-  /**
-   * Initializes an Accumulo {@link TabletLocator} based on the configuration.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @return an Accumulo tablet locator
-   * @throws TableNotFoundException
-   *           if the table name set on the configuration doesn't exist
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static TabletLocator getTabletLocator(Class<?> implementingClass, Configuration conf) throws TableNotFoundException {
-    return org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.getTabletLocator(implementingClass, conf,
-        Tables.getTableId(getInstance(implementingClass, conf), getInputTableName(implementingClass, conf)));
-  }
-
-  // InputFormat doesn't have the equivalent of OutputFormat's checkOutputSpecs(JobContext job)
-  /**
-   * Check whether a configuration is fully configured to be used with an Accumulo {@link org.apache.hadoop.mapreduce.InputFormat}.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @throws IOException
-   *           if the context is improperly configured
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void validateOptions(Class<?> implementingClass, Configuration conf) throws IOException {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.validateOptions(implementingClass, conf);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dca2ff93/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/OutputConfigurator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/OutputConfigurator.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/OutputConfigurator.java
deleted file mode 100644
index 39163a6..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/OutputConfigurator.java
+++ /dev/null
@@ -1,196 +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.core.client.mapreduce.lib.util;
-
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
- * @since 1.5.0
- */
-@Deprecated
-public class OutputConfigurator extends ConfiguratorBase {
-
-  /**
-   * Configuration keys for {@link BatchWriter}.
-   *
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static enum WriteOpts {
-    DEFAULT_TABLE_NAME, BATCH_WRITER_CONFIG
-  }
-
-  /**
-   * Configuration keys for various features.
-   *
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static enum Features {
-    CAN_CREATE_TABLES, SIMULATION_MODE
-  }
-
-  /**
-   * Sets the default table name to use if one emits a null in place of a table name for a given mutation. Table names can only be alpha-numeric and
-   * underscores.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param tableName
-   *          the table to use when the tablename is null in the write call
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void setDefaultTableName(Class<?> implementingClass, Configuration conf, String tableName) {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.OutputConfigurator.setDefaultTableName(implementingClass, conf, tableName);
-  }
-
-  /**
-   * Gets the default table name from the configuration.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @return the default table name
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   * @see #setDefaultTableName(Class, Configuration, String)
-   */
-  @Deprecated
-  public static String getDefaultTableName(Class<?> implementingClass, Configuration conf) {
-    return org.apache.accumulo.core.client.mapreduce.lib.impl.OutputConfigurator.getDefaultTableName(implementingClass, conf);
-  }
-
-  /**
-   * Sets the configuration for for the job's {@link BatchWriter} instances. If not set, a new {@link BatchWriterConfig}, with sensible built-in defaults is
-   * used. Setting the configuration multiple times overwrites any previous configuration.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param bwConfig
-   *          the configuration for the {@link BatchWriter}
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void setBatchWriterOptions(Class<?> implementingClass, Configuration conf, BatchWriterConfig bwConfig) {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.OutputConfigurator.setBatchWriterOptions(implementingClass, conf, bwConfig);
-  }
-
-  /**
-   * Gets the {@link BatchWriterConfig} settings.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @return the configuration object
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   * @see #setBatchWriterOptions(Class, Configuration, BatchWriterConfig)
-   */
-  @Deprecated
-  public static BatchWriterConfig getBatchWriterOptions(Class<?> implementingClass, Configuration conf) {
-    return org.apache.accumulo.core.client.mapreduce.lib.impl.OutputConfigurator.getBatchWriterOptions(implementingClass, conf);
-  }
-
-  /**
-   * Sets the directive to create new tables, as necessary. Table names can only be alpha-numeric and underscores.
-   *
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param enableFeature
-   *          the feature is enabled if true, disabled otherwise
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void setCreateTables(Class<?> implementingClass, Configuration conf, boolean enableFeature) {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.OutputConfigurator.setCreateTables(implementingClass, conf, enableFeature);
-  }
-
-  /**
-   * Determines whether tables are permitted to be created as needed.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @return true if the feature is disabled, false otherwise
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   * @see #setCreateTables(Class, Configuration, boolean)
-   */
-  @Deprecated
-  public static Boolean canCreateTables(Class<?> implementingClass, Configuration conf) {
-    return org.apache.accumulo.core.client.mapreduce.lib.impl.OutputConfigurator.canCreateTables(implementingClass, conf);
-  }
-
-  /**
-   * Sets the directive to use simulation mode for this job. In simulation mode, no output is produced. This is useful for testing.
-   *
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param enableFeature
-   *          the feature is enabled if true, disabled otherwise
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void setSimulationMode(Class<?> implementingClass, Configuration conf, boolean enableFeature) {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.OutputConfigurator.setSimulationMode(implementingClass, conf, enableFeature);
-  }
-
-  /**
-   * Determines whether this feature is enabled.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @return true if the feature is enabled, false otherwise
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   * @see #setSimulationMode(Class, Configuration, boolean)
-   */
-  @Deprecated
-  public static Boolean getSimulationMode(Class<?> implementingClass, Configuration conf) {
-    return org.apache.accumulo.core.client.mapreduce.lib.impl.OutputConfigurator.getSimulationMode(implementingClass, conf);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dca2ff93/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/package-info.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/package-info.java
deleted file mode 100644
index 269ffea..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-/**
- * @deprecated since 1.6.0; This package was moved out of the public API.
- * @since 1.5.0
- */
-package org.apache.accumulo.core.client.mapreduce.lib.util;
-

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dca2ff93/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
index 50d212f..ac53bed 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
@@ -31,8 +31,6 @@ import org.apache.accumulo.core.client.impl.Credentials;
 import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.util.ByteBufferUtil;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.core.util.TextUtil;
@@ -137,20 +135,6 @@ public class MockInstance implements Instance {
     return getConnector(user, TextUtil.getBytes(new Text(pass.toString())));
   }
 
-  AccumuloConfiguration conf = null;
-
-  @Deprecated
-  @Override
-  public AccumuloConfiguration getConfiguration() {
-    return conf == null ? DefaultConfiguration.getInstance() : conf;
-  }
-
-  @Override
-  @Deprecated
-  public void setConfiguration(AccumuloConfiguration conf) {
-    this.conf = conf;
-  }
-
   @Override
   public Connector getConnector(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
     Connector conn = new MockConnector(new Credentials(principal, token), acu, this);